Skip to content

Commit

Permalink
planner: replace context in planbuilder to PlanContext (#51172)
Browse files Browse the repository at this point in the history
ref #51070, close #51171
  • Loading branch information
lcwangchao authored Feb 21, 2024
1 parent a0e0969 commit 1ee58bf
Show file tree
Hide file tree
Showing 44 changed files with 227 additions and 159 deletions.
12 changes: 6 additions & 6 deletions pkg/executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1470,18 +1470,18 @@ 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)
executor := e.build(p)
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
Expand Down Expand Up @@ -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.
Expand Down
2 changes: 1 addition & 1 deletion pkg/executor/set.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
4 changes: 2 additions & 2 deletions pkg/executor/test/executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
Expand Down Expand Up @@ -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)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/lock/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
],
Expand Down
9 changes: 9 additions & 0 deletions pkg/lock/context/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -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"],
)
40 changes: 40 additions & 0 deletions pkg/lock/context/lockcontext.go
Original file line number Diff line number Diff line change
@@ -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()
}
6 changes: 3 additions & 3 deletions pkg/lock/lock.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,24 +18,24 @@ 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
}

// ErrLockedTableDropped returns error when try to drop the table with write lock
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}
}

Expand Down
1 change: 1 addition & 0 deletions pkg/planner/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
1 change: 0 additions & 1 deletion pkg/planner/cardinality/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
9 changes: 4 additions & 5 deletions pkg/planner/cardinality/row_count_column.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
Expand All @@ -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
Expand All @@ -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) {
Expand All @@ -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
Expand Down
1 change: 0 additions & 1 deletion pkg/planner/cascades/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
9 changes: 4 additions & 5 deletions pkg/planner/cascades/optimize.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
)

Expand Down Expand Up @@ -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
Expand All @@ -116,15 +115,15 @@ 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
}
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)
Expand Down Expand Up @@ -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,
}
Expand Down
3 changes: 3 additions & 0 deletions pkg/planner/context/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
],
)
15 changes: 15 additions & 0 deletions pkg/planner/context/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,22 +16,37 @@ 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"
)

// PlanContext is the context for building plan.
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
}
1 change: 1 addition & 0 deletions pkg/planner/core/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -95,6 +95,7 @@ go_library(
"//pkg/infoschema",
"//pkg/kv",
"//pkg/lock",
"//pkg/lock/context",
"//pkg/meta/autoid",
"//pkg/metrics",
"//pkg/parser",
Expand Down
16 changes: 8 additions & 8 deletions pkg/planner/core/common_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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{})
Expand Down Expand Up @@ -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
}

Expand All @@ -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
}
Expand Down Expand Up @@ -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.
Expand Down
Loading

0 comments on commit 1ee58bf

Please sign in to comment.