340 lines
11 KiB
Go
340 lines
11 KiB
Go
// Copyright 2016 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 (
|
|
goJSON "encoding/json"
|
|
"fmt"
|
|
|
|
"github.com/pingcap/parser/ast"
|
|
"github.com/pingcap/parser/model"
|
|
"github.com/pingcap/parser/mysql"
|
|
"github.com/pingcap/parser/terror"
|
|
"github.com/pingcap/tidb/sessionctx"
|
|
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
|
"github.com/pingcap/tidb/types"
|
|
"github.com/pingcap/tidb/types/json"
|
|
"github.com/pingcap/tidb/util/chunk"
|
|
)
|
|
|
|
// These are byte flags used for `HashCode()`.
|
|
const (
|
|
constantFlag byte = 0
|
|
columnFlag byte = 1
|
|
scalarFunctionFlag byte = 3
|
|
)
|
|
|
|
// EvalAstExpr evaluates ast expression directly.
|
|
var EvalAstExpr func(ctx sessionctx.Context, expr ast.ExprNode) (types.Datum, error)
|
|
|
|
// Expression represents all scalar expression in SQL.
|
|
type Expression interface {
|
|
fmt.Stringer
|
|
goJSON.Marshaler
|
|
|
|
// Eval evaluates an expression through a row.
|
|
Eval(row chunk.Row) (types.Datum, error)
|
|
|
|
// EvalInt returns the int64 representation of expression.
|
|
EvalInt(ctx sessionctx.Context, row chunk.Row) (val int64, isNull bool, err error)
|
|
|
|
// EvalReal returns the float64 representation of expression.
|
|
EvalReal(ctx sessionctx.Context, row chunk.Row) (val float64, isNull bool, err error)
|
|
|
|
// EvalString returns the string representation of expression.
|
|
EvalString(ctx sessionctx.Context, row chunk.Row) (val string, isNull bool, err error)
|
|
|
|
// EvalDecimal returns the decimal representation of expression.
|
|
EvalDecimal(ctx sessionctx.Context, row chunk.Row) (val *types.MyDecimal, isNull bool, err error)
|
|
|
|
// EvalTime returns the DATE/DATETIME/TIMESTAMP representation of expression.
|
|
EvalTime(ctx sessionctx.Context, row chunk.Row) (val types.Time, isNull bool, err error)
|
|
|
|
// EvalDuration returns the duration representation of expression.
|
|
EvalDuration(ctx sessionctx.Context, row chunk.Row) (val types.Duration, isNull bool, err error)
|
|
|
|
// EvalJSON returns the JSON representation of expression.
|
|
EvalJSON(ctx sessionctx.Context, row chunk.Row) (val json.BinaryJSON, isNull bool, err error)
|
|
|
|
// GetType gets the type that the expression returns.
|
|
GetType() *types.FieldType
|
|
|
|
// Clone copies an expression totally.
|
|
Clone() Expression
|
|
|
|
// Equal checks whether two expressions are equal.
|
|
Equal(ctx sessionctx.Context, e Expression) bool
|
|
|
|
// IsCorrelated checks if this expression has correlated key.
|
|
IsCorrelated() bool
|
|
|
|
// Decorrelate try to decorrelate the expression by schema.
|
|
Decorrelate(schema *Schema) Expression
|
|
|
|
// ResolveIndices resolves indices by the given schema. It will copy the original expression and return the copied one.
|
|
ResolveIndices(schema *Schema) (Expression, error)
|
|
|
|
// resolveIndices is called inside the `ResolveIndices` It will perform on the expression itself.
|
|
resolveIndices(schema *Schema) error
|
|
|
|
// ExplainInfo returns operator information to be explained.
|
|
ExplainInfo() string
|
|
|
|
// HashCode creates the hashcode for expression which can be used to identify itself from other expression.
|
|
// It generated as the following:
|
|
// Constant: ConstantFlag+encoded value
|
|
// Column: ColumnFlag+encoded value
|
|
// ScalarFunction: SFFlag+encoded function name + encoded arg_1 + encoded arg_2 + ...
|
|
HashCode(sc *stmtctx.StatementContext) []byte
|
|
}
|
|
|
|
// CNFExprs stands for a CNF expression.
|
|
type CNFExprs []Expression
|
|
|
|
// Clone clones itself.
|
|
func (e CNFExprs) Clone() CNFExprs {
|
|
cnf := make(CNFExprs, 0, len(e))
|
|
for _, expr := range e {
|
|
cnf = append(cnf, expr.Clone())
|
|
}
|
|
return cnf
|
|
}
|
|
|
|
// EvalBool evaluates expression list to a boolean value.
|
|
func EvalBool(ctx sessionctx.Context, exprList CNFExprs, row chunk.Row) (bool, error) {
|
|
for _, expr := range exprList {
|
|
data, err := expr.Eval(row)
|
|
if err != nil {
|
|
return false, err
|
|
}
|
|
if data.IsNull() {
|
|
return false, nil
|
|
}
|
|
|
|
i, err := data.ToBool(ctx.GetSessionVars().StmtCtx)
|
|
if err != nil {
|
|
return false, err
|
|
}
|
|
if i == 0 {
|
|
return false, nil
|
|
}
|
|
}
|
|
return true, nil
|
|
}
|
|
|
|
// composeConditionWithBinaryOp composes condition with binary operator into a balance deep tree, which benefits a lot for pb decoder/encoder.
|
|
func composeConditionWithBinaryOp(ctx sessionctx.Context, conditions []Expression, funcName string) Expression {
|
|
length := len(conditions)
|
|
if length == 0 {
|
|
return nil
|
|
}
|
|
if length == 1 {
|
|
return conditions[0]
|
|
}
|
|
expr := NewFunctionInternal(ctx, funcName,
|
|
types.NewFieldType(mysql.TypeTiny),
|
|
composeConditionWithBinaryOp(ctx, conditions[:length/2], funcName),
|
|
composeConditionWithBinaryOp(ctx, conditions[length/2:], funcName))
|
|
return expr
|
|
}
|
|
|
|
// ComposeCNFCondition composes CNF items into a balance deep CNF tree, which benefits a lot for pb decoder/encoder.
|
|
func ComposeCNFCondition(ctx sessionctx.Context, conditions ...Expression) Expression {
|
|
return composeConditionWithBinaryOp(ctx, conditions, ast.LogicAnd)
|
|
}
|
|
|
|
// ComposeDNFCondition composes DNF items into a balance deep DNF tree.
|
|
func ComposeDNFCondition(ctx sessionctx.Context, conditions ...Expression) Expression {
|
|
return composeConditionWithBinaryOp(ctx, conditions, ast.LogicOr)
|
|
}
|
|
|
|
func extractBinaryOpItems(conditions *ScalarFunction, funcName string) []Expression {
|
|
var ret []Expression
|
|
for _, arg := range conditions.GetArgs() {
|
|
if sf, ok := arg.(*ScalarFunction); ok && sf.FuncName.L == funcName {
|
|
ret = append(ret, extractBinaryOpItems(sf, funcName)...)
|
|
} else {
|
|
ret = append(ret, arg)
|
|
}
|
|
}
|
|
return ret
|
|
}
|
|
|
|
// FlattenDNFConditions extracts DNF expression's leaf item.
|
|
// e.g. or(or(a=1, a=2), or(a=3, a=4)), we'll get [a=1, a=2, a=3, a=4].
|
|
func FlattenDNFConditions(DNFCondition *ScalarFunction) []Expression {
|
|
return extractBinaryOpItems(DNFCondition, ast.LogicOr)
|
|
}
|
|
|
|
// FlattenCNFConditions extracts CNF expression's leaf item.
|
|
// e.g. and(and(a>1, a>2), and(a>3, a>4)), we'll get [a>1, a>2, a>3, a>4].
|
|
func FlattenCNFConditions(CNFCondition *ScalarFunction) []Expression {
|
|
return extractBinaryOpItems(CNFCondition, ast.LogicAnd)
|
|
}
|
|
|
|
// Assignment represents a set assignment in Update, such as
|
|
// Update t set c1 = hex(12), c2 = c3 where c2 = 1
|
|
type Assignment struct {
|
|
Col *Column
|
|
Expr Expression
|
|
}
|
|
|
|
// VarAssignment represents a variable assignment in Set, such as set global a = 1.
|
|
type VarAssignment struct {
|
|
Name string
|
|
Expr Expression
|
|
IsDefault bool
|
|
IsGlobal bool
|
|
IsSystem bool
|
|
ExtendValue *Constant
|
|
}
|
|
|
|
// splitNormalFormItems split CNF(conjunctive normal form) like "a and b and c", or DNF(disjunctive normal form) like "a or b or c"
|
|
func splitNormalFormItems(onExpr Expression, funcName string) []Expression {
|
|
switch v := onExpr.(type) {
|
|
case *ScalarFunction:
|
|
if v.FuncName.L == funcName {
|
|
var ret []Expression
|
|
for _, arg := range v.GetArgs() {
|
|
ret = append(ret, splitNormalFormItems(arg, funcName)...)
|
|
}
|
|
return ret
|
|
}
|
|
}
|
|
return []Expression{onExpr}
|
|
}
|
|
|
|
// SplitCNFItems splits CNF items.
|
|
// CNF means conjunctive normal form, e.g. "a and b and c".
|
|
func SplitCNFItems(onExpr Expression) []Expression {
|
|
return splitNormalFormItems(onExpr, ast.LogicAnd)
|
|
}
|
|
|
|
// SplitDNFItems splits DNF items.
|
|
// DNF means disjunctive normal form, e.g. "a or b or c".
|
|
func SplitDNFItems(onExpr Expression) []Expression {
|
|
return splitNormalFormItems(onExpr, ast.LogicOr)
|
|
}
|
|
|
|
// EvaluateExprWithNull sets columns in schema as null and calculate the final result of the scalar function.
|
|
// If the Expression is a non-constant value, it means the result is unknown.
|
|
func EvaluateExprWithNull(ctx sessionctx.Context, schema *Schema, expr Expression) Expression {
|
|
switch x := expr.(type) {
|
|
case *ScalarFunction:
|
|
args := make([]Expression, len(x.GetArgs()))
|
|
for i, arg := range x.GetArgs() {
|
|
args[i] = EvaluateExprWithNull(ctx, schema, arg)
|
|
}
|
|
return NewFunctionInternal(ctx, x.FuncName.L, types.NewFieldType(mysql.TypeTiny), args...)
|
|
case *Column:
|
|
if !schema.Contains(x) {
|
|
return x
|
|
}
|
|
return &Constant{Value: types.Datum{}, RetType: types.NewFieldType(mysql.TypeNull)}
|
|
case *Constant:
|
|
if x.DeferredExpr != nil {
|
|
return FoldConstant(x)
|
|
}
|
|
}
|
|
return expr
|
|
}
|
|
|
|
// TableInfo2Schema converts table info to schema with empty DBName.
|
|
func TableInfo2Schema(ctx sessionctx.Context, tbl *model.TableInfo) *Schema {
|
|
return TableInfo2SchemaWithDBName(ctx, model.CIStr{}, tbl)
|
|
}
|
|
|
|
// TableInfo2SchemaWithDBName converts table info to schema.
|
|
func TableInfo2SchemaWithDBName(ctx sessionctx.Context, dbName model.CIStr, tbl *model.TableInfo) *Schema {
|
|
cols := ColumnInfos2ColumnsWithDBName(ctx, dbName, tbl.Name, tbl.Columns)
|
|
keys := make([]KeyInfo, 0, len(tbl.Indices)+1)
|
|
for _, idx := range tbl.Indices {
|
|
if !idx.Unique || idx.State != model.StatePublic {
|
|
continue
|
|
}
|
|
ok := true
|
|
newKey := make([]*Column, 0, len(idx.Columns))
|
|
for _, idxCol := range idx.Columns {
|
|
find := false
|
|
for i, col := range tbl.Columns {
|
|
if idxCol.Name.L == col.Name.L {
|
|
if !mysql.HasNotNullFlag(col.Flag) {
|
|
break
|
|
}
|
|
newKey = append(newKey, cols[i])
|
|
find = true
|
|
break
|
|
}
|
|
}
|
|
if !find {
|
|
ok = false
|
|
break
|
|
}
|
|
}
|
|
if ok {
|
|
keys = append(keys, newKey)
|
|
}
|
|
}
|
|
if tbl.PKIsHandle {
|
|
for i, col := range tbl.Columns {
|
|
if mysql.HasPriKeyFlag(col.Flag) {
|
|
keys = append(keys, KeyInfo{cols[i]})
|
|
break
|
|
}
|
|
}
|
|
}
|
|
schema := NewSchema(cols...)
|
|
schema.SetUniqueKeys(keys)
|
|
return schema
|
|
}
|
|
|
|
// ColumnInfos2ColumnsWithDBName converts a slice of ColumnInfo to a slice of Column.
|
|
func ColumnInfos2ColumnsWithDBName(ctx sessionctx.Context, dbName, tblName model.CIStr, colInfos []*model.ColumnInfo) []*Column {
|
|
columns := make([]*Column, 0, len(colInfos))
|
|
for _, col := range colInfos {
|
|
if col.State != model.StatePublic {
|
|
continue
|
|
}
|
|
newCol := &Column{
|
|
ColName: col.Name,
|
|
TblName: tblName,
|
|
DBName: dbName,
|
|
RetType: &col.FieldType,
|
|
ID: col.ID,
|
|
UniqueID: ctx.GetSessionVars().AllocPlanColumnID(),
|
|
Index: col.Offset,
|
|
}
|
|
columns = append(columns, newCol)
|
|
}
|
|
return columns
|
|
}
|
|
|
|
// NewValuesFunc creates a new values function.
|
|
func NewValuesFunc(ctx sessionctx.Context, offset int, retTp *types.FieldType) *ScalarFunction {
|
|
fc := &valuesFunctionClass{baseFunctionClass{ast.Values, 0, 0}, offset, retTp}
|
|
bt, err := fc.getFunction(ctx, nil)
|
|
terror.Log(err)
|
|
return &ScalarFunction{
|
|
FuncName: model.NewCIStr(ast.Values),
|
|
RetType: retTp,
|
|
Function: bt,
|
|
}
|
|
}
|
|
|
|
// IsBinaryLiteral checks whether an expression is a binary literal
|
|
func IsBinaryLiteral(expr Expression) bool {
|
|
con, ok := expr.(*Constant)
|
|
return ok && con.Value.Kind() == types.KindBinaryLiteral
|
|
}
|