Files
tidb/executor/prepared.go

366 lines
11 KiB
Go

// 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,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package executor
import (
"context"
"math"
"sort"
"time"
"github.com/pingcap/errors"
"github.com/pingcap/log"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/parser"
"github.com/pingcap/tidb/parser/ast"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/planner"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/types"
driver "github.com/pingcap/tidb/types/parser_driver"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/hint"
"github.com/pingcap/tidb/util/sqlexec"
"github.com/pingcap/tidb/util/topsql"
"go.uber.org/zap"
)
var (
_ Executor = &DeallocateExec{}
_ Executor = &ExecuteExec{}
_ Executor = &PrepareExec{}
)
type paramMarkerSorter struct {
markers []ast.ParamMarkerExpr
}
func (p *paramMarkerSorter) Len() int {
return len(p.markers)
}
func (p *paramMarkerSorter) Less(i, j int) bool {
return p.markers[i].(*driver.ParamMarkerExpr).Offset < p.markers[j].(*driver.ParamMarkerExpr).Offset
}
func (p *paramMarkerSorter) Swap(i, j int) {
p.markers[i], p.markers[j] = p.markers[j], p.markers[i]
}
type paramMarkerExtractor struct {
markers []ast.ParamMarkerExpr
}
func (e *paramMarkerExtractor) Enter(in ast.Node) (ast.Node, bool) {
return in, false
}
func (e *paramMarkerExtractor) Leave(in ast.Node) (ast.Node, bool) {
if x, ok := in.(*driver.ParamMarkerExpr); ok {
e.markers = append(e.markers, x)
}
return in, true
}
// PrepareExec represents a PREPARE executor.
type PrepareExec struct {
baseExecutor
name string
sqlText string
ID uint32
ParamCount int
Fields []*ast.ResultField
// If it's generated from executing "prepare stmt from '...'", the process is parse -> plan -> executor
// If it's generated from the prepare protocol, the process is session.PrepareStmt -> NewPrepareExec
// They both generate a PrepareExec struct, but the second case needs to reset the statement context while the first already do that.
needReset bool
}
// NewPrepareExec creates a new PrepareExec.
func NewPrepareExec(ctx sessionctx.Context, sqlTxt string) *PrepareExec {
base := newBaseExecutor(ctx, nil, 0)
base.initCap = chunk.ZeroCapacity
return &PrepareExec{
baseExecutor: base,
sqlText: sqlTxt,
needReset: true,
}
}
// Next implements the Executor Next interface.
func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error {
vars := e.ctx.GetSessionVars()
if e.ID != 0 {
// Must be the case when we retry a prepare.
// Make sure it is idempotent.
_, ok := vars.PreparedStmts[e.ID]
if ok {
return nil
}
}
charset, collation := vars.GetCharsetInfo()
var (
stmts []ast.StmtNode
err error
)
if sqlParser, ok := e.ctx.(sqlexec.SQLParser); ok {
// FIXME: ok... yet another parse API, may need some api interface clean.
stmts, _, err = sqlParser.ParseSQL(ctx, e.sqlText,
parser.CharsetConnection(charset),
parser.CollationConnection(collation))
} else {
p := parser.New()
p.SetParserConfig(vars.BuildParserConfig())
var warns []error
stmts, warns, err = p.ParseSQL(e.sqlText,
parser.CharsetConnection(charset),
parser.CollationConnection(collation))
for _, warn := range warns {
e.ctx.GetSessionVars().StmtCtx.AppendWarning(util.SyntaxWarn(warn))
}
}
if err != nil {
return util.SyntaxError(err)
}
if len(stmts) != 1 {
return ErrPrepareMulti
}
stmt := stmts[0]
if e.needReset {
err = ResetContextOfStmt(e.ctx, stmt)
if err != nil {
return err
}
}
var extractor paramMarkerExtractor
stmt.Accept(&extractor)
// DDL Statements can not accept parameters
if _, ok := stmt.(ast.DDLNode); ok && len(extractor.markers) > 0 {
return ErrPrepareDDL
}
switch stmt.(type) {
case *ast.LoadDataStmt, *ast.PrepareStmt, *ast.ExecuteStmt, *ast.DeallocateStmt:
return ErrUnsupportedPs
}
// Prepare parameters should NOT over 2 bytes(MaxUint16)
// https://dev.mysql.com/doc/internals/en/com-stmt-prepare-response.html#packet-COM_STMT_PREPARE_OK.
if len(extractor.markers) > math.MaxUint16 {
return ErrPsManyParam
}
ret := &plannercore.PreprocessorReturn{}
err = plannercore.Preprocess(e.ctx, stmt, plannercore.InPrepare, plannercore.WithPreprocessorReturn(ret))
if err != nil {
return err
}
// The parameter markers are appended in visiting order, which may not
// be the same as the position order in the query string. We need to
// sort it by position.
sorter := &paramMarkerSorter{markers: extractor.markers}
sort.Sort(sorter)
e.ParamCount = len(sorter.markers)
for i := 0; i < e.ParamCount; i++ {
sorter.markers[i].SetOrder(i)
}
prepared := &ast.Prepared{
Stmt: stmt,
StmtType: GetStmtLabel(stmt),
Params: sorter.markers,
SchemaVersion: ret.InfoSchema.SchemaMetaVersion(),
}
normalizedSQL, digest := parser.NormalizeDigest(prepared.Stmt.Text())
if variable.TopSQLEnabled() {
ctx = topsql.AttachSQLInfo(ctx, normalizedSQL, digest, "", nil, vars.InRestrictedSQL)
}
if !plannercore.PreparedPlanCacheEnabled() {
prepared.UseCache = false
} else {
prepared.UseCache = plannercore.CacheableWithCtx(e.ctx, stmt, ret.InfoSchema)
}
// We try to build the real statement of preparedStmt.
for i := range prepared.Params {
param := prepared.Params[i].(*driver.ParamMarkerExpr)
param.Datum.SetNull()
param.InExecute = false
}
var p plannercore.Plan
e.ctx.GetSessionVars().PlanID = 0
e.ctx.GetSessionVars().PlanColumnID = 0
destBuilder, _ := plannercore.NewPlanBuilder().Init(e.ctx, ret.InfoSchema, &hint.BlockHintProcessor{})
p, err = destBuilder.Build(ctx, stmt)
if err != nil {
return err
}
if _, ok := stmt.(*ast.SelectStmt); ok {
e.Fields = colNames2ResultFields(p.Schema(), p.OutputNames(), vars.CurrentDB)
}
if e.ID == 0 {
e.ID = vars.GetNextPreparedStmtID()
}
if e.name != "" {
vars.PreparedStmtNameToID[e.name] = e.ID
}
preparedObj := &plannercore.CachedPrepareStmt{
PreparedAst: prepared,
VisitInfos: destBuilder.GetVisitInfo(),
NormalizedSQL: normalizedSQL,
SQLDigest: digest,
ForUpdateRead: destBuilder.GetIsForUpdateRead(),
SnapshotTSEvaluator: ret.SnapshotTSEvaluator,
}
return vars.AddPreparedStmt(e.ID, preparedObj)
}
// ExecuteExec represents an EXECUTE executor.
// It cannot be executed by itself, all it needs to do is to build
// another Executor from a prepared statement.
type ExecuteExec struct {
baseExecutor
is infoschema.InfoSchema
name string
usingVars []expression.Expression
stmtExec Executor
stmt ast.StmtNode
plan plannercore.Plan
id uint32
lowerPriority bool
outputNames []*types.FieldName
}
// Next implements the Executor Next interface.
func (e *ExecuteExec) Next(ctx context.Context, req *chunk.Chunk) error {
return nil
}
// Build builds a prepared statement into an executor.
// After Build, e.StmtExec will be used to do the real execution.
func (e *ExecuteExec) Build(b *executorBuilder) error {
if snapshotTS := e.ctx.GetSessionVars().SnapshotTS; snapshotTS != 0 {
if err := e.ctx.InitTxnWithStartTS(snapshotTS); err != nil {
return err
}
} else {
ok, err := plannercore.IsPointGetWithPKOrUniqueKeyByAutoCommit(e.ctx, e.plan)
if err != nil {
return err
}
if ok {
err = e.ctx.InitTxnWithStartTS(math.MaxUint64)
if err != nil {
return err
}
}
}
stmtExec := b.build(e.plan)
if b.err != nil {
log.Warn("rebuild plan in EXECUTE statement failed", zap.String("labelName of PREPARE statement", e.name))
return errors.Trace(b.err)
}
e.stmtExec = stmtExec
if e.ctx.GetSessionVars().StmtCtx.Priority == mysql.NoPriority {
e.lowerPriority = needLowerPriority(e.plan)
}
return nil
}
// DeallocateExec represent a DEALLOCATE executor.
type DeallocateExec struct {
baseExecutor
Name string
}
// Next implements the Executor Next interface.
func (e *DeallocateExec) Next(ctx context.Context, req *chunk.Chunk) error {
vars := e.ctx.GetSessionVars()
id, ok := vars.PreparedStmtNameToID[e.Name]
if !ok {
return errors.Trace(plannercore.ErrStmtNotFound)
}
preparedPointer := vars.PreparedStmts[id]
preparedObj, ok := preparedPointer.(*plannercore.CachedPrepareStmt)
if !ok {
return errors.Errorf("invalid CachedPrepareStmt type")
}
prepared := preparedObj.PreparedAst
delete(vars.PreparedStmtNameToID, e.Name)
if plannercore.PreparedPlanCacheEnabled() {
bindSQL := planner.GetBindSQL4PlanCache(e.ctx, prepared.Stmt)
e.ctx.PreparedPlanCache().Delete(plannercore.NewPSTMTPlanCacheKey(
vars, id, prepared.SchemaVersion, bindSQL,
))
}
vars.RemovePreparedStmt(id)
return nil
}
// CompileExecutePreparedStmt compiles a session Execute command to a stmt.Statement.
func CompileExecutePreparedStmt(ctx context.Context, sctx sessionctx.Context,
ID uint32, is infoschema.InfoSchema, snapshotTS uint64, args []types.Datum) (*ExecStmt, bool, bool, error) {
startTime := time.Now()
defer func() {
sctx.GetSessionVars().DurationCompile = time.Since(startTime)
}()
execStmt := &ast.ExecuteStmt{ExecID: ID}
if err := ResetContextOfStmt(sctx, execStmt); err != nil {
return nil, false, false, err
}
execStmt.BinaryArgs = args
execPlan, names, err := planner.Optimize(ctx, sctx, execStmt, is)
if err != nil {
return nil, false, false, err
}
stmt := &ExecStmt{
GoCtx: ctx,
InfoSchema: is,
Plan: execPlan,
StmtNode: execStmt,
Ctx: sctx,
OutputNames: names,
Ti: &TelemetryInfo{},
SnapshotTS: snapshotTS,
}
if preparedPointer, ok := sctx.GetSessionVars().PreparedStmts[ID]; ok {
preparedObj, ok := preparedPointer.(*plannercore.CachedPrepareStmt)
if !ok {
return nil, false, false, errors.Errorf("invalid CachedPrepareStmt type")
}
stmtCtx := sctx.GetSessionVars().StmtCtx
stmt.Text = preparedObj.PreparedAst.Stmt.Text()
stmtCtx.OriginalSQL = stmt.Text
stmtCtx.InitSQLDigest(preparedObj.NormalizedSQL, preparedObj.SQLDigest)
}
tiFlashPushDown, tiFlashExchangePushDown := plannercore.IsTiFlashContained(stmt.Plan)
return stmt, tiFlashPushDown, tiFlashExchangePushDown, nil
}