From efe8523d53c540a5bcfffdaa9bb7c755bace2231 Mon Sep 17 00:00:00 2001 From: EasonBall <592838129@qq.com> Date: Thu, 1 Feb 2024 18:30:25 +0800 Subject: [PATCH] planner/infoschema: add predicate pushdown for system tables (#50779) ref pingcap/tidb#50305 --- pkg/executor/infoschema_reader.go | 111 ++++++++++++++---- pkg/planner/core/logical_plan_builder.go | 5 + .../core/memtable_predicate_extractor.go | 68 +++++++++++ .../core/memtable_predicate_extractor_test.go | 92 +++++++++++++++ pkg/planner/core/rule_column_pruning.go | 3 +- .../r/executor/explainfor.result | 58 +++++++++ .../r/infoschema/infoschema.result | 57 ++++++++- .../t/executor/explainfor.test | 22 ++++ .../t/infoschema/infoschema.test | 28 ++++- 9 files changed, 414 insertions(+), 30 deletions(-) diff --git a/pkg/executor/infoschema_reader.go b/pkg/executor/infoschema_reader.go index 585aa291f0803..7d75f7875a23d 100644 --- a/pkg/executor/infoschema_reader.go +++ b/pkg/executor/infoschema_reader.go @@ -361,8 +361,18 @@ func (e *memtableRetriever) setDataFromSchemata(ctx sessionctx.Context, schemas func (e *memtableRetriever) setDataForStatistics(ctx sessionctx.Context, schemas []*model.DBInfo) { checker := privilege.GetPrivilegeManager(ctx) + extractor, ok := e.extractor.(*plannercore.InfoSchemaTablesExtractor) + if ok && extractor.SkipRequest { + return + } for _, schema := range schemas { + if ok && extractor.Filter("table_schema", schema.Name.L) { + continue + } for _, table := range schema.Tables { + if ok && extractor.Filter("table_name", table.Name.L) { + continue + } if checker != nil && !checker.RequestVerification(ctx.GetSessionVars().ActiveRoles, schema.Name.L, table.Name.L, "", mysql.AllPrivMask) { continue } @@ -459,8 +469,18 @@ func (e *memtableRetriever) setDataForStatisticsInTable(schema *model.DBInfo, ta func (e *memtableRetriever) setDataFromReferConst(sctx sessionctx.Context, schemas []*model.DBInfo) error { checker := privilege.GetPrivilegeManager(sctx) var rows [][]types.Datum + extractor, ok := e.extractor.(*plannercore.InfoSchemaTablesExtractor) + if ok && extractor.SkipRequest { + return nil + } for _, schema := range schemas { + if ok && extractor.Filter("table_schema", schema.Name.L) { + continue + } for _, table := range schema.Tables { + if ok && extractor.Filter("table_name", table.Name.L) { + continue + } if !table.IsBaseTable() { continue } @@ -496,12 +516,51 @@ func (e *memtableRetriever) setDataFromReferConst(sctx sessionctx.Context, schem return nil } +func fetchColumnsFromStatsCache(table *model.TableInfo) (rowCount uint64, avgRowLength uint64, dataLength uint64, indexLength uint64) { + cache := cache.TableRowStatsCache + if table.GetPartitionInfo() == nil { + rowCount = cache.GetTableRows(table.ID) + dataLength, indexLength = cache.GetDataAndIndexLength(table, table.ID, rowCount) + } else { + for _, pi := range table.GetPartitionInfo().Definitions { + piRowCnt := cache.GetTableRows(pi.ID) + rowCount += piRowCnt + parDataLen, parIndexLen := cache.GetDataAndIndexLength(table, pi.ID, piRowCnt) + dataLength += parDataLen + indexLength += parIndexLen + } + } + avgRowLength = uint64(0) + if rowCount != 0 { + avgRowLength = dataLength / rowCount + } + + if table.IsSequence() { + // sequence is always 1 row regardless of stats. + rowCount = 1 + } + return +} + +func (e *memtableRetriever) updateStatsCacheIfNeed(sctx sessionctx.Context) (bool, error) { + for _, col := range e.columns { + // only the following columns need stats cahce. + if col.Name.O == "AVG_ROW_LENGTH" || col.Name.O == "DATA_LENGTH" || col.Name.O == "INDEX_LENGTH" || col.Name.O == "TABLE_ROWS" { + err := cache.TableRowStatsCache.Update(sctx) + if err != nil { + return false, err + } + return true, err + } + } + return false, nil +} + func (e *memtableRetriever) setDataFromTables(sctx sessionctx.Context, schemas []*model.DBInfo) error { - err := cache.TableRowStatsCache.Update(sctx) + useStatsCache, err := e.updateStatsCacheIfNeed(sctx) if err != nil { return err } - checker := privilege.GetPrivilegeManager(sctx) var rows [][]types.Datum @@ -510,8 +569,18 @@ func (e *memtableRetriever) setDataFromTables(sctx sessionctx.Context, schemas [ if loc == nil { loc = time.Local } + extractor, ok := e.extractor.(*plannercore.InfoSchemaTablesExtractor) + if ok && extractor.SkipRequest { + return nil + } for _, schema := range schemas { + if ok && extractor.Filter("table_schema", schema.Name.L) { + continue + } for _, table := range schema.Tables { + if ok && extractor.Filter("table_name", table.Name.L) { + continue + } collation := table.Collate if collation == "" { collation = mysql.DefaultCollationName @@ -530,6 +599,7 @@ func (e *memtableRetriever) setDataFromTables(sctx sessionctx.Context, schemas [ } else if table.TableCacheStatusType == model.TableCacheStatusEnable { createOptions = "cached=on" } + var err error var autoIncID any hasAutoIncID, _ := infoschema.HasAutoIncrementColumn(table) if hasAutoIncID { @@ -538,33 +608,12 @@ func (e *memtableRetriever) setDataFromTables(sctx sessionctx.Context, schemas [ return err } } - - cache := cache.TableRowStatsCache - var rowCount, dataLength, indexLength uint64 - if table.GetPartitionInfo() == nil { - rowCount = cache.GetTableRows(table.ID) - dataLength, indexLength = cache.GetDataAndIndexLength(table, table.ID, rowCount) - } else { - for _, pi := range table.GetPartitionInfo().Definitions { - piRowCnt := cache.GetTableRows(pi.ID) - rowCount += piRowCnt - parDataLen, parIndexLen := cache.GetDataAndIndexLength(table, pi.ID, piRowCnt) - dataLength += parDataLen - indexLength += parIndexLen - } - } - avgRowLength := uint64(0) - if rowCount != 0 { - avgRowLength = dataLength / rowCount - } tableType := "BASE TABLE" if util.IsSystemView(schema.Name.L) { tableType = "SYSTEM VIEW" } if table.IsSequence() { tableType = "SEQUENCE" - // sequence is always 1 row regardless of stats. - rowCount = 1 } if table.HasClusteredIndex() { pkType = "CLUSTERED" @@ -574,6 +623,12 @@ func (e *memtableRetriever) setDataFromTables(sctx sessionctx.Context, schemas [ if table.PlacementPolicyRef != nil { policyName = table.PlacementPolicyRef.Name.O } + + var rowCount, avgRowLength, dataLength, indexLength uint64 + if useStatsCache { + rowCount, avgRowLength, dataLength, indexLength = fetchColumnsFromStatsCache(table) + } + record := types.MakeDatums( infoschema.CatalogVal, // TABLE_CATALOG schema.Name.O, // TABLE_SCHEMA @@ -1467,8 +1522,18 @@ func (e *memtableRetriever) dataForTiDBClusterInfo(ctx sessionctx.Context) error func (e *memtableRetriever) setDataFromKeyColumnUsage(ctx sessionctx.Context, schemas []*model.DBInfo) { checker := privilege.GetPrivilegeManager(ctx) rows := make([][]types.Datum, 0, len(schemas)) // The capacity is not accurate, but it is not a big problem. + extractor, ok := e.extractor.(*plannercore.InfoSchemaTablesExtractor) + if ok && extractor.SkipRequest { + return + } for _, schema := range schemas { + if ok && extractor.Filter("table_schema", schema.Name.L) { + continue + } for _, table := range schema.Tables { + if ok && extractor.Filter("table_name", table.Name.L) { + continue + } if checker != nil && !checker.RequestVerification(ctx.GetSessionVars().ActiveRoles, schema.Name.L, table.Name.L, "", mysql.AllPrivMask) { continue } diff --git a/pkg/planner/core/logical_plan_builder.go b/pkg/planner/core/logical_plan_builder.go index ae4addd85e98d..8259f5bfa33c7 100644 --- a/pkg/planner/core/logical_plan_builder.go +++ b/pkg/planner/core/logical_plan_builder.go @@ -5334,6 +5334,11 @@ func (b *PlanBuilder) buildMemTable(_ context.Context, dbName model.CIStr, table p.Extractor = &TikvRegionPeersExtractor{} case infoschema.TableColumns: p.Extractor = &ColumnsTableExtractor{} + case infoschema.TableTables, + infoschema.TableReferConst, + infoschema.TableKeyColumn, + infoschema.TableStatistics: + p.Extractor = &InfoSchemaTablesExtractor{} case infoschema.TableTiKVRegionStatus: p.Extractor = &TiKVRegionStatusExtractor{tablesID: make([]int64, 0)} } diff --git a/pkg/planner/core/memtable_predicate_extractor.go b/pkg/planner/core/memtable_predicate_extractor.go index 6ea343681928e..406e53c068b78 100644 --- a/pkg/planner/core/memtable_predicate_extractor.go +++ b/pkg/planner/core/memtable_predicate_extractor.go @@ -20,6 +20,7 @@ import ( "math" "regexp" "slices" + "sort" "strconv" "strings" "time" @@ -1676,3 +1677,70 @@ func (e *TiKVRegionStatusExtractor) explainInfo(_ *PhysicalMemTable) string { func (e *TiKVRegionStatusExtractor) GetTablesID() []int64 { return e.tablesID } + +// InfoSchemaTablesExtractor is used to extract infoSchema tables related predicates. +type InfoSchemaTablesExtractor struct { + extractHelper + // SkipRequest means the where clause always false, we don't need to request any component + SkipRequest bool + + colNames []string + ColPredicates map[string]set.StringSet +} + +// Extract implements the MemTablePredicateExtractor Extract interface +func (e *InfoSchemaTablesExtractor) Extract(_ sessionctx.Context, + schema *expression.Schema, + names []*types.FieldName, + predicates []expression.Expression, +) (remained []expression.Expression) { + var resultSet set.StringSet + e.colNames = []string{"table_schema", "table_name"} + e.ColPredicates = make(map[string]set.StringSet) + remained = predicates + for _, colName := range e.colNames { + remained, e.SkipRequest, resultSet = e.extractCol(schema, names, remained, colName, true) + e.ColPredicates[colName] = resultSet + if e.SkipRequest { + break + } + } + return remained +} + +func (e *InfoSchemaTablesExtractor) explainInfo(_ *PhysicalMemTable) string { + if e.SkipRequest { + return "skip_request:true" + } + r := new(bytes.Buffer) + colNames := make([]string, 0, len(e.ColPredicates)) + for colName := range e.ColPredicates { + colNames = append(colNames, colName) + } + sort.Strings(colNames) + for _, colName := range colNames { + if len(e.ColPredicates[colName]) > 0 { + fmt.Fprintf(r, "%s:[%s], ", colName, extractStringFromStringSet(e.ColPredicates[colName])) + } + } + + // remove the last ", " in the message info + s := r.String() + if len(s) > 2 { + return s[:len(s)-2] + } + return s +} + +// Filter use the col predicates to filter records. +func (e *InfoSchemaTablesExtractor) Filter(colName string, val string) bool { + if e.SkipRequest { + return true + } + predVals, ok := e.ColPredicates[colName] + if ok && len(predVals) > 0 { + return !predVals.Exist(val) + } + // No need to filter records since no predicate for the column exists. + return false +} diff --git a/pkg/planner/core/memtable_predicate_extractor_test.go b/pkg/planner/core/memtable_predicate_extractor_test.go index b70e4530eb109..2a8b7698e0fbb 100644 --- a/pkg/planner/core/memtable_predicate_extractor_test.go +++ b/pkg/planner/core/memtable_predicate_extractor_test.go @@ -1810,3 +1810,95 @@ func TestExtractorInPreparedStmt(t *testing.T) { ca.checker(extractor) } } + +func TestInformSchemaTableExtract(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + + se, err := session.CreateSession4Test(store) + require.NoError(t, err) + var cases = []struct { + sql string + skipRequest bool + colPredicates map[string]set.StringSet + }{ + { + sql: `select * from INFORMATION_SCHEMA.TABLES where table_name='T';`, + skipRequest: false, + colPredicates: map[string]set.StringSet{ + "table_name": set.NewStringSet("t"), + "table_schema": set.NewStringSet(), + }, + }, + { + sql: `select * from INFORMATION_SCHEMA.TABLES where table_schema='TS';`, + skipRequest: false, + colPredicates: map[string]set.StringSet{ + "table_name": set.NewStringSet(), + "table_schema": set.NewStringSet("ts"), + }, + }, + { + sql: "select * from information_schema.TABLES where table_name in ('TEST','t') and table_schema in ('A','b')", + skipRequest: false, + colPredicates: map[string]set.StringSet{ + "table_name": set.NewStringSet("test", "t"), + "table_schema": set.NewStringSet("a", "b"), + }, + }, + { + sql: "select * from information_schema.TABLES where table_name ='t' or table_name ='A'", + skipRequest: false, + colPredicates: map[string]set.StringSet{ + "table_name": set.NewStringSet("t", "a"), + "table_schema": set.NewStringSet(), + }, + }, + { + sql: "select * from information_schema.REFERENTIAL_CONSTRAINTS where table_name ='t' or table_name ='A'", + skipRequest: false, + colPredicates: map[string]set.StringSet{ + "table_name": set.NewStringSet("t", "a"), + "table_schema": set.NewStringSet(), + }, + }, + { + sql: "select * from information_schema.KEY_COLUMN_USAGE where table_name ='t' or table_name ='A'", + skipRequest: false, + colPredicates: map[string]set.StringSet{ + "table_name": set.NewStringSet("t", "a"), + "table_schema": set.NewStringSet(), + }, + }, + { + sql: "select * from information_schema.STATISTICS where table_name ='t' or table_name ='A'", + skipRequest: false, + colPredicates: map[string]set.StringSet{ + "table_name": set.NewStringSet("t", "a"), + "table_schema": set.NewStringSet(), + }, + }, + { + sql: "select * from information_schema.STATISTICS where table_name ='t' and table_name ='A'", + skipRequest: true, + colPredicates: map[string]set.StringSet{ + "table_name": set.NewStringSet(), + "table_schema": set.NewStringSet(), + }, + }, + { + sql: "select * from information_schema.STATISTICS where table_name ='t' and table_schema ='A' and table_schema = 'b'", + skipRequest: true, + colPredicates: map[string]set.StringSet{ + "table_schema": set.NewStringSet(), + }, + }, + } + parser := parser.New() + for _, ca := range cases { + logicalMemTable := getLogicalMemTable(t, dom, se, parser, ca.sql) + require.NotNil(t, logicalMemTable.Extractor) + columnsTableExtractor := logicalMemTable.Extractor.(*plannercore.InfoSchemaTablesExtractor) + require.Equal(t, ca.skipRequest, columnsTableExtractor.SkipRequest, "SQL: %v", ca.sql) + require.Equal(t, ca.colPredicates, columnsTableExtractor.ColPredicates, "SQL: %v", ca.sql) + } +} diff --git a/pkg/planner/core/rule_column_pruning.go b/pkg/planner/core/rule_column_pruning.go index 159c075d98951..b6ac53de952c9 100644 --- a/pkg/planner/core/rule_column_pruning.go +++ b/pkg/planner/core/rule_column_pruning.go @@ -396,7 +396,8 @@ func (p *LogicalMemTable) PruneColumns(parentUsedCols []*expression.Column, opt infoschema.ClusterTableTiDBTrx, infoschema.TableDataLockWaits, infoschema.TableDeadlocks, - infoschema.ClusterTableDeadlocks: + infoschema.ClusterTableDeadlocks, + infoschema.TableTables: default: return nil } diff --git a/tests/integrationtest/r/executor/explainfor.result b/tests/integrationtest/r/executor/explainfor.result index b6a8f1d0b2f34..3f63c53021581 100644 --- a/tests/integrationtest/r/executor/explainfor.result +++ b/tests/integrationtest/r/executor/explainfor.result @@ -834,3 +834,61 @@ a a select @@last_plan_from_binding; @@last_plan_from_binding 1 +create database s1; +drop table if exists t; +create table t(a int); +drop table if exists r; +create table r(a int); +desc format='brief' select * from information_schema.tables where table_name='t'; +id estRows task access object operator info +MemTableScan 10000.00 root table:TABLES table_name:["t"] +desc format='brief' select * from information_schema.REFERENTIAL_CONSTRAINTS where table_name='t'; +id estRows task access object operator info +MemTableScan 10000.00 root table:REFERENTIAL_CONSTRAINTS table_name:["t"] +desc format='brief' select * from information_schema.KEY_COLUMN_USAGE where table_name='t'; +id estRows task access object operator info +MemTableScan 10000.00 root table:KEY_COLUMN_USAGE table_name:["t"] +desc format='brief' select * from information_schema.STATISTICS where table_name='t'; +id estRows task access object operator info +MemTableScan 10000.00 root table:STATISTICS table_name:["t"] +desc format='brief' select * from information_schema.STATISTICS where table_name='t' and table_name='r'; +id estRows task access object operator info +MemTableScan 10000.00 root table:STATISTICS skip_request:true +desc format='brief' select * from information_schema.STATISTICS where table_name='t' or table_name='r'; +id estRows task access object operator info +MemTableScan 10000.00 root table:STATISTICS table_name:["r","t"] +desc format='brief' select * from information_schema.tables where table_schema='t'; +id estRows task access object operator info +MemTableScan 10000.00 root table:TABLES table_schema:["t"] +desc format='brief' select * from information_schema.KEY_COLUMN_USAGE where table_schema='s1'; +id estRows task access object operator info +MemTableScan 10000.00 root table:KEY_COLUMN_USAGE table_schema:["s1"] +desc format='brief' select * from information_schema.STATISTICS where table_schema='s1'; +id estRows task access object operator info +MemTableScan 10000.00 root table:STATISTICS table_schema:["s1"] +desc format='brief' select * from information_schema.STATISTICS where table_schema='s1' and table_schema='test'; +id estRows task access object operator info +MemTableScan 10000.00 root table:STATISTICS skip_request:true +desc format='brief' select * from information_schema.STATISTICS where table_name='s1' and table_schema='test'; +id estRows task access object operator info +MemTableScan 10000.00 root table:STATISTICS table_name:["s1"], table_schema:["test"] +desc format='brief' select * from information_schema.STATISTICS where table_schema='s1' or table_schema='test'; +id estRows task access object operator info +MemTableScan 10000.00 root table:STATISTICS table_schema:["s1","test"] +desc format='brief' SELECT column_name FROM information_schema.statistics WHERE index_name = 'idx1' AND table_schema = DATABASE () AND table_name = 't' ORDER BY seq_in_index; +id estRows task access object operator info +Projection 8000.00 root Column#8->Column#19 +└─Sort 8000.00 root Column#7 + └─Selection 8000.00 root eq(Column#6, "idx1") + └─MemTableScan 10000.00 root table:STATISTICS table_name:["t"], table_schema:["plan_cache"] +desc format='brief' SELECT table_comment FROM information_schema.tables WHERE table_schema = DATABASE () AND table_name = 't'; +id estRows task access object operator info +MemTableScan 10000.00 root table:TABLES table_name:["t"], table_schema:["plan_cache"] +desc format='brief' SELECT * FROM information_schema.referential_constraints rc JOIN information_schema.key_column_usage fk USING (constraint_schema, constraint_name) WHERE fk.referenced_column_name != 't' AND fk.table_schema = DATABASE () AND fk.table_name = 't' AND rc.constraint_schema = DATABASE () AND rc.table_name = 't'; +id estRows task access object operator info +Projection 8000.00 root Column#2, Column#3, Column#1, Column#4, Column#5, Column#6, Column#7, Column#8, Column#9, Column#10, Column#11, Column#12, Column#15, Column#16, Column#17, Column#18, Column#19, Column#20, Column#21, Column#22, Column#23 +└─HashJoin 8000.00 root inner join, equal:[eq(Column#3, Column#14)] + ├─Selection(Build) 8000.00 root eq("plan_cache", Column#13), ne(Column#23, "t") + │ └─MemTableScan 10000.00 root table:KEY_COLUMN_USAGE table_name:["t"], table_schema:["plan_cache"] + └─Selection(Probe) 8000.00 root eq(Column#2, "plan_cache") + └─MemTableScan 10000.00 root table:REFERENTIAL_CONSTRAINTS table_name:["t"] diff --git a/tests/integrationtest/r/infoschema/infoschema.result b/tests/integrationtest/r/infoschema/infoschema.result index afae9899d9d66..eec2d44bcebaa 100644 --- a/tests/integrationtest/r/infoschema/infoschema.result +++ b/tests/integrationtest/r/infoschema/infoschema.result @@ -1,6 +1,9 @@ +use infoschema__infoschema; DROP TABLE IF EXISTS `t1`; -create table t1 (c1 VARCHAR(10) NOT NULL COMMENT 'Abcdefghijabcd', c2 INTEGER COMMENT 'aBcdefghijab',c3 INTEGER COMMENT '01234567890', c4 INTEGER, c5 INTEGER, c6 INTEGER, c7 INTEGER, c8 VARCHAR(100), c9 CHAR(50), c10 DATETIME, c11 DATETIME, c12 DATETIME,c13 DATETIME, INDEX i1 (c1) COMMENT 'i1 comment',INDEX i2(c2) ) COMMENT='ABCDEFGHIJabc'; +create table test.t1 (c1 VARCHAR(10) NOT NULL COMMENT 'Abcdefghijabcd', c2 INTEGER COMMENT 'aBcdefghijab',c3 INTEGER COMMENT '01234567890', c4 INTEGER, c5 INTEGER, c6 INTEGER, c7 INTEGER, c8 VARCHAR(100), c9 CHAR(50), c10 DATETIME, c11 DATETIME, c12 DATETIME,c13 DATETIME, INDEX i1 (c1) COMMENT 'i1 comment',INDEX i2(c2) ) COMMENT='ABCDEFGHIJabc'; SELECT index_comment,char_length(index_comment),COLUMN_NAME FROM information_schema.statistics WHERE table_name='t1' and table_schema="infoschema__infoschema" ORDER BY index_comment; +index_comment char_length(index_comment) COLUMN_NAME +SELECT index_comment,char_length(index_comment),COLUMN_NAME FROM information_schema.statistics WHERE table_name='t1' and table_schema="test" ORDER BY index_comment; index_comment char_length(index_comment) COLUMN_NAME 0 c2 i1 comment 10 c1 @@ -37,10 +40,58 @@ insert infoschema__infoschema.t2 values(1,'334'),(4,'3443435'),(5,'fdf43t536653' create table infoschema__infoschema.t3 (id int primary key, a text); insert infoschema__infoschema.t3 values(1,'334'),(4,'3443435'),(5,'fdf43t536653'); rename table infoschema__infoschema.t2 to mysql.t2, infoschema__infoschema.t3 to mysql.t3; -SELECT count(*) FROM information_schema.TABLES WHERE (TABLE_SCHEMA = 'mysql') AND (TABLE_NAME = 't2'); +SELECT count(*) FROM information_schema.TABLES WHERE TABLE_SCHEMA = 'mysql' AND TABLE_NAME = 't2'; count(*) 1 -SELECT count(*) FROM information_schema.TABLES WHERE (TABLE_SCHEMA = 'mysql') AND (TABLE_NAME = 't3'); +SELECT count(*) FROM information_schema.TABLES WHERE TABLE_SCHEMA = 'mysql' AND TABLE_NAME = 't3'; count(*) 1 +SELECT count(*) FROM information_schema.TABLES WHERE TABLE_SCHEMA in ('mysql', 'test') and TABLE_NAME in ('t1', 't2'); +count(*) +3 +SELECT count(*) FROM information_schema.TABLES WHERE (TABLE_SCHEMA= 'mysql' or TABLE_SCHEMA = 'test') and (TABLE_NAME = 't1' or TABLE_NAME = 't2'); +count(*) +3 +desc format='brief' SELECT count(*) FROM information_schema.TABLES WHERE (TABLE_SCHEMA= 'mysql' or TABLE_SCHEMA = 'test') and (TABLE_NAME = 't1' or TABLE_NAME = 't2'); +id estRows task access object operator info +HashAgg 1.00 root funcs:count(1)->Column#26 +└─MemTableScan 10000.00 root table:TABLES table_name:["t1","t2"], table_schema:["mysql","test"] +desc format='brief' SELECT count(*) FROM information_schema.TABLES WHERE TABLE_SCHEMA in ('mysql', 'test') and TABLE_NAME in ('t1', 't2'); +id estRows task access object operator info +HashAgg 1.00 root funcs:count(1)->Column#26 +└─MemTableScan 10000.00 root table:TABLES table_name:["t1","t2"], table_schema:["mysql","test"] +SELECT count(*) FROM information_schema.TABLES WHERE TABLE_NAME in ('t1', 't2') and TABLE_SCHEMA = 'mysql'; +count(*) +2 +SELECT count(*) FROM information_schema.TABLES WHERE (TABLE_NAME = 't1' or TABLE_NAME = 't2') and TABLE_SCHEMA = 'mysql'; +count(*) +2 drop table mysql.t1, mysql.t2, mysql.t3; +create table infoschema__infoschema.t4(a int, INDEX i1 (a)); +create table infoschema__infoschema.t5(a int, INDEX i1 (a)); +insert into infoschema__infoschema.t4 values(1); +insert into infoschema__infoschema.t5 values(1); +SELECT count(*) FROM information_schema.STATISTICS WHERE TABLE_SCHEMA = 'infoschema__infoschema' AND TABLE_NAME = 't4'; +count(*) +1 +SELECT count(*) FROM information_schema.STATISTICS WHERE TABLE_SCHEMA = 'infoschema__infoschema' AND TABLE_NAME != 't4'; +count(*) +1 +analyze table infoschema__infoschema.t4; +analyze table infoschema__infoschema.t5; +select table_type from information_schema.tables where (table_name='t4' or table_name = 't5') and table_schema = 'infoschema__infoschema'; +table_type +BASE TABLE +BASE TABLE +select table_type, table_rows from information_schema.tables where (table_name ='t4' or table_name = 't5') and table_schema = 'infoschema__infoschema'; +table_type table_rows +BASE TABLE 1 +BASE TABLE 1 +select table_type, DATA_LENGTH from information_schema.tables where (table_name ='t4' or table_name = 't5') and table_schema = 'infoschema__infoschema'; +table_type DATA_LENGTH +BASE TABLE 8 +BASE TABLE 8 +select engine, DATA_LENGTH from information_schema.tables where (table_name ='t4' or table_name = 't5') and table_schema = 'infoschema__infoschema'; +engine DATA_LENGTH +InnoDB 8 +InnoDB 8 diff --git a/tests/integrationtest/t/executor/explainfor.test b/tests/integrationtest/t/executor/explainfor.test index 454976647e409..af4dc3de7db2f 100644 --- a/tests/integrationtest/t/executor/explainfor.test +++ b/tests/integrationtest/t/executor/explainfor.test @@ -430,3 +430,25 @@ select @@last_plan_from_cache; execute stmt_join; select @@last_plan_from_binding; + +# TestExplainMemTablePredicatePushDown +create database s1; +drop table if exists t; +create table t(a int); +drop table if exists r; +create table r(a int); +desc format='brief' select * from information_schema.tables where table_name='t'; +desc format='brief' select * from information_schema.REFERENTIAL_CONSTRAINTS where table_name='t'; +desc format='brief' select * from information_schema.KEY_COLUMN_USAGE where table_name='t'; +desc format='brief' select * from information_schema.STATISTICS where table_name='t'; +desc format='brief' select * from information_schema.STATISTICS where table_name='t' and table_name='r'; +desc format='brief' select * from information_schema.STATISTICS where table_name='t' or table_name='r'; +desc format='brief' select * from information_schema.tables where table_schema='t'; +desc format='brief' select * from information_schema.KEY_COLUMN_USAGE where table_schema='s1'; +desc format='brief' select * from information_schema.STATISTICS where table_schema='s1'; +desc format='brief' select * from information_schema.STATISTICS where table_schema='s1' and table_schema='test'; +desc format='brief' select * from information_schema.STATISTICS where table_name='s1' and table_schema='test'; +desc format='brief' select * from information_schema.STATISTICS where table_schema='s1' or table_schema='test'; +desc format='brief' SELECT column_name FROM information_schema.statistics WHERE index_name = 'idx1' AND table_schema = DATABASE () AND table_name = 't' ORDER BY seq_in_index; +desc format='brief' SELECT table_comment FROM information_schema.tables WHERE table_schema = DATABASE () AND table_name = 't'; +desc format='brief' SELECT * FROM information_schema.referential_constraints rc JOIN information_schema.key_column_usage fk USING (constraint_schema, constraint_name) WHERE fk.referenced_column_name != 't' AND fk.table_schema = DATABASE () AND fk.table_name = 't' AND rc.constraint_schema = DATABASE () AND rc.table_name = 't'; diff --git a/tests/integrationtest/t/infoschema/infoschema.test b/tests/integrationtest/t/infoschema/infoschema.test index 79d7d913cdfab..2676fcc052d0b 100644 --- a/tests/integrationtest/t/infoschema/infoschema.test +++ b/tests/integrationtest/t/infoschema/infoschema.test @@ -1,7 +1,9 @@ # TestIndexComment +use infoschema__infoschema; DROP TABLE IF EXISTS `t1`; -create table t1 (c1 VARCHAR(10) NOT NULL COMMENT 'Abcdefghijabcd', c2 INTEGER COMMENT 'aBcdefghijab',c3 INTEGER COMMENT '01234567890', c4 INTEGER, c5 INTEGER, c6 INTEGER, c7 INTEGER, c8 VARCHAR(100), c9 CHAR(50), c10 DATETIME, c11 DATETIME, c12 DATETIME,c13 DATETIME, INDEX i1 (c1) COMMENT 'i1 comment',INDEX i2(c2) ) COMMENT='ABCDEFGHIJabc'; +create table test.t1 (c1 VARCHAR(10) NOT NULL COMMENT 'Abcdefghijabcd', c2 INTEGER COMMENT 'aBcdefghijab',c3 INTEGER COMMENT '01234567890', c4 INTEGER, c5 INTEGER, c6 INTEGER, c7 INTEGER, c8 VARCHAR(100), c9 CHAR(50), c10 DATETIME, c11 DATETIME, c12 DATETIME,c13 DATETIME, INDEX i1 (c1) COMMENT 'i1 comment',INDEX i2(c2) ) COMMENT='ABCDEFGHIJabc'; SELECT index_comment,char_length(index_comment),COLUMN_NAME FROM information_schema.statistics WHERE table_name='t1' and table_schema="infoschema__infoschema" ORDER BY index_comment; +SELECT index_comment,char_length(index_comment),COLUMN_NAME FROM information_schema.statistics WHERE table_name='t1' and table_schema="test" ORDER BY index_comment; # TestIssue42440 show create table information_schema.ddl_jobs; @@ -22,6 +24,26 @@ insert infoschema__infoschema.t2 values(1,'334'),(4,'3443435'),(5,'fdf43t536653' create table infoschema__infoschema.t3 (id int primary key, a text); insert infoschema__infoschema.t3 values(1,'334'),(4,'3443435'),(5,'fdf43t536653'); rename table infoschema__infoschema.t2 to mysql.t2, infoschema__infoschema.t3 to mysql.t3; -SELECT count(*) FROM information_schema.TABLES WHERE (TABLE_SCHEMA = 'mysql') AND (TABLE_NAME = 't2'); -SELECT count(*) FROM information_schema.TABLES WHERE (TABLE_SCHEMA = 'mysql') AND (TABLE_NAME = 't3'); +SELECT count(*) FROM information_schema.TABLES WHERE TABLE_SCHEMA = 'mysql' AND TABLE_NAME = 't2'; +SELECT count(*) FROM information_schema.TABLES WHERE TABLE_SCHEMA = 'mysql' AND TABLE_NAME = 't3'; +SELECT count(*) FROM information_schema.TABLES WHERE TABLE_SCHEMA in ('mysql', 'test') and TABLE_NAME in ('t1', 't2'); +SELECT count(*) FROM information_schema.TABLES WHERE (TABLE_SCHEMA= 'mysql' or TABLE_SCHEMA = 'test') and (TABLE_NAME = 't1' or TABLE_NAME = 't2'); +desc format='brief' SELECT count(*) FROM information_schema.TABLES WHERE (TABLE_SCHEMA= 'mysql' or TABLE_SCHEMA = 'test') and (TABLE_NAME = 't1' or TABLE_NAME = 't2'); +desc format='brief' SELECT count(*) FROM information_schema.TABLES WHERE TABLE_SCHEMA in ('mysql', 'test') and TABLE_NAME in ('t1', 't2'); +SELECT count(*) FROM information_schema.TABLES WHERE TABLE_NAME in ('t1', 't2') and TABLE_SCHEMA = 'mysql'; +SELECT count(*) FROM information_schema.TABLES WHERE (TABLE_NAME = 't1' or TABLE_NAME = 't2') and TABLE_SCHEMA = 'mysql'; drop table mysql.t1, mysql.t2, mysql.t3; + +# TestTablesColumn +create table infoschema__infoschema.t4(a int, INDEX i1 (a)); +create table infoschema__infoschema.t5(a int, INDEX i1 (a)); +insert into infoschema__infoschema.t4 values(1); +insert into infoschema__infoschema.t5 values(1); +SELECT count(*) FROM information_schema.STATISTICS WHERE TABLE_SCHEMA = 'infoschema__infoschema' AND TABLE_NAME = 't4'; +SELECT count(*) FROM information_schema.STATISTICS WHERE TABLE_SCHEMA = 'infoschema__infoschema' AND TABLE_NAME != 't4'; +analyze table infoschema__infoschema.t4; +analyze table infoschema__infoschema.t5; +select table_type from information_schema.tables where (table_name='t4' or table_name = 't5') and table_schema = 'infoschema__infoschema'; +select table_type, table_rows from information_schema.tables where (table_name ='t4' or table_name = 't5') and table_schema = 'infoschema__infoschema'; +select table_type, DATA_LENGTH from information_schema.tables where (table_name ='t4' or table_name = 't5') and table_schema = 'infoschema__infoschema'; +select engine, DATA_LENGTH from information_schema.tables where (table_name ='t4' or table_name = 't5') and table_schema = 'infoschema__infoschema'; \ No newline at end of file