Skip to content

Commit

Permalink
*: add pushdown for ShowStmt and implement for show columns (#31742)
Browse files Browse the repository at this point in the history
close #29910
  • Loading branch information
hawkingrei authored Jan 20, 2022
1 parent a15f4fd commit ad1c5b5
Show file tree
Hide file tree
Showing 11 changed files with 243 additions and 8 deletions.
1 change: 1 addition & 0 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
22 changes: 19 additions & 3 deletions executor/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"context"
gjson "encoding/json"
"fmt"
"regexp"
"sort"
"strconv"
"strings"
Expand Down Expand Up @@ -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

Expand Down Expand Up @@ -513,10 +515,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) {
Expand All @@ -535,10 +550,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 {
Expand Down
6 changes: 4 additions & 2 deletions planner/cascades/implementation_rules.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down
2 changes: 1 addition & 1 deletion planner/core/find_best_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
2 changes: 2 additions & 0 deletions planner/core/logical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -1261,6 +1261,8 @@ type ShowContents struct {
type LogicalShow struct {
logicalSchemaProducer
ShowContents

Extractor ShowPredicateExtractor
}

// LogicalShowDDLJobs is for showing DDL job list.
Expand Down
29 changes: 28 additions & 1 deletion planner/core/memtable_predicate_extractor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
)

Expand Down Expand Up @@ -1659,11 +1661,36 @@ 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 <nil> ", "abclmn int(11) YES <nil> "))
tk.MustQuery("describe t id").Check(testkit.Rows("id int(11) YES <nil> "))
tk.MustQuery("describe t ID").Check(testkit.Rows("id int(11) YES <nil> "))
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,,<nil>,"))
tk.MustQuery("show columns from t like 'ABCLMN'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,<nil>,"))
tk.MustQuery("show columns from t like 'abc%'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,<nil>,"))
tk.MustQuery("show columns from t like 'ABC%'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,<nil>,"))
tk.MustQuery("show columns from t like '%lmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,<nil>,"))
tk.MustQuery("show columns from t like '%LMN'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,<nil>,"))
tk.MustQuery("show columns in t like '%lmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,<nil>,"))
tk.MustQuery("show columns in t like '%LMN'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,<nil>,"))
tk.MustQuery("show fields in t like '%lmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,<nil>,"))
tk.MustQuery("show fields in t like '%LMN'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,<nil>,"))

tk.MustQuery("show columns from t where field like '%lmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,<nil>,"))
tk.MustQuery("show columns from t where field = 'abclmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,<nil>,"))
tk.MustQuery("show columns in t where field = 'abclmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,<nil>,"))
tk.MustQuery("show fields from t where field = 'abclmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,<nil>,"))
tk.MustQuery("show fields in t where field = 'abclmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,<nil>,"))
tk.MustQuery("explain t").Check(testkit.Rows("id int(11) YES <nil> ", "abclmn int(11) YES <nil> "))

tk.MustGetErrCode("show columns from t like id", errno.ErrBadField)
tk.MustGetErrCode("show columns from t like `id`", errno.ErrBadField)
}
2 changes: 2 additions & 0 deletions planner/core/physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -1367,6 +1367,8 @@ type PhysicalShow struct {
physicalSchemaProducer

ShowContents

Extractor ShowPredicateExtractor
}

// PhysicalShowDDLJobs is for showing DDL job list.
Expand Down
8 changes: 8 additions & 0 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
97 changes: 97 additions & 0 deletions planner/core/show_predicate_extractor.go
Original file line number Diff line number Diff line change
@@ -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
}
10 changes: 9 additions & 1 deletion planner/core/stringer.go
Original file line number Diff line number Diff line change
Expand Up @@ -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:
Expand Down
72 changes: 72 additions & 0 deletions planner/core/stringer_test.go
Original file line number Diff line number Diff line change
@@ -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))
}
}

0 comments on commit ad1c5b5

Please sign in to comment.