Skip to content

Commit

Permalink
planner: move logical sequence into logicalop pkg. (#54955)
Browse files Browse the repository at this point in the history
ref #51664, ref #52714
  • Loading branch information
AilinKid authored Jul 29, 2024
1 parent 069258e commit e92baf9
Show file tree
Hide file tree
Showing 11 changed files with 24 additions and 17 deletions.
1 change: 0 additions & 1 deletion pkg/planner/core/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,6 @@ go_library(
"logical_plans.go",
"logical_projection.go",
"logical_selection.go",
"logical_sequence.go",
"logical_sort.go",
"logical_table_dual.go",
"logical_table_scan.go",
Expand Down
1 change: 1 addition & 0 deletions pkg/planner/core/core_init.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ func init() {
utilfuncp.FindBestTask4LogicalCTETable = findBestTask4LogicalCTETable
utilfuncp.FindBestTask4LogicalMemTable = findBestTask4LogicalMemTable
utilfuncp.FindBestTask4LogicalShowDDLJobs = findBestTask4LogicalShowDDLJobs
utilfuncp.ExhaustPhysicalPlans4LogicalSequence = exhaustPhysicalPlans4LogicalSequence
utilfuncp.ExhaustPhysicalPlans4LogicalMaxOneRow = exhaustPhysicalPlans4LogicalMaxOneRow

utilfuncp.AppendCandidate4PhysicalOptimizeOp = appendCandidate4PhysicalOptimizeOp
Expand Down
5 changes: 3 additions & 2 deletions pkg/planner/core/exhaust_physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -3018,7 +3018,7 @@ func canPushToCopImpl(lp base.LogicalPlan, storeTp kv.StoreType, considerDual bo
// These operators can be partially push down to TiFlash, so we don't raise warning for them.
case *LogicalLimit, *LogicalTopN:
return false
case *LogicalSequence:
case *logicalop.LogicalSequence:
return storeTp == kv.TiFlash
case *LogicalCTE:
if storeTp != kv.TiFlash {
Expand Down Expand Up @@ -3574,7 +3574,8 @@ func exhaustPhysicalPlans4LogicalCTE(p *LogicalCTE, prop *property.PhysicalPrope
return []base.PhysicalPlan{(*PhysicalCTEStorage)(pcte)}, true, nil
}

func exhaustPhysicalPlans4LogicalSequence(p *LogicalSequence, prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) {
func exhaustPhysicalPlans4LogicalSequence(lp base.LogicalPlan, prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) {
p := lp.(*logicalop.LogicalSequence)
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
2 changes: 1 addition & 1 deletion pkg/planner/core/find_best_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -182,7 +182,7 @@ func enumeratePhysicalPlans4Task(
childCnts := make([]int64, p.ChildLen())
cntPlan = 0
iteration := iteratePhysicalPlan4BaseLogical
if _, ok := p.Self().(*LogicalSequence); ok {
if _, ok := p.Self().(*logicalop.LogicalSequence); ok {
iteration = iterateChildPlan4LogicalSequence
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -4058,7 +4058,7 @@ func (b *PlanBuilder) tryToBuildSequence(ctes []*cteInfo, p base.LogicalPlan) ba
lctes = append(lctes, lcte)
}
b.optFlag |= flagPushDownSequence
seq := LogicalSequence{}.Init(b.ctx, b.getSelectOffset())
seq := logicalop.LogicalSequence{}.Init(b.ctx, b.getSelectOffset())
seq.SetChildren(append(lctes, p)...)
seq.SetOutputNames(p.OutputNames().Shallow())
return seq
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/logical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ var (
_ base.LogicalPlan = &logicalop.LogicalShowDDLJobs{}
_ base.LogicalPlan = &LogicalCTE{}
_ base.LogicalPlan = &logicalop.LogicalCTETable{}
_ base.LogicalPlan = &LogicalSequence{}
_ base.LogicalPlan = &logicalop.LogicalSequence{}
)

// ExtractNotNullFromConds extracts not-null columns from conditions.
Expand Down
1 change: 1 addition & 0 deletions pkg/planner/core/operator/logicalop/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@ go_library(
"logical_max_one_row.go",
"logical_mem_table.go",
"logical_schema_producer.go",
"logical_sequence.go",
"logical_show.go",
"logical_show_ddl_jobs.go",
],
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -12,14 +12,14 @@
// 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/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/planner/util/utilfuncp"
"github.com/pingcap/tidb/pkg/util/plancodec"
)

Expand All @@ -34,12 +34,12 @@ import (
//
// We use this property to do complex optimizations for CTEs.
type LogicalSequence struct {
logicalop.BaseLogicalPlan
BaseLogicalPlan
}

// Init initializes LogicalSequence
func (p LogicalSequence) Init(ctx base.PlanContext, offset int) *LogicalSequence {
p.BaseLogicalPlan = logicalop.NewBaseLogicalPlan(ctx, plancodec.TypeSequence, &p, offset)
p.BaseLogicalPlan = NewBaseLogicalPlan(ctx, plancodec.TypeSequence, &p, offset)
return &p
}

Expand Down Expand Up @@ -103,7 +103,7 @@ func (p *LogicalSequence) DeriveStats(childStats []*property.StatsInfo, _ *expre

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

// ExtractCorrelatedCols inherits BaseLogicalPlan.LogicalPlan.<15th> implementation.
Expand Down
11 changes: 6 additions & 5 deletions pkg/planner/core/rule_push_down_sequence.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"context"

"github.com/pingcap/tidb/pkg/planner/core/base"
"github.com/pingcap/tidb/pkg/planner/core/operator/logicalop"
"github.com/pingcap/tidb/pkg/planner/util/optimizetrace"
)

Expand All @@ -33,8 +34,8 @@ func (pdss *pushDownSequenceSolver) optimize(_ context.Context, lp base.LogicalP
return pdss.recursiveOptimize(nil, lp), planChanged, nil
}

func (pdss *pushDownSequenceSolver) recursiveOptimize(pushedSequence *LogicalSequence, lp base.LogicalPlan) base.LogicalPlan {
_, ok := lp.(*LogicalSequence)
func (pdss *pushDownSequenceSolver) recursiveOptimize(pushedSequence *logicalop.LogicalSequence, lp base.LogicalPlan) base.LogicalPlan {
_, ok := lp.(*logicalop.LogicalSequence)
if !ok && pushedSequence == nil {
newChildren := make([]base.LogicalPlan, 0, len(lp.Children()))
for _, child := range lp.Children() {
Expand All @@ -44,9 +45,9 @@ func (pdss *pushDownSequenceSolver) recursiveOptimize(pushedSequence *LogicalSeq
return lp
}
switch x := lp.(type) {
case *LogicalSequence:
case *logicalop.LogicalSequence:
if pushedSequence == nil {
pushedSequence = LogicalSequence{}.Init(lp.SCtx(), lp.QueryBlockOffset())
pushedSequence = logicalop.LogicalSequence{}.Init(lp.SCtx(), lp.QueryBlockOffset())
pushedSequence.SetChildren(lp.Children()...)
return pdss.recursiveOptimize(pushedSequence, lp.Children()[len(lp.Children())-1])
}
Expand All @@ -55,7 +56,7 @@ func (pdss *pushDownSequenceSolver) recursiveOptimize(pushedSequence *LogicalSeq
allCTEs := make([]base.LogicalPlan, 0, childLen+pushedSequence.ChildLen()-2)
allCTEs = append(allCTEs, pushedSequence.Children()[:pushedSequence.ChildLen()-1]...)
allCTEs = append(allCTEs, x.Children()[:childLen-1]...)
pushedSequence = LogicalSequence{}.Init(lp.SCtx(), lp.QueryBlockOffset())
pushedSequence = logicalop.LogicalSequence{}.Init(lp.SCtx(), lp.QueryBlockOffset())
pushedSequence.SetChildren(append(allCTEs, mainQuery)...)
return pdss.recursiveOptimize(pushedSequence, mainQuery)
case *DataSource, *LogicalAggregation, *LogicalCTE:
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/stringer.go
Original file line number Diff line number Diff line change
Expand Up @@ -212,7 +212,7 @@ func toString(in base.Plan, strs []string, idxs []int) ([]string, []int) {
}
str = name + "{" + strings.Join(children, "->") + "}"
idxs = idxs[:last]
case *LogicalSequence:
case *logicalop.LogicalSequence:
last := len(idxs) - 1
idx := idxs[last]
children := strs[idx:]
Expand Down
4 changes: 4 additions & 0 deletions pkg/planner/util/utilfuncp/func_pointer_misc.go
Original file line number Diff line number Diff line change
Expand Up @@ -109,3 +109,7 @@ var FindBestTask4LogicalShow func(lp base.LogicalPlan, prop *property.PhysicalPr
// FindBestTask4LogicalShowDDLJobs will be called by LogicalShowDDLJobs in logicalOp pkg.
var FindBestTask4LogicalShowDDLJobs func(lp base.LogicalPlan, prop *property.PhysicalProperty,
planCounter *base.PlanCounterTp, _ *optimizetrace.PhysicalOptimizeOp) (base.Task, int64, error)

// ExhaustPhysicalPlans4LogicalSequence will be called by LogicalSequence in logicalOp pkg.
var ExhaustPhysicalPlans4LogicalSequence func(lp base.LogicalPlan, prop *property.PhysicalProperty) (
[]base.PhysicalPlan, bool, error)

0 comments on commit e92baf9

Please sign in to comment.