From 05d4f2b9cc65e9f5cf18680e793eb4ad15f0cfbd Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Tue, 18 Jan 2022 11:37:44 +0800 Subject: [PATCH 01/10] *: add pushdown for ShowStmt and implement for show columns Signed-off-by: Weizhen Wang --- executor/builder.go | 1 + executor/show.go | 20 +++- planner/cascades/implementation_rules.go | 6 +- planner/core/find_best_task.go | 2 +- planner/core/logical_plans.go | 2 + .../core/memtable_predicate_extractor_test.go | 11 ++- planner/core/physical_plans.go | 2 + planner/core/planbuilder.go | 9 ++ planner/core/show_predicate_extractor.go | 94 +++++++++++++++++++ planner/core/stringer.go | 10 +- planner/core/stringer_test.go | 72 ++++++++++++++ 11 files changed, 221 insertions(+), 8 deletions(-) create mode 100644 planner/core/show_predicate_extractor.go create mode 100644 planner/core/stringer_test.go diff --git a/executor/builder.go b/executor/builder.go index 0b231dae12ac1..d1892401763b3 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -764,6 +764,7 @@ func (b *executorBuilder) buildShow(v *plannercore.PhysicalShow) Executor { IfNotExists: v.IfNotExists, GlobalScope: v.GlobalScope, Extended: v.Extended, + Extractor: v.Extractor, } if e.Tp == ast.ShowMasterStatus { // show master status need start ts. diff --git a/executor/show.go b/executor/show.go index f786259f35565..3ce7347988779 100644 --- a/executor/show.go +++ b/executor/show.go @@ -19,6 +19,7 @@ import ( "context" gjson "encoding/json" "fmt" + "regexp" "sort" "strconv" "strings" @@ -83,6 +84,7 @@ type ShowExec struct { Flag int // Some flag parsed from sql, such as FULL. Roles []*auth.RoleIdentity // Used for show grants. User *auth.UserIdentity // Used by show grants, show create user. + Extractor plannercore.ShowPredicateExtractor is infoschema.InfoSchema @@ -513,10 +515,21 @@ func (e *ShowExec) fetchShowTableStatus(ctx context.Context) error { func (e *ShowExec) fetchShowColumns(ctx context.Context) error { tb, err := e.getTable() - if err != nil { return errors.Trace(err) } + var fieldPatternsRegexp *regexp.Regexp + var FieldFilterEnable bool + var fieldFilter string + if e.Extractor != nil { + extractor := (e.Extractor).(*plannercore.ShowColumnsTableExtractor) + if extractor.FieldPatterns != "" { + fieldPatternsRegexp = regexp.MustCompile(extractor.FieldPatterns) + } + FieldFilterEnable = extractor.Field != "" + fieldFilter = extractor.Field + } + checker := privilege.GetPrivilegeManager(e.ctx) activeRoles := e.ctx.GetSessionVars().ActiveRoles if checker != nil && e.ctx.GetSessionVars().User != nil && !checker.RequestVerification(activeRoles, e.DBName.O, tb.Meta().Name.O, "", mysql.InsertPriv|mysql.SelectPriv|mysql.UpdatePriv|mysql.ReferencesPriv) { @@ -535,10 +548,11 @@ func (e *ShowExec) fetchShowColumns(ctx context.Context) error { return err } for _, col := range cols { - if e.Column != nil && e.Column.Name.L != col.Name.L { + if FieldFilterEnable && col.Name.L != fieldFilter { + continue + } else if fieldPatternsRegexp != nil && !fieldPatternsRegexp.MatchString(col.Name.L) { continue } - desc := table.NewColDesc(col) var columnDefault interface{} if desc.DefaultValue != nil { diff --git a/planner/cascades/implementation_rules.go b/planner/cascades/implementation_rules.go index b686d17d72469..7b30b67287202 100644 --- a/planner/cascades/implementation_rules.go +++ b/planner/cascades/implementation_rules.go @@ -242,12 +242,14 @@ func (r *ImplShow) Match(expr *memo.GroupExpr, prop *property.PhysicalProperty) func (r *ImplShow) OnImplement(expr *memo.GroupExpr, reqProp *property.PhysicalProperty) ([]memo.Implementation, error) { logicProp := expr.Group.Prop show := expr.ExprNode.(*plannercore.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 // overall performance. - showPhys := plannercore.PhysicalShow{ShowContents: show.ShowContents}.Init(show.SCtx()) + showPhys := plannercore.PhysicalShow{ + ShowContents: show.ShowContents, + Extractor: show.Extractor, + }.Init(show.SCtx()) showPhys.SetSchema(logicProp.Schema) return []memo.Implementation{impl.NewShowImpl(showPhys)}, nil } diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index b2d3fb3636d48..c49dfb6f316ba 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -156,7 +156,7 @@ func (p *LogicalShow) findBestTask(prop *property.PhysicalProperty, planCounter if !prop.IsEmpty() || planCounter.Empty() { return invalidTask, 0, nil } - pShow := PhysicalShow{ShowContents: p.ShowContents}.Init(p.ctx) + pShow := PhysicalShow{ShowContents: p.ShowContents, Extractor: p.Extractor}.Init(p.ctx) pShow.SetSchema(p.schema) planCounter.Dec(1) return &rootTask{p: pShow}, 1, nil diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index c322f7e9489c1..79eb82a76b47a 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -1261,6 +1261,8 @@ type ShowContents struct { type LogicalShow struct { logicalSchemaProducer ShowContents + + Extractor ShowPredicateExtractor } // LogicalShowDDLJobs is for showing DDL job list. diff --git a/planner/core/memtable_predicate_extractor_test.go b/planner/core/memtable_predicate_extractor_test.go index 43a774759b205..5b0935e1c97ad 100644 --- a/planner/core/memtable_predicate_extractor_test.go +++ b/planner/core/memtable_predicate_extractor_test.go @@ -22,12 +22,14 @@ import ( "time" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/parser" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/hint" "github.com/pingcap/tidb/util/set" + "github.com/pingcap/tidb/util/testutil" "github.com/stretchr/testify/require" ) @@ -1659,11 +1661,18 @@ func TestPredicateQuery(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t(id int, abclmn int);") - tk.MustQuery("select TABLE_NAME from information_schema.columns where table_schema = 'test' and column_name like 'i%'").Check(testkit.Rows("t")) tk.MustQuery("select TABLE_NAME from information_schema.columns where table_schema = 'TEST' and column_name like 'I%'").Check(testkit.Rows("t")) tk.MustQuery("select TABLE_NAME from information_schema.columns where table_schema = 'TEST' and column_name like 'ID'").Check(testkit.Rows("t")) tk.MustQuery("select TABLE_NAME from information_schema.columns where table_schema = 'TEST' and column_name like 'id'").Check(testkit.Rows("t")) tk.MustQuery("select column_name from information_schema.columns where table_schema = 'TEST' and (column_name like 'I%' or column_name like '%D')").Check(testkit.Rows("id")) tk.MustQuery("select column_name from information_schema.columns where table_schema = 'TEST' and (column_name like 'abc%' and column_name like '%lmn')").Check(testkit.Rows("abclmn")) + tk.MustQuery("describe t").Check(testkit.Rows("id int(11) YES ", "abclmn int(11) YES ")) + tk.MustQuery("describe t id").Check(testkit.Rows("id int(11) YES ")) + tk.MustGetErrCode("describe t 'I%'", errno.ErrParse) + tk.MustGetErrCode("describe t I%", errno.ErrParse) + + tk.MustQuery("show columns from t like `ABCLMN`").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show columns from t like 'abc%'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show columns from t like '%lmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) } diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 6293bba4b5073..876e71415b75e 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -1367,6 +1367,8 @@ type PhysicalShow struct { physicalSchemaProducer ShowContents + + Extractor ShowPredicateExtractor } // PhysicalShowDDLJobs is for showing DDL job list. diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 5e1e1faeddff2..a5260f41ed500 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -2871,6 +2871,15 @@ func (b *PlanBuilder) buildShow(ctx context.Context, show *ast.ShowStmt) (Plan, }.Init(b.ctx) isView := false isSequence := false + switch show.Tp { + case ast.ShowColumns: + var extractor ShowColumnsTableExtractor + if extractor.Extract(show) { + p.Extractor = &extractor + show.Pattern = nil + } + } + switch show.Tp { case ast.ShowTables, ast.ShowTableStatus: if p.DBName == "" { diff --git a/planner/core/show_predicate_extractor.go b/planner/core/show_predicate_extractor.go new file mode 100644 index 0000000000000..1dc97f49314e8 --- /dev/null +++ b/planner/core/show_predicate_extractor.go @@ -0,0 +1,94 @@ +// Copyright 2022 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 core + +import ( + "bytes" + "fmt" + "strings" + + "github.com/pingcap/tidb/parser/ast" + driver "github.com/pingcap/tidb/types/parser_driver" + "github.com/pingcap/tidb/util/collate" + "github.com/pingcap/tidb/util/stringutil" +) + +var _ ShowPredicateExtractor = &ShowColumnsTableExtractor{} + +// ShowPredicateExtractor is used to extract some predicates from `PatternLikeExpr` clause +// and push the predicates down to the data retrieving on reading memory table stage when use ShowStmt. +// +// e.g: +// SHOW COLUMNS FROM t LIKE '%abc%' +// We must request all components from the memory table, and filter the result by the PatternLikeExpr predicate. +// +// it is a way to fix https://github.com/pingcap/tidb/issues/29910. +type ShowPredicateExtractor interface { + // Extracts predicates which can be pushed down and returns the remained predicates + Extract(show *ast.ShowStmt) bool + explainInfo() string +} + +// ShowColumnsTableExtractor is used to extract some predicates of tables table. +type ShowColumnsTableExtractor struct { + Field string + + FieldPatterns string +} + +// Extract implements the MemTablePredicateExtractor Extract interface +func (e *ShowColumnsTableExtractor) Extract(show *ast.ShowStmt) bool { + if show.Pattern != nil && show.Pattern.Pattern != nil { + pattern := show.Pattern + switch pattern.Pattern.(type) { + case *driver.ValueExpr: + ptn := pattern.Pattern.(*driver.ValueExpr).GetString() + patValue, patTypes := stringutil.CompilePattern(ptn, pattern.Escape) + if !collate.NewCollationEnabled() && stringutil.IsExactMatch(patTypes) { + e.Field = strings.ToLower(string(patValue)) + return true + } + // (?i) mean to be case-insensitive. + e.FieldPatterns = "(?i)" + stringutil.CompileLike2Regexp(string(patValue)) + return true + case *ast.ColumnNameExpr: + e.Field = pattern.Pattern.(*ast.ColumnNameExpr).Name.Name.L + return true + } + } else if show.Column != nil && show.Column.Name.L != "" { + e.Field = show.Column.Name.L + return true + } + return false + +} + +func (e *ShowColumnsTableExtractor) explainInfo() string { + r := new(bytes.Buffer) + if len(e.Field) > 0 { + r.WriteString(fmt.Sprintf("feild:[%s], ", e.Field)) + } + + if len(e.FieldPatterns) > 0 { + r.WriteString(fmt.Sprintf("feild_pattern:[%s], ", e.FieldPatterns)) + } + + // remove the last ", " in the message info + s := r.String() + if len(s) > 2 { + return s[:len(s)-2] + } + return s +} diff --git a/planner/core/stringer.go b/planner/core/stringer.go index ce41a53bf66c8..68eb94c584861 100644 --- a/planner/core/stringer.go +++ b/planner/core/stringer.go @@ -132,8 +132,16 @@ func toString(in Plan, strs []string, idxs []int) ([]string, []int) { str = "Lock" case *ShowDDL: str = "ShowDDL" - case *LogicalShow, *PhysicalShow: + case *LogicalShow: str = "Show" + if pl := in.(*LogicalShow); pl.Extractor != nil { + str = str + "(" + pl.Extractor.explainInfo() + ")" + } + case *PhysicalShow: + str = "Show" + if pl := in.(*PhysicalShow); pl.Extractor != nil { + str = str + "(" + pl.Extractor.explainInfo() + ")" + } case *LogicalShowDDLJobs, *PhysicalShowDDLJobs: str = "ShowDDLJobs" case *LogicalSort, *PhysicalSort: diff --git a/planner/core/stringer_test.go b/planner/core/stringer_test.go new file mode 100644 index 0000000000000..b4420f4eb6ec5 --- /dev/null +++ b/planner/core/stringer_test.go @@ -0,0 +1,72 @@ +// Copyright 2022 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 core_test + +import ( + "context" + "testing" + + "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/util/hint" + "github.com/stretchr/testify/require" +) + +func TestPlanStringer(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("create table t(a int, b int, c int, index idx(a))") + tests := []struct { + sql string + plan string + }{ + { + sql: "show columns from t like 'a'", + plan: "Show(feild:[a])", + }, + { + sql: "show columns from t like 'a%'", + plan: "Show(feild_pattern:[(?i)a.*])", + }, + { + sql: "show columns from t where field = 'a'", + plan: "Show->Sel([eq(Column#13, a)])->Projection", + }, + { + sql: "desc t", + plan: "Show", + }, + { + sql: "desc t a", + plan: "Show(feild:[a])", + }, + } + parser := parser.New() + for _, tt := range tests { + stmt, err := parser.ParseOneStmt(tt.sql, "", "") + require.NoError(t, err, "for %s", tt.sql) + ret := &core.PreprocessorReturn{} + builder, _ := core.NewPlanBuilder().Init(tk.Session(), ret.InfoSchema, &hint.BlockHintProcessor{}) + p, err := builder.Build(context.TODO(), stmt) + require.NoError(t, err, "for %s", tt.sql) + p, err = core.LogicalOptimize(context.TODO(), builder.GetOptFlag(), p.(core.LogicalPlan)) + require.NoError(t, err, "for %s", tt.sql) + require.Equal(t, tt.plan, core.ToString(p)) + } +} From 08ca787b517d34266705b657a3faa10d79edba6b Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 19 Jan 2022 11:34:50 +0800 Subject: [PATCH 02/10] clean code Signed-off-by: Weizhen Wang --- executor/show.go | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/executor/show.go b/executor/show.go index 3ce7347988779..1fef2db83c915 100644 --- a/executor/show.go +++ b/executor/show.go @@ -518,9 +518,11 @@ func (e *ShowExec) fetchShowColumns(ctx context.Context) error { if err != nil { return errors.Trace(err) } - var fieldPatternsRegexp *regexp.Regexp - var FieldFilterEnable bool - var fieldFilter string + var ( + fieldPatternsRegexp *regexp.Regexp + FieldFilterEnable bool + fieldFilter string + ) if e.Extractor != nil { extractor := (e.Extractor).(*plannercore.ShowColumnsTableExtractor) if extractor.FieldPatterns != "" { From a51d9fc97032ae13b545301531b332502ddef76e Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 19 Jan 2022 16:17:46 +0800 Subject: [PATCH 03/10] clean code and fix some spelling errors Signed-off-by: Weizhen Wang --- planner/core/planbuilder.go | 5 ++--- planner/core/show_predicate_extractor.go | 5 ++++- planner/core/stringer_test.go | 4 ++-- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index a5260f41ed500..4e940cd56a089 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -2871,16 +2871,15 @@ func (b *PlanBuilder) buildShow(ctx context.Context, show *ast.ShowStmt) (Plan, }.Init(b.ctx) isView := false isSequence := false + switch show.Tp { case ast.ShowColumns: var extractor ShowColumnsTableExtractor if extractor.Extract(show) { p.Extractor = &extractor + // avoid to build Selection. show.Pattern = nil } - } - - switch show.Tp { case ast.ShowTables, ast.ShowTableStatus: if p.DBName == "" { return nil, ErrNoDB diff --git a/planner/core/show_predicate_extractor.go b/planner/core/show_predicate_extractor.go index 1dc97f49314e8..f590990e374a2 100644 --- a/planner/core/show_predicate_extractor.go +++ b/planner/core/show_predicate_extractor.go @@ -54,6 +54,7 @@ func (e *ShowColumnsTableExtractor) Extract(show *ast.ShowStmt) bool { pattern := show.Pattern switch pattern.Pattern.(type) { case *driver.ValueExpr: + // It is used in `SHOW COLUMNS FROM t LIKE `abc``. ptn := pattern.Pattern.(*driver.ValueExpr).GetString() patValue, patTypes := stringutil.CompilePattern(ptn, pattern.Escape) if !collate.NewCollationEnabled() && stringutil.IsExactMatch(patTypes) { @@ -64,10 +65,12 @@ func (e *ShowColumnsTableExtractor) Extract(show *ast.ShowStmt) bool { e.FieldPatterns = "(?i)" + stringutil.CompileLike2Regexp(string(patValue)) return true case *ast.ColumnNameExpr: + // It is used in `SHOW COLUMNS FROM t LIKE abc`. e.Field = pattern.Pattern.(*ast.ColumnNameExpr).Name.Name.L return true } } else if show.Column != nil && show.Column.Name.L != "" { + // it is used in `DESCRIBE t COLUMN`. e.Field = show.Column.Name.L return true } @@ -78,7 +81,7 @@ func (e *ShowColumnsTableExtractor) Extract(show *ast.ShowStmt) bool { func (e *ShowColumnsTableExtractor) explainInfo() string { r := new(bytes.Buffer) if len(e.Field) > 0 { - r.WriteString(fmt.Sprintf("feild:[%s], ", e.Field)) + r.WriteString(fmt.Sprintf("field:[%s], ", e.Field)) } if len(e.FieldPatterns) > 0 { diff --git a/planner/core/stringer_test.go b/planner/core/stringer_test.go index b4420f4eb6ec5..93a124676d33d 100644 --- a/planner/core/stringer_test.go +++ b/planner/core/stringer_test.go @@ -38,7 +38,7 @@ func TestPlanStringer(t *testing.T) { }{ { sql: "show columns from t like 'a'", - plan: "Show(feild:[a])", + plan: "Show(field:[a])", }, { sql: "show columns from t like 'a%'", @@ -54,7 +54,7 @@ func TestPlanStringer(t *testing.T) { }, { sql: "desc t a", - plan: "Show(feild:[a])", + plan: "Show(field:[a])", }, } parser := parser.New() From 785c94df3319d5f1be31f5141f7418371e25f303 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 19 Jan 2022 16:19:40 +0800 Subject: [PATCH 04/10] clean code and fix some spelling errors Signed-off-by: Weizhen Wang --- planner/core/show_predicate_extractor.go | 2 +- planner/core/stringer_test.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/planner/core/show_predicate_extractor.go b/planner/core/show_predicate_extractor.go index f590990e374a2..56b8bc725be35 100644 --- a/planner/core/show_predicate_extractor.go +++ b/planner/core/show_predicate_extractor.go @@ -85,7 +85,7 @@ func (e *ShowColumnsTableExtractor) explainInfo() string { } if len(e.FieldPatterns) > 0 { - r.WriteString(fmt.Sprintf("feild_pattern:[%s], ", e.FieldPatterns)) + r.WriteString(fmt.Sprintf("field_pattern:[%s], ", e.FieldPatterns)) } // remove the last ", " in the message info diff --git a/planner/core/stringer_test.go b/planner/core/stringer_test.go index 93a124676d33d..3ca5cbfc66568 100644 --- a/planner/core/stringer_test.go +++ b/planner/core/stringer_test.go @@ -42,7 +42,7 @@ func TestPlanStringer(t *testing.T) { }, { sql: "show columns from t like 'a%'", - plan: "Show(feild_pattern:[(?i)a.*])", + plan: "Show(field_pattern:[(?i)a.*])", }, { sql: "show columns from t where field = 'a'", From 3e0b34b742b156ca8d67a084a38635d02bac00c6 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Thu, 20 Jan 2022 14:16:35 +0800 Subject: [PATCH 05/10] add some tests and remove some implementation for mysql unsupported feature Signed-off-by: Weizhen Wang --- planner/core/memtable_predicate_extractor_test.go | 3 +++ planner/core/show_predicate_extractor.go | 6 +++--- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/planner/core/memtable_predicate_extractor_test.go b/planner/core/memtable_predicate_extractor_test.go index 5b0935e1c97ad..a3d97630ecf76 100644 --- a/planner/core/memtable_predicate_extractor_test.go +++ b/planner/core/memtable_predicate_extractor_test.go @@ -1675,4 +1675,7 @@ func TestPredicateQuery(t *testing.T) { tk.MustQuery("show columns from t like `ABCLMN`").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) tk.MustQuery("show columns from t like 'abc%'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) tk.MustQuery("show columns from t like '%lmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show columns from t where field like '%lmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show columns from t where field = 'abclmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustGetErrCode("show columns from t like id", errno.ErrBadField) } diff --git a/planner/core/show_predicate_extractor.go b/planner/core/show_predicate_extractor.go index 56b8bc725be35..0be76a053fd6d 100644 --- a/planner/core/show_predicate_extractor.go +++ b/planner/core/show_predicate_extractor.go @@ -36,7 +36,7 @@ var _ ShowPredicateExtractor = &ShowColumnsTableExtractor{} // // it is a way to fix https://github.com/pingcap/tidb/issues/29910. type ShowPredicateExtractor interface { - // Extracts predicates which can be pushed down and returns the remained predicates + // Extracts predicates which can be pushed down and returns whether the extractor can extract predicates. Extract(show *ast.ShowStmt) bool explainInfo() string } @@ -66,8 +66,8 @@ func (e *ShowColumnsTableExtractor) Extract(show *ast.ShowStmt) bool { return true case *ast.ColumnNameExpr: // It is used in `SHOW COLUMNS FROM t LIKE abc`. - e.Field = pattern.Pattern.(*ast.ColumnNameExpr).Name.Name.L - return true + // MySQL do not support this syntax and return the error. + return false } } else if show.Column != nil && show.Column.Name.L != "" { // it is used in `DESCRIBE t COLUMN`. From 43c2f5d5f6ad528fb625759500dca780f3721cf0 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Thu, 20 Jan 2022 14:32:23 +0800 Subject: [PATCH 06/10] add some tests and remove some implementation for mysql unsupported feature Signed-off-by: Weizhen Wang --- planner/core/memtable_predicate_extractor_test.go | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/planner/core/memtable_predicate_extractor_test.go b/planner/core/memtable_predicate_extractor_test.go index a3d97630ecf76..7f032d9dc594f 100644 --- a/planner/core/memtable_predicate_extractor_test.go +++ b/planner/core/memtable_predicate_extractor_test.go @@ -1672,10 +1672,17 @@ func TestPredicateQuery(t *testing.T) { tk.MustGetErrCode("describe t 'I%'", errno.ErrParse) tk.MustGetErrCode("describe t I%", errno.ErrParse) - tk.MustQuery("show columns from t like `ABCLMN`").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show columns from t like 'ABCLMN'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) tk.MustQuery("show columns from t like 'abc%'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) tk.MustQuery("show columns from t like '%lmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show columns from t where field like '%lmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) tk.MustQuery("show columns from t where field = 'abclmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show columns in t where field = 'abclmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show fields from t where field = 'abclmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show fields in t where field = 'abclmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("explain t").Check(testkit.Rows("id int(11) YES ", "abclmn int(11) YES ")) + tk.MustGetErrCode("show columns from t like id", errno.ErrBadField) + tk.MustGetErrCode("show columns from t like `id`", errno.ErrBadField) } From 1b3e994fb3910cfaf6931e12d295f7cb585793d6 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Thu, 20 Jan 2022 14:35:38 +0800 Subject: [PATCH 07/10] add more test Signed-off-by: Weizhen Wang --- planner/core/memtable_predicate_extractor_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/planner/core/memtable_predicate_extractor_test.go b/planner/core/memtable_predicate_extractor_test.go index 7f032d9dc594f..5a1ffe61fdcd1 100644 --- a/planner/core/memtable_predicate_extractor_test.go +++ b/planner/core/memtable_predicate_extractor_test.go @@ -1675,6 +1675,8 @@ func TestPredicateQuery(t *testing.T) { tk.MustQuery("show columns from t like 'ABCLMN'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) tk.MustQuery("show columns from t like 'abc%'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) tk.MustQuery("show columns from t like '%lmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show columns in t like '%lmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show fields in t like '%lmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) tk.MustQuery("show columns from t where field like '%lmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) tk.MustQuery("show columns from t where field = 'abclmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) From b296ff5660b4d9bfee7cc4ee1e32f3ad4674a64a Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Thu, 20 Jan 2022 14:41:27 +0800 Subject: [PATCH 08/10] add more test Signed-off-by: Weizhen Wang --- planner/core/memtable_predicate_extractor_test.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/planner/core/memtable_predicate_extractor_test.go b/planner/core/memtable_predicate_extractor_test.go index 5a1ffe61fdcd1..9a52ac4f15db7 100644 --- a/planner/core/memtable_predicate_extractor_test.go +++ b/planner/core/memtable_predicate_extractor_test.go @@ -1669,12 +1669,16 @@ func TestPredicateQuery(t *testing.T) { tk.MustQuery("select column_name from information_schema.columns where table_schema = 'TEST' and (column_name like 'abc%' and column_name like '%lmn')").Check(testkit.Rows("abclmn")) tk.MustQuery("describe t").Check(testkit.Rows("id int(11) YES ", "abclmn int(11) YES ")) tk.MustQuery("describe t id").Check(testkit.Rows("id int(11) YES ")) + tk.MustQuery("describe t ID").Check(testkit.Rows("id int(11) YES ")) tk.MustGetErrCode("describe t 'I%'", errno.ErrParse) tk.MustGetErrCode("describe t I%", errno.ErrParse) + tk.MustQuery("show columns from t like 'abclmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,") tk.MustQuery("show columns from t like 'ABCLMN'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) tk.MustQuery("show columns from t like 'abc%'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show columns from t like 'ABC%'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) tk.MustQuery("show columns from t like '%lmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show columns from t like '%LMN'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) tk.MustQuery("show columns in t like '%lmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) tk.MustQuery("show fields in t like '%lmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) From 7ce57d1480b7b0e636d00ad2a1fe547a26397cfd Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Thu, 20 Jan 2022 14:42:05 +0800 Subject: [PATCH 09/10] add more test Signed-off-by: Weizhen Wang --- planner/core/memtable_predicate_extractor_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/planner/core/memtable_predicate_extractor_test.go b/planner/core/memtable_predicate_extractor_test.go index 9a52ac4f15db7..a0c4d70271805 100644 --- a/planner/core/memtable_predicate_extractor_test.go +++ b/planner/core/memtable_predicate_extractor_test.go @@ -1680,7 +1680,9 @@ func TestPredicateQuery(t *testing.T) { tk.MustQuery("show columns from t like '%lmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) tk.MustQuery("show columns from t like '%LMN'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) tk.MustQuery("show columns in t like '%lmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show columns in t like '%LMN'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) tk.MustQuery("show fields in t like '%lmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show fields in t like '%LMN'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) tk.MustQuery("show columns from t where field like '%lmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) tk.MustQuery("show columns from t where field = 'abclmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) From 9afc4e4834069617dea330e103087832847f8865 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Thu, 20 Jan 2022 14:44:05 +0800 Subject: [PATCH 10/10] add more test Signed-off-by: Weizhen Wang --- planner/core/memtable_predicate_extractor_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/planner/core/memtable_predicate_extractor_test.go b/planner/core/memtable_predicate_extractor_test.go index a0c4d70271805..663da2cfb92d2 100644 --- a/planner/core/memtable_predicate_extractor_test.go +++ b/planner/core/memtable_predicate_extractor_test.go @@ -1673,7 +1673,7 @@ func TestPredicateQuery(t *testing.T) { tk.MustGetErrCode("describe t 'I%'", errno.ErrParse) tk.MustGetErrCode("describe t I%", errno.ErrParse) - tk.MustQuery("show columns from t like 'abclmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,") + tk.MustQuery("show columns from t like 'abclmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) tk.MustQuery("show columns from t like 'ABCLMN'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) tk.MustQuery("show columns from t like 'abc%'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) tk.MustQuery("show columns from t like 'ABC%'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,"))