// 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 core import ( "context" "strconv" "strings" "github.com/pingcap/errors" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/charset" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/opcode" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" driver "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/hint" "github.com/pingcap/tidb/util/stringutil" ) // EvalSubqueryFirstRow evaluates incorrelated subqueries once, and get first row. var EvalSubqueryFirstRow func(ctx context.Context, p PhysicalPlan, is infoschema.InfoSchema, sctx sessionctx.Context) (row []types.Datum, err error) // evalAstExpr evaluates ast expression directly. func evalAstExpr(sctx sessionctx.Context, expr ast.ExprNode) (types.Datum, error) { if val, ok := expr.(*driver.ValueExpr); ok { return val.Datum, nil } NewExpr, err := rewriteAstExpr(sctx, expr, nil, nil) if err != nil { return types.Datum{}, err } return NewExpr.Eval(chunk.Row{}) } // rewriteAstExpr rewrite ast expression directly. func rewriteAstExpr(sctx sessionctx.Context, expr ast.ExprNode, schema *expression.Schema, names types.NameSlice) (expression.Expression, error) { var is infoschema.InfoSchema if sctx.GetSessionVars().TxnCtx.InfoSchema != nil { is = sctx.GetSessionVars().TxnCtx.InfoSchema.(infoschema.InfoSchema) } b := NewPlanBuilder(sctx, is, &hint.BlockHintProcessor{}) fakePlan := LogicalTableDual{}.Init(sctx, 0) if schema != nil { fakePlan.schema = schema fakePlan.names = names } newExpr, _, err := b.rewrite(context.TODO(), expr, fakePlan, nil, true) if err != nil { return nil, err } return newExpr, nil } func (b *PlanBuilder) rewriteInsertOnDuplicateUpdate(ctx context.Context, exprNode ast.ExprNode, mockPlan LogicalPlan, insertPlan *Insert) (expression.Expression, error) { b.rewriterCounter++ defer func() { b.rewriterCounter-- }() rewriter := b.getExpressionRewriter(ctx, mockPlan) // The rewriter maybe is obtained from "b.rewriterPool", "rewriter.err" is // not nil means certain previous procedure has not handled this error. // Here we give us one more chance to make a correct behavior by handling // this missed error. if rewriter.err != nil { return nil, rewriter.err } rewriter.insertPlan = insertPlan rewriter.asScalar = true expr, _, err := b.rewriteExprNode(rewriter, exprNode, true) return expr, err } // rewrite function rewrites ast expr to expression.Expression. // aggMapper maps ast.AggregateFuncExpr to the columns offset in p's output schema. // asScalar means whether this expression must be treated as a scalar expression. // And this function returns a result expression, a new plan that may have apply or semi-join. func (b *PlanBuilder) rewrite(ctx context.Context, exprNode ast.ExprNode, p LogicalPlan, aggMapper map[*ast.AggregateFuncExpr]int, asScalar bool) (expression.Expression, LogicalPlan, error) { expr, resultPlan, err := b.rewriteWithPreprocess(ctx, exprNode, p, aggMapper, nil, asScalar, nil) return expr, resultPlan, err } // rewriteWithPreprocess is for handling the situation that we need to adjust the input ast tree // before really using its node in `expressionRewriter.Leave`. In that case, we first call // er.preprocess(expr), which returns a new expr. Then we use the new expr in `Leave`. func (b *PlanBuilder) rewriteWithPreprocess( ctx context.Context, exprNode ast.ExprNode, p LogicalPlan, aggMapper map[*ast.AggregateFuncExpr]int, windowMapper map[*ast.WindowFuncExpr]int, asScalar bool, preprocess func(ast.Node) ast.Node, ) (expression.Expression, LogicalPlan, error) { b.rewriterCounter++ defer func() { b.rewriterCounter-- }() rewriter := b.getExpressionRewriter(ctx, p) // The rewriter maybe is obtained from "b.rewriterPool", "rewriter.err" is // not nil means certain previous procedure has not handled this error. // Here we give us one more chance to make a correct behavior by handling // this missed error. if rewriter.err != nil { return nil, nil, rewriter.err } rewriter.aggrMap = aggMapper rewriter.windowMap = windowMapper rewriter.asScalar = asScalar rewriter.preprocess = preprocess expr, resultPlan, err := b.rewriteExprNode(rewriter, exprNode, asScalar) return expr, resultPlan, err } func (b *PlanBuilder) getExpressionRewriter(ctx context.Context, p LogicalPlan) (rewriter *expressionRewriter) { defer func() { if p != nil { rewriter.schema = p.Schema() rewriter.names = p.OutputNames() } }() if len(b.rewriterPool) < b.rewriterCounter { rewriter = &expressionRewriter{p: p, b: b, sctx: b.ctx, ctx: ctx} b.rewriterPool = append(b.rewriterPool, rewriter) return } rewriter = b.rewriterPool[b.rewriterCounter-1] rewriter.p = p rewriter.asScalar = false rewriter.aggrMap = nil rewriter.preprocess = nil rewriter.insertPlan = nil rewriter.disableFoldCounter = 0 rewriter.tryFoldCounter = 0 rewriter.ctxStack = rewriter.ctxStack[:0] rewriter.ctxNameStk = rewriter.ctxNameStk[:0] rewriter.ctx = ctx return } func (b *PlanBuilder) rewriteExprNode(rewriter *expressionRewriter, exprNode ast.ExprNode, asScalar bool) (expression.Expression, LogicalPlan, error) { if rewriter.p != nil { curColLen := rewriter.p.Schema().Len() defer func() { names := rewriter.p.OutputNames().Shallow()[:curColLen] for i := curColLen; i < rewriter.p.Schema().Len(); i++ { names = append(names, types.EmptyName) } // After rewriting finished, only old columns are visible. // e.g. select * from t where t.a in (select t1.a from t1); // The output columns before we enter the subquery are the columns from t. // But when we leave the subquery `t.a in (select t1.a from t1)`, we got a Apply operator // and the output columns become [t.*, t1.*]. But t1.* is used only inside the subquery. If there's another filter // which is also a subquery where t1 is involved. The name resolving will fail if we still expose the column from // the previous subquery. // So here we just reset the names to empty to avoid this situation. // TODO: implement ScalarSubQuery and resolve it during optimizing. In building phase, we will not change the plan's structure. rewriter.p.SetOutputNames(names) }() } exprNode.Accept(rewriter) if rewriter.err != nil { return nil, nil, errors.Trace(rewriter.err) } if !asScalar && len(rewriter.ctxStack) == 0 { return nil, rewriter.p, nil } if len(rewriter.ctxStack) != 1 { return nil, nil, errors.Errorf("context len %v is invalid", len(rewriter.ctxStack)) } rewriter.err = expression.CheckArgsNotMultiColumnRow(rewriter.ctxStack[0]) if rewriter.err != nil { return nil, nil, errors.Trace(rewriter.err) } return rewriter.ctxStack[0], rewriter.p, nil } type expressionRewriter struct { ctxStack []expression.Expression ctxNameStk []*types.FieldName p LogicalPlan schema *expression.Schema names []*types.FieldName err error aggrMap map[*ast.AggregateFuncExpr]int windowMap map[*ast.WindowFuncExpr]int b *PlanBuilder sctx sessionctx.Context ctx context.Context // asScalar indicates the return value must be a scalar value. // NOTE: This value can be changed during expression rewritten. asScalar bool // preprocess is called for every ast.Node in Leave. preprocess func(ast.Node) ast.Node // insertPlan is only used to rewrite the expressions inside the assignment // of the "INSERT" statement. insertPlan *Insert // disableFoldCounter controls fold-disabled scope. If > 0, rewriter will NOT do constant folding. // Typically, during visiting AST, while entering the scope(disable), the counter will +1; while // leaving the scope(enable again), the counter will -1. // NOTE: This value can be changed during expression rewritten. disableFoldCounter int tryFoldCounter int } func (er *expressionRewriter) ctxStackLen() int { return len(er.ctxStack) } func (er *expressionRewriter) ctxStackPop(num int) { l := er.ctxStackLen() er.ctxStack = er.ctxStack[:l-num] er.ctxNameStk = er.ctxNameStk[:l-num] } func (er *expressionRewriter) ctxStackAppend(col expression.Expression, name *types.FieldName) { er.ctxStack = append(er.ctxStack, col) er.ctxNameStk = append(er.ctxNameStk, name) } // constructBinaryOpFunction converts binary operator functions // 1. If op are EQ or NE or NullEQ, constructBinaryOpFunctions converts (a0,a1,a2) op (b0,b1,b2) to (a0 op b0) and (a1 op b1) and (a2 op b2) // 2. Else constructBinaryOpFunctions converts (a0,a1,a2) op (b0,b1,b2) to // `IF( a0 NE b0, a0 op b0, // // IF ( isNull(a0 NE b0), Null, // IF ( a1 NE b1, a1 op b1, // IF ( isNull(a1 NE b1), Null, a2 op b2))))` func (er *expressionRewriter) constructBinaryOpFunction(l expression.Expression, r expression.Expression, op string) (expression.Expression, error) { lLen, rLen := expression.GetRowLen(l), expression.GetRowLen(r) if lLen == 1 && rLen == 1 { return er.newFunction(op, types.NewFieldType(mysql.TypeTiny), l, r) } else if rLen != lLen { return nil, expression.ErrOperandColumns.GenWithStackByArgs(lLen) } switch op { case ast.EQ, ast.NE, ast.NullEQ: funcs := make([]expression.Expression, lLen) for i := 0; i < lLen; i++ { var err error funcs[i], err = er.constructBinaryOpFunction(expression.GetFuncArg(l, i), expression.GetFuncArg(r, i), op) if err != nil { return nil, err } } if op == ast.NE { return expression.ComposeDNFCondition(er.sctx, funcs...), nil } return expression.ComposeCNFCondition(er.sctx, funcs...), nil default: larg0, rarg0 := expression.GetFuncArg(l, 0), expression.GetFuncArg(r, 0) var expr1, expr2, expr3, expr4, expr5 expression.Expression expr1 = expression.NewFunctionInternal(er.sctx, ast.NE, types.NewFieldType(mysql.TypeTiny), larg0, rarg0) expr2 = expression.NewFunctionInternal(er.sctx, op, types.NewFieldType(mysql.TypeTiny), larg0, rarg0) expr3 = expression.NewFunctionInternal(er.sctx, ast.IsNull, types.NewFieldType(mysql.TypeTiny), expr1) var err error l, err = expression.PopRowFirstArg(er.sctx, l) if err != nil { return nil, err } r, err = expression.PopRowFirstArg(er.sctx, r) if err != nil { return nil, err } expr4, err = er.constructBinaryOpFunction(l, r, op) if err != nil { return nil, err } expr5, err = er.newFunction(ast.If, types.NewFieldType(mysql.TypeTiny), expr3, expression.NewNull(), expr4) if err != nil { return nil, err } return er.newFunction(ast.If, types.NewFieldType(mysql.TypeTiny), expr1, expr2, expr5) } } func (er *expressionRewriter) buildSubquery(ctx context.Context, subq *ast.SubqueryExpr) (LogicalPlan, error) { if er.schema != nil { outerSchema := er.schema.Clone() er.b.outerSchemas = append(er.b.outerSchemas, outerSchema) er.b.outerNames = append(er.b.outerNames, er.names) defer func() { er.b.outerSchemas = er.b.outerSchemas[0 : len(er.b.outerSchemas)-1] er.b.outerNames = er.b.outerNames[0 : len(er.b.outerNames)-1] }() } np, err := er.b.buildResultSetNode(ctx, subq.Query) if err != nil { return nil, err } // Pop the handle map generated by the subquery. er.b.handleHelper.popMap() return np, nil } // Enter implements Visitor interface. func (er *expressionRewriter) Enter(inNode ast.Node) (ast.Node, bool) { switch v := inNode.(type) { case *ast.AggregateFuncExpr: index, ok := -1, false if er.aggrMap != nil { index, ok = er.aggrMap[v] } if !ok { er.err = ErrInvalidGroupFuncUse return inNode, true } er.ctxStackAppend(er.schema.Columns[index], er.names[index]) return inNode, true case *ast.ColumnNameExpr: if index, ok := er.b.colMapper[v]; ok { er.ctxStackAppend(er.schema.Columns[index], er.names[index]) return inNode, true } case *ast.CompareSubqueryExpr: return er.handleCompareSubquery(er.ctx, v) case *ast.ExistsSubqueryExpr: return er.handleExistSubquery(er.ctx, v) case *ast.PatternInExpr: if v.Sel != nil { return er.handleInSubquery(er.ctx, v) } if len(v.List) != 1 { break } // For 10 in ((select * from t)), the parser won't set v.Sel. // So we must process this case here. x := v.List[0] for { switch y := x.(type) { case *ast.SubqueryExpr: v.Sel = y return er.handleInSubquery(er.ctx, v) case *ast.ParenthesesExpr: x = y.Expr default: return inNode, false } } case *ast.SubqueryExpr: return er.handleScalarSubquery(er.ctx, v) case *ast.ParenthesesExpr: case *ast.ValuesExpr: schema, names := er.schema, er.names // NOTE: "er.insertPlan != nil" means that we are rewriting the // expressions inside the assignment of "INSERT" statement. we have to // use the "tableSchema" of that "insertPlan". if er.insertPlan != nil { schema = er.insertPlan.tableSchema names = er.insertPlan.tableColNames } idx, err := expression.FindFieldName(names, v.Column.Name) if err != nil { er.err = err return inNode, false } if idx < 0 { er.err = ErrUnknownColumn.GenWithStackByArgs(v.Column.Name.OrigColName(), "field list") return inNode, false } col := schema.Columns[idx] er.ctxStackAppend(expression.NewValuesFunc(er.sctx, col.Index, col.RetType), types.EmptyName) return inNode, true case *ast.WindowFuncExpr: index, ok := -1, false if er.windowMap != nil { index, ok = er.windowMap[v] } if !ok { er.err = ErrWindowInvalidWindowFuncUse.GenWithStackByArgs(strings.ToLower(v.F)) return inNode, true } er.ctxStackAppend(er.schema.Columns[index], er.names[index]) return inNode, true case *ast.FuncCallExpr: er.asScalar = true if _, ok := expression.DisableFoldFunctions[v.FnName.L]; ok { er.disableFoldCounter++ } if _, ok := expression.TryFoldFunctions[v.FnName.L]; ok { er.tryFoldCounter++ } case *ast.CaseExpr: er.asScalar = true if _, ok := expression.DisableFoldFunctions["case"]; ok { er.disableFoldCounter++ } if _, ok := expression.TryFoldFunctions["case"]; ok { er.tryFoldCounter++ } case *ast.BinaryOperationExpr: er.asScalar = true if v.Op == opcode.LogicAnd || v.Op == opcode.LogicOr { er.tryFoldCounter++ } case *ast.SetCollationExpr: // Do nothing default: er.asScalar = true } return inNode, false } func (er *expressionRewriter) buildSemiApplyFromEqualSubq(np LogicalPlan, l, r expression.Expression, not bool) { if er.asScalar || not { if expression.GetRowLen(r) == 1 { rCol := r.(*expression.Column) // If both input columns of `!= all / = any` expression are not null, we can treat the expression // as normal column equal condition. if !expression.ExprNotNull(l) || !expression.ExprNotNull(rCol) { rColCopy := *rCol rColCopy.InOperand = true r = &rColCopy } } else { rowFunc := r.(*expression.ScalarFunction) rargs := rowFunc.GetArgs() args := make([]expression.Expression, 0, len(rargs)) modified := false for i, rarg := range rargs { larg := expression.GetFuncArg(l, i) if !expression.ExprNotNull(larg) || !expression.ExprNotNull(rarg) { rCol := rarg.(*expression.Column) rColCopy := *rCol rColCopy.InOperand = true rarg = &rColCopy modified = true } args = append(args, rarg) } if modified { r, er.err = er.newFunction(ast.RowFunc, args[0].GetType(), args...) if er.err != nil { return } } } } var condition expression.Expression condition, er.err = er.constructBinaryOpFunction(l, r, ast.EQ) if er.err != nil { return } er.p, er.err = er.b.buildSemiApply(er.p, np, []expression.Expression{condition}, er.asScalar, not) } func (er *expressionRewriter) handleCompareSubquery(ctx context.Context, v *ast.CompareSubqueryExpr) (ast.Node, bool) { v.L.Accept(er) if er.err != nil { return v, true } lexpr := er.ctxStack[len(er.ctxStack)-1] subq, ok := v.R.(*ast.SubqueryExpr) if !ok { er.err = errors.Errorf("Unknown compare type %T.", v.R) return v, true } np, err := er.buildSubquery(ctx, subq) if err != nil { er.err = err return v, true } // Only (a,b,c) = any (...) and (a,b,c) != all (...) can use row expression. canMultiCol := (!v.All && v.Op == opcode.EQ) || (v.All && v.Op == opcode.NE) if !canMultiCol && (expression.GetRowLen(lexpr) != 1 || np.Schema().Len() != 1) { er.err = expression.ErrOperandColumns.GenWithStackByArgs(1) return v, true } lLen := expression.GetRowLen(lexpr) if lLen != np.Schema().Len() { er.err = expression.ErrOperandColumns.GenWithStackByArgs(lLen) return v, true } var rexpr expression.Expression if np.Schema().Len() == 1 { rexpr = np.Schema().Columns[0] } else { args := make([]expression.Expression, 0, np.Schema().Len()) for _, col := range np.Schema().Columns { args = append(args, col) } rexpr, er.err = er.newFunction(ast.RowFunc, args[0].GetType(), args...) if er.err != nil { return v, true } } switch v.Op { // Only EQ, NE and NullEQ can be composed with and. case opcode.EQ, opcode.NE, opcode.NullEQ: if v.Op == opcode.EQ { if v.All { er.handleEQAll(lexpr, rexpr, np) } else { // `a = any(subq)` will be rewriten as `a in (subq)`. er.buildSemiApplyFromEqualSubq(np, lexpr, rexpr, false) if er.err != nil { return v, true } } } else if v.Op == opcode.NE { if v.All { // `a != all(subq)` will be rewriten as `a not in (subq)`. er.buildSemiApplyFromEqualSubq(np, lexpr, rexpr, true) if er.err != nil { return v, true } } else { er.handleNEAny(lexpr, rexpr, np) } } else { // TODO: Support this in future. er.err = errors.New("We don't support <=> all or <=> any now") return v, true } default: // When < all or > any , the agg function should use min. useMin := ((v.Op == opcode.LT || v.Op == opcode.LE) && v.All) || ((v.Op == opcode.GT || v.Op == opcode.GE) && !v.All) er.handleOtherComparableSubq(lexpr, rexpr, np, useMin, v.Op.String(), v.All) } if er.asScalar { // The parent expression only use the last column in schema, which represents whether the condition is matched. er.ctxStack[len(er.ctxStack)-1] = er.p.Schema().Columns[er.p.Schema().Len()-1] er.ctxNameStk[len(er.ctxNameStk)-1] = er.p.OutputNames()[er.p.Schema().Len()-1] } return v, true } // handleOtherComparableSubq handles the queries like < any, < max, etc. For example, if the query is t.id < any (select s.id from s), // it will be rewrote to t.id < (select max(s.id) from s). func (er *expressionRewriter) handleOtherComparableSubq(lexpr, rexpr expression.Expression, np LogicalPlan, useMin bool, cmpFunc string, all bool) { plan4Agg := LogicalAggregation{}.Init(er.sctx, er.b.getSelectOffset()) if hint := er.b.TableHints(); hint != nil { plan4Agg.aggHints = hint.aggHints } plan4Agg.SetChildren(np) // Create a "max" or "min" aggregation. funcName := ast.AggFuncMax if useMin { funcName = ast.AggFuncMin } funcMaxOrMin, err := aggregation.NewAggFuncDesc(er.sctx, funcName, []expression.Expression{rexpr}, false) if err != nil { er.err = err return } // Create a column and append it to the schema of that aggregation. colMaxOrMin := &expression.Column{ UniqueID: er.sctx.GetSessionVars().AllocPlanColumnID(), RetType: funcMaxOrMin.RetTp, } schema := expression.NewSchema(colMaxOrMin) plan4Agg.names = append(plan4Agg.names, types.EmptyName) plan4Agg.SetSchema(schema) plan4Agg.AggFuncs = []*aggregation.AggFuncDesc{funcMaxOrMin} cond := expression.NewFunctionInternal(er.sctx, cmpFunc, types.NewFieldType(mysql.TypeTiny), lexpr, colMaxOrMin) er.buildQuantifierPlan(plan4Agg, cond, lexpr, rexpr, all) } // buildQuantifierPlan adds extra condition for any / all subquery. func (er *expressionRewriter) buildQuantifierPlan(plan4Agg *LogicalAggregation, cond, lexpr, rexpr expression.Expression, all bool) { innerIsNull := expression.NewFunctionInternal(er.sctx, ast.IsNull, types.NewFieldType(mysql.TypeTiny), rexpr) outerIsNull := expression.NewFunctionInternal(er.sctx, ast.IsNull, types.NewFieldType(mysql.TypeTiny), lexpr) funcSum, err := aggregation.NewAggFuncDesc(er.sctx, ast.AggFuncSum, []expression.Expression{innerIsNull}, false) if err != nil { er.err = err return } colSum := &expression.Column{ UniqueID: er.sctx.GetSessionVars().AllocPlanColumnID(), RetType: funcSum.RetTp, } plan4Agg.AggFuncs = append(plan4Agg.AggFuncs, funcSum) plan4Agg.schema.Append(colSum) innerHasNull := expression.NewFunctionInternal(er.sctx, ast.NE, types.NewFieldType(mysql.TypeTiny), colSum, expression.NewZero()) // Build `count(1)` aggregation to check if subquery is empty. funcCount, err := aggregation.NewAggFuncDesc(er.sctx, ast.AggFuncCount, []expression.Expression{expression.NewOne()}, false) if err != nil { er.err = err return } colCount := &expression.Column{ UniqueID: er.sctx.GetSessionVars().AllocPlanColumnID(), RetType: funcCount.RetTp, } plan4Agg.AggFuncs = append(plan4Agg.AggFuncs, funcCount) plan4Agg.schema.Append(colCount) if all { // All of the inner record set should not contain null value. So for t.id < all(select s.id from s), it // should be rewrote to t.id < min(s.id) and if(sum(s.id is null) != 0, null, true). innerNullChecker := expression.NewFunctionInternal(er.sctx, ast.If, types.NewFieldType(mysql.TypeTiny), innerHasNull, expression.NewNull(), expression.NewOne()) cond = expression.ComposeCNFCondition(er.sctx, cond, innerNullChecker) // If the subquery is empty, it should always return true. emptyChecker := expression.NewFunctionInternal(er.sctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), colCount, expression.NewZero()) // If outer key is null, and subquery is not empty, it should always return null, even when it is `null = all (1, 2)`. outerNullChecker := expression.NewFunctionInternal(er.sctx, ast.If, types.NewFieldType(mysql.TypeTiny), outerIsNull, expression.NewNull(), expression.NewZero()) cond = expression.ComposeDNFCondition(er.sctx, cond, emptyChecker, outerNullChecker) } else { // For "any" expression, if the subquery has null and the cond returns false, the result should be NULL. // Specifically, `t.id < any (select s.id from s)` would be rewrote to `t.id < max(s.id) or if(sum(s.id is null) != 0, null, false)` innerNullChecker := expression.NewFunctionInternal(er.sctx, ast.If, types.NewFieldType(mysql.TypeTiny), innerHasNull, expression.NewNull(), expression.NewZero()) cond = expression.ComposeDNFCondition(er.sctx, cond, innerNullChecker) // If the subquery is empty, it should always return false. emptyChecker := expression.NewFunctionInternal(er.sctx, ast.NE, types.NewFieldType(mysql.TypeTiny), colCount, expression.NewZero()) // If outer key is null, and subquery is not empty, it should return null. outerNullChecker := expression.NewFunctionInternal(er.sctx, ast.If, types.NewFieldType(mysql.TypeTiny), outerIsNull, expression.NewNull(), expression.NewOne()) cond = expression.ComposeCNFCondition(er.sctx, cond, emptyChecker, outerNullChecker) } // TODO: Add a Projection if any argument of aggregate funcs or group by items are scalar functions. // plan4Agg.buildProjectionIfNecessary() if !er.asScalar { // For Semi LogicalApply without aux column, the result is no matter false or null. So we can add it to join predicate. er.p, er.err = er.b.buildSemiApply(er.p, plan4Agg, []expression.Expression{cond}, false, false) return } // If we treat the result as a scalar value, we will add a projection with a extra column to output true, false or null. outerSchemaLen := er.p.Schema().Len() er.p = er.b.buildApplyWithJoinType(er.p, plan4Agg, InnerJoin) joinSchema := er.p.Schema() proj := LogicalProjection{ Exprs: expression.Column2Exprs(joinSchema.Clone().Columns[:outerSchemaLen]), }.Init(er.sctx, er.b.getSelectOffset()) proj.names = make([]*types.FieldName, outerSchemaLen, outerSchemaLen+1) copy(proj.names, er.p.OutputNames()) proj.SetSchema(expression.NewSchema(joinSchema.Clone().Columns[:outerSchemaLen]...)) proj.Exprs = append(proj.Exprs, cond) proj.schema.Append(&expression.Column{ UniqueID: er.sctx.GetSessionVars().AllocPlanColumnID(), RetType: cond.GetType(), }) proj.names = append(proj.names, types.EmptyName) proj.SetChildren(er.p) er.p = proj } // handleNEAny handles the case of != any. For example, if the query is t.id != any (select s.id from s), it will be rewrote to // t.id != s.id or count(distinct s.id) > 1 or [any checker]. If there are two different values in s.id , // there must exist a s.id that doesn't equal to t.id. func (er *expressionRewriter) handleNEAny(lexpr, rexpr expression.Expression, np LogicalPlan) { // If there is NULL in s.id column, s.id should be the value that isn't null in condition t.id != s.id. // So use function max to filter NULL. maxFunc, err := aggregation.NewAggFuncDesc(er.sctx, ast.AggFuncMax, []expression.Expression{rexpr}, false) if err != nil { er.err = err return } countFunc, err := aggregation.NewAggFuncDesc(er.sctx, ast.AggFuncCount, []expression.Expression{rexpr}, true) if err != nil { er.err = err return } plan4Agg := LogicalAggregation{ AggFuncs: []*aggregation.AggFuncDesc{maxFunc, countFunc}, }.Init(er.sctx, er.b.getSelectOffset()) if hint := er.b.TableHints(); hint != nil { plan4Agg.aggHints = hint.aggHints } plan4Agg.SetChildren(np) maxResultCol := &expression.Column{ UniqueID: er.sctx.GetSessionVars().AllocPlanColumnID(), RetType: maxFunc.RetTp, } count := &expression.Column{ UniqueID: er.sctx.GetSessionVars().AllocPlanColumnID(), RetType: countFunc.RetTp, } plan4Agg.names = append(plan4Agg.names, types.EmptyName, types.EmptyName) plan4Agg.SetSchema(expression.NewSchema(maxResultCol, count)) gtFunc := expression.NewFunctionInternal(er.sctx, ast.GT, types.NewFieldType(mysql.TypeTiny), count, expression.NewOne()) neCond := expression.NewFunctionInternal(er.sctx, ast.NE, types.NewFieldType(mysql.TypeTiny), lexpr, maxResultCol) cond := expression.ComposeDNFCondition(er.sctx, gtFunc, neCond) er.buildQuantifierPlan(plan4Agg, cond, lexpr, rexpr, false) } // handleEQAll handles the case of = all. For example, if the query is t.id = all (select s.id from s), it will be rewrote to // t.id = (select s.id from s having count(distinct s.id) <= 1 and [all checker]). func (er *expressionRewriter) handleEQAll(lexpr, rexpr expression.Expression, np LogicalPlan) { firstRowFunc, err := aggregation.NewAggFuncDesc(er.sctx, ast.AggFuncFirstRow, []expression.Expression{rexpr}, false) if err != nil { er.err = err return } countFunc, err := aggregation.NewAggFuncDesc(er.sctx, ast.AggFuncCount, []expression.Expression{rexpr}, true) if err != nil { er.err = err return } plan4Agg := LogicalAggregation{ AggFuncs: []*aggregation.AggFuncDesc{firstRowFunc, countFunc}, }.Init(er.sctx, er.b.getSelectOffset()) if hint := er.b.TableHints(); hint != nil { plan4Agg.aggHints = hint.aggHints } plan4Agg.SetChildren(np) plan4Agg.names = append(plan4Agg.names, types.EmptyName) firstRowResultCol := &expression.Column{ UniqueID: er.sctx.GetSessionVars().AllocPlanColumnID(), RetType: firstRowFunc.RetTp, } plan4Agg.names = append(plan4Agg.names, types.EmptyName) count := &expression.Column{ UniqueID: er.sctx.GetSessionVars().AllocPlanColumnID(), RetType: countFunc.RetTp, } plan4Agg.SetSchema(expression.NewSchema(firstRowResultCol, count)) leFunc := expression.NewFunctionInternal(er.sctx, ast.LE, types.NewFieldType(mysql.TypeTiny), count, expression.NewOne()) eqCond := expression.NewFunctionInternal(er.sctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), lexpr, firstRowResultCol) cond := expression.ComposeCNFCondition(er.sctx, leFunc, eqCond) er.buildQuantifierPlan(plan4Agg, cond, lexpr, rexpr, true) } func (er *expressionRewriter) handleExistSubquery(ctx context.Context, v *ast.ExistsSubqueryExpr) (ast.Node, bool) { subq, ok := v.Sel.(*ast.SubqueryExpr) if !ok { er.err = errors.Errorf("Unknown exists type %T.", v.Sel) return v, true } np, err := er.buildSubquery(ctx, subq) if err != nil { er.err = err return v, true } np = er.popExistsSubPlan(np) if len(ExtractCorrelatedCols(np)) > 0 { er.p, er.err = er.b.buildSemiApply(er.p, np, nil, er.asScalar, v.Not) if er.err != nil || !er.asScalar { return v, true } er.ctxStackAppend(er.p.Schema().Columns[er.p.Schema().Len()-1], er.p.OutputNames()[er.p.Schema().Len()-1]) } else { physicalPlan, _, err := DoOptimize(ctx, er.sctx, er.b.optFlag, np) if err != nil { er.err = err return v, true } row, err := EvalSubqueryFirstRow(ctx, physicalPlan, er.b.is, er.b.ctx) if err != nil { er.err = err return v, true } if (row != nil && !v.Not) || (row == nil && v.Not) { er.ctxStackAppend(expression.NewOne(), types.EmptyName) } else { er.ctxStackAppend(expression.NewZero(), types.EmptyName) } } return v, true } // popExistsSubPlan will remove the useless plan in exist's child. // See comments inside the method for more details. func (er *expressionRewriter) popExistsSubPlan(p LogicalPlan) LogicalPlan { out: for { switch plan := p.(type) { // This can be removed when in exists clause, // e.g. exists(select count(*) from t order by a) is equal to exists t. case *LogicalProjection, *LogicalSort: p = p.Children()[0] case *LogicalAggregation: if len(plan.GroupByItems) == 0 { p = LogicalTableDual{RowCount: 1}.Init(er.sctx, er.b.getSelectOffset()) break out } p = p.Children()[0] default: break out } } return p } func (er *expressionRewriter) handleInSubquery(ctx context.Context, v *ast.PatternInExpr) (ast.Node, bool) { asScalar := er.asScalar er.asScalar = true v.Expr.Accept(er) if er.err != nil { return v, true } lexpr := er.ctxStack[len(er.ctxStack)-1] subq, ok := v.Sel.(*ast.SubqueryExpr) if !ok { er.err = errors.Errorf("Unknown compare type %T.", v.Sel) return v, true } np, err := er.buildSubquery(ctx, subq) if err != nil { er.err = err return v, true } lLen := expression.GetRowLen(lexpr) if lLen != np.Schema().Len() { er.err = expression.ErrOperandColumns.GenWithStackByArgs(lLen) return v, true } var rexpr expression.Expression if np.Schema().Len() == 1 { rexpr = np.Schema().Columns[0] rCol := rexpr.(*expression.Column) // For AntiSemiJoin/LeftOuterSemiJoin/AntiLeftOuterSemiJoin, we cannot treat `in` expression as // normal column equal condition, so we specially mark the inner operand here. if v.Not || asScalar { // If both input columns of `in` expression are not null, we can treat the expression // as normal column equal condition instead. if !expression.ExprNotNull(lexpr) || !expression.ExprNotNull(rCol) { rColCopy := *rCol rColCopy.InOperand = true rexpr = &rColCopy } } } else { args := make([]expression.Expression, 0, np.Schema().Len()) for i, col := range np.Schema().Columns { larg := expression.GetFuncArg(lexpr, i) if !expression.ExprNotNull(larg) || !expression.ExprNotNull(col) { rarg := *col rarg.InOperand = true col = &rarg } args = append(args, col) } rexpr, er.err = er.newFunction(ast.RowFunc, args[0].GetType(), args...) if er.err != nil { return v, true } } checkCondition, err := er.constructBinaryOpFunction(lexpr, rexpr, ast.EQ) if err != nil { er.err = err return v, true } // If the leftKey and the rightKey have different collations, don't convert the sub-query to an inner-join // since when converting we will add a distinct-agg upon the right child and this distinct-agg doesn't have the right collation. // To keep it simple, we forbid this converting if they have different collations. lt, rt := lexpr.GetType(), rexpr.GetType() collFlag := collate.CompatibleCollate(lt.Collate, rt.Collate) // If it's not the form of `not in (SUBQUERY)`, // and has no correlated column from the current level plan(if the correlated column is from upper level, // we can treat it as constant, because the upper LogicalApply cannot be eliminated since current node is a join node), // and don't need to append a scalar value, we can rewrite it to inner join. if er.sctx.GetSessionVars().GetAllowInSubqToJoinAndAgg() && !v.Not && !asScalar && len(extractCorColumnsBySchema(np, er.p.Schema())) == 0 && collFlag { // We need to try to eliminate the agg and the projection produced by this operation. er.b.optFlag |= flagEliminateAgg er.b.optFlag |= flagEliminateProjection er.b.optFlag |= flagJoinReOrder // Build distinct for the inner query. agg, err := er.b.buildDistinct(np, np.Schema().Len()) if err != nil { er.err = err return v, true } // Build inner join above the aggregation. join := LogicalJoin{JoinType: InnerJoin}.Init(er.sctx, er.b.getSelectOffset()) join.SetChildren(er.p, agg) join.SetSchema(expression.MergeSchema(er.p.Schema(), agg.schema)) join.names = make([]*types.FieldName, er.p.Schema().Len()+agg.Schema().Len()) copy(join.names, er.p.OutputNames()) copy(join.names[er.p.Schema().Len():], agg.OutputNames()) join.AttachOnConds(expression.SplitCNFItems(checkCondition)) // Set join hint for this join. if er.b.TableHints() != nil { join.setPreferredJoinType(er.b.TableHints()) } er.p = join } else { er.p, er.err = er.b.buildSemiApply(er.p, np, expression.SplitCNFItems(checkCondition), asScalar, v.Not) if er.err != nil { return v, true } } er.ctxStackPop(1) if asScalar { col := er.p.Schema().Columns[er.p.Schema().Len()-1] er.ctxStackAppend(col, er.p.OutputNames()[er.p.Schema().Len()-1]) } return v, true } func (er *expressionRewriter) handleScalarSubquery(ctx context.Context, v *ast.SubqueryExpr) (ast.Node, bool) { np, err := er.buildSubquery(ctx, v) if err != nil { er.err = err return v, true } np = er.b.buildMaxOneRow(np) if len(ExtractCorrelatedCols(np)) > 0 { er.p = er.b.buildApplyWithJoinType(er.p, np, LeftOuterJoin) if np.Schema().Len() > 1 { newCols := make([]expression.Expression, 0, np.Schema().Len()) for _, col := range np.Schema().Columns { newCols = append(newCols, col) } expr, err1 := er.newFunction(ast.RowFunc, newCols[0].GetType(), newCols...) if err1 != nil { er.err = err1 return v, true } er.ctxStackAppend(expr, types.EmptyName) } else { er.ctxStackAppend(er.p.Schema().Columns[er.p.Schema().Len()-1], er.p.OutputNames()[er.p.Schema().Len()-1]) } return v, true } physicalPlan, _, err := DoOptimize(ctx, er.sctx, er.b.optFlag, np) if err != nil { er.err = err return v, true } row, err := EvalSubqueryFirstRow(ctx, physicalPlan, er.b.is, er.b.ctx) if err != nil { er.err = err return v, true } if np.Schema().Len() > 1 { newCols := make([]expression.Expression, 0, np.Schema().Len()) for i, data := range row { newCols = append(newCols, &expression.Constant{ Value: data, RetType: np.Schema().Columns[i].GetType()}) } expr, err1 := er.newFunction(ast.RowFunc, newCols[0].GetType(), newCols...) if err1 != nil { er.err = err1 return v, true } er.ctxStackAppend(expr, types.EmptyName) } else { er.ctxStackAppend(&expression.Constant{ Value: row[0], RetType: np.Schema().Columns[0].GetType(), }, types.EmptyName) } return v, true } // Leave implements Visitor interface. func (er *expressionRewriter) Leave(originInNode ast.Node) (retNode ast.Node, ok bool) { if er.err != nil { return retNode, false } var inNode = originInNode if er.preprocess != nil { inNode = er.preprocess(inNode) } switch v := inNode.(type) { case *ast.AggregateFuncExpr, *ast.ColumnNameExpr, *ast.ParenthesesExpr, *ast.WhenClause, *ast.SubqueryExpr, *ast.ExistsSubqueryExpr, *ast.CompareSubqueryExpr, *ast.ValuesExpr, *ast.WindowFuncExpr, *ast.TableNameExpr: case *driver.ValueExpr: v.Datum.SetValue(v.Datum.GetValue(), &v.Type) value := &expression.Constant{Value: v.Datum, RetType: &v.Type} er.ctxStackAppend(value, types.EmptyName) case *driver.ParamMarkerExpr: var value expression.Expression value, er.err = expression.ParamMarkerExpression(er.sctx, v) if er.err != nil { return retNode, false } er.ctxStackAppend(value, types.EmptyName) case *ast.VariableExpr: er.rewriteVariable(v) case *ast.FuncCallExpr: if _, ok := expression.TryFoldFunctions[v.FnName.L]; ok { er.tryFoldCounter-- } er.funcCallToExpression(v) if _, ok := expression.DisableFoldFunctions[v.FnName.L]; ok { er.disableFoldCounter-- } case *ast.TableName: er.toTable(v) case *ast.ColumnName: er.toColumn(v) case *ast.UnaryOperationExpr: er.unaryOpToExpression(v) case *ast.BinaryOperationExpr: if v.Op == opcode.LogicAnd || v.Op == opcode.LogicOr { er.tryFoldCounter-- } er.binaryOpToExpression(v) case *ast.BetweenExpr: er.betweenToExpression(v) case *ast.CaseExpr: if _, ok := expression.TryFoldFunctions["case"]; ok { er.tryFoldCounter-- } er.caseToExpression(v) if _, ok := expression.DisableFoldFunctions["case"]; ok { er.disableFoldCounter-- } case *ast.FuncCastExpr: arg := er.ctxStack[len(er.ctxStack)-1] er.err = expression.CheckArgsNotMultiColumnRow(arg) if er.err != nil { return retNode, false } // check the decimal precision of "CAST(AS TIME)". er.err = er.checkTimePrecision(v.Tp) if er.err != nil { return retNode, false } er.ctxStack[len(er.ctxStack)-1] = expression.BuildCastFunction(er.sctx, arg, v.Tp) er.ctxNameStk[len(er.ctxNameStk)-1] = types.EmptyName case *ast.PatternLikeExpr: er.patternLikeToExpression(v) case *ast.PatternRegexpExpr: er.regexpToScalarFunc(v) case *ast.RowExpr: er.rowToScalarFunc(v) case *ast.PatternInExpr: if v.Sel == nil { er.inToExpression(len(v.List), v.Not, &v.Type) } case *ast.PositionExpr: er.positionToScalarFunc(v) case *ast.IsNullExpr: er.isNullToExpression(v) case *ast.IsTruthExpr: er.isTrueToScalarFunc(v) case *ast.DefaultExpr: er.evalDefaultExpr(v) // TODO: Perhaps we don't need to transcode these back to generic integers/strings case *ast.TrimDirectionExpr: er.ctxStackAppend(&expression.Constant{ Value: types.NewIntDatum(int64(v.Direction)), RetType: types.NewFieldType(mysql.TypeTiny), }, types.EmptyName) case *ast.TimeUnitExpr: er.ctxStackAppend(&expression.Constant{ Value: types.NewStringDatum(v.Unit.String()), RetType: types.NewFieldType(mysql.TypeVarchar), }, types.EmptyName) case *ast.GetFormatSelectorExpr: er.ctxStackAppend(&expression.Constant{ Value: types.NewStringDatum(v.Selector.String()), RetType: types.NewFieldType(mysql.TypeVarchar), }, types.EmptyName) case *ast.SetCollationExpr: arg := er.ctxStack[len(er.ctxStack)-1] if collate.NewCollationEnabled() { var collInfo *charset.Collation // TODO(bb7133): use charset.ValidCharsetAndCollation when its bug is fixed. if collInfo, er.err = collate.GetCollationByName(v.Collate); er.err != nil { break } chs := arg.GetType().Charset if chs != "" && collInfo.CharsetName != chs { er.err = charset.ErrCollationCharsetMismatch.GenWithStackByArgs(collInfo.Name, chs) break } } // SetCollationExpr sets the collation explicitly, even when the evaluation type of the expression is non-string. if _, ok := arg.(*expression.Column); ok { // Wrap a cast here to avoid changing the original FieldType of the column expression. exprType := arg.GetType().Clone() exprType.Collate = v.Collate casted := expression.BuildCastFunction(er.sctx, arg, exprType) er.ctxStackPop(1) er.ctxStackAppend(casted, types.EmptyName) } else { // For constant and scalar function, we can set its collate directly. arg.GetType().Collate = v.Collate } er.ctxStack[len(er.ctxStack)-1].SetCoercibility(expression.CoercibilityExplicit) default: er.err = errors.Errorf("UnknownType: %T", v) return retNode, false } if er.err != nil { return retNode, false } return originInNode, true } // newFunction chooses which expression.NewFunctionImpl() will be used. func (er *expressionRewriter) newFunction(funcName string, retType *types.FieldType, args ...expression.Expression) (expression.Expression, error) { if er.disableFoldCounter > 0 { return expression.NewFunctionBase(er.sctx, funcName, retType, args...) } if er.tryFoldCounter > 0 { return expression.NewFunctionTryFold(er.sctx, funcName, retType, args...) } return expression.NewFunction(er.sctx, funcName, retType, args...) } func (er *expressionRewriter) checkTimePrecision(ft *types.FieldType) error { if ft.EvalType() == types.ETDuration && ft.Decimal > int(types.MaxFsp) { return errTooBigPrecision.GenWithStackByArgs(ft.Decimal, "CAST", types.MaxFsp) } return nil } func (er *expressionRewriter) useCache() bool { return er.sctx.GetSessionVars().StmtCtx.UseCache } func (er *expressionRewriter) rewriteVariable(v *ast.VariableExpr) { stkLen := len(er.ctxStack) name := strings.ToLower(v.Name) sessionVars := er.b.ctx.GetSessionVars() if !v.IsSystem { if v.Value != nil { tp := er.ctxStack[stkLen-1].GetType() er.ctxStack[stkLen-1], er.err = er.newFunction(ast.SetVar, tp, expression.DatumToConstant(types.NewDatum(name), mysql.TypeString), er.ctxStack[stkLen-1]) er.ctxNameStk[stkLen-1] = types.EmptyName // Store the field type of the variable into SessionVars.UserVarTypes. // Normally we can infer the type from SessionVars.User, but we need SessionVars.UserVarTypes when // GetVar has not been executed to fill the SessionVars.Users. sessionVars.UsersLock.Lock() sessionVars.UserVarTypes[name] = tp sessionVars.UsersLock.Unlock() return } sessionVars.UsersLock.RLock() tp, ok := sessionVars.UserVarTypes[name] sessionVars.UsersLock.RUnlock() if !ok { tp = types.NewFieldType(mysql.TypeVarString) tp.Flen = mysql.MaxFieldVarCharLength } f, err := er.newFunction(ast.GetVar, tp, expression.DatumToConstant(types.NewStringDatum(name), mysql.TypeString)) if err != nil { er.err = err return } f.SetCoercibility(expression.CoercibilityImplicit) er.ctxStackAppend(f, types.EmptyName) return } var val string var err error if v.ExplicitScope { err = variable.ValidateGetSystemVar(name, v.IsGlobal) if err != nil { er.err = err return } } sysVar := variable.SysVars[name] if sysVar == nil { er.err = variable.ErrUnknownSystemVar.GenWithStackByArgs(name) return } // Variable is @@gobal.variable_name or variable is only global scope variable. if v.IsGlobal || sysVar.Scope == variable.ScopeGlobal { val, err = variable.GetGlobalSystemVar(sessionVars, name) } else { val, err = variable.GetSessionSystemVar(sessionVars, name) } if err != nil { er.err = err return } e := expression.DatumToConstant(types.NewStringDatum(val), mysql.TypeVarString) e.GetType().Charset, _ = er.sctx.GetSessionVars().GetSystemVar(variable.CharacterSetConnection) e.GetType().Collate, _ = er.sctx.GetSessionVars().GetSystemVar(variable.CollationConnection) er.ctxStackAppend(e, types.EmptyName) } func (er *expressionRewriter) unaryOpToExpression(v *ast.UnaryOperationExpr) { stkLen := len(er.ctxStack) var op string switch v.Op { case opcode.Plus: // expression (+ a) is equal to a return case opcode.Minus: op = ast.UnaryMinus case opcode.BitNeg: op = ast.BitNeg case opcode.Not, opcode.Not2: op = ast.UnaryNot default: er.err = errors.Errorf("Unknown Unary Op %T", v.Op) return } if expression.GetRowLen(er.ctxStack[stkLen-1]) != 1 { er.err = expression.ErrOperandColumns.GenWithStackByArgs(1) return } er.ctxStack[stkLen-1], er.err = er.newFunction(op, &v.Type, er.ctxStack[stkLen-1]) er.ctxNameStk[stkLen-1] = types.EmptyName } func (er *expressionRewriter) binaryOpToExpression(v *ast.BinaryOperationExpr) { stkLen := len(er.ctxStack) var function expression.Expression switch v.Op { case opcode.EQ, opcode.NE, opcode.NullEQ, opcode.GT, opcode.GE, opcode.LT, opcode.LE: function, er.err = er.constructBinaryOpFunction(er.ctxStack[stkLen-2], er.ctxStack[stkLen-1], v.Op.String()) default: lLen := expression.GetRowLen(er.ctxStack[stkLen-2]) rLen := expression.GetRowLen(er.ctxStack[stkLen-1]) if lLen != 1 || rLen != 1 { er.err = expression.ErrOperandColumns.GenWithStackByArgs(1) return } function, er.err = er.newFunction(v.Op.String(), types.NewFieldType(mysql.TypeUnspecified), er.ctxStack[stkLen-2:]...) } if er.err != nil { return } er.ctxStackPop(2) er.ctxStackAppend(function, types.EmptyName) } func (er *expressionRewriter) notToExpression(hasNot bool, op string, tp *types.FieldType, args ...expression.Expression) expression.Expression { opFunc, err := er.newFunction(op, tp, args...) if err != nil { er.err = err return nil } if !hasNot { return opFunc } opFunc, err = er.newFunction(ast.UnaryNot, tp, opFunc) if err != nil { er.err = err return nil } return opFunc } func (er *expressionRewriter) isNullToExpression(v *ast.IsNullExpr) { stkLen := len(er.ctxStack) if expression.GetRowLen(er.ctxStack[stkLen-1]) != 1 { er.err = expression.ErrOperandColumns.GenWithStackByArgs(1) return } function := er.notToExpression(v.Not, ast.IsNull, &v.Type, er.ctxStack[stkLen-1]) er.ctxStackPop(1) er.ctxStackAppend(function, types.EmptyName) } func (er *expressionRewriter) positionToScalarFunc(v *ast.PositionExpr) { pos := v.N str := strconv.Itoa(pos) if v.P != nil { stkLen := len(er.ctxStack) val := er.ctxStack[stkLen-1] intNum, isNull, err := expression.GetIntFromConstant(er.sctx, val) str = "?" if err == nil { if isNull { return } pos = intNum er.ctxStackPop(1) } er.err = err } if er.err == nil && pos > 0 && pos <= er.schema.Len() { er.ctxStackAppend(er.schema.Columns[pos-1], er.names[pos-1]) } else { er.err = ErrUnknownColumn.GenWithStackByArgs(str, clauseMsg[er.b.curClause]) } } func (er *expressionRewriter) isTrueToScalarFunc(v *ast.IsTruthExpr) { stkLen := len(er.ctxStack) op := ast.IsTruthWithoutNull if v.True == 0 { op = ast.IsFalsity } if expression.GetRowLen(er.ctxStack[stkLen-1]) != 1 { er.err = expression.ErrOperandColumns.GenWithStackByArgs(1) return } function := er.notToExpression(v.Not, op, &v.Type, er.ctxStack[stkLen-1]) er.ctxStackPop(1) er.ctxStackAppend(function, types.EmptyName) } // inToExpression converts in expression to a scalar function. The argument lLen means the length of in list. // The argument not means if the expression is not in. The tp stands for the expression type, which is always bool. // a in (b, c, d) will be rewritten as `(a = b) or (a = c) or (a = d)`. func (er *expressionRewriter) inToExpression(lLen int, not bool, tp *types.FieldType) { stkLen := len(er.ctxStack) l := expression.GetRowLen(er.ctxStack[stkLen-lLen-1]) for i := 0; i < lLen; i++ { if l != expression.GetRowLen(er.ctxStack[stkLen-lLen+i]) { er.err = expression.ErrOperandColumns.GenWithStackByArgs(l) return } } args := er.ctxStack[stkLen-lLen-1:] leftFt := args[0].GetType() leftEt, leftIsNull := leftFt.EvalType(), leftFt.Tp == mysql.TypeNull if leftIsNull { er.ctxStackPop(lLen + 1) er.ctxStackAppend(expression.NewNull(), types.EmptyName) return } if leftEt == types.ETInt { for i := 1; i < len(args); i++ { if c, ok := args[i].(*expression.Constant); ok { var isExceptional bool args[i], isExceptional = expression.RefineComparedConstant(er.sctx, *leftFt, c, opcode.EQ) if isExceptional { args[i] = c } } } } allSameType := true for _, arg := range args[1:] { if arg.GetType().Tp != mysql.TypeNull && expression.GetAccurateCmpType(args[0], arg) != leftEt { allSameType = false break } } var function expression.Expression if allSameType && l == 1 && lLen > 1 { function = er.notToExpression(not, ast.In, tp, er.ctxStack[stkLen-lLen-1:]...) } else { eqFunctions := make([]expression.Expression, 0, lLen) for i := stkLen - lLen; i < stkLen; i++ { expr, err := er.constructBinaryOpFunction(args[0], er.ctxStack[i], ast.EQ) if err != nil { er.err = err return } eqFunctions = append(eqFunctions, expr) } function = expression.ComposeDNFCondition(er.sctx, eqFunctions...) if not { var err error function, err = er.newFunction(ast.UnaryNot, tp, function) if err != nil { er.err = err return } } } er.ctxStackPop(lLen + 1) er.ctxStackAppend(function, types.EmptyName) } func (er *expressionRewriter) caseToExpression(v *ast.CaseExpr) { stkLen := len(er.ctxStack) argsLen := 2 * len(v.WhenClauses) if v.ElseClause != nil { argsLen++ } er.err = expression.CheckArgsNotMultiColumnRow(er.ctxStack[stkLen-argsLen:]...) if er.err != nil { return } // value -> ctxStack[stkLen-argsLen-1] // when clause(condition, result) -> ctxStack[stkLen-argsLen:stkLen-1]; // else clause -> ctxStack[stkLen-1] var args []expression.Expression if v.Value != nil { // args: eq scalar func(args: value, condition1), result1, // eq scalar func(args: value, condition2), result2, // ... // else clause value := er.ctxStack[stkLen-argsLen-1] args = make([]expression.Expression, 0, argsLen) for i := stkLen - argsLen; i < stkLen-1; i += 2 { arg, err := er.newFunction(ast.EQ, types.NewFieldType(mysql.TypeTiny), value, er.ctxStack[i]) if err != nil { er.err = err return } args = append(args, arg) args = append(args, er.ctxStack[i+1]) } if v.ElseClause != nil { args = append(args, er.ctxStack[stkLen-1]) } argsLen++ // for trimming the value element later } else { // args: condition1, result1, // condition2, result2, // ... // else clause args = er.ctxStack[stkLen-argsLen:] } function, err := er.newFunction(ast.Case, &v.Type, args...) if err != nil { er.err = err return } er.ctxStackPop(argsLen) er.ctxStackAppend(function, types.EmptyName) } func (er *expressionRewriter) patternLikeToExpression(v *ast.PatternLikeExpr) { l := len(er.ctxStack) er.err = expression.CheckArgsNotMultiColumnRow(er.ctxStack[l-2:]...) if er.err != nil { return } char, col := er.sctx.GetSessionVars().GetCharsetInfo() var function expression.Expression fieldType := &types.FieldType{} isPatternExactMatch := false // Treat predicate 'like' the same way as predicate '=' when it is an exact match. if patExpression, ok := er.ctxStack[l-1].(*expression.Constant); ok { patString, isNull, err := patExpression.EvalString(nil, chunk.Row{}) if err != nil { er.err = err return } if !isNull { patValue, patTypes := stringutil.CompilePattern(patString, v.Escape) if stringutil.IsExactMatch(patTypes) && er.ctxStack[l-2].GetType().EvalType() == types.ETString { op := ast.EQ if v.Not { op = ast.NE } types.DefaultTypeForValue(string(patValue), fieldType, char, col) function, er.err = er.constructBinaryOpFunction(er.ctxStack[l-2], &expression.Constant{Value: types.NewStringDatum(string(patValue)), RetType: fieldType}, op) isPatternExactMatch = true } } } if !isPatternExactMatch { types.DefaultTypeForValue(int(v.Escape), fieldType, char, col) function = er.notToExpression(v.Not, ast.Like, &v.Type, er.ctxStack[l-2], er.ctxStack[l-1], &expression.Constant{Value: types.NewIntDatum(int64(v.Escape)), RetType: fieldType}) } er.ctxStackPop(2) er.ctxStackAppend(function, types.EmptyName) } func (er *expressionRewriter) regexpToScalarFunc(v *ast.PatternRegexpExpr) { l := len(er.ctxStack) er.err = expression.CheckArgsNotMultiColumnRow(er.ctxStack[l-2:]...) if er.err != nil { return } function := er.notToExpression(v.Not, ast.Regexp, &v.Type, er.ctxStack[l-2], er.ctxStack[l-1]) er.ctxStackPop(2) er.ctxStackAppend(function, types.EmptyName) } func (er *expressionRewriter) rowToScalarFunc(v *ast.RowExpr) { stkLen := len(er.ctxStack) length := len(v.Values) rows := make([]expression.Expression, 0, length) for i := stkLen - length; i < stkLen; i++ { rows = append(rows, er.ctxStack[i]) } er.ctxStackPop(length) function, err := er.newFunction(ast.RowFunc, rows[0].GetType(), rows...) if err != nil { er.err = err return } er.ctxStackAppend(function, types.EmptyName) } func (er *expressionRewriter) betweenToExpression(v *ast.BetweenExpr) { stkLen := len(er.ctxStack) er.err = expression.CheckArgsNotMultiColumnRow(er.ctxStack[stkLen-3:]...) if er.err != nil { return } expr, lexp, rexp := er.ctxStack[stkLen-3], er.ctxStack[stkLen-2], er.ctxStack[stkLen-1] if expression.GetCmpTp4MinMax([]expression.Expression{expr, lexp, rexp}) == types.ETDatetime { expr = expression.WrapWithCastAsTime(er.sctx, expr, types.NewFieldType(mysql.TypeDatetime)) lexp = expression.WrapWithCastAsTime(er.sctx, lexp, types.NewFieldType(mysql.TypeDatetime)) rexp = expression.WrapWithCastAsTime(er.sctx, rexp, types.NewFieldType(mysql.TypeDatetime)) } var op string var l, r expression.Expression l, er.err = er.newFunction(ast.GE, &v.Type, expr, lexp) if er.err == nil { r, er.err = er.newFunction(ast.LE, &v.Type, expr, rexp) } op = ast.LogicAnd if er.err != nil { return } function, err := er.newFunction(op, &v.Type, l, r) if err != nil { er.err = err return } if v.Not { function, err = er.newFunction(ast.UnaryNot, &v.Type, function) if err != nil { er.err = err return } } er.ctxStackPop(3) er.ctxStackAppend(function, types.EmptyName) } // rewriteFuncCall handles a FuncCallExpr and generates a customized function. // It should return true if for the given FuncCallExpr a rewrite is performed so that original behavior is skipped. // Otherwise it should return false to indicate (the caller) that original behavior needs to be performed. func (er *expressionRewriter) rewriteFuncCall(v *ast.FuncCallExpr) bool { switch v.FnName.L { // when column is not null, ifnull on such column is not necessary. case ast.Ifnull: if len(v.Args) != 2 { er.err = expression.ErrIncorrectParameterCount.GenWithStackByArgs(v.FnName.O) return true } stackLen := len(er.ctxStack) arg1 := er.ctxStack[stackLen-2] col, isColumn := arg1.(*expression.Column) // if expr1 is a column and column has not null flag, then we can eliminate ifnull on // this column. if isColumn && mysql.HasNotNullFlag(col.RetType.Flag) { name := er.ctxNameStk[stackLen-2] newCol := col.Clone().(*expression.Column) er.ctxStackPop(len(v.Args)) er.ctxStackAppend(newCol, name) return true } return false case ast.Nullif: if len(v.Args) != 2 { er.err = expression.ErrIncorrectParameterCount.GenWithStackByArgs(v.FnName.O) return true } stackLen := len(er.ctxStack) param1 := er.ctxStack[stackLen-2] param2 := er.ctxStack[stackLen-1] // param1 = param2 funcCompare, err := er.constructBinaryOpFunction(param1, param2, ast.EQ) if err != nil { er.err = err return true } // NULL nullTp := types.NewFieldType(mysql.TypeNull) nullTp.Flen, nullTp.Decimal = mysql.GetDefaultFieldLengthAndDecimal(mysql.TypeNull) paramNull := &expression.Constant{ Value: types.NewDatum(nil), RetType: nullTp, } // if(param1 = param2, NULL, param1) funcIf, err := er.newFunction(ast.If, &v.Type, funcCompare, paramNull, param1) if err != nil { er.err = err return true } er.ctxStackPop(len(v.Args)) er.ctxStackAppend(funcIf, types.EmptyName) return true default: return false } } func (er *expressionRewriter) funcCallToExpression(v *ast.FuncCallExpr) { stackLen := len(er.ctxStack) args := er.ctxStack[stackLen-len(v.Args):] er.err = expression.CheckArgsNotMultiColumnRow(args...) if er.err != nil { return } if er.rewriteFuncCall(v) { return } var function expression.Expression er.ctxStackPop(len(v.Args)) if _, ok := expression.DeferredFunctions[v.FnName.L]; er.useCache() && ok { // When the expression is unix_timestamp and the number of argument is not zero, // we deal with it as normal expression. if v.FnName.L == ast.UnixTimestamp && len(v.Args) != 0 { function, er.err = er.newFunction(v.FnName.L, &v.Type, args...) er.ctxStackAppend(function, types.EmptyName) } else { function, er.err = expression.NewFunctionBase(er.sctx, v.FnName.L, &v.Type, args...) c := &expression.Constant{Value: types.NewDatum(nil), RetType: function.GetType().Clone(), DeferredExpr: function} er.ctxStackAppend(c, types.EmptyName) } } else { function, er.err = er.newFunction(v.FnName.L, &v.Type, args...) er.ctxStackAppend(function, types.EmptyName) } } // Now TableName in expression only used by sequence function like nextval(seq). // The function arg should be evaluated as a table name rather than normal column name like mysql does. func (er *expressionRewriter) toTable(v *ast.TableName) { fullName := v.Name.L if len(v.Schema.L) != 0 { fullName = v.Schema.L + "." + fullName } val := &expression.Constant{ Value: types.NewDatum(fullName), RetType: types.NewFieldType(mysql.TypeString), } er.ctxStackAppend(val, types.EmptyName) } func (er *expressionRewriter) toColumn(v *ast.ColumnName) { idx, err := expression.FindFieldName(er.names, v) if err != nil { er.err = ErrAmbiguous.GenWithStackByArgs(v.Name, clauseMsg[fieldList]) return } if idx >= 0 { column := er.schema.Columns[idx] if column.IsHidden { er.err = ErrUnknownColumn.GenWithStackByArgs(v.Name, clauseMsg[er.b.curClause]) return } er.ctxStackAppend(column, er.names[idx]) return } for i := len(er.b.outerSchemas) - 1; i >= 0; i-- { outerSchema, outerName := er.b.outerSchemas[i], er.b.outerNames[i] idx, err = expression.FindFieldName(outerName, v) if idx >= 0 { column := outerSchema.Columns[idx] er.ctxStackAppend(&expression.CorrelatedColumn{Column: *column, Data: new(types.Datum)}, outerName[idx]) return } if err != nil { er.err = ErrAmbiguous.GenWithStackByArgs(v.Name, clauseMsg[fieldList]) return } } if _, ok := er.p.(*LogicalUnionAll); ok && v.Table.O != "" { er.err = ErrTablenameNotAllowedHere.GenWithStackByArgs(v.Table.O, "SELECT", clauseMsg[er.b.curClause]) return } col, name, err := findFieldNameFromNaturalUsingJoin(er.p, v) if err != nil { er.err = err return } else if col != nil { er.ctxStackAppend(col, name) return } if er.b.curClause == globalOrderByClause { er.b.curClause = orderByClause } er.err = ErrUnknownColumn.GenWithStackByArgs(v.String(), clauseMsg[er.b.curClause]) } func findFieldNameFromNaturalUsingJoin(p LogicalPlan, v *ast.ColumnName) (col *expression.Column, name *types.FieldName, err error) { switch x := p.(type) { case *LogicalLimit, *LogicalSelection, *LogicalTopN, *LogicalSort, *LogicalMaxOneRow: return findFieldNameFromNaturalUsingJoin(p.Children()[0], v) case *LogicalJoin: if x.redundantSchema != nil { idx, err := expression.FindFieldName(x.redundantNames, v) if err != nil { return nil, nil, err } if idx >= 0 { return x.redundantSchema.Columns[idx], x.redundantNames[idx], nil } } } return nil, nil, nil } func (er *expressionRewriter) evalDefaultExpr(v *ast.DefaultExpr) { var name *types.FieldName // Here we will find the corresponding column for default function. At the same time, we need to consider the issue // of subquery and name space. // For example, we have two tables t1(a int default 1, b int) and t2(a int default -1, c int). Consider the following SQL: // select a from t1 where a > (select default(a) from t2) // Refer to the behavior of MySQL, we need to find column a in table t2. If table t2 does not have column a, then find it // in table t1. If there are none, return an error message. // Based on the above description, we need to look in er.b.allNames from back to front. for i := len(er.b.allNames) - 1; i >= 0; i-- { idx, err := expression.FindFieldName(er.b.allNames[i], v.Name) if err != nil { er.err = err return } if idx >= 0 { name = er.b.allNames[i][idx] break } } if name == nil { idx, err := expression.FindFieldName(er.names, v.Name) if err != nil { er.err = err return } if idx < 0 { er.err = ErrUnknownColumn.GenWithStackByArgs(v.Name.OrigColName(), "field list") return } name = er.names[idx] } dbName := name.DBName if dbName.O == "" { // if database name is not specified, use current database name dbName = model.NewCIStr(er.sctx.GetSessionVars().CurrentDB) } if name.OrigTblName.O == "" { // column is evaluated by some expressions, for example: // `select default(c) from (select (a+1) as c from t) as t0` // in such case, a 'no default' error is returned er.err = table.ErrNoDefaultValue.GenWithStackByArgs(name.ColName) return } var tbl table.Table tbl, er.err = er.b.is.TableByName(dbName, name.OrigTblName) if er.err != nil { return } colName := name.OrigColName.O if colName == "" { // in some cases, OrigColName is empty, use ColName instead colName = name.ColName.O } col := table.FindCol(tbl.Cols(), colName) if col == nil { er.err = ErrUnknownColumn.GenWithStackByArgs(v.Name, "field_list") return } isCurrentTimestamp := hasCurrentDatetimeDefault(col) var val *expression.Constant switch { case isCurrentTimestamp && col.Tp == mysql.TypeDatetime: // for DATETIME column with current_timestamp, use NULL to be compatible with MySQL 5.7 val = expression.NewNull() case isCurrentTimestamp && col.Tp == mysql.TypeTimestamp: // for TIMESTAMP column with current_timestamp, use 0 to be compatible with MySQL 5.7 zero := types.NewTime(types.ZeroCoreTime, mysql.TypeTimestamp, int8(col.Decimal)) val = &expression.Constant{ Value: types.NewDatum(zero), RetType: types.NewFieldType(mysql.TypeTimestamp), } default: // for other columns, just use what it is val, er.err = er.b.getDefaultValue(col) } if er.err != nil { return } er.ctxStackAppend(val, types.EmptyName) } // hasCurrentDatetimeDefault checks if column has current_timestamp default value func hasCurrentDatetimeDefault(col *table.Column) bool { x, ok := col.DefaultValue.(string) if !ok { return false } return strings.ToLower(x) == ast.CurrentTimestamp }