Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

planner: classify logical expand logic into a separate file for later pkg move. #54869

Merged
merged 1 commit into from
Jul 25, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
5 changes: 1 addition & 4 deletions pkg/planner/core/exhaust_physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -2589,10 +2589,7 @@ func choosePartitionKeys(keys []*property.MPPPartitionColumn, matches []int) []*
return newKeys
}

// ExhaustPhysicalPlans enumerate all the possible physical plan for expand operator.
// The second boolean means whether we should resort to enforcer to satisfy prop requirement.
// false means we should, while true means we should not.
func (p *LogicalExpand) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) {
func exhaustPhysicalPlans4LogicalExpand(p *LogicalExpand, prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) {
// under the mpp task type, if the sort item is not empty, refuse it, cause expanded data doesn't support any sort items.
if !prop.IsSortItemEmpty() {
// false, meaning we can add a sort enforcer.
Expand Down
111 changes: 106 additions & 5 deletions pkg/planner/core/logical_expand.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,15 @@ import (
"fmt"

"github.com/pingcap/tidb/pkg/expression"
"github.com/pingcap/tidb/pkg/planner/core/base"
"github.com/pingcap/tidb/pkg/planner/core/operator/logicalop"
fd "github.com/pingcap/tidb/pkg/planner/funcdep"
"github.com/pingcap/tidb/pkg/planner/property"
"github.com/pingcap/tidb/pkg/planner/util/optimizetrace"
"github.com/pingcap/tidb/pkg/planner/util/optimizetrace/logicaltrace"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util/dbterror/plannererrors"
"github.com/pingcap/tidb/pkg/util/plancodec"
"github.com/pingcap/tipb/go-tipb"
)

Expand Down Expand Up @@ -59,13 +64,85 @@ type LogicalExpand struct {
GPosName *types.FieldName
}

// ExtractFD implements the logical plan interface, extracting the FD from bottom up.
func (p *LogicalExpand) ExtractFD() *fd.FDSet {
// basically extract the children's fdSet.
return p.LogicalSchemaProducer.ExtractFD()
// Init initializes LogicalProjection.
func (p LogicalExpand) Init(ctx base.PlanContext, offset int) *LogicalExpand {
p.BaseLogicalPlan = logicalop.NewBaseLogicalPlan(ctx, plancodec.TypeExpand, &p, offset)
return &p
}

// *************************** start implementation of logicalPlan interface ***************************

// HashCode inherits BaseLogicalPlan.LogicalPlan.<0th> implementation.

// PredicatePushDown implements base.LogicalPlan.<1st> interface.
func (p *LogicalExpand) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) (ret []expression.Expression, retPlan base.LogicalPlan) {
// Note that, grouping column related predicates can't be pushed down, since grouping column has nullability change after Expand OP itself.
// condition related with grouping column shouldn't be pushed down through it.
// currently, since expand is adjacent to aggregate, any filter above aggregate wanted to be push down through expand only have two cases:
// 1. agg function related filters. (these condition is always above aggregate)
// 2. group-by item related filters. (there condition is always related with grouping sets columns, which can't be pushed down)
// As a whole, we banned all the predicates pushing-down logic here that remained in Expand OP, and constructing a new selection above it if any.
remained, child := p.BaseLogicalPlan.PredicatePushDown(nil, opt)
return append(remained, predicates...), child
}

// PruneColumns implement the base.LogicalPlan.<2nd> interface.
// logicExpand is built in the logical plan building phase, where all the column prune is not done yet. So the
// expand projection expressions is meaningless if it built at that time. (we only maintain its schema, while
// the level projection expressions construction is left to the last logical optimize rule)
//
// so when do the rule_column_pruning here, we just prune the schema is enough.
func (p *LogicalExpand) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error) {
// Expand need those extra redundant distinct group by columns projected from underlying projection.
// distinct GroupByCol must be used by aggregate above, to make sure this, append DistinctGroupByCol again.
parentUsedCols = append(parentUsedCols, p.DistinctGroupByCol...)
used := expression.GetUsedList(p.SCtx().GetExprCtx().GetEvalCtx(), parentUsedCols, p.Schema())
prunedColumns := make([]*expression.Column, 0)
for i := len(used) - 1; i >= 0; i-- {
if !used[i] {
prunedColumns = append(prunedColumns, p.Schema().Columns[i])
p.Schema().Columns = append(p.Schema().Columns[:i], p.Schema().Columns[i+1:]...)
p.SetOutputNames(append(p.OutputNames()[:i], p.OutputNames()[i+1:]...))
}
}
logicaltrace.AppendColumnPruneTraceStep(p, prunedColumns, opt)
// Underlying still need to keep the distinct group by columns and parent used columns.
var err error
p.Children()[0], err = p.Children()[0].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 inherits BaseLogicalPlan.LogicalPlan.<11th> implementation.

// ExtractColGroups inherits BaseLogicalPlan.LogicalPlan.<12th> implementation.

// PreparePossibleProperties inherits BaseLogicalPlan.LogicalPlan.<13th> implementation.

// ExhaustPhysicalPlans implements base.LogicalPlan.<14th> interface.
func (p *LogicalExpand) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) {
return exhaustPhysicalPlans4LogicalExpand(p, prop)
}

// ExtractCorrelatedCols implements LogicalPlan interface.
// ExtractCorrelatedCols implements base.LogicalPlan.<15th> interface.
func (p *LogicalExpand) ExtractCorrelatedCols() []*expression.CorrelatedColumn {
corCols := make([]*expression.CorrelatedColumn, 0, len(p.LevelExprs[0]))
for _, lExpr := range p.LevelExprs {
Expand All @@ -76,6 +153,30 @@ func (p *LogicalExpand) ExtractCorrelatedCols() []*expression.CorrelatedColumn {
return corCols
}

// 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 implements the base.LogicalPlan.<22nd> interface, extracting the FD from bottom up.
func (p *LogicalExpand) ExtractFD() *fd.FDSet {
// basically extract the children's fdSet.
return p.LogicalSchemaProducer.ExtractFD()
}

// GetBaseLogicalPlan inherits BaseLogicalPlan.LogicalPlan.<23rd> implementation.

// ConvertOuterToInnerJoin inherits BaseLogicalPlan.LogicalPlan.<24th> implementation.

// *************************** end implementation of logicalPlan interface ***************************

// GetUsedCols extracts all of the Columns used by proj.
func (*LogicalExpand) GetUsedCols() (usedCols []*expression.Column) {
// be careful that, expand OP itself, shouldn't output its own used cols, because
Expand Down
12 changes: 0 additions & 12 deletions pkg/planner/core/logical_initialize.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,15 +13,3 @@
// limitations under the License.

package core

import (
"github.com/pingcap/tidb/pkg/planner/core/base"
"github.com/pingcap/tidb/pkg/planner/core/operator/logicalop"
"github.com/pingcap/tidb/pkg/util/plancodec"
)

// Init initializes LogicalProjection.
func (p LogicalExpand) Init(ctx base.PlanContext, offset int) *LogicalExpand {
p.BaseLogicalPlan = logicalop.NewBaseLogicalPlan(ctx, plancodec.TypeExpand, &p, offset)
return &p
}
29 changes: 0 additions & 29 deletions pkg/planner/core/rule_column_pruning.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,35 +39,6 @@ func (*columnPruner) optimize(_ context.Context, lp base.LogicalPlan, opt *optim
return lp, planChanged, nil
}

// PruneColumns implement the Expand OP's column pruning logic.
// logicExpand is built in the logical plan building phase, where all the column prune is not done yet. So the
// expand projection expressions is meaningless if it built at that time. (we only maintain its schema, while
// the level projection expressions construction is left to the last logical optimize rule)
//
// so when do the rule_column_pruning here, we just prune the schema is enough.
func (p *LogicalExpand) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error) {
// Expand need those extra redundant distinct group by columns projected from underlying projection.
// distinct GroupByCol must be used by aggregate above, to make sure this, append DistinctGroupByCol again.
parentUsedCols = append(parentUsedCols, p.DistinctGroupByCol...)
used := expression.GetUsedList(p.SCtx().GetExprCtx().GetEvalCtx(), parentUsedCols, p.Schema())
prunedColumns := make([]*expression.Column, 0)
for i := len(used) - 1; i >= 0; i-- {
if !used[i] {
prunedColumns = append(prunedColumns, p.Schema().Columns[i])
p.Schema().Columns = append(p.Schema().Columns[:i], p.Schema().Columns[i+1:]...)
p.SetOutputNames(append(p.OutputNames()[:i], p.OutputNames()[i+1:]...))
}
}
logicaltrace.AppendColumnPruneTraceStep(p, prunedColumns, opt)
// Underlying still need to keep the distinct group by columns and parent used columns.
var err error
p.Children()[0], err = p.Children()[0].PruneColumns(parentUsedCols, opt)
if err != nil {
return nil, err
}
return p, nil
}

func pruneByItems(p base.LogicalPlan, old []*util.ByItems, opt *optimizetrace.LogicalOptimizeOp) (byItems []*util.ByItems,
parentUsedCols []*expression.Column) {
prunedByItems := make([]*util.ByItems, 0)
Expand Down
12 changes: 0 additions & 12 deletions pkg/planner/core/rule_predicate_push_down.go
Original file line number Diff line number Diff line change
Expand Up @@ -100,18 +100,6 @@ func BreakDownPredicates(p *LogicalProjection, predicates []expression.Expressio
return canBePushed, canNotBePushed
}

// PredicatePushDown implements base.LogicalPlan PredicatePushDown interface.
func (p *LogicalExpand) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) (ret []expression.Expression, retPlan base.LogicalPlan) {
// Note that, grouping column related predicates can't be pushed down, since grouping column has nullability change after Expand OP itself.
// condition related with grouping column shouldn't be pushed down through it.
// currently, since expand is adjacent to aggregate, any filter above aggregate wanted to be push down through expand only have two cases:
// 1. agg function related filters. (these condition is always above aggregate)
// 2. group-by item related filters. (there condition is always related with grouping sets columns, which can't be pushed down)
// As a whole, we banned all the predicates pushing-down logic here that remained in Expand OP, and constructing a new selection above it if any.
remained, child := p.BaseLogicalPlan.PredicatePushDown(nil, opt)
return append(remained, predicates...), child
}

// DeriveOtherConditions given a LogicalJoin, check the OtherConditions to see if we can derive more
// conditions for left/right child pushdown.
func DeriveOtherConditions(
Expand Down