Skip to content

Commit

Permalink
.
Browse files Browse the repository at this point in the history
Signed-off-by: AilinKid <314806019@qq.com>
  • Loading branch information
AilinKid committed Jul 9, 2024
1 parent 49f1427 commit e195ff0
Show file tree
Hide file tree
Showing 8 changed files with 131 additions and 52 deletions.
1 change: 1 addition & 0 deletions pkg/planner/core/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ go_library(
"logical_plans.go",
"logical_projection.go",
"logical_selection.go",
"logical_sequence.go",
"logical_show.go",
"logical_sort.go",
"logical_table_dual.go",
Expand Down
3 changes: 1 addition & 2 deletions pkg/planner/core/exhaust_physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -3649,8 +3649,7 @@ func (p *LogicalCTE) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]ba
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 {
Expand Down
6 changes: 0 additions & 6 deletions pkg/planner/core/logical_initialize.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,9 +61,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
}
19 changes: 0 additions & 19 deletions pkg/planner/core/logical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -384,22 +384,3 @@ func (p *LogicalCTE) ExtractCorrelatedCols() []*expression.CorrelatedColumn {
}
return corCols
}

// 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()
}
129 changes: 129 additions & 0 deletions pkg/planner/core/logical_sequence.go
Original file line number Diff line number Diff line change
@@ -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 ***************************
10 changes: 0 additions & 10 deletions pkg/planner/core/rule_column_pruning.go
Original file line number Diff line number Diff line change
Expand Up @@ -315,13 +315,3 @@ func preferKeyColumnFromTable(dataSource *DataSource, originColumns []*expressio
func (p *LogicalCTE) PruneColumns(_ []*expression.Column, _ *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error) {
return p, nil
}

// 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
}
9 changes: 0 additions & 9 deletions pkg/planner/core/rule_predicate_push_down.go
Original file line number Diff line number Diff line change
Expand Up @@ -709,12 +709,3 @@ func (p *LogicalCTE) PredicatePushDown(predicates []expression.Expression, _ *op
p.cte.pushDownPredicates = append(p.cte.pushDownPredicates, expression.ComposeCNFCondition(p.SCtx().GetExprCtx(), newPred...))
return predicates, p.Self()
}

// 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
}
6 changes: 0 additions & 6 deletions pkg/planner/core/stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -686,12 +686,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)
Expand Down

0 comments on commit e195ff0

Please sign in to comment.