From f5f4d66d3fcb24c8fa4b9e9efe7deb84778b4b2a Mon Sep 17 00:00:00 2001 From: qw4990 Date: Wed, 10 Aug 2022 16:35:20 +0800 Subject: [PATCH 1/2] fixup --- executor/adapter.go | 2 +- executor/executor_test.go | 16 ++++++++-------- executor/prepared.go | 6 +++--- executor/prepared_test.go | 6 +++--- planner/core/cache.go | 10 +++++----- planner/core/common_plans.go | 2 +- planner/core/plan_cache.go | 14 +++++++------- planner/core/prepare_test.go | 2 +- server/conn_stmt.go | 6 +++--- server/driver_tidb.go | 10 +++++----- session/session.go | 8 ++++---- 11 files changed, 41 insertions(+), 41 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index c26cfcc780455..6a5b3262c441a 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -242,7 +242,7 @@ type ExecStmt struct { // OutputNames will be set if using cached plan OutputNames []*types.FieldName - PsStmt *plannercore.CachedPrepareStmt + PsStmt *plannercore.PlanCacheStmt Ti *TelemetryInfo } diff --git a/executor/executor_test.go b/executor/executor_test.go index 16e40483a55fb..0fe57783a9f4c 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -3554,10 +3554,10 @@ func TestPointGetPreparedPlan(t *testing.T) { pspk1Id, _, _, err := tk.Session().PrepareStmt("select * from t where a = ?") require.NoError(t, err) - tk.Session().GetSessionVars().PreparedStmts[pspk1Id].(*plannercore.CachedPrepareStmt).PreparedAst.UseCache = false + tk.Session().GetSessionVars().PreparedStmts[pspk1Id].(*plannercore.PlanCacheStmt).PreparedAst.UseCache = false pspk2Id, _, _, err := tk.Session().PrepareStmt("select * from t where ? = a ") require.NoError(t, err) - tk.Session().GetSessionVars().PreparedStmts[pspk2Id].(*plannercore.CachedPrepareStmt).PreparedAst.UseCache = false + tk.Session().GetSessionVars().PreparedStmts[pspk2Id].(*plannercore.PlanCacheStmt).PreparedAst.UseCache = false ctx := context.Background() // first time plan generated @@ -3597,7 +3597,7 @@ func TestPointGetPreparedPlan(t *testing.T) { // unique index psuk1Id, _, _, err := tk.Session().PrepareStmt("select * from t where b = ? ") require.NoError(t, err) - tk.Session().GetSessionVars().PreparedStmts[psuk1Id].(*plannercore.CachedPrepareStmt).PreparedAst.UseCache = false + tk.Session().GetSessionVars().PreparedStmts[psuk1Id].(*plannercore.PlanCacheStmt).PreparedAst.UseCache = false rs, err = tk.Session().ExecutePreparedStmt(ctx, psuk1Id, expression.Args2Expressions4Test(1)) require.NoError(t, err) @@ -3715,7 +3715,7 @@ func TestPointGetPreparedPlanWithCommitMode(t *testing.T) { pspk1Id, _, _, err := tk1.Session().PrepareStmt("select * from t where a = ?") require.NoError(t, err) - tk1.Session().GetSessionVars().PreparedStmts[pspk1Id].(*plannercore.CachedPrepareStmt).PreparedAst.UseCache = false + tk1.Session().GetSessionVars().PreparedStmts[pspk1Id].(*plannercore.PlanCacheStmt).PreparedAst.UseCache = false ctx := context.Background() // first time plan generated @@ -3781,11 +3781,11 @@ func TestPointUpdatePreparedPlan(t *testing.T) { updateID1, pc, _, err := tk.Session().PrepareStmt(`update t set c = c + 1 where a = ?`) require.NoError(t, err) - tk.Session().GetSessionVars().PreparedStmts[updateID1].(*plannercore.CachedPrepareStmt).PreparedAst.UseCache = false + tk.Session().GetSessionVars().PreparedStmts[updateID1].(*plannercore.PlanCacheStmt).PreparedAst.UseCache = false require.Equal(t, 1, pc) updateID2, pc, _, err := tk.Session().PrepareStmt(`update t set c = c + 2 where ? = a`) require.NoError(t, err) - tk.Session().GetSessionVars().PreparedStmts[updateID2].(*plannercore.CachedPrepareStmt).PreparedAst.UseCache = false + tk.Session().GetSessionVars().PreparedStmts[updateID2].(*plannercore.PlanCacheStmt).PreparedAst.UseCache = false require.Equal(t, 1, pc) ctx := context.Background() @@ -3820,7 +3820,7 @@ func TestPointUpdatePreparedPlan(t *testing.T) { // unique index updUkID1, _, _, err := tk.Session().PrepareStmt(`update t set c = c + 10 where b = ?`) require.NoError(t, err) - tk.Session().GetSessionVars().PreparedStmts[updUkID1].(*plannercore.CachedPrepareStmt).PreparedAst.UseCache = false + tk.Session().GetSessionVars().PreparedStmts[updUkID1].(*plannercore.PlanCacheStmt).PreparedAst.UseCache = false rs, err = tk.Session().ExecutePreparedStmt(ctx, updUkID1, expression.Args2Expressions4Test(3)) require.Nil(t, rs) require.NoError(t, err) @@ -3889,7 +3889,7 @@ func TestPointUpdatePreparedPlanWithCommitMode(t *testing.T) { ctx := context.Background() updateID1, _, _, err := tk1.Session().PrepareStmt(`update t set c = c + 1 where a = ?`) - tk1.Session().GetSessionVars().PreparedStmts[updateID1].(*plannercore.CachedPrepareStmt).PreparedAst.UseCache = false + tk1.Session().GetSessionVars().PreparedStmts[updateID1].(*plannercore.PlanCacheStmt).PreparedAst.UseCache = false require.NoError(t, err) // first time plan generated diff --git a/executor/prepared.go b/executor/prepared.go index 715d438c90943..84d852c8b0d7b 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -229,7 +229,7 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { vars.PreparedStmtNameToID[e.name] = e.ID } - preparedObj := &plannercore.CachedPrepareStmt{ + preparedObj := &plannercore.PlanCacheStmt{ PreparedAst: prepared, StmtDB: e.ctx.GetSessionVars().CurrentDB, StmtText: stmt.Text(), @@ -295,9 +295,9 @@ func (e *DeallocateExec) Next(ctx context.Context, req *chunk.Chunk) error { return errors.Trace(plannercore.ErrStmtNotFound) } preparedPointer := vars.PreparedStmts[id] - preparedObj, ok := preparedPointer.(*plannercore.CachedPrepareStmt) + preparedObj, ok := preparedPointer.(*plannercore.PlanCacheStmt) if !ok { - return errors.Errorf("invalid CachedPrepareStmt type") + return errors.Errorf("invalid PlanCacheStmt type") } prepared := preparedObj.PreparedAst delete(vars.PreparedStmtNameToID, e.Name) diff --git a/executor/prepared_test.go b/executor/prepared_test.go index 4c10dd7801443..f511c2e6f8b72 100644 --- a/executor/prepared_test.go +++ b/executor/prepared_test.go @@ -1252,8 +1252,8 @@ func TestPrepareStmtAfterIsolationReadChange(t *testing.T) { require.Equal(t, rows[len(rows)-1][2], "cop[tiflash]") require.Equal(t, 1, len(tk.Session().GetSessionVars().PreparedStmts)) - require.Equal(t, "select * from `t`", tk.Session().GetSessionVars().PreparedStmts[1].(*plannercore.CachedPrepareStmt).NormalizedSQL) - require.Equal(t, "", tk.Session().GetSessionVars().PreparedStmts[1].(*plannercore.CachedPrepareStmt).NormalizedPlan) + require.Equal(t, "select * from `t`", tk.Session().GetSessionVars().PreparedStmts[1].(*plannercore.PlanCacheStmt).NormalizedSQL) + require.Equal(t, "", tk.Session().GetSessionVars().PreparedStmts[1].(*plannercore.PlanCacheStmt).NormalizedPlan) } func TestPreparePC4Binding(t *testing.T) { @@ -1267,7 +1267,7 @@ func TestPreparePC4Binding(t *testing.T) { tk.MustExec("prepare stmt from \"select * from t\"") require.Equal(t, 1, len(tk.Session().GetSessionVars().PreparedStmts)) - require.Equal(t, "select * from `test` . `t`", tk.Session().GetSessionVars().PreparedStmts[1].(*plannercore.CachedPrepareStmt).NormalizedSQL4PC) + require.Equal(t, "select * from `test` . `t`", tk.Session().GetSessionVars().PreparedStmts[1].(*plannercore.PlanCacheStmt).NormalizedSQL4PC) tk.MustQuery("execute stmt") tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("0")) diff --git a/planner/core/cache.go b/planner/core/cache.go index f9c3f5fa038b5..84d0c2a98d989 100644 --- a/planner/core/cache.go +++ b/planner/core/cache.go @@ -219,8 +219,8 @@ func NewPlanCacheValue(plan Plan, names []*types.FieldName, srcMap map[*model.Ta } } -// CachedPrepareStmt store prepared ast from PrepareExec and other related fields -type CachedPrepareStmt struct { +// PlanCacheStmt store prepared ast from PrepareExec and other related fields +type PlanCacheStmt struct { PreparedAst *ast.Prepared StmtDB string // which DB the statement will be processed over VisitInfos []visitInfo @@ -244,9 +244,9 @@ type CachedPrepareStmt struct { } // GetPreparedStmt extract the prepared statement from the execute statement. -func GetPreparedStmt(stmt *ast.ExecuteStmt, vars *variable.SessionVars) (*CachedPrepareStmt, error) { +func GetPreparedStmt(stmt *ast.ExecuteStmt, vars *variable.SessionVars) (*PlanCacheStmt, error) { if stmt.PrepStmt != nil { - return stmt.PrepStmt.(*CachedPrepareStmt), nil + return stmt.PrepStmt.(*PlanCacheStmt), nil } if stmt.Name != "" { prepStmt, err := vars.GetPreparedStmtByName(stmt.Name) @@ -254,7 +254,7 @@ func GetPreparedStmt(stmt *ast.ExecuteStmt, vars *variable.SessionVars) (*Cached return nil, err } stmt.PrepStmt = prepStmt - return prepStmt.(*CachedPrepareStmt), nil + return prepStmt.(*PlanCacheStmt), nil } return nil, ErrStmtNotFound } diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 8db57e24e4c78..0cf85917c1631 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -190,7 +190,7 @@ type Execute struct { Name string Params []expression.Expression - PrepStmt *CachedPrepareStmt + PrepStmt *PlanCacheStmt Stmt ast.StmtNode StmtType string Plan Plan diff --git a/planner/core/plan_cache.go b/planner/core/plan_cache.go index ca1d6014c3c23..35a2a15fcf5e5 100644 --- a/planner/core/plan_cache.go +++ b/planner/core/plan_cache.go @@ -44,7 +44,7 @@ import ( // GetPlanFromSessionPlanCache is the entry point of Plan Cache. // It tries to get a valid cached plan from this session's plan cache. // If there is no such a plan, it'll call the optimizer to generate a new one. -func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context, is infoschema.InfoSchema, preparedStmt *CachedPrepareStmt, +func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context, is infoschema.InfoSchema, preparedStmt *PlanCacheStmt, params []expression.Expression) (plan Plan, names []*types.FieldName, err error) { var cacheKey kvcache.Key sessVars := sctx.GetSessionVars() @@ -137,7 +137,7 @@ func getPointQueryPlan(prepared *ast.Prepared, sessVars *variable.SessionVars, s } func getGeneralPlan(ctx context.Context, sctx sessionctx.Context, cacheKey kvcache.Key, bindSQL string, - is infoschema.InfoSchema, preparedStmt *CachedPrepareStmt, paramTypes []*types.FieldType) (Plan, + is infoschema.InfoSchema, preparedStmt *PlanCacheStmt, paramTypes []*types.FieldType) (Plan, []*types.FieldName, bool, error) { sessVars := sctx.GetSessionVars() stmtCtx := sessVars.StmtCtx @@ -189,7 +189,7 @@ func getGeneralPlan(ctx context.Context, sctx sessionctx.Context, cacheKey kvcac // generateNewPlan call the optimizer to generate a new plan for current statement // and try to add it to cache -func generateNewPlan(ctx context.Context, sctx sessionctx.Context, is infoschema.InfoSchema, preparedStmt *CachedPrepareStmt, +func generateNewPlan(ctx context.Context, sctx sessionctx.Context, is infoschema.InfoSchema, preparedStmt *PlanCacheStmt, ignorePlanCache bool, cacheKey kvcache.Key, latestSchemaVersion int64, paramNum int, paramTypes []*types.FieldType, bindSQL string) (Plan, []*types.FieldName, error) { prepared := preparedStmt.PreparedAst @@ -539,7 +539,7 @@ func buildRangeForIndexScan(sctx sessionctx.Context, is *PhysicalIndexScan) (err } func checkPreparedPriv(_ context.Context, sctx sessionctx.Context, - preparedObj *CachedPrepareStmt, is infoschema.InfoSchema) error { + preparedObj *PlanCacheStmt, is infoschema.InfoSchema) error { if pm := privilege.GetPrivilegeManager(sctx); pm != nil { visitInfo := VisitInfo4PrivCheck(is, preparedObj.PreparedAst.Stmt, preparedObj.VisitInfos) if err := CheckPrivilege(sctx.GetSessionVars().ActiveRoles, pm, visitInfo); err != nil { @@ -553,7 +553,7 @@ func checkPreparedPriv(_ context.Context, sctx sessionctx.Context, // tryCachePointPlan will try to cache point execution plan, there may be some // short paths for these executions, currently "point select" and "point update" func tryCachePointPlan(_ context.Context, sctx sessionctx.Context, - preparedStmt *CachedPrepareStmt, _ infoschema.InfoSchema, p Plan) error { + preparedStmt *PlanCacheStmt, _ infoschema.InfoSchema, p Plan) error { if !sctx.GetSessionVars().StmtCtx.UseCache || sctx.GetSessionVars().StmtCtx.SkipPlanCache { return nil } @@ -600,7 +600,7 @@ func containTableDual(p Plan) bool { } // GetBindSQL4PlanCache used to get the bindSQL for plan cache to build the plan cache key. -func GetBindSQL4PlanCache(sctx sessionctx.Context, preparedStmt *CachedPrepareStmt) (string, bool) { +func GetBindSQL4PlanCache(sctx sessionctx.Context, preparedStmt *PlanCacheStmt) (string, bool) { useBinding := sctx.GetSessionVars().UsePlanBaselines ignore := false if !useBinding || preparedStmt.PreparedAst.Stmt == nil || preparedStmt.NormalizedSQL4PC == "" || preparedStmt.SQLDigest4PC == "" { @@ -636,7 +636,7 @@ func GetBindSQL4PlanCache(sctx sessionctx.Context, preparedStmt *CachedPrepareSt // IsPointPlanShortPathOK check if we can execute using plan cached in prepared structure // Be careful with the short path, current precondition is ths cached plan satisfying // IsPointGetWithPKOrUniqueKeyByAutoCommit -func IsPointPlanShortPathOK(sctx sessionctx.Context, is infoschema.InfoSchema, preparedStmt *CachedPrepareStmt) (bool, error) { +func IsPointPlanShortPathOK(sctx sessionctx.Context, is infoschema.InfoSchema, preparedStmt *PlanCacheStmt) (bool, error) { prepared := preparedStmt.PreparedAst if prepared.CachedPlan == nil || staleread.IsStmtStaleness(sctx) { return false, nil diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index ed47183902da4..10d5dd454a8ed 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -61,7 +61,7 @@ func TestPointGetPreparedPlan4PlanCache(t *testing.T) { pspk1Id, _, _, err := tk1.Session().PrepareStmt("select * from t where a = ?") require.NoError(t, err) - tk1.Session().GetSessionVars().PreparedStmts[pspk1Id].(*core.CachedPrepareStmt).PreparedAst.UseCache = false + tk1.Session().GetSessionVars().PreparedStmts[pspk1Id].(*core.PlanCacheStmt).PreparedAst.UseCache = false ctx := context.Background() // first time plan generated diff --git a/server/conn_stmt.go b/server/conn_stmt.go index 3e7ce6d6d4c0e..7a9e93133f1e5 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -724,7 +724,7 @@ func (cc *clientConn) preparedStmt2StringNoArgs(stmtID uint32) string { } preparedObj, invalid := cc.preparedStmtID2CachePreparedStmt(stmtID) if invalid { - return "invalidate CachedPrepareStmt type, ID: " + strconv.FormatUint(uint64(stmtID), 10) + return "invalidate PlanCacheStmt type, ID: " + strconv.FormatUint(uint64(stmtID), 10) } if preparedObj == nil { return "prepared statement not found, ID: " + strconv.FormatUint(uint64(stmtID), 10) @@ -732,7 +732,7 @@ func (cc *clientConn) preparedStmt2StringNoArgs(stmtID uint32) string { return preparedObj.PreparedAst.Stmt.Text() } -func (cc *clientConn) preparedStmtID2CachePreparedStmt(stmtID uint32) (_ *plannercore.CachedPrepareStmt, invalid bool) { +func (cc *clientConn) preparedStmtID2CachePreparedStmt(stmtID uint32) (_ *plannercore.PlanCacheStmt, invalid bool) { sv := cc.ctx.GetSessionVars() if sv == nil { return nil, false @@ -742,7 +742,7 @@ func (cc *clientConn) preparedStmtID2CachePreparedStmt(stmtID uint32) (_ *planne // not found return nil, false } - preparedObj, ok := preparedPointer.(*plannercore.CachedPrepareStmt) + preparedObj, ok := preparedPointer.(*plannercore.PlanCacheStmt) if !ok { // invalid cache. should never happen. return nil, true diff --git a/server/driver_tidb.go b/server/driver_tidb.go index ff8e9b67e1985..bfd76ff0552a1 100644 --- a/server/driver_tidb.go +++ b/server/driver_tidb.go @@ -85,7 +85,7 @@ func (ts *TiDBStatement) Execute(ctx context.Context, args []expression.Expressi } rs = &tidbResultSet{ recordSet: tidbRecordset, - preparedStmt: ts.ctx.GetSessionVars().PreparedStmts[ts.id].(*core.CachedPrepareStmt), + preparedStmt: ts.ctx.GetSessionVars().PreparedStmts[ts.id].(*core.PlanCacheStmt), } return } @@ -165,9 +165,9 @@ func (ts *TiDBStatement) Close() error { } else { if ts.ctx.GetSessionVars().EnablePreparedPlanCache { preparedPointer := ts.ctx.GetSessionVars().PreparedStmts[ts.id] - preparedObj, ok := preparedPointer.(*core.CachedPrepareStmt) + preparedObj, ok := preparedPointer.(*core.PlanCacheStmt) if !ok { - return errors.Errorf("invalid CachedPrepareStmt type") + return errors.Errorf("invalid PlanCacheStmt type") } bindSQL, _ := core.GetBindSQL4PlanCache(ts.ctx, preparedObj) cacheKey, err := core.NewPlanCacheKey(ts.ctx.GetSessionVars(), preparedObj.StmtText, preparedObj.StmtDB, @@ -313,7 +313,7 @@ func (tc *TiDBContext) EncodeSessionStates(ctx context.Context, sctx sessionctx. sessionVars := tc.Session.GetSessionVars() sessionStates.PreparedStmts = make(map[uint32]*sessionstates.PreparedStmtInfo, len(sessionVars.PreparedStmts)) for preparedID, preparedObj := range sessionVars.PreparedStmts { - preparedStmt, ok := preparedObj.(*core.CachedPrepareStmt) + preparedStmt, ok := preparedObj.(*core.PlanCacheStmt) if !ok { return errors.Errorf("invalid CachedPreparedStmt type") } @@ -396,7 +396,7 @@ type tidbResultSet struct { columns []*ColumnInfo rows []chunk.Row closed int32 - preparedStmt *core.CachedPrepareStmt + preparedStmt *core.PlanCacheStmt } func (trs *tidbResultSet) NewChunk(alloc chunk.Allocator) *chunk.Chunk { diff --git a/session/session.go b/session/session.go index c20b5afe8ec3d..3dd8d3d006eb0 100644 --- a/session/session.go +++ b/session/session.go @@ -345,7 +345,7 @@ func (s *session) cleanRetryInfo() { if planCacheEnabled { firstStmtID := retryInfo.DroppedPreparedStmtIDs[0] if preparedPointer, ok := s.sessionVars.PreparedStmts[firstStmtID]; ok { - preparedObj, ok := preparedPointer.(*plannercore.CachedPrepareStmt) + preparedObj, ok := preparedPointer.(*plannercore.PlanCacheStmt) if ok { preparedAst = preparedObj.PreparedAst stmtText, stmtDB = preparedObj.StmtText, preparedObj.StmtDB @@ -2269,7 +2269,7 @@ func (s *session) PrepareStmt(sql string) (stmtID uint32, paramCount int, fields return prepareExec.ID, prepareExec.ParamCount, prepareExec.Fields, nil } -func (s *session) preparedStmtExec(ctx context.Context, execStmt *ast.ExecuteStmt, prepareStmt *plannercore.CachedPrepareStmt) (sqlexec.RecordSet, error) { +func (s *session) preparedStmtExec(ctx context.Context, execStmt *ast.ExecuteStmt, prepareStmt *plannercore.PlanCacheStmt) (sqlexec.RecordSet, error) { failpoint.Inject("assertTxnManagerInPreparedStmtExec", func() { sessiontxn.RecordAssert(s, "assertTxnManagerInPreparedStmtExec", true) if prepareStmt.SnapshotTSEvaluator != nil { @@ -2326,9 +2326,9 @@ func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args [ logutil.Logger(ctx).Error("prepared statement not found", zap.Uint32("stmtID", stmtID)) return nil, err } - preparedStmt, ok := prepStmt.(*plannercore.CachedPrepareStmt) + preparedStmt, ok := prepStmt.(*plannercore.PlanCacheStmt) if !ok { - return nil, errors.Errorf("invalid CachedPrepareStmt type") + return nil, errors.Errorf("invalid PlanCacheStmt type") } execStmt := &ast.ExecuteStmt{PrepStmt: prepStmt, BinaryArgs: args} From 43acfbe90fc2a102b73a3031686c399bf367b8c5 Mon Sep 17 00:00:00 2001 From: qw4990 Date: Wed, 10 Aug 2022 16:48:51 +0800 Subject: [PATCH 2/2] fixup --- planner/core/plan_cache.go | 109 ++++++++++++++++++------------------- 1 file changed, 54 insertions(+), 55 deletions(-) diff --git a/planner/core/plan_cache.go b/planner/core/plan_cache.go index 35a2a15fcf5e5..a2ec0c8645f6a 100644 --- a/planner/core/plan_cache.go +++ b/planner/core/plan_cache.go @@ -44,13 +44,13 @@ import ( // GetPlanFromSessionPlanCache is the entry point of Plan Cache. // It tries to get a valid cached plan from this session's plan cache. // If there is no such a plan, it'll call the optimizer to generate a new one. -func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context, is infoschema.InfoSchema, preparedStmt *PlanCacheStmt, +func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context, is infoschema.InfoSchema, stmt *PlanCacheStmt, params []expression.Expression) (plan Plan, names []*types.FieldName, err error) { var cacheKey kvcache.Key sessVars := sctx.GetSessionVars() stmtCtx := sessVars.StmtCtx - prepared := preparedStmt.PreparedAst - stmtCtx.UseCache = prepared.UseCache + stmtAst := stmt.PreparedAst + stmtCtx.UseCache = stmtAst.UseCache var bindSQL string var ignorePlanCache = false @@ -59,36 +59,36 @@ func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context, i // rebuild the plan. So we set this value in rc or for update read. In other cases, let it be 0. var latestSchemaVersion int64 - if prepared.UseCache { - bindSQL, ignorePlanCache = GetBindSQL4PlanCache(sctx, preparedStmt) - if sctx.GetSessionVars().IsIsolation(ast.ReadCommitted) || preparedStmt.ForUpdateRead { + if stmtAst.UseCache { + bindSQL, ignorePlanCache = GetBindSQL4PlanCache(sctx, stmt) + if sctx.GetSessionVars().IsIsolation(ast.ReadCommitted) || stmt.ForUpdateRead { // In Rc or ForUpdateRead, we should check if the information schema has been changed since // last time. If it changed, we should rebuild the plan. Here, we use a different and more // up-to-date schema version which can lead plan cache miss and thus, the plan will be rebuilt. latestSchemaVersion = domain.GetDomain(sctx).InfoSchema().SchemaMetaVersion() } - if cacheKey, err = NewPlanCacheKey(sctx.GetSessionVars(), preparedStmt.StmtText, - preparedStmt.StmtDB, prepared.SchemaVersion, latestSchemaVersion, bindSQL); err != nil { + if cacheKey, err = NewPlanCacheKey(sctx.GetSessionVars(), stmt.StmtText, + stmt.StmtDB, stmtAst.SchemaVersion, latestSchemaVersion, bindSQL); err != nil { return nil, nil, err } } paramNum, paramTypes := parseParamTypes(sctx, params) - if prepared.UseCache && prepared.CachedPlan != nil && !ignorePlanCache { // for point query plan - if plan, names, ok, err := getPointQueryPlan(prepared, sessVars, stmtCtx); ok { + if stmtAst.UseCache && stmtAst.CachedPlan != nil && !ignorePlanCache { // for point query plan + if plan, names, ok, err := getPointQueryPlan(stmtAst, sessVars, stmtCtx); ok { return plan, names, err } } - if prepared.UseCache && !ignorePlanCache { // for general plans - if plan, names, ok, err := getGeneralPlan(ctx, sctx, cacheKey, bindSQL, is, preparedStmt, + if stmtAst.UseCache && !ignorePlanCache { // for general plans + if plan, names, ok, err := getGeneralPlan(ctx, sctx, cacheKey, bindSQL, is, stmt, paramTypes); err != nil || ok { return plan, names, err } } - return generateNewPlan(ctx, sctx, is, preparedStmt, ignorePlanCache, cacheKey, + return generateNewPlan(ctx, sctx, is, stmt, ignorePlanCache, cacheKey, latestSchemaVersion, paramNum, paramTypes, bindSQL) } @@ -112,15 +112,15 @@ func parseParamTypes(sctx sessionctx.Context, params []expression.Expression) (p return } -func getPointQueryPlan(prepared *ast.Prepared, sessVars *variable.SessionVars, stmtCtx *stmtctx.StatementContext) (Plan, +func getPointQueryPlan(stmt *ast.Prepared, sessVars *variable.SessionVars, stmtCtx *stmtctx.StatementContext) (Plan, []*types.FieldName, bool, error) { // short path for point-get plans // Rewriting the expression in the select.where condition will convert its // type from "paramMarker" to "Constant".When Point Select queries are executed, // the expression in the where condition will not be evaluated, // so you don't need to consider whether prepared.useCache is enabled. - plan := prepared.CachedPlan.(Plan) - names := prepared.CachedNames.(types.NameSlice) + plan := stmt.CachedPlan.(Plan) + names := stmt.CachedNames.(types.NameSlice) err := RebuildPlan4CachedPlan(plan) if err != nil { logutil.BgLogger().Debug("rebuild range failed", zap.Error(err)) @@ -137,13 +137,13 @@ func getPointQueryPlan(prepared *ast.Prepared, sessVars *variable.SessionVars, s } func getGeneralPlan(ctx context.Context, sctx sessionctx.Context, cacheKey kvcache.Key, bindSQL string, - is infoschema.InfoSchema, preparedStmt *PlanCacheStmt, paramTypes []*types.FieldType) (Plan, + is infoschema.InfoSchema, stmt *PlanCacheStmt, paramTypes []*types.FieldType) (Plan, []*types.FieldName, bool, error) { sessVars := sctx.GetSessionVars() stmtCtx := sessVars.StmtCtx if cacheValue, exists := sctx.PreparedPlanCache().Get(cacheKey); exists { - if err := checkPreparedPriv(ctx, sctx, preparedStmt, is); err != nil { + if err := checkPreparedPriv(ctx, sctx, stmt, is); err != nil { return nil, nil, false, err } cachedVals := cacheValue.([]*PlanCacheValue) @@ -178,7 +178,7 @@ func getGeneralPlan(ctx context.Context, sctx sessionctx.Context, cacheKey kvcac } else { planCacheCounter.Inc() } - stmtCtx.SetPlanDigest(preparedStmt.NormalizedPlan, preparedStmt.PlanDigest) + stmtCtx.SetPlanDigest(stmt.NormalizedPlan, stmt.PlanDigest) return cachedVal.Plan, cachedVal.OutPutNames, true, nil } break @@ -189,20 +189,19 @@ func getGeneralPlan(ctx context.Context, sctx sessionctx.Context, cacheKey kvcac // generateNewPlan call the optimizer to generate a new plan for current statement // and try to add it to cache -func generateNewPlan(ctx context.Context, sctx sessionctx.Context, is infoschema.InfoSchema, preparedStmt *PlanCacheStmt, +func generateNewPlan(ctx context.Context, sctx sessionctx.Context, is infoschema.InfoSchema, stmt *PlanCacheStmt, ignorePlanCache bool, cacheKey kvcache.Key, latestSchemaVersion int64, paramNum int, paramTypes []*types.FieldType, bindSQL string) (Plan, []*types.FieldName, error) { - prepared := preparedStmt.PreparedAst + stmtAst := stmt.PreparedAst sessVars := sctx.GetSessionVars() stmtCtx := sessVars.StmtCtx planCacheMissCounter.Inc() - stmt := prepared.Stmt - p, names, err := OptimizeAstNode(ctx, sctx, stmt, is) + p, names, err := OptimizeAstNode(ctx, sctx, stmtAst.Stmt, is) if err != nil { return nil, nil, err } - err = tryCachePointPlan(ctx, sctx, preparedStmt, is, p) + err = tryCachePointPlan(ctx, sctx, stmt, is, p) if err != nil { return nil, nil, err } @@ -211,20 +210,20 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, is infoschema if containTableDual(p) && paramNum > 0 { stmtCtx.SkipPlanCache = true } - if prepared.UseCache && !stmtCtx.SkipPlanCache && !ignorePlanCache { + if stmtAst.UseCache && !stmtCtx.SkipPlanCache && !ignorePlanCache { // rebuild key to exclude kv.TiFlash when stmt is not read only - if _, isolationReadContainTiFlash := sessVars.IsolationReadEngines[kv.TiFlash]; isolationReadContainTiFlash && !IsReadOnly(stmt, sessVars) { + if _, isolationReadContainTiFlash := sessVars.IsolationReadEngines[kv.TiFlash]; isolationReadContainTiFlash && !IsReadOnly(stmtAst.Stmt, sessVars) { delete(sessVars.IsolationReadEngines, kv.TiFlash) - if cacheKey, err = NewPlanCacheKey(sessVars, preparedStmt.StmtText, preparedStmt.StmtDB, - prepared.SchemaVersion, latestSchemaVersion, bindSQL); err != nil { + if cacheKey, err = NewPlanCacheKey(sessVars, stmt.StmtText, stmt.StmtDB, + stmtAst.SchemaVersion, latestSchemaVersion, bindSQL); err != nil { return nil, nil, err } sessVars.IsolationReadEngines[kv.TiFlash] = struct{}{} } cached := NewPlanCacheValue(p, names, stmtCtx.TblInfo2UnionScan, paramTypes) - preparedStmt.NormalizedPlan, preparedStmt.PlanDigest = NormalizePlan(p) + stmt.NormalizedPlan, stmt.PlanDigest = NormalizePlan(p) stmtCtx.SetPlan(p) - stmtCtx.SetPlanDigest(preparedStmt.NormalizedPlan, preparedStmt.PlanDigest) + stmtCtx.SetPlanDigest(stmt.NormalizedPlan, stmt.PlanDigest) if cacheVals, exists := sctx.PreparedPlanCache().Get(cacheKey); exists { hitVal := false for i, cacheVal := range cacheVals.([]*PlanCacheValue) { @@ -539,29 +538,29 @@ func buildRangeForIndexScan(sctx sessionctx.Context, is *PhysicalIndexScan) (err } func checkPreparedPriv(_ context.Context, sctx sessionctx.Context, - preparedObj *PlanCacheStmt, is infoschema.InfoSchema) error { + stmt *PlanCacheStmt, is infoschema.InfoSchema) error { if pm := privilege.GetPrivilegeManager(sctx); pm != nil { - visitInfo := VisitInfo4PrivCheck(is, preparedObj.PreparedAst.Stmt, preparedObj.VisitInfos) + visitInfo := VisitInfo4PrivCheck(is, stmt.PreparedAst.Stmt, stmt.VisitInfos) if err := CheckPrivilege(sctx.GetSessionVars().ActiveRoles, pm, visitInfo); err != nil { return err } } - err := CheckTableLock(sctx, is, preparedObj.VisitInfos) + err := CheckTableLock(sctx, is, stmt.VisitInfos) return err } // tryCachePointPlan will try to cache point execution plan, there may be some // short paths for these executions, currently "point select" and "point update" func tryCachePointPlan(_ context.Context, sctx sessionctx.Context, - preparedStmt *PlanCacheStmt, _ infoschema.InfoSchema, p Plan) error { + stmt *PlanCacheStmt, _ infoschema.InfoSchema, p Plan) error { if !sctx.GetSessionVars().StmtCtx.UseCache || sctx.GetSessionVars().StmtCtx.SkipPlanCache { return nil } var ( - prepared = preparedStmt.PreparedAst - ok bool - err error - names types.NameSlice + stmtAst = stmt.PreparedAst + ok bool + err error + names types.NameSlice ) if _, _ok := p.(*PointGetPlan); _ok { @@ -574,11 +573,11 @@ func tryCachePointPlan(_ context.Context, sctx sessionctx.Context, if ok { // just cache point plan now - prepared.CachedPlan = p - prepared.CachedNames = names - preparedStmt.NormalizedPlan, preparedStmt.PlanDigest = NormalizePlan(p) + stmtAst.CachedPlan = p + stmtAst.CachedNames = names + stmt.NormalizedPlan, stmt.PlanDigest = NormalizePlan(p) sctx.GetSessionVars().StmtCtx.SetPlan(p) - sctx.GetSessionVars().StmtCtx.SetPlanDigest(preparedStmt.NormalizedPlan, preparedStmt.PlanDigest) + sctx.GetSessionVars().StmtCtx.SetPlanDigest(stmt.NormalizedPlan, stmt.PlanDigest) } return err } @@ -600,17 +599,17 @@ func containTableDual(p Plan) bool { } // GetBindSQL4PlanCache used to get the bindSQL for plan cache to build the plan cache key. -func GetBindSQL4PlanCache(sctx sessionctx.Context, preparedStmt *PlanCacheStmt) (string, bool) { +func GetBindSQL4PlanCache(sctx sessionctx.Context, stmt *PlanCacheStmt) (string, bool) { useBinding := sctx.GetSessionVars().UsePlanBaselines ignore := false - if !useBinding || preparedStmt.PreparedAst.Stmt == nil || preparedStmt.NormalizedSQL4PC == "" || preparedStmt.SQLDigest4PC == "" { + if !useBinding || stmt.PreparedAst.Stmt == nil || stmt.NormalizedSQL4PC == "" || stmt.SQLDigest4PC == "" { return "", ignore } if sctx.Value(bindinfo.SessionBindInfoKeyType) == nil { return "", ignore } sessionHandle := sctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) - bindRecord := sessionHandle.GetBindRecord(preparedStmt.SQLDigest4PC, preparedStmt.NormalizedSQL4PC, "") + bindRecord := sessionHandle.GetBindRecord(stmt.SQLDigest4PC, stmt.NormalizedSQL4PC, "") if bindRecord != nil { enabledBinding := bindRecord.FindEnabledBinding() if enabledBinding != nil { @@ -622,7 +621,7 @@ func GetBindSQL4PlanCache(sctx sessionctx.Context, preparedStmt *PlanCacheStmt) if globalHandle == nil { return "", ignore } - bindRecord = globalHandle.GetBindRecord(preparedStmt.SQLDigest4PC, preparedStmt.NormalizedSQL4PC, "") + bindRecord = globalHandle.GetBindRecord(stmt.SQLDigest4PC, stmt.NormalizedSQL4PC, "") if bindRecord != nil { enabledBinding := bindRecord.FindEnabledBinding() if enabledBinding != nil { @@ -636,33 +635,33 @@ func GetBindSQL4PlanCache(sctx sessionctx.Context, preparedStmt *PlanCacheStmt) // IsPointPlanShortPathOK check if we can execute using plan cached in prepared structure // Be careful with the short path, current precondition is ths cached plan satisfying // IsPointGetWithPKOrUniqueKeyByAutoCommit -func IsPointPlanShortPathOK(sctx sessionctx.Context, is infoschema.InfoSchema, preparedStmt *PlanCacheStmt) (bool, error) { - prepared := preparedStmt.PreparedAst - if prepared.CachedPlan == nil || staleread.IsStmtStaleness(sctx) { +func IsPointPlanShortPathOK(sctx sessionctx.Context, is infoschema.InfoSchema, stmt *PlanCacheStmt) (bool, error) { + stmtAst := stmt.PreparedAst + if stmtAst.CachedPlan == nil || staleread.IsStmtStaleness(sctx) { return false, nil } // check auto commit if !IsAutoCommitTxn(sctx) { return false, nil } - if prepared.SchemaVersion != is.SchemaMetaVersion() { - prepared.CachedPlan = nil - preparedStmt.ColumnInfos = nil + if stmtAst.SchemaVersion != is.SchemaMetaVersion() { + stmtAst.CachedPlan = nil + stmt.ColumnInfos = nil return false, nil } // maybe we'd better check cached plan type here, current // only point select/update will be cached, see "getPhysicalPlan" func var ok bool var err error - switch prepared.CachedPlan.(type) { + switch stmtAst.CachedPlan.(type) { case *PointGetPlan: ok = true case *Update: - pointUpdate := prepared.CachedPlan.(*Update) + pointUpdate := stmtAst.CachedPlan.(*Update) _, ok = pointUpdate.SelectPlan.(*PointGetPlan) if !ok { err = errors.Errorf("cached update plan not point update") - prepared.CachedPlan = nil + stmtAst.CachedPlan = nil return false, err } default: