592 lines
25 KiB
Go
592 lines
25 KiB
Go
// Copyright 2015 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.
|
|
|
|
// Copyright 2013 The ql Authors. All rights reserved.
|
|
// Use of this source code is governed by a BSD-style
|
|
// license that can be found in the LICENSES/QL-LICENSE file.
|
|
|
|
package session
|
|
|
|
import (
|
|
"context"
|
|
"encoding/hex"
|
|
"fmt"
|
|
"os"
|
|
osuser "os/user"
|
|
"strconv"
|
|
"strings"
|
|
"time"
|
|
|
|
"github.com/pingcap/errors"
|
|
"github.com/pingcap/tidb/pkg/config"
|
|
"github.com/pingcap/tidb/pkg/config/kerneltype"
|
|
"github.com/pingcap/tidb/pkg/domain"
|
|
"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"
|
|
"github.com/pingcap/tidb/pkg/meta/metadef"
|
|
"github.com/pingcap/tidb/pkg/meta/model"
|
|
"github.com/pingcap/tidb/pkg/owner"
|
|
"github.com/pingcap/tidb/pkg/parser"
|
|
"github.com/pingcap/tidb/pkg/parser/ast"
|
|
"github.com/pingcap/tidb/pkg/parser/auth"
|
|
"github.com/pingcap/tidb/pkg/parser/mysql"
|
|
"github.com/pingcap/tidb/pkg/parser/terror"
|
|
"github.com/pingcap/tidb/pkg/session/sessionapi"
|
|
"github.com/pingcap/tidb/pkg/sessionctx/vardef"
|
|
"github.com/pingcap/tidb/pkg/sessionctx/variable"
|
|
storepkg "github.com/pingcap/tidb/pkg/store"
|
|
"github.com/pingcap/tidb/pkg/table/tables"
|
|
"github.com/pingcap/tidb/pkg/util/intest"
|
|
"github.com/pingcap/tidb/pkg/util/logutil"
|
|
"github.com/pingcap/tidb/pkg/util/sqlescape"
|
|
"github.com/pingcap/tidb/pkg/util/sqlexec"
|
|
"go.uber.org/zap"
|
|
)
|
|
|
|
// bootstrapOwnerKey is the key used by ddl owner mutex during boostrap.
|
|
var bootstrapOwnerKey = "/tidb/distributeDDLOwnerLock/"
|
|
|
|
// bootstrap initiates system DB for a store.
|
|
func bootstrap(s sessionapi.Session) {
|
|
startTime := time.Now()
|
|
err := InitMDLVariableForBootstrap(s.GetStore())
|
|
if err != nil {
|
|
logutil.BgLogger().Fatal("init metadata lock failed during bootstrap", zap.Error(err))
|
|
}
|
|
dom := domain.GetDomain(s)
|
|
bootLogger := logutil.SampleLoggerFactory(30*time.Second, 1)()
|
|
for {
|
|
b, err := checkBootstrapped(s)
|
|
if err != nil {
|
|
logutil.BgLogger().Fatal("check bootstrap error",
|
|
zap.Error(err))
|
|
}
|
|
// For rolling upgrade, we can't do upgrade only in the owner.
|
|
if b {
|
|
upgrade(s)
|
|
logutil.BgLogger().Info("upgrade successful in bootstrap",
|
|
zap.Duration("take time", time.Since(startTime)))
|
|
return
|
|
}
|
|
// To reduce conflict when multiple TiDB-server start at the same time.
|
|
// Actually only one server need to do the bootstrap. So we chose DDL owner to do this.
|
|
if dom.DDL().OwnerManager().IsOwner() {
|
|
doDDLWorks(s)
|
|
doDMLWorks(s)
|
|
runBootstrapSQLFile = true
|
|
logutil.BgLogger().Info("bootstrap successful",
|
|
zap.Duration("take time", time.Since(startTime)))
|
|
return
|
|
}
|
|
bootLogger.Info("bootstrap not done yet, waiting for owner to finish")
|
|
time.Sleep(200 * time.Millisecond)
|
|
}
|
|
}
|
|
|
|
const (
|
|
// varTrue is the true value in mysql.TiDB table for boolean columns.
|
|
varTrue = "True"
|
|
// varFalse is the false value in mysql.TiDB table for boolean columns.
|
|
varFalse = "False"
|
|
// The variable name in mysql.TiDB table.
|
|
// It is used for checking if the store is bootstrapped by any TiDB server.
|
|
// If the value is `True`, the store is already bootstrapped by a TiDB server.
|
|
bootstrappedVar = "bootstrapped"
|
|
// The variable name in mysql.TiDB table.
|
|
// It is used for getting the version of the TiDB server which bootstrapped the store.
|
|
tidbServerVersionVar = "tidb_server_version"
|
|
// The variable name in mysql.tidb table and it will be used when we want to know
|
|
// system timezone.
|
|
tidbSystemTZ = "system_tz"
|
|
// TidbNewCollationEnabled The variable name in mysql.tidb table and it will indicate if the new collations are enabled in the TiDB cluster.
|
|
TidbNewCollationEnabled = "new_collation_enabled"
|
|
// The variable name in mysql.tidb table and it records the default value of
|
|
// mem-quota-query when upgrade from v3.0.x to v4.0.9+.
|
|
tidbDefMemoryQuotaQuery = "default_memory_quota_query"
|
|
// The variable name in mysql.tidb table and it records the default value of
|
|
// oom-action when upgrade from v3.0.x to v4.0.11+.
|
|
tidbDefOOMAction = "default_oom_action"
|
|
// The variable name in mysql.tidb table and it records the current DDLTableVersion
|
|
tidbDDLTableVersion = "ddl_table_version"
|
|
// The variable name in mysql.tidb table and it records the cluster id of this cluster
|
|
tidbClusterID = "cluster_id"
|
|
)
|
|
|
|
// DDL owner key's expired time is ManagerSessionTTL seconds, we should wait the time and give more time to have a chance to finish it.
|
|
var internalSQLTimeout = owner.ManagerSessionTTL + 15
|
|
|
|
// whether to run the sql file in bootstrap.
|
|
var runBootstrapSQLFile = false
|
|
|
|
// DisableRunBootstrapSQLFileInTest only used for test
|
|
func DisableRunBootstrapSQLFileInTest() {
|
|
if intest.InTest {
|
|
runBootstrapSQLFile = false
|
|
}
|
|
}
|
|
|
|
func checkBootstrapped(s sessionapi.Session) (bool, error) {
|
|
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap)
|
|
// Check if system db exists.
|
|
_, err := s.ExecuteInternal(ctx, "USE %n", mysql.SystemDB)
|
|
if err != nil && infoschema.ErrDatabaseNotExists.NotEqual(err) {
|
|
logutil.BgLogger().Fatal("check bootstrap error",
|
|
zap.Error(err))
|
|
}
|
|
// Check bootstrapped variable value in TiDB table.
|
|
sVal, _, err := getTiDBVar(s, bootstrappedVar)
|
|
if err != nil {
|
|
if infoschema.ErrTableNotExists.Equal(err) {
|
|
return false, nil
|
|
}
|
|
return false, errors.Trace(err)
|
|
}
|
|
isBootstrapped := sVal == varTrue
|
|
if isBootstrapped {
|
|
// Make sure that doesn't affect the following operations.
|
|
if err = s.CommitTxn(ctx); err != nil {
|
|
return false, errors.Trace(err)
|
|
}
|
|
}
|
|
return isBootstrapped, nil
|
|
}
|
|
|
|
// getTiDBVar gets variable value from mysql.tidb table.
|
|
// Those variables are used by TiDB server.
|
|
func getTiDBVar(s sessionapi.Session, name string) (sVal string, isNull bool, e error) {
|
|
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap)
|
|
rs, err := s.ExecuteInternal(ctx, `SELECT HIGH_PRIORITY VARIABLE_VALUE FROM %n.%n WHERE VARIABLE_NAME= %?`,
|
|
mysql.SystemDB,
|
|
mysql.TiDBTable,
|
|
name,
|
|
)
|
|
if err != nil {
|
|
return "", true, errors.Trace(err)
|
|
}
|
|
if rs == nil {
|
|
return "", true, errors.New("Wrong number of Recordset")
|
|
}
|
|
defer terror.Call(rs.Close)
|
|
req := rs.NewChunk(nil)
|
|
err = rs.Next(ctx, req)
|
|
if err != nil || req.NumRows() == 0 {
|
|
return "", true, errors.Trace(err)
|
|
}
|
|
row := req.GetRow(0)
|
|
if row.IsNull(0) {
|
|
return "", true, nil
|
|
}
|
|
return row.GetString(0), false, nil
|
|
}
|
|
|
|
var (
|
|
// SupportUpgradeHTTPOpVer is exported for testing.
|
|
// The minimum version of the upgrade by paused user DDL can be notified through the HTTP API.
|
|
SupportUpgradeHTTPOpVer int64 = version174
|
|
)
|
|
|
|
func acquireLock(store kv.Storage) (func(), error) {
|
|
etcdCli, err := storepkg.NewEtcdCli(store)
|
|
if err != nil {
|
|
return nil, errors.Trace(err)
|
|
}
|
|
if etcdCli == nil {
|
|
// Special handling for test.
|
|
logutil.BgLogger().Warn("skip acquire ddl owner lock for uni-store")
|
|
return func() {
|
|
// do nothing
|
|
}, nil
|
|
}
|
|
releaseFn, err := owner.AcquireDistributedLock(context.Background(), etcdCli, bootstrapOwnerKey, 10)
|
|
if err != nil {
|
|
if err2 := etcdCli.Close(); err2 != nil {
|
|
logutil.BgLogger().Error("failed to close etcd client", zap.Error(err2))
|
|
}
|
|
return nil, errors.Trace(err)
|
|
}
|
|
return func() {
|
|
releaseFn()
|
|
if err2 := etcdCli.Close(); err2 != nil {
|
|
logutil.BgLogger().Error("failed to close etcd client", zap.Error(err2))
|
|
}
|
|
}, nil
|
|
}
|
|
|
|
// initGlobalVariableIfNotExists initialize a global variable with specific val if it does not exist.
|
|
func initGlobalVariableIfNotExists(s sessionapi.Session, name string, val any) {
|
|
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap)
|
|
rows, err := sqlexec.ExecSQL(ctx, s, "SELECT VARIABLE_VALUE FROM %n.%n WHERE VARIABLE_NAME=%?;", mysql.SystemDB, mysql.GlobalVariablesTable, name)
|
|
terror.MustNil(err)
|
|
if len(rows) != 0 {
|
|
return
|
|
}
|
|
|
|
mustExecute(s, "INSERT HIGH_PRIORITY IGNORE INTO %n.%n VALUES (%?, %?);",
|
|
mysql.SystemDB, mysql.GlobalVariablesTable, name, val)
|
|
}
|
|
|
|
func writeOOMAction(s sessionapi.Session) {
|
|
comment := "oom-action is `log` by default in v3.0.x, `cancel` by default in v4.0.11+"
|
|
mustExecute(s, `INSERT HIGH_PRIORITY INTO %n.%n VALUES (%?, %?, %?) ON DUPLICATE KEY UPDATE VARIABLE_VALUE= %?`,
|
|
mysql.SystemDB, mysql.TiDBTable, tidbDefOOMAction, vardef.OOMActionLog, comment, vardef.OOMActionLog,
|
|
)
|
|
}
|
|
|
|
// updateBootstrapVer updates bootstrap version variable in mysql.TiDB table.
|
|
func updateBootstrapVer(s sessionapi.Session) {
|
|
// Update bootstrap version.
|
|
mustExecute(s, `INSERT HIGH_PRIORITY INTO %n.%n VALUES (%?, %?, "TiDB bootstrap version.") ON DUPLICATE KEY UPDATE VARIABLE_VALUE=%?`,
|
|
mysql.SystemDB, mysql.TiDBTable, tidbServerVersionVar, currentBootstrapVersion, currentBootstrapVersion,
|
|
)
|
|
}
|
|
|
|
// getBootstrapVersion gets bootstrap version from mysql.tidb table;
|
|
func getBootstrapVersion(s sessionapi.Session) (int64, error) {
|
|
sVal, isNull, err := getTiDBVar(s, tidbServerVersionVar)
|
|
if err != nil {
|
|
return 0, errors.Trace(err)
|
|
}
|
|
if isNull {
|
|
return 0, nil
|
|
}
|
|
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.
|
|
var tablesInSystemDatabase = []TableBasicInfo{
|
|
{ID: metadef.UserTableID, Name: "user", SQL: metadef.CreateUserTable},
|
|
{ID: metadef.PasswordHistoryTableID, Name: "password_history", SQL: metadef.CreatePasswordHistoryTable},
|
|
{ID: metadef.GlobalPrivTableID, Name: "global_priv", SQL: metadef.CreateGlobalPrivTable},
|
|
{ID: metadef.DBTableID, Name: "db", SQL: metadef.CreateDBTable},
|
|
{ID: metadef.TablesPrivTableID, Name: "tables_priv", SQL: metadef.CreateTablesPrivTable},
|
|
{ID: metadef.ColumnsPrivTableID, Name: "columns_priv", SQL: metadef.CreateColumnsPrivTable},
|
|
{ID: metadef.GlobalVariablesTableID, Name: "global_variables", SQL: metadef.CreateGlobalVariablesTable},
|
|
{ID: metadef.TiDBTableID, Name: "tidb", SQL: metadef.CreateTiDBTable},
|
|
{ID: metadef.HelpTopicTableID, Name: "help_topic", SQL: metadef.CreateHelpTopicTable},
|
|
{ID: metadef.StatsMetaTableID, Name: "stats_meta", SQL: metadef.CreateStatsMetaTable},
|
|
{ID: metadef.StatsHistogramsTableID, Name: "stats_histograms", SQL: metadef.CreateStatsHistogramsTable},
|
|
{ID: metadef.StatsBucketsTableID, Name: "stats_buckets", SQL: metadef.CreateStatsBucketsTable},
|
|
{ID: metadef.GCDeleteRangeTableID, Name: "gc_delete_range", SQL: metadef.CreateGCDeleteRangeTable},
|
|
{ID: metadef.GCDeleteRangeDoneTableID, Name: "gc_delete_range_done", SQL: metadef.CreateGCDeleteRangeDoneTable},
|
|
{ID: metadef.StatsFeedbackTableID, Name: "stats_feedback", SQL: metadef.CreateStatsFeedbackTable},
|
|
{ID: metadef.RoleEdgesTableID, Name: "role_edges", SQL: metadef.CreateRoleEdgesTable},
|
|
{ID: metadef.DefaultRolesTableID, Name: "default_roles", SQL: metadef.CreateDefaultRolesTable},
|
|
{ID: metadef.BindInfoTableID, Name: "bind_info", SQL: metadef.CreateBindInfoTable},
|
|
{ID: metadef.StatsTopNTableID, Name: "stats_top_n", SQL: metadef.CreateStatsTopNTable},
|
|
{ID: metadef.ExprPushdownBlacklistTableID, Name: "expr_pushdown_blacklist", SQL: metadef.CreateExprPushdownBlacklistTable},
|
|
{ID: metadef.OptRuleBlacklistTableID, Name: "opt_rule_blacklist", SQL: metadef.CreateOptRuleBlacklistTable},
|
|
{ID: metadef.StatsExtendedTableID, Name: "stats_extended", SQL: metadef.CreateStatsExtendedTable},
|
|
{ID: metadef.StatsFMSketchTableID, Name: "stats_fm_sketch", SQL: metadef.CreateStatsFMSketchTable},
|
|
{ID: metadef.GlobalGrantsTableID, Name: "global_grants", SQL: metadef.CreateGlobalGrantsTable},
|
|
{ID: metadef.CapturePlanBaselinesBlacklistTableID, Name: "capture_plan_baselines_blacklist", SQL: metadef.CreateCapturePlanBaselinesBlacklistTable},
|
|
{ID: metadef.ColumnStatsUsageTableID, Name: "column_stats_usage", SQL: metadef.CreateColumnStatsUsageTable},
|
|
{ID: metadef.TableCacheMetaTableID, Name: "table_cache_meta", SQL: metadef.CreateTableCacheMetaTable},
|
|
{ID: metadef.AnalyzeOptionsTableID, Name: "analyze_options", SQL: metadef.CreateAnalyzeOptionsTable},
|
|
{ID: metadef.StatsHistoryTableID, Name: "stats_history", SQL: metadef.CreateStatsHistoryTable},
|
|
{ID: metadef.StatsMetaHistoryTableID, Name: "stats_meta_history", SQL: metadef.CreateStatsMetaHistoryTable},
|
|
{ID: metadef.AnalyzeJobsTableID, Name: "analyze_jobs", SQL: metadef.CreateAnalyzeJobsTable},
|
|
{ID: metadef.AdvisoryLocksTableID, Name: "advisory_locks", SQL: metadef.CreateAdvisoryLocksTable},
|
|
{ID: metadef.PlanReplayerStatusTableID, Name: "plan_replayer_status", SQL: metadef.CreatePlanReplayerStatusTable},
|
|
{ID: metadef.PlanReplayerTaskTableID, Name: "plan_replayer_task", SQL: metadef.CreatePlanReplayerTaskTable},
|
|
{ID: metadef.StatsTableLockedTableID, Name: "stats_table_locked", SQL: metadef.CreateStatsTableLockedTable},
|
|
{ID: metadef.TiDBTTLTableStatusTableID, Name: "tidb_ttl_table_status", SQL: metadef.CreateTiDBTTLTableStatusTable},
|
|
{ID: metadef.TiDBTTLTaskTableID, Name: "tidb_ttl_task", SQL: metadef.CreateTiDBTTLTaskTable},
|
|
{ID: metadef.TiDBTTLJobHistoryTableID, Name: "tidb_ttl_job_history", SQL: metadef.CreateTiDBTTLJobHistoryTable},
|
|
{ID: metadef.TiDBGlobalTaskTableID, Name: "tidb_global_task", SQL: metadef.CreateTiDBGlobalTaskTable},
|
|
{ID: metadef.TiDBGlobalTaskHistoryTableID, Name: "tidb_global_task_history", SQL: metadef.CreateTiDBGlobalTaskHistoryTable},
|
|
{ID: metadef.TiDBImportJobsTableID, Name: "tidb_import_jobs", SQL: metadef.CreateTiDBImportJobsTable},
|
|
{ID: metadef.TiDBRunawayWatchTableID, Name: "tidb_runaway_watch", SQL: metadef.CreateTiDBRunawayWatchTable},
|
|
{ID: metadef.TiDBRunawayQueriesTableID, Name: "tidb_runaway_queries", SQL: metadef.CreateTiDBRunawayQueriesTable},
|
|
{ID: metadef.TiDBTimersTableID, Name: "tidb_timers", SQL: metadef.CreateTiDBTimersTable},
|
|
{ID: metadef.TiDBRunawayWatchDoneTableID, Name: "tidb_runaway_watch_done", SQL: metadef.CreateTiDBRunawayWatchDoneTable},
|
|
{ID: metadef.DistFrameworkMetaTableID, Name: "dist_framework_meta", SQL: metadef.CreateDistFrameworkMetaTable},
|
|
{ID: metadef.RequestUnitByGroupTableID, Name: "request_unit_by_group", SQL: metadef.CreateRequestUnitByGroupTable},
|
|
{ID: metadef.TiDBPITRIDMapTableID, Name: "tidb_pitr_id_map", SQL: metadef.CreateTiDBPITRIDMapTable},
|
|
{ID: metadef.TiDBRestoreRegistryTableID, Name: "tidb_restore_registry", SQL: metadef.CreateTiDBRestoreRegistryTable},
|
|
{ID: metadef.IndexAdvisorResultsTableID, Name: "index_advisor_results", SQL: metadef.CreateIndexAdvisorResultsTable},
|
|
{ID: metadef.TiDBKernelOptionsTableID, Name: "tidb_kernel_options", SQL: metadef.CreateTiDBKernelOptionsTable},
|
|
{ID: metadef.TiDBWorkloadValuesTableID, Name: "tidb_workload_values", SQL: metadef.CreateTiDBWorkloadValuesTable},
|
|
// NOTE: if you need to add more tables to 'mysql' database, please also add
|
|
// an entry to versionedBootstrapSchemas, to make sure the table is created
|
|
// correctly in nextgen kennel.
|
|
}
|
|
|
|
type versionedBootstrapSchema struct {
|
|
ver meta.NextGenBootTableVersion
|
|
databases []DatabaseBasicInfo
|
|
}
|
|
|
|
const (
|
|
// 52 is the number of system tables as we do this change.
|
|
// as tablesInSystemDatabase is shared with classic kernel, it's simple to
|
|
// use a slice to hold all system tables in classic kernel. but in nextgen,
|
|
// we need to make those tables versioned, as we don't create system tables
|
|
// through DDL, we need this version to avoid create tables again.
|
|
// if we add more system tables later, we should increase the version, and
|
|
// add another versionedBootstrapSchema entry.
|
|
tableCountInFirstVerOnNextGen = 52
|
|
)
|
|
|
|
// used in nextgen, to create system tables directly through meta kv, without
|
|
// going through DDL, so we can create them with reversed ID range.
|
|
var versionedBootstrapSchemas = []versionedBootstrapSchema{
|
|
{ver: meta.BaseNextGenBootTableVersion, databases: []DatabaseBasicInfo{
|
|
{ID: metadef.SystemDatabaseID, Name: mysql.SystemDB, Tables: tablesInSystemDatabase[:tableCountInFirstVerOnNextGen]},
|
|
{ID: metadef.SysDatabaseID, Name: mysql.SysDB},
|
|
}},
|
|
}
|
|
|
|
func bootstrapSchemas(store kv.Storage) error {
|
|
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL)
|
|
return kv.RunInNewTxn(ctx, store, true, func(_ context.Context, txn kv.Transaction) error {
|
|
m := meta.NewMutator(txn)
|
|
currVer, err := m.GetNextGenBootTableVersion()
|
|
if err != nil {
|
|
return errors.Trace(err)
|
|
}
|
|
|
|
largestVer := currVer
|
|
for _, vt := range versionedBootstrapSchemas {
|
|
if currVer >= vt.ver {
|
|
continue
|
|
}
|
|
logutil.BgLogger().Info("bootstrap tables", zap.Int("currVer", int(currVer)),
|
|
zap.Int("targetVer", int(vt.ver)))
|
|
for _, bdb := range vt.databases {
|
|
if err = m.CreateSysDatabaseByIDIfNotExists(bdb.Name, bdb.ID); err != nil {
|
|
return err
|
|
}
|
|
if len(bdb.Tables) > 0 {
|
|
if err = createAndSplitTables(store, m, bdb.ID, bdb.Tables); err != nil {
|
|
return err
|
|
}
|
|
}
|
|
}
|
|
largestVer = max(largestVer, vt.ver)
|
|
}
|
|
if largestVer > currVer {
|
|
return m.SetNextGenBootTableVersion(largestVer)
|
|
}
|
|
return nil
|
|
})
|
|
}
|
|
|
|
// doDDLWorks executes DDL statements in bootstrap stage.
|
|
func doDDLWorks(s sessionapi.Session) {
|
|
// for nextgen, system schemas are created in bootstrapSessionImpl
|
|
if kerneltype.IsClassic() {
|
|
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.
|
|
insertBuiltinBindInfoRow(s)
|
|
// Create `mysql.tidb_mdl_view` view.
|
|
mustExecute(s, metadef.CreateTiDBMDLView)
|
|
// create `sys.schema_unused_indexes` view
|
|
mustExecute(s, metadef.CreateSchemaUnusedIndexesView)
|
|
// Create a test database.
|
|
mustExecute(s, "CREATE DATABASE IF NOT EXISTS test")
|
|
}
|
|
|
|
func checkSystemTableConstraint(tblInfo *model.TableInfo) error {
|
|
if tblInfo.Partition != nil {
|
|
return errors.New("system table should not be partitioned table")
|
|
}
|
|
if tblInfo.SepAutoInc() {
|
|
// AUTO_ID_CACHE=1 is implemented through GRPC service and requires owner
|
|
// election, system tables should not depend on that.
|
|
return errors.New("system table should not use AUTO_ID_CACHE=1")
|
|
}
|
|
return nil
|
|
}
|
|
|
|
// doBootstrapSQLFile executes SQL commands in a file as the last stage of bootstrap.
|
|
// It is useful for setting the initial value of GLOBAL variables.
|
|
func doBootstrapSQLFile(s sessionapi.Session) error {
|
|
sqlFile := config.GetGlobalConfig().InitializeSQLFile
|
|
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap)
|
|
if sqlFile == "" {
|
|
return nil
|
|
}
|
|
logutil.BgLogger().Info("executing -initialize-sql-file", zap.String("file", sqlFile))
|
|
b, err := os.ReadFile(sqlFile) //nolint:gosec
|
|
if err != nil {
|
|
if intest.InTest {
|
|
return err
|
|
}
|
|
logutil.BgLogger().Fatal("unable to read InitializeSQLFile", zap.Error(err))
|
|
}
|
|
stmts, err := s.Parse(ctx, string(b))
|
|
if err != nil {
|
|
if intest.InTest {
|
|
return err
|
|
}
|
|
logutil.BgLogger().Fatal("unable to parse InitializeSQLFile", zap.Error(err))
|
|
}
|
|
for _, stmt := range stmts {
|
|
rs, err := s.ExecuteStmt(ctx, stmt)
|
|
if err != nil {
|
|
logutil.BgLogger().Warn("InitializeSQLFile error", zap.Error(err))
|
|
}
|
|
if rs != nil {
|
|
// I don't believe we need to drain the result-set in bootstrap mode
|
|
// but if required we can do this here in future.
|
|
if err := rs.Close(); err != nil {
|
|
logutil.BgLogger().Fatal("unable to close result", zap.Error(err))
|
|
}
|
|
}
|
|
}
|
|
return nil
|
|
}
|
|
|
|
// doDMLWorks executes DML statements in bootstrap stage.
|
|
// All the statements run in a single transaction.
|
|
func doDMLWorks(s sessionapi.Session) {
|
|
mustExecute(s, "BEGIN")
|
|
if config.GetGlobalConfig().Security.SecureBootstrap {
|
|
// If secure bootstrap is enabled, we create a root@localhost account which can login with auth_socket.
|
|
// i.e. mysql -S /tmp/tidb.sock -uroot
|
|
// The auth_socket plugin will validate that the user matches $USER.
|
|
u, err := osuser.Current()
|
|
if err != nil {
|
|
logutil.BgLogger().Fatal("failed to read current user. unable to secure bootstrap.", zap.Error(err))
|
|
}
|
|
mustExecute(s, `INSERT HIGH_PRIORITY INTO mysql.user (Host,User,authentication_string,plugin,Select_priv,Insert_priv,Update_priv,Delete_priv,Create_priv,Drop_priv,Process_priv,Grant_priv,References_priv,Alter_priv,Show_db_priv,
|
|
Super_priv,Create_tmp_table_priv,Lock_tables_priv,Execute_priv,Create_view_priv,Show_view_priv,Create_routine_priv,Alter_routine_priv,Index_priv,Create_user_priv,Event_priv,Repl_slave_priv,Repl_client_priv,Trigger_priv,Create_role_priv,Drop_role_priv,Account_locked,
|
|
Shutdown_priv,Reload_priv,FILE_priv,Config_priv,Create_Tablespace_Priv,User_attributes,Token_issuer) VALUES
|
|
("localhost", "root", %?, "auth_socket", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "N", "Y", "Y", "Y", "Y", "Y", null, "")`, u.Username)
|
|
} else {
|
|
mustExecute(s, `INSERT HIGH_PRIORITY INTO mysql.user (Host,User,authentication_string,plugin,Select_priv,Insert_priv,Update_priv,Delete_priv,Create_priv,Drop_priv,Process_priv,Grant_priv,References_priv,Alter_priv,Show_db_priv,
|
|
Super_priv,Create_tmp_table_priv,Lock_tables_priv,Execute_priv,Create_view_priv,Show_view_priv,Create_routine_priv,Alter_routine_priv,Index_priv,Create_user_priv,Event_priv,Repl_slave_priv,Repl_client_priv,Trigger_priv,Create_role_priv,Drop_role_priv,Account_locked,
|
|
Shutdown_priv,Reload_priv,FILE_priv,Config_priv,Create_Tablespace_Priv,User_attributes,Token_issuer) VALUES
|
|
("%", "root", "", "mysql_native_password", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "N", "Y", "Y", "Y", "Y", "Y", null, "")`)
|
|
}
|
|
|
|
// For GLOBAL scoped system variables, insert the initial value
|
|
// into the mysql.global_variables table. This is only run on initial
|
|
// bootstrap, and in some cases we will use a different default value
|
|
// for new installs versus existing installs.
|
|
|
|
values := make([]string, 0, len(variable.GetSysVars()))
|
|
for k, v := range variable.GetSysVars() {
|
|
if !v.HasGlobalScope() {
|
|
continue
|
|
}
|
|
vVal := variable.GlobalSystemVariableInitialValue(v.Name, v.Value)
|
|
|
|
// sanitize k and vVal
|
|
value := fmt.Sprintf(`("%s", "%s")`, sqlescape.EscapeString(k), sqlescape.EscapeString(vVal))
|
|
values = append(values, value)
|
|
}
|
|
sql := fmt.Sprintf("INSERT HIGH_PRIORITY INTO %s.%s VALUES %s;", mysql.SystemDB, mysql.GlobalVariablesTable,
|
|
strings.Join(values, ", "))
|
|
mustExecute(s, sql)
|
|
|
|
mustExecute(s, `INSERT HIGH_PRIORITY INTO %n.%n VALUES(%?, %?, "Bootstrap flag. Do not delete.") ON DUPLICATE KEY UPDATE VARIABLE_VALUE=%?`,
|
|
mysql.SystemDB, mysql.TiDBTable, bootstrappedVar, varTrue, varTrue,
|
|
)
|
|
|
|
mustExecute(s, `INSERT HIGH_PRIORITY INTO %n.%n VALUES(%?, %?, "Bootstrap version. Do not delete.")`,
|
|
mysql.SystemDB, mysql.TiDBTable, tidbServerVersionVar, currentBootstrapVersion,
|
|
)
|
|
writeSystemTZ(s)
|
|
|
|
writeNewCollationParameter(s, config.GetGlobalConfig().NewCollationsEnabledOnFirstBootstrap)
|
|
|
|
writeStmtSummaryVars(s)
|
|
|
|
writeDDLTableVersion(s)
|
|
|
|
writeClusterID(s)
|
|
|
|
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap)
|
|
_, err := s.ExecuteInternal(ctx, "COMMIT")
|
|
if err != nil {
|
|
sleepTime := 1 * time.Second
|
|
logutil.BgLogger().Info("doDMLWorks failed", zap.Error(err), zap.Duration("sleeping time", sleepTime))
|
|
time.Sleep(sleepTime)
|
|
// Check if TiDB is already bootstrapped.
|
|
b, err1 := checkBootstrapped(s)
|
|
if err1 != nil {
|
|
logutil.BgLogger().Fatal("doDMLWorks failed", zap.Error(err1))
|
|
}
|
|
if b {
|
|
return
|
|
}
|
|
logutil.BgLogger().Fatal("doDMLWorks failed", zap.Error(err))
|
|
}
|
|
}
|
|
|
|
func mustExecute(s sessionapi.Session, sql string, args ...any) {
|
|
ctx, cancel := context.WithTimeout(context.Background(), time.Duration(internalSQLTimeout)*time.Second)
|
|
ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnBootstrap)
|
|
_, err := s.ExecuteInternal(ctx, sql, args...)
|
|
defer cancel()
|
|
if err != nil {
|
|
logutil.BgLogger().Fatal("mustExecute error", zap.Error(err), zap.Stack("stack"))
|
|
}
|
|
}
|
|
|
|
// oldPasswordUpgrade upgrade password to MySQL compatible format
|
|
func oldPasswordUpgrade(pass string) (string, error) {
|
|
hash1, err := hex.DecodeString(pass)
|
|
if err != nil {
|
|
return "", errors.Trace(err)
|
|
}
|
|
|
|
hash2 := auth.Sha1Hash(hash1)
|
|
newpass := fmt.Sprintf("*%X", hash2)
|
|
return newpass, nil
|
|
}
|
|
|
|
// rebuildAllPartitionValueMapAndSorted rebuilds all value map and sorted info for list column partitions with InfoSchema.
|
|
func rebuildAllPartitionValueMapAndSorted(ctx context.Context, s *session) {
|
|
type partitionExpr interface {
|
|
PartitionExpr() *tables.PartitionExpr
|
|
}
|
|
|
|
p := parser.New()
|
|
is := s.GetInfoSchema().(infoschema.InfoSchema)
|
|
dbs := is.ListTablesWithSpecialAttribute(infoschemacontext.PartitionAttribute)
|
|
for _, db := range dbs {
|
|
for _, t := range db.TableInfos {
|
|
pi := t.GetPartitionInfo()
|
|
if pi == nil || pi.Type != ast.PartitionTypeList {
|
|
continue
|
|
}
|
|
tbl, ok := is.TableByID(ctx, t.ID)
|
|
intest.Assert(ok, "table not found in infoschema")
|
|
pe := tbl.(partitionExpr).PartitionExpr()
|
|
for _, cp := range pe.ColPrunes {
|
|
if err := cp.RebuildPartitionValueMapAndSorted(p, pi.Definitions); err != nil {
|
|
logutil.BgLogger().Warn("build list column partition value map and sorted failed")
|
|
break
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|