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.
 
 

1328 lines
38 KiB

// Copyright 2018 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"
"fmt"
math2 "math"
"github.com/pingcap/errors"
"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/charset"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/parser/opcode"
"github.com/pingcap/parser/terror"
ptypes "github.com/pingcap/parser/types"
"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/privilege"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/types/parser_driver"
"github.com/pingcap/tidb/util/math"
"github.com/pingcap/tidb/util/plancodec"
"github.com/pingcap/tipb/go-tipb"
)
// PointGetPlan is a fast plan for simple point get.
// When we detect that the statement has a unique equal access condition, this plan is used.
// This plan is much faster to build and to execute because it avoid the optimization and coprocessor cost.
type PointGetPlan struct {
basePlan
dbName string
schema *expression.Schema
TblInfo *model.TableInfo
IndexInfo *model.IndexInfo
PartitionInfo *model.PartitionDefinition
Handle int64
HandleParam *driver.ParamMarkerExpr
IndexValues []types.Datum
IndexValueParams []*driver.ParamMarkerExpr
expr expression.Expression
ctx sessionctx.Context
UnsignedHandle bool
IsTableDual bool
Lock bool
outputNames []*types.FieldName
LockWaitTime int64
partitionColumnPos int
Columns []*model.ColumnInfo
Path *util.AccessPath
}
type nameValuePair struct {
colName string
value types.Datum
param *driver.ParamMarkerExpr
}
// Schema implements the Plan interface.
func (p *PointGetPlan) Schema() *expression.Schema {
return p.schema
}
// attach2Task makes the current physical plan as the father of task's physicalPlan and updates the cost of
// current task. If the child's task is cop task, some operator may close this task and return a new rootTask.
func (p *PointGetPlan) attach2Task(...task) task {
return nil
}
// ToPB converts physical plan to tipb executor.
func (p *PointGetPlan) ToPB(ctx sessionctx.Context, _ kv.StoreType) (*tipb.Executor, error) {
return nil, nil
}
// ExplainInfo implements Plan interface.
func (p *PointGetPlan) ExplainInfo() string {
accessObject, operatorInfo := p.AccessObject(false), p.OperatorInfo(false)
if len(operatorInfo) == 0 {
return accessObject
}
return accessObject + ", " + operatorInfo
}
// ExplainNormalizedInfo implements Plan interface.
func (p *PointGetPlan) ExplainNormalizedInfo() string {
accessObject, operatorInfo := p.AccessObject(true), p.OperatorInfo(true)
if len(operatorInfo) == 0 {
return accessObject
}
return accessObject + ", " + operatorInfo
}
// AccessObject implements dataAccesser interface.
func (p *PointGetPlan) AccessObject(normalized bool) string {
buffer := bytes.NewBufferString("")
tblName := p.TblInfo.Name.O
fmt.Fprintf(buffer, "table:%s", tblName)
if p.PartitionInfo != nil {
if normalized {
fmt.Fprintf(buffer, ", partition:?")
} else {
fmt.Fprintf(buffer, ", partition:%s", p.PartitionInfo.Name.L)
}
}
if p.IndexInfo != nil {
buffer.WriteString(", index:" + p.IndexInfo.Name.O + "(")
for i, idxCol := range p.IndexInfo.Columns {
if tblCol := p.TblInfo.Columns[idxCol.Offset]; tblCol.Hidden {
buffer.WriteString(tblCol.GeneratedExprString)
} else {
buffer.WriteString(idxCol.Name.O)
}
if i+1 < len(p.IndexInfo.Columns) {
buffer.WriteString(", ")
}
}
buffer.WriteString(")")
}
return buffer.String()
}
// OperatorInfo implements dataAccesser interface.
func (p *PointGetPlan) OperatorInfo(normalized bool) string {
buffer := bytes.NewBufferString("")
if p.IndexInfo == nil {
if normalized {
fmt.Fprintf(buffer, "handle:?, ")
} else {
if p.UnsignedHandle {
fmt.Fprintf(buffer, "handle:%d, ", uint64(p.Handle))
} else {
fmt.Fprintf(buffer, "handle:%d, ", p.Handle)
}
}
}
if p.Lock {
fmt.Fprintf(buffer, "lock, ")
}
if buffer.Len() >= 2 {
buffer.Truncate(buffer.Len() - 2)
}
return buffer.String()
}
// GetChildReqProps gets the required property by child index.
func (p *PointGetPlan) GetChildReqProps(idx int) *property.PhysicalProperty {
return nil
}
// StatsCount will return the the RowCount of property.StatsInfo for this plan.
func (p *PointGetPlan) StatsCount() float64 {
return 1
}
// statsInfo will return the the RowCount of property.StatsInfo for this plan.
func (p *PointGetPlan) statsInfo() *property.StatsInfo {
if p.stats == nil {
p.stats = &property.StatsInfo{}
}
p.stats.RowCount = 1
return p.stats
}
// Children gets all the children.
func (p *PointGetPlan) Children() []PhysicalPlan {
return nil
}
// SetChildren sets the children for the plan.
func (p *PointGetPlan) SetChildren(...PhysicalPlan) {}
// SetChild sets a specific child for the plan.
func (p *PointGetPlan) SetChild(i int, child PhysicalPlan) {}
// ResolveIndices resolves the indices for columns. After doing this, the columns can evaluate the rows by their indices.
func (p *PointGetPlan) ResolveIndices() error {
return resolveIndicesForVirtualColumn(p.schema.Columns, p.schema)
}
// OutputNames returns the outputting names of each column.
func (p *PointGetPlan) OutputNames() types.NameSlice {
return p.outputNames
}
// SetOutputNames sets the outputting name by the given slice.
func (p *PointGetPlan) SetOutputNames(names types.NameSlice) {
p.outputNames = names
}
// GetCost returns cost of the PointGetPlan.
func (p *PointGetPlan) GetCost(cols []*expression.Column) float64 {
sessVars := p.ctx.GetSessionVars()
var rowSize float64
cost := 0.0
if p.IndexInfo == nil {
rowSize = p.stats.HistColl.GetTableAvgRowSize(p.ctx, cols, kv.TiKV, true)
} else {
rowSize = p.stats.HistColl.GetIndexAvgRowSize(p.ctx, cols, p.IndexInfo.Unique)
}
cost += rowSize * sessVars.NetworkFactor
cost += sessVars.SeekFactor
cost /= float64(sessVars.DistSQLScanConcurrency)
return cost
}
// BatchPointGetPlan represents a physical plan which contains a bunch of
// keys reference the same table and use the same `unique key`
type BatchPointGetPlan struct {
baseSchemaProducer
ctx sessionctx.Context
dbName string
TblInfo *model.TableInfo
IndexInfo *model.IndexInfo
Handles []int64
HandleParams []*driver.ParamMarkerExpr
IndexValues [][]types.Datum
IndexValueParams [][]*driver.ParamMarkerExpr
PartitionColPos int
KeepOrder bool
Desc bool
Lock bool
LockWaitTime int64
Columns []*model.ColumnInfo
Path *util.AccessPath
}
// attach2Task makes the current physical plan as the father of task's physicalPlan and updates the cost of
// current task. If the child's task is cop task, some operator may close this task and return a new rootTask.
func (p *BatchPointGetPlan) attach2Task(...task) task {
return nil
}
// ToPB converts physical plan to tipb executor.
func (p *BatchPointGetPlan) ToPB(ctx sessionctx.Context, _ kv.StoreType) (*tipb.Executor, error) {
return nil, nil
}
// ExplainInfo implements Plan interface.
func (p *BatchPointGetPlan) ExplainInfo() string {
return p.AccessObject(false) + ", " + p.OperatorInfo(false)
}
// ExplainNormalizedInfo implements Plan interface.
func (p *BatchPointGetPlan) ExplainNormalizedInfo() string {
return p.AccessObject(true) + ", " + p.OperatorInfo(true)
}
// AccessObject implements physicalScan interface.
func (p *BatchPointGetPlan) AccessObject(_ bool) string {
buffer := bytes.NewBufferString("")
tblName := p.TblInfo.Name.O
fmt.Fprintf(buffer, "table:%s", tblName)
if p.IndexInfo != nil {
buffer.WriteString(", index:" + p.IndexInfo.Name.O + "(")
for i, idxCol := range p.IndexInfo.Columns {
if tblCol := p.TblInfo.Columns[idxCol.Offset]; tblCol.Hidden {
buffer.WriteString(tblCol.GeneratedExprString)
} else {
buffer.WriteString(idxCol.Name.O)
}
if i+1 < len(p.IndexInfo.Columns) {
buffer.WriteString(", ")
}
}
buffer.WriteString(")")
}
return buffer.String()
}
// OperatorInfo implements dataAccesser interface.
func (p *BatchPointGetPlan) OperatorInfo(normalized bool) string {
buffer := bytes.NewBufferString("")
if p.IndexInfo == nil {
if normalized {
fmt.Fprintf(buffer, "handle:?, ")
} else {
fmt.Fprintf(buffer, "handle:%v, ", p.Handles)
}
}
fmt.Fprintf(buffer, "keep order:%v, ", p.KeepOrder)
fmt.Fprintf(buffer, "desc:%v, ", p.Desc)
if p.Lock {
fmt.Fprintf(buffer, "lock, ")
}
if buffer.Len() >= 2 {
buffer.Truncate(buffer.Len() - 2)
}
return buffer.String()
}
// GetChildReqProps gets the required property by child index.
func (p *BatchPointGetPlan) GetChildReqProps(idx int) *property.PhysicalProperty {
return nil
}
// StatsCount will return the the RowCount of property.StatsInfo for this plan.
func (p *BatchPointGetPlan) StatsCount() float64 {
return p.statsInfo().RowCount
}
// statsInfo will return the the RowCount of property.StatsInfo for this plan.
func (p *BatchPointGetPlan) statsInfo() *property.StatsInfo {
return p.stats
}
// Children gets all the children.
func (p *BatchPointGetPlan) Children() []PhysicalPlan {
return nil
}
// SetChildren sets the children for the plan.
func (p *BatchPointGetPlan) SetChildren(...PhysicalPlan) {}
// SetChild sets a specific child for the plan.
func (p *BatchPointGetPlan) SetChild(i int, child PhysicalPlan) {}
// ResolveIndices resolves the indices for columns. After doing this, the columns can evaluate the rows by their indices.
func (p *BatchPointGetPlan) ResolveIndices() error {
return resolveIndicesForVirtualColumn(p.schema.Columns, p.schema)
}
// OutputNames returns the outputting names of each column.
func (p *BatchPointGetPlan) OutputNames() types.NameSlice {
return p.names
}
// SetOutputNames sets the outputting name by the given slice.
func (p *BatchPointGetPlan) SetOutputNames(names types.NameSlice) {
p.names = names
}
// GetCost returns cost of the PointGetPlan.
func (p *BatchPointGetPlan) GetCost(cols []*expression.Column) float64 {
sessVars := p.ctx.GetSessionVars()
var rowSize, rowCount float64
cost := 0.0
if p.IndexInfo == nil {
rowCount = float64(len(p.Handles))
rowSize = p.stats.HistColl.GetTableAvgRowSize(p.ctx, cols, kv.TiKV, true)
} else {
rowCount = float64(len(p.IndexValues))
rowSize = p.stats.HistColl.GetIndexAvgRowSize(p.ctx, cols, p.IndexInfo.Unique)
}
cost += rowCount * rowSize * sessVars.NetworkFactor
cost += rowCount * sessVars.SeekFactor
cost /= float64(sessVars.DistSQLScanConcurrency)
return cost
}
// TryFastPlan tries to use the PointGetPlan for the query.
func TryFastPlan(ctx sessionctx.Context, node ast.Node) (p Plan) {
ctx.GetSessionVars().PlanID = 0
ctx.GetSessionVars().PlanColumnID = 0
switch x := node.(type) {
case *ast.SelectStmt:
defer func() {
if ctx.GetSessionVars().SelectLimit != math2.MaxUint64 && p != nil {
ctx.GetSessionVars().StmtCtx.AppendWarning(errors.New("sql_select_limit is set, so point get plan is not activated"))
p = nil
}
}()
// Try to convert the `SELECT a, b, c FROM t WHERE (a, b, c) in ((1, 2, 4), (1, 3, 5))` to
// `PhysicalUnionAll` which children are `PointGet` if exists an unique key (a, b, c) in table `t`
if fp := tryWhereIn2BatchPointGet(ctx, x); fp != nil {
if checkFastPlanPrivilege(ctx, fp.dbName, fp.TblInfo.Name.L, mysql.SelectPriv) != nil {
return
}
fp.Lock, fp.LockWaitTime = getLockWaitTime(ctx, x.LockTp)
p = fp
return
}
if fp := tryPointGetPlan(ctx, x); fp != nil {
if checkFastPlanPrivilege(ctx, fp.dbName, fp.TblInfo.Name.L, mysql.SelectPriv) != nil {
return nil
}
if fp.IsTableDual {
tableDual := PhysicalTableDual{}
tableDual.names = fp.outputNames
tableDual.SetSchema(fp.Schema())
p = tableDual.Init(ctx, &property.StatsInfo{}, 0)
return
}
fp.Lock, fp.LockWaitTime = getLockWaitTime(ctx, x.LockTp)
p = fp
return
}
case *ast.UpdateStmt:
return tryUpdatePointPlan(ctx, x)
case *ast.DeleteStmt:
return tryDeletePointPlan(ctx, x)
}
return nil
}
func getLockWaitTime(ctx sessionctx.Context, lockTp ast.SelectLockType) (lock bool, waitTime int64) {
if lockTp == ast.SelectLockForUpdate || lockTp == ast.SelectLockForUpdateNoWait {
// Locking of rows for update using SELECT FOR UPDATE only applies when autocommit
// is disabled (either by beginning transaction with START TRANSACTION or by setting
// autocommit to 0. If autocommit is enabled, the rows matching the specification are not locked.
// See https://dev.mysql.com/doc/refman/5.7/en/innodb-locking-reads.html
sessVars := ctx.GetSessionVars()
if !sessVars.IsAutocommit() || sessVars.InTxn() {
lock = true
waitTime = sessVars.LockWaitTimeout
if lockTp == ast.SelectLockForUpdateNoWait {
waitTime = kv.LockNoWait
}
}
}
return
}
func newBatchPointGetPlan(
ctx sessionctx.Context, patternInExpr *ast.PatternInExpr,
handleCol *model.ColumnInfo, tbl *model.TableInfo, schema *expression.Schema,
names []*types.FieldName, whereColNames []string,
) *BatchPointGetPlan {
statsInfo := &property.StatsInfo{RowCount: float64(len(patternInExpr.List))}
var partitionColName *ast.ColumnName
if tbl.GetPartitionInfo() != nil {
partitionColName = getHashPartitionColumnName(ctx, tbl)
if partitionColName == nil {
return nil
}
}
if handleCol != nil {
var handles = make([]int64, len(patternInExpr.List))
var handleParams = make([]*driver.ParamMarkerExpr, len(patternInExpr.List))
for i, item := range patternInExpr.List {
// SELECT * FROM t WHERE (key) in ((1), (2))
if p, ok := item.(*ast.ParenthesesExpr); ok {
item = p.Expr
}
var d types.Datum
var param *driver.ParamMarkerExpr
switch x := item.(type) {
case *driver.ValueExpr:
d = x.Datum
case *driver.ParamMarkerExpr:
d = x.Datum
param = x
default:
return nil
}
if d.IsNull() {
return nil
}
if !checkCanConvertInPointGet(handleCol, d) {
return nil
}
intDatum, err := d.ConvertTo(ctx.GetSessionVars().StmtCtx, &handleCol.FieldType)
if err != nil {
return nil
}
// The converted result must be same as original datum
cmp, err := intDatum.CompareDatum(ctx.GetSessionVars().StmtCtx, &d)
if err != nil || cmp != 0 {
return nil
}
handles[i] = intDatum.GetInt64()
handleParams[i] = param
}
return BatchPointGetPlan{
TblInfo: tbl,
Handles: handles,
HandleParams: handleParams,
}.Init(ctx, statsInfo, schema, names, 0)
}
// The columns in where clause should be covered by unique index
var matchIdxInfo *model.IndexInfo
permutations := make([]int, len(whereColNames))
colInfos := make([]*model.ColumnInfo, len(whereColNames))
for i, innerCol := range whereColNames {
for _, col := range tbl.Columns {
if col.Name.L == innerCol {
colInfos[i] = col
}
}
}
for _, idxInfo := range tbl.Indices {
if !idxInfo.Unique || idxInfo.State != model.StatePublic {
continue
}
if len(idxInfo.Columns) != len(whereColNames) || idxInfo.HasPrefixIndex() {
continue
}
// TODO: not sure is there any function to reuse
matched := true
for whereColIndex, innerCol := range whereColNames {
var found bool
for i, col := range idxInfo.Columns {
if innerCol == col.Name.L {
permutations[whereColIndex] = i
found = true
break
}
}
if !found {
matched = false
break
}
}
if matched {
matchIdxInfo = idxInfo
break
}
}
if matchIdxInfo == nil {
return nil
}
indexValues := make([][]types.Datum, len(patternInExpr.List))
indexValueParams := make([][]*driver.ParamMarkerExpr, len(patternInExpr.List))
for i, item := range patternInExpr.List {
// SELECT * FROM t WHERE (key) in ((1), (2))
if p, ok := item.(*ast.ParenthesesExpr); ok {
item = p.Expr
}
var values []types.Datum
var valuesParams []*driver.ParamMarkerExpr
switch x := item.(type) {
case *ast.RowExpr:
// The `len(values) == len(valuesParams)` should be satisfied in this mode
if len(x.Values) != len(whereColNames) {
return nil
}
values = make([]types.Datum, len(x.Values))
valuesParams = make([]*driver.ParamMarkerExpr, len(x.Values))
for index, inner := range x.Values {
permIndex := permutations[index]
switch innerX := inner.(type) {
case *driver.ValueExpr:
if !checkCanConvertInPointGet(colInfos[index], innerX.Datum) {
return nil
}
values[permIndex] = innerX.Datum
case *driver.ParamMarkerExpr:
if !checkCanConvertInPointGet(colInfos[index], innerX.Datum) {
return nil
}
values[permIndex] = innerX.Datum
valuesParams[permIndex] = innerX
default:
return nil
}
}
case *driver.ValueExpr:
if !checkCanConvertInPointGet(colInfos[0], x.Datum) {
return nil
}
values = []types.Datum{x.Datum}
case *driver.ParamMarkerExpr:
if !checkCanConvertInPointGet(colInfos[0], x.Datum) {
return nil
}
values = []types.Datum{x.Datum}
valuesParams = []*driver.ParamMarkerExpr{x}
default:
return nil
}
indexValues[i] = values
indexValueParams[i] = valuesParams
}
return BatchPointGetPlan{
TblInfo: tbl,
IndexInfo: matchIdxInfo,
IndexValues: indexValues,
IndexValueParams: indexValueParams,
PartitionColPos: getPartitionColumnPos(matchIdxInfo, partitionColName),
}.Init(ctx, statsInfo, schema, names, 0)
}
func tryWhereIn2BatchPointGet(ctx sessionctx.Context, selStmt *ast.SelectStmt) *BatchPointGetPlan {
if selStmt.OrderBy != nil || selStmt.GroupBy != nil ||
selStmt.Limit != nil || selStmt.Having != nil ||
len(selStmt.WindowSpecs) > 0 {
return nil
}
in, ok := selStmt.Where.(*ast.PatternInExpr)
if !ok || in.Not || len(in.List) < 1 {
return nil
}
tblName, tblAlias := getSingleTableNameAndAlias(selStmt.From)
if tblName == nil {
return nil
}
tbl := tblName.TableInfo
if tbl == nil {
return nil
}
// Skip the optimization with partition selection.
if len(tblName.PartitionNames) > 0 {
return nil
}
for _, col := range tbl.Columns {
if col.IsGenerated() || col.State != model.StatePublic {
return nil
}
}
schema, names := buildSchemaFromFields(tblName.Schema, tbl, tblAlias, selStmt.Fields.Fields)
if schema == nil {
return nil
}
var (
handleCol *model.ColumnInfo
whereColNames []string
)
// SELECT * FROM t WHERE (key) in ((1), (2))
colExpr := in.Expr
if p, ok := colExpr.(*ast.ParenthesesExpr); ok {
colExpr = p.Expr
}
switch colName := colExpr.(type) {
case *ast.ColumnNameExpr:
if name := colName.Name.Table.L; name != "" && name != tblAlias.L {
return nil
}
// Try use handle
if tbl.PKIsHandle {
for _, col := range tbl.Columns {
if mysql.HasPriKeyFlag(col.Flag) && col.Name.L == colName.Name.Name.L {
handleCol = col
whereColNames = append(whereColNames, col.Name.L)
break
}
}
}
if handleCol == nil {
// Downgrade to use unique index
whereColNames = append(whereColNames, colName.Name.Name.L)
}
case *ast.RowExpr:
for _, col := range colName.Values {
c, ok := col.(*ast.ColumnNameExpr)
if !ok {
return nil
}
if name := c.Name.Table.L; name != "" && name != tblAlias.L {
return nil
}
whereColNames = append(whereColNames, c.Name.Name.L)
}
default:
return nil
}
p := newBatchPointGetPlan(ctx, in, handleCol, tbl, schema, names, whereColNames)
if p == nil {
return nil
}
p.dbName = tblName.Schema.L
if p.dbName == "" {
p.dbName = ctx.GetSessionVars().CurrentDB
}
return p
}
// tryPointGetPlan determine if the SelectStmt can use a PointGetPlan.
// Returns nil if not applicable.
// To use the PointGetPlan the following rules must be satisfied:
// 1. For the limit clause, the count should at least 1 and the offset is 0.
// 2. It must be a single table select.
// 3. All the columns must be public and generated.
// 4. The condition is an access path that the range is a unique key.
func tryPointGetPlan(ctx sessionctx.Context, selStmt *ast.SelectStmt) *PointGetPlan {
if selStmt.Having != nil {
return nil
} else if selStmt.Limit != nil {
count, offset, err := extractLimitCountOffset(ctx, selStmt.Limit)
if err != nil || count == 0 || offset > 0 {
return nil
}
}
tblName, tblAlias := getSingleTableNameAndAlias(selStmt.From)
if tblName == nil {
return nil
}
tbl := tblName.TableInfo
if tbl == nil {
return nil
}
pi := tbl.GetPartitionInfo()
if pi != nil && pi.Type != model.PartitionTypeHash {
return nil
}
for _, col := range tbl.Columns {
// Do not handle generated columns.
if col.IsGenerated() {
return nil
}
// Only handle tables that all columns are public.
if col.State != model.StatePublic {
return nil
}
}
schema, names := buildSchemaFromFields(tblName.Schema, tbl, tblAlias, selStmt.Fields.Fields)
if schema == nil {
return nil
}
dbName := tblName.Schema.L
if dbName == "" {
dbName = ctx.GetSessionVars().CurrentDB
}
pairs := make([]nameValuePair, 0, 4)
pairs, isTableDual := getNameValuePairs(ctx.GetSessionVars().StmtCtx, tbl, tblAlias, pairs, selStmt.Where)
if pairs == nil && !isTableDual {
return nil
}
var partitionInfo *model.PartitionDefinition
var pos int
if pi != nil {
partitionInfo, pos = getPartitionInfo(ctx, tbl, pairs)
if partitionInfo == nil {
return nil
}
// Take partition selection into consideration.
if len(tblName.PartitionNames) > 0 {
if !partitionNameInSet(partitionInfo.Name, tblName.PartitionNames) {
p := newPointGetPlan(ctx, tblName.Schema.O, schema, tbl, names)
p.IsTableDual = true
return p
}
}
}
handlePair, fieldType := findPKHandle(tbl, pairs)
if handlePair.value.Kind() != types.KindNull && len(pairs) == 1 {
if isTableDual {
p := newPointGetPlan(ctx, tblName.Schema.O, schema, tbl, names)
p.IsTableDual = true
return p
}
p := newPointGetPlan(ctx, dbName, schema, tbl, names)
p.Handle = handlePair.value.GetInt64()
p.UnsignedHandle = mysql.HasUnsignedFlag(fieldType.Flag)
p.HandleParam = handlePair.param
p.PartitionInfo = partitionInfo
return p
}
for _, idxInfo := range tbl.Indices {
if !idxInfo.Unique {
continue
}
if idxInfo.State != model.StatePublic {
continue
}
if isTableDual {
p := newPointGetPlan(ctx, tblName.Schema.O, schema, tbl, names)
p.IsTableDual = true
return p
}
idxValues, idxValueParams := getIndexValues(idxInfo, pairs)
if idxValues == nil {
continue
}
p := newPointGetPlan(ctx, dbName, schema, tbl, names)
p.IndexInfo = idxInfo
p.IndexValues = idxValues
p.IndexValueParams = idxValueParams
p.PartitionInfo = partitionInfo
if p.PartitionInfo != nil {
p.partitionColumnPos = findPartitionIdx(idxInfo, pos, pairs)
}
return p
}
return nil
}
func partitionNameInSet(name model.CIStr, pnames []model.CIStr) bool {
for _, pname := range pnames {
// Case insensitive, create table partition p0, query using P0 is OK.
if name.L == pname.L {
return true
}
}
return false
}
func newPointGetPlan(ctx sessionctx.Context, dbName string, schema *expression.Schema, tbl *model.TableInfo, names []*types.FieldName) *PointGetPlan {
p := &PointGetPlan{
basePlan: newBasePlan(ctx, plancodec.TypePointGet, 0),
dbName: dbName,
schema: schema,
TblInfo: tbl,
outputNames: names,
LockWaitTime: ctx.GetSessionVars().LockWaitTimeout,
}
ctx.GetSessionVars().StmtCtx.Tables = []stmtctx.TableEntry{{DB: dbName, Table: tbl.Name.L}}
return p
}
func checkFastPlanPrivilege(ctx sessionctx.Context, dbName, tableName string, checkTypes ...mysql.PrivilegeType) error {
pm := privilege.GetPrivilegeManager(ctx)
visitInfos := []visitInfo{}
for _, checkType := range checkTypes {
if pm != nil && !pm.RequestVerification(ctx.GetSessionVars().ActiveRoles, dbName, tableName, "", checkType) {
return errors.New("privilege check fail")
}
// This visitInfo is only for table lock check, so we do not need column field,
// just fill it empty string.
visitInfos = append(visitInfos, visitInfo{
privilege: checkType,
db: dbName,
table: tableName,
column: "",
err: nil,
})
}
infoSchema := infoschema.GetInfoSchema(ctx)
return CheckTableLock(ctx, infoSchema, visitInfos)
}
func buildSchemaFromFields(
dbName model.CIStr,
tbl *model.TableInfo,
tblName model.CIStr,
fields []*ast.SelectField,
) (
*expression.Schema,
[]*types.FieldName,
) {
columns := make([]*expression.Column, 0, len(tbl.Columns)+1)
names := make([]*types.FieldName, 0, len(tbl.Columns)+1)
if len(fields) > 0 {
for _, field := range fields {
if field.WildCard != nil {
if field.WildCard.Table.L != "" && field.WildCard.Table.L != tblName.L {
return nil, nil
}
for _, col := range tbl.Columns {
names = append(names, &types.FieldName{
DBName: dbName,
OrigTblName: tbl.Name,
TblName: tblName,
ColName: col.Name,
})
columns = append(columns, colInfoToColumn(col, len(columns)))
}
continue
}
colNameExpr, ok := field.Expr.(*ast.ColumnNameExpr)
if !ok {
return nil, nil
}
if colNameExpr.Name.Table.L != "" && colNameExpr.Name.Table.L != tblName.L {
return nil, nil
}
col := findCol(tbl, colNameExpr.Name)
if col == nil {
return nil, nil
}
asName := col.Name
if field.AsName.L != "" {
asName = field.AsName
}
names = append(names, &types.FieldName{
DBName: dbName,
OrigTblName: tbl.Name,
TblName: tblName,
ColName: asName,
})
columns = append(columns, colInfoToColumn(col, len(columns)))
}
return expression.NewSchema(columns...), names
}
// fields len is 0 for update and delete.
for _, col := range tbl.Columns {
names = append(names, &types.FieldName{
DBName: dbName,
OrigTblName: tbl.Name,
TblName: tblName,
ColName: col.Name,
})
column := colInfoToColumn(col, len(columns))
columns = append(columns, column)
}
schema := expression.NewSchema(columns...)
return schema, names
}
// getSingleTableNameAndAlias return the ast node of queried table name and the alias string.
// `tblName` is `nil` if there are multiple tables in the query.
// `tblAlias` will be the real table name if there is no table alias in the query.
func getSingleTableNameAndAlias(tableRefs *ast.TableRefsClause) (tblName *ast.TableName, tblAlias model.CIStr) {
if tableRefs == nil || tableRefs.TableRefs == nil || tableRefs.TableRefs.Right != nil {
return nil, tblAlias
}
tblSrc, ok := tableRefs.TableRefs.Left.(*ast.TableSource)
if !ok {
return nil, tblAlias
}
tblName, ok = tblSrc.Source.(*ast.TableName)
if !ok {
return nil, tblAlias
}
tblAlias = tblSrc.AsName
if tblSrc.AsName.L == "" {
tblAlias = tblName.Name
}
return tblName, tblAlias
}
// getNameValuePairs extracts `column = constant/paramMarker` conditions from expr as name value pairs.
func getNameValuePairs(stmtCtx *stmtctx.StatementContext, tbl *model.TableInfo, tblName model.CIStr, nvPairs []nameValuePair, expr ast.ExprNode) (
pairs []nameValuePair, isTableDual bool) {
binOp, ok := expr.(*ast.BinaryOperationExpr)
if !ok {
return nil, false
}
if binOp.Op == opcode.LogicAnd {
nvPairs, isTableDual = getNameValuePairs(stmtCtx, tbl, tblName, nvPairs, binOp.L)
if nvPairs == nil || isTableDual {
return nil, isTableDual
}
nvPairs, isTableDual = getNameValuePairs(stmtCtx, tbl, tblName, nvPairs, binOp.R)
if nvPairs == nil || isTableDual {
return nil, isTableDual
}
return nvPairs, isTableDual
} else if binOp.Op == opcode.EQ {
var d types.Datum
var colName *ast.ColumnNameExpr
var param *driver.ParamMarkerExpr
var ok bool
if colName, ok = binOp.L.(*ast.ColumnNameExpr); ok {
switch x := binOp.R.(type) {
case *driver.ValueExpr:
d = x.Datum
case *driver.ParamMarkerExpr:
d = x.Datum
param = x
}
} else if colName, ok = binOp.R.(*ast.ColumnNameExpr); ok {
switch x := binOp.L.(type) {
case *driver.ValueExpr:
d = x.Datum
case *driver.ParamMarkerExpr:
d = x.Datum
param = x
}
} else {
return nil, false
}
if d.IsNull() {
return nil, false
}
// Views' columns have no FieldType.
if tbl.IsView() {
return nil, false
}
if colName.Name.Table.L != "" && colName.Name.Table.L != tblName.L {
return nil, false
}
col := model.FindColumnInfo(tbl.Cols(), colName.Name.Name.L)
if col == nil || // Handling the case when the column is _tidb_rowid.
(col.Tp == mysql.TypeString && col.Collate == charset.CollationBin) { // This type we needn't to pad `\0` in here.
return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, value: d, param: param}), false
}
if !checkCanConvertInPointGet(col, d) {
return nil, false
}
dVal, err := d.ConvertTo(stmtCtx, &col.FieldType)
if err != nil {
if terror.ErrorEqual(types.ErrOverflow, err) {
return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, value: d, param: param}), true
}
// Some scenarios cast to int with error, but we may use this value in point get.
if !terror.ErrorEqual(types.ErrTruncatedWrongVal, err) {
return nil, false
}
}
// The converted result must be same as original datum.
cmp, err := d.CompareDatum(stmtCtx, &dVal)
if err != nil {
return nil, false
} else if cmp != 0 {
return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, value: dVal, param: param}), true
}
return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, value: dVal, param: param}), false
}
return nil, false
}
func checkCanConvertInPointGet(col *model.ColumnInfo, d types.Datum) bool {
kind := d.Kind()
switch col.FieldType.EvalType() {
case ptypes.ETString:
switch kind {
case types.KindInt64, types.KindUint64,
types.KindFloat32, types.KindFloat64, types.KindMysqlDecimal:
// column type is String and constant type is numeric
return false
}
}
switch col.FieldType.Tp {
case mysql.TypeBit:
switch kind {
case types.KindString:
// column type is Bit and constant type is string
return false
}
}
return true
}
func findPKHandle(tblInfo *model.TableInfo, pairs []nameValuePair) (handlePair nameValuePair, fieldType *types.FieldType) {
if !tblInfo.PKIsHandle {
rowIDIdx := findInPairs("_tidb_rowid", pairs)
if rowIDIdx != -1 {
return pairs[rowIDIdx], types.NewFieldType(mysql.TypeLonglong)
}
return handlePair, nil
}
for _, col := range tblInfo.Columns {
if mysql.HasPriKeyFlag(col.Flag) {
i := findInPairs(col.Name.L, pairs)
if i == -1 {
return handlePair, nil
}
return pairs[i], &col.FieldType
}
}
return handlePair, nil
}
func getIndexValues(idxInfo *model.IndexInfo, pairs []nameValuePair) ([]types.Datum, []*driver.ParamMarkerExpr) {
idxValues := make([]types.Datum, 0, 4)
idxValueParams := make([]*driver.ParamMarkerExpr, 0, 4)
if len(idxInfo.Columns) != len(pairs) {
return nil, nil
}
if idxInfo.HasPrefixIndex() {
return nil, nil
}
for _, idxCol := range idxInfo.Columns {
i := findInPairs(idxCol.Name.L, pairs)
if i == -1 {
return nil, nil
}
idxValues = append(idxValues, pairs[i].value)
idxValueParams = append(idxValueParams, pairs[i].param)
}
if len(idxValues) > 0 {
return idxValues, idxValueParams
}
return nil, nil
}
func findInPairs(colName string, pairs []nameValuePair) int {
for i, pair := range pairs {
if pair.colName == colName {
return i
}
}
return -1
}
func tryUpdatePointPlan(ctx sessionctx.Context, updateStmt *ast.UpdateStmt) Plan {
selStmt := &ast.SelectStmt{
Fields: &ast.FieldList{},
From: updateStmt.TableRefs,
Where: updateStmt.Where,
OrderBy: updateStmt.Order,
Limit: updateStmt.Limit,
}
pointGet := tryPointGetPlan(ctx, selStmt)
if pointGet != nil {
if pointGet.IsTableDual {
return PhysicalTableDual{
names: pointGet.outputNames,
}.Init(ctx, &property.StatsInfo{}, 0)
}
if ctx.GetSessionVars().TxnCtx.IsPessimistic {
pointGet.Lock, pointGet.LockWaitTime = getLockWaitTime(ctx, ast.SelectLockForUpdate)
}
return buildPointUpdatePlan(ctx, pointGet, pointGet.dbName, pointGet.TblInfo, updateStmt)
}
batchPointGet := tryWhereIn2BatchPointGet(ctx, selStmt)
if batchPointGet != nil {
if ctx.GetSessionVars().TxnCtx.IsPessimistic {
batchPointGet.Lock, batchPointGet.LockWaitTime = getLockWaitTime(ctx, ast.SelectLockForUpdate)
}
return buildPointUpdatePlan(ctx, batchPointGet, batchPointGet.dbName, batchPointGet.TblInfo, updateStmt)
}
return nil
}
func buildPointUpdatePlan(ctx sessionctx.Context, pointPlan PhysicalPlan, dbName string, tbl *model.TableInfo, updateStmt *ast.UpdateStmt) Plan {
if checkFastPlanPrivilege(ctx, dbName, tbl.Name.L, mysql.SelectPriv, mysql.UpdatePriv) != nil {
return nil
}
orderedList, allAssignmentsAreConstant := buildOrderedList(ctx, pointPlan, updateStmt.List)
if orderedList == nil {
return nil
}
handleCol := findHandleCol(tbl, pointPlan.Schema())
updatePlan := Update{
SelectPlan: pointPlan,
OrderedList: orderedList,
TblColPosInfos: TblColPosInfoSlice{
TblColPosInfo{
TblID: tbl.ID,
Start: 0,
End: pointPlan.Schema().Len(),
HandleOrdinal: handleCol.Index,
},
},
AllAssignmentsAreConstant: allAssignmentsAreConstant,
}.Init(ctx)
updatePlan.names = pointPlan.OutputNames()
return updatePlan
}
func buildOrderedList(ctx sessionctx.Context, plan Plan, list []*ast.Assignment,
) (orderedList []*expression.Assignment, allAssignmentsAreConstant bool) {
orderedList = make([]*expression.Assignment, 0, len(list))
allAssignmentsAreConstant = true
for _, assign := range list {
idx, err := expression.FindFieldName(plan.OutputNames(), assign.Column)
if idx == -1 || err != nil {
return nil, true
}
col := plan.Schema().Columns[idx]
newAssign := &expression.Assignment{
Col: col,
ColName: plan.OutputNames()[idx].ColName,
}
expr, err := expression.RewriteSimpleExprWithNames(ctx, assign.Expr, plan.Schema(), plan.OutputNames())
if err != nil {
return nil, true
}
expr = expression.BuildCastFunction(ctx, expr, col.GetType())
if allAssignmentsAreConstant {
_, isConst := expr.(*expression.Constant)
allAssignmentsAreConstant = isConst
}
newAssign.Expr, err = expr.ResolveIndices(plan.Schema())
if err != nil {
return nil, true
}
orderedList = append(orderedList, newAssign)
}
return orderedList, allAssignmentsAreConstant
}
func tryDeletePointPlan(ctx sessionctx.Context, delStmt *ast.DeleteStmt) Plan {
if delStmt.IsMultiTable {
return nil
}
selStmt := &ast.SelectStmt{
Fields: &ast.FieldList{},
From: delStmt.TableRefs,
Where: delStmt.Where,
OrderBy: delStmt.Order,
Limit: delStmt.Limit,
}
if pointGet := tryPointGetPlan(ctx, selStmt); pointGet != nil {
if pointGet.IsTableDual {
return PhysicalTableDual{
names: pointGet.outputNames,
}.Init(ctx, &property.StatsInfo{}, 0)
}
if ctx.GetSessionVars().TxnCtx.IsPessimistic {
pointGet.Lock, pointGet.LockWaitTime = getLockWaitTime(ctx, ast.SelectLockForUpdate)
}
return buildPointDeletePlan(ctx, pointGet, pointGet.dbName, pointGet.TblInfo)
}
if batchPointGet := tryWhereIn2BatchPointGet(ctx, selStmt); batchPointGet != nil {
if ctx.GetSessionVars().TxnCtx.IsPessimistic {
batchPointGet.Lock, batchPointGet.LockWaitTime = getLockWaitTime(ctx, ast.SelectLockForUpdate)
}
return buildPointDeletePlan(ctx, batchPointGet, batchPointGet.dbName, batchPointGet.TblInfo)
}
return nil
}
func buildPointDeletePlan(ctx sessionctx.Context, pointPlan PhysicalPlan, dbName string, tbl *model.TableInfo) Plan {
if checkFastPlanPrivilege(ctx, dbName, tbl.Name.L, mysql.SelectPriv, mysql.DeletePriv) != nil {
return nil
}
handleCol := findHandleCol(tbl, pointPlan.Schema())
delPlan := Delete{
SelectPlan: pointPlan,
TblColPosInfos: TblColPosInfoSlice{
TblColPosInfo{
TblID: tbl.ID,
Start: 0,
End: pointPlan.Schema().Len(),
HandleOrdinal: handleCol.Index,
},
},
}.Init(ctx)
return delPlan
}
func findCol(tbl *model.TableInfo, colName *ast.ColumnName) *model.ColumnInfo {
for _, col := range tbl.Columns {
if col.Name.L == colName.Name.L {
return col
}
}
return nil
}
func colInfoToColumn(col *model.ColumnInfo, idx int) *expression.Column {
return &expression.Column{
RetType: col.FieldType.Clone(),
ID: col.ID,
UniqueID: int64(col.Offset),
Index: idx,
OrigName: col.Name.L,
}
}
func findHandleCol(tbl *model.TableInfo, schema *expression.Schema) *expression.Column {
// fields len is 0 for update and delete.
var handleCol *expression.Column
if tbl.PKIsHandle {
for i, col := range tbl.Columns {
if mysql.HasPriKeyFlag(col.Flag) && tbl.PKIsHandle {
handleCol = schema.Columns[i]
}
}
}
if handleCol == nil {
handleCol = colInfoToColumn(model.NewExtraHandleColInfo(), schema.Len())
schema.Append(handleCol)
}
return handleCol
}
func getPartitionInfo(ctx sessionctx.Context, tbl *model.TableInfo, pairs []nameValuePair) (*model.PartitionDefinition, int) {
partitionColName := getHashPartitionColumnName(ctx, tbl)
if partitionColName == nil {
return nil, 0
}
pi := tbl.Partition
for i, pair := range pairs {
if partitionColName.Name.L == pair.colName {
val := pair.value.GetInt64()
pos := math.Abs(val % int64(pi.Num))
return &pi.Definitions[pos], i
}
}
return nil, 0
}
func findPartitionIdx(idxInfo *model.IndexInfo, pos int, pairs []nameValuePair) int {
for i, idxCol := range idxInfo.Columns {
if idxCol.Name.L == pairs[pos].colName {
return i
}
}
return 0
}
// getPartitionColumnPos gets the partition column's position in the index.
func getPartitionColumnPos(idx *model.IndexInfo, partitionColName *ast.ColumnName) int {
if partitionColName == nil {
return 0
}
for i, idxCol := range idx.Columns {
if partitionColName.Name.L == idxCol.Name.L {
return i
}
}
panic("unique index must include all partition columns")
}
func getHashPartitionColumnName(ctx sessionctx.Context, tbl *model.TableInfo) *ast.ColumnName {
pi := tbl.GetPartitionInfo()
if pi == nil {
return nil
}
if pi.Type != model.PartitionTypeHash {
return nil
}
is := infoschema.GetInfoSchema(ctx)
table, ok := is.TableByID(tbl.ID)
if !ok {
return nil
}
// PartitionExpr don't need columns and names for hash partition.
partitionExpr, err := table.(partitionTable).PartitionExpr()
if err != nil {
return nil
}
expr := partitionExpr.OrigExpr
col, ok := expr.(*ast.ColumnNameExpr)
if !ok {
return nil
}
return col.Name
}