Skip to content

Commit

Permalink
planner: support subquery in SHOW statement (pingcap#10942)
Browse files Browse the repository at this point in the history
  • Loading branch information
eurekaka committed Jul 26, 2019
1 parent 5bb3c06 commit ac33c5f
Show file tree
Hide file tree
Showing 13 changed files with 125 additions and 41 deletions.
9 changes: 1 addition & 8 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -576,14 +576,7 @@ func (b *executorBuilder) buildShow(v *plannercore.Show) Executor {
b.err = err
}
}
if len(v.Conditions) == 0 {
return e
}
sel := &SelectionExec{
baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID(), e),
filters: v.Conditions,
}
return sel
return e
}

func (b *executorBuilder) buildSimple(v *plannercore.Simple) Executor {
Expand Down
4 changes: 1 addition & 3 deletions planner/core/common_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -340,7 +340,7 @@ type Deallocate struct {

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

Tp ast.ShowStmtType // Databases/Tables/Columns/....
DBName string
Expand All @@ -353,8 +353,6 @@ type Show struct {
Roles []*auth.RoleIdentity // Used for show grants.
IfNotExists bool // Used for `show create database if not exists`

Conditions []expression.Expression

GlobalScope bool // Used by show variables
}

Expand Down
5 changes: 4 additions & 1 deletion planner/core/find_best_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,10 @@ func (p *LogicalTableDual) findBestTask(prop *property.PhysicalProperty) (task,
if !prop.IsEmpty() {
return invalidTask, nil
}
dual := PhysicalTableDual{RowCount: p.RowCount}.Init(p.ctx, p.stats)
dual := PhysicalTableDual{
RowCount: p.RowCount,
placeHolder: p.placeHolder,
}.Init(p.ctx, p.stats)
dual.SetSchema(p.schema)
return &rootTask{p: dual}, nil
}
Expand Down
4 changes: 3 additions & 1 deletion planner/core/initialize.go
Original file line number Diff line number Diff line change
Expand Up @@ -267,7 +267,9 @@ func (p Insert) Init(ctx sessionctx.Context) *Insert {

// Init initializes Show.
func (p Show) Init(ctx sessionctx.Context) *Show {
p.basePlan = newBasePlan(ctx, TypeShow)
p.basePhysicalPlan = newBasePhysicalPlan(ctx, TypeShow, &p)
// Just use pseudo stats to avoid panic.
p.stats = &property.StatsInfo{RowCount: 1}
return &p
}

Expand Down
61 changes: 61 additions & 0 deletions planner/core/integration_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
// Copyright 2019 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,
// See the License for the specific language governing permissions and
// limitations under the License.

package core_test

import (
. "github.com/pingcap/check"
"github.com/pingcap/tidb/util/testkit"
)

var _ = Suite(&testIntegrationSuite{})

type testIntegrationSuite struct {
}

func (s *testIntegrationSuite) TestShowSubquery(c *C) {
store, dom, err := newStoreWithBootstrap()
c.Assert(err, IsNil)
tk := testkit.NewTestKit(c, store)
defer func() {
dom.Close()
store.Close()
}()
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a varchar(10), b int, c int)")
tk.MustQuery("show columns from t where true").Check(testkit.Rows(
"a varchar(10) YES <nil> ",
"b int(11) YES <nil> ",
"c int(11) YES <nil> ",
))
tk.MustQuery("show columns from t where field = 'b'").Check(testkit.Rows(
"b int(11) YES <nil> ",
))
tk.MustQuery("show columns from t where field in (select 'b')").Check(testkit.Rows(
"b int(11) YES <nil> ",
))
tk.MustQuery("show columns from t where field in (select 'b') and true").Check(testkit.Rows(
"b int(11) YES <nil> ",
))
tk.MustQuery("show columns from t where field in (select 'b') and false").Check(testkit.Rows())
tk.MustExec("insert into t values('c', 0, 0)")
tk.MustQuery("show columns from t where field < all (select a from t)").Check(testkit.Rows(
"a varchar(10) YES <nil> ",
"b int(11) YES <nil> ",
))
tk.MustExec("insert into t values('b', 0, 0)")
tk.MustQuery("show columns from t where field < all (select a from t)").Check(testkit.Rows(
"a varchar(10) YES <nil> ",
))
}
2 changes: 1 addition & 1 deletion planner/core/logical_plan_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -829,7 +829,7 @@ func (s *testPlanSuite) TestPlanBuilder(c *C) {
},
{
sql: "show columns from t where `Key` = 'pri' like 't*'",
plan: "Show([eq(cast(key), 0)])",
plan: "Show->Sel([eq(cast(key), 0)])",
},
{
sql: "do sleep(5)",
Expand Down
4 changes: 4 additions & 0 deletions planner/core/logical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -309,6 +309,10 @@ 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
4 changes: 4 additions & 0 deletions planner/core/physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -380,6 +380,10 @@ 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
}

// PhysicalWindow is the physical operator of window function.
Expand Down
8 changes: 8 additions & 0 deletions planner/core/plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -141,6 +141,9 @@ type PhysicalPlan interface {
// SetChildren sets the children for the plan.
SetChildren(...PhysicalPlan)

// SetChild sets the ith child for the plan.
SetChild(i int, child PhysicalPlan)

// ResolveIndices resolves the indices for columns. After doing this, the columns can evaluate the rows by their indices.
ResolveIndices() error
}
Expand Down Expand Up @@ -296,6 +299,11 @@ func (p *basePhysicalPlan) SetChildren(children ...PhysicalPlan) {
p.children = children
}

// SetChild implements PhysicalPlan SetChild interface.
func (p *basePhysicalPlan) SetChild(i int, child PhysicalPlan) {
p.children[i] = child
}

func (p *basePlan) context() sessionctx.Context {
return p.ctx
}
Expand Down
45 changes: 34 additions & 11 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1131,35 +1131,58 @@ func (b *PlanBuilder) buildShow(show *ast.ShowStmt) (Plan, error) {
for _, col := range p.schema.Columns {
col.UniqueID = b.ctx.GetSessionVars().AllocPlanColumnID()
}
mockTablePlan := LogicalTableDual{}.Init(b.ctx)
mockTablePlan := LogicalTableDual{placeHolder: true}.Init(b.ctx)
mockTablePlan.SetSchema(p.schema)
var err error
var np LogicalPlan
np = mockTablePlan
if show.Pattern != nil {
show.Pattern.Expr = &ast.ColumnNameExpr{
Name: &ast.ColumnName{Name: p.Schema().Columns[0].ColName},
}
expr, _, err := b.rewrite(show.Pattern, mockTablePlan, nil, false)
np, err = b.buildSelection(np, show.Pattern, nil)
if err != nil {
return nil, err
}
p.Conditions = append(p.Conditions, expr)
}
if show.Where != nil {
conds := splitWhere(show.Where)
for _, cond := range conds {
expr, _, err := b.rewrite(cond, mockTablePlan, nil, false)
if err != nil {
return nil, err
}
p.Conditions = append(p.Conditions, expr)
np, err = b.buildSelection(np, show.Where, nil)
if err != nil {
return nil, err
}
err := p.ResolveIndices()
}
if np != mockTablePlan {
fieldsLen := len(mockTablePlan.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 {
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(b.optFlag|flagEliminateProjection, proj)
if err != nil {
return nil, err
}
return substitutePlaceHolderDual(physical, p), 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
3 changes: 3 additions & 0 deletions planner/core/point_get_plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -124,6 +124,9 @@ func (p *PointGetPlan) Children() []PhysicalPlan {
// SetChildren sets the children for the plan.
func (p *PointGetPlan) SetChildren(...PhysicalPlan) {}

// SetChild sets a specific child for the plan.
func (p *PointGetPlan) SetChild(i int, child PhysicalPlan) {}

// ResolveIndices resolves the indices for columns. After doing this, the columns can evaluate the rows by their indices.
func (p *PointGetPlan) ResolveIndices() error {
return nil
Expand Down
11 changes: 0 additions & 11 deletions planner/core/resolve_indices.go
Original file line number Diff line number Diff line change
Expand Up @@ -480,17 +480,6 @@ func (p *Insert) ResolveIndices() (err error) {
return
}

// ResolveIndices implements Plan interface.
func (p *Show) ResolveIndices() (err error) {
for i, expr := range p.Conditions {
p.Conditions[i], err = expr.ResolveIndices(p.schema)
if err != nil {
return err
}
}
return err
}

func (p *physicalSchemaProducer) ResolveIndices() (err error) {
err = p.basePhysicalPlan.ResolveIndices()
if err != nil {
Expand Down
6 changes: 1 addition & 5 deletions planner/core/stringer.go
Original file line number Diff line number Diff line change
Expand Up @@ -114,11 +114,7 @@ func toString(in Plan, strs []string, idxs []int) ([]string, []int) {
case *ShowDDL:
str = "ShowDDL"
case *Show:
if len(x.Conditions) == 0 {
str = "Show"
} else {
str = fmt.Sprintf("Show(%s)", x.Conditions)
}
str = "Show"
case *LogicalSort, *PhysicalSort:
str = "Sort"
case *LogicalJoin:
Expand Down

0 comments on commit ac33c5f

Please sign in to comment.