Skip to content

Commit

Permalink
planner: move logical cte into logicalop pkg (#55496)
Browse files Browse the repository at this point in the history
ref #51664, ref #52714
  • Loading branch information
AilinKid authored Aug 21, 2024
1 parent 1ee498e commit 0b6e2e6
Show file tree
Hide file tree
Showing 32 changed files with 231 additions and 172 deletions.
2 changes: 1 addition & 1 deletion pkg/planner/core/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@ go_library(
"indexmerge_path.go",
"indexmerge_unfinished_path.go",
"initialize.go",
"logical_cte.go",
"logical_datasource.go",
"logical_index_scan.go",
"logical_initialize.go",
Expand Down Expand Up @@ -287,6 +286,7 @@ go_test(
"//pkg/planner/core/base",
"//pkg/planner/core/operator/logicalop",
"//pkg/planner/core/operator/physicalop",
"//pkg/planner/core/rule",
"//pkg/planner/property",
"//pkg/planner/util",
"//pkg/planner/util/coretestsdk",
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/collect_column_stats_usage.go
Original file line number Diff line number Diff line change
Expand Up @@ -293,7 +293,7 @@ func (c *columnStatsUsageCollector) collectFromPlan(lp base.LogicalPlan) {
c.collectPredicateColumnsForUnionAll(x)
case *logicalop.LogicalPartitionUnionAll:
c.collectPredicateColumnsForUnionAll(&x.LogicalUnionAll)
case *LogicalCTE:
case *logicalop.LogicalCTE:
// Visit SeedPartLogicalPlan and RecursivePartLogicalPlan first.
c.collectFromPlan(x.Cte.SeedPartLogicalPlan)
if x.Cte.RecursivePartLogicalPlan != nil {
Expand Down
3 changes: 2 additions & 1 deletion pkg/planner/core/collect_column_stats_usage_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/pingcap/tidb/pkg/infoschema"
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/planner/core/base"
"github.com/pingcap/tidb/pkg/planner/core/rule"
"github.com/pingcap/tidb/pkg/util/hint"
"github.com/stretchr/testify/require"
)
Expand Down Expand Up @@ -396,7 +397,7 @@ func TestCollectHistNeededColumns(t *testing.T) {
flags := builder.GetOptFlag()
// JoinReOrder may need columns stats so collecting hist-needed columns must happen before JoinReOrder.
// Hence, we disable JoinReOrder and PruneColumnsAgain here.
flags &= ^(flagJoinReOrder | flagPrunColumnsAgain)
flags &= ^(rule.FlagJoinReOrder | rule.FlagPruneColumnsAgain)
lp, err = logicalOptimize(ctx, flags, lp)
require.NoError(t, err, comment)
checkColumnStatsUsageForStatsLoad(t, s.is, lp, tt.res, comment)
Expand Down
3 changes: 3 additions & 0 deletions pkg/planner/core/core_init.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,11 +34,13 @@ func init() {
utilfuncp.GetTaskPlanCost = getTaskPlanCost
utilfuncp.CanPushToCopImpl = canPushToCopImpl
utilfuncp.PushDownTopNForBaseLogicalPlan = pushDownTopNForBaseLogicalPlan
utilfuncp.FindBestTask4LogicalCTE = findBestTask4LogicalCTE
utilfuncp.FindBestTask4LogicalShow = findBestTask4LogicalShow
utilfuncp.FindBestTask4LogicalCTETable = findBestTask4LogicalCTETable
utilfuncp.FindBestTask4LogicalMemTable = findBestTask4LogicalMemTable
utilfuncp.FindBestTask4LogicalTableDual = findBestTask4LogicalTableDual
utilfuncp.FindBestTask4LogicalShowDDLJobs = findBestTask4LogicalShowDDLJobs
utilfuncp.ExhaustPhysicalPlans4LogicalCTE = exhaustPhysicalPlans4LogicalCTE
utilfuncp.ExhaustPhysicalPlans4LogicalSort = exhaustPhysicalPlans4LogicalSort
utilfuncp.ExhaustPhysicalPlans4LogicalTopN = exhaustPhysicalPlans4LogicalTopN
utilfuncp.ExhaustPhysicalPlans4LogicalLock = exhaustPhysicalPlans4LogicalLock
Expand All @@ -63,6 +65,7 @@ func init() {
utilfuncp.PushDownTopNForBaseLogicalPlan = pushDownTopNForBaseLogicalPlan
utilfuncp.AttachPlan2Task = attachPlan2Task
utilfuncp.WindowIsTopN = windowIsTopN
utilfuncp.DoOptimize = doOptimize

// For mv index init.
cardinality.GetTblInfoForUsedStatsByPhysicalID = getTblInfoForUsedStatsByPhysicalID
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 @@ -2480,7 +2480,7 @@ func canPushToCopImpl(lp base.LogicalPlan, storeTp kv.StoreType, considerDual bo
return false
case *logicalop.LogicalSequence:
return storeTp == kv.TiFlash
case *LogicalCTE:
case *logicalop.LogicalCTE:
if storeTp != kv.TiFlash {
return false
}
Expand Down Expand Up @@ -3073,7 +3073,8 @@ func exhaustPhysicalPlans4LogicalMaxOneRow(lp base.LogicalPlan, prop *property.P
return []base.PhysicalPlan{mor}, true, nil
}

func exhaustPhysicalPlans4LogicalCTE(p *LogicalCTE, prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) {
func exhaustPhysicalPlans4LogicalCTE(lp base.LogicalPlan, prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) {
p := lp.(*logicalop.LogicalCTE)
pcte := PhysicalCTE{CTE: p.Cte}.Init(p.SCtx(), p.StatsInfo())
if prop.IsFlashProp() {
pcte.storageSender = PhysicalExchangeSender{
Expand Down
9 changes: 5 additions & 4 deletions pkg/planner/core/expression_rewriter.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import (
"github.com/pingcap/tidb/pkg/parser/opcode"
"github.com/pingcap/tidb/pkg/planner/core/base"
"github.com/pingcap/tidb/pkg/planner/core/operator/logicalop"
"github.com/pingcap/tidb/pkg/planner/core/rule"
"github.com/pingcap/tidb/pkg/planner/util/coreusage"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/table"
Expand Down Expand Up @@ -1227,9 +1228,9 @@ func (er *expressionRewriter) handleInSubquery(ctx context.Context, planCtx *exp
// and don't need to append a scalar value, we can rewrite it to inner join.
if planCtx.builder.ctx.GetSessionVars().GetAllowInSubqToJoinAndAgg() && !v.Not && !asScalar && len(corCols) == 0 && collFlag {
// We need to try to eliminate the agg and the projection produced by this operation.
planCtx.builder.optFlag |= flagEliminateAgg
planCtx.builder.optFlag |= flagEliminateProjection
planCtx.builder.optFlag |= flagJoinReOrder
planCtx.builder.optFlag |= rule.FlagEliminateAgg
planCtx.builder.optFlag |= rule.FlagEliminateProjection
planCtx.builder.optFlag |= rule.FlagJoinReOrder
// Build distinct for the inner query.
agg, err := planCtx.builder.buildDistinct(np, np.Schema().Len())
if err != nil {
Expand Down Expand Up @@ -1375,7 +1376,7 @@ func (er *expressionRewriter) handleScalarSubquery(ctx context.Context, planCtx
}

func hasCTEConsumerInSubPlan(p base.LogicalPlan) bool {
if _, ok := p.(*LogicalCTE); ok {
if _, ok := p.(*logicalop.LogicalCTE); ok {
return true
}
for _, child := range p.Children() {
Expand Down
3 changes: 2 additions & 1 deletion pkg/planner/core/find_best_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -2900,7 +2900,8 @@ func getOriginalPhysicalIndexScan(ds *DataSource, prop *property.PhysicalPropert
return is
}

func findBestTask4LogicalCTE(p *LogicalCTE, prop *property.PhysicalProperty, counter *base.PlanCounterTp, pop *optimizetrace.PhysicalOptimizeOp) (t base.Task, cntPlan int64, err error) {
func findBestTask4LogicalCTE(lp base.LogicalPlan, prop *property.PhysicalProperty, counter *base.PlanCounterTp, pop *optimizetrace.PhysicalOptimizeOp) (t base.Task, cntPlan int64, err error) {
p := lp.(*logicalop.LogicalCTE)
if p.ChildLen() > 0 {
return p.BaseLogicalPlan.FindBestTask(prop, counter, pop)
}
Expand Down
Loading

0 comments on commit 0b6e2e6

Please sign in to comment.