bootstrap: refactor the system tables/dbs creation (#62540)

ref pingcap/tidb#61702
This commit is contained in:
D3Hunter
2025-07-23 13:07:17 +08:00
committed by GitHub
parent c3722d72ad
commit ad4f3ea0f7
22 changed files with 591 additions and 367 deletions

View File

@ -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)

View File

@ -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"],
)

View File

@ -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)
}

View File

@ -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() {

View File

@ -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)

View File

@ -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
)

View File

@ -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
}

View File

@ -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",

View File

@ -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.

View File

@ -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.

View File

@ -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

View File

@ -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.<table_name> ('")

View File

@ -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) {

View File

@ -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))")

View File

@ -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)

View File

@ -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))")

View File

@ -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))")

View File

@ -12,6 +12,7 @@ go_test(
shard_count = 6,
deps = [
"//pkg/kv",
"//pkg/session",
"//pkg/session/syssession",
"//pkg/testkit",
"//pkg/testkit/testsetup",

View File

@ -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()}

View File

@ -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)
}

View File

@ -83,6 +83,7 @@ go_test(
"//pkg/metrics",
"//pkg/parser/ast",
"//pkg/parser/mysql",
"//pkg/session",
"//pkg/session/syssession",
"//pkg/sessionctx",
"//pkg/sessionctx/vardef",

View File

@ -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()