Skip to content

Commit

Permalink
planner: move logical cte table into logicalop pkg. (#54905)
Browse files Browse the repository at this point in the history
ref #51664, ref #52714
  • Loading branch information
AilinKid authored Jul 25, 2024
1 parent d12f841 commit 77c97ef
Show file tree
Hide file tree
Showing 9 changed files with 18 additions and 11 deletions.
1 change: 0 additions & 1 deletion pkg/planner/core/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@ go_library(
"logical_aggregation.go",
"logical_apply.go",
"logical_cte.go",
"logical_cte_table.go",
"logical_datasource.go",
"logical_expand.go",
"logical_index_scan.go",
Expand Down
3 changes: 2 additions & 1 deletion pkg/planner/core/collect_column_stats_usage.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,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/operator/logicalop"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/statistics/asyncload"
"github.com/pingcap/tidb/pkg/util/filter"
Expand Down Expand Up @@ -320,7 +321,7 @@ func (c *columnStatsUsageCollector) collectFromPlan(lp base.LogicalPlan) {
c.addPredicateColumn(col)
}
}
case *LogicalCTETable:
case *logicalop.LogicalCTETable:
// Schema change from seedPlan to self.
for i, col := range x.Schema().Columns {
c.updateColMap(col, []*expression.Column{x.SeedSchema.Columns[i]})
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 @@ -35,6 +35,7 @@ func init() {
utilfuncp.GetStreamAggs = getStreamAggs
utilfuncp.GetHashAggs = getHashAggs
utilfuncp.PruneByItems = pruneByItems
utilfuncp.FindBestTask4LogicalCTETable = findBestTask4LogicalCTETable
utilfuncp.ExhaustPhysicalPlans4LogicalMaxOneRow = exhaustPhysicalPlans4LogicalMaxOneRow

utilfuncp.AppendCandidate4PhysicalOptimizeOp = appendCandidate4PhysicalOptimizeOp
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 @@ -2927,7 +2927,8 @@ func findBestTask4LogicalCTE(p *LogicalCTE, prop *property.PhysicalProperty, cou
return t, 1, nil
}

func findBestTask4LogicalCTETable(p *LogicalCTETable, prop *property.PhysicalProperty, _ *base.PlanCounterTp, _ *optimizetrace.PhysicalOptimizeOp) (t base.Task, cntPlan int64, err error) {
func findBestTask4LogicalCTETable(lp base.LogicalPlan, prop *property.PhysicalProperty, _ *base.PlanCounterTp, _ *optimizetrace.PhysicalOptimizeOp) (t base.Task, cntPlan int64, err error) {
p := lp.(*logicalop.LogicalCTETable)
if !prop.IsSortItemEmpty() {
return base.InvalidTask, 0, nil
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/planner/core/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -650,7 +650,7 @@ func (b *PlanBuilder) buildJoin(ctx context.Context, joinNode *ast.Join) (base.L
}

// The recursive part in CTE must not be on the right side of a LEFT JOIN.
if lc, ok := rightPlan.(*LogicalCTETable); ok && joinNode.Tp == ast.LeftJoin {
if lc, ok := rightPlan.(*logicalop.LogicalCTETable); ok && joinNode.Tp == ast.LeftJoin {
return nil, plannererrors.ErrCTERecursiveForbiddenJoinOrder.GenWithStackByArgs(lc.Name)
}

Expand Down Expand Up @@ -4261,7 +4261,7 @@ func (b *PlanBuilder) tryBuildCTE(ctx context.Context, tn *ast.TableName, asName
}

cte.recursiveRef = true
p := LogicalCTETable{Name: cte.def.Name.String(), IDForStorage: cte.storageID, SeedStat: cte.seedStat, SeedSchema: cte.seedLP.Schema()}.Init(b.ctx, b.getSelectOffset())
p := logicalop.LogicalCTETable{Name: cte.def.Name.String(), IDForStorage: cte.storageID, SeedStat: cte.seedStat, SeedSchema: cte.seedLP.Schema()}.Init(b.ctx, b.getSelectOffset())
p.SetSchema(getResultCTESchema(cte.seedLP.Schema(), b.ctx.GetSessionVars()))
p.SetOutputNames(cte.seedLP.OutputNames())
return p, nil
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 @@ -46,7 +46,7 @@ var (
_ base.LogicalPlan = &LogicalShow{}
_ base.LogicalPlan = &LogicalShowDDLJobs{}
_ base.LogicalPlan = &LogicalCTE{}
_ base.LogicalPlan = &LogicalCTETable{}
_ base.LogicalPlan = &logicalop.LogicalCTETable{}
_ base.LogicalPlan = &LogicalSequence{}
)

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 @@ -4,6 +4,7 @@ go_library(
name = "logicalop",
srcs = [
"base_logical_plan.go",
"logical_cte_table.go",
"logical_max_one_row.go",
"logical_schema_producer.go",
],
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -12,20 +12,20 @@
// 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"
)

// LogicalCTETable is for CTE table
type LogicalCTETable struct {
logicalop.LogicalSchemaProducer
LogicalSchemaProducer

SeedStat *property.StatsInfo
Name string
Expand All @@ -37,7 +37,7 @@ type LogicalCTETable struct {

// Init only assigns type and context.
func (p LogicalCTETable) Init(ctx base.PlanContext, offset int) *LogicalCTETable {
p.BaseLogicalPlan = logicalop.NewBaseLogicalPlan(ctx, plancodec.TypeCTETable, &p, offset)
p.BaseLogicalPlan = NewBaseLogicalPlan(ctx, plancodec.TypeCTETable, &p, offset)
return &p
}

Expand All @@ -51,7 +51,7 @@ func (p LogicalCTETable) Init(ctx base.PlanContext, offset int) *LogicalCTETable

// FindBestTask implements the base.LogicalPlan.<3rd> interface.
func (p *LogicalCTETable) FindBestTask(prop *property.PhysicalProperty, _ *base.PlanCounterTp, _ *optimizetrace.PhysicalOptimizeOp) (t base.Task, cntPlan int64, err error) {
return findBestTask4LogicalCTETable(p, prop, nil, nil)
return utilfuncp.FindBestTask4LogicalCTETable(p, prop, nil, nil)
}

// BuildKeyInfo inherits BaseLogicalPlan.LogicalPlan.<4th> implementation.
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 @@ -92,3 +92,7 @@ var PruneByItems func(p base.LogicalPlan, old []*util.ByItems, opt *optimizetrac
// ExhaustPhysicalPlans4LogicalMaxOneRow will be called by LogicalMaxOneRow in logicalOp pkg.
var ExhaustPhysicalPlans4LogicalMaxOneRow func(p base.LogicalPlan, prop *property.PhysicalProperty) (
[]base.PhysicalPlan, bool, error)

// FindBestTask4LogicalCTETable will be called by LogicalCTETable in logicalOp pkg.
var FindBestTask4LogicalCTETable func(lp base.LogicalPlan, prop *property.PhysicalProperty, _ *base.PlanCounterTp,
_ *optimizetrace.PhysicalOptimizeOp) (t base.Task, cntPlan int64, err error)

0 comments on commit 77c97ef

Please sign in to comment.