diff --git a/pkg/executor/builder.go b/pkg/executor/builder.go index bf6de8dfa8905..39141e0007092 100644 --- a/pkg/executor/builder.go +++ b/pkg/executor/builder.go @@ -2332,7 +2332,7 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) exec.Ex retriever: &hugeMemTableRetriever{ table: v.Table, columns: v.Columns, - extractor: v.Extractor.(*plannercore.ColumnsTableExtractor), + extractor: v.Extractor.(*plannercore.InfoSchemaColumnsExtractor), viewSchemaMap: make(map[int64]*expression.Schema), viewOutputNamesMap: make(map[int64]types.NameSlice), }, diff --git a/pkg/executor/infoschema_reader.go b/pkg/executor/infoschema_reader.go index 2ff025aa49d74..d85d30c5a4856 100644 --- a/pkg/executor/infoschema_reader.go +++ b/pkg/executor/infoschema_reader.go @@ -219,11 +219,10 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex case infoschema.TableKeywords: err = e.setDataFromKeywords() case infoschema.TableTiDBIndexUsage: - dbs := getAllSchemas() - err = e.setDataFromIndexUsage(ctx, sctx, dbs) + err = e.setDataFromIndexUsage(ctx, sctx) case infoschema.ClusterTableTiDBIndexUsage: dbs := getAllSchemas() - err = e.setDataForClusterIndexUsage(ctx, sctx, dbs) + err = e.setDataFromClusterIndexUsage(ctx, sctx, dbs) } if err != nil { return nil, err @@ -829,13 +828,13 @@ func (e *memtableRetriever) setDataFromTiDBCheckConstraints(ctx context.Context, type hugeMemTableRetriever struct { dummyCloser - extractor *plannercore.ColumnsTableExtractor + extractor *plannercore.InfoSchemaColumnsExtractor table *model.TableInfo columns []*model.ColumnInfo retrieved bool initialized bool rows [][]types.Datum - dbs []*model.DBInfo + dbs []model.CIStr curTables []*model.TableInfo dbsIdx int tblIdx int @@ -848,15 +847,16 @@ type hugeMemTableRetriever struct { // retrieve implements the infoschemaRetriever interface func (e *hugeMemTableRetriever) retrieve(ctx context.Context, sctx sessionctx.Context) ([][]types.Datum, error) { + if e.extractor.SkipRequest { + e.retrieved = true + } if e.retrieved { return nil, nil } if !e.initialized { e.is = sessiontxn.GetTxnManager(sctx).GetTxnInfoSchema() - dbs := e.is.AllSchemas() - slices.SortFunc(dbs, model.LessDBInfo) - e.dbs = dbs + e.dbs = e.extractor.ListSchemas(e.is) e.initialized = true e.rows = make([][]types.Datum, 0, 1024) e.batch = 1024 @@ -864,7 +864,7 @@ func (e *hugeMemTableRetriever) retrieve(ctx context.Context, sctx sessionctx.Co var err error if e.table.Name.O == infoschema.TableColumns { - err = e.setDataForColumns(ctx, sctx, e.extractor) + err = e.setDataForColumns(ctx, sctx) } if err != nil { return nil, err @@ -874,14 +874,14 @@ func (e *hugeMemTableRetriever) retrieve(ctx context.Context, sctx sessionctx.Co return adjustColumns(e.rows, e.columns, e.table), nil } -func (e *hugeMemTableRetriever) setDataForColumns(ctx context.Context, sctx sessionctx.Context, extractor *plannercore.ColumnsTableExtractor) error { +func (e *hugeMemTableRetriever) setDataForColumns(ctx context.Context, sctx sessionctx.Context) error { checker := privilege.GetPrivilegeManager(sctx) e.rows = e.rows[:0] for ; e.dbsIdx < len(e.dbs); e.dbsIdx++ { schema := e.dbs[e.dbsIdx] var table *model.TableInfo if len(e.curTables) == 0 { - tables, err := e.is.SchemaTableInfos(ctx, schema.Name) + tables, err := e.extractor.ListTables(ctx, schema, e.is) if err != nil { return errors.Trace(err) } @@ -890,7 +890,7 @@ func (e *hugeMemTableRetriever) setDataForColumns(ctx context.Context, sctx sess for e.tblIdx < len(e.curTables) { table = e.curTables[e.tblIdx] e.tblIdx++ - if e.setDataForColumnsWithOneTable(ctx, sctx, extractor, schema, table, checker) { + if e.setDataForColumnsWithOneTable(ctx, sctx, schema, table, checker) { return nil } } @@ -903,15 +903,14 @@ func (e *hugeMemTableRetriever) setDataForColumns(ctx context.Context, sctx sess func (e *hugeMemTableRetriever) setDataForColumnsWithOneTable( ctx context.Context, sctx sessionctx.Context, - extractor *plannercore.ColumnsTableExtractor, - schema *model.DBInfo, + schema model.CIStr, table *model.TableInfo, checker privilege.Manager) bool { hasPrivs := false var priv mysql.PrivilegeType if checker != nil { for _, p := range mysql.AllColumnPrivs { - if checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.Name.L, table.Name.L, "", p) { + if checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.L, table.Name.L, "", p) { hasPrivs = true priv |= p } @@ -921,11 +920,16 @@ func (e *hugeMemTableRetriever) setDataForColumnsWithOneTable( } } - e.dataForColumnsInTable(ctx, sctx, schema, table, priv, extractor) + e.dataForColumnsInTable(ctx, sctx, schema, table, priv) return len(e.rows) >= e.batch } -func (e *hugeMemTableRetriever) dataForColumnsInTable(ctx context.Context, sctx sessionctx.Context, schema *model.DBInfo, tbl *model.TableInfo, priv mysql.PrivilegeType, extractor *plannercore.ColumnsTableExtractor) { +func (e *hugeMemTableRetriever) dataForColumnsInTable( + ctx context.Context, + sctx sessionctx.Context, + schema model.CIStr, + tbl *model.TableInfo, + priv mysql.PrivilegeType) { if tbl.IsView() { e.viewMu.Lock() _, ok := e.viewSchemaMap[tbl.ID] @@ -937,7 +941,7 @@ func (e *hugeMemTableRetriever) dataForColumnsInTable(ctx context.Context, sctx is := sessiontxn.GetTxnManager(s).GetTxnInfoSchema() planBuilder, _ := plannercore.NewPlanBuilder().Init(s.GetPlanCtx(), is, hint.NewQBHintHandler(nil)) var err error - viewLogicalPlan, err = planBuilder.BuildDataSourceFromView(ctx, schema.Name, tbl, nil, nil) + viewLogicalPlan, err = planBuilder.BuildDataSourceFromView(ctx, schema, tbl, nil, nil) return errors.Trace(err) }); err != nil { sctx.GetSessionVars().StmtCtx.AppendWarning(err) @@ -950,42 +954,8 @@ func (e *hugeMemTableRetriever) dataForColumnsInTable(ctx context.Context, sctx e.viewMu.Unlock() } - var tableSchemaRegexp, tableNameRegexp, columnsRegexp []collate.WildcardPattern - var tableSchemaFilterEnable, - tableNameFilterEnable, columnsFilterEnable bool - if !extractor.SkipRequest { - tableSchemaFilterEnable = extractor.TableSchema.Count() > 0 - tableNameFilterEnable = extractor.TableName.Count() > 0 - columnsFilterEnable = extractor.ColumnName.Count() > 0 - if len(extractor.TableSchemaPatterns) > 0 { - tableSchemaRegexp = make([]collate.WildcardPattern, len(extractor.TableSchemaPatterns)) - for i, pattern := range extractor.TableSchemaPatterns { - tableSchemaRegexp[i] = collate.GetCollatorByID(collate.CollationName2ID(mysql.UTF8MB4DefaultCollation)).Pattern() - tableSchemaRegexp[i].Compile(pattern, byte('\\')) - } - } - if len(extractor.TableNamePatterns) > 0 { - tableNameRegexp = make([]collate.WildcardPattern, len(extractor.TableNamePatterns)) - for i, pattern := range extractor.TableNamePatterns { - tableNameRegexp[i] = collate.GetCollatorByID(collate.CollationName2ID(mysql.UTF8MB4DefaultCollation)).Pattern() - tableNameRegexp[i].Compile(pattern, byte('\\')) - } - } - if len(extractor.ColumnNamePatterns) > 0 { - columnsRegexp = make([]collate.WildcardPattern, len(extractor.ColumnNamePatterns)) - for i, pattern := range extractor.ColumnNamePatterns { - columnsRegexp[i] = collate.GetCollatorByID(collate.CollationName2ID(mysql.UTF8MB4DefaultCollation)).Pattern() - columnsRegexp[i].Compile(pattern, byte('\\')) - } - } - } - i := 0 -ForColumnsTag: - for _, col := range tbl.Columns { - if col.Hidden { - continue - } - i++ + cols, ordinalPos := e.extractor.ListColumns(tbl) + for i, col := range cols { ft := &(col.FieldType) if tbl.IsView() { e.viewMu.RLock() @@ -999,32 +969,6 @@ ForColumnsTag: } e.viewMu.RUnlock() } - if !extractor.SkipRequest { - if tableSchemaFilterEnable && !extractor.TableSchema.Exist(schema.Name.L) { - continue - } - if tableNameFilterEnable && !extractor.TableName.Exist(tbl.Name.L) { - continue - } - if columnsFilterEnable && !extractor.ColumnName.Exist(col.Name.L) { - continue - } - for _, re := range tableSchemaRegexp { - if !re.DoMatch(schema.Name.L) { - continue ForColumnsTag - } - } - for _, re := range tableNameRegexp { - if !re.DoMatch(tbl.Name.L) { - continue ForColumnsTag - } - } - for _, re := range columnsRegexp { - if !re.DoMatch(col.Name.L) { - continue ForColumnsTag - } - } - } var charMaxLen, charOctLen, numericPrecision, numericScale, datetimePrecision any colLen, decimal := ft.GetFlen(), ft.GetDecimal() @@ -1101,10 +1045,10 @@ ForColumnsTag: } record := types.MakeDatums( infoschema.CatalogVal, // TABLE_CATALOG - schema.Name.O, // TABLE_SCHEMA + schema.O, // TABLE_SCHEMA tbl.Name.O, // TABLE_NAME col.Name.O, // COLUMN_NAME - i, // ORDINAL_POSITION + ordinalPos[i], // ORDINAL_POSITION columnDefault, // COLUMN_DEFAULT columnDesc.Null, // IS_NULLABLE types.TypeToStr(colType, ft.GetCharset()), // DATA_TYPE @@ -3743,7 +3687,7 @@ func (e *memtableRetriever) setDataFromKeywords() error { return nil } -func (e *memtableRetriever) setDataFromIndexUsage(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) error { +func (e *memtableRetriever) setDataForClusterIndexUsage(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) error { dom := domain.GetDomain(sctx) rows := make([][]types.Datum, 0, 100) checker := privilege.GetPrivilegeManager(sctx) @@ -3802,8 +3746,64 @@ func (e *memtableRetriever) setDataFromIndexUsage(ctx context.Context, sctx sess return nil } -func (e *memtableRetriever) setDataForClusterIndexUsage(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) error { - err := e.setDataFromIndexUsage(ctx, sctx, schemas) +func (e *memtableRetriever) setDataFromIndexUsage(ctx context.Context, sctx sessionctx.Context) error { + dom := domain.GetDomain(sctx) + rows := make([][]types.Datum, 0, 100) + checker := privilege.GetPrivilegeManager(sctx) + extractor, ok := e.extractor.(*plannercore.InfoSchemaIndexUsageExtractor) + if !ok { + return errors.Errorf("wrong extractor type: %T, expected InfoSchemaIndexUsageExtractor", e.extractor) + } + if extractor.SkipRequest { + return nil + } + + schemas := extractor.ListSchemas(e.is) + for _, schema := range schemas { + tbls, err := extractor.ListTables(ctx, schema, e.is) + if err != nil { + return errors.Trace(err) + } + + for _, tbl := range tbls { + if checker != nil && !checker.RequestVerification( + sctx.GetSessionVars().ActiveRoles, + schema.L, tbl.Name.L, "", mysql.AllPrivMask) { + continue + } + + idxs := extractor.ListIndexes(tbl) + for _, idx := range idxs { + row := make([]types.Datum, 0, 14) + usage := dom.StatsHandle().GetIndexUsage(tbl.ID, idx.ID) + row = append(row, types.NewStringDatum(schema.O)) + row = append(row, types.NewStringDatum(tbl.Name.O)) + row = append(row, types.NewStringDatum(idx.Name.O)) + row = append(row, types.NewIntDatum(int64(usage.QueryTotal))) + row = append(row, types.NewIntDatum(int64(usage.KvReqTotal))) + row = append(row, types.NewIntDatum(int64(usage.RowAccessTotal))) + for _, percentage := range usage.PercentageAccess { + row = append(row, types.NewIntDatum(int64(percentage))) + } + lastUsedAt := types.Datum{} + lastUsedAt.SetNull() + if !usage.LastUsedAt.IsZero() { + t := types.NewTime(types.FromGoTime(usage.LastUsedAt), mysql.TypeTimestamp, 0) + lastUsedAt = types.NewTimeDatum(t) + } + row = append(row, lastUsedAt) + rows = append(rows, row) + } + } + } + + e.rows = rows + return nil +} + +// Currently, ClusterIndexUsage will not be be pushed down, so just use the previous logic +func (e *memtableRetriever) setDataFromClusterIndexUsage(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) error { + err := e.setDataForClusterIndexUsage(ctx, sctx, schemas) if err != nil { return errors.Trace(err) } diff --git a/pkg/executor/infoschema_reader_test.go b/pkg/executor/infoschema_reader_test.go index 88e013b4c4777..6674eb9b02177 100644 --- a/pkg/executor/infoschema_reader_test.go +++ b/pkg/executor/infoschema_reader_test.go @@ -489,6 +489,124 @@ func TestTiFlashSystemTableWithTiFlashV640(t *testing.T) { tk.MustQuery("show warnings").Check(testkit.Rows()) } +func TestColumnTable(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table tbl1(col_1 int primary key, col_2 int, col_4 int);") + tk.MustExec("create table tbl2(col_1 int primary key, col_2 int, col_3 int);") + tk.MustExec("create view view1 as select min(col_1), col_2, max(col_4) as max4 from tbl1 group by col_2;") + + tk.MustQuery("select TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME from information_schema.columns where TABLE_SCHEMA = 'test';").Check( + testkit.RowsWithSep("|", + "test|tbl1|col_1", + "test|tbl1|col_2", + "test|tbl1|col_4", + "test|tbl2|col_1", + "test|tbl2|col_2", + "test|tbl2|col_3", + "test|view1|min(col_1)", + "test|view1|col_2", + "test|view1|max4")) + tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME from information_schema.columns + where TABLE_NAME = 'view1' or TABLE_NAME = 'tbl1'`).Check( + testkit.RowsWithSep("|", + "test|tbl1|col_1", + "test|tbl1|col_2", + "test|tbl1|col_4", + "test|view1|min(col_1)", + "test|view1|col_2", + "test|view1|max4")) + tk.MustQuery("select TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME from information_schema.columns where COLUMN_NAME = \"col_2\";").Check( + testkit.RowsWithSep("|", + "test|tbl1|col_2", + "test|tbl2|col_2", + "test|view1|col_2")) + tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME from information_schema.columns + where TABLE_SCHEMA = 'test' and TABLE_NAME = 'tbl2';`).Check( + testkit.RowsWithSep("|", + "test|tbl2|col_1", + "test|tbl2|col_2", + "test|tbl2|col_3")) + tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME from information_schema.columns + where TABLE_SCHEMA = 'test' and COLUMN_NAME = 'col_4'`).Check( + testkit.RowsWithSep("|", + "test|tbl1|col_4")) + tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME from information_schema.columns + where TABLE_NAME = 'view1' and COLUMN_NAME like 'm%%';`).Check( + testkit.RowsWithSep("|", + "test|view1|min(col_1)", + "test|view1|max4")) + tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME from information_schema.columns + where TABLE_SCHEMA = 'test' and TABLE_NAME = 'tbl1' and COLUMN_NAME = 'col_2';`).Check( + testkit.RowsWithSep("|", + "test|tbl1|col_2")) + tk.MustQuery(`select count(*) from information_schema.columns;`).Check( + testkit.RowsWithSep("|", "4923")) +} + +func TestIndexUsageTable(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table idt1(col_1 int primary key, col_2 int, index idx_1(col_1), index idx_2(col_2), index idx_3(col_1, col_2));") + tk.MustExec("create table idt2(col_1 int primary key, col_2 int, index idx_1(col_1), index idx_2(col_2), index idx_4(col_2, col_1));") + + tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage + where TABLE_SCHEMA = 'test';`).Check( + testkit.RowsWithSep("|", + "test|idt1|idx_1", + "test|idt1|idx_2", + "test|idt1|idx_3", + "test|idt2|idx_1", + "test|idt2|idx_2", + "test|idt2|idx_4")) + tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_NAME = 'idt1'`).Check( + testkit.RowsWithSep("|", + "test|idt1|idx_1", + "test|idt1|idx_2", + "test|idt1|idx_3")) + tk.MustQuery("select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where INDEX_NAME = 'IDX_3'").Check( + testkit.RowsWithSep("|", + "test|idt1|idx_3")) + tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage + where TABLE_SCHEMA = 'test' and TABLE_NAME = 'idt1';`).Check( + testkit.RowsWithSep("|", + "test|idt1|idx_1", + "test|idt1|idx_2", + "test|idt1|idx_3")) + tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage + where TABLE_SCHEMA = 'test' and INDEX_NAME = 'idx_2';`).Check( + testkit.RowsWithSep("|", + "test|idt1|idx_2", + "test|idt2|idx_2")) + tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage + where TABLE_NAME = 'idt1' and INDEX_NAME = 'idx_1';`).Check( + testkit.RowsWithSep("|", + "test|idt1|idx_1")) + tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage + where TABLE_SCHEMA = 'test' and TABLE_NAME = 'idt2' and INDEX_NAME = 'idx_4';`).Check( + testkit.RowsWithSep("|", + "test|idt2|idx_4")) + tk.MustQuery(`select count(*) from information_schema.tidb_index_usage;`).Check( + testkit.RowsWithSep("|", "72")) + + tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage + where TABLE_SCHEMA = 'test1';`).Check(testkit.Rows()) + tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage + where TABLE_NAME = 'idt3';`).Check(testkit.Rows()) + tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage + where INDEX_NAME = 'IDX_5';`).Check(testkit.Rows()) + tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage + where TABLE_SCHEMA = 'test' and TABLE_NAME = 'idt0';`).Check(testkit.Rows()) + tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage + where TABLE_SCHEMA = 'test1' and INDEX_NAME = 'idx_2';`).Check(testkit.Rows()) + tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage + where TABLE_NAME = 'idt2' and INDEX_NAME = 'idx_3';`).Check(testkit.Rows()) + tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage + where TABLE_SCHEMA = 'test' and TABLE_NAME = 'idt1' and INDEX_NAME = 'idx_4';`).Check(testkit.Rows()) +} + // https://github.com/pingcap/tidb/issues/32459. func TestJoinSystemTableContainsView(t *testing.T) { store := testkit.CreateMockStore(t) diff --git a/pkg/planner/core/logical_plan_builder.go b/pkg/planner/core/logical_plan_builder.go index cf1a04448419b..508abc5628732 100644 --- a/pkg/planner/core/logical_plan_builder.go +++ b/pkg/planner/core/logical_plan_builder.go @@ -4814,7 +4814,9 @@ func (b *PlanBuilder) buildMemTable(_ context.Context, dbName model.CIStr, table case infoschema.TableTiKVRegionPeers: p.Extractor = &TikvRegionPeersExtractor{} case infoschema.TableColumns: - p.Extractor = &ColumnsTableExtractor{} + ex := &InfoSchemaColumnsExtractor{} + ex.initExtractableColNames(upTbl) + p.Extractor = ex case infoschema.TableTables: ex := &InfoSchemaTablesExtractor{} ex.initExtractableColNames(upTbl) @@ -4831,11 +4833,14 @@ func (b *PlanBuilder) buildMemTable(_ context.Context, dbName model.CIStr, table ex := &InfoSchemaSchemataExtractor{} ex.initExtractableColNames(upTbl) p.Extractor = ex + case infoschema.TableTiDBIndexUsage: + ex := &InfoSchemaIndexUsageExtractor{} + ex.initExtractableColNames(upTbl) + p.Extractor = ex case infoschema.TableReferConst, infoschema.TableSequences, infoschema.TableCheckConstraints, - infoschema.TableTiDBCheckConstraints, - infoschema.TableTiDBIndexUsage: + infoschema.TableTiDBCheckConstraints: ex := &InfoSchemaBaseExtractor{} ex.initExtractableColNames(upTbl) p.Extractor = ex diff --git a/pkg/planner/core/memtable_infoschema_extractor.go b/pkg/planner/core/memtable_infoschema_extractor.go index 0d7dde7a3459c..4583bcf42b624 100644 --- a/pkg/planner/core/memtable_infoschema_extractor.go +++ b/pkg/planner/core/memtable_infoschema_extractor.go @@ -27,9 +27,11 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/types" + "github.com/pingcap/tidb/pkg/util/collate" "github.com/pingcap/tidb/pkg/util/set" "golang.org/x/exp/maps" ) @@ -44,6 +46,7 @@ const ( _schemaName = "schema_name" _constraintSchema = "constraint_schema" _constraintName = "constraint_name" + _columnName = "column_name" ) var extractableColumns = map[string][]string{ @@ -64,6 +67,18 @@ var extractableColumns = map[string][]string{ _tableSchema, _tableName, _indexName, }, + // See infoschema.columns for full columns. + // Used by InfoSchemaColumnsExtractor and setDataFromColumns. + infoschema.TableColumns: { + _tableSchema, _tableName, + _columnName, + }, + // See infoschema.tidb_index_usage for full columns. + // Used by InfoSchemaIndexesExtractor and setDataFromIndexUsage. + infoschema.TableTiDBIndexUsage: { + _tableSchema, _tableName, + _indexName, + }, // See infoschema.schemataCols for full columns. // Used by InfoSchemaSchemataExtractor and setDataFromSchemata. infoschema.TableSchemata: { @@ -614,3 +629,299 @@ func (e *InfoSchemaBaseExtractor) getSchemaObjectNames(colName string) []model.C } return nil } + +// InfoSchemaTableNameExtractor is a base struct to list matching schemas and tables in predicates, +// so there is no need to call `Filter` for returns from `ListSchemas` and `ListTables`. +// But for other columns, Subclass **must** reimplement `Filter` method to use like operators for filtering. +// Currently, table_id is not taken into consideration. +type InfoSchemaTableNameExtractor struct { + InfoSchemaSchemataExtractor + + listTableFunc func( + ctx context.Context, + s model.CIStr, + is infoschema.InfoSchema, + ) ([]*model.TableInfo, error) + + // table names from predicate, used by `ListTables` + tableNames []model.CIStr + + // all predicates in lower case + colsPredLower map[string]set.StringSet + + // all built regexp in predicates + colsRegexp map[string][]collate.WildcardPattern + + // used for EXPLAIN only + LikePatterns map[string][]string +} + +// Extract all names and like operators in predicates +func (e *InfoSchemaTableNameExtractor) Extract( + ctx base.PlanContext, + schema *expression.Schema, + names []*types.FieldName, + predicates []expression.Expression, +) []expression.Expression { + remained := e.InfoSchemaBaseExtractor.Extract(ctx, schema, names, predicates) + if e.SkipRequest { + return remained + } + + e.LikePatterns = make(map[string][]string, len(e.colNames)) + e.colsRegexp = make(map[string][]collate.WildcardPattern, len(e.colNames)) + e.colsPredLower = make(map[string]set.StringSet, len(e.colNames)) + var likePatterns []string + for _, colName := range e.colNames { + remained, likePatterns = e.extractLikePatternCol(ctx, schema, names, remained, colName, true, false) + regexp := make([]collate.WildcardPattern, len(likePatterns)) + predColLower := set.StringSet{} + for i, pattern := range likePatterns { + regexp[i] = collate.GetCollatorByID(collate.CollationName2ID(mysql.UTF8MB4DefaultCollation)).Pattern() + regexp[i].Compile(pattern, byte('\\')) + } + if vals, ok := e.ColPredicates[colName]; ok { + vals.IterateWith(func(n string) { + predColLower.Insert(strings.ToLower(n)) + }) + } + e.colsPredLower[colName] = predColLower + e.LikePatterns[colName] = likePatterns + e.colsRegexp[colName] = regexp + } + + return remained +} + +// getPredicates gets all names and regexps related to given column names. +func (e *InfoSchemaTableNameExtractor) getPredicates(colNames ...string) ( + set.StringSet, []collate.WildcardPattern, bool) { + filters := set.StringSet{} + regexp := []collate.WildcardPattern{} + hasPredicates := false + + // Extract all filters and like patterns + for _, col := range colNames { + if rs, ok := e.colsRegexp[col]; ok && len(rs) > 0 { + regexp = append(regexp, rs...) + } + if f, ok := e.colsPredLower[col]; ok && len(f) > 0 { + if !hasPredicates { + filters = f + hasPredicates = true + } else { + filters = filters.Intersection(f) + } + } + } + + return filters, regexp, hasPredicates +} + +// Get all predicates related to schema extraction. +// Add more columns if necessary. +func (e *InfoSchemaTableNameExtractor) getSchemaNames() ( + set.StringSet, []collate.WildcardPattern, bool) { + return e.getPredicates(_tableSchema, _schemaName, _constraintSchema) +} + +// ListSchemas lists related schemas from predicates. +// Returned schemas is examined by like operators, so there is no need to call Filter again. +func (e *InfoSchemaTableNameExtractor) ListSchemas( + is infoschema.InfoSchema, +) []model.CIStr { + schemaFilters, schemaRegexp, hasPredicates := e.getSchemaNames() + + // Get all schema names + var schemas []model.CIStr + if hasPredicates { + schemas = make([]model.CIStr, 0, len(schemaFilters)) + schemaFilters.IterateWith(func(n string) { + s := model.CIStr{O: n, L: n} + if n, ok := is.SchemaByName(s); ok { + schemas = append(schemas, n.Name) + } + }) + } else { + schemas = is.AllSchemaNames() + } + slices.SortFunc(schemas, func(a, b model.CIStr) int { + return strings.Compare(a.L, b.L) + }) + + // Filter with regexp + filteredSchemas := make([]model.CIStr, 0, len(schemas)) +ForLoop: + for _, schema := range schemas { + for _, re := range schemaRegexp { + if !re.DoMatch(schema.L) { + continue ForLoop + } + } + filteredSchemas = append(filteredSchemas, schema) + } + + // TODO: add table_id here + tableNames := e.getSchemaObjectNames(_tableName) + e.tableNames = tableNames + if len(tableNames) > 0 { + e.listTableFunc = e.listSchemaTablesByName + } else { + e.listTableFunc = listSchemaTables + } + + return filteredSchemas +} + +// ListTables lists related tables for given schema from predicate. +// If no table found in predicate, it return all tables. +func (e *InfoSchemaTableNameExtractor) ListTables( + ctx context.Context, + s model.CIStr, + is infoschema.InfoSchema, +) ([]*model.TableInfo, error) { + allTbls, err := e.listTableFunc(ctx, s, is) + if err != nil { + return nil, errors.Trace(err) + } + + if regexp, ok := e.colsRegexp[_tableName]; ok { + tbls := make([]*model.TableInfo, 0, len(allTbls)) + ForLoop: + for _, tbl := range allTbls { + for _, re := range regexp { + if !re.DoMatch(tbl.Name.L) { + continue ForLoop + } + } + tbls = append(tbls, tbl) + } + allTbls = tbls + } + + return allTbls, nil +} + +func (e *InfoSchemaTableNameExtractor) listSchemaTablesByName( + ctx context.Context, + s model.CIStr, + is infoschema.InfoSchema, +) ([]*model.TableInfo, error) { + tbls := make([]*model.TableInfo, 0, len(e.tableNames)) + for _, n := range e.tableNames { + tbl, err := is.TableByName(ctx, s, n) + if err != nil { + if terror.ErrorEqual(err, infoschema.ErrTableNotExists) { + continue + } + return nil, errors.Trace(err) + } + tbls = append(tbls, tbl.Meta()) + } + + return tbls, nil +} + +func listSchemaTables( + ctx context.Context, + s model.CIStr, + is infoschema.InfoSchema, +) ([]*model.TableInfo, error) { + return is.SchemaTableInfos(ctx, s) +} + +// ExplainInfo implements base.MemTablePredicateExtractor interface. +func (e *InfoSchemaTableNameExtractor) ExplainInfo(_ base.PhysicalPlan) string { + if e.SkipRequest { + return "skip_request:true" + } + + r := new(bytes.Buffer) + + for _, colName := range e.colNames { + if pred, ok := e.ColPredicates[colName]; ok && len(pred) > 0 { + fmt.Fprintf(r, "%s:[%s], ", colName, extractStringFromStringSet(pred)) + } + } + + for _, colName := range e.colNames { + if patterns, ok := e.LikePatterns[colName]; ok && len(patterns) > 0 { + fmt.Fprintf(r, "%s_pattern:[%s], ", colName, extractStringFromStringSlice(patterns)) + } + } + + // remove the last ", " in the message info + s := r.String() + if len(s) > 2 { + return s[:len(s)-2] + } + return s +} + +// InfoSchemaColumnsExtractor is the predicate extractor for information_schema.columns. +type InfoSchemaColumnsExtractor struct { + InfoSchemaTableNameExtractor +} + +// ListColumns lists unhidden columns and corresponding ordinal positions for given table from predicates. +// If no column found in predicate, it return all visible columns. +func (e *InfoSchemaTableNameExtractor) ListColumns( + tbl *model.TableInfo, +) ([]*model.ColumnInfo, []int) { + predCol, regexp, _ := e.getPredicates(_columnName) + + columns := make([]*model.ColumnInfo, 0, len(predCol)) + ordinalPos := make([]int, 0, len(predCol)) + ord := 0 +ForLoop: + for _, column := range tbl.Columns { + if column.Hidden { + continue + } + ord++ + if len(predCol) > 0 && !predCol.Exist(column.Name.L) { + continue + } + for _, re := range regexp { + if !re.DoMatch(column.Name.L) { + continue ForLoop + } + } + columns = append(columns, column) + ordinalPos = append(ordinalPos, ord) + } + + return columns, ordinalPos +} + +// InfoSchemaIndexUsageExtractor is the predicate extractor for information_schema.tidb_index_usage. +type InfoSchemaIndexUsageExtractor struct { + InfoSchemaTableNameExtractor +} + +// ListIndexes lists related indexes for given table from predicate. +// If no index found in predicate, it return all indexes. +func (e *InfoSchemaIndexUsageExtractor) ListIndexes( + tbl *model.TableInfo, +) []*model.IndexInfo { + predCol, regexp, _ := e.getPredicates(_indexName) + if len(predCol) == 0 && len(regexp) == 0 { + return tbl.Indices + } + + indexes := make([]*model.IndexInfo, 0, len(predCol)) +ForLoop: + for _, index := range tbl.Indices { + if len(predCol) > 0 && !predCol.Exist(index.Name.L) { + continue + } + for _, re := range regexp { + if !re.DoMatch(index.Name.L) { + continue ForLoop + } + } + indexes = append(indexes, index) + } + + return indexes +} diff --git a/pkg/planner/core/memtable_predicate_extractor.go b/pkg/planner/core/memtable_predicate_extractor.go index c5ff9a92fb4ef..28214fc6a541b 100644 --- a/pkg/planner/core/memtable_predicate_extractor.go +++ b/pkg/planner/core/memtable_predicate_extractor.go @@ -1678,84 +1678,6 @@ func (e *TikvRegionPeersExtractor) ExplainInfo(_ base.PhysicalPlan) string { return s } -// ColumnsTableExtractor is used to extract some predicates of columns table. -type ColumnsTableExtractor struct { - extractHelper - - // SkipRequest means the where clause always false, we don't need to request any component - SkipRequest bool - - TableSchema set.StringSet - - TableName set.StringSet - // ColumnName represents all column name we should filter in memtable. - ColumnName set.StringSet - - TableSchemaPatterns []string - - TableNamePatterns []string - - ColumnNamePatterns []string -} - -// Extract implements the MemTablePredicateExtractor Extract interface -func (e *ColumnsTableExtractor) Extract(ctx base.PlanContext, - schema *expression.Schema, - names []*types.FieldName, - predicates []expression.Expression, -) (remained []expression.Expression) { - remained, tableSchemaSkipRequest, tableSchema := e.extractCol(ctx, schema, names, predicates, "table_schema", true) - remained, tableNameSkipRequest, tableName := e.extractCol(ctx, schema, names, remained, "table_name", true) - remained, columnNameSkipRequest, columnName := e.extractCol(ctx, schema, names, remained, "column_name", true) - e.SkipRequest = columnNameSkipRequest || tableSchemaSkipRequest || tableNameSkipRequest - if e.SkipRequest { - return - } - remained, tableSchemaPatterns := e.extractLikePatternCol(ctx, schema, names, remained, "table_schema", true, false) - remained, tableNamePatterns := e.extractLikePatternCol(ctx, schema, names, remained, "table_name", true, false) - remained, columnNamePatterns := e.extractLikePatternCol(ctx, schema, names, remained, "column_name", true, false) - - e.ColumnName = columnName - e.TableName = tableName - e.TableSchema = tableSchema - e.TableSchemaPatterns = tableSchemaPatterns - e.TableNamePatterns = tableNamePatterns - e.ColumnNamePatterns = columnNamePatterns - return remained -} - -// ExplainInfo implements base.MemTablePredicateExtractor interface. -func (e *ColumnsTableExtractor) ExplainInfo(_ base.PhysicalPlan) string { - if e.SkipRequest { - return "skip_request:true" - } - r := new(bytes.Buffer) - if len(e.TableSchema) > 0 { - fmt.Fprintf(r, "table_schema:[%s], ", extractStringFromStringSet(e.TableSchema)) - } - if len(e.TableName) > 0 { - fmt.Fprintf(r, "table_name:[%s], ", extractStringFromStringSet(e.TableName)) - } - if len(e.ColumnName) > 0 { - fmt.Fprintf(r, "column_name:[%s], ", extractStringFromStringSet(e.ColumnName)) - } - if len(e.TableSchemaPatterns) > 0 { - fmt.Fprintf(r, "table_schema_pattern:[%s], ", extractStringFromStringSlice(e.TableSchemaPatterns)) - } - if len(e.TableNamePatterns) > 0 { - fmt.Fprintf(r, "table_name_pattern:[%s], ", extractStringFromStringSlice(e.TableNamePatterns)) - } - if len(e.ColumnNamePatterns) > 0 { - fmt.Fprintf(r, "column_name_pattern:[%s], ", extractStringFromStringSlice(e.ColumnNamePatterns)) - } - // remove the last ", " in the message info - s := r.String() - if len(s) > 2 { - return s[:len(s)-2] - } - return s -} - // TiKVRegionStatusExtractor is used to extract single table region scan region from predictions type TiKVRegionStatusExtractor struct { extractHelper diff --git a/pkg/planner/core/operator/logicalop/logicalop_test/logical_mem_table_predicate_extractor_test.go b/pkg/planner/core/operator/logicalop/logicalop_test/logical_mem_table_predicate_extractor_test.go index 1fc32f2d0ece9..35352c737507a 100644 --- a/pkg/planner/core/operator/logicalop/logicalop_test/logical_mem_table_predicate_extractor_test.go +++ b/pkg/planner/core/operator/logicalop/logicalop_test/logical_mem_table_predicate_extractor_test.go @@ -1575,7 +1575,7 @@ func TestColumns(t *testing.T) { }{ { sql: `select * from INFORMATION_SCHEMA.COLUMNS where lower(column_name)=lower('T');`, - columnName: set.NewStringSet(), + columnName: set.NewStringSet("t"), }, { sql: `select * from INFORMATION_SCHEMA.COLUMNS where column_name=lower('T');`, @@ -1632,33 +1632,33 @@ func TestColumns(t *testing.T) { logicalMemTable := getLogicalMemTable(t, dom, se, parser, ca.sql) require.NotNil(t, logicalMemTable.Extractor) - columnsTableExtractor := logicalMemTable.Extractor.(*plannercore.ColumnsTableExtractor) + columnsTableExtractor := logicalMemTable.Extractor.(*plannercore.InfoSchemaColumnsExtractor) 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.columnName.Count(), columnsTableExtractor.ColPredicates["column_name"].Count()) + if ca.columnName.Count() > 0 && columnsTableExtractor.ColPredicates["column_name"].Count() > 0 { + require.EqualValues(t, ca.columnName, columnsTableExtractor.ColPredicates["column_name"], "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.tableSchema.Count(), columnsTableExtractor.ColPredicates["table_schema"].Count()) + if ca.tableSchema.Count() > 0 && columnsTableExtractor.ColPredicates["table_schema"].Count() > 0 { + require.EqualValues(t, ca.tableSchema, columnsTableExtractor.ColPredicates["table_schema"], "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, ca.tableName.Count(), columnsTableExtractor.ColPredicates["table_name"].Count()) + if ca.tableName.Count() > 0 && columnsTableExtractor.ColPredicates["table_name"].Count() > 0 { + require.EqualValues(t, ca.tableName, columnsTableExtractor.ColPredicates["table_name"], "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.tableNamePattern), len(columnsTableExtractor.LikePatterns["table_name"])) + if len(ca.tableNamePattern) > 0 && len(columnsTableExtractor.LikePatterns["table_name"]) > 0 { + require.EqualValues(t, ca.tableNamePattern, columnsTableExtractor.LikePatterns["table_name"], "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.columnNamePattern), len(columnsTableExtractor.LikePatterns["column_name"])) + if len(ca.columnNamePattern) > 0 && len(columnsTableExtractor.LikePatterns["column_name"]) > 0 { + require.EqualValues(t, ca.columnNamePattern, columnsTableExtractor.LikePatterns["column_name"], "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) + require.Equal(t, len(ca.tableSchemaPattern), len(columnsTableExtractor.LikePatterns["table_schema"])) + if len(ca.tableSchemaPattern) > 0 && len(columnsTableExtractor.LikePatterns["table_schema"]) > 0 { + require.EqualValues(t, ca.tableSchemaPattern, columnsTableExtractor.LikePatterns["table_schema"], "SQL: %v", ca.sql) } } } @@ -1763,8 +1763,8 @@ func TestExtractorInPreparedStmt(t *testing.T) { userVars: []any{`"a%"`}, params: []any{"a%"}, checker: func(extractor base.MemTablePredicateExtractor) { - rse := extractor.(*plannercore.ColumnsTableExtractor) - require.EqualValues(t, []string{"a%"}, rse.TableNamePatterns) + rse := extractor.(*plannercore.InfoSchemaColumnsExtractor) + require.EqualValues(t, []string{"a%"}, rse.LikePatterns["table_name"]) }, }, { @@ -2052,12 +2052,16 @@ func TestInfoSchemaTableExtract(t *testing.T) { base = &ex.InfoSchemaBaseExtractor case *plannercore.InfoSchemaIndexesExtractor: base = &ex.InfoSchemaBaseExtractor + case *plannercore.InfoSchemaIndexUsageExtractor: + base = &ex.InfoSchemaBaseExtractor case *plannercore.InfoSchemaViewsExtractor: base = &ex.InfoSchemaBaseExtractor case *plannercore.InfoSchemaKeyColumnUsageExtractor: base = &ex.InfoSchemaBaseExtractor case *plannercore.InfoSchemaTableConstraintsExtractor: base = &ex.InfoSchemaBaseExtractor + case *plannercore.InfoSchemaColumnsExtractor: + base = &ex.InfoSchemaBaseExtractor default: require.Failf(t, "unexpected extractor type", "%T", ex) } diff --git a/pkg/table/tables/tables_test.go b/pkg/table/tables/tables_test.go index 50f4c785e52c4..4956b1f8e3226 100644 --- a/pkg/table/tables/tables_test.go +++ b/pkg/table/tables/tables_test.go @@ -695,13 +695,13 @@ func TestViewColumns(t *testing.T) { tk.MustQuery("select column_name, table_name from information_schema.columns where table_name='v1'").Check( testkit.RowsWithSep("|", "col|v1")) tk.MustExec("drop table if exists t") - for _, testCase := range testCases { - require.Len(t, tk.MustQuery(testCase.query).Rows(), 0) - tk.MustQuery("show warnings").Sort().Check(testkit.RowsWithSep("|", - "Warning|1356|View 'test.v' references invalid table(s) or column(s) or function(s) or definer/invoker of view lack rights to use them", - "Warning|1356|View 'test.v1' references invalid table(s) or column(s) or function(s) or definer/invoker of view lack rights to use them", - "Warning|1356|View 'test.va' references invalid table(s) or column(s) or function(s) or definer/invoker of view lack rights to use them")) - } + + require.Len(t, tk.MustQuery(testCases[0].query).Rows(), 0) + tk.MustQuery("show warnings").Sort().Check(testkit.RowsWithSep("|", + "Warning|1356|View 'test.v' references invalid table(s) or column(s) or function(s) or definer/invoker of view lack rights to use them")) + require.Len(t, tk.MustQuery(testCases[1].query).Rows(), 0) + tk.MustQuery("show warnings").Sort().Check(testkit.RowsWithSep("|", + "Warning|1356|View 'test.va' references invalid table(s) or column(s) or function(s) or definer/invoker of view lack rights to use them")) // For issue 43264 tk.MustExec(`CREATE TABLE User ( diff --git a/tests/integrationtest/r/infoschema/infoschema.result b/tests/integrationtest/r/infoschema/infoschema.result index f9a0b4680981e..96a5377490043 100644 --- a/tests/integrationtest/r/infoschema/infoschema.result +++ b/tests/integrationtest/r/infoschema/infoschema.result @@ -184,6 +184,102 @@ infoschema__infoschema_2 t2 a infoschema__infoschema_2 t2 b drop table infoschema__infoschema.t1; drop table infoschema__infoschema_2.t2; +drop database if exists indexusage; +create database indexusage; +use indexusage; +create table idt1(col_1 int primary key, col_2 int, index idx_1(col_1), index idx_2(col_2), index idx_3(col_1, col_2)); +create table idt2(col_1 int primary key, col_2 int, index idx_1(col_1), index idx_2(col_2), index idx_4(col_2, col_1)); +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_SCHEMA = 'indexusage'; +TABLE_SCHEMA TABLE_NAME INDEX_NAME +indexusage idt1 idx_1 +indexusage idt1 idx_2 +indexusage idt1 idx_3 +indexusage idt2 idx_1 +indexusage idt2 idx_2 +indexusage idt2 idx_4 +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_NAME = 'idt1'; +TABLE_SCHEMA TABLE_NAME INDEX_NAME +indexusage idt1 idx_1 +indexusage idt1 idx_2 +indexusage idt1 idx_3 +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where INDEX_NAME = 'IDX_3'; +TABLE_SCHEMA TABLE_NAME INDEX_NAME +indexusage idt1 idx_3 +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_SCHEMA = 'indexusage' and TABLE_NAME = 'idt1'; +TABLE_SCHEMA TABLE_NAME INDEX_NAME +indexusage idt1 idx_1 +indexusage idt1 idx_2 +indexusage idt1 idx_3 +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_SCHEMA = 'indexusage' and INDEX_NAME = 'idx_2'; +TABLE_SCHEMA TABLE_NAME INDEX_NAME +indexusage idt1 idx_2 +indexusage idt2 idx_2 +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_NAME = 'idt1' and INDEX_NAME = 'idx_1'; +TABLE_SCHEMA TABLE_NAME INDEX_NAME +indexusage idt1 idx_1 +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_SCHEMA = 'indexusage' and TABLE_NAME = 'idt2' and INDEX_NAME = 'idx_4'; +TABLE_SCHEMA TABLE_NAME INDEX_NAME +indexusage idt2 idx_4 +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_SCHEMA = 'indexusage1'; +TABLE_SCHEMA TABLE_NAME INDEX_NAME +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_NAME = 'idt3'; +TABLE_SCHEMA TABLE_NAME INDEX_NAME +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where INDEX_NAME = 'IDX_5'; +TABLE_SCHEMA TABLE_NAME INDEX_NAME +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_SCHEMA = 'indexusage' and TABLE_NAME = 'idt0'; +TABLE_SCHEMA TABLE_NAME INDEX_NAME +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_SCHEMA = 'indexusage1' and INDEX_NAME = 'idx_2'; +TABLE_SCHEMA TABLE_NAME INDEX_NAME +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_NAME = 'idt2' and INDEX_NAME = 'idx_3'; +TABLE_SCHEMA TABLE_NAME INDEX_NAME +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_SCHEMA = 'indexusage' and TABLE_NAME = 'idt1' and INDEX_NAME = 'idx_4'; +TABLE_SCHEMA TABLE_NAME INDEX_NAME +drop database indexusage; +drop database if exists columnsinfo; +create database columnsinfo; +use columnsinfo; +create table tbl1(col_1 int primary key, col_2 int, col_4 int); +create table tbl2(col_1 int primary key, col_2 int, col_3 int); +create view view1 as select min(col_1), col_2, max(col_4) as max4 from tbl1 group by col_2; +select TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME from information_schema.columns where TABLE_SCHEMA = "columnsinfo"; +TABLE_SCHEMA TABLE_NAME COLUMN_NAME +columnsinfo tbl1 col_1 +columnsinfo tbl1 col_2 +columnsinfo tbl1 col_4 +columnsinfo tbl2 col_1 +columnsinfo tbl2 col_2 +columnsinfo tbl2 col_3 +columnsinfo view1 col_2 +columnsinfo view1 max4 +columnsinfo view1 min(col_1) +select TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME from information_schema.columns where TABLE_NAME = "view1" or TABLE_NAME = "tbl1"; +TABLE_SCHEMA TABLE_NAME COLUMN_NAME +columnsinfo tbl1 col_1 +columnsinfo tbl1 col_2 +columnsinfo tbl1 col_4 +columnsinfo view1 col_2 +columnsinfo view1 max4 +columnsinfo view1 min(col_1) +select TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME from information_schema.columns where COLUMN_NAME = "col_2"; +TABLE_SCHEMA TABLE_NAME COLUMN_NAME +columnsinfo tbl1 col_2 +columnsinfo tbl2 col_2 +columnsinfo view1 col_2 +select TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME from information_schema.columns where TABLE_SCHEMA = "columnsinfo" and TABLE_NAME = "tbl2"; +TABLE_SCHEMA TABLE_NAME COLUMN_NAME +columnsinfo tbl2 col_1 +columnsinfo tbl2 col_2 +columnsinfo tbl2 col_3 +select TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME from information_schema.columns where TABLE_SCHEMA = "columnsinfo" and COLUMN_NAME = "col_4"; +TABLE_SCHEMA TABLE_NAME COLUMN_NAME +columnsinfo tbl1 col_4 +select TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME from information_schema.columns where TABLE_NAME = "view1" and COLUMN_NAME like "m%"; +TABLE_SCHEMA TABLE_NAME COLUMN_NAME +columnsinfo view1 max4 +columnsinfo view1 min(col_1) +select TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME from information_schema.columns where TABLE_SCHEMA = 'columnsinfo' and TABLE_NAME = 'tbl1' and COLUMN_NAME = 'col_2'; +TABLE_SCHEMA TABLE_NAME COLUMN_NAME +columnsinfo tbl1 col_2 use infoschema__infoschema; select SCHEMA_NAME from information_schema.schemata where schema_name = 'infoschema__infoschema_2'; SCHEMA_NAME diff --git a/tests/integrationtest/t/infoschema/infoschema.test b/tests/integrationtest/t/infoschema/infoschema.test index fc7efaa8359ab..dd4980315e989 100644 --- a/tests/integrationtest/t/infoschema/infoschema.test +++ b/tests/integrationtest/t/infoschema/infoschema.test @@ -93,6 +93,66 @@ select TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME from information_schema.statistics drop table infoschema__infoschema.t1; drop table infoschema__infoschema_2.t2; +# TestIndexUsageColumns +drop database if exists indexusage; +create database indexusage; +use indexusage; +create table idt1(col_1 int primary key, col_2 int, index idx_1(col_1), index idx_2(col_2), index idx_3(col_1, col_2)); +create table idt2(col_1 int primary key, col_2 int, index idx_1(col_1), index idx_2(col_2), index idx_4(col_2, col_1)); +-- sorted_result +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_SCHEMA = 'indexusage'; +-- sorted_result +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_NAME = 'idt1'; +-- sorted_result +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where INDEX_NAME = 'IDX_3'; +-- sorted_result +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_SCHEMA = 'indexusage' and TABLE_NAME = 'idt1'; +-- sorted_result +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_SCHEMA = 'indexusage' and INDEX_NAME = 'idx_2'; +-- sorted_result +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_NAME = 'idt1' and INDEX_NAME = 'idx_1'; +-- sorted_result +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_SCHEMA = 'indexusage' and TABLE_NAME = 'idt2' and INDEX_NAME = 'idx_4'; + +# Empty query +-- sorted_result +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_SCHEMA = 'indexusage1'; +-- sorted_result +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_NAME = 'idt3'; +-- sorted_result +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where INDEX_NAME = 'IDX_5'; +-- sorted_result +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_SCHEMA = 'indexusage' and TABLE_NAME = 'idt0'; +-- sorted_result +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_SCHEMA = 'indexusage1' and INDEX_NAME = 'idx_2'; +-- sorted_result +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_NAME = 'idt2' and INDEX_NAME = 'idx_3'; +-- sorted_result +select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_SCHEMA = 'indexusage' and TABLE_NAME = 'idt1' and INDEX_NAME = 'idx_4'; +drop database indexusage; + +drop database if exists columnsinfo; +create database columnsinfo; +use columnsinfo; +create table tbl1(col_1 int primary key, col_2 int, col_4 int); +create table tbl2(col_1 int primary key, col_2 int, col_3 int); +create view view1 as select min(col_1), col_2, max(col_4) as max4 from tbl1 group by col_2; + +-- sorted_result +select TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME from information_schema.columns where TABLE_SCHEMA = "columnsinfo"; +-- sorted_result +select TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME from information_schema.columns where TABLE_NAME = "view1" or TABLE_NAME = "tbl1"; +-- sorted_result +select TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME from information_schema.columns where COLUMN_NAME = "col_2"; +-- sorted_result +select TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME from information_schema.columns where TABLE_SCHEMA = "columnsinfo" and TABLE_NAME = "tbl2"; +-- sorted_result +select TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME from information_schema.columns where TABLE_SCHEMA = "columnsinfo" and COLUMN_NAME = "col_4"; +-- sorted_result +select TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME from information_schema.columns where TABLE_NAME = "view1" and COLUMN_NAME like "m%"; +-- sorted_result +select TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME from information_schema.columns where TABLE_SCHEMA = 'columnsinfo' and TABLE_NAME = 'tbl1' and COLUMN_NAME = 'col_2'; + # TestSchemataColumns use infoschema__infoschema; select SCHEMA_NAME from information_schema.schemata where schema_name = 'infoschema__infoschema_2';