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.
3259 lines
118 KiB
3259 lines
118 KiB
// Copyright 2015 PingCAP, Inc.
|
|
//
|
|
// Licensed under the Apache License, Version 2.0 (the "License");
|
|
// you may not use this file except in compliance with the License.
|
|
// You may obtain a copy of the License at
|
|
//
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
|
//
|
|
// Unless required by applicable law or agreed to in writing, software
|
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
// See the License for the specific language governing permissions and
|
|
// limitations under the License.
|
|
|
|
package core
|
|
|
|
import (
|
|
"bytes"
|
|
"context"
|
|
"encoding/binary"
|
|
"fmt"
|
|
"strings"
|
|
"time"
|
|
|
|
"github.com/pingcap/errors"
|
|
"github.com/pingcap/parser"
|
|
"github.com/pingcap/parser/ast"
|
|
"github.com/pingcap/parser/charset"
|
|
"github.com/pingcap/parser/model"
|
|
"github.com/pingcap/parser/mysql"
|
|
"github.com/pingcap/parser/opcode"
|
|
"github.com/pingcap/tidb/config"
|
|
"github.com/pingcap/tidb/ddl"
|
|
"github.com/pingcap/tidb/expression"
|
|
"github.com/pingcap/tidb/infoschema"
|
|
"github.com/pingcap/tidb/kv"
|
|
"github.com/pingcap/tidb/planner/property"
|
|
"github.com/pingcap/tidb/planner/util"
|
|
"github.com/pingcap/tidb/sessionctx"
|
|
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
|
"github.com/pingcap/tidb/sessionctx/variable"
|
|
"github.com/pingcap/tidb/statistics"
|
|
"github.com/pingcap/tidb/store/tikv"
|
|
"github.com/pingcap/tidb/table"
|
|
"github.com/pingcap/tidb/types"
|
|
"github.com/pingcap/tidb/types/parser_driver"
|
|
util2 "github.com/pingcap/tidb/util"
|
|
"github.com/pingcap/tidb/util/chunk"
|
|
"github.com/pingcap/tidb/util/execdetails"
|
|
"github.com/pingcap/tidb/util/hint"
|
|
"github.com/pingcap/tidb/util/logutil"
|
|
utilparser "github.com/pingcap/tidb/util/parser"
|
|
"github.com/pingcap/tidb/util/ranger"
|
|
"github.com/pingcap/tidb/util/set"
|
|
|
|
"github.com/cznic/mathutil"
|
|
"github.com/pingcap/tidb/table/tables"
|
|
"go.uber.org/zap"
|
|
)
|
|
|
|
type visitInfo struct {
|
|
privilege mysql.PrivilegeType
|
|
db string
|
|
table string
|
|
column string
|
|
err error
|
|
}
|
|
|
|
type indexNestedLoopJoinTables struct {
|
|
inljTables []hintTableInfo
|
|
inlhjTables []hintTableInfo
|
|
inlmjTables []hintTableInfo
|
|
}
|
|
|
|
type tableHintInfo struct {
|
|
indexNestedLoopJoinTables
|
|
sortMergeJoinTables []hintTableInfo
|
|
broadcastJoinTables []hintTableInfo
|
|
broadcastJoinPreferredLocal []hintTableInfo
|
|
hashJoinTables []hintTableInfo
|
|
indexHintList []indexHintInfo
|
|
tiflashTables []hintTableInfo
|
|
tikvTables []hintTableInfo
|
|
aggHints aggHintInfo
|
|
indexMergeHintList []indexHintInfo
|
|
timeRangeHint ast.HintTimeRange
|
|
limitHints limitHintInfo
|
|
}
|
|
|
|
type limitHintInfo struct {
|
|
preferLimitToCop bool
|
|
}
|
|
|
|
type hintTableInfo struct {
|
|
dbName model.CIStr
|
|
tblName model.CIStr
|
|
partitions []model.CIStr
|
|
selectOffset int
|
|
matched bool
|
|
}
|
|
|
|
type indexHintInfo struct {
|
|
dbName model.CIStr
|
|
tblName model.CIStr
|
|
partitions []model.CIStr
|
|
indexHint *ast.IndexHint
|
|
// Matched indicates whether this index hint
|
|
// has been successfully applied to a DataSource.
|
|
// If an indexHintInfo is not matched after building
|
|
// a Select statement, we will generate a warning for it.
|
|
matched bool
|
|
}
|
|
|
|
func (hint *indexHintInfo) hintTypeString() string {
|
|
switch hint.indexHint.HintType {
|
|
case ast.HintUse:
|
|
return "use_index"
|
|
case ast.HintIgnore:
|
|
return "ignore_index"
|
|
case ast.HintForce:
|
|
return "force_index"
|
|
}
|
|
return ""
|
|
}
|
|
|
|
// indexString formats the indexHint as dbName.tableName[, indexNames].
|
|
func (hint *indexHintInfo) indexString() string {
|
|
var indexListString string
|
|
indexList := make([]string, len(hint.indexHint.IndexNames))
|
|
for i := range hint.indexHint.IndexNames {
|
|
indexList[i] = hint.indexHint.IndexNames[i].L
|
|
}
|
|
if len(indexList) > 0 {
|
|
indexListString = fmt.Sprintf(", %s", strings.Join(indexList, ", "))
|
|
}
|
|
return fmt.Sprintf("%s.%s%s", hint.dbName, hint.tblName, indexListString)
|
|
}
|
|
|
|
type aggHintInfo struct {
|
|
preferAggType uint
|
|
preferAggToCop bool
|
|
}
|
|
|
|
// QueryTimeRange represents a time range specified by TIME_RANGE hint
|
|
type QueryTimeRange struct {
|
|
From time.Time
|
|
To time.Time
|
|
}
|
|
|
|
// Condition returns a WHERE clause base on it's value
|
|
func (tr *QueryTimeRange) Condition() string {
|
|
return fmt.Sprintf("where time>='%s' and time<='%s'", tr.From.Format(MetricTableTimeFormat), tr.To.Format(MetricTableTimeFormat))
|
|
}
|
|
|
|
func tableNames2HintTableInfo(ctx sessionctx.Context, hintName string, hintTables []ast.HintTable, p *hint.BlockHintProcessor, nodeType hint.NodeType, currentOffset int) []hintTableInfo {
|
|
if len(hintTables) == 0 {
|
|
return nil
|
|
}
|
|
hintTableInfos := make([]hintTableInfo, 0, len(hintTables))
|
|
defaultDBName := model.NewCIStr(ctx.GetSessionVars().CurrentDB)
|
|
isInapplicable := false
|
|
for _, hintTable := range hintTables {
|
|
tableInfo := hintTableInfo{
|
|
dbName: hintTable.DBName,
|
|
tblName: hintTable.TableName,
|
|
partitions: hintTable.PartitionList,
|
|
selectOffset: p.GetHintOffset(hintTable.QBName, nodeType, currentOffset),
|
|
}
|
|
if tableInfo.dbName.L == "" {
|
|
tableInfo.dbName = defaultDBName
|
|
}
|
|
switch hintName {
|
|
case TiDBMergeJoin, HintSMJ, TiDBIndexNestedLoopJoin, HintINLJ, HintINLHJ, HintINLMJ, TiDBHashJoin, HintHJ:
|
|
if len(tableInfo.partitions) > 0 {
|
|
isInapplicable = true
|
|
}
|
|
}
|
|
hintTableInfos = append(hintTableInfos, tableInfo)
|
|
}
|
|
if isInapplicable {
|
|
ctx.GetSessionVars().StmtCtx.AppendWarning(
|
|
errors.New(fmt.Sprintf("Optimizer Hint %s is inapplicable on specified partitions",
|
|
restore2JoinHint(hintName, hintTableInfos))))
|
|
return nil
|
|
}
|
|
return hintTableInfos
|
|
}
|
|
|
|
// ifPreferAsLocalInBCJoin checks if there is a data source specified as local read by hint
|
|
func (info *tableHintInfo) ifPreferAsLocalInBCJoin(p LogicalPlan, blockOffset int) bool {
|
|
alias := extractTableAlias(p, blockOffset)
|
|
if alias != nil {
|
|
tableNames := make([]*hintTableInfo, 1)
|
|
tableNames[0] = alias
|
|
return info.matchTableName(tableNames, info.broadcastJoinPreferredLocal)
|
|
}
|
|
for _, c := range p.Children() {
|
|
if info.ifPreferAsLocalInBCJoin(c, blockOffset) {
|
|
return true
|
|
}
|
|
}
|
|
return false
|
|
}
|
|
|
|
func (info *tableHintInfo) ifPreferMergeJoin(tableNames ...*hintTableInfo) bool {
|
|
return info.matchTableName(tableNames, info.sortMergeJoinTables)
|
|
}
|
|
|
|
func (info *tableHintInfo) ifPreferBroadcastJoin(tableNames ...*hintTableInfo) bool {
|
|
return info.matchTableName(tableNames, info.broadcastJoinTables)
|
|
}
|
|
|
|
func (info *tableHintInfo) ifPreferHashJoin(tableNames ...*hintTableInfo) bool {
|
|
return info.matchTableName(tableNames, info.hashJoinTables)
|
|
}
|
|
|
|
func (info *tableHintInfo) ifPreferINLJ(tableNames ...*hintTableInfo) bool {
|
|
return info.matchTableName(tableNames, info.indexNestedLoopJoinTables.inljTables)
|
|
}
|
|
|
|
func (info *tableHintInfo) ifPreferINLHJ(tableNames ...*hintTableInfo) bool {
|
|
return info.matchTableName(tableNames, info.indexNestedLoopJoinTables.inlhjTables)
|
|
}
|
|
|
|
func (info *tableHintInfo) ifPreferINLMJ(tableNames ...*hintTableInfo) bool {
|
|
return info.matchTableName(tableNames, info.indexNestedLoopJoinTables.inlmjTables)
|
|
}
|
|
|
|
func (info *tableHintInfo) ifPreferTiFlash(tableName *hintTableInfo) *hintTableInfo {
|
|
if tableName == nil {
|
|
return nil
|
|
}
|
|
for i, tbl := range info.tiflashTables {
|
|
if tableName.dbName.L == tbl.dbName.L && tableName.tblName.L == tbl.tblName.L && tbl.selectOffset == tableName.selectOffset {
|
|
info.tiflashTables[i].matched = true
|
|
return &tbl
|
|
}
|
|
}
|
|
return nil
|
|
}
|
|
|
|
func (info *tableHintInfo) ifPreferTiKV(tableName *hintTableInfo) *hintTableInfo {
|
|
if tableName == nil {
|
|
return nil
|
|
}
|
|
for i, tbl := range info.tikvTables {
|
|
if tableName.dbName.L == tbl.dbName.L && tableName.tblName.L == tbl.tblName.L && tbl.selectOffset == tableName.selectOffset {
|
|
info.tikvTables[i].matched = true
|
|
return &tbl
|
|
}
|
|
}
|
|
return nil
|
|
}
|
|
|
|
// matchTableName checks whether the hint hit the need.
|
|
// Only need either side matches one on the list.
|
|
// Even though you can put 2 tables on the list,
|
|
// it doesn't mean optimizer will reorder to make them
|
|
// join directly.
|
|
// Which it joins on with depend on sequence of traverse
|
|
// and without reorder, user might adjust themselves.
|
|
// This is similar to MySQL hints.
|
|
func (info *tableHintInfo) matchTableName(tables []*hintTableInfo, hintTables []hintTableInfo) bool {
|
|
hintMatched := false
|
|
for _, table := range tables {
|
|
for i, curEntry := range hintTables {
|
|
if table == nil {
|
|
continue
|
|
}
|
|
if curEntry.dbName.L == table.dbName.L && curEntry.tblName.L == table.tblName.L && table.selectOffset == curEntry.selectOffset {
|
|
hintTables[i].matched = true
|
|
hintMatched = true
|
|
break
|
|
}
|
|
}
|
|
}
|
|
return hintMatched
|
|
}
|
|
|
|
func restore2TableHint(hintTables ...hintTableInfo) string {
|
|
buffer := bytes.NewBufferString("")
|
|
for i, table := range hintTables {
|
|
buffer.WriteString(table.tblName.L)
|
|
if len(table.partitions) > 0 {
|
|
buffer.WriteString(" PARTITION(")
|
|
for j, partition := range table.partitions {
|
|
if j > 0 {
|
|
buffer.WriteString(", ")
|
|
}
|
|
buffer.WriteString(partition.L)
|
|
}
|
|
buffer.WriteString(")")
|
|
}
|
|
if i < len(hintTables)-1 {
|
|
buffer.WriteString(", ")
|
|
}
|
|
}
|
|
return buffer.String()
|
|
}
|
|
|
|
func restore2JoinHint(hintType string, hintTables []hintTableInfo) string {
|
|
buffer := bytes.NewBufferString("/*+ ")
|
|
buffer.WriteString(strings.ToUpper(hintType))
|
|
buffer.WriteString("(")
|
|
buffer.WriteString(restore2TableHint(hintTables...))
|
|
buffer.WriteString(") */")
|
|
return buffer.String()
|
|
}
|
|
|
|
func restore2IndexHint(hintType string, hintIndex indexHintInfo) string {
|
|
buffer := bytes.NewBufferString("/*+ ")
|
|
buffer.WriteString(strings.ToUpper(hintType))
|
|
buffer.WriteString("(")
|
|
buffer.WriteString(restore2TableHint(hintTableInfo{
|
|
dbName: hintIndex.dbName,
|
|
tblName: hintIndex.tblName,
|
|
partitions: hintIndex.partitions,
|
|
}))
|
|
if hintIndex.indexHint != nil && len(hintIndex.indexHint.IndexNames) > 0 {
|
|
for i, indexName := range hintIndex.indexHint.IndexNames {
|
|
if i > 0 {
|
|
buffer.WriteString(",")
|
|
}
|
|
buffer.WriteString(" " + indexName.L)
|
|
}
|
|
}
|
|
buffer.WriteString(") */")
|
|
return buffer.String()
|
|
}
|
|
|
|
func restore2StorageHint(tiflashTables, tikvTables []hintTableInfo) string {
|
|
buffer := bytes.NewBufferString("/*+ ")
|
|
buffer.WriteString(strings.ToUpper(HintReadFromStorage))
|
|
buffer.WriteString("(")
|
|
if len(tiflashTables) > 0 {
|
|
buffer.WriteString("tiflash[")
|
|
buffer.WriteString(restore2TableHint(tiflashTables...))
|
|
buffer.WriteString("]")
|
|
if len(tikvTables) > 0 {
|
|
buffer.WriteString(", ")
|
|
}
|
|
}
|
|
if len(tikvTables) > 0 {
|
|
buffer.WriteString("tikv[")
|
|
buffer.WriteString(restore2TableHint(tikvTables...))
|
|
buffer.WriteString("]")
|
|
}
|
|
buffer.WriteString(") */")
|
|
return buffer.String()
|
|
}
|
|
|
|
func extractUnmatchedTables(hintTables []hintTableInfo) []string {
|
|
var tableNames []string
|
|
for _, table := range hintTables {
|
|
if !table.matched {
|
|
tableNames = append(tableNames, table.tblName.O)
|
|
}
|
|
}
|
|
return tableNames
|
|
}
|
|
|
|
// clauseCode indicates in which clause the column is currently.
|
|
type clauseCode int
|
|
|
|
const (
|
|
unknowClause clauseCode = iota
|
|
fieldList
|
|
havingClause
|
|
onClause
|
|
orderByClause
|
|
whereClause
|
|
groupByClause
|
|
showStatement
|
|
globalOrderByClause
|
|
)
|
|
|
|
var clauseMsg = map[clauseCode]string{
|
|
unknowClause: "",
|
|
fieldList: "field list",
|
|
havingClause: "having clause",
|
|
onClause: "on clause",
|
|
orderByClause: "order clause",
|
|
whereClause: "where clause",
|
|
groupByClause: "group statement",
|
|
showStatement: "show statement",
|
|
globalOrderByClause: "global ORDER clause",
|
|
}
|
|
|
|
type capFlagType = uint64
|
|
|
|
const (
|
|
_ capFlagType = iota
|
|
// canExpandAST indicates whether the origin AST can be expanded during plan
|
|
// building. ONLY used for `CreateViewStmt` now.
|
|
canExpandAST
|
|
// renameView indicates a view is being renamed, so we cannot use the origin
|
|
// definition of that view.
|
|
renameView
|
|
)
|
|
|
|
// PlanBuilder builds Plan from an ast.Node.
|
|
// It just builds the ast node straightforwardly.
|
|
type PlanBuilder struct {
|
|
ctx sessionctx.Context
|
|
is infoschema.InfoSchema
|
|
outerSchemas []*expression.Schema
|
|
outerNames [][]*types.FieldName
|
|
// colMapper stores the column that must be pre-resolved.
|
|
colMapper map[*ast.ColumnNameExpr]int
|
|
// visitInfo is used for privilege check.
|
|
visitInfo []visitInfo
|
|
tableHintInfo []tableHintInfo
|
|
// optFlag indicates the flags of the optimizer rules.
|
|
optFlag uint64
|
|
// capFlag indicates the capability flags.
|
|
capFlag capFlagType
|
|
|
|
curClause clauseCode
|
|
|
|
// rewriterPool stores the expressionRewriter we have created to reuse it if it has been released.
|
|
// rewriterCounter counts how many rewriter is being used.
|
|
rewriterPool []*expressionRewriter
|
|
rewriterCounter int
|
|
|
|
windowSpecs map[string]*ast.WindowSpec
|
|
inUpdateStmt bool
|
|
inDeleteStmt bool
|
|
// inStraightJoin represents whether the current "SELECT" statement has
|
|
// "STRAIGHT_JOIN" option.
|
|
inStraightJoin bool
|
|
|
|
// handleHelper records the handle column position for tables. Delete/Update/SelectLock/UnionScan may need this information.
|
|
// It collects the information by the following procedure:
|
|
// Since we build the plan tree from bottom to top, we maintain a stack to record the current handle information.
|
|
// If it's a dataSource/tableDual node, we create a new map.
|
|
// If it's a aggregation, we pop the map and push a nil map since no handle information left.
|
|
// If it's a union, we pop all children's and push a nil map.
|
|
// If it's a join, we pop its children's out then merge them and push the new map to stack.
|
|
// If we meet a subquery, it's clearly that it's a independent problem so we just pop one map out when we finish building the subquery.
|
|
handleHelper *handleColHelper
|
|
|
|
hintProcessor *hint.BlockHintProcessor
|
|
// selectOffset is the offsets of current processing select stmts.
|
|
selectOffset []int
|
|
|
|
// SelectLock need this information to locate the lock on partitions.
|
|
partitionedTable []table.PartitionedTable
|
|
// buildingViewStack is used to check whether there is a recursive view.
|
|
buildingViewStack set.StringSet
|
|
// renamingViewName is the name of the view which is being renamed.
|
|
renamingViewName string
|
|
|
|
// evalDefaultExpr needs this information to find the corresponding column.
|
|
// It stores the OutputNames before buildProjection.
|
|
allNames [][]*types.FieldName
|
|
}
|
|
|
|
type handleColHelper struct {
|
|
id2HandleMapStack []map[int64][]*expression.Column
|
|
stackTail int
|
|
}
|
|
|
|
func (hch *handleColHelper) appendColToLastMap(tblID int64, col *expression.Column) {
|
|
tailMap := hch.id2HandleMapStack[hch.stackTail-1]
|
|
tailMap[tblID] = append(tailMap[tblID], col)
|
|
}
|
|
|
|
func (hch *handleColHelper) popMap() map[int64][]*expression.Column {
|
|
ret := hch.id2HandleMapStack[hch.stackTail-1]
|
|
hch.stackTail--
|
|
hch.id2HandleMapStack = hch.id2HandleMapStack[:hch.stackTail]
|
|
return ret
|
|
}
|
|
|
|
func (hch *handleColHelper) pushMap(m map[int64][]*expression.Column) {
|
|
hch.id2HandleMapStack = append(hch.id2HandleMapStack, m)
|
|
hch.stackTail++
|
|
}
|
|
|
|
func (hch *handleColHelper) mergeAndPush(m1, m2 map[int64][]*expression.Column) {
|
|
newMap := make(map[int64][]*expression.Column, mathutil.Max(len(m1), len(m2)))
|
|
for k, v := range m1 {
|
|
newMap[k] = make([]*expression.Column, len(v))
|
|
copy(newMap[k], v)
|
|
}
|
|
for k, v := range m2 {
|
|
if _, ok := newMap[k]; ok {
|
|
newMap[k] = append(newMap[k], v...)
|
|
} else {
|
|
newMap[k] = make([]*expression.Column, len(v))
|
|
copy(newMap[k], v)
|
|
}
|
|
}
|
|
hch.pushMap(newMap)
|
|
}
|
|
|
|
func (hch *handleColHelper) tailMap() map[int64][]*expression.Column {
|
|
return hch.id2HandleMapStack[hch.stackTail-1]
|
|
}
|
|
|
|
// GetVisitInfo gets the visitInfo of the PlanBuilder.
|
|
func (b *PlanBuilder) GetVisitInfo() []visitInfo {
|
|
return b.visitInfo
|
|
}
|
|
|
|
// GetDBTableInfo gets the accessed dbs and tables info.
|
|
func (b *PlanBuilder) GetDBTableInfo() []stmtctx.TableEntry {
|
|
var tables []stmtctx.TableEntry
|
|
existsFunc := func(tbls []stmtctx.TableEntry, tbl *stmtctx.TableEntry) bool {
|
|
for _, t := range tbls {
|
|
if t == *tbl {
|
|
return true
|
|
}
|
|
}
|
|
return false
|
|
}
|
|
for _, v := range b.visitInfo {
|
|
tbl := &stmtctx.TableEntry{DB: v.db, Table: v.table}
|
|
if !existsFunc(tables, tbl) {
|
|
tables = append(tables, *tbl)
|
|
}
|
|
}
|
|
return tables
|
|
}
|
|
|
|
// GetOptFlag gets the optFlag of the PlanBuilder.
|
|
func (b *PlanBuilder) GetOptFlag() uint64 {
|
|
return b.optFlag
|
|
}
|
|
|
|
func (b *PlanBuilder) getSelectOffset() int {
|
|
if len(b.selectOffset) > 0 {
|
|
return b.selectOffset[len(b.selectOffset)-1]
|
|
}
|
|
return -1
|
|
}
|
|
|
|
func (b *PlanBuilder) pushSelectOffset(offset int) {
|
|
b.selectOffset = append(b.selectOffset, offset)
|
|
}
|
|
|
|
func (b *PlanBuilder) popSelectOffset() {
|
|
b.selectOffset = b.selectOffset[:len(b.selectOffset)-1]
|
|
}
|
|
|
|
// NewPlanBuilder creates a new PlanBuilder.
|
|
func NewPlanBuilder(sctx sessionctx.Context, is infoschema.InfoSchema, processor *hint.BlockHintProcessor) *PlanBuilder {
|
|
if processor == nil {
|
|
sctx.GetSessionVars().PlannerSelectBlockAsName = nil
|
|
} else {
|
|
sctx.GetSessionVars().PlannerSelectBlockAsName = make([]ast.HintTable, processor.MaxSelectStmtOffset()+1)
|
|
}
|
|
return &PlanBuilder{
|
|
ctx: sctx,
|
|
is: is,
|
|
colMapper: make(map[*ast.ColumnNameExpr]int),
|
|
handleHelper: &handleColHelper{id2HandleMapStack: make([]map[int64][]*expression.Column, 0)},
|
|
hintProcessor: processor,
|
|
}
|
|
}
|
|
|
|
// Build builds the ast node to a Plan.
|
|
func (b *PlanBuilder) Build(ctx context.Context, node ast.Node) (Plan, error) {
|
|
b.optFlag |= flagPrunColumns
|
|
switch x := node.(type) {
|
|
case *ast.AdminStmt:
|
|
return b.buildAdmin(ctx, x)
|
|
case *ast.DeallocateStmt:
|
|
return &Deallocate{Name: x.Name}, nil
|
|
case *ast.DeleteStmt:
|
|
return b.buildDelete(ctx, x)
|
|
case *ast.ExecuteStmt:
|
|
return b.buildExecute(ctx, x)
|
|
case *ast.ExplainStmt:
|
|
return b.buildExplain(ctx, x)
|
|
case *ast.ExplainForStmt:
|
|
return b.buildExplainFor(x)
|
|
case *ast.TraceStmt:
|
|
return b.buildTrace(x)
|
|
case *ast.InsertStmt:
|
|
return b.buildInsert(ctx, x)
|
|
case *ast.LoadDataStmt:
|
|
return b.buildLoadData(ctx, x)
|
|
case *ast.LoadStatsStmt:
|
|
return b.buildLoadStats(x), nil
|
|
case *ast.IndexAdviseStmt:
|
|
return b.buildIndexAdvise(x), nil
|
|
case *ast.PrepareStmt:
|
|
return b.buildPrepare(x), nil
|
|
case *ast.SelectStmt:
|
|
if x.SelectIntoOpt != nil {
|
|
return b.buildSelectInto(ctx, x)
|
|
}
|
|
return b.buildSelect(ctx, x)
|
|
case *ast.UnionStmt:
|
|
return b.buildUnion(ctx, x)
|
|
case *ast.UpdateStmt:
|
|
return b.buildUpdate(ctx, x)
|
|
case *ast.ShowStmt:
|
|
return b.buildShow(ctx, x)
|
|
case *ast.DoStmt:
|
|
return b.buildDo(ctx, x)
|
|
case *ast.SetStmt:
|
|
return b.buildSet(ctx, x)
|
|
case *ast.SetConfigStmt:
|
|
return b.buildSetConfig(ctx, x)
|
|
case *ast.AnalyzeTableStmt:
|
|
return b.buildAnalyze(x)
|
|
case *ast.BinlogStmt, *ast.FlushStmt, *ast.UseStmt, *ast.BRIEStmt,
|
|
*ast.BeginStmt, *ast.CommitStmt, *ast.RollbackStmt, *ast.CreateUserStmt, *ast.SetPwdStmt, *ast.AlterInstanceStmt,
|
|
*ast.GrantStmt, *ast.DropUserStmt, *ast.AlterUserStmt, *ast.RevokeStmt, *ast.KillStmt, *ast.DropStatsStmt,
|
|
*ast.GrantRoleStmt, *ast.RevokeRoleStmt, *ast.SetRoleStmt, *ast.SetDefaultRoleStmt, *ast.ShutdownStmt:
|
|
return b.buildSimple(node.(ast.StmtNode))
|
|
case ast.DDLNode:
|
|
return b.buildDDL(ctx, x)
|
|
case *ast.CreateBindingStmt:
|
|
return b.buildCreateBindPlan(x)
|
|
case *ast.DropBindingStmt:
|
|
return b.buildDropBindPlan(x)
|
|
case *ast.ChangeStmt:
|
|
return b.buildChange(x)
|
|
case *ast.SplitRegionStmt:
|
|
return b.buildSplitRegion(x)
|
|
}
|
|
return nil, ErrUnsupportedType.GenWithStack("Unsupported type %T", node)
|
|
}
|
|
|
|
func (b *PlanBuilder) buildSetConfig(ctx context.Context, v *ast.SetConfigStmt) (Plan, error) {
|
|
privErr := ErrSpecificAccessDenied.GenWithStackByArgs("CONFIG")
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.ConfigPriv, "", "", "", privErr)
|
|
mockTablePlan := LogicalTableDual{}.Init(b.ctx, b.getSelectOffset())
|
|
expr, _, err := b.rewrite(ctx, v.Value, mockTablePlan, nil, true)
|
|
return &SetConfig{Name: v.Name, Type: v.Type, Instance: v.Instance, Value: expr}, err
|
|
}
|
|
|
|
func (b *PlanBuilder) buildChange(v *ast.ChangeStmt) (Plan, error) {
|
|
exe := &Change{
|
|
ChangeStmt: v,
|
|
}
|
|
return exe, nil
|
|
}
|
|
|
|
func (b *PlanBuilder) buildExecute(ctx context.Context, v *ast.ExecuteStmt) (Plan, error) {
|
|
vars := make([]expression.Expression, 0, len(v.UsingVars))
|
|
for _, expr := range v.UsingVars {
|
|
newExpr, _, err := b.rewrite(ctx, expr, nil, nil, true)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
vars = append(vars, newExpr)
|
|
}
|
|
exe := &Execute{Name: v.Name, UsingVars: vars, ExecID: v.ExecID}
|
|
if v.BinaryArgs != nil {
|
|
exe.PrepareParams = v.BinaryArgs.([]types.Datum)
|
|
}
|
|
return exe, nil
|
|
}
|
|
|
|
func (b *PlanBuilder) buildDo(ctx context.Context, v *ast.DoStmt) (Plan, error) {
|
|
var p LogicalPlan
|
|
dual := LogicalTableDual{RowCount: 1}.Init(b.ctx, b.getSelectOffset())
|
|
dual.SetSchema(expression.NewSchema())
|
|
p = dual
|
|
proj := LogicalProjection{Exprs: make([]expression.Expression, 0, len(v.Exprs))}.Init(b.ctx, b.getSelectOffset())
|
|
proj.names = make([]*types.FieldName, len(v.Exprs))
|
|
schema := expression.NewSchema(make([]*expression.Column, 0, len(v.Exprs))...)
|
|
for _, astExpr := range v.Exprs {
|
|
expr, np, err := b.rewrite(ctx, astExpr, p, nil, true)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
p = np
|
|
proj.Exprs = append(proj.Exprs, expr)
|
|
schema.Append(&expression.Column{
|
|
UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(),
|
|
RetType: expr.GetType(),
|
|
})
|
|
}
|
|
proj.SetChildren(p)
|
|
proj.self = proj
|
|
proj.SetSchema(schema)
|
|
proj.CalculateNoDelay = true
|
|
return proj, nil
|
|
}
|
|
|
|
func (b *PlanBuilder) buildSet(ctx context.Context, v *ast.SetStmt) (Plan, error) {
|
|
p := &Set{}
|
|
for _, vars := range v.Variables {
|
|
if vars.IsGlobal {
|
|
err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER")
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", err)
|
|
}
|
|
assign := &expression.VarAssignment{
|
|
Name: vars.Name,
|
|
IsGlobal: vars.IsGlobal,
|
|
IsSystem: vars.IsSystem,
|
|
}
|
|
if _, ok := vars.Value.(*ast.DefaultExpr); !ok {
|
|
if cn, ok2 := vars.Value.(*ast.ColumnNameExpr); ok2 && cn.Name.Table.L == "" {
|
|
// Convert column name expression to string value expression.
|
|
char, col := b.ctx.GetSessionVars().GetCharsetInfo()
|
|
vars.Value = ast.NewValueExpr(cn.Name.Name.O, char, col)
|
|
}
|
|
mockTablePlan := LogicalTableDual{}.Init(b.ctx, b.getSelectOffset())
|
|
var err error
|
|
assign.Expr, _, err = b.rewrite(ctx, vars.Value, mockTablePlan, nil, true)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
} else {
|
|
assign.IsDefault = true
|
|
}
|
|
if vars.ExtendValue != nil {
|
|
assign.ExtendValue = &expression.Constant{
|
|
Value: vars.ExtendValue.(*driver.ValueExpr).Datum,
|
|
RetType: &vars.ExtendValue.(*driver.ValueExpr).Type,
|
|
}
|
|
}
|
|
p.VarAssigns = append(p.VarAssigns, assign)
|
|
}
|
|
return p, nil
|
|
}
|
|
|
|
func (b *PlanBuilder) buildDropBindPlan(v *ast.DropBindingStmt) (Plan, error) {
|
|
p := &SQLBindPlan{
|
|
SQLBindOp: OpSQLBindDrop,
|
|
NormdOrigSQL: parser.Normalize(v.OriginSel.Text()),
|
|
IsGlobal: v.GlobalScope,
|
|
Db: utilparser.GetDefaultDB(v.OriginSel, b.ctx.GetSessionVars().CurrentDB),
|
|
}
|
|
if v.HintedSel != nil {
|
|
p.BindSQL = v.HintedSel.Text()
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", nil)
|
|
return p, nil
|
|
}
|
|
|
|
func (b *PlanBuilder) buildCreateBindPlan(v *ast.CreateBindingStmt) (Plan, error) {
|
|
charSet, collation := b.ctx.GetSessionVars().GetCharsetInfo()
|
|
p := &SQLBindPlan{
|
|
SQLBindOp: OpSQLBindCreate,
|
|
NormdOrigSQL: parser.Normalize(v.OriginSel.Text()),
|
|
BindSQL: v.HintedSel.Text(),
|
|
IsGlobal: v.GlobalScope,
|
|
BindStmt: v.HintedSel,
|
|
Db: utilparser.GetDefaultDB(v.OriginSel, b.ctx.GetSessionVars().CurrentDB),
|
|
Charset: charSet,
|
|
Collation: collation,
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", nil)
|
|
return p, nil
|
|
}
|
|
|
|
// detectSelectAgg detects an aggregate function or GROUP BY clause.
|
|
func (b *PlanBuilder) detectSelectAgg(sel *ast.SelectStmt) bool {
|
|
if sel.GroupBy != nil {
|
|
return true
|
|
}
|
|
for _, f := range sel.Fields.Fields {
|
|
if ast.HasAggFlag(f.Expr) {
|
|
return true
|
|
}
|
|
}
|
|
if sel.Having != nil {
|
|
if ast.HasAggFlag(sel.Having.Expr) {
|
|
return true
|
|
}
|
|
}
|
|
if sel.OrderBy != nil {
|
|
for _, item := range sel.OrderBy.Items {
|
|
if ast.HasAggFlag(item.Expr) {
|
|
return true
|
|
}
|
|
}
|
|
}
|
|
return false
|
|
}
|
|
|
|
func (b *PlanBuilder) detectSelectWindow(sel *ast.SelectStmt) bool {
|
|
for _, f := range sel.Fields.Fields {
|
|
if ast.HasWindowFlag(f.Expr) {
|
|
return true
|
|
}
|
|
}
|
|
if sel.OrderBy != nil {
|
|
for _, item := range sel.OrderBy.Items {
|
|
if ast.HasWindowFlag(item.Expr) {
|
|
return true
|
|
}
|
|
}
|
|
}
|
|
return false
|
|
}
|
|
|
|
func getPathByIndexName(paths []*util.AccessPath, idxName model.CIStr, tblInfo *model.TableInfo) *util.AccessPath {
|
|
var tablePath *util.AccessPath
|
|
for _, path := range paths {
|
|
if path.IsTablePath {
|
|
tablePath = path
|
|
continue
|
|
}
|
|
if path.Index.Name.L == idxName.L {
|
|
return path
|
|
}
|
|
}
|
|
if isPrimaryIndex(idxName) && tblInfo.PKIsHandle {
|
|
return tablePath
|
|
}
|
|
return nil
|
|
}
|
|
|
|
func isPrimaryIndex(indexName model.CIStr) bool {
|
|
return indexName.L == "primary"
|
|
}
|
|
|
|
func genTiFlashPath(tblInfo *model.TableInfo, isGlobalRead bool) *util.AccessPath {
|
|
tiFlashPath := &util.AccessPath{IsTablePath: true, StoreType: kv.TiFlash, IsTiFlashGlobalRead: isGlobalRead}
|
|
return tiFlashPath
|
|
}
|
|
|
|
func getPossibleAccessPaths(ctx sessionctx.Context, tableHints *tableHintInfo, indexHints []*ast.IndexHint, tbl table.Table, dbName, tblName model.CIStr) ([]*util.AccessPath, error) {
|
|
tblInfo := tbl.Meta()
|
|
publicPaths := make([]*util.AccessPath, 0, len(tblInfo.Indices)+2)
|
|
tp := kv.TiKV
|
|
if tbl.Type().IsClusterTable() {
|
|
tp = kv.TiDB
|
|
}
|
|
publicPaths = append(publicPaths, &util.AccessPath{IsTablePath: true, StoreType: tp})
|
|
if tblInfo.TiFlashReplica != nil && tblInfo.TiFlashReplica.Available {
|
|
publicPaths = append(publicPaths, genTiFlashPath(tblInfo, false))
|
|
publicPaths = append(publicPaths, genTiFlashPath(tblInfo, true))
|
|
}
|
|
for _, index := range tblInfo.Indices {
|
|
if index.State == model.StatePublic {
|
|
publicPaths = append(publicPaths, &util.AccessPath{Index: index})
|
|
}
|
|
}
|
|
|
|
hasScanHint, hasUseOrForce := false, false
|
|
available := make([]*util.AccessPath, 0, len(publicPaths))
|
|
ignored := make([]*util.AccessPath, 0, len(publicPaths))
|
|
|
|
// Extract comment-style index hint like /*+ INDEX(t, idx1, idx2) */.
|
|
indexHintsLen := len(indexHints)
|
|
if tableHints != nil {
|
|
for i, hint := range tableHints.indexHintList {
|
|
if hint.dbName.L == dbName.L && hint.tblName.L == tblName.L {
|
|
indexHints = append(indexHints, hint.indexHint)
|
|
tableHints.indexHintList[i].matched = true
|
|
}
|
|
}
|
|
}
|
|
|
|
_, isolationReadEnginesHasTiKV := ctx.GetSessionVars().GetIsolationReadEngines()[kv.TiKV]
|
|
for i, hint := range indexHints {
|
|
if hint.HintScope != ast.HintForScan {
|
|
continue
|
|
}
|
|
|
|
hasScanHint = true
|
|
|
|
if !isolationReadEnginesHasTiKV {
|
|
if hint.IndexNames != nil {
|
|
engineVals, _ := ctx.GetSessionVars().GetSystemVar(variable.TiDBIsolationReadEngines)
|
|
err := errors.New(fmt.Sprintf("TiDB doesn't support index in the isolation read engines(value: '%v')", engineVals))
|
|
if i < indexHintsLen {
|
|
return nil, err
|
|
}
|
|
ctx.GetSessionVars().StmtCtx.AppendWarning(err)
|
|
}
|
|
continue
|
|
}
|
|
// It is syntactically valid to omit index_list for USE INDEX, which means “use no indexes”.
|
|
// Omitting index_list for FORCE INDEX or IGNORE INDEX is a syntax error.
|
|
// See https://dev.mysql.com/doc/refman/8.0/en/index-hints.html.
|
|
if hint.IndexNames == nil && hint.HintType != ast.HintIgnore {
|
|
if path := getTablePath(publicPaths); path != nil {
|
|
hasUseOrForce = true
|
|
path.Forced = true
|
|
available = append(available, path)
|
|
}
|
|
}
|
|
for _, idxName := range hint.IndexNames {
|
|
path := getPathByIndexName(publicPaths, idxName, tblInfo)
|
|
if path == nil {
|
|
err := ErrKeyDoesNotExist.GenWithStackByArgs(idxName, tblInfo.Name)
|
|
// if hint is from comment-style sql hints, we should throw a warning instead of error.
|
|
if i < indexHintsLen {
|
|
return nil, err
|
|
}
|
|
ctx.GetSessionVars().StmtCtx.AppendWarning(err)
|
|
continue
|
|
}
|
|
if hint.HintType == ast.HintIgnore {
|
|
// Collect all the ignored index hints.
|
|
ignored = append(ignored, path)
|
|
continue
|
|
}
|
|
// Currently we don't distinguish between "FORCE" and "USE" because
|
|
// our cost estimation is not reliable.
|
|
hasUseOrForce = true
|
|
path.Forced = true
|
|
available = append(available, path)
|
|
}
|
|
}
|
|
|
|
if !hasScanHint || !hasUseOrForce {
|
|
available = publicPaths
|
|
}
|
|
|
|
available = removeIgnoredPaths(available, ignored, tblInfo)
|
|
|
|
// If we have got "FORCE" or "USE" index hint but got no available index,
|
|
// we have to use table scan.
|
|
if len(available) == 0 {
|
|
available = append(available, &util.AccessPath{IsTablePath: true})
|
|
}
|
|
return available, nil
|
|
}
|
|
|
|
func filterPathByIsolationRead(ctx sessionctx.Context, paths []*util.AccessPath, dbName model.CIStr) ([]*util.AccessPath, error) {
|
|
// TODO: filter paths with isolation read locations.
|
|
if dbName.L == mysql.SystemDB {
|
|
return paths, nil
|
|
}
|
|
isolationReadEngines := ctx.GetSessionVars().GetIsolationReadEngines()
|
|
availableEngine := map[kv.StoreType]struct{}{}
|
|
var availableEngineStr string
|
|
for i := len(paths) - 1; i >= 0; i-- {
|
|
if _, ok := availableEngine[paths[i].StoreType]; !ok {
|
|
availableEngine[paths[i].StoreType] = struct{}{}
|
|
if availableEngineStr != "" {
|
|
availableEngineStr += ", "
|
|
}
|
|
availableEngineStr += paths[i].StoreType.Name()
|
|
}
|
|
if _, ok := isolationReadEngines[paths[i].StoreType]; !ok && paths[i].StoreType != kv.TiDB {
|
|
paths = append(paths[:i], paths[i+1:]...)
|
|
}
|
|
}
|
|
var err error
|
|
if len(paths) == 0 {
|
|
engineVals, _ := ctx.GetSessionVars().GetSystemVar(variable.TiDBIsolationReadEngines)
|
|
err = ErrInternal.GenWithStackByArgs(fmt.Sprintf("Can not find access path matching '%v'(value: '%v'). Available values are '%v'.",
|
|
variable.TiDBIsolationReadEngines, engineVals, availableEngineStr))
|
|
}
|
|
return paths, err
|
|
}
|
|
|
|
func removeIgnoredPaths(paths, ignoredPaths []*util.AccessPath, tblInfo *model.TableInfo) []*util.AccessPath {
|
|
if len(ignoredPaths) == 0 {
|
|
return paths
|
|
}
|
|
remainedPaths := make([]*util.AccessPath, 0, len(paths))
|
|
for _, path := range paths {
|
|
if path.IsTablePath || getPathByIndexName(ignoredPaths, path.Index.Name, tblInfo) == nil {
|
|
remainedPaths = append(remainedPaths, path)
|
|
}
|
|
}
|
|
return remainedPaths
|
|
}
|
|
|
|
func (b *PlanBuilder) buildSelectLock(src LogicalPlan, lock ast.SelectLockType) *LogicalLock {
|
|
selectLock := LogicalLock{
|
|
Lock: lock,
|
|
tblID2Handle: b.handleHelper.tailMap(),
|
|
partitionedTable: b.partitionedTable,
|
|
}.Init(b.ctx)
|
|
selectLock.SetChildren(src)
|
|
return selectLock
|
|
}
|
|
|
|
func (b *PlanBuilder) buildPrepare(x *ast.PrepareStmt) Plan {
|
|
p := &Prepare{
|
|
Name: x.Name,
|
|
}
|
|
if x.SQLVar != nil {
|
|
if v, ok := b.ctx.GetSessionVars().Users[strings.ToLower(x.SQLVar.Name)]; ok {
|
|
var err error
|
|
p.SQLText, err = v.ToString()
|
|
if err != nil {
|
|
b.ctx.GetSessionVars().StmtCtx.AppendWarning(err)
|
|
p.SQLText = "NULL"
|
|
}
|
|
} else {
|
|
p.SQLText = "NULL"
|
|
}
|
|
} else {
|
|
p.SQLText = x.SQLText
|
|
}
|
|
return p
|
|
}
|
|
|
|
func (b *PlanBuilder) buildAdmin(ctx context.Context, as *ast.AdminStmt) (Plan, error) {
|
|
var ret Plan
|
|
var err error
|
|
switch as.Tp {
|
|
case ast.AdminCheckTable, ast.AdminCheckIndex:
|
|
ret, err = b.buildAdminCheckTable(ctx, as)
|
|
if err != nil {
|
|
return ret, err
|
|
}
|
|
case ast.AdminRecoverIndex:
|
|
p := &RecoverIndex{Table: as.Tables[0], IndexName: as.Index}
|
|
p.setSchemaAndNames(buildRecoverIndexFields())
|
|
ret = p
|
|
case ast.AdminCleanupIndex:
|
|
p := &CleanupIndex{Table: as.Tables[0], IndexName: as.Index}
|
|
p.setSchemaAndNames(buildCleanupIndexFields())
|
|
ret = p
|
|
case ast.AdminChecksumTable:
|
|
p := &ChecksumTable{Tables: as.Tables}
|
|
p.setSchemaAndNames(buildChecksumTableSchema())
|
|
ret = p
|
|
case ast.AdminShowNextRowID:
|
|
p := &ShowNextRowID{TableName: as.Tables[0]}
|
|
p.setSchemaAndNames(buildShowNextRowID())
|
|
ret = p
|
|
case ast.AdminShowDDL:
|
|
p := &ShowDDL{}
|
|
p.setSchemaAndNames(buildShowDDLFields())
|
|
ret = p
|
|
case ast.AdminShowDDLJobs:
|
|
p := LogicalShowDDLJobs{JobNumber: as.JobNumber}.Init(b.ctx)
|
|
p.setSchemaAndNames(buildShowDDLJobsFields())
|
|
for _, col := range p.schema.Columns {
|
|
col.UniqueID = b.ctx.GetSessionVars().AllocPlanColumnID()
|
|
}
|
|
ret = p
|
|
if as.Where != nil {
|
|
ret, err = b.buildSelection(ctx, p, as.Where, nil)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
}
|
|
case ast.AdminCancelDDLJobs:
|
|
p := &CancelDDLJobs{JobIDs: as.JobIDs}
|
|
p.setSchemaAndNames(buildCancelDDLJobsFields())
|
|
ret = p
|
|
case ast.AdminCheckIndexRange:
|
|
schema, names, err := b.buildCheckIndexSchema(as.Tables[0], as.Index)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
p := &CheckIndexRange{Table: as.Tables[0], IndexName: as.Index, HandleRanges: as.HandleRanges}
|
|
p.setSchemaAndNames(schema, names)
|
|
ret = p
|
|
case ast.AdminShowDDLJobQueries:
|
|
p := &ShowDDLJobQueries{JobIDs: as.JobIDs}
|
|
p.setSchemaAndNames(buildShowDDLJobQueriesFields())
|
|
ret = p
|
|
case ast.AdminShowSlow:
|
|
p := &ShowSlow{ShowSlow: as.ShowSlow}
|
|
p.setSchemaAndNames(buildShowSlowSchema())
|
|
ret = p
|
|
case ast.AdminReloadExprPushdownBlacklist:
|
|
return &ReloadExprPushdownBlacklist{}, nil
|
|
case ast.AdminReloadOptRuleBlacklist:
|
|
return &ReloadOptRuleBlacklist{}, nil
|
|
case ast.AdminPluginEnable:
|
|
return &AdminPlugins{Action: Enable, Plugins: as.Plugins}, nil
|
|
case ast.AdminPluginDisable:
|
|
return &AdminPlugins{Action: Disable, Plugins: as.Plugins}, nil
|
|
case ast.AdminFlushBindings:
|
|
return &SQLBindPlan{SQLBindOp: OpFlushBindings}, nil
|
|
case ast.AdminCaptureBindings:
|
|
return &SQLBindPlan{SQLBindOp: OpCaptureBindings}, nil
|
|
case ast.AdminEvolveBindings:
|
|
return &SQLBindPlan{SQLBindOp: OpEvolveBindings}, nil
|
|
case ast.AdminReloadBindings:
|
|
return &SQLBindPlan{SQLBindOp: OpReloadBindings}, nil
|
|
case ast.AdminShowTelemetry:
|
|
p := &AdminShowTelemetry{}
|
|
p.setSchemaAndNames(buildShowTelemetrySchema())
|
|
ret = p
|
|
case ast.AdminResetTelemetryID:
|
|
return &AdminResetTelemetryID{}, nil
|
|
default:
|
|
return nil, ErrUnsupportedType.GenWithStack("Unsupported ast.AdminStmt(%T) for buildAdmin", as)
|
|
}
|
|
|
|
// Admin command can only be executed by administrator.
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", nil)
|
|
return ret, nil
|
|
}
|
|
|
|
// getGenExprs gets generated expressions map.
|
|
func (b *PlanBuilder) getGenExprs(ctx context.Context, dbName model.CIStr, tbl table.Table, idx *model.IndexInfo, exprCols *expression.Schema, names types.NameSlice) (
|
|
map[model.TableColumnID]expression.Expression, error) {
|
|
tblInfo := tbl.Meta()
|
|
genExprsMap := make(map[model.TableColumnID]expression.Expression)
|
|
exprs := make([]expression.Expression, 0, len(tbl.Cols()))
|
|
genExprIdxs := make([]model.TableColumnID, len(tbl.Cols()))
|
|
mockTablePlan := LogicalTableDual{}.Init(b.ctx, b.getSelectOffset())
|
|
mockTablePlan.SetSchema(exprCols)
|
|
mockTablePlan.names = names
|
|
for i, colExpr := range mockTablePlan.Schema().Columns {
|
|
col := tbl.Cols()[i]
|
|
var expr expression.Expression
|
|
expr = colExpr
|
|
if col.IsGenerated() && !col.GeneratedStored {
|
|
var err error
|
|
expr, _, err = b.rewrite(ctx, col.GeneratedExpr, mockTablePlan, nil, true)
|
|
if err != nil {
|
|
return nil, errors.Trace(err)
|
|
}
|
|
found := false
|
|
for _, column := range idx.Columns {
|
|
if strings.EqualFold(col.Name.L, column.Name.L) {
|
|
found = true
|
|
break
|
|
}
|
|
}
|
|
if found {
|
|
genColumnID := model.TableColumnID{TableID: tblInfo.ID, ColumnID: col.ColumnInfo.ID}
|
|
genExprsMap[genColumnID] = expr
|
|
genExprIdxs[i] = genColumnID
|
|
}
|
|
}
|
|
exprs = append(exprs, expr)
|
|
}
|
|
// Re-iterate expressions to handle those virtual generated columns that refers to the other generated columns.
|
|
for i, expr := range exprs {
|
|
exprs[i] = expression.ColumnSubstitute(expr, mockTablePlan.Schema(), exprs)
|
|
if _, ok := genExprsMap[genExprIdxs[i]]; ok {
|
|
genExprsMap[genExprIdxs[i]] = exprs[i]
|
|
}
|
|
}
|
|
return genExprsMap, nil
|
|
}
|
|
|
|
// FindColumnInfoByID finds ColumnInfo in cols by ID.
|
|
func FindColumnInfoByID(colInfos []*model.ColumnInfo, id int64) *model.ColumnInfo {
|
|
for _, info := range colInfos {
|
|
if info.ID == id {
|
|
return info
|
|
}
|
|
}
|
|
return nil
|
|
}
|
|
|
|
func (b *PlanBuilder) buildPhysicalIndexLookUpReader(ctx context.Context, dbName model.CIStr, tbl table.Table, idx *model.IndexInfo) (Plan, error) {
|
|
pkOffset := -1
|
|
tblInfo := tbl.Meta()
|
|
schema := expression.NewSchema(make([]*expression.Column, 0, len(idx.Columns))...)
|
|
idxReaderCols := make([]*model.ColumnInfo, 0, len(idx.Columns))
|
|
tblReaderCols := make([]*model.ColumnInfo, 0, len(tbl.Cols()))
|
|
fullExprCols, _ := expression.TableInfo2SchemaAndNames(b.ctx, dbName, tblInfo)
|
|
for _, idxCol := range idx.Columns {
|
|
for i, col := range tblInfo.Columns {
|
|
if idxCol.Name.L == col.Name.L {
|
|
idxReaderCols = append(idxReaderCols, col)
|
|
tblReaderCols = append(tblReaderCols, col)
|
|
schema.Append(fullExprCols.Columns[i])
|
|
}
|
|
}
|
|
}
|
|
idxCols, idxColLens := expression.IndexInfo2PrefixCols(tblReaderCols, schema.Columns, idx)
|
|
fullIdxCols, _ := expression.IndexInfo2Cols(tblReaderCols, schema.Columns, idx)
|
|
// Add generated columns to tblSchema and tblReaderCols.
|
|
|
|
is := PhysicalIndexScan{
|
|
Table: tblInfo,
|
|
TableAsName: &tblInfo.Name,
|
|
DBName: dbName,
|
|
Columns: idxReaderCols,
|
|
Index: idx,
|
|
IdxCols: idxCols,
|
|
IdxColLens: idxColLens,
|
|
dataSourceSchema: schema,
|
|
Ranges: ranger.FullRange(),
|
|
}.Init(b.ctx, b.getSelectOffset())
|
|
// There is no alternative plan choices, so just use pseudo stats to avoid panic.
|
|
is.stats = &property.StatsInfo{HistColl: &(statistics.PseudoTable(tblInfo)).HistColl}
|
|
// It's double read case.
|
|
ts := PhysicalTableScan{Columns: tblReaderCols, Table: is.Table, TableAsName: &tblInfo.Name}.Init(b.ctx, b.getSelectOffset())
|
|
ts.SetSchema(schema.Clone())
|
|
ts.Columns = ExpandVirtualColumn(ts.Columns, ts.schema, ts.Table.Columns)
|
|
for offset, col := range ts.Columns {
|
|
if mysql.HasPriKeyFlag(col.Flag) {
|
|
pkOffset = offset
|
|
}
|
|
}
|
|
if !tbl.Meta().PKIsHandle || pkOffset == -1 {
|
|
ts.Columns = append(ts.Columns, model.NewExtraHandleColInfo())
|
|
handleCol := &expression.Column{
|
|
RetType: types.NewFieldType(mysql.TypeLonglong),
|
|
UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(),
|
|
ID: model.ExtraHandleID,
|
|
}
|
|
ts.schema.Append(handleCol)
|
|
pkOffset = len(ts.Columns) - 1
|
|
}
|
|
|
|
if tbl.Meta().GetPartitionInfo() != nil {
|
|
pid := tbl.(table.PhysicalTable).GetPhysicalID()
|
|
is.physicalTableID = pid
|
|
is.isPartition = true
|
|
ts.physicalTableID = pid
|
|
ts.isPartition = true
|
|
}
|
|
cop := &copTask{
|
|
indexPlan: is,
|
|
tablePlan: ts,
|
|
tblColHists: is.stats.HistColl,
|
|
}
|
|
ts.HandleIdx = pkOffset
|
|
is.initSchema(idx, fullIdxCols, true)
|
|
rootT := finishCopTask(b.ctx, cop).(*rootTask)
|
|
if err := rootT.p.ResolveIndices(); err != nil {
|
|
return nil, err
|
|
}
|
|
return rootT.p, nil
|
|
}
|
|
|
|
func (b *PlanBuilder) buildPhysicalIndexLookUpReaders(ctx context.Context, dbName model.CIStr, tbl table.Table, indices []table.Index) ([]Plan, []*model.IndexInfo, error) {
|
|
tblInfo := tbl.Meta()
|
|
// get index information
|
|
indexInfos := make([]*model.IndexInfo, 0, len(tblInfo.Indices))
|
|
indexLookUpReaders := make([]Plan, 0, len(tblInfo.Indices))
|
|
for _, idx := range indices {
|
|
idxInfo := idx.Meta()
|
|
if idxInfo.State != model.StatePublic {
|
|
logutil.Logger(context.Background()).Info("build physical index lookup reader, the index isn't public",
|
|
zap.String("index", idxInfo.Name.O), zap.Stringer("state", idxInfo.State), zap.String("table", tblInfo.Name.O))
|
|
continue
|
|
}
|
|
indexInfos = append(indexInfos, idxInfo)
|
|
// For partition tables.
|
|
if pi := tbl.Meta().GetPartitionInfo(); pi != nil {
|
|
for _, def := range pi.Definitions {
|
|
t := tbl.(table.PartitionedTable).GetPartition(def.ID)
|
|
reader, err := b.buildPhysicalIndexLookUpReader(ctx, dbName, t, idxInfo)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
indexLookUpReaders = append(indexLookUpReaders, reader)
|
|
}
|
|
continue
|
|
}
|
|
// For non-partition tables.
|
|
reader, err := b.buildPhysicalIndexLookUpReader(ctx, dbName, tbl, idxInfo)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
indexLookUpReaders = append(indexLookUpReaders, reader)
|
|
}
|
|
if len(indexLookUpReaders) == 0 {
|
|
return nil, nil, nil
|
|
}
|
|
return indexLookUpReaders, indexInfos, nil
|
|
}
|
|
|
|
func (b *PlanBuilder) buildAdminCheckTable(ctx context.Context, as *ast.AdminStmt) (*CheckTable, error) {
|
|
tblName := as.Tables[0]
|
|
tableInfo := as.Tables[0].TableInfo
|
|
tbl, ok := b.is.TableByID(tableInfo.ID)
|
|
if !ok {
|
|
return nil, infoschema.ErrTableNotExists.GenWithStackByArgs(tblName.DBInfo.Name.O, tableInfo.Name.O)
|
|
}
|
|
p := &CheckTable{
|
|
DBName: tblName.Schema.O,
|
|
Table: tbl,
|
|
}
|
|
var readerPlans []Plan
|
|
var indexInfos []*model.IndexInfo
|
|
var err error
|
|
if as.Tp == ast.AdminCheckIndex {
|
|
// get index information
|
|
var idx table.Index
|
|
idxName := strings.ToLower(as.Index)
|
|
for _, index := range tbl.Indices() {
|
|
if index.Meta().Name.L == idxName {
|
|
idx = index
|
|
break
|
|
}
|
|
}
|
|
if idx == nil {
|
|
return nil, errors.Errorf("index %s do not exist", as.Index)
|
|
}
|
|
if idx.Meta().State != model.StatePublic {
|
|
return nil, errors.Errorf("index %s state %s isn't public", as.Index, idx.Meta().State)
|
|
}
|
|
p.CheckIndex = true
|
|
readerPlans, indexInfos, err = b.buildPhysicalIndexLookUpReaders(ctx, tblName.Schema, tbl, []table.Index{idx})
|
|
} else {
|
|
readerPlans, indexInfos, err = b.buildPhysicalIndexLookUpReaders(ctx, tblName.Schema, tbl, tbl.Indices())
|
|
}
|
|
if err != nil {
|
|
return nil, errors.Trace(err)
|
|
}
|
|
readers := make([]*PhysicalIndexLookUpReader, 0, len(readerPlans))
|
|
for _, plan := range readerPlans {
|
|
readers = append(readers, plan.(*PhysicalIndexLookUpReader))
|
|
}
|
|
p.IndexInfos = indexInfos
|
|
p.IndexLookUpReaders = readers
|
|
return p, nil
|
|
}
|
|
|
|
func (b *PlanBuilder) buildCheckIndex(ctx context.Context, dbName model.CIStr, as *ast.AdminStmt) (Plan, error) {
|
|
tblName := as.Tables[0]
|
|
tbl, err := b.is.TableByName(dbName, tblName.Name)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
tblInfo := tbl.Meta()
|
|
|
|
// get index information
|
|
var idx *model.IndexInfo
|
|
for _, index := range tblInfo.Indices {
|
|
if index.Name.L == strings.ToLower(as.Index) {
|
|
idx = index
|
|
break
|
|
}
|
|
}
|
|
if idx == nil {
|
|
return nil, errors.Errorf("index %s do not exist", as.Index)
|
|
}
|
|
if idx.State != model.StatePublic {
|
|
return nil, errors.Errorf("index %s state %s isn't public", as.Index, idx.State)
|
|
}
|
|
|
|
return b.buildPhysicalIndexLookUpReader(ctx, dbName, tbl, idx)
|
|
}
|
|
|
|
func (b *PlanBuilder) buildCheckIndexSchema(tn *ast.TableName, indexName string) (*expression.Schema, types.NameSlice, error) {
|
|
schema := expression.NewSchema()
|
|
var names types.NameSlice
|
|
indexName = strings.ToLower(indexName)
|
|
indicesInfo := tn.TableInfo.Indices
|
|
cols := tn.TableInfo.Cols()
|
|
for _, idxInfo := range indicesInfo {
|
|
if idxInfo.Name.L != indexName {
|
|
continue
|
|
}
|
|
for _, idxCol := range idxInfo.Columns {
|
|
col := cols[idxCol.Offset]
|
|
names = append(names, &types.FieldName{
|
|
ColName: idxCol.Name,
|
|
TblName: tn.Name,
|
|
DBName: tn.Schema,
|
|
})
|
|
schema.Append(&expression.Column{
|
|
RetType: &col.FieldType,
|
|
UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(),
|
|
ID: col.ID})
|
|
}
|
|
names = append(names, &types.FieldName{
|
|
ColName: model.NewCIStr("extra_handle"),
|
|
TblName: tn.Name,
|
|
DBName: tn.Schema,
|
|
})
|
|
schema.Append(&expression.Column{
|
|
RetType: types.NewFieldType(mysql.TypeLonglong),
|
|
UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(),
|
|
ID: -1,
|
|
})
|
|
}
|
|
if schema.Len() == 0 {
|
|
return nil, nil, errors.Errorf("index %s not found", indexName)
|
|
}
|
|
return schema, names, nil
|
|
}
|
|
|
|
// getColsInfo returns the info of index columns, normal columns and primary key.
|
|
func getColsInfo(tn *ast.TableName) (indicesInfo []*model.IndexInfo, colsInfo []*model.ColumnInfo, pkCol *model.ColumnInfo) {
|
|
tbl := tn.TableInfo
|
|
for _, col := range tbl.Columns {
|
|
// The virtual column will not store any data in TiKV, so it should be ignored when collect statistics
|
|
if col.IsGenerated() && !col.GeneratedStored {
|
|
continue
|
|
}
|
|
if tbl.PKIsHandle && mysql.HasPriKeyFlag(col.Flag) {
|
|
pkCol = col
|
|
} else {
|
|
colsInfo = append(colsInfo, col)
|
|
}
|
|
}
|
|
for _, idx := range tn.TableInfo.Indices {
|
|
if idx.State == model.StatePublic {
|
|
indicesInfo = append(indicesInfo, idx)
|
|
}
|
|
}
|
|
return
|
|
}
|
|
|
|
func getPhysicalIDsAndPartitionNames(tblInfo *model.TableInfo, partitionNames []model.CIStr) ([]int64, []string, error) {
|
|
pi := tblInfo.GetPartitionInfo()
|
|
if pi == nil {
|
|
if len(partitionNames) != 0 {
|
|
return nil, nil, errors.Trace(ddl.ErrPartitionMgmtOnNonpartitioned)
|
|
}
|
|
return []int64{tblInfo.ID}, []string{""}, nil
|
|
}
|
|
if len(partitionNames) == 0 {
|
|
ids := make([]int64, 0, len(pi.Definitions))
|
|
names := make([]string, 0, len(pi.Definitions))
|
|
for _, def := range pi.Definitions {
|
|
ids = append(ids, def.ID)
|
|
names = append(names, def.Name.O)
|
|
}
|
|
return ids, names, nil
|
|
}
|
|
ids := make([]int64, 0, len(partitionNames))
|
|
names := make([]string, 0, len(partitionNames))
|
|
for _, name := range partitionNames {
|
|
found := false
|
|
for _, def := range pi.Definitions {
|
|
if def.Name.L == name.L {
|
|
found = true
|
|
ids = append(ids, def.ID)
|
|
names = append(names, def.Name.O)
|
|
break
|
|
}
|
|
}
|
|
if !found {
|
|
return nil, nil, fmt.Errorf("can not found the specified partition name %s in the table definition", name.O)
|
|
}
|
|
}
|
|
return ids, names, nil
|
|
}
|
|
|
|
func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.AnalyzeOptionType]uint64) (Plan, error) {
|
|
p := &Analyze{Opts: opts}
|
|
for _, tbl := range as.TableNames {
|
|
if tbl.TableInfo.IsView() {
|
|
return nil, errors.Errorf("analyze view %s is not supported now.", tbl.Name.O)
|
|
}
|
|
if tbl.TableInfo.IsSequence() {
|
|
return nil, errors.Errorf("analyze sequence %s is not supported now.", tbl.Name.O)
|
|
}
|
|
idxInfo, colInfo, pkInfo := getColsInfo(tbl)
|
|
physicalIDs, names, err := getPhysicalIDsAndPartitionNames(tbl.TableInfo, as.PartitionNames)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
for _, idx := range idxInfo {
|
|
for i, id := range physicalIDs {
|
|
info := analyzeInfo{DBName: tbl.Schema.O, TableName: tbl.Name.O, PartitionName: names[i], PhysicalTableID: id, Incremental: as.Incremental}
|
|
p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{
|
|
IndexInfo: idx,
|
|
analyzeInfo: info,
|
|
TblInfo: tbl.TableInfo,
|
|
})
|
|
}
|
|
}
|
|
if len(colInfo) > 0 || pkInfo != nil {
|
|
for i, id := range physicalIDs {
|
|
info := analyzeInfo{DBName: tbl.Schema.O, TableName: tbl.Name.O, PartitionName: names[i], PhysicalTableID: id, Incremental: as.Incremental}
|
|
p.ColTasks = append(p.ColTasks, AnalyzeColumnsTask{
|
|
PKInfo: pkInfo,
|
|
ColsInfo: colInfo,
|
|
analyzeInfo: info,
|
|
TblInfo: tbl.TableInfo,
|
|
})
|
|
}
|
|
}
|
|
}
|
|
return p, nil
|
|
}
|
|
|
|
func (b *PlanBuilder) buildAnalyzeIndex(as *ast.AnalyzeTableStmt, opts map[ast.AnalyzeOptionType]uint64) (Plan, error) {
|
|
p := &Analyze{Opts: opts}
|
|
tblInfo := as.TableNames[0].TableInfo
|
|
physicalIDs, names, err := getPhysicalIDsAndPartitionNames(tblInfo, as.PartitionNames)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
for _, idxName := range as.IndexNames {
|
|
if isPrimaryIndex(idxName) && tblInfo.PKIsHandle {
|
|
pkCol := tblInfo.GetPkColInfo()
|
|
for i, id := range physicalIDs {
|
|
info := analyzeInfo{DBName: as.TableNames[0].Schema.O, TableName: as.TableNames[0].Name.O, PartitionName: names[i], PhysicalTableID: id, Incremental: as.Incremental}
|
|
p.ColTasks = append(p.ColTasks, AnalyzeColumnsTask{PKInfo: pkCol, analyzeInfo: info, TblInfo: tblInfo})
|
|
}
|
|
continue
|
|
}
|
|
idx := tblInfo.FindIndexByName(idxName.L)
|
|
if idx == nil || idx.State != model.StatePublic {
|
|
return nil, ErrAnalyzeMissIndex.GenWithStackByArgs(idxName.O, tblInfo.Name.O)
|
|
}
|
|
for i, id := range physicalIDs {
|
|
info := analyzeInfo{DBName: as.TableNames[0].Schema.O, TableName: as.TableNames[0].Name.O, PartitionName: names[i], PhysicalTableID: id, Incremental: as.Incremental}
|
|
p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{IndexInfo: idx, analyzeInfo: info, TblInfo: tblInfo})
|
|
}
|
|
}
|
|
return p, nil
|
|
}
|
|
|
|
func (b *PlanBuilder) buildAnalyzeAllIndex(as *ast.AnalyzeTableStmt, opts map[ast.AnalyzeOptionType]uint64) (Plan, error) {
|
|
p := &Analyze{Opts: opts}
|
|
tblInfo := as.TableNames[0].TableInfo
|
|
physicalIDs, names, err := getPhysicalIDsAndPartitionNames(tblInfo, as.PartitionNames)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
for _, idx := range tblInfo.Indices {
|
|
if idx.State == model.StatePublic {
|
|
for i, id := range physicalIDs {
|
|
info := analyzeInfo{DBName: as.TableNames[0].Schema.O, TableName: as.TableNames[0].Name.O, PartitionName: names[i], PhysicalTableID: id, Incremental: as.Incremental}
|
|
p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{IndexInfo: idx, analyzeInfo: info, TblInfo: tblInfo})
|
|
}
|
|
}
|
|
}
|
|
if tblInfo.PKIsHandle {
|
|
pkCol := tblInfo.GetPkColInfo()
|
|
for i, id := range physicalIDs {
|
|
info := analyzeInfo{DBName: as.TableNames[0].Schema.O, TableName: as.TableNames[0].Name.O, PartitionName: names[i], PhysicalTableID: id, Incremental: as.Incremental}
|
|
p.ColTasks = append(p.ColTasks, AnalyzeColumnsTask{PKInfo: pkCol, analyzeInfo: info})
|
|
}
|
|
}
|
|
return p, nil
|
|
}
|
|
|
|
// CMSketchSizeLimit indicates the max size(width * depth) of a CMSketch.
|
|
var CMSketchSizeLimit = kv.TxnEntrySizeLimit / binary.MaxVarintLen32
|
|
|
|
var analyzeOptionLimit = map[ast.AnalyzeOptionType]uint64{
|
|
ast.AnalyzeOptNumBuckets: 1024,
|
|
ast.AnalyzeOptNumTopN: 1024,
|
|
ast.AnalyzeOptCMSketchWidth: uint64(CMSketchSizeLimit),
|
|
ast.AnalyzeOptCMSketchDepth: uint64(CMSketchSizeLimit),
|
|
ast.AnalyzeOptNumSamples: 100000,
|
|
}
|
|
|
|
var analyzeOptionDefault = map[ast.AnalyzeOptionType]uint64{
|
|
ast.AnalyzeOptNumBuckets: 256,
|
|
ast.AnalyzeOptNumTopN: 20,
|
|
ast.AnalyzeOptCMSketchWidth: 2048,
|
|
ast.AnalyzeOptCMSketchDepth: 5,
|
|
ast.AnalyzeOptNumSamples: 10000,
|
|
}
|
|
|
|
func handleAnalyzeOptions(opts []ast.AnalyzeOpt) (map[ast.AnalyzeOptionType]uint64, error) {
|
|
optMap := make(map[ast.AnalyzeOptionType]uint64, len(analyzeOptionDefault))
|
|
for key, val := range analyzeOptionDefault {
|
|
optMap[key] = val
|
|
}
|
|
for _, opt := range opts {
|
|
if opt.Type == ast.AnalyzeOptNumTopN {
|
|
if opt.Value > analyzeOptionLimit[opt.Type] {
|
|
return nil, errors.Errorf("value of analyze option %s should not larger than %d", ast.AnalyzeOptionString[opt.Type], analyzeOptionLimit[opt.Type])
|
|
}
|
|
} else {
|
|
if opt.Value == 0 || opt.Value > analyzeOptionLimit[opt.Type] {
|
|
return nil, errors.Errorf("value of analyze option %s should be positive and not larger than %d", ast.AnalyzeOptionString[opt.Type], analyzeOptionLimit[opt.Type])
|
|
}
|
|
}
|
|
optMap[opt.Type] = opt.Value
|
|
}
|
|
if optMap[ast.AnalyzeOptCMSketchWidth]*optMap[ast.AnalyzeOptCMSketchDepth] > uint64(CMSketchSizeLimit) {
|
|
return nil, errors.Errorf("cm sketch size(depth * width) should not larger than %d", CMSketchSizeLimit)
|
|
}
|
|
return optMap, nil
|
|
}
|
|
|
|
func (b *PlanBuilder) buildAnalyze(as *ast.AnalyzeTableStmt) (Plan, error) {
|
|
// If enable fast analyze, the storage must be tikv.Storage.
|
|
if _, isTikvStorage := b.ctx.GetStore().(tikv.Storage); !isTikvStorage && b.ctx.GetSessionVars().EnableFastAnalyze {
|
|
return nil, errors.Errorf("Only support fast analyze in tikv storage.")
|
|
}
|
|
for _, tbl := range as.TableNames {
|
|
user := b.ctx.GetSessionVars().User
|
|
var insertErr, selectErr error
|
|
if user != nil {
|
|
insertErr = ErrTableaccessDenied.GenWithStackByArgs("INSERT", user.AuthUsername, user.AuthHostname, tbl.Name.O)
|
|
selectErr = ErrTableaccessDenied.GenWithStackByArgs("SELECT", user.AuthUsername, user.AuthHostname, tbl.Name.O)
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.InsertPriv, tbl.Schema.O, tbl.Name.O, "", insertErr)
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SelectPriv, tbl.Schema.O, tbl.Name.O, "", selectErr)
|
|
}
|
|
opts, err := handleAnalyzeOptions(as.AnalyzeOpts)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
if as.IndexFlag {
|
|
if len(as.IndexNames) == 0 {
|
|
return b.buildAnalyzeAllIndex(as, opts)
|
|
}
|
|
return b.buildAnalyzeIndex(as, opts)
|
|
}
|
|
return b.buildAnalyzeTable(as, opts)
|
|
}
|
|
|
|
func buildShowNextRowID() (*expression.Schema, types.NameSlice) {
|
|
schema := newColumnsWithNames(4)
|
|
schema.Append(buildColumnWithName("", "DB_NAME", mysql.TypeVarchar, mysql.MaxDatabaseNameLength))
|
|
schema.Append(buildColumnWithName("", "TABLE_NAME", mysql.TypeVarchar, mysql.MaxTableNameLength))
|
|
schema.Append(buildColumnWithName("", "COLUMN_NAME", mysql.TypeVarchar, mysql.MaxColumnNameLength))
|
|
schema.Append(buildColumnWithName("", "NEXT_GLOBAL_ROW_ID", mysql.TypeLonglong, 4))
|
|
schema.Append(buildColumnWithName("", "ID_TYPE", mysql.TypeVarchar, 15))
|
|
return schema.col2Schema(), schema.names
|
|
}
|
|
|
|
func buildShowDDLFields() (*expression.Schema, types.NameSlice) {
|
|
schema := newColumnsWithNames(6)
|
|
schema.Append(buildColumnWithName("", "SCHEMA_VER", mysql.TypeLonglong, 4))
|
|
schema.Append(buildColumnWithName("", "OWNER_ID", mysql.TypeVarchar, 64))
|
|
schema.Append(buildColumnWithName("", "OWNER_ADDRESS", mysql.TypeVarchar, 32))
|
|
schema.Append(buildColumnWithName("", "RUNNING_JOBS", mysql.TypeVarchar, 256))
|
|
schema.Append(buildColumnWithName("", "SELF_ID", mysql.TypeVarchar, 64))
|
|
schema.Append(buildColumnWithName("", "QUERY", mysql.TypeVarchar, 256))
|
|
|
|
return schema.col2Schema(), schema.names
|
|
}
|
|
|
|
func buildRecoverIndexFields() (*expression.Schema, types.NameSlice) {
|
|
schema := newColumnsWithNames(2)
|
|
schema.Append(buildColumnWithName("", "ADDED_COUNT", mysql.TypeLonglong, 4))
|
|
schema.Append(buildColumnWithName("", "SCAN_COUNT", mysql.TypeLonglong, 4))
|
|
return schema.col2Schema(), schema.names
|
|
}
|
|
|
|
func buildCleanupIndexFields() (*expression.Schema, types.NameSlice) {
|
|
schema := newColumnsWithNames(1)
|
|
schema.Append(buildColumnWithName("", "REMOVED_COUNT", mysql.TypeLonglong, 4))
|
|
return schema.col2Schema(), schema.names
|
|
}
|
|
|
|
func buildShowDDLJobsFields() (*expression.Schema, types.NameSlice) {
|
|
schema := newColumnsWithNames(11)
|
|
schema.Append(buildColumnWithName("", "JOB_ID", mysql.TypeLonglong, 4))
|
|
schema.Append(buildColumnWithName("", "DB_NAME", mysql.TypeVarchar, 64))
|
|
schema.Append(buildColumnWithName("", "TABLE_NAME", mysql.TypeVarchar, 64))
|
|
schema.Append(buildColumnWithName("", "JOB_TYPE", mysql.TypeVarchar, 64))
|
|
schema.Append(buildColumnWithName("", "SCHEMA_STATE", mysql.TypeVarchar, 64))
|
|
schema.Append(buildColumnWithName("", "SCHEMA_ID", mysql.TypeLonglong, 4))
|
|
schema.Append(buildColumnWithName("", "TABLE_ID", mysql.TypeLonglong, 4))
|
|
schema.Append(buildColumnWithName("", "ROW_COUNT", mysql.TypeLonglong, 4))
|
|
schema.Append(buildColumnWithName("", "START_TIME", mysql.TypeDatetime, 19))
|
|
schema.Append(buildColumnWithName("", "END_TIME", mysql.TypeDatetime, 19))
|
|
schema.Append(buildColumnWithName("", "STATE", mysql.TypeVarchar, 64))
|
|
return schema.col2Schema(), schema.names
|
|
}
|
|
|
|
func buildTableRegionsSchema() (*expression.Schema, types.NameSlice) {
|
|
schema := newColumnsWithNames(11)
|
|
schema.Append(buildColumnWithName("", "REGION_ID", mysql.TypeLonglong, 4))
|
|
schema.Append(buildColumnWithName("", "START_KEY", mysql.TypeVarchar, 64))
|
|
schema.Append(buildColumnWithName("", "END_KEY", mysql.TypeVarchar, 64))
|
|
schema.Append(buildColumnWithName("", "LEADER_ID", mysql.TypeLonglong, 4))
|
|
schema.Append(buildColumnWithName("", "LEADER_STORE_ID", mysql.TypeLonglong, 4))
|
|
schema.Append(buildColumnWithName("", "PEERS", mysql.TypeVarchar, 64))
|
|
schema.Append(buildColumnWithName("", "SCATTERING", mysql.TypeTiny, 1))
|
|
schema.Append(buildColumnWithName("", "WRITTEN_BYTES", mysql.TypeLonglong, 4))
|
|
schema.Append(buildColumnWithName("", "READ_BYTES", mysql.TypeLonglong, 4))
|
|
schema.Append(buildColumnWithName("", "APPROXIMATE_SIZE(MB)", mysql.TypeLonglong, 4))
|
|
schema.Append(buildColumnWithName("", "APPROXIMATE_KEYS", mysql.TypeLonglong, 4))
|
|
return schema.col2Schema(), schema.names
|
|
}
|
|
|
|
func buildSplitRegionsSchema() (*expression.Schema, types.NameSlice) {
|
|
schema := newColumnsWithNames(2)
|
|
schema.Append(buildColumnWithName("", "TOTAL_SPLIT_REGION", mysql.TypeLonglong, 4))
|
|
schema.Append(buildColumnWithName("", "SCATTER_FINISH_RATIO", mysql.TypeDouble, 8))
|
|
return schema.col2Schema(), schema.names
|
|
}
|
|
|
|
func buildShowDDLJobQueriesFields() (*expression.Schema, types.NameSlice) {
|
|
schema := newColumnsWithNames(1)
|
|
schema.Append(buildColumnWithName("", "QUERY", mysql.TypeVarchar, 256))
|
|
return schema.col2Schema(), schema.names
|
|
}
|
|
|
|
func buildShowSlowSchema() (*expression.Schema, types.NameSlice) {
|
|
longlongSize, _ := mysql.GetDefaultFieldLengthAndDecimal(mysql.TypeLonglong)
|
|
tinySize, _ := mysql.GetDefaultFieldLengthAndDecimal(mysql.TypeTiny)
|
|
timestampSize, _ := mysql.GetDefaultFieldLengthAndDecimal(mysql.TypeTimestamp)
|
|
durationSize, _ := mysql.GetDefaultFieldLengthAndDecimal(mysql.TypeDuration)
|
|
|
|
schema := newColumnsWithNames(11)
|
|
schema.Append(buildColumnWithName("", "SQL", mysql.TypeVarchar, 4096))
|
|
schema.Append(buildColumnWithName("", "START", mysql.TypeTimestamp, timestampSize))
|
|
schema.Append(buildColumnWithName("", "DURATION", mysql.TypeDuration, durationSize))
|
|
schema.Append(buildColumnWithName("", "DETAILS", mysql.TypeVarchar, 256))
|
|
schema.Append(buildColumnWithName("", "SUCC", mysql.TypeTiny, tinySize))
|
|
schema.Append(buildColumnWithName("", "CONN_ID", mysql.TypeLonglong, longlongSize))
|
|
schema.Append(buildColumnWithName("", "TRANSACTION_TS", mysql.TypeLonglong, longlongSize))
|
|
schema.Append(buildColumnWithName("", "USER", mysql.TypeVarchar, 32))
|
|
schema.Append(buildColumnWithName("", "DB", mysql.TypeVarchar, 64))
|
|
schema.Append(buildColumnWithName("", "TABLE_IDS", mysql.TypeVarchar, 256))
|
|
schema.Append(buildColumnWithName("", "INDEX_IDS", mysql.TypeVarchar, 256))
|
|
schema.Append(buildColumnWithName("", "INTERNAL", mysql.TypeTiny, tinySize))
|
|
schema.Append(buildColumnWithName("", "DIGEST", mysql.TypeVarchar, 64))
|
|
return schema.col2Schema(), schema.names
|
|
}
|
|
|
|
func buildCancelDDLJobsFields() (*expression.Schema, types.NameSlice) {
|
|
schema := newColumnsWithNames(2)
|
|
schema.Append(buildColumnWithName("", "JOB_ID", mysql.TypeVarchar, 64))
|
|
schema.Append(buildColumnWithName("", "RESULT", mysql.TypeVarchar, 128))
|
|
|
|
return schema.col2Schema(), schema.names
|
|
}
|
|
|
|
func buildBRIESchema() (*expression.Schema, types.NameSlice) {
|
|
longlongSize, _ := mysql.GetDefaultFieldLengthAndDecimal(mysql.TypeLonglong)
|
|
datetimeSize, _ := mysql.GetDefaultFieldLengthAndDecimal(mysql.TypeDatetime)
|
|
|
|
schema := newColumnsWithNames(5)
|
|
schema.Append(buildColumnWithName("", "Destination", mysql.TypeVarchar, 255))
|
|
schema.Append(buildColumnWithName("", "Size", mysql.TypeLonglong, longlongSize))
|
|
schema.Append(buildColumnWithName("", "BackupTS", mysql.TypeLonglong, longlongSize))
|
|
schema.Append(buildColumnWithName("", "Queue Time", mysql.TypeDatetime, datetimeSize))
|
|
schema.Append(buildColumnWithName("", "Execution Time", mysql.TypeDatetime, datetimeSize))
|
|
return schema.col2Schema(), schema.names
|
|
}
|
|
|
|
func buildShowTelemetrySchema() (*expression.Schema, types.NameSlice) {
|
|
schema := newColumnsWithNames(1)
|
|
schema.Append(buildColumnWithName("", "TRACKING_ID", mysql.TypeVarchar, 64))
|
|
schema.Append(buildColumnWithName("", "LAST_STATUS", mysql.TypeString, mysql.MaxBlobWidth))
|
|
schema.Append(buildColumnWithName("", "DATA_PREVIEW", mysql.TypeString, mysql.MaxBlobWidth))
|
|
return schema.col2Schema(), schema.names
|
|
}
|
|
|
|
func buildColumnWithName(tableName, name string, tp byte, size int) (*expression.Column, *types.FieldName) {
|
|
cs, cl := types.DefaultCharsetForType(tp)
|
|
flag := mysql.UnsignedFlag
|
|
if tp == mysql.TypeVarchar || tp == mysql.TypeBlob {
|
|
cs = charset.CharsetUTF8MB4
|
|
cl = charset.CollationUTF8MB4
|
|
flag = 0
|
|
}
|
|
|
|
fieldType := &types.FieldType{
|
|
Charset: cs,
|
|
Collate: cl,
|
|
Tp: tp,
|
|
Flen: size,
|
|
Flag: flag,
|
|
}
|
|
return &expression.Column{
|
|
RetType: fieldType,
|
|
}, &types.FieldName{DBName: util2.InformationSchemaName, TblName: model.NewCIStr(tableName), ColName: model.NewCIStr(name)}
|
|
}
|
|
|
|
type columnsWithNames struct {
|
|
cols []*expression.Column
|
|
names types.NameSlice
|
|
}
|
|
|
|
func newColumnsWithNames(cap int) *columnsWithNames {
|
|
return &columnsWithNames{
|
|
cols: make([]*expression.Column, 0, 2),
|
|
names: make(types.NameSlice, 0, 2),
|
|
}
|
|
}
|
|
|
|
func (cwn *columnsWithNames) Append(col *expression.Column, name *types.FieldName) {
|
|
cwn.cols = append(cwn.cols, col)
|
|
cwn.names = append(cwn.names, name)
|
|
}
|
|
|
|
func (cwn *columnsWithNames) col2Schema() *expression.Schema {
|
|
return expression.NewSchema(cwn.cols...)
|
|
}
|
|
|
|
// splitWhere split a where expression to a list of AND conditions.
|
|
func splitWhere(where ast.ExprNode) []ast.ExprNode {
|
|
var conditions []ast.ExprNode
|
|
switch x := where.(type) {
|
|
case nil:
|
|
case *ast.BinaryOperationExpr:
|
|
if x.Op == opcode.LogicAnd {
|
|
conditions = append(conditions, splitWhere(x.L)...)
|
|
conditions = append(conditions, splitWhere(x.R)...)
|
|
} else {
|
|
conditions = append(conditions, x)
|
|
}
|
|
case *ast.ParenthesesExpr:
|
|
conditions = append(conditions, splitWhere(x.Expr)...)
|
|
default:
|
|
conditions = append(conditions, where)
|
|
}
|
|
return conditions
|
|
}
|
|
|
|
func (b *PlanBuilder) buildShow(ctx context.Context, show *ast.ShowStmt) (Plan, error) {
|
|
p := LogicalShow{
|
|
ShowContents: ShowContents{
|
|
Tp: show.Tp,
|
|
DBName: show.DBName,
|
|
Table: show.Table,
|
|
Column: show.Column,
|
|
IndexName: show.IndexName,
|
|
Flag: show.Flag,
|
|
User: show.User,
|
|
Roles: show.Roles,
|
|
Full: show.Full,
|
|
IfNotExists: show.IfNotExists,
|
|
GlobalScope: show.GlobalScope,
|
|
Extended: show.Extended,
|
|
},
|
|
}.Init(b.ctx)
|
|
isView := false
|
|
isSequence := false
|
|
switch show.Tp {
|
|
case ast.ShowTables, ast.ShowTableStatus:
|
|
if p.DBName == "" {
|
|
return nil, ErrNoDB
|
|
}
|
|
case ast.ShowCreateTable, ast.ShowCreateSequence:
|
|
user := b.ctx.GetSessionVars().User
|
|
var err error
|
|
if user != nil {
|
|
err = ErrTableaccessDenied.GenWithStackByArgs("SHOW", user.AuthUsername, user.AuthHostname, show.Table.Name.L)
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.AllPrivMask, show.Table.Schema.L, show.Table.Name.L, "", err)
|
|
if table, err := b.is.TableByName(show.Table.Schema, show.Table.Name); err == nil {
|
|
isView = table.Meta().IsView()
|
|
isSequence = table.Meta().IsSequence()
|
|
}
|
|
case ast.ShowCreateView:
|
|
err := ErrSpecificAccessDenied.GenWithStackByArgs("SHOW VIEW")
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.ShowViewPriv, show.Table.Schema.L, show.Table.Name.L, "", err)
|
|
case ast.ShowBackups, ast.ShowRestores:
|
|
err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER")
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", err)
|
|
case ast.ShowTableNextRowId:
|
|
p := &ShowNextRowID{TableName: show.Table}
|
|
p.setSchemaAndNames(buildShowNextRowID())
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SelectPriv, show.Table.Schema.L, show.Table.Name.L, "", ErrPrivilegeCheckFail)
|
|
return p, nil
|
|
case ast.ShowStatsBuckets, ast.ShowStatsHistograms, ast.ShowStatsMeta, ast.ShowStatsHealthy:
|
|
user := b.ctx.GetSessionVars().User
|
|
var err error
|
|
if user != nil {
|
|
err = ErrDBaccessDenied.GenWithStackByArgs(user.AuthUsername, user.AuthHostname, mysql.SystemDB)
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SelectPriv, mysql.SystemDB, "", "", err)
|
|
}
|
|
schema, names := buildShowSchema(show, isView, isSequence)
|
|
p.SetSchema(schema)
|
|
p.names = names
|
|
for _, col := range p.schema.Columns {
|
|
col.UniqueID = b.ctx.GetSessionVars().AllocPlanColumnID()
|
|
}
|
|
var err error
|
|
var np LogicalPlan
|
|
np = p
|
|
if show.Pattern != nil {
|
|
show.Pattern.Expr = &ast.ColumnNameExpr{
|
|
Name: &ast.ColumnName{Name: p.OutputNames()[0].ColName},
|
|
}
|
|
np, err = b.buildSelection(ctx, np, show.Pattern, nil)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
}
|
|
if show.Where != nil {
|
|
np, err = b.buildSelection(ctx, np, show.Where, nil)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
}
|
|
if np != p {
|
|
b.optFlag |= flagEliminateProjection
|
|
fieldsLen := len(p.schema.Columns)
|
|
proj := LogicalProjection{Exprs: make([]expression.Expression, 0, fieldsLen)}.Init(b.ctx, 0)
|
|
schema := expression.NewSchema(make([]*expression.Column, 0, fieldsLen)...)
|
|
for _, col := range p.schema.Columns {
|
|
proj.Exprs = append(proj.Exprs, col)
|
|
newCol := col.Clone().(*expression.Column)
|
|
newCol.UniqueID = b.ctx.GetSessionVars().AllocPlanColumnID()
|
|
schema.Append(newCol)
|
|
}
|
|
proj.SetSchema(schema)
|
|
proj.SetChildren(np)
|
|
proj.SetOutputNames(np.OutputNames())
|
|
np = proj
|
|
}
|
|
if show.Tp == ast.ShowVariables || show.Tp == ast.ShowStatus {
|
|
b.curClause = orderByClause
|
|
orderByCol := np.Schema().Columns[0].Clone().(*expression.Column)
|
|
sort := LogicalSort{
|
|
ByItems: []*util.ByItems{{Expr: orderByCol}},
|
|
}.Init(b.ctx, b.getSelectOffset())
|
|
sort.SetChildren(np)
|
|
np = sort
|
|
}
|
|
return np, nil
|
|
}
|
|
|
|
func (b *PlanBuilder) buildSimple(node ast.StmtNode) (Plan, error) {
|
|
p := &Simple{Statement: node}
|
|
|
|
switch raw := node.(type) {
|
|
case *ast.FlushStmt:
|
|
err := ErrSpecificAccessDenied.GenWithStackByArgs("RELOAD")
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.ReloadPriv, "", "", "", err)
|
|
case *ast.AlterInstanceStmt:
|
|
err := ErrSpecificAccessDenied.GenWithStack("ALTER INSTANCE")
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", err)
|
|
case *ast.AlterUserStmt:
|
|
err := ErrSpecificAccessDenied.GenWithStackByArgs("CREATE USER")
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.CreateUserPriv, "", "", "", err)
|
|
case *ast.GrantStmt:
|
|
if b.ctx.GetSessionVars().CurrentDB == "" && raw.Level.DBName == "" {
|
|
if raw.Level.Level == ast.GrantLevelTable {
|
|
return nil, ErrNoDB
|
|
}
|
|
}
|
|
b.visitInfo = collectVisitInfoFromGrantStmt(b.ctx, b.visitInfo, raw)
|
|
case *ast.BRIEStmt:
|
|
p.setSchemaAndNames(buildBRIESchema())
|
|
err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER")
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", err)
|
|
case *ast.GrantRoleStmt, *ast.RevokeRoleStmt:
|
|
err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER")
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", err)
|
|
case *ast.RevokeStmt:
|
|
b.visitInfo = collectVisitInfoFromRevokeStmt(b.ctx, b.visitInfo, raw)
|
|
case *ast.KillStmt:
|
|
// If you have the SUPER privilege, you can kill all threads and statements.
|
|
// Otherwise, you can kill only your own threads and statements.
|
|
sm := b.ctx.GetSessionManager()
|
|
if sm != nil {
|
|
if pi, ok := sm.GetProcessInfo(raw.ConnectionID); ok {
|
|
loginUser := b.ctx.GetSessionVars().User
|
|
if pi.User != loginUser.Username {
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", nil)
|
|
}
|
|
}
|
|
}
|
|
case *ast.UseStmt:
|
|
if raw.DBName == "" {
|
|
return nil, ErrNoDB
|
|
}
|
|
case *ast.ShutdownStmt:
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.ShutdownPriv, "", "", "", nil)
|
|
}
|
|
return p, nil
|
|
}
|
|
|
|
func collectVisitInfoFromRevokeStmt(sctx sessionctx.Context, vi []visitInfo, stmt *ast.RevokeStmt) []visitInfo {
|
|
// To use REVOKE, you must have the GRANT OPTION privilege,
|
|
// and you must have the privileges that you are granting.
|
|
dbName := stmt.Level.DBName
|
|
tableName := stmt.Level.TableName
|
|
if dbName == "" {
|
|
dbName = sctx.GetSessionVars().CurrentDB
|
|
}
|
|
vi = appendVisitInfo(vi, mysql.GrantPriv, dbName, tableName, "", nil)
|
|
|
|
var allPrivs []mysql.PrivilegeType
|
|
for _, item := range stmt.Privs {
|
|
if item.Priv == mysql.AllPriv {
|
|
switch stmt.Level.Level {
|
|
case ast.GrantLevelGlobal:
|
|
allPrivs = mysql.AllGlobalPrivs
|
|
case ast.GrantLevelDB:
|
|
allPrivs = mysql.AllDBPrivs
|
|
case ast.GrantLevelTable:
|
|
allPrivs = mysql.AllTablePrivs
|
|
}
|
|
break
|
|
}
|
|
vi = appendVisitInfo(vi, item.Priv, dbName, tableName, "", nil)
|
|
}
|
|
|
|
for _, priv := range allPrivs {
|
|
vi = appendVisitInfo(vi, priv, dbName, tableName, "", nil)
|
|
}
|
|
|
|
return vi
|
|
}
|
|
|
|
func collectVisitInfoFromGrantStmt(sctx sessionctx.Context, vi []visitInfo, stmt *ast.GrantStmt) []visitInfo {
|
|
// To use GRANT, you must have the GRANT OPTION privilege,
|
|
// and you must have the privileges that you are granting.
|
|
dbName := stmt.Level.DBName
|
|
tableName := stmt.Level.TableName
|
|
if dbName == "" {
|
|
dbName = sctx.GetSessionVars().CurrentDB
|
|
}
|
|
vi = appendVisitInfo(vi, mysql.GrantPriv, dbName, tableName, "", nil)
|
|
|
|
var allPrivs []mysql.PrivilegeType
|
|
for _, item := range stmt.Privs {
|
|
if item.Priv == mysql.AllPriv {
|
|
switch stmt.Level.Level {
|
|
case ast.GrantLevelGlobal:
|
|
allPrivs = mysql.AllGlobalPrivs
|
|
case ast.GrantLevelDB:
|
|
allPrivs = mysql.AllDBPrivs
|
|
case ast.GrantLevelTable:
|
|
allPrivs = mysql.AllTablePrivs
|
|
}
|
|
break
|
|
}
|
|
vi = appendVisitInfo(vi, item.Priv, dbName, tableName, "", nil)
|
|
}
|
|
|
|
for _, priv := range allPrivs {
|
|
vi = appendVisitInfo(vi, priv, dbName, tableName, "", nil)
|
|
}
|
|
|
|
return vi
|
|
}
|
|
|
|
func (b *PlanBuilder) getDefaultValue(col *table.Column) (*expression.Constant, error) {
|
|
var (
|
|
value types.Datum
|
|
err error
|
|
)
|
|
if col.DefaultIsExpr && col.DefaultExpr != nil {
|
|
value, err = table.EvalColDefaultExpr(b.ctx, col.ToInfo(), col.DefaultExpr)
|
|
} else {
|
|
value, err = table.GetColDefaultValue(b.ctx, col.ToInfo())
|
|
}
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
return &expression.Constant{Value: value, RetType: &col.FieldType}, nil
|
|
}
|
|
|
|
func (b *PlanBuilder) findDefaultValue(cols []*table.Column, name *ast.ColumnName) (*expression.Constant, error) {
|
|
for _, col := range cols {
|
|
if col.Name.L == name.Name.L {
|
|
return b.getDefaultValue(col)
|
|
}
|
|
}
|
|
return nil, ErrUnknownColumn.GenWithStackByArgs(name.Name.O, "field_list")
|
|
}
|
|
|
|
// resolveGeneratedColumns resolves generated columns with their generation
|
|
// expressions respectively. onDups indicates which columns are in on-duplicate list.
|
|
func (b *PlanBuilder) resolveGeneratedColumns(ctx context.Context, columns []*table.Column, onDups map[string]struct{}, mockPlan LogicalPlan) (igc InsertGeneratedColumns, err error) {
|
|
for _, column := range columns {
|
|
if !column.IsGenerated() {
|
|
continue
|
|
}
|
|
columnName := &ast.ColumnName{Name: column.Name}
|
|
columnName.SetText(column.Name.O)
|
|
|
|
idx, err := expression.FindFieldName(mockPlan.OutputNames(), columnName)
|
|
if err != nil {
|
|
return igc, err
|
|
}
|
|
colExpr := mockPlan.Schema().Columns[idx]
|
|
|
|
expr, _, err := b.rewrite(ctx, column.GeneratedExpr, mockPlan, nil, true)
|
|
if err != nil {
|
|
return igc, err
|
|
}
|
|
|
|
igc.Columns = append(igc.Columns, columnName)
|
|
igc.Exprs = append(igc.Exprs, expr)
|
|
if onDups == nil {
|
|
continue
|
|
}
|
|
for dep := range column.Dependences {
|
|
if _, ok := onDups[dep]; ok {
|
|
assign := &expression.Assignment{Col: colExpr, ColName: column.Name, Expr: expr}
|
|
igc.OnDuplicates = append(igc.OnDuplicates, assign)
|
|
break
|
|
}
|
|
}
|
|
}
|
|
return igc, nil
|
|
}
|
|
|
|
func (b *PlanBuilder) buildInsert(ctx context.Context, insert *ast.InsertStmt) (Plan, error) {
|
|
ts, ok := insert.Table.TableRefs.Left.(*ast.TableSource)
|
|
if !ok {
|
|
return nil, infoschema.ErrTableNotExists.GenWithStackByArgs()
|
|
}
|
|
tn, ok := ts.Source.(*ast.TableName)
|
|
if !ok {
|
|
return nil, infoschema.ErrTableNotExists.GenWithStackByArgs()
|
|
}
|
|
tableInfo := tn.TableInfo
|
|
if tableInfo.IsView() {
|
|
err := errors.Errorf("insert into view %s is not supported now.", tableInfo.Name.O)
|
|
if insert.IsReplace {
|
|
err = errors.Errorf("replace into view %s is not supported now.", tableInfo.Name.O)
|
|
}
|
|
return nil, err
|
|
}
|
|
if tableInfo.IsSequence() {
|
|
err := errors.Errorf("insert into sequence %s is not supported now.", tableInfo.Name.O)
|
|
if insert.IsReplace {
|
|
err = errors.Errorf("replace into sequence %s is not supported now.", tableInfo.Name.O)
|
|
}
|
|
return nil, err
|
|
}
|
|
// Build Schema with DBName otherwise ColumnRef with DBName cannot match any Column in Schema.
|
|
schema, names := expression.TableInfo2SchemaAndNames(b.ctx, tn.Schema, tableInfo)
|
|
tableInPlan, ok := b.is.TableByID(tableInfo.ID)
|
|
if !ok {
|
|
return nil, errors.Errorf("Can't get table %s.", tableInfo.Name.O)
|
|
}
|
|
|
|
insertPlan := Insert{
|
|
Table: tableInPlan,
|
|
Columns: insert.Columns,
|
|
tableSchema: schema,
|
|
tableColNames: names,
|
|
IsReplace: insert.IsReplace,
|
|
}.Init(b.ctx)
|
|
|
|
if tableInfo.GetPartitionInfo() != nil && len(insert.PartitionNames) != 0 {
|
|
givenPartitionSets := make(map[int64]struct{}, len(insert.PartitionNames))
|
|
// check partition by name.
|
|
for _, name := range insert.PartitionNames {
|
|
id, err := tables.FindPartitionByName(tableInfo, name.L)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
givenPartitionSets[id] = struct{}{}
|
|
}
|
|
pt := tableInPlan.(table.PartitionedTable)
|
|
insertPlan.Table = tables.NewPartitionTableithGivenSets(pt, givenPartitionSets)
|
|
} else if len(insert.PartitionNames) != 0 {
|
|
return nil, ErrPartitionClauseOnNonpartitioned
|
|
}
|
|
|
|
var authErr error
|
|
if b.ctx.GetSessionVars().User != nil {
|
|
authErr = ErrTableaccessDenied.GenWithStackByArgs("INSERT", b.ctx.GetSessionVars().User.AuthUsername,
|
|
b.ctx.GetSessionVars().User.AuthHostname, tableInfo.Name.L)
|
|
}
|
|
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.InsertPriv, tn.DBInfo.Name.L,
|
|
tableInfo.Name.L, "", authErr)
|
|
|
|
mockTablePlan := LogicalTableDual{}.Init(b.ctx, b.getSelectOffset())
|
|
mockTablePlan.SetSchema(insertPlan.tableSchema)
|
|
mockTablePlan.names = insertPlan.tableColNames
|
|
|
|
checkRefColumn := func(n ast.Node) ast.Node {
|
|
if insertPlan.NeedFillDefaultValue {
|
|
return n
|
|
}
|
|
switch n.(type) {
|
|
case *ast.ColumnName, *ast.ColumnNameExpr:
|
|
insertPlan.NeedFillDefaultValue = true
|
|
}
|
|
return n
|
|
}
|
|
|
|
if len(insert.Setlist) > 0 {
|
|
// Branch for `INSERT ... SET ...`.
|
|
err := b.buildSetValuesOfInsert(ctx, insert, insertPlan, mockTablePlan, checkRefColumn)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
} else if len(insert.Lists) > 0 {
|
|
// Branch for `INSERT ... VALUES ...`.
|
|
err := b.buildValuesListOfInsert(ctx, insert, insertPlan, mockTablePlan, checkRefColumn)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
} else {
|
|
// Branch for `INSERT ... SELECT ...`.
|
|
err := b.buildSelectPlanOfInsert(ctx, insert, insertPlan)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
}
|
|
|
|
mockTablePlan.SetSchema(insertPlan.Schema4OnDuplicate)
|
|
mockTablePlan.names = insertPlan.names4OnDuplicate
|
|
|
|
onDupColSet, err := insertPlan.resolveOnDuplicate(insert.OnDuplicate, tableInfo, func(node ast.ExprNode) (expression.Expression, error) {
|
|
return b.rewriteInsertOnDuplicateUpdate(ctx, node, mockTablePlan, insertPlan)
|
|
})
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
// Calculate generated columns.
|
|
mockTablePlan.schema = insertPlan.tableSchema
|
|
mockTablePlan.names = insertPlan.tableColNames
|
|
insertPlan.GenCols, err = b.resolveGeneratedColumns(ctx, insertPlan.Table.Cols(), onDupColSet, mockTablePlan)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
err = insertPlan.ResolveIndices()
|
|
return insertPlan, err
|
|
}
|
|
|
|
func (p *Insert) resolveOnDuplicate(onDup []*ast.Assignment, tblInfo *model.TableInfo, yield func(ast.ExprNode) (expression.Expression, error)) (map[string]struct{}, error) {
|
|
onDupColSet := make(map[string]struct{}, len(onDup))
|
|
colMap := make(map[string]*table.Column, len(p.Table.Cols()))
|
|
for _, col := range p.Table.Cols() {
|
|
colMap[col.Name.L] = col
|
|
}
|
|
for _, assign := range onDup {
|
|
// Check whether the column to be updated exists in the source table.
|
|
idx, err := expression.FindFieldName(p.tableColNames, assign.Column)
|
|
if err != nil {
|
|
return nil, err
|
|
} else if idx < 0 {
|
|
return nil, ErrUnknownColumn.GenWithStackByArgs(assign.Column.OrigColName(), "field list")
|
|
}
|
|
|
|
column := colMap[assign.Column.Name.L]
|
|
if column.Hidden {
|
|
return nil, ErrUnknownColumn.GenWithStackByArgs(column.Name, clauseMsg[fieldList])
|
|
}
|
|
// Check whether the column to be updated is the generated column.
|
|
defaultExpr := extractDefaultExpr(assign.Expr)
|
|
if defaultExpr != nil {
|
|
defaultExpr.Name = assign.Column
|
|
}
|
|
// 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 column.IsGenerated() {
|
|
if defaultExpr != nil {
|
|
continue
|
|
}
|
|
return nil, ErrBadGeneratedColumn.GenWithStackByArgs(assign.Column.Name.O, tblInfo.Name.O)
|
|
}
|
|
|
|
onDupColSet[column.Name.L] = struct{}{}
|
|
|
|
expr, err := yield(assign.Expr)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
p.OnDuplicate = append(p.OnDuplicate, &expression.Assignment{
|
|
Col: p.tableSchema.Columns[idx],
|
|
ColName: p.tableColNames[idx].ColName,
|
|
Expr: expr,
|
|
})
|
|
}
|
|
return onDupColSet, nil
|
|
}
|
|
|
|
func (b *PlanBuilder) getAffectCols(insertStmt *ast.InsertStmt, insertPlan *Insert) (affectedValuesCols []*table.Column, err error) {
|
|
if len(insertStmt.Columns) > 0 {
|
|
// This branch is for the following scenarios:
|
|
// 1. `INSERT INTO tbl_name (col_name [, col_name] ...) {VALUES | VALUE} (value_list) [, (value_list)] ...`,
|
|
// 2. `INSERT INTO tbl_name (col_name [, col_name] ...) SELECT ...`.
|
|
colName := make([]string, 0, len(insertStmt.Columns))
|
|
for _, col := range insertStmt.Columns {
|
|
colName = append(colName, col.Name.O)
|
|
}
|
|
var missingColName string
|
|
affectedValuesCols, missingColName = table.FindCols(insertPlan.Table.VisibleCols(), colName, insertPlan.Table.Meta().PKIsHandle)
|
|
if missingColName != "" {
|
|
return nil, ErrUnknownColumn.GenWithStackByArgs(missingColName, clauseMsg[fieldList])
|
|
}
|
|
} else if len(insertStmt.Setlist) == 0 {
|
|
// This branch is for the following scenarios:
|
|
// 1. `INSERT INTO tbl_name {VALUES | VALUE} (value_list) [, (value_list)] ...`,
|
|
// 2. `INSERT INTO tbl_name SELECT ...`.
|
|
affectedValuesCols = insertPlan.Table.VisibleCols()
|
|
}
|
|
return affectedValuesCols, nil
|
|
}
|
|
|
|
func (b *PlanBuilder) buildSetValuesOfInsert(ctx context.Context, insert *ast.InsertStmt, insertPlan *Insert, mockTablePlan *LogicalTableDual, checkRefColumn func(n ast.Node) ast.Node) error {
|
|
tableInfo := insertPlan.Table.Meta()
|
|
colNames := make([]string, 0, len(insert.Setlist))
|
|
exprCols := make([]*expression.Column, 0, len(insert.Setlist))
|
|
for _, assign := range insert.Setlist {
|
|
idx, err := expression.FindFieldName(insertPlan.tableColNames, assign.Column)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
if idx < 0 {
|
|
return errors.Errorf("Can't find column %s", assign.Column)
|
|
}
|
|
colNames = append(colNames, assign.Column.Name.L)
|
|
exprCols = append(exprCols, insertPlan.tableSchema.Columns[idx])
|
|
}
|
|
|
|
// Check whether the column to be updated is the generated column.
|
|
tCols, missingColName := table.FindCols(insertPlan.Table.VisibleCols(), colNames, tableInfo.PKIsHandle)
|
|
if missingColName != "" {
|
|
return ErrUnknownColumn.GenWithStackByArgs(missingColName, clauseMsg[fieldList])
|
|
}
|
|
generatedColumns := make(map[string]struct{}, len(tCols))
|
|
for _, tCol := range tCols {
|
|
if tCol.IsGenerated() {
|
|
generatedColumns[tCol.Name.L] = struct{}{}
|
|
}
|
|
}
|
|
|
|
insertPlan.AllAssignmentsAreConstant = true
|
|
for i, assign := range insert.Setlist {
|
|
defaultExpr := extractDefaultExpr(assign.Expr)
|
|
if defaultExpr != nil {
|
|
defaultExpr.Name = assign.Column
|
|
}
|
|
// 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 := generatedColumns[assign.Column.Name.L]; ok {
|
|
if defaultExpr != nil {
|
|
continue
|
|
}
|
|
return ErrBadGeneratedColumn.GenWithStackByArgs(assign.Column.Name.O, tableInfo.Name.O)
|
|
}
|
|
b.curClause = fieldList
|
|
// subquery in insert values should not reference upper scope
|
|
usingPlan := mockTablePlan
|
|
if _, ok := assign.Expr.(*ast.SubqueryExpr); ok {
|
|
usingPlan = LogicalTableDual{}.Init(b.ctx, b.getSelectOffset())
|
|
}
|
|
expr, _, err := b.rewriteWithPreprocess(ctx, assign.Expr, usingPlan, nil, nil, true, checkRefColumn)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
if insertPlan.AllAssignmentsAreConstant {
|
|
_, isConstant := expr.(*expression.Constant)
|
|
insertPlan.AllAssignmentsAreConstant = isConstant
|
|
}
|
|
|
|
insertPlan.SetList = append(insertPlan.SetList, &expression.Assignment{
|
|
Col: exprCols[i],
|
|
ColName: model.NewCIStr(colNames[i]),
|
|
Expr: expr,
|
|
})
|
|
}
|
|
insertPlan.Schema4OnDuplicate = insertPlan.tableSchema
|
|
insertPlan.names4OnDuplicate = insertPlan.tableColNames
|
|
return nil
|
|
}
|
|
|
|
func (b *PlanBuilder) buildValuesListOfInsert(ctx context.Context, insert *ast.InsertStmt, insertPlan *Insert, mockTablePlan *LogicalTableDual, checkRefColumn func(n ast.Node) ast.Node) error {
|
|
affectedValuesCols, err := b.getAffectCols(insert, insertPlan)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
// If value_list and col_list are empty and we have a generated column, we can still write data to this table.
|
|
// For example, insert into t values(); can be executed successfully if t has a generated column.
|
|
if len(insert.Columns) > 0 || len(insert.Lists[0]) > 0 {
|
|
// If value_list or col_list is not empty, the length of value_list should be the same with that of col_list.
|
|
if len(insert.Lists[0]) != len(affectedValuesCols) {
|
|
return ErrWrongValueCountOnRow.GenWithStackByArgs(1)
|
|
}
|
|
}
|
|
|
|
insertPlan.AllAssignmentsAreConstant = true
|
|
totalTableCols := insertPlan.Table.Cols()
|
|
for i, valuesItem := range insert.Lists {
|
|
// The length of all the value_list should be the same.
|
|
// "insert into t values (), ()" is valid.
|
|
// "insert into t values (), (1)" is not valid.
|
|
// "insert into t values (1), ()" is not valid.
|
|
// "insert into t values (1,2), (1)" is not valid.
|
|
if i > 0 && len(insert.Lists[i-1]) != len(insert.Lists[i]) {
|
|
return ErrWrongValueCountOnRow.GenWithStackByArgs(i + 1)
|
|
}
|
|
exprList := make([]expression.Expression, 0, len(valuesItem))
|
|
for j, valueItem := range valuesItem {
|
|
var expr expression.Expression
|
|
var err error
|
|
var generatedColumnWithDefaultExpr bool
|
|
col := affectedValuesCols[j]
|
|
switch x := valueItem.(type) {
|
|
case *ast.DefaultExpr:
|
|
if col.IsGenerated() {
|
|
if x.Name != nil {
|
|
return ErrBadGeneratedColumn.GenWithStackByArgs(col.Name.O, insertPlan.Table.Meta().Name.O)
|
|
}
|
|
generatedColumnWithDefaultExpr = true
|
|
break
|
|
}
|
|
if x.Name != nil {
|
|
expr, err = b.findDefaultValue(totalTableCols, x.Name)
|
|
} else {
|
|
expr, err = b.getDefaultValue(affectedValuesCols[j])
|
|
}
|
|
case *driver.ValueExpr:
|
|
expr = &expression.Constant{
|
|
Value: x.Datum,
|
|
RetType: &x.Type,
|
|
}
|
|
default:
|
|
b.curClause = fieldList
|
|
// subquery in insert values should not reference upper scope
|
|
usingPlan := mockTablePlan
|
|
if _, ok := valueItem.(*ast.SubqueryExpr); ok {
|
|
usingPlan = LogicalTableDual{}.Init(b.ctx, b.getSelectOffset())
|
|
}
|
|
expr, _, err = b.rewriteWithPreprocess(ctx, valueItem, usingPlan, nil, nil, true, checkRefColumn)
|
|
}
|
|
if err != nil {
|
|
return err
|
|
}
|
|
if insertPlan.AllAssignmentsAreConstant {
|
|
_, isConstant := expr.(*expression.Constant)
|
|
insertPlan.AllAssignmentsAreConstant = isConstant
|
|
}
|
|
// 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 col.IsGenerated() {
|
|
if generatedColumnWithDefaultExpr {
|
|
continue
|
|
}
|
|
return ErrBadGeneratedColumn.GenWithStackByArgs(col.Name.O, insertPlan.Table.Meta().Name.O)
|
|
}
|
|
exprList = append(exprList, expr)
|
|
}
|
|
insertPlan.Lists = append(insertPlan.Lists, exprList)
|
|
}
|
|
insertPlan.Schema4OnDuplicate = insertPlan.tableSchema
|
|
insertPlan.names4OnDuplicate = insertPlan.tableColNames
|
|
return nil
|
|
}
|
|
|
|
func (b *PlanBuilder) buildSelectPlanOfInsert(ctx context.Context, insert *ast.InsertStmt, insertPlan *Insert) error {
|
|
affectedValuesCols, err := b.getAffectCols(insert, insertPlan)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
selectPlan, err := b.Build(ctx, insert.Select)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
// Check to guarantee that the length of the row returned by select is equal to that of affectedValuesCols.
|
|
if selectPlan.Schema().Len() != len(affectedValuesCols) {
|
|
return ErrWrongValueCountOnRow.GenWithStackByArgs(1)
|
|
}
|
|
|
|
// Check to guarantee that there's no generated column.
|
|
// This check should be done after the above one to make its behavior compatible with MySQL.
|
|
// For example, table t has two columns, namely a and b, and b is a generated column.
|
|
// "insert into t (b) select * from t" will raise an error that the column count is not matched.
|
|
// "insert into t select * from t" will raise an error that there's a generated column in the column list.
|
|
// If we do this check before the above one, "insert into t (b) select * from t" will raise an error
|
|
// that there's a generated column in the column list.
|
|
for _, col := range affectedValuesCols {
|
|
if col.IsGenerated() {
|
|
return ErrBadGeneratedColumn.GenWithStackByArgs(col.Name.O, insertPlan.Table.Meta().Name.O)
|
|
}
|
|
}
|
|
|
|
names := selectPlan.OutputNames()
|
|
insertPlan.SelectPlan, _, err = DoOptimize(ctx, b.ctx, b.optFlag, selectPlan.(LogicalPlan))
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
// schema4NewRow is the schema for the newly created data record based on
|
|
// the result of the select statement.
|
|
schema4NewRow := expression.NewSchema(make([]*expression.Column, len(insertPlan.Table.Cols()))...)
|
|
names4NewRow := make(types.NameSlice, len(insertPlan.Table.Cols()))
|
|
// TODO: don't clone it.
|
|
for i, selCol := range insertPlan.SelectPlan.Schema().Columns {
|
|
ordinal := affectedValuesCols[i].Offset
|
|
schema4NewRow.Columns[ordinal] = &expression.Column{}
|
|
*schema4NewRow.Columns[ordinal] = *selCol
|
|
|
|
schema4NewRow.Columns[ordinal].RetType = &types.FieldType{}
|
|
*schema4NewRow.Columns[ordinal].RetType = affectedValuesCols[i].FieldType
|
|
|
|
names4NewRow[ordinal] = names[i]
|
|
}
|
|
for i := range schema4NewRow.Columns {
|
|
if schema4NewRow.Columns[i] == nil {
|
|
schema4NewRow.Columns[i] = &expression.Column{UniqueID: insertPlan.ctx.GetSessionVars().AllocPlanColumnID()}
|
|
names4NewRow[i] = types.EmptyName
|
|
}
|
|
}
|
|
insertPlan.Schema4OnDuplicate = expression.MergeSchema(insertPlan.tableSchema, schema4NewRow)
|
|
insertPlan.names4OnDuplicate = append(insertPlan.tableColNames.Shallow(), names4NewRow...)
|
|
return nil
|
|
}
|
|
|
|
func (b *PlanBuilder) buildLoadData(ctx context.Context, ld *ast.LoadDataStmt) (Plan, error) {
|
|
p := LoadData{
|
|
IsLocal: ld.IsLocal,
|
|
OnDuplicate: ld.OnDuplicate,
|
|
Path: ld.Path,
|
|
Table: ld.Table,
|
|
Columns: ld.Columns,
|
|
FieldsInfo: ld.FieldsInfo,
|
|
LinesInfo: ld.LinesInfo,
|
|
IgnoreLines: ld.IgnoreLines,
|
|
}.Init(b.ctx)
|
|
user := b.ctx.GetSessionVars().User
|
|
var insertErr error
|
|
if user != nil {
|
|
insertErr = ErrTableaccessDenied.GenWithStackByArgs("INSERT", user.AuthUsername, user.AuthHostname, p.Table.Name.O)
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.InsertPriv, p.Table.Schema.O, p.Table.Name.O, "", insertErr)
|
|
tableInfo := p.Table.TableInfo
|
|
tableInPlan, ok := b.is.TableByID(tableInfo.ID)
|
|
if !ok {
|
|
db := b.ctx.GetSessionVars().CurrentDB
|
|
return nil, infoschema.ErrTableNotExists.GenWithStackByArgs(db, tableInfo.Name.O)
|
|
}
|
|
schema, names := expression.TableInfo2SchemaAndNames(b.ctx, model.NewCIStr(""), tableInfo)
|
|
mockTablePlan := LogicalTableDual{}.Init(b.ctx, b.getSelectOffset())
|
|
mockTablePlan.SetSchema(schema)
|
|
mockTablePlan.names = names
|
|
|
|
var err error
|
|
p.GenCols, err = b.resolveGeneratedColumns(ctx, tableInPlan.Cols(), nil, mockTablePlan)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
return p, nil
|
|
}
|
|
|
|
func (b *PlanBuilder) buildLoadStats(ld *ast.LoadStatsStmt) Plan {
|
|
p := &LoadStats{Path: ld.Path}
|
|
return p
|
|
}
|
|
|
|
func (b *PlanBuilder) buildIndexAdvise(node *ast.IndexAdviseStmt) Plan {
|
|
p := &IndexAdvise{
|
|
IsLocal: node.IsLocal,
|
|
Path: node.Path,
|
|
MaxMinutes: node.MaxMinutes,
|
|
MaxIndexNum: node.MaxIndexNum,
|
|
LinesInfo: node.LinesInfo,
|
|
}
|
|
return p
|
|
}
|
|
|
|
func (b *PlanBuilder) buildSplitRegion(node *ast.SplitRegionStmt) (Plan, error) {
|
|
if len(node.IndexName.L) != 0 {
|
|
return b.buildSplitIndexRegion(node)
|
|
}
|
|
return b.buildSplitTableRegion(node)
|
|
}
|
|
|
|
func (b *PlanBuilder) buildSplitIndexRegion(node *ast.SplitRegionStmt) (Plan, error) {
|
|
tblInfo := node.Table.TableInfo
|
|
indexInfo := tblInfo.FindIndexByName(node.IndexName.L)
|
|
if indexInfo == nil {
|
|
return nil, ErrKeyDoesNotExist.GenWithStackByArgs(node.IndexName, tblInfo.Name)
|
|
}
|
|
mockTablePlan := LogicalTableDual{}.Init(b.ctx, b.getSelectOffset())
|
|
schema, names := expression.TableInfo2SchemaAndNames(b.ctx, node.Table.Schema, tblInfo)
|
|
mockTablePlan.SetSchema(schema)
|
|
mockTablePlan.names = names
|
|
|
|
p := &SplitRegion{
|
|
TableInfo: tblInfo,
|
|
PartitionNames: node.PartitionNames,
|
|
IndexInfo: indexInfo,
|
|
}
|
|
p.names = names
|
|
p.setSchemaAndNames(buildSplitRegionsSchema())
|
|
// Split index regions by user specified value lists.
|
|
if len(node.SplitOpt.ValueLists) > 0 {
|
|
indexValues := make([][]types.Datum, 0, len(node.SplitOpt.ValueLists))
|
|
for i, valuesItem := range node.SplitOpt.ValueLists {
|
|
if len(valuesItem) > len(indexInfo.Columns) {
|
|
return nil, ErrWrongValueCountOnRow.GenWithStackByArgs(i + 1)
|
|
}
|
|
values, err := b.convertValue2ColumnType(valuesItem, mockTablePlan, indexInfo, tblInfo)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
indexValues = append(indexValues, values)
|
|
}
|
|
p.ValueLists = indexValues
|
|
return p, nil
|
|
}
|
|
|
|
// Split index regions by lower, upper value.
|
|
checkLowerUpperValue := func(valuesItem []ast.ExprNode, name string) ([]types.Datum, error) {
|
|
if len(valuesItem) == 0 {
|
|
return nil, errors.Errorf("Split index `%v` region %s value count should more than 0", indexInfo.Name, name)
|
|
}
|
|
if len(valuesItem) > len(indexInfo.Columns) {
|
|
return nil, errors.Errorf("Split index `%v` region column count doesn't match value count at %v", indexInfo.Name, name)
|
|
}
|
|
return b.convertValue2ColumnType(valuesItem, mockTablePlan, indexInfo, tblInfo)
|
|
}
|
|
lowerValues, err := checkLowerUpperValue(node.SplitOpt.Lower, "lower")
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
upperValues, err := checkLowerUpperValue(node.SplitOpt.Upper, "upper")
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
p.Lower = lowerValues
|
|
p.Upper = upperValues
|
|
|
|
maxSplitRegionNum := int64(config.GetGlobalConfig().SplitRegionMaxNum)
|
|
if node.SplitOpt.Num > maxSplitRegionNum {
|
|
return nil, errors.Errorf("Split index region num exceeded the limit %v", maxSplitRegionNum)
|
|
} else if node.SplitOpt.Num < 1 {
|
|
return nil, errors.Errorf("Split index region num should more than 0")
|
|
}
|
|
p.Num = int(node.SplitOpt.Num)
|
|
return p, nil
|
|
}
|
|
|
|
func (b *PlanBuilder) convertValue2ColumnType(valuesItem []ast.ExprNode, mockTablePlan LogicalPlan, indexInfo *model.IndexInfo, tblInfo *model.TableInfo) ([]types.Datum, error) {
|
|
values := make([]types.Datum, 0, len(valuesItem))
|
|
for j, valueItem := range valuesItem {
|
|
colOffset := indexInfo.Columns[j].Offset
|
|
value, err := b.convertValue(valueItem, mockTablePlan, tblInfo.Columns[colOffset])
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
values = append(values, value)
|
|
}
|
|
return values, nil
|
|
}
|
|
|
|
func (b *PlanBuilder) convertValue(valueItem ast.ExprNode, mockTablePlan LogicalPlan, col *model.ColumnInfo) (d types.Datum, err error) {
|
|
var expr expression.Expression
|
|
switch x := valueItem.(type) {
|
|
case *driver.ValueExpr:
|
|
expr = &expression.Constant{
|
|
Value: x.Datum,
|
|
RetType: &x.Type,
|
|
}
|
|
default:
|
|
expr, _, err = b.rewrite(context.TODO(), valueItem, mockTablePlan, nil, true)
|
|
if err != nil {
|
|
return d, err
|
|
}
|
|
}
|
|
constant, ok := expr.(*expression.Constant)
|
|
if !ok {
|
|
return d, errors.New("Expect constant values")
|
|
}
|
|
value, err := constant.Eval(chunk.Row{})
|
|
if err != nil {
|
|
return d, err
|
|
}
|
|
d, err = value.ConvertTo(b.ctx.GetSessionVars().StmtCtx, &col.FieldType)
|
|
if err != nil {
|
|
if !types.ErrTruncated.Equal(err) && !types.ErrTruncatedWrongVal.Equal(err) {
|
|
return d, err
|
|
}
|
|
valStr, err1 := value.ToString()
|
|
if err1 != nil {
|
|
return d, err
|
|
}
|
|
return d, types.ErrTruncated.GenWithStack("Incorrect value: '%-.128s' for column '%.192s'", valStr, col.Name.O)
|
|
}
|
|
return d, nil
|
|
}
|
|
|
|
func (b *PlanBuilder) buildSplitTableRegion(node *ast.SplitRegionStmt) (Plan, error) {
|
|
tblInfo := node.Table.TableInfo
|
|
var pkCol *model.ColumnInfo
|
|
if tblInfo.PKIsHandle {
|
|
if col := tblInfo.GetPkColInfo(); col != nil {
|
|
pkCol = col
|
|
}
|
|
}
|
|
if pkCol == nil {
|
|
pkCol = model.NewExtraHandleColInfo()
|
|
}
|
|
mockTablePlan := LogicalTableDual{}.Init(b.ctx, b.getSelectOffset())
|
|
schema, names := expression.TableInfo2SchemaAndNames(b.ctx, node.Table.Schema, tblInfo)
|
|
mockTablePlan.SetSchema(schema)
|
|
mockTablePlan.names = names
|
|
|
|
p := &SplitRegion{
|
|
TableInfo: tblInfo,
|
|
PartitionNames: node.PartitionNames,
|
|
}
|
|
p.setSchemaAndNames(buildSplitRegionsSchema())
|
|
if len(node.SplitOpt.ValueLists) > 0 {
|
|
values := make([][]types.Datum, 0, len(node.SplitOpt.ValueLists))
|
|
for i, valuesItem := range node.SplitOpt.ValueLists {
|
|
if len(valuesItem) > 1 {
|
|
return nil, ErrWrongValueCountOnRow.GenWithStackByArgs(i + 1)
|
|
}
|
|
value, err := b.convertValue(valuesItem[0], mockTablePlan, pkCol)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
values = append(values, []types.Datum{value})
|
|
}
|
|
p.ValueLists = values
|
|
return p, nil
|
|
}
|
|
|
|
checkLowerUpperValue := func(valuesItem []ast.ExprNode, name string) (types.Datum, error) {
|
|
if len(valuesItem) != 1 {
|
|
return types.Datum{}, errors.Errorf("Split table region %s value count should be 1", name)
|
|
}
|
|
return b.convertValue(valuesItem[0], mockTablePlan, pkCol)
|
|
}
|
|
lowerValues, err := checkLowerUpperValue(node.SplitOpt.Lower, "lower")
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
upperValue, err := checkLowerUpperValue(node.SplitOpt.Upper, "upper")
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
p.Lower = []types.Datum{lowerValues}
|
|
p.Upper = []types.Datum{upperValue}
|
|
|
|
maxSplitRegionNum := int64(config.GetGlobalConfig().SplitRegionMaxNum)
|
|
if node.SplitOpt.Num > maxSplitRegionNum {
|
|
return nil, errors.Errorf("Split table region num exceeded the limit %v", maxSplitRegionNum)
|
|
} else if node.SplitOpt.Num < 1 {
|
|
return nil, errors.Errorf("Split table region num should more than 0")
|
|
}
|
|
p.Num = int(node.SplitOpt.Num)
|
|
return p, nil
|
|
}
|
|
|
|
func (b *PlanBuilder) buildDDL(ctx context.Context, node ast.DDLNode) (Plan, error) {
|
|
var authErr error
|
|
switch v := node.(type) {
|
|
case *ast.AlterDatabaseStmt:
|
|
if v.AlterDefaultDatabase {
|
|
v.Name = b.ctx.GetSessionVars().CurrentDB
|
|
}
|
|
if v.Name == "" {
|
|
return nil, ErrNoDB
|
|
}
|
|
if b.ctx.GetSessionVars().User != nil {
|
|
authErr = ErrDBaccessDenied.GenWithStackByArgs("ALTER", b.ctx.GetSessionVars().User.AuthUsername,
|
|
b.ctx.GetSessionVars().User.AuthHostname, v.Name)
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.AlterPriv, v.Name, "", "", authErr)
|
|
case *ast.AlterTableStmt:
|
|
if b.ctx.GetSessionVars().User != nil {
|
|
authErr = ErrTableaccessDenied.GenWithStackByArgs("ALTER", b.ctx.GetSessionVars().User.AuthUsername,
|
|
b.ctx.GetSessionVars().User.AuthHostname, v.Table.Name.L)
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.AlterPriv, v.Table.Schema.L,
|
|
v.Table.Name.L, "", authErr)
|
|
for _, spec := range v.Specs {
|
|
if spec.Tp == ast.AlterTableRenameTable {
|
|
if b.ctx.GetSessionVars().User != nil {
|
|
authErr = ErrTableaccessDenied.GenWithStackByArgs("DROP", b.ctx.GetSessionVars().User.AuthUsername,
|
|
b.ctx.GetSessionVars().User.AuthHostname, v.Table.Name.L)
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.DropPriv, v.Table.Schema.L,
|
|
v.Table.Name.L, "", authErr)
|
|
|
|
if b.ctx.GetSessionVars().User != nil {
|
|
authErr = ErrTableaccessDenied.GenWithStackByArgs("CREATE", b.ctx.GetSessionVars().User.AuthUsername,
|
|
b.ctx.GetSessionVars().User.AuthHostname, spec.NewTable.Name.L)
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.CreatePriv, spec.NewTable.Schema.L,
|
|
spec.NewTable.Name.L, "", authErr)
|
|
|
|
if b.ctx.GetSessionVars().User != nil {
|
|
authErr = ErrTableaccessDenied.GenWithStackByArgs("INSERT", b.ctx.GetSessionVars().User.AuthUsername,
|
|
b.ctx.GetSessionVars().User.AuthHostname, spec.NewTable.Name.L)
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.InsertPriv, spec.NewTable.Schema.L,
|
|
spec.NewTable.Name.L, "", authErr)
|
|
} else if spec.Tp == ast.AlterTableDropPartition {
|
|
if b.ctx.GetSessionVars().User != nil {
|
|
authErr = ErrTableaccessDenied.GenWithStackByArgs("DROP", b.ctx.GetSessionVars().User.AuthUsername,
|
|
b.ctx.GetSessionVars().User.AuthHostname, v.Table.Name.L)
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.DropPriv, v.Table.Schema.L,
|
|
v.Table.Name.L, "", authErr)
|
|
}
|
|
}
|
|
case *ast.CreateDatabaseStmt:
|
|
if b.ctx.GetSessionVars().User != nil {
|
|
authErr = ErrDBaccessDenied.GenWithStackByArgs(b.ctx.GetSessionVars().User.AuthUsername,
|
|
b.ctx.GetSessionVars().User.AuthHostname, v.Name)
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.CreatePriv, v.Name,
|
|
"", "", authErr)
|
|
case *ast.CreateIndexStmt:
|
|
if b.ctx.GetSessionVars().User != nil {
|
|
authErr = ErrTableaccessDenied.GenWithStackByArgs("INDEX", b.ctx.GetSessionVars().User.AuthUsername,
|
|
b.ctx.GetSessionVars().User.AuthHostname, v.Table.Name.L)
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.IndexPriv, v.Table.Schema.L,
|
|
v.Table.Name.L, "", authErr)
|
|
case *ast.CreateTableStmt:
|
|
if b.ctx.GetSessionVars().User != nil {
|
|
authErr = ErrTableaccessDenied.GenWithStackByArgs("CREATE", b.ctx.GetSessionVars().User.AuthUsername,
|
|
b.ctx.GetSessionVars().User.AuthHostname, v.Table.Name.L)
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.CreatePriv, v.Table.Schema.L,
|
|
v.Table.Name.L, "", authErr)
|
|
if v.ReferTable != nil {
|
|
if b.ctx.GetSessionVars().User != nil {
|
|
authErr = ErrTableaccessDenied.GenWithStackByArgs("CREATE", b.ctx.GetSessionVars().User.AuthUsername,
|
|
b.ctx.GetSessionVars().User.AuthHostname, v.ReferTable.Name.L)
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SelectPriv, v.ReferTable.Schema.L,
|
|
v.ReferTable.Name.L, "", authErr)
|
|
}
|
|
case *ast.CreateViewStmt:
|
|
b.capFlag |= canExpandAST | renameView
|
|
b.renamingViewName = v.ViewName.Schema.L + "." + v.ViewName.Name.L
|
|
defer func() {
|
|
b.capFlag &= ^canExpandAST
|
|
b.capFlag &= ^renameView
|
|
}()
|
|
plan, err := b.Build(ctx, v.Select)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
schema := plan.Schema()
|
|
names := plan.OutputNames()
|
|
if v.Cols == nil {
|
|
adjustOverlongViewColname(plan.(LogicalPlan))
|
|
v.Cols = make([]model.CIStr, len(schema.Columns))
|
|
for i, name := range names {
|
|
v.Cols[i] = name.ColName
|
|
}
|
|
}
|
|
if len(v.Cols) != schema.Len() {
|
|
return nil, ddl.ErrViewWrongList
|
|
}
|
|
if b.ctx.GetSessionVars().User != nil {
|
|
authErr = ErrTableaccessDenied.GenWithStackByArgs("CREATE VIEW", b.ctx.GetSessionVars().User.AuthUsername,
|
|
b.ctx.GetSessionVars().User.AuthHostname, v.ViewName.Name.L)
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.CreateViewPriv, v.ViewName.Schema.L,
|
|
v.ViewName.Name.L, "", authErr)
|
|
if v.Definer.CurrentUser && b.ctx.GetSessionVars().User != nil {
|
|
v.Definer = b.ctx.GetSessionVars().User
|
|
}
|
|
if b.ctx.GetSessionVars().User != nil && v.Definer.String() != b.ctx.GetSessionVars().User.String() {
|
|
err = ErrSpecificAccessDenied.GenWithStackByArgs("SUPER")
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "",
|
|
"", "", err)
|
|
}
|
|
case *ast.CreateSequenceStmt:
|
|
if b.ctx.GetSessionVars().User != nil {
|
|
authErr = ErrTableaccessDenied.GenWithStackByArgs("CREATE", b.ctx.GetSessionVars().User.AuthUsername,
|
|
b.ctx.GetSessionVars().User.AuthHostname, v.Name.Name.L)
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.CreatePriv, v.Name.Schema.L,
|
|
v.Name.Name.L, "", authErr)
|
|
case *ast.DropDatabaseStmt:
|
|
if b.ctx.GetSessionVars().User != nil {
|
|
authErr = ErrDBaccessDenied.GenWithStackByArgs(b.ctx.GetSessionVars().User.AuthUsername,
|
|
b.ctx.GetSessionVars().User.AuthHostname, v.Name)
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.DropPriv, v.Name,
|
|
"", "", authErr)
|
|
case *ast.DropIndexStmt:
|
|
if b.ctx.GetSessionVars().User != nil {
|
|
authErr = ErrTableaccessDenied.GenWithStackByArgs("INDEx", b.ctx.GetSessionVars().User.AuthUsername,
|
|
b.ctx.GetSessionVars().User.AuthHostname, v.Table.Name.L)
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.IndexPriv, v.Table.Schema.L,
|
|
v.Table.Name.L, "", authErr)
|
|
case *ast.DropTableStmt:
|
|
for _, tableVal := range v.Tables {
|
|
if b.ctx.GetSessionVars().User != nil {
|
|
authErr = ErrTableaccessDenied.GenWithStackByArgs("DROP", b.ctx.GetSessionVars().User.AuthUsername,
|
|
b.ctx.GetSessionVars().User.AuthHostname, tableVal.Name.L)
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.DropPriv, tableVal.Schema.L,
|
|
tableVal.Name.L, "", authErr)
|
|
}
|
|
case *ast.DropSequenceStmt:
|
|
for _, sequence := range v.Sequences {
|
|
if b.ctx.GetSessionVars().User != nil {
|
|
authErr = ErrTableaccessDenied.GenWithStackByArgs("DROP", b.ctx.GetSessionVars().User.AuthUsername,
|
|
b.ctx.GetSessionVars().User.AuthHostname, sequence.Name.L)
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.DropPriv, sequence.Schema.L,
|
|
sequence.Name.L, "", authErr)
|
|
}
|
|
case *ast.TruncateTableStmt:
|
|
if b.ctx.GetSessionVars().User != nil {
|
|
authErr = ErrTableaccessDenied.GenWithStackByArgs("DROP", b.ctx.GetSessionVars().User.AuthUsername,
|
|
b.ctx.GetSessionVars().User.AuthHostname, v.Table.Name.L)
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.DropPriv, v.Table.Schema.L,
|
|
v.Table.Name.L, "", authErr)
|
|
case *ast.RenameTableStmt:
|
|
if b.ctx.GetSessionVars().User != nil {
|
|
authErr = ErrTableaccessDenied.GenWithStackByArgs("ALTER", b.ctx.GetSessionVars().User.AuthUsername,
|
|
b.ctx.GetSessionVars().User.AuthHostname, v.OldTable.Name.L)
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.AlterPriv, v.OldTable.Schema.L,
|
|
v.OldTable.Name.L, "", authErr)
|
|
|
|
if b.ctx.GetSessionVars().User != nil {
|
|
authErr = ErrTableaccessDenied.GenWithStackByArgs("DROP", b.ctx.GetSessionVars().User.AuthUsername,
|
|
b.ctx.GetSessionVars().User.AuthHostname, v.OldTable.Name.L)
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.DropPriv, v.OldTable.Schema.L,
|
|
v.OldTable.Name.L, "", authErr)
|
|
|
|
if b.ctx.GetSessionVars().User != nil {
|
|
authErr = ErrTableaccessDenied.GenWithStackByArgs("CREATE", b.ctx.GetSessionVars().User.AuthUsername,
|
|
b.ctx.GetSessionVars().User.AuthHostname, v.NewTable.Name.L)
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.CreatePriv, v.NewTable.Schema.L,
|
|
v.NewTable.Name.L, "", authErr)
|
|
|
|
if b.ctx.GetSessionVars().User != nil {
|
|
authErr = ErrTableaccessDenied.GenWithStackByArgs("INSERT", b.ctx.GetSessionVars().User.AuthUsername,
|
|
b.ctx.GetSessionVars().User.AuthHostname, v.NewTable.Name.L)
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.InsertPriv, v.NewTable.Schema.L,
|
|
v.NewTable.Name.L, "", authErr)
|
|
case *ast.RecoverTableStmt, *ast.FlashBackTableStmt:
|
|
// Recover table command can only be executed by administrator.
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", nil)
|
|
case *ast.LockTablesStmt, *ast.UnlockTablesStmt:
|
|
// TODO: add Lock Table privilege check.
|
|
case *ast.CleanupTableLockStmt:
|
|
// This command can only be executed by administrator.
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", nil)
|
|
case *ast.RepairTableStmt:
|
|
// Repair table command can only be executed by administrator.
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", nil)
|
|
}
|
|
p := &DDL{Statement: node}
|
|
return p, nil
|
|
}
|
|
|
|
const (
|
|
// TraceFormatRow indicates row tracing format.
|
|
TraceFormatRow = "row"
|
|
// TraceFormatJSON indicates json tracing format.
|
|
TraceFormatJSON = "json"
|
|
// TraceFormatLog indicates log tracing format.
|
|
TraceFormatLog = "log"
|
|
)
|
|
|
|
// buildTrace builds a trace plan. Inside this method, it first optimize the
|
|
// underlying query and then constructs a schema, which will be used to constructs
|
|
// rows result.
|
|
func (b *PlanBuilder) buildTrace(trace *ast.TraceStmt) (Plan, error) {
|
|
p := &Trace{StmtNode: trace.Stmt, Format: trace.Format}
|
|
switch trace.Format {
|
|
case TraceFormatRow:
|
|
schema := newColumnsWithNames(3)
|
|
schema.Append(buildColumnWithName("", "operation", mysql.TypeString, mysql.MaxBlobWidth))
|
|
schema.Append(buildColumnWithName("", "startTS", mysql.TypeString, mysql.MaxBlobWidth))
|
|
schema.Append(buildColumnWithName("", "duration", mysql.TypeString, mysql.MaxBlobWidth))
|
|
p.SetSchema(schema.col2Schema())
|
|
p.names = schema.names
|
|
case TraceFormatJSON:
|
|
schema := newColumnsWithNames(1)
|
|
schema.Append(buildColumnWithName("", "operation", mysql.TypeString, mysql.MaxBlobWidth))
|
|
p.SetSchema(schema.col2Schema())
|
|
p.names = schema.names
|
|
case TraceFormatLog:
|
|
schema := newColumnsWithNames(4)
|
|
schema.Append(buildColumnWithName("", "time", mysql.TypeTimestamp, mysql.MaxBlobWidth))
|
|
schema.Append(buildColumnWithName("", "event", mysql.TypeString, mysql.MaxBlobWidth))
|
|
schema.Append(buildColumnWithName("", "tags", mysql.TypeString, mysql.MaxBlobWidth))
|
|
schema.Append(buildColumnWithName("", "spanName", mysql.TypeString, mysql.MaxBlobWidth))
|
|
p.SetSchema(schema.col2Schema())
|
|
p.names = schema.names
|
|
default:
|
|
return nil, errors.New("trace format should be one of 'row', 'log' or 'json'")
|
|
}
|
|
return p, nil
|
|
}
|
|
|
|
func (b *PlanBuilder) buildExplainPlan(targetPlan Plan, format string, explainRows [][]string, analyze bool, execStmt ast.StmtNode, runtimeStats *execdetails.RuntimeStatsColl) (Plan, error) {
|
|
p := &Explain{
|
|
TargetPlan: targetPlan,
|
|
Format: format,
|
|
Analyze: analyze,
|
|
ExecStmt: execStmt,
|
|
ExplainRows: explainRows,
|
|
RuntimeStatsColl: runtimeStats,
|
|
}
|
|
p.ctx = b.ctx
|
|
return p, p.prepareSchema()
|
|
}
|
|
|
|
// buildExplainFor gets *last* (maybe running or finished) query plan from connection #connection id.
|
|
// See https://dev.mysql.com/doc/refman/8.0/en/explain-for-connection.html.
|
|
func (b *PlanBuilder) buildExplainFor(explainFor *ast.ExplainForStmt) (Plan, error) {
|
|
processInfo, ok := b.ctx.GetSessionManager().GetProcessInfo(explainFor.ConnectionID)
|
|
if !ok {
|
|
return nil, ErrNoSuchThread.GenWithStackByArgs(explainFor.ConnectionID)
|
|
}
|
|
if b.ctx.GetSessionVars() != nil && b.ctx.GetSessionVars().User != nil {
|
|
if b.ctx.GetSessionVars().User.Username != processInfo.User {
|
|
err := ErrAccessDenied.GenWithStackByArgs(b.ctx.GetSessionVars().User.Username, b.ctx.GetSessionVars().User.Hostname)
|
|
// Different from MySQL's behavior and document.
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", err)
|
|
}
|
|
}
|
|
|
|
targetPlan, ok := processInfo.Plan.(Plan)
|
|
if !ok || targetPlan == nil {
|
|
return &Explain{Format: explainFor.Format}, nil
|
|
}
|
|
var explainRows [][]string
|
|
if explainFor.Format == ast.ExplainFormatROW {
|
|
explainRows = processInfo.PlanExplainRows
|
|
}
|
|
return b.buildExplainPlan(targetPlan, explainFor.Format, explainRows, false, nil, processInfo.RuntimeStatsColl)
|
|
}
|
|
|
|
func (b *PlanBuilder) buildExplain(ctx context.Context, explain *ast.ExplainStmt) (Plan, error) {
|
|
if show, ok := explain.Stmt.(*ast.ShowStmt); ok {
|
|
return b.buildShow(ctx, show)
|
|
}
|
|
targetPlan, _, err := OptimizeAstNode(ctx, b.ctx, explain.Stmt, b.is)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
return b.buildExplainPlan(targetPlan, explain.Format, nil, explain.Analyze, explain.Stmt, nil)
|
|
}
|
|
|
|
func (b *PlanBuilder) buildSelectInto(ctx context.Context, sel *ast.SelectStmt) (Plan, error) {
|
|
selectIntoInfo := sel.SelectIntoOpt
|
|
sel.SelectIntoOpt = nil
|
|
targetPlan, _, err := OptimizeAstNode(ctx, b.ctx, sel, b.is)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.FilePriv, "", "", "", ErrSpecificAccessDenied.GenWithStackByArgs("FILE"))
|
|
return &SelectInto{
|
|
TargetPlan: targetPlan,
|
|
IntoOpt: selectIntoInfo,
|
|
}, nil
|
|
}
|
|
|
|
func buildShowProcedureSchema() (*expression.Schema, []*types.FieldName) {
|
|
tblName := "ROUTINES"
|
|
schema := newColumnsWithNames(11)
|
|
schema.Append(buildColumnWithName(tblName, "Db", mysql.TypeVarchar, 128))
|
|
schema.Append(buildColumnWithName(tblName, "Name", mysql.TypeVarchar, 128))
|
|
schema.Append(buildColumnWithName(tblName, "Type", mysql.TypeVarchar, 128))
|
|
schema.Append(buildColumnWithName(tblName, "Definer", mysql.TypeVarchar, 128))
|
|
schema.Append(buildColumnWithName(tblName, "Modified", mysql.TypeDatetime, 19))
|
|
schema.Append(buildColumnWithName(tblName, "Created", mysql.TypeDatetime, 19))
|
|
schema.Append(buildColumnWithName(tblName, "Security_type", mysql.TypeVarchar, 128))
|
|
schema.Append(buildColumnWithName(tblName, "Comment", mysql.TypeBlob, 196605))
|
|
schema.Append(buildColumnWithName(tblName, "character_set_client", mysql.TypeVarchar, 32))
|
|
schema.Append(buildColumnWithName(tblName, "collation_connection", mysql.TypeVarchar, 32))
|
|
schema.Append(buildColumnWithName(tblName, "Database Collation", mysql.TypeVarchar, 32))
|
|
return schema.col2Schema(), schema.names
|
|
}
|
|
|
|
func buildShowTriggerSchema() (*expression.Schema, []*types.FieldName) {
|
|
tblName := "TRIGGERS"
|
|
schema := newColumnsWithNames(11)
|
|
schema.Append(buildColumnWithName(tblName, "Trigger", mysql.TypeVarchar, 128))
|
|
schema.Append(buildColumnWithName(tblName, "Event", mysql.TypeVarchar, 128))
|
|
schema.Append(buildColumnWithName(tblName, "Table", mysql.TypeVarchar, 128))
|
|
schema.Append(buildColumnWithName(tblName, "Statement", mysql.TypeBlob, 196605))
|
|
schema.Append(buildColumnWithName(tblName, "Timing", mysql.TypeVarchar, 128))
|
|
schema.Append(buildColumnWithName(tblName, "Created", mysql.TypeDatetime, 19))
|
|
schema.Append(buildColumnWithName(tblName, "sql_mode", mysql.TypeBlob, 8192))
|
|
schema.Append(buildColumnWithName(tblName, "Definer", mysql.TypeVarchar, 128))
|
|
schema.Append(buildColumnWithName(tblName, "character_set_client", mysql.TypeVarchar, 32))
|
|
schema.Append(buildColumnWithName(tblName, "collation_connection", mysql.TypeVarchar, 32))
|
|
schema.Append(buildColumnWithName(tblName, "Database Collation", mysql.TypeVarchar, 32))
|
|
return schema.col2Schema(), schema.names
|
|
}
|
|
|
|
func buildShowEventsSchema() (*expression.Schema, []*types.FieldName) {
|
|
tblName := "EVENTS"
|
|
schema := newColumnsWithNames(15)
|
|
schema.Append(buildColumnWithName(tblName, "Db", mysql.TypeVarchar, 128))
|
|
schema.Append(buildColumnWithName(tblName, "Name", mysql.TypeVarchar, 128))
|
|
schema.Append(buildColumnWithName(tblName, "Time zone", mysql.TypeVarchar, 32))
|
|
schema.Append(buildColumnWithName(tblName, "Definer", mysql.TypeVarchar, 128))
|
|
schema.Append(buildColumnWithName(tblName, "Type", mysql.TypeVarchar, 128))
|
|
schema.Append(buildColumnWithName(tblName, "Execute At", mysql.TypeDatetime, 19))
|
|
schema.Append(buildColumnWithName(tblName, "Interval Value", mysql.TypeVarchar, 128))
|
|
schema.Append(buildColumnWithName(tblName, "Interval Field", mysql.TypeVarchar, 128))
|
|
schema.Append(buildColumnWithName(tblName, "Starts", mysql.TypeDatetime, 19))
|
|
schema.Append(buildColumnWithName(tblName, "Ends", mysql.TypeDatetime, 19))
|
|
schema.Append(buildColumnWithName(tblName, "Status", mysql.TypeVarchar, 32))
|
|
schema.Append(buildColumnWithName(tblName, "Originator", mysql.TypeInt24, 4))
|
|
schema.Append(buildColumnWithName(tblName, "character_set_client", mysql.TypeVarchar, 32))
|
|
schema.Append(buildColumnWithName(tblName, "collation_connection", mysql.TypeVarchar, 32))
|
|
schema.Append(buildColumnWithName(tblName, "Database Collation", mysql.TypeVarchar, 32))
|
|
return schema.col2Schema(), schema.names
|
|
}
|
|
|
|
func buildShowWarningsSchema() (*expression.Schema, types.NameSlice) {
|
|
tblName := "WARNINGS"
|
|
schema := newColumnsWithNames(3)
|
|
schema.Append(buildColumnWithName(tblName, "Level", mysql.TypeVarchar, 64))
|
|
schema.Append(buildColumnWithName(tblName, "Code", mysql.TypeLong, 19))
|
|
schema.Append(buildColumnWithName(tblName, "Message", mysql.TypeVarchar, 64))
|
|
return schema.col2Schema(), schema.names
|
|
}
|
|
|
|
// buildShowSchema builds column info for ShowStmt including column name and type.
|
|
func buildShowSchema(s *ast.ShowStmt, isView bool, isSequence bool) (schema *expression.Schema, outputNames []*types.FieldName) {
|
|
var names []string
|
|
var ftypes []byte
|
|
switch s.Tp {
|
|
case ast.ShowProcedureStatus:
|
|
return buildShowProcedureSchema()
|
|
case ast.ShowTriggers:
|
|
return buildShowTriggerSchema()
|
|
case ast.ShowEvents:
|
|
return buildShowEventsSchema()
|
|
case ast.ShowWarnings, ast.ShowErrors:
|
|
return buildShowWarningsSchema()
|
|
case ast.ShowRegions:
|
|
return buildTableRegionsSchema()
|
|
case ast.ShowEngines:
|
|
names = []string{"Engine", "Support", "Comment", "Transactions", "XA", "Savepoints"}
|
|
case ast.ShowConfig:
|
|
names = []string{"Type", "Instance", "Name", "Value"}
|
|
case ast.ShowDatabases:
|
|
names = []string{"Database"}
|
|
case ast.ShowOpenTables:
|
|
names = []string{"Database", "Table", "In_use", "Name_locked"}
|
|
ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeLong, mysql.TypeLong}
|
|
case ast.ShowTables:
|
|
names = []string{fmt.Sprintf("Tables_in_%s", s.DBName)}
|
|
if s.Full {
|
|
names = append(names, "Table_type")
|
|
}
|
|
case ast.ShowTableStatus:
|
|
names = []string{"Name", "Engine", "Version", "Row_format", "Rows", "Avg_row_length",
|
|
"Data_length", "Max_data_length", "Index_length", "Data_free", "Auto_increment",
|
|
"Create_time", "Update_time", "Check_time", "Collation", "Checksum",
|
|
"Create_options", "Comment"}
|
|
ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeLonglong, mysql.TypeVarchar, mysql.TypeLonglong, mysql.TypeLonglong,
|
|
mysql.TypeLonglong, mysql.TypeLonglong, mysql.TypeLonglong, mysql.TypeLonglong, mysql.TypeLonglong,
|
|
mysql.TypeDatetime, mysql.TypeDatetime, mysql.TypeDatetime, mysql.TypeVarchar, mysql.TypeVarchar,
|
|
mysql.TypeVarchar, mysql.TypeVarchar}
|
|
case ast.ShowColumns:
|
|
names = table.ColDescFieldNames(s.Full)
|
|
case ast.ShowCharset:
|
|
names = []string{"Charset", "Description", "Default collation", "Maxlen"}
|
|
ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeLonglong}
|
|
case ast.ShowVariables, ast.ShowStatus:
|
|
names = []string{"Variable_name", "Value"}
|
|
case ast.ShowCollation:
|
|
names = []string{"Collation", "Charset", "Id", "Default", "Compiled", "Sortlen"}
|
|
ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeLonglong,
|
|
mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeLonglong}
|
|
case ast.ShowCreateTable, ast.ShowCreateSequence:
|
|
if isSequence {
|
|
names = []string{"Sequence", "Create Sequence"}
|
|
} else if isView {
|
|
names = []string{"View", "Create View", "character_set_client", "collation_connection"}
|
|
} else {
|
|
names = []string{"Table", "Create Table"}
|
|
}
|
|
case ast.ShowCreateUser:
|
|
if s.User != nil {
|
|
names = []string{fmt.Sprintf("CREATE USER for %s", s.User)}
|
|
}
|
|
case ast.ShowCreateView:
|
|
names = []string{"View", "Create View", "character_set_client", "collation_connection"}
|
|
case ast.ShowCreateDatabase:
|
|
names = []string{"Database", "Create Database"}
|
|
case ast.ShowDrainerStatus:
|
|
names = []string{"NodeID", "Address", "State", "Max_Commit_Ts", "Update_Time"}
|
|
ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeLonglong, mysql.TypeVarchar}
|
|
case ast.ShowGrants:
|
|
if s.User != nil {
|
|
names = []string{fmt.Sprintf("Grants for %s", s.User)}
|
|
} else {
|
|
// Don't know the name yet, so just say "user"
|
|
names = []string{"Grants for User"}
|
|
}
|
|
case ast.ShowIndex:
|
|
names = []string{"Table", "Non_unique", "Key_name", "Seq_in_index",
|
|
"Column_name", "Collation", "Cardinality", "Sub_part", "Packed",
|
|
"Null", "Index_type", "Comment", "Index_comment", "Visible", "Expression"}
|
|
ftypes = []byte{mysql.TypeVarchar, mysql.TypeLonglong, mysql.TypeVarchar, mysql.TypeLonglong,
|
|
mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeLonglong, mysql.TypeLonglong,
|
|
mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar,
|
|
mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar}
|
|
case ast.ShowPlugins:
|
|
names = []string{"Name", "Status", "Type", "Library", "License", "Version"}
|
|
ftypes = []byte{
|
|
mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar,
|
|
}
|
|
case ast.ShowProcessList:
|
|
names = []string{"Id", "User", "Host", "db", "Command", "Time", "State", "Info"}
|
|
ftypes = []byte{mysql.TypeLonglong, mysql.TypeVarchar, mysql.TypeVarchar,
|
|
mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeLong, mysql.TypeVarchar, mysql.TypeString}
|
|
case ast.ShowPumpStatus:
|
|
names = []string{"NodeID", "Address", "State", "Max_Commit_Ts", "Update_Time"}
|
|
ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeLonglong, mysql.TypeVarchar}
|
|
case ast.ShowStatsMeta:
|
|
names = []string{"Db_name", "Table_name", "Partition_name", "Update_time", "Modify_count", "Row_count"}
|
|
ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeDatetime, mysql.TypeLonglong, mysql.TypeLonglong}
|
|
case ast.ShowStatsHistograms:
|
|
names = []string{"Db_name", "Table_name", "Partition_name", "Column_name", "Is_index", "Update_time", "Distinct_count", "Null_count", "Avg_col_size", "Correlation"}
|
|
ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeTiny, mysql.TypeDatetime,
|
|
mysql.TypeLonglong, mysql.TypeLonglong, mysql.TypeDouble, mysql.TypeDouble}
|
|
case ast.ShowStatsBuckets:
|
|
names = []string{"Db_name", "Table_name", "Partition_name", "Column_name", "Is_index", "Bucket_id", "Count",
|
|
"Repeats", "Lower_Bound", "Upper_Bound"}
|
|
ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeTiny, mysql.TypeLonglong,
|
|
mysql.TypeLonglong, mysql.TypeLonglong, mysql.TypeVarchar, mysql.TypeVarchar}
|
|
case ast.ShowStatsHealthy:
|
|
names = []string{"Db_name", "Table_name", "Partition_name", "Healthy"}
|
|
ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeLonglong}
|
|
case ast.ShowProfiles: // ShowProfiles is deprecated.
|
|
names = []string{"Query_ID", "Duration", "Query"}
|
|
ftypes = []byte{mysql.TypeLong, mysql.TypeDouble, mysql.TypeVarchar}
|
|
case ast.ShowMasterStatus:
|
|
names = []string{"File", "Position", "Binlog_Do_DB", "Binlog_Ignore_DB", "Executed_Gtid_Set"}
|
|
ftypes = []byte{mysql.TypeVarchar, mysql.TypeLonglong, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar}
|
|
case ast.ShowPrivileges:
|
|
names = []string{"Privilege", "Context", "Comment"}
|
|
ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar}
|
|
case ast.ShowBindings:
|
|
names = []string{"Original_sql", "Bind_sql", "Default_db", "Status", "Create_time", "Update_time", "Charset", "Collation", "Source"}
|
|
ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeDatetime, mysql.TypeDatetime, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar}
|
|
case ast.ShowAnalyzeStatus:
|
|
names = []string{"Table_schema", "Table_name", "Partition_name", "Job_info", "Processed_rows", "Start_time", "State"}
|
|
ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeLonglong, mysql.TypeDatetime, mysql.TypeVarchar}
|
|
case ast.ShowBuiltins:
|
|
names = []string{"Supported_builtin_functions"}
|
|
ftypes = []byte{mysql.TypeVarchar}
|
|
case ast.ShowBackups, ast.ShowRestores:
|
|
names = []string{"Destination", "State", "Progress", "Queue_time", "Execution_time", "Finish_time", "Connection"}
|
|
ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeDouble, mysql.TypeDatetime, mysql.TypeDatetime, mysql.TypeDatetime, mysql.TypeLonglong}
|
|
}
|
|
|
|
schema = expression.NewSchema(make([]*expression.Column, 0, len(names))...)
|
|
outputNames = make([]*types.FieldName, 0, len(names))
|
|
for i := range names {
|
|
col := &expression.Column{}
|
|
outputNames = append(outputNames, &types.FieldName{ColName: model.NewCIStr(names[i])})
|
|
// User varchar as the default return column type.
|
|
tp := mysql.TypeVarchar
|
|
if len(ftypes) != 0 && ftypes[i] != mysql.TypeUnspecified {
|
|
tp = ftypes[i]
|
|
}
|
|
fieldType := types.NewFieldType(tp)
|
|
fieldType.Flen, fieldType.Decimal = mysql.GetDefaultFieldLengthAndDecimal(tp)
|
|
fieldType.Charset, fieldType.Collate = types.DefaultCharsetForType(tp)
|
|
col.RetType = fieldType
|
|
schema.Append(col)
|
|
}
|
|
return
|
|
}
|
|
|
|
func buildChecksumTableSchema() (*expression.Schema, []*types.FieldName) {
|
|
schema := newColumnsWithNames(5)
|
|
schema.Append(buildColumnWithName("", "Db_name", mysql.TypeVarchar, 128))
|
|
schema.Append(buildColumnWithName("", "Table_name", mysql.TypeVarchar, 128))
|
|
schema.Append(buildColumnWithName("", "Checksum_crc64_xor", mysql.TypeLonglong, 22))
|
|
schema.Append(buildColumnWithName("", "Total_kvs", mysql.TypeLonglong, 22))
|
|
schema.Append(buildColumnWithName("", "Total_bytes", mysql.TypeLonglong, 22))
|
|
return schema.col2Schema(), schema.names
|
|
}
|
|
|
|
// adjustOverlongViewColname adjusts the overlong outputNames of a view to
|
|
// `new_exp_$off` where `$off` is the offset of the output column, $off starts from 1.
|
|
// There is still some MySQL compatible problems.
|
|
func adjustOverlongViewColname(plan LogicalPlan) {
|
|
outputNames := plan.OutputNames()
|
|
for i := range outputNames {
|
|
if outputName := outputNames[i].ColName.L; len(outputName) > mysql.MaxColumnNameLength {
|
|
outputNames[i].ColName = model.NewCIStr(fmt.Sprintf("name_exp_%d", i+1))
|
|
}
|
|
}
|
|
}
|
|
|