Skip to content

Commit

Permalink
planner: remove some empty file after logical ops are all classified. (
Browse files Browse the repository at this point in the history
  • Loading branch information
AilinKid authored Sep 27, 2024
1 parent 95edb84 commit 2419267
Show file tree
Hide file tree
Showing 11 changed files with 77 additions and 102 deletions.
2 changes: 0 additions & 2 deletions pkg/planner/core/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -17,15 +17,13 @@ go_library(
"flat_plan.go",
"foreign_key.go",
"fragment.go",
"hashcode.go",
"hint_utils.go",
"index_join_path.go",
"indexmerge_path.go",
"indexmerge_unfinished_path.go",
"initialize.go",
"logical_initialize.go",
"logical_plan_builder.go",
"logical_plans.go",
"memtable_infoschema_extractor.go",
"memtable_predicate_extractor.go",
"mock.go",
Expand Down
1 change: 0 additions & 1 deletion pkg/planner/core/core_init.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@ func init() {
utilfuncp.AddSelection = addSelection
utilfuncp.FindBestTask = findBestTask
utilfuncp.PruneByItems = pruneByItems
utilfuncp.HasMaxOneRowUtil = HasMaxOneRow
utilfuncp.GetTaskPlanCost = getTaskPlanCost
utilfuncp.CanPushToCopImpl = canPushToCopImpl
utilfuncp.PushDownTopNForBaseLogicalPlan = pushDownTopNForBaseLogicalPlan
Expand Down
15 changes: 0 additions & 15 deletions pkg/planner/core/hashcode.go

This file was deleted.

48 changes: 0 additions & 48 deletions pkg/planner/core/logical_plans.go

This file was deleted.

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_max_one_row.go",
"logical_mem_table.go",
"logical_partition_union_all.go",
"logical_plans_misc.go",
"logical_projection.go",
"logical_schema_producer.go",
"logical_selection.go",
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/operator/logicalop/base_logical_plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -151,7 +151,7 @@ func (p *BaseLogicalPlan) BuildKeyInfo(_ *expression.Schema, _ []*expression.Sch
for i := range p.children {
childMaxOneRow[i] = p.children[i].MaxOneRow()
}
p.maxOneRow = utilfuncp.HasMaxOneRowUtil(p.self, childMaxOneRow)
p.maxOneRow = HasMaxOneRow(p.self, childMaxOneRow)
}

// PushDownTopN implements the LogicalPlan.<5th> interface.
Expand Down
72 changes: 72 additions & 0 deletions pkg/planner/core/operator/logicalop/logical_plans_misc.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,72 @@
// Copyright 2024 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package logicalop

import (
"github.com/pingcap/tidb/pkg/planner/core/base"
)

var (
_ base.LogicalPlan = &LogicalJoin{}
_ base.LogicalPlan = &LogicalAggregation{}
_ base.LogicalPlan = &LogicalProjection{}
_ base.LogicalPlan = &LogicalSelection{}
_ base.LogicalPlan = &LogicalApply{}
_ base.LogicalPlan = &LogicalMaxOneRow{}
_ base.LogicalPlan = &LogicalTableDual{}
_ base.LogicalPlan = &DataSource{}
_ base.LogicalPlan = &TiKVSingleGather{}
_ base.LogicalPlan = &LogicalTableScan{}
_ base.LogicalPlan = &LogicalIndexScan{}
_ base.LogicalPlan = &LogicalUnionAll{}
_ base.LogicalPlan = &LogicalPartitionUnionAll{}
_ base.LogicalPlan = &LogicalSort{}
_ base.LogicalPlan = &LogicalLock{}
_ base.LogicalPlan = &LogicalLimit{}
_ base.LogicalPlan = &LogicalWindow{}
_ base.LogicalPlan = &LogicalExpand{}
_ base.LogicalPlan = &LogicalUnionScan{}
_ base.LogicalPlan = &LogicalMemTable{}
_ base.LogicalPlan = &LogicalShow{}
_ base.LogicalPlan = &LogicalShowDDLJobs{}
_ base.LogicalPlan = &LogicalCTE{}
_ base.LogicalPlan = &LogicalCTETable{}
_ base.LogicalPlan = &LogicalSequence{}
)

// HasMaxOneRow returns if the LogicalPlan will output at most one row.
func HasMaxOneRow(p base.LogicalPlan, childMaxOneRow []bool) bool {
if len(childMaxOneRow) == 0 {
// The reason why we use this check is that, this function
// is used both in planner/core and planner/cascades.
// In cascades planner, LogicalPlan may have no `children`.
return false
}
switch x := p.(type) {
case *LogicalLock, *LogicalLimit, *LogicalSort, *LogicalSelection,
*LogicalApply, *LogicalProjection, *LogicalWindow, *LogicalAggregation:
return childMaxOneRow[0]
case *LogicalMaxOneRow:
return true
case *LogicalJoin:
switch x.JoinType {
case SemiJoin, AntiSemiJoin, LeftOuterSemiJoin, AntiLeftOuterSemiJoin:
return childMaxOneRow[0]
default:
return childMaxOneRow[0] && childMaxOneRow[1]
}
}
return false
}
26 changes: 0 additions & 26 deletions pkg/planner/core/plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import (
"github.com/pingcap/tidb/pkg/expression"
"github.com/pingcap/tidb/pkg/planner/cardinality"
"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/sessionctx"
Expand Down Expand Up @@ -241,28 +240,3 @@ func getActualProbeCntFromProbeParents(pps []base.PhysicalPlan, statsColl *execd
}
return res
}

// HasMaxOneRow returns if the LogicalPlan will output at most one row.
func HasMaxOneRow(p base.LogicalPlan, childMaxOneRow []bool) bool {
if len(childMaxOneRow) == 0 {
// The reason why we use this check is that, this function
// is used both in planner/core and planner/cascades.
// In cascades planner, LogicalPlan may have no `children`.
return false
}
switch x := p.(type) {
case *logicalop.LogicalLock, *logicalop.LogicalLimit, *logicalop.LogicalSort, *logicalop.LogicalSelection,
*logicalop.LogicalApply, *logicalop.LogicalProjection, *logicalop.LogicalWindow, *logicalop.LogicalAggregation:
return childMaxOneRow[0]
case *logicalop.LogicalMaxOneRow:
return true
case *logicalop.LogicalJoin:
switch x.JoinType {
case logicalop.SemiJoin, logicalop.AntiSemiJoin, logicalop.LeftOuterSemiJoin, logicalop.AntiLeftOuterSemiJoin:
return childMaxOneRow[0]
default:
return childMaxOneRow[0] && childMaxOneRow[1]
}
}
return false
}
2 changes: 1 addition & 1 deletion pkg/planner/memo/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -14,9 +14,9 @@ go_library(
"//pkg/expression",
"//pkg/planner/core",
"//pkg/planner/core/base",
"//pkg/planner/core/operator/logicalop",
"//pkg/planner/pattern",
"//pkg/planner/property",
"//pkg/planner/util/utilfuncp",
],
)

Expand Down
4 changes: 2 additions & 2 deletions pkg/planner/memo/group.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,9 +22,9 @@ import (
// import core pkg first to call its init func.
_ "github.com/pingcap/tidb/pkg/planner/core"
"github.com/pingcap/tidb/pkg/planner/core/base"
"github.com/pingcap/tidb/pkg/planner/core/operator/logicalop"
"github.com/pingcap/tidb/pkg/planner/pattern"
"github.com/pingcap/tidb/pkg/planner/property"
"github.com/pingcap/tidb/pkg/planner/util/utilfuncp"
)

// ExploreMark is uses to mark whether a Group or GroupExpr has
Expand Down Expand Up @@ -223,5 +223,5 @@ func (g *Group) BuildKeyInfo() {
g.Prop.Schema.Keys = childSchema[0].Keys
}
e.ExprNode.BuildKeyInfo(g.Prop.Schema, childSchema)
g.Prop.MaxOneRow = e.ExprNode.MaxOneRow() || utilfuncp.HasMaxOneRowUtil(e.ExprNode, childMaxOneRow)
g.Prop.MaxOneRow = e.ExprNode.MaxOneRow() || logicalop.HasMaxOneRow(e.ExprNode, childMaxOneRow)
}
6 changes: 0 additions & 6 deletions pkg/planner/util/utilfuncp/func_pointer_misc.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,12 +28,6 @@ import (

// this file is used for passing function pointer at init(){} to avoid some import cycles.

// HasMaxOneRowUtil is used in baseLogicalPlan implementation of LogicalPlan interface, while
// the original HasMaxOneRowUtil has some dependency of original core pkg: like Datasource which
// hasn't been moved out of core pkg, so associative func pointer is introduced.
// todo: (1) arenatlx, remove this func pointer when concrete Logical Operators moved out of core.
var HasMaxOneRowUtil func(p base.LogicalPlan, childMaxOneRow []bool) bool

// AppendCandidate4PhysicalOptimizeOp is used in all logicalOp's findBestTask to trace the physical
// optimizing steps. Since we try to move baseLogicalPlan out of core, then other concrete logical
// operators, this appendCandidate4PhysicalOptimizeOp will make logicalOp/pkg back import core/pkg;
Expand Down

0 comments on commit 2419267

Please sign in to comment.