From 1ee58bf15bb4cbbadb740fff32612159577c4111 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E8=B6=85?= Date: Wed, 21 Feb 2024 18:49:57 +0800 Subject: [PATCH] planner: replace context in planbuilder to PlanContext (#51172) ref pingcap/tidb#51070, close pingcap/tidb#51171 --- pkg/executor/executor.go | 12 +++---- pkg/executor/set.go | 2 +- pkg/executor/test/executor/executor_test.go | 4 +-- pkg/lock/BUILD.bazel | 2 +- pkg/lock/context/BUILD.bazel | 9 +++++ pkg/lock/context/lockcontext.go | 40 +++++++++++++++++++++ pkg/lock/lock.go | 6 ++-- pkg/planner/BUILD.bazel | 1 + pkg/planner/cardinality/BUILD.bazel | 1 - pkg/planner/cardinality/row_count_column.go | 9 +++-- pkg/planner/cascades/BUILD.bazel | 1 - pkg/planner/cascades/optimize.go | 9 +++-- pkg/planner/context/BUILD.bazel | 3 ++ pkg/planner/context/context.go | 15 ++++++++ pkg/planner/core/BUILD.bazel | 1 + pkg/planner/core/common_plans.go | 16 ++++----- pkg/planner/core/debugtrace.go | 8 ++--- pkg/planner/core/exhaust_physical_plans.go | 3 +- pkg/planner/core/expression_rewriter.go | 5 ++- pkg/planner/core/foreign_key.go | 23 ++++++------ pkg/planner/core/fragment.go | 2 +- pkg/planner/core/logical_plan_builder.go | 8 ++--- pkg/planner/core/optimizer.go | 5 +-- pkg/planner/core/pb_to_plan.go | 5 ++- pkg/planner/core/physical_plans.go | 6 ++-- pkg/planner/core/plan.go | 9 +++++ pkg/planner/core/plan_cache.go | 8 ++--- pkg/planner/core/plan_cacheable_checker.go | 25 +++++++------ pkg/planner/core/planbuilder.go | 38 ++++++++++++-------- pkg/planner/core/point_get_plan.go | 35 +++++++++--------- pkg/planner/core/preprocess.go | 4 +-- pkg/planner/core/util.go | 3 +- pkg/planner/optimize.go | 11 +++--- pkg/planner/util/BUILD.bazel | 1 - pkg/planner/util/expression.go | 6 ++-- pkg/sessionctx/BUILD.bazel | 2 ++ pkg/sessionctx/context.go | 25 ++++--------- pkg/sessiontxn/isolation/optimistic.go | 2 +- pkg/sessiontxn/staleread/BUILD.bazel | 1 + pkg/sessiontxn/staleread/processor.go | 4 +-- pkg/sessiontxn/staleread/util.go | 8 ++--- pkg/table/column.go | 4 +-- pkg/util/ranger/BUILD.bazel | 1 - pkg/util/ranger/detacher.go | 3 +- 44 files changed, 227 insertions(+), 159 deletions(-) create mode 100644 pkg/lock/context/BUILD.bazel create mode 100644 pkg/lock/context/lockcontext.go diff --git a/pkg/executor/executor.go b/pkg/executor/executor.go index 36c4297e672c0..7bddc34fb341b 100644 --- a/pkg/executor/executor.go +++ b/pkg/executor/executor.go @@ -1470,10 +1470,10 @@ func init() { r, ctx := tracing.StartRegionEx(ctx, "executor.EvalSubQuery") defer r.End() - sctx, ok := pctx.(sessionctx.Context) - intest.Assert(ok) - if !ok { - return nil, errors.New("plan context should be sessionctx.Context to EvalSubqueryFirstRow") + sctx, err := plannercore.AsSctx(pctx) + intest.AssertNoError(err) + if err != nil { + return nil, err } e := newExecutorBuilder(sctx, is, nil) @@ -1481,7 +1481,7 @@ func init() { if e.err != nil { return nil, e.err } - err := exec.Open(ctx, executor) + err = exec.Open(ctx, executor) defer func() { terror.Log(exec.Close(executor)) }() if err != nil { return nil, err @@ -2321,7 +2321,7 @@ func setOptionForTopSQL(sc *stmtctx.StatementContext, snapshot kv.Snapshot) { func isWeakConsistencyRead(ctx sessionctx.Context, node ast.Node) bool { sessionVars := ctx.GetSessionVars() return sessionVars.ConnectionID > 0 && sessionVars.ReadConsistency.IsWeak() && - plannercore.IsAutoCommitTxn(ctx) && plannercore.IsReadOnly(node, sessionVars) + plannercore.IsAutoCommitTxn(sessionVars) && plannercore.IsReadOnly(node, sessionVars) } // FastCheckTableExec represents a check table executor. diff --git a/pkg/executor/set.go b/pkg/executor/set.go index ba552122f4eea..ffc7632b41d32 100644 --- a/pkg/executor/set.go +++ b/pkg/executor/set.go @@ -223,7 +223,7 @@ func (e *SetExecutor) setSysVariable(ctx context.Context, name string, v *expres newSnapshotIsSet := newSnapshotTS > 0 && newSnapshotTS != oldSnapshotTS if newSnapshotIsSet { if name == variable.TiDBTxnReadTS { - err = sessionctx.ValidateStaleReadTS(ctx, e.Ctx(), newSnapshotTS) + err = sessionctx.ValidateStaleReadTS(ctx, e.Ctx().GetSessionVars().StmtCtx, e.Ctx().GetStore(), newSnapshotTS) } else { err = sessionctx.ValidateSnapshotReadTS(ctx, e.Ctx(), newSnapshotTS) // Also check gc safe point for snapshot read. diff --git a/pkg/executor/test/executor/executor_test.go b/pkg/executor/test/executor/executor_test.go index c0120292685dc..ff78bdcb997b3 100644 --- a/pkg/executor/test/executor/executor_test.go +++ b/pkg/executor/test/executor/executor_test.go @@ -1942,7 +1942,7 @@ func TestIsPointGet(t *testing.T) { require.NoError(t, err) p, _, err := planner.Optimize(context.TODO(), ctx, stmtNode, preprocessorReturn.InfoSchema) require.NoError(t, err) - ret, err := plannercore.IsPointGetWithPKOrUniqueKeyByAutoCommit(ctx, p) + ret, err := plannercore.IsPointGetWithPKOrUniqueKeyByAutoCommit(ctx.GetSessionVars(), p) require.NoError(t, err) require.Equal(t, result, ret) } @@ -1983,7 +1983,7 @@ func TestClusteredIndexIsPointGet(t *testing.T) { require.NoError(t, err) p, _, err := planner.Optimize(context.TODO(), ctx, stmtNode, preprocessorReturn.InfoSchema) require.NoError(t, err) - ret, err := plannercore.IsPointGetWithPKOrUniqueKeyByAutoCommit(ctx, p) + ret, err := plannercore.IsPointGetWithPKOrUniqueKeyByAutoCommit(ctx.GetSessionVars(), p) require.NoError(t, err) require.Equal(t, result, ret) } diff --git a/pkg/lock/BUILD.bazel b/pkg/lock/BUILD.bazel index 989c953b38695..1c70f2b622626 100644 --- a/pkg/lock/BUILD.bazel +++ b/pkg/lock/BUILD.bazel @@ -7,9 +7,9 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/infoschema", + "//pkg/lock/context", "//pkg/parser/model", "//pkg/parser/mysql", - "//pkg/sessionctx", "//pkg/table", "//pkg/util", ], diff --git a/pkg/lock/context/BUILD.bazel b/pkg/lock/context/BUILD.bazel new file mode 100644 index 0000000000000..12d37593a6e6b --- /dev/null +++ b/pkg/lock/context/BUILD.bazel @@ -0,0 +1,9 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "context", + srcs = ["lockcontext.go"], + importpath = "github.com/pingcap/tidb/pkg/lock/context", + visibility = ["//visibility:public"], + deps = ["//pkg/parser/model"], +) diff --git a/pkg/lock/context/lockcontext.go b/pkg/lock/context/lockcontext.go new file mode 100644 index 0000000000000..34e3ef5a7a7a1 --- /dev/null +++ b/pkg/lock/context/lockcontext.go @@ -0,0 +1,40 @@ +// Copyright 2024 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 context + +import "github.com/pingcap/tidb/pkg/parser/model" + +// TableLockReadContext is the interface to get table lock information. +type TableLockReadContext interface { + // CheckTableLocked checks the table lock. + CheckTableLocked(tblID int64) (bool, model.TableLockType) + // GetAllTableLocks gets all table locks table id and db id hold by the session. + GetAllTableLocks() []model.TableLockTpInfo + // HasLockedTables uses to check whether this session locked any tables. + HasLockedTables() bool +} + +// TableLockContext is the interface to operate table lock. +type TableLockContext interface { + TableLockReadContext + // AddTableLock adds table lock to the session lock map. + AddTableLock([]model.TableLockTpInfo) + // ReleaseTableLocks releases table locks in the session lock map. + ReleaseTableLocks(locks []model.TableLockTpInfo) + // ReleaseTableLockByTableIDs releases table locks in the session lock map by table IDs. + ReleaseTableLockByTableIDs(tableIDs []int64) + // ReleaseAllTableLocks releases all table locks hold by the session. + ReleaseAllTableLocks() +} diff --git a/pkg/lock/lock.go b/pkg/lock/lock.go index c1064c93dbb94..ed0c87010df97 100644 --- a/pkg/lock/lock.go +++ b/pkg/lock/lock.go @@ -18,16 +18,16 @@ import ( "errors" "github.com/pingcap/tidb/pkg/infoschema" + "github.com/pingcap/tidb/pkg/lock/context" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/util" ) // Checker uses to check tables lock. type Checker struct { - ctx sessionctx.Context + ctx context.TableLockReadContext is infoschema.InfoSchema } @@ -35,7 +35,7 @@ type Checker struct { var ErrLockedTableDropped = errors.New("other table can be accessed after locked table dropped") // NewChecker return new lock Checker. -func NewChecker(ctx sessionctx.Context, is infoschema.InfoSchema) *Checker { +func NewChecker(ctx context.TableLockReadContext, is infoschema.InfoSchema) *Checker { return &Checker{ctx: ctx, is: is} } diff --git a/pkg/planner/BUILD.bazel b/pkg/planner/BUILD.bazel index cd23d78cad00d..56ef5da2c61e2 100644 --- a/pkg/planner/BUILD.bazel +++ b/pkg/planner/BUILD.bazel @@ -13,6 +13,7 @@ go_library( "//pkg/metrics", "//pkg/parser/ast", "//pkg/planner/cascades", + "//pkg/planner/context", "//pkg/planner/core", "//pkg/planner/util/debugtrace", "//pkg/privilege", diff --git a/pkg/planner/cardinality/BUILD.bazel b/pkg/planner/cardinality/BUILD.bazel index 8010960fb1f5f..af8f1416a89a4 100644 --- a/pkg/planner/cardinality/BUILD.bazel +++ b/pkg/planner/cardinality/BUILD.bazel @@ -26,7 +26,6 @@ go_library( "//pkg/planner/property", "//pkg/planner/util", "//pkg/planner/util/debugtrace", - "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", "//pkg/statistics", "//pkg/tablecodec", diff --git a/pkg/planner/cardinality/row_count_column.go b/pkg/planner/cardinality/row_count_column.go index 7a01e399d12fd..a57b5f7873cea 100644 --- a/pkg/planner/cardinality/row_count_column.go +++ b/pkg/planner/cardinality/row_count_column.go @@ -18,7 +18,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/planner/context" "github.com/pingcap/tidb/pkg/planner/util/debugtrace" - "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/codec" @@ -315,7 +314,7 @@ func betweenRowCountOnColumn(sctx context.PlanContext, c *statistics.Column, l, // functions below are mainly for testing. // ColumnGreaterRowCount estimates the row count where the column greater than value. -func ColumnGreaterRowCount(sctx sessionctx.Context, t *statistics.Table, value types.Datum, colID int64) float64 { +func ColumnGreaterRowCount(sctx context.PlanContext, t *statistics.Table, value types.Datum, colID int64) float64 { c, ok := t.Columns[colID] if !ok || c.IsInvalid(sctx, t.Pseudo) { return float64(t.RealtimeCount) / pseudoLessRate @@ -324,7 +323,7 @@ func ColumnGreaterRowCount(sctx sessionctx.Context, t *statistics.Table, value t } // columnLessRowCount estimates the row count where the column less than value. Note that null values are not counted. -func columnLessRowCount(sctx sessionctx.Context, t *statistics.Table, value types.Datum, colID int64) float64 { +func columnLessRowCount(sctx context.PlanContext, t *statistics.Table, value types.Datum, colID int64) float64 { c, ok := t.Columns[colID] if !ok || c.IsInvalid(sctx, t.Pseudo) { return float64(t.RealtimeCount) / pseudoLessRate @@ -333,7 +332,7 @@ func columnLessRowCount(sctx sessionctx.Context, t *statistics.Table, value type } // columnBetweenRowCount estimates the row count where column greater or equal to a and less than b. -func columnBetweenRowCount(sctx sessionctx.Context, t *statistics.Table, a, b types.Datum, colID int64) (float64, error) { +func columnBetweenRowCount(sctx context.PlanContext, t *statistics.Table, a, b types.Datum, colID int64) (float64, error) { sc := sctx.GetSessionVars().StmtCtx c, ok := t.Columns[colID] if !ok || c.IsInvalid(sctx, t.Pseudo) { @@ -357,7 +356,7 @@ func columnBetweenRowCount(sctx sessionctx.Context, t *statistics.Table, a, b ty } // ColumnEqualRowCount estimates the row count where the column equals to value. -func ColumnEqualRowCount(sctx sessionctx.Context, t *statistics.Table, value types.Datum, colID int64) (float64, error) { +func ColumnEqualRowCount(sctx context.PlanContext, t *statistics.Table, value types.Datum, colID int64) (float64, error) { c, ok := t.Columns[colID] if !ok || c.IsInvalid(sctx, t.Pseudo) { return float64(t.RealtimeCount) / pseudoEqualRate, nil diff --git a/pkg/planner/cascades/BUILD.bazel b/pkg/planner/cascades/BUILD.bazel index 8c75d382124c7..0105b9a459ffa 100644 --- a/pkg/planner/cascades/BUILD.bazel +++ b/pkg/planner/cascades/BUILD.bazel @@ -23,7 +23,6 @@ go_library( "//pkg/planner/memo", "//pkg/planner/property", "//pkg/planner/util", - "//pkg/sessionctx", "//pkg/types", "//pkg/util/dbterror/plannererrors", "//pkg/util/ranger", diff --git a/pkg/planner/cascades/optimize.go b/pkg/planner/cascades/optimize.go index 4066a1c068d32..6833b94bef37f 100644 --- a/pkg/planner/cascades/optimize.go +++ b/pkg/planner/cascades/optimize.go @@ -22,7 +22,6 @@ import ( plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/memo" "github.com/pingcap/tidb/pkg/planner/property" - "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/util/dbterror/plannererrors" ) @@ -98,7 +97,7 @@ func (opt *Optimizer) GetImplementationRules(node plannercore.LogicalPlan) []Imp // for each expression in each group under the required physical property. A // memo structure is used for a group to reduce the repeated search on the same // required physical property. -func (opt *Optimizer) FindBestPlan(sctx sessionctx.Context, logical plannercore.LogicalPlan) (p plannercore.PhysicalPlan, cost float64, err error) { +func (opt *Optimizer) FindBestPlan(sctx plannercore.PlanContext, logical plannercore.LogicalPlan) (p plannercore.PhysicalPlan, cost float64, err error) { logical, err = opt.onPhasePreprocessing(sctx, logical) if err != nil { return nil, 0, err @@ -116,7 +115,7 @@ func (opt *Optimizer) FindBestPlan(sctx sessionctx.Context, logical plannercore. return p, cost, err } -func (*Optimizer) onPhasePreprocessing(_ sessionctx.Context, plan plannercore.LogicalPlan) (plannercore.LogicalPlan, error) { +func (*Optimizer) onPhasePreprocessing(_ plannercore.PlanContext, plan plannercore.LogicalPlan) (plannercore.LogicalPlan, error) { err := plan.PruneColumns(plan.Schema().Columns, nil, plan) if err != nil { return nil, err @@ -124,7 +123,7 @@ func (*Optimizer) onPhasePreprocessing(_ sessionctx.Context, plan plannercore.Lo return plan, nil } -func (opt *Optimizer) onPhaseExploration(_ sessionctx.Context, g *memo.Group) error { +func (opt *Optimizer) onPhaseExploration(_ plannercore.PlanContext, g *memo.Group) error { for round, ruleBatch := range opt.transformationRuleBatches { for !g.Explored(round) { err := opt.exploreGroup(g, round, ruleBatch) @@ -241,7 +240,7 @@ func (opt *Optimizer) fillGroupStats(g *memo.Group) (err error) { } // onPhaseImplementation starts implementation physical operators from given root Group. -func (opt *Optimizer) onPhaseImplementation(_ sessionctx.Context, g *memo.Group) (plannercore.PhysicalPlan, float64, error) { +func (opt *Optimizer) onPhaseImplementation(_ plannercore.PlanContext, g *memo.Group) (plannercore.PhysicalPlan, float64, error) { prop := &property.PhysicalProperty{ ExpectedCnt: math.MaxFloat64, } diff --git a/pkg/planner/context/BUILD.bazel b/pkg/planner/context/BUILD.bazel index 77ce920dab3fd..b557968892b7d 100644 --- a/pkg/planner/context/BUILD.bazel +++ b/pkg/planner/context/BUILD.bazel @@ -7,9 +7,12 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/expression/context", + "//pkg/infoschema/context", "//pkg/kv", + "//pkg/lock/context", "//pkg/parser/model", "//pkg/sessionctx/variable", + "//pkg/util", "//pkg/util/context", ], ) diff --git a/pkg/planner/context/context.go b/pkg/planner/context/context.go index a66f82223768e..9485f01404fbc 100644 --- a/pkg/planner/context/context.go +++ b/pkg/planner/context/context.go @@ -16,9 +16,12 @@ package context import ( exprctx "github.com/pingcap/tidb/pkg/expression/context" + infoschema "github.com/pingcap/tidb/pkg/infoschema/context" "github.com/pingcap/tidb/pkg/kv" + tablelock "github.com/pingcap/tidb/pkg/lock/context" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/util" contextutil "github.com/pingcap/tidb/pkg/util/context" ) @@ -26,12 +29,24 @@ import ( type PlanContext interface { exprctx.BuildContext contextutil.ValueStoreContext + tablelock.TableLockReadContext // GetSessionVars gets the session variables. GetSessionVars() *variable.SessionVars + // GetInfoSchema returns the current infoschema + GetInfoSchema() infoschema.InfoSchemaMetaVersion // UpdateColStatsUsage updates the column stats usage. UpdateColStatsUsage(predicateColumns []model.TableItemID) // GetClient gets a kv.Client. GetClient() kv.Client // GetMPPClient gets a kv.MPPClient. GetMPPClient() kv.MPPClient + // GetSessionManager gets the session manager. + GetSessionManager() util.SessionManager + // Txn returns the current transaction which is created before executing a statement. + // The returned kv.Transaction is not nil, but it maybe pending or invalid. + // If the active parameter is true, call this function will wait for the pending txn + // to become valid. + Txn(active bool) (kv.Transaction, error) + // HasDirtyContent checks whether there's dirty update on the given table. + HasDirtyContent(tid int64) bool } diff --git a/pkg/planner/core/BUILD.bazel b/pkg/planner/core/BUILD.bazel index 7488c4b1c2eb3..5e0b171de7063 100644 --- a/pkg/planner/core/BUILD.bazel +++ b/pkg/planner/core/BUILD.bazel @@ -95,6 +95,7 @@ go_library( "//pkg/infoschema", "//pkg/kv", "//pkg/lock", + "//pkg/lock/context", "//pkg/meta/autoid", "//pkg/metrics", "//pkg/parser", diff --git a/pkg/planner/core/common_plans.go b/pkg/planner/core/common_plans.go index 3b9af0eea6650..97f7542848363 100644 --- a/pkg/planner/core/common_plans.go +++ b/pkg/planner/core/common_plans.go @@ -28,7 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/property" - "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" @@ -205,7 +205,7 @@ type Execute struct { // Check if result of GetVar expr is BinaryLiteral // Because GetVar use String to represent BinaryLiteral, here we need to convert string back to BinaryLiteral. -func isGetVarBinaryLiteral(sctx sessionctx.Context, expr expression.Expression) (res bool) { +func isGetVarBinaryLiteral(sctx PlanContext, expr expression.Expression) (res bool) { scalarFunc, ok := expr.(*expression.ScalarFunction) if ok && scalarFunc.FuncName.L == ast.GetVar { name, isNull, err := scalarFunc.GetArgs()[0].EvalString(sctx, chunk.Row{}) @@ -1392,8 +1392,8 @@ func (e *Explain) prepareTaskDot(p PhysicalPlan, taskTp string, buffer *bytes.Bu // 1. ctx is auto commit tagged // 2. session is not InTxn // 3. plan is point get by pk, or point get by unique index (no double read) -func IsPointGetWithPKOrUniqueKeyByAutoCommit(ctx sessionctx.Context, p Plan) (bool, error) { - if !IsAutoCommitTxn(ctx) { +func IsPointGetWithPKOrUniqueKeyByAutoCommit(vars *variable.SessionVars, p Plan) (bool, error) { + if !IsAutoCommitTxn(vars) { return false, nil } @@ -1405,13 +1405,13 @@ func IsPointGetWithPKOrUniqueKeyByAutoCommit(ctx sessionctx.Context, p Plan) (bo switch v := p.(type) { case *PhysicalIndexReader: indexScan := v.IndexPlans[0].(*PhysicalIndexScan) - return indexScan.IsPointGetByUniqueKey(ctx), nil + return indexScan.IsPointGetByUniqueKey(vars.StmtCtx.TypeCtx()), nil case *PhysicalTableReader: tableScan, ok := v.TablePlans[0].(*PhysicalTableScan) if !ok { return false, nil } - isPointRange := len(tableScan.Ranges) == 1 && tableScan.Ranges[0].IsPointNonNullable(ctx.GetSessionVars().StmtCtx.TypeCtx()) + isPointRange := len(tableScan.Ranges) == 1 && tableScan.Ranges[0].IsPointNonNullable(vars.StmtCtx.TypeCtx()) if !isPointRange { return false, nil } @@ -1442,8 +1442,8 @@ func IsPointGetWithPKOrUniqueKeyByAutoCommit(ctx sessionctx.Context, p Plan) (bo // IsAutoCommitTxn checks if session is in autocommit mode and not InTxn // used for fast plan like point get -func IsAutoCommitTxn(ctx sessionctx.Context) bool { - return ctx.GetSessionVars().IsAutocommit() && !ctx.GetSessionVars().InTxn() +func IsAutoCommitTxn(vars *variable.SessionVars) bool { + return vars.IsAutocommit() && !vars.InTxn() } // AdminShowBDRRole represents a show bdr role plan. diff --git a/pkg/planner/core/debugtrace.go b/pkg/planner/core/debugtrace.go index 7262f38cac1fc..fe7b2e195c2e8 100644 --- a/pkg/planner/core/debugtrace.go +++ b/pkg/planner/core/debugtrace.go @@ -23,9 +23,9 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/planner/context" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/debugtrace" - "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/util/hint" ) @@ -72,7 +72,7 @@ func (info *binaryParamInfo) MarshalJSON() ([]byte, error) { } // DebugTraceReceivedCommand records the received command from the client to the debug trace. -func DebugTraceReceivedCommand(s sessionctx.Context, cmd byte, stmtNode ast.StmtNode) { +func DebugTraceReceivedCommand(s PlanContext, cmd byte, stmtNode ast.StmtNode) { sessionVars := s.GetSessionVars() trace := debugtrace.GetOrInitDebugTraceRoot(s) traceInfo := new(receivedCmdInfo) @@ -135,7 +135,7 @@ func (b *bindingHint) MarshalJSON() ([]byte, error) { } // DebugTraceTryBinding records the hint that might be chosen to the debug trace. -func DebugTraceTryBinding(s sessionctx.Context, binding *hint.HintsSet) { +func DebugTraceTryBinding(s context.PlanContext, binding *hint.HintsSet) { root := debugtrace.GetOrInitDebugTraceRoot(s) traceInfo := &bindingHint{ Hint: binding, @@ -145,7 +145,7 @@ func DebugTraceTryBinding(s sessionctx.Context, binding *hint.HintsSet) { } // DebugTraceBestBinding records the chosen hint to the debug trace. -func DebugTraceBestBinding(s sessionctx.Context, binding *hint.HintsSet) { +func DebugTraceBestBinding(s context.PlanContext, binding *hint.HintsSet) { root := debugtrace.GetOrInitDebugTraceRoot(s) traceInfo := &bindingHint{ Hint: binding, diff --git a/pkg/planner/core/exhaust_physical_plans.go b/pkg/planner/core/exhaust_physical_plans.go index 0980eaa3bdba4..c04e078771198 100644 --- a/pkg/planner/core/exhaust_physical_plans.go +++ b/pkg/planner/core/exhaust_physical_plans.go @@ -34,7 +34,6 @@ import ( "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/fixcontrol" - "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -1471,7 +1470,7 @@ func (cwc *ColWithCmpFuncManager) CompareRow(lhs, rhs chunk.Row) int { } // BuildRangesByRow will build range of the given row. It will eval each function's arg then call BuildRange. -func (cwc *ColWithCmpFuncManager) BuildRangesByRow(ctx sessionctx.Context, row chunk.Row) ([]*ranger.Range, error) { +func (cwc *ColWithCmpFuncManager) BuildRangesByRow(ctx PlanContext, row chunk.Row) ([]*ranger.Range, error) { exprs := make([]expression.Expression, len(cwc.OpType)) for i, opType := range cwc.OpType { constantArg, err := cwc.opArg[i].Eval(ctx, row) diff --git a/pkg/planner/core/expression_rewriter.go b/pkg/planner/core/expression_rewriter.go index ef6fdfb01ec70..2a5daf326ad17 100644 --- a/pkg/planner/core/expression_rewriter.go +++ b/pkg/planner/core/expression_rewriter.go @@ -31,7 +31,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/opcode" - "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" @@ -54,7 +53,7 @@ var EvalSubqueryFirstRow func(ctx context.Context, p PhysicalPlan, is infoschema // evalAstExprWithPlanCtx evaluates ast expression with plan context. // Different with expression.EvalSimpleAst, it uses planner context and is more powerful to build some special expressions // like subquery, window function, etc. -func evalAstExprWithPlanCtx(sctx sessionctx.Context, expr ast.ExprNode) (types.Datum, error) { +func evalAstExprWithPlanCtx(sctx PlanContext, expr ast.ExprNode) (types.Datum, error) { if val, ok := expr.(*driver.ValueExpr); ok { return val.Datum, nil } @@ -80,7 +79,7 @@ func evalAstExpr(ctx expression.BuildContext, expr ast.ExprNode) (types.Datum, e // rewriteAstExprWithPlanCtx rewrites ast expression directly. // Different with expression.BuildSimpleExpr, it uses planner context and is more powerful to build some special expressions // like subquery, window function, etc. -func rewriteAstExprWithPlanCtx(sctx sessionctx.Context, expr ast.ExprNode, schema *expression.Schema, names types.NameSlice, allowCastArray bool) (expression.Expression, error) { +func rewriteAstExprWithPlanCtx(sctx PlanContext, expr ast.ExprNode, schema *expression.Schema, names types.NameSlice, allowCastArray bool) (expression.Expression, error) { var is infoschema.InfoSchema // in tests, it may be null if s, ok := sctx.GetInfoSchema().(infoschema.InfoSchema); ok { diff --git a/pkg/planner/core/foreign_key.go b/pkg/planner/core/foreign_key.go index 3375b86f73e61..653956d0cba9c 100644 --- a/pkg/planner/core/foreign_key.go +++ b/pkg/planner/core/foreign_key.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/util/dbterror/plannererrors" ) @@ -141,7 +140,7 @@ func (f *FKCascade) MemoryUsage() (sum int64) { return } -func (p *Insert) buildOnInsertFKTriggers(ctx sessionctx.Context, is infoschema.InfoSchema, dbName string) error { +func (p *Insert) buildOnInsertFKTriggers(ctx PlanContext, is infoschema.InfoSchema, dbName string) error { if !ctx.GetSessionVars().ForeignKeyChecks { return nil } @@ -198,7 +197,7 @@ func (p *Insert) buildOnDuplicateUpdateColumns() map[string]struct{} { return m } -func (*Insert) buildOnReplaceReferredFKTriggers(ctx sessionctx.Context, is infoschema.InfoSchema, dbName string, tblInfo *model.TableInfo) ([]*FKCheck, []*FKCascade, error) { +func (*Insert) buildOnReplaceReferredFKTriggers(ctx PlanContext, is infoschema.InfoSchema, dbName string, tblInfo *model.TableInfo) ([]*FKCheck, []*FKCascade, error) { referredFKs := is.GetTableReferredForeignKeys(dbName, tblInfo.Name.L) fkChecks := make([]*FKCheck, 0, len(referredFKs)) fkCascades := make([]*FKCascade, 0, len(referredFKs)) @@ -217,7 +216,7 @@ func (*Insert) buildOnReplaceReferredFKTriggers(ctx sessionctx.Context, is infos return fkChecks, fkCascades, nil } -func (updt *Update) buildOnUpdateFKTriggers(ctx sessionctx.Context, is infoschema.InfoSchema, tblID2table map[int64]table.Table) error { +func (updt *Update) buildOnUpdateFKTriggers(ctx PlanContext, is infoschema.InfoSchema, tblID2table map[int64]table.Table) error { if !ctx.GetSessionVars().ForeignKeyChecks { return nil } @@ -258,7 +257,7 @@ func (updt *Update) buildOnUpdateFKTriggers(ctx sessionctx.Context, is infoschem return nil } -func (del *Delete) buildOnDeleteFKTriggers(ctx sessionctx.Context, is infoschema.InfoSchema, tblID2table map[int64]table.Table) error { +func (del *Delete) buildOnDeleteFKTriggers(ctx PlanContext, is infoschema.InfoSchema, tblID2table map[int64]table.Table) error { if !ctx.GetSessionVars().ForeignKeyChecks { return nil } @@ -289,7 +288,7 @@ func (del *Delete) buildOnDeleteFKTriggers(ctx sessionctx.Context, is infoschema return nil } -func buildOnUpdateReferredFKTriggers(ctx sessionctx.Context, is infoschema.InfoSchema, dbName string, tblInfo *model.TableInfo, updateCols map[string]struct{}) ([]*FKCheck, []*FKCascade, error) { +func buildOnUpdateReferredFKTriggers(ctx PlanContext, is infoschema.InfoSchema, dbName string, tblInfo *model.TableInfo, updateCols map[string]struct{}) ([]*FKCheck, []*FKCascade, error) { referredFKs := is.GetTableReferredForeignKeys(dbName, tblInfo.Name.L) fkChecks := make([]*FKCheck, 0, len(referredFKs)) fkCascades := make([]*FKCascade, 0, len(referredFKs)) @@ -311,7 +310,7 @@ func buildOnUpdateReferredFKTriggers(ctx sessionctx.Context, is infoschema.InfoS return fkChecks, fkCascades, nil } -func buildOnUpdateChildFKChecks(ctx sessionctx.Context, is infoschema.InfoSchema, dbName string, tblInfo *model.TableInfo, updateCols map[string]struct{}) ([]*FKCheck, error) { +func buildOnUpdateChildFKChecks(ctx PlanContext, is infoschema.InfoSchema, dbName string, tblInfo *model.TableInfo, updateCols map[string]struct{}) ([]*FKCheck, error) { fkChecks := make([]*FKCheck, 0, len(tblInfo.ForeignKeys)) for _, fk := range tblInfo.ForeignKeys { if fk.Version < 1 { @@ -366,7 +365,7 @@ func (updt *Update) buildTbl2UpdateColumns() map[int64]map[string]struct{} { return tblID2UpdateColumns } -func buildOnDeleteOrUpdateFKTrigger(ctx sessionctx.Context, is infoschema.InfoSchema, referredFK *model.ReferredFKInfo, tp FKCascadeType) (*FKCheck, *FKCascade, error) { +func buildOnDeleteOrUpdateFKTrigger(ctx PlanContext, is infoschema.InfoSchema, referredFK *model.ReferredFKInfo, tp FKCascadeType) (*FKCheck, *FKCascade, error) { childTable, err := is.TableByName(referredFK.ChildSchema, referredFK.ChildTable) if err != nil { return nil, nil, nil @@ -406,7 +405,7 @@ func isMapContainAnyCols(colsMap map[string]struct{}, cols ...model.CIStr) bool return false } -func buildFKCheckOnModifyChildTable(ctx sessionctx.Context, is infoschema.InfoSchema, fk *model.FKInfo, failedErr error) (*FKCheck, error) { +func buildFKCheckOnModifyChildTable(ctx PlanContext, is infoschema.InfoSchema, fk *model.FKInfo, failedErr error) (*FKCheck, error) { referTable, err := is.TableByName(fk.RefSchema, fk.RefTable) if err != nil { return nil, nil @@ -420,7 +419,7 @@ func buildFKCheckOnModifyChildTable(ctx sessionctx.Context, is infoschema.InfoSc return fkCheck, nil } -func buildFKCheckForReferredFK(ctx sessionctx.Context, childTable table.Table, fk *model.FKInfo, referredFK *model.ReferredFKInfo) (*FKCheck, error) { +func buildFKCheckForReferredFK(ctx PlanContext, childTable table.Table, fk *model.FKInfo, referredFK *model.ReferredFKInfo) (*FKCheck, error) { failedErr := plannererrors.ErrRowIsReferenced2.GenWithStackByArgs(fk.String(referredFK.ChildSchema.L, referredFK.ChildTable.L)) fkCheck, err := buildFKCheck(ctx, childTable, fk.Cols, failedErr) if err != nil { @@ -431,7 +430,7 @@ func buildFKCheckForReferredFK(ctx sessionctx.Context, childTable table.Table, f return fkCheck, nil } -func buildFKCheck(ctx sessionctx.Context, tbl table.Table, cols []model.CIStr, failedErr error) (*FKCheck, error) { +func buildFKCheck(ctx PlanContext, tbl table.Table, cols []model.CIStr, failedErr error) (*FKCheck, error) { tblInfo := tbl.Meta() if tblInfo.PKIsHandle && len(cols) == 1 { refColInfo := model.FindColumnInfo(tblInfo.Columns, cols[0].L) @@ -468,7 +467,7 @@ func buildFKCheck(ctx sessionctx.Context, tbl table.Table, cols []model.CIStr, f }.Init(ctx), nil } -func buildFKCascade(ctx sessionctx.Context, tp FKCascadeType, referredFK *model.ReferredFKInfo, childTable table.Table, fk *model.FKInfo) (*FKCascade, error) { +func buildFKCascade(ctx PlanContext, tp FKCascadeType, referredFK *model.ReferredFKInfo, childTable table.Table, fk *model.FKInfo) (*FKCascade, error) { cols := make([]*model.ColumnInfo, len(fk.Cols)) childTableColumns := childTable.Meta().Columns for i, c := range fk.Cols { diff --git a/pkg/planner/core/fragment.go b/pkg/planner/core/fragment.go index 95eea5bd349ad..f812e90b823d4 100644 --- a/pkg/planner/core/fragment.go +++ b/pkg/planner/core/fragment.go @@ -487,7 +487,7 @@ func (e *mppTaskGenerator) addReaderTasksForCTEStorage(storageID int, tasks ...* } } -func partitionPruning(ctx sessionctx.Context, tbl table.PartitionedTable, conds []expression.Expression, partitionNames []model.CIStr, +func partitionPruning(ctx PlanContext, tbl table.PartitionedTable, conds []expression.Expression, partitionNames []model.CIStr, columns []*expression.Column, columnNames types.NameSlice) ([]table.PhysicalTable, error) { idxArr, err := PartitionPruning(ctx, tbl, conds, partitionNames, columns, columnNames) if err != nil { diff --git a/pkg/planner/core/logical_plan_builder.go b/pkg/planner/core/logical_plan_builder.go index 580f654b6c216..42ef9ac26f97a 100644 --- a/pkg/planner/core/logical_plan_builder.go +++ b/pkg/planner/core/logical_plan_builder.go @@ -80,7 +80,7 @@ const ( // aggOrderByResolver is currently resolving expressions of order by clause // in aggregate function GROUP_CONCAT. type aggOrderByResolver struct { - ctx sessionctx.Context + ctx PlanContext err error args []ast.ExprNode exprDepth int // exprDepth is the depth of current expression in expression tree. @@ -2376,7 +2376,7 @@ CheckReferenced: // getUintFromNode gets uint64 value from ast.Node. // For ordinary statement, node should be uint64 constant value. // For prepared statement, node is string. We should convert it to uint64. -func getUintFromNode(ctx sessionctx.Context, n ast.Node, mustInt64orUint64 bool) (uVal uint64, isNull bool, isExpectedType bool) { +func getUintFromNode(ctx PlanContext, n ast.Node, mustInt64orUint64 bool) (uVal uint64, isNull bool, isExpectedType bool) { var val any switch v := n.(type) { case *driver.ValueExpr: @@ -2438,7 +2438,7 @@ func CheckParamTypeInt64orUint64(param *driver.ParamMarkerExpr) (bool, uint64) { return false, 0 } -func extractLimitCountOffset(ctx sessionctx.Context, limit *ast.Limit) (count uint64, +func extractLimitCountOffset(ctx PlanContext, limit *ast.Limit) (count uint64, offset uint64, err error) { var isExpectedType bool if limit.Count != nil { @@ -3213,7 +3213,7 @@ func (b *PlanBuilder) resolveCorrelatedAggregates(ctx context.Context, sel *ast. // gbyResolver resolves group by items from select fields. type gbyResolver struct { - ctx sessionctx.Context + ctx PlanContext fields []*ast.SelectField schema *expression.Schema names []*types.FieldName diff --git a/pkg/planner/core/optimizer.go b/pkg/planner/core/optimizer.go index e25333e8c429f..57f8439af1731 100644 --- a/pkg/planner/core/optimizer.go +++ b/pkg/planner/core/optimizer.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/lock" + tablelock "github.com/pingcap/tidb/pkg/lock/context" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/model" @@ -176,7 +177,7 @@ type logicalOptRule interface { } // BuildLogicalPlanForTest builds a logical plan for testing purpose from ast.Node. -func BuildLogicalPlanForTest(ctx context.Context, sctx sessionctx.Context, node ast.Node, infoSchema infoschema.InfoSchema) (Plan, error) { +func BuildLogicalPlanForTest(ctx context.Context, sctx PlanContext, node ast.Node, infoSchema infoschema.InfoSchema) (Plan, error) { sctx.GetSessionVars().PlanID.Store(0) sctx.GetSessionVars().PlanColumnID.Store(0) builder, _ := NewPlanBuilder().Init(sctx, infoSchema, utilhint.NewQBHintHandler(nil)) @@ -281,7 +282,7 @@ func needCheckTmpTablePriv(is infoschema.InfoSchema, v visitInfo) bool { } // CheckTableLock checks the table lock. -func CheckTableLock(ctx sessionctx.Context, is infoschema.InfoSchema, vs []visitInfo) error { +func CheckTableLock(ctx tablelock.TableLockReadContext, is infoschema.InfoSchema, vs []visitInfo) error { if !config.TableLockEnabled() { return nil } diff --git a/pkg/planner/core/pb_to_plan.go b/pkg/planner/core/pb_to_plan.go index cb5b41854f255..04dcb78a24e7d 100644 --- a/pkg/planner/core/pb_to_plan.go +++ b/pkg/planner/core/pb_to_plan.go @@ -26,21 +26,20 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" - "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tipb/go-tipb" ) // PBPlanBuilder uses to build physical plan from dag protocol buffers. type PBPlanBuilder struct { - sctx sessionctx.Context + sctx PlanContext tps []*types.FieldType is infoschema.InfoSchema ranges []*coprocessor.KeyRange } // NewPBPlanBuilder creates a new pb plan builder. -func NewPBPlanBuilder(sctx sessionctx.Context, is infoschema.InfoSchema, ranges []*coprocessor.KeyRange) *PBPlanBuilder { +func NewPBPlanBuilder(sctx PlanContext, is infoschema.InfoSchema, ranges []*coprocessor.KeyRange) *PBPlanBuilder { return &PBPlanBuilder{sctx: sctx, is: is, ranges: ranges} } diff --git a/pkg/planner/core/physical_plans.go b/pkg/planner/core/physical_plans.go index 043b9aaec886b..c156a9b386e79 100644 --- a/pkg/planner/core/physical_plans.go +++ b/pkg/planner/core/physical_plans.go @@ -2146,11 +2146,11 @@ func (p *PhysicalIndexScan) IsPartition() (bool, int64) { } // IsPointGetByUniqueKey checks whether is a point get by unique key. -func (p *PhysicalIndexScan) IsPointGetByUniqueKey(sctx sessionctx.Context) bool { +func (p *PhysicalIndexScan) IsPointGetByUniqueKey(tc types.Context) bool { return len(p.Ranges) == 1 && p.Index.Unique && len(p.Ranges[0].LowVal) == len(p.Index.Columns) && - p.Ranges[0].IsPointNonNullable(sctx.GetSessionVars().StmtCtx.TypeCtx()) + p.Ranges[0].IsPointNonNullable(tc) } // PhysicalSelection represents a filter. @@ -2488,7 +2488,7 @@ func (p *PhysicalShowDDLJobs) MemoryUsage() (sum int64) { } // BuildMergeJoinPlan builds a PhysicalMergeJoin from the given fields. Currently, it is only used for test purpose. -func BuildMergeJoinPlan(ctx sessionctx.Context, joinType JoinType, leftKeys, rightKeys []*expression.Column) *PhysicalMergeJoin { +func BuildMergeJoinPlan(ctx PlanContext, joinType JoinType, leftKeys, rightKeys []*expression.Column) *PhysicalMergeJoin { baseJoin := basePhysicalJoin{ JoinType: joinType, DefaultValues: []types.Datum{types.NewDatum(1), types.NewDatum(1)}, diff --git a/pkg/planner/core/plan.go b/pkg/planner/core/plan.go index 3a1294412ede2..ef838d54ec2ab 100644 --- a/pkg/planner/core/plan.go +++ b/pkg/planner/core/plan.go @@ -38,6 +38,15 @@ import ( // PlanContext is the context for building plan. type PlanContext = context.PlanContext +// AsSctx converts PlanContext to sessionctx.Context. +func AsSctx(pctx PlanContext) (sessionctx.Context, error) { + sctx, ok := pctx.(sessionctx.Context) + if !ok { + return nil, errors.New("the current PlanContext cannot be converted to sessionctx.Context") + } + return sctx, nil +} + // Plan is the description of an execution flow. // It is created from ast.Node first, then optimized by the optimizer, // finally used by the executor to create a Cursor which executes the statement. diff --git a/pkg/planner/core/plan_cache.go b/pkg/planner/core/plan_cache.go index 5b56c9c621a80..d86672594284d 100644 --- a/pkg/planner/core/plan_cache.go +++ b/pkg/planner/core/plan_cache.go @@ -54,7 +54,7 @@ type PlanCacheKeyTestIssue46760 struct{} type PlanCacheKeyTestIssue47133 struct{} // SetParameterValuesIntoSCtx sets these parameters into session context. -func SetParameterValuesIntoSCtx(sctx sessionctx.Context, isNonPrep bool, markers []ast.ParamMarkerExpr, params []expression.Expression) error { +func SetParameterValuesIntoSCtx(sctx PlanContext, isNonPrep bool, markers []ast.ParamMarkerExpr, params []expression.Expression) error { vars := sctx.GetSessionVars() vars.PlanCacheParams.Reset() for i, usingParam := range params { @@ -758,7 +758,7 @@ func CheckPreparedPriv(sctx sessionctx.Context, stmt *PlanCacheStmt, is infosche // 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, +func tryCachePointPlan(_ context.Context, sctx PlanContext, stmt *PlanCacheStmt, p Plan, names types.NameSlice) error { if !sctx.GetSessionVars().StmtCtx.UseCache { return nil @@ -770,7 +770,7 @@ func tryCachePointPlan(_ context.Context, sctx sessionctx.Context, ) if plan, _ok := p.(*PointGetPlan); _ok { - ok, err = IsPointGetWithPKOrUniqueKeyByAutoCommit(sctx, p) + ok, err = IsPointGetWithPKOrUniqueKeyByAutoCommit(sctx.GetSessionVars(), p) if err != nil { return err } @@ -799,7 +799,7 @@ func IsPointGetPlanShortPathOK(sctx sessionctx.Context, is infoschema.InfoSchema return false, nil } // check auto commit - if !IsAutoCommitTxn(sctx) { + if !IsAutoCommitTxn(sctx.GetSessionVars()) { return false, nil } if stmtAst.SchemaVersion != is.SchemaMetaVersion() { diff --git a/pkg/planner/core/plan_cacheable_checker.go b/pkg/planner/core/plan_cacheable_checker.go index aa539aee07664..c52ee52f2339f 100644 --- a/pkg/planner/core/plan_cacheable_checker.go +++ b/pkg/planner/core/plan_cacheable_checker.go @@ -30,7 +30,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" core_metrics "github.com/pingcap/tidb/pkg/planner/core/metrics" "github.com/pingcap/tidb/pkg/planner/util/fixcontrol" - "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/types" driver "github.com/pingcap/tidb/pkg/types/parser_driver" "github.com/pingcap/tidb/pkg/util/filter" @@ -49,14 +48,14 @@ func Cacheable(node ast.Node, is infoschema.InfoSchema) bool { // CacheableWithCtx checks whether the input ast(query) is cacheable. // TODO: only for test, remove this function later on. -func CacheableWithCtx(sctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema) (bool, string) { +func CacheableWithCtx(sctx PlanContext, node ast.Node, is infoschema.InfoSchema) (bool, string) { return IsASTCacheable(nil, sctx, node, is) } // IsASTCacheable checks whether the input ast(query) is cacheable. // Handle "ignore_plan_cache()" hint // If there are multiple hints, only one will take effect -func IsASTCacheable(ctx context.Context, sctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema) (bool, string) { +func IsASTCacheable(ctx context.Context, sctx PlanContext, node ast.Node, is infoschema.InfoSchema) (bool, string) { switch node.(type) { case *ast.SelectStmt, *ast.UpdateStmt, *ast.InsertStmt, *ast.DeleteStmt, *ast.SetOprStmt: default: @@ -77,7 +76,7 @@ func IsASTCacheable(ctx context.Context, sctx sessionctx.Context, node ast.Node, // cacheableChecker checks whether a query can be cached: type cacheableChecker struct { ctx context.Context - sctx sessionctx.Context + sctx PlanContext cacheable bool schema infoschema.InfoSchema reason string // reason why cannot use plan-cache @@ -213,7 +212,7 @@ func (checker *cacheableChecker) Leave(in ast.Node) (out ast.Node, ok bool) { var nonPrepCacheCheckerPool = &sync.Pool{New: func() any { return &nonPreparedPlanCacheableChecker{} }} // NonPreparedPlanCacheableWithCtx checks whether this SQL is cacheable for non-prepared plan cache. -func NonPreparedPlanCacheableWithCtx(sctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema) (ok bool, reason string) { +func NonPreparedPlanCacheableWithCtx(sctx PlanContext, node ast.Node, is infoschema.InfoSchema) (ok bool, reason string) { selStmt, isSelect := node.(*ast.SelectStmt) if !sctx.GetSessionVars().EnableNonPreparedPlanCacheForDML && (!isSelect || selStmt.LockInfo != nil) { @@ -303,7 +302,7 @@ func NonPreparedPlanCacheableWithCtx(sctx sessionctx.Context, node ast.Node, is } // isSelectStmtNonPrepCacheableFastCheck checks whether the input select statement is cacheable for non-prepared plan cache. -func isSelectStmtNonPrepCacheableFastCheck(sctx sessionctx.Context, selectStmt *ast.SelectStmt) (names []*ast.TableName, ok bool, reason string) { +func isSelectStmtNonPrepCacheableFastCheck(sctx PlanContext, selectStmt *ast.SelectStmt) (names []*ast.TableName, ok bool, reason string) { if selectStmt.Kind != ast.SelectStmtKindSelect { return nil, false, "not a select statement" } @@ -371,7 +370,7 @@ func extractTableNames(node ast.ResultSetNode, names []*ast.TableName) ([]*ast.T // nonPreparedPlanCacheableChecker checks whether a query's plan can be cached for non-prepared plan cache. // NOTE: we can add more rules in the future. type nonPreparedPlanCacheableChecker struct { - sctx sessionctx.Context + sctx PlanContext cacheable bool reason string // reason why this statement cannot hit the cache schema infoschema.InfoSchema @@ -384,7 +383,7 @@ type nonPreparedPlanCacheableChecker struct { maxNumberParam int // the maximum number of parameters for a query to be cached. } -func (checker *nonPreparedPlanCacheableChecker) reset(sctx sessionctx.Context, schema infoschema.InfoSchema, tableNodes []*ast.TableName, maxNumberParam int) { +func (checker *nonPreparedPlanCacheableChecker) reset(sctx PlanContext, schema infoschema.InfoSchema, tableNodes []*ast.TableName, maxNumberParam int) { checker.sctx = sctx checker.cacheable = true checker.schema = schema @@ -532,7 +531,7 @@ func getColType(schema infoschema.InfoSchema, tbl *ast.TableName, col *ast.Colum } // isPlanCacheable returns whether this plan is cacheable and the reason if not. -func isPlanCacheable(sctx sessionctx.Context, p Plan, paramNum, limitParamNum int, hasSubQuery bool) (cacheable bool, reason string) { +func isPlanCacheable(sctx PlanContext, p Plan, paramNum, limitParamNum int, hasSubQuery bool) (cacheable bool, reason string) { var pp PhysicalPlan switch x := p.(type) { case *Insert: @@ -562,7 +561,7 @@ func isPlanCacheable(sctx sessionctx.Context, p Plan, paramNum, limitParamNum in } // isPhysicalPlanCacheable returns whether this physical plan is cacheable and return the reason if not. -func isPhysicalPlanCacheable(sctx sessionctx.Context, p PhysicalPlan, paramNum, limitParamNum int, underIndexMerge bool) (cacheable bool, reason string) { +func isPhysicalPlanCacheable(sctx PlanContext, p PhysicalPlan, paramNum, limitParamNum int, underIndexMerge bool) (cacheable bool, reason string) { var subPlans []PhysicalPlan switch x := p.(type) { case *PhysicalTableDual: @@ -605,7 +604,7 @@ func isPhysicalPlanCacheable(sctx sessionctx.Context, p PhysicalPlan, paramNum, } // getMaxParamLimit returns the maximum number of parameters for a query that can be cached in the Plan Cache. -func getMaxParamLimit(sctx sessionctx.Context) int { +func getMaxParamLimit(sctx PlanContext) int { v := 200 if sctx == nil || sctx.GetSessionVars() == nil || sctx.GetSessionVars().OptimizerFixControl == nil { return v @@ -620,7 +619,7 @@ func getMaxParamLimit(sctx sessionctx.Context) int { return v } -func enablePlanCacheForGeneratedCols(sctx sessionctx.Context) bool { +func enablePlanCacheForGeneratedCols(sctx PlanContext) bool { // disable this by default since it's not well tested. // TODO: complete its test and enable it by default. if sctx == nil || sctx.GetSessionVars() == nil || sctx.GetSessionVars().GetOptimizerFixControlMap() == nil { @@ -630,7 +629,7 @@ func enablePlanCacheForGeneratedCols(sctx sessionctx.Context) bool { } // checkTableCacheable checks whether a query accessing this table is cacheable. -func checkTableCacheable(ctx context.Context, sctx sessionctx.Context, schema infoschema.InfoSchema, node *ast.TableName, isNonPrep bool) (cacheable bool, reason string) { +func checkTableCacheable(ctx context.Context, sctx PlanContext, schema infoschema.InfoSchema, node *ast.TableName, isNonPrep bool) (cacheable bool, reason string) { tableSchema := node.Schema if tableSchema.L == "" { tableSchema.O = sctx.GetSessionVars().CurrentDB diff --git a/pkg/planner/core/planbuilder.go b/pkg/planner/core/planbuilder.go index 3a1a8e09f94b0..fa3928dcec33e 100644 --- a/pkg/planner/core/planbuilder.go +++ b/pkg/planner/core/planbuilder.go @@ -200,7 +200,7 @@ const ( // PlanBuilder builds Plan from an ast.Node. // It just builds the ast node straightforwardly. type PlanBuilder struct { - ctx sessionctx.Context + ctx PlanContext is infoschema.InfoSchema outerSchemas []*expression.Schema outerNames [][]*types.FieldName @@ -444,7 +444,7 @@ func NewPlanBuilder(opts ...PlanBuilderOpt) *PlanBuilder { // PlannerSelectBlockAsName should be restored after using this builder. // This is The comman code pattern to use it: // NewPlanBuilder().Init(sctx, is, processor) -func (b *PlanBuilder) Init(sctx sessionctx.Context, is infoschema.InfoSchema, processor *hint.QBHintHandler) (*PlanBuilder, []ast.HintTable) { +func (b *PlanBuilder) Init(sctx PlanContext, is infoschema.InfoSchema, processor *hint.QBHintHandler) (*PlanBuilder, []ast.HintTable) { savedBlockNames := sctx.GetSessionVars().PlannerSelectBlockAsName.Load() if processor == nil { sctx.GetSessionVars().PlannerSelectBlockAsName.Store(&[]ast.HintTable{}) @@ -806,7 +806,7 @@ func checkHintedSQL(sql, charset, collation, db string) error { return nil } -func fetchRecordFromClusterStmtSummary(sctx sessionctx.Context, planDigest string) ([]chunk.Row, error) { +func fetchRecordFromClusterStmtSummary(sctx PlanContext, planDigest string) ([]chunk.Row, error) { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBindInfo) exec, _ := sctx.(sqlexec.SQLExecutor) fields := "stmt_type, schema_name, digest_text, sample_user, prepared, query_sample_text, charset, collation, plan_hint, plan_digest" @@ -1819,7 +1819,7 @@ func getColsInfo(tn *ast.TableName) (indicesInfo []*model.IndexInfo, colsInfo [] } // BuildHandleColsForAnalyze returns HandleCols for ANALYZE. -func BuildHandleColsForAnalyze(ctx sessionctx.Context, tblInfo *model.TableInfo, allColumns bool, colsInfo []*model.ColumnInfo) HandleCols { +func BuildHandleColsForAnalyze(ctx PlanContext, tblInfo *model.TableInfo, allColumns bool, colsInfo []*model.ColumnInfo) HandleCols { var handleCols HandleCols switch { case tblInfo.PKIsHandle: @@ -3400,7 +3400,7 @@ func (b *PlanBuilder) buildSimple(ctx context.Context, node ast.StmtNode) (Plan, if err != nil { return nil, err } - if err := sessionctx.ValidateStaleReadTS(ctx, b.ctx, startTS); err != nil { + if err := sessionctx.ValidateStaleReadTS(ctx, b.ctx.GetSessionVars().StmtCtx, b.ctx.GetStore(), startTS); err != nil { return nil, err } p.StaleTxnStartTS = startTS @@ -3410,11 +3410,11 @@ func (b *PlanBuilder) buildSimple(ctx context.Context, node ast.StmtNode) (Plan, b.ctx.GetSessionVars().TxnReadTS.UseTxnReadTS() } else if b.ctx.GetSessionVars().EnableExternalTSRead && !b.ctx.GetSessionVars().InRestrictedSQL { // try to get the stale ts from external timestamp - startTS, err := staleread.GetExternalTimestamp(ctx, b.ctx) + startTS, err := staleread.GetExternalTimestamp(ctx, b.ctx.GetSessionVars().StmtCtx) if err != nil { return nil, err } - if err := sessionctx.ValidateStaleReadTS(ctx, b.ctx, startTS); err != nil { + if err := sessionctx.ValidateStaleReadTS(ctx, b.ctx.GetSessionVars().StmtCtx, b.ctx.GetStore(), startTS); err != nil { return nil, err } p.StaleTxnStartTS = startTS @@ -3423,7 +3423,7 @@ func (b *PlanBuilder) buildSimple(ctx context.Context, node ast.StmtNode) (Plan, return p, nil } -func collectVisitInfoFromRevokeStmt(sctx sessionctx.Context, vi []visitInfo, stmt *ast.RevokeStmt) ([]visitInfo, error) { +func collectVisitInfoFromRevokeStmt(sctx PlanContext, vi []visitInfo, stmt *ast.RevokeStmt) ([]visitInfo, error) { // To use REVOKE, you must have the GRANT OPTION privilege, // and you must have the privileges that you are granting. dbName := stmt.Level.DBName @@ -3475,7 +3475,7 @@ func collectVisitInfoFromRevokeStmt(sctx sessionctx.Context, vi []visitInfo, stm // appendVisitInfoIsRestrictedUser appends additional visitInfo if the user has a // special privilege called "RESTRICTED_USER_ADMIN". It only applies when SEM is enabled. -func appendVisitInfoIsRestrictedUser(visitInfo []visitInfo, sctx sessionctx.Context, user *auth.UserIdentity, priv string) []visitInfo { +func appendVisitInfoIsRestrictedUser(visitInfo []visitInfo, sctx PlanContext, user *auth.UserIdentity, priv string) []visitInfo { if !sem.IsEnabled() { return visitInfo } @@ -3487,7 +3487,7 @@ func appendVisitInfoIsRestrictedUser(visitInfo []visitInfo, sctx sessionctx.Cont return visitInfo } -func collectVisitInfoFromGrantStmt(sctx sessionctx.Context, vi []visitInfo, stmt *ast.GrantStmt) ([]visitInfo, error) { +func collectVisitInfoFromGrantStmt(sctx PlanContext, vi []visitInfo, stmt *ast.GrantStmt) ([]visitInfo, error) { // To use GRANT, you must have the GRANT OPTION privilege, // and you must have the privileges that you are granting. dbName := stmt.Level.DBName @@ -3546,7 +3546,7 @@ func collectVisitInfoFromGrantStmt(sctx sessionctx.Context, vi []visitInfo, stmt return vi, nil } -func genAuthErrForGrantStmt(sctx sessionctx.Context, dbName string) error { +func genAuthErrForGrantStmt(sctx PlanContext, dbName string) error { if !strings.EqualFold(dbName, variable.PerformanceSchema) { return nil } @@ -5010,7 +5010,13 @@ func (b *PlanBuilder) buildExplain(ctx context.Context, explain *ast.ExplainStmt if show, ok := explain.Stmt.(*ast.ShowStmt); ok { return b.buildShow(ctx, show) } - targetPlan, _, err := OptimizeAstNode(ctx, b.ctx, explain.Stmt, b.is) + + sctx, err := AsSctx(b.ctx) + if err != nil { + return nil, err + } + + targetPlan, _, err := OptimizeAstNode(ctx, sctx, explain.Stmt, b.is) if err != nil { return nil, err } @@ -5024,7 +5030,11 @@ func (b *PlanBuilder) buildSelectInto(ctx context.Context, sel *ast.SelectStmt) } selectIntoInfo := sel.SelectIntoOpt sel.SelectIntoOpt = nil - targetPlan, _, err := OptimizeAstNode(ctx, b.ctx, sel, b.is) + sctx, err := AsSctx(b.ctx) + if err != nil { + return nil, err + } + targetPlan, _, err := OptimizeAstNode(ctx, sctx, sel, b.is) if err != nil { return nil, err } @@ -5328,7 +5338,7 @@ func (b *PlanBuilder) buildPlanReplayer(pc *ast.PlanReplayerStmt) Plan { return p } -func calcTSForPlanReplayer(sctx sessionctx.Context, tsExpr ast.ExprNode) uint64 { +func calcTSForPlanReplayer(sctx PlanContext, tsExpr ast.ExprNode) uint64 { tsVal, err := evalAstExprWithPlanCtx(sctx, tsExpr) if err != nil { sctx.GetSessionVars().StmtCtx.AppendWarning(err) diff --git a/pkg/planner/core/point_get_plan.go b/pkg/planner/core/point_get_plan.go index 418963e1034ac..1f835b3623f18 100644 --- a/pkg/planner/core/point_get_plan.go +++ b/pkg/planner/core/point_get_plan.go @@ -39,7 +39,6 @@ import ( "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" - "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/types" @@ -555,7 +554,7 @@ type PointPlanVal struct { } // TryFastPlan tries to use the PointGetPlan for the query. -func TryFastPlan(ctx sessionctx.Context, node ast.Node) (p Plan) { +func TryFastPlan(ctx PlanContext, node ast.Node) (p Plan) { if checkStableResultMode(ctx) { // the rule of stabilizing results has not taken effect yet, so cannot generate a plan here in this mode return nil @@ -651,7 +650,7 @@ func getLockWaitTime(ctx PlanContext, lockInfo *ast.SelectLockInfo) (lock bool, } func newBatchPointGetPlan( - ctx sessionctx.Context, patternInExpr *ast.PatternInExpr, + ctx PlanContext, patternInExpr *ast.PatternInExpr, handleCol *model.ColumnInfo, tbl *model.TableInfo, schema *expression.Schema, names []*types.FieldName, whereColNames []string, indexHints []*ast.IndexHint, ) *BatchPointGetPlan { @@ -937,7 +936,7 @@ func newBatchPointGetPlan( return p.Init(ctx, statsInfo, schema, names, 0) } -func tryWhereIn2BatchPointGet(ctx sessionctx.Context, selStmt *ast.SelectStmt) *BatchPointGetPlan { +func tryWhereIn2BatchPointGet(ctx PlanContext, selStmt *ast.SelectStmt) *BatchPointGetPlan { if selStmt.OrderBy != nil || selStmt.GroupBy != nil || selStmt.Limit != nil || selStmt.Having != nil || selStmt.Distinct || len(selStmt.WindowSpecs) > 0 { @@ -1037,7 +1036,7 @@ func tryWhereIn2BatchPointGet(ctx sessionctx.Context, selStmt *ast.SelectStmt) * // 2. It must be a single table select. // 3. All the columns must be public and not generated. // 4. The condition is an access path that the range is a unique key. -func tryPointGetPlan(ctx sessionctx.Context, selStmt *ast.SelectStmt, check bool) *PointGetPlan { +func tryPointGetPlan(ctx PlanContext, selStmt *ast.SelectStmt, check bool) *PointGetPlan { if selStmt.Having != nil || selStmt.OrderBy != nil { return nil } else if selStmt.Limit != nil { @@ -1125,7 +1124,7 @@ func tryPointGetPlan(ctx sessionctx.Context, selStmt *ast.SelectStmt, check bool return checkTblIndexForPointPlan(ctx, tblName, schema, names, pairs, partitionDef, pairIdx, false, isTableDual, check) } -func checkTblIndexForPointPlan(ctx sessionctx.Context, tblName *ast.TableName, schema *expression.Schema, +func checkTblIndexForPointPlan(ctx PlanContext, tblName *ast.TableName, schema *expression.Schema, names []*types.FieldName, pairs []nameValuePair, partitionDef *model.PartitionDefinition, pos int, globalIndexCheck, isTableDual, check bool) *PointGetPlan { if globalIndexCheck { @@ -1256,7 +1255,7 @@ func partitionNameInSet(name model.CIStr, pnames []model.CIStr) bool { return false } -func newPointGetPlan(ctx sessionctx.Context, dbName string, schema *expression.Schema, tbl *model.TableInfo, names []*types.FieldName) *PointGetPlan { +func newPointGetPlan(ctx PlanContext, dbName string, schema *expression.Schema, tbl *model.TableInfo, names []*types.FieldName) *PointGetPlan { p := &PointGetPlan{ Plan: base.NewBasePlan(ctx, plancodec.TypePointGet, 0), dbName: dbName, @@ -1269,7 +1268,7 @@ func newPointGetPlan(ctx sessionctx.Context, dbName string, schema *expression.S return p } -func checkFastPlanPrivilege(ctx sessionctx.Context, dbName, tableName string, checkTypes ...mysql.PrivilegeType) error { +func checkFastPlanPrivilege(ctx PlanContext, dbName, tableName string, checkTypes ...mysql.PrivilegeType) error { pm := privilege.GetPrivilegeManager(ctx) visitInfos := make([]visitInfo, 0, len(checkTypes)) for _, checkType := range checkTypes { @@ -1420,7 +1419,7 @@ func getSingleTableNameAndAlias(tableRefs *ast.TableRefsClause) (tblName *ast.Ta } // getNameValuePairs extracts `column = constant/paramMarker` conditions from expr as name value pairs. -func getNameValuePairs(ctx sessionctx.Context, tbl *model.TableInfo, tblName model.CIStr, nvPairs []nameValuePair, expr ast.ExprNode) ( +func getNameValuePairs(ctx PlanContext, tbl *model.TableInfo, tblName model.CIStr, nvPairs []nameValuePair, expr ast.ExprNode) ( pairs []nameValuePair, isTableDual bool) { stmtCtx := ctx.GetSessionVars().StmtCtx binOp, ok := expr.(*ast.BinaryOperationExpr) @@ -1662,7 +1661,7 @@ func checkIfAssignmentListHasSubQuery(list []*ast.Assignment) bool { return false } -func tryUpdatePointPlan(ctx sessionctx.Context, updateStmt *ast.UpdateStmt) Plan { +func tryUpdatePointPlan(ctx PlanContext, updateStmt *ast.UpdateStmt) Plan { // Avoid using the point_get when assignment_list contains the sub-query in the UPDATE. if checkIfAssignmentListHasSubQuery(updateStmt.List) { return nil @@ -1697,7 +1696,7 @@ func tryUpdatePointPlan(ctx sessionctx.Context, updateStmt *ast.UpdateStmt) Plan return nil } -func buildPointUpdatePlan(ctx sessionctx.Context, pointPlan PhysicalPlan, dbName string, tbl *model.TableInfo, updateStmt *ast.UpdateStmt) Plan { +func buildPointUpdatePlan(ctx PlanContext, 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 } @@ -1721,7 +1720,7 @@ func buildPointUpdatePlan(ctx sessionctx.Context, pointPlan PhysicalPlan, dbName VirtualAssignmentsOffset: len(orderedList), }.Init(ctx) updatePlan.names = pointPlan.OutputNames() - is := sessiontxn.GetTxnManager(ctx).GetTxnInfoSchema() + is := ctx.GetInfoSchema().(infoschema.InfoSchema) t, _ := is.TableByID(tbl.ID) updatePlan.tblID2Table = map[int64]table.Table{ tbl.ID: t, @@ -1753,7 +1752,7 @@ func buildPointUpdatePlan(ctx sessionctx.Context, pointPlan PhysicalPlan, dbName return updatePlan } -func buildOrderedList(ctx sessionctx.Context, plan Plan, list []*ast.Assignment, +func buildOrderedList(ctx PlanContext, plan Plan, list []*ast.Assignment, ) (orderedList []*expression.Assignment, allAssignmentsAreConstant bool) { orderedList = make([]*expression.Assignment, 0, len(list)) allAssignmentsAreConstant = true @@ -1790,7 +1789,7 @@ func buildOrderedList(ctx sessionctx.Context, plan Plan, list []*ast.Assignment, return orderedList, allAssignmentsAreConstant } -func tryDeletePointPlan(ctx sessionctx.Context, delStmt *ast.DeleteStmt) Plan { +func tryDeletePointPlan(ctx PlanContext, delStmt *ast.DeleteStmt) Plan { if delStmt.IsMultiTable { return nil } @@ -1821,7 +1820,7 @@ func tryDeletePointPlan(ctx sessionctx.Context, delStmt *ast.DeleteStmt) Plan { return nil } -func buildPointDeletePlan(ctx sessionctx.Context, pointPlan PhysicalPlan, dbName string, tbl *model.TableInfo) Plan { +func buildPointDeletePlan(ctx PlanContext, pointPlan PhysicalPlan, dbName string, tbl *model.TableInfo) Plan { if checkFastPlanPrivilege(ctx, dbName, tbl.Name.L, mysql.SelectPriv, mysql.DeletePriv) != nil { return nil } @@ -1838,7 +1837,7 @@ func buildPointDeletePlan(ctx sessionctx.Context, pointPlan PhysicalPlan, dbName }, }.Init(ctx) var err error - is := sessiontxn.GetTxnManager(ctx).GetTxnInfoSchema() + is := ctx.GetInfoSchema().(infoschema.InfoSchema) t, _ := is.TableByID(tbl.ID) if t != nil { tblID2Table := map[int64]table.Table{tbl.ID: t} @@ -1874,7 +1873,7 @@ func colInfoToColumn(col *model.ColumnInfo, idx int) *expression.Column { } } -func buildHandleCols(ctx sessionctx.Context, tbl *model.TableInfo, schema *expression.Schema) HandleCols { +func buildHandleCols(ctx PlanContext, tbl *model.TableInfo, schema *expression.Schema) HandleCols { // fields len is 0 for update and delete. if tbl.PKIsHandle { for i, col := range tbl.Columns { @@ -1894,7 +1893,7 @@ func buildHandleCols(ctx sessionctx.Context, tbl *model.TableInfo, schema *expre return &IntHandleCols{col: handleCol} } -func getPartitionDef(ctx sessionctx.Context, tbl *model.TableInfo, pairs []nameValuePair) (*model.PartitionDefinition, int, int, bool) { +func getPartitionDef(ctx PlanContext, tbl *model.TableInfo, pairs []nameValuePair) (*model.PartitionDefinition, int, int, bool) { partitionExpr := getPartitionExpr(ctx, tbl) if partitionExpr == nil { return nil, 0, 0, false diff --git a/pkg/planner/core/preprocess.go b/pkg/planner/core/preprocess.go index a04e689c3b213..8039a5f8b2566 100644 --- a/pkg/planner/core/preprocess.go +++ b/pkg/planner/core/preprocess.go @@ -1788,7 +1788,7 @@ func (p *preprocessor) hasAutoConvertWarning(colDef *ast.ColumnDef) bool { return false } -func tryLockMDLAndUpdateSchemaIfNecessary(sctx sessionctx.Context, dbName model.CIStr, tbl table.Table, is infoschema.InfoSchema) (table.Table, error) { +func tryLockMDLAndUpdateSchemaIfNecessary(sctx PlanContext, dbName model.CIStr, tbl table.Table, is infoschema.InfoSchema) (table.Table, error) { if !sctx.GetSessionVars().TxnCtx.EnableMDL { return tbl, nil } @@ -1808,7 +1808,7 @@ func tryLockMDLAndUpdateSchemaIfNecessary(sctx sessionctx.Context, dbName model. } else if tbl.Meta().TempTableType == model.TempTableGlobal { skipLock = true } - if IsAutoCommitTxn(sctx) && sctx.GetSessionVars().StmtCtx.IsReadOnly { + if IsAutoCommitTxn(sctx.GetSessionVars()) && sctx.GetSessionVars().StmtCtx.IsReadOnly { return tbl, nil } tableInfo := tbl.Meta() diff --git a/pkg/planner/core/util.go b/pkg/planner/core/util.go index afff2f3a4ee05..d557c5685aba7 100644 --- a/pkg/planner/core/util.go +++ b/pkg/planner/core/util.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core/internal/base" - "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/mathutil" @@ -386,7 +385,7 @@ func extractStringFromBoolSlice(slice []bool) string { return strings.Join(l, ",") } -func tableHasDirtyContent(ctx sessionctx.Context, tableInfo *model.TableInfo) bool { +func tableHasDirtyContent(ctx PlanContext, tableInfo *model.TableInfo) bool { pi := tableInfo.GetPartitionInfo() if pi == nil { return ctx.HasDirtyContent(tableInfo.ID) diff --git a/pkg/planner/optimize.go b/pkg/planner/optimize.go index de92f0bce9336..b82ae327de93d 100644 --- a/pkg/planner/optimize.go +++ b/pkg/planner/optimize.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/metrics" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/planner/cascades" + pctx "github.com/pingcap/tidb/pkg/planner/context" "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/util/debugtrace" "github.com/pingcap/tidb/pkg/privilege" @@ -372,7 +373,7 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in // OptimizeForForeignKeyCascade does optimization and creates a Plan for foreign key cascade. // Compare to Optimize, OptimizeForForeignKeyCascade only build plan by StmtNode, // doesn't consider plan cache and plan binding, also doesn't do privilege check. -func OptimizeForForeignKeyCascade(ctx context.Context, sctx sessionctx.Context, node ast.StmtNode, is infoschema.InfoSchema) (core.Plan, error) { +func OptimizeForForeignKeyCascade(ctx context.Context, sctx pctx.PlanContext, node ast.StmtNode, is infoschema.InfoSchema) (core.Plan, error) { builder := planBuilderPool.Get().(*core.PlanBuilder) defer planBuilderPool.Put(builder.ResetForReuse()) hintProcessor := hint.NewQBHintHandler(sctx.GetSessionVars().StmtCtx) @@ -387,7 +388,7 @@ func OptimizeForForeignKeyCascade(ctx context.Context, sctx sessionctx.Context, return p, nil } -func allowInReadOnlyMode(sctx sessionctx.Context, node ast.Node) (bool, error) { +func allowInReadOnlyMode(sctx pctx.PlanContext, node ast.Node) (bool, error) { pm := privilege.GetPrivilegeManager(sctx) if pm == nil { return true, nil @@ -436,7 +437,7 @@ var planBuilderPool = sync.Pool{ // optimizeCnt is a global variable only used for test. var optimizeCnt int -func optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema) (core.Plan, types.NameSlice, float64, error) { +func optimize(ctx context.Context, sctx pctx.PlanContext, node ast.Node, is infoschema.InfoSchema) (core.Plan, types.NameSlice, float64, error) { failpoint.Inject("checkOptimizeCountOne", func(val failpoint.Value) { // only count the optimization for SQL with specified text if testSQL, ok := val.(string); ok && testSQL == node.OriginalText() { @@ -532,7 +533,7 @@ func OptimizeExecStmt(ctx context.Context, sctx sessionctx.Context, return exec, names, nil } -func buildLogicalPlan(ctx context.Context, sctx sessionctx.Context, node ast.Node, builder *core.PlanBuilder) (core.Plan, error) { +func buildLogicalPlan(ctx context.Context, sctx pctx.PlanContext, node ast.Node, builder *core.PlanBuilder) (core.Plan, error) { sctx.GetSessionVars().PlanID.Store(0) sctx.GetSessionVars().PlanColumnID.Store(0) sctx.GetSessionVars().MapScalarSubQ = nil @@ -558,7 +559,7 @@ func buildLogicalPlan(ctx context.Context, sctx sessionctx.Context, node ast.Nod return p, nil } -func handleInvalidBinding(ctx context.Context, sctx sessionctx.Context, level string, binding bindinfo.Binding) { +func handleInvalidBinding(ctx context.Context, sctx pctx.PlanContext, level string, binding bindinfo.Binding) { sessionHandle := sctx.Value(bindinfo.SessionBindInfoKeyType).(bindinfo.SessionBindingHandle) err := sessionHandle.DropSessionBinding(binding.SQLDigest) if err != nil { diff --git a/pkg/planner/util/BUILD.bazel b/pkg/planner/util/BUILD.bazel index c2597d6777ec7..af04ffaa5c027 100644 --- a/pkg/planner/util/BUILD.bazel +++ b/pkg/planner/util/BUILD.bazel @@ -16,7 +16,6 @@ go_library( "//pkg/parser/ast", "//pkg/parser/model", "//pkg/planner/context", - "//pkg/sessionctx", "//pkg/types", "//pkg/util/collate", "//pkg/util/ranger", diff --git a/pkg/planner/util/expression.go b/pkg/planner/util/expression.go index 49e185c1dd98f..dce117ad396b4 100644 --- a/pkg/planner/util/expression.go +++ b/pkg/planner/util/expression.go @@ -17,7 +17,7 @@ package util import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/planner/context" "github.com/pingcap/tidb/pkg/types" ) @@ -25,11 +25,11 @@ import ( // Different with expression.EvalSimpleAst, it uses planner context and is more powerful to build // some special expressions like subquery, window function, etc. // If you only want to evaluate simple expressions, use `expression.EvalSimpleAst` instead. -var EvalAstExprWithPlanCtx func(ctx sessionctx.Context, expr ast.ExprNode) (types.Datum, error) +var EvalAstExprWithPlanCtx func(ctx context.PlanContext, expr ast.ExprNode) (types.Datum, error) // RewriteAstExprWithPlanCtx rewrites ast expression directly. // Different with expression.BuildSimpleExpr, it uses planner context and is more powerful to build // some special expressions like subquery, window function, etc. // If you only want to build simple expressions, use `expression.BuildSimpleExpr` instead. -var RewriteAstExprWithPlanCtx func(ctx sessionctx.Context, expr ast.ExprNode, +var RewriteAstExprWithPlanCtx func(ctx context.PlanContext, expr ast.ExprNode, schema *expression.Schema, names types.NameSlice, allowCastArray bool) (expression.Expression, error) diff --git a/pkg/sessionctx/BUILD.bazel b/pkg/sessionctx/BUILD.bazel index 628caa9868df2..e50fdeefc5fcb 100644 --- a/pkg/sessionctx/BUILD.bazel +++ b/pkg/sessionctx/BUILD.bazel @@ -10,10 +10,12 @@ go_library( "//pkg/extension", "//pkg/infoschema/context", "//pkg/kv", + "//pkg/lock/context", "//pkg/metrics", "//pkg/parser/model", "//pkg/planner/context", "//pkg/sessionctx/sessionstates", + "//pkg/sessionctx/stmtctx", "//pkg/sessionctx/variable", "//pkg/statistics/handle/usage/indexusage", "//pkg/util", diff --git a/pkg/sessionctx/context.go b/pkg/sessionctx/context.go index 5af0a3ab1335a..afbb3199d17d0 100644 --- a/pkg/sessionctx/context.go +++ b/pkg/sessionctx/context.go @@ -24,10 +24,12 @@ import ( "github.com/pingcap/tidb/pkg/extension" infoschema "github.com/pingcap/tidb/pkg/infoschema/context" "github.com/pingcap/tidb/pkg/kv" + tablelock "github.com/pingcap/tidb/pkg/lock/context" "github.com/pingcap/tidb/pkg/metrics" "github.com/pingcap/tidb/pkg/parser/model" planctx "github.com/pingcap/tidb/pkg/planner/context" "github.com/pingcap/tidb/pkg/sessionctx/sessionstates" + "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics/handle/usage/indexusage" "github.com/pingcap/tidb/pkg/util" @@ -66,6 +68,7 @@ type Context interface { exprctx.EvalContext exprctx.BuildContext planctx.PlanContext + tablelock.TableLockContext // SetDiskFullOpt set the disk full opt when tikv disk full happened. SetDiskFullOpt(level kvrpcpb.DiskFullOpt) // RollbackTxn rolls back the current transaction. @@ -124,20 +127,6 @@ type Context interface { StmtGetMutation(int64) *binlog.TableMutation // IsDDLOwner checks whether this session is DDL owner. IsDDLOwner() bool - // AddTableLock adds table lock to the session lock map. - AddTableLock([]model.TableLockTpInfo) - // ReleaseTableLocks releases table locks in the session lock map. - ReleaseTableLocks(locks []model.TableLockTpInfo) - // ReleaseTableLockByTableIDs releases table locks in the session lock map by table IDs. - ReleaseTableLockByTableIDs(tableIDs []int64) - // CheckTableLocked checks the table lock. - CheckTableLocked(tblID int64) (bool, model.TableLockType) - // GetAllTableLocks gets all table locks table id and db id hold by the session. - GetAllTableLocks() []model.TableLockTpInfo - // ReleaseAllTableLocks releases all table locks hold by the session. - ReleaseAllTableLocks() - // HasLockedTables uses to check whether this session locked any tables. - HasLockedTables() bool // PrepareTSFuture uses to prepare timestamp by future. PrepareTSFuture(ctx context.Context, future oracle.Future, scope string) error // GetPreparedTxnFuture returns the TxnFuture if it is valid or pending. @@ -230,15 +219,15 @@ func ValidateSnapshotReadTS(ctx context.Context, sctx Context, readTS uint64) er const allowedTimeFromNow = 100 * time.Millisecond // ValidateStaleReadTS validates that readTS does not exceed the current time not strictly. -func ValidateStaleReadTS(ctx context.Context, sctx Context, readTS uint64) error { - currentTS, err := sctx.GetSessionVars().StmtCtx.GetStaleTSO() +func ValidateStaleReadTS(ctx context.Context, sc *stmtctx.StatementContext, store kv.Storage, readTS uint64) error { + currentTS, err := sc.GetStaleTSO() if currentTS == 0 || err != nil { - currentTS, err = sctx.GetStore().GetOracle().GetStaleTimestamp(ctx, oracle.GlobalTxnScope, 0) + currentTS, err = store.GetOracle().GetStaleTimestamp(ctx, oracle.GlobalTxnScope, 0) } // If we fail to calculate currentTS from local time, fallback to get a timestamp from PD if err != nil { metrics.ValidateReadTSFromPDCount.Inc() - currentVer, err := sctx.GetStore().CurrentVersion(oracle.GlobalTxnScope) + currentVer, err := store.CurrentVersion(oracle.GlobalTxnScope) if err != nil { return errors.Errorf("fail to validate read timestamp: %v", err) } diff --git a/pkg/sessiontxn/isolation/optimistic.go b/pkg/sessiontxn/isolation/optimistic.go index d560fee244f89..c68ef59a6f5fb 100644 --- a/pkg/sessiontxn/isolation/optimistic.go +++ b/pkg/sessiontxn/isolation/optimistic.go @@ -129,7 +129,7 @@ func (p *OptimisticTxnContextProvider) AdviseOptimizeWithPlan(plan any) (err err realPlan = execute.Plan } - ok, err = plannercore.IsPointGetWithPKOrUniqueKeyByAutoCommit(p.sctx, realPlan) + ok, err = plannercore.IsPointGetWithPKOrUniqueKeyByAutoCommit(p.sctx.GetSessionVars(), realPlan) if err != nil { return err } diff --git a/pkg/sessiontxn/staleread/BUILD.bazel b/pkg/sessiontxn/staleread/BUILD.bazel index 7ac7d55ce64e5..45c11e9e2e410 100644 --- a/pkg/sessiontxn/staleread/BUILD.bazel +++ b/pkg/sessiontxn/staleread/BUILD.bazel @@ -19,6 +19,7 @@ go_library( "//pkg/kv", "//pkg/parser/ast", "//pkg/parser/mysql", + "//pkg/planner/context", "//pkg/planner/util", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", diff --git a/pkg/sessiontxn/staleread/processor.go b/pkg/sessiontxn/staleread/processor.go index 1d505297793e5..24aa1dc0b28c5 100644 --- a/pkg/sessiontxn/staleread/processor.go +++ b/pkg/sessiontxn/staleread/processor.go @@ -286,7 +286,7 @@ func parseAndValidateAsOf(ctx context.Context, sctx sessionctx.Context, asOf *as return 0, err } - if err = sessionctx.ValidateStaleReadTS(ctx, sctx, ts); err != nil { + if err = sessionctx.ValidateStaleReadTS(ctx, sctx.GetSessionVars().StmtCtx, sctx.GetStore(), ts); err != nil { return 0, err } @@ -306,7 +306,7 @@ func getTsEvaluatorFromReadStaleness(sctx sessionctx.Context) StalenessTSEvaluat func getTSFromExternalTS(ctx context.Context, sctx sessionctx.Context) (uint64, error) { if sctx.GetSessionVars().EnableExternalTSRead && !sctx.GetSessionVars().InRestrictedSQL { - externalTimestamp, err := GetExternalTimestamp(ctx, sctx) + externalTimestamp, err := GetExternalTimestamp(ctx, sctx.GetSessionVars().StmtCtx) if err != nil { return 0, err } diff --git a/pkg/sessiontxn/staleread/util.go b/pkg/sessiontxn/staleread/util.go index 6aebc436b2989..c5d30d4ce38b4 100644 --- a/pkg/sessiontxn/staleread/util.go +++ b/pkg/sessiontxn/staleread/util.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" + pctx "github.com/pingcap/tidb/pkg/planner/context" plannerutil "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" @@ -32,7 +33,7 @@ import ( ) // CalculateAsOfTsExpr calculates the TsExpr of AsOfClause to get a StartTS. -func CalculateAsOfTsExpr(ctx context.Context, sctx sessionctx.Context, tsExpr ast.ExprNode) (uint64, error) { +func CalculateAsOfTsExpr(ctx context.Context, sctx pctx.PlanContext, tsExpr ast.ExprNode) (uint64, error) { sctx.GetSessionVars().StmtCtx.SetStaleTSOProvider(func() (uint64, error) { failpoint.Inject("mockStaleReadTSO", func(val failpoint.Value) (uint64, error) { return uint64(val.(int)), nil @@ -82,10 +83,9 @@ func IsStmtStaleness(sctx sessionctx.Context) bool { } // GetExternalTimestamp returns the external timestamp in cache, or get and store it in cache -func GetExternalTimestamp(ctx context.Context, sctx sessionctx.Context) (uint64, error) { +func GetExternalTimestamp(ctx context.Context, sc *stmtctx.StatementContext) (uint64, error) { // Try to get from the stmt cache to make sure this function is deterministic. - stmtCtx := sctx.GetSessionVars().StmtCtx - externalTimestamp, err := stmtCtx.GetOrEvaluateStmtCache(stmtctx.StmtExternalTSCacheKey, func() (any, error) { + externalTimestamp, err := sc.GetOrEvaluateStmtCache(stmtctx.StmtExternalTSCacheKey, func() (any, error) { return variable.GetExternalTimestamp(ctx) }) diff --git a/pkg/table/column.go b/pkg/table/column.go index cb15de5ad8a70..c34faaefe3c03 100644 --- a/pkg/table/column.go +++ b/pkg/table/column.go @@ -546,13 +546,13 @@ func GetColDefaultValue(ctx expression.BuildContext, col *model.ColumnInfo) (typ } // EvalColDefaultExpr eval default expr node to explicit default value. -func EvalColDefaultExpr(ctx sessionctx.Context, col *model.ColumnInfo, defaultExpr ast.ExprNode) (types.Datum, error) { +func EvalColDefaultExpr(ctx expression.BuildContext, col *model.ColumnInfo, defaultExpr ast.ExprNode) (types.Datum, error) { d, err := expression.EvalSimpleAst(ctx, defaultExpr) if err != nil { return types.Datum{}, err } // Check the evaluated data type by cast. - value, err := CastValue(ctx, d, col, false, false) + value, err := CastColumnValue(ctx.GetSessionVars(), d, col, false, false) if err != nil { return types.Datum{}, err } diff --git a/pkg/util/ranger/BUILD.bazel b/pkg/util/ranger/BUILD.bazel index f1be0c5c6d121..d8fa71b24f11a 100644 --- a/pkg/util/ranger/BUILD.bazel +++ b/pkg/util/ranger/BUILD.bazel @@ -23,7 +23,6 @@ go_library( "//pkg/parser/terror", "//pkg/planner/context", "//pkg/planner/util/fixcontrol", - "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", "//pkg/types", "//pkg/types/parser_driver", diff --git a/pkg/util/ranger/detacher.go b/pkg/util/ranger/detacher.go index a1f56792b495a..f77796277569b 100644 --- a/pkg/util/ranger/detacher.go +++ b/pkg/util/ranger/detacher.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" planctx "github.com/pingcap/tidb/pkg/planner/context" "github.com/pingcap/tidb/pkg/planner/util/fixcontrol" - "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -945,7 +944,7 @@ func (d *rangeDetacher) detachCondAndBuildRangeForCols() (*DetachRangeResult, er // It will find the point query column firstly and then extract the range query column. // rangeMaxSize is the max memory limit for ranges. O indicates no memory limit. If you ask that all conditions must be used // for building ranges, set rangeMemQuota to 0 to avoid range fallback. -func DetachSimpleCondAndBuildRangeForIndex(sctx sessionctx.Context, conditions []expression.Expression, +func DetachSimpleCondAndBuildRangeForIndex(sctx planctx.PlanContext, conditions []expression.Expression, cols []*expression.Column, lengths []int, rangeMaxSize int64) (Ranges, []expression.Expression, error) { newTpSlice := make([]*types.FieldType, 0, len(cols)) for _, col := range cols {