Skip to content

Commit

Permalink
planner: move logical show into logicalop pkg. (pingcap#54928)
Browse files Browse the repository at this point in the history
  • Loading branch information
AilinKid authored and hawkingrei committed Aug 1, 2024
1 parent a42b256 commit a382718
Show file tree
Hide file tree
Showing 12 changed files with 38 additions and 15 deletions.
2 changes: 1 addition & 1 deletion pkg/planner/cascades/implementation_rules.go
Original file line number Diff line number Diff line change
Expand Up @@ -244,7 +244,7 @@ func (*ImplShow) Match(_ *memo.GroupExpr, prop *property.PhysicalProperty) (matc
// OnImplement implements ImplementationRule OnImplement interface.
func (*ImplShow) OnImplement(expr *memo.GroupExpr, _ *property.PhysicalProperty) ([]memo.Implementation, error) {
logicProp := expr.Group.Prop
show := expr.ExprNode.(*plannercore.LogicalShow)
show := expr.ExprNode.(*logicalop.LogicalShow)
// TODO(zz-jason): unifying LogicalShow and PhysicalShow to a single
// struct. So that we don't need to create a new PhysicalShow object, which
// can help us to reduce the gc pressure of golang runtime and improve the
Expand Down
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_projection.go",
"logical_selection.go",
"logical_sequence.go",
"logical_show.go",
"logical_show_ddl_jobs.go",
"logical_sort.go",
"logical_table_dual.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 @@ -35,6 +35,7 @@ func init() {
utilfuncp.GetStreamAggs = getStreamAggs
utilfuncp.GetHashAggs = getHashAggs
utilfuncp.PruneByItems = pruneByItems
utilfuncp.FindBestTask4LogicalShow = findBestTask4LogicalShow
utilfuncp.FindBestTask4LogicalCTETable = findBestTask4LogicalCTETable
utilfuncp.FindBestTask4LogicalMemTable = findBestTask4LogicalMemTable
utilfuncp.ExhaustPhysicalPlans4LogicalMaxOneRow = exhaustPhysicalPlans4LogicalMaxOneRow
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 @@ -109,7 +109,8 @@ func findBestTask4LogicalTableDual(p *LogicalTableDual, prop *property.PhysicalP
return rt, 1, nil
}

func findBestTask4LogicalShow(p *LogicalShow, prop *property.PhysicalProperty, planCounter *base.PlanCounterTp, _ *optimizetrace.PhysicalOptimizeOp) (base.Task, int64, error) {
func findBestTask4LogicalShow(lp base.LogicalPlan, prop *property.PhysicalProperty, planCounter *base.PlanCounterTp, _ *optimizetrace.PhysicalOptimizeOp) (base.Task, int64, error) {
p := lp.(*logicalop.LogicalShow)
if !prop.IsSortItemEmpty() || planCounter.Empty() {
return base.InvalidTask, 0, 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 @@ -43,7 +43,7 @@ var (
_ base.LogicalPlan = &LogicalExpand{}
_ base.LogicalPlan = &LogicalUnionScan{}
_ base.LogicalPlan = &logicalop.LogicalMemTable{}
_ base.LogicalPlan = &LogicalShow{}
_ base.LogicalPlan = &logicalop.LogicalShow{}
_ base.LogicalPlan = &LogicalShowDDLJobs{}
_ base.LogicalPlan = &LogicalCTE{}
_ base.LogicalPlan = &logicalop.LogicalCTETable{}
Expand Down
4 changes: 4 additions & 0 deletions pkg/planner/core/operator/logicalop/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -8,13 +8,16 @@ go_library(
"logical_max_one_row.go",
"logical_mem_table.go",
"logical_schema_producer.go",
"logical_show.go",
],
importpath = "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop",
visibility = ["//visibility:public"],
deps = [
"//pkg/expression",
"//pkg/infoschema",
"//pkg/kv",
"//pkg/parser/ast",
"//pkg/parser/auth",
"//pkg/parser/model",
"//pkg/planner/core/base",
"//pkg/planner/core/operator/baseimpl",
Expand All @@ -28,6 +31,7 @@ go_library(
"//pkg/types",
"//pkg/util/dbterror/plannererrors",
"//pkg/util/plancodec",
"//pkg/util/size",
"//pkg/util/tracing",
],
)
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 (
"unsafe"
Expand All @@ -22,16 +22,16 @@ import (
"github.com/pingcap/tidb/pkg/parser/auth"
"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/optimizetrace"
"github.com/pingcap/tidb/pkg/planner/util/utilfuncp"
"github.com/pingcap/tidb/pkg/util/plancodec"
"github.com/pingcap/tidb/pkg/util/size"
)

// LogicalShow represents a show plan.
type LogicalShow struct {
logicalop.LogicalSchemaProducer
LogicalSchemaProducer
ShowContents

Extractor base.ShowPredicateExtractor
Expand Down Expand Up @@ -76,7 +76,7 @@ func (s *ShowContents) MemoryUsage() (sum int64) {

// Init initializes LogicalShow.
func (p LogicalShow) Init(ctx base.PlanContext) *LogicalShow {
p.BaseLogicalPlan = logicalop.NewBaseLogicalPlan(ctx, plancodec.TypeShow, &p, 0)
p.BaseLogicalPlan = NewBaseLogicalPlan(ctx, plancodec.TypeShow, &p, 0)
return &p
}

Expand All @@ -90,7 +90,7 @@ func (p LogicalShow) Init(ctx base.PlanContext) *LogicalShow {

// FindBestTask implements the base.LogicalPlan.<3rd> interface.
func (p *LogicalShow) FindBestTask(prop *property.PhysicalProperty, planCounter *base.PlanCounterTp, _ *optimizetrace.PhysicalOptimizeOp) (base.Task, int64, error) {
return findBestTask4LogicalShow(p, prop, planCounter, nil)
return utilfuncp.FindBestTask4LogicalShow(p, prop, planCounter, nil)
}

// BuildKeyInfo inherits BaseLogicalPlan.LogicalPlan.<4th> implementation.
Expand Down Expand Up @@ -138,3 +138,15 @@ func (p *LogicalShow) DeriveStats(_ []*property.StatsInfo, selfSchema *expressio
// ConvertOuterToInnerJoin inherits BaseLogicalPlan.LogicalPlan.<24th> implementation.

// *************************** end implementation of logicalPlan interface ***************************

// todo: merge getFakeStats with the one in logical_show_ddl_jobs.go
func getFakeStats(schema *expression.Schema) *property.StatsInfo {
profile := &property.StatsInfo{
RowCount: 1,
ColNDVs: make(map[int64]float64, schema.Len()),
}
for _, col := range schema.Columns {
profile.ColNDVs[col.UniqueID] = 1
}
return profile
}
3 changes: 2 additions & 1 deletion pkg/planner/core/physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/pingcap/tidb/pkg/planner/cardinality"
"github.com/pingcap/tidb/pkg/planner/core/base"
"github.com/pingcap/tidb/pkg/planner/core/cost"
"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/coreusage"
Expand Down Expand Up @@ -2547,7 +2548,7 @@ func CollectPlanStatsVersion(plan base.PhysicalPlan, statsInfos map[string]uint6
type PhysicalShow struct {
physicalSchemaProducer

ShowContents
logicalop.ShowContents

Extractor base.ShowPredicateExtractor
}
Expand Down
5 changes: 3 additions & 2 deletions pkg/planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@ import (
"github.com/pingcap/tidb/pkg/parser/opcode"
"github.com/pingcap/tidb/pkg/parser/terror"
"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/privilege"
Expand Down Expand Up @@ -3169,8 +3170,8 @@ func splitWhere(where ast.ExprNode) []ast.ExprNode {
}

func (b *PlanBuilder) buildShow(ctx context.Context, show *ast.ShowStmt) (base.Plan, error) {
p := LogicalShow{
ShowContents: ShowContents{
p := logicalop.LogicalShow{
ShowContents: logicalop.ShowContents{
Tp: show.Tp,
CountWarningsOrErrors: show.CountWarningsOrErrors,
DBName: show.DBName,
Expand Down
4 changes: 2 additions & 2 deletions pkg/planner/core/stringer.go
Original file line number Diff line number Diff line change
Expand Up @@ -175,9 +175,9 @@ func toString(in base.Plan, strs []string, idxs []int) ([]string, []int) {
str = "Lock"
case *ShowDDL:
str = "ShowDDL"
case *LogicalShow:
case *logicalop.LogicalShow:
str = "Show"
if pl := in.(*LogicalShow); pl.Extractor != nil {
if pl := in.(*logicalop.LogicalShow); pl.Extractor != nil {
str = str + "(" + pl.Extractor.ExplainInfo() + ")"
}
case *PhysicalShow:
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 @@ -113,7 +113,7 @@ func GetOperand(p base.LogicalPlan) Operand {
return OperandMemTableScan
case *plannercore.LogicalIndexScan:
return OperandIndexScan
case *plannercore.LogicalShow:
case *logicalop.LogicalShow:
return OperandShow
case *plannercore.LogicalWindow:
return OperandWindow
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 @@ -101,3 +101,7 @@ var FindBestTask4LogicalCTETable func(lp base.LogicalPlan, prop *property.Physic
var FindBestTask4LogicalMemTable func(lp base.LogicalPlan, prop *property.PhysicalProperty,
planCounter *base.PlanCounterTp, opt *optimizetrace.PhysicalOptimizeOp) (t base.Task,
cntPlan int64, err error)

// FindBestTask4LogicalShow will be called by LogicalShow in logicalOp pkg.
var FindBestTask4LogicalShow func(lp base.LogicalPlan, prop *property.PhysicalProperty, planCounter *base.PlanCounterTp,
_ *optimizetrace.PhysicalOptimizeOp) (base.Task, int64, error)

0 comments on commit a382718

Please sign in to comment.