Skip to content

Commit

Permalink
planner: don't DoOptimze when build show (pingcap#12005)
Browse files Browse the repository at this point in the history
  • Loading branch information
winoros authored and sre-bot committed Sep 9, 2019
1 parent 619b9a9 commit 4b911b8
Show file tree
Hide file tree
Showing 10 changed files with 85 additions and 69 deletions.
4 changes: 2 additions & 2 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -150,7 +150,7 @@ func (b *executorBuilder) build(p plannercore.Plan) Executor {
return b.buildShowDDLJobQueries(v)
case *plannercore.ShowSlow:
return b.buildShowSlow(v)
case *plannercore.Show:
case *plannercore.PhysicalShow:
return b.buildShow(v)
case *plannercore.Simple:
return b.buildSimple(v)
Expand Down Expand Up @@ -604,7 +604,7 @@ func (b *executorBuilder) buildExecute(v *plannercore.Execute) Executor {
return e
}

func (b *executorBuilder) buildShow(v *plannercore.Show) Executor {
func (b *executorBuilder) buildShow(v *plannercore.PhysicalShow) Executor {
e := &ShowExec{
baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()),
Tp: v.Tp,
Expand Down
19 changes: 0 additions & 19 deletions planner/core/common_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@ import (

"github.com/pingcap/errors"
"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/auth"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/expression"
Expand Down Expand Up @@ -367,24 +366,6 @@ type Deallocate struct {
Name string
}

// Show represents a show plan.
type Show struct {
physicalSchemaProducer

Tp ast.ShowStmtType // Databases/Tables/Columns/....
DBName string
Table *ast.TableName // Used for showing columns.
Column *ast.ColumnName // Used for `desc table column`.
IndexName model.CIStr
Flag int // Some flag parsed from sql, such as FULL.
User *auth.UserIdentity // Used for show grants.
Roles []*auth.RoleIdentity // Used for show grants.
Full bool
IfNotExists bool // Used for `show create database if not exists`

GlobalScope bool // Used by show variables
}

// Set represents a plan for set stmt.
type Set struct {
baseSchemaProducer
Expand Down
12 changes: 10 additions & 2 deletions planner/core/find_best_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -68,13 +68,21 @@ func (p *LogicalTableDual) findBestTask(prop *property.PhysicalProperty) (task,
return invalidTask, nil
}
dual := PhysicalTableDual{
RowCount: p.RowCount,
placeHolder: p.placeHolder,
RowCount: p.RowCount,
}.Init(p.ctx, p.stats)
dual.SetSchema(p.schema)
return &rootTask{p: dual}, nil
}

func (p *LogicalShow) findBestTask(prop *property.PhysicalProperty) (task, error) {
if !prop.IsEmpty() {
return invalidTask, nil
}
pShow := PhysicalShow{baseShowContent: p.baseShowContent}.Init(p.ctx)
pShow.SetSchema(p.schema)
return &rootTask{p: pShow}, nil
}

// findBestTask implements LogicalPlan interface.
func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty) (bestTask task, err error) {
// If p is an inner plan in an IndexJoin, the IndexJoin will generate an inner plan by itself,
Expand Down
10 changes: 8 additions & 2 deletions planner/core/initialize.go
Original file line number Diff line number Diff line change
Expand Up @@ -267,8 +267,14 @@ func (p Insert) Init(ctx sessionctx.Context) *Insert {
return &p
}

// Init initializes Show.
func (p Show) Init(ctx sessionctx.Context) *Show {
// Init initializes LogicalShow.
func (p LogicalShow) Init(ctx sessionctx.Context) *LogicalShow {
p.baseLogicalPlan = newBaseLogicalPlan(ctx, TypeShow, &p)
return &p
}

// Init initializes PhysicalShow.
func (p PhysicalShow) Init(ctx sessionctx.Context) *PhysicalShow {
p.basePhysicalPlan = newBasePhysicalPlan(ctx, TypeShow, &p)
// Just use pseudo stats to avoid panic.
p.stats = &property.StatsInfo{RowCount: 1}
Expand Down
2 changes: 1 addition & 1 deletion planner/core/logical_plan_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -938,7 +938,7 @@ func (s *testPlanSuite) TestPlanBuilder(c *C) {
},
{
sql: "show columns from t where `Key` = 'pri' like 't*'",
plan: "Show->Sel([eq(cast(key), 0)])",
plan: "Show->Sel([eq(cast(key), 0)])->Projection",
},
{
sql: "do sleep(5)",
Expand Down
26 changes: 22 additions & 4 deletions planner/core/logical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"math"

"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/auth"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/expression"
Expand Down Expand Up @@ -316,10 +317,6 @@ type LogicalTableDual struct {
logicalSchemaProducer

RowCount int
// placeHolder indicates if this dual plan is a place holder in query optimization
// for data sources like `Show`, if true, the dual plan would be substituted by
// `Show` in the final plan.
placeHolder bool
}

// LogicalUnionScan is only used in non read-only txn.
Expand Down Expand Up @@ -781,3 +778,24 @@ func extractCorColumnsBySchema(p LogicalPlan, schema *expression.Schema) []*expr
}
return resultCorCols[:length]
}

type baseShowContent struct {
Tp ast.ShowStmtType // Databases/Tables/Columns/....
DBName string
Table *ast.TableName // Used for showing columns.
Column *ast.ColumnName // Used for `desc table column`.
IndexName model.CIStr
Flag int // Some flag parsed from sql, such as FULL.
User *auth.UserIdentity // Used for show grants.
Roles []*auth.RoleIdentity // Used for show grants.
Full bool
IfNotExists bool // Used for `show create database if not exists`.

GlobalScope bool // Used by show variables.
}

// LogicalShow represents a show plan.
type LogicalShow struct {
logicalSchemaProducer
baseShowContent
}
11 changes: 7 additions & 4 deletions planner/core/physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -407,10 +407,6 @@ type PhysicalTableDual struct {
physicalSchemaProducer

RowCount int
// placeHolder indicates if this dual plan is a place holder in query optimization
// for data sources like `Show`, if true, the dual plan would be substituted by
// `Show` in the final plan.
placeHolder bool

// names is used for OutputNames() method. Dual may be inited when building point get plan.
// So it needs to hold names for itself.
Expand Down Expand Up @@ -455,6 +451,13 @@ func CollectPlanStatsVersion(plan PhysicalPlan, statsInfos map[string]uint64) ma
return statsInfos
}

// PhysicalShow represents a show plan.
type PhysicalShow struct {
physicalSchemaProducer

baseShowContent
}

// BuildMergeJoinPlan builds a PhysicalMergeJoin from the given fields. Currently, it is only used for test purpose.
func BuildMergeJoinPlan(ctx sessionctx.Context, joinType JoinType, leftKeys, rightKeys []*expression.Column) *PhysicalMergeJoin {
baseJoin := basePhysicalJoin{
Expand Down
54 changes: 20 additions & 34 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1361,18 +1361,20 @@ func splitWhere(where ast.ExprNode) []ast.ExprNode {
}

func (b *PlanBuilder) buildShow(ctx context.Context, show *ast.ShowStmt) (Plan, error) {
p := Show{
Tp: show.Tp,
DBName: show.DBName,
Table: show.Table,
Column: show.Column,
IndexName: show.IndexName,
Flag: show.Flag,
Full: show.Full,
User: show.User,
Roles: show.Roles,
IfNotExists: show.IfNotExists,
GlobalScope: show.GlobalScope,
p := LogicalShow{
baseShowContent: baseShowContent{
Tp: show.Tp,
DBName: show.DBName,
Table: show.Table,
Column: show.Column,
IndexName: show.IndexName,
Flag: show.Flag,
Full: show.Full,
User: show.User,
Roles: show.Roles,
IfNotExists: show.IfNotExists,
GlobalScope: show.GlobalScope,
},
}.Init(b.ctx)
isView := false
switch show.Tp {
Expand All @@ -1398,11 +1400,9 @@ func (b *PlanBuilder) buildShow(ctx context.Context, show *ast.ShowStmt) (Plan,
for _, col := range p.schema.Columns {
col.UniqueID = b.ctx.GetSessionVars().AllocPlanColumnID()
}
mockTablePlan := LogicalTableDual{placeHolder: true}.Init(b.ctx)
mockTablePlan.SetSchema(p.schema)
var err error
var np LogicalPlan
np = mockTablePlan
np = p
if show.Pattern != nil {
show.Pattern.Expr = &ast.ColumnNameExpr{
Name: &ast.ColumnName{Name: p.Schema().Columns[0].ColName},
Expand All @@ -1418,38 +1418,24 @@ func (b *PlanBuilder) buildShow(ctx context.Context, show *ast.ShowStmt) (Plan,
return nil, err
}
}
if np != mockTablePlan {
fieldsLen := len(mockTablePlan.schema.Columns)
if np != p {
b.optFlag |= flagEliminateProjection
fieldsLen := len(p.schema.Columns)
proj := LogicalProjection{Exprs: make([]expression.Expression, 0, fieldsLen)}.Init(b.ctx)
schema := expression.NewSchema(make([]*expression.Column, 0, fieldsLen)...)
for _, col := range mockTablePlan.schema.Columns {
for _, col := range p.schema.Columns {
proj.Exprs = append(proj.Exprs, col)
newCol := col.Clone().(*expression.Column)
newCol.UniqueID = b.ctx.GetSessionVars().AllocPlanColumnID()
schema.Append(newCol)
}
proj.SetSchema(schema)
proj.SetChildren(np)
physical, err := DoOptimize(ctx, b.optFlag|flagEliminateProjection, proj)
if err != nil {
return nil, err
}
return substitutePlaceHolderDual(physical, p), nil
return proj, nil
}
return p, nil
}

func substitutePlaceHolderDual(src PhysicalPlan, dst PhysicalPlan) PhysicalPlan {
if dual, ok := src.(*PhysicalTableDual); ok && dual.placeHolder {
return dst
}
for i, child := range src.Children() {
newChild := substitutePlaceHolderDual(child, dst)
src.SetChild(i, newChild)
}
return src
}

func (b *PlanBuilder) buildSimple(node ast.StmtNode) (Plan, error) {
p := &Simple{Statement: node}

Expand Down
14 changes: 14 additions & 0 deletions planner/core/stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,20 @@ func (p *LogicalTableDual) DeriveStats(childStats []*property.StatsInfo) (*prope
return p.stats, nil
}

// DeriveStats implement LogicalPlan DeriveStats interface.
func (p *LogicalShow) DeriveStats(childStats []*property.StatsInfo) (*property.StatsInfo, error) {
// A fake count, just to avoid panic now.
profile := &property.StatsInfo{
RowCount: 1,
Cardinality: make([]float64, p.Schema().Len()),
}
for i := range profile.Cardinality {
profile.Cardinality[i] = 1
}
p.stats = profile
return p.stats, nil
}

func (p *baseLogicalPlan) recursiveDeriveStats() (*property.StatsInfo, error) {
if p.stats != nil {
return p.stats, nil
Expand Down
2 changes: 1 addition & 1 deletion planner/core/stringer.go
Original file line number Diff line number Diff line change
Expand Up @@ -113,7 +113,7 @@ func toString(in Plan, strs []string, idxs []int) ([]string, []int) {
str = "Lock"
case *ShowDDL:
str = "ShowDDL"
case *Show:
case *LogicalShow, *PhysicalShow:
str = "Show"
case *LogicalSort, *PhysicalSort:
str = "Sort"
Expand Down

0 comments on commit 4b911b8

Please sign in to comment.