Skip to content

Commit

Permalink
planner: move logical union scan into logicalop pkg. (pingcap#55159)
Browse files Browse the repository at this point in the history
  • Loading branch information
AilinKid authored and hawkingrei committed Aug 8, 2024
1 parent 9da1eec commit 767f2e6
Show file tree
Hide file tree
Showing 12 changed files with 27 additions and 21 deletions.
1 change: 0 additions & 1 deletion pkg/planner/core/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,6 @@ go_library(
"logical_table_scan.go",
"logical_tikv_single_gather.go",
"logical_union_all.go",
"logical_union_scan.go",
"memtable_infoschema_extractor.go",
"memtable_predicate_extractor.go",
"mock.go",
Expand Down
3 changes: 2 additions & 1 deletion pkg/planner/core/core_init.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,9 +44,10 @@ func init() {
utilfuncp.ExhaustPhysicalPlans4LogicalTopN = exhaustPhysicalPlans4LogicalTopN
utilfuncp.ExhaustPhysicalPlans4LogicalLock = exhaustPhysicalPlans4LogicalLock
utilfuncp.ExhaustPhysicalPlans4LogicalLimit = exhaustPhysicalPlans4LogicalLimit
utilfuncp.ExhaustLogicalWindowPhysicalPlans = exhaustLogicalWindowPhysicalPlans
utilfuncp.ExhaustPhysicalPlans4LogicalWindow = exhaustPhysicalPlans4LogicalWindow
utilfuncp.ExhaustPhysicalPlans4LogicalSequence = exhaustPhysicalPlans4LogicalSequence
utilfuncp.ExhaustPhysicalPlans4LogicalMaxOneRow = exhaustPhysicalPlans4LogicalMaxOneRow
utilfuncp.ExhaustPhysicalPlans4LogicalUnionScan = exhaustPhysicalPlans4LogicalUnionScan
utilfuncp.ExhaustPhysicalPlans4LogicalProjection = exhaustPhysicalPlans4LogicalProjection

utilfuncp.AppendCandidate4PhysicalOptimizeOp = appendCandidate4PhysicalOptimizeOp
Expand Down
11 changes: 6 additions & 5 deletions pkg/planner/core/exhaust_physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,8 @@ import (
"go.uber.org/zap"
)

func exhaustPhysicalPlans4LogicalUnionScan(p *LogicalUnionScan, prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) {
func exhaustPhysicalPlans4LogicalUnionScan(lp base.LogicalPlan, prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) {
p := lp.(*logicalop.LogicalUnionScan)
if prop.IsFlashProp() {
p.SCtx().GetSessionVars().RaiseWarningWhenMPPEnforced(
"MPP mode may be blocked because operator `UnionScan` is not supported now.")
Expand Down Expand Up @@ -769,7 +770,7 @@ childLoop:
return nil
}
wrapper.zippedChildren = append(wrapper.zippedChildren, child)
case *LogicalUnionScan:
case *logicalop.LogicalUnionScan:
wrapper.hasDitryWrite = true
wrapper.zippedChildren = append(wrapper.zippedChildren, child)
default:
Expand Down Expand Up @@ -1029,7 +1030,7 @@ func constructInnerTableScanTask(
func constructInnerByZippedChildren(prop *property.PhysicalProperty, zippedChildren []base.LogicalPlan, child base.PhysicalPlan) base.PhysicalPlan {
for i := len(zippedChildren) - 1; i >= 0; i-- {
switch x := zippedChildren[i].(type) {
case *LogicalUnionScan:
case *logicalop.LogicalUnionScan:
child = constructInnerUnionScan(prop, x, child)
case *logicalop.LogicalProjection:
child = constructInnerProj(prop, x, child)
Expand Down Expand Up @@ -1077,7 +1078,7 @@ func constructInnerProj(prop *property.PhysicalProperty, proj *logicalop.Logical
return physicalProj
}

func constructInnerUnionScan(prop *property.PhysicalProperty, us *LogicalUnionScan, reader base.PhysicalPlan) base.PhysicalPlan {
func constructInnerUnionScan(prop *property.PhysicalProperty, us *logicalop.LogicalUnionScan, reader base.PhysicalPlan) base.PhysicalPlan {
if us == nil {
return reader
}
Expand Down Expand Up @@ -2282,7 +2283,7 @@ func tryToGetMppWindows(lw *logicalop.LogicalWindow, prop *property.PhysicalProp
return []base.PhysicalPlan{window}
}

func exhaustLogicalWindowPhysicalPlans(lp base.LogicalPlan, prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) {
func exhaustPhysicalPlans4LogicalWindow(lp base.LogicalPlan, prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) {
lw := lp.(*logicalop.LogicalWindow)
windows := make([]base.PhysicalPlan, 0, 2)

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 @@ -4762,7 +4762,7 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as
var result base.LogicalPlan = ds
dirty := tableHasDirtyContent(b.ctx, tableInfo)
if dirty || tableInfo.TempTableType == model.TempTableLocal || tableInfo.TableCacheStatusType == model.TableCacheStatusEnable {
us := LogicalUnionScan{HandleCols: handleCols}.Init(b.ctx, b.getSelectOffset())
us := logicalop.LogicalUnionScan{HandleCols: handleCols}.Init(b.ctx, b.getSelectOffset())
us.SetChildren(ds)
if tableInfo.Partition != nil && b.optFlag&flagPartitionProcessor == 0 {
// Adding ExtraPhysTblIDCol for UnionScan (transaction buffer handling)
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 @@ -41,7 +41,7 @@ var (
_ base.LogicalPlan = &logicalop.LogicalLimit{}
_ base.LogicalPlan = &logicalop.LogicalWindow{}
_ base.LogicalPlan = &LogicalExpand{}
_ base.LogicalPlan = &LogicalUnionScan{}
_ base.LogicalPlan = &logicalop.LogicalUnionScan{}
_ base.LogicalPlan = &logicalop.LogicalMemTable{}
_ base.LogicalPlan = &logicalop.LogicalShow{}
_ base.LogicalPlan = &logicalop.LogicalShowDDLJobs{}
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 @@ -17,6 +17,7 @@ go_library(
"logical_sort.go",
"logical_table_dual.go",
"logical_top_n.go",
"logical_union_scan.go",
"logical_window.go",
],
importpath = "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.

package core
package logicalop

import (
"bytes"
Expand All @@ -21,16 +21,16 @@ import (
"github.com/pingcap/tidb/pkg/expression"
"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/planner/property"
"github.com/pingcap/tidb/pkg/planner/util"
"github.com/pingcap/tidb/pkg/planner/util/optimizetrace"
"github.com/pingcap/tidb/pkg/planner/util/utilfuncp"
"github.com/pingcap/tidb/pkg/util/plancodec"
)

// LogicalUnionScan is used in non read-only txn or for scanning a local temporary table whose snapshot data is located in memory.
type LogicalUnionScan struct {
logicalop.BaseLogicalPlan
BaseLogicalPlan

Conditions []expression.Expression

Expand All @@ -39,7 +39,7 @@ type LogicalUnionScan struct {

// Init initializes LogicalUnionScan.
func (p LogicalUnionScan) Init(ctx base.PlanContext, qbOffset int) *LogicalUnionScan {
p.BaseLogicalPlan = logicalop.NewBaseLogicalPlan(ctx, plancodec.TypeUnionScan, &p, qbOffset)
p.BaseLogicalPlan = NewBaseLogicalPlan(ctx, plancodec.TypeUnionScan, &p, qbOffset)
return &p
}

Expand Down Expand Up @@ -125,7 +125,7 @@ func (p *LogicalUnionScan) PruneColumns(parentUsedCols []*expression.Column, opt

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

// ExtractCorrelatedCols inherits BaseLogicalPlan.LogicalPlan.<15th> implementation.
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/operator/logicalop/logical_window.go
Original file line number Diff line number Diff line change
Expand Up @@ -312,7 +312,7 @@ func (p *LogicalWindow) PreparePossibleProperties(_ *expression.Schema, _ ...[][

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

// ExtractCorrelatedCols implements base.LogicalPlan.<15th> interface.
Expand Down
4 changes: 2 additions & 2 deletions pkg/planner/core/rule_partition_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,7 @@ func (s *PartitionProcessor) rewriteDataSource(lp base.LogicalPlan, opt *optimiz
switch p := lp.(type) {
case *DataSource:
return s.prune(p, opt)
case *LogicalUnionScan:
case *logicalop.LogicalUnionScan:
ds := p.Children()[0]
ds, err := s.prune(ds.(*DataSource), opt)
if err != nil {
Expand All @@ -93,7 +93,7 @@ func (s *PartitionProcessor) rewriteDataSource(lp base.LogicalPlan, opt *optimiz
// Union->(UnionScan->DataSource1), (UnionScan->DataSource2)
children := make([]base.LogicalPlan, 0, len(ua.Children()))
for _, child := range ua.Children() {
us := LogicalUnionScan{
us := logicalop.LogicalUnionScan{
Conditions: p.Conditions,
HandleCols: p.HandleCols,
}.Init(ua.SCtx(), ua.QueryBlockOffset())
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/pattern/pattern.go
Original file line number Diff line number Diff line change
Expand Up @@ -93,7 +93,7 @@ func GetOperand(p base.LogicalPlan) Operand {
return OperandTableDual
case *plannercore.DataSource:
return OperandDataSource
case *plannercore.LogicalUnionScan:
case *logicalop.LogicalUnionScan:
return OperandUnionScan
case *plannercore.LogicalUnionAll:
return OperandUnionAll
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/pattern/pattern_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ func TestGetOperand(t *testing.T) {
require.Equal(t, OperandMaxOneRow, GetOperand(&logicalop.LogicalMaxOneRow{}))
require.Equal(t, OperandTableDual, GetOperand(&logicalop.LogicalTableDual{}))
require.Equal(t, OperandDataSource, GetOperand(&plannercore.DataSource{}))
require.Equal(t, OperandUnionScan, GetOperand(&plannercore.LogicalUnionScan{}))
require.Equal(t, OperandUnionScan, GetOperand(&logicalop.LogicalUnionScan{}))
require.Equal(t, OperandUnionAll, GetOperand(&plannercore.LogicalUnionAll{}))
require.Equal(t, OperandSort, GetOperand(&logicalop.LogicalSort{}))
require.Equal(t, OperandTopN, GetOperand(&logicalop.LogicalTopN{}))
Expand Down
8 changes: 6 additions & 2 deletions pkg/planner/util/utilfuncp/func_pointer_misc.go
Original file line number Diff line number Diff line change
Expand Up @@ -134,10 +134,14 @@ var ExhaustPhysicalPlans4LogicalLimit func(lp base.LogicalPlan, prop *property.P
var ExhaustPhysicalPlans4LogicalProjection func(lp base.LogicalPlan,
prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)

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

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

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

0 comments on commit 767f2e6

Please sign in to comment.