diff --git a/pkg/planner/core/BUILD.bazel b/pkg/planner/core/BUILD.bazel index fa1f33b2a579b..bb6fa5445703e 100644 --- a/pkg/planner/core/BUILD.bazel +++ b/pkg/planner/core/BUILD.bazel @@ -38,6 +38,7 @@ go_library( "logical_plans.go", "logical_projection.go", "logical_selection.go", + "logical_sequence.go", "logical_show.go", "logical_show_ddl_jobs.go", "logical_sort.go", diff --git a/pkg/planner/core/exhaust_physical_plans.go b/pkg/planner/core/exhaust_physical_plans.go index d5dc8f8274c05..9100e42fbd367 100644 --- a/pkg/planner/core/exhaust_physical_plans.go +++ b/pkg/planner/core/exhaust_physical_plans.go @@ -3648,8 +3648,7 @@ func exhaustPhysicalPlans4LogicalCTE(p *LogicalCTE, prop *property.PhysicalPrope return []base.PhysicalPlan{(*PhysicalCTEStorage)(pcte)}, true, nil } -// ExhaustPhysicalPlans implements LogicalPlan interface. -func (p *LogicalSequence) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { +func exhaustPhysicalPlans4LogicalSequence(p *LogicalSequence, prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { possibleChildrenProps := make([][]*property.PhysicalProperty, 0, 2) anyType := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.AnyType, CanAddEnforcer: true, RejectSort: true, CTEProducerStatus: prop.CTEProducerStatus} if prop.TaskTp == property.MppTaskType { diff --git a/pkg/planner/core/logical_initialize.go b/pkg/planner/core/logical_initialize.go index 52cf82725d152..12b920702ab43 100644 --- a/pkg/planner/core/logical_initialize.go +++ b/pkg/planner/core/logical_initialize.go @@ -49,9 +49,3 @@ func (p LogicalCTETable) Init(ctx base.PlanContext, offset int) *LogicalCTETable p.BaseLogicalPlan = logicalop.NewBaseLogicalPlan(ctx, plancodec.TypeCTETable, &p, offset) return &p } - -// Init initializes LogicalSequence -func (p LogicalSequence) Init(ctx base.PlanContext, offset int) *LogicalSequence { - p.BaseLogicalPlan = logicalop.NewBaseLogicalPlan(ctx, plancodec.TypeSequence, &p, offset) - return &p -} diff --git a/pkg/planner/core/logical_plans.go b/pkg/planner/core/logical_plans.go index e5c57c59a1baa..5c060109dd235 100644 --- a/pkg/planner/core/logical_plans.go +++ b/pkg/planner/core/logical_plans.go @@ -355,22 +355,3 @@ type LogicalCTETable struct { // seedSchema is only used in columnStatsUsageCollector to get column mapping seedSchema *expression.Schema } - -// LogicalSequence is used to mark the CTE producer in the main query tree. -// Its last child is main query. The previous children are cte producers. -// And there might be dependencies between the CTE producers: -// -// Suppose that the sequence has 4 children, naming c0, c1, c2, c3. -// From the definition, c3 is the main query. c0, c1, c2 are CTE producers. -// It's possible that c1 references c0, c2 references c1 and c2. -// But it's no possible that c0 references c1 or c2. -// -// We use this property to do complex optimizations for CTEs. -type LogicalSequence struct { - logicalop.BaseLogicalPlan -} - -// Schema returns its last child(which is the main query plan)'s schema. -func (p *LogicalSequence) Schema() *expression.Schema { - return p.Children()[p.ChildLen()-1].Schema() -} diff --git a/pkg/planner/core/logical_sequence.go b/pkg/planner/core/logical_sequence.go new file mode 100644 index 0000000000000..c3350bdba22e0 --- /dev/null +++ b/pkg/planner/core/logical_sequence.go @@ -0,0 +1,129 @@ +// 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 core + +import ( + "github.com/pingcap/tidb/pkg/expression" + "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/optimizetrace" + "github.com/pingcap/tidb/pkg/util/plancodec" +) + +// LogicalSequence is used to mark the CTE producer in the main query tree. +// Its last child is main query. The previous children are cte producers. +// And there might be dependencies between the CTE producers: +// +// Suppose that the sequence has 4 children, naming c0, c1, c2, c3. +// From the definition, c3 is the main query. c0, c1, c2 are CTE producers. +// It's possible that c1 references c0, c2 references c1 and c2. +// But it's not possible that c0 references c1 or c2. +// +// We use this property to do complex optimizations for CTEs. +type LogicalSequence struct { + logicalop.BaseLogicalPlan +} + +// Init initializes LogicalSequence +func (p LogicalSequence) Init(ctx base.PlanContext, offset int) *LogicalSequence { + p.BaseLogicalPlan = logicalop.NewBaseLogicalPlan(ctx, plancodec.TypeSequence, &p, offset) + return &p +} + +// *************************** start implementation of Plan interface *************************** + +// Schema returns its last child(which is the main query plan)'s schema. +func (p *LogicalSequence) Schema() *expression.Schema { + return p.Children()[p.ChildLen()-1].Schema() +} + +// *************************** end implementation of Plan interface *************************** + +// *************************** start implementation of logicalPlan interface *************************** + +// HashCode inherits the BaseLogicalPlan.LogicalPlan.<0th> implementation. + +// PredicatePushDown implements the base.LogicalPlan.<1st> interface. +// Currently, we only maintain the main query tree. +func (p *LogicalSequence) PredicatePushDown(predicates []expression.Expression, op *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan) { + lastIdx := p.ChildLen() - 1 + remained, newLastChild := p.Children()[lastIdx].PredicatePushDown(predicates, op) + p.SetChild(lastIdx, newLastChild) + return remained, p +} + +// PruneColumns implements the base.LogicalPlan.<2nd> interface. +func (p *LogicalSequence) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error) { + var err error + p.Children()[p.ChildLen()-1], err = p.Children()[p.ChildLen()-1].PruneColumns(parentUsedCols, opt) + if err != nil { + return nil, err + } + return p, nil +} + +// FindBestTask inherits BaseLogicalPlan.LogicalPlan.<3rd> implementation. + +// BuildKeyInfo inherits BaseLogicalPlan.LogicalPlan.<4th> implementation. + +// PushDownTopN inherits BaseLogicalPlan.LogicalPlan.<5th> implementation. + +// DeriveTopN inherits BaseLogicalPlan.LogicalPlan.<6th> implementation. + +// PredicateSimplification inherits BaseLogicalPlan.LogicalPlan.<7th> implementation. + +// ConstantPropagation inherits BaseLogicalPlan.LogicalPlan.<8th> implementation. + +// PullUpConstantPredicates inherits BaseLogicalPlan.LogicalPlan.<9th> implementation. + +// RecursiveDeriveStats inherits BaseLogicalPlan.LogicalPlan.<10th> implementation. + +// DeriveStats implements the base.LogicalPlan.<11th> interface. +func (p *LogicalSequence) DeriveStats(childStats []*property.StatsInfo, _ *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error) { + p.SetStats(childStats[len(childStats)-1]) + return p.StatsInfo(), nil +} + +// ExtractColGroups inherits BaseLogicalPlan.LogicalPlan.<12th> implementation. + +// PreparePossibleProperties inherits BaseLogicalPlan.LogicalPlan.<13th> implementation. + +// ExhaustPhysicalPlans implements the base.LogicalPlan.<14th> interface. +func (p *LogicalSequence) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) { + return exhaustPhysicalPlans4LogicalSequence(p, prop) +} + +// ExtractCorrelatedCols inherits BaseLogicalPlan.LogicalPlan.<15th> implementation. + +// MaxOneRow inherits BaseLogicalPlan.LogicalPlan.<16th> implementation. + +// Children inherits BaseLogicalPlan.LogicalPlan.<17th> implementation. + +// SetChildren inherits BaseLogicalPlan.LogicalPlan.<18th> implementation. + +// SetChild inherits BaseLogicalPlan.LogicalPlan.<19th> implementation. + +// RollBackTaskMap inherits BaseLogicalPlan.LogicalPlan.<20th> implementation. + +// CanPushToCop inherits BaseLogicalPlan.LogicalPlan.<21st> implementation. + +// ExtractFD inherits BaseLogicalPlan.LogicalPlan.<22nd> implementation. + +// GetBaseLogicalPlan inherits BaseLogicalPlan.LogicalPlan.<23rd> implementation. + +// ConvertOuterToInnerJoin inherits BaseLogicalPlan.LogicalPlan.<24th> implementation. + +// *************************** end implementation of logicalPlan interface *************************** diff --git a/pkg/planner/core/rule_column_pruning.go b/pkg/planner/core/rule_column_pruning.go index 1b8ecdd33e9e6..cd160586220da 100644 --- a/pkg/planner/core/rule_column_pruning.go +++ b/pkg/planner/core/rule_column_pruning.go @@ -309,13 +309,3 @@ func preferKeyColumnFromTable(dataSource *DataSource, originColumns []*expressio } return resultColumn, resultColumnInfo } - -// PruneColumns implements the interface of base.LogicalPlan. -func (p *LogicalSequence) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error) { - var err error - p.Children()[p.ChildLen()-1], err = p.Children()[p.ChildLen()-1].PruneColumns(parentUsedCols, opt) - if err != nil { - return nil, err - } - return p, nil -} diff --git a/pkg/planner/core/rule_predicate_push_down.go b/pkg/planner/core/rule_predicate_push_down.go index 311bdef0c24c9..c93388dc9f2ea 100644 --- a/pkg/planner/core/rule_predicate_push_down.go +++ b/pkg/planner/core/rule_predicate_push_down.go @@ -674,12 +674,3 @@ func (adder *exprPrefixAdder) addExprPrefix4DNFCond(condition *expression.Scalar return []expression.Expression{expression.ComposeDNFCondition(exprCtx, newAccessItems...)}, nil } - -// PredicatePushDown implements the base.LogicalPlan interface. -// Currently, we only maintain the main query tree. -func (p *LogicalSequence) PredicatePushDown(predicates []expression.Expression, op *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan) { - lastIdx := p.ChildLen() - 1 - remained, newLastChild := p.Children()[lastIdx].PredicatePushDown(predicates, op) - p.SetChild(lastIdx, newLastChild) - return remained, p -} diff --git a/pkg/planner/core/stats.go b/pkg/planner/core/stats.go index 506ea43f5b9f7..1a1263e9763df 100644 --- a/pkg/planner/core/stats.go +++ b/pkg/planner/core/stats.go @@ -621,12 +621,6 @@ func (p *LogicalCTETable) DeriveStats(_ []*property.StatsInfo, _ *expression.Sch return p.StatsInfo(), nil } -// DeriveStats implement LogicalPlan DeriveStats interface. -func (p *LogicalSequence) DeriveStats(childStats []*property.StatsInfo, _ *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error) { - p.SetStats(childStats[len(childStats)-1]) - return p.StatsInfo(), nil -} - // loadTableStats loads the stats of the table and store it in the statement `UsedStatsInfo` if it didn't exist func loadTableStats(ctx sessionctx.Context, tblInfo *model.TableInfo, pid int64) { statsRecord := ctx.GetSessionVars().StmtCtx.GetUsedStatsInfo(true)