From ff54f1cc14f79bf90ac0e83e9dc55ea72a08b02d Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Tue, 16 Jul 2024 16:25:14 +0800 Subject: [PATCH 01/12] *: remove infoschema.SchemaTables() API, replace it with SchemaTableInfos() --- br/pkg/restore/log_client/client.go | 3 +- br/pkg/task/restore_data.go | 3 +- pkg/ddl/ddl_api.go | 11 +- pkg/ddl/foreign_key.go | 12 +- pkg/ddl/placement_policy.go | 3 +- pkg/executor/infoschema_reader.go | 163 ++++++++---------- pkg/executor/memtable_reader.go | 2 +- pkg/executor/show.go | 42 ++--- pkg/executor/show_placement.go | 6 +- pkg/executor/show_stats.go | 40 ++--- pkg/infoschema/context/infoschema.go | 4 +- pkg/infoschema/infoschema.go | 26 ++- pkg/infoschema/infoschema_test.go | 16 +- pkg/infoschema/infoschema_v2.go | 58 +------ pkg/infoschema/infoschema_v2_test.go | 13 +- pkg/infoschema/interface.go | 1 - .../test/clustertablestest/tables_test.go | 14 +- pkg/lock/lock.go | 4 +- pkg/server/handler/tests/http_handler_test.go | 6 +- .../handler/tikvhandler/tikv_handler.go | 16 +- .../handle/autoanalyze/autoanalyze.go | 7 +- .../handle/autoanalyze/refresher/refresher.go | 8 +- 22 files changed, 183 insertions(+), 275 deletions(-) diff --git a/br/pkg/restore/log_client/client.go b/br/pkg/restore/log_client/client.go index a2ffb4ee5054f..ec85c1ef78202 100644 --- a/br/pkg/restore/log_client/client.go +++ b/br/pkg/restore/log_client/client.go @@ -683,8 +683,7 @@ func (rc *LogClient) InitSchemasReplaceForDDL( info := rc.dom.InfoSchema() shcemas := info.AllSchemaNames() for _, schema := range shcemas { - for _, table := range info.SchemaTables(schema) { - tableInfo := table.Meta() + for _, tableInfo := range info.SchemaTableInfos(ctx, schema) { if tableInfo.TiFlashReplica != nil && tableInfo.TiFlashReplica.Count > 0 { return nil, errors.Errorf("exist table(s) have tiflash replica, please remove it before restore") } diff --git a/br/pkg/task/restore_data.go b/br/pkg/task/restore_data.go index ecfbf77f870ce..db9cd7d10e949 100644 --- a/br/pkg/task/restore_data.go +++ b/br/pkg/task/restore_data.go @@ -182,8 +182,7 @@ func resetTiFlashReplicas(ctx context.Context, g glue.Glue, storage kv.Storage, expectTiFlashStoreCount := uint64(0) needTiFlash := false for _, s := range allSchemaName { - for _, t := range info.SchemaTables(s) { - t := t.Meta() + for _, t := range info.SchemaTableInfos(ctx, s) { if t.TiFlashReplica != nil { expectTiFlashStoreCount = max(expectTiFlashStoreCount, t.TiFlashReplica.Count) recorder.AddTable(t.ID, *t.TiFlashReplica) diff --git a/pkg/ddl/ddl_api.go b/pkg/ddl/ddl_api.go index 08230de72fc7f..e9dd8116f5df1 100644 --- a/pkg/ddl/ddl_api.go +++ b/pkg/ddl/ddl_api.go @@ -406,7 +406,7 @@ func (d *ddl) ModifySchemaSetTiFlashReplica(sctx sessionctx.Context, stmt *ast.A return errors.Trace(dbterror.ErrUnsupportedTiFlashOperationForSysOrMemTable) } - tbls := is.SchemaTables(dbInfo.Name) + tbls := is.SchemaTableInfos(context.Background(), dbInfo.Name) total := len(tbls) succ := 0 skip := 0 @@ -429,7 +429,6 @@ func (d *ddl) ModifySchemaSetTiFlashReplica(sctx sessionctx.Context, stmt *ast.A threshold := uint32(sctx.GetSessionVars().BatchPendingTiFlashCount) for _, tbl := range tbls { - tbl := tbl.Meta() done, killed := isSessionDone(sctx) if done { logutil.DDLLogger().Info("abort batch add TiFlash replica", zap.Int64("schemaID", dbInfo.ID), zap.Uint32("isKilled", killed)) @@ -674,7 +673,7 @@ func (d *ddl) DropSchema(ctx sessionctx.Context, stmt *ast.DropDatabaseStmt) (er return infoschema.ErrDatabaseDropExists.GenWithStackByArgs(stmt.Name) } fkCheck := ctx.GetSessionVars().ForeignKeyChecks - err = checkDatabaseHasForeignKeyReferred(is, old.Name, fkCheck) + err = checkDatabaseHasForeignKeyReferred(d.ctx, is, old.Name, fkCheck) if err != nil { return err } @@ -708,11 +707,11 @@ func (d *ddl) DropSchema(ctx sessionctx.Context, stmt *ast.DropDatabaseStmt) (er return nil } // Clear table locks hold by the session. - tbs := is.SchemaTables(stmt.Name) + tbs := is.SchemaTableInfos(d.ctx, stmt.Name) lockTableIDs := make([]int64, 0) for _, tb := range tbs { - if ok, _ := ctx.CheckTableLocked(tb.Meta().ID); ok { - lockTableIDs = append(lockTableIDs, tb.Meta().ID) + if ok, _ := ctx.CheckTableLocked(tb.ID); ok { + lockTableIDs = append(lockTableIDs, tb.ID) } } ctx.ReleaseTableLockByTableIDs(lockTableIDs) diff --git a/pkg/ddl/foreign_key.go b/pkg/ddl/foreign_key.go index fba4ed5eaec71..bd744c7566342 100644 --- a/pkg/ddl/foreign_key.go +++ b/pkg/ddl/foreign_key.go @@ -601,18 +601,18 @@ func (h *foreignKeyHelper) getTableFromStorage(is infoschema.InfoSchema, t *meta return result, nil } -func checkDatabaseHasForeignKeyReferred(is infoschema.InfoSchema, schema model.CIStr, fkCheck bool) error { +func checkDatabaseHasForeignKeyReferred(ctx context.Context, is infoschema.InfoSchema, schema model.CIStr, fkCheck bool) error { if !fkCheck { return nil } - tables := is.SchemaTables(schema) + tables := is.SchemaTableInfos(ctx, schema) tableNames := make([]ast.Ident, len(tables)) for i := range tables { - tableNames[i] = ast.Ident{Schema: schema, Name: tables[i].Meta().Name} + tableNames[i] = ast.Ident{Schema: schema, Name: tables[i].Name} } for _, tbl := range tables { - if referredFK := checkTableHasForeignKeyReferred(is, schema.L, tbl.Meta().Name.L, tableNames, fkCheck); referredFK != nil { - return errors.Trace(dbterror.ErrForeignKeyCannotDropParent.GenWithStackByArgs(tbl.Meta().Name, referredFK.ChildFKName, referredFK.ChildTable)) + if referredFK := checkTableHasForeignKeyReferred(is, schema.L, tbl.Name.L, tableNames, fkCheck); referredFK != nil { + return errors.Trace(dbterror.ErrForeignKeyCannotDropParent.GenWithStackByArgs(tbl.Name, referredFK.ChildFKName, referredFK.ChildTable)) } } return nil @@ -635,7 +635,7 @@ func checkDatabaseHasForeignKeyReferredInOwner(d *ddlCtx, t *meta.Meta, job *mod if err != nil { return errors.Trace(err) } - err = checkDatabaseHasForeignKeyReferred(is, model.NewCIStr(job.SchemaName), fkCheck) + err = checkDatabaseHasForeignKeyReferred(d.ctx, is, model.NewCIStr(job.SchemaName), fkCheck) if err != nil { job.State = model.JobStateCancelled } diff --git a/pkg/ddl/placement_policy.go b/pkg/ddl/placement_policy.go index 799da52e5efb3..7ace7811dc86d 100644 --- a/pkg/ddl/placement_policy.go +++ b/pkg/ddl/placement_policy.go @@ -364,8 +364,7 @@ func CheckPlacementPolicyNotInUseFromInfoSchema(is infoschema.InfoSchema, policy return dbterror.ErrPlacementPolicyInUse.GenWithStackByArgs(policy.Name) } - for _, tbl := range is.SchemaTables(dbInfo.Name) { - tblInfo := tbl.Meta() + for _, tblInfo := range is.SchemaTableInfos(context.Background(), dbInfo.Name) { if err := checkPlacementPolicyNotUsedByTable(tblInfo, policy); err != nil { return err } diff --git a/pkg/executor/infoschema_reader.go b/pkg/executor/infoschema_reader.go index bdbf3eaa13ada..a558d409403c2 100644 --- a/pkg/executor/infoschema_reader.go +++ b/pkg/executor/infoschema_reader.go @@ -121,23 +121,23 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex case infoschema.TableSchemata: e.setDataFromSchemata(sctx, dbs) case infoschema.TableStatistics: - e.setDataForStatistics(sctx, dbs) + e.setDataForStatistics(ctx, sctx, dbs) case infoschema.TableTables: err = e.setDataFromTables(ctx, sctx, dbs) case infoschema.TableReferConst: - err = e.setDataFromReferConst(sctx, dbs) + err = e.setDataFromReferConst(ctx, sctx, dbs) case infoschema.TableSequences: - e.setDataFromSequences(sctx, dbs) + e.setDataFromSequences(ctx, sctx, dbs) case infoschema.TablePartitions: - err = e.setDataFromPartitions(sctx, dbs) + err = e.setDataFromPartitions(ctx, sctx, dbs) case infoschema.TableClusterInfo: err = e.dataForTiDBClusterInfo(sctx) case infoschema.TableAnalyzeStatus: err = e.setDataForAnalyzeStatus(ctx, sctx) case infoschema.TableTiDBIndexes: - e.setDataFromIndexes(sctx, dbs) + e.setDataFromIndexes(ctx, sctx, dbs) case infoschema.TableViews: - e.setDataFromViews(sctx, dbs) + e.setDataFromViews(ctx, sctx, dbs) case infoschema.TableEngines: e.setDataFromEngines() case infoschema.TableCharacterSets: @@ -145,7 +145,7 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex case infoschema.TableCollations: e.setDataFromCollations() case infoschema.TableKeyColumn: - e.setDataFromKeyColumnUsage(sctx, dbs) + e.setDataFromKeyColumnUsage(ctx, sctx, dbs) case infoschema.TableMetricTables: e.setDataForMetricTables() case infoschema.TableProfiling: @@ -163,13 +163,13 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex case infoschema.TableTiDBHotRegions: err = e.setDataForTiDBHotRegions(ctx, sctx) case infoschema.TableConstraints: - e.setDataFromTableConstraints(sctx, dbs) + e.setDataFromTableConstraints(ctx, sctx, dbs) case infoschema.TableSessionVar: e.rows, err = infoschema.GetDataFromSessionVariables(ctx, sctx) case infoschema.TableTiDBServersInfo: err = e.setDataForServersInfo(sctx) case infoschema.TableTiFlashReplica: - e.dataForTableTiFlashReplica(sctx, dbs) + e.dataForTableTiFlashReplica(ctx, sctx, dbs) case infoschema.TableTiKVStoreStatus: err = e.dataForTiKVStoreStatus(ctx, sctx) case infoschema.TableClientErrorsSummaryGlobal, @@ -201,15 +201,15 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex case infoschema.TableRunawayWatches: err = e.setDataFromRunawayWatches(sctx) case infoschema.TableCheckConstraints: - err = e.setDataFromCheckConstraints(sctx, dbs) + err = e.setDataFromCheckConstraints(ctx, sctx, dbs) case infoschema.TableTiDBCheckConstraints: - err = e.setDataFromTiDBCheckConstraints(sctx, dbs) + err = e.setDataFromTiDBCheckConstraints(ctx, sctx, dbs) case infoschema.TableKeywords: err = e.setDataFromKeywords() case infoschema.TableTiDBIndexUsage: - e.setDataFromIndexUsage(sctx, dbs) + e.setDataFromIndexUsage(ctx, sctx, dbs) case infoschema.ClusterTableTiDBIndexUsage: - err = e.setDataForClusterIndexUsage(sctx, dbs) + err = e.setDataForClusterIndexUsage(ctx, sctx, dbs) } if err != nil { return nil, err @@ -372,8 +372,8 @@ func (e *memtableRetriever) setDataFromSchemata(ctx sessionctx.Context, schemas e.rows = rows } -func (e *memtableRetriever) setDataForStatistics(ctx sessionctx.Context, schemas []model.CIStr) { - checker := privilege.GetPrivilegeManager(ctx) +func (e *memtableRetriever) setDataForStatistics(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) { + checker := privilege.GetPrivilegeManager(sctx) extractor, ok := e.extractor.(*plannercore.InfoSchemaTablesExtractor) if ok && extractor.SkipRequest { return @@ -382,13 +382,12 @@ func (e *memtableRetriever) setDataForStatistics(ctx sessionctx.Context, schemas if ok && extractor.Filter("table_schema", schema.L) { continue } - tables := e.is.SchemaTables(schema) + tables := e.is.SchemaTableInfos(ctx, schema) for _, table := range tables { - table := table.Meta() if ok && extractor.Filter("table_name", table.Name.L) { continue } - if checker != nil && !checker.RequestVerification(ctx.GetSessionVars().ActiveRoles, schema.L, table.Name.L, "", mysql.AllPrivMask) { + if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.L, table.Name.L, "", mysql.AllPrivMask) { continue } e.setDataForStatisticsInTable(schema, table) @@ -481,7 +480,7 @@ func (e *memtableRetriever) setDataForStatisticsInTable(schema model.CIStr, tabl e.rows = append(e.rows, rows...) } -func (e *memtableRetriever) setDataFromReferConst(sctx sessionctx.Context, schemas []model.CIStr) error { +func (e *memtableRetriever) setDataFromReferConst(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) error { checker := privilege.GetPrivilegeManager(sctx) var rows [][]types.Datum extractor, ok := e.extractor.(*plannercore.InfoSchemaTablesExtractor) @@ -492,9 +491,8 @@ func (e *memtableRetriever) setDataFromReferConst(sctx sessionctx.Context, schem if ok && extractor.Filter("table_schema", schema.L) { continue } - tables := e.is.SchemaTables(schema) + tables := e.is.SchemaTableInfos(ctx, schema) for _, table := range tables { - table := table.Meta() if ok && extractor.Filter("table_name", table.Name.L) { continue } @@ -561,9 +559,8 @@ func (e *memtableRetriever) setDataFromTables(ctx context.Context, sctx sessionc if ok && extractor.Filter("table_schema", schema.L) { continue } - tables := e.is.SchemaTables(schema) + tables := e.is.SchemaTableInfos(ctx, schema) for _, table := range tables { - table := table.Meta() if ok && extractor.Filter("table_name", table.Name.L) { continue } @@ -695,13 +692,12 @@ func (e *memtableRetriever) setDataFromTables(ctx context.Context, sctx sessionc // Data for inforation_schema.CHECK_CONSTRAINTS // This is standards (ISO/IEC 9075-11) compliant and is compatible with the implementation in MySQL as well. -func (e *memtableRetriever) setDataFromCheckConstraints(sctx sessionctx.Context, schemas []model.CIStr) error { +func (e *memtableRetriever) setDataFromCheckConstraints(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) error { var rows [][]types.Datum checker := privilege.GetPrivilegeManager(sctx) for _, schema := range schemas { - tables := e.is.SchemaTables(schema) + tables := e.is.SchemaTableInfos(ctx, schema) for _, table := range tables { - table := table.Meta() if len(table.Constraints) > 0 { if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.L, table.Name.L, "", mysql.SelectPriv) { continue @@ -727,13 +723,12 @@ func (e *memtableRetriever) setDataFromCheckConstraints(sctx sessionctx.Context, // Data for inforation_schema.TIDB_CHECK_CONSTRAINTS // This has non-standard TiDB specific extensions. -func (e *memtableRetriever) setDataFromTiDBCheckConstraints(sctx sessionctx.Context, schemas []model.CIStr) error { +func (e *memtableRetriever) setDataFromTiDBCheckConstraints(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) error { var rows [][]types.Datum checker := privilege.GetPrivilegeManager(sctx) for _, schema := range schemas { - tables := e.is.SchemaTables(schema) + tables := e.is.SchemaTableInfos(ctx, schema) for _, table := range tables { - table := table.Meta() if len(table.Constraints) > 0 { if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.L, table.Name.L, "", mysql.SelectPriv) { continue @@ -768,7 +763,7 @@ type hugeMemTableRetriever struct { initialized bool rows [][]types.Datum dbs []*model.DBInfo - curTables []table.Table + curTables []*model.TableInfo dbsIdx int tblIdx int viewMu syncutil.RWMutex @@ -813,10 +808,10 @@ func (e *hugeMemTableRetriever) setDataForColumns(ctx context.Context, sctx sess schema := e.dbs[e.dbsIdx] var table *model.TableInfo if len(e.curTables) == 0 { - e.curTables = e.is.SchemaTables(schema.Name) + e.curTables = e.is.SchemaTableInfos(ctx, schema.Name) } for e.tblIdx < len(e.curTables) { - table = e.curTables[e.tblIdx].Meta() + table = e.curTables[e.tblIdx] e.tblIdx++ if e.setDataForColumnsWithOneTable(ctx, sctx, extractor, schema, table, checker) { return nil @@ -1062,7 +1057,7 @@ func calcCharOctLength(lenInChar int, cs string) int { return lenInBytes } -func (e *memtableRetriever) setDataFromPartitions(sctx sessionctx.Context, schemas []model.CIStr) error { +func (e *memtableRetriever) setDataFromPartitions(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) error { cache := cache.TableRowStatsCache err := cache.Update(sctx) if err != nil { @@ -1072,9 +1067,8 @@ func (e *memtableRetriever) setDataFromPartitions(sctx sessionctx.Context, schem var rows [][]types.Datum createTimeTp := mysql.TypeDatetime for _, schema := range schemas { - tables := e.is.SchemaTables(schema) + tables := e.is.SchemaTableInfos(ctx, schema) for _, table := range tables { - table := table.Meta() if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.L, table.Name.L, "", mysql.SelectPriv) { continue } @@ -1217,14 +1211,13 @@ func (e *memtableRetriever) setDataFromPartitions(sctx sessionctx.Context, schem return nil } -func (e *memtableRetriever) setDataFromIndexes(ctx sessionctx.Context, schemas []model.CIStr) { - checker := privilege.GetPrivilegeManager(ctx) +func (e *memtableRetriever) setDataFromIndexes(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) { + checker := privilege.GetPrivilegeManager(sctx) var rows [][]types.Datum for _, schema := range schemas { - tables := e.is.SchemaTables(schema) + tables := e.is.SchemaTableInfos(ctx, schema) for _, tb := range tables { - tb := tb.Meta() - if checker != nil && !checker.RequestVerification(ctx.GetSessionVars().ActiveRoles, schema.L, tb.Name.L, "", mysql.AllPrivMask) { + if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.L, tb.Name.L, "", mysql.AllPrivMask) { continue } @@ -1305,13 +1298,12 @@ func (e *memtableRetriever) setDataFromIndexes(ctx sessionctx.Context, schemas [ e.rows = rows } -func (e *memtableRetriever) setDataFromViews(ctx sessionctx.Context, schemas []model.CIStr) { - checker := privilege.GetPrivilegeManager(ctx) +func (e *memtableRetriever) setDataFromViews(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) { + checker := privilege.GetPrivilegeManager(sctx) var rows [][]types.Datum for _, schema := range schemas { - tables := e.is.SchemaTables(schema) + tables := e.is.SchemaTableInfos(ctx, schema) for _, table := range tables { - table := table.Meta() if !table.IsView() { continue } @@ -1323,7 +1315,7 @@ func (e *memtableRetriever) setDataFromViews(ctx sessionctx.Context, schemas []m if charset == "" { charset = mysql.DefaultCharset } - if checker != nil && !checker.RequestVerification(ctx.GetSessionVars().ActiveRoles, schema.L, table.Name.L, "", mysql.AllPrivMask) { + if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.L, table.Name.L, "", mysql.AllPrivMask) { continue } record := types.MakeDatums( @@ -1593,8 +1585,8 @@ func (e *memtableRetriever) dataForTiDBClusterInfo(ctx sessionctx.Context) error return nil } -func (e *memtableRetriever) setDataFromKeyColumnUsage(ctx sessionctx.Context, schemas []model.CIStr) { - checker := privilege.GetPrivilegeManager(ctx) +func (e *memtableRetriever) setDataFromKeyColumnUsage(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) { + checker := privilege.GetPrivilegeManager(sctx) 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 { @@ -1604,13 +1596,12 @@ func (e *memtableRetriever) setDataFromKeyColumnUsage(ctx sessionctx.Context, sc if ok && extractor.Filter("table_schema", schema.L) { continue } - tables := e.is.SchemaTables(schema) + tables := e.is.SchemaTableInfos(ctx, schema) for _, table := range tables { - table := table.Meta() if ok && extractor.Filter("table_name", table.Name.L) { continue } - if checker != nil && !checker.RequestVerification(ctx.GetSessionVars().ActiveRoles, schema.L, table.Name.L, "", mysql.AllPrivMask) { + if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.L, table.Name.L, "", mysql.AllPrivMask) { continue } rs := keyColumnUsageInTable(schema, table) @@ -1769,16 +1760,16 @@ func keyColumnUsageInTable(schema model.CIStr, table *model.TableInfo) [][]types return rows } -func ensureSchemaTables(is infoschema.InfoSchema, schemaNames []model.CIStr) []*model.DBInfo { +func ensureSchemaTables(ctx context.Context, is infoschema.InfoSchema, schemaNames []model.CIStr) []*model.DBInfo { // For infoschema v2, Tables of DBInfo could be missing. res := make([]*model.DBInfo, 0, len(schemaNames)) for _, dbName := range schemaNames { dbInfoRaw, _ := is.SchemaByName(dbName) dbInfo := dbInfoRaw.Clone() dbInfo.Tables = dbInfo.Tables[:0] - tbls := is.SchemaTables(dbName) + tbls := is.SchemaTableInfos(ctx, dbName) for _, tbl := range tbls { - dbInfo.Tables = append(dbInfo.Tables, tbl.Meta()) + dbInfo.Tables = append(dbInfo.Tables, tbl) } res = append(res, dbInfo) } @@ -1827,7 +1818,7 @@ func (e *memtableRetriever) setDataForTiKVRegionStatus(ctx context.Context, sctx } } schemaNames := is.AllSchemaNames() - schemas := ensureSchemaTables(is, schemaNames) + schemas := ensureSchemaTables(ctx, is, schemaNames) tableInfos := tikvHelper.GetRegionsTableInfo(allRegionsInfo, schemas) for i := range allRegionsInfo.Regions { regionTableList := tableInfos[allRegionsInfo.Regions[i].ID] @@ -1999,14 +1990,13 @@ func (e *memtableRetriever) setDataForHotRegionByMetrics(metrics []helper.HotTab } // setDataFromTableConstraints constructs data for table information_schema.constraints.See https://dev.mysql.com/doc/refman/5.7/en/table-constraints-table.html -func (e *memtableRetriever) setDataFromTableConstraints(ctx sessionctx.Context, schemas []model.CIStr) { - checker := privilege.GetPrivilegeManager(ctx) +func (e *memtableRetriever) setDataFromTableConstraints(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) { + checker := privilege.GetPrivilegeManager(sctx) var rows [][]types.Datum for _, schema := range schemas { - tables := e.is.SchemaTables(schema) + tables := e.is.SchemaTableInfos(ctx, schema) for _, tbl := range tables { - tbl := tbl.Meta() - if checker != nil && !checker.RequestVerification(ctx.GetSessionVars().ActiveRoles, schema.L, tbl.Name.L, "", mysql.AllPrivMask) { + if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.L, tbl.Name.L, "", mysql.AllPrivMask) { continue } @@ -2080,7 +2070,7 @@ func (e *tableStorageStatsRetriever) retrieve(ctx context.Context, sctx sessionc return nil, nil } if !e.initialized { - err := e.initialize(sctx) + err := e.initialize(ctx, sctx) if err != nil { return nil, err } @@ -2113,7 +2103,7 @@ type initialTable struct { *model.TableInfo } -func (e *tableStorageStatsRetriever) initialize(sctx sessionctx.Context) error { +func (e *tableStorageStatsRetriever) initialize(ctx context.Context, sctx sessionctx.Context) error { is := sctx.GetInfoSchema().(infoschema.InfoSchema) var databases []string schemas := e.extractor.TableSchema @@ -2144,11 +2134,11 @@ func (e *tableStorageStatsRetriever) initialize(sctx sessionctx.Context) error { for _, DB := range databases { // The user didn't specified the table, extract all tables of this db to initialTable. if len(tables) == 0 { - tbs := is.SchemaTables(model.NewCIStr(DB)) + tbs := is.SchemaTableInfos(ctx, model.NewCIStr(DB)) for _, tb := range tbs { // For every db.table, check it's privileges. - if checker(DB, tb.Meta().Name.L) { - e.initialTables = append(e.initialTables, &initialTable{DB, tb.Meta()}) + if checker(DB, tb.Name.L) { + e.initialTables = append(e.initialTables, &initialTable{DB, tb}) } } } else { @@ -2441,17 +2431,16 @@ func (e *memtableRetriever) setDataForServersInfo(ctx sessionctx.Context) error return nil } -func (e *memtableRetriever) setDataFromSequences(ctx sessionctx.Context, schemas []model.CIStr) { - checker := privilege.GetPrivilegeManager(ctx) +func (e *memtableRetriever) setDataFromSequences(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) { + checker := privilege.GetPrivilegeManager(sctx) var rows [][]types.Datum for _, schema := range schemas { - tables := e.is.SchemaTables(schema) + tables := e.is.SchemaTableInfos(ctx, schema) for _, table := range tables { - table := table.Meta() if !table.IsSequence() { continue } - if checker != nil && !checker.RequestVerification(ctx.GetSessionVars().ActiveRoles, schema.L, table.Name.L, "", mysql.AllPrivMask) { + if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.L, table.Name.L, "", mysql.AllPrivMask) { continue } record := types.MakeDatums( @@ -2474,20 +2463,17 @@ func (e *memtableRetriever) setDataFromSequences(ctx sessionctx.Context, schemas } // dataForTableTiFlashReplica constructs data for table tiflash replica info. -func (e *memtableRetriever) dataForTableTiFlashReplica(ctx sessionctx.Context, schemas []model.CIStr) { +func (e *memtableRetriever) dataForTableTiFlashReplica(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) { var ( - checker = privilege.GetPrivilegeManager(ctx) + checker = privilege.GetPrivilegeManager(sctx) rows [][]types.Datum tiFlashStores map[int64]pd.StoreInfo ) - for _, schema := range schemas { - tables := e.is.SchemaTables(schema) - for _, tbl := range tables { - tbl := tbl.Meta() - if tbl.TiFlashReplica == nil { - continue - } - if checker != nil && !checker.RequestVerification(ctx.GetSessionVars().ActiveRoles, schema.L, tbl.Name.L, "", mysql.AllPrivMask) { + tableInfoResult := e.is.ListTablesWithSpecialAttribute(infoschema.TiFlashAttribute) + for _, res := range tableInfoResult { + schema := res.DBName + for _, tbl := range res.TableInfos { + if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema, tbl.Name.L, "", mysql.AllPrivMask) { continue } var progress float64 @@ -2510,7 +2496,7 @@ func (e *memtableRetriever) dataForTableTiFlashReplica(ctx sessionctx.Context, s progressString := types.TruncateFloatToString(progress, 2) progress, _ = strconv.ParseFloat(progressString, 64) record := types.MakeDatums( - schema.O, // TABLE_SCHEMA + schema, // TABLE_SCHEMA tbl.Name.O, // TABLE_NAME tbl.ID, // TABLE_ID int64(tbl.TiFlashReplica.Count), // REPLICA_COUNT @@ -3578,17 +3564,16 @@ func (e *memtableRetriever) setDataFromKeywords() error { return nil } -func (e *memtableRetriever) setDataFromIndexUsage(ctx sessionctx.Context, schemas []model.CIStr) { - dom := domain.GetDomain(ctx) +func (e *memtableRetriever) setDataFromIndexUsage(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) { + dom := domain.GetDomain(sctx) rows := make([][]types.Datum, 0, 100) - checker := privilege.GetPrivilegeManager(ctx) + checker := privilege.GetPrivilegeManager(sctx) for _, schema := range schemas { - tables := dom.InfoSchema().SchemaTables(schema) + tables := dom.InfoSchema().SchemaTableInfos(ctx, schema) for _, tbl := range tables { - tbl := tbl.Meta() allowed := checker == nil || checker.RequestVerification( - ctx.GetSessionVars().ActiveRoles, + sctx.GetSessionVars().ActiveRoles, schema.L, tbl.Name.L, "", mysql.AllPrivMask) if !allowed { continue @@ -3622,9 +3607,9 @@ func (e *memtableRetriever) setDataFromIndexUsage(ctx sessionctx.Context, schema e.rows = rows } -func (e *memtableRetriever) setDataForClusterIndexUsage(ctx sessionctx.Context, schemas []model.CIStr) error { - e.setDataFromIndexUsage(ctx, schemas) - rows, err := infoschema.AppendHostInfoToRows(ctx, e.rows) +func (e *memtableRetriever) setDataForClusterIndexUsage(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) error { + e.setDataFromIndexUsage(ctx, sctx, schemas) + rows, err := infoschema.AppendHostInfoToRows(sctx, e.rows) if err != nil { return err } diff --git a/pkg/executor/memtable_reader.go b/pkg/executor/memtable_reader.go index fecbc5cfb28df..8a7644c2feb54 100644 --- a/pkg/executor/memtable_reader.go +++ b/pkg/executor/memtable_reader.go @@ -792,7 +792,7 @@ func (e *hotRegionsHistoryRetriver) retrieve(ctx context.Context, sctx sessionct tz := sctx.GetSessionVars().Location() is := sessiontxn.GetTxnManager(sctx).GetTxnInfoSchema() allSchemaNames := is.AllSchemaNames() - schemas := ensureSchemaTables(is, allSchemaNames) + schemas := ensureSchemaTables(ctx, is, allSchemaNames) schemas = tikvHelper.FilterMemDBs(schemas) tables := tikvHelper.GetTablesInfoWithKeyRange(schemas) for e.heap.Len() > 0 && len(finalRows) < hotRegionsHistoryBatchSize { diff --git a/pkg/executor/show.go b/pkg/executor/show.go index da6918f228c95..3786bbc5163bb 100644 --- a/pkg/executor/show.go +++ b/pkg/executor/show.go @@ -214,7 +214,7 @@ func (e *ShowExec) fetchAll(ctx context.Context) error { case ast.ShowStatus: return e.fetchShowStatus() case ast.ShowTables: - return e.fetchShowTables() + return e.fetchShowTables(ctx) case ast.ShowOpenTables: return e.fetchShowOpenTables() case ast.ShowTableStatus: @@ -232,25 +232,25 @@ func (e *ShowExec) fetchAll(ctx context.Context) error { case ast.ShowEvents: // empty result case ast.ShowStatsExtended: - return e.fetchShowStatsExtended() + return e.fetchShowStatsExtended(ctx) case ast.ShowStatsMeta: - return e.fetchShowStatsMeta() + return e.fetchShowStatsMeta(ctx) case ast.ShowStatsHistograms: - return e.fetchShowStatsHistogram() + return e.fetchShowStatsHistogram(ctx) case ast.ShowStatsBuckets: - return e.fetchShowStatsBuckets() + return e.fetchShowStatsBuckets(ctx) case ast.ShowStatsTopN: - return e.fetchShowStatsTopN() + return e.fetchShowStatsTopN(ctx) case ast.ShowStatsHealthy: - e.fetchShowStatsHealthy() + e.fetchShowStatsHealthy(ctx) return nil case ast.ShowStatsLocked: - return e.fetchShowStatsLocked() + return e.fetchShowStatsLocked(ctx) case ast.ShowHistogramsInFlight: e.fetchShowHistogramsInFlight() return nil case ast.ShowColumnStatsUsage: - return e.fetchShowColumnStatsUsage() + return e.fetchShowColumnStatsUsage(ctx) case ast.ShowPlugins: return e.fetchShowPlugins() case ast.ShowProfiles: @@ -493,7 +493,7 @@ func (*ShowExec) fetchShowOpenTables() error { return nil } -func (e *ShowExec) fetchShowTables() error { +func (e *ShowExec) fetchShowTables(ctx context.Context) error { checker := privilege.GetPrivilegeManager(e.Ctx()) if checker != nil && e.Ctx().GetSessionVars().User != nil { if !checker.DBIsVisible(e.Ctx().GetSessionVars().ActiveRoles, e.DBName.O) { @@ -504,7 +504,7 @@ func (e *ShowExec) fetchShowTables() error { return exeerrors.ErrBadDB.GenWithStackByArgs(e.DBName) } // sort for tables - schemaTables := e.is.SchemaTables(e.DBName) + schemaTables := e.is.SchemaTableInfos(ctx, e.DBName) tableNames := make([]string, 0, len(schemaTables)) activeRoles := e.Ctx().GetSessionVars().ActiveRoles var ( @@ -520,22 +520,22 @@ func (e *ShowExec) fetchShowTables() error { for _, v := range schemaTables { // Test with mysql.AllPrivMask means any privilege would be OK. // TODO: Should consider column privileges, which also make a table visible. - if checker != nil && !checker.RequestVerification(activeRoles, e.DBName.O, v.Meta().Name.O, "", mysql.AllPrivMask) { + if checker != nil && !checker.RequestVerification(activeRoles, e.DBName.O, v.Name.O, "", mysql.AllPrivMask) { continue - } else if fieldFilter != "" && v.Meta().Name.L != fieldFilter { + } else if fieldFilter != "" && v.Name.L != fieldFilter { continue - } else if fieldPatternsLike != nil && !fieldPatternsLike.DoMatch(v.Meta().Name.L) { + } else if fieldPatternsLike != nil && !fieldPatternsLike.DoMatch(v.Name.L) { continue } - tableNames = append(tableNames, v.Meta().Name.O) - if v.Meta().IsView() { - tableTypes[v.Meta().Name.O] = "VIEW" - } else if v.Meta().IsSequence() { - tableTypes[v.Meta().Name.O] = "SEQUENCE" + tableNames = append(tableNames, v.Name.O) + if v.IsView() { + tableTypes[v.Name.O] = "VIEW" + } else if v.IsSequence() { + tableTypes[v.Name.O] = "SEQUENCE" } else if util.IsSystemView(e.DBName.L) { - tableTypes[v.Meta().Name.O] = "SYSTEM VIEW" + tableTypes[v.Name.O] = "SYSTEM VIEW" } else { - tableTypes[v.Meta().Name.O] = "BASE TABLE" + tableTypes[v.Name.O] = "BASE TABLE" } } slices.Sort(tableNames) diff --git a/pkg/executor/show_placement.go b/pkg/executor/show_placement.go index ba69acb5f84e7..9a0eafa8c74fd 100644 --- a/pkg/executor/show_placement.go +++ b/pkg/executor/show_placement.go @@ -331,8 +331,7 @@ func (e *ShowExec) fetchAllDBPlacements(ctx context.Context, scheduleState map[i func (e *ShowExec) fetchDBScheduleState(ctx context.Context, scheduleState map[int64]infosync.PlacementScheduleState, db *model.DBInfo) (infosync.PlacementScheduleState, error) { state := infosync.PlacementScheduleStateScheduled - for _, table := range e.is.SchemaTables(db.Name) { - tbl := table.Meta() + for _, tbl := range e.is.SchemaTableInfos(ctx, db.Name) { schedule, err := fetchTableScheduleState(ctx, scheduleState, tbl) if err != nil { return state, err @@ -360,8 +359,7 @@ func (e *ShowExec) fetchAllTablePlacements(ctx context.Context, scheduleState ma for _, dbName := range dbs { tableRowSets := make([]tableRowSet, 0) - for _, tbl := range e.is.SchemaTables(dbName) { - tblInfo := tbl.Meta() + for _, tblInfo := range e.is.SchemaTableInfos(ctx, dbName) { if checker != nil && !checker.RequestVerification(activeRoles, dbName.O, tblInfo.Name.O, "", mysql.AllPrivMask) { continue } diff --git a/pkg/executor/show_stats.go b/pkg/executor/show_stats.go index aa619c3818b6a..def3afdf6340b 100644 --- a/pkg/executor/show_stats.go +++ b/pkg/executor/show_stats.go @@ -33,14 +33,13 @@ import ( "github.com/tikv/client-go/v2/oracle" ) -func (e *ShowExec) fetchShowStatsExtended() error { +func (e *ShowExec) fetchShowStatsExtended(ctx context.Context) error { do := domain.GetDomain(e.Ctx()) h := do.StatsHandle() dbs := do.InfoSchema().AllSchemaNames() for _, db := range dbs { - tables := do.InfoSchema().SchemaTables(db) + tables := do.InfoSchema().SchemaTableInfos(ctx, db) for _, tblInfo := range tables { - tblInfo := tblInfo.Meta() pi := tblInfo.GetPartitionInfo() // Extended statistics for partitioned table is not supported now. if pi != nil { @@ -107,14 +106,13 @@ func (e *ShowExec) appendTableForStatsExtended(dbName string, tbl *model.TableIn } } -func (e *ShowExec) fetchShowStatsMeta() error { +func (e *ShowExec) fetchShowStatsMeta(ctx context.Context) error { do := domain.GetDomain(e.Ctx()) h := do.StatsHandle() dbs := do.InfoSchema().AllSchemaNames() for _, db := range dbs { - tables := do.InfoSchema().SchemaTables(db) + tables := do.InfoSchema().SchemaTableInfos(ctx, db) for _, tbl := range tables { - tbl := tbl.Meta() pi := tbl.GetPartitionInfo() if pi == nil || e.Ctx().GetSessionVars().IsDynamicPartitionPruneEnabled() { partitionName := "" @@ -173,7 +171,7 @@ func (e *ShowExec) appendTableForStatsLocked(dbName, tblName, partitionName stri }) } -func (e *ShowExec) fetchShowStatsLocked() error { +func (e *ShowExec) fetchShowStatsLocked(ctx context.Context) error { do := domain.GetDomain(e.Ctx()) h := do.StatsHandle() dbs := do.InfoSchema().AllSchemaNames() @@ -186,9 +184,8 @@ func (e *ShowExec) fetchShowStatsLocked() error { tableInfo := make(map[int64]*LockedTableInfo) for _, db := range dbs { - tables := do.InfoSchema().SchemaTables(db) + tables := do.InfoSchema().SchemaTableInfos(ctx, db) for _, tbl := range tables { - tbl := tbl.Meta() pi := tbl.GetPartitionInfo() if pi == nil || e.Ctx().GetSessionVars().IsDynamicPartitionPruneEnabled() { partitionName := "" @@ -231,14 +228,13 @@ func (e *ShowExec) fetchShowStatsLocked() error { return nil } -func (e *ShowExec) fetchShowStatsHistogram() error { +func (e *ShowExec) fetchShowStatsHistogram(ctx context.Context) error { do := domain.GetDomain(e.Ctx()) h := do.StatsHandle() dbs := do.InfoSchema().AllSchemaNames() for _, db := range dbs { - tables := do.InfoSchema().SchemaTables(db) + tables := do.InfoSchema().SchemaTableInfos(ctx, db) for _, tbl := range tables { - tbl := tbl.Meta() pi := tbl.GetPartitionInfo() if pi == nil || e.Ctx().GetSessionVars().IsDynamicPartitionPruneEnabled() { partitionName := "" @@ -309,14 +305,13 @@ func (*ShowExec) versionToTime(version uint64) types.Time { return types.NewTime(types.FromGoTime(t), mysql.TypeDatetime, 0) } -func (e *ShowExec) fetchShowStatsBuckets() error { +func (e *ShowExec) fetchShowStatsBuckets(ctx context.Context) error { do := domain.GetDomain(e.Ctx()) h := do.StatsHandle() dbs := do.InfoSchema().AllSchemaNames() for _, db := range dbs { - tables := do.InfoSchema().SchemaTables(db) + tables := do.InfoSchema().SchemaTableInfos(ctx, db) for _, tbl := range tables { - tbl := tbl.Meta() pi := tbl.GetPartitionInfo() if pi == nil || e.Ctx().GetSessionVars().IsDynamicPartitionPruneEnabled() { partitionName := "" @@ -370,14 +365,13 @@ func (e *ShowExec) appendTableForStatsBuckets(dbName, tblName, partitionName str return nil } -func (e *ShowExec) fetchShowStatsTopN() error { +func (e *ShowExec) fetchShowStatsTopN(ctx context.Context) error { do := domain.GetDomain(e.Ctx()) h := do.StatsHandle() dbs := do.InfoSchema().AllSchemaNames() for _, db := range dbs { - tables := do.InfoSchema().SchemaTables(db) + tables := do.InfoSchema().SchemaTableInfos(ctx, db) for _, tbl := range tables { - tbl := tbl.Meta() pi := tbl.GetPartitionInfo() if pi == nil || e.Ctx().GetSessionVars().IsDynamicPartitionPruneEnabled() { partitionName := "" @@ -488,7 +482,7 @@ func (e *ShowExec) bucketsToRows(dbName, tblName, partitionName, colName string, return nil } -func (e *ShowExec) fetchShowStatsHealthy() { +func (e *ShowExec) fetchShowStatsHealthy(ctx context.Context) { do := domain.GetDomain(e.Ctx()) h := do.StatsHandle() dbs := do.InfoSchema().AllSchemaNames() @@ -506,9 +500,8 @@ func (e *ShowExec) fetchShowStatsHealthy() { } else if fieldPatternsLike != nil && !fieldPatternsLike.DoMatch(db.L) { continue } - tables := do.InfoSchema().SchemaTables(db) + tables := do.InfoSchema().SchemaTableInfos(ctx, db) for _, tbl := range tables { - tbl := tbl.Meta() pi := tbl.GetPartitionInfo() if pi == nil || e.Ctx().GetSessionVars().IsDynamicPartitionPruneEnabled() { partitionName := "" @@ -561,7 +554,7 @@ func (e *ShowExec) fetchShowAnalyzeStatus(ctx context.Context) error { return nil } -func (e *ShowExec) fetchShowColumnStatsUsage() error { +func (e *ShowExec) fetchShowColumnStatsUsage(ctx context.Context) error { do := domain.GetDomain(e.Ctx()) h := do.StatsHandle() colStatsMap, err := h.LoadColumnStatsUsage(e.Ctx().GetSessionVars().Location()) @@ -603,9 +596,8 @@ func (e *ShowExec) fetchShowColumnStatsUsage() error { } for _, db := range dbs { - tables := do.InfoSchema().SchemaTables(db) + tables := do.InfoSchema().SchemaTableInfos(ctx, db) for _, tbl := range tables { - tbl := tbl.Meta() pi := tbl.GetPartitionInfo() // Though partition tables in static pruning mode don't have global stats, we dump predicate columns of partitions with table ID // rather than partition ID. Hence appendTableForColumnStatsUsage needs to be called for both partition and global in both dynamic diff --git a/pkg/infoschema/context/infoschema.go b/pkg/infoschema/context/infoschema.go index 3ecbe0551f926..e9fa240f8edc1 100644 --- a/pkg/infoschema/context/infoschema.go +++ b/pkg/infoschema/context/infoschema.go @@ -15,6 +15,8 @@ package context import ( + stdctx "context" + "github.com/pingcap/tidb/pkg/ddl/placement" "github.com/pingcap/tidb/pkg/parser/model" ) @@ -33,7 +35,7 @@ type MetaOnlyInfoSchema interface { SchemaByID(id int64) (*model.DBInfo, bool) AllSchemas() []*model.DBInfo AllSchemaNames() []model.CIStr - SchemaTableInfos(schema model.CIStr) []*model.TableInfo + SchemaTableInfos(ctx stdctx.Context, schema model.CIStr) []*model.TableInfo Misc } diff --git a/pkg/infoschema/infoschema.go b/pkg/infoschema/infoschema.go index 0fdf003c5c8ad..9d2df51b7fbc6 100644 --- a/pkg/infoschema/infoschema.go +++ b/pkg/infoschema/infoschema.go @@ -318,8 +318,16 @@ func (is *infoSchema) FindTableInfoByPartitionID( } // SchemaTableInfos implements InfoSchema.FindTableInfoByPartitionID -func (is *infoSchema) SchemaTableInfos(schema model.CIStr) []*model.TableInfo { - return getTableInfoList(is.SchemaTables(schema)) +func (is *infoSchema) SchemaTableInfos(ctx stdctx.Context, schema model.CIStr) []*model.TableInfo { + schemaTables, ok := is.schemaMap[schema.L] + if !ok { + return nil + } + tables := make([]*model.TableInfo, 0, len(schemaTables.tables)) + for _, tbl := range schemaTables.tables { + tables = append(tables, tbl.Meta()) + } + return tables } type tableInfoResult struct { @@ -331,7 +339,7 @@ func (is *infoSchema) ListTablesWithSpecialAttribute(filter specialAttributeFilt ret := make([]tableInfoResult, 0, 10) for _, dbName := range is.AllSchemaNames() { res := tableInfoResult{DBName: dbName.O} - for _, tblInfo := range is.SchemaTableInfos(dbName) { + for _, tblInfo := range is.SchemaTableInfos(stdctx.Background(), dbName) { if !filter(tblInfo) { continue } @@ -366,18 +374,6 @@ func (is *infoSchema) AllSchemaNames() (schemas []model.CIStr) { return rs } -func (is *infoSchema) SchemaTables(schema model.CIStr) []table.Table { - schemaTables, ok := is.schemaMap[schema.L] - if !ok { - return nil - } - tables := make([]table.Table, 0, len(schemaTables.tables)) - for _, tbl := range schemaTables.tables { - tables = append(tables, tbl) - } - return tables -} - // FindTableByPartitionID finds the partition-table info by the partitionID. // FindTableByPartitionID will traverse all the tables to find the partitionID partition in which partition-table. func (is *infoSchema) FindTableByPartitionID(partitionID int64) (table.Table, *model.DBInfo, *model.PartitionDefinition) { diff --git a/pkg/infoschema/infoschema_test.go b/pkg/infoschema/infoschema_test.go index 04dd3496301ad..728e95488293f 100644 --- a/pkg/infoschema/infoschema_test.go +++ b/pkg/infoschema/infoschema_test.go @@ -206,16 +206,10 @@ func TestBasic(t *testing.T) { require.Nil(t, gotTblInfo) require.False(t, ok) - tbs := is.SchemaTables(dbName) - require.Len(t, tbs, 1) - - tblInfos := is.SchemaTableInfos(dbName) + tblInfos := is.SchemaTableInfos(context.Background(), dbName) require.Len(t, tblInfos, 1) require.Same(t, tbs[0].Meta(), tblInfos[0]) - tbs = is.SchemaTables(noexist) - require.Len(t, tbs, 0) - tblInfos = is.SchemaTableInfos(noexist) require.Len(t, tblInfos, 0) @@ -444,10 +438,10 @@ func TestBuildSchemaWithGlobalTemporaryTable(t *testing.T) { // full load data = infoschema.NewData() newDB, ok := newIS.SchemaByName(model.NewCIStr("test")) - tables := newIS.SchemaTables(newDB.Name) + tables := newIS.SchemaTableInfos(newDB.Name) tblInfos := make([]*model.TableInfo, 0, len(tables)) for _, table := range tables { - tblInfos = append(tblInfos, table.Meta()) + tblInfos = append(tblInfos, table) } newDB.Tables = tblInfos require.True(t, ok) @@ -577,9 +571,9 @@ func TestBuildBundle(t *testing.T) { assertBundle(is, p1.ID, p1Bundle) if len(db.Tables) == 0 { - tbls := is.SchemaTables(db.Name) + tbls := is.SchemaTableInfos(context.Background(), db.Name) for _, tbl := range tbls { - db.Tables = append(db.Tables, tbl.Meta()) + db.Tables = append(db.Tables, tbl) } } builder, err := infoschema.NewBuilder(dom, nil, infoschema.NewData()).InitWithDBInfos([]*model.DBInfo{db}, is.AllPlacementPolicies(), is.AllResourceGroups(), is.SchemaMetaVersion()) diff --git a/pkg/infoschema/infoschema_v2.go b/pkg/infoschema/infoschema_v2.go index 176896e3af026..be726524e733d 100644 --- a/pkg/infoschema/infoschema_v2.go +++ b/pkg/infoschema/infoschema_v2.go @@ -504,7 +504,7 @@ func (is *infoschemaV2) TableInfoByID(id int64) (*model.TableInfo, bool) { } // SchemaTableInfos implements InfoSchema.FindTableInfoByPartitionID -func (is *infoschemaV2) SchemaTableInfos(schema model.CIStr) []*model.TableInfo { +func (is *infoschemaV2) SchemaTableInfos(ctx context.Context, schema model.CIStr) []*model.TableInfo { if IsSpecialDB(schema.L) { raw, ok := is.Data.specials.Load(schema.L) if ok { @@ -727,57 +727,6 @@ func (is *infoschemaV2) SchemaByID(id int64) (*model.DBInfo, bool) { return is.SchemaByName(model.NewCIStr(name)) } -func (is *infoschemaV2) SchemaTables(schema model.CIStr) (tables []table.Table) { - if IsSpecialDB(schema.L) { - raw, ok := is.Data.specials.Load(schema.L) - if ok { - schTbls := raw.(*schemaTables) - tables := make([]table.Table, 0, len(schTbls.tables)) - for _, tbl := range schTbls.tables { - tables = append(tables, tbl) - } - return tables - } - } - -retry: - dbInfo, ok := is.SchemaByName(schema) - if !ok { - return - } - snapshot := is.r.Store().GetSnapshot(kv.NewVersion(is.ts)) - // Using the KV timeout read feature to address the issue of potential DDL lease expiration when - // the meta region leader is slow. - snapshot.SetOption(kv.TiKVClientReadTimeout, uint64(3000)) // 3000ms. - m := meta.NewSnapshotMeta(snapshot) - tblInfos, err := m.ListSimpleTables(dbInfo.ID) - if err != nil { - if meta.ErrDBNotExists.Equal(err) { - return nil - } - // Flashback statement could cause such kind of error. - // In theory that error should be handled in the lower layer, like client-go. - // But it's not done, so we retry here. - if strings.Contains(err.Error(), "in flashback progress") { - time.Sleep(200 * time.Millisecond) - goto retry - } - // TODO: error could happen, so do not panic! - panic(err) - } - - tables = make([]table.Table, 0, len(tblInfos)) - for _, tblInfo := range tblInfos { - tbl, ok := is.tableByID(tblInfo.ID, true) - if !ok { - // what happen? - continue - } - tables = append(tables, tbl) - } - return -} - func loadTableInfo(ctx context.Context, r autoid.Requirement, infoData *Data, tblID, dbID int64, ts uint64, schemaVersion int64) (table.Table, error) { defer tracing.StartRegion(ctx, "infoschema.loadTableInfo").End() failpoint.Inject("mockLoadTableInfoError", func(_ failpoint.Value) { @@ -986,9 +935,8 @@ func (b *Builder) applyDropSchemaV2(diff *model.SchemaDiff) []int64 { } tableIDs := make([]int64, 0, len(di.Tables)) - tables := b.infoschemaV2.SchemaTables(di.Name) - for _, table := range tables { - tbl := table.Meta() + tables := b.infoschemaV2.SchemaTableInfos(context.Background(), di.Name) + for _, tbl := range tables { tableIDs = appendAffectedIDs(tableIDs, tbl) } diff --git a/pkg/infoschema/infoschema_v2_test.go b/pkg/infoschema/infoschema_v2_test.go index dcfc37bb6d5ea..64bf3f1d2301c 100644 --- a/pkg/infoschema/infoschema_v2_test.go +++ b/pkg/infoschema/infoschema_v2_test.go @@ -45,13 +45,12 @@ func TestV2Basic(t *testing.T) { internal.AddTable(t, r.Store(), dbInfo, tblInfo) is.base().schemaMetaVersion = 1 require.Equal(t, 1, len(is.AllSchemas())) - require.Equal(t, 0, len(is.SchemaTables(is.AllSchemas()[0].Name))) ver, err := r.Store().CurrentVersion(kv.GlobalTxnScope) require.NoError(t, err) is.base().schemaMetaVersion = 2 is.ts = ver.Ver require.Equal(t, 1, len(is.AllSchemas())) - require.Equal(t, 1, len(is.SchemaTables(is.AllSchemas()[0].Name))) + require.Equal(t, 1, len(is.SchemaTableInfos(context.Background(), is.AllSchemas()[0].Name))) getDBInfo, ok := is.SchemaByName(schemaName) require.True(t, ok) @@ -98,18 +97,18 @@ func TestV2Basic(t *testing.T) { require.False(t, ok) require.Nil(t, gotTblInfo) - tables := is.SchemaTables(schemaName) + tables := is.SchemaTableInfos(context.Background(), schemaName) require.Equal(t, 1, len(tables)) require.Equal(t, tblInfo.ID, tables[0].Meta().ID) - tblInfos := is.SchemaTableInfos(schemaName) + tblInfos := is.SchemaTableInfos(context.Background(), schemaName) require.Equal(t, 1, len(tblInfos)) require.Equal(t, tables[0].Meta(), tblInfos[0]) - tables = is.SchemaTables(model.NewCIStr("notexist")) + tables = is.SchemaTableInfos(context.Background(), model.NewCIStr("notexist")) require.Equal(t, 0, len(tables)) - tblInfos = is.SchemaTableInfos(model.NewCIStr("notexist")) + tblInfos = is.SchemaTableInfos(context.Background(), model.NewCIStr("notexist")) require.Equal(t, 0, len(tblInfos)) require.Equal(t, int64(2), is.SchemaMetaVersion()) @@ -269,7 +268,7 @@ func TestBundles(t *testing.T) { _, err = builder.ApplyDiff(meta.NewMeta(txn), &model.SchemaDiff{Type: model.ActionCreateTable, Version: 2, SchemaID: dbInfo.ID, TableID: tblInfo.ID}) require.NoError(t, err) is = builder.Build(math.MaxUint64) - require.Equal(t, 1, len(is.SchemaTables(dbInfo.Name))) + require.Equal(t, 1, len(is.SchemaTableInfos(dbInfo.Name))) require.NoError(t, txn.Rollback()) // test create policy diff --git a/pkg/infoschema/interface.go b/pkg/infoschema/interface.go index 8fa0d4eed6492..0b7701532ed94 100644 --- a/pkg/infoschema/interface.go +++ b/pkg/infoschema/interface.go @@ -29,7 +29,6 @@ type InfoSchema interface { context.MetaOnlyInfoSchema TableByName(ctx stdctx.Context, schema, table model.CIStr) (table.Table, error) TableByID(id int64) (table.Table, bool) - SchemaTables(schema model.CIStr) []table.Table FindTableByPartitionID(partitionID int64) (table.Table, *model.DBInfo, *model.PartitionDefinition) ListTablesWithSpecialAttribute(filter specialAttributeFilter) []tableInfoResult base() *infoSchema diff --git a/pkg/infoschema/test/clustertablestest/tables_test.go b/pkg/infoschema/test/clustertablestest/tables_test.go index add194ea7090b..c4e13cb31c056 100644 --- a/pkg/infoschema/test/clustertablestest/tables_test.go +++ b/pkg/infoschema/test/clustertablestest/tables_test.go @@ -622,17 +622,17 @@ func checkSystemSchemaTableID(t *testing.T, dom *domain.Domain, dbName string, d require.True(t, ok) require.Equal(t, dbID, db.ID) // Test for information_schema table id. - tables := is.SchemaTables(model.NewCIStr(dbName)) + tables := is.SchemaTableInfos(context.Background(), model.NewCIStr(dbName)) require.Greater(t, len(tables), 0) for _, tbl := range tables { - tid := tbl.Meta().ID - require.Greaterf(t, tid&autoid.SystemSchemaIDFlag, int64(0), "table name is %v", tbl.Meta().Name) - require.Greaterf(t, tid&^autoid.SystemSchemaIDFlag, start, "table name is %v", tbl.Meta().Name) - require.Lessf(t, tid&^autoid.SystemSchemaIDFlag, end, "table name is %v", tbl.Meta().Name) + tid := tbl.ID + require.Greaterf(t, tid&autoid.SystemSchemaIDFlag, int64(0), "table name is %v", tbl.Name) + require.Greaterf(t, tid&^autoid.SystemSchemaIDFlag, start, "table name is %v", tbl.Name) + require.Lessf(t, tid&^autoid.SystemSchemaIDFlag, end, "table name is %v", tbl.Name) name, ok := uniqueIDMap[tid] - require.Falsef(t, ok, "schema id of %v is duplicate with %v, both is %v", name, tbl.Meta().Name, tid) - uniqueIDMap[tid] = tbl.Meta().Name.O + require.Falsef(t, ok, "schema id of %v is duplicate with %v, both is %v", name, tbl.Name, tid) + uniqueIDMap[tid] = tbl.Name.O } } diff --git a/pkg/lock/lock.go b/pkg/lock/lock.go index b49ba2dec29cc..db86c3a8aa629 100644 --- a/pkg/lock/lock.go +++ b/pkg/lock/lock.go @@ -141,9 +141,9 @@ func (c *Checker) CheckLockInDB(db string, privilege mysql.PrivilegeType) error if privilege == mysql.CreatePriv { return nil } - tables := c.is.SchemaTables(model.NewCIStr(db)) + tables := c.is.SchemaTableInfos(stdctx.Background(), model.NewCIStr(db)) for _, tbl := range tables { - err := c.CheckTableLock(db, tbl.Meta().Name.L, privilege, false) + err := c.CheckTableLock(db, tbl.Name.L, privilege, false) if err != nil { return err } diff --git a/pkg/server/handler/tests/http_handler_test.go b/pkg/server/handler/tests/http_handler_test.go index f3cce0664f4ac..8021203ec3899 100644 --- a/pkg/server/handler/tests/http_handler_test.go +++ b/pkg/server/handler/tests/http_handler_test.go @@ -1130,7 +1130,7 @@ func TestWriteDBTablesData(t *testing.T) { // No table in a schema. info := infoschema.MockInfoSchema([]*model.TableInfo{}) rc := httptest.NewRecorder() - tbs := info.SchemaTableInfos(model.NewCIStr("test")) + tbs := info.SchemaTableInfos(context.Background(), model.NewCIStr("test")) require.Equal(t, 0, len(tbs)) tikvhandler.WriteDBTablesData(rc, tbs) var ti []*model.TableInfo @@ -1142,7 +1142,7 @@ func TestWriteDBTablesData(t *testing.T) { // One table in a schema. info = infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable()}) rc = httptest.NewRecorder() - tbs = info.SchemaTableInfos(model.NewCIStr("test")) + tbs = info.SchemaTableInfos(context.Background(), model.NewCIStr("test")) require.Equal(t, 1, len(tbs)) tikvhandler.WriteDBTablesData(rc, tbs) decoder = json.NewDecoder(rc.Body) @@ -1155,7 +1155,7 @@ func TestWriteDBTablesData(t *testing.T) { // Two tables in a schema. info = infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) rc = httptest.NewRecorder() - tbs = info.SchemaTableInfos(model.NewCIStr("test")) + tbs = info.SchemaTableInfos(context.Background(), model.NewCIStr("test")) require.Equal(t, 2, len(tbs)) tikvhandler.WriteDBTablesData(rc, tbs) decoder = json.NewDecoder(rc.Body) diff --git a/pkg/server/handler/tikvhandler/tikv_handler.go b/pkg/server/handler/tikvhandler/tikv_handler.go index 5dca21bd53e4f..07827556d4592 100644 --- a/pkg/server/handler/tikvhandler/tikv_handler.go +++ b/pkg/server/handler/tikvhandler/tikv_handler.go @@ -643,13 +643,13 @@ func (h FlashReplicaHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) return } replicaInfos := make([]*TableFlashReplicaInfo, 0) - allDBs := schema.AllSchemaNames() - for _, db := range allDBs { - tbls := schema.SchemaTables(db) - for _, tbl := range tbls { - replicaInfos = h.getTiFlashReplicaInfo(tbl.Meta(), replicaInfos) + tableInfoRes := schema.ListTablesWithSpecialAttribute(infoschema.TiFlashAttribute) + for _, res := range tableInfoRes { + for _, tbl := range res.TableInfos { + replicaInfos = h.getTiFlashReplicaInfo(tbl, replicaInfos) } } + dropedOrTruncateReplicaInfos, err := h.getDropOrTruncateTableTiflash(schema) if err != nil { handler.WriteError(w, err) @@ -987,7 +987,7 @@ func (h SchemaHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { } // all table schemas in a specified database if schema.SchemaExists(cDBName) { - tbs := schema.SchemaTableInfos(cDBName) + tbs := schema.SchemaTableInfos(context.Background(), cDBName) WriteDBTablesData(w, tbs) return } @@ -1481,9 +1481,9 @@ func (h RegionHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { if util.IsMemDB(dbName.L) { continue } - tables := schema.SchemaTables(dbName) + tables := schema.SchemaTableInfos(context.Background(), dbName) for _, tableVal := range tables { - regionDetail.addTableInRange(dbName.String(), tableVal.Meta(), frameRange) + regionDetail.addTableInRange(dbName.String(), tableVal, frameRange) } } handler.WriteData(w, regionDetail) diff --git a/pkg/statistics/handle/autoanalyze/autoanalyze.go b/pkg/statistics/handle/autoanalyze/autoanalyze.go index 784a6e446a5a3..a1de081e60f7e 100644 --- a/pkg/statistics/handle/autoanalyze/autoanalyze.go +++ b/pkg/statistics/handle/autoanalyze/autoanalyze.go @@ -353,7 +353,7 @@ func RandomPickOneTableAndTryAutoAnalyze( continue } - tbls := is.SchemaTables(model.NewCIStr(db)) + tbls := is.SchemaTableInfos(context.Background(), model.NewCIStr(db)) // We shuffle dbs and tbls so that the order of iterating tables is random. If the order is fixed and the auto // analyze job of one table fails for some reason, it may always analyze the same table and fail again and again // when the HandleAutoAnalyze is triggered. Randomizing the order can avoid the problem. @@ -363,7 +363,7 @@ func RandomPickOneTableAndTryAutoAnalyze( }) // We need to check every partition of every table to see if it needs to be analyzed. - for _, tbl := range tbls { + for _, tblInfo := range tbls { // Sometimes the tables are too many. Auto-analyze will take too much time on it. // so we need to check the available time. if !timeutil.WithinDayTimePeriod(start, end, time.Now()) { @@ -371,11 +371,10 @@ func RandomPickOneTableAndTryAutoAnalyze( } // If table locked, skip analyze all partitions of the table. // FIXME: This check is not accurate, because other nodes may change the table lock status at any time. - if _, ok := lockedTables[tbl.Meta().ID]; ok { + if _, ok := lockedTables[tblInfo.ID]; ok { continue } - tblInfo := tbl.Meta() if tblInfo.IsView() { continue } diff --git a/pkg/statistics/handle/autoanalyze/refresher/refresher.go b/pkg/statistics/handle/autoanalyze/refresher/refresher.go index 7112ddcec7cfe..f13ffbe0ab4e2 100644 --- a/pkg/statistics/handle/autoanalyze/refresher/refresher.go +++ b/pkg/statistics/handle/autoanalyze/refresher/refresher.go @@ -15,6 +15,7 @@ package refresher import ( + "context" "time" "github.com/pingcap/tidb/pkg/infoschema" @@ -180,15 +181,14 @@ func (r *Refresher) RebuildTableAnalysisJobQueue() error { continue } - tbls := is.SchemaTables(db) + tbls := is.SchemaTableInfos(context.Background(), db) // We need to check every partition of every table to see if it needs to be analyzed. - for _, tbl := range tbls { + for _, tblInfo := range tbls { // If table locked, skip analyze all partitions of the table. - if _, ok := lockedTables[tbl.Meta().ID]; ok { + if _, ok := lockedTables[tblInfo.ID]; ok { continue } - tblInfo := tbl.Meta() if tblInfo.IsView() { continue } From 690260c14498e0147c83025b111a6748b0ed2969 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Tue, 16 Jul 2024 17:00:09 +0800 Subject: [PATCH 02/12] fix build --- pkg/executor/infoschema_reader_internal_test.go | 5 +++-- pkg/infoschema/infoschema_test.go | 8 +++++--- pkg/infoschema/infoschema_v2_test.go | 6 +++--- 3 files changed, 11 insertions(+), 8 deletions(-) diff --git a/pkg/executor/infoschema_reader_internal_test.go b/pkg/executor/infoschema_reader_internal_test.go index 6274e4f24650f..2bb33bca1050b 100644 --- a/pkg/executor/infoschema_reader_internal_test.go +++ b/pkg/executor/infoschema_reader_internal_test.go @@ -15,6 +15,7 @@ package executor import ( + "context" "testing" "github.com/pingcap/tidb/pkg/infoschema" @@ -75,7 +76,7 @@ func TestSetDataFromCheckConstraints(t *testing.T) { dbs := []model.CIStr{ model.NewCIStr("test"), } - err := mt.setDataFromCheckConstraints(sctx, dbs) + err := mt.setDataFromCheckConstraints(context.Background(), sctx, dbs) require.NoError(t, err) require.Equal(t, 1, len(mt.rows)) // 1 row @@ -138,7 +139,7 @@ func TestSetDataFromTiDBCheckConstraints(t *testing.T) { dbs := []model.CIStr{ model.NewCIStr("test"), } - err := mt.setDataFromTiDBCheckConstraints(sctx, dbs) + err := mt.setDataFromTiDBCheckConstraints(context.Background(), sctx, dbs) require.NoError(t, err) require.Equal(t, 1, len(mt.rows)) // 1 row diff --git a/pkg/infoschema/infoschema_test.go b/pkg/infoschema/infoschema_test.go index 728e95488293f..4401e8f585812 100644 --- a/pkg/infoschema/infoschema_test.go +++ b/pkg/infoschema/infoschema_test.go @@ -208,9 +208,11 @@ func TestBasic(t *testing.T) { tblInfos := is.SchemaTableInfos(context.Background(), dbName) require.Len(t, tblInfos, 1) - require.Same(t, tbs[0].Meta(), tblInfos[0]) + tbl, ok := is.TableByID(tblInfos[0].ID) + require.True(t, ok) + require.Same(t, tbl.Meta(), tblInfos[0]) - tblInfos = is.SchemaTableInfos(noexist) + tblInfos = is.SchemaTableInfos(context.Background(), noexist) require.Len(t, tblInfos, 0) // Make sure partitions table exists @@ -438,7 +440,7 @@ func TestBuildSchemaWithGlobalTemporaryTable(t *testing.T) { // full load data = infoschema.NewData() newDB, ok := newIS.SchemaByName(model.NewCIStr("test")) - tables := newIS.SchemaTableInfos(newDB.Name) + tables := newIS.SchemaTableInfos(context.Background(), newDB.Name) tblInfos := make([]*model.TableInfo, 0, len(tables)) for _, table := range tables { tblInfos = append(tblInfos, table) diff --git a/pkg/infoschema/infoschema_v2_test.go b/pkg/infoschema/infoschema_v2_test.go index 64bf3f1d2301c..d3768ba55cf8b 100644 --- a/pkg/infoschema/infoschema_v2_test.go +++ b/pkg/infoschema/infoschema_v2_test.go @@ -99,11 +99,11 @@ func TestV2Basic(t *testing.T) { tables := is.SchemaTableInfos(context.Background(), schemaName) require.Equal(t, 1, len(tables)) - require.Equal(t, tblInfo.ID, tables[0].Meta().ID) + require.Equal(t, tblInfo.ID, tables[0].ID) tblInfos := is.SchemaTableInfos(context.Background(), schemaName) require.Equal(t, 1, len(tblInfos)) - require.Equal(t, tables[0].Meta(), tblInfos[0]) + require.Equal(t, tables[0], tblInfos[0]) tables = is.SchemaTableInfos(context.Background(), model.NewCIStr("notexist")) require.Equal(t, 0, len(tables)) @@ -268,7 +268,7 @@ func TestBundles(t *testing.T) { _, err = builder.ApplyDiff(meta.NewMeta(txn), &model.SchemaDiff{Type: model.ActionCreateTable, Version: 2, SchemaID: dbInfo.ID, TableID: tblInfo.ID}) require.NoError(t, err) is = builder.Build(math.MaxUint64) - require.Equal(t, 1, len(is.SchemaTableInfos(dbInfo.Name))) + require.Equal(t, 1, len(is.SchemaTableInfos(context.Background(), dbInfo.Name))) require.NoError(t, txn.Rollback()) // test create policy From df3bf087a5606c0f4594008280330dcf527ddd11 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Tue, 16 Jul 2024 17:22:35 +0800 Subject: [PATCH 03/12] make lint happy --- pkg/executor/infoschema_reader.go | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/pkg/executor/infoschema_reader.go b/pkg/executor/infoschema_reader.go index a558d409403c2..a48c4c133b30b 100644 --- a/pkg/executor/infoschema_reader.go +++ b/pkg/executor/infoschema_reader.go @@ -1767,10 +1767,7 @@ func ensureSchemaTables(ctx context.Context, is infoschema.InfoSchema, schemaNam dbInfoRaw, _ := is.SchemaByName(dbName) dbInfo := dbInfoRaw.Clone() dbInfo.Tables = dbInfo.Tables[:0] - tbls := is.SchemaTableInfos(ctx, dbName) - for _, tbl := range tbls { - dbInfo.Tables = append(dbInfo.Tables, tbl) - } + dbInfo.Tables = is.SchemaTableInfos(ctx, dbName) res = append(res, dbInfo) } return res @@ -2463,7 +2460,7 @@ func (e *memtableRetriever) setDataFromSequences(ctx context.Context, sctx sessi } // dataForTableTiFlashReplica constructs data for table tiflash replica info. -func (e *memtableRetriever) dataForTableTiFlashReplica(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) { +func (e *memtableRetriever) dataForTableTiFlashReplica(_ context.Context, sctx sessionctx.Context, schemas []model.CIStr) { var ( checker = privilege.GetPrivilegeManager(sctx) rows [][]types.Datum From 6d1ed8b7a0a3693d96d0258133d197b5b701d283 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Wed, 17 Jul 2024 10:20:31 +0800 Subject: [PATCH 04/12] address conflict --- pkg/infoschema/context/infoschema.go | 2 +- pkg/infoschema/infoschema.go | 2 +- pkg/infoschema/infoschema_v2.go | 2 +- pkg/server/handler/tikvhandler/tikv_handler.go | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/pkg/infoschema/context/infoschema.go b/pkg/infoschema/context/infoschema.go index aef55aa7fb2d0..8d9c90e78095e 100644 --- a/pkg/infoschema/context/infoschema.go +++ b/pkg/infoschema/context/infoschema.go @@ -35,7 +35,7 @@ type MetaOnlyInfoSchema interface { SchemaByID(id int64) (*model.DBInfo, bool) AllSchemas() []*model.DBInfo AllSchemaNames() []model.CIStr - SchemaTableInfos(schema model.CIStr) []*model.TableInfo + SchemaTableInfos(ctx stdctx.Context, schema model.CIStr) []*model.TableInfo SchemaSimpleTableInfos(ctx stdctx.Context, schema model.CIStr) []*model.TableNameInfo Misc } diff --git a/pkg/infoschema/infoschema.go b/pkg/infoschema/infoschema.go index ee3853b9a491a..9872a26cbc779 100644 --- a/pkg/infoschema/infoschema.go +++ b/pkg/infoschema/infoschema.go @@ -331,7 +331,7 @@ func (is *infoSchema) SchemaTableInfos(ctx stdctx.Context, schema model.CIStr) [ } // SchemaSimpleTableInfos implements MetaOnlyInfoSchema. -func (is *infoSchema) SchemaSimpleTableInfos(schema model.CIStr) []*model.TableNameInfo { +func (is *infoSchema) SchemaSimpleTableInfos(ctx stdctx.Context, schema model.CIStr) []*model.TableNameInfo { schemaTables, ok := is.schemaMap[schema.L] if !ok { return nil diff --git a/pkg/infoschema/infoschema_v2.go b/pkg/infoschema/infoschema_v2.go index 75453fea73b8a..7700fc7fefec9 100644 --- a/pkg/infoschema/infoschema_v2.go +++ b/pkg/infoschema/infoschema_v2.go @@ -547,7 +547,7 @@ retry: } // SchemaSimpleTableInfos implements MetaOnlyInfoSchema. -func (is *infoschemaV2) SchemaSimpleTableInfos(schema model.CIStr) []*model.TableNameInfo { +func (is *infoschemaV2) SchemaSimpleTableInfos(ctx context.Context, schema model.CIStr) []*model.TableNameInfo { if IsSpecialDB(schema.L) { raw, ok := is.Data.specials.Load(schema.L) if ok { diff --git a/pkg/server/handler/tikvhandler/tikv_handler.go b/pkg/server/handler/tikvhandler/tikv_handler.go index 531eabb9c48f5..6ae8844cffada 100644 --- a/pkg/server/handler/tikvhandler/tikv_handler.go +++ b/pkg/server/handler/tikvhandler/tikv_handler.go @@ -1008,7 +1008,7 @@ func (h SchemaHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { // all table schemas in a specified database if schema.SchemaExists(cDBName) { if a := req.FormValue(handler.IDNameOnly); a == "true" { - tbs := schema.SchemaSimpleTableInfos(cDBName) + tbs := schema.SchemaSimpleTableInfos(context.Background(), cDBName) writeDBSimpleTablesData(w, tbs) return } From 9910abbad3476abb7deabe64076515e2dad36ba0 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Wed, 17 Jul 2024 10:40:41 +0800 Subject: [PATCH 05/12] make lint happy --- pkg/infoschema/infoschema_test.go | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/pkg/infoschema/infoschema_test.go b/pkg/infoschema/infoschema_test.go index 4401e8f585812..bfdc68b572b99 100644 --- a/pkg/infoschema/infoschema_test.go +++ b/pkg/infoschema/infoschema_test.go @@ -440,11 +440,7 @@ func TestBuildSchemaWithGlobalTemporaryTable(t *testing.T) { // full load data = infoschema.NewData() newDB, ok := newIS.SchemaByName(model.NewCIStr("test")) - tables := newIS.SchemaTableInfos(context.Background(), newDB.Name) - tblInfos := make([]*model.TableInfo, 0, len(tables)) - for _, table := range tables { - tblInfos = append(tblInfos, table) - } + tblInfos := newIS.SchemaTableInfos(context.Background(), newDB.Name) newDB.Tables = tblInfos require.True(t, ok) builder, err = infoschema.NewBuilder(re, nil, data).InitWithDBInfos([]*model.DBInfo{newDB}, newIS.AllPlacementPolicies(), newIS.AllResourceGroups(), newIS.SchemaMetaVersion()) @@ -573,10 +569,7 @@ func TestBuildBundle(t *testing.T) { assertBundle(is, p1.ID, p1Bundle) if len(db.Tables) == 0 { - tbls := is.SchemaTableInfos(context.Background(), db.Name) - for _, tbl := range tbls { - db.Tables = append(db.Tables, tbl) - } + db.Tables = is.SchemaTableInfos(context.Background(), db.Name) } builder, err := infoschema.NewBuilder(dom, nil, infoschema.NewData()).InitWithDBInfos([]*model.DBInfo{db}, is.AllPlacementPolicies(), is.AllResourceGroups(), is.SchemaMetaVersion()) require.NoError(t, err) From f9974e567597e183620fc74fc617ea73bd812e3c Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Wed, 17 Jul 2024 13:49:52 +0800 Subject: [PATCH 06/12] fix ci --- .../test/clustertablestest/BUILD.bazel | 1 + .../test/clustertablestest/tables_test.go | 22 ++++++++++++++++++- 2 files changed, 22 insertions(+), 1 deletion(-) diff --git a/pkg/infoschema/test/clustertablestest/BUILD.bazel b/pkg/infoschema/test/clustertablestest/BUILD.bazel index a1782d2c1b325..05d801917705c 100644 --- a/pkg/infoschema/test/clustertablestest/BUILD.bazel +++ b/pkg/infoschema/test/clustertablestest/BUILD.bazel @@ -18,6 +18,7 @@ go_test( "//pkg/infoschema", "//pkg/infoschema/internal", "//pkg/kv", + "//pkg/meta", "//pkg/meta/autoid", "//pkg/parser", "//pkg/parser/auth", diff --git a/pkg/infoschema/test/clustertablestest/tables_test.go b/pkg/infoschema/test/clustertablestest/tables_test.go index c4e13cb31c056..467f0cdbc9b99 100644 --- a/pkg/infoschema/test/clustertablestest/tables_test.go +++ b/pkg/infoschema/test/clustertablestest/tables_test.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/infoschema/internal" "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/auth" @@ -636,6 +637,15 @@ func checkSystemSchemaTableID(t *testing.T, dom *domain.Domain, dbName string, d } } +func updateTableMeta(t *testing.T, store kv.Storage, dbID int64, tableInfo *model.TableInfo) { + ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) + err := kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error { + m := meta.NewMeta(txn) + return m.UpdateTable(dbID, tableInfo) + }) + require.NoError(t, err) +} + func TestSelectHiddenColumn(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) @@ -647,18 +657,28 @@ func TestSelectHiddenColumn(t *testing.T) { tk.MustQuery("select count(*) from INFORMATION_SCHEMA.COLUMNS where table_name = 'hidden'").Check(testkit.Rows("3")) tb, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test_hidden"), model.NewCIStr("hidden")) require.NoError(t, err) - colInfo := tb.Meta().Columns + tbInfo := tb.Meta() + colInfo := tbInfo.Columns + // Set column b to hidden colInfo[1].Hidden = true + updateTableMeta(t, store, tbInfo.DBID, tbInfo) + tk.MustQuery("select count(*) from INFORMATION_SCHEMA.COLUMNS where table_name = 'hidden'").Check(testkit.Rows("2")) tk.MustQuery("select count(*) from INFORMATION_SCHEMA.COLUMNS where table_name = 'hidden' and column_name = 'b'").Check(testkit.Rows("0")) + // Set column b to visible colInfo[1].Hidden = false + updateTableMeta(t, store, tbInfo.DBID, tbInfo) + tk.MustQuery("select count(*) from INFORMATION_SCHEMA.COLUMNS where table_name = 'hidden' and column_name = 'b'").Check(testkit.Rows("1")) + // Set a, b ,c to hidden colInfo[0].Hidden = true colInfo[1].Hidden = true colInfo[2].Hidden = true + updateTableMeta(t, store, tbInfo.DBID, tbInfo) + tk.MustQuery("select count(*) from INFORMATION_SCHEMA.COLUMNS where table_name = 'hidden'").Check(testkit.Rows("0")) } From 748665cdccedbda0d9e73323693b3bc617a0860c Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 18 Jul 2024 11:13:39 +0800 Subject: [PATCH 07/12] return error for SchemaTableInfos() API --- br/pkg/restore/log_client/client.go | 6 +- br/pkg/task/restore_data.go | 6 +- pkg/ddl/ddl_api.go | 12 +- pkg/ddl/foreign_key.go | 5 +- pkg/ddl/placement_policy.go | 6 +- pkg/executor/infoschema_reader.go | 120 +++++++++++++----- pkg/executor/show.go | 5 +- pkg/executor/show_placement.go | 13 +- pkg/executor/show_stats.go | 39 ++++-- pkg/infoschema/context/infoschema.go | 2 +- pkg/infoschema/infoschema.go | 11 +- pkg/infoschema/infoschema_test.go | 13 +- pkg/infoschema/infoschema_v2.go | 19 +-- pkg/infoschema/infoschema_v2_test.go | 20 ++- .../test/clustertablestest/tables_test.go | 3 +- pkg/lock/lock.go | 5 +- pkg/server/handler/tests/http_handler_test.go | 9 +- .../handler/tikvhandler/tikv_handler.go | 12 +- pkg/statistics/handle/autoanalyze/BUILD.bazel | 1 + .../handle/autoanalyze/autoanalyze.go | 4 +- .../handle/autoanalyze/refresher/refresher.go | 5 +- 21 files changed, 232 insertions(+), 84 deletions(-) diff --git a/br/pkg/restore/log_client/client.go b/br/pkg/restore/log_client/client.go index ec85c1ef78202..656028dfb6b82 100644 --- a/br/pkg/restore/log_client/client.go +++ b/br/pkg/restore/log_client/client.go @@ -683,7 +683,11 @@ func (rc *LogClient) InitSchemasReplaceForDDL( info := rc.dom.InfoSchema() shcemas := info.AllSchemaNames() for _, schema := range shcemas { - for _, tableInfo := range info.SchemaTableInfos(ctx, schema) { + tblInfos, err := info.SchemaTableInfos(ctx, schema) + if err != nil { + return nil, errors.Trace(err) + } + for _, tableInfo := range tblInfos { if tableInfo.TiFlashReplica != nil && tableInfo.TiFlashReplica.Count > 0 { return nil, errors.Errorf("exist table(s) have tiflash replica, please remove it before restore") } diff --git a/br/pkg/task/restore_data.go b/br/pkg/task/restore_data.go index db9cd7d10e949..57dc184fa182a 100644 --- a/br/pkg/task/restore_data.go +++ b/br/pkg/task/restore_data.go @@ -182,7 +182,11 @@ func resetTiFlashReplicas(ctx context.Context, g glue.Glue, storage kv.Storage, expectTiFlashStoreCount := uint64(0) needTiFlash := false for _, s := range allSchemaName { - for _, t := range info.SchemaTableInfos(ctx, s) { + tblInfos, err := info.SchemaTableInfos(ctx, s) + if err != nil { + return errors.Trace(err) + } + for _, t := range tblInfos { if t.TiFlashReplica != nil { expectTiFlashStoreCount = max(expectTiFlashStoreCount, t.TiFlashReplica.Count) recorder.AddTable(t.ID, *t.TiFlashReplica) diff --git a/pkg/ddl/ddl_api.go b/pkg/ddl/ddl_api.go index bda10fa34be53..955e69bfb1156 100644 --- a/pkg/ddl/ddl_api.go +++ b/pkg/ddl/ddl_api.go @@ -398,7 +398,10 @@ func (d *ddl) ModifySchemaSetTiFlashReplica(sctx sessionctx.Context, stmt *ast.A return errors.Trace(dbterror.ErrUnsupportedTiFlashOperationForSysOrMemTable) } - tbls := is.SchemaTableInfos(context.Background(), dbInfo.Name) + tbls, err := is.SchemaTableInfos(context.Background(), dbInfo.Name) + if err != nil { + return errors.Trace(err) + } total := len(tbls) succ := 0 skip := 0 @@ -408,7 +411,7 @@ func (d *ddl) ModifySchemaSetTiFlashReplica(sctx sessionctx.Context, stmt *ast.A if total == 0 { return infoschema.ErrEmptyDatabase.GenWithStack("Empty database '%v'", dbName.O) } - err := checkTiFlashReplicaCount(sctx, tiflashReplica.Count) + err = checkTiFlashReplicaCount(sctx, tiflashReplica.Count) if err != nil { return errors.Trace(err) } @@ -699,7 +702,10 @@ func (d *ddl) DropSchema(ctx sessionctx.Context, stmt *ast.DropDatabaseStmt) (er return nil } // Clear table locks hold by the session. - tbs := is.SchemaTableInfos(d.ctx, stmt.Name) + tbs, err := is.SchemaTableInfos(d.ctx, stmt.Name) + if err != nil { + return errors.Trace(err) + } lockTableIDs := make([]int64, 0) for _, tb := range tbs { if ok, _ := ctx.CheckTableLocked(tb.ID); ok { diff --git a/pkg/ddl/foreign_key.go b/pkg/ddl/foreign_key.go index bd744c7566342..62308440a7099 100644 --- a/pkg/ddl/foreign_key.go +++ b/pkg/ddl/foreign_key.go @@ -605,7 +605,10 @@ func checkDatabaseHasForeignKeyReferred(ctx context.Context, is infoschema.InfoS if !fkCheck { return nil } - tables := is.SchemaTableInfos(ctx, schema) + tables, err := is.SchemaTableInfos(ctx, schema) + if err != nil { + return errors.Trace(err) + } tableNames := make([]ast.Ident, len(tables)) for i := range tables { tableNames[i] = ast.Ident{Schema: schema, Name: tables[i].Name} diff --git a/pkg/ddl/placement_policy.go b/pkg/ddl/placement_policy.go index 7ace7811dc86d..2277ab3199fe2 100644 --- a/pkg/ddl/placement_policy.go +++ b/pkg/ddl/placement_policy.go @@ -364,7 +364,11 @@ func CheckPlacementPolicyNotInUseFromInfoSchema(is infoschema.InfoSchema, policy return dbterror.ErrPlacementPolicyInUse.GenWithStackByArgs(policy.Name) } - for _, tblInfo := range is.SchemaTableInfos(context.Background(), dbInfo.Name) { + tblInfos, err := is.SchemaTableInfos(context.Background(), dbInfo.Name) + if err != nil { + return errors.Trace(err) + } + for _, tblInfo := range tblInfos { if err := checkPlacementPolicyNotUsedByTable(tblInfo, policy); err != nil { return err } diff --git a/pkg/executor/infoschema_reader.go b/pkg/executor/infoschema_reader.go index 652df56e2664d..d729a2e6ad677 100644 --- a/pkg/executor/infoschema_reader.go +++ b/pkg/executor/infoschema_reader.go @@ -47,6 +47,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/parser/terror" plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/privilege" @@ -121,13 +122,13 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex case infoschema.TableSchemata: e.setDataFromSchemata(sctx, dbs) case infoschema.TableStatistics: - e.setDataForStatistics(ctx, sctx, dbs) + err = e.setDataForStatistics(ctx, sctx, dbs) case infoschema.TableTables: err = e.setDataFromTables(ctx, sctx, dbs) case infoschema.TableReferConst: err = e.setDataFromReferConst(ctx, sctx, dbs) case infoschema.TableSequences: - e.setDataFromSequences(ctx, sctx, dbs) + err = e.setDataFromSequences(ctx, sctx, dbs) case infoschema.TablePartitions: err = e.setDataFromPartitions(ctx, sctx, dbs) case infoschema.TableClusterInfo: @@ -135,9 +136,9 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex case infoschema.TableAnalyzeStatus: err = e.setDataForAnalyzeStatus(ctx, sctx) case infoschema.TableTiDBIndexes: - e.setDataFromIndexes(ctx, sctx, dbs) + err = e.setDataFromIndexes(ctx, sctx, dbs) case infoschema.TableViews: - e.setDataFromViews(ctx, sctx, dbs) + err = e.setDataFromViews(ctx, sctx, dbs) case infoschema.TableEngines: e.setDataFromEngines() case infoschema.TableCharacterSets: @@ -145,7 +146,7 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex case infoschema.TableCollations: e.setDataFromCollations() case infoschema.TableKeyColumn: - e.setDataFromKeyColumnUsage(ctx, sctx, dbs) + err = e.setDataFromKeyColumnUsage(ctx, sctx, dbs) case infoschema.TableMetricTables: e.setDataForMetricTables() case infoschema.TableProfiling: @@ -163,7 +164,7 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex case infoschema.TableTiDBHotRegions: err = e.setDataForTiDBHotRegions(ctx, sctx) case infoschema.TableConstraints: - e.setDataFromTableConstraints(ctx, sctx, dbs) + err = e.setDataFromTableConstraints(ctx, sctx, dbs) case infoschema.TableSessionVar: e.rows, err = infoschema.GetDataFromSessionVariables(ctx, sctx) case infoschema.TableTiDBServersInfo: @@ -207,7 +208,7 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex case infoschema.TableKeywords: err = e.setDataFromKeywords() case infoschema.TableTiDBIndexUsage: - e.setDataFromIndexUsage(ctx, sctx, dbs) + err = e.setDataFromIndexUsage(ctx, sctx, dbs) case infoschema.ClusterTableTiDBIndexUsage: err = e.setDataForClusterIndexUsage(ctx, sctx, dbs) } @@ -372,17 +373,20 @@ func (e *memtableRetriever) setDataFromSchemata(ctx sessionctx.Context, schemas e.rows = rows } -func (e *memtableRetriever) setDataForStatistics(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) { +func (e *memtableRetriever) setDataForStatistics(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) error { checker := privilege.GetPrivilegeManager(sctx) extractor, ok := e.extractor.(*plannercore.InfoSchemaTablesExtractor) if ok && extractor.SkipRequest { - return + return nil } for _, schema := range schemas { if ok && extractor.Filter("table_schema", schema.L) { continue } - tables := e.is.SchemaTableInfos(ctx, schema) + tables, err := e.is.SchemaTableInfos(ctx, schema) + if err != nil { + return errors.Trace(err) + } for _, table := range tables { if ok && extractor.Filter("table_name", table.Name.L) { continue @@ -393,6 +397,7 @@ func (e *memtableRetriever) setDataForStatistics(ctx context.Context, sctx sessi e.setDataForStatisticsInTable(schema, table) } } + return nil } func (e *memtableRetriever) setDataForStatisticsInTable(schema model.CIStr, table *model.TableInfo) { @@ -491,7 +496,10 @@ func (e *memtableRetriever) setDataFromReferConst(ctx context.Context, sctx sess if ok && extractor.Filter("table_schema", schema.L) { continue } - tables := e.is.SchemaTableInfos(ctx, schema) + tables, err := e.is.SchemaTableInfos(ctx, schema) + if err != nil { + return errors.Trace(err) + } for _, table := range tables { if ok && extractor.Filter("table_name", table.Name.L) { continue @@ -559,7 +567,10 @@ func (e *memtableRetriever) setDataFromTables(ctx context.Context, sctx sessionc if ok && extractor.Filter("table_schema", schema.L) { continue } - tables := e.is.SchemaTableInfos(ctx, schema) + tables, err := e.is.SchemaTableInfos(ctx, schema) + if err != nil { + return errors.Trace(err) + } for _, table := range tables { if ok && extractor.Filter("table_name", table.Name.L) { continue @@ -696,7 +707,10 @@ func (e *memtableRetriever) setDataFromCheckConstraints(ctx context.Context, sct var rows [][]types.Datum checker := privilege.GetPrivilegeManager(sctx) for _, schema := range schemas { - tables := e.is.SchemaTableInfos(ctx, schema) + tables, err := e.is.SchemaTableInfos(ctx, schema) + if err != nil { + return errors.Trace(err) + } for _, table := range tables { if len(table.Constraints) > 0 { if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.L, table.Name.L, "", mysql.SelectPriv) { @@ -727,7 +741,10 @@ func (e *memtableRetriever) setDataFromTiDBCheckConstraints(ctx context.Context, var rows [][]types.Datum checker := privilege.GetPrivilegeManager(sctx) for _, schema := range schemas { - tables := e.is.SchemaTableInfos(ctx, schema) + tables, err := e.is.SchemaTableInfos(ctx, schema) + if err != nil { + return errors.Trace(err) + } for _, table := range tables { if len(table.Constraints) > 0 { if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.L, table.Name.L, "", mysql.SelectPriv) { @@ -808,7 +825,11 @@ func (e *hugeMemTableRetriever) setDataForColumns(ctx context.Context, sctx sess schema := e.dbs[e.dbsIdx] var table *model.TableInfo if len(e.curTables) == 0 { - e.curTables = e.is.SchemaTableInfos(ctx, schema.Name) + tables, err := e.is.SchemaTableInfos(ctx, schema.Name) + if err != nil { + return errors.Trace(err) + } + e.curTables = tables } for e.tblIdx < len(e.curTables) { table = e.curTables[e.tblIdx] @@ -1067,7 +1088,10 @@ func (e *memtableRetriever) setDataFromPartitions(ctx context.Context, sctx sess var rows [][]types.Datum createTimeTp := mysql.TypeDatetime for _, schema := range schemas { - tables := e.is.SchemaTableInfos(ctx, schema) + tables, err := e.is.SchemaTableInfos(ctx, schema) + if err != nil { + return errors.Trace(err) + } for _, table := range tables { if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.L, table.Name.L, "", mysql.SelectPriv) { continue @@ -1211,11 +1235,14 @@ func (e *memtableRetriever) setDataFromPartitions(ctx context.Context, sctx sess return nil } -func (e *memtableRetriever) setDataFromIndexes(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) { +func (e *memtableRetriever) setDataFromIndexes(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) error { checker := privilege.GetPrivilegeManager(sctx) var rows [][]types.Datum for _, schema := range schemas { - tables := e.is.SchemaTableInfos(ctx, schema) + tables, err := e.is.SchemaTableInfos(ctx, schema) + if err != nil { + return errors.Trace(err) + } for _, tb := range tables { if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.L, tb.Name.L, "", mysql.AllPrivMask) { continue @@ -1296,13 +1323,17 @@ func (e *memtableRetriever) setDataFromIndexes(ctx context.Context, sctx session } } e.rows = rows + return nil } -func (e *memtableRetriever) setDataFromViews(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) { +func (e *memtableRetriever) setDataFromViews(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) error { checker := privilege.GetPrivilegeManager(sctx) var rows [][]types.Datum for _, schema := range schemas { - tables := e.is.SchemaTableInfos(ctx, schema) + tables, err := e.is.SchemaTableInfos(ctx, schema) + if err != nil { + return errors.Trace(err) + } for _, table := range tables { if !table.IsView() { continue @@ -1334,6 +1365,7 @@ func (e *memtableRetriever) setDataFromViews(ctx context.Context, sctx sessionct } } e.rows = rows + return nil } func (e *memtableRetriever) dataForTiKVStoreStatus(ctx context.Context, sctx sessionctx.Context) (err error) { @@ -1585,18 +1617,21 @@ func (e *memtableRetriever) dataForTiDBClusterInfo(ctx sessionctx.Context) error return nil } -func (e *memtableRetriever) setDataFromKeyColumnUsage(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) { +func (e *memtableRetriever) setDataFromKeyColumnUsage(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) error { checker := privilege.GetPrivilegeManager(sctx) 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 + return nil } for _, schema := range schemas { if ok && extractor.Filter("table_schema", schema.L) { continue } - tables := e.is.SchemaTableInfos(ctx, schema) + tables, err := e.is.SchemaTableInfos(ctx, schema) + if err != nil { + return errors.Trace(err) + } for _, table := range tables { if ok && extractor.Filter("table_name", table.Name.L) { continue @@ -1609,6 +1644,7 @@ func (e *memtableRetriever) setDataFromKeyColumnUsage(ctx context.Context, sctx } } e.rows = rows + return nil } func (e *memtableRetriever) setDataForClusterProcessList(ctx sessionctx.Context) error { @@ -1767,7 +1803,9 @@ func ensureSchemaTables(ctx context.Context, is infoschema.InfoSchema, schemaNam dbInfoRaw, _ := is.SchemaByName(dbName) dbInfo := dbInfoRaw.Clone() dbInfo.Tables = dbInfo.Tables[:0] - dbInfo.Tables = is.SchemaTableInfos(ctx, dbName) + var err error + dbInfo.Tables, err = is.SchemaTableInfos(ctx, dbName) + terror.Log(err) res = append(res, dbInfo) } return res @@ -1987,11 +2025,14 @@ func (e *memtableRetriever) setDataForHotRegionByMetrics(metrics []helper.HotTab } // setDataFromTableConstraints constructs data for table information_schema.constraints.See https://dev.mysql.com/doc/refman/5.7/en/table-constraints-table.html -func (e *memtableRetriever) setDataFromTableConstraints(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) { +func (e *memtableRetriever) setDataFromTableConstraints(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) error { checker := privilege.GetPrivilegeManager(sctx) var rows [][]types.Datum for _, schema := range schemas { - tables := e.is.SchemaTableInfos(ctx, schema) + tables, err := e.is.SchemaTableInfos(ctx, schema) + if err != nil { + return errors.Trace(err) + } for _, tbl := range tables { if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.L, tbl.Name.L, "", mysql.AllPrivMask) { continue @@ -2046,6 +2087,7 @@ func (e *memtableRetriever) setDataFromTableConstraints(ctx context.Context, sct } } e.rows = rows + return nil } // tableStorageStatsRetriever is used to read slow log data. @@ -2131,7 +2173,10 @@ func (e *tableStorageStatsRetriever) initialize(ctx context.Context, sctx sessio for _, DB := range databases { // The user didn't specified the table, extract all tables of this db to initialTable. if len(tables) == 0 { - tbs := is.SchemaTableInfos(ctx, model.NewCIStr(DB)) + tbs, err := is.SchemaTableInfos(ctx, model.NewCIStr(DB)) + if err != nil { + return errors.Trace(err) + } for _, tb := range tbs { // For every db.table, check it's privileges. if checker(DB, tb.Name.L) { @@ -2428,11 +2473,14 @@ func (e *memtableRetriever) setDataForServersInfo(ctx sessionctx.Context) error return nil } -func (e *memtableRetriever) setDataFromSequences(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) { +func (e *memtableRetriever) setDataFromSequences(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) error { checker := privilege.GetPrivilegeManager(sctx) var rows [][]types.Datum for _, schema := range schemas { - tables := e.is.SchemaTableInfos(ctx, schema) + tables, err := e.is.SchemaTableInfos(ctx, schema) + if err != nil { + return errors.Trace(err) + } for _, table := range tables { if !table.IsSequence() { continue @@ -2457,6 +2505,7 @@ func (e *memtableRetriever) setDataFromSequences(ctx context.Context, sctx sessi } } e.rows = rows + return nil } func schemasContain(schemas []model.CIStr, schema string) bool { @@ -3574,13 +3623,16 @@ func (e *memtableRetriever) setDataFromKeywords() error { return nil } -func (e *memtableRetriever) setDataFromIndexUsage(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) { +func (e *memtableRetriever) setDataFromIndexUsage(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) error { dom := domain.GetDomain(sctx) rows := make([][]types.Datum, 0, 100) checker := privilege.GetPrivilegeManager(sctx) for _, schema := range schemas { - tables := dom.InfoSchema().SchemaTableInfos(ctx, schema) + tables, err := dom.InfoSchema().SchemaTableInfos(ctx, schema) + if err != nil { + return errors.Trace(err) + } for _, tbl := range tables { allowed := checker == nil || checker.RequestVerification( sctx.GetSessionVars().ActiveRoles, @@ -3615,10 +3667,14 @@ func (e *memtableRetriever) setDataFromIndexUsage(ctx context.Context, sctx sess } e.rows = rows + return nil } func (e *memtableRetriever) setDataForClusterIndexUsage(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) error { - e.setDataFromIndexUsage(ctx, sctx, schemas) + err := e.setDataFromIndexUsage(ctx, sctx, schemas) + if err != nil { + return errors.Trace(err) + } rows, err := infoschema.AppendHostInfoToRows(sctx, e.rows) if err != nil { return err diff --git a/pkg/executor/show.go b/pkg/executor/show.go index 3786bbc5163bb..ec8166174a896 100644 --- a/pkg/executor/show.go +++ b/pkg/executor/show.go @@ -504,7 +504,10 @@ func (e *ShowExec) fetchShowTables(ctx context.Context) error { return exeerrors.ErrBadDB.GenWithStackByArgs(e.DBName) } // sort for tables - schemaTables := e.is.SchemaTableInfos(ctx, e.DBName) + schemaTables, err := e.is.SchemaTableInfos(ctx, e.DBName) + if err != nil { + return errors.Trace(err) + } tableNames := make([]string, 0, len(schemaTables)) activeRoles := e.Ctx().GetSessionVars().ActiveRoles var ( diff --git a/pkg/executor/show_placement.go b/pkg/executor/show_placement.go index 9a0eafa8c74fd..7c955d914f780 100644 --- a/pkg/executor/show_placement.go +++ b/pkg/executor/show_placement.go @@ -331,7 +331,11 @@ func (e *ShowExec) fetchAllDBPlacements(ctx context.Context, scheduleState map[i func (e *ShowExec) fetchDBScheduleState(ctx context.Context, scheduleState map[int64]infosync.PlacementScheduleState, db *model.DBInfo) (infosync.PlacementScheduleState, error) { state := infosync.PlacementScheduleStateScheduled - for _, tbl := range e.is.SchemaTableInfos(ctx, db.Name) { + tblInfos, err := e.is.SchemaTableInfos(ctx, db.Name) + if err != nil { + return state, errors.Trace(err) + } + for _, tbl := range tblInfos { schedule, err := fetchTableScheduleState(ctx, scheduleState, tbl) if err != nil { return state, err @@ -358,8 +362,11 @@ func (e *ShowExec) fetchAllTablePlacements(ctx context.Context, scheduleState ma for _, dbName := range dbs { tableRowSets := make([]tableRowSet, 0) - - for _, tblInfo := range e.is.SchemaTableInfos(ctx, dbName) { + tblInfos, err := e.is.SchemaTableInfos(ctx, dbName) + if err != nil { + return errors.Trace(err) + } + for _, tblInfo := range tblInfos { if checker != nil && !checker.RequestVerification(activeRoles, dbName.O, tblInfo.Name.O, "", mysql.AllPrivMask) { continue } diff --git a/pkg/executor/show_stats.go b/pkg/executor/show_stats.go index def3afdf6340b..cd5030f524bfb 100644 --- a/pkg/executor/show_stats.go +++ b/pkg/executor/show_stats.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "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/cardinality" "github.com/pingcap/tidb/pkg/statistics" statsStorage "github.com/pingcap/tidb/pkg/statistics/handle/storage" @@ -38,7 +39,10 @@ func (e *ShowExec) fetchShowStatsExtended(ctx context.Context) error { h := do.StatsHandle() dbs := do.InfoSchema().AllSchemaNames() for _, db := range dbs { - tables := do.InfoSchema().SchemaTableInfos(ctx, db) + tables, err := do.InfoSchema().SchemaTableInfos(ctx, db) + if err != nil { + return errors.Trace(err) + } for _, tblInfo := range tables { pi := tblInfo.GetPartitionInfo() // Extended statistics for partitioned table is not supported now. @@ -111,7 +115,10 @@ func (e *ShowExec) fetchShowStatsMeta(ctx context.Context) error { h := do.StatsHandle() dbs := do.InfoSchema().AllSchemaNames() for _, db := range dbs { - tables := do.InfoSchema().SchemaTableInfos(ctx, db) + tables, err := do.InfoSchema().SchemaTableInfos(ctx, db) + if err != nil { + return errors.Trace(err) + } for _, tbl := range tables { pi := tbl.GetPartitionInfo() if pi == nil || e.Ctx().GetSessionVars().IsDynamicPartitionPruneEnabled() { @@ -184,7 +191,10 @@ func (e *ShowExec) fetchShowStatsLocked(ctx context.Context) error { tableInfo := make(map[int64]*LockedTableInfo) for _, db := range dbs { - tables := do.InfoSchema().SchemaTableInfos(ctx, db) + tables, err := do.InfoSchema().SchemaTableInfos(ctx, db) + if err != nil { + return errors.Trace(err) + } for _, tbl := range tables { pi := tbl.GetPartitionInfo() if pi == nil || e.Ctx().GetSessionVars().IsDynamicPartitionPruneEnabled() { @@ -233,7 +243,10 @@ func (e *ShowExec) fetchShowStatsHistogram(ctx context.Context) error { h := do.StatsHandle() dbs := do.InfoSchema().AllSchemaNames() for _, db := range dbs { - tables := do.InfoSchema().SchemaTableInfos(ctx, db) + tables, err := do.InfoSchema().SchemaTableInfos(ctx, db) + if err != nil { + return errors.Trace(err) + } for _, tbl := range tables { pi := tbl.GetPartitionInfo() if pi == nil || e.Ctx().GetSessionVars().IsDynamicPartitionPruneEnabled() { @@ -310,7 +323,10 @@ func (e *ShowExec) fetchShowStatsBuckets(ctx context.Context) error { h := do.StatsHandle() dbs := do.InfoSchema().AllSchemaNames() for _, db := range dbs { - tables := do.InfoSchema().SchemaTableInfos(ctx, db) + tables, err := do.InfoSchema().SchemaTableInfos(ctx, db) + if err != nil { + return errors.Trace(err) + } for _, tbl := range tables { pi := tbl.GetPartitionInfo() if pi == nil || e.Ctx().GetSessionVars().IsDynamicPartitionPruneEnabled() { @@ -370,7 +386,10 @@ func (e *ShowExec) fetchShowStatsTopN(ctx context.Context) error { h := do.StatsHandle() dbs := do.InfoSchema().AllSchemaNames() for _, db := range dbs { - tables := do.InfoSchema().SchemaTableInfos(ctx, db) + tables, err := do.InfoSchema().SchemaTableInfos(ctx, db) + if err != nil { + return errors.Trace(err) + } for _, tbl := range tables { pi := tbl.GetPartitionInfo() if pi == nil || e.Ctx().GetSessionVars().IsDynamicPartitionPruneEnabled() { @@ -500,7 +519,8 @@ func (e *ShowExec) fetchShowStatsHealthy(ctx context.Context) { } else if fieldPatternsLike != nil && !fieldPatternsLike.DoMatch(db.L) { continue } - tables := do.InfoSchema().SchemaTableInfos(ctx, db) + tables, err := do.InfoSchema().SchemaTableInfos(ctx, db) + terror.Log(err) for _, tbl := range tables { pi := tbl.GetPartitionInfo() if pi == nil || e.Ctx().GetSessionVars().IsDynamicPartitionPruneEnabled() { @@ -596,7 +616,10 @@ func (e *ShowExec) fetchShowColumnStatsUsage(ctx context.Context) error { } for _, db := range dbs { - tables := do.InfoSchema().SchemaTableInfos(ctx, db) + tables, err := do.InfoSchema().SchemaTableInfos(ctx, db) + if err != nil { + return errors.Trace(err) + } for _, tbl := range tables { pi := tbl.GetPartitionInfo() // Though partition tables in static pruning mode don't have global stats, we dump predicate columns of partitions with table ID diff --git a/pkg/infoschema/context/infoschema.go b/pkg/infoschema/context/infoschema.go index 8d9c90e78095e..1796c20428f42 100644 --- a/pkg/infoschema/context/infoschema.go +++ b/pkg/infoschema/context/infoschema.go @@ -35,7 +35,7 @@ type MetaOnlyInfoSchema interface { SchemaByID(id int64) (*model.DBInfo, bool) AllSchemas() []*model.DBInfo AllSchemaNames() []model.CIStr - SchemaTableInfos(ctx stdctx.Context, schema model.CIStr) []*model.TableInfo + SchemaTableInfos(ctx stdctx.Context, schema model.CIStr) ([]*model.TableInfo, error) SchemaSimpleTableInfos(ctx stdctx.Context, schema model.CIStr) []*model.TableNameInfo Misc } diff --git a/pkg/infoschema/infoschema.go b/pkg/infoschema/infoschema.go index 9872a26cbc779..8fb86d5a445a8 100644 --- a/pkg/infoschema/infoschema.go +++ b/pkg/infoschema/infoschema.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/meta/autoid" "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/table" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/intest" @@ -318,16 +319,16 @@ func (is *infoSchema) FindTableInfoByPartitionID( } // SchemaTableInfos implements InfoSchema.FindTableInfoByPartitionID -func (is *infoSchema) SchemaTableInfos(ctx stdctx.Context, schema model.CIStr) []*model.TableInfo { +func (is *infoSchema) SchemaTableInfos(ctx stdctx.Context, schema model.CIStr) ([]*model.TableInfo, error) { schemaTables, ok := is.schemaMap[schema.L] if !ok { - return nil + return nil, nil } tables := make([]*model.TableInfo, 0, len(schemaTables.tables)) for _, tbl := range schemaTables.tables { tables = append(tables, tbl.Meta()) } - return tables + return tables, nil } // SchemaSimpleTableInfos implements MetaOnlyInfoSchema. @@ -355,7 +356,9 @@ func (is *infoSchema) ListTablesWithSpecialAttribute(filter specialAttributeFilt ret := make([]tableInfoResult, 0, 10) for _, dbName := range is.AllSchemaNames() { res := tableInfoResult{DBName: dbName.O} - for _, tblInfo := range is.SchemaTableInfos(stdctx.Background(), dbName) { + tblInfos, err := is.SchemaTableInfos(stdctx.Background(), dbName) + terror.Log(err) + for _, tblInfo := range tblInfos { if !filter(tblInfo) { continue } diff --git a/pkg/infoschema/infoschema_test.go b/pkg/infoschema/infoschema_test.go index bfdc68b572b99..d052312df8b94 100644 --- a/pkg/infoschema/infoschema_test.go +++ b/pkg/infoschema/infoschema_test.go @@ -206,13 +206,15 @@ func TestBasic(t *testing.T) { require.Nil(t, gotTblInfo) require.False(t, ok) - tblInfos := is.SchemaTableInfos(context.Background(), dbName) + tblInfos, err := is.SchemaTableInfos(context.Background(), dbName) + require.NoError(t, err) require.Len(t, tblInfos, 1) tbl, ok := is.TableByID(tblInfos[0].ID) require.True(t, ok) require.Same(t, tbl.Meta(), tblInfos[0]) - tblInfos = is.SchemaTableInfos(context.Background(), noexist) + tblInfos, err = is.SchemaTableInfos(context.Background(), noexist) + require.NoError(t, err) require.Len(t, tblInfos, 0) // Make sure partitions table exists @@ -440,7 +442,8 @@ func TestBuildSchemaWithGlobalTemporaryTable(t *testing.T) { // full load data = infoschema.NewData() newDB, ok := newIS.SchemaByName(model.NewCIStr("test")) - tblInfos := newIS.SchemaTableInfos(context.Background(), newDB.Name) + tblInfos, err := newIS.SchemaTableInfos(context.Background(), newDB.Name) + require.NoError(t, err) newDB.Tables = tblInfos require.True(t, ok) builder, err = infoschema.NewBuilder(re, nil, data).InitWithDBInfos([]*model.DBInfo{newDB}, newIS.AllPlacementPolicies(), newIS.AllResourceGroups(), newIS.SchemaMetaVersion()) @@ -569,7 +572,9 @@ func TestBuildBundle(t *testing.T) { assertBundle(is, p1.ID, p1Bundle) if len(db.Tables) == 0 { - db.Tables = is.SchemaTableInfos(context.Background(), db.Name) + var err error + db.Tables, err = is.SchemaTableInfos(context.Background(), db.Name) + require.NoError(t, err) } builder, err := infoschema.NewBuilder(dom, nil, infoschema.NewData()).InitWithDBInfos([]*model.DBInfo{db}, is.AllPlacementPolicies(), is.AllResourceGroups(), is.SchemaMetaVersion()) require.NoError(t, err) diff --git a/pkg/infoschema/infoschema_v2.go b/pkg/infoschema/infoschema_v2.go index 7700fc7fefec9..810d301e5e84c 100644 --- a/pkg/infoschema/infoschema_v2.go +++ b/pkg/infoschema/infoschema_v2.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/metrics" "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/util" @@ -504,7 +505,7 @@ func (is *infoschemaV2) TableInfoByID(id int64) (*model.TableInfo, bool) { } // SchemaTableInfos implements MetaOnlyInfoSchema. -func (is *infoschemaV2) SchemaTableInfos(ctx context.Context, schema model.CIStr) []*model.TableInfo { +func (is *infoschemaV2) SchemaTableInfos(ctx context.Context, schema model.CIStr) ([]*model.TableInfo, error) { if IsSpecialDB(schema.L) { raw, ok := is.Data.specials.Load(schema.L) if ok { @@ -513,15 +514,15 @@ func (is *infoschemaV2) SchemaTableInfos(ctx context.Context, schema model.CIStr for _, tbl := range schTbls.tables { tables = append(tables, tbl) } - return getTableInfoList(tables) + return getTableInfoList(tables), nil } - return nil // something wrong? + return nil, nil // something wrong? } retry: dbInfo, ok := is.SchemaByName(schema) if !ok { - return nil + return nil, nil } snapshot := is.r.Store().GetSnapshot(kv.NewVersion(is.ts)) // Using the KV timeout read feature to address the issue of potential DDL lease expiration when @@ -531,7 +532,7 @@ retry: tblInfos, err := m.ListTables(dbInfo.ID) if err != nil { if meta.ErrDBNotExists.Equal(err) { - return nil + return nil, nil } // Flashback statement could cause such kind of error. // In theory that error should be handled in the lower layer, like client-go. @@ -540,10 +541,9 @@ retry: time.Sleep(200 * time.Millisecond) goto retry } - // TODO: error could happen, so do not panic! - panic(err) + return nil, errors.Trace(err) } - return tblInfos + return tblInfos, nil } // SchemaSimpleTableInfos implements MetaOnlyInfoSchema. @@ -981,7 +981,8 @@ func (b *Builder) applyDropSchemaV2(diff *model.SchemaDiff) []int64 { } tableIDs := make([]int64, 0, len(di.Tables)) - tables := b.infoschemaV2.SchemaTableInfos(context.Background(), di.Name) + tables, err := b.infoschemaV2.SchemaTableInfos(context.Background(), di.Name) + terror.Log(err) for _, tbl := range tables { tableIDs = appendAffectedIDs(tableIDs, tbl) } diff --git a/pkg/infoschema/infoschema_v2_test.go b/pkg/infoschema/infoschema_v2_test.go index d3768ba55cf8b..4a2cf4d8bfcf9 100644 --- a/pkg/infoschema/infoschema_v2_test.go +++ b/pkg/infoschema/infoschema_v2_test.go @@ -50,7 +50,9 @@ func TestV2Basic(t *testing.T) { is.base().schemaMetaVersion = 2 is.ts = ver.Ver require.Equal(t, 1, len(is.AllSchemas())) - require.Equal(t, 1, len(is.SchemaTableInfos(context.Background(), is.AllSchemas()[0].Name))) + tblInfos, err := is.SchemaTableInfos(context.Background(), is.AllSchemas()[0].Name) + require.NoError(t, err) + require.Equal(t, 1, len(tblInfos)) getDBInfo, ok := is.SchemaByName(schemaName) require.True(t, ok) @@ -97,18 +99,22 @@ func TestV2Basic(t *testing.T) { require.False(t, ok) require.Nil(t, gotTblInfo) - tables := is.SchemaTableInfos(context.Background(), schemaName) + tables, err := is.SchemaTableInfos(context.Background(), schemaName) + require.NoError(t, err) require.Equal(t, 1, len(tables)) require.Equal(t, tblInfo.ID, tables[0].ID) - tblInfos := is.SchemaTableInfos(context.Background(), schemaName) + tblInfos, err := is.SchemaTableInfos(context.Background(), schemaName) + require.NoError(t, err) require.Equal(t, 1, len(tblInfos)) require.Equal(t, tables[0], tblInfos[0]) - tables = is.SchemaTableInfos(context.Background(), model.NewCIStr("notexist")) + tables, err = is.SchemaTableInfos(context.Background(), model.NewCIStr("notexist")) + require.NoError(t, err) require.Equal(t, 0, len(tables)) - tblInfos = is.SchemaTableInfos(context.Background(), model.NewCIStr("notexist")) + tblInfos, err = is.SchemaTableInfos(context.Background(), model.NewCIStr("notexist")) + require.NoError(t, err) require.Equal(t, 0, len(tblInfos)) require.Equal(t, int64(2), is.SchemaMetaVersion()) @@ -268,7 +274,9 @@ func TestBundles(t *testing.T) { _, err = builder.ApplyDiff(meta.NewMeta(txn), &model.SchemaDiff{Type: model.ActionCreateTable, Version: 2, SchemaID: dbInfo.ID, TableID: tblInfo.ID}) require.NoError(t, err) is = builder.Build(math.MaxUint64) - require.Equal(t, 1, len(is.SchemaTableInfos(context.Background(), dbInfo.Name))) + tblInfos, err := is.SchemaTableInfos(context.Background(), dbInfo.Name) + require.NoError(t, err) + require.Equal(t, 1, len(tblInfos)) require.NoError(t, txn.Rollback()) // test create policy diff --git a/pkg/infoschema/test/clustertablestest/tables_test.go b/pkg/infoschema/test/clustertablestest/tables_test.go index 467f0cdbc9b99..2a8225566ab25 100644 --- a/pkg/infoschema/test/clustertablestest/tables_test.go +++ b/pkg/infoschema/test/clustertablestest/tables_test.go @@ -623,7 +623,8 @@ func checkSystemSchemaTableID(t *testing.T, dom *domain.Domain, dbName string, d require.True(t, ok) require.Equal(t, dbID, db.ID) // Test for information_schema table id. - tables := is.SchemaTableInfos(context.Background(), model.NewCIStr(dbName)) + tables, err := is.SchemaTableInfos(context.Background(), model.NewCIStr(dbName)) + require.NoError(t, err) require.Greater(t, len(tables), 0) for _, tbl := range tables { tid := tbl.ID diff --git a/pkg/lock/lock.go b/pkg/lock/lock.go index db86c3a8aa629..9aa281841475f 100644 --- a/pkg/lock/lock.go +++ b/pkg/lock/lock.go @@ -141,7 +141,10 @@ func (c *Checker) CheckLockInDB(db string, privilege mysql.PrivilegeType) error if privilege == mysql.CreatePriv { return nil } - tables := c.is.SchemaTableInfos(stdctx.Background(), model.NewCIStr(db)) + tables, err := c.is.SchemaTableInfos(stdctx.Background(), model.NewCIStr(db)) + if err != nil { + return err + } for _, tbl := range tables { err := c.CheckTableLock(db, tbl.Name.L, privilege, false) if err != nil { diff --git a/pkg/server/handler/tests/http_handler_test.go b/pkg/server/handler/tests/http_handler_test.go index 48061eb6026f9..5c030e2dbccd3 100644 --- a/pkg/server/handler/tests/http_handler_test.go +++ b/pkg/server/handler/tests/http_handler_test.go @@ -1191,7 +1191,8 @@ func TestWriteDBTablesData(t *testing.T) { // No table in a schema. info := infoschema.MockInfoSchema([]*model.TableInfo{}) rc := httptest.NewRecorder() - tbs := info.SchemaTableInfos(context.Background(), model.NewCIStr("test")) + tbs, err := info.SchemaTableInfos(context.Background(), model.NewCIStr("test")) + require.NoError(t, err) require.Equal(t, 0, len(tbs)) tikvhandler.WriteDBTablesData(rc, tbs) var ti []*model.TableInfo @@ -1203,7 +1204,8 @@ func TestWriteDBTablesData(t *testing.T) { // One table in a schema. info = infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable()}) rc = httptest.NewRecorder() - tbs = info.SchemaTableInfos(context.Background(), model.NewCIStr("test")) + tbs, err = info.SchemaTableInfos(context.Background(), model.NewCIStr("test")) + require.NoError(t, err) require.Equal(t, 1, len(tbs)) tikvhandler.WriteDBTablesData(rc, tbs) decoder = json.NewDecoder(rc.Body) @@ -1216,7 +1218,8 @@ func TestWriteDBTablesData(t *testing.T) { // Two tables in a schema. info = infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) rc = httptest.NewRecorder() - tbs = info.SchemaTableInfos(context.Background(), model.NewCIStr("test")) + tbs, err = info.SchemaTableInfos(context.Background(), model.NewCIStr("test")) + require.NoError(t, err) require.Equal(t, 2, len(tbs)) tikvhandler.WriteDBTablesData(rc, tbs) decoder = json.NewDecoder(rc.Body) diff --git a/pkg/server/handler/tikvhandler/tikv_handler.go b/pkg/server/handler/tikvhandler/tikv_handler.go index 6ae8844cffada..f39ab7ab1eed9 100644 --- a/pkg/server/handler/tikvhandler/tikv_handler.go +++ b/pkg/server/handler/tikvhandler/tikv_handler.go @@ -1012,7 +1012,11 @@ func (h SchemaHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { writeDBSimpleTablesData(w, tbs) return } - tbs := schema.SchemaTableInfos(context.Background(), cDBName) + tbs, err := schema.SchemaTableInfos(context.Background(), cDBName) + if err != nil { + handler.WriteError(w, err) + return + } WriteDBTablesData(w, tbs) return } @@ -1528,7 +1532,11 @@ func (h RegionHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { if util.IsMemDB(dbName.L) { continue } - tables := schema.SchemaTableInfos(context.Background(), dbName) + tables, err := schema.SchemaTableInfos(context.Background(), dbName) + if err != nil { + handler.WriteError(w, err) + return + } for _, tableVal := range tables { regionDetail.addTableInRange(dbName.String(), tableVal, frameRange) } diff --git a/pkg/statistics/handle/autoanalyze/BUILD.bazel b/pkg/statistics/handle/autoanalyze/BUILD.bazel index 7fa5cee5617b4..34dabaa1ed867 100644 --- a/pkg/statistics/handle/autoanalyze/BUILD.bazel +++ b/pkg/statistics/handle/autoanalyze/BUILD.bazel @@ -9,6 +9,7 @@ go_library( "//pkg/domain/infosync", "//pkg/infoschema", "//pkg/parser/model", + "//pkg/parser/terror", "//pkg/sessionctx", "//pkg/sessionctx/sysproctrack", "//pkg/sessionctx/variable", diff --git a/pkg/statistics/handle/autoanalyze/autoanalyze.go b/pkg/statistics/handle/autoanalyze/autoanalyze.go index a1de081e60f7e..f4f835bffabcf 100644 --- a/pkg/statistics/handle/autoanalyze/autoanalyze.go +++ b/pkg/statistics/handle/autoanalyze/autoanalyze.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/domain/infosync" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/sysproctrack" "github.com/pingcap/tidb/pkg/sessionctx/variable" @@ -353,7 +354,8 @@ func RandomPickOneTableAndTryAutoAnalyze( continue } - tbls := is.SchemaTableInfos(context.Background(), model.NewCIStr(db)) + tbls, err := is.SchemaTableInfos(context.Background(), model.NewCIStr(db)) + terror.Log(err) // We shuffle dbs and tbls so that the order of iterating tables is random. If the order is fixed and the auto // analyze job of one table fails for some reason, it may always analyze the same table and fail again and again // when the HandleAutoAnalyze is triggered. Randomizing the order can avoid the problem. diff --git a/pkg/statistics/handle/autoanalyze/refresher/refresher.go b/pkg/statistics/handle/autoanalyze/refresher/refresher.go index f13ffbe0ab4e2..c563bf692d72e 100644 --- a/pkg/statistics/handle/autoanalyze/refresher/refresher.go +++ b/pkg/statistics/handle/autoanalyze/refresher/refresher.go @@ -181,7 +181,10 @@ func (r *Refresher) RebuildTableAnalysisJobQueue() error { continue } - tbls := is.SchemaTableInfos(context.Background(), db) + tbls, err := is.SchemaTableInfos(context.Background(), db) + if err != nil { + return err + } // We need to check every partition of every table to see if it needs to be analyzed. for _, tblInfo := range tbls { // If table locked, skip analyze all partitions of the table. From a65b2567d368064eca0f9f362ab4fafdd81e6ae9 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 18 Jul 2024 11:48:52 +0800 Subject: [PATCH 08/12] fix build --- pkg/infoschema/infoschema_v2_test.go | 4 ++-- pkg/server/handler/tests/http_handler_test.go | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/infoschema/infoschema_v2_test.go b/pkg/infoschema/infoschema_v2_test.go index 4a2cf4d8bfcf9..23f4948495195 100644 --- a/pkg/infoschema/infoschema_v2_test.go +++ b/pkg/infoschema/infoschema_v2_test.go @@ -104,8 +104,8 @@ func TestV2Basic(t *testing.T) { require.Equal(t, 1, len(tables)) require.Equal(t, tblInfo.ID, tables[0].ID) - tblInfos, err := is.SchemaTableInfos(context.Background(), schemaName) - require.NoError(t, err) + tblInfos, err1 := is.SchemaTableInfos(context.Background(), schemaName) + require.NoError(t, err1) require.Equal(t, 1, len(tblInfos)) require.Equal(t, tables[0], tblInfos[0]) diff --git a/pkg/server/handler/tests/http_handler_test.go b/pkg/server/handler/tests/http_handler_test.go index 5c030e2dbccd3..d8d45547c4726 100644 --- a/pkg/server/handler/tests/http_handler_test.go +++ b/pkg/server/handler/tests/http_handler_test.go @@ -1197,7 +1197,7 @@ func TestWriteDBTablesData(t *testing.T) { tikvhandler.WriteDBTablesData(rc, tbs) var ti []*model.TableInfo decoder := json.NewDecoder(rc.Body) - err := decoder.Decode(&ti) + err = decoder.Decode(&ti) require.NoError(t, err) require.Equal(t, 0, len(ti)) From f3399a008587460a82de1ee5cdf77509994a181b Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 18 Jul 2024 14:21:28 +0800 Subject: [PATCH 09/12] try to fix ci --- pkg/infoschema/infoschema.go | 2 +- pkg/server/handler/tikvhandler/tikv_handler.go | 11 ++++++++--- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/pkg/infoschema/infoschema.go b/pkg/infoschema/infoschema.go index 8fb86d5a445a8..10b3174dfd18b 100644 --- a/pkg/infoschema/infoschema.go +++ b/pkg/infoschema/infoschema.go @@ -318,7 +318,7 @@ func (is *infoSchema) FindTableInfoByPartitionID( return getTableInfo(tbl), db, partDef } -// SchemaTableInfos implements InfoSchema.FindTableInfoByPartitionID +// SchemaTableInfos implements MetaOnlyInfoSchema. func (is *infoSchema) SchemaTableInfos(ctx stdctx.Context, schema model.CIStr) ([]*model.TableInfo, error) { schemaTables, ok := is.schemaMap[schema.L] if !ok { diff --git a/pkg/server/handler/tikvhandler/tikv_handler.go b/pkg/server/handler/tikvhandler/tikv_handler.go index f39ab7ab1eed9..1145c062e0b29 100644 --- a/pkg/server/handler/tikvhandler/tikv_handler.go +++ b/pkg/server/handler/tikvhandler/tikv_handler.go @@ -643,9 +643,14 @@ func (h FlashReplicaHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) return } replicaInfos := make([]*TableFlashReplicaInfo, 0) - tableInfoRes := schema.ListTablesWithSpecialAttribute(infoschema.TiFlashAttribute) - for _, res := range tableInfoRes { - for _, tbl := range res.TableInfos { + allDBs := schema.AllSchemaNames() + for _, db := range allDBs { + tbls, err := schema.SchemaTableInfos(context.Background(), db) + if err != nil { + handler.WriteError(w, err) + return + } + for _, tbl := range tbls { replicaInfos = h.getTiFlashReplicaInfo(tbl, replicaInfos) } } From b4ec4ad3fc506ab95c2331b52933eb57125efd42 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 18 Jul 2024 17:41:36 +0800 Subject: [PATCH 10/12] fix ci --- pkg/executor/infoschema_reader.go | 31 ++++++++++++------------------- 1 file changed, 12 insertions(+), 19 deletions(-) diff --git a/pkg/executor/infoschema_reader.go b/pkg/executor/infoschema_reader.go index d729a2e6ad677..ecd8539e9fa0c 100644 --- a/pkg/executor/infoschema_reader.go +++ b/pkg/executor/infoschema_reader.go @@ -170,7 +170,7 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex case infoschema.TableTiDBServersInfo: err = e.setDataForServersInfo(sctx) case infoschema.TableTiFlashReplica: - e.dataForTableTiFlashReplica(sctx, dbs) + err = e.dataForTableTiFlashReplica(ctx, sctx, dbs) case infoschema.TableTiKVStoreStatus: err = e.dataForTiKVStoreStatus(ctx, sctx) case infoschema.TableClientErrorsSummaryGlobal, @@ -2508,31 +2508,23 @@ func (e *memtableRetriever) setDataFromSequences(ctx context.Context, sctx sessi return nil } -func schemasContain(schemas []model.CIStr, schema string) bool { - for _, s := range schemas { - if s.L == schema { - return true - } - } - return false -} - // dataForTableTiFlashReplica constructs data for table tiflash replica info. -func (e *memtableRetriever) dataForTableTiFlashReplica(sctx sessionctx.Context, schemas []model.CIStr) { +func (e *memtableRetriever) dataForTableTiFlashReplica(ctx context.Context, sctx sessionctx.Context, schemas []model.CIStr) error { var ( checker = privilege.GetPrivilegeManager(sctx) rows [][]types.Datum tiFlashStores map[int64]pd.StoreInfo ) - tableInfoResult := e.is.ListTablesWithSpecialAttribute(infoschema.TiFlashAttribute) - for _, res := range tableInfoResult { - schema := res.DBName - if !schemasContain(schemas, schema) { - continue + for _, schema := range schemas { + tables, err := e.is.SchemaTableInfos(ctx, schema) + if err != nil { + return errors.Trace(err) } - - for _, tbl := range res.TableInfos { - if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema, tbl.Name.L, "", mysql.AllPrivMask) { + for _, tbl := range tables { + if tbl.TiFlashReplica == nil { + continue + } + if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.L, tbl.Name.L, "", mysql.AllPrivMask) { continue } var progress float64 @@ -2567,6 +2559,7 @@ func (e *memtableRetriever) dataForTableTiFlashReplica(sctx sessionctx.Context, } } e.rows = rows + return nil } func (e *memtableRetriever) setDataForClientErrorsSummary(ctx sessionctx.Context, tableName string) error { From c7094b3f75f5f2994b2b9d65bd22f856c73fe959 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Mon, 22 Jul 2024 16:22:02 +0800 Subject: [PATCH 11/12] ci stable --- pkg/ddl/tiflash_replica_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/ddl/tiflash_replica_test.go b/pkg/ddl/tiflash_replica_test.go index 27f7c3bb44603..fe96e93cd9e35 100644 --- a/pkg/ddl/tiflash_replica_test.go +++ b/pkg/ddl/tiflash_replica_test.go @@ -190,6 +190,7 @@ func TestInfoSchemaForTiFlashReplica(t *testing.T) { require.NoError(t, err) tbl.Meta().TiFlashReplica.Available = true updateTableMeta(t, store, tbl.Meta().DBID, tbl.Meta()) + dom.Reload() tk.MustQuery("select TABLE_SCHEMA,TABLE_NAME,REPLICA_COUNT,LOCATION_LABELS,AVAILABLE,PROGRESS from information_schema.tiflash_replica").Check(testkit.Rows("test t 2 a,b 1 0")) } From d1b5c05075efade73e2e7f6bc62e1a361c1e0494 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Mon, 22 Jul 2024 17:05:08 +0800 Subject: [PATCH 12/12] fix build after merge master --- pkg/ddl/tiflash_replica_test.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pkg/ddl/tiflash_replica_test.go b/pkg/ddl/tiflash_replica_test.go index fe96e93cd9e35..9d586b0896497 100644 --- a/pkg/ddl/tiflash_replica_test.go +++ b/pkg/ddl/tiflash_replica_test.go @@ -186,7 +186,8 @@ func TestInfoSchemaForTiFlashReplica(t *testing.T) { tk.MustExec("create table t (a int, b int, index idx(a))") tk.MustExec("alter table t set tiflash replica 2 location labels 'a','b';") tk.MustQuery("select TABLE_SCHEMA,TABLE_NAME,REPLICA_COUNT,LOCATION_LABELS,AVAILABLE,PROGRESS from information_schema.tiflash_replica").Check(testkit.Rows("test t 2 a,b 0 0")) - tbl, err := domain.GetDomain(tk.Session()).InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + dom := domain.GetDomain(tk.Session()) + tbl, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica.Available = true updateTableMeta(t, store, tbl.Meta().DBID, tbl.Meta())