From 03d58e0fc1b4ebdd37064b5605375e13d2a4e19e Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Thu, 20 Jan 2022 20:41:45 +0800 Subject: [PATCH] cherry pick #31742 to release-5.4 Signed-off-by: ti-srebot --- executor/builder.go | 1 + executor/show.go | 25 ++- 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 | 152 ++++++++++++++++++ planner/core/physical_plans.go | 2 + planner/core/planbuilder.go | 8 + planner/core/show_predicate_extractor.go | 97 +++++++++++ planner/core/stringer.go | 10 +- planner/core/stringer_test.go | 72 +++++++++ 11 files changed, 370 insertions(+), 7 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 ad216d44e3965..5426e6779647d 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 8449483a3787c..45df919cec29e 100644 --- a/executor/show.go +++ b/executor/show.go @@ -19,7 +19,11 @@ import ( "context" gjson "encoding/json" "fmt" +<<<<<<< HEAD "reflect" +======= + "regexp" +>>>>>>> ad1c5b508... *: add pushdown for ShowStmt and implement for show columns (#31742) "sort" "strconv" "strings" @@ -84,6 +88,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 @@ -514,10 +519,23 @@ 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 + FieldFilterEnable bool + 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) { @@ -536,10 +554,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 ff90a92b9b497..c2535efd48db4 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -155,7 +155,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 311b9ae4a8838..2cd9e290f0da8 100644 --- a/planner/core/memtable_predicate_extractor_test.go +++ b/planner/core/memtable_predicate_extractor_test.go @@ -22,13 +22,22 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb/domain" +<<<<<<< HEAD "github.com/pingcap/tidb/kv" +======= + "github.com/pingcap/tidb/errno" +>>>>>>> ad1c5b508... *: add pushdown for ShowStmt and implement for show columns (#31742) "github.com/pingcap/tidb/parser" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/util/hint" "github.com/pingcap/tidb/util/set" +<<<<<<< HEAD +======= + "github.com/pingcap/tidb/util/testutil" + "github.com/stretchr/testify/require" +>>>>>>> ad1c5b508... *: add pushdown for ShowStmt and implement for show columns (#31742) ) var _ = Suite(&extractorSuite{}) @@ -1420,3 +1429,146 @@ func (s *extractorSuite) TestTiDBHotRegionsHistoryTableExtractor(c *C) { } } } +<<<<<<< HEAD +======= + +func TestColumns(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + se, err := session.CreateSession4Test(store) + require.NoError(t, err) + + var cases = []struct { + sql string + columnName set.StringSet + tableSchema set.StringSet + tableName set.StringSet + columnNamePattern []string + tableSchemaPattern []string + tableNamePattern []string + skipRequest bool + }{ + { + sql: `select * from INFORMATION_SCHEMA.COLUMNS where column_name='T';`, + columnName: set.NewStringSet("t"), + }, + { + sql: `select * from INFORMATION_SCHEMA.COLUMNS where table_schema='TEST';`, + tableSchema: set.NewStringSet("test"), + }, + { + sql: `select * from INFORMATION_SCHEMA.COLUMNS where table_name='TEST';`, + tableName: set.NewStringSet("test"), + }, + { + sql: "select * from information_schema.COLUMNS where table_name in ('TEST','t') and column_name in ('A','b')", + columnName: set.NewStringSet("a", "b"), + tableName: set.NewStringSet("test", "t"), + }, + { + sql: `select * from information_schema.COLUMNS where table_name='a' and table_name in ('a', 'B');`, + tableName: set.NewStringSet("a"), + }, + { + sql: `select * from information_schema.COLUMNS where table_name='a' and table_name='B';`, + skipRequest: true, + }, + { + sql: `select * from information_schema.COLUMNS where table_name like 'T%';`, + tableNamePattern: []string{"(?i)T.*"}, + }, + { + sql: `select * from information_schema.COLUMNS where column_name like 'T%';`, + columnNamePattern: []string{"(?i)T.*"}, + }, + { + sql: `select * from information_schema.COLUMNS where column_name like 'i%';`, + columnNamePattern: []string{"(?i)i.*"}, + }, + { + sql: `select * from information_schema.COLUMNS where column_name like 'abc%' or column_name like "def%";`, + columnNamePattern: []string{"(?i)abc.*|def.*"}, + }, + { + sql: `select * from information_schema.COLUMNS where column_name like 'abc%' and column_name like "%def";`, + columnNamePattern: []string{"(?i)abc.*", "(?i).*def"}, + }, + } + parser := parser.New() + for _, ca := range cases { + logicalMemTable := getLogicalMemTable(t, dom, se, parser, ca.sql) + require.NotNil(t, logicalMemTable.Extractor) + + columnsTableExtractor := logicalMemTable.Extractor.(*plannercore.ColumnsTableExtractor) + require.Equal(t, ca.skipRequest, columnsTableExtractor.SkipRequest, "SQL: %v", ca.sql) + + require.Equal(t, ca.columnName.Count(), columnsTableExtractor.ColumnName.Count()) + if ca.columnName.Count() > 0 && columnsTableExtractor.ColumnName.Count() > 0 { + require.EqualValues(t, ca.columnName, columnsTableExtractor.ColumnName, "SQL: %v", ca.sql) + } + + require.Equal(t, ca.tableSchema.Count(), columnsTableExtractor.TableSchema.Count()) + if ca.tableSchema.Count() > 0 && columnsTableExtractor.TableSchema.Count() > 0 { + require.EqualValues(t, ca.tableSchema, columnsTableExtractor.TableSchema, "SQL: %v", ca.sql) + } + require.Equal(t, ca.tableName.Count(), columnsTableExtractor.TableName.Count()) + if ca.tableName.Count() > 0 && columnsTableExtractor.TableName.Count() > 0 { + require.EqualValues(t, ca.tableName, columnsTableExtractor.TableName, "SQL: %v", ca.sql) + } + require.Equal(t, len(ca.tableNamePattern), len(columnsTableExtractor.TableNamePatterns)) + if len(ca.tableNamePattern) > 0 && len(columnsTableExtractor.TableNamePatterns) > 0 { + require.EqualValues(t, ca.tableNamePattern, columnsTableExtractor.TableNamePatterns, "SQL: %v", ca.sql) + } + require.Equal(t, len(ca.columnNamePattern), len(columnsTableExtractor.ColumnNamePatterns)) + if len(ca.columnNamePattern) > 0 && len(columnsTableExtractor.ColumnNamePatterns) > 0 { + require.EqualValues(t, ca.columnNamePattern, columnsTableExtractor.ColumnNamePatterns, "SQL: %v", ca.sql) + } + require.Equal(t, len(ca.tableSchemaPattern), len(columnsTableExtractor.TableSchemaPatterns)) + if len(ca.tableSchemaPattern) > 0 && len(columnsTableExtractor.TableSchemaPatterns) > 0 { + require.EqualValues(t, ca.tableSchemaPattern, columnsTableExtractor.TableSchemaPatterns, "SQL: %v", ca.sql) + } + } +} + +func TestPredicateQuery(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + 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.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 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,,,")) + 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) +} +>>>>>>> ad1c5b508... *: add pushdown for ShowStmt and implement for show columns (#31742) 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 0dfd7ea71c743..026162be967d6 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -2871,7 +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 + } 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 new file mode 100644 index 0000000000000..0be76a053fd6d --- /dev/null +++ b/planner/core/show_predicate_extractor.go @@ -0,0 +1,97 @@ +// 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 whether the extractor can extract 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: + // 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) { + 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: + // It is used in `SHOW COLUMNS FROM t LIKE abc`. + // 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`. + 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("field:[%s], ", e.Field)) + } + + if len(e.FieldPatterns) > 0 { + r.WriteString(fmt.Sprintf("field_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..3ca5cbfc66568 --- /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(field:[a])", + }, + { + sql: "show columns from t like 'a%'", + plan: "Show(field_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(field:[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)) + } +}