From ac33c5f6662f33ae1d805b23839f9a2b10da7694 Mon Sep 17 00:00:00 2001 From: Kenan Yao Date: Thu, 11 Jul 2019 23:56:51 +0800 Subject: [PATCH] planner: support subquery in `SHOW` statement (#10942) --- executor/builder.go | 9 +---- planner/core/common_plans.go | 4 +- planner/core/find_best_task.go | 5 ++- planner/core/initialize.go | 4 +- planner/core/integration_test.go | 61 +++++++++++++++++++++++++++++++ planner/core/logical_plan_test.go | 2 +- planner/core/logical_plans.go | 4 ++ planner/core/physical_plans.go | 4 ++ planner/core/plan.go | 8 ++++ planner/core/planbuilder.go | 45 +++++++++++++++++------ planner/core/point_get_plan.go | 3 ++ planner/core/resolve_indices.go | 11 ------ planner/core/stringer.go | 6 +-- 13 files changed, 125 insertions(+), 41 deletions(-) create mode 100644 planner/core/integration_test.go diff --git a/executor/builder.go b/executor/builder.go index 361a79c6a8f44..a31d958aafabc 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -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 { diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index e737701646d13..cb45d2fe126eb 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -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 @@ -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 } diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 98b2468f5d193..40e20ea43287d 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -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 } diff --git a/planner/core/initialize.go b/planner/core/initialize.go index ff2a3dbc7b059..f0f1bddb393bd 100644 --- a/planner/core/initialize.go +++ b/planner/core/initialize.go @@ -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 } diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go new file mode 100644 index 0000000000000..ff7ca01e6dd92 --- /dev/null +++ b/planner/core/integration_test.go @@ -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 ", + "b int(11) YES ", + "c int(11) YES ", + )) + tk.MustQuery("show columns from t where field = 'b'").Check(testkit.Rows( + "b int(11) YES ", + )) + tk.MustQuery("show columns from t where field in (select 'b')").Check(testkit.Rows( + "b int(11) YES ", + )) + tk.MustQuery("show columns from t where field in (select 'b') and true").Check(testkit.Rows( + "b int(11) YES ", + )) + 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 ", + "b int(11) YES ", + )) + 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 ", + )) +} diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 228ed07bde88e..19476e63d0a8b 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -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)", diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index 8ff9f45cf3912..6ea6cc21f8395 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -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. diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index d570f5b589b00..67048b70f7e7c 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -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. diff --git a/planner/core/plan.go b/planner/core/plan.go index a141115e36140..12fd4b402e67e 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -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 } @@ -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 } diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index e9e33ca8f697f..545f1fc933a64 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -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} diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index 6bf2b6e197066..7be3ba1e49b07 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -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 diff --git a/planner/core/resolve_indices.go b/planner/core/resolve_indices.go index def6cea1f32f5..2340e747bb731 100644 --- a/planner/core/resolve_indices.go +++ b/planner/core/resolve_indices.go @@ -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 { diff --git a/planner/core/stringer.go b/planner/core/stringer.go index beba86d8cd527..f4178509bc742 100644 --- a/planner/core/stringer.go +++ b/planner/core/stringer.go @@ -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: