You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
 
 

4736 lines
157 KiB

// 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"
"fmt"
"math"
"math/bits"
"reflect"
"sort"
"strconv"
"strings"
"time"
"unicode"
"github.com/cznic/mathutil"
"github.com/pingcap/errors"
"github.com/pingcap/parser"
"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/format"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/parser/opcode"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/expression/aggregation"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/planner/property"
"github.com/pingcap/tidb/planner/util"
"github.com/pingcap/tidb/privilege"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/table/tables"
"github.com/pingcap/tidb/types"
driver "github.com/pingcap/tidb/types/parser_driver"
util2 "github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/chunk"
utilhint "github.com/pingcap/tidb/util/hint"
"github.com/pingcap/tidb/util/plancodec"
"github.com/pingcap/tidb/util/set"
)
const (
// TiDBMergeJoin is hint enforce merge join.
TiDBMergeJoin = "tidb_smj"
// HintSMJ is hint enforce merge join.
HintSMJ = "merge_join"
// TiDBBroadCastJoin indicates applying broadcast join by force.
TiDBBroadCastJoin = "tidb_bcj"
// HintBCJ indicates applying broadcast join by force.
HintBCJ = "broadcast_join"
// HintBCJPreferLocal specifies the preferred local read table
HintBCJPreferLocal = "broadcast_join_local"
// TiDBIndexNestedLoopJoin is hint enforce index nested loop join.
TiDBIndexNestedLoopJoin = "tidb_inlj"
// HintINLJ is hint enforce index nested loop join.
HintINLJ = "inl_join"
// HintINLHJ is hint enforce index nested loop hash join.
HintINLHJ = "inl_hash_join"
// HintINLMJ is hint enforce index nested loop merge join.
HintINLMJ = "inl_merge_join"
// TiDBHashJoin is hint enforce hash join.
TiDBHashJoin = "tidb_hj"
// HintHJ is hint enforce hash join.
HintHJ = "hash_join"
// HintHashAgg is hint enforce hash aggregation.
HintHashAgg = "hash_agg"
// HintStreamAgg is hint enforce stream aggregation.
HintStreamAgg = "stream_agg"
// HintUseIndex is hint enforce using some indexes.
HintUseIndex = "use_index"
// HintIgnoreIndex is hint enforce ignoring some indexes.
HintIgnoreIndex = "ignore_index"
// HintAggToCop is hint enforce pushing aggregation to coprocessor.
HintAggToCop = "agg_to_cop"
// HintReadFromStorage is hint enforce some tables read from specific type of storage.
HintReadFromStorage = "read_from_storage"
// HintTiFlash is a label represents the tiflash storage type.
HintTiFlash = "tiflash"
// HintTiKV is a label represents the tikv storage type.
HintTiKV = "tikv"
// HintIndexMerge is a hint to enforce using some indexes at the same time.
HintIndexMerge = "use_index_merge"
// HintTimeRange is a hint to specify the time range for metrics summary tables
HintTimeRange = "time_range"
// HintIgnorePlanCache is a hint to enforce ignoring plan cache
HintIgnorePlanCache = "ignore_plan_cache"
// HintLimitToCop is a hint enforce pushing limit or topn to coprocessor.
HintLimitToCop = "limit_to_cop"
)
const (
// ErrExprInSelect is in select fields for the error of ErrFieldNotInGroupBy
ErrExprInSelect = "SELECT list"
// ErrExprInOrderBy is in order by items for the error of ErrFieldNotInGroupBy
ErrExprInOrderBy = "ORDER BY"
)
func (la *LogicalAggregation) collectGroupByColumns() {
la.groupByCols = la.groupByCols[:0]
for _, item := range la.GroupByItems {
if col, ok := item.(*expression.Column); ok {
la.groupByCols = append(la.groupByCols, col)
}
}
}
// aggOrderByResolver is currently resolving expressions of order by clause
// in aggregate function GROUP_CONCAT.
type aggOrderByResolver struct {
ctx sessionctx.Context
err error
args []ast.ExprNode
exprDepth int // exprDepth is the depth of current expression in expression tree.
}
func (a *aggOrderByResolver) Enter(inNode ast.Node) (ast.Node, bool) {
a.exprDepth++
switch n := inNode.(type) {
case *driver.ParamMarkerExpr:
if a.exprDepth == 1 {
_, isNull, isExpectedType := getUintFromNode(a.ctx, n)
// For constant uint expression in top level, it should be treated as position expression.
if !isNull && isExpectedType {
return expression.ConstructPositionExpr(n), true
}
}
}
return inNode, false
}
func (a *aggOrderByResolver) Leave(inNode ast.Node) (ast.Node, bool) {
switch v := inNode.(type) {
case *ast.PositionExpr:
pos, isNull, err := expression.PosFromPositionExpr(a.ctx, v)
if err != nil {
a.err = err
}
if err != nil || isNull {
return inNode, false
}
if pos < 1 || pos > len(a.args) {
errPos := strconv.Itoa(pos)
if v.P != nil {
errPos = "?"
}
a.err = ErrUnknownColumn.FastGenByArgs(errPos, "order clause")
return inNode, false
}
ret := a.args[pos-1]
return ret, true
}
return inNode, true
}
func (b *PlanBuilder) buildAggregation(ctx context.Context, p LogicalPlan, aggFuncList []*ast.AggregateFuncExpr, gbyItems []expression.Expression) (LogicalPlan, map[int]int, error) {
b.optFlag |= flagBuildKeyInfo
b.optFlag |= flagPushDownAgg
// We may apply aggregation eliminate optimization.
// So we add the flagMaxMinEliminate to try to convert max/min to topn and flagPushDownTopN to handle the newly added topn operator.
b.optFlag |= flagMaxMinEliminate
b.optFlag |= flagPushDownTopN
// when we eliminate the max and min we may add `is not null` filter.
b.optFlag |= flagPredicatePushDown
b.optFlag |= flagEliminateAgg
b.optFlag |= flagEliminateProjection
plan4Agg := LogicalAggregation{AggFuncs: make([]*aggregation.AggFuncDesc, 0, len(aggFuncList))}.Init(b.ctx, b.getSelectOffset())
if hint := b.TableHints(); hint != nil {
plan4Agg.aggHints = hint.aggHints
}
schema4Agg := expression.NewSchema(make([]*expression.Column, 0, len(aggFuncList)+p.Schema().Len())...)
names := make(types.NameSlice, 0, len(aggFuncList)+p.Schema().Len())
// aggIdxMap maps the old index to new index after applying common aggregation functions elimination.
aggIndexMap := make(map[int]int)
allAggsFirstRow := true
for i, aggFunc := range aggFuncList {
newArgList := make([]expression.Expression, 0, len(aggFunc.Args))
for _, arg := range aggFunc.Args {
newArg, np, err := b.rewrite(ctx, arg, p, nil, true)
if err != nil {
return nil, nil, err
}
p = np
newArgList = append(newArgList, newArg)
}
newFunc, err := aggregation.NewAggFuncDesc(b.ctx, aggFunc.F, newArgList, aggFunc.Distinct)
if err != nil {
return nil, nil, err
}
if newFunc.Name != ast.AggFuncFirstRow {
allAggsFirstRow = false
}
if aggFunc.Order != nil {
trueArgs := aggFunc.Args[:len(aggFunc.Args)-1] // the last argument is SEPARATOR, remote it.
resolver := &aggOrderByResolver{
ctx: b.ctx,
args: trueArgs,
}
for _, byItem := range aggFunc.Order.Items {
resolver.exprDepth = 0
resolver.err = nil
retExpr, _ := byItem.Expr.Accept(resolver)
if resolver.err != nil {
return nil, nil, errors.Trace(resolver.err)
}
newByItem, np, err := b.rewrite(ctx, retExpr.(ast.ExprNode), p, nil, true)
if err != nil {
return nil, nil, err
}
p = np
newFunc.OrderByItems = append(newFunc.OrderByItems, &util.ByItems{Expr: newByItem, Desc: byItem.Desc})
}
}
combined := false
for j, oldFunc := range plan4Agg.AggFuncs {
if oldFunc.Equal(b.ctx, newFunc) {
aggIndexMap[i] = j
combined = true
break
}
}
if !combined {
position := len(plan4Agg.AggFuncs)
aggIndexMap[i] = position
plan4Agg.AggFuncs = append(plan4Agg.AggFuncs, newFunc)
schema4Agg.Append(&expression.Column{
UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(),
RetType: newFunc.RetTp,
})
names = append(names, types.EmptyName)
}
}
for i, col := range p.Schema().Columns {
newFunc, err := aggregation.NewAggFuncDesc(b.ctx, ast.AggFuncFirstRow, []expression.Expression{col}, false)
if err != nil {
return nil, nil, err
}
plan4Agg.AggFuncs = append(plan4Agg.AggFuncs, newFunc)
newCol, _ := col.Clone().(*expression.Column)
newCol.RetType = newFunc.RetTp
schema4Agg.Append(newCol)
names = append(names, p.OutputNames()[i])
}
hasGroupBy := len(gbyItems) > 0
for i, aggFunc := range plan4Agg.AggFuncs {
err := aggFunc.UpdateNotNullFlag4RetType(hasGroupBy, allAggsFirstRow)
if err != nil {
return nil, nil, err
}
schema4Agg.Columns[i].RetType = aggFunc.RetTp
}
plan4Agg.names = names
plan4Agg.SetChildren(p)
plan4Agg.GroupByItems = gbyItems
plan4Agg.SetSchema(schema4Agg)
plan4Agg.collectGroupByColumns()
return plan4Agg, aggIndexMap, nil
}
func (b *PlanBuilder) buildResultSetNode(ctx context.Context, node ast.ResultSetNode) (p LogicalPlan, err error) {
switch x := node.(type) {
case *ast.Join:
return b.buildJoin(ctx, x)
case *ast.TableSource:
var isTableName bool
switch v := x.Source.(type) {
case *ast.SelectStmt:
p, err = b.buildSelect(ctx, v)
case *ast.UnionStmt:
p, err = b.buildUnion(ctx, v)
case *ast.TableName:
p, err = b.buildDataSource(ctx, v, &x.AsName)
isTableName = true
default:
err = ErrUnsupportedType.GenWithStackByArgs(v)
}
if err != nil {
return nil, err
}
for _, name := range p.OutputNames() {
if name.Hidden {
continue
}
if x.AsName.L != "" {
name.TblName = x.AsName
}
}
// `TableName` is not a select block, so we do not need to handle it.
if !isTableName && b.ctx.GetSessionVars().PlannerSelectBlockAsName != nil {
b.ctx.GetSessionVars().PlannerSelectBlockAsName[p.SelectBlockOffset()] = ast.HintTable{DBName: p.OutputNames()[0].DBName, TableName: p.OutputNames()[0].TblName}
}
// Duplicate column name in one table is not allowed.
// "select * from (select 1, 1) as a;" is duplicate
dupNames := make(map[string]struct{}, len(p.Schema().Columns))
for _, name := range p.OutputNames() {
colName := name.ColName.O
if _, ok := dupNames[colName]; ok {
return nil, ErrDupFieldName.GenWithStackByArgs(colName)
}
dupNames[colName] = struct{}{}
}
return p, nil
case *ast.SelectStmt:
return b.buildSelect(ctx, x)
case *ast.UnionStmt:
return b.buildUnion(ctx, x)
default:
return nil, ErrUnsupportedType.GenWithStack("Unsupported ast.ResultSetNode(%T) for buildResultSetNode()", x)
}
}
// pushDownConstExpr checks if the condition is from filter condition, if true, push it down to both
// children of join, whatever the join type is; if false, push it down to inner child of outer join,
// and both children of non-outer-join.
func (p *LogicalJoin) pushDownConstExpr(expr expression.Expression, leftCond []expression.Expression,
rightCond []expression.Expression, filterCond bool) ([]expression.Expression, []expression.Expression) {
switch p.JoinType {
case LeftOuterJoin, LeftOuterSemiJoin, AntiLeftOuterSemiJoin:
if filterCond {
leftCond = append(leftCond, expr)
// Append the expr to right join condition instead of `rightCond`, to make it able to be
// pushed down to children of join.
p.RightConditions = append(p.RightConditions, expr)
} else {
rightCond = append(rightCond, expr)
}
case RightOuterJoin:
if filterCond {
rightCond = append(rightCond, expr)
p.LeftConditions = append(p.LeftConditions, expr)
} else {
leftCond = append(leftCond, expr)
}
case SemiJoin, InnerJoin:
leftCond = append(leftCond, expr)
rightCond = append(rightCond, expr)
case AntiSemiJoin:
if filterCond {
leftCond = append(leftCond, expr)
}
rightCond = append(rightCond, expr)
}
return leftCond, rightCond
}
func (p *LogicalJoin) extractOnCondition(conditions []expression.Expression, deriveLeft bool,
deriveRight bool) (eqCond []*expression.ScalarFunction, leftCond []expression.Expression,
rightCond []expression.Expression, otherCond []expression.Expression) {
return p.ExtractOnCondition(conditions, p.children[0].Schema(), p.children[1].Schema(), deriveLeft, deriveRight)
}
// ExtractOnCondition divide conditions in CNF of join node into 4 groups.
// These conditions can be where conditions, join conditions, or collection of both.
// If deriveLeft/deriveRight is set, we would try to derive more conditions for left/right plan.
func (p *LogicalJoin) ExtractOnCondition(
conditions []expression.Expression,
leftSchema *expression.Schema,
rightSchema *expression.Schema,
deriveLeft bool,
deriveRight bool) (eqCond []*expression.ScalarFunction, leftCond []expression.Expression,
rightCond []expression.Expression, otherCond []expression.Expression) {
for _, expr := range conditions {
// For queries like `select a in (select a from s where s.b = t.b) from t`,
// if subquery is empty caused by `s.b = t.b`, the result should always be
// false even if t.a is null or s.a is null. To make this join "empty aware",
// we should differentiate `t.a = s.a` from other column equal conditions, so
// we put it into OtherConditions instead of EqualConditions of join.
if expression.IsEQCondFromIn(expr) {
otherCond = append(otherCond, expr)
continue
}
binop, ok := expr.(*expression.ScalarFunction)
if ok && len(binop.GetArgs()) == 2 {
ctx := binop.GetCtx()
arg0, lOK := binop.GetArgs()[0].(*expression.Column)
arg1, rOK := binop.GetArgs()[1].(*expression.Column)
if lOK && rOK {
leftCol := leftSchema.RetrieveColumn(arg0)
rightCol := rightSchema.RetrieveColumn(arg1)
if leftCol == nil || rightCol == nil {
leftCol = leftSchema.RetrieveColumn(arg1)
rightCol = rightSchema.RetrieveColumn(arg0)
arg0, arg1 = arg1, arg0
}
if leftCol != nil && rightCol != nil {
if deriveLeft {
if isNullRejected(ctx, leftSchema, expr) && !mysql.HasNotNullFlag(leftCol.RetType.Flag) {
notNullExpr := expression.BuildNotNullExpr(ctx, leftCol)
leftCond = append(leftCond, notNullExpr)
}
}
if deriveRight {
if isNullRejected(ctx, rightSchema, expr) && !mysql.HasNotNullFlag(rightCol.RetType.Flag) {
notNullExpr := expression.BuildNotNullExpr(ctx, rightCol)
rightCond = append(rightCond, notNullExpr)
}
}
if binop.FuncName.L == ast.EQ {
cond := expression.NewFunctionInternal(ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), arg0, arg1)
eqCond = append(eqCond, cond.(*expression.ScalarFunction))
continue
}
}
}
}
columns := expression.ExtractColumns(expr)
// `columns` may be empty, if the condition is like `correlated_column op constant`, or `constant`,
// push this kind of constant condition down according to join type.
if len(columns) == 0 {
leftCond, rightCond = p.pushDownConstExpr(expr, leftCond, rightCond, deriveLeft || deriveRight)
continue
}
allFromLeft, allFromRight := true, true
for _, col := range columns {
if !leftSchema.Contains(col) {
allFromLeft = false
}
if !rightSchema.Contains(col) {
allFromRight = false
}
}
if allFromRight {
rightCond = append(rightCond, expr)
} else if allFromLeft {
leftCond = append(leftCond, expr)
} else {
// Relax expr to two supersets: leftRelaxedCond and rightRelaxedCond, the expression now is
// `expr AND leftRelaxedCond AND rightRelaxedCond`. Motivation is to push filters down to
// children as much as possible.
if deriveLeft {
leftRelaxedCond := expression.DeriveRelaxedFiltersFromDNF(expr, leftSchema)
if leftRelaxedCond != nil {
leftCond = append(leftCond, leftRelaxedCond)
}
}
if deriveRight {
rightRelaxedCond := expression.DeriveRelaxedFiltersFromDNF(expr, rightSchema)
if rightRelaxedCond != nil {
rightCond = append(rightCond, rightRelaxedCond)
}
}
otherCond = append(otherCond, expr)
}
}
return
}
// extractTableAlias returns table alias of the LogicalPlan's columns.
// It will return nil when there are multiple table alias, because the alias is only used to check if
// the logicalPlan match some optimizer hints, and hints are not expected to take effect in this case.
func extractTableAlias(p Plan, parentOffset int) *hintTableInfo {
if len(p.OutputNames()) > 0 && p.OutputNames()[0].TblName.L != "" {
firstName := p.OutputNames()[0]
for _, name := range p.OutputNames() {
if name.TblName.L != firstName.TblName.L || name.DBName.L != firstName.DBName.L {
return nil
}
}
blockOffset := p.SelectBlockOffset()
blockAsNames := p.SCtx().GetSessionVars().PlannerSelectBlockAsName
// For sub-queries like `(select * from t) t1`, t1 should belong to its surrounding select block.
if blockOffset != parentOffset && blockAsNames != nil && blockAsNames[blockOffset].TableName.L != "" {
blockOffset = parentOffset
}
return &hintTableInfo{dbName: firstName.DBName, tblName: firstName.TblName, selectOffset: blockOffset}
}
return nil
}
func (p *LogicalJoin) getPreferredBCJLocalIndex() (hasPrefer bool, prefer int) {
if p.hintInfo == nil {
return
}
if p.hintInfo.ifPreferAsLocalInBCJoin(p.children[0], p.blockOffset) {
return true, 0
}
if p.hintInfo.ifPreferAsLocalInBCJoin(p.children[1], p.blockOffset) {
return true, 1
}
return false, 0
}
func (p *LogicalJoin) setPreferredJoinType(hintInfo *tableHintInfo) {
if hintInfo == nil {
return
}
lhsAlias := extractTableAlias(p.children[0], p.blockOffset)
rhsAlias := extractTableAlias(p.children[1], p.blockOffset)
if hintInfo.ifPreferMergeJoin(lhsAlias, rhsAlias) {
p.preferJoinType |= preferMergeJoin
}
if hintInfo.ifPreferBroadcastJoin(lhsAlias, rhsAlias) {
p.preferJoinType |= preferBCJoin
}
if hintInfo.ifPreferHashJoin(lhsAlias, rhsAlias) {
p.preferJoinType |= preferHashJoin
}
if hintInfo.ifPreferINLJ(lhsAlias) {
p.preferJoinType |= preferLeftAsINLJInner
}
if hintInfo.ifPreferINLJ(rhsAlias) {
p.preferJoinType |= preferRightAsINLJInner
}
if hintInfo.ifPreferINLHJ(lhsAlias) {
p.preferJoinType |= preferLeftAsINLHJInner
}
if hintInfo.ifPreferINLHJ(rhsAlias) {
p.preferJoinType |= preferRightAsINLHJInner
}
if hintInfo.ifPreferINLMJ(lhsAlias) {
// TODO: reopen index merge join change to preferLeftAsINLMJInner in future
p.preferJoinType |= preferLeftAsINLJInner
}
if hintInfo.ifPreferINLMJ(rhsAlias) {
// TODO: reopen index merge join change to preferRightAsINLMJInner in future
p.preferJoinType |= preferRightAsINLJInner
}
if containDifferentJoinTypes(p.preferJoinType) {
errMsg := "Join hints are conflict, you can only specify one type of join"
warning := ErrInternal.GenWithStack(errMsg)
p.ctx.GetSessionVars().StmtCtx.AppendWarning(warning)
p.preferJoinType = 0
}
// set hintInfo for further usage if this hint info can be used.
if p.preferJoinType != 0 {
p.hintInfo = hintInfo
}
}
func (ds *DataSource) setPreferredStoreType(hintInfo *tableHintInfo) {
if hintInfo == nil {
return
}
var alias *hintTableInfo
if len(ds.TableAsName.L) != 0 {
alias = &hintTableInfo{dbName: ds.DBName, tblName: *ds.TableAsName, selectOffset: ds.SelectBlockOffset()}
} else {
alias = &hintTableInfo{dbName: ds.DBName, tblName: ds.tableInfo.Name, selectOffset: ds.SelectBlockOffset()}
}
if hintTbl := hintInfo.ifPreferTiKV(alias); hintTbl != nil {
for _, path := range ds.possibleAccessPaths {
if path.StoreType == kv.TiKV {
ds.preferStoreType |= preferTiKV
ds.preferPartitions[preferTiKV] = hintTbl.partitions
break
}
}
if ds.preferStoreType&preferTiKV == 0 {
errMsg := fmt.Sprintf("No available path for table %s.%s with the store type %s of the hint /*+ read_from_storage */, "+
"please check the status of the table replica and variable value of tidb_isolation_read_engines(%v)",
ds.DBName.O, ds.table.Meta().Name.O, kv.TiKV.Name(), ds.ctx.GetSessionVars().GetIsolationReadEngines())
warning := ErrInternal.GenWithStack(errMsg)
ds.ctx.GetSessionVars().StmtCtx.AppendWarning(warning)
}
}
if hintTbl := hintInfo.ifPreferTiFlash(alias); hintTbl != nil {
// 1. `ds.tableInfo.Partition == nil`, which means the hint takes effect in the whole table.
// 2. `ds.preferStoreType != 0`, which means there's a hint hit the both TiKV value and TiFlash value for table.
// If it's satisfied the above two conditions, then we can make sure there are some hints conflicted.
if ds.preferStoreType != 0 && ds.tableInfo.Partition == nil {
errMsg := fmt.Sprintf("Storage hints are conflict, you can only specify one storage type of table %s.%s",
alias.dbName.L, alias.tblName.L)
warning := ErrInternal.GenWithStack(errMsg)
ds.ctx.GetSessionVars().StmtCtx.AppendWarning(warning)
ds.preferStoreType = 0
return
}
for _, path := range ds.possibleAccessPaths {
if path.StoreType == kv.TiFlash {
ds.preferStoreType |= preferTiFlash
ds.preferPartitions[preferTiFlash] = hintTbl.partitions
break
}
}
if ds.preferStoreType&preferTiFlash == 0 {
errMsg := fmt.Sprintf("No available path for table %s.%s with the store type %s of the hint /*+ read_from_storage */, "+
"please check the status of the table replica and variable value of tidb_isolation_read_engines(%v)",
ds.DBName.O, ds.table.Meta().Name.O, kv.TiFlash.Name(), ds.ctx.GetSessionVars().GetIsolationReadEngines())
warning := ErrInternal.GenWithStack(errMsg)
ds.ctx.GetSessionVars().StmtCtx.AppendWarning(warning)
}
}
}
func resetNotNullFlag(schema *expression.Schema, start, end int) {
for i := start; i < end; i++ {
col := *schema.Columns[i]
newFieldType := *col.RetType
newFieldType.Flag &= ^mysql.NotNullFlag
col.RetType = &newFieldType
schema.Columns[i] = &col
}
}
func (b *PlanBuilder) buildJoin(ctx context.Context, joinNode *ast.Join) (LogicalPlan, error) {
// We will construct a "Join" node for some statements like "INSERT",
// "DELETE", "UPDATE", "REPLACE". For this scenario "joinNode.Right" is nil
// and we only build the left "ResultSetNode".
if joinNode.Right == nil {
return b.buildResultSetNode(ctx, joinNode.Left)
}
b.optFlag = b.optFlag | flagPredicatePushDown
leftPlan, err := b.buildResultSetNode(ctx, joinNode.Left)
if err != nil {
return nil, err
}
rightPlan, err := b.buildResultSetNode(ctx, joinNode.Right)
if err != nil {
return nil, err
}
handleMap1 := b.handleHelper.popMap()
handleMap2 := b.handleHelper.popMap()
b.handleHelper.mergeAndPush(handleMap1, handleMap2)
joinPlan := LogicalJoin{StraightJoin: joinNode.StraightJoin || b.inStraightJoin}.Init(b.ctx, b.getSelectOffset())
joinPlan.SetChildren(leftPlan, rightPlan)
joinPlan.SetSchema(expression.MergeSchema(leftPlan.Schema(), rightPlan.Schema()))
joinPlan.names = make([]*types.FieldName, leftPlan.Schema().Len()+rightPlan.Schema().Len())
copy(joinPlan.names, leftPlan.OutputNames())
copy(joinPlan.names[leftPlan.Schema().Len():], rightPlan.OutputNames())
// Set join type.
switch joinNode.Tp {
case ast.LeftJoin:
// left outer join need to be checked elimination
b.optFlag = b.optFlag | flagEliminateOuterJoin
joinPlan.JoinType = LeftOuterJoin
resetNotNullFlag(joinPlan.schema, leftPlan.Schema().Len(), joinPlan.schema.Len())
case ast.RightJoin:
// right outer join need to be checked elimination
b.optFlag = b.optFlag | flagEliminateOuterJoin
joinPlan.JoinType = RightOuterJoin
resetNotNullFlag(joinPlan.schema, 0, leftPlan.Schema().Len())
default:
b.optFlag = b.optFlag | flagJoinReOrder
joinPlan.JoinType = InnerJoin
}
// Merge sub join's redundantSchema into this join plan. When handle query like
// select t2.a from (t1 join t2 using (a)) join t3 using (a);
// we can simply search in the top level join plan to find redundant column.
var (
lRedundantSchema, rRedundantSchema *expression.Schema
lRedundantNames, rRedundantNames types.NameSlice
)
if left, ok := leftPlan.(*LogicalJoin); ok && left.redundantSchema != nil {
lRedundantSchema = left.redundantSchema
lRedundantNames = left.redundantNames
}
if right, ok := rightPlan.(*LogicalJoin); ok && right.redundantSchema != nil {
rRedundantSchema = right.redundantSchema
rRedundantNames = right.redundantNames
}
joinPlan.redundantSchema = expression.MergeSchema(lRedundantSchema, rRedundantSchema)
joinPlan.redundantNames = make([]*types.FieldName, len(lRedundantNames)+len(rRedundantNames))
copy(joinPlan.redundantNames, lRedundantNames)
copy(joinPlan.redundantNames[len(lRedundantNames):], rRedundantNames)
// Set preferred join algorithm if some join hints is specified by user.
joinPlan.setPreferredJoinType(b.TableHints())
// "NATURAL JOIN" doesn't have "ON" or "USING" conditions.
//
// The "NATURAL [LEFT] JOIN" of two tables is defined to be semantically
// equivalent to an "INNER JOIN" or a "LEFT JOIN" with a "USING" clause
// that names all columns that exist in both tables.
//
// See https://dev.mysql.com/doc/refman/5.7/en/join.html for more detail.
if joinNode.NaturalJoin {
err = b.buildNaturalJoin(joinPlan, leftPlan, rightPlan, joinNode)
if err != nil {
return nil, err
}
} else if joinNode.Using != nil {
err = b.buildUsingClause(joinPlan, leftPlan, rightPlan, joinNode)
if err != nil {
return nil, err
}
} else if joinNode.On != nil {
b.curClause = onClause
onExpr, newPlan, err := b.rewrite(ctx, joinNode.On.Expr, joinPlan, nil, false)
if err != nil {
return nil, err
}
if newPlan != joinPlan {
return nil, errors.New("ON condition doesn't support subqueries yet")
}
onCondition := expression.SplitCNFItems(onExpr)
joinPlan.AttachOnConds(onCondition)
} else if joinPlan.JoinType == InnerJoin {
// If a inner join without "ON" or "USING" clause, it's a cartesian
// product over the join tables.
joinPlan.cartesianJoin = true
}
return joinPlan, nil
}
// buildUsingClause eliminate the redundant columns and ordering columns based
// on the "USING" clause.
//
// According to the standard SQL, columns are ordered in the following way:
// 1. coalesced common columns of "leftPlan" and "rightPlan", in the order they
// appears in "leftPlan".
// 2. the rest columns in "leftPlan", in the order they appears in "leftPlan".
// 3. the rest columns in "rightPlan", in the order they appears in "rightPlan".
func (b *PlanBuilder) buildUsingClause(p *LogicalJoin, leftPlan, rightPlan LogicalPlan, join *ast.Join) error {
filter := make(map[string]bool, len(join.Using))
for _, col := range join.Using {
filter[col.Name.L] = true
}
err := b.coalesceCommonColumns(p, leftPlan, rightPlan, join.Tp, filter)
if err != nil {
return err
}
// We do not need to coalesce columns for update and delete.
if b.inUpdateStmt || b.inDeleteStmt {
p.setSchemaAndNames(expression.MergeSchema(p.Children()[0].Schema(), p.Children()[1].Schema()),
append(p.Children()[0].OutputNames(), p.Children()[1].OutputNames()...))
}
return nil
}
// buildNaturalJoin builds natural join output schema. It finds out all the common columns
// then using the same mechanism as buildUsingClause to eliminate redundant columns and build join conditions.
// According to standard SQL, producing this display order:
//
// All the common columns
// Every column in the first (left) table that is not a common column
// Every column in the second (right) table that is not a common column
func (b *PlanBuilder) buildNaturalJoin(p *LogicalJoin, leftPlan, rightPlan LogicalPlan, join *ast.Join) error {
err := b.coalesceCommonColumns(p, leftPlan, rightPlan, join.Tp, nil)
if err != nil {
return err
}
// We do not need to coalesce columns for update and delete.
if b.inUpdateStmt || b.inDeleteStmt {
p.setSchemaAndNames(expression.MergeSchema(p.Children()[0].Schema(), p.Children()[1].Schema()),
append(p.Children()[0].OutputNames(), p.Children()[1].OutputNames()...))
}
return nil
}
// coalesceCommonColumns is used by buildUsingClause and buildNaturalJoin. The filter is used by buildUsingClause.
func (b *PlanBuilder) coalesceCommonColumns(p *LogicalJoin, leftPlan, rightPlan LogicalPlan, joinTp ast.JoinType, filter map[string]bool) error {
lsc := leftPlan.Schema().Clone()
rsc := rightPlan.Schema().Clone()
if joinTp == ast.LeftJoin {
resetNotNullFlag(rsc, 0, rsc.Len())
} else if joinTp == ast.RightJoin {
resetNotNullFlag(lsc, 0, lsc.Len())
}
lColumns, rColumns := lsc.Columns, rsc.Columns
lNames, rNames := leftPlan.OutputNames().Shallow(), rightPlan.OutputNames().Shallow()
if joinTp == ast.RightJoin {
lNames, rNames = rNames, lNames
lColumns, rColumns = rsc.Columns, lsc.Columns
}
// Find out all the common columns and put them ahead.
commonLen := 0
for i, lName := range lNames {
for j := commonLen; j < len(rNames); j++ {
if lName.ColName.L != rNames[j].ColName.L {
continue
}
if len(filter) > 0 {
if !filter[lName.ColName.L] {
break
}
// Mark this column exist.
filter[lName.ColName.L] = false
}
col := lColumns[i]
copy(lColumns[commonLen+1:i+1], lColumns[commonLen:i])
lColumns[commonLen] = col
name := lNames[i]
copy(lNames[commonLen+1:i+1], lNames[commonLen:i])
lNames[commonLen] = name
col = rColumns[j]
copy(rColumns[commonLen+1:j+1], rColumns[commonLen:j])
rColumns[commonLen] = col
name = rNames[j]
copy(rNames[commonLen+1:j+1], rNames[commonLen:j])
rNames[commonLen] = name
commonLen++
break
}
}
if len(filter) > 0 && len(filter) != commonLen {
for col, notExist := range filter {
if notExist {
return ErrUnknownColumn.GenWithStackByArgs(col, "from clause")
}
}
}
schemaCols := make([]*expression.Column, len(lColumns)+len(rColumns)-commonLen)
copy(schemaCols[:len(lColumns)], lColumns)
copy(schemaCols[len(lColumns):], rColumns[commonLen:])
names := make(types.NameSlice, len(schemaCols))
copy(names, lNames)
copy(names[len(lNames):], rNames[commonLen:])
conds := make([]expression.Expression, 0, commonLen)
for i := 0; i < commonLen; i++ {
lc, rc := lsc.Columns[i], rsc.Columns[i]
cond, err := expression.NewFunction(b.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), lc, rc)
if err != nil {
return err
}
conds = append(conds, cond)
}
p.SetSchema(expression.NewSchema(schemaCols...))
p.names = names
if joinTp == ast.RightJoin {
leftPlan, rightPlan = rightPlan, leftPlan
}
// We record the full `rightPlan.Schema` as `redundantSchema` in order to
// record the redundant column in `rightPlan` and the output columns order
// of the `rightPlan`.
// For SQL like `select t1.*, t2.* from t1 left join t2 using(a)`, we can
// retrieve the column order of `t2.*` from the `redundantSchema`.
p.redundantSchema = expression.MergeSchema(p.redundantSchema, expression.NewSchema(rightPlan.Schema().Clone().Columns...))
p.redundantNames = append(p.redundantNames.Shallow(), rightPlan.OutputNames().Shallow()...)
if joinTp == ast.RightJoin || joinTp == ast.LeftJoin {
resetNotNullFlag(p.redundantSchema, 0, p.redundantSchema.Len())
}
p.OtherConditions = append(conds, p.OtherConditions...)
return nil
}
func (b *PlanBuilder) buildSelection(ctx context.Context, p LogicalPlan, where ast.ExprNode, AggMapper map[*ast.AggregateFuncExpr]int) (LogicalPlan, error) {
b.optFlag |= flagPredicatePushDown
if b.curClause != havingClause {
b.curClause = whereClause
}
conditions := splitWhere(where)
expressions := make([]expression.Expression, 0, len(conditions))
selection := LogicalSelection{}.Init(b.ctx, b.getSelectOffset())
for _, cond := range conditions {
expr, np, err := b.rewrite(ctx, cond, p, AggMapper, false)
if err != nil {
return nil, err
}
p = np
if expr == nil {
continue
}
cnfItems := expression.SplitCNFItems(expr)
for _, item := range cnfItems {
if con, ok := item.(*expression.Constant); ok && con.DeferredExpr == nil && con.ParamMarker == nil {
ret, _, err := expression.EvalBool(b.ctx, expression.CNFExprs{con}, chunk.Row{})
if err != nil || ret {
continue
}
// If there is condition which is always false, return dual plan directly.
dual := LogicalTableDual{}.Init(b.ctx, b.getSelectOffset())
dual.names = p.OutputNames()
dual.SetSchema(p.Schema())
return dual, nil
}
expressions = append(expressions, item)
}
}
if len(expressions) == 0 {
return p, nil
}
selection.Conditions = expressions
selection.SetChildren(p)
return selection, nil
}
// buildProjectionFieldNameFromColumns builds the field name, table name and database name when field expression is a column reference.
func (b *PlanBuilder) buildProjectionFieldNameFromColumns(origField *ast.SelectField, colNameField *ast.ColumnNameExpr, name *types.FieldName) (colName, origColName, tblName, origTblName, dbName model.CIStr) {
origTblName, origColName, dbName = name.OrigTblName, name.OrigColName, name.DBName
if origField.AsName.L == "" {
colName = colNameField.Name.Name
} else {
colName = origField.AsName
}
if tblName.L == "" {
tblName = name.TblName
} else {
tblName = colNameField.Name.Table
}
return
}
// buildProjectionFieldNameFromExpressions builds the field name when field expression is a normal expression.
func (b *PlanBuilder) buildProjectionFieldNameFromExpressions(ctx context.Context, field *ast.SelectField) (model.CIStr, error) {
if agg, ok := field.Expr.(*ast.AggregateFuncExpr); ok && agg.F == ast.AggFuncFirstRow {
// When the query is select t.a from t group by a; The Column Name should be a but not t.a;
return agg.Args[0].(*ast.ColumnNameExpr).Name.Name, nil
}
innerExpr := getInnerFromParenthesesAndUnaryPlus(field.Expr)
funcCall, isFuncCall := innerExpr.(*ast.FuncCallExpr)
// When used to produce a result set column, NAME_CONST() causes the column to have the given name.
// See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_name-const for details
if isFuncCall && funcCall.FnName.L == ast.NameConst {
if v, err := evalAstExpr(b.ctx, funcCall.Args[0]); err == nil {
if s, err := v.ToString(); err == nil {
return model.NewCIStr(s), nil
}
}
return model.NewCIStr(""), ErrWrongArguments.GenWithStackByArgs("NAME_CONST")
}
valueExpr, isValueExpr := innerExpr.(*driver.ValueExpr)
// Non-literal: Output as inputed, except that comments need to be removed.
if !isValueExpr {
return model.NewCIStr(parser.SpecFieldPattern.ReplaceAllStringFunc(field.Text(), parser.TrimComment)), nil
}
// Literal: Need special processing
switch valueExpr.Kind() {
case types.KindString:
projName := valueExpr.GetString()
projOffset := valueExpr.GetProjectionOffset()
if projOffset >= 0 {
projName = projName[:projOffset]
}
// See #3686, #3994:
// For string literals, string content is used as column name. Non-graph initial characters are trimmed.
fieldName := strings.TrimLeftFunc(projName, func(r rune) bool {
return !unicode.IsOneOf(mysql.RangeGraph, r)
})
return model.NewCIStr(fieldName), nil
case types.KindNull:
// See #4053, #3685
return model.NewCIStr("NULL"), nil
case types.KindBinaryLiteral:
// Don't rewrite BIT literal or HEX literals
return model.NewCIStr(field.Text()), nil
case types.KindInt64:
// See #9683
// TRUE or FALSE can be a int64
if mysql.HasIsBooleanFlag(valueExpr.Type.Flag) {
if i := valueExpr.GetValue().(int64); i == 0 {
return model.NewCIStr("FALSE"), nil
}
return model.NewCIStr("TRUE"), nil
}
fallthrough
default:
fieldName := field.Text()
fieldName = strings.TrimLeft(fieldName, "\t\n +(")
fieldName = strings.TrimRight(fieldName, "\t\n )")
return model.NewCIStr(fieldName), nil
}
}
// buildProjectionField builds the field object according to SelectField in projection.
func (b *PlanBuilder) buildProjectionField(ctx context.Context, p LogicalPlan, field *ast.SelectField, expr expression.Expression) (*expression.Column, *types.FieldName, error) {
var origTblName, tblName, origColName, colName, dbName model.CIStr
innerNode := getInnerFromParenthesesAndUnaryPlus(field.Expr)
col, isCol := expr.(*expression.Column)
// Correlated column won't affect the final output names. So we can put it in any of the three logic block.
// Don't put it into the first block just for simplifying the codes.
if colNameField, ok := innerNode.(*ast.ColumnNameExpr); ok && isCol {
// Field is a column reference.
idx := p.Schema().ColumnIndex(col)
var name *types.FieldName
// The column maybe the one from join's redundant part.
if idx == -1 {
name = findColFromNaturalUsingJoin(p, col)
} else {
name = p.OutputNames()[idx]
}
colName, origColName, tblName, origTblName, dbName = b.buildProjectionFieldNameFromColumns(field, colNameField, name)
} else if field.AsName.L != "" {
// Field has alias.
colName = field.AsName
} else {
// Other: field is an expression.
var err error
if colName, err = b.buildProjectionFieldNameFromExpressions(ctx, field); err != nil {
return nil, nil, err
}
}
name := &types.FieldName{
TblName: tblName,
OrigTblName: origTblName,
ColName: colName,
OrigColName: origColName,
DBName: dbName,
}
if isCol {
return col, name, nil
}
newCol := &expression.Column{
UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(),
RetType: expr.GetType(),
}
return newCol, name, nil
}
type userVarTypeProcessor struct {
ctx context.Context
plan LogicalPlan
builder *PlanBuilder
mapper map[*ast.AggregateFuncExpr]int
err error
}
func (p *userVarTypeProcessor) Enter(in ast.Node) (ast.Node, bool) {
v, ok := in.(*ast.VariableExpr)
if !ok {
return in, false
}
if v.IsSystem || v.Value == nil {
return in, true
}
_, p.plan, p.err = p.builder.rewrite(p.ctx, v, p.plan, p.mapper, true)
return in, true
}
func (p *userVarTypeProcessor) Leave(in ast.Node) (ast.Node, bool) {
return in, p.err == nil
}
func (b *PlanBuilder) preprocessUserVarTypes(ctx context.Context, p LogicalPlan, fields []*ast.SelectField, mapper map[*ast.AggregateFuncExpr]int) error {
aggMapper := make(map[*ast.AggregateFuncExpr]int)
for agg, i := range mapper {
aggMapper[agg] = i
}
processor := userVarTypeProcessor{
ctx: ctx,
plan: p,
builder: b,
mapper: aggMapper,
}
for _, field := range fields {
field.Expr.Accept(&processor)
if processor.err != nil {
return processor.err
}
}
return nil
}
// findColFromNaturalUsingJoin is used to recursively find the column from the
// underlying natural-using-join.
// e.g. For SQL like `select t2.a from t1 join t2 using(a) where t2.a > 0`, the
// plan will be `join->selection->projection`. The schema of the `selection`
// will be `[t1.a]`, thus we need to recursively retrieve the `t2.a` from the
// underlying join.
func findColFromNaturalUsingJoin(p LogicalPlan, col *expression.Column) (name *types.FieldName) {
switch x := p.(type) {
case *LogicalLimit, *LogicalSelection, *LogicalTopN, *LogicalSort, *LogicalMaxOneRow:
return findColFromNaturalUsingJoin(p.Children()[0], col)
case *LogicalJoin:
if x.redundantSchema != nil {
idx := x.redundantSchema.ColumnIndex(col)
return x.redundantNames[idx]
}
}
return nil
}
// buildProjection returns a Projection plan and non-aux columns length.
func (b *PlanBuilder) buildProjection(ctx context.Context, p LogicalPlan, fields []*ast.SelectField, mapper map[*ast.AggregateFuncExpr]int, windowMapper map[*ast.WindowFuncExpr]int, considerWindow bool, expandGenerateColumn bool) (LogicalPlan, int, error) {
err := b.preprocessUserVarTypes(ctx, p, fields, mapper)
if err != nil {
return nil, 0, err
}
b.optFlag |= flagEliminateProjection
b.curClause = fieldList
proj := LogicalProjection{Exprs: make([]expression.Expression, 0, len(fields))}.Init(b.ctx, b.getSelectOffset())
schema := expression.NewSchema(make([]*expression.Column, 0, len(fields))...)
oldLen := 0
newNames := make([]*types.FieldName, 0, len(fields))
for i, field := range fields {
if !field.Auxiliary {
oldLen++
}
isWindowFuncField := ast.HasWindowFlag(field.Expr)
// Although window functions occurs in the select fields, but it has to be processed after having clause.
// So when we build the projection for select fields, we need to skip the window function.
// When `considerWindow` is false, we will only build fields for non-window functions, so we add fake placeholders.
// for window functions. These fake placeholders will be erased in column pruning.
// When `considerWindow` is true, all the non-window fields have been built, so we just use the schema columns.
if considerWindow && !isWindowFuncField {
col := p.Schema().Columns[i]
proj.Exprs = append(proj.Exprs, col)
schema.Append(col)
newNames = append(newNames, p.OutputNames()[i])
continue
} else if !considerWindow && isWindowFuncField {
expr := expression.NewZero()
proj.Exprs = append(proj.Exprs, expr)
col, name, err := b.buildProjectionField(ctx, p, field, expr)
if err != nil {
return nil, 0, err
}
schema.Append(col)
newNames = append(newNames, name)
continue
}
newExpr, np, err := b.rewriteWithPreprocess(ctx, field.Expr, p, mapper, windowMapper, true, nil)
if err != nil {
return nil, 0, err
}
// For window functions in the order by clause, we will append an field for it.
// We need rewrite the window mapper here so order by clause could find the added field.
if considerWindow && isWindowFuncField && field.Auxiliary {
if windowExpr, ok := field.Expr.(*ast.WindowFuncExpr); ok {
windowMapper[windowExpr] = i
}
}
p = np
proj.Exprs = append(proj.Exprs, newExpr)
col, name, err := b.buildProjectionField(ctx, p, field, newExpr)
if err != nil {
return nil, 0, err
}
schema.Append(col)
newNames = append(newNames, name)
}
proj.SetSchema(schema)
proj.names = newNames
if expandGenerateColumn {
// Sometimes we need to add some fields to the projection so that we can use generate column substitute
// optimization. For example: select a+1 from t order by a+1, with a virtual generate column c as (a+1) and
// an index on c. We need to add c into the projection so that we can replace a+1 with c.
exprToColumn := make(ExprColumnMap)
collectGenerateColumn(p, exprToColumn)
for expr, col := range exprToColumn {
idx := p.Schema().ColumnIndex(col)
if idx == -1 {
continue
}
if proj.schema.Contains(col) {
continue
}
proj.schema.Columns = append(proj.schema.Columns, col)
proj.Exprs = append(proj.Exprs, expr)
proj.names = append(proj.names, p.OutputNames()[idx])
}
}
proj.SetChildren(p)
return proj, oldLen, nil
}
func (b *PlanBuilder) buildDistinct(child LogicalPlan, length int) (*LogicalAggregation, error) {
b.optFlag = b.optFlag | flagBuildKeyInfo
b.optFlag = b.optFlag | flagPushDownAgg
plan4Agg := LogicalAggregation{
AggFuncs: make([]*aggregation.AggFuncDesc, 0, child.Schema().Len()),
GroupByItems: expression.Column2Exprs(child.Schema().Clone().Columns[:length]),
}.Init(b.ctx, child.SelectBlockOffset())
if hint := b.TableHints(); hint != nil {
plan4Agg.aggHints = hint.aggHints
}
plan4Agg.collectGroupByColumns()
for _, col := range child.Schema().Columns {
aggDesc, err := aggregation.NewAggFuncDesc(b.ctx, ast.AggFuncFirstRow, []expression.Expression{col}, false)
if err != nil {
return nil, err
}
plan4Agg.AggFuncs = append(plan4Agg.AggFuncs, aggDesc)
}
plan4Agg.SetChildren(child)
plan4Agg.SetSchema(child.Schema().Clone())
plan4Agg.names = child.OutputNames()
// Distinct will be rewritten as first_row, we reset the type here since the return type
// of first_row is not always the same as the column arg of first_row.
for i, col := range plan4Agg.schema.Columns {
col.RetType = plan4Agg.AggFuncs[i].RetTp
}
return plan4Agg, nil
}
// unionJoinFieldType finds the type which can carry the given types in Union.
func unionJoinFieldType(a, b *types.FieldType) *types.FieldType {
resultTp := types.NewFieldType(types.MergeFieldType(a.Tp, b.Tp))
// This logic will be intelligible when it is associated with the buildProjection4Union logic.
if resultTp.Tp == mysql.TypeNewDecimal {
// The decimal result type will be unsigned only when all the decimals to be united are unsigned.
resultTp.Flag &= b.Flag & mysql.UnsignedFlag
} else {
// Non-decimal results will be unsigned when the first SQL statement result in the union is unsigned.
resultTp.Flag |= a.Flag & mysql.UnsignedFlag
}
resultTp.Decimal = mathutil.Max(a.Decimal, b.Decimal)
// `Flen - Decimal` is the fraction before '.'
resultTp.Flen = mathutil.Max(a.Flen-a.Decimal, b.Flen-b.Decimal) + resultTp.Decimal
if resultTp.EvalType() != types.ETInt && (a.EvalType() == types.ETInt || b.EvalType() == types.ETInt) && resultTp.Flen < mysql.MaxIntWidth {
resultTp.Flen = mysql.MaxIntWidth
}
resultTp.Charset = a.Charset
resultTp.Collate = a.Collate
expression.SetBinFlagOrBinStr(b, resultTp)
return resultTp
}
func (b *PlanBuilder) buildProjection4Union(ctx context.Context, u *LogicalUnionAll) {
unionCols := make([]*expression.Column, 0, u.children[0].Schema().Len())
names := make([]*types.FieldName, 0, u.children[0].Schema().Len())
// Infer union result types by its children's schema.
for i, col := range u.children[0].Schema().Columns {
resultTp := col.RetType
for j := 1; j < len(u.children); j++ {
childTp := u.children[j].Schema().Columns[i].RetType
resultTp = unionJoinFieldType(resultTp, childTp)
}
names = append(names, &types.FieldName{ColName: u.children[0].OutputNames()[i].ColName})
unionCols = append(unionCols, &expression.Column{
RetType: resultTp,
UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(),
})
}
u.schema = expression.NewSchema(unionCols...)
u.names = names
// Process each child and add a projection above original child.
// So the schema of `UnionAll` can be the same with its children's.
for childID, child := range u.children {
exprs := make([]expression.Expression, len(child.Schema().Columns))
for i, srcCol := range child.Schema().Columns {
dstType := unionCols[i].RetType
srcType := srcCol.RetType
if !srcType.Equal(dstType) {
exprs[i] = expression.BuildCastFunction4Union(b.ctx, srcCol, dstType)
} else {
exprs[i] = srcCol
}
}
b.optFlag |= flagEliminateProjection
proj := LogicalProjection{Exprs: exprs, AvoidColumnEvaluator: true}.Init(b.ctx, b.getSelectOffset())
proj.SetSchema(u.schema.Clone())
proj.SetChildren(child)
u.children[childID] = proj
}
}
func (b *PlanBuilder) buildUnion(ctx context.Context, union *ast.UnionStmt) (LogicalPlan, error) {
distinctSelectPlans, allSelectPlans, err := b.divideUnionSelectPlans(ctx, union.SelectList.Selects)
if err != nil {
return nil, err
}
unionDistinctPlan := b.buildUnionAll(ctx, distinctSelectPlans)
if unionDistinctPlan != nil {
unionDistinctPlan, err = b.buildDistinct(unionDistinctPlan, unionDistinctPlan.Schema().Len())
if err != nil {
return nil, err
}
if len(allSelectPlans) > 0 {
// Can't change the statements order in order to get the correct column info.
allSelectPlans = append([]LogicalPlan{unionDistinctPlan}, allSelectPlans...)
}
}
unionAllPlan := b.buildUnionAll(ctx, allSelectPlans)
unionPlan := unionDistinctPlan
if unionAllPlan != nil {
unionPlan = unionAllPlan
}
oldLen := unionPlan.Schema().Len()
for i := 0; i < len(union.SelectList.Selects); i++ {
b.handleHelper.popMap()
}
b.handleHelper.pushMap(nil)
if union.OrderBy != nil {
unionPlan, err = b.buildSort(ctx, unionPlan, union.OrderBy.Items, nil, nil)
if err != nil {
return nil, err
}
}
if union.Limit != nil {
unionPlan, err = b.buildLimit(unionPlan, union.Limit)
if err != nil {
return nil, err
}
}
// Fix issue #8189 (https://github.com/pingcap/tidb/issues/8189).
// If there are extra expressions generated from `ORDER BY` clause, generate a `Projection` to remove them.
if oldLen != unionPlan.Schema().Len() {
proj := LogicalProjection{Exprs: expression.Column2Exprs(unionPlan.Schema().Columns[:oldLen])}.Init(b.ctx, b.getSelectOffset())
proj.SetChildren(unionPlan)
schema := expression.NewSchema(unionPlan.Schema().Clone().Columns[:oldLen]...)
for _, col := range schema.Columns {
col.UniqueID = b.ctx.GetSessionVars().AllocPlanColumnID()
}
proj.names = unionPlan.OutputNames()[:oldLen]
proj.SetSchema(schema)
return proj, nil
}
return unionPlan, nil
}
// divideUnionSelectPlans resolves union's select stmts to logical plans.
// and divide result plans into "union-distinct" and "union-all" parts.
// divide rule ref: https://dev.mysql.com/doc/refman/5.7/en/union.html
// "Mixed UNION types are treated such that a DISTINCT union overrides any ALL union to its left."
func (b *PlanBuilder) divideUnionSelectPlans(ctx context.Context, selects []*ast.SelectStmt) (distinctSelects []LogicalPlan, allSelects []LogicalPlan, err error) {
firstUnionAllIdx, columnNums := 0, -1
// The last slot is reserved for appending distinct union outside this function.
children := make([]LogicalPlan, len(selects), len(selects)+1)
for i := len(selects) - 1; i >= 0; i-- {
stmt := selects[i]
if firstUnionAllIdx == 0 && stmt.IsAfterUnionDistinct {
firstUnionAllIdx = i + 1
}
selectPlan, err := b.buildSelect(ctx, stmt)
if err != nil {
return nil, nil, err
}
if columnNums == -1 {
columnNums = selectPlan.Schema().Len()
}
if selectPlan.Schema().Len() != columnNums {
return nil, nil, ErrWrongNumberOfColumnsInSelect.GenWithStackByArgs()
}
children[i] = selectPlan
}
return children[:firstUnionAllIdx], children[firstUnionAllIdx:], nil
}
func (b *PlanBuilder) buildUnionAll(ctx context.Context, subPlan []LogicalPlan) LogicalPlan {
if len(subPlan) == 0 {
return nil
}
u := LogicalUnionAll{}.Init(b.ctx, b.getSelectOffset())
u.children = subPlan
b.buildProjection4Union(ctx, u)
return u
}
// itemTransformer transforms ParamMarkerExpr to PositionExpr in the context of ByItem
type itemTransformer struct {
}
func (t *itemTransformer) Enter(inNode ast.Node) (ast.Node, bool) {
switch n := inNode.(type) {
case *driver.ParamMarkerExpr:
newNode := expression.ConstructPositionExpr(n)
return newNode, true
}
return inNode, false
}
func (t *itemTransformer) Leave(inNode ast.Node) (ast.Node, bool) {
return inNode, false
}
func (b *PlanBuilder) buildSort(ctx context.Context, p LogicalPlan, byItems []*ast.ByItem, aggMapper map[*ast.AggregateFuncExpr]int, windowMapper map[*ast.WindowFuncExpr]int) (*LogicalSort, error) {
if _, isUnion := p.(*LogicalUnionAll); isUnion {
b.curClause = globalOrderByClause
} else {
b.curClause = orderByClause
}
sort := LogicalSort{}.Init(b.ctx, b.getSelectOffset())
exprs := make([]*util.ByItems, 0, len(byItems))
transformer := &itemTransformer{}
for _, item := range byItems {
newExpr, _ := item.Expr.Accept(transformer)
item.Expr = newExpr.(ast.ExprNode)
it, np, err := b.rewriteWithPreprocess(ctx, item.Expr, p, aggMapper, windowMapper, true, nil)
if err != nil {
return nil, err
}
p = np
exprs = append(exprs, &util.ByItems{Expr: it, Desc: item.Desc})
}
sort.ByItems = exprs
sort.SetChildren(p)
return sort, nil
}
// getUintFromNode gets uint64 value from ast.Node.
// For ordinary statement, node should be uint64 constant value.
// For prepared statement, node is string. We should convert it to uint64.
func getUintFromNode(ctx sessionctx.Context, n ast.Node) (uVal uint64, isNull bool, isExpectedType bool) {
var val interface{}
switch v := n.(type) {
case *driver.ValueExpr:
val = v.GetValue()
case *driver.ParamMarkerExpr:
if !v.InExecute {
return 0, false, true
}
param, err := expression.ParamMarkerExpression(ctx, v)
if err != nil {
return 0, false, false
}
str, isNull, err := expression.GetStringFromConstant(ctx, param)
if err != nil {
return 0, false, false
}
if isNull {
return 0, true, true
}
val = str
default:
return 0, false, false
}
switch v := val.(type) {
case uint64:
return v, false, true
case int64:
if v >= 0 {
return uint64(v), false, true
}
case string:
sc := ctx.GetSessionVars().StmtCtx
uVal, err := types.StrToUint(sc, v, false)
if err != nil {
return 0, false, false
}
return uVal, false, true
}
return 0, false, false
}
func extractLimitCountOffset(ctx sessionctx.Context, limit *ast.Limit) (count uint64,
offset uint64, err error) {
var isExpectedType bool
if limit.Count != nil {
count, _, isExpectedType = getUintFromNode(ctx, limit.Count)
if !isExpectedType {
return 0, 0, ErrWrongArguments.GenWithStackByArgs("LIMIT")
}
}
if limit.Offset != nil {
offset, _, isExpectedType = getUintFromNode(ctx, limit.Offset)
if !isExpectedType {
return 0, 0, ErrWrongArguments.GenWithStackByArgs("LIMIT")
}
}
return count, offset, nil
}
func (b *PlanBuilder) buildLimit(src LogicalPlan, limit *ast.Limit) (LogicalPlan, error) {
b.optFlag = b.optFlag | flagPushDownTopN
var (
offset, count uint64
err error
)
if count, offset, err = extractLimitCountOffset(b.ctx, limit); err != nil {
return nil, err
}
if count > math.MaxUint64-offset {
count = math.MaxUint64 - offset
}
if offset+count == 0 {
tableDual := LogicalTableDual{RowCount: 0}.Init(b.ctx, b.getSelectOffset())
tableDual.schema = src.Schema()
tableDual.names = src.OutputNames()
return tableDual, nil
}
li := LogicalLimit{
Offset: offset,
Count: count,
}.Init(b.ctx, b.getSelectOffset())
if hint := b.TableHints(); hint != nil {
li.limitHints = hint.limitHints
}
li.SetChildren(src)
return li, nil
}
// colMatch means that if a match b, e.g. t.a can match test.t.a but test.t.a can't match t.a.
// Because column a want column from database test exactly.
func colMatch(a *ast.ColumnName, b *ast.ColumnName) bool {
if a.Schema.L == "" || a.Schema.L == b.Schema.L {
if a.Table.L == "" || a.Table.L == b.Table.L {
return a.Name.L == b.Name.L
}
}
return false
}
func matchField(f *ast.SelectField, col *ast.ColumnNameExpr, ignoreAsName bool) bool {
// if col specify a table name, resolve from table source directly.
if col.Name.Table.L == "" {
if f.AsName.L == "" || ignoreAsName {
if curCol, isCol := f.Expr.(*ast.ColumnNameExpr); isCol {
return curCol.Name.Name.L == col.Name.Name.L
} else if _, isFunc := f.Expr.(*ast.FuncCallExpr); isFunc {
// Fix issue 7331
// If there are some function calls in SelectField, we check if
// ColumnNameExpr in GroupByClause matches one of these function calls.
// Example: select concat(k1,k2) from t group by `concat(k1,k2)`,
// `concat(k1,k2)` matches with function call concat(k1, k2).
return strings.ToLower(f.Text()) == col.Name.Name.L
}
// a expression without as name can't be matched.
return false
}
return f.AsName.L == col.Name.Name.L
}
return false
}
func resolveFromSelectFields(v *ast.ColumnNameExpr, fields []*ast.SelectField, ignoreAsName bool) (index int, err error) {
var matchedExpr ast.ExprNode
index = -1
for i, field := range fields {
if field.Auxiliary {
continue
}
if matchField(field, v, ignoreAsName) {
curCol, isCol := field.Expr.(*ast.ColumnNameExpr)
if !isCol {
return i, nil
}
if matchedExpr == nil {
matchedExpr = curCol
index = i
} else if !colMatch(matchedExpr.(*ast.ColumnNameExpr).Name, curCol.Name) &&
!colMatch(curCol.Name, matchedExpr.(*ast.ColumnNameExpr).Name) {
return -1, ErrAmbiguous.GenWithStackByArgs(curCol.Name.Name.L, clauseMsg[fieldList])
}
}
}
return
}
// havingWindowAndOrderbyExprResolver visits Expr tree.
// It converts ColunmNameExpr to AggregateFuncExpr and collects AggregateFuncExpr.
type havingWindowAndOrderbyExprResolver struct {
inAggFunc bool
inWindowFunc bool
inWindowSpec bool
inExpr bool
err error
p LogicalPlan
selectFields []*ast.SelectField
aggMapper map[*ast.AggregateFuncExpr]int
colMapper map[*ast.ColumnNameExpr]int
gbyItems []*ast.ByItem
outerSchemas []*expression.Schema
outerNames [][]*types.FieldName
curClause clauseCode
}
// Enter implements Visitor interface.
func (a *havingWindowAndOrderbyExprResolver) Enter(n ast.Node) (node ast.Node, skipChildren bool) {
switch n.(type) {
case *ast.AggregateFuncExpr:
a.inAggFunc = true
case *ast.WindowFuncExpr:
a.inWindowFunc = true
case *ast.WindowSpec:
a.inWindowSpec = true
case *driver.ParamMarkerExpr, *ast.ColumnNameExpr, *ast.ColumnName:
case *ast.SubqueryExpr, *ast.ExistsSubqueryExpr:
// Enter a new context, skip it.
// For example: select sum(c) + c + exists(select c from t) from t;
return n, true
default:
a.inExpr = true
}
return n, false
}
func (a *havingWindowAndOrderbyExprResolver) resolveFromPlan(v *ast.ColumnNameExpr, p LogicalPlan) (int, error) {
idx, err := expression.FindFieldName(p.OutputNames(), v.Name)
if err != nil {
return -1, err
}
schemaCols, outputNames := p.Schema().Columns, p.OutputNames()
if idx < 0 {
// For SQL like `select t2.a from t1 join t2 using(a) where t2.a > 0
// order by t2.a`, the query plan will be `join->selection->sort`. The
// schema of selection will be `[t1.a]`, thus we need to recursively
// retrieve the `t2.a` from the underlying join.
switch x := p.(type) {
case *LogicalLimit, *LogicalSelection, *LogicalTopN, *LogicalSort, *LogicalMaxOneRow:
return a.resolveFromPlan(v, p.Children()[0])
case *LogicalJoin:
if len(x.redundantNames) != 0 {
idx, err = expression.FindFieldName(x.redundantNames, v.Name)
schemaCols, outputNames = x.redundantSchema.Columns, x.redundantNames
}
}
if err != nil || idx < 0 {
return -1, err
}
}
col := schemaCols[idx]
if col.IsHidden {
return -1, ErrUnknownColumn.GenWithStackByArgs(v.Name, clauseMsg[a.curClause])
}
name := outputNames[idx]
newColName := &ast.ColumnName{
Schema: name.DBName,
Table: name.TblName,
Name: name.ColName,
}
for i, field := range a.selectFields {
if c, ok := field.Expr.(*ast.ColumnNameExpr); ok && colMatch(c.Name, newColName) {
return i, nil
}
}
sf := &ast.SelectField{
Expr: &ast.ColumnNameExpr{Name: newColName},
Auxiliary: true,
}
sf.Expr.SetType(col.GetType())
a.selectFields = append(a.selectFields, sf)
return len(a.selectFields) - 1, nil
}
// Leave implements Visitor interface.
func (a *havingWindowAndOrderbyExprResolver) Leave(n ast.Node) (node ast.Node, ok bool) {
switch v := n.(type) {
case *ast.AggregateFuncExpr:
a.inAggFunc = false
a.aggMapper[v] = len(a.selectFields)
a.selectFields = append(a.selectFields, &ast.SelectField{
Auxiliary: true,
Expr: v,
AsName: model.NewCIStr(fmt.Sprintf("sel_agg_%d", len(a.selectFields))),
})
case *ast.WindowFuncExpr:
a.inWindowFunc = false
if a.curClause == havingClause {
a.err = ErrWindowInvalidWindowFuncUse.GenWithStackByArgs(strings.ToLower(v.F))
return node, false
}
if a.curClause == orderByClause {
a.selectFields = append(a.selectFields, &ast.SelectField{
Auxiliary: true,
Expr: v,
AsName: model.NewCIStr(fmt.Sprintf("sel_window_%d", len(a.selectFields))),
})
}
case *ast.WindowSpec:
a.inWindowSpec = false
case *ast.ColumnNameExpr:
resolveFieldsFirst := true
if a.inAggFunc || a.inWindowFunc || a.inWindowSpec || (a.curClause == orderByClause && a.inExpr) || a.curClause == fieldList {
resolveFieldsFirst = false
}
if !a.inAggFunc && a.curClause != orderByClause {
for _, item := range a.gbyItems {
if col, ok := item.Expr.(*ast.ColumnNameExpr); ok &&
(colMatch(v.Name, col.Name) || colMatch(col.Name, v.Name)) {
resolveFieldsFirst = false
break
}
}
}
var index int
if resolveFieldsFirst {
index, a.err = resolveFromSelectFields(v, a.selectFields, false)
if a.err != nil {
return node, false
}
if index != -1 && a.curClause == havingClause && ast.HasWindowFlag(a.selectFields[index].Expr) {
a.err = ErrWindowInvalidWindowFuncAliasUse.GenWithStackByArgs(v.Name.Name.O)
return node, false
}
if index == -1 {
if a.curClause == orderByClause {
index, a.err = a.resolveFromPlan(v, a.p)
} else if a.curClause == havingClause && v.Name.Table.L != "" {
// For SQLs like:
// select a from t b having b.a;
index, a.err = a.resolveFromPlan(v, a.p)
if a.err != nil {
return node, false
}
if index != -1 {
// For SQLs like:
// select a+1 from t having t.a;
newV := v
newV.Name = &ast.ColumnName{Name: v.Name.Name}
index, a.err = resolveFromSelectFields(newV, a.selectFields, true)
}
} else {
index, a.err = resolveFromSelectFields(v, a.selectFields, true)
}
}
} else {
// We should ignore the err when resolving from schema. Because we could resolve successfully
// when considering select fields.
var err error
index, err = a.resolveFromPlan(v, a.p)
_ = err
if index == -1 && a.curClause != fieldList {
index, a.err = resolveFromSelectFields(v, a.selectFields, false)
if index != -1 && a.curClause == havingClause && ast.HasWindowFlag(a.selectFields[index].Expr) {
a.err = ErrWindowInvalidWindowFuncAliasUse.GenWithStackByArgs(v.Name.Name.O)
return node, false
}
}
}
if a.err != nil {
return node, false
}
if index == -1 {
// If we can't find it any where, it may be a correlated columns.
for _, names := range a.outerNames {
idx, err1 := expression.FindFieldName(names, v.Name)
if err1 != nil {
a.err = err1
return node, false
}
if idx >= 0 {
return n, true
}
}
a.err = ErrUnknownColumn.GenWithStackByArgs(v.Name.OrigColName(), clauseMsg[a.curClause])
return node, false
}
if a.inAggFunc {
return a.selectFields[index].Expr, true
}
a.colMapper[v] = index
}
return n, true
}
// resolveHavingAndOrderBy will process aggregate functions and resolve the columns that don't exist in select fields.
// If we found some columns that are not in select fields, we will append it to select fields and update the colMapper.
// When we rewrite the order by / having expression, we will find column in map at first.
func (b *PlanBuilder) resolveHavingAndOrderBy(sel *ast.SelectStmt, p LogicalPlan) (
map[*ast.AggregateFuncExpr]int, map[*ast.AggregateFuncExpr]int, error) {
extractor := &havingWindowAndOrderbyExprResolver{
p: p,
selectFields: sel.Fields.Fields,
aggMapper: make(map[*ast.AggregateFuncExpr]int),
colMapper: b.colMapper,
outerSchemas: b.outerSchemas,
outerNames: b.outerNames,
}
if sel.GroupBy != nil {
extractor.gbyItems = sel.GroupBy.Items
}
// Extract agg funcs from having clause.
if sel.Having != nil {
extractor.curClause = havingClause
n, ok := sel.Having.Expr.Accept(extractor)
if !ok {
return nil, nil, errors.Trace(extractor.err)
}
sel.Having.Expr = n.(ast.ExprNode)
}
havingAggMapper := extractor.aggMapper
extractor.aggMapper = make(map[*ast.AggregateFuncExpr]int)
extractor.inExpr = false
// Extract agg funcs from order by clause.
if sel.OrderBy != nil {
extractor.curClause = orderByClause
for _, item := range sel.OrderBy.Items {
if ast.HasWindowFlag(item.Expr) {
continue
}
n, ok := item.Expr.Accept(extractor)
if !ok {
return nil, nil, errors.Trace(extractor.err)
}
item.Expr = n.(ast.ExprNode)
}
}
sel.Fields.Fields = extractor.selectFields
return havingAggMapper, extractor.aggMapper, nil
}
func (b *PlanBuilder) extractAggFuncs(fields []*ast.SelectField) ([]*ast.AggregateFuncExpr, map[*ast.AggregateFuncExpr]int) {
extractor := &AggregateFuncExtractor{}
for _, f := range fields {
n, _ := f.Expr.Accept(extractor)
f.Expr = n.(ast.ExprNode)
}
aggList := extractor.AggFuncs
totalAggMapper := make(map[*ast.AggregateFuncExpr]int, len(aggList))
for i, agg := range aggList {
totalAggMapper[agg] = i
}
return aggList, totalAggMapper
}
// resolveWindowFunction will process window functions and resolve the columns that don't exist in select fields.
func (b *PlanBuilder) resolveWindowFunction(sel *ast.SelectStmt, p LogicalPlan) (
map[*ast.AggregateFuncExpr]int, error) {
extractor := &havingWindowAndOrderbyExprResolver{
p: p,
selectFields: sel.Fields.Fields,
aggMapper: make(map[*ast.AggregateFuncExpr]int),
colMapper: b.colMapper,
outerSchemas: b.outerSchemas,
outerNames: b.outerNames,
}
extractor.curClause = fieldList
for _, field := range sel.Fields.Fields {
if !ast.HasWindowFlag(field.Expr) {
continue
}
n, ok := field.Expr.Accept(extractor)
if !ok {
return nil, extractor.err
}
field.Expr = n.(ast.ExprNode)
}
for _, spec := range sel.WindowSpecs {
_, ok := spec.Accept(extractor)
if !ok {
return nil, extractor.err
}
}
if sel.OrderBy != nil {
extractor.curClause = orderByClause
for _, item := range sel.OrderBy.Items {
if !ast.HasWindowFlag(item.Expr) {
continue
}
n, ok := item.Expr.Accept(extractor)
if !ok {
return nil, extractor.err
}
item.Expr = n.(ast.ExprNode)
}
}
sel.Fields.Fields = extractor.selectFields
return extractor.aggMapper, nil
}
// gbyResolver resolves group by items from select fields.
type gbyResolver struct {
ctx sessionctx.Context
fields []*ast.SelectField
schema *expression.Schema
names []*types.FieldName
err error
inExpr bool
isParam bool
exprDepth int // exprDepth is the depth of current expression in expression tree.
}
func (g *gbyResolver) Enter(inNode ast.Node) (ast.Node, bool) {
g.exprDepth++
switch n := inNode.(type) {
case *ast.SubqueryExpr, *ast.CompareSubqueryExpr, *ast.ExistsSubqueryExpr:
return inNode, true
case *driver.ParamMarkerExpr:
g.isParam = true
if g.exprDepth == 1 {
_, isNull, isExpectedType := getUintFromNode(g.ctx, n)
// For constant uint expression in top level, it should be treated as position expression.
if !isNull && isExpectedType {
return expression.ConstructPositionExpr(n), true
}
}
return n, true
case *driver.ValueExpr, *ast.ColumnNameExpr, *ast.ParenthesesExpr, *ast.ColumnName:
default:
g.inExpr = true
}
return inNode, false
}
func (g *gbyResolver) Leave(inNode ast.Node) (ast.Node, bool) {
extractor := &AggregateFuncExtractor{}
switch v := inNode.(type) {
case *ast.ColumnNameExpr:
idx, err := expression.FindFieldName(g.names, v.Name)
if idx < 0 || !g.inExpr {
var index int
index, g.err = resolveFromSelectFields(v, g.fields, false)
if g.err != nil {
return inNode, false
}
if idx >= 0 {
return inNode, true
}
if index != -1 {
ret := g.fields[index].Expr
ret.Accept(extractor)
if len(extractor.AggFuncs) != 0 {
err = ErrIllegalReference.GenWithStackByArgs(v.Name.OrigColName(), "reference to group function")
} else if ast.HasWindowFlag(ret) {
err = ErrIllegalReference.GenWithStackByArgs(v.Name.OrigColName(), "reference to window function")
} else {
return ret, true
}
}
g.err = err
return inNode, false
}
case *ast.PositionExpr:
pos, isNull, err := expression.PosFromPositionExpr(g.ctx, v)
if err != nil {
g.err = ErrUnknown.GenWithStackByArgs()
}
if err != nil || isNull {
return inNode, false
}
if pos < 1 || pos > len(g.fields) {
g.err = errors.Errorf("Unknown column '%d' in 'group statement'", pos)
return inNode, false
}
ret := g.fields[pos-1].Expr
ret.Accept(extractor)
if len(extractor.AggFuncs) != 0 || ast.HasWindowFlag(ret) {
fieldName := g.fields[pos-1].AsName.String()
if fieldName == "" {
fieldName = g.fields[pos-1].Text()
}
g.err = ErrWrongGroupField.GenWithStackByArgs(fieldName)
return inNode, false
}
return ret, true
case *ast.ValuesExpr:
if v.Column == nil {
g.err = ErrUnknownColumn.GenWithStackByArgs("", "VALUES() function")
}
}
return inNode, true
}
func tblInfoFromCol(from ast.ResultSetNode, name *types.FieldName) *model.TableInfo {
var tableList []*ast.TableName
tableList = extractTableList(from, tableList, true)
for _, field := range tableList {
if field.Name.L == name.TblName.L {
return field.TableInfo
}
if field.Name.L != name.TblName.L {
continue
}
if field.Schema.L == name.DBName.L {
return field.TableInfo
}
}
return nil
}
func buildFuncDependCol(p LogicalPlan, cond ast.ExprNode) (*types.FieldName, *types.FieldName) {
binOpExpr, ok := cond.(*ast.BinaryOperationExpr)
if !ok {
return nil, nil
}
if binOpExpr.Op != opcode.EQ {
return nil, nil
}
lColExpr, ok := binOpExpr.L.(*ast.ColumnNameExpr)
if !ok {
return nil, nil
}
rColExpr, ok := binOpExpr.R.(*ast.ColumnNameExpr)
if !ok {
return nil, nil
}
lIdx, err := expression.FindFieldName(p.OutputNames(), lColExpr.Name)
if err != nil {
return nil, nil
}
rIdx, err := expression.FindFieldName(p.OutputNames(), rColExpr.Name)
if err != nil {
return nil, nil
}
return p.OutputNames()[lIdx], p.OutputNames()[rIdx]
}
func buildWhereFuncDepend(p LogicalPlan, where ast.ExprNode) map[*types.FieldName]*types.FieldName {
whereConditions := splitWhere(where)
colDependMap := make(map[*types.FieldName]*types.FieldName, 2*len(whereConditions))
for _, cond := range whereConditions {
lCol, rCol := buildFuncDependCol(p, cond)
if lCol == nil || rCol == nil {
continue
}
colDependMap[lCol] = rCol
colDependMap[rCol] = lCol
}
return colDependMap
}
func buildJoinFuncDepend(p LogicalPlan, from ast.ResultSetNode) map[*types.FieldName]*types.FieldName {
switch x := from.(type) {
case *ast.Join:
if x.On == nil {
return nil
}
onConditions := splitWhere(x.On.Expr)
colDependMap := make(map[*types.FieldName]*types.FieldName, len(onConditions))
for _, cond := range onConditions {
lCol, rCol := buildFuncDependCol(p, cond)
if lCol == nil || rCol == nil {
continue
}
lTbl := tblInfoFromCol(x.Left, lCol)
if lTbl == nil {
lCol, rCol = rCol, lCol
}
switch x.Tp {
case ast.CrossJoin:
colDependMap[lCol] = rCol
colDependMap[rCol] = lCol
case ast.LeftJoin:
colDependMap[rCol] = lCol
case ast.RightJoin:
colDependMap[lCol] = rCol
}
}
return colDependMap
default:
return nil
}
}
func checkColFuncDepend(
p LogicalPlan,
name *types.FieldName,
tblInfo *model.TableInfo,
gbyColNames map[*types.FieldName]struct{},
whereDependNames, joinDependNames map[*types.FieldName]*types.FieldName,
) bool {
for _, index := range tblInfo.Indices {
if !index.Unique {
continue
}
funcDepend := true
for _, indexCol := range index.Columns {
iColInfo := tblInfo.Columns[indexCol.Offset]
if !mysql.HasNotNullFlag(iColInfo.Flag) {
funcDepend = false
break
}
cn := &ast.ColumnName{
Schema: name.DBName,
Table: name.TblName,
Name: iColInfo.Name,
}
iIdx, err := expression.FindFieldName(p.OutputNames(), cn)
if err != nil || iIdx < 0 {
funcDepend = false
break
}
iName := p.OutputNames()[iIdx]
if _, ok := gbyColNames[iName]; ok {
continue
}
if wCol, ok := whereDependNames[iName]; ok {
if _, ok = gbyColNames[wCol]; ok {
continue
}
}
if jCol, ok := joinDependNames[iName]; ok {
if _, ok = gbyColNames[jCol]; ok {
continue
}
}
funcDepend = false
break
}
if funcDepend {
return true
}
}
primaryFuncDepend := true
hasPrimaryField := false
for _, colInfo := range tblInfo.Columns {
if !mysql.HasPriKeyFlag(colInfo.Flag) {
continue
}
hasPrimaryField = true
pkName := &ast.ColumnName{
Schema: name.DBName,
Table: name.TblName,
Name: colInfo.Name,
}
pIdx, err := expression.FindFieldName(p.OutputNames(), pkName)
if err != nil {
primaryFuncDepend = false
break
}
pCol := p.OutputNames()[pIdx]
if _, ok := gbyColNames[pCol]; ok {
continue
}
if wCol, ok := whereDependNames[pCol]; ok {
if _, ok = gbyColNames[wCol]; ok {
continue
}
}
if jCol, ok := joinDependNames[pCol]; ok {
if _, ok = gbyColNames[jCol]; ok {
continue
}
}
primaryFuncDepend = false
break
}
return primaryFuncDepend && hasPrimaryField
}
// ErrExprLoc is for generate the ErrFieldNotInGroupBy error info
type ErrExprLoc struct {
Offset int
Loc string
}
func checkExprInGroupBy(
p LogicalPlan,
expr ast.ExprNode,
offset int,
loc string,
gbyColNames map[*types.FieldName]struct{},
gbyExprs []ast.ExprNode,
notInGbyColNames map[*types.FieldName]ErrExprLoc,
) {
if _, ok := expr.(*ast.AggregateFuncExpr); ok {
return
}
if _, ok := expr.(*ast.ColumnNameExpr); !ok {
for _, gbyExpr := range gbyExprs {
if reflect.DeepEqual(gbyExpr, expr) {
return
}
}
}
// Function `any_value` can be used in aggregation, even `ONLY_FULL_GROUP_BY` is set.
// See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_any-value for details
if f, ok := expr.(*ast.FuncCallExpr); ok {
if f.FnName.L == ast.AnyValue {
return
}
}
colMap := make(map[*types.FieldName]struct{}, len(p.Schema().Columns))
allColFromExprNode(p, expr, colMap)
for col := range colMap {
if _, ok := gbyColNames[col]; !ok {
notInGbyColNames[col] = ErrExprLoc{Offset: offset, Loc: loc}
}
}
}
func (b *PlanBuilder) checkOnlyFullGroupBy(p LogicalPlan, sel *ast.SelectStmt) (err error) {
if sel.GroupBy != nil {
err = b.checkOnlyFullGroupByWithGroupClause(p, sel)
} else {
err = b.checkOnlyFullGroupByWithOutGroupClause(p, sel.Fields.Fields)
}
return err
}
func (b *PlanBuilder) checkOnlyFullGroupByWithGroupClause(p LogicalPlan, sel *ast.SelectStmt) error {
gbyColNames := make(map[*types.FieldName]struct{}, len(sel.Fields.Fields))
gbyExprs := make([]ast.ExprNode, 0, len(sel.Fields.Fields))
for _, byItem := range sel.GroupBy.Items {
expr := getInnerFromParenthesesAndUnaryPlus(byItem.Expr)
if colExpr, ok := expr.(*ast.ColumnNameExpr); ok {
idx, err := expression.FindFieldName(p.OutputNames(), colExpr.Name)
if err != nil || idx < 0 {
continue
}
gbyColNames[p.OutputNames()[idx]] = struct{}{}
} else {
gbyExprs = append(gbyExprs, expr)
}
}
notInGbyColNames := make(map[*types.FieldName]ErrExprLoc, len(sel.Fields.Fields))
for offset, field := range sel.Fields.Fields {
if field.Auxiliary {
continue
}
checkExprInGroupBy(p, getInnerFromParenthesesAndUnaryPlus(field.Expr), offset, ErrExprInSelect, gbyColNames, gbyExprs, notInGbyColNames)
}
if sel.OrderBy != nil {
for offset, item := range sel.OrderBy.Items {
if colName, ok := item.Expr.(*ast.ColumnNameExpr); ok {
index, err := resolveFromSelectFields(colName, sel.Fields.Fields, false)
if err != nil {
return err
}
// If the ByItem is in fields list, it has been checked already in above.
if index >= 0 {
continue
}
}
checkExprInGroupBy(p, item.Expr, offset, ErrExprInOrderBy, gbyColNames, gbyExprs, notInGbyColNames)
}
}
if len(notInGbyColNames) == 0 {
return nil
}
whereDepends := buildWhereFuncDepend(p, sel.Where)
joinDepends := buildJoinFuncDepend(p, sel.From.TableRefs)
tblMap := make(map[*model.TableInfo]struct{}, len(notInGbyColNames))
for name, errExprLoc := range notInGbyColNames {
tblInfo := tblInfoFromCol(sel.From.TableRefs, name)
if tblInfo == nil {
continue
}
if _, ok := tblMap[tblInfo]; ok {
continue
}
if checkColFuncDepend(p, name, tblInfo, gbyColNames, whereDepends, joinDepends) {
tblMap[tblInfo] = struct{}{}
continue
}
switch errExprLoc.Loc {
case ErrExprInSelect:
return ErrFieldNotInGroupBy.GenWithStackByArgs(errExprLoc.Offset+1, errExprLoc.Loc, name.DBName.O+"."+name.TblName.O+"."+name.OrigColName.O)
case ErrExprInOrderBy:
return ErrFieldNotInGroupBy.GenWithStackByArgs(errExprLoc.Offset+1, errExprLoc.Loc, sel.OrderBy.Items[errExprLoc.Offset].Expr.Text())
}
return nil
}
return nil
}
func (b *PlanBuilder) checkOnlyFullGroupByWithOutGroupClause(p LogicalPlan, fields []*ast.SelectField) error {
resolver := colResolverForOnlyFullGroupBy{}
for idx, field := range fields {
resolver.exprIdx = idx
field.Accept(&resolver)
err := resolver.Check()
if err != nil {
return err
}
}
return nil
}
// colResolverForOnlyFullGroupBy visits Expr tree to find out if an Expr tree is an aggregation function.
// If so, find out the first column name that not in an aggregation function.
type colResolverForOnlyFullGroupBy struct {
firstNonAggCol *ast.ColumnName
exprIdx int
firstNonAggColIdx int
hasAggFuncOrAnyValue bool
}
func (c *colResolverForOnlyFullGroupBy) Enter(node ast.Node) (ast.Node, bool) {
switch t := node.(type) {
case *ast.AggregateFuncExpr:
c.hasAggFuncOrAnyValue = true
return node, true
case *ast.FuncCallExpr:
// enable function `any_value` in aggregation even `ONLY_FULL_GROUP_BY` is set
if t.FnName.L == ast.AnyValue {
c.hasAggFuncOrAnyValue = true
return node, true
}
case *ast.ColumnNameExpr:
if c.firstNonAggCol == nil {
c.firstNonAggCol, c.firstNonAggColIdx = t.Name, c.exprIdx
}
return node, true
case *ast.SubqueryExpr:
return node, true
}
return node, false
}
func (c *colResolverForOnlyFullGroupBy) Leave(node ast.Node) (ast.Node, bool) {
return node, true
}
func (c *colResolverForOnlyFullGroupBy) Check() error {
if c.hasAggFuncOrAnyValue && c.firstNonAggCol != nil {
return ErrMixOfGroupFuncAndFields.GenWithStackByArgs(c.firstNonAggColIdx+1, c.firstNonAggCol.Name.O)
}
return nil
}
type colNameResolver struct {
p LogicalPlan
names map[*types.FieldName]struct{}
}
func (c *colNameResolver) Enter(inNode ast.Node) (ast.Node, bool) {
switch inNode.(type) {
case *ast.ColumnNameExpr, *ast.SubqueryExpr, *ast.AggregateFuncExpr:
return inNode, true
}
return inNode, false
}
func (c *colNameResolver) Leave(inNode ast.Node) (ast.Node, bool) {
switch v := inNode.(type) {
case *ast.ColumnNameExpr:
idx, err := expression.FindFieldName(c.p.OutputNames(), v.Name)
if err == nil && idx >= 0 {
c.names[c.p.OutputNames()[idx]] = struct{}{}
}
}
return inNode, true
}
func allColFromExprNode(p LogicalPlan, n ast.Node, names map[*types.FieldName]struct{}) {
extractor := &colNameResolver{
p: p,
names: names,
}
n.Accept(extractor)
}
func (b *PlanBuilder) resolveGbyExprs(ctx context.Context, p LogicalPlan, gby *ast.GroupByClause, fields []*ast.SelectField) (LogicalPlan, []expression.Expression, error) {
b.curClause = groupByClause
exprs := make([]expression.Expression, 0, len(gby.Items))
resolver := &gbyResolver{
ctx: b.ctx,
fields: fields,
schema: p.Schema(),
names: p.OutputNames(),
}
for _, item := range gby.Items {
resolver.inExpr = false
resolver.exprDepth = 0
resolver.isParam = false
retExpr, _ := item.Expr.Accept(resolver)
if resolver.err != nil {
return nil, nil, errors.Trace(resolver.err)
}
if !resolver.isParam {
item.Expr = retExpr.(ast.ExprNode)
}
itemExpr := retExpr.(ast.ExprNode)
expr, np, err := b.rewrite(ctx, itemExpr, p, nil, true)
if err != nil {
return nil, nil, err
}
exprs = append(exprs, expr)
p = np
}
return p, exprs, nil
}
func (b *PlanBuilder) unfoldWildStar(p LogicalPlan, selectFields []*ast.SelectField) (resultList []*ast.SelectField, err error) {
join, isJoin := p.(*LogicalJoin)
for i, field := range selectFields {
if field.WildCard == nil {
resultList = append(resultList, field)
continue
}
if field.WildCard.Table.L == "" && i > 0 {
return nil, ErrInvalidWildCard
}
list := unfoldWildStar(field, p.OutputNames(), p.Schema().Columns)
// For sql like `select t1.*, t2.* from t1 join t2 using(a)`, we should
// not coalesce the `t2.a` in the output result. Thus we need to unfold
// the wildstar from the underlying join.redundantSchema.
if isJoin && join.redundantSchema != nil && field.WildCard.Table.L != "" {
redundantList := unfoldWildStar(field, join.redundantNames, join.redundantSchema.Columns)
if len(redundantList) > len(list) {
list = redundantList
}
}
if len(list) == 0 {
return nil, ErrBadTable.GenWithStackByArgs(field.WildCard.Table)
}
resultList = append(resultList, list...)
}
return resultList, nil
}
func unfoldWildStar(field *ast.SelectField, outputName types.NameSlice, column []*expression.Column) (resultList []*ast.SelectField) {
dbName := field.WildCard.Schema
tblName := field.WildCard.Table
for i, name := range outputName {
col := column[i]
if col.IsHidden {
continue
}
if (dbName.L == "" || dbName.L == name.DBName.L) &&
(tblName.L == "" || tblName.L == name.TblName.L) &&
col.ID != model.ExtraHandleID {
colName := &ast.ColumnNameExpr{
Name: &ast.ColumnName{
Schema: name.DBName,
Table: name.TblName,
Name: name.ColName,
}}
colName.SetType(col.GetType())
field := &ast.SelectField{Expr: colName}
field.SetText(name.ColName.O)
resultList = append(resultList, field)
}
}
return resultList
}
func (b *PlanBuilder) pushHintWithoutTableWarning(hint *ast.TableOptimizerHint) {
var sb strings.Builder
ctx := format.NewRestoreCtx(0, &sb)
if err := hint.Restore(ctx); err != nil {
return
}
errMsg := fmt.Sprintf("Hint %s is inapplicable. Please specify the table names in the arguments.", sb.String())
b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack(errMsg))
}
func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, nodeType utilhint.NodeType, currentLevel int) {
hints = b.hintProcessor.GetCurrentStmtHints(hints, nodeType, currentLevel)
var (
sortMergeTables, INLJTables, INLHJTables, INLMJTables, hashJoinTables, BCTables, BCJPreferLocalTables []hintTableInfo
indexHintList, indexMergeHintList []indexHintInfo
tiflashTables, tikvTables []hintTableInfo
aggHints aggHintInfo
timeRangeHint ast.HintTimeRange
limitHints limitHintInfo
)
for _, hint := range hints {
// Set warning for the hint that requires the table name.
switch hint.HintName.L {
case TiDBMergeJoin, HintSMJ, TiDBIndexNestedLoopJoin, HintINLJ, HintINLHJ, HintINLMJ,
TiDBHashJoin, HintHJ, HintUseIndex, HintIgnoreIndex, HintIndexMerge:
if len(hint.Tables) == 0 {
b.pushHintWithoutTableWarning(hint)
continue
}
}
switch hint.HintName.L {
case TiDBMergeJoin, HintSMJ:
sortMergeTables = append(sortMergeTables, tableNames2HintTableInfo(b.ctx, hint.HintName.L, hint.Tables, b.hintProcessor, nodeType, currentLevel)...)
case TiDBBroadCastJoin, HintBCJ:
BCTables = append(BCTables, tableNames2HintTableInfo(b.ctx, hint.HintName.L, hint.Tables, b.hintProcessor, nodeType, currentLevel)...)
case HintBCJPreferLocal:
BCJPreferLocalTables = append(BCJPreferLocalTables, tableNames2HintTableInfo(b.ctx, hint.HintName.L, hint.Tables, b.hintProcessor, nodeType, currentLevel)...)
case TiDBIndexNestedLoopJoin, HintINLJ:
INLJTables = append(INLJTables, tableNames2HintTableInfo(b.ctx, hint.HintName.L, hint.Tables, b.hintProcessor, nodeType, currentLevel)...)
case HintINLHJ:
INLHJTables = append(INLHJTables, tableNames2HintTableInfo(b.ctx, hint.HintName.L, hint.Tables, b.hintProcessor, nodeType, currentLevel)...)
case HintINLMJ:
INLMJTables = append(INLMJTables, tableNames2HintTableInfo(b.ctx, hint.HintName.L, hint.Tables, b.hintProcessor, nodeType, currentLevel)...)
case TiDBHashJoin, HintHJ:
hashJoinTables = append(hashJoinTables, tableNames2HintTableInfo(b.ctx, hint.HintName.L, hint.Tables, b.hintProcessor, nodeType, currentLevel)...)
case HintHashAgg:
aggHints.preferAggType |= preferHashAgg
case HintStreamAgg:
aggHints.preferAggType |= preferStreamAgg
case HintAggToCop:
aggHints.preferAggToCop = true
case HintUseIndex:
dbName := hint.Tables[0].DBName
if dbName.L == "" {
dbName = model.NewCIStr(b.ctx.GetSessionVars().CurrentDB)
}
indexHintList = append(indexHintList, indexHintInfo{
dbName: dbName,
tblName: hint.Tables[0].TableName,
partitions: hint.Tables[0].PartitionList,
indexHint: &ast.IndexHint{
IndexNames: hint.Indexes,
HintType: ast.HintUse,
HintScope: ast.HintForScan,
},
})
case HintIgnoreIndex:
dbName := hint.Tables[0].DBName
if dbName.L == "" {
dbName = model.NewCIStr(b.ctx.GetSessionVars().CurrentDB)
}
indexHintList = append(indexHintList, indexHintInfo{
dbName: dbName,
tblName: hint.Tables[0].TableName,
partitions: hint.Tables[0].PartitionList,
indexHint: &ast.IndexHint{
IndexNames: hint.Indexes,
HintType: ast.HintIgnore,
HintScope: ast.HintForScan,
},
})
case HintReadFromStorage:
switch hint.HintData.(model.CIStr).L {
case HintTiFlash:
tiflashTables = append(tiflashTables, tableNames2HintTableInfo(b.ctx, hint.HintName.L, hint.Tables, b.hintProcessor, nodeType, currentLevel)...)
case HintTiKV:
tikvTables = append(tikvTables, tableNames2HintTableInfo(b.ctx, hint.HintName.L, hint.Tables, b.hintProcessor, nodeType, currentLevel)...)
}
case HintIndexMerge:
dbName := hint.Tables[0].DBName
if dbName.L == "" {
dbName = model.NewCIStr(b.ctx.GetSessionVars().CurrentDB)
}
indexMergeHintList = append(indexMergeHintList, indexHintInfo{
dbName: dbName,
tblName: hint.Tables[0].TableName,
partitions: hint.Tables[0].PartitionList,
indexHint: &ast.IndexHint{
IndexNames: hint.Indexes,
HintType: ast.HintUse,
HintScope: ast.HintForScan,
},
})
case HintTimeRange:
timeRangeHint = hint.HintData.(ast.HintTimeRange)
case HintLimitToCop:
limitHints.preferLimitToCop = true
default:
// ignore hints that not implemented
}
}
b.tableHintInfo = append(b.tableHintInfo, tableHintInfo{
sortMergeJoinTables: sortMergeTables,
broadcastJoinTables: BCTables,
broadcastJoinPreferredLocal: BCJPreferLocalTables,
indexNestedLoopJoinTables: indexNestedLoopJoinTables{INLJTables, INLHJTables, INLMJTables},
hashJoinTables: hashJoinTables,
indexHintList: indexHintList,
tiflashTables: tiflashTables,
tikvTables: tikvTables,
aggHints: aggHints,
indexMergeHintList: indexMergeHintList,
timeRangeHint: timeRangeHint,
limitHints: limitHints,
})
}
func (b *PlanBuilder) popTableHints() {
hintInfo := b.tableHintInfo[len(b.tableHintInfo)-1]
b.appendUnmatchedIndexHintWarning(hintInfo.indexHintList, false)
b.appendUnmatchedIndexHintWarning(hintInfo.indexMergeHintList, true)
b.appendUnmatchedJoinHintWarning(HintINLJ, TiDBIndexNestedLoopJoin, hintInfo.indexNestedLoopJoinTables.inljTables)
b.appendUnmatchedJoinHintWarning(HintINLHJ, "", hintInfo.indexNestedLoopJoinTables.inlhjTables)
b.appendUnmatchedJoinHintWarning(HintINLMJ, "", hintInfo.indexNestedLoopJoinTables.inlmjTables)
b.appendUnmatchedJoinHintWarning(HintSMJ, TiDBMergeJoin, hintInfo.sortMergeJoinTables)
b.appendUnmatchedJoinHintWarning(HintBCJ, TiDBBroadCastJoin, hintInfo.broadcastJoinTables)
b.appendUnmatchedJoinHintWarning(HintBCJPreferLocal, "", hintInfo.broadcastJoinPreferredLocal)
b.appendUnmatchedJoinHintWarning(HintHJ, TiDBHashJoin, hintInfo.hashJoinTables)
b.appendUnmatchedStorageHintWarning(hintInfo.tiflashTables, hintInfo.tikvTables)
b.tableHintInfo = b.tableHintInfo[:len(b.tableHintInfo)-1]
}
func (b *PlanBuilder) appendUnmatchedIndexHintWarning(indexHints []indexHintInfo, usedForIndexMerge bool) {
for _, hint := range indexHints {
if !hint.matched {
var hintTypeString string
if usedForIndexMerge {
hintTypeString = "use_index_merge"
} else {
hintTypeString = hint.hintTypeString()
}
errMsg := fmt.Sprintf("%s(%s) is inapplicable, check whether the table(%s.%s) exists",
hintTypeString,
hint.indexString(),
hint.dbName,
hint.tblName,
)
b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack(errMsg))
}
}
}
func (b *PlanBuilder) appendUnmatchedJoinHintWarning(joinType string, joinTypeAlias string, hintTables []hintTableInfo) {
unMatchedTables := extractUnmatchedTables(hintTables)
if len(unMatchedTables) == 0 {
return
}
if len(joinTypeAlias) != 0 {
joinTypeAlias = fmt.Sprintf(" or %s", restore2JoinHint(joinTypeAlias, hintTables))
}
errMsg := fmt.Sprintf("There are no matching table names for (%s) in optimizer hint %s%s. Maybe you can use the table alias name",
strings.Join(unMatchedTables, ", "), restore2JoinHint(joinType, hintTables), joinTypeAlias)
b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack(errMsg))
}
func (b *PlanBuilder) appendUnmatchedStorageHintWarning(tiflashTables, tikvTables []hintTableInfo) {
unMatchedTiFlashTables := extractUnmatchedTables(tiflashTables)
unMatchedTiKVTables := extractUnmatchedTables(tikvTables)
if len(unMatchedTiFlashTables)+len(unMatchedTiKVTables) == 0 {
return
}
errMsg := fmt.Sprintf("There are no matching table names for (%s) in optimizer hint %s. Maybe you can use the table alias name",
strings.Join(append(unMatchedTiFlashTables, unMatchedTiKVTables...), ", "),
restore2StorageHint(tiflashTables, tikvTables))
b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack(errMsg))
}
// TableHints returns the *tableHintInfo of PlanBuilder.
func (b *PlanBuilder) TableHints() *tableHintInfo {
if len(b.tableHintInfo) == 0 {
return nil
}
return &(b.tableHintInfo[len(b.tableHintInfo)-1])
}
func (b *PlanBuilder) buildSelect(ctx context.Context, sel *ast.SelectStmt) (p LogicalPlan, err error) {
b.pushSelectOffset(sel.QueryBlockOffset)
b.pushTableHints(sel.TableHints, utilhint.TypeSelect, sel.QueryBlockOffset)
defer func() {
b.popSelectOffset()
// table hints are only visible in the current SELECT statement.
b.popTableHints()
}()
enableNoopFuncs := b.ctx.GetSessionVars().EnableNoopFuncs
if sel.SelectStmtOpts != nil {
origin := b.inStraightJoin
b.inStraightJoin = sel.SelectStmtOpts.StraightJoin
defer func() { b.inStraightJoin = origin }()
}
var (
aggFuncs []*ast.AggregateFuncExpr
havingMap, orderMap, totalMap map[*ast.AggregateFuncExpr]int
windowAggMap map[*ast.AggregateFuncExpr]int
gbyCols []expression.Expression
)
if sel.From != nil {
p, err = b.buildResultSetNode(ctx, sel.From.TableRefs)
if err != nil {
return nil, err
}
} else {
p = b.buildTableDual()
}
originalFields := sel.Fields.Fields
sel.Fields.Fields, err = b.unfoldWildStar(p, sel.Fields.Fields)
if err != nil {
return nil, err
}
if b.capFlag&canExpandAST != 0 {
originalFields = sel.Fields.Fields
}
if sel.GroupBy != nil {
p, gbyCols, err = b.resolveGbyExprs(ctx, p, sel.GroupBy, sel.Fields.Fields)
if err != nil {
return nil, err
}
}
if b.ctx.GetSessionVars().SQLMode.HasOnlyFullGroupBy() && sel.From != nil {
err = b.checkOnlyFullGroupBy(p, sel)
if err != nil {
return nil, err
}
}
hasWindowFuncField := b.detectSelectWindow(sel)
if hasWindowFuncField {
windowAggMap, err = b.resolveWindowFunction(sel, p)
if err != nil {
return nil, err
}
}
// We must resolve having and order by clause before build projection,
// because when the query is "select a+1 as b from t having sum(b) < 0", we must replace sum(b) to sum(a+1),
// which only can be done before building projection and extracting Agg functions.
havingMap, orderMap, err = b.resolveHavingAndOrderBy(sel, p)
if err != nil {
return nil, err
}
// b.allNames will be used in evalDefaultExpr(). Default function is special because it needs to find the
// corresponding column name, but does not need the value in the column.
// For example, select a from t order by default(b), the column b will not be in select fields. Also because
// buildSort is after buildProjection, so we need get OutputNames before BuildProjection and store in allNames.
// Otherwise, we will get select fields instead of all OutputNames, so that we can't find the column b in the
// above example.
b.allNames = append(b.allNames, p.OutputNames())
defer func() { b.allNames = b.allNames[:len(b.allNames)-1] }()
if sel.Where != nil {
p, err = b.buildSelection(ctx, p, sel.Where, nil)
if err != nil {
return nil, err
}
}
if sel.LockTp != ast.SelectLockNone {
if sel.LockTp == ast.SelectLockInShareMode && !enableNoopFuncs {
err = expression.ErrFunctionsNoopImpl.GenWithStackByArgs("LOCK IN SHARE MODE")
return nil, err
}
p = b.buildSelectLock(p, sel.LockTp)
}
b.handleHelper.popMap()
b.handleHelper.pushMap(nil)
hasAgg := b.detectSelectAgg(sel)
if hasAgg {
aggFuncs, totalMap = b.extractAggFuncs(sel.Fields.Fields)
var aggIndexMap map[int]int
p, aggIndexMap, err = b.buildAggregation(ctx, p, aggFuncs, gbyCols)
if err != nil {
return nil, err
}
for k, v := range totalMap {
totalMap[k] = aggIndexMap[v]
}
}
var oldLen int
// According to https://dev.mysql.com/doc/refman/8.0/en/window-functions-usage.html,
// we can only process window functions after having clause, so `considerWindow` is false now.
p, oldLen, err = b.buildProjection(ctx, p, sel.Fields.Fields, totalMap, nil, false, sel.OrderBy != nil)
if err != nil {
return nil, err
}
if sel.Having != nil {
b.curClause = havingClause
p, err = b.buildSelection(ctx, p, sel.Having.Expr, havingMap)
if err != nil {
return nil, err
}
}
b.windowSpecs, err = buildWindowSpecs(sel.WindowSpecs)
if err != nil {
return nil, err
}
var windowMapper map[*ast.WindowFuncExpr]int
if hasWindowFuncField {
windowFuncs := extractWindowFuncs(sel.Fields.Fields)
// we need to check the func args first before we check the window spec
err := b.checkWindowFuncArgs(ctx, p, windowFuncs, windowAggMap)
if err != nil {
return nil, err
}
groupedFuncs, err := b.groupWindowFuncs(windowFuncs)
if err != nil {
return nil, err
}
p, windowMapper, err = b.buildWindowFunctions(ctx, p, groupedFuncs, windowAggMap)
if err != nil {
return nil, err
}
// Now we build the window function fields.
p, oldLen, err = b.buildProjection(ctx, p, sel.Fields.Fields, windowAggMap, windowMapper, true, false)
if err != nil {
return nil, err
}
}
if sel.Distinct {
p, err = b.buildDistinct(p, oldLen)
if err != nil {
return nil, err
}
}
if sel.OrderBy != nil {
p, err = b.buildSort(ctx, p, sel.OrderBy.Items, orderMap, windowMapper)
if err != nil {
return nil, err
}
}
if sel.Limit != nil {
p, err = b.buildLimit(p, sel.Limit)
if err != nil {
return nil, err
}
}
sel.Fields.Fields = originalFields
if oldLen != p.Schema().Len() {
proj := LogicalProjection{Exprs: expression.Column2Exprs(p.Schema().Columns[:oldLen])}.Init(b.ctx, b.getSelectOffset())
proj.SetChildren(p)
schema := expression.NewSchema(p.Schema().Clone().Columns[:oldLen]...)
for _, col := range schema.Columns {
col.UniqueID = b.ctx.GetSessionVars().AllocPlanColumnID()
}
proj.names = p.OutputNames()[:oldLen]
proj.SetSchema(schema)
return proj, nil
}
return p, nil
}
func (b *PlanBuilder) buildTableDual() *LogicalTableDual {
b.handleHelper.pushMap(nil)
return LogicalTableDual{RowCount: 1}.Init(b.ctx, b.getSelectOffset())
}
func (ds *DataSource) newExtraHandleSchemaCol() *expression.Column {
return &expression.Column{
RetType: types.NewFieldType(mysql.TypeLonglong),
UniqueID: ds.ctx.GetSessionVars().AllocPlanColumnID(),
ID: model.ExtraHandleID,
OrigName: fmt.Sprintf("%v.%v.%v", ds.DBName, ds.tableInfo.Name, model.ExtraHandleName),
}
}
// getStatsTable gets statistics information for a table specified by "tableID".
// A pseudo statistics table is returned in any of the following scenario:
// 1. tidb-server started and statistics handle has not been initialized.
// 2. table row count from statistics is zero.
// 3. statistics is outdated.
func getStatsTable(ctx sessionctx.Context, tblInfo *model.TableInfo, pid int64) *statistics.Table {
statsHandle := domain.GetDomain(ctx).StatsHandle()
// 1. tidb-server started and statistics handle has not been initialized.
if statsHandle == nil {
return statistics.PseudoTable(tblInfo)
}
var statsTbl *statistics.Table
if pid != tblInfo.ID {
statsTbl = statsHandle.GetPartitionStats(tblInfo, pid)
} else {
statsTbl = statsHandle.GetTableStats(tblInfo)
}
// 2. table row count from statistics is zero.
if statsTbl.Count == 0 {
return statistics.PseudoTable(tblInfo)
}
// 3. statistics is outdated.
if statsTbl.IsOutdated() {
tbl := *statsTbl
tbl.Pseudo = true
statsTbl = &tbl
metrics.PseudoEstimation.Inc()
}
return statsTbl
}
func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, asName *model.CIStr) (LogicalPlan, error) {
dbName := tn.Schema
sessionVars := b.ctx.GetSessionVars()
if dbName.L == "" {
dbName = model.NewCIStr(sessionVars.CurrentDB)
}
tbl, err := b.is.TableByName(dbName, tn.Name)
if err != nil {
return nil, err
}
tableInfo := tbl.Meta()
var authErr error
if sessionVars.User != nil {
authErr = ErrTableaccessDenied.FastGenByArgs("SELECT", sessionVars.User.AuthUsername, sessionVars.User.AuthHostname, tableInfo.Name.L)
}
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SelectPriv, dbName.L, tableInfo.Name.L, "", authErr)
if tbl.Type().IsVirtualTable() {
return b.buildMemTable(ctx, dbName, tableInfo)
}
if tableInfo.IsView() {
return b.BuildDataSourceFromView(ctx, dbName, tableInfo)
}
if tableInfo.GetPartitionInfo() != nil {
b.optFlag = b.optFlag | flagPartitionProcessor
pt := tbl.(table.PartitionedTable)
// check partition by name.
if len(tn.PartitionNames) > 0 {
pids := make(map[int64]struct{}, len(tn.PartitionNames))
for _, name := range tn.PartitionNames {
pid, err := tables.FindPartitionByName(tableInfo, name.L)
if err != nil {
return nil, err
}
pids[pid] = struct{}{}
}
pt = tables.NewPartitionTableithGivenSets(pt, pids)
}
b.partitionedTable = append(b.partitionedTable, pt)
} else if len(tn.PartitionNames) != 0 {
return nil, ErrPartitionClauseOnNonpartitioned
}
tblName := *asName
if tblName.L == "" {
tblName = tn.Name
}
possiblePaths, err := getPossibleAccessPaths(b.ctx, b.TableHints(), tn.IndexHints, tbl, dbName, tblName)
if err != nil {
return nil, err
}
possiblePaths, err = filterPathByIsolationRead(b.ctx, possiblePaths, dbName)
if err != nil {
return nil, err
}
// Try to substitute generate column only if there is an index on generate column.
for _, index := range tableInfo.Indices {
if index.State != model.StatePublic {
continue
}
for _, indexCol := range index.Columns {
colInfo := tbl.Cols()[indexCol.Offset]
if colInfo.IsGenerated() && !colInfo.GeneratedStored {
b.optFlag |= flagGcSubstitute
break
}
}
}
var columns []*table.Column
if b.inUpdateStmt {
// create table t(a int, b int).
// Imagine that, There are 2 TiDB instances in the cluster, name A, B. We add a column `c` to table t in the TiDB cluster.
// One of the TiDB, A, the column type in its infoschema is changed to public. And in the other TiDB, the column type is
// still StateWriteReorganization.
// TiDB A: insert into t values(1, 2, 3);
// TiDB B: update t set a = 2 where b = 2;
// If we use tbl.Cols() here, the update statement, will ignore the col `c`, and the data `3` will lost.
columns = tbl.WritableCols()
} else if b.inDeleteStmt {
columns = tbl.DeletableCols()
} else {
columns = tbl.Cols()
}
var statisticTable *statistics.Table
if _, ok := tbl.(table.PartitionedTable); !ok {
statisticTable = getStatsTable(b.ctx, tbl.Meta(), tbl.Meta().ID)
}
// extract the IndexMergeHint
var indexMergeHints []indexHintInfo
if hints := b.TableHints(); hints != nil {
for i, hint := range hints.indexMergeHintList {
if hint.tblName.L == tblName.L && hint.dbName.L == dbName.L {
hints.indexMergeHintList[i].matched = true
// check whether the index names in IndexMergeHint are valid.
invalidIdxNames := make([]string, 0, len(hint.indexHint.IndexNames))
for _, idxName := range hint.indexHint.IndexNames {
hasIdxName := false
for _, path := range possiblePaths {
if path.IsTablePath {
if idxName.L == "primary" {
hasIdxName = true
break
}
continue
}
if idxName.L == path.Index.Name.L {
hasIdxName = true
break
}
}
if !hasIdxName {
invalidIdxNames = append(invalidIdxNames, idxName.String())
}
}
if len(invalidIdxNames) == 0 {
indexMergeHints = append(indexMergeHints, hint)
} else {
// Append warning if there are invalid index names.
errMsg := fmt.Sprintf("use_index_merge(%s) is inapplicable, check whether the indexes (%s) "+
"exist, or the indexes are conflicted with use_index/ignore_index hints.",
hint.indexString(), strings.Join(invalidIdxNames, ", "))
b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack(errMsg))
}
}
}
}
ds := DataSource{
DBName: dbName,
TableAsName: asName,
table: tbl,
tableInfo: tableInfo,
statisticTable: statisticTable,
astIndexHints: tn.IndexHints,
IndexHints: b.TableHints().indexHintList,
indexMergeHints: indexMergeHints,
possibleAccessPaths: possiblePaths,
Columns: make([]*model.ColumnInfo, 0, len(columns)),
partitionNames: tn.PartitionNames,
TblCols: make([]*expression.Column, 0, len(columns)),
preferPartitions: make(map[int][]model.CIStr),
}.Init(b.ctx, b.getSelectOffset())
var handleCol *expression.Column
schema := expression.NewSchema(make([]*expression.Column, 0, len(columns))...)
names := make([]*types.FieldName, 0, len(columns))
for i, col := range columns {
ds.Columns = append(ds.Columns, col.ToInfo())
names = append(names, &types.FieldName{
DBName: dbName,
TblName: tableInfo.Name,
ColName: col.Name,
OrigTblName: tableInfo.Name,
OrigColName: col.Name,
Hidden: col.Hidden,
})
newCol := &expression.Column{
UniqueID: sessionVars.AllocPlanColumnID(),
ID: col.ID,
RetType: col.FieldType.Clone(),
OrigName: names[i].String(),
IsHidden: col.Hidden,
}
if tableInfo.PKIsHandle && mysql.HasPriKeyFlag(col.Flag) {
handleCol = newCol
}
schema.Append(newCol)
ds.TblCols = append(ds.TblCols, newCol)
}
// We append an extra handle column to the schema when the handle
// column is not the primary key of "ds".
if handleCol == nil {
ds.Columns = append(ds.Columns, model.NewExtraHandleColInfo())
handleCol = ds.newExtraHandleSchemaCol()
schema.Append(handleCol)
names = append(names, &types.FieldName{
DBName: dbName,
TblName: tableInfo.Name,
ColName: model.ExtraHandleName,
OrigColName: model.ExtraHandleName,
})
ds.TblCols = append(ds.TblCols, handleCol)
}
if handleCol != nil {
ds.handleCol = handleCol
handleMap := make(map[int64][]*expression.Column)
handleMap[tableInfo.ID] = []*expression.Column{handleCol}
b.handleHelper.pushMap(handleMap)
} else {
b.handleHelper.pushMap(nil)
}
ds.SetSchema(schema)
ds.names = names
ds.setPreferredStoreType(b.TableHints())
// Init FullIdxCols, FullIdxColLens for accessPaths.
for _, path := range ds.possibleAccessPaths {
if !path.IsTablePath {
path.FullIdxCols, path.FullIdxColLens = expression.IndexInfo2Cols(ds.Columns, ds.schema.Columns, path.Index)
}
}
var result LogicalPlan = ds
dirty := tableHasDirtyContent(b.ctx, tableInfo)
if dirty {
us := LogicalUnionScan{handleCol: handleCol}.Init(b.ctx, b.getSelectOffset())
us.SetChildren(ds)
result = us
}
if sessionVars.StmtCtx.TblInfo2UnionScan == nil {
sessionVars.StmtCtx.TblInfo2UnionScan = make(map[*model.TableInfo]bool)
}
sessionVars.StmtCtx.TblInfo2UnionScan[tableInfo] = dirty
for i, colExpr := range ds.Schema().Columns {
var expr expression.Expression
if i < len(columns) {
if columns[i].IsGenerated() && !columns[i].GeneratedStored {
var err error
expr, _, err = b.rewrite(ctx, columns[i].GeneratedExpr, ds, nil, true)
if err != nil {
return nil, err
}
colExpr.VirtualExpr = expr.Clone()
}
}
}
return result, nil
}
func (b *PlanBuilder) timeRangeForSummaryTable() QueryTimeRange {
const defaultSummaryDuration = 30 * time.Minute
hints := b.TableHints()
// User doesn't use TIME_RANGE hint
if hints == nil || (hints.timeRangeHint.From == "" && hints.timeRangeHint.To == "") {
to := time.Now()
from := to.Add(-defaultSummaryDuration)
return QueryTimeRange{From: from, To: to}
}
// Parse time specified by user via TIM_RANGE hint
parse := func(s string) (time.Time, bool) {
t, err := time.ParseInLocation(MetricTableTimeFormat, s, time.Local)
if err != nil {
b.ctx.GetSessionVars().StmtCtx.AppendWarning(err)
}
return t, err == nil
}
from, fromValid := parse(hints.timeRangeHint.From)
to, toValid := parse(hints.timeRangeHint.To)
switch {
case !fromValid && !toValid:
to = time.Now()
from = to.Add(-defaultSummaryDuration)
case fromValid && !toValid:
to = from.Add(defaultSummaryDuration)
case !fromValid && toValid:
from = to.Add(-defaultSummaryDuration)
}
return QueryTimeRange{From: from, To: to}
}
func (b *PlanBuilder) buildMemTable(_ context.Context, dbName model.CIStr, tableInfo *model.TableInfo) (LogicalPlan, error) {
// We can use the `tableInfo.Columns` directly because the memory table has
// a stable schema and there is no online DDL on the memory table.
schema := expression.NewSchema(make([]*expression.Column, 0, len(tableInfo.Columns))...)
names := make([]*types.FieldName, 0, len(tableInfo.Columns))
var handleCol *expression.Column
for _, col := range tableInfo.Columns {
names = append(names, &types.FieldName{
DBName: dbName,
TblName: tableInfo.Name,
ColName: col.Name,
OrigTblName: tableInfo.Name,
OrigColName: col.Name,
})
// NOTE: Rewrite the expression if memory table supports generated columns in the future
newCol := &expression.Column{
UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(),
ID: col.ID,
RetType: &col.FieldType,
}
if tableInfo.PKIsHandle && mysql.HasPriKeyFlag(col.Flag) {
handleCol = newCol
}
schema.Append(newCol)
}
if handleCol != nil {
handleMap := make(map[int64][]*expression.Column)
handleMap[tableInfo.ID] = []*expression.Column{handleCol}
b.handleHelper.pushMap(handleMap)
} else {
b.handleHelper.pushMap(nil)
}
// NOTE: Add a `LogicalUnionScan` if we support update memory table in the future
p := LogicalMemTable{
DBName: dbName,
TableInfo: tableInfo,
}.Init(b.ctx, b.getSelectOffset())
p.SetSchema(schema)
p.names = names
// Some memory tables can receive some predicates
switch dbName.L {
case util2.MetricSchemaName.L:
p.Extractor = newMetricTableExtractor()
case util2.InformationSchemaName.L:
switch strings.ToUpper(tableInfo.Name.O) {
case infoschema.TableClusterConfig, infoschema.TableClusterLoad, infoschema.TableClusterHardware, infoschema.TableClusterSystemInfo:
p.Extractor = &ClusterTableExtractor{}
case infoschema.TableClusterLog:
p.Extractor = &ClusterLogTableExtractor{}
case infoschema.TableInspectionResult:
p.Extractor = &InspectionResultTableExtractor{}
p.QueryTimeRange = b.timeRangeForSummaryTable()
case infoschema.TableInspectionSummary:
p.Extractor = &InspectionSummaryTableExtractor{}
p.QueryTimeRange = b.timeRangeForSummaryTable()
case infoschema.TableInspectionRules:
p.Extractor = &InspectionRuleTableExtractor{}
case infoschema.TableMetricSummary, infoschema.TableMetricSummaryByLabel:
p.Extractor = &MetricSummaryTableExtractor{}
p.QueryTimeRange = b.timeRangeForSummaryTable()
case infoschema.TableSlowQuery:
p.Extractor = &SlowQueryExtractor{}
case infoschema.TableTiFlashTables, infoschema.TableTiFlashSegments:
p.Extractor = &TiFlashSystemTableExtractor{}
case infoschema.TableStorageStats:
p.Extractor = &TableStorageStatsExtractor{}
}
}
return p, nil
}
// checkRecursiveView checks whether this view is recursively defined.
func (b *PlanBuilder) checkRecursiveView(dbName model.CIStr, tableName model.CIStr) (func(), error) {
viewFullName := dbName.L + "." + tableName.L
if b.buildingViewStack == nil {
b.buildingViewStack = set.NewStringSet()
}
// If this view has already been on the building stack, it means
// this view contains a recursive definition.
if b.buildingViewStack.Exist(viewFullName) {
return nil, ErrViewRecursive.GenWithStackByArgs(dbName.O, tableName.O)
}
// If the view is being renamed, we return the mysql compatible error message.
if b.capFlag&renameView != 0 && viewFullName == b.renamingViewName {
return nil, ErrNoSuchTable.GenWithStackByArgs(dbName.O, tableName.O)
}
b.buildingViewStack.Insert(viewFullName)
return func() { delete(b.buildingViewStack, viewFullName) }, nil
}
// BuildDataSourceFromView is used to build LogicalPlan from view
func (b *PlanBuilder) BuildDataSourceFromView(ctx context.Context, dbName model.CIStr, tableInfo *model.TableInfo) (LogicalPlan, error) {
deferFunc, err := b.checkRecursiveView(dbName, tableInfo.Name)
if err != nil {
return nil, err
}
defer deferFunc()
charset, collation := b.ctx.GetSessionVars().GetCharsetInfo()
viewParser := parser.New()
viewParser.EnableWindowFunc(b.ctx.GetSessionVars().EnableWindowFunction)
selectNode, err := viewParser.ParseOneStmt(tableInfo.View.SelectStmt, charset, collation)
if err != nil {
return nil, err
}
originalVisitInfo := b.visitInfo
b.visitInfo = make([]visitInfo, 0)
selectLogicalPlan, err := b.Build(ctx, selectNode)
if err != nil {
if terror.ErrorNotEqual(err, ErrViewRecursive) &&
terror.ErrorNotEqual(err, ErrNoSuchTable) {
err = ErrViewInvalid.GenWithStackByArgs(dbName.O, tableInfo.Name.O)
}
return nil, err
}
if tableInfo.View.Security == model.SecurityDefiner {
if pm := privilege.GetPrivilegeManager(b.ctx); pm != nil {
for _, v := range b.visitInfo {
if !pm.RequestVerificationWithUser(v.db, v.table, v.column, v.privilege, tableInfo.View.Definer) {
return nil, ErrViewInvalid.GenWithStackByArgs(dbName.O, tableInfo.Name.O)
}
}
}
b.visitInfo = b.visitInfo[:0]
}
b.visitInfo = append(originalVisitInfo, b.visitInfo...)
if b.ctx.GetSessionVars().StmtCtx.InExplainStmt {
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.ShowViewPriv, dbName.L, tableInfo.Name.L, "", ErrViewNoExplain)
}
if len(tableInfo.Columns) != selectLogicalPlan.Schema().Len() {
return nil, ErrViewInvalid.GenWithStackByArgs(dbName.O, tableInfo.Name.O)
}
return b.buildProjUponView(ctx, dbName, tableInfo, selectLogicalPlan)
}
func (b *PlanBuilder) buildProjUponView(ctx context.Context, dbName model.CIStr, tableInfo *model.TableInfo, selectLogicalPlan Plan) (LogicalPlan, error) {
columnInfo := tableInfo.Cols()
cols := selectLogicalPlan.Schema().Clone().Columns
outputNamesOfUnderlyingSelect := selectLogicalPlan.OutputNames().Shallow()
// In the old version of VIEW implementation, tableInfo.View.Cols is used to
// store the origin columns' names of the underlying SelectStmt used when
// creating the view.
if tableInfo.View.Cols != nil {
cols = cols[:0]
outputNamesOfUnderlyingSelect = outputNamesOfUnderlyingSelect[:0]
for _, info := range columnInfo {
idx := expression.FindFieldNameIdxByColName(selectLogicalPlan.OutputNames(), info.Name.L)
if idx == -1 {
return nil, ErrViewInvalid.GenWithStackByArgs(dbName.O, tableInfo.Name.O)
}
cols = append(cols, selectLogicalPlan.Schema().Columns[idx])
outputNamesOfUnderlyingSelect = append(outputNamesOfUnderlyingSelect, selectLogicalPlan.OutputNames()[idx])
}
}
projSchema := expression.NewSchema(make([]*expression.Column, 0, len(tableInfo.Columns))...)
projExprs := make([]expression.Expression, 0, len(tableInfo.Columns))
projNames := make(types.NameSlice, 0, len(tableInfo.Columns))
for i, name := range outputNamesOfUnderlyingSelect {
origColName := name.ColName
if tableInfo.View.Cols != nil {
origColName = tableInfo.View.Cols[i]
}
projNames = append(projNames, &types.FieldName{
// TblName is the of view instead of the name of the underlying table.
TblName: tableInfo.Name,
OrigTblName: name.OrigTblName,
ColName: columnInfo[i].Name,
OrigColName: origColName,
DBName: dbName,
})
projSchema.Append(&expression.Column{
UniqueID: cols[i].UniqueID,
RetType: cols[i].GetType(),
})
projExprs = append(projExprs, cols[i])
}
projUponView := LogicalProjection{Exprs: projExprs}.Init(b.ctx, b.getSelectOffset())
projUponView.names = projNames
projUponView.SetChildren(selectLogicalPlan.(LogicalPlan))
projUponView.SetSchema(projSchema)
return projUponView, nil
}
// buildApplyWithJoinType builds apply plan with outerPlan and innerPlan, which apply join with particular join type for
// every row from outerPlan and the whole innerPlan.
func (b *PlanBuilder) buildApplyWithJoinType(outerPlan, innerPlan LogicalPlan, tp JoinType) LogicalPlan {
b.optFlag = b.optFlag | flagPredicatePushDown | flagBuildKeyInfo | flagDecorrelate
ap := LogicalApply{LogicalJoin: LogicalJoin{JoinType: tp}}.Init(b.ctx, b.getSelectOffset())
ap.SetChildren(outerPlan, innerPlan)
ap.names = make([]*types.FieldName, outerPlan.Schema().Len()+innerPlan.Schema().Len())
copy(ap.names, outerPlan.OutputNames())
ap.SetSchema(expression.MergeSchema(outerPlan.Schema(), innerPlan.Schema()))
// Note that, tp can only be LeftOuterJoin or InnerJoin, so we don't consider other outer joins.
if tp == LeftOuterJoin {
b.optFlag = b.optFlag | flagEliminateOuterJoin
resetNotNullFlag(ap.schema, outerPlan.Schema().Len(), ap.schema.Len())
}
for i := outerPlan.Schema().Len(); i < ap.Schema().Len(); i++ {
ap.names[i] = types.EmptyName
}
return ap
}
// buildSemiApply builds apply plan with outerPlan and innerPlan, which apply semi-join for every row from outerPlan and the whole innerPlan.
func (b *PlanBuilder) buildSemiApply(outerPlan, innerPlan LogicalPlan, condition []expression.Expression, asScalar, not bool) (LogicalPlan, error) {
b.optFlag = b.optFlag | flagPredicatePushDown | flagBuildKeyInfo | flagDecorrelate
join, err := b.buildSemiJoin(outerPlan, innerPlan, condition, asScalar, not)
if err != nil {
return nil, err
}
ap := &LogicalApply{LogicalJoin: *join}
ap.tp = plancodec.TypeApply
ap.self = ap
return ap, nil
}
func (b *PlanBuilder) buildMaxOneRow(p LogicalPlan) LogicalPlan {
maxOneRow := LogicalMaxOneRow{}.Init(b.ctx, b.getSelectOffset())
maxOneRow.SetChildren(p)
return maxOneRow
}
func (b *PlanBuilder) buildSemiJoin(outerPlan, innerPlan LogicalPlan, onCondition []expression.Expression, asScalar bool, not bool) (*LogicalJoin, error) {
joinPlan := LogicalJoin{}.Init(b.ctx, b.getSelectOffset())
for i, expr := range onCondition {
onCondition[i] = expr.Decorrelate(outerPlan.Schema())
}
joinPlan.SetChildren(outerPlan, innerPlan)
joinPlan.AttachOnConds(onCondition)
joinPlan.names = make([]*types.FieldName, outerPlan.Schema().Len(), outerPlan.Schema().Len()+innerPlan.Schema().Len()+1)
copy(joinPlan.names, outerPlan.OutputNames())
if asScalar {
newSchema := outerPlan.Schema().Clone()
newSchema.Append(&expression.Column{
RetType: types.NewFieldType(mysql.TypeTiny),
UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(),
})
joinPlan.names = append(joinPlan.names, types.EmptyName)
joinPlan.SetSchema(newSchema)
if not {
joinPlan.JoinType = AntiLeftOuterSemiJoin
} else {
joinPlan.JoinType = LeftOuterSemiJoin
}
} else {
joinPlan.SetSchema(outerPlan.Schema().Clone())
if not {
joinPlan.JoinType = AntiSemiJoin
} else {
joinPlan.JoinType = SemiJoin
}
}
// Apply forces to choose hash join currently, so don't worry the hints will take effect if the semi join is in one apply.
if b.TableHints() != nil {
outerAlias := extractTableAlias(outerPlan, joinPlan.blockOffset)
innerAlias := extractTableAlias(innerPlan, joinPlan.blockOffset)
if b.TableHints().ifPreferMergeJoin(outerAlias, innerAlias) {
joinPlan.preferJoinType |= preferMergeJoin
}
if b.TableHints().ifPreferHashJoin(outerAlias, innerAlias) {
joinPlan.preferJoinType |= preferHashJoin
}
if b.TableHints().ifPreferINLJ(innerAlias) {
joinPlan.preferJoinType = preferRightAsINLJInner
}
if b.TableHints().ifPreferINLHJ(innerAlias) {
joinPlan.preferJoinType = preferRightAsINLHJInner
}
if b.TableHints().ifPreferINLMJ(innerAlias) {
joinPlan.preferJoinType = preferRightAsINLMJInner
}
// If there're multiple join hints, they're conflict.
if bits.OnesCount(joinPlan.preferJoinType) > 1 {
return nil, errors.New("Join hints are conflict, you can only specify one type of join")
}
}
return joinPlan, nil
}
func getTableOffset(names []*types.FieldName, handleName *types.FieldName) (int, error) {
for i, name := range names {
if name.DBName.L == handleName.DBName.L && name.TblName.L == handleName.TblName.L {
return i, nil
}
}
return -1, errors.Errorf("Couldn't get column information when do update/delete")
}
// TblColPosInfo represents an mapper from column index to handle index.
type TblColPosInfo struct {
TblID int64
// Start and End represent the ordinal range [Start, End) of the consecutive columns.
Start, End int
// HandleOrdinal represents the ordinal of the handle column.
HandleOrdinal int
}
// TblColPosInfoSlice attaches the methods of sort.Interface to []TblColPosInfos sorting in increasing order.
type TblColPosInfoSlice []TblColPosInfo
// Len implements sort.Interface#Len.
func (c TblColPosInfoSlice) Len() int {
return len(c)
}
// Swap implements sort.Interface#Swap.
func (c TblColPosInfoSlice) Swap(i, j int) {
c[i], c[j] = c[j], c[i]
}
// Less implements sort.Interface#Less.
func (c TblColPosInfoSlice) Less(i, j int) bool {
return c[i].Start < c[j].Start
}
// FindHandle finds the ordinal of the corresponding handle column.
func (c TblColPosInfoSlice) FindHandle(colOrdinal int) (int, bool) {
if len(c) == 0 {
return 0, false
}
// find the smallest index of the range that its start great than colOrdinal.
// @see https://godoc.org/sort#Search
rangeBehindOrdinal := sort.Search(len(c), func(i int) bool { return c[i].Start > colOrdinal })
if rangeBehindOrdinal == 0 {
return 0, false
}
return c[rangeBehindOrdinal-1].HandleOrdinal, true
}
// buildColumns2Handle builds columns to handle mapping.
func buildColumns2Handle(
names []*types.FieldName,
tblID2Handle map[int64][]*expression.Column,
tblID2Table map[int64]table.Table,
onlyWritableCol bool,
) (TblColPosInfoSlice, error) {
var cols2Handles TblColPosInfoSlice
for tblID, handleCols := range tblID2Handle {
tbl := tblID2Table[tblID]
var tblLen int
if onlyWritableCol {
tblLen = len(tbl.WritableCols())
} else {
tblLen = len(tbl.Cols())
}
for _, handleCol := range handleCols {
offset, err := getTableOffset(names, names[handleCol.Index])
if err != nil {
return nil, err
}
end := offset + tblLen
cols2Handles = append(cols2Handles, TblColPosInfo{tblID, offset, end, handleCol.Index})
}
}
sort.Sort(cols2Handles)
return cols2Handles, nil
}
func (b *PlanBuilder) buildUpdate(ctx context.Context, update *ast.UpdateStmt) (Plan, error) {
b.pushSelectOffset(0)
b.pushTableHints(update.TableHints, utilhint.TypeUpdate, 0)
defer func() {
b.popSelectOffset()
// table hints are only visible in the current UPDATE statement.
b.popTableHints()
}()
// update subquery table should be forbidden
var asNameList []string
asNameList = extractTableSourceAsNames(update.TableRefs.TableRefs, asNameList, true)
for _, asName := range asNameList {
for _, assign := range update.List {
if assign.Column.Table.L == asName {
return nil, ErrNonUpdatableTable.GenWithStackByArgs(asName, "UPDATE")
}
}
}
b.inUpdateStmt = true
p, err := b.buildResultSetNode(ctx, update.TableRefs.TableRefs)
if err != nil {
return nil, err
}
var tableList []*ast.TableName
tableList = extractTableList(update.TableRefs.TableRefs, tableList, false)
for _, t := range tableList {
dbName := t.Schema.L
if dbName == "" {
dbName = b.ctx.GetSessionVars().CurrentDB
}
if t.TableInfo.IsView() {
return nil, errors.Errorf("update view %s is not supported now.", t.Name.O)
}
if t.TableInfo.IsSequence() {
return nil, errors.Errorf("update sequence %s is not supported now.", t.Name.O)
}
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SelectPriv, dbName, t.Name.L, "", nil)
}
oldSchemaLen := p.Schema().Len()
if update.Where != nil {
p, err = b.buildSelection(ctx, p, update.Where, nil)
if err != nil {
return nil, err
}
}
if b.ctx.GetSessionVars().TxnCtx.IsPessimistic {
if update.TableRefs.TableRefs.Right == nil {
// buildSelectLock is an optimization that can reduce RPC call.
// We only need do this optimization for single table update which is the most common case.
// When TableRefs.Right is nil, it is single table update.
p = b.buildSelectLock(p, ast.SelectLockForUpdate)
}
}
if update.Order != nil {
p, err = b.buildSort(ctx, p, update.Order.Items, nil, nil)
if err != nil {
return nil, err
}
}
if update.Limit != nil {
p, err = b.buildLimit(p, update.Limit)
if err != nil {
return nil, err
}
}
// Add project to freeze the order of output columns.
proj := LogicalProjection{Exprs: expression.Column2Exprs(p.Schema().Columns[:oldSchemaLen])}.Init(b.ctx, b.getSelectOffset())
proj.SetSchema(expression.NewSchema(make([]*expression.Column, oldSchemaLen)...))
proj.names = make(types.NameSlice, len(p.OutputNames()))
copy(proj.names, p.OutputNames())
copy(proj.schema.Columns, p.Schema().Columns[:oldSchemaLen])
proj.SetChildren(p)
p = proj
var updateTableList []*ast.TableName
updateTableList = extractTableList(update.TableRefs.TableRefs, updateTableList, true)
orderedList, np, allAssignmentsAreConstant, err := b.buildUpdateLists(ctx, updateTableList, update.List, p)
if err != nil {
return nil, err
}
p = np
updt := Update{
OrderedList: orderedList,
AllAssignmentsAreConstant: allAssignmentsAreConstant,
}.Init(b.ctx)
updt.names = p.OutputNames()
// We cannot apply projection elimination when building the subplan, because
// columns in orderedList cannot be resolved.
updt.SelectPlan, _, err = DoOptimize(ctx, b.ctx, b.optFlag&^flagEliminateProjection, p)
if err != nil {
return nil, err
}
err = updt.ResolveIndices()
if err != nil {
return nil, err
}
tblID2Handle, err := resolveIndicesForTblID2Handle(b.handleHelper.tailMap(), updt.SelectPlan.Schema())
if err != nil {
return nil, err
}
tblID2table := make(map[int64]table.Table, len(tblID2Handle))
for id := range tblID2Handle {
tblID2table[id], _ = b.is.TableByID(id)
}
updt.TblColPosInfos, err = buildColumns2Handle(updt.OutputNames(), tblID2Handle, tblID2table, true)
if err == nil {
err = checkUpdateList(b.ctx, tblID2table, updt)
}
updt.PartitionedTable = b.partitionedTable
return updt, err
}
// GetUpdateColumns gets the columns of updated lists.
func GetUpdateColumns(ctx sessionctx.Context, orderedList []*expression.Assignment, schemaLen int) ([]bool, error) {
assignFlag := make([]bool, schemaLen)
for _, v := range orderedList {
if !ctx.GetSessionVars().AllowWriteRowID && v.Col.ID == model.ExtraHandleID {
return nil, errors.Errorf("insert, update and replace statements for _tidb_rowid are not supported.")
}
idx := v.Col.Index
assignFlag[idx] = true
}
return assignFlag, nil
}
func checkUpdateList(ctx sessionctx.Context, tblID2table map[int64]table.Table, updt *Update) error {
assignFlags, err := GetUpdateColumns(ctx, updt.OrderedList, updt.SelectPlan.Schema().Len())
if err != nil {
return err
}
isPKUpdated := make(map[int64]model.CIStr)
for _, content := range updt.TblColPosInfos {
tbl := tblID2table[content.TblID]
flags := assignFlags[content.Start:content.End]
var updatePK bool
for i, col := range tbl.WritableCols() {
if flags[i] && col.State != model.StatePublic {
return ErrUnknownColumn.GenWithStackByArgs(col.Name, clauseMsg[fieldList])
}
// Check for multi-updates on primary key,
// see https://dev.mysql.com/doc/mysql-errors/5.7/en/server-error-reference.html#error_er_multi_update_key_conflict
if !flags[i] {
continue
}
if mysql.HasPriKeyFlag(col.Flag) {
updatePK = true
}
}
if updatePK {
if otherTblName, ok := isPKUpdated[tbl.Meta().ID]; ok {
return ErrMultiUpdateKeyConflict.GenWithStackByArgs(otherTblName.O, updt.names[content.Start].TblName.O)
}
isPKUpdated[tbl.Meta().ID] = updt.names[content.Start].TblName
}
}
return nil
}
func (b *PlanBuilder) buildUpdateLists(
ctx context.Context,
tableList []*ast.TableName,
list []*ast.Assignment,
p LogicalPlan,
) (newList []*expression.Assignment,
po LogicalPlan,
allAssignmentsAreConstant bool,
e error,
) {
b.curClause = fieldList
// modifyColumns indicates which columns are in set list,
// and if it is set to `DEFAULT`
modifyColumns := make(map[string]bool, p.Schema().Len())
var columnsIdx map[*ast.ColumnName]int
cacheColumnsIdx := false
if len(p.OutputNames()) > 16 {
cacheColumnsIdx = true
columnsIdx = make(map[*ast.ColumnName]int, len(list))
}
for _, assign := range list {
idx, err := expression.FindFieldName(p.OutputNames(), assign.Column)
if err != nil {
return nil, nil, false, err
}
if idx < 0 {
return nil, nil, false, ErrUnknownColumn.GenWithStackByArgs(assign.Column.Name, "field list")
}
if cacheColumnsIdx {
columnsIdx[assign.Column] = idx
}
name := p.OutputNames()[idx]
columnFullName := fmt.Sprintf("%s.%s.%s", name.DBName.L, name.TblName.L, name.ColName.L)
// We save a flag for the column in map `modifyColumns`
// This flag indicated if assign keyword `DEFAULT` to the column
if extractDefaultExpr(assign.Expr) != nil {
modifyColumns[columnFullName] = true
} else {
modifyColumns[columnFullName] = false
}
}
// If columns in set list contains generated columns, raise error.
// And, fill virtualAssignments here; that's for generated columns.
virtualAssignments := make([]*ast.Assignment, 0)
for _, tn := range tableList {
tableInfo := tn.TableInfo
tableVal, found := b.is.TableByID(tableInfo.ID)
if !found {
return nil, nil, false, infoschema.ErrTableNotExists.GenWithStackByArgs(tn.DBInfo.Name.O, tableInfo.Name.O)
}
for i, colInfo := range tableInfo.Columns {
if !colInfo.IsGenerated() {
continue
}
columnFullName := fmt.Sprintf("%s.%s.%s", tn.Schema.L, tn.Name.L, colInfo.Name.L)
isDefault, ok := modifyColumns[columnFullName]
if ok && colInfo.Hidden {
return nil, nil, false, ErrUnknownColumn.GenWithStackByArgs(colInfo.Name, clauseMsg[fieldList])
}
// Note: For INSERT, REPLACE, and UPDATE, if a generated column is inserted into, replaced, or updated explicitly, the only permitted value is DEFAULT.
// see https://dev.mysql.com/doc/refman/8.0/en/create-table-generated-columns.html
if ok && !isDefault {
return nil, nil, false, ErrBadGeneratedColumn.GenWithStackByArgs(colInfo.Name.O, tableInfo.Name.O)
}
virtualAssignments = append(virtualAssignments, &ast.Assignment{
Column: &ast.ColumnName{Schema: tn.Schema, Table: tn.Name, Name: colInfo.Name},
Expr: tableVal.Cols()[i].GeneratedExpr,
})
}
}
allAssignmentsAreConstant = true
newList = make([]*expression.Assignment, 0, p.Schema().Len())
tblDbMap := make(map[string]string, len(tableList))
for _, tbl := range tableList {
tblDbMap[tbl.Name.L] = tbl.DBInfo.Name.L
}
allAssignments := append(list, virtualAssignments...)
for i, assign := range allAssignments {
var idx int
var err error
if cacheColumnsIdx {
if i, ok := columnsIdx[assign.Column]; ok {
idx = i
} else {
idx, err = expression.FindFieldName(p.OutputNames(), assign.Column)
}
} else {
idx, err = expression.FindFieldName(p.OutputNames(), assign.Column)
}
if err != nil {
return nil, nil, false, err
}
col := p.Schema().Columns[idx]
name := p.OutputNames()[idx]
var newExpr expression.Expression
var np LogicalPlan
if i < len(list) {
// If assign `DEFAULT` to column, fill the `defaultExpr.Name` before rewrite expression
if expr := extractDefaultExpr(assign.Expr); expr != nil {
expr.Name = assign.Column
}
newExpr, np, err = b.rewrite(ctx, assign.Expr, p, nil, false)
} else {
// rewrite with generation expression
rewritePreprocess := func(expr ast.Node) ast.Node {
switch x := expr.(type) {
case *ast.ColumnName:
return &ast.ColumnName{
Schema: assign.Column.Schema,
Table: assign.Column.Table,
Name: x.Name,
}
default:
return expr
}
}
newExpr, np, err = b.rewriteWithPreprocess(ctx, assign.Expr, p, nil, nil, false, rewritePreprocess)
}
if err != nil {
return nil, nil, false, err
}
if _, isConst := newExpr.(*expression.Constant); !isConst {
allAssignmentsAreConstant = false
}
p = np
newList = append(newList, &expression.Assignment{Col: col, ColName: name.ColName, Expr: newExpr})
dbName := name.DBName.L
// To solve issue#10028, we need to get database name by the table alias name.
if dbNameTmp, ok := tblDbMap[name.TblName.L]; ok {
dbName = dbNameTmp
}
if dbName == "" {
dbName = b.ctx.GetSessionVars().CurrentDB
}
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.UpdatePriv, dbName, name.OrigTblName.L, "", nil)
}
return newList, p, allAssignmentsAreConstant, nil
}
// extractDefaultExpr extract a `DefaultExpr` from `ExprNode`,
// If it is a `DEFAULT` function like `DEFAULT(a)`, return nil.
// Only if it is `DEFAULT` keyword, it will return the `DefaultExpr`.
func extractDefaultExpr(node ast.ExprNode) *ast.DefaultExpr {
if expr, ok := node.(*ast.DefaultExpr); ok && expr.Name == nil {
return expr
}
return nil
}
func (b *PlanBuilder) buildDelete(ctx context.Context, delete *ast.DeleteStmt) (Plan, error) {
b.pushSelectOffset(0)
b.pushTableHints(delete.TableHints, utilhint.TypeDelete, 0)
defer func() {
b.popSelectOffset()
// table hints are only visible in the current DELETE statement.
b.popTableHints()
}()
b.inDeleteStmt = true
p, err := b.buildResultSetNode(ctx, delete.TableRefs.TableRefs)
if err != nil {
return nil, err
}
oldSchema := p.Schema()
oldLen := oldSchema.Len()
if delete.Where != nil {
p, err = b.buildSelection(ctx, p, delete.Where, nil)
if err != nil {
return nil, err
}
}
if b.ctx.GetSessionVars().TxnCtx.IsPessimistic {
if !delete.IsMultiTable {
p = b.buildSelectLock(p, ast.SelectLockForUpdate)
}
}
if delete.Order != nil {
p, err = b.buildSort(ctx, p, delete.Order.Items, nil, nil)
if err != nil {
return nil, err
}
}
if delete.Limit != nil {
p, err = b.buildLimit(p, delete.Limit)
if err != nil {
return nil, err
}
}
proj := LogicalProjection{Exprs: expression.Column2Exprs(p.Schema().Columns[:oldLen])}.Init(b.ctx, b.getSelectOffset())
proj.SetChildren(p)
proj.SetSchema(oldSchema.Clone())
proj.names = p.OutputNames()[:oldLen]
p = proj
del := Delete{
IsMultiTable: delete.IsMultiTable,
}.Init(b.ctx)
del.names = p.OutputNames()
del.SelectPlan, _, err = DoOptimize(ctx, b.ctx, b.optFlag, p)
if err != nil {
return nil, err
}
var tableList []*ast.TableName
tableList = extractTableList(delete.TableRefs.TableRefs, tableList, true)
// Collect visitInfo.
if delete.Tables != nil {
// Delete a, b from a, b, c, d... add a and b.
for _, tn := range delete.Tables.Tables {
foundMatch := false
for _, v := range tableList {
dbName := v.Schema
if dbName.L == "" {
dbName = model.NewCIStr(b.ctx.GetSessionVars().CurrentDB)
}
if (tn.Schema.L == "" || tn.Schema.L == dbName.L) && tn.Name.L == v.Name.L {
tn.Schema = dbName
tn.DBInfo = v.DBInfo
tn.TableInfo = v.TableInfo
foundMatch = true
break
}
}
if !foundMatch {
var asNameList []string
asNameList = extractTableSourceAsNames(delete.TableRefs.TableRefs, asNameList, false)
for _, asName := range asNameList {
tblName := tn.Name.L
if tn.Schema.L != "" {
tblName = tn.Schema.L + "." + tblName
}
if asName == tblName {
// check sql like: `delete a from (select * from t) as a, t`
return nil, ErrNonUpdatableTable.GenWithStackByArgs(tn.Name.O, "DELETE")
}
}
// check sql like: `delete b from (select * from t) as a, t`
return nil, ErrUnknownTable.GenWithStackByArgs(tn.Name.O, "MULTI DELETE")
}
if tn.TableInfo.IsView() {
return nil, errors.Errorf("delete view %s is not supported now.", tn.Name.O)
}
if tn.TableInfo.IsSequence() {
return nil, errors.Errorf("delete sequence %s is not supported now.", tn.Name.O)
}
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.DeletePriv, tn.Schema.L, tn.TableInfo.Name.L, "", nil)
}
} else {
// Delete from a, b, c, d.
for _, v := range tableList {
if v.TableInfo.IsView() {
return nil, errors.Errorf("delete view %s is not supported now.", v.Name.O)
}
if v.TableInfo.IsSequence() {
return nil, errors.Errorf("delete sequence %s is not supported now.", v.Name.O)
}
dbName := v.Schema.L
if dbName == "" {
dbName = b.ctx.GetSessionVars().CurrentDB
}
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.DeletePriv, dbName, v.Name.L, "", nil)
}
}
tblID2Handle, err := resolveIndicesForTblID2Handle(b.handleHelper.tailMap(), del.SelectPlan.Schema())
if err != nil {
return nil, err
}
if del.IsMultiTable {
// tblID2TableName is the table map value is an array which contains table aliases.
// Table ID may not be unique for deleting multiple tables, for statements like
// `delete from t as t1, t as t2`, the same table has two alias, we have to identify a table
// by its alias instead of ID.
tblID2TableName := make(map[int64][]*ast.TableName, len(delete.Tables.Tables))
for _, tn := range delete.Tables.Tables {
tblID2TableName[tn.TableInfo.ID] = append(tblID2TableName[tn.TableInfo.ID], tn)
}
tblID2Handle = del.cleanTblID2HandleMap(tblID2TableName, tblID2Handle, del.names)
}
tblID2table := make(map[int64]table.Table, len(tblID2Handle))
for id := range tblID2Handle {
tblID2table[id], _ = b.is.TableByID(id)
}
del.TblColPosInfos, err = buildColumns2Handle(del.names, tblID2Handle, tblID2table, false)
return del, err
}
func resolveIndicesForTblID2Handle(tblID2Handle map[int64][]*expression.Column, schema *expression.Schema) (map[int64][]*expression.Column, error) {
newMap := make(map[int64][]*expression.Column, len(tblID2Handle))
for i, cols := range tblID2Handle {
for _, col := range cols {
resolvedCol, err := col.ResolveIndices(schema)
if err != nil {
return nil, err
}
newMap[i] = append(newMap[i], resolvedCol.(*expression.Column))
}
}
return newMap, nil
}
func (p *Delete) cleanTblID2HandleMap(
tablesToDelete map[int64][]*ast.TableName,
tblID2Handle map[int64][]*expression.Column,
outputNames []*types.FieldName,
) map[int64][]*expression.Column {
for id, cols := range tblID2Handle {
names, ok := tablesToDelete[id]
if !ok {
delete(tblID2Handle, id)
continue
}
for i := len(cols) - 1; i >= 0; i-- {
if !p.matchingDeletingTable(names, outputNames[cols[i].Index]) {
cols = append(cols[:i], cols[i+1:]...)
}
}
if len(cols) == 0 {
delete(tblID2Handle, id)
continue
}
tblID2Handle[id] = cols
}
return tblID2Handle
}
// matchingDeletingTable checks whether this column is from the table which is in the deleting list.
func (p *Delete) matchingDeletingTable(names []*ast.TableName, name *types.FieldName) bool {
for _, n := range names {
if (name.DBName.L == "" || name.DBName.L == n.Schema.L) && name.TblName.L == n.Name.L {
return true
}
}
return false
}
func getWindowName(name string) string {
if name == "" {
return "<unnamed window>"
}
return name
}
// buildProjectionForWindow builds the projection for expressions in the window specification that is not an column,
// so after the projection, window functions only needs to deal with columns.
func (b *PlanBuilder) buildProjectionForWindow(ctx context.Context, p LogicalPlan, spec *ast.WindowSpec, args []ast.ExprNode, aggMap map[*ast.AggregateFuncExpr]int) (LogicalPlan, []property.Item, []property.Item, []expression.Expression, error) {
b.optFlag |= flagEliminateProjection
var partitionItems, orderItems []*ast.ByItem
if spec.PartitionBy != nil {
partitionItems = spec.PartitionBy.Items
}
if spec.OrderBy != nil {
orderItems = spec.OrderBy.Items
}
projLen := len(p.Schema().Columns) + len(partitionItems) + len(orderItems) + len(args)
proj := LogicalProjection{Exprs: make([]expression.Expression, 0, projLen)}.Init(b.ctx, b.getSelectOffset())
proj.SetSchema(expression.NewSchema(make([]*expression.Column, 0, projLen)...))
proj.names = make([]*types.FieldName, p.Schema().Len(), projLen)
for _, col := range p.Schema().Columns {
proj.Exprs = append(proj.Exprs, col)
proj.schema.Append(col)
}
copy(proj.names, p.OutputNames())
propertyItems := make([]property.Item, 0, len(partitionItems)+len(orderItems))
var err error
p, propertyItems, err = b.buildByItemsForWindow(ctx, p, proj, partitionItems, propertyItems, aggMap)
if err != nil {
return nil, nil, nil, nil, err
}
lenPartition := len(propertyItems)
p, propertyItems, err = b.buildByItemsForWindow(ctx, p, proj, orderItems, propertyItems, aggMap)
if err != nil {
return nil, nil, nil, nil, err
}
newArgList := make([]expression.Expression, 0, len(args))
for _, arg := range args {
newArg, np, err := b.rewrite(ctx, arg, p, aggMap, true)
if err != nil {
return nil, nil, nil, nil, err
}
p = np
switch newArg.(type) {
case *expression.Column, *expression.Constant:
newArgList = append(newArgList, newArg)
continue
}
proj.Exprs = append(proj.Exprs, newArg)
proj.names = append(proj.names, types.EmptyName)
col := &expression.Column{
UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(),
RetType: newArg.GetType(),
}
proj.schema.Append(col)
newArgList = append(newArgList, col)
}
proj.SetChildren(p)
return proj, propertyItems[:lenPartition], propertyItems[lenPartition:], newArgList, nil
}
func (b *PlanBuilder) buildArgs4WindowFunc(ctx context.Context, p LogicalPlan, args []ast.ExprNode, aggMap map[*ast.AggregateFuncExpr]int) ([]expression.Expression, error) {
b.optFlag |= flagEliminateProjection
newArgList := make([]expression.Expression, 0, len(args))
// use below index for created a new col definition
// it's okay here because we only want to return the args used in window function
newColIndex := 0
for _, arg := range args {
newArg, np, err := b.rewrite(ctx, arg, p, aggMap, true)
if err != nil {
return nil, err
}
p = np
switch newArg.(type) {
case *expression.Column, *expression.Constant:
newArgList = append(newArgList, newArg)
continue
}
col := &expression.Column{
UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(),
RetType: newArg.GetType(),
}
newColIndex += 1
newArgList = append(newArgList, col)
}
return newArgList, nil
}
func (b *PlanBuilder) buildByItemsForWindow(
ctx context.Context,
p LogicalPlan,
proj *LogicalProjection,
items []*ast.ByItem,
retItems []property.Item,
aggMap map[*ast.AggregateFuncExpr]int,
) (LogicalPlan, []property.Item, error) {
transformer := &itemTransformer{}
for _, item := range items {
newExpr, _ := item.Expr.Accept(transformer)
item.Expr = newExpr.(ast.ExprNode)
it, np, err := b.rewrite(ctx, item.Expr, p, aggMap, true)
if err != nil {
return nil, nil, err
}
p = np
if it.GetType().Tp == mysql.TypeNull {
continue
}
if col, ok := it.(*expression.Column); ok {
retItems = append(retItems, property.Item{Col: col, Desc: item.Desc})
continue
}
proj.Exprs = append(proj.Exprs, it)
proj.names = append(proj.names, types.EmptyName)
col := &expression.Column{
UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(),
RetType: it.GetType(),
}
proj.schema.Append(col)
retItems = append(retItems, property.Item{Col: col, Desc: item.Desc})
}
return p, retItems, nil
}
// buildWindowFunctionFrameBound builds the bounds of window function frames.
// For type `Rows`, the bound expr must be an unsigned integer.
// For type `Range`, the bound expr must be temporal or numeric types.
func (b *PlanBuilder) buildWindowFunctionFrameBound(ctx context.Context, spec *ast.WindowSpec, orderByItems []property.Item, boundClause *ast.FrameBound) (*FrameBound, error) {
frameType := spec.Frame.Type
bound := &FrameBound{Type: boundClause.Type, UnBounded: boundClause.UnBounded}
if bound.UnBounded {
return bound, nil
}
if frameType == ast.Rows {
if bound.Type == ast.CurrentRow {
return bound, nil
}
numRows, _, _ := getUintFromNode(b.ctx, boundClause.Expr)
bound.Num = numRows
return bound, nil
}
bound.CalcFuncs = make([]expression.Expression, len(orderByItems))
bound.CmpFuncs = make([]expression.CompareFunc, len(orderByItems))
if bound.Type == ast.CurrentRow {
for i, item := range orderByItems {
col := item.Col
bound.CalcFuncs[i] = col
bound.CmpFuncs[i] = expression.GetCmpFunction(b.ctx, col, col)
}
return bound, nil
}
col := orderByItems[0].Col
// TODO: We also need to raise error for non-deterministic expressions, like rand().
val, err := evalAstExpr(b.ctx, boundClause.Expr)
if err != nil {
return nil, ErrWindowRangeBoundNotConstant.GenWithStackByArgs(getWindowName(spec.Name.O))
}
expr := expression.Constant{Value: val, RetType: boundClause.Expr.GetType()}
checker := &paramMarkerInPrepareChecker{}
boundClause.Expr.Accept(checker)
// If it has paramMarker and is in prepare stmt. We don't need to eval it since its value is not decided yet.
if !checker.inPrepareStmt {
// Do not raise warnings for truncate.
oriIgnoreTruncate := b.ctx.GetSessionVars().StmtCtx.IgnoreTruncate
b.ctx.GetSessionVars().StmtCtx.IgnoreTruncate = true
uVal, isNull, err := expr.EvalInt(b.ctx, chunk.Row{})
b.ctx.GetSessionVars().StmtCtx.IgnoreTruncate = oriIgnoreTruncate
if uVal < 0 || isNull || err != nil {
return nil, ErrWindowFrameIllegal.GenWithStackByArgs(getWindowName(spec.Name.O))
}
}
desc := orderByItems[0].Desc
if boundClause.Unit != ast.TimeUnitInvalid {
// TODO: Perhaps we don't need to transcode this back to generic string
unitVal := boundClause.Unit.String()
unit := expression.Constant{
Value: types.NewStringDatum(unitVal),
RetType: types.NewFieldType(mysql.TypeVarchar),
}
// When the order is asc:
// `+` for following, and `-` for the preceding
// When the order is desc, `+` becomes `-` and vice-versa.
funcName := ast.DateAdd
if (!desc && bound.Type == ast.Preceding) || (desc && bound.Type == ast.Following) {
funcName = ast.DateSub
}
bound.CalcFuncs[0], err = expression.NewFunctionBase(b.ctx, funcName, col.RetType, col, &expr, &unit)
if err != nil {
return nil, err
}
bound.CmpFuncs[0] = expression.GetCmpFunction(b.ctx, orderByItems[0].Col, bound.CalcFuncs[0])
return bound, nil
}
// When the order is asc:
// `+` for following, and `-` for the preceding
// When the order is desc, `+` becomes `-` and vice-versa.
funcName := ast.Plus
if (!desc && bound.Type == ast.Preceding) || (desc && bound.Type == ast.Following) {
funcName = ast.Minus
}
bound.CalcFuncs[0], err = expression.NewFunctionBase(b.ctx, funcName, col.RetType, col, &expr)
if err != nil {
return nil, err
}
bound.CmpFuncs[0] = expression.GetCmpFunction(b.ctx, orderByItems[0].Col, bound.CalcFuncs[0])
return bound, nil
}
// paramMarkerInPrepareChecker checks whether the given ast tree has paramMarker and is in prepare statement.
type paramMarkerInPrepareChecker struct {
inPrepareStmt bool
}
// Enter implements Visitor Interface.
func (pc *paramMarkerInPrepareChecker) Enter(in ast.Node) (out ast.Node, skipChildren bool) {
switch v := in.(type) {
case *driver.ParamMarkerExpr:
pc.inPrepareStmt = !v.InExecute
return v, true
}
return in, false
}
// Leave implements Visitor Interface.
func (pc *paramMarkerInPrepareChecker) Leave(in ast.Node) (out ast.Node, ok bool) {
return in, true
}
// buildWindowFunctionFrame builds the window function frames.
// See https://dev.mysql.com/doc/refman/8.0/en/window-functions-frames.html
func (b *PlanBuilder) buildWindowFunctionFrame(ctx context.Context, spec *ast.WindowSpec, orderByItems []property.Item) (*WindowFrame, error) {
frameClause := spec.Frame
if frameClause == nil {
return nil, nil
}
frame := &WindowFrame{Type: frameClause.Type}
var err error
frame.Start, err = b.buildWindowFunctionFrameBound(ctx, spec, orderByItems, &frameClause.Extent.Start)
if err != nil {
return nil, err
}
frame.End, err = b.buildWindowFunctionFrameBound(ctx, spec, orderByItems, &frameClause.Extent.End)
return frame, err
}
func (b *PlanBuilder) checkWindowFuncArgs(ctx context.Context, p LogicalPlan, windowFuncExprs []*ast.WindowFuncExpr, windowAggMap map[*ast.AggregateFuncExpr]int) error {
for _, windowFuncExpr := range windowFuncExprs {
if strings.ToLower(windowFuncExpr.F) == ast.AggFuncGroupConcat {
return ErrNotSupportedYet.GenWithStackByArgs("group_concat as window function")
}
args, err := b.buildArgs4WindowFunc(ctx, p, windowFuncExpr.Args, windowAggMap)
if err != nil {
return err
}
desc, err := aggregation.NewWindowFuncDesc(b.ctx, windowFuncExpr.F, args)
if err != nil {
return err
}
if desc == nil {
return ErrWrongArguments.GenWithStackByArgs(strings.ToLower(windowFuncExpr.F))
}
}
return nil
}
func getAllByItems(itemsBuf []*ast.ByItem, spec *ast.WindowSpec) []*ast.ByItem {
itemsBuf = itemsBuf[:0]
if spec.PartitionBy != nil {
itemsBuf = append(itemsBuf, spec.PartitionBy.Items...)
}
if spec.OrderBy != nil {
itemsBuf = append(itemsBuf, spec.OrderBy.Items...)
}
return itemsBuf
}
func restoreByItemText(item *ast.ByItem) string {
var sb strings.Builder
ctx := format.NewRestoreCtx(0, &sb)
err := item.Expr.Restore(ctx)
if err != nil {
return ""
}
return sb.String()
}
func compareItems(lItems []*ast.ByItem, rItems []*ast.ByItem) bool {
minLen := mathutil.Min(len(lItems), len(rItems))
for i := 0; i < minLen; i++ {
res := strings.Compare(restoreByItemText(lItems[i]), restoreByItemText(rItems[i]))
if res != 0 {
return res < 0
}
res = compareBool(lItems[i].Desc, rItems[i].Desc)
if res != 0 {
return res < 0
}
}
return len(lItems) < len(rItems)
}
type windowFuncs struct {
spec *ast.WindowSpec
funcs []*ast.WindowFuncExpr
}
// sortWindowSpecs sorts the window specifications by reversed alphabetical order, then we could add less `Sort` operator
// in physical plan because the window functions with the same partition by and order by clause will be at near places.
func sortWindowSpecs(groupedFuncs map[*ast.WindowSpec][]*ast.WindowFuncExpr) []windowFuncs {
windows := make([]windowFuncs, 0, len(groupedFuncs))
for spec, funcs := range groupedFuncs {
windows = append(windows, windowFuncs{spec, funcs})
}
lItemsBuf := make([]*ast.ByItem, 0, 4)
rItemsBuf := make([]*ast.ByItem, 0, 4)
sort.SliceStable(windows, func(i, j int) bool {
lItemsBuf = getAllByItems(lItemsBuf, windows[i].spec)
rItemsBuf = getAllByItems(rItemsBuf, windows[j].spec)
return !compareItems(lItemsBuf, rItemsBuf)
})
return windows
}
func (b *PlanBuilder) buildWindowFunctions(ctx context.Context, p LogicalPlan, groupedFuncs map[*ast.WindowSpec][]*ast.WindowFuncExpr, aggMap map[*ast.AggregateFuncExpr]int) (LogicalPlan, map[*ast.WindowFuncExpr]int, error) {
args := make([]ast.ExprNode, 0, 4)
windowMap := make(map[*ast.WindowFuncExpr]int)
for _, window := range sortWindowSpecs(groupedFuncs) {
args = args[:0]
spec, funcs := window.spec, window.funcs
for _, windowFunc := range funcs {
args = append(args, windowFunc.Args...)
}
np, partitionBy, orderBy, args, err := b.buildProjectionForWindow(ctx, p, spec, args, aggMap)
if err != nil {
return nil, nil, err
}
err = b.checkOriginWindowSpecs(funcs, orderBy)
if err != nil {
return nil, nil, err
}
frame, err := b.buildWindowFunctionFrame(ctx, spec, orderBy)
if err != nil {
return nil, nil, err
}
window := LogicalWindow{
PartitionBy: partitionBy,
OrderBy: orderBy,
Frame: frame,
}.Init(b.ctx, b.getSelectOffset())
window.names = make([]*types.FieldName, np.Schema().Len())
copy(window.names, np.OutputNames())
schema := np.Schema().Clone()
descs := make([]*aggregation.WindowFuncDesc, 0, len(funcs))
preArgs := 0
for _, windowFunc := range funcs {
desc, err := aggregation.NewWindowFuncDesc(b.ctx, windowFunc.F, args[preArgs:preArgs+len(windowFunc.Args)])
if err != nil {
return nil, nil, err
}
if desc == nil {
return nil, nil, ErrWrongArguments.GenWithStackByArgs(strings.ToLower(windowFunc.F))
}
preArgs += len(windowFunc.Args)
desc.WrapCastForAggArgs(b.ctx)
descs = append(descs, desc)
windowMap[windowFunc] = schema.Len()
schema.Append(&expression.Column{
UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(),
RetType: desc.RetTp,
})
window.names = append(window.names, types.EmptyName)
}
window.WindowFuncDescs = descs
window.SetChildren(np)
window.SetSchema(schema)
p = window
}
return p, windowMap, nil
}
// checkOriginWindowSpecs checks the validation for origin window specifications for a group of functions.
// Because of the grouped specification is different from it, we should especially check them before build window frame.
func (b *PlanBuilder) checkOriginWindowSpecs(funcs []*ast.WindowFuncExpr, orderByItems []property.Item) error {
for _, f := range funcs {
if f.IgnoreNull {
return ErrNotSupportedYet.GenWithStackByArgs("IGNORE NULLS")
}
if f.Distinct {
return ErrNotSupportedYet.GenWithStackByArgs("<window function>(DISTINCT ..)")
}
if f.FromLast {
return ErrNotSupportedYet.GenWithStackByArgs("FROM LAST")
}
spec := &f.Spec
if f.Spec.Name.L != "" {
spec = b.windowSpecs[f.Spec.Name.L]
}
if spec.Frame == nil {
continue
}
if spec.Frame.Type == ast.Groups {
return ErrNotSupportedYet.GenWithStackByArgs("GROUPS")
}
start, end := spec.Frame.Extent.Start, spec.Frame.Extent.End
if start.Type == ast.Following && start.UnBounded {
return ErrWindowFrameStartIllegal.GenWithStackByArgs(getWindowName(spec.Name.O))
}
if end.Type == ast.Preceding && end.UnBounded {
return ErrWindowFrameEndIllegal.GenWithStackByArgs(getWindowName(spec.Name.O))
}
if start.Type == ast.Following && (end.Type == ast.Preceding || end.Type == ast.CurrentRow) {
return ErrWindowFrameIllegal.GenWithStackByArgs(getWindowName(spec.Name.O))
}
if (start.Type == ast.Following || start.Type == ast.CurrentRow) && end.Type == ast.Preceding {
return ErrWindowFrameIllegal.GenWithStackByArgs(getWindowName(spec.Name.O))
}
err := b.checkOriginWindowFrameBound(&start, spec, orderByItems)
if err != nil {
return err
}
err = b.checkOriginWindowFrameBound(&end, spec, orderByItems)
if err != nil {
return err
}
}
return nil
}
func (b *PlanBuilder) checkOriginWindowFrameBound(bound *ast.FrameBound, spec *ast.WindowSpec, orderByItems []property.Item) error {
if bound.Type == ast.CurrentRow || bound.UnBounded {
return nil
}
frameType := spec.Frame.Type
if frameType == ast.Rows {
if bound.Unit != ast.TimeUnitInvalid {
return ErrWindowRowsIntervalUse.GenWithStackByArgs(getWindowName(spec.Name.O))
}
_, isNull, isExpectedType := getUintFromNode(b.ctx, bound.Expr)
if isNull || !isExpectedType {
return ErrWindowFrameIllegal.GenWithStackByArgs(getWindowName(spec.Name.O))
}
return nil
}
if len(orderByItems) != 1 {
return ErrWindowRangeFrameOrderType.GenWithStackByArgs(getWindowName(spec.Name.O))
}
orderItemType := orderByItems[0].Col.RetType.Tp
isNumeric, isTemporal := types.IsTypeNumeric(orderItemType), types.IsTypeTemporal(orderItemType)
if !isNumeric && !isTemporal {
return ErrWindowRangeFrameOrderType.GenWithStackByArgs(getWindowName(spec.Name.O))
}
if bound.Unit != ast.TimeUnitInvalid && !isTemporal {
return ErrWindowRangeFrameNumericType.GenWithStackByArgs(getWindowName(spec.Name.O))
}
if bound.Unit == ast.TimeUnitInvalid && !isNumeric {
return ErrWindowRangeFrameTemporalType.GenWithStackByArgs(getWindowName(spec.Name.O))
}
return nil
}
func extractWindowFuncs(fields []*ast.SelectField) []*ast.WindowFuncExpr {
extractor := &WindowFuncExtractor{}
for _, f := range fields {
n, _ := f.Expr.Accept(extractor)
f.Expr = n.(ast.ExprNode)
}
return extractor.windowFuncs
}
func (b *PlanBuilder) handleDefaultFrame(spec *ast.WindowSpec, windowFuncName string) (*ast.WindowSpec, bool) {
needFrame := aggregation.NeedFrame(windowFuncName)
// According to MySQL, In the absence of a frame clause, the default frame depends on whether an ORDER BY clause is present:
// (1) With order by, the default frame is equivalent to "RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW";
// (2) Without order by, the default frame is equivalent to "RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING",
// which is the same as an empty frame.
if needFrame && spec.Frame == nil && spec.OrderBy != nil {
newSpec := *spec
newSpec.Frame = &ast.FrameClause{
Type: ast.Ranges,
Extent: ast.FrameExtent{
Start: ast.FrameBound{Type: ast.Preceding, UnBounded: true},
End: ast.FrameBound{Type: ast.CurrentRow},
},
}
return &newSpec, true
}
// For functions that operate on the entire partition, the frame clause will be ignored.
if !needFrame && spec.Frame != nil {
specName := spec.Name.O
b.ctx.GetSessionVars().StmtCtx.AppendNote(ErrWindowFunctionIgnoresFrame.GenWithStackByArgs(windowFuncName, getWindowName(specName)))
newSpec := *spec
newSpec.Frame = nil
return &newSpec, true
}
return spec, false
}
// groupWindowFuncs groups the window functions according to the window specification name.
// TODO: We can group the window function by the definition of window specification.
func (b *PlanBuilder) groupWindowFuncs(windowFuncs []*ast.WindowFuncExpr) (map[*ast.WindowSpec][]*ast.WindowFuncExpr, error) {
// updatedSpecMap is used to handle the specifications that have frame clause changed.
updatedSpecMap := make(map[string]*ast.WindowSpec)
groupedWindow := make(map[*ast.WindowSpec][]*ast.WindowFuncExpr)
for _, windowFunc := range windowFuncs {
if windowFunc.Spec.Name.L == "" {
spec := &windowFunc.Spec
if spec.Ref.L != "" {
ref, ok := b.windowSpecs[spec.Ref.L]
if !ok {
return nil, ErrWindowNoSuchWindow.GenWithStackByArgs(getWindowName(spec.Ref.O))
}
err := mergeWindowSpec(spec, ref)
if err != nil {
return nil, err
}
}
spec, _ = b.handleDefaultFrame(spec, windowFunc.F)
groupedWindow[spec] = append(groupedWindow[spec], windowFunc)
continue
}
name := windowFunc.Spec.Name.L
spec, ok := b.windowSpecs[name]
if !ok {
return nil, ErrWindowNoSuchWindow.GenWithStackByArgs(windowFunc.Spec.Name.O)
}
newSpec, updated := b.handleDefaultFrame(spec, windowFunc.F)
if !updated {
groupedWindow[spec] = append(groupedWindow[spec], windowFunc)
} else {
if _, ok := updatedSpecMap[name]; !ok {
updatedSpecMap[name] = newSpec
}
updatedSpec := updatedSpecMap[name]
groupedWindow[updatedSpec] = append(groupedWindow[updatedSpec], windowFunc)
}
}
return groupedWindow, nil
}
// resolveWindowSpec resolve window specifications for sql like `select ... from t window w1 as (w2), w2 as (partition by a)`.
// We need to resolve the referenced window to get the definition of current window spec.
func resolveWindowSpec(spec *ast.WindowSpec, specs map[string]*ast.WindowSpec, inStack map[string]bool) error {
if inStack[spec.Name.L] {
return errors.Trace(ErrWindowCircularityInWindowGraph)
}
if spec.Ref.L == "" {
return nil
}
ref, ok := specs[spec.Ref.L]
if !ok {
return ErrWindowNoSuchWindow.GenWithStackByArgs(spec.Ref.O)
}
inStack[spec.Name.L] = true
err := resolveWindowSpec(ref, specs, inStack)
if err != nil {
return err
}
inStack[spec.Name.L] = false
return mergeWindowSpec(spec, ref)
}
func mergeWindowSpec(spec, ref *ast.WindowSpec) error {
if ref.Frame != nil {
return ErrWindowNoInherentFrame.GenWithStackByArgs(ref.Name.O)
}
if spec.PartitionBy != nil {
return errors.Trace(ErrWindowNoChildPartitioning)
}
if ref.OrderBy != nil {
if spec.OrderBy != nil {
return ErrWindowNoRedefineOrderBy.GenWithStackByArgs(getWindowName(spec.Name.O), ref.Name.O)
}
spec.OrderBy = ref.OrderBy
}
spec.PartitionBy = ref.PartitionBy
spec.Ref = model.NewCIStr("")
return nil
}
func buildWindowSpecs(specs []ast.WindowSpec) (map[string]*ast.WindowSpec, error) {
specsMap := make(map[string]*ast.WindowSpec, len(specs))
for _, spec := range specs {
if _, ok := specsMap[spec.Name.L]; ok {
return nil, ErrWindowDuplicateName.GenWithStackByArgs(spec.Name.O)
}
newSpec := spec
specsMap[spec.Name.L] = &newSpec
}
inStack := make(map[string]bool, len(specs))
for _, spec := range specsMap {
err := resolveWindowSpec(spec, specsMap, inStack)
if err != nil {
return nil, err
}
}
return specsMap, nil
}
// extractTableList extracts all the TableNames from node.
// If asName is true, extract AsName prior to OrigName.
// Privilege check should use OrigName, while expression may use AsName.
func extractTableList(node ast.ResultSetNode, input []*ast.TableName, asName bool) []*ast.TableName {
switch x := node.(type) {
case *ast.Join:
input = extractTableList(x.Left, input, asName)
input = extractTableList(x.Right, input, asName)
case *ast.TableSource:
if s, ok := x.Source.(*ast.TableName); ok {
if x.AsName.L != "" && asName {
newTableName := *s
newTableName.Name = x.AsName
newTableName.Schema = model.NewCIStr("")
input = append(input, &newTableName)
} else {
input = append(input, s)
}
}
}
return input
}
// extractTableSourceAsNames extracts TableSource.AsNames from node.
// if onlySelectStmt is set to be true, only extracts AsNames when TableSource.Source.(type) == *ast.SelectStmt
func extractTableSourceAsNames(node ast.ResultSetNode, input []string, onlySelectStmt bool) []string {
switch x := node.(type) {
case *ast.Join:
input = extractTableSourceAsNames(x.Left, input, onlySelectStmt)
input = extractTableSourceAsNames(x.Right, input, onlySelectStmt)
case *ast.TableSource:
if _, ok := x.Source.(*ast.SelectStmt); !ok && onlySelectStmt {
break
}
if s, ok := x.Source.(*ast.TableName); ok {
if x.AsName.L == "" {
input = append(input, s.Name.L)
break
}
}
input = append(input, x.AsName.L)
}
return input
}
func appendVisitInfo(vi []visitInfo, priv mysql.PrivilegeType, db, tbl, col string, err error) []visitInfo {
return append(vi, visitInfo{
privilege: priv,
db: db,
table: tbl,
column: col,
err: err,
})
}
func getInnerFromParenthesesAndUnaryPlus(expr ast.ExprNode) ast.ExprNode {
if pexpr, ok := expr.(*ast.ParenthesesExpr); ok {
return getInnerFromParenthesesAndUnaryPlus(pexpr.Expr)
}
if uexpr, ok := expr.(*ast.UnaryOperationExpr); ok && uexpr.Op == opcode.Plus {
return getInnerFromParenthesesAndUnaryPlus(uexpr.V)
}
return expr
}
// containDifferentJoinTypes checks whether `preferJoinType` contains different
// join types.
func containDifferentJoinTypes(preferJoinType uint) bool {
inlMask := preferRightAsINLJInner ^ preferLeftAsINLJInner
inlhjMask := preferRightAsINLHJInner ^ preferLeftAsINLHJInner
inlmjMask := preferRightAsINLMJInner ^ preferLeftAsINLMJInner
mask := inlMask ^ inlhjMask ^ inlmjMask
onesCount := bits.OnesCount(preferJoinType & ^mask)
if onesCount > 1 || onesCount == 1 && preferJoinType&mask > 0 {
return true
}
cnt := 0
if preferJoinType&inlMask > 0 {
cnt++
}
if preferJoinType&inlhjMask > 0 {
cnt++
}
if preferJoinType&inlmjMask > 0 {
cnt++
}
return cnt > 1
}