From ad4f3ea0f7db1e34ae61fa2cf4a0c014d02ba5f3 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Wed, 23 Jul 2025 13:07:17 +0800 Subject: [PATCH] bootstrap: refactor the system tables/dbs creation (#62540) ref pingcap/tidb#61702 --- br/pkg/restore/log_client/client_test.go | 6 +- build/debug-linter/BUILD.bazel | 18 + build/debug-linter/main.go | 26 ++ build/linter/bootstrap/analyzer.go | 262 +++++++++----- pkg/ddl/multi_schema_change_test.go | 2 +- pkg/meta/metadef/system.go | 110 +++++- pkg/privilege/privileges/privileges_test.go | 6 +- pkg/session/BUILD.bazel | 2 +- pkg/session/bootstrap.go | 341 +++++++++--------- pkg/session/bootstrap_test.go | 22 ++ pkg/session/session.go | 6 + pkg/session/session_test.go | 20 +- pkg/session/upgrade.go | 67 ++-- ...mic_partitioned_table_analysis_job_test.go | 4 +- .../priorityqueue/interval_test.go | 4 +- ...non_partitioned_table_analysis_job_test.go | 4 +- ...tic_partitioned_table_analysis_job_test.go | 2 +- pkg/timer/BUILD.bazel | 1 + pkg/timer/store_intergartion_test.go | 15 +- pkg/timer/tablestore/sql.go | 28 -- pkg/ttl/ttlworker/BUILD.bazel | 1 + pkg/ttl/ttlworker/timer_sync_test.go | 11 +- 22 files changed, 591 insertions(+), 367 deletions(-) create mode 100644 build/debug-linter/BUILD.bazel create mode 100644 build/debug-linter/main.go diff --git a/br/pkg/restore/log_client/client_test.go b/br/pkg/restore/log_client/client_test.go index d75077c2d7..cb87e97014 100644 --- a/br/pkg/restore/log_client/client_test.go +++ b/br/pkg/restore/log_client/client_test.go @@ -1367,7 +1367,7 @@ func TestInitSchemasReplaceForDDL(t *testing.T) { { s := utiltest.CreateRestoreSchemaSuite(t) tk := testkit.NewTestKit(t, s.Mock.Storage) - tk.Exec(session.CreatePITRIDMap) + tk.Exec(session.CreateTiDBPITRIDMapTable) g := gluetidb.New() se, err := g.CreateSession(s.Mock.Storage) require.NoError(t, err) @@ -1437,7 +1437,7 @@ func TestPITRIDMap(t *testing.T) { ctx := context.Background() s := utiltest.CreateRestoreSchemaSuite(t) tk := testkit.NewTestKit(t, s.Mock.Storage) - tk.MustExec(session.CreatePITRIDMap) + tk.MustExec(session.CreateTiDBPITRIDMapTable) g := gluetidb.New() se, err := g.CreateSession(s.Mock.Storage) require.NoError(t, err) @@ -1537,7 +1537,7 @@ func TestPITRIDMapOnCheckpointStorage(t *testing.T) { ctx := context.Background() s := utiltest.CreateRestoreSchemaSuite(t) tk := testkit.NewTestKit(t, s.Mock.Storage) - tk.MustExec(session.CreatePITRIDMap) + tk.MustExec(session.CreateTiDBPITRIDMapTable) g := gluetidb.New() se, err := g.CreateSession(s.Mock.Storage) require.NoError(t, err) diff --git a/build/debug-linter/BUILD.bazel b/build/debug-linter/BUILD.bazel new file mode 100644 index 0000000000..a3cdb328af --- /dev/null +++ b/build/debug-linter/BUILD.bazel @@ -0,0 +1,18 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_binary", "go_library") + +go_library( + name = "debug-linter_lib", + srcs = ["main.go"], + importpath = "github.com/pingcap/tidb/build/debug-linter", + visibility = ["//visibility:private"], + deps = [ + "//build/linter/bootstrap", + "@org_golang_x_tools//go/analysis/singlechecker", + ], +) + +go_binary( + name = "debug-linter", + embed = [":debug-linter_lib"], + visibility = ["//visibility:public"], +) diff --git a/build/debug-linter/main.go b/build/debug-linter/main.go new file mode 100644 index 0000000000..3940e252c3 --- /dev/null +++ b/build/debug-linter/main.go @@ -0,0 +1,26 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package main + +import ( + "github.com/pingcap/tidb/build/linter/bootstrap" + "golang.org/x/tools/go/analysis/singlechecker" +) + +// THIS IS FOR DEBUG PURPOSES ONLY. +func main() { + // just change the linter here. + singlechecker.Main(bootstrap.Analyzer) +} diff --git a/build/linter/bootstrap/analyzer.go b/build/linter/bootstrap/analyzer.go index 72f12cf4f7..ae3d0db8fa 100644 --- a/build/linter/bootstrap/analyzer.go +++ b/build/linter/bootstrap/analyzer.go @@ -33,33 +33,127 @@ var Analyzer = &analysis.Analyzer{ } const ( - bootstrapCodeFile = "/upgrade.go" + bootstrapCodeFile = "/bootstrap.go" + upgradeCodeFile = "/upgrade.go" ) func run(pass *analysis.Pass) (any, error) { for _, file := range pass.Files { - if !strings.HasSuffix(pass.Fset.File(file.Pos()).Name(), bootstrapCodeFile) { - continue + if strings.HasSuffix(pass.Fset.File(file.Pos()).Name(), bootstrapCodeFile) { + checkBootstrapDotGo(pass, file) } + if strings.HasSuffix(pass.Fset.File(file.Pos()).Name(), upgradeCodeFile) { + checkUpgradeDotGo(pass, file) + } + } + return nil, nil +} - var ( - maxVerVariable int - maxVerVariablePos token.Pos - curVerVariable int - curVerVariablePos token.Pos - maxVerFunc int - maxVerFuncPos token.Pos - maxVerFuncUsed int - maxVerFuncUsedPos token.Pos - err error - ) +func checkBootstrapDotGo(pass *analysis.Pass, file *ast.File) { + var found bool + for _, decl := range file.Decls { + switch v := decl.(type) { + case *ast.GenDecl: + switch { + case len(v.Specs) == 1: + spec := v.Specs[0] + v2, ok := spec.(*ast.ValueSpec) + if !ok { + continue + } + if len(v2.Names) != 1 { + continue + } + switch v2.Names[0].Name { + case "tablesInSystemDatabase": + compositeLit := v2.Values[0].(*ast.CompositeLit) + for _, elt := range compositeLit.Elts { + idIdentPkg := elt.(*ast.CompositeLit).Elts[0].(*ast.KeyValueExpr).Value.(*ast.SelectorExpr).X.(*ast.Ident).Name + if idIdentPkg != "metadef" { + pass.Reportf(elt.Pos(), "table ID must be defined in metadef pkg, but got %q", idIdentPkg) + } + idIdentName := elt.(*ast.CompositeLit).Elts[0].(*ast.KeyValueExpr).Value.(*ast.SelectorExpr).Sel.Name + quotedName := elt.(*ast.CompositeLit).Elts[1].(*ast.KeyValueExpr).Value.(*ast.BasicLit).Value + tableName, err := strconv.Unquote(quotedName) + if err != nil { + pass.Reportf(elt.Pos(), "the name of the table in tablesInSystemDatabase must be a string literal, but got %q", quotedName) + continue + } + sqlIdentName := elt.(*ast.CompositeLit).Elts[2].(*ast.KeyValueExpr).Value.(*ast.Ident).Name + if !strings.HasSuffix(idIdentName, "TableID") { + pass.Reportf(elt.Pos(), "the name of the constant of table ID in tablesInSystemDatabase must end with TableID, but got %q", idIdentName) + } + if !strings.HasPrefix(sqlIdentName, "Create") || !strings.HasSuffix(sqlIdentName, "Table") { + pass.Reportf(elt.Pos(), "the name of the constant of the create table SQL in tablesInSystemDatabase must start 'CreateXXXTable' style, but got %q", sqlIdentName) + } + if strings.TrimSuffix(idIdentName, "TableID") != + strings.TrimSuffix(strings.TrimPrefix(sqlIdentName, "Create"), "Table") { + pass.Reportf(elt.Pos(), "the name of the constant of table ID in tablesInSystemDatabase must match the name of the create table SQL, but got %q and %q", idIdentName, sqlIdentName) + } + nameInCamel := strings.ReplaceAll(tableName, "_", "") + if strings.ToLower(strings.TrimSuffix(idIdentName, "TableID")) != nameInCamel { + pass.Reportf(elt.Pos(), "the name of the constant of table ID in tablesInSystemDatabase must match the name of the table, but got %q and %q", idIdentName, tableName) + } + } + found = true + } + } + } + } + if !found { + pass.Reportf(file.Pos(), "bootstrap.go should must have a variable named 'tablesInSystemDatabase'") + } +} - for _, decl := range file.Decls { - switch v := decl.(type) { - case *ast.GenDecl: - switch { - case len(v.Specs) == 1: - spec := v.Specs[0] +func checkUpgradeDotGo(pass *analysis.Pass, file *ast.File) { + var ( + maxVerVariable int + maxVerVariablePos token.Pos + curVerVariable int + curVerVariablePos token.Pos + maxVerFunc int + maxVerFuncPos token.Pos + maxVerFuncUsed int + maxVerFuncUsedPos token.Pos + err error + ) + + for _, decl := range file.Decls { + switch v := decl.(type) { + case *ast.GenDecl: + switch { + case len(v.Specs) == 1: + spec := v.Specs[0] + v2, ok := spec.(*ast.ValueSpec) + if !ok { + continue + } + if len(v2.Names) != 1 { + continue + } + switch v2.Names[0].Name { + case "upgradeToVerFunctions": + composeLit := v2.Values[0].(*ast.CompositeLit) + lastElm := composeLit.Elts[len(composeLit.Elts)-1] + ident := lastElm.(*ast.CompositeLit).Elts[1].(*ast.KeyValueExpr).Value.(*ast.Ident) + maxVerFuncUsed, err = strconv.Atoi(ident.Name[len("upgradeToVer"):]) + if err != nil { + panic("unexpected value of upgradeToVerFunctions: " + ident.Name) + } + maxVerFuncUsedPos = lastElm.Pos() + case "currentBootstrapVersion": + valueIdent := v2.Values[0].(*ast.Ident) + curVerVariablePos = valueIdent.Pos() + value := v2.Values[0].(*ast.Ident).Name + curVerVariable, err = strconv.Atoi(value[len("version"):]) + if err != nil { + panic("unexpected value of currentBootstrapVersion: " + value) + } + default: + continue + } + case v.Tok == token.CONST && len(v.Specs) > 1: + for _, spec := range v.Specs { v2, ok := spec.(*ast.ValueSpec) if !ok { continue @@ -67,99 +161,67 @@ func run(pass *analysis.Pass) (any, error) { if len(v2.Names) != 1 { continue } - switch v2.Names[0].Name { - case "upgradeToVerFunctions": - composeLit := v2.Values[0].(*ast.CompositeLit) - lastElm := composeLit.Elts[len(composeLit.Elts)-1] - ident := lastElm.(*ast.CompositeLit).Elts[1].(*ast.KeyValueExpr).Value.(*ast.Ident) - maxVerFuncUsed, err = strconv.Atoi(ident.Name[len("upgradeToVer"):]) - if err != nil { - panic("unexpected value of upgradeToVerFunctions: " + ident.Name) - } - maxVerFuncUsedPos = lastElm.Pos() - case "currentBootstrapVersion": - valueIdent := v2.Values[0].(*ast.Ident) - curVerVariablePos = valueIdent.Pos() - value := v2.Values[0].(*ast.Ident).Name - curVerVariable, err = strconv.Atoi(value[len("version"):]) - if err != nil { - panic("unexpected value of currentBootstrapVersion: " + value) - } - default: + name := v2.Names[0].Name + if !strings.HasPrefix(name, "version") { continue } - case v.Tok == token.CONST && len(v.Specs) > 1: - for _, spec := range v.Specs { - v2, ok := spec.(*ast.ValueSpec) - if !ok { - continue - } - if len(v2.Names) != 1 { - continue - } - name := v2.Names[0].Name - if !strings.HasPrefix(name, "version") { - continue - } - valInName, err := strconv.Atoi(name[len("version"):]) - if err != nil { - continue - } + valInName, err := strconv.Atoi(name[len("version"):]) + if err != nil { + continue + } - if valInName < maxVerVariable { - pass.Reportf(spec.Pos(), "version variable %q is not valid, we should have a increment list of version variables", name) - continue - } + if valInName < maxVerVariable { + pass.Reportf(spec.Pos(), "version variable %q is not valid, we should have a increment list of version variables", name) + continue + } - maxVerVariable = valInName - maxVerVariablePos = v2.Names[0].Pos() + maxVerVariable = valInName + maxVerVariablePos = v2.Names[0].Pos() - if len(v2.Values) != 1 { - pass.Reportf(spec.Pos(), "the value of version variable %q must be specified explicitly", name) - continue - } + if len(v2.Values) != 1 { + pass.Reportf(spec.Pos(), "the value of version variable %q must be specified explicitly", name) + continue + } - valStr := v2.Values[0].(*ast.BasicLit).Value - val, err := strconv.Atoi(valStr) - if err != nil { - pass.Reportf(spec.Pos(), "unexpected value of version variable %q: %q", name, valStr) - continue - } + valStr := v2.Values[0].(*ast.BasicLit).Value + val, err := strconv.Atoi(valStr) + if err != nil { + pass.Reportf(spec.Pos(), "unexpected value of version variable %q: %q", name, valStr) + continue + } - if val != valInName { - pass.Reportf(spec.Pos(), "the value of version variable %q must be '%d', but now is '%d'", name, valInName, val) - continue - } + if val != valInName { + pass.Reportf(spec.Pos(), "the value of version variable %q must be '%d', but now is '%d'", name, valInName, val) + continue } } - case *ast.FuncDecl: - name := v.Name.Name - if !strings.HasPrefix(name, "upgradeToVer") { - continue - } - t, err := strconv.Atoi(name[len("upgradeToVer"):]) - if err != nil { - continue - } - if t > maxVerFunc { - maxVerFunc = t - maxVerFuncPos = v.Pos() - } + } + case *ast.FuncDecl: + name := v.Name.Name + if !strings.HasPrefix(name, "upgradeToVer") { + continue + } + t, err := strconv.Atoi(name[len("upgradeToVer"):]) + if err != nil { + continue + } + if t > maxVerFunc { + maxVerFunc = t + maxVerFuncPos = v.Pos() } } - minv := min(maxVerVariable, maxVerFunc, maxVerFuncUsed, curVerVariable) - maxv := max(maxVerVariable, maxVerFunc, maxVerFuncUsed, curVerVariable) - if minv == maxv && minv != 0 { - return nil, nil - } - pass.Reportf(maxVerFuncUsedPos, "found inconsistent bootstrap versions:") - pass.Reportf(maxVerFuncUsedPos, "max version function used: %d", maxVerFuncUsed) - pass.Reportf(maxVerFuncPos, "max version function: %d", maxVerFunc) - pass.Reportf(maxVerVariablePos, "max version variable: %d", maxVerVariable) - pass.Reportf(curVerVariablePos, "current version variable: %d", curVerVariable) } - return nil, nil + minv := min(maxVerVariable, maxVerFunc, maxVerFuncUsed, curVerVariable) + maxv := max(maxVerVariable, maxVerFunc, maxVerFuncUsed, curVerVariable) + if minv == maxv && minv != 0 { + return + } + pass.Reportf(maxVerFuncUsedPos, "found inconsistent bootstrap versions:") + pass.Reportf(maxVerFuncUsedPos, "max version function used: %d", maxVerFuncUsed) + pass.Reportf(maxVerFuncPos, "max version function: %d", maxVerFunc) + pass.Reportf(maxVerVariablePos, "max version variable: %d", maxVerVariable) + pass.Reportf(curVerVariablePos, "current version variable: %d", curVerVariable) } func init() { diff --git a/pkg/ddl/multi_schema_change_test.go b/pkg/ddl/multi_schema_change_test.go index ab4acbc7d9..4477f132ea 100644 --- a/pkg/ddl/multi_schema_change_test.go +++ b/pkg/ddl/multi_schema_change_test.go @@ -829,7 +829,7 @@ func TestMultiSchemaChangeMDLView(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - unistoreMDLView := session.CreateMDLView + unistoreMDLView := session.CreateTiDBMDLView unistoreMDLView = strings.ReplaceAll(unistoreMDLView, "cluster_processlist", "processlist") unistoreMDLView = strings.ReplaceAll(unistoreMDLView, "cluster_tidb_trx", "tidb_trx") tk.MustExec(unistoreMDLView) diff --git a/pkg/meta/metadef/system.go b/pkg/meta/metadef/system.go index 88bda9dc93..2b18843468 100644 --- a/pkg/meta/metadef/system.go +++ b/pkg/meta/metadef/system.go @@ -18,7 +18,7 @@ const ( // ReservedGlobalIDUpperBound is the max value of any physical schema object ID. // due to history reasons, the first 2 bytes are planned to be used for multi // tenancy, but it's replaced by keyspace. - ReservedGlobalIDUpperBound = 0x0000FFFFFFFFFFFF + ReservedGlobalIDUpperBound int64 = 0x0000FFFFFFFFFFFF // ReservedGlobalIDLowerBound reserves 1000 IDs. // valid usable ID range for user schema objects is [1, ReservedGlobalIDLowerBound]. // @@ -30,6 +30,8 @@ const ( ) const ( + // SystemDatabaseID is the database ID of `mysql`. + SystemDatabaseID = ReservedGlobalIDUpperBound // TiDBDDLJobTableID is the table ID of `tidb_ddl_job`. TiDBDDLJobTableID = ReservedGlobalIDUpperBound - 1 // TiDBDDLReorgTableID is the table ID of `tidb_ddl_reorg`. @@ -44,4 +46,110 @@ const ( TiDBBackgroundSubtaskHistoryTableID = ReservedGlobalIDUpperBound - 6 // TiDBDDLNotifierTableID is the table ID of `tidb_ddl_notifier`. TiDBDDLNotifierTableID = ReservedGlobalIDUpperBound - 7 + // UserTableID is the table ID of `user`. + UserTableID = ReservedGlobalIDUpperBound - 8 + // PasswordHistoryTableID is the table ID of `password_history`. + PasswordHistoryTableID = ReservedGlobalIDUpperBound - 9 + // GlobalPrivTableID is the table ID of `global_priv`. + GlobalPrivTableID = ReservedGlobalIDUpperBound - 10 + // DBTableID is the table ID of `db`. + DBTableID = ReservedGlobalIDUpperBound - 11 + // TablesPrivTableID is the table ID of `table_priv`. + TablesPrivTableID = ReservedGlobalIDUpperBound - 12 + // ColumnsPrivTableID is the table ID of `column_priv`. + ColumnsPrivTableID = ReservedGlobalIDUpperBound - 13 + // GlobalVariablesTableID is the table ID of `global_variables`. + GlobalVariablesTableID = ReservedGlobalIDUpperBound - 14 + // TiDBTableID is the table ID of `tidb`. + TiDBTableID = ReservedGlobalIDUpperBound - 15 + // HelpTopicTableID is the table ID of `help_topic`. + HelpTopicTableID = ReservedGlobalIDUpperBound - 16 + // StatsMetaTableID is the table ID of `stats_meta`. + StatsMetaTableID = ReservedGlobalIDUpperBound - 17 + // StatsHistogramsTableID is the table ID of `stats_histograms`. + StatsHistogramsTableID = ReservedGlobalIDUpperBound - 18 + // StatsBucketsTableID is the table ID of `stats_buckets`. + StatsBucketsTableID = ReservedGlobalIDUpperBound - 19 + // GCDeleteRangeTableID is the table ID of `gc_delete_range`. + GCDeleteRangeTableID = ReservedGlobalIDUpperBound - 20 + // GCDeleteRangeDoneTableID is the table ID of `gc_delete_range_done`. + GCDeleteRangeDoneTableID = ReservedGlobalIDUpperBound - 21 + // StatsFeedbackTableID is the table ID of `stats_feedback`. + StatsFeedbackTableID = ReservedGlobalIDUpperBound - 22 + // RoleEdgesTableID is the table ID of `role_edges`. + RoleEdgesTableID = ReservedGlobalIDUpperBound - 23 + // DefaultRolesTableID is the table ID of `default_roles`. + DefaultRolesTableID = ReservedGlobalIDUpperBound - 24 + // BindInfoTableID is the table ID of `bind_info`. + BindInfoTableID = ReservedGlobalIDUpperBound - 25 + // StatsTopNTableID is the table ID of `stats_top_n`. + StatsTopNTableID = ReservedGlobalIDUpperBound - 26 + // ExprPushdownBlacklistTableID is the table ID of `expr_pushdown_blacklist`. + ExprPushdownBlacklistTableID = ReservedGlobalIDUpperBound - 27 + // OptRuleBlacklistTableID is the table ID of `opt_rule_blacklist`. + OptRuleBlacklistTableID = ReservedGlobalIDUpperBound - 28 + // StatsExtendedTableID is the table ID of `stats_extended`. + StatsExtendedTableID = ReservedGlobalIDUpperBound - 29 + // StatsFMSketchTableID is the table ID of `stats_fm_sketch`. + StatsFMSketchTableID = ReservedGlobalIDUpperBound - 30 + // GlobalGrantsTableID is the table ID of `global_grants`. + GlobalGrantsTableID = ReservedGlobalIDUpperBound - 31 + // CapturePlanBaselinesBlacklistTableID is the table ID of `capture_plan_baselines_blacklist`. + CapturePlanBaselinesBlacklistTableID = ReservedGlobalIDUpperBound - 32 + // ColumnStatsUsageTableID is the table ID of `column_stats_usage`. + ColumnStatsUsageTableID = ReservedGlobalIDUpperBound - 33 + // TableCacheMetaTableID is the table ID of `table_cache_meta`. + TableCacheMetaTableID = ReservedGlobalIDUpperBound - 34 + // AnalyzeOptionsTableID is the table ID of `analyze_options`. + AnalyzeOptionsTableID = ReservedGlobalIDUpperBound - 35 + // StatsHistoryTableID is the table ID of `stats_history`. + StatsHistoryTableID = ReservedGlobalIDUpperBound - 36 + // StatsMetaHistoryTableID is the table ID of `stats_meta_history`. + StatsMetaHistoryTableID = ReservedGlobalIDUpperBound - 37 + // AnalyzeJobsTableID is the table ID of `analyze_jobs`. + AnalyzeJobsTableID = ReservedGlobalIDUpperBound - 38 + // AdvisoryLocksTableID is the table ID of `advisory_locks`. + AdvisoryLocksTableID = ReservedGlobalIDUpperBound - 39 + // PlanReplayerStatusTableID is the table ID of `plan_replayer_status`. + PlanReplayerStatusTableID = ReservedGlobalIDUpperBound - 40 + // PlanReplayerTaskTableID is the table ID of `plan_replayer_task`. + PlanReplayerTaskTableID = ReservedGlobalIDUpperBound - 41 + // StatsTableLockedTableID is the table ID of `stats_table_locked`. + StatsTableLockedTableID = ReservedGlobalIDUpperBound - 42 + // TiDBTTLTableStatusTableID is the table ID of `tidb_ttl_table_status`. + TiDBTTLTableStatusTableID = ReservedGlobalIDUpperBound - 43 + // TiDBTTLTaskTableID is the table ID of `tidb_ttl_task`. + TiDBTTLTaskTableID = ReservedGlobalIDUpperBound - 44 + // TiDBTTLJobHistoryTableID is the table ID of `tidb_ttl_job_history`. + TiDBTTLJobHistoryTableID = ReservedGlobalIDUpperBound - 45 + // TiDBGlobalTaskTableID is the table ID of `tidb_global_task`. + TiDBGlobalTaskTableID = ReservedGlobalIDUpperBound - 46 + // TiDBGlobalTaskHistoryTableID is the table ID of `tidb_global_task_history`. + TiDBGlobalTaskHistoryTableID = ReservedGlobalIDUpperBound - 47 + // TiDBImportJobsTableID is the table ID of `tidb_import_jobs`. + TiDBImportJobsTableID = ReservedGlobalIDUpperBound - 48 + // TiDBRunawayWatchTableID is the table ID of `tidb_runaway_watch`. + TiDBRunawayWatchTableID = ReservedGlobalIDUpperBound - 49 + // TiDBRunawayQueriesTableID is the table ID of `tidb_runaway`. + TiDBRunawayQueriesTableID = ReservedGlobalIDUpperBound - 50 + // TiDBTimersTableID is the table ID of `tidb_timers`. + TiDBTimersTableID = ReservedGlobalIDUpperBound - 51 + // TiDBRunawayWatchDoneTableID is the table ID of `tidb_done_runaway_watch`. + TiDBRunawayWatchDoneTableID = ReservedGlobalIDUpperBound - 52 + // DistFrameworkMetaTableID is the table ID of `dist_framework_meta`. + DistFrameworkMetaTableID = ReservedGlobalIDUpperBound - 53 + // RequestUnitByGroupTableID is the table ID of `request_unit_by_group`. + RequestUnitByGroupTableID = ReservedGlobalIDUpperBound - 54 + // TiDBPITRIDMapTableID is the table ID of `tidb_pitr_id_map`. + TiDBPITRIDMapTableID = ReservedGlobalIDUpperBound - 55 + // TiDBRestoreRegistryTableID is the table ID of `tidb_restore_registry`. + TiDBRestoreRegistryTableID = ReservedGlobalIDUpperBound - 56 + // IndexAdvisorResultsTableID is the table ID of `index_advisor`. + IndexAdvisorResultsTableID = ReservedGlobalIDUpperBound - 57 + // TiDBKernelOptionsTableID is the table ID of `tidb_kernel_options`. + TiDBKernelOptionsTableID = ReservedGlobalIDUpperBound - 58 + // TiDBWorkloadValuesTableID is the table ID of `tidb_workload_values`. + TiDBWorkloadValuesTableID = ReservedGlobalIDUpperBound - 59 + // SysDatabaseID is the database ID of `sys`. + SysDatabaseID = ReservedGlobalIDUpperBound - 60 ) diff --git a/pkg/privilege/privileges/privileges_test.go b/pkg/privilege/privileges/privileges_test.go index f5d4da0b7d..ee0deb6cc8 100644 --- a/pkg/privilege/privileges/privileges_test.go +++ b/pkg/privilege/privileges/privileges_test.go @@ -1574,9 +1574,9 @@ func createStoreAndPrepareDB(t *testing.T) kv.Storage { tk.MustExec(`CREATE TABLE test(id INT NOT NULL DEFAULT 1, name varchar(255), PRIMARY KEY(id));`) tk.MustExec(fmt.Sprintf("create database if not exists %s;", mysql.SystemDB)) tk.MustExec(session.CreateUserTable) - tk.MustExec(session.CreateDBPrivTable) - tk.MustExec(session.CreateTablePrivTable) - tk.MustExec(session.CreateColumnPrivTable) + tk.MustExec(session.CreateDBTable) + tk.MustExec(session.CreateTablesPrivTable) + tk.MustExec(session.CreateColumnsPrivTable) return store } diff --git a/pkg/session/BUILD.bazel b/pkg/session/BUILD.bazel index 9638d611d3..7a9a8b3bb1 100644 --- a/pkg/session/BUILD.bazel +++ b/pkg/session/BUILD.bazel @@ -101,7 +101,6 @@ go_library( "//pkg/tablecodec", "//pkg/telemetry", "//pkg/testkit/testenv", - "//pkg/timer/tablestore", "//pkg/types", "//pkg/types/parser_driver", "//pkg/util", @@ -173,6 +172,7 @@ go_test( "//pkg/keyspace", "//pkg/kv", "//pkg/meta", + "//pkg/meta/metadef", "//pkg/parser/ast", "//pkg/parser/auth", "//pkg/session/sessionapi", diff --git a/pkg/session/bootstrap.go b/pkg/session/bootstrap.go index 7f05357c66..1e51bc0035 100644 --- a/pkg/session/bootstrap.go +++ b/pkg/session/bootstrap.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" infoschemacontext "github.com/pingcap/tidb/pkg/infoschema/context" "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/meta/metadef" "github.com/pingcap/tidb/pkg/owner" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" @@ -45,7 +46,6 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx/variable" storepkg "github.com/pingcap/tidb/pkg/store" "github.com/pingcap/tidb/pkg/table/tables" - timertable "github.com/pingcap/tidb/pkg/timer/tablestore" "github.com/pingcap/tidb/pkg/util/intest" "github.com/pingcap/tidb/pkg/util/logutil" "github.com/pingcap/tidb/pkg/util/sqlescape" @@ -130,8 +130,8 @@ const ( // * The 'GRANT'/'REVOKE' could be case-insensitive for new clusters(compatible with MySQL). // * Keep all behaviors unchanged for upgraded cluster. - // CreateDBPrivTable is the SQL statement creates DB scope privilege table in system db. - CreateDBPrivTable = `CREATE TABLE IF NOT EXISTS mysql.db ( + // CreateDBTable is the SQL statement creates DB scope privilege table in system db. + CreateDBTable = `CREATE TABLE IF NOT EXISTS mysql.db ( Host CHAR(255), DB CHAR(64) CHARSET utf8mb4 COLLATE utf8mb4_general_ci, User CHAR(32), @@ -156,8 +156,8 @@ const ( Trigger_priv ENUM('N','Y') NOT NULL DEFAULT 'N', PRIMARY KEY (Host, DB, User), KEY i_user (User));` - // CreateTablePrivTable is the SQL statement creates table scope privilege table in system db. - CreateTablePrivTable = `CREATE TABLE IF NOT EXISTS mysql.tables_priv ( + // CreateTablesPrivTable is the SQL statement creates table scope privilege table in system db. + CreateTablesPrivTable = `CREATE TABLE IF NOT EXISTS mysql.tables_priv ( Host CHAR(255), DB CHAR(64) CHARSET utf8mb4 COLLATE utf8mb4_general_ci, User CHAR(32), @@ -168,8 +168,8 @@ const ( Column_priv SET('Select','Insert','Update','References'), PRIMARY KEY (Host, DB, User, Table_name), KEY i_user (User));` - // CreateColumnPrivTable is the SQL statement creates column scope privilege table in system db. - CreateColumnPrivTable = `CREATE TABLE IF NOT EXISTS mysql.columns_priv( + // CreateColumnsPrivTable is the SQL statement creates column scope privilege table in system db. + CreateColumnsPrivTable = `CREATE TABLE IF NOT EXISTS mysql.columns_priv ( Host CHAR(255), DB CHAR(64) CHARSET utf8mb4 COLLATE utf8mb4_general_ci, User CHAR(32), @@ -183,20 +183,20 @@ const ( // TODO: MySQL puts GLOBAL_VARIABLES table in INFORMATION_SCHEMA db. // INFORMATION_SCHEMA is a virtual db in TiDB. So we put this table in system db. // Maybe we will put it back to INFORMATION_SCHEMA. - CreateGlobalVariablesTable = `CREATE TABLE IF NOT EXISTS mysql.GLOBAL_VARIABLES( + CreateGlobalVariablesTable = `CREATE TABLE IF NOT EXISTS mysql.global_variables ( VARIABLE_NAME VARCHAR(64) NOT NULL PRIMARY KEY, VARIABLE_VALUE VARCHAR(16383) DEFAULT NULL);` // CreateTiDBTable is the SQL statement creates a table in system db. // This table is a key-value struct contains some information used by TiDB. // Currently we only put bootstrapped in it which indicates if the system is already bootstrapped. - CreateTiDBTable = `CREATE TABLE IF NOT EXISTS mysql.tidb( + CreateTiDBTable = `CREATE TABLE IF NOT EXISTS mysql.tidb ( VARIABLE_NAME VARCHAR(64) NOT NULL PRIMARY KEY, VARIABLE_VALUE VARCHAR(1024) DEFAULT NULL, COMMENT VARCHAR(1024));` - // CreateHelpTopic is the SQL statement creates help_topic table in system db. + // CreateHelpTopicTable is the SQL statement creates help_topic table in system db. // See: https://dev.mysql.com/doc/refman/5.5/en/system-database.html#system-database-help-tables - CreateHelpTopic = `CREATE TABLE IF NOT EXISTS mysql.help_topic ( + CreateHelpTopicTable = `CREATE TABLE IF NOT EXISTS mysql.help_topic ( help_topic_id INT(10) UNSIGNED NOT NULL, name CHAR(64) NOT NULL, help_category_id SMALLINT(5) UNSIGNED NOT NULL, @@ -219,8 +219,8 @@ const ( UNIQUE INDEX tbl(table_id) );` - // CreateStatsColsTable stores the statistics of table columns. - CreateStatsColsTable = `CREATE TABLE IF NOT EXISTS mysql.stats_histograms ( + // CreateStatsHistogramsTable stores the statistics of table columns. + CreateStatsHistogramsTable = `CREATE TABLE IF NOT EXISTS mysql.stats_histograms ( table_id BIGINT(64) NOT NULL, is_index TINYINT(2) NOT NULL, hist_id BIGINT(64) NOT NULL, @@ -337,20 +337,20 @@ const ( INDEX tbl(table_id, is_index, hist_id) );` - // CreateExprPushdownBlacklist stores the expressions which are not allowed to be pushed down. - CreateExprPushdownBlacklist = `CREATE TABLE IF NOT EXISTS mysql.expr_pushdown_blacklist ( + // CreateExprPushdownBlacklistTable stores the expressions which are not allowed to be pushed down. + CreateExprPushdownBlacklistTable = `CREATE TABLE IF NOT EXISTS mysql.expr_pushdown_blacklist ( name CHAR(100) NOT NULL, store_type CHAR(100) NOT NULL DEFAULT 'tikv,tiflash,tidb', reason VARCHAR(200) );` - // CreateOptRuleBlacklist stores the list of disabled optimizing operations. - CreateOptRuleBlacklist = `CREATE TABLE IF NOT EXISTS mysql.opt_rule_blacklist ( + // CreateOptRuleBlacklistTable stores the list of disabled optimizing operations. + CreateOptRuleBlacklistTable = `CREATE TABLE IF NOT EXISTS mysql.opt_rule_blacklist ( name CHAR(100) NOT NULL );` - // CreateStatsExtended stores the registered extended statistics. - CreateStatsExtended = `CREATE TABLE IF NOT EXISTS mysql.stats_extended ( + // CreateStatsExtendedTable stores the registered extended statistics. + CreateStatsExtendedTable = `CREATE TABLE IF NOT EXISTS mysql.stats_extended ( name varchar(32) NOT NULL, type tinyint(4) NOT NULL, table_id bigint(64) NOT NULL, @@ -381,8 +381,8 @@ const ( PRIMARY KEY (USER,HOST,PRIV), KEY i_user (USER) );` - // CreateCapturePlanBaselinesBlacklist stores the baseline capture filter rules. - CreateCapturePlanBaselinesBlacklist = `CREATE TABLE IF NOT EXISTS mysql.capture_plan_baselines_blacklist ( + // CreateCapturePlanBaselinesBlacklistTable stores the baseline capture filter rules. + CreateCapturePlanBaselinesBlacklistTable = `CREATE TABLE IF NOT EXISTS mysql.capture_plan_baselines_blacklist ( id bigint(64) auto_increment, filter_type varchar(32) NOT NULL COMMENT "type of the filter, only db, table and frequency supported now", filter_value varchar(32) NOT NULL, @@ -416,8 +416,8 @@ const ( column_ids TEXT(19372), PRIMARY KEY (table_id) CLUSTERED );` - // CreateStatsHistory stores the historical stats. - CreateStatsHistory = `CREATE TABLE IF NOT EXISTS mysql.stats_history ( + // CreateStatsHistoryTable stores the historical stats. + CreateStatsHistoryTable = `CREATE TABLE IF NOT EXISTS mysql.stats_history ( table_id bigint(64) NOT NULL, stats_data longblob NOT NULL, seq_no bigint(64) NOT NULL comment 'sequence number of the gzipped data slice', @@ -427,8 +427,8 @@ const ( KEY table_create_time (table_id, create_time, seq_no), KEY idx_create_time (create_time) );` - // CreateStatsMetaHistory stores the historical meta stats. - CreateStatsMetaHistory = `CREATE TABLE IF NOT EXISTS mysql.stats_meta_history ( + // CreateStatsMetaHistoryTable stores the historical meta stats. + CreateStatsMetaHistoryTable = `CREATE TABLE IF NOT EXISTS mysql.stats_meta_history ( table_id bigint(64) NOT NULL, modify_count bigint(64) NOT NULL, count bigint(64) NOT NULL, @@ -439,8 +439,8 @@ const ( KEY table_create_time (table_id, create_time), KEY idx_create_time (create_time) );` - // CreateAnalyzeJobs stores the analyze jobs. - CreateAnalyzeJobs = `CREATE TABLE IF NOT EXISTS mysql.analyze_jobs ( + // CreateAnalyzeJobsTable stores the analyze jobs. + CreateAnalyzeJobsTable = `CREATE TABLE IF NOT EXISTS mysql.analyze_jobs ( id BIGINT(64) UNSIGNED NOT NULL AUTO_INCREMENT, update_time TIMESTAMP NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP, table_schema CHAR(64) NOT NULL DEFAULT '', @@ -459,12 +459,12 @@ const ( INDEX idx_schema_table_state (table_schema, table_name, state), INDEX idx_schema_table_partition_state (table_schema, table_name, partition_name, state) );` - // CreateAdvisoryLocks stores the advisory locks (get_lock, release_lock). - CreateAdvisoryLocks = `CREATE TABLE IF NOT EXISTS mysql.advisory_locks ( + // CreateAdvisoryLocksTable stores the advisory locks (get_lock, release_lock). + CreateAdvisoryLocksTable = `CREATE TABLE IF NOT EXISTS mysql.advisory_locks ( lock_name VARCHAR(64) NOT NULL PRIMARY KEY );` - // CreateMDLView is a view about metadata locks. - CreateMDLView = `CREATE OR REPLACE SQL SECURITY INVOKER VIEW mysql.tidb_mdl_view as ( + // CreateTiDBMDLView is a view about metadata locks. + CreateTiDBMDLView = `CREATE OR REPLACE SQL SECURITY INVOKER VIEW mysql.tidb_mdl_view as ( SELECT tidb_mdl_info.job_id, JSON_UNQUOTE(JSON_EXTRACT(cast(cast(job_meta as char) as json), "$.schema_name")) as db_name, JSON_UNQUOTE(JSON_EXTRACT(cast(cast(job_meta as char) as json), "$.table_name")) as table_name, @@ -496,16 +496,16 @@ const ( update_time TIMESTAMP NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP, PRIMARY KEY (sql_digest,plan_digest));` - // CreateStatsTableLocked stores the locked tables - CreateStatsTableLocked = `CREATE TABLE IF NOT EXISTS mysql.stats_table_locked( + // CreateStatsTableLockedTable stores the locked tables + CreateStatsTableLockedTable = `CREATE TABLE IF NOT EXISTS mysql.stats_table_locked ( table_id bigint(64) NOT NULL, modify_count bigint(64) NOT NULL DEFAULT 0, count bigint(64) NOT NULL DEFAULT 0, version bigint(64) UNSIGNED NOT NULL DEFAULT 0, PRIMARY KEY (table_id));` - // CreatePasswordHistory is a table save history passwd. - CreatePasswordHistory = `CREATE TABLE IF NOT EXISTS mysql.password_history ( + // CreatePasswordHistoryTable is a table save history passwd. + CreatePasswordHistoryTable = `CREATE TABLE IF NOT EXISTS mysql.password_history ( Host char(255) NOT NULL DEFAULT '', User char(32) NOT NULL DEFAULT '', Password_timestamp timestamp(6) NOT NULL DEFAULT CURRENT_TIMESTAMP(6), @@ -513,8 +513,8 @@ const ( PRIMARY KEY (Host,User,Password_timestamp ) ) COMMENT='Password history for user accounts' ` - // CreateTTLTableStatus is a table about TTL job schedule - CreateTTLTableStatus = `CREATE TABLE IF NOT EXISTS mysql.tidb_ttl_table_status ( + // CreateTiDBTTLTableStatusTable is a table about TTL job schedule + CreateTiDBTTLTableStatusTable = `CREATE TABLE IF NOT EXISTS mysql.tidb_ttl_table_status ( table_id bigint(64) PRIMARY KEY, parent_table_id bigint(64), table_statistics text DEFAULT NULL, @@ -533,8 +533,8 @@ const ( current_job_status varchar(64) DEFAULT NULL, current_job_status_update_time timestamp NULL DEFAULT NULL);` - // CreateTTLTask is a table about parallel ttl tasks - CreateTTLTask = `CREATE TABLE IF NOT EXISTS mysql.tidb_ttl_task ( + // CreateTiDBTTLTaskTable is a table about parallel ttl tasks + CreateTiDBTTLTaskTable = `CREATE TABLE IF NOT EXISTS mysql.tidb_ttl_task ( job_id varchar(64) NOT NULL, table_id bigint(64) NOT NULL, scan_id int NOT NULL, @@ -551,8 +551,8 @@ const ( primary key(job_id, scan_id), key(created_time));` - // CreateTTLJobHistory is a table that stores ttl job's history - CreateTTLJobHistory = `CREATE TABLE IF NOT EXISTS mysql.tidb_ttl_job_history ( + // CreateTiDBTTLJobHistoryTable is a table that stores ttl job's history + CreateTiDBTTLJobHistoryTable = `CREATE TABLE IF NOT EXISTS mysql.tidb_ttl_job_history ( job_id varchar(64) PRIMARY KEY, table_id bigint(64) NOT NULL, parent_table_id bigint(64) NOT NULL, @@ -572,8 +572,8 @@ const ( key(create_time) );` - // CreateGlobalTask is a table about global task. - CreateGlobalTask = `CREATE TABLE IF NOT EXISTS mysql.tidb_global_task ( + // CreateTiDBGlobalTaskTable is a table about global task. + CreateTiDBGlobalTaskTable = `CREATE TABLE IF NOT EXISTS mysql.tidb_global_task ( id BIGINT(20) NOT NULL AUTO_INCREMENT PRIMARY KEY, task_key VARCHAR(256) NOT NULL, type VARCHAR(256) NOT NULL, @@ -598,8 +598,8 @@ const ( UNIQUE KEY task_key(task_key) );` - // CreateGlobalTaskHistory is a table about history global task. - CreateGlobalTaskHistory = `CREATE TABLE IF NOT EXISTS mysql.tidb_global_task_history ( + // CreateTiDBGlobalTaskHistoryTable is a table about history global task. + CreateTiDBGlobalTaskHistoryTable = `CREATE TABLE IF NOT EXISTS mysql.tidb_global_task_history ( id BIGINT(20) NOT NULL AUTO_INCREMENT PRIMARY KEY, task_key VARCHAR(256) NOT NULL, type VARCHAR(256) NOT NULL, @@ -624,16 +624,16 @@ const ( UNIQUE KEY task_key(task_key) );` - // CreateDistFrameworkMeta create a system table that distributed task framework use to store meta information - CreateDistFrameworkMeta = `CREATE TABLE IF NOT EXISTS mysql.dist_framework_meta ( + // CreateDistFrameworkMetaTable create a system table that distributed task framework use to store meta information + CreateDistFrameworkMetaTable = `CREATE TABLE IF NOT EXISTS mysql.dist_framework_meta ( host VARCHAR(261) NOT NULL PRIMARY KEY, role VARCHAR(64), cpu_count int default 0, keyspace_id bigint(8) NOT NULL DEFAULT -1 );` - // CreateRunawayTable stores the query which is identified as runaway or quarantined because of in watch list. - CreateRunawayTable = `CREATE TABLE IF NOT EXISTS mysql.tidb_runaway_queries ( + // CreateTiDBRunawayQueriesTable stores the query which is identified as runaway or quarantined because of in watch list. + CreateTiDBRunawayQueriesTable = `CREATE TABLE IF NOT EXISTS mysql.tidb_runaway_queries ( resource_group_name varchar(32) not null, start_time TIMESTAMP NOT NULL, repeats int default 1, @@ -648,8 +648,34 @@ const ( INDEX time_index(start_time) COMMENT "accelerate the speed when querying with active watch" ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;` - // CreateRunawayWatchTable stores the condition which is used to check whether query should be quarantined. - CreateRunawayWatchTable = `CREATE TABLE IF NOT EXISTS mysql.tidb_runaway_watch ( + // CreateTiDBTimersTable is a table to store all timers for tidb + CreateTiDBTimersTable = `CREATE TABLE IF NOT EXISTS mysql.tidb_timers ( + ID BIGINT(64) UNSIGNED NOT NULL AUTO_INCREMENT, + NAMESPACE VARCHAR(256) NOT NULL, + TIMER_KEY VARCHAR(256) NOT NULL, + TIMER_DATA BLOB, + TIMEZONE VARCHAR(64) NOT NULL, + SCHED_POLICY_TYPE VARCHAR(32) NOT NULL, + SCHED_POLICY_EXPR VARCHAR(256) NOT NULL, + HOOK_CLASS VARCHAR(64) NOT NULL, + WATERMARK TIMESTAMP DEFAULT NULL, + ENABLE TINYINT(2) NOT NULL, + TIMER_EXT JSON NOT NULL, + EVENT_STATUS VARCHAR(32) NOT NULL, + EVENT_ID VARCHAR(64) NOT NULL, + EVENT_DATA BLOB, + EVENT_START TIMESTAMP DEFAULT NULL, + SUMMARY_DATA BLOB, + CREATE_TIME TIMESTAMP NOT NULL DEFAULT CURRENT_TIMESTAMP, + UPDATE_TIME TIMESTAMP NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP, + VERSION BIGINT(64) UNSIGNED NOT NULL, + PRIMARY KEY (ID), + UNIQUE KEY timer_key(NAMESPACE, TIMER_KEY), + KEY hook_class(HOOK_CLASS) + )` + + // CreateTiDBRunawayWatchTable stores the condition which is used to check whether query should be quarantined. + CreateTiDBRunawayWatchTable = `CREATE TABLE IF NOT EXISTS mysql.tidb_runaway_watch ( id BIGINT(20) NOT NULL AUTO_INCREMENT PRIMARY KEY, resource_group_name varchar(32) not null, start_time datetime(6) NOT NULL, @@ -664,8 +690,8 @@ const ( INDEX time_index(end_time) COMMENT "accelerate the speed when querying with active watch" ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;` - // CreateDoneRunawayWatchTable stores the condition which is used to check whether query should be quarantined. - CreateDoneRunawayWatchTable = `CREATE TABLE IF NOT EXISTS mysql.tidb_runaway_watch_done ( + // CreateTiDBRunawayWatchDoneTable stores the condition which is used to check whether query should be quarantined. + CreateTiDBRunawayWatchDoneTable = `CREATE TABLE IF NOT EXISTS mysql.tidb_runaway_watch_done ( id BIGINT(20) NOT NULL AUTO_INCREMENT PRIMARY KEY, record_id BIGINT(20) not null, resource_group_name varchar(32) not null, @@ -690,8 +716,8 @@ const ( KEY (resource_group) );` - // CreateImportJobs is a table that IMPORT INTO uses. - CreateImportJobs = `CREATE TABLE IF NOT EXISTS mysql.tidb_import_jobs ( + // CreateTiDBImportJobsTable is a table that IMPORT INTO uses. + CreateTiDBImportJobsTable = `CREATE TABLE IF NOT EXISTS mysql.tidb_import_jobs ( id bigint(64) NOT NULL AUTO_INCREMENT, create_time TIMESTAMP(6) NOT NULL DEFAULT CURRENT_TIMESTAMP(6), start_time TIMESTAMP(6) NULL DEFAULT NULL, @@ -711,10 +737,10 @@ const ( KEY (created_by), KEY (status));` - // CreatePITRIDMap is a table that records the id map from upstream to downstream for PITR. + // CreateTiDBPITRIDMapTable is a table that records the id map from upstream to downstream for PITR. // set restore id default to 0 to make it compatible for old BR tool to restore to a new TiDB, such case should be // rare though. - CreatePITRIDMap = `CREATE TABLE IF NOT EXISTS mysql.tidb_pitr_id_map ( + CreateTiDBPITRIDMapTable = `CREATE TABLE IF NOT EXISTS mysql.tidb_pitr_id_map ( restore_id BIGINT NOT NULL DEFAULT 0, restored_ts BIGINT NOT NULL, upstream_cluster_id BIGINT NOT NULL, @@ -723,8 +749,8 @@ const ( update_time TIMESTAMP DEFAULT CURRENT_TIMESTAMP, PRIMARY KEY (restore_id, restored_ts, upstream_cluster_id, segment_id));` - // CreateRestoreRegistryTable is a table that tracks active restore tasks to prevent conflicts. - CreateRestoreRegistryTable = `CREATE TABLE IF NOT EXISTS mysql.tidb_restore_registry ( + // CreateTiDBRestoreRegistryTable is a table that tracks active restore tasks to prevent conflicts. + CreateTiDBRestoreRegistryTable = `CREATE TABLE IF NOT EXISTS mysql.tidb_restore_registry ( id BIGINT UNSIGNED NOT NULL AUTO_INCREMENT PRIMARY KEY, filter_strings TEXT NOT NULL, filter_hash VARCHAR(64) NOT NULL, @@ -749,9 +775,6 @@ const ( // DropMySQLIndexUsageTable removes the table `mysql.schema_index_usage` DropMySQLIndexUsageTable = "DROP TABLE IF EXISTS mysql.schema_index_usage" - // CreateSysSchema creates a new schema called `sys`. - CreateSysSchema = `CREATE DATABASE IF NOT EXISTS sys;` - // CreateSchemaUnusedIndexesView creates a view to use `information_schema.tidb_index_usage` to get the unused indexes. CreateSchemaUnusedIndexesView = `CREATE OR REPLACE VIEW sys.schema_unused_indexes AS SELECT @@ -766,8 +789,8 @@ const ( HAVING sum(last_access_time) is null;` - // CreateIndexAdvisorTable is a table to store the index advisor results. - CreateIndexAdvisorTable = `CREATE TABLE IF NOT EXISTS mysql.index_advisor_results ( + // CreateIndexAdvisorResultsTable is a table to store the index advisor results. + CreateIndexAdvisorResultsTable = `CREATE TABLE IF NOT EXISTS mysql.index_advisor_results ( id bigint primary key not null auto_increment, created_at datetime not null, updated_at datetime not null, @@ -785,8 +808,8 @@ const ( index idx_update(updated_at), unique index idx(schema_name, table_name, index_columns));` - // CreateKernelOptionsTable is a table to store kernel options for tidb. - CreateKernelOptionsTable = `CREATE TABLE IF NOT EXISTS mysql.tidb_kernel_options ( + // CreateTiDBKernelOptionsTable is a table to store kernel options for tidb. + CreateTiDBKernelOptionsTable = `CREATE TABLE IF NOT EXISTS mysql.tidb_kernel_options ( module varchar(128), name varchar(128), value varchar(128), @@ -807,9 +830,6 @@ const ( index idx_table_id (table_id));` ) -// CreateTimers is a table to store all timers for tidb -var CreateTimers = timertable.CreateTimerTableSQL("mysql", "tidb_timers") - // bootstrap initiates system DB for a store. func bootstrap(s sessionapi.Session) { startTime := time.Now() @@ -1082,120 +1102,85 @@ func getBootstrapVersion(s sessionapi.Session) (int64, error) { return strconv.ParseInt(sVal, 10, 64) } +var systemDatabases = []DatabaseBasicInfo{ + {ID: metadef.SystemDatabaseID, Name: mysql.SystemDB}, + {ID: metadef.SysDatabaseID, Name: mysql.SysDB}, +} + +// tablesInSystemDatabase contains the definitions of system tables in the mysql +// database, or the system database, except DDL related tables, see ddlTableVersionTables. +// TODO: the reserved ID will be used later. +var tablesInSystemDatabase = []TableBasicInfo{ + {ID: metadef.UserTableID, Name: "user", SQL: CreateUserTable}, + {ID: metadef.PasswordHistoryTableID, Name: "password_history", SQL: CreatePasswordHistoryTable}, + {ID: metadef.GlobalPrivTableID, Name: "global_priv", SQL: CreateGlobalPrivTable}, + {ID: metadef.DBTableID, Name: "db", SQL: CreateDBTable}, + {ID: metadef.TablesPrivTableID, Name: "tables_priv", SQL: CreateTablesPrivTable}, + {ID: metadef.ColumnsPrivTableID, Name: "columns_priv", SQL: CreateColumnsPrivTable}, + {ID: metadef.GlobalVariablesTableID, Name: "global_variables", SQL: CreateGlobalVariablesTable}, + {ID: metadef.TiDBTableID, Name: "tidb", SQL: CreateTiDBTable}, + {ID: metadef.HelpTopicTableID, Name: "help_topic", SQL: CreateHelpTopicTable}, + {ID: metadef.StatsMetaTableID, Name: "stats_meta", SQL: CreateStatsMetaTable}, + {ID: metadef.StatsHistogramsTableID, Name: "stats_histograms", SQL: CreateStatsHistogramsTable}, + {ID: metadef.StatsBucketsTableID, Name: "stats_buckets", SQL: CreateStatsBucketsTable}, + {ID: metadef.GCDeleteRangeTableID, Name: "gc_delete_range", SQL: CreateGCDeleteRangeTable}, + {ID: metadef.GCDeleteRangeDoneTableID, Name: "gc_delete_range_done", SQL: CreateGCDeleteRangeDoneTable}, + {ID: metadef.StatsFeedbackTableID, Name: "stats_feedback", SQL: CreateStatsFeedbackTable}, + {ID: metadef.RoleEdgesTableID, Name: "role_edges", SQL: CreateRoleEdgesTable}, + {ID: metadef.DefaultRolesTableID, Name: "default_roles", SQL: CreateDefaultRolesTable}, + {ID: metadef.BindInfoTableID, Name: "bind_info", SQL: CreateBindInfoTable}, + {ID: metadef.StatsTopNTableID, Name: "stats_top_n", SQL: CreateStatsTopNTable}, + {ID: metadef.ExprPushdownBlacklistTableID, Name: "expr_pushdown_blacklist", SQL: CreateExprPushdownBlacklistTable}, + {ID: metadef.OptRuleBlacklistTableID, Name: "opt_rule_blacklist", SQL: CreateOptRuleBlacklistTable}, + {ID: metadef.StatsExtendedTableID, Name: "stats_extended", SQL: CreateStatsExtendedTable}, + {ID: metadef.StatsFMSketchTableID, Name: "stats_fm_sketch", SQL: CreateStatsFMSketchTable}, + {ID: metadef.GlobalGrantsTableID, Name: "global_grants", SQL: CreateGlobalGrantsTable}, + {ID: metadef.CapturePlanBaselinesBlacklistTableID, Name: "capture_plan_baselines_blacklist", SQL: CreateCapturePlanBaselinesBlacklistTable}, + {ID: metadef.ColumnStatsUsageTableID, Name: "column_stats_usage", SQL: CreateColumnStatsUsageTable}, + {ID: metadef.TableCacheMetaTableID, Name: "table_cache_meta", SQL: CreateTableCacheMetaTable}, + {ID: metadef.AnalyzeOptionsTableID, Name: "analyze_options", SQL: CreateAnalyzeOptionsTable}, + {ID: metadef.StatsHistoryTableID, Name: "stats_history", SQL: CreateStatsHistoryTable}, + {ID: metadef.StatsMetaHistoryTableID, Name: "stats_meta_history", SQL: CreateStatsMetaHistoryTable}, + {ID: metadef.AnalyzeJobsTableID, Name: "analyze_jobs", SQL: CreateAnalyzeJobsTable}, + {ID: metadef.AdvisoryLocksTableID, Name: "advisory_locks", SQL: CreateAdvisoryLocksTable}, + {ID: metadef.PlanReplayerStatusTableID, Name: "plan_replayer_status", SQL: CreatePlanReplayerStatusTable}, + {ID: metadef.PlanReplayerTaskTableID, Name: "plan_replayer_task", SQL: CreatePlanReplayerTaskTable}, + {ID: metadef.StatsTableLockedTableID, Name: "stats_table_locked", SQL: CreateStatsTableLockedTable}, + {ID: metadef.TiDBTTLTableStatusTableID, Name: "tidb_ttl_table_status", SQL: CreateTiDBTTLTableStatusTable}, + {ID: metadef.TiDBTTLTaskTableID, Name: "tidb_ttl_task", SQL: CreateTiDBTTLTaskTable}, + {ID: metadef.TiDBTTLJobHistoryTableID, Name: "tidb_ttl_job_history", SQL: CreateTiDBTTLJobHistoryTable}, + {ID: metadef.TiDBGlobalTaskTableID, Name: "tidb_global_task", SQL: CreateTiDBGlobalTaskTable}, + {ID: metadef.TiDBGlobalTaskHistoryTableID, Name: "tidb_global_task_history", SQL: CreateTiDBGlobalTaskHistoryTable}, + {ID: metadef.TiDBImportJobsTableID, Name: "tidb_import_jobs", SQL: CreateTiDBImportJobsTable}, + {ID: metadef.TiDBRunawayWatchTableID, Name: "tidb_runaway_watch", SQL: CreateTiDBRunawayWatchTable}, + {ID: metadef.TiDBRunawayQueriesTableID, Name: "tidb_runaway_queries", SQL: CreateTiDBRunawayQueriesTable}, + {ID: metadef.TiDBTimersTableID, Name: "tidb_timers", SQL: CreateTiDBTimersTable}, + {ID: metadef.TiDBRunawayWatchDoneTableID, Name: "tidb_runaway_watch_done", SQL: CreateTiDBRunawayWatchDoneTable}, + {ID: metadef.DistFrameworkMetaTableID, Name: "dist_framework_meta", SQL: CreateDistFrameworkMetaTable}, + {ID: metadef.RequestUnitByGroupTableID, Name: "request_unit_by_group", SQL: CreateRequestUnitByGroupTable}, + {ID: metadef.TiDBPITRIDMapTableID, Name: "tidb_pitr_id_map", SQL: CreateTiDBPITRIDMapTable}, + {ID: metadef.TiDBRestoreRegistryTableID, Name: "tidb_restore_registry", SQL: CreateTiDBRestoreRegistryTable}, + {ID: metadef.IndexAdvisorResultsTableID, Name: "index_advisor_results", SQL: CreateIndexAdvisorResultsTable}, + {ID: metadef.TiDBKernelOptionsTableID, Name: "tidb_kernel_options", SQL: CreateTiDBKernelOptionsTable}, + {ID: metadef.TiDBWorkloadValuesTableID, Name: "tidb_workload_values", SQL: CreateTiDBWorkloadValuesTable}, +} + // doDDLWorks executes DDL statements in bootstrap stage. func doDDLWorks(s sessionapi.Session) { - // Create a test database. - mustExecute(s, "CREATE DATABASE IF NOT EXISTS test") - // Create system db. - mustExecute(s, "CREATE DATABASE IF NOT EXISTS %n", mysql.SystemDB) - // Create user table. - mustExecute(s, CreateUserTable) - // Create password history. - mustExecute(s, CreatePasswordHistory) - // Create privilege tables. - mustExecute(s, CreateGlobalPrivTable) - mustExecute(s, CreateDBPrivTable) - mustExecute(s, CreateTablePrivTable) - mustExecute(s, CreateColumnPrivTable) - // Create global system variable table. - mustExecute(s, CreateGlobalVariablesTable) - // Create TiDB table. - mustExecute(s, CreateTiDBTable) - // Create help table. - mustExecute(s, CreateHelpTopic) - // Create stats_meta table. - mustExecute(s, CreateStatsMetaTable) - // Create stats_columns table. - mustExecute(s, CreateStatsColsTable) - // Create stats_buckets table. - mustExecute(s, CreateStatsBucketsTable) - // Create gc_delete_range table. - mustExecute(s, CreateGCDeleteRangeTable) - // Create gc_delete_range_done table. - mustExecute(s, CreateGCDeleteRangeDoneTable) - // Create stats_feedback table. - // NOTE: Feedback is deprecated, but we still need to create this table for compatibility. - mustExecute(s, CreateStatsFeedbackTable) - // Create role_edges table. - mustExecute(s, CreateRoleEdgesTable) - // Create default_roles table. - mustExecute(s, CreateDefaultRolesTable) + for _, db := range systemDatabases { + mustExecute(s, "CREATE DATABASE IF NOT EXISTS %n", db.Name) + } + for _, tbl := range tablesInSystemDatabase { + mustExecute(s, tbl.SQL) + } // Create bind_info table. - initBindInfoTable(s) - // Create stats_topn_store table. - mustExecute(s, CreateStatsTopNTable) - // Create expr_pushdown_blacklist table. - mustExecute(s, CreateExprPushdownBlacklist) - // Create opt_rule_blacklist table. - mustExecute(s, CreateOptRuleBlacklist) - // Create stats_extended table. - mustExecute(s, CreateStatsExtended) - // Create stats_fm_sketch table. - mustExecute(s, CreateStatsFMSketchTable) - // Create global_grants - mustExecute(s, CreateGlobalGrantsTable) - // Create capture_plan_baselines_blacklist - mustExecute(s, CreateCapturePlanBaselinesBlacklist) - // Create column_stats_usage table - mustExecute(s, CreateColumnStatsUsageTable) - // Create table_cache_meta table. - mustExecute(s, CreateTableCacheMetaTable) - // Create analyze_options table. - mustExecute(s, CreateAnalyzeOptionsTable) - // Create stats_history table. - mustExecute(s, CreateStatsHistory) - // Create stats_meta_history table. - mustExecute(s, CreateStatsMetaHistory) - // Create analyze_jobs table. - mustExecute(s, CreateAnalyzeJobs) - // Create advisory_locks table. - mustExecute(s, CreateAdvisoryLocks) - // Create mdl view. - mustExecute(s, CreateMDLView) - // Create plan_replayer_status table - mustExecute(s, CreatePlanReplayerStatusTable) - // Create plan_replayer_task table - mustExecute(s, CreatePlanReplayerTaskTable) - // Create stats_meta_table_locked table - mustExecute(s, CreateStatsTableLocked) - // Create tidb_ttl_table_status table - mustExecute(s, CreateTTLTableStatus) - // Create tidb_ttl_task table - mustExecute(s, CreateTTLTask) - // Create tidb_ttl_job_history table - mustExecute(s, CreateTTLJobHistory) - // Create tidb_global_task table - mustExecute(s, CreateGlobalTask) - // Create tidb_global_task_history table - mustExecute(s, CreateGlobalTaskHistory) - // Create tidb_import_jobs - mustExecute(s, CreateImportJobs) - // create runaway_watch - mustExecute(s, CreateRunawayWatchTable) - // create runaway_queries - mustExecute(s, CreateRunawayTable) - // create tidb_timers - mustExecute(s, CreateTimers) - // create runaway_watch done - mustExecute(s, CreateDoneRunawayWatchTable) - // create dist_framework_meta - mustExecute(s, CreateDistFrameworkMeta) - // create request_unit_by_group - mustExecute(s, CreateRequestUnitByGroupTable) - // create tidb_pitr_id_map - mustExecute(s, CreatePITRIDMap) - // create tidb_restore_registry - mustExecute(s, CreateRestoreRegistryTable) - // create `sys` schema - mustExecute(s, CreateSysSchema) + insertBuiltinBindInfoRow(s) + // Create `mysql.tidb_mdl_view` view. + mustExecute(s, CreateTiDBMDLView) // create `sys.schema_unused_indexes` view mustExecute(s, CreateSchemaUnusedIndexesView) - // create mysql.index_advisor_results - mustExecute(s, CreateIndexAdvisorTable) - // create mysql.tidb_kernel_options - mustExecute(s, CreateKernelOptionsTable) - // create mysql.tidb_workload_values - mustExecute(s, CreateTiDBWorkloadValuesTable) + // Create a test database. + mustExecute(s, "CREATE DATABASE IF NOT EXISTS test") } // doBootstrapSQLFile executes SQL commands in a file as the last stage of bootstrap. diff --git a/pkg/session/bootstrap_test.go b/pkg/session/bootstrap_test.go index e5fd06b980..417bf70807 100644 --- a/pkg/session/bootstrap_test.go +++ b/pkg/session/bootstrap_test.go @@ -18,6 +18,7 @@ import ( "context" "crypto/tls" "fmt" + "slices" "strings" "testing" "time" @@ -31,6 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/keyspace" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta" + "github.com/pingcap/tidb/pkg/meta/metadef" "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/session/sessionapi" "github.com/pingcap/tidb/pkg/sessionctx" @@ -46,6 +48,26 @@ import ( "go.uber.org/zap/zapcore" ) +func TestMySQLDBTables(t *testing.T) { + testTableBasicInfoSlice(t, tablesInSystemDatabase) + reservedIDs := make([]int64, 0, len(ddlTableVersionTables)*2) + for _, v := range ddlTableVersionTables { + for _, tbl := range v.tables { + reservedIDs = append(reservedIDs, tbl.ID) + } + } + for _, tbl := range tablesInSystemDatabase { + reservedIDs = append(reservedIDs, tbl.ID) + } + for _, db := range systemDatabases { + reservedIDs = append(reservedIDs, db.ID) + } + slices.Sort(reservedIDs) + require.IsIncreasing(t, reservedIDs, "used IDs should be in increasing order") + require.Greater(t, reservedIDs[0], metadef.ReservedGlobalIDLowerBound, "reserved ID should be greater than ReservedGlobalIDLowerBound") + require.LessOrEqual(t, reservedIDs[len(reservedIDs)-1], metadef.ReservedGlobalIDUpperBound, "reserved ID should be less than or equal to ReservedGlobalIDUpperBound") +} + // This test file have many problem. // 1. Please use testkit to create dom, session and store. // 2. Don't use CreateStoreAndBootstrap and BootstrapSession together. It will cause data race. diff --git a/pkg/session/session.go b/pkg/session/session.go index 25f756ce23..931f370a6d 100644 --- a/pkg/session/session.go +++ b/pkg/session/session.go @@ -3272,6 +3272,12 @@ func loadCollationParameter(ctx context.Context, se *session) (bool, error) { return false, nil } +// DatabaseBasicInfo contains the basic information of a database. +type DatabaseBasicInfo struct { + ID int64 + Name string +} + // TableBasicInfo contains the basic information of a table used in DDL. type TableBasicInfo struct { ID int64 diff --git a/pkg/session/session_test.go b/pkg/session/session_test.go index 954537de97..68141bac74 100644 --- a/pkg/session/session_test.go +++ b/pkg/session/session_test.go @@ -22,6 +22,7 @@ import ( "testing" "github.com/pingcap/tidb/pkg/ddl" + "github.com/pingcap/tidb/pkg/meta/metadef" "github.com/stretchr/testify/require" ) @@ -40,10 +41,23 @@ func TestDDLTableVersionTables(t *testing.T) { for _, v := range ddlTableVersionTables { allDDLTables = append(allDDLTables, v.tables...) } - require.True(t, slices.IsSortedFunc(allDDLTables, func(a, b TableBasicInfo) int { + testTableBasicInfoSlice(t, allDDLTables) +} + +func testTableBasicInfoSlice(t *testing.T, allTables []TableBasicInfo) { + t.Helper() + require.True(t, slices.IsSortedFunc(allTables, func(a, b TableBasicInfo) int { + if a.ID == b.ID { + t.Errorf("table IDs should be unique, a=%d, b=%d", a.ID, b.ID) + } + if a.Name == b.Name { + t.Errorf("table names should be unique, a=%s, b=%s", a.Name, b.Name) + } return cmp.Compare(b.ID, a.ID) - }), "ddlTableVersionTables should be sorted by table ID in descending order") - for _, vt := range allDDLTables { + }), "tables should be sorted by table ID in descending order") + for _, vt := range allTables { + require.Greater(t, vt.ID, metadef.ReservedGlobalIDLowerBound, "table ID should be greater than ReservedGlobalIDLowerBound") + require.LessOrEqual(t, vt.ID, metadef.ReservedGlobalIDUpperBound, "table ID should be less than or equal to ReservedGlobalIDUpperBound") require.Equal(t, strings.ToLower(vt.Name), vt.Name, "table name should be in lower case") require.Contains(t, vt.SQL, fmt.Sprintf(" mysql.%s (", vt.Name), "table SQL should contain table name and follow the format 'mysql. ('") diff --git a/pkg/session/upgrade.go b/pkg/session/upgrade.go index 17699eb7bb..35bde71393 100644 --- a/pkg/session/upgrade.go +++ b/pkg/session/upgrade.go @@ -678,7 +678,7 @@ func upgradeToVer4(s sessionapi.Session, _ int64) { } func upgradeToVer5(s sessionapi.Session, _ int64) { - mustExecute(s, CreateStatsColsTable) + mustExecute(s, CreateStatsHistogramsTable) mustExecute(s, CreateStatsBucketsTable) } @@ -922,11 +922,11 @@ func upgradeToVer32(s sessionapi.Session, _ int64) { } func upgradeToVer33(s sessionapi.Session, _ int64) { - doReentrantDDL(s, CreateExprPushdownBlacklist) + doReentrantDDL(s, CreateExprPushdownBlacklistTable) } func upgradeToVer34(s sessionapi.Session, _ int64) { - doReentrantDDL(s, CreateOptRuleBlacklist) + doReentrantDDL(s, CreateOptRuleBlacklistTable) } func upgradeToVer35(s sessionapi.Session, _ int64) { @@ -1108,18 +1108,13 @@ func upgradeToVer55(s sessionapi.Session, _ int64) { // If we upgrade from v4.0 to a newer version, the real upgradeToVer49 will be missed. // So we redo upgradeToVer49 here to make sure the upgrading from v4.0 succeeds. func upgradeToVer56(s sessionapi.Session, _ int64) { - doReentrantDDL(s, CreateStatsExtended) + doReentrantDDL(s, CreateStatsExtendedTable) } func upgradeToVer57(s sessionapi.Session, _ int64) { insertBuiltinBindInfoRow(s) } -func initBindInfoTable(s sessionapi.Session) { - mustExecute(s, CreateBindInfoTable) - insertBuiltinBindInfoRow(s) -} - func insertBuiltinBindInfoRow(s sessionapi.Session) { mustExecute(s, `INSERT HIGH_PRIORITY INTO mysql.bind_info(original_sql, bind_sql, default_db, status, create_time, update_time, charset, collation, source) VALUES (%?, %?, "mysql", %?, "0000-00-00 00:00:00", "0000-00-00 00:00:00", "", "", %?)`, @@ -1141,7 +1136,7 @@ func upgradeToVer59(s sessionapi.Session, _ int64) { func upgradeToVer60(s sessionapi.Session, _ int64) { mustExecute(s, "DROP TABLE IF EXISTS mysql.stats_extended") - doReentrantDDL(s, CreateStatsExtended) + doReentrantDDL(s, CreateStatsExtendedTable) } type bindInfo struct { @@ -1297,7 +1292,7 @@ func upgradeToVer72(s sessionapi.Session, _ int64) { } func upgradeToVer73(s sessionapi.Session, _ int64) { - doReentrantDDL(s, CreateCapturePlanBaselinesBlacklist) + doReentrantDDL(s, CreateCapturePlanBaselinesBlacklistTable) } func upgradeToVer74(s sessionapi.Session, _ int64) { @@ -1350,11 +1345,11 @@ func upgradeToVer82(s sessionapi.Session, _ int64) { } func upgradeToVer83(s sessionapi.Session, _ int64) { - doReentrantDDL(s, CreateStatsHistory) + doReentrantDDL(s, CreateStatsHistoryTable) } func upgradeToVer84(s sessionapi.Session, _ int64) { - doReentrantDDL(s, CreateStatsMetaHistory) + doReentrantDDL(s, CreateStatsMetaHistoryTable) } func upgradeToVer85(s sessionapi.Session, _ int64) { @@ -1366,7 +1361,7 @@ func upgradeToVer86(s sessionapi.Session, _ int64) { } func upgradeToVer87(s sessionapi.Session, _ int64) { - doReentrantDDL(s, CreateAnalyzeJobs) + doReentrantDDL(s, CreateAnalyzeJobsTable) } func upgradeToVer88(s sessionapi.Session, _ int64) { @@ -1375,7 +1370,7 @@ func upgradeToVer88(s sessionapi.Session, _ int64) { } func upgradeToVer89(s sessionapi.Session, _ int64) { - doReentrantDDL(s, CreateAdvisoryLocks) + doReentrantDDL(s, CreateAdvisoryLocksTable) } // importConfigOption is a one-time import. @@ -1425,7 +1420,7 @@ func upgradeToVer93(s sessionapi.Session, _ int64) { } func upgradeToVer94(s sessionapi.Session, _ int64) { - mustExecute(s, CreateMDLView) + mustExecute(s, CreateTiDBMDLView) } func upgradeToVer95(s sessionapi.Session, _ int64) { @@ -1472,7 +1467,7 @@ func upgradeToVer102(s sessionapi.Session, _ int64) { } func upgradeToVer103(s sessionapi.Session, _ int64) { - doReentrantDDL(s, CreateStatsTableLocked) + doReentrantDDL(s, CreateStatsTableLockedTable) } func upgradeToVer104(s sessionapi.Session, _ int64) { @@ -1486,7 +1481,7 @@ func upgradeToVer105(s sessionapi.Session, _ int64) { } func upgradeToVer106(s sessionapi.Session, _ int64) { - doReentrantDDL(s, CreatePasswordHistory) + doReentrantDDL(s, CreatePasswordHistoryTable) doReentrantDDL(s, "Alter table mysql.user add COLUMN IF NOT EXISTS `Password_reuse_history` smallint unsigned DEFAULT NULL AFTER `Create_Tablespace_Priv` ") doReentrantDDL(s, "Alter table mysql.user add COLUMN IF NOT EXISTS `Password_reuse_time` smallint unsigned DEFAULT NULL AFTER `Password_reuse_history`") } @@ -1498,7 +1493,7 @@ func upgradeToVer107(s sessionapi.Session, _ int64) { } func upgradeToVer108(s sessionapi.Session, _ int64) { - doReentrantDDL(s, CreateTTLTableStatus) + doReentrantDDL(s, CreateTiDBTTLTableStatusTable) } // For users that upgrade TiDB from a 6.2-6.4 version, we want to disable tidb gc_aware_memory_track by default. @@ -1518,12 +1513,12 @@ func upgradeToVer130(s sessionapi.Session, _ int64) { } func upgradeToVer131(s sessionapi.Session, _ int64) { - doReentrantDDL(s, CreateTTLTask) - doReentrantDDL(s, CreateTTLJobHistory) + doReentrantDDL(s, CreateTiDBTTLTaskTable) + doReentrantDDL(s, CreateTiDBTTLJobHistoryTable) } func upgradeToVer132(s sessionapi.Session, _ int64) { - doReentrantDDL(s, CreateMDLView) + doReentrantDDL(s, CreateTiDBMDLView) } func upgradeToVer133(s sessionapi.Session, _ int64) { @@ -1545,7 +1540,7 @@ func upgradeToVer135(s sessionapi.Session, _ int64) { } func upgradeToVer136(s sessionapi.Session, _ int64) { - mustExecute(s, CreateGlobalTask) + mustExecute(s, CreateTiDBGlobalTaskTable) doReentrantDDL(s, "ALTER TABLE mysql.tidb_background_subtask DROP INDEX namespace", dbterror.ErrCantDropFieldOrKey) doReentrantDDL(s, "ALTER TABLE mysql.tidb_background_subtask ADD INDEX idx_task_key(task_key)", dbterror.ErrDupKeyName) } @@ -1612,15 +1607,15 @@ func upgradeToVer167(s sessionapi.Session, _ int64) { } func upgradeToVer168(s sessionapi.Session, _ int64) { - mustExecute(s, CreateImportJobs) + mustExecute(s, CreateTiDBImportJobsTable) } func upgradeToVer169(s sessionapi.Session, _ int64) { - mustExecute(s, CreateRunawayTable) + mustExecute(s, CreateTiDBRunawayQueriesTable) } func upgradeToVer170(s sessionapi.Session, _ int64) { - mustExecute(s, CreateTimers) + mustExecute(s, CreateTiDBTimersTable) } func upgradeToVer171(s sessionapi.Session, _ int64) { @@ -1629,8 +1624,8 @@ func upgradeToVer171(s sessionapi.Session, _ int64) { func upgradeToVer172(s sessionapi.Session, _ int64) { mustExecute(s, "DROP TABLE IF EXISTS mysql.tidb_runaway_quarantined_watch") - mustExecute(s, CreateRunawayWatchTable) - mustExecute(s, CreateDoneRunawayWatchTable) + mustExecute(s, CreateTiDBRunawayWatchTable) + mustExecute(s, CreateTiDBRunawayWatchDoneTable) } func upgradeToVer173(s sessionapi.Session, _ int64) { @@ -1698,12 +1693,12 @@ func upgradeToVer175(s sessionapi.Session, _ int64) { } func upgradeToVer176(s sessionapi.Session, _ int64) { - mustExecute(s, CreateGlobalTaskHistory) + mustExecute(s, CreateTiDBGlobalTaskHistoryTable) } func upgradeToVer177(s sessionapi.Session, _ int64) { // ignore error when upgrading from v7.4 to higher version. - doReentrantDDL(s, CreateDistFrameworkMeta, infoschema.ErrTableExists) + doReentrantDDL(s, CreateDistFrameworkMetaTable, infoschema.ErrTableExists) err := s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.TiDBEnableAsyncMergeGlobalStats, vardef.Off) if err != nil { logutil.BgLogger().Fatal("upgradeToVer177 error", zap.Error(err)) @@ -1778,7 +1773,7 @@ func upgradeToVer192(s sessionapi.Session, _ int64) { } func upgradeToVer193(s sessionapi.Session, _ int64) { - doReentrantDDL(s, CreateMDLView) + doReentrantDDL(s, CreateTiDBMDLView) } func upgradeToVer194(s sessionapi.Session, _ int64) { @@ -1795,7 +1790,7 @@ func upgradeToVer196(s sessionapi.Session, _ int64) { } func upgradeToVer197(s sessionapi.Session, _ int64) { - doReentrantDDL(s, CreateMDLView) + doReentrantDDL(s, CreateTiDBMDLView) } func upgradeToVer198(s sessionapi.Session, _ int64) { @@ -1852,12 +1847,12 @@ func upgradeToVer212(s sessionapi.Session, ver int64) { } func upgradeToVer213(s sessionapi.Session, _ int64) { - mustExecute(s, CreatePITRIDMap) + mustExecute(s, CreateTiDBPITRIDMapTable) } func upgradeToVer214(s sessionapi.Session, _ int64) { - mustExecute(s, CreateIndexAdvisorTable) - mustExecute(s, CreateKernelOptionsTable) + mustExecute(s, CreateIndexAdvisorResultsTable) + mustExecute(s, CreateTiDBKernelOptionsTable) } func upgradeToVer215(s sessionapi.Session, _ int64) { @@ -2004,7 +1999,7 @@ func upgradeToVer248(s sessionapi.Session, _ int64) { } func upgradeToVer249(s sessionapi.Session, _ int64) { - doReentrantDDL(s, CreateRestoreRegistryTable) + doReentrantDDL(s, CreateTiDBRestoreRegistryTable) } func upgradeToVer250(s sessionapi.Session, _ int64) { diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/dynamic_partitioned_table_analysis_job_test.go b/pkg/statistics/handle/autoanalyze/priorityqueue/dynamic_partitioned_table_analysis_job_test.go index 9d62c93f3a..32065b4777 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/dynamic_partitioned_table_analysis_job_test.go +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/dynamic_partitioned_table_analysis_job_test.go @@ -127,7 +127,7 @@ func TestAnalyzeDynamicPartitionedTableIndexes(t *testing.T) { func TestValidateAndPrepareForDynamicPartitionedTable(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) - tk.MustExec(session.CreateAnalyzeJobs) + tk.MustExec(session.CreateAnalyzeJobsTable) tk.MustExec("create database example_schema") tk.MustExec("use example_schema") tk.MustExec("create table example_table (a int, b int, index idx(a)) partition by range (a) (partition p0 values less than (2), partition p1 values less than (4))") @@ -183,7 +183,7 @@ func TestValidateAndPrepareForDynamicPartitionedTable(t *testing.T) { func TestPerformanceOfValidateAndPrepare(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) - tk.MustExec(session.CreateAnalyzeJobs) + tk.MustExec(session.CreateAnalyzeJobsTable) tk.MustExec("create database example_schema") tk.MustExec("use example_schema") tk.MustExec("create table example_table (a int, b int, index idx(a)) partition by range (a) (partition p0 values less than (2), partition p1 values less than (4))") diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/interval_test.go b/pkg/statistics/handle/autoanalyze/priorityqueue/interval_test.go index 980c41fc5e..d7678323d9 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/interval_test.go +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/interval_test.go @@ -29,7 +29,7 @@ func TestGetAverageAnalysisDuration(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.MustExec(session.CreateAnalyzeJobs) + tk.MustExec(session.CreateAnalyzeJobsTable) // Empty table. se := tk.Session() sctx := se.(sessionctx.Context) @@ -94,7 +94,7 @@ func TestGetLastFailedAnalysisDuration(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.MustExec(session.CreateAnalyzeJobs) + tk.MustExec(session.CreateAnalyzeJobsTable) // Empty table. se := tk.Session() sctx := se.(sessionctx.Context) diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/non_partitioned_table_analysis_job_test.go b/pkg/statistics/handle/autoanalyze/priorityqueue/non_partitioned_table_analysis_job_test.go index 4dc2136454..fbf41e9254 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/non_partitioned_table_analysis_job_test.go +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/non_partitioned_table_analysis_job_test.go @@ -132,7 +132,7 @@ func TestAnalyzeNonPartitionedIndexes(t *testing.T) { func TestNonPartitionedTableValidateAndPrepare(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) - tk.MustExec(session.CreateAnalyzeJobs) + tk.MustExec(session.CreateAnalyzeJobsTable) tk.MustExec("create schema example_schema") tk.MustExec("use example_schema") tk.MustExec("create table example_table1 (a int, b int, index idx(a))") @@ -176,7 +176,7 @@ func TestNonPartitionedTableValidateAndPrepare(t *testing.T) { func TestValidateAndPrepareWhenOnlyHasFailedAnalysisRecords(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) - tk.MustExec(session.CreateAnalyzeJobs) + tk.MustExec(session.CreateAnalyzeJobsTable) tk.MustExec("create schema example_schema") tk.MustExec("use example_schema") tk.MustExec("create table example_table1 (a int, b int, index idx(a))") diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/static_partitioned_table_analysis_job_test.go b/pkg/statistics/handle/autoanalyze/priorityqueue/static_partitioned_table_analysis_job_test.go index 7c17386140..608b0ce6d2 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/static_partitioned_table_analysis_job_test.go +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/static_partitioned_table_analysis_job_test.go @@ -151,7 +151,7 @@ func TestAnalyzeStaticPartitionedTableIndexes(t *testing.T) { func TestStaticPartitionedTableValidateAndPrepare(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) - tk.MustExec(session.CreateAnalyzeJobs) + tk.MustExec(session.CreateAnalyzeJobsTable) tk.MustExec("create schema example_schema") tk.MustExec("use example_schema") tk.MustExec("create table example_table (a int, b int, index idx(a)) partition by range (a) (partition p0 values less than (2), partition p1 values less than (4))") diff --git a/pkg/timer/BUILD.bazel b/pkg/timer/BUILD.bazel index 1f24b68609..f5699ba738 100644 --- a/pkg/timer/BUILD.bazel +++ b/pkg/timer/BUILD.bazel @@ -12,6 +12,7 @@ go_test( shard_count = 6, deps = [ "//pkg/kv", + "//pkg/session", "//pkg/session/syssession", "//pkg/testkit", "//pkg/testkit/testsetup", diff --git a/pkg/timer/store_intergartion_test.go b/pkg/timer/store_intergartion_test.go index 69b3381db7..d62c0e9e76 100644 --- a/pkg/timer/store_intergartion_test.go +++ b/pkg/timer/store_intergartion_test.go @@ -17,6 +17,7 @@ package timer_test import ( "context" "fmt" + "strings" "sync/atomic" "testing" "time" @@ -24,6 +25,7 @@ import ( "github.com/google/uuid" "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/session/syssession" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/timer/api" @@ -63,6 +65,11 @@ func TestMemTimerStore(t *testing.T) { runTimerStoreWatchTest(t, store) } +// createTimerTableSQL returns a SQL to create timer table +func createTimerTableSQL(dbName, tableName string) string { + return strings.Replace(session.CreateTiDBTimersTable, "mysql.tidb_timers", fmt.Sprintf("`%s`.`%s`", dbName, tableName), 1) +} + func TestTableTimerStore(t *testing.T) { timeutil.SetSystemTZ("Asia/Shanghai") store, do := testkit.CreateMockStoreAndDomain(t) @@ -70,7 +77,7 @@ func TestTableTimerStore(t *testing.T) { dbName := "test" tblName := "timerstore" tk.MustExec("use test") - tk.MustExec(tablestore.CreateTimerTableSQL(dbName, tblName)) + tk.MustExec(createTimerTableSQL(dbName, tblName)) pool := &mockSessionPool{t: t, pool: do.AdvancedSysSessionPool()} // test CURD @@ -96,7 +103,7 @@ func TestTableTimerStore(t *testing.T) { cli := testEtcdCluster.RandClient() tk.MustExec("drop table " + tblName) - tk.MustExec(tablestore.CreateTimerTableSQL(dbName, tblName)) + tk.MustExec(createTimerTableSQL(dbName, tblName)) timerStore = tablestore.NewTableTimerStore(1, pool, dbName, tblName, cli) defer timerStore.Close() runTimerStoreWatchTest(t, timerStore) @@ -811,7 +818,7 @@ func TestTableStoreManualTrigger(t *testing.T) { dbName := "test" tblName := "timerstore" tk.MustExec("use test") - tk.MustExec(tablestore.CreateTimerTableSQL(dbName, tblName)) + tk.MustExec(createTimerTableSQL(dbName, tblName)) timerStore := tablestore.NewTableTimerStore(1, do.AdvancedSysSessionPool(), dbName, tblName, nil) defer timerStore.Close() @@ -900,7 +907,7 @@ func TestTimerStoreWithTimeZone(t *testing.T) { dbName := "test" tblName := "timerstore" tk.MustExec("use test") - tk.MustExec(tablestore.CreateTimerTableSQL(dbName, tblName)) + tk.MustExec(createTimerTableSQL(dbName, tblName)) tk.MustExec("set @@time_zone = 'America/Los_Angeles'") pool := &mockSessionPool{t: t, pool: do.AdvancedSysSessionPool()} diff --git a/pkg/timer/tablestore/sql.go b/pkg/timer/tablestore/sql.go index 681b20c317..0d651b008e 100644 --- a/pkg/timer/tablestore/sql.go +++ b/pkg/timer/tablestore/sql.go @@ -30,34 +30,6 @@ type timerExt struct { Event *eventExtObj `json:"event,omitempty"` } -// CreateTimerTableSQL returns a SQL to create timer table -func CreateTimerTableSQL(dbName, tableName string) string { - return fmt.Sprintf(`CREATE TABLE IF NOT EXISTS %s ( - ID BIGINT(64) UNSIGNED NOT NULL AUTO_INCREMENT, - NAMESPACE VARCHAR(256) NOT NULL, - TIMER_KEY VARCHAR(256) NOT NULL, - TIMER_DATA BLOB, - TIMEZONE VARCHAR(64) NOT NULL, - SCHED_POLICY_TYPE VARCHAR(32) NOT NULL, - SCHED_POLICY_EXPR VARCHAR(256) NOT NULL, - HOOK_CLASS VARCHAR(64) NOT NULL, - WATERMARK TIMESTAMP DEFAULT NULL, - ENABLE TINYINT(2) NOT NULL, - TIMER_EXT JSON NOT NULL, - EVENT_STATUS VARCHAR(32) NOT NULL, - EVENT_ID VARCHAR(64) NOT NULL, - EVENT_DATA BLOB, - EVENT_START TIMESTAMP DEFAULT NULL, - SUMMARY_DATA BLOB, - CREATE_TIME TIMESTAMP NOT NULL DEFAULT CURRENT_TIMESTAMP, - UPDATE_TIME TIMESTAMP NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP, - VERSION BIGINT(64) UNSIGNED NOT NULL, - PRIMARY KEY (ID), - UNIQUE KEY timer_key(NAMESPACE, TIMER_KEY), - KEY hook_class(HOOK_CLASS) - )`, indentString(dbName, tableName)) -} - func indentString(dbName, tableName string) string { return fmt.Sprintf("`%s`.`%s`", dbName, tableName) } diff --git a/pkg/ttl/ttlworker/BUILD.bazel b/pkg/ttl/ttlworker/BUILD.bazel index 3cfa4c04b6..da80e8431f 100644 --- a/pkg/ttl/ttlworker/BUILD.bazel +++ b/pkg/ttl/ttlworker/BUILD.bazel @@ -83,6 +83,7 @@ go_test( "//pkg/metrics", "//pkg/parser/ast", "//pkg/parser/mysql", + "//pkg/session", "//pkg/session/syssession", "//pkg/sessionctx", "//pkg/sessionctx/vardef", diff --git a/pkg/ttl/ttlworker/timer_sync_test.go b/pkg/ttl/ttlworker/timer_sync_test.go index ed85e39b14..b947bdedad 100644 --- a/pkg/ttl/ttlworker/timer_sync_test.go +++ b/pkg/ttl/ttlworker/timer_sync_test.go @@ -18,6 +18,7 @@ import ( "context" "encoding/json" "fmt" + "strings" "testing" "time" @@ -26,6 +27,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/metrics" "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/testkit" timerapi "github.com/pingcap/tidb/pkg/timer/api" "github.com/pingcap/tidb/pkg/timer/tablestore" @@ -35,12 +37,17 @@ import ( "github.com/stretchr/testify/require" ) +// createTimerTableSQL returns a SQL to create timer table +func createTimerTableSQL(dbName, tableName string) string { + return strings.Replace(session.CreateTiDBTimersTable, "mysql.tidb_timers", fmt.Sprintf("`%s`.`%s`", dbName, tableName), 1) +} + func TestTTLManualTriggerOneTimer(t *testing.T) { store, do := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.MustExec(tablestore.CreateTimerTableSQL("test", "test_timers")) + tk.MustExec(createTimerTableSQL("test", "test_timers")) timerStore := tablestore.NewTableTimerStore(1, do.AdvancedSysSessionPool(), "test", "test_timers", nil) defer timerStore.Close() var zeroWatermark time.Time @@ -216,7 +223,7 @@ func TestTTLTimerSync(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.MustExec(tablestore.CreateTimerTableSQL("test", "test_timers")) + tk.MustExec(createTimerTableSQL("test", "test_timers")) timerStore := tablestore.NewTableTimerStore(1, do.AdvancedSysSessionPool(), "test", "test_timers", nil) defer timerStore.Close()