Files
tidb/expression/case.go
2015-09-23 10:18:56 +08:00

190 lines
4.8 KiB
Go

// Copyright 2015 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.
package expression
import (
"fmt"
"strings"
"github.com/pingcap/tidb/context"
"github.com/pingcap/tidb/parser/opcode"
"github.com/pingcap/tidb/util/types"
)
// See https://dev.mysql.com/doc/refman/5.7/en/control-flow-functions.html#operator_case
var (
_ Expression = (*FunctionCase)(nil)
_ Expression = (*WhenClause)(nil)
)
// WhenClause is the expression in Case expression for "when condition then result".
type WhenClause struct {
// Expr is the condition expression in WhenClause.
Expr Expression
// Result is the result expression in WhenClause.
Result Expression
}
// Check if satisified the target condition.
// If satisified, the second returned value is true.
func (w *WhenClause) evalAndCheck(ctx context.Context, args map[interface{}]interface{}, target interface{}) (interface{}, bool, error) {
o := NewBinaryOperation(opcode.EQ, &Value{target}, w.Expr)
// types.Compare wil return true/false for NULL
// We must use BinaryOperation with opcode.Eq
eq, err := o.Eval(ctx, args)
if err != nil {
return nil, false, err
}
if eq == nil {
return nil, false, err
}
beq, err := types.ToBool(eq)
if beq == 0 || err != nil {
return nil, false, err
}
rv, err := w.Result.Eval(ctx, args)
return rv, true, err
}
// Eval implements the Expression Eval interface.
func (w *WhenClause) Eval(ctx context.Context, args map[interface{}]interface{}) (interface{}, error) {
return w.Result.Eval(ctx, args)
}
// Accept implements Expression Accept interface.
func (w *WhenClause) Accept(v Visitor) (Expression, error) {
return v.VisitWhenClause(w)
}
// String implements the Expression String interface.
func (w *WhenClause) String() string {
return fmt.Sprintf("WHEN %s THEN %s", w.Expr.String(), w.Result.String())
}
// Clone implements the Expression Clone interface.
func (w *WhenClause) Clone() Expression {
ne := w.Expr.Clone()
nr := w.Result.Clone()
return &WhenClause{Expr: ne, Result: nr}
}
// IsStatic implements the Expression IsStatic interface.
func (w *WhenClause) IsStatic() bool {
return w.Expr.IsStatic() && w.Result.IsStatic()
}
// FunctionCase is the case expression.
type FunctionCase struct {
// Value is the compare value expression.
Value Expression
// WhenClauses is the condition check expression.
WhenClauses []*WhenClause
// ElseClause is the else result expression.
ElseClause Expression
}
// Clone implements the Expression Clone interface.
func (f *FunctionCase) Clone() Expression {
var (
nv Expression
ne Expression
nw Expression
)
if f.Value != nil {
nv = f.Value.Clone()
}
ws := make([]*WhenClause, 0, len(f.WhenClauses))
for _, w := range f.WhenClauses {
nw = w.Clone()
ws = append(ws, nw.(*WhenClause))
}
if f.ElseClause != nil {
ne = f.ElseClause.Clone()
}
return &FunctionCase{
Value: nv,
WhenClauses: ws,
ElseClause: ne,
}
}
// IsStatic implements the Expression IsStatic interface.
func (f *FunctionCase) IsStatic() bool {
if f.Value != nil && !f.Value.IsStatic() {
return false
}
for _, w := range f.WhenClauses {
if !w.IsStatic() {
return false
}
}
if f.ElseClause != nil && !f.ElseClause.IsStatic() {
return false
}
return true
}
// String implements the Expression String interface.
func (f *FunctionCase) String() string {
strs := make([]string, 0, len(f.WhenClauses)+2)
if f.Value != nil {
strs = append(strs, f.Value.String())
}
for _, w := range f.WhenClauses {
strs = append(strs, w.String())
}
if f.ElseClause != nil {
strs = append(strs, "ELSE "+f.ElseClause.String())
}
return fmt.Sprintf("CASE %s END", strings.Join(strs, " "))
}
// Eval implements the Expression Eval interface.
func (f *FunctionCase) Eval(ctx context.Context, args map[interface{}]interface{}) (interface{}, error) {
var (
target interface{} = true
err error
)
if f.Value != nil {
target, err = f.Value.Eval(ctx, args)
if err != nil {
return nil, err
}
}
for _, w := range f.WhenClauses {
r, match, err := w.evalAndCheck(ctx, args, target)
if err != nil {
return nil, err
}
if match {
return r, nil
}
}
// If there was no matching result value, the result after ELSE is returned,
// or NULL if there is no ELSE part.
if f.ElseClause != nil {
return f.ElseClause.Eval(ctx, args)
}
return nil, nil
}
// Accept implements Expression Accept interface.
func (f *FunctionCase) Accept(v Visitor) (Expression, error) {
return v.VisitFunctionCase(f)
}