diff --git a/pkg/executor/executor.go b/pkg/executor/executor.go index 104c06e698..d6e6252d51 100644 --- a/pkg/executor/executor.go +++ b/pkg/executor/executor.go @@ -55,6 +55,7 @@ import ( planctx "github.com/pingcap/tidb/pkg/planner/context" plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" plannerutil "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/fixcontrol" "github.com/pingcap/tidb/pkg/privilege" @@ -1151,7 +1152,7 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { return err } // If there's no handle or it's not a `SELECT FOR UPDATE` statement. - if len(e.tblID2Handle) == 0 || (!plannercore.IsSelectForUpdateLockType(e.Lock.LockType)) { + if len(e.tblID2Handle) == 0 || (!logicalop.IsSelectForUpdateLockType(e.Lock.LockType)) { return nil } diff --git a/pkg/planner/core/BUILD.bazel b/pkg/planner/core/BUILD.bazel index 87838e35e6..cd93322909 100644 --- a/pkg/planner/core/BUILD.bazel +++ b/pkg/planner/core/BUILD.bazel @@ -30,7 +30,6 @@ go_library( "logical_index_scan.go", "logical_initialize.go", "logical_join.go", - "logical_lock.go", "logical_partition_union_all.go", "logical_plan_builder.go", "logical_plans.go", diff --git a/pkg/planner/core/core_init.go b/pkg/planner/core/core_init.go index 128792b9b4..bd1d5d67bf 100644 --- a/pkg/planner/core/core_init.go +++ b/pkg/planner/core/core_init.go @@ -42,6 +42,7 @@ func init() { utilfuncp.FindBestTask4LogicalShowDDLJobs = findBestTask4LogicalShowDDLJobs utilfuncp.ExhaustPhysicalPlans4LogicalSort = exhaustPhysicalPlans4LogicalSort utilfuncp.ExhaustPhysicalPlans4LogicalTopN = exhaustPhysicalPlans4LogicalTopN + utilfuncp.ExhaustPhysicalPlans4LogicalLock = exhaustPhysicalPlans4LogicalLock utilfuncp.ExhaustPhysicalPlans4LogicalLimit = exhaustPhysicalPlans4LogicalLimit utilfuncp.ExhaustLogicalWindowPhysicalPlans = exhaustLogicalWindowPhysicalPlans utilfuncp.ExhaustPhysicalPlans4LogicalSequence = exhaustPhysicalPlans4LogicalSequence diff --git a/pkg/planner/core/exhaust_physical_plans.go b/pkg/planner/core/exhaust_physical_plans.go index bfff75375f..b974d7bef2 100644 --- a/pkg/planner/core/exhaust_physical_plans.go +++ b/pkg/planner/core/exhaust_physical_plans.go @@ -2821,7 +2821,8 @@ func getLimitPhysicalPlans(p *logicalop.LogicalLimit, prop *property.PhysicalPro return ret, true, nil } -func getLockPhysicalPlans(p *LogicalLock, prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { +func exhaustPhysicalPlans4LogicalLock(lp base.LogicalPlan, prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { + p := lp.(*logicalop.LogicalLock) if prop.IsFlashProp() { p.SCtx().GetSessionVars().RaiseWarningWhenMPPEnforced( "MPP mode may be blocked because operator `Lock` is not supported now.") diff --git a/pkg/planner/core/logical_plans.go b/pkg/planner/core/logical_plans.go index 861732b353..054810bb36 100644 --- a/pkg/planner/core/logical_plans.go +++ b/pkg/planner/core/logical_plans.go @@ -37,7 +37,7 @@ var ( _ base.LogicalPlan = &LogicalIndexScan{} _ base.LogicalPlan = &LogicalUnionAll{} _ base.LogicalPlan = &logicalop.LogicalSort{} - _ base.LogicalPlan = &LogicalLock{} + _ base.LogicalPlan = &logicalop.LogicalLock{} _ base.LogicalPlan = &logicalop.LogicalLimit{} _ base.LogicalPlan = &logicalop.LogicalWindow{} _ base.LogicalPlan = &LogicalExpand{} diff --git a/pkg/planner/core/operator/logicalop/BUILD.bazel b/pkg/planner/core/operator/logicalop/BUILD.bazel index 6690041405..ad45e9d81f 100644 --- a/pkg/planner/core/operator/logicalop/BUILD.bazel +++ b/pkg/planner/core/operator/logicalop/BUILD.bazel @@ -6,6 +6,7 @@ go_library( "base_logical_plan.go", "logical_cte_table.go", "logical_limit.go", + "logical_lock.go", "logical_max_one_row.go", "logical_mem_table.go", "logical_projection.go", diff --git a/pkg/planner/core/logical_lock.go b/pkg/planner/core/operator/logicalop/logical_lock.go similarity index 88% rename from pkg/planner/core/logical_lock.go rename to pkg/planner/core/operator/logicalop/logical_lock.go index 6c360ae574..f4fe2f3a1d 100644 --- a/pkg/planner/core/logical_lock.go +++ b/pkg/planner/core/operator/logicalop/logical_lock.go @@ -12,22 +12,22 @@ // See the License for the specific language governing permissions and // limitations under the License. -package core +package logicalop import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/planner/core/base" - "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/optimizetrace" + "github.com/pingcap/tidb/pkg/planner/util/utilfuncp" "github.com/pingcap/tidb/pkg/util/plancodec" ) // LogicalLock represents a select lock plan. type LogicalLock struct { - logicalop.BaseLogicalPlan + BaseLogicalPlan Lock *ast.SelectLockInfo TblID2Handle map[int64][]util.HandleCols @@ -40,7 +40,7 @@ type LogicalLock struct { // Init initializes LogicalLock. func (p LogicalLock) Init(ctx base.PlanContext) *LogicalLock { - p.BaseLogicalPlan = logicalop.NewBaseLogicalPlan(ctx, plancodec.TypeLock, &p, 0) + p.BaseLogicalPlan = NewBaseLogicalPlan(ctx, plancodec.TypeLock, &p, 0) return &p } @@ -89,9 +89,9 @@ func (p *LogicalLock) PruneColumns(parentUsedCols []*expression.Column, opt *opt // PushDownTopN implements the base.LogicalPlan.<5th> interface. func (p *LogicalLock) PushDownTopN(topNLogicalPlan base.LogicalPlan, opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan { - var topN *logicalop.LogicalTopN + var topN *LogicalTopN if topNLogicalPlan != nil { - topN = topNLogicalPlan.(*logicalop.LogicalTopN) + topN = topNLogicalPlan.(*LogicalTopN) } if topN != nil { p.Children()[0] = p.Children()[0].PushDownTopN(topN, opt) @@ -117,7 +117,7 @@ func (p *LogicalLock) PushDownTopN(topNLogicalPlan base.LogicalPlan, opt *optimi // ExhaustPhysicalPlans implements base.LogicalPlan.<14th> interface. func (p *LogicalLock) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { - return getLockPhysicalPlans(p, prop) + return utilfuncp.ExhaustPhysicalPlans4LogicalLock(p, prop) } // ExtractCorrelatedCols inherits BaseLogicalPlan.LogicalPlan.<15th> implementation. @@ -141,3 +141,14 @@ func (p *LogicalLock) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]b // ConvertOuterToInnerJoin inherits BaseLogicalPlan.LogicalPlan.<24th> implementation. // *************************** end implementation of logicalPlan interface *************************** + +// IsSelectForUpdateLockType checks if the select lock type is for update type. +func IsSelectForUpdateLockType(lockType ast.SelectLockType) bool { + if lockType == ast.SelectLockForUpdate || + lockType == ast.SelectLockForShare || + lockType == ast.SelectLockForUpdateNoWait || + lockType == ast.SelectLockForUpdateWaitN { + return true + } + return false +} diff --git a/pkg/planner/core/plan.go b/pkg/planner/core/plan.go index a923268e03..ca99a6ecc4 100644 --- a/pkg/planner/core/plan.go +++ b/pkg/planner/core/plan.go @@ -383,7 +383,7 @@ func HasMaxOneRow(p base.LogicalPlan, childMaxOneRow []bool) bool { return false } switch x := p.(type) { - case *LogicalLock, *logicalop.LogicalLimit, *logicalop.LogicalSort, *LogicalSelection, + case *logicalop.LogicalLock, *logicalop.LogicalLimit, *logicalop.LogicalSort, *LogicalSelection, *LogicalApply, *logicalop.LogicalProjection, *logicalop.LogicalWindow, *LogicalAggregation: return childMaxOneRow[0] case *logicalop.LogicalMaxOneRow: diff --git a/pkg/planner/core/planbuilder.go b/pkg/planner/core/planbuilder.go index b74e2ad377..eb88cbbaf1 100644 --- a/pkg/planner/core/planbuilder.go +++ b/pkg/planner/core/planbuilder.go @@ -1311,7 +1311,7 @@ func removeGlobalIndexPaths(paths []*util.AccessPath) []*util.AccessPath { return paths[:i] } -func (b *PlanBuilder) buildSelectLock(src base.LogicalPlan, lock *ast.SelectLockInfo) (*LogicalLock, error) { +func (b *PlanBuilder) buildSelectLock(src base.LogicalPlan, lock *ast.SelectLockInfo) (*logicalop.LogicalLock, error) { var tblID2PhysTblIDCol map[int64]*expression.Column if len(b.partitionedTable) > 0 { tblID2PhysTblIDCol = make(map[int64]*expression.Column) @@ -1325,7 +1325,7 @@ func (b *PlanBuilder) buildSelectLock(src base.LogicalPlan, lock *ast.SelectLock // since it would otherwise be lost in the PartitionUnion executor. setExtraPhysTblIDColsOnDataSource(src, tblID2PhysTblIDCol) } - selectLock := LogicalLock{ + selectLock := logicalop.LogicalLock{ Lock: lock, TblID2Handle: b.handleHelper.tailMap(), TblID2PhysTblIDCol: tblID2PhysTblIDCol, diff --git a/pkg/planner/core/point_get_plan.go b/pkg/planner/core/point_get_plan.go index d42ae26433..5892910398 100644 --- a/pkg/planner/core/point_get_plan.go +++ b/pkg/planner/core/point_get_plan.go @@ -34,6 +34,7 @@ import ( ptypes "github.com/pingcap/tidb/pkg/parser/types" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/operator/baseimpl" + "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/costusage" @@ -943,20 +944,9 @@ func TryFastPlan(ctx base.PlanContext, node ast.Node) (p base.Plan) { return nil } -// IsSelectForUpdateLockType checks if the select lock type is for update type. -func IsSelectForUpdateLockType(lockType ast.SelectLockType) bool { - if lockType == ast.SelectLockForUpdate || - lockType == ast.SelectLockForShare || - lockType == ast.SelectLockForUpdateNoWait || - lockType == ast.SelectLockForUpdateWaitN { - return true - } - return false -} - func getLockWaitTime(ctx base.PlanContext, lockInfo *ast.SelectLockInfo) (lock bool, waitTime int64) { if lockInfo != nil { - if IsSelectForUpdateLockType(lockInfo.LockType) { + if logicalop.IsSelectForUpdateLockType(lockInfo.LockType) { // Locking of rows for update using SELECT FOR UPDATE only applies when autocommit // is disabled (either by beginning transaction with START TRANSACTION or by setting // autocommit to 0. If autocommit is enabled, the rows matching the specification are not locked. diff --git a/pkg/planner/core/stringer.go b/pkg/planner/core/stringer.go index 4b98065c42..517714f0db 100644 --- a/pkg/planner/core/stringer.go +++ b/pkg/planner/core/stringer.go @@ -171,7 +171,7 @@ func toString(in base.Plan, strs []string, idxs []int) ([]string, []int) { str = "MaxOneRow" case *logicalop.LogicalLimit, *PhysicalLimit: str = "Limit" - case *PhysicalLock, *LogicalLock: + case *PhysicalLock, *logicalop.LogicalLock: str = "Lock" case *ShowDDL: str = "ShowDDL" diff --git a/pkg/planner/pattern/pattern.go b/pkg/planner/pattern/pattern.go index 2b78e95fb2..d3c3a7d2fd 100644 --- a/pkg/planner/pattern/pattern.go +++ b/pkg/planner/pattern/pattern.go @@ -101,7 +101,7 @@ func GetOperand(p base.LogicalPlan) Operand { return OperandSort case *logicalop.LogicalTopN: return OperandTopN - case *plannercore.LogicalLock: + case *logicalop.LogicalLock: return OperandLock case *logicalop.LogicalLimit: return OperandLimit diff --git a/pkg/planner/pattern/pattern_test.go b/pkg/planner/pattern/pattern_test.go index 779d652c5f..8651838501 100644 --- a/pkg/planner/pattern/pattern_test.go +++ b/pkg/planner/pattern/pattern_test.go @@ -35,7 +35,7 @@ func TestGetOperand(t *testing.T) { require.Equal(t, OperandUnionAll, GetOperand(&plannercore.LogicalUnionAll{})) require.Equal(t, OperandSort, GetOperand(&logicalop.LogicalSort{})) require.Equal(t, OperandTopN, GetOperand(&logicalop.LogicalTopN{})) - require.Equal(t, OperandLock, GetOperand(&plannercore.LogicalLock{})) + require.Equal(t, OperandLock, GetOperand(&logicalop.LogicalLock{})) require.Equal(t, OperandLimit, GetOperand(&logicalop.LogicalLimit{})) } diff --git a/pkg/planner/util/utilfuncp/func_pointer_misc.go b/pkg/planner/util/utilfuncp/func_pointer_misc.go index d2b2ef7740..15e6c5f904 100644 --- a/pkg/planner/util/utilfuncp/func_pointer_misc.go +++ b/pkg/planner/util/utilfuncp/func_pointer_misc.go @@ -137,3 +137,7 @@ var ExhaustPhysicalPlans4LogicalProjection func(lp base.LogicalPlan, // ExhaustLogicalWindowPhysicalPlans will be called by LogicalWindow in logicalOp pkg. var ExhaustLogicalWindowPhysicalPlans func(lp base.LogicalPlan, prop *property.PhysicalProperty) ( []base.PhysicalPlan, bool, error) + +// ExhaustPhysicalPlans4LogicalLock will be called by LogicalLock in logicalOp pkg. +var ExhaustPhysicalPlans4LogicalLock func(lp base.LogicalPlan, prop *property.PhysicalProperty) ( + []base.PhysicalPlan, bool, error)