From 44c00a9ce73e45c260b3e386f3e1021d1ba1ddb2 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 9 Oct 2024 19:15:33 +0200 Subject: [PATCH] This is an automated cherry-pick of #56082 Signed-off-by: ti-chi-bot --- pkg/ddl/executor.go | 6776 +++++++++++++++++ pkg/ddl/partition.go | 172 +- pkg/ddl/rollingback.go | 158 + pkg/ddl/tests/partition/BUILD.bazel | 10 + pkg/ddl/tests/partition/multi_domain_test.go | 342 + pkg/meta/model/table.go | 1277 ++++ .../integration_partition_suite_out.json | 14 +- .../testdata/partition_pruner_in.json | 56 +- .../testdata/partition_pruner_out.json | 112 +- pkg/planner/core/partition_prune.go | 49 + pkg/planner/core/point_get_plan.go | 322 + pkg/planner/core/rule_partition_processor.go | 65 +- pkg/table/tables/partition.go | 40 +- pkg/testkit/mockstore.go | 7 +- .../integrationtest/r/ddl/db_partition.result | 4 +- .../planner/core/integration_partition.result | 6 + .../r/planner/core/partition_pruner.result | 4 +- 17 files changed, 9292 insertions(+), 122 deletions(-) create mode 100644 pkg/ddl/executor.go create mode 100644 pkg/ddl/tests/partition/multi_domain_test.go create mode 100644 pkg/meta/model/table.go diff --git a/pkg/ddl/executor.go b/pkg/ddl/executor.go new file mode 100644 index 0000000000000..3fa5c1c147590 --- /dev/null +++ b/pkg/ddl/executor.go @@ -0,0 +1,6776 @@ +// Copyright 2016 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 ddl + +import ( + "bytes" + "context" + "fmt" + "math" + "strings" + "sync/atomic" + "time" + "unicode/utf8" + + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/pkg/config" + "github.com/pingcap/tidb/pkg/ddl/label" + "github.com/pingcap/tidb/pkg/ddl/logutil" + "github.com/pingcap/tidb/pkg/ddl/resourcegroup" + sess "github.com/pingcap/tidb/pkg/ddl/session" + ddlutil "github.com/pingcap/tidb/pkg/ddl/util" + "github.com/pingcap/tidb/pkg/errctx" + "github.com/pingcap/tidb/pkg/expression" + "github.com/pingcap/tidb/pkg/infoschema" + "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/meta" + "github.com/pingcap/tidb/pkg/meta/autoid" + "github.com/pingcap/tidb/pkg/meta/metabuild" + "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/metrics" + "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/parser/charset" + pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/parser/terror" + "github.com/pingcap/tidb/pkg/privilege" + rg "github.com/pingcap/tidb/pkg/resourcegroup" + "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessiontxn" + "github.com/pingcap/tidb/pkg/statistics/handle" + "github.com/pingcap/tidb/pkg/table" + "github.com/pingcap/tidb/pkg/table/tables" + "github.com/pingcap/tidb/pkg/tablecodec" + "github.com/pingcap/tidb/pkg/types" + "github.com/pingcap/tidb/pkg/util" + "github.com/pingcap/tidb/pkg/util/collate" + "github.com/pingcap/tidb/pkg/util/dbterror" + "github.com/pingcap/tidb/pkg/util/dbterror/exeerrors" + "github.com/pingcap/tidb/pkg/util/domainutil" + "github.com/pingcap/tidb/pkg/util/generic" + "github.com/pingcap/tidb/pkg/util/intest" + "github.com/pingcap/tidb/pkg/util/mathutil" + "github.com/pingcap/tidb/pkg/util/stringutil" + "github.com/tikv/client-go/v2/oracle" + pdhttp "github.com/tikv/pd/client/http" + "go.uber.org/zap" +) + +const ( + expressionIndexPrefix = "_V$" + changingColumnPrefix = "_Col$_" + changingIndexPrefix = "_Idx$_" + tableNotExist = -1 + tinyBlobMaxLength = 255 + blobMaxLength = 65535 + mediumBlobMaxLength = 16777215 + longBlobMaxLength = 4294967295 + // When setting the placement policy with "PLACEMENT POLICY `default`", + // it means to remove placement policy from the specified object. + defaultPlacementPolicyName = "default" + tiflashCheckPendingTablesWaitTime = 3000 * time.Millisecond + // Once tiflashCheckPendingTablesLimit is reached, we trigger a limiter detection. + tiflashCheckPendingTablesLimit = 100 + tiflashCheckPendingTablesRetry = 7 +) + +var errCheckConstraintIsOff = errors.NewNoStackError(variable.TiDBEnableCheckConstraint + " is off") + +// Executor is the interface for executing DDL statements. +// it's mostly called by SQL executor. +// DDL statements are converted into DDL jobs, JobSubmitter will submit the jobs +// to DDL job table. Then jobScheduler will schedule them to run on workers +// asynchronously in parallel. Executor will wait them to finish. +type Executor interface { + CreateSchema(ctx sessionctx.Context, stmt *ast.CreateDatabaseStmt) error + AlterSchema(sctx sessionctx.Context, stmt *ast.AlterDatabaseStmt) error + DropSchema(ctx sessionctx.Context, stmt *ast.DropDatabaseStmt) error + CreateTable(ctx sessionctx.Context, stmt *ast.CreateTableStmt) error + CreateView(ctx sessionctx.Context, stmt *ast.CreateViewStmt) error + DropTable(ctx sessionctx.Context, stmt *ast.DropTableStmt) (err error) + RecoverTable(ctx sessionctx.Context, recoverTableInfo *model.RecoverTableInfo) (err error) + RecoverSchema(ctx sessionctx.Context, recoverSchemaInfo *model.RecoverSchemaInfo) error + DropView(ctx sessionctx.Context, stmt *ast.DropTableStmt) (err error) + CreateIndex(ctx sessionctx.Context, stmt *ast.CreateIndexStmt) error + DropIndex(ctx sessionctx.Context, stmt *ast.DropIndexStmt) error + AlterTable(ctx context.Context, sctx sessionctx.Context, stmt *ast.AlterTableStmt) error + TruncateTable(ctx sessionctx.Context, tableIdent ast.Ident) error + RenameTable(ctx sessionctx.Context, stmt *ast.RenameTableStmt) error + LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error + UnlockTables(ctx sessionctx.Context, lockedTables []model.TableLockTpInfo) error + CleanupTableLock(ctx sessionctx.Context, tables []*ast.TableName) error + UpdateTableReplicaInfo(ctx sessionctx.Context, physicalID int64, available bool) error + RepairTable(ctx sessionctx.Context, createStmt *ast.CreateTableStmt) error + CreateSequence(ctx sessionctx.Context, stmt *ast.CreateSequenceStmt) error + DropSequence(ctx sessionctx.Context, stmt *ast.DropSequenceStmt) (err error) + AlterSequence(ctx sessionctx.Context, stmt *ast.AlterSequenceStmt) error + CreatePlacementPolicy(ctx sessionctx.Context, stmt *ast.CreatePlacementPolicyStmt) error + DropPlacementPolicy(ctx sessionctx.Context, stmt *ast.DropPlacementPolicyStmt) error + AlterPlacementPolicy(ctx sessionctx.Context, stmt *ast.AlterPlacementPolicyStmt) error + AddResourceGroup(ctx sessionctx.Context, stmt *ast.CreateResourceGroupStmt) error + AlterResourceGroup(ctx sessionctx.Context, stmt *ast.AlterResourceGroupStmt) error + DropResourceGroup(ctx sessionctx.Context, stmt *ast.DropResourceGroupStmt) error + FlashbackCluster(ctx sessionctx.Context, flashbackTS uint64) error + + // CreateSchemaWithInfo creates a database (schema) given its database info. + // + // WARNING: the DDL owns the `info` after calling this function, and will modify its fields + // in-place. If you want to keep using `info`, please call Clone() first. + CreateSchemaWithInfo( + ctx sessionctx.Context, + info *model.DBInfo, + onExist OnExist) error + + // CreateTableWithInfo creates a table, view or sequence given its table info. + // + // WARNING: the DDL owns the `info` after calling this function, and will modify its fields + // in-place. If you want to keep using `info`, please call Clone() first. + CreateTableWithInfo( + ctx sessionctx.Context, + schema pmodel.CIStr, + info *model.TableInfo, + involvingRef []model.InvolvingSchemaInfo, + cs ...CreateTableOption) error + + // BatchCreateTableWithInfo is like CreateTableWithInfo, but can handle multiple tables. + BatchCreateTableWithInfo(ctx sessionctx.Context, + schema pmodel.CIStr, + info []*model.TableInfo, + cs ...CreateTableOption) error + + // CreatePlacementPolicyWithInfo creates a placement policy + // + // WARNING: the DDL owns the `policy` after calling this function, and will modify its fields + // in-place. If you want to keep using `policy`, please call Clone() first. + CreatePlacementPolicyWithInfo(ctx sessionctx.Context, policy *model.PolicyInfo, onExist OnExist) error +} + +// ExecutorForTest is the interface for executing DDL statements in tests. +// TODO remove it later +type ExecutorForTest interface { + // DoDDLJob does the DDL job, it's exported for test. + DoDDLJob(ctx sessionctx.Context, job *model.Job) error + // DoDDLJobWrapper similar to DoDDLJob, but with JobWrapper as input. + DoDDLJobWrapper(ctx sessionctx.Context, jobW *JobWrapper) error +} + +// all fields are shared with ddl now. +type executor struct { + sessPool *sess.Pool + statsHandle *handle.Handle + + ctx context.Context + uuid string + store kv.Storage + autoidCli *autoid.ClientDiscover + infoCache *infoschema.InfoCache + limitJobCh chan *JobWrapper + lease time.Duration // lease is schema lease, default 45s, see config.Lease. + // ddlJobDoneChMap is used to notify the session that the DDL job is finished. + // jobID -> chan struct{} + ddlJobDoneChMap *generic.SyncMap[int64, chan struct{}] +} + +var _ Executor = (*executor)(nil) +var _ ExecutorForTest = (*executor)(nil) + +func (e *executor) CreateSchema(ctx sessionctx.Context, stmt *ast.CreateDatabaseStmt) (err error) { + var placementPolicyRef *model.PolicyRefInfo + sessionVars := ctx.GetSessionVars() + + // If no charset and/or collation is specified use collation_server and character_set_server + charsetOpt := ast.CharsetOpt{} + if sessionVars.GlobalVarsAccessor != nil { + charsetOpt.Col, err = sessionVars.GetSessionOrGlobalSystemVar(context.Background(), variable.CollationServer) + if err != nil { + return err + } + charsetOpt.Chs, err = sessionVars.GetSessionOrGlobalSystemVar(context.Background(), variable.CharacterSetServer) + if err != nil { + return err + } + } + + explicitCharset := false + explicitCollation := false + for _, val := range stmt.Options { + switch val.Tp { + case ast.DatabaseOptionCharset: + charsetOpt.Chs = val.Value + explicitCharset = true + case ast.DatabaseOptionCollate: + charsetOpt.Col = val.Value + explicitCollation = true + case ast.DatabaseOptionPlacementPolicy: + placementPolicyRef = &model.PolicyRefInfo{ + Name: pmodel.NewCIStr(val.Value), + } + } + } + + if charsetOpt.Col != "" { + coll, err := collate.GetCollationByName(charsetOpt.Col) + if err != nil { + return err + } + + // The collation is not valid for the specified character set. + // Try to remove any of them, but not if they are explicitly defined. + if coll.CharsetName != charsetOpt.Chs { + if explicitCollation && !explicitCharset { + // Use the explicitly set collation, not the implicit charset. + charsetOpt.Chs = "" + } + if !explicitCollation && explicitCharset { + // Use the explicitly set charset, not the (session) collation. + charsetOpt.Col = "" + } + } + } + if !explicitCollation && explicitCharset { + coll := getDefaultCollationForUTF8MB4(charsetOpt.Chs, ctx.GetSessionVars().DefaultCollationForUTF8MB4) + if len(coll) != 0 { + charsetOpt.Col = coll + } + } + dbInfo := &model.DBInfo{Name: stmt.Name} + chs, coll, err := ResolveCharsetCollation([]ast.CharsetOpt{charsetOpt}, ctx.GetSessionVars().DefaultCollationForUTF8MB4) + if err != nil { + return errors.Trace(err) + } + dbInfo.Charset = chs + dbInfo.Collate = coll + dbInfo.PlacementPolicyRef = placementPolicyRef + + onExist := OnExistError + if stmt.IfNotExists { + onExist = OnExistIgnore + } + return e.CreateSchemaWithInfo(ctx, dbInfo, onExist) +} + +func (e *executor) CreateSchemaWithInfo( + ctx sessionctx.Context, + dbInfo *model.DBInfo, + onExist OnExist, +) error { + is := e.infoCache.GetLatest() + _, ok := is.SchemaByName(dbInfo.Name) + if ok { + // since this error may be seen as error, keep it stack info. + err := infoschema.ErrDatabaseExists.GenWithStackByArgs(dbInfo.Name) + switch onExist { + case OnExistIgnore: + ctx.GetSessionVars().StmtCtx.AppendNote(err) + return nil + case OnExistError, OnExistReplace: + // FIXME: can we implement MariaDB's CREATE OR REPLACE SCHEMA? + return err + } + } + + if err := checkTooLongSchema(dbInfo.Name); err != nil { + return errors.Trace(err) + } + + if err := checkCharsetAndCollation(dbInfo.Charset, dbInfo.Collate); err != nil { + return errors.Trace(err) + } + + if err := handleDatabasePlacement(ctx, dbInfo); err != nil { + return errors.Trace(err) + } + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaName: dbInfo.Name.L, + Type: model.ActionCreateSchema, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + InvolvingSchemaInfo: []model.InvolvingSchemaInfo{{ + Database: dbInfo.Name.L, + Table: model.InvolvingAll, + }}, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.CreateSchemaArgs{ + DBInfo: dbInfo, + } + if ref := dbInfo.PlacementPolicyRef; ref != nil { + job.InvolvingSchemaInfo = append(job.InvolvingSchemaInfo, model.InvolvingSchemaInfo{ + Policy: ref.Name.L, + Mode: model.SharedInvolving, + }) + } + + err := e.doDDLJob2(ctx, job, args) + + if infoschema.ErrDatabaseExists.Equal(err) && onExist == OnExistIgnore { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + return nil + } + + return errors.Trace(err) +} + +func (e *executor) ModifySchemaCharsetAndCollate(ctx sessionctx.Context, stmt *ast.AlterDatabaseStmt, toCharset, toCollate string) (err error) { + if toCollate == "" { + if toCollate, err = GetDefaultCollation(toCharset, ctx.GetSessionVars().DefaultCollationForUTF8MB4); err != nil { + return errors.Trace(err) + } + } + + // Check if need to change charset/collation. + dbName := stmt.Name + is := e.infoCache.GetLatest() + dbInfo, ok := is.SchemaByName(dbName) + if !ok { + return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(dbName.O) + } + if dbInfo.Charset == toCharset && dbInfo.Collate == toCollate { + return nil + } + // Do the DDL job. + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: dbInfo.ID, + SchemaName: dbInfo.Name.L, + Type: model.ActionModifySchemaCharsetAndCollate, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + InvolvingSchemaInfo: []model.InvolvingSchemaInfo{{ + Database: dbInfo.Name.L, + Table: model.InvolvingAll, + }}, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.ModifySchemaArgs{ + ToCharset: toCharset, + ToCollate: toCollate, + } + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +func (e *executor) ModifySchemaDefaultPlacement(ctx sessionctx.Context, stmt *ast.AlterDatabaseStmt, placementPolicyRef *model.PolicyRefInfo) (err error) { + dbName := stmt.Name + is := e.infoCache.GetLatest() + dbInfo, ok := is.SchemaByName(dbName) + if !ok { + return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(dbName.O) + } + + if checkIgnorePlacementDDL(ctx) { + return nil + } + + placementPolicyRef, err = checkAndNormalizePlacementPolicy(ctx, placementPolicyRef) + if err != nil { + return err + } + + // Do the DDL job. + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: dbInfo.ID, + SchemaName: dbInfo.Name.L, + Type: model.ActionModifySchemaDefaultPlacement, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + InvolvingSchemaInfo: []model.InvolvingSchemaInfo{{ + Database: dbInfo.Name.L, + Table: model.InvolvingAll, + }}, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.ModifySchemaArgs{PolicyRef: placementPolicyRef} + + if placementPolicyRef != nil { + job.InvolvingSchemaInfo = append(job.InvolvingSchemaInfo, model.InvolvingSchemaInfo{ + Policy: placementPolicyRef.Name.L, + Mode: model.SharedInvolving, + }) + } + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +// getPendingTiFlashTableCount counts unavailable TiFlash replica by iterating all tables in infoCache. +func (e *executor) getPendingTiFlashTableCount(originVersion int64, pendingCount uint32) (int64, uint32) { + is := e.infoCache.GetLatest() + // If there are no schema change since last time(can be weird). + if is.SchemaMetaVersion() == originVersion { + return originVersion, pendingCount + } + cnt := uint32(0) + dbs := is.ListTablesWithSpecialAttribute(infoschema.TiFlashAttribute) + for _, db := range dbs { + if util.IsMemOrSysDB(db.DBName.L) { + continue + } + for _, tbl := range db.TableInfos { + if tbl.TiFlashReplica != nil && !tbl.TiFlashReplica.Available { + cnt++ + } + } + } + return is.SchemaMetaVersion(), cnt +} + +func isSessionDone(sctx sessionctx.Context) (bool, uint32) { + done := false + killed := sctx.GetSessionVars().SQLKiller.HandleSignal() == exeerrors.ErrQueryInterrupted + if killed { + return true, 1 + } + failpoint.Inject("BatchAddTiFlashSendDone", func(val failpoint.Value) { + done = val.(bool) + }) + return done, 0 +} + +func (e *executor) waitPendingTableThreshold(sctx sessionctx.Context, schemaID int64, tableID int64, originVersion int64, pendingCount uint32, threshold uint32) (bool, int64, uint32, bool) { + configRetry := tiflashCheckPendingTablesRetry + configWaitTime := tiflashCheckPendingTablesWaitTime + failpoint.Inject("FastFailCheckTiFlashPendingTables", func(value failpoint.Value) { + configRetry = value.(int) + configWaitTime = time.Millisecond * 200 + }) + + for retry := 0; retry < configRetry; retry++ { + done, killed := isSessionDone(sctx) + if done { + logutil.DDLLogger().Info("abort batch add TiFlash replica", zap.Int64("schemaID", schemaID), zap.Uint32("isKilled", killed)) + return true, originVersion, pendingCount, false + } + originVersion, pendingCount = e.getPendingTiFlashTableCount(originVersion, pendingCount) + delay := time.Duration(0) + if pendingCount < threshold { + // If there are not many unavailable tables, we don't need a force check. + return false, originVersion, pendingCount, false + } + logutil.DDLLogger().Info("too many unavailable tables, wait", + zap.Uint32("threshold", threshold), + zap.Uint32("currentPendingCount", pendingCount), + zap.Int64("schemaID", schemaID), + zap.Int64("tableID", tableID), + zap.Duration("time", configWaitTime)) + delay = configWaitTime + time.Sleep(delay) + } + logutil.DDLLogger().Info("too many unavailable tables, timeout", zap.Int64("schemaID", schemaID), zap.Int64("tableID", tableID)) + // If timeout here, we will trigger a ddl job, to force sync schema. However, it doesn't mean we remove limiter, + // so there is a force check immediately after that. + return false, originVersion, pendingCount, true +} + +func (e *executor) ModifySchemaSetTiFlashReplica(sctx sessionctx.Context, stmt *ast.AlterDatabaseStmt, tiflashReplica *ast.TiFlashReplicaSpec) error { + dbName := stmt.Name + is := e.infoCache.GetLatest() + dbInfo, ok := is.SchemaByName(dbName) + if !ok { + return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(dbName.O) + } + + if util.IsMemOrSysDB(dbInfo.Name.L) { + return errors.Trace(dbterror.ErrUnsupportedTiFlashOperationForSysOrMemTable) + } + + tbls, err := is.SchemaTableInfos(context.Background(), dbInfo.Name) + if err != nil { + return errors.Trace(err) + } + + total := len(tbls) + succ := 0 + skip := 0 + fail := 0 + oneFail := int64(0) + + if total == 0 { + return infoschema.ErrEmptyDatabase.GenWithStack("Empty database '%v'", dbName.O) + } + err = checkTiFlashReplicaCount(sctx, tiflashReplica.Count) + if err != nil { + return errors.Trace(err) + } + + var originVersion int64 + var pendingCount uint32 + forceCheck := false + + logutil.DDLLogger().Info("start batch add TiFlash replicas", zap.Int("total", total), zap.Int64("schemaID", dbInfo.ID)) + threshold := uint32(sctx.GetSessionVars().BatchPendingTiFlashCount) + + for _, tbl := range tbls { + done, killed := isSessionDone(sctx) + if done { + logutil.DDLLogger().Info("abort batch add TiFlash replica", zap.Int64("schemaID", dbInfo.ID), zap.Uint32("isKilled", killed)) + return nil + } + + tbReplicaInfo := tbl.TiFlashReplica + if !shouldModifyTiFlashReplica(tbReplicaInfo, tiflashReplica) { + logutil.DDLLogger().Info("skip repeated processing table", + zap.Int64("tableID", tbl.ID), + zap.Int64("schemaID", dbInfo.ID), + zap.String("tableName", tbl.Name.String()), + zap.String("schemaName", dbInfo.Name.String())) + skip++ + continue + } + + // If table is not supported, add err to warnings. + err = isTableTiFlashSupported(dbName, tbl) + if err != nil { + logutil.DDLLogger().Info("skip processing table", zap.Int64("tableID", tbl.ID), + zap.Int64("schemaID", dbInfo.ID), + zap.String("tableName", tbl.Name.String()), + zap.String("schemaName", dbInfo.Name.String()), + zap.Error(err)) + sctx.GetSessionVars().StmtCtx.AppendNote(err) + skip++ + continue + } + + // Alter `tiflashCheckPendingTablesLimit` tables are handled, we need to check if we have reached threshold. + if (succ+fail)%tiflashCheckPendingTablesLimit == 0 || forceCheck { + // We can execute one probing ddl to the latest schema, if we timeout in `pendingFunc`. + // However, we shall mark `forceCheck` to true, because we may still reach `threshold`. + finished := false + finished, originVersion, pendingCount, forceCheck = e.waitPendingTableThreshold(sctx, dbInfo.ID, tbl.ID, originVersion, pendingCount, threshold) + if finished { + logutil.DDLLogger().Info("abort batch add TiFlash replica", zap.Int64("schemaID", dbInfo.ID)) + return nil + } + } + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: dbInfo.ID, + SchemaName: dbInfo.Name.L, + TableID: tbl.ID, + Type: model.ActionSetTiFlashReplica, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: sctx.GetSessionVars().CDCWriteSource, + InvolvingSchemaInfo: []model.InvolvingSchemaInfo{{ + Database: dbInfo.Name.L, + Table: model.InvolvingAll, + }}, + SQLMode: sctx.GetSessionVars().SQLMode, + } + args := &model.SetTiFlashReplicaArgs{TiflashReplica: *tiflashReplica} + err := e.doDDLJob2(sctx, job, args) + if err != nil { + oneFail = tbl.ID + fail++ + logutil.DDLLogger().Info("processing schema table error", + zap.Int64("tableID", tbl.ID), + zap.Int64("schemaID", dbInfo.ID), + zap.Stringer("tableName", tbl.Name), + zap.Stringer("schemaName", dbInfo.Name), + zap.Error(err)) + } else { + succ++ + } + } + failStmt := "" + if fail > 0 { + failStmt = fmt.Sprintf("(including table %v)", oneFail) + } + msg := fmt.Sprintf("In total %v tables: %v succeed, %v failed%v, %v skipped", total, succ, fail, failStmt, skip) + sctx.GetSessionVars().StmtCtx.SetMessage(msg) + logutil.DDLLogger().Info("finish batch add TiFlash replica", zap.Int64("schemaID", dbInfo.ID)) + return nil +} + +func (e *executor) AlterTablePlacement(ctx sessionctx.Context, ident ast.Ident, placementPolicyRef *model.PolicyRefInfo) (err error) { + is := e.infoCache.GetLatest() + schema, ok := is.SchemaByName(ident.Schema) + if !ok { + return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ident.Schema) + } + + tb, err := is.TableByName(e.ctx, ident.Schema, ident.Name) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ident.Schema, ident.Name)) + } + + if checkIgnorePlacementDDL(ctx) { + return nil + } + + tblInfo := tb.Meta() + if tblInfo.TempTableType != model.TempTableNone { + return errors.Trace(dbterror.ErrOptOnTemporaryTable.GenWithStackByArgs("placement")) + } + + placementPolicyRef, err = checkAndNormalizePlacementPolicy(ctx, placementPolicyRef) + if err != nil { + return err + } + + var involvingSchemaInfo []model.InvolvingSchemaInfo + if placementPolicyRef != nil { + involvingSchemaInfo = []model.InvolvingSchemaInfo{ + { + Database: schema.Name.L, + Table: tblInfo.Name.L, + }, + { + Policy: placementPolicyRef.Name.L, + Mode: model.SharedInvolving, + }, + } + } + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: schema.ID, + TableID: tblInfo.ID, + SchemaName: schema.Name.L, + TableName: tblInfo.Name.L, + Type: model.ActionAlterTablePlacement, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + InvolvingSchemaInfo: involvingSchemaInfo, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.AlterTablePlacementArgs{ + PlacementPolicyRef: placementPolicyRef, + } + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +func checkMultiSchemaSpecs(_ sessionctx.Context, specs []*ast.DatabaseOption) error { + hasSetTiFlashReplica := false + if len(specs) == 1 { + return nil + } + for _, spec := range specs { + if spec.Tp == ast.DatabaseSetTiFlashReplica { + if hasSetTiFlashReplica { + return dbterror.ErrRunMultiSchemaChanges.FastGenByArgs(model.ActionSetTiFlashReplica.String()) + } + hasSetTiFlashReplica = true + } + } + return nil +} + +func (e *executor) AlterSchema(sctx sessionctx.Context, stmt *ast.AlterDatabaseStmt) (err error) { + // Resolve target charset and collation from options. + var ( + toCharset, toCollate string + isAlterCharsetAndCollate bool + placementPolicyRef *model.PolicyRefInfo + tiflashReplica *ast.TiFlashReplicaSpec + ) + + err = checkMultiSchemaSpecs(sctx, stmt.Options) + if err != nil { + return err + } + + for _, val := range stmt.Options { + switch val.Tp { + case ast.DatabaseOptionCharset: + if toCharset == "" { + toCharset = val.Value + } else if toCharset != val.Value { + return dbterror.ErrConflictingDeclarations.GenWithStackByArgs(toCharset, val.Value) + } + isAlterCharsetAndCollate = true + case ast.DatabaseOptionCollate: + info, errGetCollate := collate.GetCollationByName(val.Value) + if errGetCollate != nil { + return errors.Trace(errGetCollate) + } + if toCharset == "" { + toCharset = info.CharsetName + } else if toCharset != info.CharsetName { + return dbterror.ErrConflictingDeclarations.GenWithStackByArgs(toCharset, info.CharsetName) + } + toCollate = info.Name + isAlterCharsetAndCollate = true + case ast.DatabaseOptionPlacementPolicy: + placementPolicyRef = &model.PolicyRefInfo{Name: pmodel.NewCIStr(val.Value)} + case ast.DatabaseSetTiFlashReplica: + tiflashReplica = val.TiFlashReplica + } + } + + if isAlterCharsetAndCollate { + if err = e.ModifySchemaCharsetAndCollate(sctx, stmt, toCharset, toCollate); err != nil { + return err + } + } + if placementPolicyRef != nil { + if err = e.ModifySchemaDefaultPlacement(sctx, stmt, placementPolicyRef); err != nil { + return err + } + } + if tiflashReplica != nil { + if err = e.ModifySchemaSetTiFlashReplica(sctx, stmt, tiflashReplica); err != nil { + return err + } + } + return nil +} + +func (e *executor) DropSchema(ctx sessionctx.Context, stmt *ast.DropDatabaseStmt) (err error) { + is := e.infoCache.GetLatest() + old, ok := is.SchemaByName(stmt.Name) + if !ok { + if stmt.IfExists { + return nil + } + return infoschema.ErrDatabaseDropExists.GenWithStackByArgs(stmt.Name) + } + fkCheck := ctx.GetSessionVars().ForeignKeyChecks + err = checkDatabaseHasForeignKeyReferred(e.ctx, is, old.Name, fkCheck) + if err != nil { + return err + } + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: old.ID, + SchemaName: old.Name.L, + SchemaState: old.State, + Type: model.ActionDropSchema, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + InvolvingSchemaInfo: []model.InvolvingSchemaInfo{{ + Database: old.Name.L, + Table: model.InvolvingAll, + }}, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.DropSchemaArgs{ + FKCheck: fkCheck, + } + + err = e.doDDLJob2(ctx, job, args) + if err != nil { + if infoschema.ErrDatabaseNotExists.Equal(err) { + if stmt.IfExists { + return nil + } + return infoschema.ErrDatabaseDropExists.GenWithStackByArgs(stmt.Name) + } + return errors.Trace(err) + } + if !config.TableLockEnabled() { + return nil + } + + // Clear table locks hold by the session. + tbs, err := is.SchemaSimpleTableInfos(e.ctx, stmt.Name) + if err != nil { + return errors.Trace(err) + } + + lockTableIDs := make([]int64, 0) + for _, tb := range tbs { + if ok, _ := ctx.CheckTableLocked(tb.ID); ok { + lockTableIDs = append(lockTableIDs, tb.ID) + } + } + ctx.ReleaseTableLockByTableIDs(lockTableIDs) + return nil +} + +func (e *executor) RecoverSchema(ctx sessionctx.Context, recoverSchemaInfo *model.RecoverSchemaInfo) error { + involvedSchemas := []model.InvolvingSchemaInfo{{ + Database: recoverSchemaInfo.DBInfo.Name.L, + Table: model.InvolvingAll, + }} + if recoverSchemaInfo.OldSchemaName.L != recoverSchemaInfo.DBInfo.Name.L { + involvedSchemas = append(involvedSchemas, model.InvolvingSchemaInfo{ + Database: recoverSchemaInfo.OldSchemaName.L, + Table: model.InvolvingAll, + }) + } + recoverSchemaInfo.State = model.StateNone + job := &model.Job{ + Version: model.GetJobVerInUse(), + Type: model.ActionRecoverSchema, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + InvolvingSchemaInfo: involvedSchemas, + SQLMode: ctx.GetSessionVars().SQLMode, + } + + args := &model.RecoverArgs{ + RecoverInfo: recoverSchemaInfo, + CheckFlag: recoverCheckFlagNone, + } + err := e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +func checkTooLongSchema(schema pmodel.CIStr) error { + if utf8.RuneCountInString(schema.L) > mysql.MaxDatabaseNameLength { + return dbterror.ErrTooLongIdent.GenWithStackByArgs(schema) + } + return nil +} + +func checkTooLongTable(table pmodel.CIStr) error { + if utf8.RuneCountInString(table.L) > mysql.MaxTableNameLength { + return dbterror.ErrTooLongIdent.GenWithStackByArgs(table) + } + return nil +} + +func checkTooLongIndex(index pmodel.CIStr) error { + if utf8.RuneCountInString(index.L) > mysql.MaxIndexIdentifierLen { + return dbterror.ErrTooLongIdent.GenWithStackByArgs(index) + } + return nil +} + +func checkTooLongColumn(col pmodel.CIStr) error { + if utf8.RuneCountInString(col.L) > mysql.MaxColumnNameLength { + return dbterror.ErrTooLongIdent.GenWithStackByArgs(col) + } + return nil +} + +func checkTooLongForeignKey(fk pmodel.CIStr) error { + if utf8.RuneCountInString(fk.L) > mysql.MaxForeignKeyIdentifierLen { + return dbterror.ErrTooLongIdent.GenWithStackByArgs(fk) + } + return nil +} + +func getDefaultCollationForUTF8MB4(cs string, defaultUTF8MB4Coll string) string { + if cs == charset.CharsetUTF8MB4 { + return defaultUTF8MB4Coll + } + return "" +} + +// GetDefaultCollation returns the default collation for charset and handle the default collation for UTF8MB4. +func GetDefaultCollation(cs string, defaultUTF8MB4Collation string) (string, error) { + coll := getDefaultCollationForUTF8MB4(cs, defaultUTF8MB4Collation) + if coll != "" { + return coll, nil + } + + coll, err := charset.GetDefaultCollation(cs) + if err != nil { + return "", errors.Trace(err) + } + return coll, nil +} + +// ResolveCharsetCollation will resolve the charset and collate by the order of parameters: +// * If any given ast.CharsetOpt is not empty, the resolved charset and collate will be returned. +// * If all ast.CharsetOpts are empty, the default charset and collate will be returned. +func ResolveCharsetCollation(charsetOpts []ast.CharsetOpt, utf8MB4DefaultColl string) (chs string, coll string, err error) { + for _, v := range charsetOpts { + if v.Col != "" { + collation, err := collate.GetCollationByName(v.Col) + if err != nil { + return "", "", errors.Trace(err) + } + if v.Chs != "" && collation.CharsetName != v.Chs { + return "", "", charset.ErrCollationCharsetMismatch.GenWithStackByArgs(v.Col, v.Chs) + } + return collation.CharsetName, v.Col, nil + } + if v.Chs != "" { + coll, err := GetDefaultCollation(v.Chs, utf8MB4DefaultColl) + if err != nil { + return "", "", errors.Trace(err) + } + return v.Chs, coll, nil + } + } + chs, coll = charset.GetDefaultCharsetAndCollate() + utf8mb4Coll := getDefaultCollationForUTF8MB4(chs, utf8MB4DefaultColl) + if utf8mb4Coll != "" { + return chs, utf8mb4Coll, nil + } + return chs, coll, nil +} + +// IsAutoRandomColumnID returns true if the given column ID belongs to an auto_random column. +func IsAutoRandomColumnID(tblInfo *model.TableInfo, colID int64) bool { + if !tblInfo.ContainsAutoRandomBits() { + return false + } + if tblInfo.PKIsHandle { + return tblInfo.GetPkColInfo().ID == colID + } else if tblInfo.IsCommonHandle { + pk := tables.FindPrimaryIndex(tblInfo) + if pk == nil { + return false + } + offset := pk.Columns[0].Offset + return tblInfo.Columns[offset].ID == colID + } + return false +} + +// checkInvisibleIndexOnPK check if primary key is invisible index. +// Note: PKIsHandle == true means the table already has a visible primary key, +// we do not need do a check for this case and return directly, +// because whether primary key is invisible has been check when creating table. +func checkInvisibleIndexOnPK(tblInfo *model.TableInfo) error { + if tblInfo.PKIsHandle { + return nil + } + pk := tblInfo.GetPrimaryKey() + if pk != nil && pk.Invisible { + return dbterror.ErrPKIndexCantBeInvisible + } + return nil +} + +// checkGlobalIndex check if the index is allowed to have global index +func checkGlobalIndex(ec errctx.Context, tblInfo *model.TableInfo, indexInfo *model.IndexInfo) error { + pi := tblInfo.GetPartitionInfo() + isPartitioned := pi != nil && pi.Type != pmodel.PartitionTypeNone + if indexInfo.Global { + if !isPartitioned { + // Makes no sense with LOCAL/GLOBAL index for non-partitioned tables, since we don't support + // partitioning an index differently from the table partitioning. + return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("Global Index on non-partitioned table") + } + // TODO: remove limitation + if !indexInfo.Unique { + return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("GLOBAL IndexOption on non-unique index") + } + // TODO: remove limitation + // check that not all partitioned columns are included. + inAllPartitionColumns, err := checkPartitionKeysConstraint(pi, indexInfo.Columns, tblInfo) + if err != nil { + return errors.Trace(err) + } + if inAllPartitionColumns { + return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("Global Index including all columns in the partitioning expression") + } + validateGlobalIndexWithGeneratedColumns(ec, tblInfo, indexInfo.Name.O, indexInfo.Columns) + } + return nil +} + +// checkGlobalIndexes check if global index is supported. +func checkGlobalIndexes(ec errctx.Context, tblInfo *model.TableInfo) error { + for _, indexInfo := range tblInfo.Indices { + err := checkGlobalIndex(ec, tblInfo, indexInfo) + if err != nil { + return err + } + } + return nil +} + +func (e *executor) CreateTable(ctx sessionctx.Context, s *ast.CreateTableStmt) (err error) { + ident := ast.Ident{Schema: s.Table.Schema, Name: s.Table.Name} + is := e.infoCache.GetLatest() + schema, ok := is.SchemaByName(ident.Schema) + if !ok { + return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ident.Schema) + } + + var ( + referTbl table.Table + involvingRef []model.InvolvingSchemaInfo + ) + if s.ReferTable != nil { + referIdent := ast.Ident{Schema: s.ReferTable.Schema, Name: s.ReferTable.Name} + _, ok := is.SchemaByName(referIdent.Schema) + if !ok { + return infoschema.ErrTableNotExists.GenWithStackByArgs(referIdent.Schema, referIdent.Name) + } + referTbl, err = is.TableByName(e.ctx, referIdent.Schema, referIdent.Name) + if err != nil { + return infoschema.ErrTableNotExists.GenWithStackByArgs(referIdent.Schema, referIdent.Name) + } + involvingRef = append(involvingRef, model.InvolvingSchemaInfo{ + Database: s.ReferTable.Schema.L, + Table: s.ReferTable.Name.L, + Mode: model.SharedInvolving, + }) + } + + // build tableInfo + metaBuildCtx := NewMetaBuildContextWithSctx(ctx) + var tbInfo *model.TableInfo + if s.ReferTable != nil { + tbInfo, err = BuildTableInfoWithLike(ident, referTbl.Meta(), s) + } else { + tbInfo, err = BuildTableInfoWithStmt(metaBuildCtx, s, schema.Charset, schema.Collate, schema.PlacementPolicyRef) + } + if err != nil { + return errors.Trace(err) + } + + if s.Partition != nil { + rewritePartitionQueryString(ctx, s.Partition, tbInfo) + } + + if err = checkTableInfoValidWithStmt(metaBuildCtx, tbInfo, s); err != nil { + return err + } + if err = checkTableForeignKeysValid(ctx, is, schema.Name.L, tbInfo); err != nil { + return err + } + + onExist := OnExistError + if s.IfNotExists { + onExist = OnExistIgnore + } + + return e.CreateTableWithInfo(ctx, schema.Name, tbInfo, involvingRef, WithOnExist(onExist)) +} + +// createTableWithInfoJob returns the table creation job. +// WARNING: it may return a nil job, which means you don't need to submit any DDL job. +func (e *executor) createTableWithInfoJob( + ctx sessionctx.Context, + dbName pmodel.CIStr, + tbInfo *model.TableInfo, + involvingRef []model.InvolvingSchemaInfo, + cfg CreateTableConfig, +) (jobW *JobWrapper, err error) { + is := e.infoCache.GetLatest() + schema, ok := is.SchemaByName(dbName) + if !ok { + return nil, infoschema.ErrDatabaseNotExists.GenWithStackByArgs(dbName) + } + + if err = handleTablePlacement(ctx, tbInfo); err != nil { + return nil, errors.Trace(err) + } + + var oldViewTblID int64 + if oldTable, err := is.TableByName(e.ctx, schema.Name, tbInfo.Name); err == nil { + err = infoschema.ErrTableExists.GenWithStackByArgs(ast.Ident{Schema: schema.Name, Name: tbInfo.Name}) + switch cfg.OnExist { + case OnExistIgnore: + ctx.GetSessionVars().StmtCtx.AppendNote(err) + return nil, nil + case OnExistReplace: + // only CREATE OR REPLACE VIEW is supported at the moment. + if tbInfo.View != nil { + if oldTable.Meta().IsView() { + oldViewTblID = oldTable.Meta().ID + break + } + // The object to replace isn't a view. + return nil, dbterror.ErrWrongObject.GenWithStackByArgs(dbName, tbInfo.Name, "VIEW") + } + return nil, err + default: + return nil, err + } + } + + if err := checkTableInfoValidExtra(ctx.GetSessionVars().StmtCtx.ErrCtx(), ctx.GetStore(), tbInfo); err != nil { + return nil, err + } + + var actionType model.ActionType + switch { + case tbInfo.View != nil: + actionType = model.ActionCreateView + case tbInfo.Sequence != nil: + actionType = model.ActionCreateSequence + default: + actionType = model.ActionCreateTable + } + + var involvingSchemas []model.InvolvingSchemaInfo + sharedInvolvingFromTableInfo := getSharedInvolvingSchemaInfo(tbInfo) + + if sum := len(involvingRef) + len(sharedInvolvingFromTableInfo); sum > 0 { + involvingSchemas = make([]model.InvolvingSchemaInfo, 0, sum+1) + involvingSchemas = append(involvingSchemas, model.InvolvingSchemaInfo{ + Database: schema.Name.L, + Table: tbInfo.Name.L, + }) + involvingSchemas = append(involvingSchemas, involvingRef...) + involvingSchemas = append(involvingSchemas, sharedInvolvingFromTableInfo...) + } + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: schema.ID, + SchemaName: schema.Name.L, + TableName: tbInfo.Name.L, + Type: actionType, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + InvolvingSchemaInfo: involvingSchemas, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.CreateTableArgs{ + TableInfo: tbInfo, + OnExistReplace: cfg.OnExist == OnExistReplace, + OldViewTblID: oldViewTblID, + FKCheck: ctx.GetSessionVars().ForeignKeyChecks, + } + return NewJobWrapperWithArgs(job, args, cfg.IDAllocated), nil +} + +func getSharedInvolvingSchemaInfo(info *model.TableInfo) []model.InvolvingSchemaInfo { + ret := make([]model.InvolvingSchemaInfo, 0, len(info.ForeignKeys)+1) + for _, fk := range info.ForeignKeys { + ret = append(ret, model.InvolvingSchemaInfo{ + Database: fk.RefSchema.L, + Table: fk.RefTable.L, + Mode: model.SharedInvolving, + }) + } + if ref := info.PlacementPolicyRef; ref != nil { + ret = append(ret, model.InvolvingSchemaInfo{ + Policy: ref.Name.L, + Mode: model.SharedInvolving, + }) + } + return ret +} + +func (e *executor) createTableWithInfoPost( + ctx sessionctx.Context, + tbInfo *model.TableInfo, + schemaID int64, +) error { + var err error + var partitions []model.PartitionDefinition + if pi := tbInfo.GetPartitionInfo(); pi != nil { + partitions = pi.Definitions + } + preSplitAndScatter(ctx, e.store, tbInfo, partitions) + if tbInfo.AutoIncID > 1 { + // Default tableAutoIncID base is 0. + // If the first ID is expected to greater than 1, we need to do rebase. + newEnd := tbInfo.AutoIncID - 1 + var allocType autoid.AllocatorType + if tbInfo.SepAutoInc() { + allocType = autoid.AutoIncrementType + } else { + allocType = autoid.RowIDAllocType + } + if err = e.handleAutoIncID(tbInfo, schemaID, newEnd, allocType); err != nil { + return errors.Trace(err) + } + } + // For issue https://github.com/pingcap/tidb/issues/46093 + if tbInfo.AutoIncIDExtra != 0 { + if err = e.handleAutoIncID(tbInfo, schemaID, tbInfo.AutoIncIDExtra-1, autoid.RowIDAllocType); err != nil { + return errors.Trace(err) + } + } + if tbInfo.AutoRandID > 1 { + // Default tableAutoRandID base is 0. + // If the first ID is expected to greater than 1, we need to do rebase. + newEnd := tbInfo.AutoRandID - 1 + err = e.handleAutoIncID(tbInfo, schemaID, newEnd, autoid.AutoRandomType) + } + return err +} + +func (e *executor) CreateTableWithInfo( + ctx sessionctx.Context, + dbName pmodel.CIStr, + tbInfo *model.TableInfo, + involvingRef []model.InvolvingSchemaInfo, + cs ...CreateTableOption, +) (err error) { + c := GetCreateTableConfig(cs) + + jobW, err := e.createTableWithInfoJob(ctx, dbName, tbInfo, involvingRef, c) + if err != nil { + return err + } + if jobW == nil { + return nil + } + + err = e.DoDDLJobWrapper(ctx, jobW) + if err != nil { + // table exists, but if_not_exists flags is true, so we ignore this error. + if c.OnExist == OnExistIgnore && infoschema.ErrTableExists.Equal(err) { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + err = nil + } + } else { + err = e.createTableWithInfoPost(ctx, tbInfo, jobW.SchemaID) + } + + return errors.Trace(err) +} + +func (e *executor) BatchCreateTableWithInfo(ctx sessionctx.Context, + dbName pmodel.CIStr, + infos []*model.TableInfo, + cs ...CreateTableOption, +) error { + failpoint.Inject("RestoreBatchCreateTableEntryTooLarge", func(val failpoint.Value) { + injectBatchSize := val.(int) + if len(infos) > injectBatchSize { + failpoint.Return(kv.ErrEntryTooLarge) + } + }) + c := GetCreateTableConfig(cs) + + job := &model.Job{ + Version: model.GetJobVerInUse(), + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + + var err error + + // check if there are any duplicated table names + duplication := make(map[string]struct{}) + // TODO filter those duplicated info out. + for _, info := range infos { + if _, ok := duplication[info.Name.L]; ok { + err = infoschema.ErrTableExists.FastGenByArgs("can not batch create tables with same name") + if c.OnExist == OnExistIgnore && infoschema.ErrTableExists.Equal(err) { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + err = nil + } + } + if err != nil { + return errors.Trace(err) + } + + duplication[info.Name.L] = struct{}{} + } + + args := &model.BatchCreateTableArgs{ + Tables: make([]*model.CreateTableArgs, 0, len(infos)), + } + for _, info := range infos { + jobItem, err := e.createTableWithInfoJob(ctx, dbName, info, nil, c) + if err != nil { + return errors.Trace(err) + } + if jobItem == nil { + continue + } + + // if jobW.Type == model.ActionCreateTables, it is initialized + // if not, initialize jobW by job.XXXX + if job.Type != model.ActionCreateTables { + job.Type = model.ActionCreateTables + job.SchemaID = jobItem.SchemaID + job.SchemaName = jobItem.SchemaName + } + + // append table job args + args.Tables = append(args.Tables, jobItem.JobArgs.(*model.CreateTableArgs)) + job.InvolvingSchemaInfo = append(job.InvolvingSchemaInfo, model.InvolvingSchemaInfo{ + Database: dbName.L, + Table: info.Name.L, + }) + if sharedInv := getSharedInvolvingSchemaInfo(info); len(sharedInv) > 0 { + job.InvolvingSchemaInfo = append(job.InvolvingSchemaInfo, sharedInv...) + } + } + if len(args.Tables) == 0 { + return nil + } + + jobW := NewJobWrapperWithArgs(job, args, c.IDAllocated) + err = e.DoDDLJobWrapper(ctx, jobW) + if err != nil { + // table exists, but if_not_exists flags is true, so we ignore this error. + if c.OnExist == OnExistIgnore && infoschema.ErrTableExists.Equal(err) { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + err = nil + } + return errors.Trace(err) + } + + for _, tblArgs := range args.Tables { + if err = e.createTableWithInfoPost(ctx, tblArgs.TableInfo, jobW.SchemaID); err != nil { + return errors.Trace(err) + } + } + + return nil +} + +func (e *executor) CreatePlacementPolicyWithInfo(ctx sessionctx.Context, policy *model.PolicyInfo, onExist OnExist) error { + if checkIgnorePlacementDDL(ctx) { + return nil + } + + policyName := policy.Name + if policyName.L == defaultPlacementPolicyName { + return errors.Trace(infoschema.ErrReservedSyntax.GenWithStackByArgs(policyName)) + } + + // Check policy existence. + _, ok := e.infoCache.GetLatest().PolicyByName(policyName) + if ok { + err := infoschema.ErrPlacementPolicyExists.GenWithStackByArgs(policyName) + switch onExist { + case OnExistIgnore: + ctx.GetSessionVars().StmtCtx.AppendNote(err) + return nil + case OnExistError: + return err + } + } + + if err := checkPolicyValidation(policy.PlacementSettings); err != nil { + return err + } + + policyID, err := e.genPlacementPolicyID() + if err != nil { + return err + } + policy.ID = policyID + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaName: policy.Name.L, + Type: model.ActionCreatePlacementPolicy, + BinlogInfo: &model.HistoryInfo{}, + InvolvingSchemaInfo: []model.InvolvingSchemaInfo{{ + Policy: policy.Name.L, + }}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + + args := &model.PlacementPolicyArgs{ + Policy: policy, + ReplaceOnExist: onExist == OnExistReplace, + } + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +// preSplitAndScatter performs pre-split and scatter of the table's regions. +// If `pi` is not nil, will only split region for `pi`, this is used when add partition. +func preSplitAndScatter(ctx sessionctx.Context, store kv.Storage, tbInfo *model.TableInfo, parts []model.PartitionDefinition) { + if tbInfo.TempTableType != model.TempTableNone { + return + } + sp, ok := store.(kv.SplittableStore) + if !ok || atomic.LoadUint32(&EnableSplitTableRegion) == 0 { + return + } + var ( + preSplit func() + scatterScope string + ) + val, ok := ctx.GetSessionVars().GetSystemVar(variable.TiDBScatterRegion) + if !ok { + logutil.DDLLogger().Warn("get system variable met problem, won't scatter region") + } else { + scatterScope = val + } + if len(parts) > 0 { + preSplit = func() { splitPartitionTableRegion(ctx, sp, tbInfo, parts, scatterScope) } + } else { + preSplit = func() { splitTableRegion(ctx, sp, tbInfo, scatterScope) } + } + if scatterScope != variable.ScatterOff { + preSplit() + } else { + go preSplit() + } +} + +func (e *executor) FlashbackCluster(ctx sessionctx.Context, flashbackTS uint64) error { + logutil.DDLLogger().Info("get flashback cluster job", zap.Stringer("flashbackTS", oracle.GetTimeFromTS(flashbackTS))) + nowTS, err := ctx.GetStore().GetOracle().GetTimestamp(e.ctx, &oracle.Option{}) + if err != nil { + return errors.Trace(err) + } + gap := time.Until(oracle.GetTimeFromTS(nowTS)).Abs() + if gap > 1*time.Second { + ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("Gap between local time and PD TSO is %s, please check PD/system time", gap)) + } + job := &model.Job{ + Version: model.GetJobVerInUse(), + Type: model.ActionFlashbackCluster, + BinlogInfo: &model.HistoryInfo{}, + // The value for global variables is meaningless, it will cover during flashback cluster. + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + // FLASHBACK CLUSTER affects all schemas and tables. + InvolvingSchemaInfo: []model.InvolvingSchemaInfo{{ + Database: model.InvolvingAll, + Table: model.InvolvingAll, + }}, + SQLMode: ctx.GetSessionVars().SQLMode, + } + + args := &model.FlashbackClusterArgs{ + FlashbackTS: flashbackTS, + PDScheduleValue: map[string]any{}, + EnableGC: true, + EnableAutoAnalyze: true, + EnableTTLJob: true, + } + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +func (e *executor) RecoverTable(ctx sessionctx.Context, recoverTableInfo *model.RecoverTableInfo) (err error) { + is := e.infoCache.GetLatest() + schemaID, tbInfo := recoverTableInfo.SchemaID, recoverTableInfo.TableInfo + // Check schema exist. + schema, ok := is.SchemaByID(schemaID) + if !ok { + return errors.Trace(infoschema.ErrDatabaseNotExists.GenWithStackByArgs( + fmt.Sprintf("(Schema ID %d)", schemaID), + )) + } + // Check not exist table with same name. + if ok := is.TableExists(schema.Name, tbInfo.Name); ok { + return infoschema.ErrTableExists.GenWithStackByArgs(tbInfo.Name) + } + + // for "flashback table xxx to yyy" + // Note: this case only allow change table name, schema remains the same. + var involvedSchemas []model.InvolvingSchemaInfo + if recoverTableInfo.OldTableName != tbInfo.Name.L { + involvedSchemas = []model.InvolvingSchemaInfo{ + {Database: schema.Name.L, Table: recoverTableInfo.OldTableName}, + {Database: schema.Name.L, Table: tbInfo.Name.L}, + } + } + + tbInfo.State = model.StateNone + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: schemaID, + TableID: tbInfo.ID, + SchemaName: schema.Name.L, + TableName: tbInfo.Name.L, + Type: model.ActionRecoverTable, + BinlogInfo: &model.HistoryInfo{}, + InvolvingSchemaInfo: involvedSchemas, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + + args := &model.RecoverArgs{ + RecoverInfo: &model.RecoverSchemaInfo{ + RecoverTableInfos: []*model.RecoverTableInfo{recoverTableInfo}, + }, + CheckFlag: recoverCheckFlagNone} + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +func (e *executor) CreateView(ctx sessionctx.Context, s *ast.CreateViewStmt) (err error) { + viewInfo, err := BuildViewInfo(s) + if err != nil { + return err + } + + cols := make([]*table.Column, len(s.Cols)) + for i, v := range s.Cols { + cols[i] = table.ToColumn(&model.ColumnInfo{ + Name: v, + ID: int64(i), + Offset: i, + State: model.StatePublic, + }) + } + + tblCharset := "" + tblCollate := "" + if v, ok := ctx.GetSessionVars().GetSystemVar(variable.CharacterSetConnection); ok { + tblCharset = v + } + if v, ok := ctx.GetSessionVars().GetSystemVar(variable.CollationConnection); ok { + tblCollate = v + } + + tbInfo, err := BuildTableInfo(NewMetaBuildContextWithSctx(ctx), s.ViewName.Name, cols, nil, tblCharset, tblCollate) + if err != nil { + return err + } + tbInfo.View = viewInfo + + onExist := OnExistError + if s.OrReplace { + onExist = OnExistReplace + } + + return e.CreateTableWithInfo(ctx, s.ViewName.Schema, tbInfo, nil, WithOnExist(onExist)) +} + +func checkCharsetAndCollation(cs string, co string) error { + if !charset.ValidCharsetAndCollation(cs, co) { + return dbterror.ErrUnknownCharacterSet.GenWithStackByArgs(cs) + } + if co != "" { + if _, err := collate.GetCollationByName(co); err != nil { + return errors.Trace(err) + } + } + return nil +} + +// handleAutoIncID handles auto_increment option in DDL. It creates a ID counter for the table and initiates the counter to a proper value. +// For example if the option sets auto_increment to 10. The counter will be set to 9. So the next allocated ID will be 10. +func (e *executor) handleAutoIncID(tbInfo *model.TableInfo, schemaID int64, newEnd int64, tp autoid.AllocatorType) error { + allocs := autoid.NewAllocatorsFromTblInfo(e.getAutoIDRequirement(), schemaID, tbInfo) + if alloc := allocs.Get(tp); alloc != nil { + err := alloc.Rebase(context.Background(), newEnd, false) + if err != nil { + return errors.Trace(err) + } + } + return nil +} + +func (e *executor) getAutoIDRequirement() autoid.Requirement { + return &asAutoIDRequirement{ + store: e.store, + autoidCli: e.autoidCli, + } +} + +func shardingBits(tblInfo *model.TableInfo) uint64 { + if tblInfo.ShardRowIDBits > 0 { + return tblInfo.ShardRowIDBits + } + return tblInfo.AutoRandomBits +} + +// isIgnorableSpec checks if the spec type is ignorable. +// Some specs are parsed by ignored. This is for compatibility. +func isIgnorableSpec(tp ast.AlterTableType) bool { + // AlterTableLock/AlterTableAlgorithm are ignored. + return tp == ast.AlterTableLock || tp == ast.AlterTableAlgorithm +} + +// GetCharsetAndCollateInTableOption will iterate the charset and collate in the options, +// and returns the last charset and collate in options. If there is no charset in the options, +// the returns charset will be "", the same as collate. +func GetCharsetAndCollateInTableOption(startIdx int, options []*ast.TableOption, defaultUTF8MB4Coll string) (chs, coll string, err error) { + for i := startIdx; i < len(options); i++ { + opt := options[i] + // we set the charset to the last option. example: alter table t charset latin1 charset utf8 collate utf8_bin; + // the charset will be utf8, collate will be utf8_bin + switch opt.Tp { + case ast.TableOptionCharset: + info, err := charset.GetCharsetInfo(opt.StrValue) + if err != nil { + return "", "", err + } + if len(chs) == 0 { + chs = info.Name + } else if chs != info.Name { + return "", "", dbterror.ErrConflictingDeclarations.GenWithStackByArgs(chs, info.Name) + } + if len(coll) == 0 { + defaultColl := getDefaultCollationForUTF8MB4(chs, defaultUTF8MB4Coll) + if len(defaultColl) == 0 { + coll = info.DefaultCollation + } else { + coll = defaultColl + } + } + case ast.TableOptionCollate: + info, err := collate.GetCollationByName(opt.StrValue) + if err != nil { + return "", "", err + } + if len(chs) == 0 { + chs = info.CharsetName + } else if chs != info.CharsetName { + return "", "", dbterror.ErrCollationCharsetMismatch.GenWithStackByArgs(info.Name, chs) + } + coll = info.Name + } + } + return +} + +// NeedToOverwriteColCharset return true for altering charset and specified CONVERT TO. +func NeedToOverwriteColCharset(options []*ast.TableOption) bool { + for i := len(options) - 1; i >= 0; i-- { + opt := options[i] + if opt.Tp == ast.TableOptionCharset { + // Only overwrite columns charset if the option contains `CONVERT TO`. + return opt.UintValue == ast.TableOptionCharsetWithConvertTo + } + } + return false +} + +// resolveAlterTableAddColumns splits "add columns" to multiple spec. For example, +// `ALTER TABLE ADD COLUMN (c1 INT, c2 INT)` is split into +// `ALTER TABLE ADD COLUMN c1 INT, ADD COLUMN c2 INT`. +func resolveAlterTableAddColumns(spec *ast.AlterTableSpec) []*ast.AlterTableSpec { + specs := make([]*ast.AlterTableSpec, 0, len(spec.NewColumns)+len(spec.NewConstraints)) + for _, col := range spec.NewColumns { + t := *spec + t.NewColumns = []*ast.ColumnDef{col} + t.NewConstraints = []*ast.Constraint{} + specs = append(specs, &t) + } + // Split the add constraints from AlterTableSpec. + for _, con := range spec.NewConstraints { + t := *spec + t.NewColumns = []*ast.ColumnDef{} + t.NewConstraints = []*ast.Constraint{} + t.Constraint = con + t.Tp = ast.AlterTableAddConstraint + specs = append(specs, &t) + } + return specs +} + +// ResolveAlterTableSpec resolves alter table algorithm and removes ignore table spec in specs. +// returns valid specs, and the occurred error. +func ResolveAlterTableSpec(ctx sessionctx.Context, specs []*ast.AlterTableSpec) ([]*ast.AlterTableSpec, error) { + validSpecs := make([]*ast.AlterTableSpec, 0, len(specs)) + algorithm := ast.AlgorithmTypeDefault + for _, spec := range specs { + if spec.Tp == ast.AlterTableAlgorithm { + // Find the last AlterTableAlgorithm. + algorithm = spec.Algorithm + } + if isIgnorableSpec(spec.Tp) { + continue + } + if spec.Tp == ast.AlterTableAddColumns && (len(spec.NewColumns) > 1 || len(spec.NewConstraints) > 0) { + validSpecs = append(validSpecs, resolveAlterTableAddColumns(spec)...) + } else { + validSpecs = append(validSpecs, spec) + } + // TODO: Only allow REMOVE PARTITIONING as a single ALTER TABLE statement? + } + + // Verify whether the algorithm is supported. + for _, spec := range validSpecs { + resolvedAlgorithm, err := ResolveAlterAlgorithm(spec, algorithm) + if err != nil { + // If TiDB failed to choose a better algorithm, report the error + if resolvedAlgorithm == ast.AlgorithmTypeDefault { + return nil, errors.Trace(err) + } + // For the compatibility, we return warning instead of error when a better algorithm is chosed by TiDB + ctx.GetSessionVars().StmtCtx.AppendError(err) + } + + spec.Algorithm = resolvedAlgorithm + } + + // Only handle valid specs. + return validSpecs, nil +} + +func isMultiSchemaChanges(specs []*ast.AlterTableSpec) bool { + if len(specs) > 1 { + return true + } + if len(specs) == 1 && len(specs[0].NewColumns) > 1 && specs[0].Tp == ast.AlterTableAddColumns { + return true + } + return false +} + +func (e *executor) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt *ast.AlterTableStmt) (err error) { + ident := ast.Ident{Schema: stmt.Table.Schema, Name: stmt.Table.Name} + validSpecs, err := ResolveAlterTableSpec(sctx, stmt.Specs) + if err != nil { + return errors.Trace(err) + } + + is := e.infoCache.GetLatest() + tb, err := is.TableByName(ctx, ident.Schema, ident.Name) + if err != nil { + return errors.Trace(err) + } + if tb.Meta().IsView() || tb.Meta().IsSequence() { + return dbterror.ErrWrongObject.GenWithStackByArgs(ident.Schema, ident.Name, "BASE TABLE") + } + if tb.Meta().TableCacheStatusType != model.TableCacheStatusDisable { + if len(validSpecs) != 1 { + return dbterror.ErrOptOnCacheTable.GenWithStackByArgs("Alter Table") + } + if validSpecs[0].Tp != ast.AlterTableCache && validSpecs[0].Tp != ast.AlterTableNoCache { + return dbterror.ErrOptOnCacheTable.GenWithStackByArgs("Alter Table") + } + } + if isMultiSchemaChanges(validSpecs) && (sctx.GetSessionVars().EnableRowLevelChecksum || variable.EnableRowLevelChecksum.Load()) { + return dbterror.ErrRunMultiSchemaChanges.GenWithStack("Unsupported multi schema change when row level checksum is enabled") + } + // set name for anonymous foreign key. + maxForeignKeyID := tb.Meta().MaxForeignKeyID + for _, spec := range validSpecs { + if spec.Tp == ast.AlterTableAddConstraint && spec.Constraint.Tp == ast.ConstraintForeignKey && spec.Constraint.Name == "" { + maxForeignKeyID++ + spec.Constraint.Name = fmt.Sprintf("fk_%d", maxForeignKeyID) + } + } + + if len(validSpecs) > 1 { + // after MultiSchemaInfo is set, DoDDLJob will collect all jobs into + // MultiSchemaInfo and skip running them. Then we will run them in + // d.multiSchemaChange all at once. + sctx.GetSessionVars().StmtCtx.MultiSchemaInfo = model.NewMultiSchemaInfo() + } + for _, spec := range validSpecs { + var handledCharsetOrCollate bool + var ttlOptionsHandled bool + switch spec.Tp { + case ast.AlterTableAddColumns: + err = e.AddColumn(sctx, ident, spec) + case ast.AlterTableAddPartitions, ast.AlterTableAddLastPartition: + err = e.AddTablePartitions(sctx, ident, spec) + case ast.AlterTableCoalescePartitions: + err = e.CoalescePartitions(sctx, ident, spec) + case ast.AlterTableReorganizePartition: + err = e.ReorganizePartitions(sctx, ident, spec) + case ast.AlterTableReorganizeFirstPartition: + err = dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("MERGE FIRST PARTITION") + case ast.AlterTableReorganizeLastPartition: + err = dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("SPLIT LAST PARTITION") + case ast.AlterTableCheckPartitions: + err = errors.Trace(dbterror.ErrUnsupportedCheckPartition) + case ast.AlterTableRebuildPartition: + err = errors.Trace(dbterror.ErrUnsupportedRebuildPartition) + case ast.AlterTableOptimizePartition: + err = errors.Trace(dbterror.ErrUnsupportedOptimizePartition) + case ast.AlterTableRemovePartitioning: + err = e.RemovePartitioning(sctx, ident, spec) + case ast.AlterTableRepairPartition: + err = errors.Trace(dbterror.ErrUnsupportedRepairPartition) + case ast.AlterTableDropColumn: + err = e.DropColumn(sctx, ident, spec) + case ast.AlterTableDropIndex: + err = e.dropIndex(sctx, ident, pmodel.NewCIStr(spec.Name), spec.IfExists, false) + case ast.AlterTableDropPrimaryKey: + err = e.dropIndex(sctx, ident, pmodel.NewCIStr(mysql.PrimaryKeyName), spec.IfExists, false) + case ast.AlterTableRenameIndex: + err = e.RenameIndex(sctx, ident, spec) + case ast.AlterTableDropPartition, ast.AlterTableDropFirstPartition: + err = e.DropTablePartition(sctx, ident, spec) + case ast.AlterTableTruncatePartition: + err = e.TruncateTablePartition(sctx, ident, spec) + case ast.AlterTableWriteable: + if !config.TableLockEnabled() { + return nil + } + tName := &ast.TableName{Schema: ident.Schema, Name: ident.Name} + if spec.Writeable { + err = e.CleanupTableLock(sctx, []*ast.TableName{tName}) + } else { + lockStmt := &ast.LockTablesStmt{ + TableLocks: []ast.TableLock{ + { + Table: tName, + Type: pmodel.TableLockReadOnly, + }, + }, + } + err = e.LockTables(sctx, lockStmt) + } + case ast.AlterTableExchangePartition: + err = e.ExchangeTablePartition(sctx, ident, spec) + case ast.AlterTableAddConstraint: + constr := spec.Constraint + switch spec.Constraint.Tp { + case ast.ConstraintKey, ast.ConstraintIndex: + err = e.createIndex(sctx, ident, ast.IndexKeyTypeNone, pmodel.NewCIStr(constr.Name), + spec.Constraint.Keys, constr.Option, constr.IfNotExists) + case ast.ConstraintUniq, ast.ConstraintUniqIndex, ast.ConstraintUniqKey: + err = e.createIndex(sctx, ident, ast.IndexKeyTypeUnique, pmodel.NewCIStr(constr.Name), + spec.Constraint.Keys, constr.Option, false) // IfNotExists should be not applied + case ast.ConstraintForeignKey: + // NOTE: we do not handle `symbol` and `index_name` well in the parser and we do not check ForeignKey already exists, + // so we just also ignore the `if not exists` check. + err = e.CreateForeignKey(sctx, ident, pmodel.NewCIStr(constr.Name), spec.Constraint.Keys, spec.Constraint.Refer) + case ast.ConstraintPrimaryKey: + err = e.CreatePrimaryKey(sctx, ident, pmodel.NewCIStr(constr.Name), spec.Constraint.Keys, constr.Option) + case ast.ConstraintFulltext: + sctx.GetSessionVars().StmtCtx.AppendWarning(dbterror.ErrTableCantHandleFt) + case ast.ConstraintCheck: + if !variable.EnableCheckConstraint.Load() { + sctx.GetSessionVars().StmtCtx.AppendWarning(errCheckConstraintIsOff) + } else { + err = e.CreateCheckConstraint(sctx, ident, pmodel.NewCIStr(constr.Name), spec.Constraint) + } + case ast.ConstraintVector: + err = e.createVectorIndex(sctx, ident, pmodel.NewCIStr(constr.Name), spec.Constraint.Keys, constr.Option, constr.IfNotExists) + default: + // Nothing to do now. + } + case ast.AlterTableDropForeignKey: + // NOTE: we do not check `if not exists` and `if exists` for ForeignKey now. + err = e.DropForeignKey(sctx, ident, pmodel.NewCIStr(spec.Name)) + case ast.AlterTableModifyColumn: + err = e.ModifyColumn(ctx, sctx, ident, spec) + case ast.AlterTableChangeColumn: + err = e.ChangeColumn(ctx, sctx, ident, spec) + case ast.AlterTableRenameColumn: + err = e.RenameColumn(sctx, ident, spec) + case ast.AlterTableAlterColumn: + err = e.AlterColumn(sctx, ident, spec) + case ast.AlterTableRenameTable: + newIdent := ast.Ident{Schema: spec.NewTable.Schema, Name: spec.NewTable.Name} + isAlterTable := true + err = e.renameTable(sctx, ident, newIdent, isAlterTable) + case ast.AlterTablePartition: + err = e.AlterTablePartitioning(sctx, ident, spec) + case ast.AlterTableOption: + var placementPolicyRef *model.PolicyRefInfo + for i, opt := range spec.Options { + switch opt.Tp { + case ast.TableOptionShardRowID: + if opt.UintValue > variable.MaxShardRowIDBits { + opt.UintValue = variable.MaxShardRowIDBits + } + err = e.ShardRowID(sctx, ident, opt.UintValue) + case ast.TableOptionAutoIncrement: + err = e.RebaseAutoID(sctx, ident, int64(opt.UintValue), autoid.AutoIncrementType, opt.BoolValue) + case ast.TableOptionAutoIdCache: + if opt.UintValue > uint64(math.MaxInt64) { + // TODO: Refine this error. + return errors.New("table option auto_id_cache overflows int64") + } + err = e.AlterTableAutoIDCache(sctx, ident, int64(opt.UintValue)) + case ast.TableOptionAutoRandomBase: + err = e.RebaseAutoID(sctx, ident, int64(opt.UintValue), autoid.AutoRandomType, opt.BoolValue) + case ast.TableOptionComment: + spec.Comment = opt.StrValue + err = e.AlterTableComment(sctx, ident, spec) + case ast.TableOptionCharset, ast.TableOptionCollate: + // GetCharsetAndCollateInTableOption will get the last charset and collate in the options, + // so it should be handled only once. + if handledCharsetOrCollate { + continue + } + var toCharset, toCollate string + toCharset, toCollate, err = GetCharsetAndCollateInTableOption(i, spec.Options, sctx.GetSessionVars().DefaultCollationForUTF8MB4) + if err != nil { + return err + } + needsOverwriteCols := NeedToOverwriteColCharset(spec.Options) + err = e.AlterTableCharsetAndCollate(sctx, ident, toCharset, toCollate, needsOverwriteCols) + handledCharsetOrCollate = true + case ast.TableOptionPlacementPolicy: + placementPolicyRef = &model.PolicyRefInfo{ + Name: pmodel.NewCIStr(opt.StrValue), + } + case ast.TableOptionEngine: + case ast.TableOptionRowFormat: + case ast.TableOptionTTL, ast.TableOptionTTLEnable, ast.TableOptionTTLJobInterval: + var ttlInfo *model.TTLInfo + var ttlEnable *bool + var ttlJobInterval *string + + if ttlOptionsHandled { + continue + } + ttlInfo, ttlEnable, ttlJobInterval, err = getTTLInfoInOptions(spec.Options) + if err != nil { + return err + } + err = e.AlterTableTTLInfoOrEnable(sctx, ident, ttlInfo, ttlEnable, ttlJobInterval) + + ttlOptionsHandled = true + default: + err = dbterror.ErrUnsupportedAlterTableOption + } + + if err != nil { + return errors.Trace(err) + } + } + + if placementPolicyRef != nil { + err = e.AlterTablePlacement(sctx, ident, placementPolicyRef) + } + case ast.AlterTableSetTiFlashReplica: + err = e.AlterTableSetTiFlashReplica(sctx, ident, spec.TiFlashReplica) + case ast.AlterTableOrderByColumns: + err = e.OrderByColumns(sctx, ident) + case ast.AlterTableIndexInvisible: + err = e.AlterIndexVisibility(sctx, ident, spec.IndexName, spec.Visibility) + case ast.AlterTableAlterCheck: + if !variable.EnableCheckConstraint.Load() { + sctx.GetSessionVars().StmtCtx.AppendWarning(errCheckConstraintIsOff) + } else { + err = e.AlterCheckConstraint(sctx, ident, pmodel.NewCIStr(spec.Constraint.Name), spec.Constraint.Enforced) + } + case ast.AlterTableDropCheck: + if !variable.EnableCheckConstraint.Load() { + sctx.GetSessionVars().StmtCtx.AppendWarning(errCheckConstraintIsOff) + } else { + err = e.DropCheckConstraint(sctx, ident, pmodel.NewCIStr(spec.Constraint.Name)) + } + case ast.AlterTableWithValidation: + sctx.GetSessionVars().StmtCtx.AppendWarning(dbterror.ErrUnsupportedAlterTableWithValidation) + case ast.AlterTableWithoutValidation: + sctx.GetSessionVars().StmtCtx.AppendWarning(dbterror.ErrUnsupportedAlterTableWithoutValidation) + case ast.AlterTableAddStatistics: + err = e.AlterTableAddStatistics(sctx, ident, spec.Statistics, spec.IfNotExists) + case ast.AlterTableDropStatistics: + err = e.AlterTableDropStatistics(sctx, ident, spec.Statistics, spec.IfExists) + case ast.AlterTableAttributes: + err = e.AlterTableAttributes(sctx, ident, spec) + case ast.AlterTablePartitionAttributes: + err = e.AlterTablePartitionAttributes(sctx, ident, spec) + case ast.AlterTablePartitionOptions: + err = e.AlterTablePartitionOptions(sctx, ident, spec) + case ast.AlterTableCache: + err = e.AlterTableCache(sctx, ident) + case ast.AlterTableNoCache: + err = e.AlterTableNoCache(sctx, ident) + case ast.AlterTableDisableKeys, ast.AlterTableEnableKeys: + // Nothing to do now, see https://github.com/pingcap/tidb/issues/1051 + // MyISAM specific + case ast.AlterTableRemoveTTL: + // the parser makes sure we have only one `ast.AlterTableRemoveTTL` in an alter statement + err = e.AlterTableRemoveTTL(sctx, ident) + default: + err = errors.Trace(dbterror.ErrUnsupportedAlterTableSpec) + } + + if err != nil { + return errors.Trace(err) + } + } + + if sctx.GetSessionVars().StmtCtx.MultiSchemaInfo != nil { + info := sctx.GetSessionVars().StmtCtx.MultiSchemaInfo + sctx.GetSessionVars().StmtCtx.MultiSchemaInfo = nil + err = e.multiSchemaChange(sctx, ident, info) + if err != nil { + return errors.Trace(err) + } + } + + return nil +} + +func (e *executor) multiSchemaChange(ctx sessionctx.Context, ti ast.Ident, info *model.MultiSchemaInfo) error { + subJobs := info.SubJobs + if len(subJobs) == 0 { + return nil + } + schema, t, err := e.getSchemaAndTableByIdent(ti) + if err != nil { + return errors.Trace(err) + } + + logFn := logutil.DDLLogger().Warn + if intest.InTest { + logFn = logutil.DDLLogger().Fatal + } + + // to do:(joccau) + // we need refactor this part to support V2 job version after refactor all of ddl types. + var involvingSchemaInfo []model.InvolvingSchemaInfo + for _, j := range subJobs { + if j.Type == model.ActionAddForeignKey { + ref, ok := j.Args[0].(*model.FKInfo) + if !ok { + logFn("unexpected type of foreign key info", + zap.Any("args[0]", j.Args[0]), + zap.String("type", fmt.Sprintf("%T", j.Args[0]))) + continue + } + involvingSchemaInfo = append(involvingSchemaInfo, model.InvolvingSchemaInfo{ + Database: ref.RefSchema.L, + Table: ref.RefTable.L, + Mode: model.SharedInvolving, + }) + } + } + + if len(involvingSchemaInfo) > 0 { + involvingSchemaInfo = append(involvingSchemaInfo, model.InvolvingSchemaInfo{ + Database: schema.Name.L, + Table: t.Meta().Name.L, + }) + } + + job := &model.Job{ + SchemaID: schema.ID, + TableID: t.Meta().ID, + SchemaName: schema.Name.L, + TableName: t.Meta().Name.L, + Type: model.ActionMultiSchemaChange, + BinlogInfo: &model.HistoryInfo{}, + Args: nil, + MultiSchemaInfo: info, + ReorgMeta: nil, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + InvolvingSchemaInfo: involvingSchemaInfo, + SQLMode: ctx.GetSessionVars().SQLMode, + } + if containsDistTaskSubJob(subJobs) { + job.ReorgMeta, err = newReorgMetaFromVariables(job, ctx) + if err != nil { + return err + } + } else { + job.ReorgMeta = NewDDLReorgMeta(ctx) + } + + err = checkMultiSchemaInfo(info, t) + if err != nil { + return errors.Trace(err) + } + mergeAddIndex(info) + return e.DoDDLJob(ctx, job) +} + +func containsDistTaskSubJob(subJobs []*model.SubJob) bool { + for _, sub := range subJobs { + if sub.Type == model.ActionAddIndex || + sub.Type == model.ActionAddPrimaryKey { + return true + } + } + return false +} + +func (e *executor) RebaseAutoID(ctx sessionctx.Context, ident ast.Ident, newBase int64, tp autoid.AllocatorType, force bool) error { + schema, t, err := e.getSchemaAndTableByIdent(ident) + if err != nil { + return errors.Trace(err) + } + tbInfo := t.Meta() + var actionType model.ActionType + switch tp { + case autoid.AutoRandomType: + pkCol := tbInfo.GetPkColInfo() + if tbInfo.AutoRandomBits == 0 || pkCol == nil { + return errors.Trace(dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomRebaseNotApplicable)) + } + shardFmt := autoid.NewShardIDFormat(&pkCol.FieldType, tbInfo.AutoRandomBits, tbInfo.AutoRandomRangeBits) + if shardFmt.IncrementalMask()&newBase != newBase { + errMsg := fmt.Sprintf(autoid.AutoRandomRebaseOverflow, newBase, shardFmt.IncrementalBitsCapacity()) + return errors.Trace(dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(errMsg)) + } + actionType = model.ActionRebaseAutoRandomBase + case autoid.RowIDAllocType: + actionType = model.ActionRebaseAutoID + case autoid.AutoIncrementType: + actionType = model.ActionRebaseAutoID + default: + panic(fmt.Sprintf("unimplemented rebase autoid type %s", tp)) + } + + if !force { + newBaseTemp, err := adjustNewBaseToNextGlobalID(ctx.GetTableCtx(), t, tp, newBase) + if err != nil { + return err + } + if newBase != newBaseTemp { + ctx.GetSessionVars().StmtCtx.AppendWarning( + errors.NewNoStackErrorf("Can't reset AUTO_INCREMENT to %d without FORCE option, using %d instead", + newBase, newBaseTemp, + )) + } + newBase = newBaseTemp + } + job := &model.Job{ + Version: model.JobVersion1, + SchemaID: schema.ID, + TableID: tbInfo.ID, + SchemaName: schema.Name.L, + TableName: tbInfo.Name.L, + Type: actionType, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.RebaseAutoIDArgs{ + NewBase: newBase, + Force: force, + } + // need fill args, the job will be pushed subjob. + job.FillArgs(args) + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +func adjustNewBaseToNextGlobalID(ctx table.AllocatorContext, t table.Table, tp autoid.AllocatorType, newBase int64) (int64, error) { + alloc := t.Allocators(ctx).Get(tp) + if alloc == nil { + return newBase, nil + } + autoID, err := alloc.NextGlobalAutoID() + if err != nil { + return newBase, errors.Trace(err) + } + // If newBase < autoID, we need to do a rebase before returning. + // Assume there are 2 TiDB servers: TiDB-A with allocator range of 0 ~ 30000; TiDB-B with allocator range of 30001 ~ 60000. + // If the user sends SQL `alter table t1 auto_increment = 100` to TiDB-B, + // and TiDB-B finds 100 < 30001 but returns without any handling, + // then TiDB-A may still allocate 99 for auto_increment column. This doesn't make sense for the user. + return int64(mathutil.Max(uint64(newBase), uint64(autoID))), nil +} + +// ShardRowID shards the implicit row ID by adding shard value to the row ID's first few bits. +func (e *executor) ShardRowID(ctx sessionctx.Context, tableIdent ast.Ident, uVal uint64) error { + schema, t, err := e.getSchemaAndTableByIdent(tableIdent) + if err != nil { + return errors.Trace(err) + } + tbInfo := t.Meta() + if tbInfo.TempTableType != model.TempTableNone { + return dbterror.ErrOptOnTemporaryTable.GenWithStackByArgs("shard_row_id_bits") + } + if uVal == tbInfo.ShardRowIDBits { + // Nothing need to do. + return nil + } + if uVal > 0 && tbInfo.HasClusteredIndex() { + return dbterror.ErrUnsupportedShardRowIDBits + } + err = verifyNoOverflowShardBits(e.sessPool, t, uVal) + if err != nil { + return err + } + job := &model.Job{ + Version: model.GetJobVerInUse(), + Type: model.ActionShardRowID, + SchemaID: schema.ID, + TableID: tbInfo.ID, + SchemaName: schema.Name.L, + TableName: tbInfo.Name.L, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.ShardRowIDArgs{ShardRowIDBits: uVal} + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +func (e *executor) getSchemaAndTableByIdent(tableIdent ast.Ident) (dbInfo *model.DBInfo, t table.Table, err error) { + is := e.infoCache.GetLatest() + schema, ok := is.SchemaByName(tableIdent.Schema) + if !ok { + return nil, nil, infoschema.ErrDatabaseNotExists.GenWithStackByArgs(tableIdent.Schema) + } + t, err = is.TableByName(e.ctx, tableIdent.Schema, tableIdent.Name) + if err != nil { + return nil, nil, infoschema.ErrTableNotExists.GenWithStackByArgs(tableIdent.Schema, tableIdent.Name) + } + return schema, t, nil +} + +// AddColumn will add a new column to the table. +func (e *executor) AddColumn(ctx sessionctx.Context, ti ast.Ident, spec *ast.AlterTableSpec) error { + specNewColumn := spec.NewColumns[0] + schema, t, err := e.getSchemaAndTableByIdent(ti) + if err != nil { + return errors.Trace(err) + } + failpoint.InjectCall("afterGetSchemaAndTableByIdent", ctx) + tbInfo := t.Meta() + if err = checkAddColumnTooManyColumns(len(t.Cols()) + 1); err != nil { + return errors.Trace(err) + } + col, err := checkAndCreateNewColumn(ctx, ti, schema, spec, t, specNewColumn) + if err != nil { + return errors.Trace(err) + } + // Added column has existed and if_not_exists flag is true. + if col == nil { + return nil + } + err = CheckAfterPositionExists(tbInfo, spec.Position) + if err != nil { + return errors.Trace(err) + } + + txn, err := ctx.Txn(true) + if err != nil { + return errors.Trace(err) + } + bdrRole, err := meta.NewMutator(txn).GetBDRRole() + if err != nil { + return errors.Trace(err) + } + if bdrRole == string(ast.BDRRolePrimary) && deniedByBDRWhenAddColumn(specNewColumn.Options) { + return dbterror.ErrBDRRestrictedDDL.FastGenByArgs(bdrRole) + } + + job := &model.Job{ + Version: model.JobVersion1, + SchemaID: schema.ID, + TableID: tbInfo.ID, + SchemaName: schema.Name.L, + TableName: tbInfo.Name.L, + Type: model.ActionAddColumn, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.TableColumnArgs{ + Col: col.ColumnInfo, + Pos: spec.Position, + IgnoreExistenceErr: spec.IfNotExists, + } + job.FillArgs(args) + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +// AddTablePartitions will add a new partition to the table. +func (e *executor) AddTablePartitions(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { + is := e.infoCache.GetLatest() + schema, ok := is.SchemaByName(ident.Schema) + if !ok { + return errors.Trace(infoschema.ErrDatabaseNotExists.GenWithStackByArgs(schema)) + } + t, err := is.TableByName(e.ctx, ident.Schema, ident.Name) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ident.Schema, ident.Name)) + } + + meta := t.Meta() + pi := meta.GetPartitionInfo() + if pi == nil { + return errors.Trace(dbterror.ErrPartitionMgmtOnNonpartitioned) + } + if pi.Type == pmodel.PartitionTypeHash || pi.Type == pmodel.PartitionTypeKey { + // Add partition for hash/key is actually a reorganize partition + // operation and not a metadata only change! + switch spec.Tp { + case ast.AlterTableAddLastPartition: + return errors.Trace(dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("LAST PARTITION of HASH/KEY partitioned table")) + case ast.AlterTableAddPartitions: + // only thing supported + default: + return errors.Trace(dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("ADD PARTITION of HASH/KEY partitioned table")) + } + return e.hashPartitionManagement(ctx, ident, spec, pi) + } + + partInfo, err := BuildAddedPartitionInfo(ctx.GetExprCtx(), meta, spec) + if err != nil { + return errors.Trace(err) + } + if pi.Type == pmodel.PartitionTypeList { + // TODO: make sure that checks in ddl_api and ddl_worker is the same. + if meta.Partition.GetDefaultListPartition() != -1 { + return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("ADD List partition, already contains DEFAULT partition. Please use REORGANIZE PARTITION instead") + } + } + + // partInfo contains only the new added partition, we have to combine it with the + // old partitions to check all partitions is strictly increasing. + clonedMeta := meta.Clone() + tmp := *partInfo + tmp.Definitions = append(pi.Definitions, tmp.Definitions...) + clonedMeta.Partition = &tmp + if err := checkPartitionDefinitionConstraints(ctx.GetExprCtx(), clonedMeta); err != nil { + if dbterror.ErrSameNamePartition.Equal(err) && spec.IfNotExists { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + return nil + } + return errors.Trace(err) + } + + if err = handlePartitionPlacement(ctx, partInfo); err != nil { + return errors.Trace(err) + } + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: schema.ID, + TableID: meta.ID, + SchemaName: schema.Name.L, + TableName: t.Meta().Name.L, + Type: model.ActionAddTablePartition, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.TablePartitionArgs{ + PartInfo: partInfo, + } + + if spec.Tp == ast.AlterTableAddLastPartition && spec.Partition != nil { + query, ok := ctx.Value(sessionctx.QueryString).(string) + if ok { + sqlMode := ctx.GetSessionVars().SQLMode + var buf bytes.Buffer + AppendPartitionDefs(partInfo, &buf, sqlMode) + + syntacticSugar := spec.Partition.PartitionMethod.OriginalText() + syntacticStart := spec.Partition.PartitionMethod.OriginTextPosition() + newQuery := query[:syntacticStart] + "ADD PARTITION (" + buf.String() + ")" + query[syntacticStart+len(syntacticSugar):] + defer ctx.SetValue(sessionctx.QueryString, query) + ctx.SetValue(sessionctx.QueryString, newQuery) + } + } + err = e.doDDLJob2(ctx, job, args) + if dbterror.ErrSameNamePartition.Equal(err) && spec.IfNotExists { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + return nil + } + return errors.Trace(err) +} + +// getReorganizedDefinitions return the definitions as they would look like after the REORGANIZE PARTITION is done. +func getReorganizedDefinitions(pi *model.PartitionInfo, firstPartIdx, lastPartIdx int, idMap map[int]struct{}) []model.PartitionDefinition { + tmpDefs := make([]model.PartitionDefinition, 0, len(pi.Definitions)+len(pi.AddingDefinitions)-len(idMap)) + if pi.Type == pmodel.PartitionTypeList { + replaced := false + for i := range pi.Definitions { + if _, ok := idMap[i]; ok { + if !replaced { + tmpDefs = append(tmpDefs, pi.AddingDefinitions...) + replaced = true + } + continue + } + tmpDefs = append(tmpDefs, pi.Definitions[i]) + } + if !replaced { + // For safety, for future non-partitioned table -> partitioned + tmpDefs = append(tmpDefs, pi.AddingDefinitions...) + } + return tmpDefs + } + // Range + tmpDefs = append(tmpDefs, pi.Definitions[:firstPartIdx]...) + tmpDefs = append(tmpDefs, pi.AddingDefinitions...) + if len(pi.Definitions) > (lastPartIdx + 1) { + tmpDefs = append(tmpDefs, pi.Definitions[lastPartIdx+1:]...) + } + return tmpDefs +} + +func getReplacedPartitionIDs(names []string, pi *model.PartitionInfo) (firstPartIdx int, lastPartIdx int, idMap map[int]struct{}, err error) { + idMap = make(map[int]struct{}) + firstPartIdx, lastPartIdx = -1, -1 + for _, name := range names { + nameL := strings.ToLower(name) + partIdx := pi.FindPartitionDefinitionByName(nameL) + if partIdx == -1 { + return 0, 0, nil, errors.Trace(dbterror.ErrWrongPartitionName) + } + if _, ok := idMap[partIdx]; ok { + return 0, 0, nil, errors.Trace(dbterror.ErrSameNamePartition) + } + idMap[partIdx] = struct{}{} + if firstPartIdx == -1 { + firstPartIdx = partIdx + } else { + firstPartIdx = mathutil.Min[int](firstPartIdx, partIdx) + } + if lastPartIdx == -1 { + lastPartIdx = partIdx + } else { + lastPartIdx = mathutil.Max[int](lastPartIdx, partIdx) + } + } + switch pi.Type { + case pmodel.PartitionTypeRange: + if len(idMap) != (lastPartIdx - firstPartIdx + 1) { + return 0, 0, nil, errors.Trace(dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs( + "REORGANIZE PARTITION of RANGE; not adjacent partitions")) + } + case pmodel.PartitionTypeHash, pmodel.PartitionTypeKey: + if len(idMap) != len(pi.Definitions) { + return 0, 0, nil, errors.Trace(dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs( + "REORGANIZE PARTITION of HASH/RANGE; must reorganize all partitions")) + } + } + + return firstPartIdx, lastPartIdx, idMap, nil +} + +func getPartitionInfoTypeNone() *model.PartitionInfo { + return &model.PartitionInfo{ + Type: pmodel.PartitionTypeNone, + Enable: true, + Definitions: []model.PartitionDefinition{{ + Name: pmodel.NewCIStr("pFullTable"), + Comment: "Intermediate partition during ALTER TABLE ... PARTITION BY ...", + }}, + Num: 1, + } +} + +// AlterTablePartitioning reorganize one set of partitions to a new set of partitions. +func (e *executor) AlterTablePartitioning(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { + schema, t, err := e.getSchemaAndTableByIdent(ident) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.FastGenByArgs(ident.Schema, ident.Name)) + } + + meta := t.Meta().Clone() + piOld := meta.GetPartitionInfo() + var partNames []string + if piOld != nil { + partNames = make([]string, 0, len(piOld.Definitions)) + for i := range piOld.Definitions { + partNames = append(partNames, piOld.Definitions[i].Name.L) + } + } else { + piOld = getPartitionInfoTypeNone() + meta.Partition = piOld + partNames = append(partNames, piOld.Definitions[0].Name.L) + } + newMeta := meta.Clone() + + err = buildTablePartitionInfo(NewMetaBuildContextWithSctx(ctx), spec.Partition, newMeta) + if err != nil { + return err + } + + newPartInfo := newMeta.Partition + rewritePartitionQueryString(ctx, spec.Partition, newMeta) + + if err = handlePartitionPlacement(ctx, newPartInfo); err != nil { + return errors.Trace(err) + } + + newPartInfo.DDLType = piOld.Type + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: schema.ID, + TableID: meta.ID, + SchemaName: schema.Name.L, + TableName: t.Meta().Name.L, + Type: model.ActionAlterTablePartitioning, + BinlogInfo: &model.HistoryInfo{}, + ReorgMeta: NewDDLReorgMeta(ctx), + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + + args := &model.TablePartitionArgs{ + PartNames: partNames, + PartInfo: newPartInfo, + } + // No preSplitAndScatter here, it will be done by the worker in onReorganizePartition instead. + err = e.doDDLJob2(ctx, job, args) + if err == nil { + ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackError("The statistics of new partitions will be outdated after reorganizing partitions. Please use 'ANALYZE TABLE' statement if you want to update it now")) + } + return errors.Trace(err) +} + +// ReorganizePartitions reorganize one set of partitions to a new set of partitions. +func (e *executor) ReorganizePartitions(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { + schema, t, err := e.getSchemaAndTableByIdent(ident) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.FastGenByArgs(ident.Schema, ident.Name)) + } + + meta := t.Meta() + pi := meta.GetPartitionInfo() + if pi == nil { + return dbterror.ErrPartitionMgmtOnNonpartitioned + } + switch pi.Type { + case pmodel.PartitionTypeRange, pmodel.PartitionTypeList: + case pmodel.PartitionTypeHash, pmodel.PartitionTypeKey: + if spec.Tp != ast.AlterTableCoalescePartitions && + spec.Tp != ast.AlterTableAddPartitions { + return errors.Trace(dbterror.ErrUnsupportedReorganizePartition) + } + default: + return errors.Trace(dbterror.ErrUnsupportedReorganizePartition) + } + partNames := make([]string, 0, len(spec.PartitionNames)) + for _, name := range spec.PartitionNames { + partNames = append(partNames, name.L) + } + firstPartIdx, lastPartIdx, idMap, err := getReplacedPartitionIDs(partNames, pi) + if err != nil { + return errors.Trace(err) + } + partInfo, err := BuildAddedPartitionInfo(ctx.GetExprCtx(), meta, spec) + if err != nil { + return errors.Trace(err) + } + if err = checkReorgPartitionDefs(ctx, model.ActionReorganizePartition, meta, partInfo, firstPartIdx, lastPartIdx, idMap); err != nil { + return errors.Trace(err) + } + if err = handlePartitionPlacement(ctx, partInfo); err != nil { + return errors.Trace(err) + } + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: schema.ID, + TableID: meta.ID, + SchemaName: schema.Name.L, + TableName: t.Meta().Name.L, + Type: model.ActionReorganizePartition, + BinlogInfo: &model.HistoryInfo{}, + ReorgMeta: NewDDLReorgMeta(ctx), + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.TablePartitionArgs{ + PartNames: partNames, + PartInfo: partInfo, + } + + // No preSplitAndScatter here, it will be done by the worker in onReorganizePartition instead. + err = e.doDDLJob2(ctx, job, args) + failpoint.InjectCall("afterReorganizePartition") + if err == nil { + ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackError("The statistics of related partitions will be outdated after reorganizing partitions. Please use 'ANALYZE TABLE' statement if you want to update it now")) + } + return errors.Trace(err) +} + +// RemovePartitioning removes partitioning from a table. +func (e *executor) RemovePartitioning(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { + schema, t, err := e.getSchemaAndTableByIdent(ident) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.FastGenByArgs(ident.Schema, ident.Name)) + } + + meta := t.Meta().Clone() + pi := meta.GetPartitionInfo() + if pi == nil { + return dbterror.ErrPartitionMgmtOnNonpartitioned + } + // TODO: Optimize for remove partitioning with a single partition + // TODO: Add the support for this in onReorganizePartition + // skip if only one partition + // If there are only one partition, then we can do: + // change the table id to the partition id + // and keep the statistics for the partition id (which should be similar to the global statistics) + // and it let the GC clean up the old table metadata including possible global index. + + newSpec := &ast.AlterTableSpec{} + newSpec.Tp = spec.Tp + defs := make([]*ast.PartitionDefinition, 1) + defs[0] = &ast.PartitionDefinition{} + defs[0].Name = pmodel.NewCIStr("CollapsedPartitions") + newSpec.PartDefinitions = defs + partNames := make([]string, len(pi.Definitions)) + for i := range pi.Definitions { + partNames[i] = pi.Definitions[i].Name.L + } + meta.Partition.Type = pmodel.PartitionTypeNone + partInfo, err := BuildAddedPartitionInfo(ctx.GetExprCtx(), meta, newSpec) + if err != nil { + return errors.Trace(err) + } + // TODO: check where the default placement comes from (i.e. table level) + if err = handlePartitionPlacement(ctx, partInfo); err != nil { + return errors.Trace(err) + } + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: schema.ID, + TableID: meta.ID, + SchemaName: schema.Name.L, + TableName: meta.Name.L, + Type: model.ActionRemovePartitioning, + BinlogInfo: &model.HistoryInfo{}, + ReorgMeta: NewDDLReorgMeta(ctx), + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.TablePartitionArgs{ + PartNames: partNames, + PartInfo: partInfo, + } + + // No preSplitAndScatter here, it will be done by the worker in onReorganizePartition instead. + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +func checkReorgPartitionDefs(ctx sessionctx.Context, action model.ActionType, tblInfo *model.TableInfo, partInfo *model.PartitionInfo, firstPartIdx, lastPartIdx int, idMap map[int]struct{}) error { + // partInfo contains only the new added partition, we have to combine it with the + // old partitions to check all partitions is strictly increasing. + pi := tblInfo.Partition + clonedMeta := tblInfo.Clone() + switch action { + case model.ActionRemovePartitioning, model.ActionAlterTablePartitioning: + clonedMeta.Partition = partInfo + clonedMeta.ID = partInfo.NewTableID + case model.ActionReorganizePartition: + clonedMeta.Partition.AddingDefinitions = partInfo.Definitions + clonedMeta.Partition.Definitions = getReorganizedDefinitions(clonedMeta.Partition, firstPartIdx, lastPartIdx, idMap) + default: + return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("partition type") + } + if err := checkPartitionDefinitionConstraints(ctx.GetExprCtx(), clonedMeta); err != nil { + return errors.Trace(err) + } + if action == model.ActionReorganizePartition { + if pi.Type == pmodel.PartitionTypeRange { + if lastPartIdx == len(pi.Definitions)-1 { + // Last partition dropped, OK to change the end range + // Also includes MAXVALUE + return nil + } + // Check if the replaced end range is the same as before + lastAddingPartition := partInfo.Definitions[len(partInfo.Definitions)-1] + lastOldPartition := pi.Definitions[lastPartIdx] + if len(pi.Columns) > 0 { + newGtOld, err := checkTwoRangeColumns(ctx.GetExprCtx(), &lastAddingPartition, &lastOldPartition, pi, tblInfo) + if err != nil { + return errors.Trace(err) + } + if newGtOld { + return errors.Trace(dbterror.ErrRangeNotIncreasing) + } + oldGtNew, err := checkTwoRangeColumns(ctx.GetExprCtx(), &lastOldPartition, &lastAddingPartition, pi, tblInfo) + if err != nil { + return errors.Trace(err) + } + if oldGtNew { + return errors.Trace(dbterror.ErrRangeNotIncreasing) + } + return nil + } + + isUnsigned := isPartExprUnsigned(ctx.GetExprCtx().GetEvalCtx(), tblInfo) + currentRangeValue, _, err := getRangeValue(ctx.GetExprCtx(), pi.Definitions[lastPartIdx].LessThan[0], isUnsigned) + if err != nil { + return errors.Trace(err) + } + newRangeValue, _, err := getRangeValue(ctx.GetExprCtx(), partInfo.Definitions[len(partInfo.Definitions)-1].LessThan[0], isUnsigned) + if err != nil { + return errors.Trace(err) + } + + if currentRangeValue != newRangeValue { + return errors.Trace(dbterror.ErrRangeNotIncreasing) + } + } + } else { + if len(pi.Definitions) != (lastPartIdx - firstPartIdx + 1) { + // if not ActionReorganizePartition, require all partitions to be changed. + return errors.Trace(dbterror.ErrAlterOperationNotSupported) + } + } + return nil +} + +// CoalescePartitions coalesce partitions can be used with a table that is partitioned by hash or key to reduce the number of partitions by number. +func (e *executor) CoalescePartitions(sctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { + is := e.infoCache.GetLatest() + schema, ok := is.SchemaByName(ident.Schema) + if !ok { + return errors.Trace(infoschema.ErrDatabaseNotExists.GenWithStackByArgs(schema)) + } + t, err := is.TableByName(e.ctx, ident.Schema, ident.Name) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ident.Schema, ident.Name)) + } + + pi := t.Meta().GetPartitionInfo() + if pi == nil { + return errors.Trace(dbterror.ErrPartitionMgmtOnNonpartitioned) + } + + switch pi.Type { + case pmodel.PartitionTypeHash, pmodel.PartitionTypeKey: + return e.hashPartitionManagement(sctx, ident, spec, pi) + + // Coalesce partition can only be used on hash/key partitions. + default: + return errors.Trace(dbterror.ErrCoalesceOnlyOnHashPartition) + } +} + +func (e *executor) hashPartitionManagement(sctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec, pi *model.PartitionInfo) error { + newSpec := *spec + newSpec.PartitionNames = make([]pmodel.CIStr, len(pi.Definitions)) + for i := 0; i < len(pi.Definitions); i++ { + // reorganize ALL partitions into the new number of partitions + newSpec.PartitionNames[i] = pi.Definitions[i].Name + } + for i := 0; i < len(newSpec.PartDefinitions); i++ { + switch newSpec.PartDefinitions[i].Clause.(type) { + case *ast.PartitionDefinitionClauseNone: + // OK, expected + case *ast.PartitionDefinitionClauseIn: + return errors.Trace(ast.ErrPartitionWrongValues.FastGenByArgs("LIST", "IN")) + case *ast.PartitionDefinitionClauseLessThan: + return errors.Trace(ast.ErrPartitionWrongValues.FastGenByArgs("RANGE", "LESS THAN")) + case *ast.PartitionDefinitionClauseHistory: + return errors.Trace(ast.ErrPartitionWrongValues.FastGenByArgs("SYSTEM_TIME", "HISTORY")) + + default: + return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs( + "partitioning clause") + } + } + if newSpec.Num < uint64(len(newSpec.PartDefinitions)) { + newSpec.Num = uint64(len(newSpec.PartDefinitions)) + } + if spec.Tp == ast.AlterTableCoalescePartitions { + if newSpec.Num < 1 { + return ast.ErrCoalescePartitionNoPartition + } + if newSpec.Num >= uint64(len(pi.Definitions)) { + return dbterror.ErrDropLastPartition + } + if isNonDefaultPartitionOptionsUsed(pi.Definitions) { + // The partition definitions will be copied in buildHashPartitionDefinitions() + // if there is a non-empty list of definitions + newSpec.PartDefinitions = []*ast.PartitionDefinition{{}} + } + } + + return e.ReorganizePartitions(sctx, ident, &newSpec) +} + +func (e *executor) TruncateTablePartition(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { + is := e.infoCache.GetLatest() + schema, ok := is.SchemaByName(ident.Schema) + if !ok { + return errors.Trace(infoschema.ErrDatabaseNotExists.GenWithStackByArgs(schema)) + } + t, err := is.TableByName(e.ctx, ident.Schema, ident.Name) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ident.Schema, ident.Name)) + } + meta := t.Meta() + if meta.GetPartitionInfo() == nil { + return errors.Trace(dbterror.ErrPartitionMgmtOnNonpartitioned) + } + + getTruncatedParts := func(pi *model.PartitionInfo) (*model.PartitionInfo, error) { + if spec.OnAllPartitions { + return pi.Clone(), nil + } + var defs []model.PartitionDefinition + // MySQL allows duplicate partition names in truncate partition + // so we filter them out through a hash + posMap := make(map[int]bool) + for _, name := range spec.PartitionNames { + pos := pi.FindPartitionDefinitionByName(name.L) + if pos < 0 { + return nil, errors.Trace(table.ErrUnknownPartition.GenWithStackByArgs(name.L, ident.Name.O)) + } + if _, ok := posMap[pos]; !ok { + defs = append(defs, pi.Definitions[pos]) + posMap[pos] = true + } + } + pi = pi.Clone() + pi.Definitions = defs + return pi, nil + } + pi, err := getTruncatedParts(meta.GetPartitionInfo()) + if err != nil { + return err + } + pids := make([]int64, 0, len(pi.Definitions)) + for i := range pi.Definitions { + pids = append(pids, pi.Definitions[i].ID) + } + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: schema.ID, + TableID: meta.ID, + SchemaName: schema.Name.L, + SchemaState: model.StatePublic, + TableName: t.Meta().Name.L, + Type: model.ActionTruncateTablePartition, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.TruncateTableArgs{ + OldPartitionIDs: pids, + // job submitter will fill new partition IDs. + } + + err = e.doDDLJob2(ctx, job, args) + if err != nil { + return errors.Trace(err) + } + return nil +} + +func (e *executor) DropTablePartition(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { + is := e.infoCache.GetLatest() + schema, ok := is.SchemaByName(ident.Schema) + if !ok { + return errors.Trace(infoschema.ErrDatabaseNotExists.GenWithStackByArgs(schema)) + } + t, err := is.TableByName(e.ctx, ident.Schema, ident.Name) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ident.Schema, ident.Name)) + } + meta := t.Meta() + if meta.GetPartitionInfo() == nil { + return errors.Trace(dbterror.ErrPartitionMgmtOnNonpartitioned) + } + + if spec.Tp == ast.AlterTableDropFirstPartition { + intervalOptions := getPartitionIntervalFromTable(ctx.GetExprCtx(), meta) + if intervalOptions == nil { + return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs( + "FIRST PARTITION, does not seem like an INTERVAL partitioned table") + } + if len(spec.Partition.Definitions) != 0 { + return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs( + "FIRST PARTITION, table info already contains partition definitions") + } + spec.Partition.Interval = intervalOptions + err = GeneratePartDefsFromInterval(ctx.GetExprCtx(), spec.Tp, meta, spec.Partition) + if err != nil { + return err + } + pNullOffset := 0 + if intervalOptions.NullPart { + pNullOffset = 1 + } + if len(spec.Partition.Definitions) == 0 || + len(spec.Partition.Definitions) >= len(meta.Partition.Definitions)-pNullOffset { + return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs( + "FIRST PARTITION, number of partitions does not match") + } + if len(spec.PartitionNames) != 0 || len(spec.Partition.Definitions) <= 1 { + return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs( + "FIRST PARTITION, given value does not generate a list of partition names to be dropped") + } + for i := range spec.Partition.Definitions { + spec.PartitionNames = append(spec.PartitionNames, meta.Partition.Definitions[i+pNullOffset].Name) + } + // Use the last generated partition as First, i.e. do not drop the last name in the slice + spec.PartitionNames = spec.PartitionNames[:len(spec.PartitionNames)-1] + + query, ok := ctx.Value(sessionctx.QueryString).(string) + if ok { + partNames := make([]string, 0, len(spec.PartitionNames)) + sqlMode := ctx.GetSessionVars().SQLMode + for i := range spec.PartitionNames { + partNames = append(partNames, stringutil.Escape(spec.PartitionNames[i].O, sqlMode)) + } + syntacticSugar := spec.Partition.PartitionMethod.OriginalText() + syntacticStart := spec.Partition.PartitionMethod.OriginTextPosition() + newQuery := query[:syntacticStart] + "DROP PARTITION " + strings.Join(partNames, ", ") + query[syntacticStart+len(syntacticSugar):] + defer ctx.SetValue(sessionctx.QueryString, query) + ctx.SetValue(sessionctx.QueryString, newQuery) + } + } + partNames := make([]string, len(spec.PartitionNames)) + for i, partCIName := range spec.PartitionNames { + partNames[i] = partCIName.L + } + err = CheckDropTablePartition(meta, partNames) + if err != nil { + if dbterror.ErrDropPartitionNonExistent.Equal(err) && spec.IfExists { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + return nil + } + return errors.Trace(err) + } + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: schema.ID, + TableID: meta.ID, + SchemaName: schema.Name.L, + SchemaState: model.StatePublic, + TableName: meta.Name.L, + Type: model.ActionDropTablePartition, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.TablePartitionArgs{ + PartNames: partNames, + } + + err = e.doDDLJob2(ctx, job, args) + if err != nil { + if dbterror.ErrDropPartitionNonExistent.Equal(err) && spec.IfExists { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + return nil + } + return errors.Trace(err) + } + return errors.Trace(err) +} + +func checkFieldTypeCompatible(ft *types.FieldType, other *types.FieldType) bool { + // int(1) could match the type with int(8) + partialEqual := ft.GetType() == other.GetType() && + ft.GetDecimal() == other.GetDecimal() && + ft.GetCharset() == other.GetCharset() && + ft.GetCollate() == other.GetCollate() && + (ft.GetFlen() == other.GetFlen() || ft.StorageLength() != types.VarStorageLen) && + mysql.HasUnsignedFlag(ft.GetFlag()) == mysql.HasUnsignedFlag(other.GetFlag()) && + mysql.HasAutoIncrementFlag(ft.GetFlag()) == mysql.HasAutoIncrementFlag(other.GetFlag()) && + mysql.HasNotNullFlag(ft.GetFlag()) == mysql.HasNotNullFlag(other.GetFlag()) && + mysql.HasZerofillFlag(ft.GetFlag()) == mysql.HasZerofillFlag(other.GetFlag()) && + mysql.HasBinaryFlag(ft.GetFlag()) == mysql.HasBinaryFlag(other.GetFlag()) && + mysql.HasPriKeyFlag(ft.GetFlag()) == mysql.HasPriKeyFlag(other.GetFlag()) + if !partialEqual || len(ft.GetElems()) != len(other.GetElems()) { + return false + } + for i := range ft.GetElems() { + if ft.GetElems()[i] != other.GetElems()[i] { + return false + } + } + return true +} + +func checkTiFlashReplicaCompatible(source *model.TiFlashReplicaInfo, target *model.TiFlashReplicaInfo) bool { + if source == target { + return true + } + if source == nil || target == nil { + return false + } + if source.Count != target.Count || + source.Available != target.Available || len(source.LocationLabels) != len(target.LocationLabels) { + return false + } + for i, lable := range source.LocationLabels { + if target.LocationLabels[i] != lable { + return false + } + } + return true +} + +func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) error { + // check temp table + if target.TempTableType != model.TempTableNone { + return errors.Trace(dbterror.ErrPartitionExchangeTempTable.FastGenByArgs(target.Name)) + } + + // check auto_random + if source.AutoRandomBits != target.AutoRandomBits || + source.AutoRandomRangeBits != target.AutoRandomRangeBits || + source.Charset != target.Charset || + source.Collate != target.Collate || + source.ShardRowIDBits != target.ShardRowIDBits || + source.MaxShardRowIDBits != target.MaxShardRowIDBits || + !checkTiFlashReplicaCompatible(source.TiFlashReplica, target.TiFlashReplica) { + return errors.Trace(dbterror.ErrTablesDifferentMetadata) + } + if len(source.Cols()) != len(target.Cols()) { + return errors.Trace(dbterror.ErrTablesDifferentMetadata) + } + // Col compatible check + for i, sourceCol := range source.Cols() { + targetCol := target.Cols()[i] + if sourceCol.IsVirtualGenerated() != targetCol.IsVirtualGenerated() { + return dbterror.ErrUnsupportedOnGeneratedColumn.GenWithStackByArgs("Exchanging partitions for non-generated columns") + } + // It should strictyle compare expressions for generated columns + if sourceCol.Name.L != targetCol.Name.L || + sourceCol.Hidden != targetCol.Hidden || + !checkFieldTypeCompatible(&sourceCol.FieldType, &targetCol.FieldType) || + sourceCol.GeneratedExprString != targetCol.GeneratedExprString { + return errors.Trace(dbterror.ErrTablesDifferentMetadata) + } + if sourceCol.State != model.StatePublic || + targetCol.State != model.StatePublic { + return errors.Trace(dbterror.ErrTablesDifferentMetadata) + } + if sourceCol.ID != targetCol.ID { + return dbterror.ErrPartitionExchangeDifferentOption.GenWithStackByArgs(fmt.Sprintf("column: %s", sourceCol.Name)) + } + } + if len(source.Indices) != len(target.Indices) { + return errors.Trace(dbterror.ErrTablesDifferentMetadata) + } + for _, sourceIdx := range source.Indices { + if sourceIdx.Global { + return dbterror.ErrPartitionExchangeDifferentOption.GenWithStackByArgs(fmt.Sprintf("global index: %s", sourceIdx.Name)) + } + var compatIdx *model.IndexInfo + for _, targetIdx := range target.Indices { + if strings.EqualFold(sourceIdx.Name.L, targetIdx.Name.L) { + compatIdx = targetIdx + } + } + // No match index + if compatIdx == nil { + return errors.Trace(dbterror.ErrTablesDifferentMetadata) + } + // Index type is not compatible + if sourceIdx.Tp != compatIdx.Tp || + sourceIdx.Unique != compatIdx.Unique || + sourceIdx.Primary != compatIdx.Primary { + return errors.Trace(dbterror.ErrTablesDifferentMetadata) + } + // The index column + if len(sourceIdx.Columns) != len(compatIdx.Columns) { + return errors.Trace(dbterror.ErrTablesDifferentMetadata) + } + for i, sourceIdxCol := range sourceIdx.Columns { + compatIdxCol := compatIdx.Columns[i] + if sourceIdxCol.Length != compatIdxCol.Length || + sourceIdxCol.Name.L != compatIdxCol.Name.L { + return errors.Trace(dbterror.ErrTablesDifferentMetadata) + } + } + if sourceIdx.ID != compatIdx.ID { + return dbterror.ErrPartitionExchangeDifferentOption.GenWithStackByArgs(fmt.Sprintf("index: %s", sourceIdx.Name)) + } + } + + return nil +} + +func checkExchangePartition(pt *model.TableInfo, nt *model.TableInfo) error { + if nt.IsView() || nt.IsSequence() { + return errors.Trace(dbterror.ErrCheckNoSuchTable) + } + if pt.GetPartitionInfo() == nil { + return errors.Trace(dbterror.ErrPartitionMgmtOnNonpartitioned) + } + if nt.GetPartitionInfo() != nil { + return errors.Trace(dbterror.ErrPartitionExchangePartTable.GenWithStackByArgs(nt.Name)) + } + + if len(nt.ForeignKeys) > 0 { + return errors.Trace(dbterror.ErrPartitionExchangeForeignKey.GenWithStackByArgs(nt.Name)) + } + + return nil +} + +func (e *executor) ExchangeTablePartition(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { + ptSchema, pt, err := e.getSchemaAndTableByIdent(ident) + if err != nil { + return errors.Trace(err) + } + + ptMeta := pt.Meta() + + ntIdent := ast.Ident{Schema: spec.NewTable.Schema, Name: spec.NewTable.Name} + + // We should check local temporary here using session's info schema because the local temporary tables are only stored in session. + ntLocalTempTable, err := sessiontxn.GetTxnManager(ctx).GetTxnInfoSchema().TableByName(context.Background(), ntIdent.Schema, ntIdent.Name) + if err == nil && ntLocalTempTable.Meta().TempTableType == model.TempTableLocal { + return errors.Trace(dbterror.ErrPartitionExchangeTempTable.FastGenByArgs(ntLocalTempTable.Meta().Name)) + } + + ntSchema, nt, err := e.getSchemaAndTableByIdent(ntIdent) + if err != nil { + return errors.Trace(err) + } + + ntMeta := nt.Meta() + + err = checkExchangePartition(ptMeta, ntMeta) + if err != nil { + return errors.Trace(err) + } + + partName := spec.PartitionNames[0].L + + // NOTE: if pt is subPartitioned, it should be checked + + defID, err := tables.FindPartitionByName(ptMeta, partName) + if err != nil { + return errors.Trace(err) + } + + err = checkTableDefCompatible(ptMeta, ntMeta) + if err != nil { + return errors.Trace(err) + } + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: ntSchema.ID, + TableID: ntMeta.ID, + SchemaName: ntSchema.Name.L, + TableName: ntMeta.Name.L, + Type: model.ActionExchangeTablePartition, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + InvolvingSchemaInfo: []model.InvolvingSchemaInfo{ + {Database: ptSchema.Name.L, Table: ptMeta.Name.L}, + {Database: ntSchema.Name.L, Table: ntMeta.Name.L}, + }, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.ExchangeTablePartitionArgs{ + PartitionID: defID, + PTSchemaID: ptSchema.ID, + PTTableID: ptMeta.ID, + PartitionName: partName, + WithValidation: spec.WithValidation, + } + + err = e.doDDLJob2(ctx, job, args) + if err != nil { + return errors.Trace(err) + } + ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackError("after the exchange, please analyze related table of the exchange to update statistics")) + return nil +} + +// DropColumn will drop a column from the table, now we don't support drop the column with index covered. +func (e *executor) DropColumn(ctx sessionctx.Context, ti ast.Ident, spec *ast.AlterTableSpec) error { + schema, t, err := e.getSchemaAndTableByIdent(ti) + if err != nil { + return errors.Trace(err) + } + failpoint.InjectCall("afterGetSchemaAndTableByIdent", ctx) + + isDropable, err := checkIsDroppableColumn(ctx, e.infoCache.GetLatest(), schema, t, spec) + if err != nil { + return err + } + if !isDropable { + return nil + } + colName := spec.OldColumnName.Name + err = checkVisibleColumnCnt(t, 0, 1) + if err != nil { + return err + } + + job := &model.Job{ + // to do(joccau) + // we should set Version = model.GetJobVerInUse() after refactor the actionMultiSchemaChange. + Version: model.JobVersion1, + SchemaID: schema.ID, + TableID: t.Meta().ID, + SchemaName: schema.Name.L, + SchemaState: model.StatePublic, + TableName: t.Meta().Name.L, + Type: model.ActionDropColumn, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.TableColumnArgs{ + Col: &model.ColumnInfo{Name: colName}, + IgnoreExistenceErr: spec.IfExists, + } + // we need fill args here, because it will be added subjob which contains args and rawArgs from job. + job.FillArgs(args) + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +func checkIsDroppableColumn(ctx sessionctx.Context, is infoschema.InfoSchema, schema *model.DBInfo, t table.Table, spec *ast.AlterTableSpec) (isDrapable bool, err error) { + tblInfo := t.Meta() + // Check whether dropped column has existed. + colName := spec.OldColumnName.Name + col := table.FindCol(t.VisibleCols(), colName.L) + if col == nil { + err = dbterror.ErrCantDropFieldOrKey.GenWithStackByArgs(colName) + if spec.IfExists { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + return false, nil + } + return false, err + } + + if err = isDroppableColumn(tblInfo, colName); err != nil { + return false, errors.Trace(err) + } + if err = checkDropColumnWithPartitionConstraint(t, colName); err != nil { + return false, errors.Trace(err) + } + // Check the column with foreign key. + err = checkDropColumnWithForeignKeyConstraint(is, schema.Name.L, tblInfo, colName.L) + if err != nil { + return false, errors.Trace(err) + } + // Check the column with TTL config + err = checkDropColumnWithTTLConfig(tblInfo, colName.L) + if err != nil { + return false, errors.Trace(err) + } + // We don't support dropping column with PK handle covered now. + if col.IsPKHandleColumn(tblInfo) { + return false, dbterror.ErrUnsupportedPKHandle + } + if mysql.HasAutoIncrementFlag(col.GetFlag()) && !ctx.GetSessionVars().AllowRemoveAutoInc { + return false, dbterror.ErrCantDropColWithAutoInc + } + return true, nil +} + +// checkDropColumnWithPartitionConstraint is used to check the partition constraint of the drop column. +func checkDropColumnWithPartitionConstraint(t table.Table, colName pmodel.CIStr) error { + if t.Meta().Partition == nil { + return nil + } + pt, ok := t.(table.PartitionedTable) + if !ok { + // Should never happen! + return errors.Trace(dbterror.ErrDependentByPartitionFunctional.GenWithStackByArgs(colName.L)) + } + for _, name := range pt.GetPartitionColumnNames() { + if strings.EqualFold(name.L, colName.L) { + return errors.Trace(dbterror.ErrDependentByPartitionFunctional.GenWithStackByArgs(colName.L)) + } + } + return nil +} + +func checkVisibleColumnCnt(t table.Table, addCnt, dropCnt int) error { + tblInfo := t.Meta() + visibleColumCnt := 0 + for _, column := range tblInfo.Columns { + if !column.Hidden { + visibleColumCnt++ + } + } + if visibleColumCnt+addCnt > dropCnt { + return nil + } + if len(tblInfo.Columns)-visibleColumCnt > 0 { + // There are only invisible columns. + return dbterror.ErrTableMustHaveColumns + } + return dbterror.ErrCantRemoveAllFields +} + +// checkModifyCharsetAndCollation returns error when the charset or collation is not modifiable. +// needRewriteCollationData is used when trying to modify the collation of a column, it is true when the column is with +// index because index of a string column is collation-aware. +func checkModifyCharsetAndCollation(toCharset, toCollate, origCharset, origCollate string, needRewriteCollationData bool) error { + if !charset.ValidCharsetAndCollation(toCharset, toCollate) { + return dbterror.ErrUnknownCharacterSet.GenWithStack("Unknown character set: '%s', collation: '%s'", toCharset, toCollate) + } + + if needRewriteCollationData && collate.NewCollationEnabled() && !collate.CompatibleCollate(origCollate, toCollate) { + return dbterror.ErrUnsupportedModifyCollation.GenWithStackByArgs(origCollate, toCollate) + } + + if (origCharset == charset.CharsetUTF8 && toCharset == charset.CharsetUTF8MB4) || + (origCharset == charset.CharsetUTF8 && toCharset == charset.CharsetUTF8) || + (origCharset == charset.CharsetUTF8MB4 && toCharset == charset.CharsetUTF8MB4) || + (origCharset == charset.CharsetLatin1 && toCharset == charset.CharsetUTF8MB4) { + // TiDB only allow utf8/latin1 to be changed to utf8mb4, or changing the collation when the charset is utf8/utf8mb4/latin1. + return nil + } + + if toCharset != origCharset { + msg := fmt.Sprintf("charset from %s to %s", origCharset, toCharset) + return dbterror.ErrUnsupportedModifyCharset.GenWithStackByArgs(msg) + } + if toCollate != origCollate { + msg := fmt.Sprintf("change collate from %s to %s", origCollate, toCollate) + return dbterror.ErrUnsupportedModifyCharset.GenWithStackByArgs(msg) + } + return nil +} + +func (e *executor) getModifiableColumnJob(ctx context.Context, sctx sessionctx.Context, ident ast.Ident, originalColName pmodel.CIStr, + spec *ast.AlterTableSpec) (*model.Job, error) { + is := e.infoCache.GetLatest() + schema, ok := is.SchemaByName(ident.Schema) + if !ok { + return nil, errors.Trace(infoschema.ErrDatabaseNotExists) + } + t, err := is.TableByName(ctx, ident.Schema, ident.Name) + if err != nil { + return nil, errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ident.Schema, ident.Name)) + } + + return GetModifiableColumnJob(ctx, sctx, is, ident, originalColName, schema, t, spec) +} + +// ChangeColumn renames an existing column and modifies the column's definition, +// currently we only support limited kind of changes +// that do not need to change or check data on the table. +func (e *executor) ChangeColumn(ctx context.Context, sctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { + specNewColumn := spec.NewColumns[0] + if len(specNewColumn.Name.Schema.O) != 0 && ident.Schema.L != specNewColumn.Name.Schema.L { + return dbterror.ErrWrongDBName.GenWithStackByArgs(specNewColumn.Name.Schema.O) + } + if len(spec.OldColumnName.Schema.O) != 0 && ident.Schema.L != spec.OldColumnName.Schema.L { + return dbterror.ErrWrongDBName.GenWithStackByArgs(spec.OldColumnName.Schema.O) + } + if len(specNewColumn.Name.Table.O) != 0 && ident.Name.L != specNewColumn.Name.Table.L { + return dbterror.ErrWrongTableName.GenWithStackByArgs(specNewColumn.Name.Table.O) + } + if len(spec.OldColumnName.Table.O) != 0 && ident.Name.L != spec.OldColumnName.Table.L { + return dbterror.ErrWrongTableName.GenWithStackByArgs(spec.OldColumnName.Table.O) + } + + job, err := e.getModifiableColumnJob(ctx, sctx, ident, spec.OldColumnName.Name, spec) + if err != nil { + if infoschema.ErrColumnNotExists.Equal(err) && spec.IfExists { + sctx.GetSessionVars().StmtCtx.AppendNote(infoschema.ErrColumnNotExists.FastGenByArgs(spec.OldColumnName.Name, ident.Name)) + return nil + } + return errors.Trace(err) + } + + err = e.DoDDLJob(sctx, job) + // column not exists, but if_exists flags is true, so we ignore this error. + if infoschema.ErrColumnNotExists.Equal(err) && spec.IfExists { + sctx.GetSessionVars().StmtCtx.AppendNote(err) + return nil + } + return errors.Trace(err) +} + +// RenameColumn renames an existing column. +func (e *executor) RenameColumn(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { + oldColName := spec.OldColumnName.Name + newColName := spec.NewColumnName.Name + + schema, tbl, err := e.getSchemaAndTableByIdent(ident) + if err != nil { + return errors.Trace(err) + } + + oldCol := table.FindCol(tbl.VisibleCols(), oldColName.L) + if oldCol == nil { + return infoschema.ErrColumnNotExists.GenWithStackByArgs(oldColName, ident.Name) + } + // check if column can rename with check constraint + err = IsColumnRenameableWithCheckConstraint(oldCol.Name, tbl.Meta()) + if err != nil { + return err + } + + if oldColName.L == newColName.L { + return nil + } + if newColName.L == model.ExtraHandleName.L { + return dbterror.ErrWrongColumnName.GenWithStackByArgs(newColName.L) + } + + allCols := tbl.Cols() + colWithNewNameAlreadyExist := table.FindCol(allCols, newColName.L) != nil + if colWithNewNameAlreadyExist { + return infoschema.ErrColumnExists.GenWithStackByArgs(newColName) + } + + // Check generated expression. + err = checkModifyColumnWithGeneratedColumnsConstraint(allCols, oldColName) + if err != nil { + return errors.Trace(err) + } + err = checkDropColumnWithPartitionConstraint(tbl, oldColName) + if err != nil { + return errors.Trace(err) + } + + newCol := oldCol.Clone() + newCol.Name = newColName + job := &model.Job{ + SchemaID: schema.ID, + TableID: tbl.Meta().ID, + SchemaName: schema.Name.L, + TableName: tbl.Meta().Name.L, + Type: model.ActionModifyColumn, + BinlogInfo: &model.HistoryInfo{}, + ReorgMeta: NewDDLReorgMeta(ctx), + Args: []any{&newCol, oldColName, spec.Position, 0, 0}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + err = e.DoDDLJob(ctx, job) + return errors.Trace(err) +} + +// ModifyColumn does modification on an existing column, currently we only support limited kind of changes +// that do not need to change or check data on the table. +func (e *executor) ModifyColumn(ctx context.Context, sctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { + specNewColumn := spec.NewColumns[0] + if len(specNewColumn.Name.Schema.O) != 0 && ident.Schema.L != specNewColumn.Name.Schema.L { + return dbterror.ErrWrongDBName.GenWithStackByArgs(specNewColumn.Name.Schema.O) + } + if len(specNewColumn.Name.Table.O) != 0 && ident.Name.L != specNewColumn.Name.Table.L { + return dbterror.ErrWrongTableName.GenWithStackByArgs(specNewColumn.Name.Table.O) + } + + originalColName := specNewColumn.Name.Name + job, err := e.getModifiableColumnJob(ctx, sctx, ident, originalColName, spec) + if err != nil { + if infoschema.ErrColumnNotExists.Equal(err) && spec.IfExists { + sctx.GetSessionVars().StmtCtx.AppendNote(infoschema.ErrColumnNotExists.FastGenByArgs(originalColName, ident.Name)) + return nil + } + return errors.Trace(err) + } + + err = e.DoDDLJob(sctx, job) + // column not exists, but if_exists flags is true, so we ignore this error. + if infoschema.ErrColumnNotExists.Equal(err) && spec.IfExists { + sctx.GetSessionVars().StmtCtx.AppendNote(err) + return nil + } + return errors.Trace(err) +} + +func (e *executor) AlterColumn(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { + specNewColumn := spec.NewColumns[0] + is := e.infoCache.GetLatest() + schema, ok := is.SchemaByName(ident.Schema) + if !ok { + return infoschema.ErrTableNotExists.GenWithStackByArgs(ident.Schema, ident.Name) + } + t, err := is.TableByName(e.ctx, ident.Schema, ident.Name) + if err != nil { + return infoschema.ErrTableNotExists.GenWithStackByArgs(ident.Schema, ident.Name) + } + + colName := specNewColumn.Name.Name + // Check whether alter column has existed. + oldCol := table.FindCol(t.Cols(), colName.L) + if oldCol == nil { + return dbterror.ErrBadField.GenWithStackByArgs(colName, ident.Name) + } + col := table.ToColumn(oldCol.Clone()) + + // Clean the NoDefaultValueFlag value. + col.DelFlag(mysql.NoDefaultValueFlag) + col.DefaultIsExpr = false + if len(specNewColumn.Options) == 0 { + err = col.SetDefaultValue(nil) + if err != nil { + return errors.Trace(err) + } + col.AddFlag(mysql.NoDefaultValueFlag) + } else { + if IsAutoRandomColumnID(t.Meta(), col.ID) { + return dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomIncompatibleWithDefaultValueErrMsg) + } + hasDefaultValue, err := SetDefaultValue(ctx.GetExprCtx(), col, specNewColumn.Options[0]) + if err != nil { + return errors.Trace(err) + } + if err = checkDefaultValue(ctx.GetExprCtx(), col, hasDefaultValue); err != nil { + return errors.Trace(err) + } + } + + job := &model.Job{ + Version: model.JobVersion1, + SchemaID: schema.ID, + TableID: t.Meta().ID, + SchemaName: schema.Name.L, + TableName: t.Meta().Name.L, + Type: model.ActionSetDefaultValue, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.SetDefaultValueArgs{ + Col: col.ColumnInfo, + } + job.FillArgs(args) + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +// AlterTableComment updates the table comment information. +func (e *executor) AlterTableComment(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { + is := e.infoCache.GetLatest() + schema, ok := is.SchemaByName(ident.Schema) + if !ok { + return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ident.Schema) + } + + tb, err := is.TableByName(e.ctx, ident.Schema, ident.Name) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ident.Schema, ident.Name)) + } + sessionVars := ctx.GetSessionVars() + if _, err = validateCommentLength(sessionVars.StmtCtx.ErrCtx(), sessionVars.SQLMode, ident.Name.L, &spec.Comment, dbterror.ErrTooLongTableComment); err != nil { + return errors.Trace(err) + } + + job := &model.Job{ + Version: model.JobVersion1, + SchemaID: schema.ID, + TableID: tb.Meta().ID, + SchemaName: schema.Name.L, + TableName: tb.Meta().Name.L, + Type: model.ActionModifyTableComment, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.ModifyTableCommentArgs{Comment: spec.Comment} + job.FillArgs(args) + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +// AlterTableAutoIDCache updates the table comment information. +func (e *executor) AlterTableAutoIDCache(ctx sessionctx.Context, ident ast.Ident, newCache int64) error { + schema, tb, err := e.getSchemaAndTableByIdent(ident) + if err != nil { + return errors.Trace(err) + } + tbInfo := tb.Meta() + if (newCache == 1 && tbInfo.AutoIDCache != 1) || + (newCache != 1 && tbInfo.AutoIDCache == 1) { + return fmt.Errorf("Can't Alter AUTO_ID_CACHE between 1 and non-1, the underlying implementation is different") + } + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: schema.ID, + TableID: tb.Meta().ID, + SchemaName: schema.Name.L, + TableName: tb.Meta().Name.L, + Type: model.ActionModifyTableAutoIDCache, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.ModifyTableAutoIDCacheArgs{ + NewCache: newCache, + } + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +// AlterTableCharsetAndCollate changes the table charset and collate. +func (e *executor) AlterTableCharsetAndCollate(ctx sessionctx.Context, ident ast.Ident, toCharset, toCollate string, needsOverwriteCols bool) error { + // use the last one. + if toCharset == "" && toCollate == "" { + return dbterror.ErrUnknownCharacterSet.GenWithStackByArgs(toCharset) + } + + is := e.infoCache.GetLatest() + schema, ok := is.SchemaByName(ident.Schema) + if !ok { + return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ident.Schema) + } + + tb, err := is.TableByName(e.ctx, ident.Schema, ident.Name) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ident.Schema, ident.Name)) + } + + if toCharset == "" { + // charset does not change. + toCharset = tb.Meta().Charset + } + + if toCollate == "" { + // Get the default collation of the charset. + toCollate, err = GetDefaultCollation(toCharset, ctx.GetSessionVars().DefaultCollationForUTF8MB4) + if err != nil { + return errors.Trace(err) + } + } + doNothing, err := checkAlterTableCharset(tb.Meta(), schema, toCharset, toCollate, needsOverwriteCols) + if err != nil { + return err + } + if doNothing { + return nil + } + + job := &model.Job{ + Version: model.JobVersion1, + SchemaID: schema.ID, + TableID: tb.Meta().ID, + SchemaName: schema.Name.L, + TableName: tb.Meta().Name.L, + Type: model.ActionModifyTableCharsetAndCollate, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + + args := &model.ModifyTableCharsetAndCollateArgs{ + ToCharset: toCharset, + ToCollate: toCollate, + NeedsOverwriteCols: needsOverwriteCols, + } + job.FillArgs(args) + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +func shouldModifyTiFlashReplica(tbReplicaInfo *model.TiFlashReplicaInfo, replicaInfo *ast.TiFlashReplicaSpec) bool { + if tbReplicaInfo != nil && tbReplicaInfo.Count == replicaInfo.Count && + len(tbReplicaInfo.LocationLabels) == len(replicaInfo.Labels) { + for i, label := range tbReplicaInfo.LocationLabels { + if replicaInfo.Labels[i] != label { + return true + } + } + return false + } + return true +} + +// addHypoTiFlashReplicaIntoCtx adds this hypothetical tiflash replica into this ctx. +func (*executor) setHypoTiFlashReplica(ctx sessionctx.Context, schemaName, tableName pmodel.CIStr, replicaInfo *ast.TiFlashReplicaSpec) error { + sctx := ctx.GetSessionVars() + if sctx.HypoTiFlashReplicas == nil { + sctx.HypoTiFlashReplicas = make(map[string]map[string]struct{}) + } + if sctx.HypoTiFlashReplicas[schemaName.L] == nil { + sctx.HypoTiFlashReplicas[schemaName.L] = make(map[string]struct{}) + } + if replicaInfo.Count > 0 { // add replicas + sctx.HypoTiFlashReplicas[schemaName.L][tableName.L] = struct{}{} + } else { // delete replicas + delete(sctx.HypoTiFlashReplicas[schemaName.L], tableName.L) + } + return nil +} + +// AlterTableSetTiFlashReplica sets the TiFlash replicas info. +func (e *executor) AlterTableSetTiFlashReplica(ctx sessionctx.Context, ident ast.Ident, replicaInfo *ast.TiFlashReplicaSpec) error { + schema, tb, err := e.getSchemaAndTableByIdent(ident) + if err != nil { + return errors.Trace(err) + } + + err = isTableTiFlashSupported(schema.Name, tb.Meta()) + if err != nil { + return errors.Trace(err) + } + + tbReplicaInfo := tb.Meta().TiFlashReplica + if !shouldModifyTiFlashReplica(tbReplicaInfo, replicaInfo) { + return nil + } + + if replicaInfo.Hypo { + return e.setHypoTiFlashReplica(ctx, schema.Name, tb.Meta().Name, replicaInfo) + } + + err = checkTiFlashReplicaCount(ctx, replicaInfo.Count) + if err != nil { + return errors.Trace(err) + } + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: schema.ID, + TableID: tb.Meta().ID, + SchemaName: schema.Name.L, + TableName: tb.Meta().Name.L, + Type: model.ActionSetTiFlashReplica, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.SetTiFlashReplicaArgs{TiflashReplica: *replicaInfo} + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +// AlterTableTTLInfoOrEnable submit ddl job to change table info according to the ttlInfo, or ttlEnable +// at least one of the `ttlInfo`, `ttlEnable` or `ttlCronJobSchedule` should be not nil. +// When `ttlInfo` is nil, and `ttlEnable` is not, it will use the original `.TTLInfo` in the table info and modify the +// `.Enable`. If the `.TTLInfo` in the table info is empty, this function will return an error. +// When `ttlInfo` is nil, and `ttlCronJobSchedule` is not, it will use the original `.TTLInfo` in the table info and modify the +// `.JobInterval`. If the `.TTLInfo` in the table info is empty, this function will return an error. +// When `ttlInfo` is not nil, it simply submits the job with the `ttlInfo` and ignore the `ttlEnable`. +func (e *executor) AlterTableTTLInfoOrEnable(ctx sessionctx.Context, ident ast.Ident, ttlInfo *model.TTLInfo, ttlEnable *bool, ttlCronJobSchedule *string) error { + is := e.infoCache.GetLatest() + schema, ok := is.SchemaByName(ident.Schema) + if !ok { + return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ident.Schema) + } + + tb, err := is.TableByName(e.ctx, ident.Schema, ident.Name) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ident.Schema, ident.Name)) + } + + tblInfo := tb.Meta().Clone() + tableID := tblInfo.ID + tableName := tblInfo.Name.L + + var job *model.Job + if ttlInfo != nil { + tblInfo.TTLInfo = ttlInfo + err = checkTTLInfoValid(ident.Schema, tblInfo, is) + if err != nil { + return err + } + } else { + if tblInfo.TTLInfo == nil { + if ttlEnable != nil { + return errors.Trace(dbterror.ErrSetTTLOptionForNonTTLTable.FastGenByArgs("TTL_ENABLE")) + } + if ttlCronJobSchedule != nil { + return errors.Trace(dbterror.ErrSetTTLOptionForNonTTLTable.FastGenByArgs("TTL_JOB_INTERVAL")) + } + } + } + + job = &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: schema.ID, + TableID: tableID, + SchemaName: schema.Name.L, + TableName: tableName, + Type: model.ActionAlterTTLInfo, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.AlterTTLInfoArgs{ + TTLInfo: ttlInfo, + TTLEnable: ttlEnable, + TTLCronJobSchedule: ttlCronJobSchedule, + } + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +func (e *executor) AlterTableRemoveTTL(ctx sessionctx.Context, ident ast.Ident) error { + is := e.infoCache.GetLatest() + + schema, ok := is.SchemaByName(ident.Schema) + if !ok { + return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ident.Schema) + } + + tb, err := is.TableByName(e.ctx, ident.Schema, ident.Name) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ident.Schema, ident.Name)) + } + + tblInfo := tb.Meta().Clone() + tableID := tblInfo.ID + tableName := tblInfo.Name.L + + if tblInfo.TTLInfo != nil { + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: schema.ID, + TableID: tableID, + SchemaName: schema.Name.L, + TableName: tableName, + Type: model.ActionAlterTTLRemove, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + err = e.doDDLJob2(ctx, job, &model.EmptyArgs{}) + return errors.Trace(err) + } + + return nil +} + +func isTableTiFlashSupported(dbName pmodel.CIStr, tbl *model.TableInfo) error { + // Memory tables and system tables are not supported by TiFlash + if util.IsMemOrSysDB(dbName.L) { + return errors.Trace(dbterror.ErrUnsupportedTiFlashOperationForSysOrMemTable) + } else if tbl.TempTableType != model.TempTableNone { + return dbterror.ErrOptOnTemporaryTable.GenWithStackByArgs("set on tiflash") + } else if tbl.IsView() || tbl.IsSequence() { + return dbterror.ErrWrongObject.GenWithStackByArgs(dbName, tbl.Name, "BASE TABLE") + } + + // Tables that has charset are not supported by TiFlash + for _, col := range tbl.Cols() { + _, ok := charset.TiFlashSupportedCharsets[col.GetCharset()] + if !ok { + return dbterror.ErrUnsupportedTiFlashOperationForUnsupportedCharsetTable.GenWithStackByArgs(col.GetCharset()) + } + } + + return nil +} + +func checkTiFlashReplicaCount(ctx sessionctx.Context, replicaCount uint64) error { + // Check the tiflash replica count should be less than the total tiflash stores. + tiflashStoreCnt, err := infoschema.GetTiFlashStoreCount(ctx.GetStore()) + if err != nil { + return errors.Trace(err) + } + if replicaCount > tiflashStoreCnt { + return errors.Errorf("the tiflash replica count: %d should be less than the total tiflash server count: %d", replicaCount, tiflashStoreCnt) + } + return nil +} + +// AlterTableAddStatistics registers extended statistics for a table. +func (e *executor) AlterTableAddStatistics(ctx sessionctx.Context, ident ast.Ident, stats *ast.StatisticsSpec, ifNotExists bool) error { + if !ctx.GetSessionVars().EnableExtendedStats { + return errors.New("Extended statistics feature is not generally available now, and tidb_enable_extended_stats is OFF") + } + // Not support Cardinality and Dependency statistics type for now. + if stats.StatsType == ast.StatsTypeCardinality || stats.StatsType == ast.StatsTypeDependency { + return errors.New("Cardinality and Dependency statistics types are not supported now") + } + _, tbl, err := e.getSchemaAndTableByIdent(ident) + if err != nil { + return err + } + tblInfo := tbl.Meta() + if tblInfo.GetPartitionInfo() != nil { + return errors.New("Extended statistics on partitioned tables are not supported now") + } + colIDs := make([]int64, 0, 2) + colIDSet := make(map[int64]struct{}, 2) + // Check whether columns exist. + for _, colName := range stats.Columns { + col := table.FindCol(tbl.VisibleCols(), colName.Name.L) + if col == nil { + return infoschema.ErrColumnNotExists.GenWithStackByArgs(colName.Name, ident.Name) + } + if stats.StatsType == ast.StatsTypeCorrelation && tblInfo.PKIsHandle && mysql.HasPriKeyFlag(col.GetFlag()) { + ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackError("No need to create correlation statistics on the integer primary key column")) + return nil + } + if _, exist := colIDSet[col.ID]; exist { + return errors.Errorf("Cannot create extended statistics on duplicate column names '%s'", colName.Name.L) + } + colIDSet[col.ID] = struct{}{} + colIDs = append(colIDs, col.ID) + } + if len(colIDs) != 2 && (stats.StatsType == ast.StatsTypeCorrelation || stats.StatsType == ast.StatsTypeDependency) { + return errors.New("Only support Correlation and Dependency statistics types on 2 columns") + } + if len(colIDs) < 1 && stats.StatsType == ast.StatsTypeCardinality { + return errors.New("Only support Cardinality statistics type on at least 2 columns") + } + // TODO: check whether covering index exists for cardinality / dependency types. + + // Call utilities of statistics.Handle to modify system tables instead of doing DML directly, + // because locking in Handle can guarantee the correctness of `version` in system tables. + return e.statsHandle.InsertExtendedStats(stats.StatsName, colIDs, int(stats.StatsType), tblInfo.ID, ifNotExists) +} + +// AlterTableDropStatistics logically deletes extended statistics for a table. +func (e *executor) AlterTableDropStatistics(ctx sessionctx.Context, ident ast.Ident, stats *ast.StatisticsSpec, ifExists bool) error { + if !ctx.GetSessionVars().EnableExtendedStats { + return errors.New("Extended statistics feature is not generally available now, and tidb_enable_extended_stats is OFF") + } + _, tbl, err := e.getSchemaAndTableByIdent(ident) + if err != nil { + return err + } + tblInfo := tbl.Meta() + // Call utilities of statistics.Handle to modify system tables instead of doing DML directly, + // because locking in Handle can guarantee the correctness of `version` in system tables. + return e.statsHandle.MarkExtendedStatsDeleted(stats.StatsName, tblInfo.ID, ifExists) +} + +// UpdateTableReplicaInfo updates the table flash replica infos. +func (e *executor) UpdateTableReplicaInfo(ctx sessionctx.Context, physicalID int64, available bool) error { + is := e.infoCache.GetLatest() + tb, ok := is.TableByID(e.ctx, physicalID) + if !ok { + tb, _, _ = is.FindTableByPartitionID(physicalID) + if tb == nil { + return infoschema.ErrTableNotExists.GenWithStack("Table which ID = %d does not exist.", physicalID) + } + } + tbInfo := tb.Meta() + if tbInfo.TiFlashReplica == nil || (tbInfo.ID == physicalID && tbInfo.TiFlashReplica.Available == available) || + (tbInfo.ID != physicalID && available == tbInfo.TiFlashReplica.IsPartitionAvailable(physicalID)) { + return nil + } + + db, ok := infoschema.SchemaByTable(is, tbInfo) + if !ok { + return infoschema.ErrDatabaseNotExists.GenWithStack("Database of table `%s` does not exist.", tb.Meta().Name) + } + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: db.ID, + TableID: tb.Meta().ID, + SchemaName: db.Name.L, + TableName: tb.Meta().Name.L, + Type: model.ActionUpdateTiFlashReplicaStatus, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.UpdateTiFlashReplicaStatusArgs{ + Available: available, + PhysicalID: physicalID, + } + err := e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +// checkAlterTableCharset uses to check is it possible to change the charset of table. +// This function returns 2 variable: +// doNothing: if doNothing is true, means no need to change any more, because the target charset is same with the charset of table. +// err: if err is not nil, means it is not possible to change table charset to target charset. +func checkAlterTableCharset(tblInfo *model.TableInfo, dbInfo *model.DBInfo, toCharset, toCollate string, needsOverwriteCols bool) (doNothing bool, err error) { + origCharset := tblInfo.Charset + origCollate := tblInfo.Collate + // Old version schema charset maybe modified when load schema if TreatOldVersionUTF8AsUTF8MB4 was enable. + // So even if the origCharset equal toCharset, we still need to do the ddl for old version schema. + if origCharset == toCharset && origCollate == toCollate && tblInfo.Version >= model.TableInfoVersion2 { + // nothing to do. + doNothing = true + for _, col := range tblInfo.Columns { + if col.GetCharset() == charset.CharsetBin { + continue + } + if col.GetCharset() == toCharset && col.GetCollate() == toCollate { + continue + } + doNothing = false + } + if doNothing { + return doNothing, nil + } + } + + // This DDL will update the table charset to default charset. + origCharset, origCollate, err = ResolveCharsetCollation([]ast.CharsetOpt{ + {Chs: origCharset, Col: origCollate}, + {Chs: dbInfo.Charset, Col: dbInfo.Collate}, + }, "") + if err != nil { + return doNothing, err + } + + if err = checkModifyCharsetAndCollation(toCharset, toCollate, origCharset, origCollate, false); err != nil { + return doNothing, err + } + if !needsOverwriteCols { + // If we don't change the charset and collation of columns, skip the next checks. + return doNothing, nil + } + + for _, col := range tblInfo.Columns { + if col.GetType() == mysql.TypeVarchar { + if err = types.IsVarcharTooBigFieldLength(col.GetFlen(), col.Name.O, toCharset); err != nil { + return doNothing, err + } + } + if col.GetCharset() == charset.CharsetBin { + continue + } + if len(col.GetCharset()) == 0 { + continue + } + if err = checkModifyCharsetAndCollation(toCharset, toCollate, col.GetCharset(), col.GetCollate(), isColumnWithIndex(col.Name.L, tblInfo.Indices)); err != nil { + if strings.Contains(err.Error(), "Unsupported modifying collation") { + colErrMsg := "Unsupported converting collation of column '%s' from '%s' to '%s' when index is defined on it." + err = dbterror.ErrUnsupportedModifyCollation.GenWithStack(colErrMsg, col.Name.L, col.GetCollate(), toCollate) + } + return doNothing, err + } + } + return doNothing, nil +} + +// RenameIndex renames an index. +// In TiDB, indexes are case-insensitive (so index 'a' and 'A" are considered the same index), +// but index names are case-sensitive (we can rename index 'a' to 'A') +func (e *executor) RenameIndex(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { + is := e.infoCache.GetLatest() + schema, ok := is.SchemaByName(ident.Schema) + if !ok { + return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ident.Schema) + } + + tb, err := is.TableByName(e.ctx, ident.Schema, ident.Name) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ident.Schema, ident.Name)) + } + if tb.Meta().TableCacheStatusType != model.TableCacheStatusDisable { + return errors.Trace(dbterror.ErrOptOnCacheTable.GenWithStackByArgs("Rename Index")) + } + duplicate, err := ValidateRenameIndex(spec.FromKey, spec.ToKey, tb.Meta()) + if duplicate { + return nil + } + if err != nil { + return errors.Trace(err) + } + + job := &model.Job{ + SchemaID: schema.ID, + TableID: tb.Meta().ID, + SchemaName: schema.Name.L, + TableName: tb.Meta().Name.L, + Type: model.ActionRenameIndex, + BinlogInfo: &model.HistoryInfo{}, + Args: []any{spec.FromKey, spec.ToKey}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + + err = e.DoDDLJob(ctx, job) + return errors.Trace(err) +} + +// If one drop those tables by mistake, it's difficult to recover. +// In the worst case, the whole TiDB cluster fails to bootstrap, so we prevent user from dropping them. +var systemTables = map[string]struct{}{ + "tidb": {}, + "gc_delete_range": {}, + "gc_delete_range_done": {}, +} + +func isUndroppableTable(schema, table string) bool { + if schema != mysql.SystemDB { + return false + } + if _, ok := systemTables[table]; ok { + return true + } + return false +} + +type objectType int + +const ( + tableObject objectType = iota + viewObject + sequenceObject +) + +// dropTableObject provides common logic to DROP TABLE/VIEW/SEQUENCE. +func (e *executor) dropTableObject( + ctx sessionctx.Context, + objects []*ast.TableName, + ifExists bool, + tableObjectType objectType, +) error { + var ( + notExistTables []string + sessVars = ctx.GetSessionVars() + is = e.infoCache.GetLatest() + dropExistErr *terror.Error + jobType model.ActionType + ) + + var ( + objectIdents []ast.Ident + fkCheck bool + ) + switch tableObjectType { + case tableObject: + dropExistErr = infoschema.ErrTableDropExists + jobType = model.ActionDropTable + objectIdents = make([]ast.Ident, len(objects)) + fkCheck = ctx.GetSessionVars().ForeignKeyChecks + for i, tn := range objects { + objectIdents[i] = ast.Ident{Schema: tn.Schema, Name: tn.Name} + } + for _, tn := range objects { + if referredFK := checkTableHasForeignKeyReferred(is, tn.Schema.L, tn.Name.L, objectIdents, fkCheck); referredFK != nil { + return errors.Trace(dbterror.ErrForeignKeyCannotDropParent.GenWithStackByArgs(tn.Name, referredFK.ChildFKName, referredFK.ChildTable)) + } + } + case viewObject: + dropExistErr = infoschema.ErrTableDropExists + jobType = model.ActionDropView + case sequenceObject: + dropExistErr = infoschema.ErrSequenceDropExists + jobType = model.ActionDropSequence + } + for _, tn := range objects { + fullti := ast.Ident{Schema: tn.Schema, Name: tn.Name} + schema, ok := is.SchemaByName(tn.Schema) + if !ok { + // TODO: we should return special error for table not exist, checking "not exist" is not enough, + // because some other errors may contain this error string too. + notExistTables = append(notExistTables, fullti.String()) + continue + } + tableInfo, err := is.TableByName(e.ctx, tn.Schema, tn.Name) + if err != nil && infoschema.ErrTableNotExists.Equal(err) { + notExistTables = append(notExistTables, fullti.String()) + continue + } else if err != nil { + return err + } + + // prechecks before build DDL job + + // Protect important system table from been dropped by a mistake. + // I can hardly find a case that a user really need to do this. + if isUndroppableTable(tn.Schema.L, tn.Name.L) { + return errors.Errorf("Drop tidb system table '%s.%s' is forbidden", tn.Schema.L, tn.Name.L) + } + switch tableObjectType { + case tableObject: + if !tableInfo.Meta().IsBaseTable() { + notExistTables = append(notExistTables, fullti.String()) + continue + } + + tempTableType := tableInfo.Meta().TempTableType + if config.CheckTableBeforeDrop && tempTableType == model.TempTableNone { + logutil.DDLLogger().Warn("admin check table before drop", + zap.String("database", fullti.Schema.O), + zap.String("table", fullti.Name.O), + ) + exec := ctx.GetRestrictedSQLExecutor() + internalCtx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) + _, _, err := exec.ExecRestrictedSQL(internalCtx, nil, "admin check table %n.%n", fullti.Schema.O, fullti.Name.O) + if err != nil { + return err + } + } + + if tableInfo.Meta().TableCacheStatusType != model.TableCacheStatusDisable { + return dbterror.ErrOptOnCacheTable.GenWithStackByArgs("Drop Table") + } + case viewObject: + if !tableInfo.Meta().IsView() { + return dbterror.ErrWrongObject.GenWithStackByArgs(fullti.Schema, fullti.Name, "VIEW") + } + case sequenceObject: + if !tableInfo.Meta().IsSequence() { + err = dbterror.ErrWrongObject.GenWithStackByArgs(fullti.Schema, fullti.Name, "SEQUENCE") + if ifExists { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + continue + } + return err + } + } + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: schema.ID, + TableID: tableInfo.Meta().ID, + SchemaName: schema.Name.L, + SchemaState: schema.State, + TableName: tableInfo.Meta().Name.L, + Type: jobType, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.DropTableArgs{ + Identifiers: objectIdents, + FKCheck: fkCheck, + } + + err = e.doDDLJob2(ctx, job, args) + if infoschema.ErrDatabaseNotExists.Equal(err) || infoschema.ErrTableNotExists.Equal(err) { + notExistTables = append(notExistTables, fullti.String()) + continue + } else if err != nil { + return errors.Trace(err) + } + + // unlock table after drop + if tableObjectType != tableObject { + continue + } + if !config.TableLockEnabled() { + continue + } + if ok, _ := ctx.CheckTableLocked(tableInfo.Meta().ID); ok { + ctx.ReleaseTableLockByTableIDs([]int64{tableInfo.Meta().ID}) + } + } + if len(notExistTables) > 0 && !ifExists { + return dropExistErr.FastGenByArgs(strings.Join(notExistTables, ",")) + } + // We need add warning when use if exists. + if len(notExistTables) > 0 && ifExists { + for _, table := range notExistTables { + sessVars.StmtCtx.AppendNote(dropExistErr.FastGenByArgs(table)) + } + } + return nil +} + +// DropTable will proceed even if some table in the list does not exists. +func (e *executor) DropTable(ctx sessionctx.Context, stmt *ast.DropTableStmt) (err error) { + return e.dropTableObject(ctx, stmt.Tables, stmt.IfExists, tableObject) +} + +// DropView will proceed even if some view in the list does not exists. +func (e *executor) DropView(ctx sessionctx.Context, stmt *ast.DropTableStmt) (err error) { + return e.dropTableObject(ctx, stmt.Tables, stmt.IfExists, viewObject) +} + +func (e *executor) TruncateTable(ctx sessionctx.Context, ti ast.Ident) error { + schema, tb, err := e.getSchemaAndTableByIdent(ti) + if err != nil { + return errors.Trace(err) + } + tblInfo := tb.Meta() + if tblInfo.IsView() || tblInfo.IsSequence() { + return infoschema.ErrTableNotExists.GenWithStackByArgs(schema.Name.O, tblInfo.Name.O) + } + if tblInfo.TableCacheStatusType != model.TableCacheStatusDisable { + return dbterror.ErrOptOnCacheTable.GenWithStackByArgs("Truncate Table") + } + fkCheck := ctx.GetSessionVars().ForeignKeyChecks + referredFK := checkTableHasForeignKeyReferred(e.infoCache.GetLatest(), ti.Schema.L, ti.Name.L, []ast.Ident{{Name: ti.Name, Schema: ti.Schema}}, fkCheck) + if referredFK != nil { + msg := fmt.Sprintf("`%s`.`%s` CONSTRAINT `%s`", referredFK.ChildSchema, referredFK.ChildTable, referredFK.ChildFKName) + return errors.Trace(dbterror.ErrTruncateIllegalForeignKey.GenWithStackByArgs(msg)) + } + + var oldPartitionIDs []int64 + if tblInfo.Partition != nil { + oldPartitionIDs = make([]int64, 0, len(tblInfo.Partition.Definitions)) + for _, def := range tblInfo.Partition.Definitions { + oldPartitionIDs = append(oldPartitionIDs, def.ID) + } + } + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: schema.ID, + TableID: tblInfo.ID, + SchemaName: schema.Name.L, + TableName: tblInfo.Name.L, + Type: model.ActionTruncateTable, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.TruncateTableArgs{ + FKCheck: fkCheck, + OldPartitionIDs: oldPartitionIDs, + } + err = e.doDDLJob2(ctx, job, args) + if err != nil { + return errors.Trace(err) + } + + return nil +} + +func (e *executor) RenameTable(ctx sessionctx.Context, s *ast.RenameTableStmt) error { + isAlterTable := false + var err error + if len(s.TableToTables) == 1 { + oldIdent := ast.Ident{Schema: s.TableToTables[0].OldTable.Schema, Name: s.TableToTables[0].OldTable.Name} + newIdent := ast.Ident{Schema: s.TableToTables[0].NewTable.Schema, Name: s.TableToTables[0].NewTable.Name} + err = e.renameTable(ctx, oldIdent, newIdent, isAlterTable) + } else { + oldIdents := make([]ast.Ident, 0, len(s.TableToTables)) + newIdents := make([]ast.Ident, 0, len(s.TableToTables)) + for _, tables := range s.TableToTables { + oldIdent := ast.Ident{Schema: tables.OldTable.Schema, Name: tables.OldTable.Name} + newIdent := ast.Ident{Schema: tables.NewTable.Schema, Name: tables.NewTable.Name} + oldIdents = append(oldIdents, oldIdent) + newIdents = append(newIdents, newIdent) + } + err = e.renameTables(ctx, oldIdents, newIdents, isAlterTable) + } + return err +} + +func (e *executor) renameTable(ctx sessionctx.Context, oldIdent, newIdent ast.Ident, isAlterTable bool) error { + is := e.infoCache.GetLatest() + tables := make(map[string]int64) + schemas, tableID, err := ExtractTblInfos(is, oldIdent, newIdent, isAlterTable, tables) + if err != nil { + return err + } + + if schemas == nil { + return nil + } + + if tbl, ok := is.TableByID(e.ctx, tableID); ok { + if tbl.Meta().TableCacheStatusType != model.TableCacheStatusDisable { + return errors.Trace(dbterror.ErrOptOnCacheTable.GenWithStackByArgs("Rename Table")) + } + } + + job := &model.Job{ + SchemaID: schemas[1].ID, + TableID: tableID, + SchemaName: schemas[1].Name.L, + TableName: oldIdent.Name.L, + Type: model.ActionRenameTable, + Version: model.GetJobVerInUse(), + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + InvolvingSchemaInfo: []model.InvolvingSchemaInfo{ + {Database: schemas[0].Name.L, Table: oldIdent.Name.L}, + {Database: schemas[1].Name.L, Table: newIdent.Name.L}, + }, + SQLMode: ctx.GetSessionVars().SQLMode, + } + + args := &model.RenameTableArgs{ + OldSchemaID: schemas[0].ID, + OldSchemaName: schemas[0].Name, + NewTableName: newIdent.Name, + } + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +func (e *executor) renameTables(ctx sessionctx.Context, oldIdents, newIdents []ast.Ident, isAlterTable bool) error { + is := e.infoCache.GetLatest() + involveSchemaInfo := make([]model.InvolvingSchemaInfo, 0, len(oldIdents)*2) + + var schemas []*model.DBInfo + var tableID int64 + var err error + + tables := make(map[string]int64) + infos := make([]*model.RenameTableArgs, 0, len(oldIdents)) + for i := 0; i < len(oldIdents); i++ { + schemas, tableID, err = ExtractTblInfos(is, oldIdents[i], newIdents[i], isAlterTable, tables) + if err != nil { + return err + } + + if t, ok := is.TableByID(e.ctx, tableID); ok { + if t.Meta().TableCacheStatusType != model.TableCacheStatusDisable { + return errors.Trace(dbterror.ErrOptOnCacheTable.GenWithStackByArgs("Rename Tables")) + } + } + + infos = append(infos, &model.RenameTableArgs{ + OldSchemaID: schemas[0].ID, + OldSchemaName: schemas[0].Name, + OldTableName: oldIdents[i].Name, + NewSchemaID: schemas[1].ID, + NewTableName: newIdents[i].Name, + TableID: tableID, + }) + + involveSchemaInfo = append(involveSchemaInfo, + model.InvolvingSchemaInfo{ + Database: schemas[0].Name.L, Table: oldIdents[i].Name.L, + }, + model.InvolvingSchemaInfo{ + Database: schemas[1].Name.L, Table: newIdents[i].Name.L, + }, + ) + } + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: schemas[1].ID, + TableID: infos[0].TableID, + SchemaName: schemas[1].Name.L, + Type: model.ActionRenameTables, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + InvolvingSchemaInfo: involveSchemaInfo, + SQLMode: ctx.GetSessionVars().SQLMode, + } + + args := &model.RenameTablesArgs{RenameTableInfos: infos} + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +// ExtractTblInfos extracts the table information from the infoschema. +func ExtractTblInfos(is infoschema.InfoSchema, oldIdent, newIdent ast.Ident, isAlterTable bool, tables map[string]int64) ([]*model.DBInfo, int64, error) { + oldSchema, ok := is.SchemaByName(oldIdent.Schema) + if !ok { + if isAlterTable { + return nil, 0, infoschema.ErrTableNotExists.GenWithStackByArgs(oldIdent.Schema, oldIdent.Name) + } + if tableExists(is, newIdent, tables) { + return nil, 0, infoschema.ErrTableExists.GenWithStackByArgs(newIdent) + } + return nil, 0, infoschema.ErrTableNotExists.GenWithStackByArgs(oldIdent.Schema, oldIdent.Name) + } + if !tableExists(is, oldIdent, tables) { + if isAlterTable { + return nil, 0, infoschema.ErrTableNotExists.GenWithStackByArgs(oldIdent.Schema, oldIdent.Name) + } + if tableExists(is, newIdent, tables) { + return nil, 0, infoschema.ErrTableExists.GenWithStackByArgs(newIdent) + } + return nil, 0, infoschema.ErrTableNotExists.GenWithStackByArgs(oldIdent.Schema, oldIdent.Name) + } + if isAlterTable && newIdent.Schema.L == oldIdent.Schema.L && newIdent.Name.L == oldIdent.Name.L { + // oldIdent is equal to newIdent, do nothing + return nil, 0, nil + } + //View can be renamed only in the same schema. Compatible with mysql + if infoschema.TableIsView(is, oldIdent.Schema, oldIdent.Name) { + if oldIdent.Schema != newIdent.Schema { + return nil, 0, infoschema.ErrForbidSchemaChange.GenWithStackByArgs(oldIdent.Schema, newIdent.Schema) + } + } + + newSchema, ok := is.SchemaByName(newIdent.Schema) + if !ok { + return nil, 0, dbterror.ErrErrorOnRename.GenWithStackByArgs( + fmt.Sprintf("%s.%s", oldIdent.Schema, oldIdent.Name), + fmt.Sprintf("%s.%s", newIdent.Schema, newIdent.Name), + 168, + fmt.Sprintf("Database `%s` doesn't exist", newIdent.Schema)) + } + if tableExists(is, newIdent, tables) { + return nil, 0, infoschema.ErrTableExists.GenWithStackByArgs(newIdent) + } + if err := checkTooLongTable(newIdent.Name); err != nil { + return nil, 0, errors.Trace(err) + } + oldTableID := getTableID(is, oldIdent, tables) + oldIdentKey := getIdentKey(oldIdent) + tables[oldIdentKey] = tableNotExist + newIdentKey := getIdentKey(newIdent) + tables[newIdentKey] = oldTableID + return []*model.DBInfo{oldSchema, newSchema}, oldTableID, nil +} + +func tableExists(is infoschema.InfoSchema, ident ast.Ident, tables map[string]int64) bool { + identKey := getIdentKey(ident) + tableID, ok := tables[identKey] + if (ok && tableID != tableNotExist) || (!ok && is.TableExists(ident.Schema, ident.Name)) { + return true + } + return false +} + +func getTableID(is infoschema.InfoSchema, ident ast.Ident, tables map[string]int64) int64 { + identKey := getIdentKey(ident) + tableID, ok := tables[identKey] + if !ok { + oldTbl, err := is.TableByName(context.Background(), ident.Schema, ident.Name) + if err != nil { + return tableNotExist + } + tableID = oldTbl.Meta().ID + } + return tableID +} + +func getIdentKey(ident ast.Ident) string { + return fmt.Sprintf("%s.%s", ident.Schema.L, ident.Name.L) +} + +func getAnonymousIndexPrefix(isVector bool) string { + colName := "expression_index" + if isVector { + colName = "vector_index" + } + return colName +} + +// GetName4AnonymousIndex returns a valid name for anonymous index. +func GetName4AnonymousIndex(t table.Table, colName pmodel.CIStr, idxName pmodel.CIStr) pmodel.CIStr { + // `id` is used to indicated the index name's suffix. + id := 2 + l := len(t.Indices()) + indexName := colName + if idxName.O != "" { + // Use the provided index name, it only happens when the original index name is too long and be truncated. + indexName = idxName + id = 3 + } + if strings.EqualFold(indexName.L, mysql.PrimaryKeyName) { + indexName = pmodel.NewCIStr(fmt.Sprintf("%s_%d", colName.O, id)) + id = 3 + } + for i := 0; i < l; i++ { + if t.Indices()[i].Meta().Name.L == indexName.L { + indexName = pmodel.NewCIStr(fmt.Sprintf("%s_%d", colName.O, id)) + if err := checkTooLongIndex(indexName); err != nil { + indexName = GetName4AnonymousIndex(t, pmodel.NewCIStr(colName.O[:30]), pmodel.NewCIStr(fmt.Sprintf("%s_%d", colName.O[:30], 2))) + } + i = -1 + id++ + } + } + return indexName +} + +func (e *executor) CreatePrimaryKey(ctx sessionctx.Context, ti ast.Ident, indexName pmodel.CIStr, + indexPartSpecifications []*ast.IndexPartSpecification, indexOption *ast.IndexOption) error { + if indexOption != nil && indexOption.PrimaryKeyTp == pmodel.PrimaryKeyTypeClustered { + return dbterror.ErrUnsupportedModifyPrimaryKey.GenWithStack("Adding clustered primary key is not supported. " + + "Please consider adding NONCLUSTERED primary key instead") + } + schema, t, err := e.getSchemaAndTableByIdent(ti) + if err != nil { + return errors.Trace(err) + } + + if err = checkTooLongIndex(indexName); err != nil { + return dbterror.ErrTooLongIdent.GenWithStackByArgs(mysql.PrimaryKeyName) + } + + indexName = pmodel.NewCIStr(mysql.PrimaryKeyName) + if indexInfo := t.Meta().FindIndexByName(indexName.L); indexInfo != nil || + // If the table's PKIsHandle is true, it also means that this table has a primary key. + t.Meta().PKIsHandle { + return infoschema.ErrMultiplePriKey + } + + // Primary keys cannot include expression index parts. A primary key requires the generated column to be stored, + // but expression index parts are implemented as virtual generated columns, not stored generated columns. + for _, idxPart := range indexPartSpecifications { + if idxPart.Expr != nil { + return dbterror.ErrFunctionalIndexPrimaryKey + } + } + + tblInfo := t.Meta() + // Check before the job is put to the queue. + // This check is redundant, but useful. If DDL check fail before the job is put + // to job queue, the fail path logic is particularly fast. + // After DDL job is put to the queue, and if the check fail, TiDB will run the DDL cancel logic. + // The recover step causes DDL wait a few seconds, makes the unit test painfully slow. + // For same reason, decide whether index is global here. + indexColumns, _, err := buildIndexColumns(NewMetaBuildContextWithSctx(ctx), tblInfo.Columns, indexPartSpecifications, false) + if err != nil { + return errors.Trace(err) + } + if _, err = CheckPKOnGeneratedColumn(tblInfo, indexPartSpecifications); err != nil { + return err + } + + if tblInfo.GetPartitionInfo() != nil { + ck, err := checkPartitionKeysConstraint(tblInfo.GetPartitionInfo(), indexColumns, tblInfo) + if err != nil { + return err + } + if !ck { + // index columns does not contain all partition columns, must be global + if indexOption == nil || !indexOption.Global { + return dbterror.ErrGlobalIndexNotExplicitlySet.GenWithStackByArgs("PRIMARY") + } + validateGlobalIndexWithGeneratedColumns(ctx.GetSessionVars().StmtCtx.ErrCtx(), tblInfo, indexName.O, indexColumns) + } + } + + // May be truncate comment here, when index comment too long and sql_mode is't strict. + if indexOption != nil { + sessionVars := ctx.GetSessionVars() + if _, err = validateCommentLength(sessionVars.StmtCtx.ErrCtx(), sessionVars.SQLMode, indexName.String(), &indexOption.Comment, dbterror.ErrTooLongIndexComment); err != nil { + return errors.Trace(err) + } + } + + unique := true + sqlMode := ctx.GetSessionVars().SQLMode + // global is set to 'false' is just there to be backwards compatible, + // to avoid unmarshal issues, it is now part of indexOption. + global := false + job := &model.Job{ + SchemaID: schema.ID, + TableID: t.Meta().ID, + SchemaName: schema.Name.L, + TableName: t.Meta().Name.L, + Type: model.ActionAddPrimaryKey, + BinlogInfo: &model.HistoryInfo{}, + ReorgMeta: nil, + Args: []any{unique, indexName, indexPartSpecifications, indexOption, sqlMode, nil, global}, + Priority: ctx.GetSessionVars().DDLReorgPriority, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + reorgMeta, err := newReorgMetaFromVariables(job, ctx) + if err != nil { + return err + } + job.ReorgMeta = reorgMeta + + err = e.DoDDLJob(ctx, job) + return errors.Trace(err) +} + +func checkIndexNameAndColumns(ctx *metabuild.Context, t table.Table, indexName pmodel.CIStr, + indexPartSpecifications []*ast.IndexPartSpecification, isVector, ifNotExists bool) (pmodel.CIStr, []*model.ColumnInfo, error) { + // Deal with anonymous index. + if len(indexName.L) == 0 { + colName := pmodel.NewCIStr(getAnonymousIndexPrefix(isVector)) + if indexPartSpecifications[0].Column != nil { + colName = indexPartSpecifications[0].Column.Name + } + indexName = GetName4AnonymousIndex(t, colName, pmodel.NewCIStr("")) + } + + var err error + if indexInfo := t.Meta().FindIndexByName(indexName.L); indexInfo != nil { + if indexInfo.State != model.StatePublic { + // NOTE: explicit error message. See issue #18363. + err = dbterror.ErrDupKeyName.GenWithStack("index already exist %s; "+ + "a background job is trying to add the same index, "+ + "please check by `ADMIN SHOW DDL JOBS`", indexName) + } else { + err = dbterror.ErrDupKeyName.GenWithStackByArgs(indexName) + } + if ifNotExists { + ctx.AppendNote(err) + return pmodel.CIStr{}, nil, nil + } + return pmodel.CIStr{}, nil, err + } + + if err = checkTooLongIndex(indexName); err != nil { + return pmodel.CIStr{}, nil, errors.Trace(err) + } + + // Build hidden columns if necessary. + var hiddenCols []*model.ColumnInfo + if !isVector { + hiddenCols, err = buildHiddenColumnInfoWithCheck(ctx, indexPartSpecifications, indexName, t.Meta(), t.Cols()) + if err != nil { + return pmodel.CIStr{}, nil, err + } + } + if err = checkAddColumnTooManyColumns(len(t.Cols()) + len(hiddenCols)); err != nil { + return pmodel.CIStr{}, nil, errors.Trace(err) + } + + return indexName, hiddenCols, nil +} + +func checkTableTypeForVectorIndex(tblInfo *model.TableInfo) error { + if tblInfo.TableCacheStatusType != model.TableCacheStatusDisable { + return dbterror.ErrOptOnCacheTable.GenWithStackByArgs("Create Vector Index") + } + if tblInfo.TempTableType != model.TempTableNone { + return dbterror.ErrOptOnTemporaryTable.FastGenByArgs("vector index") + } + if tblInfo.GetPartitionInfo() != nil { + return dbterror.ErrUnsupportedAddVectorIndex.FastGenByArgs("unsupported partition table") + } + if tblInfo.TiFlashReplica == nil || tblInfo.TiFlashReplica.Count == 0 { + return dbterror.ErrUnsupportedAddVectorIndex.FastGenByArgs("unsupported empty TiFlash replica, the replica is nil") + } + + return nil +} + +func (e *executor) createVectorIndex(ctx sessionctx.Context, ti ast.Ident, indexName pmodel.CIStr, + indexPartSpecifications []*ast.IndexPartSpecification, indexOption *ast.IndexOption, ifNotExists bool) error { + schema, t, err := e.getSchemaAndTableByIdent(ti) + if err != nil { + return errors.Trace(err) + } + + tblInfo := t.Meta() + if err := checkTableTypeForVectorIndex(tblInfo); err != nil { + return errors.Trace(err) + } + + metaBuildCtx := NewMetaBuildContextWithSctx(ctx) + indexName, _, err = checkIndexNameAndColumns(metaBuildCtx, t, indexName, indexPartSpecifications, true, ifNotExists) + if err != nil { + return errors.Trace(err) + } + _, funcExpr, err := buildVectorInfoWithCheck(indexPartSpecifications, tblInfo) + if err != nil { + return errors.Trace(err) + } + + // Check before the job is put to the queue. + // This check is redundant, but useful. If DDL check fail before the job is put + // to job queue, the fail path logic is particularly fast. + // After DDL job is put to the queue, and if the check fail, TiDB will run the DDL cancel logic. + // The recover step causes DDL wait a few seconds, makes the unit test painfully slow. + // For same reason, decide whether index is global here. + _, _, err = buildIndexColumns(metaBuildCtx, tblInfo.Columns, indexPartSpecifications, true) + if err != nil { + return errors.Trace(err) + } + + // May be truncate comment here, when index comment too long and sql_mode it's strict. + sessionVars := ctx.GetSessionVars() + if _, err = validateCommentLength(sessionVars.StmtCtx.ErrCtx(), sessionVars.SQLMode, indexName.String(), &indexOption.Comment, dbterror.ErrTooLongTableComment); err != nil { + return errors.Trace(err) + } + + job, err := buildAddIndexJobWithoutTypeAndArgs(ctx, schema, t) + if err != nil { + return errors.Trace(err) + } + job.Type = model.ActionAddVectorIndex + indexPartSpecifications[0].Expr = nil + job.Args = []any{indexName, indexPartSpecifications[0], indexOption, funcExpr} + // TODO: support CDCWriteSource + + err = e.DoDDLJob(ctx, job) + // key exists, but if_not_exists flags is true, so we ignore this error. + if dbterror.ErrDupKeyName.Equal(err) && ifNotExists { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + return nil + } + return errors.Trace(err) +} + +func buildAddIndexJobWithoutTypeAndArgs(ctx sessionctx.Context, schema *model.DBInfo, t table.Table) (*model.Job, error) { + tzName, tzOffset := ddlutil.GetTimeZone(ctx) + charset, collate := ctx.GetSessionVars().GetCharsetInfo() + job := &model.Job{ + SchemaID: schema.ID, + TableID: t.Meta().ID, + SchemaName: schema.Name.L, + TableName: t.Meta().Name.L, + BinlogInfo: &model.HistoryInfo{}, + ReorgMeta: &model.DDLReorgMeta{ + SQLMode: ctx.GetSessionVars().SQLMode, + Warnings: make(map[errors.ErrorID]*terror.Error), + WarningsCount: make(map[errors.ErrorID]int64), + Location: &model.TimeZoneLocation{Name: tzName, Offset: tzOffset}, + }, + Priority: ctx.GetSessionVars().DDLReorgPriority, + Charset: charset, + Collate: collate, + SQLMode: ctx.GetSessionVars().SQLMode, + } + reorgMeta, err := newReorgMetaFromVariables(job, ctx) + if err != nil { + return nil, errors.Trace(err) + } + job.ReorgMeta = reorgMeta + return job, nil +} + +func (e *executor) CreateIndex(ctx sessionctx.Context, stmt *ast.CreateIndexStmt) error { + ident := ast.Ident{Schema: stmt.Table.Schema, Name: stmt.Table.Name} + return e.createIndex(ctx, ident, stmt.KeyType, pmodel.NewCIStr(stmt.IndexName), + stmt.IndexPartSpecifications, stmt.IndexOption, stmt.IfNotExists) +} + +// addHypoIndexIntoCtx adds this index as a hypo-index into this ctx. +func (*executor) addHypoIndexIntoCtx(ctx sessionctx.Context, schemaName, tableName pmodel.CIStr, indexInfo *model.IndexInfo) error { + sctx := ctx.GetSessionVars() + indexName := indexInfo.Name + + if sctx.HypoIndexes == nil { + sctx.HypoIndexes = make(map[string]map[string]map[string]*model.IndexInfo) + } + if sctx.HypoIndexes[schemaName.L] == nil { + sctx.HypoIndexes[schemaName.L] = make(map[string]map[string]*model.IndexInfo) + } + if sctx.HypoIndexes[schemaName.L][tableName.L] == nil { + sctx.HypoIndexes[schemaName.L][tableName.L] = make(map[string]*model.IndexInfo) + } + if _, exist := sctx.HypoIndexes[schemaName.L][tableName.L][indexName.L]; exist { + return errors.Trace(errors.Errorf("conflict hypo index name %s", indexName.L)) + } + + sctx.HypoIndexes[schemaName.L][tableName.L][indexName.L] = indexInfo + return nil +} + +func (e *executor) createIndex(ctx sessionctx.Context, ti ast.Ident, keyType ast.IndexKeyType, indexName pmodel.CIStr, + indexPartSpecifications []*ast.IndexPartSpecification, indexOption *ast.IndexOption, ifNotExists bool) error { + // not support Spatial and FullText index + if keyType == ast.IndexKeyTypeFullText || keyType == ast.IndexKeyTypeSpatial { + return dbterror.ErrUnsupportedIndexType.GenWithStack("FULLTEXT and SPATIAL index is not supported") + } + if keyType == ast.IndexKeyTypeVector { + return e.createVectorIndex(ctx, ti, indexName, indexPartSpecifications, indexOption, ifNotExists) + } + unique := keyType == ast.IndexKeyTypeUnique + schema, t, err := e.getSchemaAndTableByIdent(ti) + if err != nil { + return errors.Trace(err) + } + + if t.Meta().TableCacheStatusType != model.TableCacheStatusDisable { + return errors.Trace(dbterror.ErrOptOnCacheTable.GenWithStackByArgs("Create Index")) + } + metaBuildCtx := NewMetaBuildContextWithSctx(ctx) + indexName, hiddenCols, err := checkIndexNameAndColumns(metaBuildCtx, t, indexName, indexPartSpecifications, false, ifNotExists) + if err != nil { + return errors.Trace(err) + } + + tblInfo := t.Meta() + finalColumns := make([]*model.ColumnInfo, len(tblInfo.Columns), len(tblInfo.Columns)+len(hiddenCols)) + copy(finalColumns, tblInfo.Columns) + finalColumns = append(finalColumns, hiddenCols...) + // Check before the job is put to the queue. + // This check is redundant, but useful. If DDL check fail before the job is put + // to job queue, the fail path logic is particularly fast. + // After DDL job is put to the queue, and if the check fail, TiDB will run the DDL cancel logic. + // The recover step causes DDL wait a few seconds, makes the unit test painfully slow. + // For same reason, decide whether index is global here. + indexColumns, _, err := buildIndexColumns(metaBuildCtx, finalColumns, indexPartSpecifications, false) + if err != nil { + return errors.Trace(err) + } + + globalIndex := false + if indexOption != nil && indexOption.Global { + globalIndex = true + } + if globalIndex { + if tblInfo.GetPartitionInfo() == nil { + return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("Global Index on non-partitioned table") + } + if !unique { + // TODO: remove this limitation + return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("Global IndexOption on non-unique index") + } + } + if unique && tblInfo.GetPartitionInfo() != nil { + ck, err := checkPartitionKeysConstraint(tblInfo.GetPartitionInfo(), indexColumns, tblInfo) + if err != nil { + return err + } + if !ck { + // index columns does not contain all partition columns, must be global + if !globalIndex { + return dbterror.ErrGlobalIndexNotExplicitlySet.GenWithStackByArgs(indexName.O) + } + validateGlobalIndexWithGeneratedColumns(ctx.GetSessionVars().StmtCtx.ErrCtx(), tblInfo, indexName.O, indexColumns) + } else if globalIndex { + // TODO: remove this restriction + return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("Global IndexOption on index including all columns in the partitioning expression") + } + } + // May be truncate comment here, when index comment too long and sql_mode is't strict. + if indexOption != nil { + sessionVars := ctx.GetSessionVars() + if _, err = validateCommentLength(sessionVars.StmtCtx.ErrCtx(), sessionVars.SQLMode, indexName.String(), &indexOption.Comment, dbterror.ErrTooLongIndexComment); err != nil { + return errors.Trace(err) + } + } + + if indexOption != nil && indexOption.Tp == pmodel.IndexTypeHypo { // for hypo-index + indexInfo, err := BuildIndexInfo(metaBuildCtx, tblInfo, indexName, false, unique, false, + indexPartSpecifications, indexOption, model.StatePublic) + if err != nil { + return err + } + return e.addHypoIndexIntoCtx(ctx, ti.Schema, ti.Name, indexInfo) + } + + // global is set to 'false' is just there to be backwards compatible, + // to avoid unmarshal issues, it is now part of indexOption. + global := false + job, err := buildAddIndexJobWithoutTypeAndArgs(ctx, schema, t) + if err != nil { + return errors.Trace(err) + } + job.Type = model.ActionAddIndex + job.Args = []any{unique, indexName, indexPartSpecifications, indexOption, hiddenCols, global} + job.CDCWriteSource = ctx.GetSessionVars().CDCWriteSource + + err = e.DoDDLJob(ctx, job) + // key exists, but if_not_exists flags is true, so we ignore this error. + if dbterror.ErrDupKeyName.Equal(err) && ifNotExists { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + return nil + } + return errors.Trace(err) +} + +func newReorgMetaFromVariables(job *model.Job, sctx sessionctx.Context) (*model.DDLReorgMeta, error) { + reorgMeta := NewDDLReorgMeta(sctx) + reorgMeta.IsDistReorg = variable.EnableDistTask.Load() + reorgMeta.IsFastReorg = variable.EnableFastReorg.Load() + reorgMeta.TargetScope = variable.ServiceScope.Load() + if sv, ok := sctx.GetSessionVars().GetSystemVar(variable.TiDBDDLReorgWorkerCount); ok { + reorgMeta.Concurrency = variable.TidbOptInt(sv, 0) + } + if sv, ok := sctx.GetSessionVars().GetSystemVar(variable.TiDBDDLReorgBatchSize); ok { + reorgMeta.BatchSize = variable.TidbOptInt(sv, 0) + } + + if reorgMeta.IsDistReorg && !reorgMeta.IsFastReorg { + return nil, dbterror.ErrUnsupportedDistTask + } + if hasSysDB(job) { + if reorgMeta.IsDistReorg { + logutil.DDLLogger().Info("cannot use distributed task execution on system DB", + zap.Stringer("job", job)) + } + reorgMeta.IsDistReorg = false + reorgMeta.IsFastReorg = false + failpoint.Inject("reorgMetaRecordFastReorgDisabled", func(_ failpoint.Value) { + LastReorgMetaFastReorgDisabled = true + }) + } + + logutil.DDLLogger().Info("initialize reorg meta", + zap.String("jobSchema", job.SchemaName), + zap.String("jobTable", job.TableName), + zap.Stringer("jobType", job.Type), + zap.Bool("enableDistTask", reorgMeta.IsDistReorg), + zap.Bool("enableFastReorg", reorgMeta.IsFastReorg), + zap.String("targetScope", reorgMeta.TargetScope), + zap.Int("concurrency", reorgMeta.Concurrency), + zap.Int("batchSize", reorgMeta.BatchSize), + ) + return reorgMeta, nil +} + +// LastReorgMetaFastReorgDisabled is used for test. +var LastReorgMetaFastReorgDisabled bool + +func buildFKInfo(fkName pmodel.CIStr, keys []*ast.IndexPartSpecification, refer *ast.ReferenceDef, cols []*table.Column) (*model.FKInfo, error) { + if len(keys) != len(refer.IndexPartSpecifications) { + return nil, infoschema.ErrForeignKeyNotMatch.GenWithStackByArgs(fkName, "Key reference and table reference don't match") + } + if err := checkTooLongForeignKey(fkName); err != nil { + return nil, err + } + if err := checkTooLongSchema(refer.Table.Schema); err != nil { + return nil, err + } + if err := checkTooLongTable(refer.Table.Name); err != nil { + return nil, err + } + + // all base columns of stored generated columns + baseCols := make(map[string]struct{}) + for _, col := range cols { + if col.IsGenerated() && col.GeneratedStored { + for name := range col.Dependences { + baseCols[name] = struct{}{} + } + } + } + + fkInfo := &model.FKInfo{ + Name: fkName, + RefSchema: refer.Table.Schema, + RefTable: refer.Table.Name, + Cols: make([]pmodel.CIStr, len(keys)), + } + if variable.EnableForeignKey.Load() { + fkInfo.Version = model.FKVersion1 + } + + for i, key := range keys { + // Check add foreign key to generated columns + // For more detail, see https://dev.mysql.com/doc/refman/8.0/en/innodb-foreign-key-constraints.html#innodb-foreign-key-generated-columns + for _, col := range cols { + if col.Name.L != key.Column.Name.L { + continue + } + if col.IsGenerated() { + // Check foreign key on virtual generated columns + if !col.GeneratedStored { + return nil, infoschema.ErrForeignKeyCannotUseVirtualColumn.GenWithStackByArgs(fkInfo.Name.O, col.Name.O) + } + + // Check wrong reference options of foreign key on stored generated columns + switch refer.OnUpdate.ReferOpt { + case pmodel.ReferOptionCascade, pmodel.ReferOptionSetNull, pmodel.ReferOptionSetDefault: + //nolint: gosec + return nil, dbterror.ErrWrongFKOptionForGeneratedColumn.GenWithStackByArgs("ON UPDATE " + refer.OnUpdate.ReferOpt.String()) + } + switch refer.OnDelete.ReferOpt { + case pmodel.ReferOptionSetNull, pmodel.ReferOptionSetDefault: + //nolint: gosec + return nil, dbterror.ErrWrongFKOptionForGeneratedColumn.GenWithStackByArgs("ON DELETE " + refer.OnDelete.ReferOpt.String()) + } + continue + } + // Check wrong reference options of foreign key on base columns of stored generated columns + if _, ok := baseCols[col.Name.L]; ok { + switch refer.OnUpdate.ReferOpt { + case pmodel.ReferOptionCascade, pmodel.ReferOptionSetNull, pmodel.ReferOptionSetDefault: + return nil, infoschema.ErrCannotAddForeign + } + switch refer.OnDelete.ReferOpt { + case pmodel.ReferOptionCascade, pmodel.ReferOptionSetNull, pmodel.ReferOptionSetDefault: + return nil, infoschema.ErrCannotAddForeign + } + } + } + col := table.FindCol(cols, key.Column.Name.O) + if col == nil { + return nil, dbterror.ErrKeyColumnDoesNotExits.GenWithStackByArgs(key.Column.Name) + } + if mysql.HasNotNullFlag(col.GetFlag()) && (refer.OnDelete.ReferOpt == pmodel.ReferOptionSetNull || refer.OnUpdate.ReferOpt == pmodel.ReferOptionSetNull) { + return nil, infoschema.ErrForeignKeyColumnNotNull.GenWithStackByArgs(col.Name.O, fkName) + } + fkInfo.Cols[i] = key.Column.Name + } + + fkInfo.RefCols = make([]pmodel.CIStr, len(refer.IndexPartSpecifications)) + for i, key := range refer.IndexPartSpecifications { + if err := checkTooLongColumn(key.Column.Name); err != nil { + return nil, err + } + fkInfo.RefCols[i] = key.Column.Name + } + + fkInfo.OnDelete = int(refer.OnDelete.ReferOpt) + fkInfo.OnUpdate = int(refer.OnUpdate.ReferOpt) + + return fkInfo, nil +} + +func (e *executor) CreateForeignKey(ctx sessionctx.Context, ti ast.Ident, fkName pmodel.CIStr, keys []*ast.IndexPartSpecification, refer *ast.ReferenceDef) error { + is := e.infoCache.GetLatest() + schema, ok := is.SchemaByName(ti.Schema) + if !ok { + return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ti.Schema) + } + + t, err := is.TableByName(context.Background(), ti.Schema, ti.Name) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ti.Schema, ti.Name)) + } + if t.Meta().TempTableType != model.TempTableNone { + return infoschema.ErrCannotAddForeign + } + + if fkName.L == "" { + fkName = pmodel.NewCIStr(fmt.Sprintf("fk_%d", t.Meta().MaxForeignKeyID+1)) + } + err = checkFKDupName(t.Meta(), fkName) + if err != nil { + return err + } + fkInfo, err := buildFKInfo(fkName, keys, refer, t.Cols()) + if err != nil { + return errors.Trace(err) + } + fkCheck := ctx.GetSessionVars().ForeignKeyChecks + err = checkAddForeignKeyValid(is, schema.Name.L, t.Meta(), fkInfo, fkCheck) + if err != nil { + return err + } + if model.FindIndexByColumns(t.Meta(), t.Meta().Indices, fkInfo.Cols...) == nil { + // Need to auto create index for fk cols + if ctx.GetSessionVars().StmtCtx.MultiSchemaInfo == nil { + ctx.GetSessionVars().StmtCtx.MultiSchemaInfo = model.NewMultiSchemaInfo() + } + indexPartSpecifications := make([]*ast.IndexPartSpecification, 0, len(fkInfo.Cols)) + for _, col := range fkInfo.Cols { + indexPartSpecifications = append(indexPartSpecifications, &ast.IndexPartSpecification{ + Column: &ast.ColumnName{Name: col}, + Length: types.UnspecifiedLength, // Index prefixes on foreign key columns are not supported. + }) + } + indexOption := &ast.IndexOption{} + err = e.createIndex(ctx, ti, ast.IndexKeyTypeNone, fkInfo.Name, indexPartSpecifications, indexOption, false) + if err != nil { + return err + } + } + + job := &model.Job{ + Version: model.JobVersion1, + SchemaID: schema.ID, + TableID: t.Meta().ID, + SchemaName: schema.Name.L, + TableName: t.Meta().Name.L, + Type: model.ActionAddForeignKey, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + InvolvingSchemaInfo: []model.InvolvingSchemaInfo{ + { + Database: schema.Name.L, + Table: t.Meta().Name.L, + }, + { + Database: fkInfo.RefSchema.L, + Table: fkInfo.RefTable.L, + Mode: model.SharedInvolving, + }, + }, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.AddForeignKeyArgs{ + FkInfo: fkInfo, + FkCheck: fkCheck, + } + job.FillArgs(args) + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +func (e *executor) DropForeignKey(ctx sessionctx.Context, ti ast.Ident, fkName pmodel.CIStr) error { + is := e.infoCache.GetLatest() + schema, ok := is.SchemaByName(ti.Schema) + if !ok { + return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ti.Schema) + } + + t, err := is.TableByName(context.Background(), ti.Schema, ti.Name) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ti.Schema, ti.Name)) + } + + job := &model.Job{ + Version: model.JobVersion1, + SchemaID: schema.ID, + TableID: t.Meta().ID, + SchemaName: schema.Name.L, + SchemaState: model.StatePublic, + TableName: t.Meta().Name.L, + Type: model.ActionDropForeignKey, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.DropForeignKeyArgs{FkName: fkName} + job.FillArgs(args) + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +func (e *executor) DropIndex(ctx sessionctx.Context, stmt *ast.DropIndexStmt) error { + ti := ast.Ident{Schema: stmt.Table.Schema, Name: stmt.Table.Name} + err := e.dropIndex(ctx, ti, pmodel.NewCIStr(stmt.IndexName), stmt.IfExists, stmt.IsHypo) + if (infoschema.ErrDatabaseNotExists.Equal(err) || infoschema.ErrTableNotExists.Equal(err)) && stmt.IfExists { + err = nil + } + return err +} + +// dropHypoIndexFromCtx drops this hypo-index from this ctx. +func (*executor) dropHypoIndexFromCtx(ctx sessionctx.Context, schema, table, index pmodel.CIStr, ifExists bool) error { + sctx := ctx.GetSessionVars() + if sctx.HypoIndexes != nil && + sctx.HypoIndexes[schema.L] != nil && + sctx.HypoIndexes[schema.L][table.L] != nil && + sctx.HypoIndexes[schema.L][table.L][index.L] != nil { + delete(sctx.HypoIndexes[schema.L][table.L], index.L) + return nil + } + if !ifExists { + return dbterror.ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", index) + } + return nil +} + +// dropIndex drops the specified index. +// isHypo is used to indicate whether this operation is for a hypo-index. +func (e *executor) dropIndex(ctx sessionctx.Context, ti ast.Ident, indexName pmodel.CIStr, ifExists, isHypo bool) error { + is := e.infoCache.GetLatest() + schema, ok := is.SchemaByName(ti.Schema) + if !ok { + return errors.Trace(infoschema.ErrDatabaseNotExists) + } + t, err := is.TableByName(context.Background(), ti.Schema, ti.Name) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ti.Schema, ti.Name)) + } + if t.Meta().TableCacheStatusType != model.TableCacheStatusDisable { + return errors.Trace(dbterror.ErrOptOnCacheTable.GenWithStackByArgs("Drop Index")) + } + + if isHypo { + return e.dropHypoIndexFromCtx(ctx, ti.Schema, ti.Name, indexName, ifExists) + } + + indexInfo := t.Meta().FindIndexByName(indexName.L) + + isPK, err := CheckIsDropPrimaryKey(indexName, indexInfo, t) + if err != nil { + return err + } + + if !ctx.GetSessionVars().InRestrictedSQL && ctx.GetSessionVars().PrimaryKeyRequired && isPK { + return infoschema.ErrTableWithoutPrimaryKey + } + + if indexInfo == nil { + err = dbterror.ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) + if ifExists { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + return nil + } + return err + } + + err = checkIndexNeededInForeignKey(is, schema.Name.L, t.Meta(), indexInfo) + if err != nil { + return err + } + + jobTp := model.ActionDropIndex + if isPK { + jobTp = model.ActionDropPrimaryKey + } + + job := &model.Job{ + SchemaID: schema.ID, + TableID: t.Meta().ID, + SchemaName: schema.Name.L, + SchemaState: indexInfo.State, + TableName: t.Meta().Name.L, + Type: jobTp, + BinlogInfo: &model.HistoryInfo{}, + Args: []any{indexName, ifExists}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + + err = e.DoDDLJob(ctx, job) + return errors.Trace(err) +} + +// CheckIsDropPrimaryKey checks if we will drop PK, there are many PK implementations so we provide a helper function. +func CheckIsDropPrimaryKey(indexName pmodel.CIStr, indexInfo *model.IndexInfo, t table.Table) (bool, error) { + var isPK bool + if indexName.L == strings.ToLower(mysql.PrimaryKeyName) && + // Before we fixed #14243, there might be a general index named `primary` but not a primary key. + (indexInfo == nil || indexInfo.Primary) { + isPK = true + } + if isPK { + // If the table's PKIsHandle is true, we can't find the index from the table. So we check the value of PKIsHandle. + if indexInfo == nil && !t.Meta().PKIsHandle { + return isPK, dbterror.ErrCantDropFieldOrKey.GenWithStackByArgs("PRIMARY") + } + if t.Meta().IsCommonHandle || t.Meta().PKIsHandle { + return isPK, dbterror.ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when the table is using clustered index") + } + } + + return isPK, nil +} + +// validateCommentLength checks comment length of table, column, or index +// If comment length is more than the standard length truncate it +// and store the comment length upto the standard comment length size. +func validateCommentLength(ec errctx.Context, sqlMode mysql.SQLMode, name string, comment *string, errTooLongComment *terror.Error) (string, error) { + if comment == nil { + return "", nil + } + + maxLen := MaxCommentLength + // The maximum length of table comment in MySQL 5.7 is 2048 + // Other comment is 1024 + switch errTooLongComment { + case dbterror.ErrTooLongTableComment: + maxLen *= 2 + case dbterror.ErrTooLongFieldComment, dbterror.ErrTooLongIndexComment, dbterror.ErrTooLongTablePartitionComment: + default: + // add more types of terror.Error if need + } + if len(*comment) > maxLen { + err := errTooLongComment.GenWithStackByArgs(name, maxLen) + if sqlMode.HasStrictMode() { + // may be treated like an error. + return "", err + } + ec.AppendWarning(err) + *comment = (*comment)[:maxLen] + } + return *comment, nil +} + +func validateGlobalIndexWithGeneratedColumns(ec errctx.Context, tblInfo *model.TableInfo, indexName string, indexColumns []*model.IndexColumn) { + // Auto analyze is not effective when a global index contains prefix columns or virtual generated columns. + for _, col := range indexColumns { + colInfo := tblInfo.Columns[col.Offset] + isPrefixCol := col.Length != types.UnspecifiedLength + if colInfo.IsVirtualGenerated() || isPrefixCol { + ec.AppendWarning(dbterror.ErrWarnGlobalIndexNeedManuallyAnalyze.FastGenByArgs(indexName)) + return + } + } +} + +// BuildAddedPartitionInfo build alter table add partition info +func BuildAddedPartitionInfo(ctx expression.BuildContext, meta *model.TableInfo, spec *ast.AlterTableSpec) (*model.PartitionInfo, error) { + numParts := uint64(0) + switch meta.Partition.Type { + case pmodel.PartitionTypeNone: + // OK + case pmodel.PartitionTypeList: + if len(spec.PartDefinitions) == 0 { + return nil, ast.ErrPartitionsMustBeDefined.GenWithStackByArgs(meta.Partition.Type) + } + err := checkListPartitions(spec.PartDefinitions) + if err != nil { + return nil, err + } + + case pmodel.PartitionTypeRange: + if spec.Tp == ast.AlterTableAddLastPartition { + err := buildAddedPartitionDefs(ctx, meta, spec) + if err != nil { + return nil, err + } + spec.PartDefinitions = spec.Partition.Definitions + } else { + if len(spec.PartDefinitions) == 0 { + return nil, ast.ErrPartitionsMustBeDefined.GenWithStackByArgs(meta.Partition.Type) + } + } + case pmodel.PartitionTypeHash, pmodel.PartitionTypeKey: + switch spec.Tp { + case ast.AlterTableRemovePartitioning: + numParts = 1 + default: + return nil, errors.Trace(dbterror.ErrUnsupportedAddPartition) + case ast.AlterTableCoalescePartitions: + if int(spec.Num) >= len(meta.Partition.Definitions) { + return nil, dbterror.ErrDropLastPartition + } + numParts = uint64(len(meta.Partition.Definitions)) - spec.Num + case ast.AlterTableAddPartitions: + if len(spec.PartDefinitions) > 0 { + numParts = uint64(len(meta.Partition.Definitions)) + uint64(len(spec.PartDefinitions)) + } else { + numParts = uint64(len(meta.Partition.Definitions)) + spec.Num + } + } + default: + // we don't support ADD PARTITION for all other partition types yet. + return nil, errors.Trace(dbterror.ErrUnsupportedAddPartition) + } + + part := &model.PartitionInfo{ + Type: meta.Partition.Type, + Expr: meta.Partition.Expr, + Columns: meta.Partition.Columns, + Enable: meta.Partition.Enable, + } + + defs, err := buildPartitionDefinitionsInfo(ctx, spec.PartDefinitions, meta, numParts) + if err != nil { + return nil, err + } + + part.Definitions = defs + part.Num = uint64(len(defs)) + return part, nil +} + +func buildAddedPartitionDefs(ctx expression.BuildContext, meta *model.TableInfo, spec *ast.AlterTableSpec) error { + partInterval := getPartitionIntervalFromTable(ctx, meta) + if partInterval == nil { + return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs( + "LAST PARTITION, does not seem like an INTERVAL partitioned table") + } + if partInterval.MaxValPart { + return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("LAST PARTITION when MAXVALUE partition exists") + } + + spec.Partition.Interval = partInterval + + if len(spec.PartDefinitions) > 0 { + return errors.Trace(dbterror.ErrUnsupportedAddPartition) + } + return GeneratePartDefsFromInterval(ctx, spec.Tp, meta, spec.Partition) +} + +// LockTables uses to execute lock tables statement. +func (e *executor) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error { + lockTables := make([]model.TableLockTpInfo, 0, len(stmt.TableLocks)) + sessionInfo := model.SessionInfo{ + ServerID: e.uuid, + SessionID: ctx.GetSessionVars().ConnectionID, + } + uniqueTableID := make(map[int64]struct{}) + involveSchemaInfo := make([]model.InvolvingSchemaInfo, 0, len(stmt.TableLocks)) + // Check whether the table was already locked by another. + for _, tl := range stmt.TableLocks { + tb := tl.Table + err := throwErrIfInMemOrSysDB(ctx, tb.Schema.L) + if err != nil { + return err + } + schema, t, err := e.getSchemaAndTableByIdent(ast.Ident{Schema: tb.Schema, Name: tb.Name}) + if err != nil { + return errors.Trace(err) + } + if t.Meta().IsView() || t.Meta().IsSequence() { + return table.ErrUnsupportedOp.GenWithStackByArgs() + } + + err = checkTableLocked(t.Meta(), tl.Type, sessionInfo) + if err != nil { + return err + } + if _, ok := uniqueTableID[t.Meta().ID]; ok { + return infoschema.ErrNonuniqTable.GenWithStackByArgs(t.Meta().Name) + } + uniqueTableID[t.Meta().ID] = struct{}{} + lockTables = append(lockTables, model.TableLockTpInfo{SchemaID: schema.ID, TableID: t.Meta().ID, Tp: tl.Type}) + involveSchemaInfo = append(involveSchemaInfo, model.InvolvingSchemaInfo{ + Database: schema.Name.L, + Table: t.Meta().Name.L, + }) + } + + unlockTables := ctx.GetAllTableLocks() + args := &model.LockTablesArgs{ + LockTables: lockTables, + UnlockTables: unlockTables, + SessionInfo: sessionInfo, + } + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: lockTables[0].SchemaID, + TableID: lockTables[0].TableID, + Type: model.ActionLockTable, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + InvolvingSchemaInfo: involveSchemaInfo, + SQLMode: ctx.GetSessionVars().SQLMode, + } + // AddTableLock here is avoiding this job was executed successfully but the session was killed before return. + ctx.AddTableLock(lockTables) + err := e.doDDLJob2(ctx, job, args) + if err == nil { + ctx.ReleaseTableLocks(unlockTables) + ctx.AddTableLock(lockTables) + } + return errors.Trace(err) +} + +// UnlockTables uses to execute unlock tables statement. +func (e *executor) UnlockTables(ctx sessionctx.Context, unlockTables []model.TableLockTpInfo) error { + if len(unlockTables) == 0 { + return nil + } + args := &model.LockTablesArgs{ + UnlockTables: unlockTables, + SessionInfo: model.SessionInfo{ + ServerID: e.uuid, + SessionID: ctx.GetSessionVars().ConnectionID, + }, + } + + involveSchemaInfo := make([]model.InvolvingSchemaInfo, 0, len(unlockTables)) + is := e.infoCache.GetLatest() + for _, t := range unlockTables { + schema, ok := is.SchemaByID(t.SchemaID) + if !ok { + continue + } + tbl, ok := is.TableByID(e.ctx, t.TableID) + if !ok { + continue + } + involveSchemaInfo = append(involveSchemaInfo, model.InvolvingSchemaInfo{ + Database: schema.Name.L, + Table: tbl.Meta().Name.L, + }) + } + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: unlockTables[0].SchemaID, + TableID: unlockTables[0].TableID, + Type: model.ActionUnlockTable, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + InvolvingSchemaInfo: involveSchemaInfo, + SQLMode: ctx.GetSessionVars().SQLMode, + } + + err := e.doDDLJob2(ctx, job, args) + if err == nil { + ctx.ReleaseAllTableLocks() + } + return errors.Trace(err) +} + +func throwErrIfInMemOrSysDB(ctx sessionctx.Context, dbLowerName string) error { + if util.IsMemOrSysDB(dbLowerName) { + if ctx.GetSessionVars().User != nil { + return infoschema.ErrAccessDenied.GenWithStackByArgs(ctx.GetSessionVars().User.Username, ctx.GetSessionVars().User.Hostname) + } + return infoschema.ErrAccessDenied.GenWithStackByArgs("", "") + } + return nil +} + +func (e *executor) CleanupTableLock(ctx sessionctx.Context, tables []*ast.TableName) error { + uniqueTableID := make(map[int64]struct{}) + cleanupTables := make([]model.TableLockTpInfo, 0, len(tables)) + unlockedTablesNum := 0 + involvingSchemaInfo := make([]model.InvolvingSchemaInfo, 0, len(tables)) + // Check whether the table was already locked by another. + for _, tb := range tables { + err := throwErrIfInMemOrSysDB(ctx, tb.Schema.L) + if err != nil { + return err + } + schema, t, err := e.getSchemaAndTableByIdent(ast.Ident{Schema: tb.Schema, Name: tb.Name}) + if err != nil { + return errors.Trace(err) + } + if t.Meta().IsView() || t.Meta().IsSequence() { + return table.ErrUnsupportedOp + } + // Maybe the table t was not locked, but still try to unlock this table. + // If we skip unlock the table here, the job maybe not consistent with the job.Query. + // eg: unlock tables t1,t2; If t2 is not locked and skip here, then the job will only unlock table t1, + // and this behaviour is not consistent with the sql query. + if !t.Meta().IsLocked() { + unlockedTablesNum++ + } + if _, ok := uniqueTableID[t.Meta().ID]; ok { + return infoschema.ErrNonuniqTable.GenWithStackByArgs(t.Meta().Name) + } + uniqueTableID[t.Meta().ID] = struct{}{} + cleanupTables = append(cleanupTables, model.TableLockTpInfo{SchemaID: schema.ID, TableID: t.Meta().ID}) + involvingSchemaInfo = append(involvingSchemaInfo, model.InvolvingSchemaInfo{ + Database: schema.Name.L, + Table: t.Meta().Name.L, + }) + } + // If the num of cleanupTables is 0, or all cleanupTables is unlocked, just return here. + if len(cleanupTables) == 0 || len(cleanupTables) == unlockedTablesNum { + return nil + } + + args := &model.LockTablesArgs{ + UnlockTables: cleanupTables, + IsCleanup: true, + } + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: cleanupTables[0].SchemaID, + TableID: cleanupTables[0].TableID, + Type: model.ActionUnlockTable, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + InvolvingSchemaInfo: involvingSchemaInfo, + SQLMode: ctx.GetSessionVars().SQLMode, + } + err := e.doDDLJob2(ctx, job, args) + if err == nil { + ctx.ReleaseTableLocks(cleanupTables) + } + return errors.Trace(err) +} + +func (e *executor) RepairTable(ctx sessionctx.Context, createStmt *ast.CreateTableStmt) error { + // Existence of DB and table has been checked in the preprocessor. + oldTableInfo, ok := (ctx.Value(domainutil.RepairedTable)).(*model.TableInfo) + if !ok || oldTableInfo == nil { + return dbterror.ErrRepairTableFail.GenWithStack("Failed to get the repaired table") + } + oldDBInfo, ok := (ctx.Value(domainutil.RepairedDatabase)).(*model.DBInfo) + if !ok || oldDBInfo == nil { + return dbterror.ErrRepairTableFail.GenWithStack("Failed to get the repaired database") + } + // By now only support same DB repair. + if createStmt.Table.Schema.L != oldDBInfo.Name.L { + return dbterror.ErrRepairTableFail.GenWithStack("Repaired table should in same database with the old one") + } + + // It is necessary to specify the table.ID and partition.ID manually. + newTableInfo, err := buildTableInfoWithCheck(NewMetaBuildContextWithSctx(ctx), ctx.GetStore(), createStmt, + oldTableInfo.Charset, oldTableInfo.Collate, oldTableInfo.PlacementPolicyRef) + if err != nil { + return errors.Trace(err) + } + if createStmt.Partition != nil { + rewritePartitionQueryString(ctx, createStmt.Partition, newTableInfo) + } + // Override newTableInfo with oldTableInfo's element necessary. + // TODO: There may be more element assignments here, and the new TableInfo should be verified with the actual data. + newTableInfo.ID = oldTableInfo.ID + if err = checkAndOverridePartitionID(newTableInfo, oldTableInfo); err != nil { + return err + } + newTableInfo.AutoIncID = oldTableInfo.AutoIncID + // If any old columnInfo has lost, that means the old column ID lost too, repair failed. + for i, newOne := range newTableInfo.Columns { + old := oldTableInfo.FindPublicColumnByName(newOne.Name.L) + if old == nil { + return dbterror.ErrRepairTableFail.GenWithStackByArgs("Column " + newOne.Name.L + " has lost") + } + if newOne.GetType() != old.GetType() { + return dbterror.ErrRepairTableFail.GenWithStackByArgs("Column " + newOne.Name.L + " type should be the same") + } + if newOne.GetFlen() != old.GetFlen() { + logutil.DDLLogger().Warn("admin repair table : Column " + newOne.Name.L + " flen is not equal to the old one") + } + newTableInfo.Columns[i].ID = old.ID + } + // If any old indexInfo has lost, that means the index ID lost too, so did the data, repair failed. + for i, newOne := range newTableInfo.Indices { + old := getIndexInfoByNameAndColumn(oldTableInfo, newOne) + if old == nil { + return dbterror.ErrRepairTableFail.GenWithStackByArgs("Index " + newOne.Name.L + " has lost") + } + if newOne.Tp != old.Tp { + return dbterror.ErrRepairTableFail.GenWithStackByArgs("Index " + newOne.Name.L + " type should be the same") + } + newTableInfo.Indices[i].ID = old.ID + } + + newTableInfo.State = model.StatePublic + err = checkTableInfoValid(newTableInfo) + if err != nil { + return err + } + newTableInfo.State = model.StateNone + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: oldDBInfo.ID, + TableID: newTableInfo.ID, + SchemaName: oldDBInfo.Name.L, + TableName: newTableInfo.Name.L, + Type: model.ActionRepairTable, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + + args := &model.RepairTableArgs{TableInfo: newTableInfo} + err = e.doDDLJob2(ctx, job, args) + if err == nil { + // Remove the old TableInfo from repairInfo before domain reload. + domainutil.RepairInfo.RemoveFromRepairInfo(oldDBInfo.Name.L, oldTableInfo.Name.L) + } + return errors.Trace(err) +} + +func (e *executor) OrderByColumns(ctx sessionctx.Context, ident ast.Ident) error { + _, tb, err := e.getSchemaAndTableByIdent(ident) + if err != nil { + return errors.Trace(err) + } + if tb.Meta().GetPkColInfo() != nil { + ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("ORDER BY ignored as there is a user-defined clustered index in the table '%s'", ident.Name)) + } + return nil +} + +func (e *executor) CreateSequence(ctx sessionctx.Context, stmt *ast.CreateSequenceStmt) error { + ident := ast.Ident{Name: stmt.Name.Name, Schema: stmt.Name.Schema} + sequenceInfo, err := buildSequenceInfo(stmt, ident) + if err != nil { + return err + } + // TiDB describe the sequence within a tableInfo, as a same-level object of a table and view. + tbInfo, err := BuildTableInfo(NewMetaBuildContextWithSctx(ctx), ident.Name, nil, nil, "", "") + if err != nil { + return err + } + tbInfo.Sequence = sequenceInfo + + onExist := OnExistError + if stmt.IfNotExists { + onExist = OnExistIgnore + } + + return e.CreateTableWithInfo(ctx, ident.Schema, tbInfo, nil, WithOnExist(onExist)) +} + +func (e *executor) AlterSequence(ctx sessionctx.Context, stmt *ast.AlterSequenceStmt) error { + ident := ast.Ident{Name: stmt.Name.Name, Schema: stmt.Name.Schema} + is := e.infoCache.GetLatest() + // Check schema existence. + db, ok := is.SchemaByName(ident.Schema) + if !ok { + return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ident.Schema) + } + // Check table existence. + tbl, err := is.TableByName(context.Background(), ident.Schema, ident.Name) + if err != nil { + if stmt.IfExists { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + return nil + } + return err + } + if !tbl.Meta().IsSequence() { + return dbterror.ErrWrongObject.GenWithStackByArgs(ident.Schema, ident.Name, "SEQUENCE") + } + + // Validate the new sequence option value in old sequenceInfo. + oldSequenceInfo := tbl.Meta().Sequence + copySequenceInfo := *oldSequenceInfo + _, _, err = alterSequenceOptions(stmt.SeqOptions, ident, ©SequenceInfo) + if err != nil { + return err + } + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: db.ID, + TableID: tbl.Meta().ID, + SchemaName: db.Name.L, + TableName: tbl.Meta().Name.L, + Type: model.ActionAlterSequence, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.AlterSequenceArgs{ + Ident: ident, + SeqOptions: stmt.SeqOptions, + } + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +func (e *executor) DropSequence(ctx sessionctx.Context, stmt *ast.DropSequenceStmt) (err error) { + return e.dropTableObject(ctx, stmt.Sequences, stmt.IfExists, sequenceObject) +} + +func (e *executor) AlterIndexVisibility(ctx sessionctx.Context, ident ast.Ident, indexName pmodel.CIStr, visibility ast.IndexVisibility) error { + schema, tb, err := e.getSchemaAndTableByIdent(ident) + if err != nil { + return err + } + + invisible := false + if visibility == ast.IndexVisibilityInvisible { + invisible = true + } + + skip, err := validateAlterIndexVisibility(ctx, indexName, invisible, tb.Meta()) + if err != nil { + return errors.Trace(err) + } + if skip { + return nil + } + + job := &model.Job{ + Version: model.JobVersion1, + SchemaID: schema.ID, + TableID: tb.Meta().ID, + SchemaName: schema.Name.L, + TableName: tb.Meta().Name.L, + Type: model.ActionAlterIndexVisibility, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.AlterIndexVisibilityArgs{ + IndexName: indexName, + Invisible: invisible, + } + job.FillArgs(args) + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +func (e *executor) AlterTableAttributes(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { + schema, tb, err := e.getSchemaAndTableByIdent(ident) + if err != nil { + return errors.Trace(err) + } + meta := tb.Meta() + + rule := label.NewRule() + err = rule.ApplyAttributesSpec(spec.AttributesSpec) + if err != nil { + return dbterror.ErrInvalidAttributesSpec.GenWithStackByArgs(err) + } + ids := getIDs([]*model.TableInfo{meta}) + rule.Reset(schema.Name.L, meta.Name.L, "", ids...) + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: schema.ID, + TableID: meta.ID, + SchemaName: schema.Name.L, + TableName: meta.Name.L, + Type: model.ActionAlterTableAttributes, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + + pdLabelRule := (*pdhttp.LabelRule)(rule) + args := &model.AlterTableAttributesArgs{LabelRule: pdLabelRule} + err = e.doDDLJob2(ctx, job, args) + if err != nil { + return errors.Trace(err) + } + + return errors.Trace(err) +} + +func (e *executor) AlterTablePartitionAttributes(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) (err error) { + schema, tb, err := e.getSchemaAndTableByIdent(ident) + if err != nil { + return errors.Trace(err) + } + + meta := tb.Meta() + if meta.Partition == nil { + return errors.Trace(dbterror.ErrPartitionMgmtOnNonpartitioned) + } + + partitionID, err := tables.FindPartitionByName(meta, spec.PartitionNames[0].L) + if err != nil { + return errors.Trace(err) + } + + rule := label.NewRule() + err = rule.ApplyAttributesSpec(spec.AttributesSpec) + if err != nil { + return dbterror.ErrInvalidAttributesSpec.GenWithStackByArgs(err) + } + rule.Reset(schema.Name.L, meta.Name.L, spec.PartitionNames[0].L, partitionID) + + pdLabelRule := (*pdhttp.LabelRule)(rule) + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: schema.ID, + TableID: meta.ID, + SchemaName: schema.Name.L, + TableName: meta.Name.L, + Type: model.ActionAlterTablePartitionAttributes, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.AlterTablePartitionArgs{ + PartitionID: partitionID, + LabelRule: pdLabelRule, + } + + err = e.doDDLJob2(ctx, job, args) + if err != nil { + return errors.Trace(err) + } + + return errors.Trace(err) +} + +func (e *executor) AlterTablePartitionOptions(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) (err error) { + var policyRefInfo *model.PolicyRefInfo + if spec.Options != nil { + for _, op := range spec.Options { + switch op.Tp { + case ast.TableOptionPlacementPolicy: + policyRefInfo = &model.PolicyRefInfo{ + Name: pmodel.NewCIStr(op.StrValue), + } + default: + return errors.Trace(errors.New("unknown partition option")) + } + } + } + + if policyRefInfo != nil { + err = e.AlterTablePartitionPlacement(ctx, ident, spec, policyRefInfo) + if err != nil { + return errors.Trace(err) + } + } + + return nil +} + +func (e *executor) AlterTablePartitionPlacement(ctx sessionctx.Context, tableIdent ast.Ident, spec *ast.AlterTableSpec, policyRefInfo *model.PolicyRefInfo) (err error) { + schema, tb, err := e.getSchemaAndTableByIdent(tableIdent) + if err != nil { + return errors.Trace(err) + } + + tblInfo := tb.Meta() + if tblInfo.Partition == nil { + return errors.Trace(dbterror.ErrPartitionMgmtOnNonpartitioned) + } + + partitionID, err := tables.FindPartitionByName(tblInfo, spec.PartitionNames[0].L) + if err != nil { + return errors.Trace(err) + } + + if checkIgnorePlacementDDL(ctx) { + return nil + } + + policyRefInfo, err = checkAndNormalizePlacementPolicy(ctx, policyRefInfo) + if err != nil { + return errors.Trace(err) + } + + var involveSchemaInfo []model.InvolvingSchemaInfo + if policyRefInfo != nil { + involveSchemaInfo = []model.InvolvingSchemaInfo{ + { + Database: schema.Name.L, + Table: tblInfo.Name.L, + }, + { + Policy: policyRefInfo.Name.L, + Mode: model.SharedInvolving, + }, + } + } + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: schema.ID, + TableID: tblInfo.ID, + SchemaName: schema.Name.L, + TableName: tblInfo.Name.L, + Type: model.ActionAlterTablePartitionPlacement, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + InvolvingSchemaInfo: involveSchemaInfo, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.AlterTablePartitionArgs{ + PartitionID: partitionID, + PolicyRefInfo: policyRefInfo, + } + + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +// AddResourceGroup implements the DDL interface, creates a resource group. +func (e *executor) AddResourceGroup(ctx sessionctx.Context, stmt *ast.CreateResourceGroupStmt) (err error) { + groupName := stmt.ResourceGroupName + groupInfo := &model.ResourceGroupInfo{Name: groupName, ResourceGroupSettings: model.NewResourceGroupSettings()} + groupInfo, err = buildResourceGroup(groupInfo, stmt.ResourceGroupOptionList) + if err != nil { + return err + } + + if _, ok := e.infoCache.GetLatest().ResourceGroupByName(groupName); ok { + if stmt.IfNotExists { + err = infoschema.ErrResourceGroupExists.FastGenByArgs(groupName) + ctx.GetSessionVars().StmtCtx.AppendNote(err) + return nil + } + return infoschema.ErrResourceGroupExists.GenWithStackByArgs(groupName) + } + + if err := checkResourceGroupValidation(groupInfo); err != nil { + return err + } + + logutil.DDLLogger().Debug("create resource group", zap.String("name", groupName.O), zap.Stringer("resource group settings", groupInfo.ResourceGroupSettings)) + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaName: groupName.L, + Type: model.ActionCreateResourceGroup, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + InvolvingSchemaInfo: []model.InvolvingSchemaInfo{{ + ResourceGroup: groupInfo.Name.L, + }}, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.ResourceGroupArgs{RGInfo: groupInfo} + err = e.doDDLJob2(ctx, job, args) + return err +} + +// DropResourceGroup implements the DDL interface. +func (e *executor) DropResourceGroup(ctx sessionctx.Context, stmt *ast.DropResourceGroupStmt) (err error) { + groupName := stmt.ResourceGroupName + if groupName.L == rg.DefaultResourceGroupName { + return resourcegroup.ErrDroppingInternalResourceGroup + } + is := e.infoCache.GetLatest() + // Check group existence. + group, ok := is.ResourceGroupByName(groupName) + if !ok { + err = infoschema.ErrResourceGroupNotExists.GenWithStackByArgs(groupName) + if stmt.IfExists { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + return nil + } + return err + } + + // check to see if some user has dependency on the group + checker := privilege.GetPrivilegeManager(ctx) + if checker == nil { + return errors.New("miss privilege checker") + } + user, matched := checker.MatchUserResourceGroupName(groupName.L) + if matched { + err = errors.Errorf("user [%s] depends on the resource group to drop", user) + return err + } + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: group.ID, + SchemaName: group.Name.L, + Type: model.ActionDropResourceGroup, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + InvolvingSchemaInfo: []model.InvolvingSchemaInfo{{ + ResourceGroup: groupName.L, + }}, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.ResourceGroupArgs{RGInfo: &model.ResourceGroupInfo{Name: groupName}} + err = e.doDDLJob2(ctx, job, args) + return err +} + +// AlterResourceGroup implements the DDL interface. +func (e *executor) AlterResourceGroup(ctx sessionctx.Context, stmt *ast.AlterResourceGroupStmt) (err error) { + groupName := stmt.ResourceGroupName + is := e.infoCache.GetLatest() + // Check group existence. + group, ok := is.ResourceGroupByName(groupName) + if !ok { + err := infoschema.ErrResourceGroupNotExists.GenWithStackByArgs(groupName) + if stmt.IfExists { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + return nil + } + return err + } + newGroupInfo, err := buildResourceGroup(group, stmt.ResourceGroupOptionList) + if err != nil { + return errors.Trace(err) + } + + if err := checkResourceGroupValidation(newGroupInfo); err != nil { + return err + } + + logutil.DDLLogger().Debug("alter resource group", zap.String("name", groupName.L), zap.Stringer("new resource group settings", newGroupInfo.ResourceGroupSettings)) + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: newGroupInfo.ID, + SchemaName: newGroupInfo.Name.L, + Type: model.ActionAlterResourceGroup, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + InvolvingSchemaInfo: []model.InvolvingSchemaInfo{{ + ResourceGroup: newGroupInfo.Name.L, + }}, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.ResourceGroupArgs{RGInfo: newGroupInfo} + err = e.doDDLJob2(ctx, job, args) + return err +} + +func (e *executor) CreatePlacementPolicy(ctx sessionctx.Context, stmt *ast.CreatePlacementPolicyStmt) (err error) { + if checkIgnorePlacementDDL(ctx) { + return nil + } + + if stmt.OrReplace && stmt.IfNotExists { + return dbterror.ErrWrongUsage.GenWithStackByArgs("OR REPLACE", "IF NOT EXISTS") + } + + policyInfo, err := buildPolicyInfo(stmt.PolicyName, stmt.PlacementOptions) + if err != nil { + return errors.Trace(err) + } + + var onExists OnExist + switch { + case stmt.IfNotExists: + onExists = OnExistIgnore + case stmt.OrReplace: + onExists = OnExistReplace + default: + onExists = OnExistError + } + + return e.CreatePlacementPolicyWithInfo(ctx, policyInfo, onExists) +} + +func (e *executor) DropPlacementPolicy(ctx sessionctx.Context, stmt *ast.DropPlacementPolicyStmt) (err error) { + if checkIgnorePlacementDDL(ctx) { + return nil + } + policyName := stmt.PolicyName + is := e.infoCache.GetLatest() + // Check policy existence. + policy, ok := is.PolicyByName(policyName) + if !ok { + err = infoschema.ErrPlacementPolicyNotExists.GenWithStackByArgs(policyName) + if stmt.IfExists { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + return nil + } + return err + } + + if err = CheckPlacementPolicyNotInUseFromInfoSchema(is, policy); err != nil { + return err + } + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: policy.ID, + SchemaName: policy.Name.L, + Type: model.ActionDropPlacementPolicy, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + InvolvingSchemaInfo: []model.InvolvingSchemaInfo{{ + Policy: policyName.L, + }}, + SQLMode: ctx.GetSessionVars().SQLMode, + } + + args := &model.PlacementPolicyArgs{ + PolicyName: policyName, + PolicyID: policy.ID, + } + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +func (e *executor) AlterPlacementPolicy(ctx sessionctx.Context, stmt *ast.AlterPlacementPolicyStmt) (err error) { + if checkIgnorePlacementDDL(ctx) { + return nil + } + policyName := stmt.PolicyName + is := e.infoCache.GetLatest() + // Check policy existence. + policy, ok := is.PolicyByName(policyName) + if !ok { + return infoschema.ErrPlacementPolicyNotExists.GenWithStackByArgs(policyName) + } + + newPolicyInfo, err := buildPolicyInfo(policy.Name, stmt.PlacementOptions) + if err != nil { + return errors.Trace(err) + } + + err = checkPolicyValidation(newPolicyInfo.PlacementSettings) + if err != nil { + return err + } + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: policy.ID, + SchemaName: policy.Name.L, + Type: model.ActionAlterPlacementPolicy, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + InvolvingSchemaInfo: []model.InvolvingSchemaInfo{{ + Policy: newPolicyInfo.Name.L, + }}, + SQLMode: ctx.GetSessionVars().SQLMode, + } + args := &model.PlacementPolicyArgs{ + Policy: newPolicyInfo, + PolicyID: policy.ID, + } + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +func (e *executor) AlterTableCache(sctx sessionctx.Context, ti ast.Ident) (err error) { + schema, t, err := e.getSchemaAndTableByIdent(ti) + if err != nil { + return err + } + // if a table is already in cache state, return directly + if t.Meta().TableCacheStatusType == model.TableCacheStatusEnable { + return nil + } + + // forbidden cache table in system database. + if util.IsMemOrSysDB(schema.Name.L) { + return errors.Trace(dbterror.ErrUnsupportedAlterCacheForSysTable) + } else if t.Meta().TempTableType != model.TempTableNone { + return dbterror.ErrOptOnTemporaryTable.GenWithStackByArgs("alter temporary table cache") + } + + if t.Meta().Partition != nil { + return dbterror.ErrOptOnCacheTable.GenWithStackByArgs("partition mode") + } + + succ, err := checkCacheTableSize(e.store, t.Meta().ID) + if err != nil { + return errors.Trace(err) + } + if !succ { + return dbterror.ErrOptOnCacheTable.GenWithStackByArgs("table too large") + } + + ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) + ddlQuery, _ := sctx.Value(sessionctx.QueryString).(string) + // Initialize the cached table meta lock info in `mysql.table_cache_meta`. + // The operation shouldn't fail in most cases, and if it does, return the error directly. + // This DML and the following DDL is not atomic, that's not a problem. + _, _, err = sctx.GetRestrictedSQLExecutor().ExecRestrictedSQL(ctx, nil, + "replace into mysql.table_cache_meta values (%?, 'NONE', 0, 0)", t.Meta().ID) + if err != nil { + return errors.Trace(err) + } + + sctx.SetValue(sessionctx.QueryString, ddlQuery) + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: schema.ID, + SchemaName: schema.Name.L, + TableName: t.Meta().Name.L, + TableID: t.Meta().ID, + Type: model.ActionAlterCacheTable, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: sctx.GetSessionVars().CDCWriteSource, + SQLMode: sctx.GetSessionVars().SQLMode, + } + + return e.doDDLJob2(sctx, job, &model.EmptyArgs{}) +} + +func checkCacheTableSize(store kv.Storage, tableID int64) (bool, error) { + const cacheTableSizeLimit = 64 * (1 << 20) // 64M + succ := true + ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnCacheTable) + err := kv.RunInNewTxn(ctx, store, true, func(_ context.Context, txn kv.Transaction) error { + txn.SetOption(kv.RequestSourceType, kv.InternalTxnCacheTable) + prefix := tablecodec.GenTablePrefix(tableID) + it, err := txn.Iter(prefix, prefix.PrefixNext()) + if err != nil { + return errors.Trace(err) + } + defer it.Close() + + totalSize := 0 + for it.Valid() && it.Key().HasPrefix(prefix) { + key := it.Key() + value := it.Value() + totalSize += len(key) + totalSize += len(value) + + if totalSize > cacheTableSizeLimit { + succ = false + break + } + + err = it.Next() + if err != nil { + return errors.Trace(err) + } + } + return nil + }) + return succ, err +} + +func (e *executor) AlterTableNoCache(ctx sessionctx.Context, ti ast.Ident) (err error) { + schema, t, err := e.getSchemaAndTableByIdent(ti) + if err != nil { + return err + } + // if a table is not in cache state, return directly + if t.Meta().TableCacheStatusType == model.TableCacheStatusDisable { + return nil + } + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: schema.ID, + SchemaName: schema.Name.L, + TableName: t.Meta().Name.L, + TableID: t.Meta().ID, + Type: model.ActionAlterNoCacheTable, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + return e.doDDLJob2(ctx, job, &model.EmptyArgs{}) +} + +func (e *executor) CreateCheckConstraint(ctx sessionctx.Context, ti ast.Ident, constrName pmodel.CIStr, constr *ast.Constraint) error { + schema, t, err := e.getSchemaAndTableByIdent(ti) + if err != nil { + return errors.Trace(err) + } + if constraintInfo := t.Meta().FindConstraintInfoByName(constrName.L); constraintInfo != nil { + return infoschema.ErrCheckConstraintDupName.GenWithStackByArgs(constrName.L) + } + + // allocate the temporary constraint name for dependency-check-error-output below. + constrNames := map[string]bool{} + for _, constr := range t.Meta().Constraints { + constrNames[constr.Name.L] = true + } + setEmptyCheckConstraintName(t.Meta().Name.L, constrNames, []*ast.Constraint{constr}) + + // existedColsMap can be used to check the existence of depended. + existedColsMap := make(map[string]struct{}) + cols := t.Cols() + for _, v := range cols { + existedColsMap[v.Name.L] = struct{}{} + } + // check expression if supported + if ok, err := table.IsSupportedExpr(constr); !ok { + return err + } + + dependedColsMap := findDependentColsInExpr(constr.Expr) + dependedCols := make([]pmodel.CIStr, 0, len(dependedColsMap)) + for k := range dependedColsMap { + if _, ok := existedColsMap[k]; !ok { + // The table constraint depended on a non-existed column. + return dbterror.ErrBadField.GenWithStackByArgs(k, "check constraint "+constr.Name+" expression") + } + dependedCols = append(dependedCols, pmodel.NewCIStr(k)) + } + + // build constraint meta info. + tblInfo := t.Meta() + + // check auto-increment column + if table.ContainsAutoIncrementCol(dependedCols, tblInfo) { + return dbterror.ErrCheckConstraintRefersAutoIncrementColumn.GenWithStackByArgs(constr.Name) + } + // check foreign key + if err := table.HasForeignKeyRefAction(tblInfo.ForeignKeys, nil, constr, dependedCols); err != nil { + return err + } + constraintInfo, err := buildConstraintInfo(tblInfo, dependedCols, constr, model.StateNone) + if err != nil { + return errors.Trace(err) + } + // check if the expression is bool type + if err := table.IfCheckConstraintExprBoolType(ctx.GetExprCtx().GetEvalCtx(), constraintInfo, tblInfo); err != nil { + return err + } + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: schema.ID, + TableID: tblInfo.ID, + SchemaName: schema.Name.L, + TableName: tblInfo.Name.L, + Type: model.ActionAddCheckConstraint, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + Priority: ctx.GetSessionVars().DDLReorgPriority, + SQLMode: ctx.GetSessionVars().SQLMode, + } + + args := &model.AddCheckConstraintArgs{ + Constraint: constraintInfo, + } + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +func (e *executor) DropCheckConstraint(ctx sessionctx.Context, ti ast.Ident, constrName pmodel.CIStr) error { + is := e.infoCache.GetLatest() + schema, ok := is.SchemaByName(ti.Schema) + if !ok { + return errors.Trace(infoschema.ErrDatabaseNotExists) + } + t, err := is.TableByName(context.Background(), ti.Schema, ti.Name) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ti.Schema, ti.Name)) + } + tblInfo := t.Meta() + + constraintInfo := tblInfo.FindConstraintInfoByName(constrName.L) + if constraintInfo == nil { + return dbterror.ErrConstraintNotFound.GenWithStackByArgs(constrName) + } + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: schema.ID, + TableID: tblInfo.ID, + SchemaName: schema.Name.L, + TableName: tblInfo.Name.L, + Type: model.ActionDropCheckConstraint, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + + args := &model.CheckConstraintArgs{ + ConstraintName: constrName, + } + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +func (e *executor) AlterCheckConstraint(ctx sessionctx.Context, ti ast.Ident, constrName pmodel.CIStr, enforced bool) error { + is := e.infoCache.GetLatest() + schema, ok := is.SchemaByName(ti.Schema) + if !ok { + return errors.Trace(infoschema.ErrDatabaseNotExists) + } + t, err := is.TableByName(context.Background(), ti.Schema, ti.Name) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ti.Schema, ti.Name)) + } + tblInfo := t.Meta() + + constraintInfo := tblInfo.FindConstraintInfoByName(constrName.L) + if constraintInfo == nil { + return dbterror.ErrConstraintNotFound.GenWithStackByArgs(constrName) + } + + job := &model.Job{ + Version: model.GetJobVerInUse(), + SchemaID: schema.ID, + TableID: tblInfo.ID, + SchemaName: schema.Name.L, + TableName: tblInfo.Name.L, + Type: model.ActionAlterCheckConstraint, + BinlogInfo: &model.HistoryInfo{}, + CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, + SQLMode: ctx.GetSessionVars().SQLMode, + } + + args := &model.CheckConstraintArgs{ + ConstraintName: constrName, + Enforced: enforced, + } + err = e.doDDLJob2(ctx, job, args) + return errors.Trace(err) +} + +func (e *executor) genPlacementPolicyID() (int64, error) { + var ret int64 + ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) + err := kv.RunInNewTxn(ctx, e.store, true, func(_ context.Context, txn kv.Transaction) error { + m := meta.NewMutator(txn) + var err error + ret, err = m.GenPlacementPolicyID() + return err + }) + + return ret, err +} + +// DoDDLJob will return +// - nil: found in history DDL job and no job error +// - context.Cancel: job has been sent to worker, but not found in history DDL job before cancel +// - other: found in history DDL job and return that job error +func (e *executor) DoDDLJob(ctx sessionctx.Context, job *model.Job) error { + return e.DoDDLJobWrapper(ctx, NewJobWrapper(job, false)) +} + +func (e *executor) doDDLJob2(ctx sessionctx.Context, job *model.Job, args model.JobArgs) error { + return e.DoDDLJobWrapper(ctx, NewJobWrapperWithArgs(job, args, false)) +} + +// DoDDLJobWrapper submit DDL job and wait it finishes. +// When fast create is enabled, we might merge multiple jobs into one, so do not +// depend on job.ID, use JobID from jobSubmitResult. +func (e *executor) DoDDLJobWrapper(ctx sessionctx.Context, jobW *JobWrapper) (resErr error) { + job := jobW.Job + job.TraceInfo = &model.TraceInfo{ + ConnectionID: ctx.GetSessionVars().ConnectionID, + SessionAlias: ctx.GetSessionVars().SessionAlias, + } + if mci := ctx.GetSessionVars().StmtCtx.MultiSchemaInfo; mci != nil { + // In multiple schema change, we don't run the job. + // Instead, we merge all the jobs into one pending job. + return appendToSubJobs(mci, jobW) + } + // Get a global job ID and put the DDL job in the queue. + setDDLJobQuery(ctx, job) + e.deliverJobTask(jobW) + + failpoint.Inject("mockParallelSameDDLJobTwice", func(val failpoint.Value) { + if val.(bool) { + <-jobW.ResultCh[0] + // The same job will be put to the DDL queue twice. + job = job.Clone() + newJobW := NewJobWrapper(job, jobW.IDAllocated) + e.deliverJobTask(newJobW) + // The second job result is used for test. + jobW = newJobW + } + }) + + // worker should restart to continue handling tasks in limitJobCh, and send back through jobW.err + result := <-jobW.ResultCh[0] + // job.ID must be allocated after previous channel receive returns nil. + jobID, err := result.jobID, result.err + defer e.delJobDoneCh(jobID) + if err != nil { + // The transaction of enqueuing job is failed. + return errors.Trace(err) + } + failpoint.InjectCall("waitJobSubmitted") + + sessVars := ctx.GetSessionVars() + sessVars.StmtCtx.IsDDLJobInQueue = true + + ddlAction := job.Type + if result.merged { + logutil.DDLLogger().Info("DDL job submitted", zap.Int64("job_id", jobID), zap.String("query", job.Query), zap.String("merged", "true")) + } else { + logutil.DDLLogger().Info("DDL job submitted", zap.Stringer("job", job), zap.String("query", job.Query)) + } + + // lock tables works on table ID, for some DDLs which changes table ID, we need + // make sure the session still tracks it. + // we need add it here to avoid this ddl job was executed successfully but the + // session was killed before return. The session will release all table locks + // it holds, if we don't add the new locking table id here, the session may forget + // to release the new locked table id when this ddl job was executed successfully + // but the session was killed before return. + if config.TableLockEnabled() { + HandleLockTablesOnSuccessSubmit(ctx, jobW) + defer func() { + HandleLockTablesOnFinish(ctx, jobW, resErr) + }() + } + + var historyJob *model.Job + + // Attach the context of the jobId to the calling session so that + // KILL can cancel this DDL job. + ctx.GetSessionVars().StmtCtx.DDLJobID = jobID + + // For a job from start to end, the state of it will be none -> delete only -> write only -> reorganization -> public + // For every state changes, we will wait as lease 2 * lease time, so here the ticker check is 10 * lease. + // But we use etcd to speed up, normally it takes less than 0.5s now, so we use 0.5s or 1s or 3s as the max value. + initInterval, _ := getJobCheckInterval(ddlAction, 0) + ticker := time.NewTicker(chooseLeaseTime(10*e.lease, initInterval)) + startTime := time.Now() + metrics.JobsGauge.WithLabelValues(ddlAction.String()).Inc() + defer func() { + ticker.Stop() + metrics.JobsGauge.WithLabelValues(ddlAction.String()).Dec() + metrics.HandleJobHistogram.WithLabelValues(ddlAction.String(), metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) + recordLastDDLInfo(ctx, historyJob) + }() + i := 0 + notifyCh, _ := e.getJobDoneCh(jobID) + for { + failpoint.InjectCall("storeCloseInLoop") + select { + case _, ok := <-notifyCh: + if !ok { + // when fast create enabled, jobs might be merged, and we broadcast + // the result by closing the channel, to avoid this loop keeps running + // without sleeping on retryable error, we set it to nil. + notifyCh = nil + } + case <-ticker.C: + i++ + ticker = updateTickerInterval(ticker, 10*e.lease, ddlAction, i) + case <-e.ctx.Done(): + logutil.DDLLogger().Info("DoDDLJob will quit because context done") + return context.Canceled + } + + // If the connection being killed, we need to CANCEL the DDL job. + if sessVars.SQLKiller.HandleSignal() == exeerrors.ErrQueryInterrupted { + if atomic.LoadInt32(&sessVars.ConnectionStatus) == variable.ConnStatusShutdown { + logutil.DDLLogger().Info("DoDDLJob will quit because context done") + return context.Canceled + } + if sessVars.StmtCtx.DDLJobID != 0 { + se, err := e.sessPool.Get() + if err != nil { + logutil.DDLLogger().Error("get session failed, check again", zap.Error(err)) + continue + } + sessVars.StmtCtx.DDLJobID = 0 // Avoid repeat. + errs, err := CancelJobsBySystem(se, []int64{jobID}) + e.sessPool.Put(se) + if len(errs) > 0 { + logutil.DDLLogger().Warn("error canceling DDL job", zap.Error(errs[0])) + } + if err != nil { + logutil.DDLLogger().Warn("Kill command could not cancel DDL job", zap.Error(err)) + continue + } + } + } + + se, err := e.sessPool.Get() + if err != nil { + logutil.DDLLogger().Error("get session failed, check again", zap.Error(err)) + continue + } + historyJob, err = GetHistoryJobByID(se, jobID) + e.sessPool.Put(se) + if err != nil { + logutil.DDLLogger().Error("get history DDL job failed, check again", zap.Error(err)) + continue + } + if historyJob == nil { + logutil.DDLLogger().Debug("DDL job is not in history, maybe not run", zap.Int64("jobID", jobID)) + continue + } + + e.checkHistoryJobInTest(ctx, historyJob) + + // If a job is a history job, the state must be JobStateSynced or JobStateRollbackDone or JobStateCancelled. + if historyJob.IsSynced() { + // Judge whether there are some warnings when executing DDL under the certain SQL mode. + if historyJob.ReorgMeta != nil && len(historyJob.ReorgMeta.Warnings) != 0 { + if len(historyJob.ReorgMeta.Warnings) != len(historyJob.ReorgMeta.WarningsCount) { + logutil.DDLLogger().Info("DDL warnings doesn't match the warnings count", zap.Int64("jobID", jobID)) + } else { + for key, warning := range historyJob.ReorgMeta.Warnings { + keyCount := historyJob.ReorgMeta.WarningsCount[key] + if keyCount == 1 { + ctx.GetSessionVars().StmtCtx.AppendWarning(warning) + } else { + newMsg := fmt.Sprintf("%d warnings with this error code, first warning: "+warning.GetMsg(), keyCount) + newWarning := dbterror.ClassTypes.Synthesize(terror.ErrCode(warning.Code()), newMsg) + ctx.GetSessionVars().StmtCtx.AppendWarning(newWarning) + } + } + } + } + appendMultiChangeWarningsToOwnerCtx(ctx, historyJob) + + logutil.DDLLogger().Info("DDL job is finished", zap.Int64("jobID", jobID)) + return nil + } + + if historyJob.Error != nil { + logutil.DDLLogger().Info("DDL job is failed", zap.Int64("jobID", jobID)) + return errors.Trace(historyJob.Error) + } + panic("When the state is JobStateRollbackDone or JobStateCancelled, historyJob.Error should never be nil") + } +} + +func getRenameTableUniqueIDs(jobW *JobWrapper, schema bool) []int64 { + if !schema { + return []int64{jobW.TableID} + } + + oldSchemaID := jobW.JobArgs.(*model.RenameTableArgs).OldSchemaID + return []int64{oldSchemaID, jobW.SchemaID} +} + +// HandleLockTablesOnSuccessSubmit handles the table lock for the job which is submitted +// successfully. exported for testing purpose. +func HandleLockTablesOnSuccessSubmit(ctx sessionctx.Context, jobW *JobWrapper) { + if jobW.Type == model.ActionTruncateTable { + if ok, lockTp := ctx.CheckTableLocked(jobW.TableID); ok { + ctx.AddTableLock([]model.TableLockTpInfo{{ + SchemaID: jobW.SchemaID, + TableID: jobW.JobArgs.(*model.TruncateTableArgs).NewTableID, + Tp: lockTp, + }}) + } + } +} + +// HandleLockTablesOnFinish handles the table lock for the job which is finished. +// exported for testing purpose. +func HandleLockTablesOnFinish(ctx sessionctx.Context, jobW *JobWrapper, ddlErr error) { + if jobW.Type == model.ActionTruncateTable { + if ddlErr != nil { + newTableID := jobW.JobArgs.(*model.TruncateTableArgs).NewTableID + ctx.ReleaseTableLockByTableIDs([]int64{newTableID}) + return + } + if ok, _ := ctx.CheckTableLocked(jobW.TableID); ok { + ctx.ReleaseTableLockByTableIDs([]int64{jobW.TableID}) + } + } +} + +func (e *executor) getJobDoneCh(jobID int64) (chan struct{}, bool) { + return e.ddlJobDoneChMap.Load(jobID) +} + +func (e *executor) delJobDoneCh(jobID int64) { + e.ddlJobDoneChMap.Delete(jobID) +} + +func (e *executor) deliverJobTask(task *JobWrapper) { + // TODO this might block forever, as the consumer part considers context cancel. + e.limitJobCh <- task +} + +func updateTickerInterval(ticker *time.Ticker, lease time.Duration, action model.ActionType, i int) *time.Ticker { + interval, changed := getJobCheckInterval(action, i) + if !changed { + return ticker + } + // For now we should stop old ticker and create a new ticker + ticker.Stop() + return time.NewTicker(chooseLeaseTime(lease, interval)) +} + +func recordLastDDLInfo(ctx sessionctx.Context, job *model.Job) { + if job == nil { + return + } + ctx.GetSessionVars().LastDDLInfo.Query = job.Query + ctx.GetSessionVars().LastDDLInfo.SeqNum = job.SeqNum +} + +func setDDLJobQuery(ctx sessionctx.Context, job *model.Job) { + switch job.Type { + case model.ActionUpdateTiFlashReplicaStatus, model.ActionUnlockTable: + job.Query = "" + default: + job.Query, _ = ctx.Value(sessionctx.QueryString).(string) + } +} + +var ( + fastDDLIntervalPolicy = []time.Duration{ + 500 * time.Millisecond, + } + normalDDLIntervalPolicy = []time.Duration{ + 500 * time.Millisecond, + 500 * time.Millisecond, + 1 * time.Second, + } + slowDDLIntervalPolicy = []time.Duration{ + 500 * time.Millisecond, + 500 * time.Millisecond, + 1 * time.Second, + 1 * time.Second, + 3 * time.Second, + } +) + +func getIntervalFromPolicy(policy []time.Duration, i int) (time.Duration, bool) { + plen := len(policy) + if i < plen { + return policy[i], true + } + return policy[plen-1], false +} + +func getJobCheckInterval(action model.ActionType, i int) (time.Duration, bool) { + switch action { + case model.ActionAddIndex, model.ActionAddPrimaryKey, model.ActionModifyColumn, + model.ActionReorganizePartition, + model.ActionRemovePartitioning, + model.ActionAlterTablePartitioning: + return getIntervalFromPolicy(slowDDLIntervalPolicy, i) + case model.ActionCreateTable, model.ActionCreateSchema: + return getIntervalFromPolicy(fastDDLIntervalPolicy, i) + default: + return getIntervalFromPolicy(normalDDLIntervalPolicy, i) + } +} + +// NewDDLReorgMeta create a DDL ReorgMeta. +func NewDDLReorgMeta(ctx sessionctx.Context) *model.DDLReorgMeta { + tzName, tzOffset := ddlutil.GetTimeZone(ctx) + return &model.DDLReorgMeta{ + SQLMode: ctx.GetSessionVars().SQLMode, + Warnings: make(map[errors.ErrorID]*terror.Error), + WarningsCount: make(map[errors.ErrorID]int64), + Location: &model.TimeZoneLocation{Name: tzName, Offset: tzOffset}, + ResourceGroupName: ctx.GetSessionVars().StmtCtx.ResourceGroupName, + Version: model.CurrentReorgMetaVersion, + } +} diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index f0f13b7e39236..ebbd3bc6137f2 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -95,7 +95,11 @@ func checkAddPartition(t *meta.Meta, job *model.Job) (*model.TableInfo, *model.P func (w *worker) onAddTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { // Handle the rolling back job if job.IsRollingback() { +<<<<<<< HEAD ver, err := w.onDropTablePartition(d, t, job) +======= + ver, err := w.rollbackLikeDropPartition(jobCtx, job) +>>>>>>> 1e24d396082 (*: Drop partition DDL handling for overlapping partitions during State Changes (#56082)) if err != nil { return ver, errors.Trace(err) } @@ -341,20 +345,6 @@ func rollbackAddingPartitionInfo(tblInfo *model.TableInfo) ([]int64, []string, [ return physicalTableIDs, partNames, rollbackBundles } -// Check if current table already contains DEFAULT list partition -func checkAddListPartitions(tblInfo *model.TableInfo) error { - for i := range tblInfo.Partition.Definitions { - for j := range tblInfo.Partition.Definitions[i].InValues { - for _, val := range tblInfo.Partition.Definitions[i].InValues[j] { - if val == "DEFAULT" { // should already be normalized - return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("ADD List partition, already contains DEFAULT partition. Please use REORGANIZE PARTITION instead") - } - } - } - } - return nil -} - // checkAddPartitionValue check add Partition Values, // For Range: values less than value must be strictly increasing for each partition. // For List: if a Default partition exists, @@ -394,10 +384,16 @@ func checkAddPartitionValue(meta *model.TableInfo, part *model.PartitionInfo) er currentRangeValue = nextRangeValue } } +<<<<<<< HEAD case model.PartitionTypeList: err := checkAddListPartitions(meta) if err != nil { return err +======= + case pmodel.PartitionTypeList: + if meta.Partition.GetDefaultListPartition() != -1 { + return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("ADD List partition, already contains DEFAULT partition. Please use REORGANIZE PARTITION instead") +>>>>>>> 1e24d396082 (*: Drop partition DDL handling for overlapping partitions during State Changes (#56082)) } } return nil @@ -1951,7 +1947,79 @@ func dropLabelRules(_ *ddlCtx, schemaName, tableName string, partNames []string) return infosync.UpdateLabelRules(context.TODO(), patch) } +// rollbackLikeDropPartition does rollback for Reorganize partition and Add partition. +// It will drop newly created partitions that has not yet been used, including cleaning +// up label rules and bundles as well as changed indexes due to global flag. +func (w *worker) rollbackLikeDropPartition(jobCtx *jobContext, job *model.Job) (ver int64, _ error) { + args, err := model.GetTablePartitionArgs(job) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + partInfo := args.PartInfo + metaMut := jobCtx.metaMut + tblInfo, err := GetTableInfoAndCancelFaultJob(metaMut, job, job.SchemaID) + if err != nil { + return ver, errors.Trace(err) + } + tblInfo.Partition.DroppingDefinitions = nil + physicalTableIDs, pNames, rollbackBundles := rollbackAddingPartitionInfo(tblInfo) + err = infosync.PutRuleBundlesWithDefaultRetry(context.TODO(), rollbackBundles) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Wrapf(err, "failed to notify PD the placement rules") + } + // TODO: Will this drop LabelRules for existing partitions, if the new partitions have the same name? + err = dropLabelRules(w.ctx, job.SchemaName, tblInfo.Name.L, pNames) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Wrapf(err, "failed to notify PD the label rules") + } + + if _, err := alterTableLabelRule(job.SchemaName, tblInfo, getIDs([]*model.TableInfo{tblInfo})); err != nil { + job.State = model.JobStateCancelled + return ver, err + } + if partInfo.Type != pmodel.PartitionTypeNone { + // ALTER TABLE ... PARTITION BY + // Also remove anything with the new table id + physicalTableIDs = append(physicalTableIDs, partInfo.NewTableID) + // Reset if it was normal table before + if tblInfo.Partition.Type == pmodel.PartitionTypeNone || + tblInfo.Partition.DDLType == pmodel.PartitionTypeNone { + tblInfo.Partition = nil + } + } + + var dropIndices []*model.IndexInfo + for _, indexInfo := range tblInfo.Indices { + if indexInfo.Unique && + indexInfo.State == model.StateDeleteReorganization && + tblInfo.Partition.DDLState == model.StateDeleteReorganization { + dropIndices = append(dropIndices, indexInfo) + } + } + for _, indexInfo := range dropIndices { + DropIndexColumnFlag(tblInfo, indexInfo) + RemoveDependentHiddenColumns(tblInfo, indexInfo) + removeIndexInfo(tblInfo, indexInfo) + } + if tblInfo.Partition != nil { + tblInfo.Partition.ClearReorgIntermediateInfo() + } + + ver, err = updateVersionAndTableInfo(jobCtx, job, tblInfo, true) + if err != nil { + return ver, errors.Trace(err) + } + job.FinishTableJob(model.JobStateRollbackDone, model.StateNone, ver, tblInfo) + args.OldPhysicalTblIDs = physicalTableIDs + job.FillFinishedArgs(args) + return ver, nil +} + // onDropTablePartition deletes old partition meta. +<<<<<<< HEAD func (w *worker) onDropTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { var partNames []string partInfo := model.PartitionInfo{} @@ -2009,6 +2077,49 @@ func (w *worker) onDropTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) ( job.Args = []interface{}{physicalTableIDs} return ver, nil } +======= +// States: +// StateNone +// +// Old partitions are queued to be deleted (delete_range), global index up-to-date +// +// StateDeleteReorganization +// +// Old partitions are not accessible/used by any sessions. +// Inserts/updates of global index which still have entries pointing to old partitions +// will overwrite those entries +// In the background we are reading all old partitions and deleting their entries from +// the global indexes. +// +// StateDeleteOnly +// +// old partitions are no longer visible, but if there is inserts/updates to the global indexes, +// duplicate key errors will be given, even if the entries are from dropped partitions +// Note that overlapping ranges (i.e. a dropped partitions with 'less than (N)' will now .. ?!? +// +// StateWriteOnly +// +// old partitions are blocked for read and write. But for read we are allowing +// "overlapping" partition to be read instead. Which means that write can only +// happen in the 'overlapping' partitions original range, not into the extended +// range open by the dropped partitions. +// +// StatePublic +// +// Original state, unaware of DDL +func (w *worker) onDropTablePartition(jobCtx *jobContext, job *model.Job) (ver int64, _ error) { + args, err := model.GetTablePartitionArgs(job) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + partNames := args.PartNames + metaMut := jobCtx.metaMut + tblInfo, err := GetTableInfoAndCancelFaultJob(metaMut, job, job.SchemaID) + if err != nil { + return ver, errors.Trace(err) + } +>>>>>>> 1e24d396082 (*: Drop partition DDL handling for overlapping partitions during State Changes (#56082)) var physicalTableIDs []int64 // In order to skip maintaining the state check in partitionDefinition, TiDB use droppingDefinition instead of state field. @@ -2016,15 +2127,30 @@ func (w *worker) onDropTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) ( originalState := job.SchemaState switch job.SchemaState { case model.StatePublic: - // If an error occurs, it returns that it cannot delete all partitions or that the partition doesn't exist. + // Here we mark the partitions to be dropped, so they are not read or written err = CheckDropTablePartition(tblInfo, partNames) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) } + // Reason, see https://github.com/pingcap/tidb/issues/55888 + // Only mark the partitions as to be dropped, so they are not used, but not yet removed. + originalDefs := tblInfo.Partition.Definitions + physicalTableIDs = updateDroppingPartitionInfo(tblInfo, partNames) + tblInfo.Partition.Definitions = originalDefs + tblInfo.Partition.DDLState = model.StateWriteOnly + tblInfo.Partition.DDLAction = model.ActionDropTablePartition + + job.SchemaState = model.StateWriteOnly + ver, err = updateVersionAndTableInfo(jobCtx, job, tblInfo, originalState != job.SchemaState) + case model.StateWriteOnly: + // Since the previous state do not use the dropping partitions, + // we can now actually remove them, allowing to write into the overlapping range + // of the higher range partition or LIST default partition. physicalTableIDs = updateDroppingPartitionInfo(tblInfo, partNames) err = dropLabelRules(d, job.SchemaName, tblInfo.Name.L, partNames) if err != nil { + // TODO: Add failpoint error/cancel injection and test failure/rollback and cancellation! job.State = model.JobStateCancelled return ver, errors.Wrapf(err, "failed to notify PD the label rules") } @@ -2060,12 +2186,14 @@ func (w *worker) onDropTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) ( return ver, err } + tblInfo.Partition.DDLState = model.StateDeleteOnly job.SchemaState = model.StateDeleteOnly ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, originalState != job.SchemaState) case model.StateDeleteOnly: - // This state is not a real 'DeleteOnly' state, because tidb does not maintaining the state check in partitionDefinition. + // This state is not a real 'DeleteOnly' state, because tidb does not maintain the state check in partitionDefinition. // Insert this state to confirm all servers can not see the old partitions when reorg is running, // so that no new data will be inserted into old partitions when reorganizing. + tblInfo.Partition.DDLState = model.StateDeleteReorganization job.SchemaState = model.StateDeleteReorganization ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, originalState != job.SchemaState) case model.StateDeleteReorganization: @@ -2125,6 +2253,8 @@ func (w *worker) onDropTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) ( } droppedDefs := tblInfo.Partition.DroppingDefinitions tblInfo.Partition.DroppingDefinitions = nil + tblInfo.Partition.DDLState = model.StateNone + tblInfo.Partition.DDLAction = model.ActionNone // used by ApplyDiff in updateSchemaVersion job.CtxVars = []interface{}{physicalTableIDs} ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, true) @@ -2245,6 +2375,7 @@ func (w *worker) onTruncateTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo pi.DroppingDefinitions = truncatingDefinitions pi.NewPartitionIDs = newIDs[:] + tblInfo.Partition.DDLAction = model.ActionTruncateTablePartition job.SchemaState = model.StateDeleteOnly ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, true) case model.StateDeleteOnly: @@ -2763,7 +2894,11 @@ func getReorgPartitionInfo(t *meta.Meta, job *model.Job) (*model.TableInfo, []st func (w *worker) onReorganizePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { // Handle the rolling back job if job.IsRollingback() { +<<<<<<< HEAD ver, err := w.onDropTablePartition(d, t, job) +======= + ver, err := w.rollbackLikeDropPartition(jobCtx, job) +>>>>>>> 1e24d396082 (*: Drop partition DDL handling for overlapping partitions during State Changes (#56082)) if err != nil { return ver, errors.Trace(err) } @@ -2882,7 +3017,12 @@ func (w *worker) onReorganizePartition(d *ddlCtx, t *meta.Meta, job *model.Job) metrics.GetBackfillProgressByLabel(metrics.LblReorgPartition, job.SchemaName, tblInfo.Name.String()).Set(0.1 / float64(math.MaxUint64)) job.SchemaState = model.StateDeleteOnly tblInfo.Partition.DDLState = model.StateDeleteOnly +<<<<<<< HEAD ver, err = updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, true) +======= + tblInfo.Partition.DDLAction = job.Type + ver, err = updateVersionAndTableInfoWithCheck(jobCtx, job, tblInfo, true) +>>>>>>> 1e24d396082 (*: Drop partition DDL handling for overlapping partitions during State Changes (#56082)) if err != nil { return ver, errors.Trace(err) } diff --git a/pkg/ddl/rollingback.go b/pkg/ddl/rollingback.go index bcfbb7ffb413a..fb9bd8a2e286a 100644 --- a/pkg/ddl/rollingback.go +++ b/pkg/ddl/rollingback.go @@ -339,6 +339,7 @@ func convertAddTablePartitionJob2RollbackJob(d *ddlCtx, t *meta.Meta, job *model for _, pd := range addingDefinitions { partNames = append(partNames, pd.Name.L) } +<<<<<<< HEAD if job.Type == model.ActionReorganizePartition || job.Type == model.ActionAlterTablePartitioning || job.Type == model.ActionRemovePartitioning { @@ -353,15 +354,172 @@ func convertAddTablePartitionJob2RollbackJob(d *ddlCtx, t *meta.Meta, job *model job.Args = []any{partNames} } ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, true) +======= + args, err := model.GetTablePartitionArgs(job) if err != nil { return ver, errors.Trace(err) } + args.PartNames = partNames + model.FillRollbackArgsForAddPartition(job, args) + ver, err = updateVersionAndTableInfo(jobCtx, job, tblInfo, true) +>>>>>>> 1e24d396082 (*: Drop partition DDL handling for overlapping partitions during State Changes (#56082)) + if err != nil { + return ver, errors.Trace(err) + } + tblInfo.Partition.DDLState = model.StateNone + tblInfo.Partition.DDLAction = model.ActionNone job.State = model.JobStateRollingback return ver, errors.Trace(otherwiseErr) } +<<<<<<< HEAD func rollingbackAddTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { tblInfo, _, addingDefinitions, err := checkAddPartition(t, job) +======= +func rollbackReorganizePartitionWithErr(jobCtx *jobContext, job *model.Job, otherwiseErr error) (ver int64, err error) { + if job.SchemaState == model.StateNone { + job.State = model.JobStateCancelled + return ver, otherwiseErr + } + + tblInfo, err := GetTableInfoAndCancelFaultJob(jobCtx.metaMut, job, job.SchemaID) + if err != nil { + return ver, errors.Trace(err) + } + + // addingDefinitions is also in tblInfo, here pass the tblInfo as parameter directly. + return convertReorgPartitionJob2RollbackJob(jobCtx, job, otherwiseErr, tblInfo) +} + +func convertReorgPartitionJob2RollbackJob(jobCtx *jobContext, job *model.Job, otherwiseErr error, tblInfo *model.TableInfo) (ver int64, err error) { + pi := tblInfo.Partition + addingDefinitions := pi.AddingDefinitions + partNames := make([]string, 0, len(addingDefinitions)) + for _, pd := range addingDefinitions { + partNames = append(partNames, pd.Name.L) + } + var dropIndices []*model.IndexInfo + // When Global Index is duplicated to a non Global, we later need + // to know if if it was Global before (marked to be dropped) or not. + globalToUniqueDupMap := make(map[string]int64) + for _, indexInfo := range tblInfo.Indices { + if !indexInfo.Unique { + continue + } + switch indexInfo.State { + case model.StateWriteReorganization, model.StateDeleteOnly, + model.StateWriteOnly: + dropIndices = append(dropIndices, indexInfo) + case model.StateDeleteReorganization: + if pi.DDLState != model.StateDeleteReorganization { + continue + } + // Old index marked to be dropped, rollback by making it public again + indexInfo.State = model.StatePublic + if indexInfo.Global { + if id, ok := globalToUniqueDupMap[indexInfo.Name.L]; ok { + return ver, errors.NewNoStackErrorf("Duplicate global index names '%s', %d != %d", indexInfo.Name.O, indexInfo.ID, id) + } + globalToUniqueDupMap[indexInfo.Name.L] = indexInfo.ID + } + case model.StatePublic: + if pi.DDLState != model.StateDeleteReorganization { + continue + } + // We cannot drop the index here, we need to wait until + // the next schema version + // i.e. rollback in rollbackLikeDropPartition + // New index that became public in this state, + // mark it to be dropped in next schema version + if indexInfo.Global { + indexInfo.State = model.StateDeleteReorganization + } else { + // How to know if this index was created as a duplicate or not? + if id, ok := globalToUniqueDupMap[indexInfo.Name.L]; ok { + // The original index + if id >= indexInfo.ID { + return ver, errors.NewNoStackErrorf("Indexes in wrong order during rollback, '%s', %d >= %d", indexInfo.Name.O, id, indexInfo.ID) + } + indexInfo.State = model.StateDeleteReorganization + } else { + globalToUniqueDupMap[indexInfo.Name.L] = indexInfo.ID + } + } + } + } + for _, indexInfo := range dropIndices { + DropIndexColumnFlag(tblInfo, indexInfo) + RemoveDependentHiddenColumns(tblInfo, indexInfo) + removeIndexInfo(tblInfo, indexInfo) + } + if pi.DDLState == model.StateDeleteReorganization { + // New partitions are public, + // but old is still double written. + // OK to revert. + // Remove the AddingDefinitions + // Add back the DroppingDefinitions + if job.Type == model.ActionReorganizePartition { + // Reassemble the list of partitions in the OriginalPartitionIDsOrder + // Special handling, since for LIST partitioning, + // only pi.OriginalPartitionIDsOrder shows how to merge back the DroppingDefinitions. + // Implicitly it will also filter away AddingPartitions. + // pi.Definitions and pi.DroppingDefinitions contain the original partitions + // in the original order, but where the DroppingDefinitions should be placed, + // can only be known through pi.OriginalPartitionIDsOrder. + // RANGE/HASH/KEY would have consecutive added/dropped partitions, but use + // the same code to avoid confusion. + defPos := 0 + dropPos := 0 + newDefs := make([]model.PartitionDefinition, 0, len(pi.OriginalPartitionIDsOrder)) + for _, id := range pi.OriginalPartitionIDsOrder { + if defPos < len(pi.Definitions) && pi.Definitions[defPos].ID == id { + newDefs = append(newDefs, pi.Definitions[defPos]) + defPos++ + continue + } + if dropPos < len(pi.DroppingDefinitions) && id == pi.DroppingDefinitions[dropPos].ID { + newDefs = append(newDefs, pi.DroppingDefinitions[dropPos]) + dropPos++ + continue + } + for { + defPos++ + if defPos < len(pi.Definitions) && pi.Definitions[defPos].ID == id { + newDefs = append(newDefs, pi.Definitions[defPos]) + break + } + } + } + if len(newDefs) != len(pi.OriginalPartitionIDsOrder) { + return ver, errors.Trace(errors.New("Internal error, failed to find original partition definitions")) + } + pi.Definitions = newDefs + pi.Num = uint64(len(pi.Definitions)) + } else { + pi.Type, pi.DDLType = pi.DDLType, pi.Type + pi.Expr, pi.DDLExpr = pi.DDLExpr, pi.Expr + pi.Columns, pi.DDLColumns = pi.DDLColumns, pi.Columns + pi.Definitions = pi.DroppingDefinitions + } + } + + args, err := model.GetTablePartitionArgs(job) + if err != nil { + return ver, errors.Trace(err) + } + args.PartNames = partNames + job.FillArgs(args) + ver, err = updateVersionAndTableInfo(jobCtx, job, tblInfo, true) + if err != nil { + return ver, errors.Trace(err) + } + job.State = model.JobStateRollingback + return ver, errors.Trace(otherwiseErr) +} + +func rollingbackAddTablePartition(jobCtx *jobContext, job *model.Job) (ver int64, err error) { + tblInfo, _, addingDefinitions, err := checkAddPartition(jobCtx.metaMut, job) +>>>>>>> 1e24d396082 (*: Drop partition DDL handling for overlapping partitions during State Changes (#56082)) if err != nil { return ver, errors.Trace(err) } diff --git a/pkg/ddl/tests/partition/BUILD.bazel b/pkg/ddl/tests/partition/BUILD.bazel index 1183232ee444d..45e9b73d6de20 100644 --- a/pkg/ddl/tests/partition/BUILD.bazel +++ b/pkg/ddl/tests/partition/BUILD.bazel @@ -6,6 +6,12 @@ go_test( srcs = [ "db_partition_test.go", "main_test.go", +<<<<<<< HEAD +======= + "multi_domain_test.go", + "placement_test.go", + "reorg_partition_test.go", +>>>>>>> 1e24d396082 (*: Drop partition DDL handling for overlapping partitions during State Changes (#56082)) ], flaky = True, shard_count = 49, @@ -36,6 +42,10 @@ go_test( "//pkg/util/codec", "//pkg/util/dbterror", "//pkg/util/logutil", +<<<<<<< HEAD +======= + "//pkg/util/mathutil", +>>>>>>> 1e24d396082 (*: Drop partition DDL handling for overlapping partitions during State Changes (#56082)) "@com_github_pingcap_errors//:errors", "@com_github_pingcap_failpoint//:failpoint", "@com_github_stretchr_testify//assert", diff --git a/pkg/ddl/tests/partition/multi_domain_test.go b/pkg/ddl/tests/partition/multi_domain_test.go new file mode 100644 index 0000000000000..1c567c9a4afaf --- /dev/null +++ b/pkg/ddl/tests/partition/multi_domain_test.go @@ -0,0 +1,342 @@ +// Copyright 2024 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 partition + +import ( + "testing" + "time" + + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/session" + "github.com/pingcap/tidb/pkg/testkit" + "github.com/pingcap/tidb/pkg/util/logutil" + "github.com/stretchr/testify/require" + "go.uber.org/zap" +) + +func TestMultiSchemaDropRangePartition(t *testing.T) { + createSQL := `create table t (a int primary key, b varchar(255)) partition by range (a) (partition p0 values less than (100), partition p1 values less than (200))` + initFn := func(tkO *testkit.TestKit) { + tkO.MustExec(`insert into t values (1,1),(2,2),(101,101),(102,102)`) + } + alterSQL := `alter table t drop partition p0` + loopFn := func(tkO, tkNO *testkit.TestKit) { + res := tkO.MustQuery(`select schema_state from information_schema.DDL_JOBS where table_name = 't' order by job_id desc limit 1`) + schemaState := res.Rows()[0][0].(string) + // TODO: Test both static and dynamic partition pruning! + switch schemaState { + case "write only": + // tkNO are unaware of the DDL + // tkO see non-readable/non-writable p0 partition, and should try to read from p1 + // in case there is something written to overlapping p1 + tkO.MustContainErrMsg(`insert into t values (1,1)`, "[table:1526]Table has no partition for value matching a partition being dropped, 'p0'") + tkNO.MustContainErrMsg(`insert into t values (1,1)`, "[kv:1062]Duplicate entry '1' for key 't.PRIMARY'") + tkO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.PRIMARY'") + tkNO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.PRIMARY'") + tkNO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 1", "101 101", "102 102", "2 2")) + tkO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("101 101", "102 102")) + case "delete only": + // tkNO see non-readable/non-writable p0 partition, and should try to read from p1 + // in case there is something written to overlapping p1 + // tkO is not aware of p0. + tkO.MustExec(`insert into t values (1,2)`) + tkNO.MustContainErrMsg(`insert into t values (1,2)`, "[table:1526]Table has no partition for value matching a partition being dropped, 'p0'") + tkO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.PRIMARY'") + tkNO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.PRIMARY'") + tkNO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 2", "101 101", "102 102")) + // Original row should not be seen in StateWriteOnly + tkNO.MustQuery(`select * from t partition (p0)`).Sort().Check(testkit.Rows()) + tkNO.MustContainErrMsg(`select * from t partition (pNonExisting)`, "[table:1735]Unknown partition 'pnonexisting' in table 't'") + tkNO.MustQuery(`select * from t partition (p1)`).Sort().Check(testkit.Rows("1 2", "101 101", "102 102")) + tkNO.MustQuery(`select * from t where a < 1000`).Sort().Check(testkit.Rows("1 2", "101 101", "102 102")) + tkNO.MustQuery(`select * from t where a > 0`).Sort().Check(testkit.Rows("1 2", "101 101", "102 102")) + tkNO.MustQuery(`select * from t where a = 1`).Sort().Check(testkit.Rows("1 2")) + tkNO.MustQuery(`select * from t where a = 1 or a = 2 or a = 3`).Sort().Check(testkit.Rows("1 2")) + tkNO.MustQuery(`select * from t where a in (1,2,3)`).Sort().Check(testkit.Rows("1 2")) + tkNO.MustQuery(`select * from t where a < 100`).Sort().Check(testkit.Rows("1 2")) + + tkNO.MustQuery(`select * from t where b = 2`).Sort().Check(testkit.Rows("1 2")) + // TODO: Test update and delete! + // TODO: test key, hash and list partition without default partition :) + tkNO.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE (`a`)\n" + + "(PARTITION `p0` VALUES LESS THAN (100),\n" + + " PARTITION `p1` VALUES LESS THAN (200))")) + tkO.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE (`a`)\n" + + "(PARTITION `p1` VALUES LESS THAN (200))")) + case "delete reorganization": + // just to not fail :) + case "none": + // just to not fail :) + default: + require.Failf(t, "unhandled schema state '%s'", schemaState) + } + } + runMultiSchemaTest(t, createSQL, alterSQL, initFn, func(kit *testkit.TestKit) {}, loopFn) +} + +func TestMultiSchemaDropListDefaultPartition(t *testing.T) { + createSQL := `create table t (a int primary key, b varchar(255)) partition by list (a) (partition p0 values in (1,2,3), partition p1 values in (100,101,102,DEFAULT))` + initFn := func(tkO *testkit.TestKit) { + tkO.MustExec(`insert into t values (1,1),(2,2),(101,101),(102,102)`) + } + alterSQL := `alter table t drop partition p0` + loopFn := func(tkO, tkNO *testkit.TestKit) { + res := tkO.MustQuery(`select schema_state from information_schema.DDL_JOBS where table_name = 't' order by job_id desc limit 1`) + schemaState := res.Rows()[0][0].(string) + // TODO: Test both static and dynamic partition pruning! + switch schemaState { + case "write only": + // tkNO are unaware of the DDL + // tkO see non-readable/non-writable p0 partition, and should try to read from p1 + // in case there is something written to overlapping p1 + tkO.MustContainErrMsg(`insert into t values (1,1)`, "[table:1526]Table has no partition for value matching a partition being dropped, 'p0'") + tkNO.MustContainErrMsg(`insert into t values (1,1)`, "[kv:1062]Duplicate entry '1' for key 't.PRIMARY'") + tkO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.PRIMARY'") + tkNO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.PRIMARY'") + tkNO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 1", "101 101", "102 102", "2 2")) + tkO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("101 101", "102 102")) + case "delete only": + // tkNO see non-readable/non-writable p0 partition, and should try to read from p1 + // in case there is something written to overlapping p1 + // tkO is not aware of p0. + tkO.MustExec(`insert into t values (1,2)`) + tkNO.MustContainErrMsg(`insert into t values (1,2)`, "[table:1526]Table has no partition for value matching a partition being dropped, 'p0'") + tkO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.PRIMARY'") + tkNO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.PRIMARY'") + tkNO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 2", "101 101", "102 102")) + // Original row should not be seen in StateWriteOnly + tkNO.MustQuery(`select * from t partition (p0)`).Sort().Check(testkit.Rows()) + tkNO.MustContainErrMsg(`select * from t partition (pNonExisting)`, "[table:1735]Unknown partition 'pnonexisting' in table 't'") + tkNO.MustQuery(`select * from t partition (p1)`).Sort().Check(testkit.Rows("1 2", "101 101", "102 102")) + tkNO.MustQuery(`select * from t where a < 1000`).Sort().Check(testkit.Rows("1 2", "101 101", "102 102")) + tkNO.MustQuery(`select * from t where a > 0`).Sort().Check(testkit.Rows("1 2", "101 101", "102 102")) + tkNO.MustQuery(`select * from t where a = 1`).Sort().Check(testkit.Rows("1 2")) + tkNO.MustQuery(`select * from t where a = 1 or a = 2 or a = 3`).Sort().Check(testkit.Rows("1 2")) + tkNO.MustQuery(`select * from t where a in (1,2,3)`).Sort().Check(testkit.Rows("1 2")) + tkNO.MustQuery(`select * from t where a < 100`).Sort().Check(testkit.Rows("1 2")) + + tkNO.MustQuery(`select * from t where b = 2`).Sort().Check(testkit.Rows("1 2")) + // TODO: Test update and delete! + // TODO: test key, hash and list partition without default partition :) + // Should we see the partition or not?!? + tkNO.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY LIST (`a`)\n" + + "(PARTITION `p0` VALUES IN (1,2,3),\n" + + " PARTITION `p1` VALUES IN (100,101,102,DEFAULT))")) + tkO.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY LIST (`a`)\n" + + "(PARTITION `p1` VALUES IN (100,101,102,DEFAULT))")) + case "delete reorganization": + // just to not fail :) + case "none": + // just to not fail :) + default: + require.Failf(t, "unhandled schema state '%s'", schemaState) + } + } + runMultiSchemaTest(t, createSQL, alterSQL, initFn, func(kit *testkit.TestKit) {}, loopFn) +} + +func TestMultiSchemaDropListColumnsDefaultPartition(t *testing.T) { + createSQL := `create table t (a int, b varchar(255), c varchar (255), primary key (a,b)) partition by list columns (a,b) (partition p0 values in ((1,"1"),(2,"2"),(3,"3")), partition p1 values in ((100,"100"),(101,"101"),(102,"102"),DEFAULT))` + initFn := func(tkO *testkit.TestKit) { + tkO.MustExec(`insert into t values (1,1,1),(2,2,2),(101,101,101),(102,102,102)`) + } + alterSQL := `alter table t drop partition p0` + loopFn := func(tkO, tkNO *testkit.TestKit) { + res := tkO.MustQuery(`select schema_state from information_schema.DDL_JOBS where table_name = 't' order by job_id desc limit 1`) + schemaState := res.Rows()[0][0].(string) + // TODO: Test both static and dynamic partition pruning! + switch schemaState { + case "write only": + // tkNO are unaware of the DDL + // tkO see non-readable/non-writable p0 partition, and should try to read from p1 + // in case there is something written to overlapping p1 + tkO.MustContainErrMsg(`insert into t values (1,1,1)`, "[table:1526]Table has no partition for value matching a partition being dropped, 'p0'") + tkNO.MustContainErrMsg(`insert into t values (1,1,1)`, "[kv:1062]Duplicate entry '1-1' for key 't.PRIMARY'") + tkO.MustContainErrMsg(`insert into t values (101,101,101)`, "[kv:1062]Duplicate entry '101-101' for key 't.PRIMARY'") + tkNO.MustContainErrMsg(`insert into t values (101,101,101)`, "[kv:1062]Duplicate entry '101-101' for key 't.PRIMARY'") + tkNO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 1 1", "101 101 101", "102 102 102", "2 2 2")) + tkO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("101 101 101", "102 102 102")) + case "delete only": + // tkNO see non-readable/non-writable p0 partition, and should try to read from p1 + // in case there is something written to overlapping p1 + // tkO is not aware of p0. + tkO.MustExec(`insert into t values (1,1,2)`) + tkNO.MustContainErrMsg(`insert into t values (1,1,2)`, "[table:1526]Table has no partition for value matching a partition being dropped, 'p0'") + tkO.MustContainErrMsg(`insert into t values (101,101,101)`, "[kv:1062]Duplicate entry '101-101' for key 't.PRIMARY'") + tkNO.MustContainErrMsg(`insert into t values (101,101,101)`, "[kv:1062]Duplicate entry '101-101' for key 't.PRIMARY'") + tkNO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 1 2", "101 101 101", "102 102 102")) + // Original row should not be seen in StateWriteOnly + tkNO.MustQuery(`select * from t partition (p0)`).Sort().Check(testkit.Rows()) + tkNO.MustContainErrMsg(`select * from t partition (pNonExisting)`, "[table:1735]Unknown partition 'pnonexisting' in table 't'") + tkNO.MustQuery(`select * from t partition (p1)`).Sort().Check(testkit.Rows("1 1 2", "101 101 101", "102 102 102")) + tkNO.MustQuery(`select * from t where a < 1000`).Sort().Check(testkit.Rows("1 1 2", "101 101 101", "102 102 102")) + tkNO.MustQuery(`select * from t where a > 0`).Sort().Check(testkit.Rows("1 1 2", "101 101 101", "102 102 102")) + tkNO.MustQuery(`select * from t where a = 1`).Sort().Check(testkit.Rows("1 1 2")) + tkNO.MustQuery(`select * from t where a = 1 or a = 2 or a = 3`).Sort().Check(testkit.Rows("1 1 2")) + tkNO.MustQuery(`select * from t where a in (1,2,3) or b in ("1","2")`).Sort().Check(testkit.Rows("1 1 2")) + tkNO.MustQuery(`select * from t where a in (1,2,3)`).Sort().Check(testkit.Rows("1 1 2")) + tkNO.MustQuery(`select * from t where a < 100`).Sort().Check(testkit.Rows("1 1 2")) + + tkNO.MustQuery(`select * from t where c = "2"`).Sort().Check(testkit.Rows("1 1 2")) + tkNO.MustQuery(`select * from t where b = "1"`).Sort().Check(testkit.Rows("1 1 2")) + // TODO: Test update and delete! + // TODO: test key, hash and list partition without default partition :) + // Should we see the partition or not?!? + tkNO.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) NOT NULL,\n" + + " `c` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`,`b`) /*T![clustered_index] CLUSTERED */\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY LIST COLUMNS(`a`,`b`)\n" + + "(PARTITION `p0` VALUES IN ((1,'1'),(2,'2'),(3,'3')),\n" + + " PARTITION `p1` VALUES IN ((100,'100'),(101,'101'),(102,'102'),DEFAULT))")) + tkO.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) NOT NULL,\n" + + " `c` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`,`b`) /*T![clustered_index] CLUSTERED */\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY LIST COLUMNS(`a`,`b`)\n" + + "(PARTITION `p1` VALUES IN ((100,'100'),(101,'101'),(102,'102'),DEFAULT))")) + case "delete reorganization": + // just to not fail :) + case "none": + // just to not fail :) + default: + require.Failf(t, "unhandled schema state '%s'", schemaState) + } + } + runMultiSchemaTest(t, createSQL, alterSQL, initFn, func(kit *testkit.TestKit) {}, loopFn) +} + +func runMultiSchemaTest(t *testing.T, createSQL, alterSQL string, initFn, postFn func(*testkit.TestKit), loopFn func(tO, tNO *testkit.TestKit)) { + distCtx := testkit.NewDistExecutionContextWithLease(t, 2, 15*time.Second) + store := distCtx.Store + domOwner := distCtx.GetDomain(0) + domNonOwner := distCtx.GetDomain(1) + defer func() { + domOwner.Close() + domNonOwner.Close() + store.Close() + }() + + if !domOwner.DDL().OwnerManager().IsOwner() { + domOwner, domNonOwner = domNonOwner, domOwner + } + + seOwner, err := session.CreateSessionWithDomain(store, domOwner) + require.NoError(t, err) + seNonOwner, err := session.CreateSessionWithDomain(store, domNonOwner) + require.NoError(t, err) + + tkDDLOwner := testkit.NewTestKitWithSession(t, store, seOwner) + tkDDLOwner.MustExec(`use test`) + tkDDLOwner.MustExec(`set @@global.tidb_enable_global_index = 1`) + tkDDLOwner.MustExec(`set @@session.tidb_enable_global_index = 1`) + tkO := testkit.NewTestKitWithSession(t, store, seOwner) + tkO.MustExec(`use test`) + tkNO := testkit.NewTestKitWithSession(t, store, seNonOwner) + tkNO.MustExec(`use test`) + + tkDDLOwner.MustExec(createSQL) + domOwner.Reload() + domNonOwner.Reload() + initFn(tkO) + verStart := domNonOwner.InfoSchema().SchemaMetaVersion() + hookChan := make(chan struct{}) + hookFunc := func(job *model.Job) { + hookChan <- struct{}{} + logutil.BgLogger().Info("XXXXXXXXXXX Hook now waiting", zap.String("job.State", job.State.String()), zap.String("job.SchemaStage", job.SchemaState.String())) + <-hookChan + logutil.BgLogger().Info("XXXXXXXXXXX Hook released", zap.String("job.State", job.State.String()), zap.String("job.SchemaStage", job.SchemaState.String())) + } + failpoint.EnableCall("github.com/pingcap/tidb/pkg/ddl/onJobRunAfter", hookFunc) + defer failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/onJobRunAfter") + alterChan := make(chan struct{}) + go func() { + tkDDLOwner.MustExec(alterSQL) + logutil.BgLogger().Info("XXXXXXXXXXX drop partition done!") + alterChan <- struct{}{} + }() + // Skip the first state, since we want to compare before vs after in the loop + <-hookChan + hookChan <- struct{}{} + verCurr := verStart + 1 + i := 0 + for { + // Waiting for the next State change to be done (i.e. blocking the state after) + releaseHook := true + for { + select { + case <-hookChan: + case <-alterChan: + releaseHook = false + logutil.BgLogger().Info("XXXXXXXXXXX release hook") + break + } + domOwner.Reload() + if domNonOwner.InfoSchema().SchemaMetaVersion() == domOwner.InfoSchema().SchemaMetaVersion() { + // looping over reorganize data/indexes + hookChan <- struct{}{} + continue + } + break + } + logutil.BgLogger().Info("XXXXXXXXXXX states loop", zap.Int64("verCurr", verCurr), zap.Int64("NonOwner ver", domNonOwner.InfoSchema().SchemaMetaVersion()), zap.Int64("Owner ver", domOwner.InfoSchema().SchemaMetaVersion())) + domOwner.Reload() + require.Equal(t, verCurr-1, domNonOwner.InfoSchema().SchemaMetaVersion()) + require.Equal(t, verCurr, domOwner.InfoSchema().SchemaMetaVersion()) + loopFn(tkO, tkNO) + domNonOwner.Reload() + if !releaseHook { + // Alter done! + break + } + // Continue to next state + verCurr++ + i++ + hookChan <- struct{}{} + } + logutil.BgLogger().Info("XXXXXXXXXXX states loop done") + postFn(tkO) +} diff --git a/pkg/meta/model/table.go b/pkg/meta/model/table.go new file mode 100644 index 0000000000000..a643e07cec9b7 --- /dev/null +++ b/pkg/meta/model/table.go @@ -0,0 +1,1277 @@ +// Copyright 2024 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 model + +import ( + "bytes" + "fmt" + "strconv" + "strings" + "time" + "unsafe" + + "github.com/pingcap/tidb/pkg/parser/auth" + "github.com/pingcap/tidb/pkg/parser/duration" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/mysql" +) + +// ExtraHandleID is the column ID of column which we need to append to schema to occupy the handle's position +// for use of execution phase. +const ExtraHandleID = -1 + +// ExtraPhysTblID is the column ID of column that should be filled in with the physical table id. +// Primarily used for table partition dynamic prune mode, to return which partition (physical table id) the row came from. +// If used with a global index, the partition ID decoded from the key value will be filled in. +const ExtraPhysTblID = -3 + +// Deprecated: Use ExtraPhysTblID instead. +// const ExtraPidColID = -2 + +// ExtraRowChecksumID is the column ID of column which holds the row checksum info. +const ExtraRowChecksumID = -4 + +const ( + // TableInfoVersion0 means the table info version is 0. + // Upgrade from v2.1.1 or v2.1.2 to v2.1.3 and later, and then execute a "change/modify column" statement + // that does not specify a charset value for column. Then the following error may be reported: + // ERROR 1105 (HY000): unsupported modify charset from utf8mb4 to utf8. + // To eliminate this error, we will not modify the charset of this column + // when executing a change/modify column statement that does not specify a charset value for column. + // This behavior is not compatible with MySQL. + TableInfoVersion0 = uint16(0) + // TableInfoVersion1 means the table info version is 1. + // When we execute a change/modify column statement that does not specify a charset value for column, + // we set the charset of this column to the charset of table. This behavior is compatible with MySQL. + TableInfoVersion1 = uint16(1) + // TableInfoVersion2 means the table info version is 2. + // This is for v2.1.7 to Compatible with older versions charset problem. + // Old version such as v2.0.8 treat utf8 as utf8mb4, because there is no UTF8 check in v2.0.8. + // After version V2.1.2 (PR#8738) , TiDB add UTF8 check, then the user upgrade from v2.0.8 insert some UTF8MB4 characters will got error. + // This is not compatibility for user. Then we try to fix this in PR #9820, and increase the version number. + TableInfoVersion2 = uint16(2) + // TableInfoVersion3 means the table info version is 3. + // This version aims to deal with upper-cased charset name in TableInfo stored by versions prior to TiDB v2.1.9: + // TiDB always suppose all charsets / collations as lower-cased and try to convert them if they're not. + // However, the convert is missed in some scenarios before v2.1.9, so for all those tables prior to TableInfoVersion3, their + // charsets / collations will be converted to lower-case while loading from the storage. + TableInfoVersion3 = uint16(3) + // TableInfoVersion4 is not used. + TableInfoVersion4 = uint16(4) + // TableInfoVersion5 indicates that the auto_increment allocator in TiDB has been separated from + // _tidb_rowid allocator when AUTO_ID_CACHE is 1. This version is introduced to preserve the compatibility of old tables: + // the tables with version <= TableInfoVersion4 still use a single allocator for auto_increment and _tidb_rowid. + // Also see https://github.com/pingcap/tidb/issues/982. + TableInfoVersion5 = uint16(5) + + // CurrLatestTableInfoVersion means the latest table info in the current TiDB. + CurrLatestTableInfoVersion = TableInfoVersion5 +) + +// ExtraHandleName is the name of ExtraHandle Column. +var ExtraHandleName = model.NewCIStr("_tidb_rowid") + +// ExtraPhysTblIDName is the name of ExtraPhysTblID Column. +var ExtraPhysTblIDName = model.NewCIStr("_tidb_tid") + +// Deprecated: Use ExtraPhysTblIDName instead. +// var ExtraPartitionIdName = NewCIStr("_tidb_pid") //nolint:revive + +// TableInfo provides meta data describing a DB table. +type TableInfo struct { + ID int64 `json:"id"` + Name model.CIStr `json:"name"` + Charset string `json:"charset"` + Collate string `json:"collate"` + // Columns are listed in the order in which they appear in the schema. + Columns []*ColumnInfo `json:"cols"` + Indices []*IndexInfo `json:"index_info"` + Constraints []*ConstraintInfo `json:"constraint_info"` + ForeignKeys []*FKInfo `json:"fk_info"` + State SchemaState `json:"state"` + // PKIsHandle is true when primary key is a single integer column. + PKIsHandle bool `json:"pk_is_handle"` + // IsCommonHandle is true when clustered index feature is + // enabled and the primary key is not a single integer column. + IsCommonHandle bool `json:"is_common_handle"` + // CommonHandleVersion is the version of the clustered index. + // 0 for the clustered index created == 5.0.0 RC. + // 1 for the clustered index created > 5.0.0 RC. + CommonHandleVersion uint16 `json:"common_handle_version"` + + Comment string `json:"comment"` + AutoIncID int64 `json:"auto_inc_id"` + + // Only used by BR when: + // 1. SepAutoInc() is true + // 2. The table is nonclustered and has auto_increment column. + // In that case, both auto_increment_id and tidb_rowid need to be backup & recover. + // See also https://github.com/pingcap/tidb/issues/46093 + // + // It should have been named TiDBRowID, but for historial reasons, we do not use separate meta key for _tidb_rowid and auto_increment_id, + // and field `AutoIncID` is used to serve both _tidb_rowid and auto_increment_id. + // If we introduce a TiDBRowID here, it could make furthur misunderstanding: + // in most cases, AutoIncID is _tidb_rowid and TiDBRowID is null + // but in some cases, AutoIncID is auto_increment_id and TiDBRowID is _tidb_rowid + // So let's just use another name AutoIncIDExtra to avoid misconception. + AutoIncIDExtra int64 `json:"auto_inc_id_extra,omitempty"` + + AutoIDCache int64 `json:"auto_id_cache"` + AutoRandID int64 `json:"auto_rand_id"` + MaxColumnID int64 `json:"max_col_id"` + MaxIndexID int64 `json:"max_idx_id"` + MaxForeignKeyID int64 `json:"max_fk_id"` + MaxConstraintID int64 `json:"max_cst_id"` + // UpdateTS is used to record the timestamp of updating the table's schema information. + // These changing schema operations don't include 'truncate table' and 'rename table'. + UpdateTS uint64 `json:"update_timestamp"` + // OldSchemaID : + // Because auto increment ID has schemaID as prefix, + // We need to save original schemaID to keep autoID unchanged + // while renaming a table from one database to another. + // Only set if table has been renamed across schemas + // Old name 'old_schema_id' is kept for backwards compatibility + AutoIDSchemaID int64 `json:"old_schema_id,omitempty"` + + // ShardRowIDBits specify if the implicit row ID is sharded. + ShardRowIDBits uint64 + // MaxShardRowIDBits uses to record the max ShardRowIDBits be used so far. + MaxShardRowIDBits uint64 `json:"max_shard_row_id_bits"` + // AutoRandomBits is used to set the bit number to shard automatically when PKIsHandle. + AutoRandomBits uint64 `json:"auto_random_bits"` + // AutoRandomRangeBits represents the bit number of the int primary key that will be used by TiDB. + AutoRandomRangeBits uint64 `json:"auto_random_range_bits"` + // PreSplitRegions specify the pre-split region when create table. + // The pre-split region num is 2^(PreSplitRegions-1). + // And the PreSplitRegions should less than or equal to ShardRowIDBits. + PreSplitRegions uint64 `json:"pre_split_regions"` + + Partition *PartitionInfo `json:"partition"` + + Compression string `json:"compression"` + + View *ViewInfo `json:"view"` + + Sequence *SequenceInfo `json:"sequence"` + + // Lock represent the table lock info. + Lock *TableLockInfo `json:"Lock"` + + // Version means the version of the table info. + Version uint16 `json:"version"` + + // TiFlashReplica means the TiFlash replica info. + TiFlashReplica *TiFlashReplicaInfo `json:"tiflash_replica"` + + // IsColumnar means the table is column-oriented. + // It's true when the engine of the table is TiFlash only. + IsColumnar bool `json:"is_columnar"` + + TempTableType `json:"temp_table_type"` + TableCacheStatusType `json:"cache_table_status"` + PlacementPolicyRef *PolicyRefInfo `json:"policy_ref_info"` + + // StatsOptions is used when do analyze/auto-analyze for each table + StatsOptions *StatsOptions `json:"stats_options"` + + ExchangePartitionInfo *ExchangePartitionInfo `json:"exchange_partition_info"` + + TTLInfo *TTLInfo `json:"ttl_info"` + + // Revision is per table schema's version, it will be increased when the schema changed. + Revision uint64 `json:"revision"` + + DBID int64 `json:"-"` +} + +// SepAutoInc decides whether _rowid and auto_increment id use separate allocator. +func (t *TableInfo) SepAutoInc() bool { + return t.Version >= TableInfoVersion5 && t.AutoIDCache == 1 +} + +// GetPartitionInfo returns the partition information. +func (t *TableInfo) GetPartitionInfo() *PartitionInfo { + if t.Partition != nil && t.Partition.Enable { + return t.Partition + } + return nil +} + +// GetUpdateTime gets the table's updating time. +func (t *TableInfo) GetUpdateTime() time.Time { + return TSConvert2Time(t.UpdateTS) +} + +// Clone clones TableInfo. +func (t *TableInfo) Clone() *TableInfo { + nt := *t + nt.Columns = make([]*ColumnInfo, len(t.Columns)) + nt.Indices = make([]*IndexInfo, len(t.Indices)) + nt.ForeignKeys = make([]*FKInfo, len(t.ForeignKeys)) + + for i := range t.Columns { + nt.Columns[i] = t.Columns[i].Clone() + } + + for i := range t.Indices { + nt.Indices[i] = t.Indices[i].Clone() + } + + for i := range t.ForeignKeys { + nt.ForeignKeys[i] = t.ForeignKeys[i].Clone() + } + + if t.Partition != nil { + nt.Partition = t.Partition.Clone() + } + if t.TTLInfo != nil { + nt.TTLInfo = t.TTLInfo.Clone() + } + + return &nt +} + +// GetPkName will return the pk name if pk exists. +func (t *TableInfo) GetPkName() model.CIStr { + for _, colInfo := range t.Columns { + if mysql.HasPriKeyFlag(colInfo.GetFlag()) { + return colInfo.Name + } + } + return model.CIStr{} +} + +// GetPkColInfo gets the ColumnInfo of pk if exists. +// Make sure PkIsHandle checked before call this method. +func (t *TableInfo) GetPkColInfo() *ColumnInfo { + for _, colInfo := range t.Columns { + if mysql.HasPriKeyFlag(colInfo.GetFlag()) { + return colInfo + } + } + return nil +} + +// GetAutoIncrementColInfo gets the ColumnInfo of auto_increment column if exists. +func (t *TableInfo) GetAutoIncrementColInfo() *ColumnInfo { + for _, colInfo := range t.Columns { + if mysql.HasAutoIncrementFlag(colInfo.GetFlag()) { + return colInfo + } + } + return nil +} + +// IsAutoIncColUnsigned checks whether the auto increment column is unsigned. +func (t *TableInfo) IsAutoIncColUnsigned() bool { + col := t.GetAutoIncrementColInfo() + if col == nil { + return false + } + return mysql.HasUnsignedFlag(col.GetFlag()) +} + +// ContainsAutoRandomBits indicates whether a table contains auto_random column. +func (t *TableInfo) ContainsAutoRandomBits() bool { + return t.AutoRandomBits != 0 +} + +// IsAutoRandomBitColUnsigned indicates whether the auto_random column is unsigned. Make sure the table contains auto_random before calling this method. +func (t *TableInfo) IsAutoRandomBitColUnsigned() bool { + if !t.PKIsHandle || t.AutoRandomBits == 0 { + return false + } + return mysql.HasUnsignedFlag(t.GetPkColInfo().GetFlag()) +} + +// Cols returns the columns of the table in public state. +func (t *TableInfo) Cols() []*ColumnInfo { + publicColumns := make([]*ColumnInfo, len(t.Columns)) + maxOffset := -1 + for _, col := range t.Columns { + if col.State != StatePublic { + continue + } + publicColumns[col.Offset] = col + if maxOffset < col.Offset { + maxOffset = col.Offset + } + } + return publicColumns[0 : maxOffset+1] +} + +// FindIndexByName finds index by name. +func (t *TableInfo) FindIndexByName(idxName string) *IndexInfo { + for _, idx := range t.Indices { + if idx.Name.L == idxName { + return idx + } + } + return nil +} + +// FindColumnByID finds ColumnInfo by id. +func (t *TableInfo) FindColumnByID(id int64) *ColumnInfo { + for _, col := range t.Columns { + if col.ID == id { + return col + } + } + return nil +} + +// FindIndexByID finds index by id. +func (t *TableInfo) FindIndexByID(id int64) *IndexInfo { + for _, idx := range t.Indices { + if idx.ID == id { + return idx + } + } + return nil +} + +// FindPublicColumnByName finds the public column by name. +func (t *TableInfo) FindPublicColumnByName(colNameL string) *ColumnInfo { + for _, col := range t.Cols() { + if col.Name.L == colNameL { + return col + } + } + return nil +} + +// IsLocked checks whether the table was locked. +func (t *TableInfo) IsLocked() bool { + return t.Lock != nil && len(t.Lock.Sessions) > 0 +} + +// MoveColumnInfo moves a column to another offset. It maintains the offsets of all affects columns and index columns, +func (t *TableInfo) MoveColumnInfo(from, to int) { + if from == to { + return + } + updatedOffsets := make(map[int]int) + src := t.Columns[from] + if from < to { + for i := from; i < to; i++ { + t.Columns[i] = t.Columns[i+1] + t.Columns[i].Offset = i + updatedOffsets[i+1] = i + } + } else if from > to { + for i := from; i > to; i-- { + t.Columns[i] = t.Columns[i-1] + t.Columns[i].Offset = i + updatedOffsets[i-1] = i + } + } + t.Columns[to] = src + t.Columns[to].Offset = to + updatedOffsets[from] = to + for _, idx := range t.Indices { + for _, idxCol := range idx.Columns { + newOffset, ok := updatedOffsets[idxCol.Offset] + if ok { + idxCol.Offset = newOffset + } + } + } +} + +// ClearPlacement clears all table and partitions' placement settings +func (t *TableInfo) ClearPlacement() { + t.PlacementPolicyRef = nil + if t.Partition != nil { + for i := range t.Partition.Definitions { + def := &t.Partition.Definitions[i] + def.PlacementPolicyRef = nil + } + } +} + +// GetPrimaryKey extract the primary key in a table and return `IndexInfo` +// The returned primary key could be explicit or implicit. +// If there is no explicit primary key in table, +// the first UNIQUE INDEX on NOT NULL columns will be the implicit primary key. +// For more information about implicit primary key, see +// https://dev.mysql.com/doc/refman/8.0/en/invisible-indexes.html +func (t *TableInfo) GetPrimaryKey() *IndexInfo { + var implicitPK *IndexInfo + + for _, key := range t.Indices { + if key.Primary { + // table has explicit primary key + return key + } + // The case index without any columns should never happen, but still do a check here + if len(key.Columns) == 0 { + continue + } + // find the first unique key with NOT NULL columns + if implicitPK == nil && key.Unique { + // ensure all columns in unique key have NOT NULL flag + allColNotNull := true + skip := false + for _, idxCol := range key.Columns { + col := FindColumnInfo(t.Cols(), idxCol.Name.L) + // This index has a column in DeleteOnly state, + // or it is expression index (it defined on a hidden column), + // it can not be implicit PK, go to next index iterator + if col == nil || col.Hidden { + skip = true + break + } + if !mysql.HasNotNullFlag(col.GetFlag()) { + allColNotNull = false + break + } + } + if skip { + continue + } + if allColNotNull { + implicitPK = key + } + } + } + return implicitPK +} + +// ColumnIsInIndex checks whether c is included in any indices of t. +func (t *TableInfo) ColumnIsInIndex(c *ColumnInfo) bool { + for _, index := range t.Indices { + for _, column := range index.Columns { + if column.Name.L == c.Name.L { + return true + } + } + } + return false +} + +// HasClusteredIndex checks whether the table has a clustered index. +func (t *TableInfo) HasClusteredIndex() bool { + return t.PKIsHandle || t.IsCommonHandle +} + +// IsView checks if TableInfo is a view. +func (t *TableInfo) IsView() bool { + return t.View != nil +} + +// IsSequence checks if TableInfo is a sequence. +func (t *TableInfo) IsSequence() bool { + return t.Sequence != nil +} + +// IsBaseTable checks to see the table is neither a view or a sequence. +func (t *TableInfo) IsBaseTable() bool { + return t.Sequence == nil && t.View == nil +} + +// FindConstraintInfoByName finds constraintInfo by name. +func (t *TableInfo) FindConstraintInfoByName(constrName string) *ConstraintInfo { + lowConstrName := strings.ToLower(constrName) + for _, chk := range t.Constraints { + if chk.Name.L == lowConstrName { + return chk + } + } + return nil +} + +// FindIndexNameByID finds index name by id. +func (t *TableInfo) FindIndexNameByID(id int64) string { + indexInfo := FindIndexInfoByID(t.Indices, id) + if indexInfo != nil { + return indexInfo.Name.L + } + return "" +} + +// FindColumnNameByID finds column name by id. +func (t *TableInfo) FindColumnNameByID(id int64) string { + colInfo := FindColumnInfoByID(t.Columns, id) + if colInfo != nil { + return colInfo.Name.L + } + return "" +} + +// GetColumnByID finds the column by ID. +func (t *TableInfo) GetColumnByID(id int64) *ColumnInfo { + for _, col := range t.Columns { + if col.State != StatePublic { + continue + } + if col.ID == id { + return col + } + } + return nil +} + +// FindFKInfoByName finds FKInfo in fks by lowercase name. +func FindFKInfoByName(fks []*FKInfo, name string) *FKInfo { + for _, fk := range fks { + if fk.Name.L == name { + return fk + } + } + return nil +} + +// TableNameInfo provides meta data describing a table name info. +type TableNameInfo struct { + ID int64 `json:"id"` + Name model.CIStr `json:"name"` +} + +// TableCacheStatusType is the type of the table cache status +type TableCacheStatusType int + +// TableCacheStatusType values. +const ( + TableCacheStatusDisable TableCacheStatusType = iota + TableCacheStatusEnable + TableCacheStatusSwitching +) + +// String implements fmt.Stringer interface. +func (t TableCacheStatusType) String() string { + switch t { + case TableCacheStatusDisable: + return "disable" + case TableCacheStatusEnable: + return "enable" + case TableCacheStatusSwitching: + return "switching" + default: + return "" + } +} + +// TempTableType is the type of the temp table +type TempTableType byte + +// TempTableType values. +const ( + TempTableNone TempTableType = iota + TempTableGlobal + TempTableLocal +) + +// String implements fmt.Stringer interface. +func (t TempTableType) String() string { + switch t { + case TempTableGlobal: + return "global" + case TempTableLocal: + return "local" + default: + return "" + } +} + +// TableLockInfo provides meta data describing a table lock. +type TableLockInfo struct { + Tp model.TableLockType + // Use array because there may be multiple sessions holding the same read lock. + Sessions []SessionInfo + State TableLockState + // TS is used to record the timestamp this table lock been locked. + TS uint64 +} + +// SessionInfo contain the session ID and the server ID. +type SessionInfo struct { + ServerID string + SessionID uint64 +} + +// String implements fmt.Stringer interface. +func (s SessionInfo) String() string { + return "server: " + s.ServerID + "_session: " + strconv.FormatUint(s.SessionID, 10) +} + +// TableLockTpInfo is composed by schema ID, table ID and table lock type. +type TableLockTpInfo struct { + SchemaID int64 + TableID int64 + Tp model.TableLockType +} + +// TableLockState is the state for table lock. +type TableLockState byte + +const ( + // TableLockStateNone means this table lock is absent. + TableLockStateNone TableLockState = iota + // TableLockStatePreLock means this table lock is pre-lock state. Other session doesn't hold this lock should't do corresponding operation according to the lock type. + TableLockStatePreLock + // TableLockStatePublic means this table lock is public state. + TableLockStatePublic +) + +// String implements fmt.Stringer interface. +func (t TableLockState) String() string { + switch t { + case TableLockStatePreLock: + return "pre-lock" + case TableLockStatePublic: + return "public" + default: + return "none" + } +} + +// TiFlashReplicaInfo means the flash replica info. +type TiFlashReplicaInfo struct { + Count uint64 + LocationLabels []string + Available bool + AvailablePartitionIDs []int64 +} + +// IsPartitionAvailable checks whether the partition table replica was available. +func (tr *TiFlashReplicaInfo) IsPartitionAvailable(pid int64) bool { + for _, id := range tr.AvailablePartitionIDs { + if id == pid { + return true + } + } + return false +} + +// ViewInfo provides meta data describing a DB view. +type ViewInfo struct { + Algorithm model.ViewAlgorithm `json:"view_algorithm"` + Definer *auth.UserIdentity `json:"view_definer"` + Security model.ViewSecurity `json:"view_security"` + SelectStmt string `json:"view_select"` + CheckOption model.ViewCheckOption `json:"view_checkoption"` + Cols []model.CIStr `json:"view_cols"` +} + +// Some constants for sequence. +const ( + DefaultSequenceCacheBool = true + DefaultSequenceCycleBool = false + DefaultSequenceOrderBool = false + DefaultSequenceCacheValue = int64(1000) + DefaultSequenceIncrementValue = int64(1) + DefaultPositiveSequenceStartValue = int64(1) + DefaultNegativeSequenceStartValue = int64(-1) + DefaultPositiveSequenceMinValue = int64(1) + DefaultPositiveSequenceMaxValue = int64(9223372036854775806) + DefaultNegativeSequenceMaxValue = int64(-1) + DefaultNegativeSequenceMinValue = int64(-9223372036854775807) +) + +// SequenceInfo provide meta data describing a DB sequence. +type SequenceInfo struct { + Start int64 `json:"sequence_start"` + Cache bool `json:"sequence_cache"` + Cycle bool `json:"sequence_cycle"` + MinValue int64 `json:"sequence_min_value"` + MaxValue int64 `json:"sequence_max_value"` + Increment int64 `json:"sequence_increment"` + CacheValue int64 `json:"sequence_cache_value"` + Comment string `json:"sequence_comment"` +} + +// ExchangePartitionInfo provides exchange partition info. +type ExchangePartitionInfo struct { + // It is nt tableID when table which has the info is a partition table, else pt tableID. + ExchangePartitionTableID int64 `json:"exchange_partition_id"` + ExchangePartitionDefID int64 `json:"exchange_partition_def_id"` + // Deprecated, not used + XXXExchangePartitionFlag bool `json:"exchange_partition_flag"` +} + +// UpdateIndexInfo is to carry the entries in the list of indexes in UPDATE INDEXES +// during ALTER TABLE t PARTITION BY ... UPDATE INDEXES (idx_a GLOBAL, idx_b LOCAL...) +type UpdateIndexInfo struct { + IndexName string `json:"index_name"` + Global bool `json:"global"` +} + +// PartitionInfo provides table partition info. +type PartitionInfo struct { + Type model.PartitionType `json:"type"` + Expr string `json:"expr"` + Columns []model.CIStr `json:"columns"` + + // User may already create table with partition but table partition is not + // yet supported back then. When Enable is true, write/read need use tid + // rather than pid. + Enable bool `json:"enable"` + + // IsEmptyColumns is for syntax like `partition by key()`. + // When IsEmptyColums is true, it will not display column name in `show create table` stmt. + IsEmptyColumns bool `json:"is_empty_columns"` + + Definitions []PartitionDefinition `json:"definitions"` + // AddingDefinitions is filled when adding partitions that is in the mid state. + AddingDefinitions []PartitionDefinition `json:"adding_definitions"` + // DroppingDefinitions is filled when dropping/truncating partitions that is in the mid state. + DroppingDefinitions []PartitionDefinition `json:"dropping_definitions"` + // NewPartitionIDs is filled when truncating partitions that is in the mid state. + NewPartitionIDs []int64 `json:"new_partition_ids,omitempty"` + // OriginalPartitionIDsOrder is only needed for rollback of Reorganize Partition for + // LIST partitions, since in StateDeleteReorganize we don't know the old order any longer. + OriginalPartitionIDsOrder []int64 `json:"original_partition_ids_order,omitempty"` + + States []PartitionState `json:"states"` + Num uint64 `json:"num"` + // Indicate which DDL Action is currently on going + DDLAction ActionType `json:"ddl_action,omitempty"` + // Only used during ReorganizePartition so far + DDLState SchemaState `json:"ddl_state"` + // Set during ALTER TABLE ... if the table id needs to change + // like if there is a global index or going between non-partitioned + // and partitioned table, to make the data dropping / range delete + // optimized. + NewTableID int64 `json:"new_table_id"` + // Set during ALTER TABLE ... PARTITION BY ... + // First as the new partition scheme, then in StateDeleteReorg as the old + DDLType model.PartitionType `json:"ddl_type"` + DDLExpr string `json:"ddl_expr"` + DDLColumns []model.CIStr `json:"ddl_columns"` + // For ActionAlterTablePartitioning, UPDATE INDEXES + DDLUpdateIndexes []UpdateIndexInfo `json:"ddl_update_indexes"` +} + +// Clone clones itself. +func (pi *PartitionInfo) Clone() *PartitionInfo { + newPi := *pi + newPi.Columns = make([]model.CIStr, len(pi.Columns)) + copy(newPi.Columns, pi.Columns) + + newPi.Definitions = make([]PartitionDefinition, len(pi.Definitions)) + for i := range pi.Definitions { + newPi.Definitions[i] = pi.Definitions[i].Clone() + } + + newPi.AddingDefinitions = make([]PartitionDefinition, len(pi.AddingDefinitions)) + for i := range pi.AddingDefinitions { + newPi.AddingDefinitions[i] = pi.AddingDefinitions[i].Clone() + } + + newPi.DroppingDefinitions = make([]PartitionDefinition, len(pi.DroppingDefinitions)) + for i := range pi.DroppingDefinitions { + newPi.DroppingDefinitions[i] = pi.DroppingDefinitions[i].Clone() + } + + return &newPi +} + +// GetNameByID gets the partition name by ID. +// TODO: Remove the need for this function! +func (pi *PartitionInfo) GetNameByID(id int64) string { + definitions := pi.Definitions + // do not convert this loop to `for _, def := range definitions`. + // see https://github.com/pingcap/parser/pull/1072 for the benchmark. + for i := range definitions { + if id == definitions[i].ID { + return definitions[i].Name.O + } + } + return "" +} + +// GetStateByID gets the partition state by ID. +func (pi *PartitionInfo) GetStateByID(id int64) SchemaState { + for _, pstate := range pi.States { + if pstate.ID == id { + return pstate.State + } + } + return StatePublic +} + +// SetStateByID sets the state of the partition by ID. +func (pi *PartitionInfo) SetStateByID(id int64, state SchemaState) { + newState := PartitionState{ID: id, State: state} + for i, pstate := range pi.States { + if pstate.ID == id { + pi.States[i] = newState + return + } + } + if pi.States == nil { + pi.States = make([]PartitionState, 0, 1) + } + pi.States = append(pi.States, newState) +} + +// GCPartitionStates cleans up the partition state. +func (pi *PartitionInfo) GCPartitionStates() { + if len(pi.States) < 1 { + return + } + newStates := make([]PartitionState, 0, len(pi.Definitions)) + for _, state := range pi.States { + found := false + for _, def := range pi.Definitions { + if def.ID == state.ID { + found = true + break + } + } + if found { + newStates = append(newStates, state) + } + } + pi.States = newStates +} + +// HasTruncatingPartitionID checks whether the pid is truncating. +func (pi *PartitionInfo) HasTruncatingPartitionID(pid int64) bool { + for i := range pi.NewPartitionIDs { + if pi.NewPartitionIDs[i] == pid { + return true + } + } + return false +} + +// ClearReorgIntermediateInfo remove intermediate information used during reorganize partition. +func (pi *PartitionInfo) ClearReorgIntermediateInfo() { + pi.DDLAction = ActionNone + pi.DDLState = StateNone + pi.DDLType = model.PartitionTypeNone + pi.DDLExpr = "" + pi.DDLColumns = nil + pi.NewTableID = 0 +} + +// FindPartitionDefinitionByName finds PartitionDefinition by name. +func (pi *PartitionInfo) FindPartitionDefinitionByName(partitionDefinitionName string) int { + lowConstrName := strings.ToLower(partitionDefinitionName) + definitions := pi.Definitions + for i := range definitions { + if definitions[i].Name.L == lowConstrName { + return i + } + } + return -1 +} + +// GetPartitionIDByName gets the partition ID by name. +func (pi *PartitionInfo) GetPartitionIDByName(partitionDefinitionName string) int64 { + lowConstrName := strings.ToLower(partitionDefinitionName) + for _, definition := range pi.Definitions { + if definition.Name.L == lowConstrName { + return definition.ID + } + } + return -1 +} + +// GetDefaultListPartition return the index of Definitions +// that contains the LIST Default partition otherwise it returns -1 +func (pi *PartitionInfo) GetDefaultListPartition() int { + if pi.Type != model.PartitionTypeList { + return -1 + } + defs := pi.Definitions + for i := range defs { + if len(defs[i].InValues) == 0 { + return i + } + for _, vs := range defs[i].InValues { + if len(vs) == 1 && vs[0] == "DEFAULT" { + return i + } + } + } + + return -1 +} + +// CanHaveOverlappingDroppingPartition returns true if special handling +// is needed during DDL of partitioned tables, +// where range or list with default partition can have +// overlapping partitions. +// Example: +// ... PARTITION BY RANGE (a) +// (PARTITION p0 VALUES LESS THAN (10), +// PARTITION p1 VALUES LESS THAN (20)) +// ALTER TABLE t DROP PARTITION p0; +// When p0 is gone, then p1 can have values < 10, +// so if p0 is visible for one session, while another session +// have dropped p0, a value '9' will then be in p1, instead of p0, +// i.e. an "overlapping" partition, that needs special handling. +// Same can happen for LIST partitioning, if there is a DEFAULT partition. +func (pi *PartitionInfo) CanHaveOverlappingDroppingPartition() bool { + if pi.DDLAction == ActionDropTablePartition && + pi.DDLState == StateWriteOnly { + return true + } + return false +} + +// ReplaceWithOverlappingPartitionIdx returns the overlapping partition +// if there is one and a previous error. +// Functions based on locatePartitionCommon, like GetPartitionIdxByRow +// will return the found partition, with an error, +// since it is being dropped. +// This function will correct the partition index and error if it can. +// For example of Overlapping partition, +// see CanHaveOverlappingDroppingPartition +// This function should not be used for writing, since we should block +// writes to partitions that are being dropped. +// But for read, we should replace the dropping partitions with +// the overlapping partition if it exists, so we can read new data +// from sessions one step ahead in the DDL State. +func (pi *PartitionInfo) ReplaceWithOverlappingPartitionIdx(idx int, err error) (int, error) { + if err != nil && idx >= 0 { + idx = pi.GetOverlappingDroppingPartitionIdx(idx) + if idx >= 0 { + err = nil + } + } + return idx, err +} + +// GetOverlappingDroppingPartitionIdx takes the index of Definitions +// and returns possible overlapping partition to use instead. +// Only used during DROP PARTITION! +// For RANGE, DROP PARTITION must be a consecutive range of partitions. +// For LIST, it only takes effect if there is default partition. +// returns same idx if no overlapping partition +// return -1 if the partition is being dropped, with no overlapping partition, +// like for last range partition dropped or no default list partition. +// See CanHaveOverlappingDroppingPartition() for more info about +// Overlapping dropping partition. +func (pi *PartitionInfo) GetOverlappingDroppingPartitionIdx(idx int) int { + if idx < 0 || idx >= len(pi.Definitions) { + return -1 + } + if pi.CanHaveOverlappingDroppingPartition() { + switch pi.Type { + case model.PartitionTypeRange: + for i := idx; i < len(pi.Definitions); i++ { + if pi.IsDropping(i) { + continue + } + return i + } + // Last partition is also dropped! + return -1 + case model.PartitionTypeList: + if pi.IsDropping(idx) { + defaultIdx := pi.GetDefaultListPartition() + if defaultIdx == idx { + return -1 + } + return defaultIdx + } + return idx + } + } + return idx +} + +// IsDropping returns true if the partition +// is being dropped (i.e. in DroppingDefinitions) +func (pi *PartitionInfo) IsDropping(idx int) bool { + for _, def := range pi.DroppingDefinitions { + if def.ID == pi.Definitions[idx].ID { + return true + } + } + return false +} + +// SetOriginalPartitionIDs sets the order of the original partition IDs +// in case it needs to be rolled back. LIST Partitioning would not know otherwise. +func (pi *PartitionInfo) SetOriginalPartitionIDs() { + ids := make([]int64, 0, len(pi.Definitions)) + for _, def := range pi.Definitions { + ids = append(ids, def.ID) + } + pi.OriginalPartitionIDsOrder = ids +} + +// PartitionState is the state of the partition. +type PartitionState struct { + ID int64 `json:"id"` + State SchemaState `json:"state"` +} + +// PartitionDefinition defines a single partition. +type PartitionDefinition struct { + ID int64 `json:"id"` + Name model.CIStr `json:"name"` + LessThan []string `json:"less_than"` + InValues [][]string `json:"in_values"` + PlacementPolicyRef *PolicyRefInfo `json:"policy_ref_info"` + Comment string `json:"comment,omitempty"` +} + +// Clone clones ConstraintInfo. +func (ci *PartitionDefinition) Clone() PartitionDefinition { + nci := *ci + nci.LessThan = make([]string, len(ci.LessThan)) + copy(nci.LessThan, ci.LessThan) + return nci +} + +const emptyPartitionDefinitionSize = int64(unsafe.Sizeof(PartitionState{})) + +// MemoryUsage return the memory usage of PartitionDefinition +func (ci *PartitionDefinition) MemoryUsage() (sum int64) { + if ci == nil { + return + } + + sum = emptyPartitionDefinitionSize + ci.Name.MemoryUsage() + if ci.PlacementPolicyRef != nil { + sum += int64(unsafe.Sizeof(ci.PlacementPolicyRef.ID)) + ci.PlacementPolicyRef.Name.MemoryUsage() + } + + for _, str := range ci.LessThan { + sum += int64(len(str)) + } + for _, strs := range ci.InValues { + for _, str := range strs { + sum += int64(len(str)) + } + } + return +} + +// ConstraintInfo provides meta data describing check-expression constraint. +type ConstraintInfo struct { + ID int64 `json:"id"` + Name model.CIStr `json:"constraint_name"` + Table model.CIStr `json:"tbl_name"` // Table name. + ConstraintCols []model.CIStr `json:"constraint_cols"` // Depended column names. + Enforced bool `json:"enforced"` + InColumn bool `json:"in_column"` // Indicate whether the constraint is column type check. + ExprString string `json:"expr_string"` + State SchemaState `json:"state"` +} + +// Clone clones ConstraintInfo. +func (ci *ConstraintInfo) Clone() *ConstraintInfo { + nci := *ci + + nci.ConstraintCols = make([]model.CIStr, len(ci.ConstraintCols)) + copy(nci.ConstraintCols, ci.ConstraintCols) + return &nci +} + +// FKInfo provides meta data describing a foreign key constraint. +type FKInfo struct { + ID int64 `json:"id"` + Name model.CIStr `json:"fk_name"` + RefSchema model.CIStr `json:"ref_schema"` + RefTable model.CIStr `json:"ref_table"` + RefCols []model.CIStr `json:"ref_cols"` + Cols []model.CIStr `json:"cols"` + OnDelete int `json:"on_delete"` + OnUpdate int `json:"on_update"` + State SchemaState `json:"state"` + Version int `json:"version"` +} + +// String returns the string representation of FKInfo. +func (fk *FKInfo) String(db, tb string) string { + buf := bytes.Buffer{} + buf.WriteString("`" + db + "`.`") + buf.WriteString(tb + "`, CONSTRAINT `") + buf.WriteString(fk.Name.O + "` FOREIGN KEY (") + for i, col := range fk.Cols { + if i > 0 { + buf.WriteString(", ") + } + buf.WriteString("`" + col.O + "`") + } + buf.WriteString(") REFERENCES `") + if fk.RefSchema.L != db { + buf.WriteString(fk.RefSchema.L) + buf.WriteString("`.`") + } + buf.WriteString(fk.RefTable.L) + buf.WriteString("` (") + for i, col := range fk.RefCols { + if i > 0 { + buf.WriteString(", ") + } + buf.WriteString("`" + col.O + "`") + } + buf.WriteString(")") + if onDelete := model.ReferOptionType(fk.OnDelete); onDelete != model.ReferOptionNoOption { + buf.WriteString(" ON DELETE ") + buf.WriteString(onDelete.String()) + } + if onUpdate := model.ReferOptionType(fk.OnUpdate); onUpdate != model.ReferOptionNoOption { + buf.WriteString(" ON UPDATE ") + buf.WriteString(onUpdate.String()) + } + return buf.String() +} + +// Clone clones FKInfo. +func (fk *FKInfo) Clone() *FKInfo { + nfk := *fk + + nfk.RefCols = make([]model.CIStr, len(fk.RefCols)) + nfk.Cols = make([]model.CIStr, len(fk.Cols)) + copy(nfk.RefCols, fk.RefCols) + copy(nfk.Cols, fk.Cols) + + return &nfk +} + +const ( + // FKVersion0 indicate the FKInfo version is 0. + // In FKVersion0, TiDB only supported syntax of foreign key, but the foreign key constraint doesn't take effect. + FKVersion0 = 0 + // FKVersion1 indicate the FKInfo version is 1. + // In FKVersion1, TiDB supports the foreign key constraint. + FKVersion1 = 1 +) + +// ReferredFKInfo provides the cited foreign key in the child table. +type ReferredFKInfo struct { + Cols []model.CIStr `json:"cols"` + ChildSchema model.CIStr `json:"child_schema"` + ChildTable model.CIStr `json:"child_table"` + ChildFKName model.CIStr `json:"child_fk_name"` +} + +// TableItemID is composed by table ID and column/index ID +type TableItemID struct { + TableID int64 + ID int64 + IsIndex bool + IsSyncLoadFailed bool +} + +// Key is used to generate unique key for TableItemID to use in the syncload +func (t TableItemID) Key() string { + return fmt.Sprintf("%d#%d#%t", t.ID, t.TableID, t.IsIndex) +} + +// StatsLoadItem represents the load unit for statistics's memory loading. +type StatsLoadItem struct { + TableItemID + FullLoad bool +} + +// Key is used to generate unique key for TableItemID to use in the syncload +func (s StatsLoadItem) Key() string { + return fmt.Sprintf("%s#%t", s.TableItemID.Key(), s.FullLoad) +} + +// StatsOptions is the struct to store the stats options. +type StatsOptions struct { + *StatsWindowSettings + AutoRecalc bool `json:"auto_recalc"` + ColumnChoice model.ColumnChoice `json:"column_choice"` + ColumnList []model.CIStr `json:"column_list"` + SampleNum uint64 `json:"sample_num"` + SampleRate float64 `json:"sample_rate"` + Buckets uint64 `json:"buckets"` + TopN uint64 `json:"topn"` + Concurrency uint `json:"concurrency"` +} + +// NewStatsOptions creates a new StatsOptions. +func NewStatsOptions() *StatsOptions { + return &StatsOptions{ + AutoRecalc: true, + ColumnChoice: model.DefaultChoice, + ColumnList: []model.CIStr{}, + SampleNum: uint64(0), + SampleRate: 0.0, + Buckets: uint64(0), + TopN: uint64(0), + Concurrency: uint(0), + } +} + +// StatsWindowSettings is the settings of the stats window. +type StatsWindowSettings struct { + WindowStart time.Time `json:"window_start"` + WindowEnd time.Time `json:"window_end"` + RepeatType WindowRepeatType `json:"repeat_type"` + RepeatInterval uint `json:"repeat_interval"` +} + +// WindowRepeatType is the type of the window repeat. +type WindowRepeatType byte + +// WindowRepeatType values. +const ( + Never WindowRepeatType = iota + Day + Week + Month +) + +// String implements fmt.Stringer interface. +func (s WindowRepeatType) String() string { + switch s { + case Never: + return "Never" + case Day: + return "Day" + case Week: + return "Week" + case Month: + return "Month" + default: + return "" + } +} + +// DefaultJobInterval sets the default interval between TTL jobs +const DefaultJobInterval = time.Hour + +// TTLInfo records the TTL config +type TTLInfo struct { + ColumnName model.CIStr `json:"column"` + IntervalExprStr string `json:"interval_expr"` + // `IntervalTimeUnit` is actually ast.TimeUnitType. Use `int` to avoid cycle dependency + IntervalTimeUnit int `json:"interval_time_unit"` + Enable bool `json:"enable"` + // JobInterval is the interval between two TTL scan jobs. + // It's suggested to get a duration with `(*TTLInfo).GetJobInterval` + JobInterval string `json:"job_interval"` +} + +// Clone clones TTLInfo +func (t *TTLInfo) Clone() *TTLInfo { + cloned := *t + return &cloned +} + +// GetJobInterval parses the job interval and return +// if the job interval is an empty string, the "1h" will be returned, to keep compatible with 6.5 (in which +// TTL_JOB_INTERVAL attribute doesn't exist) +// Didn't set TTL_JOB_INTERVAL during upgrade and bootstrap because setting default value here is much simpler +// and could avoid bugs blocking users from upgrading or bootstrapping the cluster. +func (t *TTLInfo) GetJobInterval() (time.Duration, error) { + if len(t.JobInterval) == 0 { + return DefaultJobInterval, nil + } + + return duration.ParseDuration(t.JobInterval) +} diff --git a/pkg/planner/core/casetest/partition/testdata/integration_partition_suite_out.json b/pkg/planner/core/casetest/partition/testdata/integration_partition_suite_out.json index 8ae2b35395618..9bbf9cd567efc 100644 --- a/pkg/planner/core/casetest/partition/testdata/integration_partition_suite_out.json +++ b/pkg/planner/core/casetest/partition/testdata/integration_partition_suite_out.json @@ -1121,7 +1121,7 @@ { "SQL": "select * from tlist1 where a = 1 and b in (1,2)", "DynamicPlan": [ - "IndexReader 2.00 root partition:p0,p1 index:IndexRangeScan", + "IndexReader 2.00 root partition:all index:IndexRangeScan", "└─IndexRangeScan 2.00 cop[tikv] table:tlist1, index:PRIMARY(a, b) range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo" ], "StaticPlan": [ @@ -1139,7 +1139,7 @@ { "SQL": "select * from tlist1 where a = 1 and b in (1,2) order by b", "DynamicPlan": [ - "IndexReader 2.00 root partition:p0,p1 index:IndexRangeScan", + "IndexReader 2.00 root partition:all index:IndexRangeScan", "└─IndexRangeScan 2.00 cop[tikv] table:tlist1, index:PRIMARY(a, b) range:[1 1,1 1], [1 2,1 2], keep order:true, stats:pseudo" ], "StaticPlan": [ @@ -1158,7 +1158,7 @@ { "SQL": "select * from tlist1 where a = 1 and b in (1,2) order by b desc", "DynamicPlan": [ - "IndexReader 2.00 root partition:p0,p1 index:IndexRangeScan", + "IndexReader 2.00 root partition:all index:IndexRangeScan", "└─IndexRangeScan 2.00 cop[tikv] table:tlist1, index:PRIMARY(a, b) range:[1 1,1 1], [1 2,1 2], keep order:true, desc, stats:pseudo" ], "StaticPlan": [ @@ -1422,7 +1422,7 @@ { "SQL": "select * from tlist2 where a = 1 and b in (1,2)", "DynamicPlan": [ - "TableReader 2.00 root partition:p0,p1 data:TableRangeScan", + "TableReader 2.00 root partition:all data:TableRangeScan", "└─TableRangeScan 2.00 cop[tikv] table:tlist2 range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo" ], "StaticPlan": [ @@ -1440,7 +1440,7 @@ { "SQL": "select * from tlist2 where a = 1 and b in (1,2) order by b", "DynamicPlan": [ - "TableReader 2.00 root partition:p0,p1 data:TableRangeScan", + "TableReader 2.00 root partition:all data:TableRangeScan", "└─TableRangeScan 2.00 cop[tikv] table:tlist2 range:[1 1,1 1], [1 2,1 2], keep order:true, stats:pseudo" ], "StaticPlan": [ @@ -1460,7 +1460,7 @@ "SQL": "select * from tlist2 where a = 1 and b in (1,2) order by b desc", "DynamicPlan": [ "Sort 0.02 root test.tlist2.b:desc", - "└─TableReader 2.00 root partition:p0,p1 data:TableRangeScan", + "└─TableReader 2.00 root partition:all data:TableRangeScan", " └─TableRangeScan 2.00 cop[tikv] table:tlist2 range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo" ], "StaticPlan": [ @@ -1701,7 +1701,7 @@ { "SQL": "select * from tlist3 where a in (1,3) and 1 = 1", "DynamicPlan": [ - "TableReader 2.00 root partition:p0,p1 data:TableRangeScan", + "TableReader 2.00 root partition:all data:TableRangeScan", "└─TableRangeScan 2.00 cop[tikv] table:tlist3 range:[1,1], [3,3], keep order:false, stats:pseudo" ], "StaticPlan": [ diff --git a/pkg/planner/core/casetest/partition/testdata/partition_pruner_in.json b/pkg/planner/core/casetest/partition/testdata/partition_pruner_in.json index 1ece78adf46cf..aa35b16978330 100644 --- a/pkg/planner/core/casetest/partition/testdata/partition_pruner_in.json +++ b/pkg/planner/core/casetest/partition/testdata/partition_pruner_in.json @@ -66,7 +66,7 @@ }, { "SQL": "select * from t1 where a in (1,2,3) or b in (4,5,6)", - "Pruner": "t1: p0,p1" + "Pruner": "t1: all" }, { "SQL": "select * from t1 where a in (1,2,3) and b in (4,5,6)", @@ -86,7 +86,7 @@ }, { "SQL": "select * from t1 where ( a=1 and b=1) or (a=6 and b=6)", - "Pruner": "t1: p0,p1" + "Pruner": "t1: all" }, { "SQL": "select * from t1 where a = 100 and b = 100", @@ -130,7 +130,7 @@ }, { "SQL": "select * from t1 where a = 1 or (a = 10 and b is null)", - "Pruner": "t1: p0,p1" + "Pruner": "t1: all" }, { "SQL": "select * from t1 where a = 8 or (a = 10 and b is null)", @@ -162,7 +162,7 @@ }, { "SQL": "select * from t1 where a = 100 or b in (1,6)", - "Pruner": "t1: p0,p1" + "Pruner": "t1: all" }, { "SQL": "select * from t1 where a = 100 or b in (100,200)", @@ -170,15 +170,15 @@ }, { "SQL": "select * from t1 where a in (1,6) or b in (1,2) or (a=3 and b =3)", - "Pruner": "t1: p0,p1" + "Pruner": "t1: all" }, { "SQL": "select * from t1 where a in (1,6)", - "Pruner": "t1: p0,p1" + "Pruner": "t1: all" }, { "SQL": "select * from t1 where a in (1,6) or (a=3 and b =3)", - "Pruner": "t1: p0,p1" + "Pruner": "t1: all" }, { "SQL": "select * from t1 where a in (1,6) and (a=3 and b =3)", @@ -218,11 +218,11 @@ }, { "SQL": "select * from t1 where t1.a in (select b from t2 where a in (1,2)) order by a", - "Pruner": "t1: p0,p1; t2: p0" + "Pruner": "t1: all; t2: p0" }, { "SQL": "select * from t1 where t1.a in (select b from t1 where a in (1,2)) order by a", - "Pruner": "t1: p0; t1: p0,p1" + "Pruner": "t1: all; t1: p0" }, { "SQL": "select * from t1 left join t2 on t1.id = t2.id where (t1.a=1 or t1.a = 3) and t2.a in (6,7,8)", @@ -238,15 +238,15 @@ }, { "SQL": "select count(*) from t1 join t2 on t1.b = t2.b where t1.a in (1,2) and t2.a in (1,6) and t1.b in (1,6)", - "Pruner": "t1: p0; t2: p0,p1" + "Pruner": "t1: p0; t2: all" }, { "SQL": "select /*+ INL_JOIN(t2,t1) */ count(*) from t2 join t1 on t2.b = t1.b where t2.a in (1,2) and t1.a in (1,6) and t1.b in (1,6)", - "Pruner": "t1: p0,p1; t2: p0" + "Pruner": "t1: all; t2: p0" }, { "SQL": "select /*+ INL_HASH_JOIN(t1,t2) */ count(*) from t2 join t1 on t2.b = t1.b where t2.a in (1,2) and t1.a in (1,6) and t1.b in (6,1)", - "Pruner": "t1: p0,p1; t2: p0" + "Pruner": "t1: all; t2: p0" }, { "SQL": "select /*+ INL_HASH_JOIN(t1,t2) */ count(*) from t2 join t1 on t2.b = t1.b where t2.a in (1,2) and t1.a in (1,6) and t1.b in (100,9,6)", @@ -254,7 +254,7 @@ }, { "SQL": "select /*+ INL_HASH_JOIN(t1,t2) */ count(*) from t2 join t1 on t2.b = t1.b where t2.a in (1,2) and t1.a in (1,6) and t1.b in (100,9,6,1)", - "Pruner": "t1: p0,p1; t2: p0" + "Pruner": "t1: all; t2: p0" }, { "SQL": "select * from t1 where a in (1,2,3) union select * from t1 where b in (6,7,8) order by a", @@ -275,7 +275,7 @@ }, { "SQL": "select * from t1 where a < 3 or b > 4", - "Pruner": "t1: p0,p1" + "Pruner": "t1: all" }, { "SQL": "select * from t1 where a < 3 and b > 4", @@ -295,11 +295,11 @@ }, { "SQL": "select * from t1 where (a<=1 and b<=1) or (a >=6 and b>=6)", - "Pruner": "t1: p0,p1" + "Pruner": "t1: all" }, { "SQL": "select * from t1 where a <= 100 and b <= 100", - "Pruner": "t1: p0,p1" + "Pruner": "t1: all" }, { "SQL": "select * from t1 join t2 on t1.id = t2.id where (t1.a <= 3 and (t1.b >= 3 and t1.b <= 5)) and (t2.a >= 6 and t2.a <= 8) and t2.b>=7 and t2.id>=7", @@ -331,11 +331,11 @@ }, { "SQL": "select * from t1 where a <= 1 or (a <= 10 and b is null)", - "Pruner": "t1: p0,p1" + "Pruner": "t1: all" }, { "SQL": "select * from t1 where a <= 8 or b <= 9", - "Pruner": "t1: p0,p1" + "Pruner": "t1: all" }, { "SQL": "select * from t1 where a <= 3 and false", @@ -363,7 +363,7 @@ }, { "SQL": "select * from t1 where a = 100 or b >= 1 and b <= 6", - "Pruner": "t1: p0,p1" + "Pruner": "t1: all" }, { "SQL": "select * from t1 where a = 100 or (b >= 100 and b <= 200)", @@ -371,15 +371,15 @@ }, { "SQL": "select * from t1 where (a >= 1 and a <= 6) or (b >= 1 and b <= 2) or (a<=3 and b <=3)", - "Pruner": "t1: p0,p1" + "Pruner": "t1: all" }, { "SQL": "select * from t1 where a >= 1 and a <= 6", - "Pruner": "t1: p0,p1" + "Pruner": "t1: all" }, { "SQL": "select * from t1 where (a >= 1 and a <= 6) or (a>=3 and b >=3)", - "Pruner": "t1: p0,p1" + "Pruner": "t1: all" }, { "SQL": "select * from t1 where a in (1,6) and (a=3 and b =3)", @@ -411,19 +411,19 @@ }, { "SQL": "select * from t1 join t2 on t1.id = t2.id where (t1.a<=1 or t1.a <= 3) and (t2.a <= 6 and t2.b <= 6)", - "Pruner": "t1: p0; t2: p0,p1" + "Pruner": "t1: p0; t2: all" }, { "SQL": "select * from t1 join t1 as t2 on t1.id = t2.id where (t1.a<=1 or t1.a <= 3) and (t2.a <= 6 and t2.b <= 6)", - "Pruner": "t1: p0; t2: p0,p1" + "Pruner": "t1: p0; t2: all" }, { "SQL": "select * from t1 where t1.a in (select b from t2 where a BETWEEN 1 AND 2) order by a", - "Pruner": "t1: p0,p1; t2: p0" + "Pruner": "t1: all; t2: p0" }, { "SQL": "select * from t1 where t1.a in (select b from t1 where a BETWEEN 1 AND 2) order by a", - "Pruner": "t1: p0; t1: p0,p1" + "Pruner": "t1: all; t1: p0" }, { "SQL": "select * from t1 left join t2 on t1.id = t2.id where (t1.a<=1 or t1.a <= 3) and t2.a BETWEEN 6 AND 8", @@ -439,11 +439,11 @@ }, { "SQL": "select count(*) from t1 join t2 on t1.b = t2.b where t1.a BETWEEN 1 AND 2 and t2.a BETWEEN 1 AND 6 and t1.b BETWEEN 1 AND 6", - "Pruner": "t1: p0; t2: p0,p1" + "Pruner": "t1: p0; t2: all" }, { "SQL": "select /*+ INL_JOIN(t2,t1) */ count(*) from t2 join t1 on t2.b = t1.b where t2.a BETWEEN 1 AND 2 and t1.a BETWEEN 1 AND 6 and t1.b BETWEEN 1 AND 6", - "Pruner": "t1: p0,p1; t2: p0" + "Pruner": "t1: all; t2: p0" } ] } diff --git a/pkg/planner/core/casetest/partition/testdata/partition_pruner_out.json b/pkg/planner/core/casetest/partition/testdata/partition_pruner_out.json index ee3b7bb9d2e0d..1aeb1d10bd8f0 100644 --- a/pkg/planner/core/casetest/partition/testdata/partition_pruner_out.json +++ b/pkg/planner/core/casetest/partition/testdata/partition_pruner_out.json @@ -362,12 +362,12 @@ "6 6 6" ], "Plan": [ - "TableReader 59.91 root partition:p0,p1 data:Selection", + "TableReader 59.91 root partition:all data:Selection", "└─Selection 59.91 cop[tikv] or(in(test_partition.t1.a, 1, 2, 3), in(test_partition.t1.b, 4, 5, 6))", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "IndexPlan": [ - "IndexReader 59.91 root partition:p0,p1 index:Selection", + "IndexReader 59.91 root partition:all index:Selection", "└─Selection 59.91 cop[tikv] or(in(test_partition_1.t1.a, 1, 2, 3), in(test_partition_1.t1.b, 4, 5, 6))", " └─IndexFullScan 10000.00 cop[tikv] table:t1, index:a(a, b, id) keep order:false, stats:pseudo" ] @@ -442,12 +442,12 @@ "6 6 6" ], "Plan": [ - "TableReader 0.02 root partition:p0,p1 data:Selection", + "TableReader 0.02 root partition:all data:Selection", "└─Selection 0.02 cop[tikv] or(and(eq(test_partition.t1.a, 1), eq(test_partition.t1.b, 1)), and(eq(test_partition.t1.a, 6), eq(test_partition.t1.b, 6)))", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "IndexPlan": [ - "IndexReader 0.20 root partition:p0,p1 index:IndexRangeScan", + "IndexReader 0.20 root partition:all index:IndexRangeScan", "└─IndexRangeScan 0.20 cop[tikv] table:t1, index:a(a, b, id) range:[1 1,1 1], [6 6,6 6], keep order:false, stats:pseudo" ] }, @@ -631,12 +631,12 @@ " 10 " ], "Plan": [ - "TableReader 10.01 root partition:p0,p1 data:Selection", + "TableReader 10.01 root partition:all data:Selection", "└─Selection 10.01 cop[tikv] or(eq(test_partition.t1.a, 1), and(eq(test_partition.t1.a, 10), isnull(test_partition.t1.b)))", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "IndexPlan": [ - "IndexReader 10.10 root partition:p0,p1 index:IndexRangeScan", + "IndexReader 10.10 root partition:all index:IndexRangeScan", "└─IndexRangeScan 10.10 cop[tikv] table:t1, index:a(a, b, id) range:[1,1], [10 NULL,10 NULL], keep order:false, stats:pseudo" ] }, @@ -763,12 +763,12 @@ "6 6 6" ], "Plan": [ - "TableReader 29.98 root partition:p0,p1 data:Selection", + "TableReader 29.98 root partition:all data:Selection", "└─Selection 29.98 cop[tikv] or(eq(test_partition.t1.a, 100), in(test_partition.t1.b, 1, 6))", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "IndexPlan": [ - "IndexReader 29.98 root partition:p0,p1 index:Selection", + "IndexReader 29.98 root partition:all index:Selection", "└─Selection 29.98 cop[tikv] or(eq(test_partition_1.t1.a, 100), in(test_partition_1.t1.b, 1, 6))", " └─IndexFullScan 10000.00 cop[tikv] table:t1, index:a(a, b, id) keep order:false, stats:pseudo" ] @@ -796,12 +796,12 @@ "6 6 6" ], "Plan": [ - "TableReader 39.97 root partition:p0,p1 data:Selection", + "TableReader 39.97 root partition:all data:Selection", "└─Selection 39.97 cop[tikv] or(in(test_partition.t1.a, 1, 6), or(in(test_partition.t1.b, 1, 2), and(eq(test_partition.t1.a, 3), eq(test_partition.t1.b, 3))))", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "IndexPlan": [ - "IndexReader 40.06 root partition:p0,p1 index:Selection", + "IndexReader 40.06 root partition:all index:Selection", "└─Selection 40.06 cop[tikv] or(in(test_partition_1.t1.a, 1, 6), or(in(test_partition_1.t1.b, 1, 2), and(eq(test_partition_1.t1.a, 3), eq(test_partition_1.t1.b, 3))))", " └─IndexFullScan 10000.00 cop[tikv] table:t1, index:a(a, b, id) keep order:false, stats:pseudo" ] @@ -813,12 +813,12 @@ "6 6 6" ], "Plan": [ - "TableReader 20.00 root partition:p0,p1 data:Selection", + "TableReader 20.00 root partition:all data:Selection", "└─Selection 20.00 cop[tikv] in(test_partition.t1.a, 1, 6)", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "IndexPlan": [ - "IndexReader 20.00 root partition:p0,p1 index:IndexRangeScan", + "IndexReader 20.00 root partition:all index:IndexRangeScan", "└─IndexRangeScan 20.00 cop[tikv] table:t1, index:a(a, b, id) range:[1,1], [6,6], keep order:false, stats:pseudo" ] }, @@ -830,12 +830,12 @@ "6 6 6" ], "Plan": [ - "TableReader 20.01 root partition:p0,p1 data:Selection", + "TableReader 20.01 root partition:all data:Selection", "└─Selection 20.01 cop[tikv] or(in(test_partition.t1.a, 1, 6), and(eq(test_partition.t1.a, 3), eq(test_partition.t1.b, 3)))", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "IndexPlan": [ - "IndexReader 20.10 root partition:p0,p1 index:IndexRangeScan", + "IndexReader 20.10 root partition:all index:IndexRangeScan", "└─IndexRangeScan 20.10 cop[tikv] table:t1, index:a(a, b, id) range:[1,1], [3 3,3 3], [6,6], keep order:false, stats:pseudo" ] }, @@ -998,7 +998,7 @@ " │ └─HashAgg 15.98 cop[tikv] group by:test_partition.t2.b, ", " │ └─Selection 19.98 cop[tikv] in(test_partition.t2.a, 1, 2), not(isnull(test_partition.t2.b))", " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - " └─TableReader(Probe) 9990.00 root partition:p0,p1 data:Selection", + " └─TableReader(Probe) 9990.00 root partition:all data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test_partition.t1.a))", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], @@ -1009,7 +1009,7 @@ " │ └─IndexReader 159.84 root partition:p0 index:HashAgg", " │ └─HashAgg 159.84 cop[tikv] group by:test_partition_1.t2.b, ", " │ └─IndexRangeScan 199.80 cop[tikv] table:t2, index:a(a, b, id) range:[1 -inf,1 +inf], [2 -inf,2 +inf], keep order:false, stats:pseudo", - " └─IndexReader(Probe) 199.80 root partition:p0,p1 index:Selection", + " └─IndexReader(Probe) 199.80 root partition:all index:Selection", " └─Selection 199.80 cop[tikv] not(isnull(test_partition_1.t1.a))", " └─IndexRangeScan 200.00 cop[tikv] table:t1, index:a(a, b, id) range: decided by [eq(test_partition_1.t1.a, test_partition_1.t2.b)], keep order:false, stats:pseudo" ] @@ -1028,7 +1028,7 @@ " │ └─HashAgg 15.98 cop[tikv] group by:test_partition.t1.b, ", " │ └─Selection 19.98 cop[tikv] in(test_partition.t1.a, 1, 2), not(isnull(test_partition.t1.b))", " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - " └─TableReader(Probe) 9990.00 root partition:p0,p1 data:Selection", + " └─TableReader(Probe) 9990.00 root partition:all data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test_partition.t1.a))", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], @@ -1039,7 +1039,7 @@ " │ └─IndexReader 159.84 root partition:p0 index:HashAgg", " │ └─HashAgg 159.84 cop[tikv] group by:test_partition_1.t1.b, ", " │ └─IndexRangeScan 199.80 cop[tikv] table:t1, index:a(a, b, id) range:[1 -inf,1 +inf], [2 -inf,2 +inf], keep order:false, stats:pseudo", - " └─IndexReader(Probe) 199.80 root partition:p0,p1 index:Selection", + " └─IndexReader(Probe) 199.80 root partition:all index:Selection", " └─Selection 199.80 cop[tikv] not(isnull(test_partition_1.t1.a))", " └─IndexRangeScan 200.00 cop[tikv] table:t1, index:a(a, b, id) range: decided by [eq(test_partition_1.t1.a, test_partition_1.t1.b)], keep order:false, stats:pseudo" ] @@ -1123,7 +1123,7 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(1)->Column#9", "└─HashJoin 0.00 root inner join, equal:[eq(test_partition.t1.b, test_partition.t2.b)]", - " ├─TableReader(Build) 0.04 root partition:p0,p1 data:Selection", + " ├─TableReader(Build) 0.04 root partition:all data:Selection", " │ └─Selection 0.04 cop[tikv] in(test_partition.t2.a, 1, 6), in(test_partition.t2.b, 1, 6), not(isnull(test_partition.t2.b))", " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", " └─TableReader(Probe) 0.04 root partition:p0 data:Selection", @@ -1133,7 +1133,7 @@ "IndexPlan": [ "StreamAgg 1.00 root funcs:count(1)->Column#9", "└─HashJoin 0.10 root inner join, equal:[eq(test_partition_1.t1.b, test_partition_1.t2.b)]", - " ├─IndexReader(Build) 0.40 root partition:p0,p1 index:Selection", + " ├─IndexReader(Build) 0.40 root partition:all index:Selection", " │ └─Selection 0.40 cop[tikv] not(isnull(test_partition_1.t2.b))", " │ └─IndexRangeScan 0.40 cop[tikv] table:t2, index:a(a, b, id) range:[1 1,1 1], [1 6,1 6], [6 1,6 1], [6 6,6 6], keep order:false, stats:pseudo", " └─IndexReader(Probe) 0.40 root partition:p0 index:Selection", @@ -1149,7 +1149,7 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(1)->Column#9", "└─HashJoin 0.00 root inner join, equal:[eq(test_partition.t2.b, test_partition.t1.b)]", - " ├─TableReader(Build) 0.04 root partition:p0,p1 data:Selection", + " ├─TableReader(Build) 0.04 root partition:all data:Selection", " │ └─Selection 0.04 cop[tikv] in(test_partition.t1.a, 1, 6), in(test_partition.t1.b, 1, 6), not(isnull(test_partition.t1.b))", " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", " └─TableReader(Probe) 0.04 root partition:p0 data:Selection", @@ -1162,7 +1162,7 @@ " ├─IndexReader(Build) 0.40 root partition:p0 index:Selection", " │ └─Selection 0.40 cop[tikv] not(isnull(test_partition_1.t2.b))", " │ └─IndexRangeScan 0.40 cop[tikv] table:t2, index:a(a, b, id) range:[1 1,1 1], [1 6,1 6], [2 1,2 1], [2 6,2 6], keep order:false, stats:pseudo", - " └─IndexReader(Probe) 0.13 root partition:p0,p1 index:Selection", + " └─IndexReader(Probe) 0.13 root partition:all index:Selection", " └─Selection 0.13 cop[tikv] in(test_partition_1.t1.b, 1, 6), not(isnull(test_partition_1.t1.b))", " └─IndexRangeScan 63.94 cop[tikv] table:t1, index:a(a, b, id) range: decided by [eq(test_partition_1.t1.b, test_partition_1.t2.b) in(test_partition_1.t1.a, 1, 6)], keep order:false, stats:pseudo" ] @@ -1175,7 +1175,7 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(1)->Column#9", "└─HashJoin 0.00 root inner join, equal:[eq(test_partition.t2.b, test_partition.t1.b)]", - " ├─TableReader(Build) 0.04 root partition:p0,p1 data:Selection", + " ├─TableReader(Build) 0.04 root partition:all data:Selection", " │ └─Selection 0.04 cop[tikv] in(test_partition.t1.a, 1, 6), in(test_partition.t1.b, 6, 1), not(isnull(test_partition.t1.b))", " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", " └─TableReader(Probe) 0.04 root partition:p0 data:Selection", @@ -1188,7 +1188,7 @@ " ├─IndexReader(Build) 0.40 root partition:p0 index:Selection", " │ └─Selection 0.40 cop[tikv] not(isnull(test_partition_1.t2.b))", " │ └─IndexRangeScan 0.40 cop[tikv] table:t2, index:a(a, b, id) range:[1 1,1 1], [1 6,1 6], [2 1,2 1], [2 6,2 6], keep order:false, stats:pseudo", - " └─IndexReader(Probe) 0.13 root partition:p0,p1 index:Selection", + " └─IndexReader(Probe) 0.13 root partition:all index:Selection", " └─Selection 0.13 cop[tikv] in(test_partition_1.t1.b, 6, 1), not(isnull(test_partition_1.t1.b))", " └─IndexRangeScan 63.94 cop[tikv] table:t1, index:a(a, b, id) range: decided by [eq(test_partition_1.t1.b, test_partition_1.t2.b) in(test_partition_1.t1.a, 1, 6)], keep order:false, stats:pseudo" ] @@ -1227,7 +1227,7 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(1)->Column#9", "└─HashJoin 0.01 root inner join, equal:[eq(test_partition.t2.b, test_partition.t1.b)]", - " ├─TableReader(Build) 0.08 root partition:p0,p1 data:Selection", + " ├─TableReader(Build) 0.08 root partition:all data:Selection", " │ └─Selection 0.08 cop[tikv] in(test_partition.t1.a, 1, 6), in(test_partition.t1.b, 100, 9, 6, 1), not(isnull(test_partition.t1.b))", " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", " └─TableReader(Probe) 0.08 root partition:p0 data:Selection", @@ -1240,7 +1240,7 @@ " ├─IndexReader(Build) 0.80 root partition:p0 index:Selection", " │ └─Selection 0.80 cop[tikv] not(isnull(test_partition_1.t2.b))", " │ └─IndexRangeScan 0.80 cop[tikv] table:t2, index:a(a, b, id) range:[1 1,1 1], [1 6,1 6], [1 9,1 9], [1 100,1 100], [2 1,2 1], [2 6,2 6], [2 9,2 9], [2 100,2 100], keep order:false, stats:pseudo", - " └─IndexReader(Probe) 0.51 root partition:p0,p1 index:Selection", + " └─IndexReader(Probe) 0.51 root partition:all index:Selection", " └─Selection 0.51 cop[tikv] in(test_partition_1.t1.b, 100, 9, 6, 1), not(isnull(test_partition_1.t1.b))", " └─IndexRangeScan 127.87 cop[tikv] table:t1, index:a(a, b, id) range: decided by [eq(test_partition_1.t1.b, test_partition_1.t2.b) in(test_partition_1.t1.a, 1, 6)], keep order:false, stats:pseudo" ] @@ -1340,12 +1340,12 @@ "9 9 9" ], "Plan": [ - "TableReader 5548.89 root partition:p0,p1 data:Selection", + "TableReader 5548.89 root partition:all data:Selection", "└─Selection 5548.89 cop[tikv] or(lt(test_partition.t1.a, 3), gt(test_partition.t1.b, 4))", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "IndexPlan": [ - "IndexReader 5548.89 root partition:p0,p1 index:Selection", + "IndexReader 5548.89 root partition:all index:Selection", "└─Selection 5548.89 cop[tikv] or(lt(test_partition_1.t1.a, 3), gt(test_partition_1.t1.b, 4))", " └─IndexFullScan 10000.00 cop[tikv] table:t1, index:a(a, b, id) keep order:false, stats:pseudo" ] @@ -1424,12 +1424,12 @@ "9 9 9" ], "Plan": [ - "TableReader 2092.85 root partition:p0,p1 data:Selection", + "TableReader 2092.85 root partition:all data:Selection", "└─Selection 2092.85 cop[tikv] or(and(le(test_partition.t1.a, 1), le(test_partition.t1.b, 1)), and(ge(test_partition.t1.a, 6), ge(test_partition.t1.b, 6)))", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "IndexPlan": [ - "IndexReader 5325.33 root partition:p0,p1 index:Selection", + "IndexReader 5325.33 root partition:all index:Selection", "└─Selection 5325.33 cop[tikv] or(and(le(test_partition_1.t1.a, 1), le(test_partition_1.t1.b, 1)), and(ge(test_partition_1.t1.a, 6), ge(test_partition_1.t1.b, 6)))", " └─IndexRangeScan 6656.67 cop[tikv] table:t1, index:a(a, b, id) range:[-inf,1], [6,+inf], keep order:false, stats:pseudo" ] @@ -1449,12 +1449,12 @@ "9 9 9" ], "Plan": [ - "TableReader 1104.45 root partition:p0,p1 data:Selection", + "TableReader 1104.45 root partition:all data:Selection", "└─Selection 1104.45 cop[tikv] le(test_partition.t1.a, 100), le(test_partition.t1.b, 100)", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "IndexPlan": [ - "IndexReader 1104.45 root partition:p0,p1 index:Selection", + "IndexReader 1104.45 root partition:all index:Selection", "└─Selection 1104.45 cop[tikv] le(test_partition_1.t1.b, 100)", " └─IndexRangeScan 3323.33 cop[tikv] table:t1, index:a(a, b, id) range:[-inf,100], keep order:false, stats:pseudo" ] @@ -1603,12 +1603,12 @@ " 10 " ], "Plan": [ - "TableReader 3325.55 root partition:p0,p1 data:Selection", + "TableReader 3325.55 root partition:all data:Selection", "└─Selection 3325.55 cop[tikv] or(le(test_partition.t1.a, 1), and(le(test_partition.t1.a, 10), isnull(test_partition.t1.b)))", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "IndexPlan": [ - "IndexReader 2658.67 root partition:p0,p1 index:Selection", + "IndexReader 2658.67 root partition:all index:Selection", "└─Selection 2658.67 cop[tikv] or(le(test_partition_1.t1.a, 1), and(le(test_partition_1.t1.a, 10), isnull(test_partition_1.t1.b)))", " └─IndexRangeScan 3323.33 cop[tikv] table:t1, index:a(a, b, id) range:[-inf,10], keep order:false, stats:pseudo" ] @@ -1627,12 +1627,12 @@ "9 9 9" ], "Plan": [ - "TableReader 5542.21 root partition:p0,p1 data:Selection", + "TableReader 5542.21 root partition:all data:Selection", "└─Selection 5542.21 cop[tikv] or(le(test_partition.t1.a, 8), le(test_partition.t1.b, 9))", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "IndexPlan": [ - "IndexReader 5542.21 root partition:p0,p1 index:Selection", + "IndexReader 5542.21 root partition:all index:Selection", "└─Selection 5542.21 cop[tikv] or(le(test_partition_1.t1.a, 8), le(test_partition_1.t1.b, 9))", " └─IndexFullScan 10000.00 cop[tikv] table:t1, index:a(a, b, id) keep order:false, stats:pseudo" ] @@ -1752,12 +1752,12 @@ "6 6 6" ], "Plan": [ - "TableReader 259.75 root partition:p0,p1 data:Selection", + "TableReader 259.75 root partition:all data:Selection", "└─Selection 259.75 cop[tikv] or(eq(test_partition.t1.a, 100), and(ge(test_partition.t1.b, 1), le(test_partition.t1.b, 6)))", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "IndexPlan": [ - "IndexReader 259.75 root partition:p0,p1 index:Selection", + "IndexReader 259.75 root partition:all index:Selection", "└─Selection 259.75 cop[tikv] or(eq(test_partition_1.t1.a, 100), and(ge(test_partition_1.t1.b, 1), le(test_partition_1.t1.b, 6)))", " └─IndexFullScan 10000.00 cop[tikv] table:t1, index:a(a, b, id) keep order:false, stats:pseudo" ] @@ -1787,12 +1787,12 @@ "6 6 6" ], "Plan": [ - "TableReader 1543.67 root partition:p0,p1 data:Selection", + "TableReader 1543.67 root partition:all data:Selection", "└─Selection 1543.67 cop[tikv] or(and(ge(test_partition.t1.a, 1), le(test_partition.t1.a, 6)), or(and(ge(test_partition.t1.b, 1), le(test_partition.t1.b, 2)), and(le(test_partition.t1.a, 3), le(test_partition.t1.b, 3))))", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "IndexPlan": [ - "IndexReader 1543.67 root partition:p0,p1 index:Selection", + "IndexReader 1543.67 root partition:all index:Selection", "└─Selection 1543.67 cop[tikv] or(and(ge(test_partition_1.t1.a, 1), le(test_partition_1.t1.a, 6)), or(and(ge(test_partition_1.t1.b, 1), le(test_partition_1.t1.b, 2)), and(le(test_partition_1.t1.a, 3), le(test_partition_1.t1.b, 3))))", " └─IndexFullScan 10000.00 cop[tikv] table:t1, index:a(a, b, id) keep order:false, stats:pseudo" ] @@ -1808,12 +1808,12 @@ "6 6 6" ], "Plan": [ - "TableReader 250.00 root partition:p0,p1 data:Selection", + "TableReader 250.00 root partition:all data:Selection", "└─Selection 250.00 cop[tikv] ge(test_partition.t1.a, 1), le(test_partition.t1.a, 6)", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "IndexPlan": [ - "IndexReader 250.00 root partition:p0,p1 index:IndexRangeScan", + "IndexReader 250.00 root partition:all index:IndexRangeScan", "└─IndexRangeScan 250.00 cop[tikv] table:t1, index:a(a, b, id) range:[1,6], keep order:false, stats:pseudo" ] }, @@ -1832,12 +1832,12 @@ "9 9 9" ], "Plan": [ - "TableReader 1333.33 root partition:p0,p1 data:Selection", + "TableReader 1333.33 root partition:all data:Selection", "└─Selection 1333.33 cop[tikv] or(and(ge(test_partition.t1.a, 1), le(test_partition.t1.a, 6)), and(ge(test_partition.t1.a, 3), ge(test_partition.t1.b, 3)))", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "IndexPlan": [ - "IndexReader 2666.67 root partition:p0,p1 index:Selection", + "IndexReader 2666.67 root partition:all index:Selection", "└─Selection 2666.67 cop[tikv] or(and(ge(test_partition_1.t1.a, 1), le(test_partition_1.t1.a, 6)), and(ge(test_partition_1.t1.a, 3), ge(test_partition_1.t1.b, 3)))", " └─IndexRangeScan 3333.33 cop[tikv] table:t1, index:a(a, b, id) range:[1,+inf], keep order:false, stats:pseudo" ] @@ -1956,7 +1956,7 @@ "Plan": [ "Projection 1379.19 root test_partition.t1.id, test_partition.t1.a, test_partition.t1.b, test_partition.t2.id, test_partition.t2.a, test_partition.t2.b", "└─HashJoin 1379.19 root inner join, equal:[eq(test_partition.t2.id, test_partition.t1.id)]", - " ├─TableReader(Build) 1103.35 root partition:p0,p1 data:Selection", + " ├─TableReader(Build) 1103.35 root partition:all data:Selection", " │ └─Selection 1103.35 cop[tikv] le(test_partition.t2.a, 6), le(test_partition.t2.b, 6), not(isnull(test_partition.t2.id))", " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", " └─TableReader(Probe) 3320.01 root partition:p0 data:Selection", @@ -1966,7 +1966,7 @@ "IndexPlan": [ "Projection 1379.19 root test_partition_1.t1.id, test_partition_1.t1.a, test_partition_1.t1.b, test_partition_1.t2.id, test_partition_1.t2.a, test_partition_1.t2.b", "└─HashJoin 1379.19 root inner join, equal:[eq(test_partition_1.t2.id, test_partition_1.t1.id)]", - " ├─IndexReader(Build) 1103.35 root partition:p0,p1 index:Selection", + " ├─IndexReader(Build) 1103.35 root partition:all index:Selection", " │ └─Selection 1103.35 cop[tikv] le(test_partition_1.t2.b, 6), not(isnull(test_partition_1.t2.id))", " │ └─IndexRangeScan 3323.33 cop[tikv] table:t2, index:a(a, b, id) range:[-inf,6], keep order:false, stats:pseudo", " └─IndexReader(Probe) 3320.01 root partition:p0 index:Selection", @@ -1984,7 +1984,7 @@ "Plan": [ "Projection 1379.19 root test_partition.t1.id, test_partition.t1.a, test_partition.t1.b, test_partition.t1.id, test_partition.t1.a, test_partition.t1.b", "└─HashJoin 1379.19 root inner join, equal:[eq(test_partition.t1.id, test_partition.t1.id)]", - " ├─TableReader(Build) 1103.35 root partition:p0,p1 data:Selection", + " ├─TableReader(Build) 1103.35 root partition:all data:Selection", " │ └─Selection 1103.35 cop[tikv] le(test_partition.t1.a, 6), le(test_partition.t1.b, 6), not(isnull(test_partition.t1.id))", " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", " └─TableReader(Probe) 3320.01 root partition:p0 data:Selection", @@ -1994,7 +1994,7 @@ "IndexPlan": [ "Projection 1379.19 root test_partition_1.t1.id, test_partition_1.t1.a, test_partition_1.t1.b, test_partition_1.t1.id, test_partition_1.t1.a, test_partition_1.t1.b", "└─HashJoin 1379.19 root inner join, equal:[eq(test_partition_1.t1.id, test_partition_1.t1.id)]", - " ├─IndexReader(Build) 1103.35 root partition:p0,p1 index:Selection", + " ├─IndexReader(Build) 1103.35 root partition:all index:Selection", " │ └─Selection 1103.35 cop[tikv] le(test_partition_1.t1.b, 6), not(isnull(test_partition_1.t1.id))", " │ └─IndexRangeScan 3323.33 cop[tikv] table:t2, index:a(a, b, id) range:[-inf,6], keep order:false, stats:pseudo", " └─IndexReader(Probe) 3320.01 root partition:p0 index:Selection", @@ -2016,7 +2016,7 @@ " │ └─HashAgg 199.80 cop[tikv] group by:test_partition.t2.b, ", " │ └─Selection 249.75 cop[tikv] ge(test_partition.t2.a, 1), le(test_partition.t2.a, 2), not(isnull(test_partition.t2.b))", " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - " └─TableReader(Probe) 9990.00 root partition:p0,p1 data:Selection", + " └─TableReader(Probe) 9990.00 root partition:all data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test_partition.t1.a))", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], @@ -2028,7 +2028,7 @@ " │ └─HashAgg 199.80 cop[tikv] group by:test_partition_1.t2.b, ", " │ └─Selection 249.75 cop[tikv] not(isnull(test_partition_1.t2.b))", " │ └─IndexRangeScan 250.00 cop[tikv] table:t2, index:a(a, b, id) range:[1,2], keep order:false, stats:pseudo", - " └─IndexReader(Probe) 249.75 root partition:p0,p1 index:Selection", + " └─IndexReader(Probe) 249.75 root partition:all index:Selection", " └─Selection 249.75 cop[tikv] not(isnull(test_partition_1.t1.a))", " └─IndexRangeScan 250.00 cop[tikv] table:t1, index:a(a, b, id) range: decided by [eq(test_partition_1.t1.a, test_partition_1.t2.b)], keep order:false, stats:pseudo" ] @@ -2047,7 +2047,7 @@ " │ └─HashAgg 199.80 cop[tikv] group by:test_partition.t1.b, ", " │ └─Selection 249.75 cop[tikv] ge(test_partition.t1.a, 1), le(test_partition.t1.a, 2), not(isnull(test_partition.t1.b))", " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - " └─TableReader(Probe) 9990.00 root partition:p0,p1 data:Selection", + " └─TableReader(Probe) 9990.00 root partition:all data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test_partition.t1.a))", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], @@ -2059,7 +2059,7 @@ " │ └─HashAgg 199.80 cop[tikv] group by:test_partition_1.t1.b, ", " │ └─Selection 249.75 cop[tikv] not(isnull(test_partition_1.t1.b))", " │ └─IndexRangeScan 250.00 cop[tikv] table:t1, index:a(a, b, id) range:[1,2], keep order:false, stats:pseudo", - " └─IndexReader(Probe) 249.75 root partition:p0,p1 index:Selection", + " └─IndexReader(Probe) 249.75 root partition:all index:Selection", " └─Selection 249.75 cop[tikv] not(isnull(test_partition_1.t1.a))", " └─IndexRangeScan 250.00 cop[tikv] table:t1, index:a(a, b, id) range: decided by [eq(test_partition_1.t1.a, test_partition_1.t1.b)], keep order:false, stats:pseudo" ] @@ -2149,7 +2149,7 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(1)->Column#9", "└─HashJoin 7.81 root inner join, equal:[eq(test_partition.t1.b, test_partition.t2.b)]", - " ├─TableReader(Build) 6.25 root partition:p0,p1 data:Selection", + " ├─TableReader(Build) 6.25 root partition:all data:Selection", " │ └─Selection 6.25 cop[tikv] ge(test_partition.t2.a, 1), ge(test_partition.t2.b, 1), le(test_partition.t2.a, 6), le(test_partition.t2.b, 6), not(isnull(test_partition.t2.b))", " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", " └─TableReader(Probe) 6.25 root partition:p0 data:Selection", @@ -2159,7 +2159,7 @@ "IndexPlan": [ "StreamAgg 1.00 root funcs:count(1)->Column#9", "└─HashJoin 7.81 root inner join, equal:[eq(test_partition_1.t1.b, test_partition_1.t2.b)]", - " ├─IndexReader(Build) 6.25 root partition:p0,p1 index:Selection", + " ├─IndexReader(Build) 6.25 root partition:all index:Selection", " │ └─Selection 6.25 cop[tikv] ge(test_partition_1.t2.b, 1), le(test_partition_1.t2.b, 6), not(isnull(test_partition_1.t2.b))", " │ └─IndexRangeScan 250.00 cop[tikv] table:t2, index:a(a, b, id) range:[1,6], keep order:false, stats:pseudo", " └─IndexReader(Probe) 6.25 root partition:p0 index:Selection", @@ -2175,7 +2175,7 @@ "Plan": [ "StreamAgg 1.00 root funcs:count(1)->Column#9", "└─HashJoin 7.81 root inner join, equal:[eq(test_partition.t2.b, test_partition.t1.b)]", - " ├─TableReader(Build) 6.25 root partition:p0,p1 data:Selection", + " ├─TableReader(Build) 6.25 root partition:all data:Selection", " │ └─Selection 6.25 cop[tikv] ge(test_partition.t1.a, 1), ge(test_partition.t1.b, 1), le(test_partition.t1.a, 6), le(test_partition.t1.b, 6), not(isnull(test_partition.t1.b))", " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", " └─TableReader(Probe) 6.25 root partition:p0 data:Selection", @@ -2185,7 +2185,7 @@ "IndexPlan": [ "StreamAgg 1.00 root funcs:count(1)->Column#9", "└─HashJoin 7.81 root inner join, equal:[eq(test_partition_1.t2.b, test_partition_1.t1.b)]", - " ├─IndexReader(Build) 6.25 root partition:p0,p1 index:Selection", + " ├─IndexReader(Build) 6.25 root partition:all index:Selection", " │ └─Selection 6.25 cop[tikv] ge(test_partition_1.t1.b, 1), le(test_partition_1.t1.b, 6), not(isnull(test_partition_1.t1.b))", " │ └─IndexRangeScan 250.00 cop[tikv] table:t1, index:a(a, b, id) range:[1,6], keep order:false, stats:pseudo", " └─IndexReader(Probe) 6.25 root partition:p0 index:Selection", diff --git a/pkg/planner/core/partition_prune.go b/pkg/planner/core/partition_prune.go index 29defea381f68..32122a2f9c473 100644 --- a/pkg/planner/core/partition_prune.go +++ b/pkg/planner/core/partition_prune.go @@ -16,6 +16,7 @@ package core import ( "github.com/pingcap/tidb/pkg/expression" + tmodel "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/table" @@ -38,9 +39,57 @@ func PartitionPruning(ctx sessionctx.Context, tbl table.PartitionedTable, conds return nil, err } ret := s.convertToIntSlice(rangeOr, pi, partitionNames) + ret = handleDroppingForRange(pi, partitionNames, ret) return ret, nil case model.PartitionTypeList: return s.pruneListPartition(ctx, tbl, partitionNames, conds, columns) } return []int{FullRange}, nil } + +func handleDroppingForRange(pi *tmodel.PartitionInfo, partitionNames []model.CIStr, usedPartitions []int) []int { + if pi.CanHaveOverlappingDroppingPartition() { + if len(usedPartitions) == 1 && usedPartitions[0] == FullRange { + usedPartitions = make([]int, 0, len(pi.Definitions)) + for i := range pi.Definitions { + usedPartitions = append(usedPartitions, i) + } + } + ret := make([]int, 0, len(usedPartitions)) + for i := range usedPartitions { + idx := pi.GetOverlappingDroppingPartitionIdx(usedPartitions[i]) + if idx == -1 { + // dropped without overlapping partition, skip it + continue + } + if idx == usedPartitions[i] { + // non-dropped partition + ret = append(ret, idx) + continue + } + // partition being dropped, remove the consecutive range of dropping partitions + // and add the overlapping partition. + end := i + 1 + for ; end < len(usedPartitions) && usedPartitions[end] < idx; end++ { + continue + } + // add the overlapping partition, if not already included + if end >= len(usedPartitions) || usedPartitions[end] != idx { + // It must also match partitionNames if explicitly given + s := PartitionProcessor{} + if len(partitionNames) == 0 || s.findByName(partitionNames, pi.Definitions[idx].Name.L) { + ret = append(ret, idx) + } + } + if end < len(usedPartitions) { + ret = append(ret, usedPartitions[end:]...) + } + break + } + usedPartitions = ret + } + if len(usedPartitions) == len(pi.Definitions) { + return []int{FullRange} + } + return usedPartitions +} diff --git a/pkg/planner/core/point_get_plan.go b/pkg/planner/core/point_get_plan.go index 546c97340e342..2b0831d8e2b24 100644 --- a/pkg/planner/core/point_get_plan.go +++ b/pkg/planner/core/point_get_plan.go @@ -306,6 +306,96 @@ func (p *PointGetPlan) MemoryUsage() (sum int64) { return } +<<<<<<< HEAD +======= +// LoadTableStats preloads the stats data for the physical table +func (p *PointGetPlan) LoadTableStats(ctx sessionctx.Context) { + tableID := p.TblInfo.ID + if idx := p.PartitionIdx; idx != nil { + if *idx < 0 { + // No matching partitions + return + } + if pi := p.TblInfo.GetPartitionInfo(); pi != nil { + tableID = pi.Definitions[*idx].ID + } + } + loadTableStats(ctx, p.TblInfo, tableID) +} + +// PrunePartitions will check which partition to use +// returns true if no matching partition +func (p *PointGetPlan) PrunePartitions(sctx sessionctx.Context) bool { + pi := p.TblInfo.GetPartitionInfo() + if pi == nil { + return false + } + if p.IndexInfo != nil && p.IndexInfo.Global { + // reading for the Global Index / table id + return false + } + // _tidb_rowid + specify a partition + if p.IndexInfo == nil && !p.TblInfo.HasClusteredIndex() && len(p.PartitionNames) == 1 { + for i, def := range pi.Definitions { + if def.Name.L == p.PartitionNames[0].L { + idx := i + p.PartitionIdx = &idx + break + } + } + return false + } + // If tryPointGetPlan did generate the plan, + // then PartitionIdx is not set and needs to be set here! + // There are two ways to get here from static mode partition pruning: + // 1) Converting a set of partitions into a Union scan + // - This should NOT be cached and should already be having PartitionIdx set! + // 2) Converted to PointGet from checkTblIndexForPointPlan + // and it does not have the PartitionIdx set + if !p.SCtx().GetSessionVars().StmtCtx.UseCache() && + p.PartitionIdx != nil { + return false + } + is := sessiontxn.GetTxnManager(sctx).GetTxnInfoSchema() + tbl, ok := is.TableByID(context.Background(), p.TblInfo.ID) + if tbl == nil || !ok { + // Can this happen? + intest.Assert(false) + return false + } + pt := tbl.GetPartitionedTable() + if pt == nil { + // Can this happen? + intest.Assert(false) + return false + } + row := make([]types.Datum, len(p.TblInfo.Columns)) + if p.HandleConstant == nil && len(p.IndexValues) > 0 { + for i := range p.IndexInfo.Columns { + // TODO: Skip copying non-partitioning columns? + p.IndexValues[i].Copy(&row[p.IndexInfo.Columns[i].Offset]) + } + } else { + var dVal types.Datum + if p.UnsignedHandle { + dVal = types.NewUintDatum(uint64(p.Handle.IntValue())) + } else { + dVal = types.NewIntDatum(p.Handle.IntValue()) + } + dVal.Copy(&row[p.HandleColOffset]) + } + partIdx, err := pt.GetPartitionIdxByRow(sctx.GetExprCtx().GetEvalCtx(), row) + partIdx, err = pt.Meta().Partition.ReplaceWithOverlappingPartitionIdx(partIdx, err) + if err != nil || !isInExplicitPartitions(pi, partIdx, p.PartitionNames) { + partIdx = -1 + p.PartitionIdx = &partIdx + return true + } + p.PartitionIdx = &partIdx + return false +} + +>>>>>>> 1e24d396082 (*: Drop partition DDL handling for overlapping partitions during State Changes (#56082)) // BatchPointGetPlan represents a physical plan which contains a bunch of // keys reference the same table and use the same `unique key` type BatchPointGetPlan struct { @@ -526,6 +616,238 @@ func (p *BatchPointGetPlan) MemoryUsage() (sum int64) { return } +<<<<<<< HEAD +======= +// LoadTableStats preloads the stats data for the physical table +func (p *BatchPointGetPlan) LoadTableStats(ctx sessionctx.Context) { + // as a `BatchPointGet` can access multiple partitions, and we cannot distinguish how many rows come from each + // partitions in the existing statistics information, we treat all index usage through a `BatchPointGet` just + // like a normal global index. + loadTableStats(ctx, p.TblInfo, p.TblInfo.ID) +} + +func isInExplicitPartitions(pi *model.PartitionInfo, idx int, names []pmodel.CIStr) bool { + if len(names) == 0 { + return true + } + s := pi.Definitions[idx].Name.L + for _, name := range names { + if s == name.L { + return true + } + } + return false +} + +// Map each index value to Partition ID +func (p *BatchPointGetPlan) getPartitionIdxs(sctx sessionctx.Context) []int { + is := sessiontxn.GetTxnManager(sctx).GetTxnInfoSchema() + tbl, ok := is.TableByID(context.Background(), p.TblInfo.ID) + intest.Assert(ok) + pTbl, ok := tbl.(table.PartitionedTable) + intest.Assert(ok) + intest.Assert(pTbl != nil) + r := make([]types.Datum, len(pTbl.Cols())) + rows := p.IndexValues + idxs := make([]int, 0, len(rows)) + for i := range rows { + for j := range rows[i] { + rows[i][j].Copy(&r[p.IndexInfo.Columns[j].Offset]) + } + pIdx, err := pTbl.GetPartitionIdxByRow(sctx.GetExprCtx().GetEvalCtx(), r) + pIdx, err = pTbl.Meta().Partition.ReplaceWithOverlappingPartitionIdx(pIdx, err) + if err != nil { + // Skip on any error, like: + // No matching partition, overflow etc. + idxs = append(idxs, -1) + continue + } + idxs = append(idxs, pIdx) + } + return idxs +} + +// PrunePartitionsAndValues will check which partition to use +// returns: +// slice of non-duplicated handles (or nil if IndexValues is used) +// true if no matching partition (TableDual plan can be used) +func (p *BatchPointGetPlan) PrunePartitionsAndValues(sctx sessionctx.Context) ([]kv.Handle, bool) { + pi := p.TblInfo.GetPartitionInfo() + if p.IndexInfo != nil && p.IndexInfo.Global { + // Reading from a global index, i.e. base table ID + // Skip pruning partitions here + pi = nil + } + // reset the PartitionIDs + if pi != nil && !p.SinglePartition { + p.PartitionIdxs = p.PartitionIdxs[:0] + } + if p.IndexInfo != nil && !(p.TblInfo.IsCommonHandle && p.IndexInfo.Primary) { + filteredVals := p.IndexValues[:0] + for _, idxVals := range p.IndexValues { + // For all x, 'x IN (null)' evaluate to null, so the query get no result. + if !types.DatumsContainNull(idxVals) { + filteredVals = append(filteredVals, idxVals) + } + } + p.IndexValues = filteredVals + if pi != nil { + partIdxs := p.getPartitionIdxs(sctx) + partitionsFound := 0 + for i, idx := range partIdxs { + if idx < 0 || + (p.SinglePartition && + idx != p.PartitionIdxs[0]) || + !isInExplicitPartitions(pi, idx, p.PartitionNames) { + // Index value does not match any partitions, + // remove it from the plan + partIdxs[i] = -1 + } else { + partitionsFound++ + } + } + if partitionsFound == 0 { + return nil, true + } + skipped := 0 + for i, idx := range partIdxs { + if idx < 0 { + curr := i - skipped + next := curr + 1 + p.IndexValues = append(p.IndexValues[:curr], p.IndexValues[next:]...) + skipped++ + } else if !p.SinglePartition { + p.PartitionIdxs = append(p.PartitionIdxs, idx) + } + } + intest.Assert(p.SinglePartition || partitionsFound == len(p.PartitionIdxs)) + intest.Assert(partitionsFound == len(p.IndexValues)) + } + return nil, false + } + handles := make([]kv.Handle, 0, len(p.Handles)) + dedup := kv.NewHandleMap() + if p.IndexInfo == nil { + for _, handle := range p.Handles { + if _, found := dedup.Get(handle); found { + continue + } + dedup.Set(handle, true) + handles = append(handles, handle) + } + if pi != nil { + is := sessiontxn.GetTxnManager(sctx).GetTxnInfoSchema() + tbl, ok := is.TableByID(context.Background(), p.TblInfo.ID) + intest.Assert(ok) + pTbl, ok := tbl.(table.PartitionedTable) + intest.Assert(ok) + intest.Assert(pTbl != nil) + r := make([]types.Datum, p.HandleColOffset+1) + partIdxs := make([]int, 0, len(handles)) + partitionsFound := 0 + for _, handle := range handles { + var d types.Datum + if mysql.HasUnsignedFlag(p.TblInfo.Columns[p.HandleColOffset].GetFlag()) { + d = types.NewUintDatum(uint64(handle.IntValue())) + } else { + d = types.NewIntDatum(handle.IntValue()) + } + d.Copy(&r[p.HandleColOffset]) + pIdx, err := pTbl.GetPartitionIdxByRow(sctx.GetExprCtx().GetEvalCtx(), r) + pIdx, err = pi.ReplaceWithOverlappingPartitionIdx(pIdx, err) + if err != nil || + !isInExplicitPartitions(pi, pIdx, p.PartitionNames) || + (p.SinglePartition && + p.PartitionIdxs[0] != pIdx) { + { + pIdx = -1 + } + } else { + partitionsFound++ + } + partIdxs = append(partIdxs, pIdx) + } + if partitionsFound == 0 { + return nil, true + } + skipped := 0 + for i, idx := range partIdxs { + if idx < 0 { + curr := i - skipped + next := curr + 1 + handles = append(handles[:curr], handles[next:]...) + skipped++ + } else if !p.SinglePartition { + p.PartitionIdxs = append(p.PartitionIdxs, idx) + } + } + intest.Assert(p.SinglePartition || partitionsFound == len(p.PartitionIdxs)) + intest.Assert(p.SinglePartition || partitionsFound == len(handles)) + } + p.Handles = handles + } else { + usedValues := make([]bool, len(p.IndexValues)) + for i, value := range p.IndexValues { + if types.DatumsContainNull(value) { + continue + } + handleBytes, err := EncodeUniqueIndexValuesForKey(sctx, p.TblInfo, p.IndexInfo, value) + if err != nil { + if kv.ErrNotExist.Equal(err) { + continue + } + intest.Assert(false) + continue + } + handle, err := kv.NewCommonHandle(handleBytes) + if err != nil { + intest.Assert(false) + continue + } + if _, found := dedup.Get(handle); found { + continue + } + dedup.Set(handle, true) + handles = append(handles, handle) + usedValues[i] = true + } + skipped := 0 + for i, use := range usedValues { + if !use { + curr := i - skipped + p.IndexValues = append(p.IndexValues[:curr], p.IndexValues[curr+1:]...) + skipped++ + } + } + if pi != nil { + partIdxs := p.getPartitionIdxs(sctx) + skipped = 0 + partitionsFound := 0 + for i, idx := range partIdxs { + if partIdxs[i] < 0 || + (p.SinglePartition && + partIdxs[i] != p.PartitionIdxs[0]) || + !isInExplicitPartitions(pi, idx, p.PartitionNames) { + curr := i - skipped + handles = append(handles[:curr], handles[curr+1:]...) + p.IndexValues = append(p.IndexValues[:curr], p.IndexValues[curr+1:]...) + skipped++ + continue + } else if !p.SinglePartition { + p.PartitionIdxs = append(p.PartitionIdxs, idx) + } + partitionsFound++ + } + if partitionsFound == 0 { + return nil, true + } + intest.Assert(p.SinglePartition || partitionsFound == len(p.PartitionIdxs)) + } + } + return handles, false +} + +>>>>>>> 1e24d396082 (*: Drop partition DDL handling for overlapping partitions during State Changes (#56082)) // PointPlanKey is used to get point plan that is pre-built for multi-statement query. const PointPlanKey = stringutil.StringerStr("pointPlanKey") diff --git a/pkg/planner/core/rule_partition_processor.go b/pkg/planner/core/rule_partition_processor.go index 34811b32ffa3b..063287d134142 100644 --- a/pkg/planner/core/rule_partition_processor.go +++ b/pkg/planner/core/rule_partition_processor.go @@ -656,14 +656,27 @@ func (l *listPartitionPruner) locateColumnPartitionsByCondition(cond expression. } } for _, location := range locations { - if len(l.partitionNames) > 0 { - for _, pg := range location { + for _, pg := range location { + idx := l.pi.GetOverlappingDroppingPartitionIdx(pg.PartIdx) + if idx == -1 { + // Skip dropping partitions + continue + } + if idx != pg.PartIdx { + pg = tables.ListPartitionGroup{ + PartIdx: idx, + // TODO: Test this!!! + // How does it work with intersection for example? + GroupIdxs: []int{-1}, // Special group! + } + } + if len(l.partitionNames) > 0 { if l.findByName(l.partitionNames, l.pi.Definitions[pg.PartIdx].Name.L) { helper.UnionPartitionGroup(pg) } + } else { + helper.UnionPartitionGroup(pg) } - } else { - helper.Union(location) } } } @@ -727,6 +740,7 @@ func (l *listPartitionPruner) findUsedListPartitions(conds []expression.Expressi return nil, err } partitionIdx := l.listPrune.LocatePartition(value, isNull) + partitionIdx = l.pi.GetOverlappingDroppingPartitionIdx(partitionIdx) if partitionIdx == -1 { continue } @@ -755,8 +769,23 @@ func (s *partitionProcessor) findUsedListPartitions(ctx sessionctx.Context, tbl return nil, err } if _, ok := used[FullRange]; ok { - or := partitionRangeOR{partitionRange{0, len(pi.Definitions)}} - return s.convertToIntSlice(or, pi, partitionNames), nil + ret := make([]int, 0, len(pi.Definitions)) + for i := 0; i < len(pi.Definitions); i++ { + if len(partitionNames) > 0 && !listPruner.findByName(partitionNames, pi.Definitions[i].Name.L) { + continue + } + if i != pi.GetOverlappingDroppingPartitionIdx(i) { + continue + } + ret = append(ret, i) + } + if len(ret) == len(pi.Definitions) { + return []int{FullRange}, nil + } + return ret, nil + } + if len(used) == len(pi.Definitions) { + return []int{FullRange}, nil } ret := make([]int, 0, len(used)) for k := range used { @@ -1718,32 +1747,50 @@ func (s *partitionProcessor) makeUnionAllChildren(ds *DataSource, pi *model.Part usedDefinition := make(map[int64]model.PartitionDefinition) for _, r := range or { for i := r.start; i < r.end; i++ { + partIdx := pi.GetOverlappingDroppingPartitionIdx(i) + if partIdx < 0 { + continue + } + // This is for `table partition (p0,p1)` syntax, only union the specified partition if has specified partitions. +<<<<<<< HEAD if len(ds.partitionNames) != 0 { if !s.findByName(ds.partitionNames, pi.Definitions[i].Name.L) { +======= + if len(ds.PartitionNames) != 0 { + if !s.findByName(ds.PartitionNames, pi.Definitions[partIdx].Name.L) { +>>>>>>> 1e24d396082 (*: Drop partition DDL handling for overlapping partitions during State Changes (#56082)) continue } } + if _, found := usedDefinition[pi.Definitions[partIdx].ID]; found { + continue + } // Not a deep copy. newDataSource := *ds newDataSource.baseLogicalPlan = newBaseLogicalPlan(ds.SCtx(), plancodec.TypeTableScan, &newDataSource, ds.SelectBlockOffset()) newDataSource.schema = ds.schema.Clone() newDataSource.Columns = make([]*model.ColumnInfo, len(ds.Columns)) copy(newDataSource.Columns, ds.Columns) +<<<<<<< HEAD newDataSource.isPartition = true newDataSource.physicalTableID = pi.Definitions[i].ID +======= + newDataSource.PartitionDefIdx = &partIdx + newDataSource.PhysicalTableID = pi.Definitions[partIdx].ID +>>>>>>> 1e24d396082 (*: Drop partition DDL handling for overlapping partitions during State Changes (#56082)) // There are many expression nodes in the plan tree use the original datasource // id as FromID. So we set the id of the newDataSource with the original one to // avoid traversing the whole plan tree to update the references. newDataSource.SetID(ds.ID()) - err := s.resolveOptimizeHint(&newDataSource, pi.Definitions[i].Name) - partitionNameSet.Insert(pi.Definitions[i].Name.L) + err := s.resolveOptimizeHint(&newDataSource, pi.Definitions[partIdx].Name) + partitionNameSet.Insert(pi.Definitions[partIdx].Name.L) if err != nil { return nil, err } children = append(children, &newDataSource) - usedDefinition[pi.Definitions[i].ID] = pi.Definitions[i] + usedDefinition[pi.Definitions[partIdx].ID] = pi.Definitions[partIdx] } } s.checkHintsApplicable(ds, partitionNameSet) diff --git a/pkg/table/tables/partition.go b/pkg/table/tables/partition.go index a7af6b55b5239..554ccc7ae8986 100644 --- a/pkg/table/tables/partition.go +++ b/pkg/table/tables/partition.go @@ -138,6 +138,11 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part partitions[p.ID] = &t } ret.partitions = partitions + if pi.DDLAction != model.ActionReorganizePartition && + pi.DDLAction != model.ActionRemovePartitioning && + pi.DDLAction != model.ActionAlterTablePartitioning { + return ret, nil + } // In StateWriteReorganization we are using the 'old' partition definitions // and if any new change happens in DroppingDefinitions, it needs to be done // also in AddingDefinitions (with new evaluation of the new expression) @@ -1291,18 +1296,43 @@ func (t *partitionedTable) locatePartitionCommon(ctx sessionctx.Context, tp mode } else { idx, err = t.locateRangePartition(ctx, partitionExpr, r) } +<<<<<<< HEAD case model.PartitionTypeHash: +======= + if err != nil { + return -1, err + } + pi := t.Meta().Partition + if pi.CanHaveOverlappingDroppingPartition() { + if pi.IsDropping(idx) { + // Give an error, since it should not be written to! + // For read it can check the Overlapping partition and ignore the error. + // One should use the next non-dropping partition for range, or the default + // partition for list partitioned table with default partition, for read. + return idx, table.ErrNoPartitionForGivenValue.GenWithStackByArgs(fmt.Sprintf("matching a partition being dropped, '%s'", pi.Definitions[idx].Name.String())) + } + } + case pmodel.PartitionTypeHash: +>>>>>>> 1e24d396082 (*: Drop partition DDL handling for overlapping partitions during State Changes (#56082)) // Note that only LIST and RANGE supports REORGANIZE PARTITION idx, err = t.locateHashPartition(ctx, partitionExpr, num, r) case model.PartitionTypeKey: idx, err = partitionExpr.LocateKeyPartition(num, r) case model.PartitionTypeList: idx, err = partitionExpr.locateListPartition(ctx, r) +<<<<<<< HEAD case model.PartitionTypeNone: +======= + pi := t.Meta().Partition + if idx != pi.GetOverlappingDroppingPartitionIdx(idx) { + return idx, table.ErrNoPartitionForGivenValue.GenWithStackByArgs(fmt.Sprintf("matching a partition being dropped, '%s'", pi.Definitions[idx].Name.String())) + } + case pmodel.PartitionTypeNone: +>>>>>>> 1e24d396082 (*: Drop partition DDL handling for overlapping partitions during State Changes (#56082)) idx = 0 } if err != nil { - return 0, errors.Trace(err) + return -1, errors.Trace(err) } return idx, nil } @@ -1310,7 +1340,15 @@ func (t *partitionedTable) locatePartitionCommon(ctx sessionctx.Context, tp mode func (t *partitionedTable) locatePartition(ctx sessionctx.Context, r []types.Datum) (int64, error) { pi := t.Meta().GetPartitionInfo() columnsSet := len(t.meta.Partition.Columns) > 0 +<<<<<<< HEAD idx, err := t.locatePartitionCommon(ctx, pi.Type, t.partitionExpr, pi.Num, columnsSet, r) +======= + return t.locatePartitionCommon(ctx, pi.Type, t.partitionExpr, pi.Num, columnsSet, r) +} + +func (t *partitionedTable) locatePartition(ctx expression.EvalContext, r []types.Datum) (int64, error) { + idx, err := t.locatePartitionIdx(ctx, r) +>>>>>>> 1e24d396082 (*: Drop partition DDL handling for overlapping partitions during State Changes (#56082)) if err != nil { return 0, errors.Trace(err) } diff --git a/pkg/testkit/mockstore.go b/pkg/testkit/mockstore.go index b5cdb8726d7c7..f954c82dff1a4 100644 --- a/pkg/testkit/mockstore.go +++ b/pkg/testkit/mockstore.go @@ -169,6 +169,11 @@ func (d *DistExecutionContext) GetDomain(idx int) *domain.Domain { // NewDistExecutionContext create DistExecutionContext for testing. func NewDistExecutionContext(t testing.TB, serverNum int) *DistExecutionContext { + return NewDistExecutionContextWithLease(t, serverNum, 500*time.Millisecond) +} + +// NewDistExecutionContextWithLease create DistExecutionContext for testing. +func NewDistExecutionContextWithLease(t testing.TB, serverNum int, lease time.Duration) *DistExecutionContext { store, err := mockstore.NewMockStore() require.NoError(t, err) gctuner.GlobalMemoryLimitTuner.Stop() @@ -177,7 +182,7 @@ func NewDistExecutionContext(t testing.TB, serverNum int) *DistExecutionContext var domInfo []string for i := 0; i < serverNum; i++ { - dom := bootstrap4DistExecution(t, store, 500*time.Millisecond) + dom := bootstrap4DistExecution(t, store, lease) if i != serverNum-1 { dom.SetOnClose(func() { /* don't delete the store in domain map */ }) } diff --git a/tests/integrationtest/r/ddl/db_partition.result b/tests/integrationtest/r/ddl/db_partition.result index 1c37b2bcfeb5f..728d3ed5b4fa3 100644 --- a/tests/integrationtest/r/ddl/db_partition.result +++ b/tests/integrationtest/r/ddl/db_partition.result @@ -617,7 +617,7 @@ b a 4 4 explain format = 'brief' select * from t where a = 4; id estRows task access object operator info -TableReader 4.80 root partition:p0,p1,p2 data:Selection +TableReader 4.80 root partition:all data:Selection └─Selection 4.80 cop[tikv] eq(cast(ddl__db_partition.t.a, double BINARY), 4) └─TableFullScan 6.00 cop[tikv] table:t keep order:false select * from t where a = 3; @@ -625,7 +625,7 @@ b a 3 3 explain format = 'brief' select * from t where a = 3; id estRows task access object operator info -TableReader 4.80 root partition:p0,p1,p2 data:Selection +TableReader 4.80 root partition:all data:Selection └─Selection 4.80 cop[tikv] eq(cast(ddl__db_partition.t.a, double BINARY), 3) └─TableFullScan 6.00 cop[tikv] table:t keep order:false explain format = 'brief' select * from t where a = "3"; diff --git a/tests/integrationtest/r/planner/core/integration_partition.result b/tests/integrationtest/r/planner/core/integration_partition.result index 8110fb3f66387..65c31e35fe734 100644 --- a/tests/integrationtest/r/planner/core/integration_partition.result +++ b/tests/integrationtest/r/planner/core/integration_partition.result @@ -430,6 +430,7 @@ create table tcollist (a int, b int, key(a)) partition by list columns (a) (part alter table tcollist alter index a invisible; explain select a from tcollist where a>=0 and a<=5; id estRows task access object operator info +<<<<<<< HEAD PartitionUnion_8 500.00 root ├─TableReader_11 250.00 root data:Selection_10 │ └─Selection_10 250.00 cop[tikv] ge(list_partition_invisible_idx.tcollist.a, 0), le(list_partition_invisible_idx.tcollist.a, 5) @@ -437,6 +438,11 @@ PartitionUnion_8 500.00 root └─TableReader_14 250.00 root data:Selection_13 └─Selection_13 250.00 cop[tikv] ge(list_partition_invisible_idx.tcollist.a, 0), le(list_partition_invisible_idx.tcollist.a, 5) └─TableFullScan_12 10000.00 cop[tikv] table:tcollist, partition:p1 keep order:false, stats:pseudo +======= +TableReader_7 250.00 root partition:all data:Selection_6 +└─Selection_6 250.00 cop[tikv] ge(list_partition_invisible_idx.tcollist.a, 0), le(list_partition_invisible_idx.tcollist.a, 5) + └─TableFullScan_5 10000.00 cop[tikv] table:tcollist keep order:false, stats:pseudo +>>>>>>> 1e24d396082 (*: Drop partition DDL handling for overlapping partitions during State Changes (#56082)) create database list_partition_cte; use list_partition_cte; drop table if exists tlist; diff --git a/tests/integrationtest/r/planner/core/partition_pruner.result b/tests/integrationtest/r/planner/core/partition_pruner.result index 5cff3913e6afa..723d04981f591 100644 --- a/tests/integrationtest/r/planner/core/partition_pruner.result +++ b/tests/integrationtest/r/planner/core/partition_pruner.result @@ -3435,7 +3435,7 @@ a b 2 2 explain format='brief' select * from t where a in (1,2) and b in (1,2); id estRows task access object operator info -TableReader 3.43 root partition:p1,p2,pDef data:Selection +TableReader 3.43 root partition:all data:Selection └─Selection 3.43 cop[tikv] in(listdefaultprune.t.a, 1, 2), in(listdefaultprune.t.b, 1, 2) └─TableFullScan 7.00 cop[tikv] table:t keep order:false select * from t where a in (1) and b in (1); @@ -3466,7 +3466,7 @@ a b 2 2 explain format='brief' select * from t where a in (1,2) and b in (1,2); id estRows task access object operator info -TableReader 3.43 root partition:p1,p2,pDef data:Selection +TableReader 3.43 root partition:all data:Selection └─Selection 3.43 cop[tikv] in(listdefaultprune.t.a, 1, 2), in(listdefaultprune.t.b, 1, 2) └─TableFullScan 7.00 cop[tikv] table:t keep order:false drop table t;