From 26ec5b47abdd22e65c38496a024762d016573c7e Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Thu, 1 Aug 2024 13:36:10 +0800 Subject: [PATCH 01/13] change --- pkg/ddl/column.go | 24 ++ pkg/ddl/ddl.go | 49 ++- pkg/ddl/executor.go | 762 ++------------------------------------- pkg/ddl/index.go | 17 + pkg/ddl/modify_column.go | 618 +++++++++++++++++++++++++++++++ pkg/ddl/partition.go | 60 +++ 6 files changed, 789 insertions(+), 741 deletions(-) create mode 100644 pkg/ddl/modify_column.go diff --git a/pkg/ddl/column.go b/pkg/ddl/column.go index 7d81d99142d76..713af458c4328 100644 --- a/pkg/ddl/column.go +++ b/pkg/ddl/column.go @@ -358,6 +358,30 @@ func checkDropColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (*model.TableInfo, return tblInfo, colInfo, idxInfos, false, nil } +func isDroppableColumn(tblInfo *model.TableInfo, colName model.CIStr) error { + if ok, dep, isHidden := hasDependentByGeneratedColumn(tblInfo, colName); ok { + if isHidden { + return dbterror.ErrDependentByFunctionalIndex.GenWithStackByArgs(dep) + } + return dbterror.ErrDependentByGeneratedColumn.GenWithStackByArgs(dep) + } + + if len(tblInfo.Columns) == 1 { + return dbterror.ErrCantRemoveAllFields.GenWithStack("can't drop only column %s in table %s", + colName, tblInfo.Name) + } + // We only support dropping column with single-value none Primary Key index covered now. + err := isColumnCanDropWithIndex(colName.L, tblInfo.Indices) + if err != nil { + return err + } + err = IsColumnDroppableWithCheckConstraint(colName, tblInfo) + if err != nil { + return err + } + return nil +} + func onSetDefaultValue(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { newCol := &model.ColumnInfo{} err := job.DecodeArgs(newCol) diff --git a/pkg/ddl/ddl.go b/pkg/ddl/ddl.go index c0fed601acc67..3c4d3ea7235aa 100644 --- a/pkg/ddl/ddl.go +++ b/pkg/ddl/ddl.go @@ -250,9 +250,7 @@ type ddl struct { delRangeMgr delRangeManager enableTiFlashPoll *atomicutil.Bool // get notification if any DDL job submitted or finished. - ddlJobNotifyCh chan struct{} - sysTblMgr systable.Manager - minJobIDRefresher *systable.MinJobIDRefresher + ddlJobNotifyCh chan struct{} // globalIDLock locks global id to reduce write conflict. globalIDLock sync.Mutex @@ -542,10 +540,6 @@ func (dc *ddlCtx) notifyReorgWorkerJobStateChange(job *model.Job) { rc.notifyJobState(job.State) } -func (dc *ddlCtx) initJobDoneCh(jobID int64) { - dc.ddlJobDoneChMap.Store(jobID, make(chan struct{}, 1)) -} - func (dc *ddlCtx) notifyJobDone(jobID int64) { if ch, ok := dc.ddlJobDoneChMap.Delete(jobID); ok { // broadcast done event as we might merge multiple jobs into one when fast @@ -574,6 +568,8 @@ func (d *ddl) IsTiFlashPollEnabled() bool { } // RegisterStatsHandle registers statistics handle and its corresponding even channel for ddl. +// TODO this is called after ddl started, will cause panic if related DDL are executed +// in between. func (d *ddl) RegisterStatsHandle(h *handle.Handle) { d.ddlCtx.statsHandle = h d.executor.statsHandle = h @@ -706,6 +702,7 @@ func newDDL(ctx context.Context, options ...Option) (*ddl, *executor) { ddlJobNotifyCh: d.ddlJobNotifyCh, mu: &d.mu, globalIDLock: &d.globalIDLock, + stateSyncer: d.stateSyncer, } d.executor = e @@ -741,13 +738,13 @@ func (d *ddl) Start(ctxPool *pools.ResourcePool) error { d.sessPool = sess.NewSessionPool(ctxPool) d.executor.sessPool = d.sessPool - d.sysTblMgr = systable.NewManager(d.sessPool) - d.minJobIDRefresher = systable.NewMinJobIDRefresher(d.sysTblMgr) + d.executor.sysTblMgr = systable.NewManager(d.sessPool) + d.executor.minJobIDRefresher = systable.NewMinJobIDRefresher(d.executor.sysTblMgr) d.wg.Run(func() { - d.limitDDLJobs() + d.executor.limitDDLJobs() }) d.wg.Run(func() { - d.minJobIDRefresher.Start(d.ctx) + d.executor.minJobIDRefresher.Start(d.ctx) }) d.delRangeMgr = d.newDeleteRangeManager(ctxPool == nil) @@ -916,7 +913,7 @@ func (d *ddl) SetHook(h Callback) { } func (d *ddl) GetMinJobIDRefresher() *systable.MinJobIDRefresher { - return d.minJobIDRefresher + return d.executor.minJobIDRefresher } func (d *ddl) startCleanDeadTableLock() { @@ -940,7 +937,7 @@ func (d *ddl) startCleanDeadTableLock() { continue } for se, tables := range deadLockTables { - err := d.CleanDeadTableLock(tables, se) + err := d.cleanDeadTableLock(tables, se) if err != nil { logutil.DDLLogger().Info("clean dead table lock failed.", zap.Error(err)) } @@ -951,6 +948,32 @@ func (d *ddl) startCleanDeadTableLock() { } } +// cleanDeadTableLock uses to clean dead table locks. +func (d *ddl) cleanDeadTableLock(unlockTables []model.TableLockTpInfo, se model.SessionInfo) error { + if len(unlockTables) == 0 { + return nil + } + arg := &LockTablesArg{ + UnlockTables: unlockTables, + SessionInfo: se, + } + job := &model.Job{ + SchemaID: unlockTables[0].SchemaID, + TableID: unlockTables[0].TableID, + Type: model.ActionUnlockTable, + BinlogInfo: &model.HistoryInfo{}, + Args: []any{arg}, + } + + ctx, err := d.sessPool.Get() + if err != nil { + return err + } + defer d.sessPool.Put(ctx) + err = d.executor.DoDDLJob(ctx, job) + return errors.Trace(err) +} + // SwitchMDL enables MDL or disable MDL. func (d *ddl) SwitchMDL(enable bool) error { isEnableBefore := variable.EnableMDL.Load() diff --git a/pkg/ddl/executor.go b/pkg/ddl/executor.go index df2d9c5f4b806..4fb85e0ff6fda 100644 --- a/pkg/ddl/executor.go +++ b/pkg/ddl/executor.go @@ -39,6 +39,8 @@ import ( "github.com/pingcap/tidb/pkg/ddl/logutil" "github.com/pingcap/tidb/pkg/ddl/resourcegroup" sess "github.com/pingcap/tidb/pkg/ddl/session" + "github.com/pingcap/tidb/pkg/ddl/syncer" + "github.com/pingcap/tidb/pkg/ddl/systable" ddlutil "github.com/pingcap/tidb/pkg/ddl/util" rg "github.com/pingcap/tidb/pkg/domain/resourcegroup" "github.com/pingcap/tidb/pkg/errctx" @@ -50,7 +52,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/metrics" "github.com/pingcap/tidb/pkg/owner" - "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/format" @@ -186,7 +187,6 @@ type ExecutorForTest interface { // all fields are shared with ddl now. type executor struct { - sessPool *sess.Pool statsHandle *handle.Handle ctx context.Context @@ -203,6 +203,12 @@ type executor struct { ddlJobNotifyCh chan struct{} mu *hookStruct // TODO remove it. globalIDLock *sync.Mutex + stateSyncer syncer.StateSyncer + + // those fields are initialized on ddl.Start + sessPool *sess.Pool + sysTblMgr systable.Manager + minJobIDRefresher *systable.MinJobIDRefresher } var _ Executor = (*executor)(nil) @@ -2497,23 +2503,6 @@ func addIndexForForeignKey(ctx sessionctx.Context, tbInfo *model.TableInfo) erro return nil } -func indexColumnsLen(cols []*model.ColumnInfo, idxCols []*model.IndexColumn) (colLen int, err error) { - for _, idxCol := range idxCols { - col := model.FindColumnInfo(cols, idxCol.Name.L) - if col == nil { - err = dbterror.ErrKeyColumnDoesNotExits.GenWithStack("column does not exist: %s", idxCol.Name.L) - return - } - var l int - l, err = getIndexColumnLength(col, idxCol.Length) - if err != nil { - return - } - colLen += l - } - return -} - func isSingleIntPK(constr *ast.Constraint, lastCol *model.ColumnInfo) bool { if len(constr.Keys) != 1 { return false @@ -5855,35 +5844,6 @@ func checkModifyCharsetAndCollation(toCharset, toCollate, origCharset, origColla return nil } -// checkModifyTypes checks if the 'origin' type can be modified to 'to' type no matter directly change -// or change by reorg. It returns error if the two types are incompatible and correlated change are not -// supported. However, even the two types can be change, if the "origin" type contains primary key, error will be returned. -func checkModifyTypes(origin *types.FieldType, to *types.FieldType, needRewriteCollationData bool) error { - canReorg, err := types.CheckModifyTypeCompatible(origin, to) - if err != nil { - if !canReorg { - return errors.Trace(dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs(err.Error())) - } - if mysql.HasPriKeyFlag(origin.GetFlag()) { - msg := "this column has primary key flag" - return dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs(msg) - } - } - - err = checkModifyCharsetAndCollation(to.GetCharset(), to.GetCollate(), origin.GetCharset(), origin.GetCollate(), needRewriteCollationData) - - if err != nil { - if to.GetCharset() == charset.CharsetGBK || origin.GetCharset() == charset.CharsetGBK { - return errors.Trace(err) - } - // column type change can handle the charset change between these two types in the process of the reorg. - if dbterror.ErrUnsupportedModifyCharset.Equal(err) && canReorg { - return nil - } - } - return errors.Trace(err) -} - // SetDefaultValue sets the default value of the column. func SetDefaultValue(ctx sessionctx.Context, col *table.Column, option *ast.ColumnOption) (hasDefaultValue bool, err error) { var value any @@ -5951,85 +5911,6 @@ func setColumnComment(ctx sessionctx.Context, col *table.Column, option *ast.Col return errors.Trace(err) } -// ProcessModifyColumnOptions process column options. -func ProcessModifyColumnOptions(ctx sessionctx.Context, col *table.Column, options []*ast.ColumnOption) error { - var sb strings.Builder - restoreFlags := format.RestoreStringSingleQuotes | format.RestoreKeyWordLowercase | format.RestoreNameBackQuotes | - format.RestoreSpacesAroundBinaryOperation | format.RestoreWithoutSchemaName | format.RestoreWithoutSchemaName - restoreCtx := format.NewRestoreCtx(restoreFlags, &sb) - - var hasDefaultValue, setOnUpdateNow bool - var err error - var hasNullFlag bool - for _, opt := range options { - switch opt.Tp { - case ast.ColumnOptionDefaultValue: - hasDefaultValue, err = SetDefaultValue(ctx, col, opt) - if err != nil { - return errors.Trace(err) - } - case ast.ColumnOptionComment: - err := setColumnComment(ctx, col, opt) - if err != nil { - return errors.Trace(err) - } - case ast.ColumnOptionNotNull: - col.AddFlag(mysql.NotNullFlag) - case ast.ColumnOptionNull: - hasNullFlag = true - col.DelFlag(mysql.NotNullFlag) - case ast.ColumnOptionAutoIncrement: - col.AddFlag(mysql.AutoIncrementFlag) - case ast.ColumnOptionPrimaryKey: - return errors.Trace(dbterror.ErrUnsupportedModifyColumn.GenWithStack("can't change column constraint (PRIMARY KEY)")) - case ast.ColumnOptionUniqKey: - return errors.Trace(dbterror.ErrUnsupportedModifyColumn.GenWithStack("can't change column constraint (UNIQUE KEY)")) - case ast.ColumnOptionOnUpdate: - // TODO: Support other time functions. - if !(col.GetType() == mysql.TypeTimestamp || col.GetType() == mysql.TypeDatetime) { - return dbterror.ErrInvalidOnUpdate.GenWithStackByArgs(col.Name) - } - if !expression.IsValidCurrentTimestampExpr(opt.Expr, &col.FieldType) { - return dbterror.ErrInvalidOnUpdate.GenWithStackByArgs(col.Name) - } - col.AddFlag(mysql.OnUpdateNowFlag) - setOnUpdateNow = true - case ast.ColumnOptionGenerated: - sb.Reset() - err = opt.Expr.Restore(restoreCtx) - if err != nil { - return errors.Trace(err) - } - col.GeneratedExprString = sb.String() - col.GeneratedStored = opt.Stored - col.Dependences = make(map[string]struct{}) - // Only used by checkModifyGeneratedColumn, there is no need to set a ctor for it. - col.GeneratedExpr = table.NewClonableExprNode(nil, opt.Expr) - for _, colName := range FindColumnNamesInExpr(opt.Expr) { - col.Dependences[colName.Name.L] = struct{}{} - } - case ast.ColumnOptionCollate: - col.SetCollate(opt.StrValue) - case ast.ColumnOptionReference: - return errors.Trace(dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("can't modify with references")) - case ast.ColumnOptionFulltext: - return errors.Trace(dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("can't modify with full text")) - case ast.ColumnOptionCheck: - return errors.Trace(dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("can't modify with check")) - // Ignore ColumnOptionAutoRandom. It will be handled later. - case ast.ColumnOptionAutoRandom: - default: - return errors.Trace(dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs(fmt.Sprintf("unknown column option type: %d", opt.Tp))) - } - } - - if err = processAndCheckDefaultValueAndColumn(ctx, col, nil, hasDefaultValue, setOnUpdateNow, hasNullFlag); err != nil { - return errors.Trace(err) - } - - return nil -} - func processAndCheckDefaultValueAndColumn(ctx sessionctx.Context, col *table.Column, outPriKeyConstraint *ast.Constraint, hasDefaultValue, setOnUpdateNow, hasNullFlag bool) error { processDefaultValue(col, hasDefaultValue, setOnUpdateNow) @@ -6066,408 +5947,6 @@ func (e *executor) getModifiableColumnJob(ctx context.Context, sctx sessionctx.C return GetModifiableColumnJob(ctx, sctx, is, ident, originalColName, schema, t, spec) } -func checkModifyColumnWithGeneratedColumnsConstraint(allCols []*table.Column, oldColName model.CIStr) error { - for _, col := range allCols { - if col.GeneratedExpr == nil { - continue - } - dependedColNames := FindColumnNamesInExpr(col.GeneratedExpr.Internal()) - for _, name := range dependedColNames { - if name.Name.L == oldColName.L { - if col.Hidden { - return dbterror.ErrDependentByFunctionalIndex.GenWithStackByArgs(oldColName.O) - } - return dbterror.ErrDependentByGeneratedColumn.GenWithStackByArgs(oldColName.O) - } - } - } - return nil -} - -// ProcessColumnCharsetAndCollation process column charset and collation -func ProcessColumnCharsetAndCollation(sctx sessionctx.Context, col *table.Column, newCol *table.Column, meta *model.TableInfo, specNewColumn *ast.ColumnDef, schema *model.DBInfo) error { - var chs, coll string - var err error - // TODO: Remove it when all table versions are greater than or equal to TableInfoVersion1. - // If newCol's charset is empty and the table's version less than TableInfoVersion1, - // we will not modify the charset of the column. This behavior is not compatible with MySQL. - if len(newCol.FieldType.GetCharset()) == 0 && meta.Version < model.TableInfoVersion1 { - chs = col.FieldType.GetCharset() - coll = col.FieldType.GetCollate() - } else { - chs, coll, err = getCharsetAndCollateInColumnDef(sctx.GetSessionVars(), specNewColumn) - if err != nil { - return errors.Trace(err) - } - chs, coll, err = ResolveCharsetCollation(sctx.GetSessionVars(), - ast.CharsetOpt{Chs: chs, Col: coll}, - ast.CharsetOpt{Chs: meta.Charset, Col: meta.Collate}, - ast.CharsetOpt{Chs: schema.Charset, Col: schema.Collate}, - ) - chs, coll = OverwriteCollationWithBinaryFlag(sctx.GetSessionVars(), specNewColumn, chs, coll) - if err != nil { - return errors.Trace(err) - } - } - - if err = setCharsetCollationFlenDecimal(&newCol.FieldType, newCol.Name.O, chs, coll, sctx.GetSessionVars()); err != nil { - return errors.Trace(err) - } - decodeEnumSetBinaryLiteralToUTF8(&newCol.FieldType, chs) - return nil -} - -// GetModifiableColumnJob returns a DDL job of model.ActionModifyColumn. -func GetModifiableColumnJob( - ctx context.Context, - sctx sessionctx.Context, - is infoschema.InfoSchema, // WARN: is maybe nil here. - ident ast.Ident, - originalColName model.CIStr, - schema *model.DBInfo, - t table.Table, - spec *ast.AlterTableSpec, -) (*model.Job, error) { - var err error - specNewColumn := spec.NewColumns[0] - - col := table.FindCol(t.Cols(), originalColName.L) - if col == nil { - return nil, infoschema.ErrColumnNotExists.GenWithStackByArgs(originalColName, ident.Name) - } - newColName := specNewColumn.Name.Name - if newColName.L == model.ExtraHandleName.L { - return nil, dbterror.ErrWrongColumnName.GenWithStackByArgs(newColName.L) - } - errG := checkModifyColumnWithGeneratedColumnsConstraint(t.Cols(), originalColName) - - // If we want to rename the column name, we need to check whether it already exists. - if newColName.L != originalColName.L { - c := table.FindCol(t.Cols(), newColName.L) - if c != nil { - return nil, infoschema.ErrColumnExists.GenWithStackByArgs(newColName) - } - - // And also check the generated columns dependency, if some generated columns - // depend on this column, we can't rename the column name. - if errG != nil { - return nil, errors.Trace(errG) - } - } - - // Constraints in the new column means adding new constraints. Errors should thrown, - // which will be done by `processColumnOptions` later. - if specNewColumn.Tp == nil { - // Make sure the column definition is simple field type. - return nil, errors.Trace(dbterror.ErrUnsupportedModifyColumn) - } - - if err = checkColumnAttributes(specNewColumn.Name.OrigColName(), specNewColumn.Tp); err != nil { - return nil, errors.Trace(err) - } - - newCol := table.ToColumn(&model.ColumnInfo{ - ID: col.ID, - // We use this PR(https://github.com/pingcap/tidb/pull/6274) as the dividing line to define whether it is a new version or an old version TiDB. - // The old version TiDB initializes the column's offset and state here. - // The new version TiDB doesn't initialize the column's offset and state, and it will do the initialization in run DDL function. - // When we do the rolling upgrade the following may happen: - // a new version TiDB builds the DDL job that doesn't be set the column's offset and state, - // and the old version TiDB is the DDL owner, it doesn't get offset and state from the store. Then it will encounter errors. - // So here we set offset and state to support the rolling upgrade. - Offset: col.Offset, - State: col.State, - OriginDefaultValue: col.OriginDefaultValue, - OriginDefaultValueBit: col.OriginDefaultValueBit, - FieldType: *specNewColumn.Tp, - Name: newColName, - Version: col.Version, - }) - - if err = ProcessColumnCharsetAndCollation(sctx, col, newCol, t.Meta(), specNewColumn, schema); err != nil { - return nil, err - } - - if err = checkModifyColumnWithForeignKeyConstraint(is, schema.Name.L, t.Meta(), col.ColumnInfo, newCol.ColumnInfo); err != nil { - return nil, errors.Trace(err) - } - - // Copy index related options to the new spec. - indexFlags := col.FieldType.GetFlag() & (mysql.PriKeyFlag | mysql.UniqueKeyFlag | mysql.MultipleKeyFlag) - newCol.FieldType.AddFlag(indexFlags) - if mysql.HasPriKeyFlag(col.FieldType.GetFlag()) { - newCol.FieldType.AddFlag(mysql.NotNullFlag) - // TODO: If user explicitly set NULL, we should throw error ErrPrimaryCantHaveNull. - } - - if err = ProcessModifyColumnOptions(sctx, newCol, specNewColumn.Options); err != nil { - return nil, errors.Trace(err) - } - - if err = checkModifyTypes(&col.FieldType, &newCol.FieldType, isColumnWithIndex(col.Name.L, t.Meta().Indices)); err != nil { - if strings.Contains(err.Error(), "Unsupported modifying collation") { - colErrMsg := "Unsupported modifying collation of column '%s' from '%s' to '%s' when index is defined on it." - err = dbterror.ErrUnsupportedModifyCollation.GenWithStack(colErrMsg, col.Name.L, col.GetCollate(), newCol.GetCollate()) - } - return nil, errors.Trace(err) - } - needChangeColData := needChangeColumnData(col.ColumnInfo, newCol.ColumnInfo) - if needChangeColData { - if err = isGeneratedRelatedColumn(t.Meta(), newCol.ColumnInfo, col.ColumnInfo); err != nil { - return nil, errors.Trace(err) - } - if t.Meta().Partition != nil { - return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("table is partition table") - } - } - - // Check that the column change does not affect the partitioning column - // It must keep the same type, int [unsigned], [var]char, date[time] - if t.Meta().Partition != nil { - pt, ok := t.(table.PartitionedTable) - if !ok { - // Should never happen! - return nil, dbterror.ErrNotAllowedTypeInPartition.GenWithStackByArgs(newCol.Name.O) - } - isPartitioningColumn := false - for _, name := range pt.GetPartitionColumnNames() { - if strings.EqualFold(name.L, col.Name.L) { - isPartitioningColumn = true - break - } - } - if isPartitioningColumn { - // TODO: update the partitioning columns with new names if column is renamed - // Would be an extension from MySQL which does not support it. - if col.Name.L != newCol.Name.L { - return nil, dbterror.ErrDependentByPartitionFunctional.GenWithStackByArgs(col.Name.L) - } - if !isColTypeAllowedAsPartitioningCol(t.Meta().Partition.Type, newCol.FieldType) { - return nil, dbterror.ErrNotAllowedTypeInPartition.GenWithStackByArgs(newCol.Name.O) - } - pi := pt.Meta().GetPartitionInfo() - if len(pi.Columns) == 0 { - // non COLUMNS partitioning, only checks INTs, not their actual range - // There are many edge cases, like when truncating SQL Mode is allowed - // which will change the partitioning expression value resulting in a - // different partition. Better be safe and not allow decreasing of length. - // TODO: Should we allow it in strict mode? Wait for a use case / request. - if newCol.FieldType.GetFlen() < col.FieldType.GetFlen() { - return nil, dbterror.ErrUnsupportedModifyCollation.GenWithStack("Unsupported modify column, decreasing length of int may result in truncation and change of partition") - } - } - // Basically only allow changes of the length/decimals for the column - // Note that enum is not allowed, so elems are not checked - // TODO: support partition by ENUM - if newCol.FieldType.EvalType() != col.FieldType.EvalType() || - newCol.FieldType.GetFlag() != col.FieldType.GetFlag() || - newCol.FieldType.GetCollate() != col.FieldType.GetCollate() || - newCol.FieldType.GetCharset() != col.FieldType.GetCharset() { - return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("can't change the partitioning column, since it would require reorganize all partitions") - } - // Generate a new PartitionInfo and validate it together with the new column definition - // Checks if all partition definition values are compatible. - // Similar to what buildRangePartitionDefinitions would do in terms of checks. - - tblInfo := pt.Meta() - newTblInfo := *tblInfo - // Replace col with newCol and see if we can generate a new SHOW CREATE TABLE - // and reparse it and build new partition definitions (which will do additional - // checks columns vs partition definition values - newCols := make([]*model.ColumnInfo, 0, len(newTblInfo.Columns)) - for _, c := range newTblInfo.Columns { - if c.ID == col.ID { - newCols = append(newCols, newCol.ColumnInfo) - continue - } - newCols = append(newCols, c) - } - newTblInfo.Columns = newCols - - var buf bytes.Buffer - AppendPartitionInfo(tblInfo.GetPartitionInfo(), &buf, mysql.ModeNone) - // The parser supports ALTER TABLE ... PARTITION BY ... even if the ddl code does not yet :) - // Ignoring warnings - stmt, _, err := parser.New().ParseSQL("ALTER TABLE t " + buf.String()) - if err != nil { - // Should never happen! - return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStack("cannot parse generated PartitionInfo") - } - at, ok := stmt[0].(*ast.AlterTableStmt) - if !ok || len(at.Specs) != 1 || at.Specs[0].Partition == nil { - return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStack("cannot parse generated PartitionInfo") - } - pAst := at.Specs[0].Partition - _, err = buildPartitionDefinitionsInfo( - exprctx.CtxWithHandleTruncateErrLevel(sctx.GetExprCtx(), errctx.LevelError), - pAst.Definitions, &newTblInfo, uint64(len(newTblInfo.Partition.Definitions)), - ) - if err != nil { - return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStack("New column does not match partition definitions: %s", err.Error()) - } - } - } - - // We don't support modifying column from not_auto_increment to auto_increment. - if !mysql.HasAutoIncrementFlag(col.GetFlag()) && mysql.HasAutoIncrementFlag(newCol.GetFlag()) { - return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("can't set auto_increment") - } - // Not support auto id with default value. - if mysql.HasAutoIncrementFlag(newCol.GetFlag()) && newCol.GetDefaultValue() != nil { - return nil, dbterror.ErrInvalidDefaultValue.GenWithStackByArgs(newCol.Name) - } - // Disallow modifying column from auto_increment to not auto_increment if the session variable `AllowRemoveAutoInc` is false. - if !sctx.GetSessionVars().AllowRemoveAutoInc && mysql.HasAutoIncrementFlag(col.GetFlag()) && !mysql.HasAutoIncrementFlag(newCol.GetFlag()) { - return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("can't remove auto_increment without @@tidb_allow_remove_auto_inc enabled") - } - - // We support modifying the type definitions of 'null' to 'not null' now. - var modifyColumnTp byte - if !mysql.HasNotNullFlag(col.GetFlag()) && mysql.HasNotNullFlag(newCol.GetFlag()) { - if err = checkForNullValue(ctx, sctx, true, ident.Schema, ident.Name, newCol.ColumnInfo, col.ColumnInfo); err != nil { - return nil, errors.Trace(err) - } - // `modifyColumnTp` indicates that there is a type modification. - modifyColumnTp = mysql.TypeNull - } - - if err = checkColumnWithIndexConstraint(t.Meta(), col.ColumnInfo, newCol.ColumnInfo); err != nil { - return nil, err - } - - // As same with MySQL, we don't support modifying the stored status for generated columns. - if err = checkModifyGeneratedColumn(sctx, schema.Name, t, col, newCol, specNewColumn, spec.Position); err != nil { - return nil, errors.Trace(err) - } - if errG != nil { - // According to issue https://github.com/pingcap/tidb/issues/24321, - // changing the type of a column involving generating a column is prohibited. - return nil, dbterror.ErrUnsupportedOnGeneratedColumn.GenWithStackByArgs(errG.Error()) - } - - if t.Meta().TTLInfo != nil { - // the column referenced by TTL should be a time type - if t.Meta().TTLInfo.ColumnName.L == originalColName.L && !types.IsTypeTime(newCol.ColumnInfo.FieldType.GetType()) { - return nil, errors.Trace(dbterror.ErrUnsupportedColumnInTTLConfig.GenWithStackByArgs(newCol.ColumnInfo.Name.O)) - } - } - - var newAutoRandBits uint64 - if newAutoRandBits, err = checkAutoRandom(t.Meta(), col, specNewColumn); err != nil { - return nil, errors.Trace(err) - } - - txn, err := sctx.Txn(true) - if err != nil { - return nil, errors.Trace(err) - } - bdrRole, err := meta.NewMeta(txn).GetBDRRole() - if err != nil { - return nil, errors.Trace(err) - } - if bdrRole == string(ast.BDRRolePrimary) && - deniedByBDRWhenModifyColumn(newCol.FieldType, col.FieldType, specNewColumn.Options) { - return nil, dbterror.ErrBDRRestrictedDDL.FastGenByArgs(bdrRole) - } - - job := &model.Job{ - SchemaID: schema.ID, - TableID: t.Meta().ID, - SchemaName: schema.Name.L, - TableName: t.Meta().Name.L, - Type: model.ActionModifyColumn, - BinlogInfo: &model.HistoryInfo{}, - ReorgMeta: NewDDLReorgMeta(sctx), - CtxVars: []any{needChangeColData}, - Args: []any{&newCol.ColumnInfo, originalColName, spec.Position, modifyColumnTp, newAutoRandBits}, - CDCWriteSource: sctx.GetSessionVars().CDCWriteSource, - SQLMode: sctx.GetSessionVars().SQLMode, - } - return job, nil -} - -// checkColumnWithIndexConstraint is used to check the related index constraint of the modified column. -// Index has a max-prefix-length constraint. eg: a varchar(100), index idx(a), modifying column a to a varchar(4000) -// will cause index idx to break the max-prefix-length constraint. -func checkColumnWithIndexConstraint(tbInfo *model.TableInfo, originalCol, newCol *model.ColumnInfo) error { - columns := make([]*model.ColumnInfo, 0, len(tbInfo.Columns)) - columns = append(columns, tbInfo.Columns...) - // Replace old column with new column. - for i, col := range columns { - if col.Name.L != originalCol.Name.L { - continue - } - columns[i] = newCol.Clone() - columns[i].Name = originalCol.Name - break - } - - pkIndex := tables.FindPrimaryIndex(tbInfo) - - checkOneIndex := func(indexInfo *model.IndexInfo) (err error) { - var modified bool - for _, col := range indexInfo.Columns { - if col.Name.L == originalCol.Name.L { - modified = true - break - } - } - if !modified { - return - } - err = checkIndexInModifiableColumns(columns, indexInfo.Columns) - if err != nil { - return - } - err = checkIndexPrefixLength(columns, indexInfo.Columns) - return - } - - // Check primary key first. - var err error - - if pkIndex != nil { - err = checkOneIndex(pkIndex) - if err != nil { - return err - } - } - - // Check secondary indexes. - for _, indexInfo := range tbInfo.Indices { - if indexInfo.Primary { - continue - } - // the second param should always be set to true, check index length only if it was modified - // checkOneIndex needs one param only. - err = checkOneIndex(indexInfo) - if err != nil { - return err - } - } - return nil -} - -func checkIndexInModifiableColumns(columns []*model.ColumnInfo, idxColumns []*model.IndexColumn) error { - for _, ic := range idxColumns { - col := model.FindColumnInfo(columns, ic.Name.L) - if col == nil { - return dbterror.ErrKeyColumnDoesNotExits.GenWithStack("column does not exist: %s", ic.Name) - } - - prefixLength := types.UnspecifiedLength - if types.IsTypePrefixable(col.FieldType.GetType()) && col.FieldType.GetFlen() > ic.Length { - // When the index column is changed, prefix length is only valid - // if the type is still prefixable and larger than old prefix length. - prefixLength = ic.Length - } - if err := checkIndexColumn(nil, col, prefixLength); err != nil { - return err - } - } - return nil -} - func isClusteredPKColumn(col *table.Column, tblInfo *model.TableInfo) bool { switch { case tblInfo.PKIsHandle: @@ -6485,73 +5964,6 @@ func isClusteredPKColumn(col *table.Column, tblInfo *model.TableInfo) bool { } } -func checkAutoRandom(tableInfo *model.TableInfo, originCol *table.Column, specNewColumn *ast.ColumnDef) (uint64, error) { - var oldShardBits, oldRangeBits uint64 - if isClusteredPKColumn(originCol, tableInfo) { - oldShardBits = tableInfo.AutoRandomBits - oldRangeBits = tableInfo.AutoRandomRangeBits - } - newShardBits, newRangeBits, err := extractAutoRandomBitsFromColDef(specNewColumn) - if err != nil { - return 0, errors.Trace(err) - } - switch { - case oldShardBits == newShardBits: - case oldShardBits < newShardBits: - addingAutoRandom := oldShardBits == 0 - if addingAutoRandom { - convFromAutoInc := mysql.HasAutoIncrementFlag(originCol.GetFlag()) && originCol.IsPKHandleColumn(tableInfo) - if !convFromAutoInc { - return 0, dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomAlterChangeFromAutoInc) - } - } - if autoid.AutoRandomShardBitsMax < newShardBits { - errMsg := fmt.Sprintf(autoid.AutoRandomOverflowErrMsg, - autoid.AutoRandomShardBitsMax, newShardBits, specNewColumn.Name.Name.O) - return 0, dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(errMsg) - } - // increasing auto_random shard bits is allowed. - case oldShardBits > newShardBits: - if newShardBits == 0 { - return 0, dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomAlterErrMsg) - } - return 0, dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomDecreaseBitErrMsg) - } - - modifyingAutoRandCol := oldShardBits > 0 || newShardBits > 0 - if modifyingAutoRandCol { - // Disallow changing the column field type. - if originCol.GetType() != specNewColumn.Tp.GetType() { - return 0, dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomModifyColTypeErrMsg) - } - if originCol.GetType() != mysql.TypeLonglong { - return 0, dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(fmt.Sprintf(autoid.AutoRandomOnNonBigIntColumn, types.TypeStr(originCol.GetType()))) - } - // Disallow changing from auto_random to auto_increment column. - if containsColumnOption(specNewColumn, ast.ColumnOptionAutoIncrement) { - return 0, dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomIncompatibleWithAutoIncErrMsg) - } - // Disallow specifying a default value on auto_random column. - if containsColumnOption(specNewColumn, ast.ColumnOptionDefaultValue) { - return 0, dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomIncompatibleWithDefaultValueErrMsg) - } - } - if rangeBitsIsChanged(oldRangeBits, newRangeBits) { - return 0, dbterror.ErrInvalidAutoRandom.FastGenByArgs(autoid.AutoRandomUnsupportedAlterRangeBits) - } - return newShardBits, nil -} - -func rangeBitsIsChanged(oldBits, newBits uint64) bool { - if oldBits == 0 { - oldBits = autoid.AutoRandomRangeBitsDefault - } - if newBits == 0 { - newBits = autoid.AutoRandomRangeBitsDefault - } - return oldBits != newBits -} - // 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. @@ -8425,30 +7837,6 @@ func CheckIsDropPrimaryKey(indexName model.CIStr, indexInfo *model.IndexInfo, t return isPK, nil } -func isDroppableColumn(tblInfo *model.TableInfo, colName model.CIStr) error { - if ok, dep, isHidden := hasDependentByGeneratedColumn(tblInfo, colName); ok { - if isHidden { - return dbterror.ErrDependentByFunctionalIndex.GenWithStackByArgs(dep) - } - return dbterror.ErrDependentByGeneratedColumn.GenWithStackByArgs(dep) - } - - if len(tblInfo.Columns) == 1 { - return dbterror.ErrCantRemoveAllFields.GenWithStack("can't drop only column %s in table %s", - colName, tblInfo.Name) - } - // We only support dropping column with single-value none Primary Key index covered now. - err := isColumnCanDropWithIndex(colName.L, tblInfo.Indices) - if err != nil { - return err - } - err = IsColumnDroppableWithCheckConstraint(colName, tblInfo) - if err != nil { - return err - } - return 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. @@ -8564,66 +7952,6 @@ func buildAddedPartitionDefs(ctx expression.BuildContext, meta *model.TableInfo, return GeneratePartDefsFromInterval(ctx, spec.Tp, meta, spec.Partition) } -func checkAndGetColumnsTypeAndValuesMatch(ctx expression.BuildContext, colTypes []types.FieldType, exprs []ast.ExprNode) ([]types.Datum, error) { - // Validate() has already checked len(colNames) = len(exprs) - // create table ... partition by range columns (cols) - // partition p0 values less than (expr) - // check the type of cols[i] and expr is consistent. - valDatums := make([]types.Datum, 0, len(colTypes)) - for i, colExpr := range exprs { - if _, ok := colExpr.(*ast.MaxValueExpr); ok { - valDatums = append(valDatums, types.NewStringDatum(partitionMaxValue)) - continue - } - if d, ok := colExpr.(*ast.DefaultExpr); ok { - if d.Name != nil { - return nil, dbterror.ErrWrongTypeColumnValue.GenWithStackByArgs() - } - continue - } - colType := colTypes[i] - val, err := expression.EvalSimpleAst(ctx, colExpr) - if err != nil { - return nil, err - } - // Check val.ConvertTo(colType) doesn't work, so we need this case by case check. - vkind := val.Kind() - switch colType.GetType() { - case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeDuration: - switch vkind { - case types.KindString, types.KindBytes, types.KindNull: - default: - return nil, dbterror.ErrWrongTypeColumnValue.GenWithStackByArgs() - } - case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong: - switch vkind { - case types.KindInt64, types.KindUint64, types.KindNull: - default: - return nil, dbterror.ErrWrongTypeColumnValue.GenWithStackByArgs() - } - case mysql.TypeFloat, mysql.TypeDouble: - switch vkind { - case types.KindFloat32, types.KindFloat64, types.KindNull: - default: - return nil, dbterror.ErrWrongTypeColumnValue.GenWithStackByArgs() - } - case mysql.TypeString, mysql.TypeVarString: - switch vkind { - case types.KindString, types.KindBytes, types.KindNull, types.KindBinaryLiteral: - default: - return nil, dbterror.ErrWrongTypeColumnValue.GenWithStackByArgs() - } - } - evalCtx := ctx.GetEvalCtx() - newVal, err := val.ConvertTo(evalCtx.TypeCtx(), &colType) - if err != nil { - return nil, dbterror.ErrWrongTypeColumnValue.GenWithStackByArgs() - } - valDatums = append(valDatums, newVal) - } - return valDatums, nil -} - // 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)) @@ -8736,32 +8064,6 @@ func (e *executor) UnlockTables(ctx sessionctx.Context, unlockTables []model.Tab return errors.Trace(err) } -// CleanDeadTableLock uses to clean dead table locks. -func (d *ddl) CleanDeadTableLock(unlockTables []model.TableLockTpInfo, se model.SessionInfo) error { - if len(unlockTables) == 0 { - return nil - } - arg := &LockTablesArg{ - UnlockTables: unlockTables, - SessionInfo: se, - } - job := &model.Job{ - SchemaID: unlockTables[0].SchemaID, - TableID: unlockTables[0].TableID, - Type: model.ActionUnlockTable, - BinlogInfo: &model.HistoryInfo{}, - Args: []any{arg}, - } - - ctx, err := d.sessPool.Get() - if err != nil { - return err - } - defer d.sessPool.Put(ctx) - err = d.executor.DoDDLJob(ctx, job) - return errors.Trace(err) -} - func throwErrIfInMemOrSysDB(ctx sessionctx.Context, dbLowerName string) error { if util.IsMemOrSysDB(dbLowerName) { if ctx.GetSessionVars().User != nil { @@ -10057,11 +9359,15 @@ func (e *executor) delJobDoneCh(jobID int64) { e.ddlJobDoneChMap.Delete(jobID) } -func (d *ddl) limitDDLJobs() { +func (e *executor) initJobDoneCh(jobID int64) { + e.ddlJobDoneChMap.Store(jobID, make(chan struct{}, 1)) +} + +func (e *executor) limitDDLJobs() { defer util.Recover(metrics.LabelDDL, "limitDDLJobs", nil, true) jobWs := make([]*JobWrapper, 0, batchAddingJobs) - ch := d.limitJobCh + ch := e.limitJobCh for { select { // the channel is never closed @@ -10073,8 +9379,8 @@ func (d *ddl) limitDDLJobs() { for i := 0; i < jobLen; i++ { jobWs = append(jobWs, <-ch) } - d.addBatchDDLJobs(jobWs) - case <-d.ctx.Done(): + e.addBatchDDLJobs(jobWs) + case <-e.ctx.Done(): return } } @@ -10117,7 +9423,7 @@ func (e *executor) deliverJobTask(task *JobWrapper) { } // addBatchDDLJobs gets global job IDs and puts the DDL jobs in the DDL queue. -func (d *ddl) addBatchDDLJobs(jobWs []*JobWrapper) { +func (e *executor) addBatchDDLJobs(jobWs []*JobWrapper) { startTime := time.Now() var ( err error @@ -10135,9 +9441,9 @@ func (d *ddl) addBatchDDLJobs(jobWs []*JobWrapper) { jobWs = newWs } } - err = d.addBatchDDLJobs2Table(jobWs) + err = e.addBatchDDLJobs2Table(jobWs) } else { - err = d.addBatchDDLJobs2Queue(jobWs) + err = e.addBatchDDLJobs2Queue(jobWs) } var jobs string for _, jobW := range jobWs { @@ -10160,12 +9466,12 @@ func (d *ddl) addBatchDDLJobs(jobWs []*JobWrapper) { } } -func (d *ddl) addBatchDDLJobs2Queue(jobWs []*JobWrapper) error { +func (e *executor) addBatchDDLJobs2Queue(jobWs []*JobWrapper) error { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) // lock to reduce conflict - d.globalIDLock.Lock() - defer d.globalIDLock.Unlock() - return kv.RunInNewTxn(ctx, d.store, true, func(_ context.Context, txn kv.Transaction) error { + e.globalIDLock.Lock() + defer e.globalIDLock.Unlock() + return kv.RunInNewTxn(ctx, e.store, true, func(_ context.Context, txn kv.Transaction) error { t := meta.NewMeta(txn) count := getRequiredGIDCount(jobWs) @@ -10175,7 +9481,7 @@ func (d *ddl) addBatchDDLJobs2Queue(jobWs []*JobWrapper) error { } assignGIDsForJobs(jobWs, ids) - if err := d.checkFlashbackJobInQueue(t); err != nil { + if err := e.checkFlashbackJobInQueue(t); err != nil { return errors.Trace(err) } @@ -10204,7 +9510,7 @@ func (d *ddl) addBatchDDLJobs2Queue(jobWs []*JobWrapper) error { }) } -func (*ddl) checkFlashbackJobInQueue(t *meta.Meta) error { +func (*executor) checkFlashbackJobInQueue(t *meta.Meta) error { jobs, err := t.GetAllDDLJobsInQueue(meta.DefaultJobListKey) if err != nil { return errors.Trace(err) @@ -10218,20 +9524,20 @@ func (*ddl) checkFlashbackJobInQueue(t *meta.Meta) error { } // addBatchDDLJobs2Table gets global job IDs and puts the DDL jobs in the DDL job table. -func (d *ddl) addBatchDDLJobs2Table(jobWs []*JobWrapper) error { +func (e *executor) addBatchDDLJobs2Table(jobWs []*JobWrapper) error { var err error if len(jobWs) == 0 { return nil } - ctx := kv.WithInternalSourceType(d.ctx, kv.InternalTxnDDL) - se, err := d.sessPool.Get() + ctx := kv.WithInternalSourceType(e.ctx, kv.InternalTxnDDL) + se, err := e.sessPool.Get() if err != nil { return errors.Trace(err) } - defer d.sessPool.Put(se) - found, err := d.sysTblMgr.HasFlashbackClusterJob(ctx, d.minJobIDRefresher.GetCurrMinJobID()) + defer e.sessPool.Put(se) + found, err := e.sysTblMgr.HasFlashbackClusterJob(ctx, e.minJobIDRefresher.GetCurrMinJobID()) if err != nil { return errors.Trace(err) } @@ -10244,7 +9550,7 @@ func (d *ddl) addBatchDDLJobs2Table(jobWs []*JobWrapper) error { bdrRole = string(ast.BDRRoleNone) ) - err = kv.RunInNewTxn(ctx, d.store, true, func(_ context.Context, txn kv.Transaction) error { + err = kv.RunInNewTxn(ctx, e.store, true, func(_ context.Context, txn kv.Transaction) error { t := meta.NewMeta(txn) bdrRole, err = t.GetBDRRole() @@ -10254,7 +9560,7 @@ func (d *ddl) addBatchDDLJobs2Table(jobWs []*JobWrapper) error { startTS = txn.StartTS() if variable.DDLForce2Queue.Load() { - if err := d.checkFlashbackJobInQueue(t); err != nil { + if err := e.checkFlashbackJobInQueue(t); err != nil { return err } } @@ -10286,7 +9592,7 @@ func (d *ddl) addBatchDDLJobs2Table(jobWs []*JobWrapper) error { setJobStateToQueueing(job) - if d.stateSyncer.IsUpgradingState() && !hasSysDB(job) { + if e.stateSyncer.IsUpgradingState() && !hasSysDB(job) { if err = pauseRunningJob(sess.NewSession(se), job, model.AdminCommandBySystem); err != nil { logutil.DDLUpgradingLogger().Warn("pause user DDL by system failed", zap.Stringer("job", job), zap.Error(err)) jobW.cacheErr = err @@ -10302,7 +9608,7 @@ func (d *ddl) addBatchDDLJobs2Table(jobWs []*JobWrapper) error { return errors.Trace(err) } for _, jobW := range jobWs { - d.initJobDoneCh(jobW.ID) + e.initJobDoneCh(jobW.ID) } return nil diff --git a/pkg/ddl/index.go b/pkg/ddl/index.go index 7d36e050b3b10..44f86e5e7556e 100644 --- a/pkg/ddl/index.go +++ b/pkg/ddl/index.go @@ -188,6 +188,23 @@ func checkIndexPrefixLength(columns []*model.ColumnInfo, idxColumns []*model.Ind return nil } +func indexColumnsLen(cols []*model.ColumnInfo, idxCols []*model.IndexColumn) (colLen int, err error) { + for _, idxCol := range idxCols { + col := model.FindColumnInfo(cols, idxCol.Name.L) + if col == nil { + err = dbterror.ErrKeyColumnDoesNotExits.GenWithStack("column does not exist: %s", idxCol.Name.L) + return + } + var l int + l, err = getIndexColumnLength(col, idxCol.Length) + if err != nil { + return + } + colLen += l + } + return +} + func checkIndexColumn(ctx sessionctx.Context, col *model.ColumnInfo, indexColumnLen int) error { if col.GetFlen() == 0 && (types.IsTypeChar(col.FieldType.GetType()) || types.IsTypeVarchar(col.FieldType.GetType())) { if col.Hidden { diff --git a/pkg/ddl/modify_column.go b/pkg/ddl/modify_column.go new file mode 100644 index 0000000000000..413616c779619 --- /dev/null +++ b/pkg/ddl/modify_column.go @@ -0,0 +1,618 @@ +// 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 ddl + +import ( + "bytes" + "context" + "fmt" + "strings" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/pkg/errctx" + "github.com/pingcap/tidb/pkg/expression" + exprctx "github.com/pingcap/tidb/pkg/expression/context" + "github.com/pingcap/tidb/pkg/infoschema" + "github.com/pingcap/tidb/pkg/meta" + "github.com/pingcap/tidb/pkg/meta/autoid" + "github.com/pingcap/tidb/pkg/parser" + "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/parser/charset" + "github.com/pingcap/tidb/pkg/parser/format" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/table" + "github.com/pingcap/tidb/pkg/table/tables" + "github.com/pingcap/tidb/pkg/types" + "github.com/pingcap/tidb/pkg/util/dbterror" +) + +func checkModifyColumnWithGeneratedColumnsConstraint(allCols []*table.Column, oldColName model.CIStr) error { + for _, col := range allCols { + if col.GeneratedExpr == nil { + continue + } + dependedColNames := FindColumnNamesInExpr(col.GeneratedExpr.Internal()) + for _, name := range dependedColNames { + if name.Name.L == oldColName.L { + if col.Hidden { + return dbterror.ErrDependentByFunctionalIndex.GenWithStackByArgs(oldColName.O) + } + return dbterror.ErrDependentByGeneratedColumn.GenWithStackByArgs(oldColName.O) + } + } + } + return nil +} + +// GetModifiableColumnJob returns a DDL job of model.ActionModifyColumn. +func GetModifiableColumnJob( + ctx context.Context, + sctx sessionctx.Context, + is infoschema.InfoSchema, // WARN: is maybe nil here. + ident ast.Ident, + originalColName model.CIStr, + schema *model.DBInfo, + t table.Table, + spec *ast.AlterTableSpec, +) (*model.Job, error) { + var err error + specNewColumn := spec.NewColumns[0] + + col := table.FindCol(t.Cols(), originalColName.L) + if col == nil { + return nil, infoschema.ErrColumnNotExists.GenWithStackByArgs(originalColName, ident.Name) + } + newColName := specNewColumn.Name.Name + if newColName.L == model.ExtraHandleName.L { + return nil, dbterror.ErrWrongColumnName.GenWithStackByArgs(newColName.L) + } + errG := checkModifyColumnWithGeneratedColumnsConstraint(t.Cols(), originalColName) + + // If we want to rename the column name, we need to check whether it already exists. + if newColName.L != originalColName.L { + c := table.FindCol(t.Cols(), newColName.L) + if c != nil { + return nil, infoschema.ErrColumnExists.GenWithStackByArgs(newColName) + } + + // And also check the generated columns dependency, if some generated columns + // depend on this column, we can't rename the column name. + if errG != nil { + return nil, errors.Trace(errG) + } + } + + // Constraints in the new column means adding new constraints. Errors should thrown, + // which will be done by `processColumnOptions` later. + if specNewColumn.Tp == nil { + // Make sure the column definition is simple field type. + return nil, errors.Trace(dbterror.ErrUnsupportedModifyColumn) + } + + if err = checkColumnAttributes(specNewColumn.Name.OrigColName(), specNewColumn.Tp); err != nil { + return nil, errors.Trace(err) + } + + newCol := table.ToColumn(&model.ColumnInfo{ + ID: col.ID, + // We use this PR(https://github.com/pingcap/tidb/pull/6274) as the dividing line to define whether it is a new version or an old version TiDB. + // The old version TiDB initializes the column's offset and state here. + // The new version TiDB doesn't initialize the column's offset and state, and it will do the initialization in run DDL function. + // When we do the rolling upgrade the following may happen: + // a new version TiDB builds the DDL job that doesn't be set the column's offset and state, + // and the old version TiDB is the DDL owner, it doesn't get offset and state from the store. Then it will encounter errors. + // So here we set offset and state to support the rolling upgrade. + Offset: col.Offset, + State: col.State, + OriginDefaultValue: col.OriginDefaultValue, + OriginDefaultValueBit: col.OriginDefaultValueBit, + FieldType: *specNewColumn.Tp, + Name: newColName, + Version: col.Version, + }) + + if err = ProcessColumnCharsetAndCollation(sctx, col, newCol, t.Meta(), specNewColumn, schema); err != nil { + return nil, err + } + + if err = checkModifyColumnWithForeignKeyConstraint(is, schema.Name.L, t.Meta(), col.ColumnInfo, newCol.ColumnInfo); err != nil { + return nil, errors.Trace(err) + } + + // Copy index related options to the new spec. + indexFlags := col.FieldType.GetFlag() & (mysql.PriKeyFlag | mysql.UniqueKeyFlag | mysql.MultipleKeyFlag) + newCol.FieldType.AddFlag(indexFlags) + if mysql.HasPriKeyFlag(col.FieldType.GetFlag()) { + newCol.FieldType.AddFlag(mysql.NotNullFlag) + // TODO: If user explicitly set NULL, we should throw error ErrPrimaryCantHaveNull. + } + + if err = ProcessModifyColumnOptions(sctx, newCol, specNewColumn.Options); err != nil { + return nil, errors.Trace(err) + } + + if err = checkModifyTypes(&col.FieldType, &newCol.FieldType, isColumnWithIndex(col.Name.L, t.Meta().Indices)); err != nil { + if strings.Contains(err.Error(), "Unsupported modifying collation") { + colErrMsg := "Unsupported modifying collation of column '%s' from '%s' to '%s' when index is defined on it." + err = dbterror.ErrUnsupportedModifyCollation.GenWithStack(colErrMsg, col.Name.L, col.GetCollate(), newCol.GetCollate()) + } + return nil, errors.Trace(err) + } + needChangeColData := needChangeColumnData(col.ColumnInfo, newCol.ColumnInfo) + if needChangeColData { + if err = isGeneratedRelatedColumn(t.Meta(), newCol.ColumnInfo, col.ColumnInfo); err != nil { + return nil, errors.Trace(err) + } + if t.Meta().Partition != nil { + return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("table is partition table") + } + } + + // Check that the column change does not affect the partitioning column + // It must keep the same type, int [unsigned], [var]char, date[time] + if t.Meta().Partition != nil { + pt, ok := t.(table.PartitionedTable) + if !ok { + // Should never happen! + return nil, dbterror.ErrNotAllowedTypeInPartition.GenWithStackByArgs(newCol.Name.O) + } + isPartitioningColumn := false + for _, name := range pt.GetPartitionColumnNames() { + if strings.EqualFold(name.L, col.Name.L) { + isPartitioningColumn = true + break + } + } + if isPartitioningColumn { + // TODO: update the partitioning columns with new names if column is renamed + // Would be an extension from MySQL which does not support it. + if col.Name.L != newCol.Name.L { + return nil, dbterror.ErrDependentByPartitionFunctional.GenWithStackByArgs(col.Name.L) + } + if !isColTypeAllowedAsPartitioningCol(t.Meta().Partition.Type, newCol.FieldType) { + return nil, dbterror.ErrNotAllowedTypeInPartition.GenWithStackByArgs(newCol.Name.O) + } + pi := pt.Meta().GetPartitionInfo() + if len(pi.Columns) == 0 { + // non COLUMNS partitioning, only checks INTs, not their actual range + // There are many edge cases, like when truncating SQL Mode is allowed + // which will change the partitioning expression value resulting in a + // different partition. Better be safe and not allow decreasing of length. + // TODO: Should we allow it in strict mode? Wait for a use case / request. + if newCol.FieldType.GetFlen() < col.FieldType.GetFlen() { + return nil, dbterror.ErrUnsupportedModifyCollation.GenWithStack("Unsupported modify column, decreasing length of int may result in truncation and change of partition") + } + } + // Basically only allow changes of the length/decimals for the column + // Note that enum is not allowed, so elems are not checked + // TODO: support partition by ENUM + if newCol.FieldType.EvalType() != col.FieldType.EvalType() || + newCol.FieldType.GetFlag() != col.FieldType.GetFlag() || + newCol.FieldType.GetCollate() != col.FieldType.GetCollate() || + newCol.FieldType.GetCharset() != col.FieldType.GetCharset() { + return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("can't change the partitioning column, since it would require reorganize all partitions") + } + // Generate a new PartitionInfo and validate it together with the new column definition + // Checks if all partition definition values are compatible. + // Similar to what buildRangePartitionDefinitions would do in terms of checks. + + tblInfo := pt.Meta() + newTblInfo := *tblInfo + // Replace col with newCol and see if we can generate a new SHOW CREATE TABLE + // and reparse it and build new partition definitions (which will do additional + // checks columns vs partition definition values + newCols := make([]*model.ColumnInfo, 0, len(newTblInfo.Columns)) + for _, c := range newTblInfo.Columns { + if c.ID == col.ID { + newCols = append(newCols, newCol.ColumnInfo) + continue + } + newCols = append(newCols, c) + } + newTblInfo.Columns = newCols + + var buf bytes.Buffer + AppendPartitionInfo(tblInfo.GetPartitionInfo(), &buf, mysql.ModeNone) + // The parser supports ALTER TABLE ... PARTITION BY ... even if the ddl code does not yet :) + // Ignoring warnings + stmt, _, err := parser.New().ParseSQL("ALTER TABLE t " + buf.String()) + if err != nil { + // Should never happen! + return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStack("cannot parse generated PartitionInfo") + } + at, ok := stmt[0].(*ast.AlterTableStmt) + if !ok || len(at.Specs) != 1 || at.Specs[0].Partition == nil { + return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStack("cannot parse generated PartitionInfo") + } + pAst := at.Specs[0].Partition + _, err = buildPartitionDefinitionsInfo( + exprctx.CtxWithHandleTruncateErrLevel(sctx.GetExprCtx(), errctx.LevelError), + pAst.Definitions, &newTblInfo, uint64(len(newTblInfo.Partition.Definitions)), + ) + if err != nil { + return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStack("New column does not match partition definitions: %s", err.Error()) + } + } + } + + // We don't support modifying column from not_auto_increment to auto_increment. + if !mysql.HasAutoIncrementFlag(col.GetFlag()) && mysql.HasAutoIncrementFlag(newCol.GetFlag()) { + return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("can't set auto_increment") + } + // Not support auto id with default value. + if mysql.HasAutoIncrementFlag(newCol.GetFlag()) && newCol.GetDefaultValue() != nil { + return nil, dbterror.ErrInvalidDefaultValue.GenWithStackByArgs(newCol.Name) + } + // Disallow modifying column from auto_increment to not auto_increment if the session variable `AllowRemoveAutoInc` is false. + if !sctx.GetSessionVars().AllowRemoveAutoInc && mysql.HasAutoIncrementFlag(col.GetFlag()) && !mysql.HasAutoIncrementFlag(newCol.GetFlag()) { + return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("can't remove auto_increment without @@tidb_allow_remove_auto_inc enabled") + } + + // We support modifying the type definitions of 'null' to 'not null' now. + var modifyColumnTp byte + if !mysql.HasNotNullFlag(col.GetFlag()) && mysql.HasNotNullFlag(newCol.GetFlag()) { + if err = checkForNullValue(ctx, sctx, true, ident.Schema, ident.Name, newCol.ColumnInfo, col.ColumnInfo); err != nil { + return nil, errors.Trace(err) + } + // `modifyColumnTp` indicates that there is a type modification. + modifyColumnTp = mysql.TypeNull + } + + if err = checkColumnWithIndexConstraint(t.Meta(), col.ColumnInfo, newCol.ColumnInfo); err != nil { + return nil, err + } + + // As same with MySQL, we don't support modifying the stored status for generated columns. + if err = checkModifyGeneratedColumn(sctx, schema.Name, t, col, newCol, specNewColumn, spec.Position); err != nil { + return nil, errors.Trace(err) + } + if errG != nil { + // According to issue https://github.com/pingcap/tidb/issues/24321, + // changing the type of a column involving generating a column is prohibited. + return nil, dbterror.ErrUnsupportedOnGeneratedColumn.GenWithStackByArgs(errG.Error()) + } + + if t.Meta().TTLInfo != nil { + // the column referenced by TTL should be a time type + if t.Meta().TTLInfo.ColumnName.L == originalColName.L && !types.IsTypeTime(newCol.ColumnInfo.FieldType.GetType()) { + return nil, errors.Trace(dbterror.ErrUnsupportedColumnInTTLConfig.GenWithStackByArgs(newCol.ColumnInfo.Name.O)) + } + } + + var newAutoRandBits uint64 + if newAutoRandBits, err = checkAutoRandom(t.Meta(), col, specNewColumn); err != nil { + return nil, errors.Trace(err) + } + + txn, err := sctx.Txn(true) + if err != nil { + return nil, errors.Trace(err) + } + bdrRole, err := meta.NewMeta(txn).GetBDRRole() + if err != nil { + return nil, errors.Trace(err) + } + if bdrRole == string(ast.BDRRolePrimary) && + deniedByBDRWhenModifyColumn(newCol.FieldType, col.FieldType, specNewColumn.Options) { + return nil, dbterror.ErrBDRRestrictedDDL.FastGenByArgs(bdrRole) + } + + job := &model.Job{ + SchemaID: schema.ID, + TableID: t.Meta().ID, + SchemaName: schema.Name.L, + TableName: t.Meta().Name.L, + Type: model.ActionModifyColumn, + BinlogInfo: &model.HistoryInfo{}, + ReorgMeta: NewDDLReorgMeta(sctx), + CtxVars: []any{needChangeColData}, + Args: []any{&newCol.ColumnInfo, originalColName, spec.Position, modifyColumnTp, newAutoRandBits}, + CDCWriteSource: sctx.GetSessionVars().CDCWriteSource, + SQLMode: sctx.GetSessionVars().SQLMode, + } + return job, nil +} + +// ProcessColumnCharsetAndCollation process column charset and collation +func ProcessColumnCharsetAndCollation(sctx sessionctx.Context, col *table.Column, newCol *table.Column, meta *model.TableInfo, specNewColumn *ast.ColumnDef, schema *model.DBInfo) error { + var chs, coll string + var err error + // TODO: Remove it when all table versions are greater than or equal to TableInfoVersion1. + // If newCol's charset is empty and the table's version less than TableInfoVersion1, + // we will not modify the charset of the column. This behavior is not compatible with MySQL. + if len(newCol.FieldType.GetCharset()) == 0 && meta.Version < model.TableInfoVersion1 { + chs = col.FieldType.GetCharset() + coll = col.FieldType.GetCollate() + } else { + chs, coll, err = getCharsetAndCollateInColumnDef(sctx.GetSessionVars(), specNewColumn) + if err != nil { + return errors.Trace(err) + } + chs, coll, err = ResolveCharsetCollation(sctx.GetSessionVars(), + ast.CharsetOpt{Chs: chs, Col: coll}, + ast.CharsetOpt{Chs: meta.Charset, Col: meta.Collate}, + ast.CharsetOpt{Chs: schema.Charset, Col: schema.Collate}, + ) + chs, coll = OverwriteCollationWithBinaryFlag(sctx.GetSessionVars(), specNewColumn, chs, coll) + if err != nil { + return errors.Trace(err) + } + } + + if err = setCharsetCollationFlenDecimal(&newCol.FieldType, newCol.Name.O, chs, coll, sctx.GetSessionVars()); err != nil { + return errors.Trace(err) + } + decodeEnumSetBinaryLiteralToUTF8(&newCol.FieldType, chs) + return nil +} + +// checkColumnWithIndexConstraint is used to check the related index constraint of the modified column. +// Index has a max-prefix-length constraint. eg: a varchar(100), index idx(a), modifying column a to a varchar(4000) +// will cause index idx to break the max-prefix-length constraint. +func checkColumnWithIndexConstraint(tbInfo *model.TableInfo, originalCol, newCol *model.ColumnInfo) error { + columns := make([]*model.ColumnInfo, 0, len(tbInfo.Columns)) + columns = append(columns, tbInfo.Columns...) + // Replace old column with new column. + for i, col := range columns { + if col.Name.L != originalCol.Name.L { + continue + } + columns[i] = newCol.Clone() + columns[i].Name = originalCol.Name + break + } + + pkIndex := tables.FindPrimaryIndex(tbInfo) + + checkOneIndex := func(indexInfo *model.IndexInfo) (err error) { + var modified bool + for _, col := range indexInfo.Columns { + if col.Name.L == originalCol.Name.L { + modified = true + break + } + } + if !modified { + return + } + err = checkIndexInModifiableColumns(columns, indexInfo.Columns) + if err != nil { + return + } + err = checkIndexPrefixLength(columns, indexInfo.Columns) + return + } + + // Check primary key first. + var err error + + if pkIndex != nil { + err = checkOneIndex(pkIndex) + if err != nil { + return err + } + } + + // Check secondary indexes. + for _, indexInfo := range tbInfo.Indices { + if indexInfo.Primary { + continue + } + // the second param should always be set to true, check index length only if it was modified + // checkOneIndex needs one param only. + err = checkOneIndex(indexInfo) + if err != nil { + return err + } + } + return nil +} + +func checkIndexInModifiableColumns(columns []*model.ColumnInfo, idxColumns []*model.IndexColumn) error { + for _, ic := range idxColumns { + col := model.FindColumnInfo(columns, ic.Name.L) + if col == nil { + return dbterror.ErrKeyColumnDoesNotExits.GenWithStack("column does not exist: %s", ic.Name) + } + + prefixLength := types.UnspecifiedLength + if types.IsTypePrefixable(col.FieldType.GetType()) && col.FieldType.GetFlen() > ic.Length { + // When the index column is changed, prefix length is only valid + // if the type is still prefixable and larger than old prefix length. + prefixLength = ic.Length + } + if err := checkIndexColumn(nil, col, prefixLength); err != nil { + return err + } + } + return nil +} + +// checkModifyTypes checks if the 'origin' type can be modified to 'to' type no matter directly change +// or change by reorg. It returns error if the two types are incompatible and correlated change are not +// supported. However, even the two types can be change, if the "origin" type contains primary key, error will be returned. +func checkModifyTypes(origin *types.FieldType, to *types.FieldType, needRewriteCollationData bool) error { + canReorg, err := types.CheckModifyTypeCompatible(origin, to) + if err != nil { + if !canReorg { + return errors.Trace(dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs(err.Error())) + } + if mysql.HasPriKeyFlag(origin.GetFlag()) { + msg := "this column has primary key flag" + return dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs(msg) + } + } + + err = checkModifyCharsetAndCollation(to.GetCharset(), to.GetCollate(), origin.GetCharset(), origin.GetCollate(), needRewriteCollationData) + + if err != nil { + if to.GetCharset() == charset.CharsetGBK || origin.GetCharset() == charset.CharsetGBK { + return errors.Trace(err) + } + // column type change can handle the charset change between these two types in the process of the reorg. + if dbterror.ErrUnsupportedModifyCharset.Equal(err) && canReorg { + return nil + } + } + return errors.Trace(err) +} + +// ProcessModifyColumnOptions process column options. +func ProcessModifyColumnOptions(ctx sessionctx.Context, col *table.Column, options []*ast.ColumnOption) error { + var sb strings.Builder + restoreFlags := format.RestoreStringSingleQuotes | format.RestoreKeyWordLowercase | format.RestoreNameBackQuotes | + format.RestoreSpacesAroundBinaryOperation | format.RestoreWithoutSchemaName | format.RestoreWithoutSchemaName + restoreCtx := format.NewRestoreCtx(restoreFlags, &sb) + + var hasDefaultValue, setOnUpdateNow bool + var err error + var hasNullFlag bool + for _, opt := range options { + switch opt.Tp { + case ast.ColumnOptionDefaultValue: + hasDefaultValue, err = SetDefaultValue(ctx, col, opt) + if err != nil { + return errors.Trace(err) + } + case ast.ColumnOptionComment: + err := setColumnComment(ctx, col, opt) + if err != nil { + return errors.Trace(err) + } + case ast.ColumnOptionNotNull: + col.AddFlag(mysql.NotNullFlag) + case ast.ColumnOptionNull: + hasNullFlag = true + col.DelFlag(mysql.NotNullFlag) + case ast.ColumnOptionAutoIncrement: + col.AddFlag(mysql.AutoIncrementFlag) + case ast.ColumnOptionPrimaryKey: + return errors.Trace(dbterror.ErrUnsupportedModifyColumn.GenWithStack("can't change column constraint (PRIMARY KEY)")) + case ast.ColumnOptionUniqKey: + return errors.Trace(dbterror.ErrUnsupportedModifyColumn.GenWithStack("can't change column constraint (UNIQUE KEY)")) + case ast.ColumnOptionOnUpdate: + // TODO: Support other time functions. + if !(col.GetType() == mysql.TypeTimestamp || col.GetType() == mysql.TypeDatetime) { + return dbterror.ErrInvalidOnUpdate.GenWithStackByArgs(col.Name) + } + if !expression.IsValidCurrentTimestampExpr(opt.Expr, &col.FieldType) { + return dbterror.ErrInvalidOnUpdate.GenWithStackByArgs(col.Name) + } + col.AddFlag(mysql.OnUpdateNowFlag) + setOnUpdateNow = true + case ast.ColumnOptionGenerated: + sb.Reset() + err = opt.Expr.Restore(restoreCtx) + if err != nil { + return errors.Trace(err) + } + col.GeneratedExprString = sb.String() + col.GeneratedStored = opt.Stored + col.Dependences = make(map[string]struct{}) + // Only used by checkModifyGeneratedColumn, there is no need to set a ctor for it. + col.GeneratedExpr = table.NewClonableExprNode(nil, opt.Expr) + for _, colName := range FindColumnNamesInExpr(opt.Expr) { + col.Dependences[colName.Name.L] = struct{}{} + } + case ast.ColumnOptionCollate: + col.SetCollate(opt.StrValue) + case ast.ColumnOptionReference: + return errors.Trace(dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("can't modify with references")) + case ast.ColumnOptionFulltext: + return errors.Trace(dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("can't modify with full text")) + case ast.ColumnOptionCheck: + return errors.Trace(dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("can't modify with check")) + // Ignore ColumnOptionAutoRandom. It will be handled later. + case ast.ColumnOptionAutoRandom: + default: + return errors.Trace(dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs(fmt.Sprintf("unknown column option type: %d", opt.Tp))) + } + } + + if err = processAndCheckDefaultValueAndColumn(ctx, col, nil, hasDefaultValue, setOnUpdateNow, hasNullFlag); err != nil { + return errors.Trace(err) + } + + return nil +} + +func checkAutoRandom(tableInfo *model.TableInfo, originCol *table.Column, specNewColumn *ast.ColumnDef) (uint64, error) { + var oldShardBits, oldRangeBits uint64 + if isClusteredPKColumn(originCol, tableInfo) { + oldShardBits = tableInfo.AutoRandomBits + oldRangeBits = tableInfo.AutoRandomRangeBits + } + newShardBits, newRangeBits, err := extractAutoRandomBitsFromColDef(specNewColumn) + if err != nil { + return 0, errors.Trace(err) + } + switch { + case oldShardBits == newShardBits: + case oldShardBits < newShardBits: + addingAutoRandom := oldShardBits == 0 + if addingAutoRandom { + convFromAutoInc := mysql.HasAutoIncrementFlag(originCol.GetFlag()) && originCol.IsPKHandleColumn(tableInfo) + if !convFromAutoInc { + return 0, dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomAlterChangeFromAutoInc) + } + } + if autoid.AutoRandomShardBitsMax < newShardBits { + errMsg := fmt.Sprintf(autoid.AutoRandomOverflowErrMsg, + autoid.AutoRandomShardBitsMax, newShardBits, specNewColumn.Name.Name.O) + return 0, dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(errMsg) + } + // increasing auto_random shard bits is allowed. + case oldShardBits > newShardBits: + if newShardBits == 0 { + return 0, dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomAlterErrMsg) + } + return 0, dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomDecreaseBitErrMsg) + } + + modifyingAutoRandCol := oldShardBits > 0 || newShardBits > 0 + if modifyingAutoRandCol { + // Disallow changing the column field type. + if originCol.GetType() != specNewColumn.Tp.GetType() { + return 0, dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomModifyColTypeErrMsg) + } + if originCol.GetType() != mysql.TypeLonglong { + return 0, dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(fmt.Sprintf(autoid.AutoRandomOnNonBigIntColumn, types.TypeStr(originCol.GetType()))) + } + // Disallow changing from auto_random to auto_increment column. + if containsColumnOption(specNewColumn, ast.ColumnOptionAutoIncrement) { + return 0, dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomIncompatibleWithAutoIncErrMsg) + } + // Disallow specifying a default value on auto_random column. + if containsColumnOption(specNewColumn, ast.ColumnOptionDefaultValue) { + return 0, dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomIncompatibleWithDefaultValueErrMsg) + } + } + if rangeBitsIsChanged(oldRangeBits, newRangeBits) { + return 0, dbterror.ErrInvalidAutoRandom.FastGenByArgs(autoid.AutoRandomUnsupportedAlterRangeBits) + } + return newShardBits, nil +} + +func rangeBitsIsChanged(oldBits, newBits uint64) bool { + if oldBits == 0 { + oldBits = autoid.AutoRandomRangeBitsDefault + } + if newBits == 0 { + newBits = autoid.AutoRandomRangeBitsDefault + } + return oldBits != newBits +} diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index d86e749206cc4..a3402ab93c5ac 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -1030,6 +1030,66 @@ func generatePartitionDefinitionsFromInterval(ctx expression.BuildContext, partO return nil } +func checkAndGetColumnsTypeAndValuesMatch(ctx expression.BuildContext, colTypes []types.FieldType, exprs []ast.ExprNode) ([]types.Datum, error) { + // Validate() has already checked len(colNames) = len(exprs) + // create table ... partition by range columns (cols) + // partition p0 values less than (expr) + // check the type of cols[i] and expr is consistent. + valDatums := make([]types.Datum, 0, len(colTypes)) + for i, colExpr := range exprs { + if _, ok := colExpr.(*ast.MaxValueExpr); ok { + valDatums = append(valDatums, types.NewStringDatum(partitionMaxValue)) + continue + } + if d, ok := colExpr.(*ast.DefaultExpr); ok { + if d.Name != nil { + return nil, dbterror.ErrWrongTypeColumnValue.GenWithStackByArgs() + } + continue + } + colType := colTypes[i] + val, err := expression.EvalSimpleAst(ctx, colExpr) + if err != nil { + return nil, err + } + // Check val.ConvertTo(colType) doesn't work, so we need this case by case check. + vkind := val.Kind() + switch colType.GetType() { + case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeDuration: + switch vkind { + case types.KindString, types.KindBytes, types.KindNull: + default: + return nil, dbterror.ErrWrongTypeColumnValue.GenWithStackByArgs() + } + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong: + switch vkind { + case types.KindInt64, types.KindUint64, types.KindNull: + default: + return nil, dbterror.ErrWrongTypeColumnValue.GenWithStackByArgs() + } + case mysql.TypeFloat, mysql.TypeDouble: + switch vkind { + case types.KindFloat32, types.KindFloat64, types.KindNull: + default: + return nil, dbterror.ErrWrongTypeColumnValue.GenWithStackByArgs() + } + case mysql.TypeString, mysql.TypeVarString: + switch vkind { + case types.KindString, types.KindBytes, types.KindNull, types.KindBinaryLiteral: + default: + return nil, dbterror.ErrWrongTypeColumnValue.GenWithStackByArgs() + } + } + evalCtx := ctx.GetEvalCtx() + newVal, err := val.ConvertTo(evalCtx.TypeCtx(), &colType) + if err != nil { + return nil, dbterror.ErrWrongTypeColumnValue.GenWithStackByArgs() + } + valDatums = append(valDatums, newVal) + } + return valDatums, nil +} + func astIntValueExprFromStr(s string, unsigned bool) (ast.ExprNode, error) { if unsigned { u, err := strconv.ParseUint(s, 10, 64) From ddfe673c057cc49a7ed9ace4b9ea1ad730368da0 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Thu, 1 Aug 2024 15:44:28 +0800 Subject: [PATCH 02/13] create table --- pkg/ddl/create_table.go | 739 ++++++++++++++++++++++++++++++++++++++++ pkg/ddl/executor.go | 702 -------------------------------------- 2 files changed, 739 insertions(+), 702 deletions(-) create mode 100644 pkg/ddl/create_table.go diff --git a/pkg/ddl/create_table.go b/pkg/ddl/create_table.go new file mode 100644 index 0000000000000..6cc4baf0d76dd --- /dev/null +++ b/pkg/ddl/create_table.go @@ -0,0 +1,739 @@ +// 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 ddl + +import ( + "context" + "fmt" + "math" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/pkg/config" + "github.com/pingcap/tidb/pkg/infoschema" + "github.com/pingcap/tidb/pkg/meta/autoid" + "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/mysql" + field_types "github.com/pingcap/tidb/pkg/parser/types" + "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/table" + "github.com/pingcap/tidb/pkg/table/tables" + "github.com/pingcap/tidb/pkg/types" + driver "github.com/pingcap/tidb/pkg/types/parser_driver" + "github.com/pingcap/tidb/pkg/util/dbterror" + "github.com/pingcap/tidb/pkg/util/mock" +) + +// BuildTableInfoFromAST builds model.TableInfo from a SQL statement. +// Note: TableID and PartitionID are left as uninitialized value. +func BuildTableInfoFromAST(s *ast.CreateTableStmt) (*model.TableInfo, error) { + return buildTableInfoWithCheck(mock.NewContext(), s, mysql.DefaultCharset, "", nil) +} + +// buildTableInfoWithCheck builds model.TableInfo from a SQL statement. +// Note: TableID and PartitionIDs are left as uninitialized value. +func buildTableInfoWithCheck(ctx sessionctx.Context, s *ast.CreateTableStmt, dbCharset, dbCollate string, placementPolicyRef *model.PolicyRefInfo) (*model.TableInfo, error) { + tbInfo, err := BuildTableInfoWithStmt(ctx, s, dbCharset, dbCollate, placementPolicyRef) + if err != nil { + return nil, err + } + // Fix issue 17952 which will cause partition range expr can't be parsed as Int. + // checkTableInfoValidWithStmt will do the constant fold the partition expression first, + // then checkTableInfoValidExtra will pass the tableInfo check successfully. + if err = checkTableInfoValidWithStmt(ctx, tbInfo, s); err != nil { + return nil, err + } + if err = checkTableInfoValidExtra(tbInfo); err != nil { + return nil, err + } + return tbInfo, nil +} + +// BuildSessionTemporaryTableInfo builds model.TableInfo from a SQL statement. +func BuildSessionTemporaryTableInfo(ctx sessionctx.Context, is infoschema.InfoSchema, s *ast.CreateTableStmt, dbCharset, dbCollate string, placementPolicyRef *model.PolicyRefInfo) (*model.TableInfo, error) { + ident := ast.Ident{Schema: s.Table.Schema, Name: s.Table.Name} + //build tableInfo + var tbInfo *model.TableInfo + var referTbl table.Table + var err error + if s.ReferTable != nil { + referIdent := ast.Ident{Schema: s.ReferTable.Schema, Name: s.ReferTable.Name} + _, ok := is.SchemaByName(referIdent.Schema) + if !ok { + return nil, infoschema.ErrTableNotExists.GenWithStackByArgs(referIdent.Schema, referIdent.Name) + } + referTbl, err = is.TableByName(context.Background(), referIdent.Schema, referIdent.Name) + if err != nil { + return nil, infoschema.ErrTableNotExists.GenWithStackByArgs(referIdent.Schema, referIdent.Name) + } + tbInfo, err = BuildTableInfoWithLike(ctx, ident, referTbl.Meta(), s) + } else { + tbInfo, err = buildTableInfoWithCheck(ctx, s, dbCharset, dbCollate, placementPolicyRef) + } + return tbInfo, err +} + +// BuildTableInfoWithStmt builds model.TableInfo from a SQL statement without validity check +func BuildTableInfoWithStmt(ctx sessionctx.Context, s *ast.CreateTableStmt, dbCharset, dbCollate string, placementPolicyRef *model.PolicyRefInfo) (*model.TableInfo, error) { + colDefs := s.Cols + tableCharset, tableCollate, err := GetCharsetAndCollateInTableOption(ctx.GetSessionVars(), 0, s.Options) + if err != nil { + return nil, errors.Trace(err) + } + tableCharset, tableCollate, err = ResolveCharsetCollation(ctx.GetSessionVars(), + ast.CharsetOpt{Chs: tableCharset, Col: tableCollate}, + ast.CharsetOpt{Chs: dbCharset, Col: dbCollate}, + ) + if err != nil { + return nil, errors.Trace(err) + } + + // The column charset haven't been resolved here. + cols, newConstraints, err := buildColumnsAndConstraints(ctx, colDefs, s.Constraints, tableCharset, tableCollate) + if err != nil { + return nil, errors.Trace(err) + } + err = checkConstraintNames(s.Table.Name, newConstraints) + if err != nil { + return nil, errors.Trace(err) + } + + var tbInfo *model.TableInfo + tbInfo, err = BuildTableInfo(ctx, s.Table.Name, cols, newConstraints, tableCharset, tableCollate) + if err != nil { + return nil, errors.Trace(err) + } + if err = setTemporaryType(ctx, tbInfo, s); err != nil { + return nil, errors.Trace(err) + } + + if err = setTableAutoRandomBits(ctx, tbInfo, colDefs); err != nil { + return nil, errors.Trace(err) + } + + if err = handleTableOptions(s.Options, tbInfo); err != nil { + return nil, errors.Trace(err) + } + + sessionVars := ctx.GetSessionVars() + if _, err = validateCommentLength(sessionVars.StmtCtx.ErrCtx(), sessionVars.SQLMode, tbInfo.Name.L, &tbInfo.Comment, dbterror.ErrTooLongTableComment); err != nil { + return nil, errors.Trace(err) + } + + if tbInfo.TempTableType == model.TempTableNone && tbInfo.PlacementPolicyRef == nil && placementPolicyRef != nil { + // Set the defaults from Schema. Note: they are mutual exclusive! + tbInfo.PlacementPolicyRef = placementPolicyRef + } + + // After handleTableOptions, so the partitions can get defaults from Table level + err = buildTablePartitionInfo(ctx, s.Partition, tbInfo) + if err != nil { + return nil, errors.Trace(err) + } + + return tbInfo, nil +} + +func setTableAutoRandomBits(ctx sessionctx.Context, tbInfo *model.TableInfo, colDefs []*ast.ColumnDef) error { + for _, col := range colDefs { + if containsColumnOption(col, ast.ColumnOptionAutoRandom) { + if col.Tp.GetType() != mysql.TypeLonglong { + return dbterror.ErrInvalidAutoRandom.GenWithStackByArgs( + fmt.Sprintf(autoid.AutoRandomOnNonBigIntColumn, types.TypeStr(col.Tp.GetType()))) + } + switch { + case tbInfo.PKIsHandle: + if tbInfo.GetPkName().L != col.Name.Name.L { + errMsg := fmt.Sprintf(autoid.AutoRandomMustFirstColumnInPK, col.Name.Name.O) + return dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(errMsg) + } + case tbInfo.IsCommonHandle: + pk := tables.FindPrimaryIndex(tbInfo) + if pk == nil { + return dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomNoClusteredPKErrMsg) + } + if col.Name.Name.L != pk.Columns[0].Name.L { + errMsg := fmt.Sprintf(autoid.AutoRandomMustFirstColumnInPK, col.Name.Name.O) + return dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(errMsg) + } + default: + return dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomNoClusteredPKErrMsg) + } + + if containsColumnOption(col, ast.ColumnOptionAutoIncrement) { + return dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomIncompatibleWithAutoIncErrMsg) + } + if containsColumnOption(col, ast.ColumnOptionDefaultValue) { + return dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomIncompatibleWithDefaultValueErrMsg) + } + + shardBits, rangeBits, err := extractAutoRandomBitsFromColDef(col) + if err != nil { + return errors.Trace(err) + } + tbInfo.AutoRandomBits = shardBits + tbInfo.AutoRandomRangeBits = rangeBits + + shardFmt := autoid.NewShardIDFormat(col.Tp, shardBits, rangeBits) + if shardFmt.IncrementalBits < autoid.AutoRandomIncBitsMin { + return dbterror.ErrInvalidAutoRandom.FastGenByArgs(autoid.AutoRandomIncrementalBitsTooSmall) + } + msg := fmt.Sprintf(autoid.AutoRandomAvailableAllocTimesNote, shardFmt.IncrementalBitsCapacity()) + ctx.GetSessionVars().StmtCtx.AppendNote(errors.NewNoStackError(msg)) + } + } + return nil +} + +func extractAutoRandomBitsFromColDef(colDef *ast.ColumnDef) (shardBits, rangeBits uint64, err error) { + for _, op := range colDef.Options { + if op.Tp == ast.ColumnOptionAutoRandom { + shardBits, err = autoid.AutoRandomShardBitsNormalize(op.AutoRandOpt.ShardBits, colDef.Name.Name.O) + if err != nil { + return 0, 0, err + } + rangeBits, err = autoid.AutoRandomRangeBitsNormalize(op.AutoRandOpt.RangeBits) + if err != nil { + return 0, 0, err + } + return shardBits, rangeBits, nil + } + } + return 0, 0, nil +} + +// handleTableOptions updates tableInfo according to table options. +func handleTableOptions(options []*ast.TableOption, tbInfo *model.TableInfo) error { + var ttlOptionsHandled bool + + for _, op := range options { + switch op.Tp { + case ast.TableOptionAutoIncrement: + tbInfo.AutoIncID = int64(op.UintValue) + case ast.TableOptionAutoIdCache: + if op.UintValue > uint64(math.MaxInt64) { + // TODO: Refine this error. + return errors.New("table option auto_id_cache overflows int64") + } + tbInfo.AutoIdCache = int64(op.UintValue) + case ast.TableOptionAutoRandomBase: + tbInfo.AutoRandID = int64(op.UintValue) + case ast.TableOptionComment: + tbInfo.Comment = op.StrValue + case ast.TableOptionCompression: + tbInfo.Compression = op.StrValue + case ast.TableOptionShardRowID: + if op.UintValue > 0 && tbInfo.HasClusteredIndex() { + return dbterror.ErrUnsupportedShardRowIDBits + } + tbInfo.ShardRowIDBits = op.UintValue + if tbInfo.ShardRowIDBits > shardRowIDBitsMax { + tbInfo.ShardRowIDBits = shardRowIDBitsMax + } + tbInfo.MaxShardRowIDBits = tbInfo.ShardRowIDBits + case ast.TableOptionPreSplitRegion: + if tbInfo.TempTableType != model.TempTableNone { + return errors.Trace(dbterror.ErrOptOnTemporaryTable.GenWithStackByArgs("pre split regions")) + } + tbInfo.PreSplitRegions = op.UintValue + case ast.TableOptionCharset, ast.TableOptionCollate: + // We don't handle charset and collate here since they're handled in `GetCharsetAndCollateInTableOption`. + case ast.TableOptionPlacementPolicy: + tbInfo.PlacementPolicyRef = &model.PolicyRefInfo{ + Name: model.NewCIStr(op.StrValue), + } + case ast.TableOptionTTL, ast.TableOptionTTLEnable, ast.TableOptionTTLJobInterval: + if ttlOptionsHandled { + continue + } + + ttlInfo, ttlEnable, ttlJobInterval, err := getTTLInfoInOptions(options) + if err != nil { + return err + } + // It's impossible that `ttlInfo` and `ttlEnable` are all nil, because we have met this option. + // After exclude the situation `ttlInfo == nil && ttlEnable != nil`, we could say `ttlInfo != nil` + if ttlInfo == nil { + if ttlEnable != nil { + return errors.Trace(dbterror.ErrSetTTLOptionForNonTTLTable.FastGenByArgs("TTL_ENABLE")) + } + if ttlJobInterval != nil { + return errors.Trace(dbterror.ErrSetTTLOptionForNonTTLTable.FastGenByArgs("TTL_JOB_INTERVAL")) + } + } + + tbInfo.TTLInfo = ttlInfo + ttlOptionsHandled = true + } + } + shardingBits := shardingBits(tbInfo) + if tbInfo.PreSplitRegions > shardingBits { + tbInfo.PreSplitRegions = shardingBits + } + return nil +} + +func setTemporaryType(_ sessionctx.Context, tbInfo *model.TableInfo, s *ast.CreateTableStmt) error { + switch s.TemporaryKeyword { + case ast.TemporaryGlobal: + tbInfo.TempTableType = model.TempTableGlobal + // "create global temporary table ... on commit preserve rows" + if !s.OnCommitDelete { + return errors.Trace(dbterror.ErrUnsupportedOnCommitPreserve) + } + case ast.TemporaryLocal: + tbInfo.TempTableType = model.TempTableLocal + default: + tbInfo.TempTableType = model.TempTableNone + } + return nil +} + +func buildColumnsAndConstraints( + ctx sessionctx.Context, + colDefs []*ast.ColumnDef, + constraints []*ast.Constraint, + tblCharset string, + tblCollate string, +) ([]*table.Column, []*ast.Constraint, error) { + // outPriKeyConstraint is the primary key constraint out of column definition. such as: create table t1 (id int , age int, primary key(id)); + var outPriKeyConstraint *ast.Constraint + for _, v := range constraints { + if v.Tp == ast.ConstraintPrimaryKey { + outPriKeyConstraint = v + break + } + } + cols := make([]*table.Column, 0, len(colDefs)) + colMap := make(map[string]*table.Column, len(colDefs)) + + for i, colDef := range colDefs { + if field_types.TiDBStrictIntegerDisplayWidth { + switch colDef.Tp.GetType() { + case mysql.TypeTiny: + // No warning for BOOL-like tinyint(1) + if colDef.Tp.GetFlen() != types.UnspecifiedLength && colDef.Tp.GetFlen() != 1 { + ctx.GetSessionVars().StmtCtx.AppendWarning( + dbterror.ErrWarnDeprecatedIntegerDisplayWidth.FastGenByArgs(), + ) + } + case mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong: + if colDef.Tp.GetFlen() != types.UnspecifiedLength { + ctx.GetSessionVars().StmtCtx.AppendWarning( + dbterror.ErrWarnDeprecatedIntegerDisplayWidth.FastGenByArgs(), + ) + } + } + } + col, cts, err := buildColumnAndConstraint(ctx, i, colDef, outPriKeyConstraint, tblCharset, tblCollate) + if err != nil { + return nil, nil, errors.Trace(err) + } + col.State = model.StatePublic + if mysql.HasZerofillFlag(col.GetFlag()) { + ctx.GetSessionVars().StmtCtx.AppendWarning( + dbterror.ErrWarnDeprecatedZerofill.FastGenByArgs(), + ) + } + constraints = append(constraints, cts...) + cols = append(cols, col) + colMap[colDef.Name.Name.L] = col + } + // Traverse table Constraints and set col.flag. + for _, v := range constraints { + setColumnFlagWithConstraint(colMap, v) + } + return cols, constraints, nil +} + +func setColumnFlagWithConstraint(colMap map[string]*table.Column, v *ast.Constraint) { + switch v.Tp { + case ast.ConstraintPrimaryKey: + for _, key := range v.Keys { + if key.Expr != nil { + continue + } + c, ok := colMap[key.Column.Name.L] + if !ok { + continue + } + c.AddFlag(mysql.PriKeyFlag) + // Primary key can not be NULL. + c.AddFlag(mysql.NotNullFlag) + setNoDefaultValueFlag(c, c.DefaultValue != nil) + } + case ast.ConstraintUniq, ast.ConstraintUniqIndex, ast.ConstraintUniqKey: + for i, key := range v.Keys { + if key.Expr != nil { + continue + } + c, ok := colMap[key.Column.Name.L] + if !ok { + continue + } + if i == 0 { + // Only the first column can be set + // if unique index has multi columns, + // the flag should be MultipleKeyFlag. + // See https://dev.mysql.com/doc/refman/5.7/en/show-columns.html + if len(v.Keys) > 1 { + c.AddFlag(mysql.MultipleKeyFlag) + } else { + c.AddFlag(mysql.UniqueKeyFlag) + } + } + } + case ast.ConstraintKey, ast.ConstraintIndex: + for i, key := range v.Keys { + if key.Expr != nil { + continue + } + c, ok := colMap[key.Column.Name.L] + if !ok { + continue + } + if i == 0 { + // Only the first column can be set. + c.AddFlag(mysql.MultipleKeyFlag) + } + } + } +} + +// BuildTableInfoWithLike builds a new table info according to CREATE TABLE ... LIKE statement. +func BuildTableInfoWithLike(ctx sessionctx.Context, ident ast.Ident, referTblInfo *model.TableInfo, s *ast.CreateTableStmt) (*model.TableInfo, error) { + // Check the referred table is a real table object. + if referTblInfo.IsSequence() || referTblInfo.IsView() { + return nil, dbterror.ErrWrongObject.GenWithStackByArgs(ident.Schema, referTblInfo.Name, "BASE TABLE") + } + tblInfo := *referTblInfo + if err := setTemporaryType(ctx, &tblInfo, s); err != nil { + return nil, errors.Trace(err) + } + // Check non-public column and adjust column offset. + newColumns := referTblInfo.Cols() + newIndices := make([]*model.IndexInfo, 0, len(tblInfo.Indices)) + for _, idx := range tblInfo.Indices { + if idx.State == model.StatePublic { + newIndices = append(newIndices, idx) + } + } + tblInfo.Columns = newColumns + tblInfo.Indices = newIndices + tblInfo.Name = ident.Name + tblInfo.AutoIncID = 0 + tblInfo.ForeignKeys = nil + // Ignore TiFlash replicas for temporary tables. + if s.TemporaryKeyword != ast.TemporaryNone { + tblInfo.TiFlashReplica = nil + } else if tblInfo.TiFlashReplica != nil { + replica := *tblInfo.TiFlashReplica + // Keep the tiflash replica setting, remove the replica available status. + replica.AvailablePartitionIDs = nil + replica.Available = false + tblInfo.TiFlashReplica = &replica + } + if referTblInfo.Partition != nil { + pi := *referTblInfo.Partition + pi.Definitions = make([]model.PartitionDefinition, len(referTblInfo.Partition.Definitions)) + copy(pi.Definitions, referTblInfo.Partition.Definitions) + tblInfo.Partition = &pi + } + + if referTblInfo.TTLInfo != nil { + tblInfo.TTLInfo = referTblInfo.TTLInfo.Clone() + } + renameCheckConstraint(&tblInfo) + return &tblInfo, nil +} + +func renameCheckConstraint(tblInfo *model.TableInfo) { + for _, cons := range tblInfo.Constraints { + cons.Name = model.NewCIStr("") + cons.Table = tblInfo.Name + } + setNameForConstraintInfo(tblInfo.Name.L, map[string]bool{}, tblInfo.Constraints) +} + +// BuildTableInfo creates a TableInfo. +func BuildTableInfo( + ctx sessionctx.Context, + tableName model.CIStr, + cols []*table.Column, + constraints []*ast.Constraint, + charset string, + collate string, +) (tbInfo *model.TableInfo, err error) { + tbInfo = &model.TableInfo{ + Name: tableName, + Version: model.CurrLatestTableInfoVersion, + Charset: charset, + Collate: collate, + } + tblColumns := make([]*table.Column, 0, len(cols)) + existedColsMap := make(map[string]struct{}, len(cols)) + for _, v := range cols { + v.ID = AllocateColumnID(tbInfo) + tbInfo.Columns = append(tbInfo.Columns, v.ToInfo()) + tblColumns = append(tblColumns, table.ToColumn(v.ToInfo())) + existedColsMap[v.Name.L] = struct{}{} + } + foreignKeyID := tbInfo.MaxForeignKeyID + for _, constr := range constraints { + // Build hidden columns if necessary. + hiddenCols, err := buildHiddenColumnInfoWithCheck(ctx, constr.Keys, model.NewCIStr(constr.Name), tbInfo, tblColumns) + if err != nil { + return nil, err + } + for _, hiddenCol := range hiddenCols { + hiddenCol.State = model.StatePublic + hiddenCol.ID = AllocateColumnID(tbInfo) + hiddenCol.Offset = len(tbInfo.Columns) + tbInfo.Columns = append(tbInfo.Columns, hiddenCol) + tblColumns = append(tblColumns, table.ToColumn(hiddenCol)) + } + // Check clustered on non-primary key. + if constr.Option != nil && constr.Option.PrimaryKeyTp != model.PrimaryKeyTypeDefault && + constr.Tp != ast.ConstraintPrimaryKey { + return nil, dbterror.ErrUnsupportedClusteredSecondaryKey + } + if constr.Tp == ast.ConstraintForeignKey { + var fkName model.CIStr + foreignKeyID++ + if constr.Name != "" { + fkName = model.NewCIStr(constr.Name) + } else { + fkName = model.NewCIStr(fmt.Sprintf("fk_%d", foreignKeyID)) + } + if model.FindFKInfoByName(tbInfo.ForeignKeys, fkName.L) != nil { + return nil, infoschema.ErrCannotAddForeign + } + fk, err := buildFKInfo(fkName, constr.Keys, constr.Refer, cols) + if err != nil { + return nil, err + } + fk.State = model.StatePublic + + tbInfo.ForeignKeys = append(tbInfo.ForeignKeys, fk) + continue + } + if constr.Tp == ast.ConstraintPrimaryKey { + lastCol, err := CheckPKOnGeneratedColumn(tbInfo, constr.Keys) + if err != nil { + return nil, err + } + isSingleIntPK := isSingleIntPK(constr, lastCol) + if ShouldBuildClusteredIndex(ctx, constr.Option, isSingleIntPK) { + if isSingleIntPK { + tbInfo.PKIsHandle = true + } else { + tbInfo.IsCommonHandle = true + tbInfo.CommonHandleVersion = 1 + } + } + if tbInfo.HasClusteredIndex() { + // Primary key cannot be invisible. + if constr.Option != nil && constr.Option.Visibility == ast.IndexVisibilityInvisible { + return nil, dbterror.ErrPKIndexCantBeInvisible + } + } + if tbInfo.PKIsHandle { + continue + } + } + + if constr.Tp == ast.ConstraintFulltext { + ctx.GetSessionVars().StmtCtx.AppendWarning(dbterror.ErrTableCantHandleFt.FastGenByArgs()) + continue + } + + var ( + indexName = constr.Name + primary, unique bool + ) + + // Check if the index is primary or unique. + switch constr.Tp { + case ast.ConstraintPrimaryKey: + primary = true + unique = true + indexName = mysql.PrimaryKeyName + case ast.ConstraintUniq, ast.ConstraintUniqKey, ast.ConstraintUniqIndex: + unique = true + } + + // check constraint + if constr.Tp == ast.ConstraintCheck { + if !variable.EnableCheckConstraint.Load() { + ctx.GetSessionVars().StmtCtx.AppendWarning(errCheckConstraintIsOff) + continue + } + // Since column check constraint dependency has been done in columnDefToCol. + // Here do the table check constraint dependency check, table constraint + // can only refer the columns in defined columns of the table. + // Refer: https://dev.mysql.com/doc/refman/8.0/en/create-table-check-constraints.html + if ok, err := table.IsSupportedExpr(constr); !ok { + return nil, err + } + var dependedCols []model.CIStr + dependedColsMap := findDependentColsInExpr(constr.Expr) + if !constr.InColumn { + dependedCols = make([]model.CIStr, 0, len(dependedColsMap)) + for k := range dependedColsMap { + if _, ok := existedColsMap[k]; !ok { + // The table constraint depended on a non-existed column. + return nil, dbterror.ErrTableCheckConstraintReferUnknown.GenWithStackByArgs(constr.Name, k) + } + dependedCols = append(dependedCols, model.NewCIStr(k)) + } + } else { + // Check the column-type constraint dependency. + if len(dependedColsMap) > 1 { + return nil, dbterror.ErrColumnCheckConstraintReferOther.GenWithStackByArgs(constr.Name) + } else if len(dependedColsMap) == 0 { + // If dependedCols is empty, the expression must be true/false. + valExpr, ok := constr.Expr.(*driver.ValueExpr) + if !ok || !mysql.HasIsBooleanFlag(valExpr.GetType().GetFlag()) { + return nil, errors.Trace(errors.New("unsupported expression in check constraint")) + } + } else { + if _, ok := dependedColsMap[constr.InColumnName]; !ok { + return nil, dbterror.ErrColumnCheckConstraintReferOther.GenWithStackByArgs(constr.Name) + } + dependedCols = []model.CIStr{model.NewCIStr(constr.InColumnName)} + } + } + // check auto-increment column + if table.ContainsAutoIncrementCol(dependedCols, tbInfo) { + return nil, dbterror.ErrCheckConstraintRefersAutoIncrementColumn.GenWithStackByArgs(constr.Name) + } + // check foreign key + if err := table.HasForeignKeyRefAction(tbInfo.ForeignKeys, constraints, constr, dependedCols); err != nil { + return nil, err + } + // build constraint meta info. + constraintInfo, err := buildConstraintInfo(tbInfo, dependedCols, constr, model.StatePublic) + if err != nil { + return nil, errors.Trace(err) + } + // check if the expression is bool type + if err := table.IfCheckConstraintExprBoolType(ctx.GetExprCtx().GetEvalCtx(), constraintInfo, tbInfo); err != nil { + return nil, err + } + constraintInfo.ID = allocateConstraintID(tbInfo) + tbInfo.Constraints = append(tbInfo.Constraints, constraintInfo) + continue + } + + // build index info. + idxInfo, err := BuildIndexInfo( + ctx, + tbInfo.Columns, + model.NewCIStr(indexName), + primary, + unique, + false, + constr.Keys, + constr.Option, + model.StatePublic, + ) + if err != nil { + return nil, errors.Trace(err) + } + + if len(hiddenCols) > 0 { + AddIndexColumnFlag(tbInfo, idxInfo) + } + sessionVars := ctx.GetSessionVars() + _, err = validateCommentLength(sessionVars.StmtCtx.ErrCtx(), sessionVars.SQLMode, idxInfo.Name.String(), &idxInfo.Comment, dbterror.ErrTooLongIndexComment) + if err != nil { + return nil, errors.Trace(err) + } + idxInfo.ID = AllocateIndexID(tbInfo) + tbInfo.Indices = append(tbInfo.Indices, idxInfo) + } + + err = addIndexForForeignKey(ctx, tbInfo) + return tbInfo, err +} + +// addIndexForForeignKey uses to auto create an index for the foreign key if the table doesn't have any index cover the +// foreign key columns. +func addIndexForForeignKey(ctx sessionctx.Context, tbInfo *model.TableInfo) error { + if len(tbInfo.ForeignKeys) == 0 { + return nil + } + var handleCol *model.ColumnInfo + if tbInfo.PKIsHandle { + handleCol = tbInfo.GetPkColInfo() + } + for _, fk := range tbInfo.ForeignKeys { + if fk.Version < model.FKVersion1 { + continue + } + if handleCol != nil && len(fk.Cols) == 1 && handleCol.Name.L == fk.Cols[0].L { + continue + } + if model.FindIndexByColumns(tbInfo, tbInfo.Indices, fk.Cols...) != nil { + continue + } + idxName := fk.Name + if tbInfo.FindIndexByName(idxName.L) != nil { + return dbterror.ErrDupKeyName.GenWithStack("duplicate key name %s", fk.Name.O) + } + keys := make([]*ast.IndexPartSpecification, 0, len(fk.Cols)) + for _, col := range fk.Cols { + keys = append(keys, &ast.IndexPartSpecification{ + Column: &ast.ColumnName{Name: col}, + Length: types.UnspecifiedLength, + }) + } + idxInfo, err := BuildIndexInfo(ctx, tbInfo.Columns, idxName, false, false, false, keys, nil, model.StatePublic) + if err != nil { + return errors.Trace(err) + } + idxInfo.ID = AllocateIndexID(tbInfo) + tbInfo.Indices = append(tbInfo.Indices, idxInfo) + } + return nil +} + +func isSingleIntPK(constr *ast.Constraint, lastCol *model.ColumnInfo) bool { + if len(constr.Keys) != 1 { + return false + } + switch lastCol.GetType() { + case mysql.TypeLong, mysql.TypeLonglong, + mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24: + return true + } + return false +} + +// ShouldBuildClusteredIndex is used to determine whether the CREATE TABLE statement should build a clustered index table. +func ShouldBuildClusteredIndex(ctx sessionctx.Context, opt *ast.IndexOption, isSingleIntPK bool) bool { + if opt == nil || opt.PrimaryKeyTp == model.PrimaryKeyTypeDefault { + switch ctx.GetSessionVars().EnableClusteredIndex { + case variable.ClusteredIndexDefModeOn: + return true + case variable.ClusteredIndexDefModeIntOnly: + return !config.GetGlobalConfig().AlterPrimaryKey && isSingleIntPK + default: + return false + } + } + return opt.PrimaryKeyTp == model.PrimaryKeyTypeClustered +} diff --git a/pkg/ddl/executor.go b/pkg/ddl/executor.go index 4fb85e0ff6fda..f6f8092e6c1c6 100644 --- a/pkg/ddl/executor.go +++ b/pkg/ddl/executor.go @@ -79,7 +79,6 @@ import ( "github.com/pingcap/tidb/pkg/util/hack" "github.com/pingcap/tidb/pkg/util/intest" "github.com/pingcap/tidb/pkg/util/mathutil" - "github.com/pingcap/tidb/pkg/util/mock" "github.com/pingcap/tidb/pkg/util/set" "github.com/pingcap/tidb/pkg/util/sqlkiller" "github.com/pingcap/tidb/pkg/util/stringutil" @@ -886,117 +885,6 @@ func checkTooLongForeignKey(fk model.CIStr) error { return nil } -func setColumnFlagWithConstraint(colMap map[string]*table.Column, v *ast.Constraint) { - switch v.Tp { - case ast.ConstraintPrimaryKey: - for _, key := range v.Keys { - if key.Expr != nil { - continue - } - c, ok := colMap[key.Column.Name.L] - if !ok { - continue - } - c.AddFlag(mysql.PriKeyFlag) - // Primary key can not be NULL. - c.AddFlag(mysql.NotNullFlag) - setNoDefaultValueFlag(c, c.DefaultValue != nil) - } - case ast.ConstraintUniq, ast.ConstraintUniqIndex, ast.ConstraintUniqKey: - for i, key := range v.Keys { - if key.Expr != nil { - continue - } - c, ok := colMap[key.Column.Name.L] - if !ok { - continue - } - if i == 0 { - // Only the first column can be set - // if unique index has multi columns, - // the flag should be MultipleKeyFlag. - // See https://dev.mysql.com/doc/refman/5.7/en/show-columns.html - if len(v.Keys) > 1 { - c.AddFlag(mysql.MultipleKeyFlag) - } else { - c.AddFlag(mysql.UniqueKeyFlag) - } - } - } - case ast.ConstraintKey, ast.ConstraintIndex: - for i, key := range v.Keys { - if key.Expr != nil { - continue - } - c, ok := colMap[key.Column.Name.L] - if !ok { - continue - } - if i == 0 { - // Only the first column can be set. - c.AddFlag(mysql.MultipleKeyFlag) - } - } - } -} - -func buildColumnsAndConstraints( - ctx sessionctx.Context, - colDefs []*ast.ColumnDef, - constraints []*ast.Constraint, - tblCharset string, - tblCollate string, -) ([]*table.Column, []*ast.Constraint, error) { - // outPriKeyConstraint is the primary key constraint out of column definition. such as: create table t1 (id int , age int, primary key(id)); - var outPriKeyConstraint *ast.Constraint - for _, v := range constraints { - if v.Tp == ast.ConstraintPrimaryKey { - outPriKeyConstraint = v - break - } - } - cols := make([]*table.Column, 0, len(colDefs)) - colMap := make(map[string]*table.Column, len(colDefs)) - - for i, colDef := range colDefs { - if field_types.TiDBStrictIntegerDisplayWidth { - switch colDef.Tp.GetType() { - case mysql.TypeTiny: - // No warning for BOOL-like tinyint(1) - if colDef.Tp.GetFlen() != types.UnspecifiedLength && colDef.Tp.GetFlen() != 1 { - ctx.GetSessionVars().StmtCtx.AppendWarning( - dbterror.ErrWarnDeprecatedIntegerDisplayWidth.FastGenByArgs(), - ) - } - case mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong: - if colDef.Tp.GetFlen() != types.UnspecifiedLength { - ctx.GetSessionVars().StmtCtx.AppendWarning( - dbterror.ErrWarnDeprecatedIntegerDisplayWidth.FastGenByArgs(), - ) - } - } - } - col, cts, err := buildColumnAndConstraint(ctx, i, colDef, outPriKeyConstraint, tblCharset, tblCollate) - if err != nil { - return nil, nil, errors.Trace(err) - } - col.State = model.StatePublic - if mysql.HasZerofillFlag(col.GetFlag()) { - ctx.GetSessionVars().StmtCtx.AppendWarning( - dbterror.ErrWarnDeprecatedZerofill.FastGenByArgs(), - ) - } - constraints = append(constraints, cts...) - cols = append(cols, col) - colMap[colDef.Name.Name.L] = col - } - // Traverse table Constraints and set col.flag. - for _, v := range constraints { - setColumnFlagWithConstraint(colMap, v) - } - return cols, constraints, nil -} - func getDefaultCollationForUTF8MB4(sessVars *variable.SessionVars, cs string) (string, error) { if sessVars == nil || cs != charset.CharsetUTF8MB4 { return "", nil @@ -2192,344 +2080,6 @@ func checkInvisibleIndexOnPK(tblInfo *model.TableInfo) error { return nil } -func setTableAutoRandomBits(ctx sessionctx.Context, tbInfo *model.TableInfo, colDefs []*ast.ColumnDef) error { - for _, col := range colDefs { - if containsColumnOption(col, ast.ColumnOptionAutoRandom) { - if col.Tp.GetType() != mysql.TypeLonglong { - return dbterror.ErrInvalidAutoRandom.GenWithStackByArgs( - fmt.Sprintf(autoid.AutoRandomOnNonBigIntColumn, types.TypeStr(col.Tp.GetType()))) - } - switch { - case tbInfo.PKIsHandle: - if tbInfo.GetPkName().L != col.Name.Name.L { - errMsg := fmt.Sprintf(autoid.AutoRandomMustFirstColumnInPK, col.Name.Name.O) - return dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(errMsg) - } - case tbInfo.IsCommonHandle: - pk := tables.FindPrimaryIndex(tbInfo) - if pk == nil { - return dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomNoClusteredPKErrMsg) - } - if col.Name.Name.L != pk.Columns[0].Name.L { - errMsg := fmt.Sprintf(autoid.AutoRandomMustFirstColumnInPK, col.Name.Name.O) - return dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(errMsg) - } - default: - return dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomNoClusteredPKErrMsg) - } - - if containsColumnOption(col, ast.ColumnOptionAutoIncrement) { - return dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomIncompatibleWithAutoIncErrMsg) - } - if containsColumnOption(col, ast.ColumnOptionDefaultValue) { - return dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomIncompatibleWithDefaultValueErrMsg) - } - - shardBits, rangeBits, err := extractAutoRandomBitsFromColDef(col) - if err != nil { - return errors.Trace(err) - } - tbInfo.AutoRandomBits = shardBits - tbInfo.AutoRandomRangeBits = rangeBits - - shardFmt := autoid.NewShardIDFormat(col.Tp, shardBits, rangeBits) - if shardFmt.IncrementalBits < autoid.AutoRandomIncBitsMin { - return dbterror.ErrInvalidAutoRandom.FastGenByArgs(autoid.AutoRandomIncrementalBitsTooSmall) - } - msg := fmt.Sprintf(autoid.AutoRandomAvailableAllocTimesNote, shardFmt.IncrementalBitsCapacity()) - ctx.GetSessionVars().StmtCtx.AppendNote(errors.NewNoStackError(msg)) - } - } - return nil -} - -func extractAutoRandomBitsFromColDef(colDef *ast.ColumnDef) (shardBits, rangeBits uint64, err error) { - for _, op := range colDef.Options { - if op.Tp == ast.ColumnOptionAutoRandom { - shardBits, err = autoid.AutoRandomShardBitsNormalize(op.AutoRandOpt.ShardBits, colDef.Name.Name.O) - if err != nil { - return 0, 0, err - } - rangeBits, err = autoid.AutoRandomRangeBitsNormalize(op.AutoRandOpt.RangeBits) - if err != nil { - return 0, 0, err - } - return shardBits, rangeBits, nil - } - } - return 0, 0, nil -} - -// BuildTableInfo creates a TableInfo. -func BuildTableInfo( - ctx sessionctx.Context, - tableName model.CIStr, - cols []*table.Column, - constraints []*ast.Constraint, - charset string, - collate string, -) (tbInfo *model.TableInfo, err error) { - tbInfo = &model.TableInfo{ - Name: tableName, - Version: model.CurrLatestTableInfoVersion, - Charset: charset, - Collate: collate, - } - tblColumns := make([]*table.Column, 0, len(cols)) - existedColsMap := make(map[string]struct{}, len(cols)) - for _, v := range cols { - v.ID = AllocateColumnID(tbInfo) - tbInfo.Columns = append(tbInfo.Columns, v.ToInfo()) - tblColumns = append(tblColumns, table.ToColumn(v.ToInfo())) - existedColsMap[v.Name.L] = struct{}{} - } - foreignKeyID := tbInfo.MaxForeignKeyID - for _, constr := range constraints { - // Build hidden columns if necessary. - hiddenCols, err := buildHiddenColumnInfoWithCheck(ctx, constr.Keys, model.NewCIStr(constr.Name), tbInfo, tblColumns) - if err != nil { - return nil, err - } - for _, hiddenCol := range hiddenCols { - hiddenCol.State = model.StatePublic - hiddenCol.ID = AllocateColumnID(tbInfo) - hiddenCol.Offset = len(tbInfo.Columns) - tbInfo.Columns = append(tbInfo.Columns, hiddenCol) - tblColumns = append(tblColumns, table.ToColumn(hiddenCol)) - } - // Check clustered on non-primary key. - if constr.Option != nil && constr.Option.PrimaryKeyTp != model.PrimaryKeyTypeDefault && - constr.Tp != ast.ConstraintPrimaryKey { - return nil, dbterror.ErrUnsupportedClusteredSecondaryKey - } - if constr.Tp == ast.ConstraintForeignKey { - var fkName model.CIStr - foreignKeyID++ - if constr.Name != "" { - fkName = model.NewCIStr(constr.Name) - } else { - fkName = model.NewCIStr(fmt.Sprintf("fk_%d", foreignKeyID)) - } - if model.FindFKInfoByName(tbInfo.ForeignKeys, fkName.L) != nil { - return nil, infoschema.ErrCannotAddForeign - } - fk, err := buildFKInfo(fkName, constr.Keys, constr.Refer, cols) - if err != nil { - return nil, err - } - fk.State = model.StatePublic - - tbInfo.ForeignKeys = append(tbInfo.ForeignKeys, fk) - continue - } - if constr.Tp == ast.ConstraintPrimaryKey { - lastCol, err := CheckPKOnGeneratedColumn(tbInfo, constr.Keys) - if err != nil { - return nil, err - } - isSingleIntPK := isSingleIntPK(constr, lastCol) - if ShouldBuildClusteredIndex(ctx, constr.Option, isSingleIntPK) { - if isSingleIntPK { - tbInfo.PKIsHandle = true - } else { - tbInfo.IsCommonHandle = true - tbInfo.CommonHandleVersion = 1 - } - } - if tbInfo.HasClusteredIndex() { - // Primary key cannot be invisible. - if constr.Option != nil && constr.Option.Visibility == ast.IndexVisibilityInvisible { - return nil, dbterror.ErrPKIndexCantBeInvisible - } - } - if tbInfo.PKIsHandle { - continue - } - } - - if constr.Tp == ast.ConstraintFulltext { - ctx.GetSessionVars().StmtCtx.AppendWarning(dbterror.ErrTableCantHandleFt.FastGenByArgs()) - continue - } - - var ( - indexName = constr.Name - primary, unique bool - ) - - // Check if the index is primary or unique. - switch constr.Tp { - case ast.ConstraintPrimaryKey: - primary = true - unique = true - indexName = mysql.PrimaryKeyName - case ast.ConstraintUniq, ast.ConstraintUniqKey, ast.ConstraintUniqIndex: - unique = true - } - - // check constraint - if constr.Tp == ast.ConstraintCheck { - if !variable.EnableCheckConstraint.Load() { - ctx.GetSessionVars().StmtCtx.AppendWarning(errCheckConstraintIsOff) - continue - } - // Since column check constraint dependency has been done in columnDefToCol. - // Here do the table check constraint dependency check, table constraint - // can only refer the columns in defined columns of the table. - // Refer: https://dev.mysql.com/doc/refman/8.0/en/create-table-check-constraints.html - if ok, err := table.IsSupportedExpr(constr); !ok { - return nil, err - } - var dependedCols []model.CIStr - dependedColsMap := findDependentColsInExpr(constr.Expr) - if !constr.InColumn { - dependedCols = make([]model.CIStr, 0, len(dependedColsMap)) - for k := range dependedColsMap { - if _, ok := existedColsMap[k]; !ok { - // The table constraint depended on a non-existed column. - return nil, dbterror.ErrTableCheckConstraintReferUnknown.GenWithStackByArgs(constr.Name, k) - } - dependedCols = append(dependedCols, model.NewCIStr(k)) - } - } else { - // Check the column-type constraint dependency. - if len(dependedColsMap) > 1 { - return nil, dbterror.ErrColumnCheckConstraintReferOther.GenWithStackByArgs(constr.Name) - } else if len(dependedColsMap) == 0 { - // If dependedCols is empty, the expression must be true/false. - valExpr, ok := constr.Expr.(*driver.ValueExpr) - if !ok || !mysql.HasIsBooleanFlag(valExpr.GetType().GetFlag()) { - return nil, errors.Trace(errors.New("unsupported expression in check constraint")) - } - } else { - if _, ok := dependedColsMap[constr.InColumnName]; !ok { - return nil, dbterror.ErrColumnCheckConstraintReferOther.GenWithStackByArgs(constr.Name) - } - dependedCols = []model.CIStr{model.NewCIStr(constr.InColumnName)} - } - } - // check auto-increment column - if table.ContainsAutoIncrementCol(dependedCols, tbInfo) { - return nil, dbterror.ErrCheckConstraintRefersAutoIncrementColumn.GenWithStackByArgs(constr.Name) - } - // check foreign key - if err := table.HasForeignKeyRefAction(tbInfo.ForeignKeys, constraints, constr, dependedCols); err != nil { - return nil, err - } - // build constraint meta info. - constraintInfo, err := buildConstraintInfo(tbInfo, dependedCols, constr, model.StatePublic) - if err != nil { - return nil, errors.Trace(err) - } - // check if the expression is bool type - if err := table.IfCheckConstraintExprBoolType(ctx.GetExprCtx().GetEvalCtx(), constraintInfo, tbInfo); err != nil { - return nil, err - } - constraintInfo.ID = allocateConstraintID(tbInfo) - tbInfo.Constraints = append(tbInfo.Constraints, constraintInfo) - continue - } - - // build index info. - idxInfo, err := BuildIndexInfo( - ctx, - tbInfo.Columns, - model.NewCIStr(indexName), - primary, - unique, - false, - constr.Keys, - constr.Option, - model.StatePublic, - ) - if err != nil { - return nil, errors.Trace(err) - } - - if len(hiddenCols) > 0 { - AddIndexColumnFlag(tbInfo, idxInfo) - } - sessionVars := ctx.GetSessionVars() - _, err = validateCommentLength(sessionVars.StmtCtx.ErrCtx(), sessionVars.SQLMode, idxInfo.Name.String(), &idxInfo.Comment, dbterror.ErrTooLongIndexComment) - if err != nil { - return nil, errors.Trace(err) - } - idxInfo.ID = AllocateIndexID(tbInfo) - tbInfo.Indices = append(tbInfo.Indices, idxInfo) - } - - err = addIndexForForeignKey(ctx, tbInfo) - return tbInfo, err -} - -// addIndexForForeignKey uses to auto create an index for the foreign key if the table doesn't have any index cover the -// foreign key columns. -func addIndexForForeignKey(ctx sessionctx.Context, tbInfo *model.TableInfo) error { - if len(tbInfo.ForeignKeys) == 0 { - return nil - } - var handleCol *model.ColumnInfo - if tbInfo.PKIsHandle { - handleCol = tbInfo.GetPkColInfo() - } - for _, fk := range tbInfo.ForeignKeys { - if fk.Version < model.FKVersion1 { - continue - } - if handleCol != nil && len(fk.Cols) == 1 && handleCol.Name.L == fk.Cols[0].L { - continue - } - if model.FindIndexByColumns(tbInfo, tbInfo.Indices, fk.Cols...) != nil { - continue - } - idxName := fk.Name - if tbInfo.FindIndexByName(idxName.L) != nil { - return dbterror.ErrDupKeyName.GenWithStack("duplicate key name %s", fk.Name.O) - } - keys := make([]*ast.IndexPartSpecification, 0, len(fk.Cols)) - for _, col := range fk.Cols { - keys = append(keys, &ast.IndexPartSpecification{ - Column: &ast.ColumnName{Name: col}, - Length: types.UnspecifiedLength, - }) - } - idxInfo, err := BuildIndexInfo(ctx, tbInfo.Columns, idxName, false, false, false, keys, nil, model.StatePublic) - if err != nil { - return errors.Trace(err) - } - idxInfo.ID = AllocateIndexID(tbInfo) - tbInfo.Indices = append(tbInfo.Indices, idxInfo) - } - return nil -} - -func isSingleIntPK(constr *ast.Constraint, lastCol *model.ColumnInfo) bool { - if len(constr.Keys) != 1 { - return false - } - switch lastCol.GetType() { - case mysql.TypeLong, mysql.TypeLonglong, - mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24: - return true - } - return false -} - -// ShouldBuildClusteredIndex is used to determine whether the CREATE TABLE statement should build a clustered index table. -func ShouldBuildClusteredIndex(ctx sessionctx.Context, opt *ast.IndexOption, isSingleIntPK bool) bool { - if opt == nil || opt.PrimaryKeyTp == model.PrimaryKeyTypeDefault { - switch ctx.GetSessionVars().EnableClusteredIndex { - case variable.ClusteredIndexDefModeOn: - return true - case variable.ClusteredIndexDefModeIntOnly: - return !config.GetGlobalConfig().AlterPrimaryKey && isSingleIntPK - default: - return false - } - } - return opt.PrimaryKeyTp == model.PrimaryKeyTypeClustered -} - // checkTableInfoValidExtra is like checkTableInfoValid, but also assumes the // table info comes from untrusted source and performs further checks such as // name length and column count. @@ -2641,171 +2191,6 @@ func checkTableInfoValid(tblInfo *model.TableInfo) error { return checkInvisibleIndexOnPK(tblInfo) } -// BuildTableInfoWithLike builds a new table info according to CREATE TABLE ... LIKE statement. -func BuildTableInfoWithLike(ctx sessionctx.Context, ident ast.Ident, referTblInfo *model.TableInfo, s *ast.CreateTableStmt) (*model.TableInfo, error) { - // Check the referred table is a real table object. - if referTblInfo.IsSequence() || referTblInfo.IsView() { - return nil, dbterror.ErrWrongObject.GenWithStackByArgs(ident.Schema, referTblInfo.Name, "BASE TABLE") - } - tblInfo := *referTblInfo - if err := setTemporaryType(ctx, &tblInfo, s); err != nil { - return nil, errors.Trace(err) - } - // Check non-public column and adjust column offset. - newColumns := referTblInfo.Cols() - newIndices := make([]*model.IndexInfo, 0, len(tblInfo.Indices)) - for _, idx := range tblInfo.Indices { - if idx.State == model.StatePublic { - newIndices = append(newIndices, idx) - } - } - tblInfo.Columns = newColumns - tblInfo.Indices = newIndices - tblInfo.Name = ident.Name - tblInfo.AutoIncID = 0 - tblInfo.ForeignKeys = nil - // Ignore TiFlash replicas for temporary tables. - if s.TemporaryKeyword != ast.TemporaryNone { - tblInfo.TiFlashReplica = nil - } else if tblInfo.TiFlashReplica != nil { - replica := *tblInfo.TiFlashReplica - // Keep the tiflash replica setting, remove the replica available status. - replica.AvailablePartitionIDs = nil - replica.Available = false - tblInfo.TiFlashReplica = &replica - } - if referTblInfo.Partition != nil { - pi := *referTblInfo.Partition - pi.Definitions = make([]model.PartitionDefinition, len(referTblInfo.Partition.Definitions)) - copy(pi.Definitions, referTblInfo.Partition.Definitions) - tblInfo.Partition = &pi - } - - if referTblInfo.TTLInfo != nil { - tblInfo.TTLInfo = referTblInfo.TTLInfo.Clone() - } - renameCheckConstraint(&tblInfo) - return &tblInfo, nil -} - -func renameCheckConstraint(tblInfo *model.TableInfo) { - for _, cons := range tblInfo.Constraints { - cons.Name = model.NewCIStr("") - cons.Table = tblInfo.Name - } - setNameForConstraintInfo(tblInfo.Name.L, map[string]bool{}, tblInfo.Constraints) -} - -// BuildTableInfoFromAST builds model.TableInfo from a SQL statement. -// Note: TableID and PartitionID are left as uninitialized value. -func BuildTableInfoFromAST(s *ast.CreateTableStmt) (*model.TableInfo, error) { - return buildTableInfoWithCheck(mock.NewContext(), s, mysql.DefaultCharset, "", nil) -} - -// buildTableInfoWithCheck builds model.TableInfo from a SQL statement. -// Note: TableID and PartitionIDs are left as uninitialized value. -func buildTableInfoWithCheck(ctx sessionctx.Context, s *ast.CreateTableStmt, dbCharset, dbCollate string, placementPolicyRef *model.PolicyRefInfo) (*model.TableInfo, error) { - tbInfo, err := BuildTableInfoWithStmt(ctx, s, dbCharset, dbCollate, placementPolicyRef) - if err != nil { - return nil, err - } - // Fix issue 17952 which will cause partition range expr can't be parsed as Int. - // checkTableInfoValidWithStmt will do the constant fold the partition expression first, - // then checkTableInfoValidExtra will pass the tableInfo check successfully. - if err = checkTableInfoValidWithStmt(ctx, tbInfo, s); err != nil { - return nil, err - } - if err = checkTableInfoValidExtra(tbInfo); err != nil { - return nil, err - } - return tbInfo, nil -} - -// BuildSessionTemporaryTableInfo builds model.TableInfo from a SQL statement. -func BuildSessionTemporaryTableInfo(ctx sessionctx.Context, is infoschema.InfoSchema, s *ast.CreateTableStmt, dbCharset, dbCollate string, placementPolicyRef *model.PolicyRefInfo) (*model.TableInfo, error) { - ident := ast.Ident{Schema: s.Table.Schema, Name: s.Table.Name} - //build tableInfo - var tbInfo *model.TableInfo - var referTbl table.Table - var err error - if s.ReferTable != nil { - referIdent := ast.Ident{Schema: s.ReferTable.Schema, Name: s.ReferTable.Name} - _, ok := is.SchemaByName(referIdent.Schema) - if !ok { - return nil, infoschema.ErrTableNotExists.GenWithStackByArgs(referIdent.Schema, referIdent.Name) - } - referTbl, err = is.TableByName(context.Background(), referIdent.Schema, referIdent.Name) - if err != nil { - return nil, infoschema.ErrTableNotExists.GenWithStackByArgs(referIdent.Schema, referIdent.Name) - } - tbInfo, err = BuildTableInfoWithLike(ctx, ident, referTbl.Meta(), s) - } else { - tbInfo, err = buildTableInfoWithCheck(ctx, s, dbCharset, dbCollate, placementPolicyRef) - } - return tbInfo, err -} - -// BuildTableInfoWithStmt builds model.TableInfo from a SQL statement without validity check -func BuildTableInfoWithStmt(ctx sessionctx.Context, s *ast.CreateTableStmt, dbCharset, dbCollate string, placementPolicyRef *model.PolicyRefInfo) (*model.TableInfo, error) { - colDefs := s.Cols - tableCharset, tableCollate, err := GetCharsetAndCollateInTableOption(ctx.GetSessionVars(), 0, s.Options) - if err != nil { - return nil, errors.Trace(err) - } - tableCharset, tableCollate, err = ResolveCharsetCollation(ctx.GetSessionVars(), - ast.CharsetOpt{Chs: tableCharset, Col: tableCollate}, - ast.CharsetOpt{Chs: dbCharset, Col: dbCollate}, - ) - if err != nil { - return nil, errors.Trace(err) - } - - // The column charset haven't been resolved here. - cols, newConstraints, err := buildColumnsAndConstraints(ctx, colDefs, s.Constraints, tableCharset, tableCollate) - if err != nil { - return nil, errors.Trace(err) - } - err = checkConstraintNames(s.Table.Name, newConstraints) - if err != nil { - return nil, errors.Trace(err) - } - - var tbInfo *model.TableInfo - tbInfo, err = BuildTableInfo(ctx, s.Table.Name, cols, newConstraints, tableCharset, tableCollate) - if err != nil { - return nil, errors.Trace(err) - } - if err = setTemporaryType(ctx, tbInfo, s); err != nil { - return nil, errors.Trace(err) - } - - if err = setTableAutoRandomBits(ctx, tbInfo, colDefs); err != nil { - return nil, errors.Trace(err) - } - - if err = handleTableOptions(s.Options, tbInfo); err != nil { - return nil, errors.Trace(err) - } - - sessionVars := ctx.GetSessionVars() - if _, err = validateCommentLength(sessionVars.StmtCtx.ErrCtx(), sessionVars.SQLMode, tbInfo.Name.L, &tbInfo.Comment, dbterror.ErrTooLongTableComment); err != nil { - return nil, errors.Trace(err) - } - - if tbInfo.TempTableType == model.TempTableNone && tbInfo.PlacementPolicyRef == nil && placementPolicyRef != nil { - // Set the defaults from Schema. Note: they are mutual exclusive! - tbInfo.PlacementPolicyRef = placementPolicyRef - } - - // After handleTableOptions, so the partitions can get defaults from Table level - err = buildTablePartitionInfo(ctx, s.Partition, tbInfo) - if err != nil { - return nil, errors.Trace(err) - } - - return tbInfo, nil -} - func (e *executor) assignPartitionIDs(defs []model.PartitionDefinition) error { genIDs, err := e.genGlobalIDs(len(defs)) if err != nil { @@ -2872,22 +2257,6 @@ func (e *executor) CreateTable(ctx sessionctx.Context, s *ast.CreateTableStmt) ( return e.CreateTableWithInfo(ctx, schema.Name, tbInfo, involvingRef, WithOnExist(onExist)) } -func setTemporaryType(_ sessionctx.Context, tbInfo *model.TableInfo, s *ast.CreateTableStmt) error { - switch s.TemporaryKeyword { - case ast.TemporaryGlobal: - tbInfo.TempTableType = model.TempTableGlobal - // "create global temporary table ... on commit preserve rows" - if !s.OnCommitDelete { - return errors.Trace(dbterror.ErrUnsupportedOnCommitPreserve) - } - case ast.TemporaryLocal: - tbInfo.TempTableType = model.TempTableLocal - default: - tbInfo.TempTableType = model.TempTableNone - } - return nil -} - // 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( @@ -3819,77 +3188,6 @@ func parseBackgroundJobTypes(t string) ([]string, error) { return res, nil } -// handleTableOptions updates tableInfo according to table options. -func handleTableOptions(options []*ast.TableOption, tbInfo *model.TableInfo) error { - var ttlOptionsHandled bool - - for _, op := range options { - switch op.Tp { - case ast.TableOptionAutoIncrement: - tbInfo.AutoIncID = int64(op.UintValue) - case ast.TableOptionAutoIdCache: - if op.UintValue > uint64(math.MaxInt64) { - // TODO: Refine this error. - return errors.New("table option auto_id_cache overflows int64") - } - tbInfo.AutoIdCache = int64(op.UintValue) - case ast.TableOptionAutoRandomBase: - tbInfo.AutoRandID = int64(op.UintValue) - case ast.TableOptionComment: - tbInfo.Comment = op.StrValue - case ast.TableOptionCompression: - tbInfo.Compression = op.StrValue - case ast.TableOptionShardRowID: - if op.UintValue > 0 && tbInfo.HasClusteredIndex() { - return dbterror.ErrUnsupportedShardRowIDBits - } - tbInfo.ShardRowIDBits = op.UintValue - if tbInfo.ShardRowIDBits > shardRowIDBitsMax { - tbInfo.ShardRowIDBits = shardRowIDBitsMax - } - tbInfo.MaxShardRowIDBits = tbInfo.ShardRowIDBits - case ast.TableOptionPreSplitRegion: - if tbInfo.TempTableType != model.TempTableNone { - return errors.Trace(dbterror.ErrOptOnTemporaryTable.GenWithStackByArgs("pre split regions")) - } - tbInfo.PreSplitRegions = op.UintValue - case ast.TableOptionCharset, ast.TableOptionCollate: - // We don't handle charset and collate here since they're handled in `GetCharsetAndCollateInTableOption`. - case ast.TableOptionPlacementPolicy: - tbInfo.PlacementPolicyRef = &model.PolicyRefInfo{ - Name: model.NewCIStr(op.StrValue), - } - case ast.TableOptionTTL, ast.TableOptionTTLEnable, ast.TableOptionTTLJobInterval: - if ttlOptionsHandled { - continue - } - - ttlInfo, ttlEnable, ttlJobInterval, err := getTTLInfoInOptions(options) - if err != nil { - return err - } - // It's impossible that `ttlInfo` and `ttlEnable` are all nil, because we have met this option. - // After exclude the situation `ttlInfo == nil && ttlEnable != nil`, we could say `ttlInfo != nil` - if ttlInfo == nil { - if ttlEnable != nil { - return errors.Trace(dbterror.ErrSetTTLOptionForNonTTLTable.FastGenByArgs("TTL_ENABLE")) - } - if ttlJobInterval != nil { - return errors.Trace(dbterror.ErrSetTTLOptionForNonTTLTable.FastGenByArgs("TTL_JOB_INTERVAL")) - } - } - - tbInfo.TTLInfo = ttlInfo - ttlOptionsHandled = true - } - } - shardingBits := shardingBits(tbInfo) - if tbInfo.PreSplitRegions > shardingBits { - tbInfo.PreSplitRegions = shardingBits - } - return nil -} - func shardingBits(tblInfo *model.TableInfo) uint64 { if tblInfo.ShardRowIDBits > 0 { return tblInfo.ShardRowIDBits From 0e892940e554cbcf16230ac36aa7513a268a842f Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Thu, 1 Aug 2024 16:26:07 +0800 Subject: [PATCH 03/13] split --- pkg/ddl/add_column.go | 1135 +++++++++++++++++++++++++++++++++ pkg/ddl/create_table.go | 188 ++++++ pkg/ddl/executor.go | 1292 -------------------------------------- pkg/ddl/modify_column.go | 17 + 4 files changed, 1340 insertions(+), 1292 deletions(-) create mode 100644 pkg/ddl/add_column.go diff --git a/pkg/ddl/add_column.go b/pkg/ddl/add_column.go new file mode 100644 index 0000000000000..0413bb12fbf2e --- /dev/null +++ b/pkg/ddl/add_column.go @@ -0,0 +1,1135 @@ +// 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 ddl + +import ( + "fmt" + "strconv" + "strings" + "time" + "unicode/utf8" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/pkg/config" + "github.com/pingcap/tidb/pkg/ddl/logutil" + "github.com/pingcap/tidb/pkg/errctx" + "github.com/pingcap/tidb/pkg/expression" + exprctx "github.com/pingcap/tidb/pkg/expression/context" + "github.com/pingcap/tidb/pkg/infoschema" + "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/parser/charset" + "github.com/pingcap/tidb/pkg/parser/format" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/parser/terror" + field_types "github.com/pingcap/tidb/pkg/parser/types" + "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/table" + "github.com/pingcap/tidb/pkg/types" + driver "github.com/pingcap/tidb/pkg/types/parser_driver" + "github.com/pingcap/tidb/pkg/util/collate" + "github.com/pingcap/tidb/pkg/util/dbterror" + "github.com/pingcap/tidb/pkg/util/hack" + "go.uber.org/zap" +) + +func checkAndCreateNewColumn(ctx sessionctx.Context, ti ast.Ident, schema *model.DBInfo, spec *ast.AlterTableSpec, t table.Table, specNewColumn *ast.ColumnDef) (*table.Column, error) { + err := checkUnsupportedColumnConstraint(specNewColumn, ti) + if err != nil { + return nil, errors.Trace(err) + } + + colName := specNewColumn.Name.Name.O + // Check whether added column has existed. + col := table.FindCol(t.Cols(), colName) + if col != nil { + err = infoschema.ErrColumnExists.GenWithStackByArgs(colName) + if spec.IfNotExists { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + return nil, nil + } + return nil, err + } + if err = checkColumnAttributes(colName, specNewColumn.Tp); err != nil { + return nil, errors.Trace(err) + } + if utf8.RuneCountInString(colName) > mysql.MaxColumnNameLength { + return nil, dbterror.ErrTooLongIdent.GenWithStackByArgs(colName) + } + + return CreateNewColumn(ctx, schema, spec, t, specNewColumn) +} + +// CreateNewColumn creates a new column according to the column information. +func CreateNewColumn(ctx sessionctx.Context, schema *model.DBInfo, spec *ast.AlterTableSpec, t table.Table, specNewColumn *ast.ColumnDef) (*table.Column, error) { + // If new column is a generated column, do validation. + // NOTE: we do check whether the column refers other generated + // columns occurring later in a table, but we don't handle the col offset. + for _, option := range specNewColumn.Options { + if option.Tp == ast.ColumnOptionGenerated { + if err := checkIllegalFn4Generated(specNewColumn.Name.Name.L, typeColumn, option.Expr); err != nil { + return nil, errors.Trace(err) + } + + if option.Stored { + return nil, dbterror.ErrUnsupportedOnGeneratedColumn.GenWithStackByArgs("Adding generated stored column through ALTER TABLE") + } + + _, dependColNames, err := findDependedColumnNames(schema.Name, t.Meta().Name, specNewColumn) + if err != nil { + return nil, errors.Trace(err) + } + if !ctx.GetSessionVars().EnableAutoIncrementInGenerated { + if err := checkAutoIncrementRef(specNewColumn.Name.Name.L, dependColNames, t.Meta()); err != nil { + return nil, errors.Trace(err) + } + } + duplicateColNames := make(map[string]struct{}, len(dependColNames)) + for k := range dependColNames { + duplicateColNames[k] = struct{}{} + } + cols := t.Cols() + + if err := checkDependedColExist(dependColNames, cols); err != nil { + return nil, errors.Trace(err) + } + + if err := verifyColumnGenerationSingle(duplicateColNames, cols, spec.Position); err != nil { + return nil, errors.Trace(err) + } + } + // Specially, since sequence has been supported, if a newly added column has a + // sequence nextval function as it's default value option, it won't fill the + // known rows with specific sequence next value under current add column logic. + // More explanation can refer: TestSequenceDefaultLogic's comment in sequence_test.go + if option.Tp == ast.ColumnOptionDefaultValue { + if f, ok := option.Expr.(*ast.FuncCallExpr); ok { + switch f.FnName.L { + case ast.NextVal: + if _, err := getSequenceDefaultValue(option); err != nil { + return nil, errors.Trace(err) + } + return nil, errors.Trace(dbterror.ErrAddColumnWithSequenceAsDefault.GenWithStackByArgs(specNewColumn.Name.Name.O)) + case ast.Rand, ast.UUID, ast.UUIDToBin, ast.Replace, ast.Upper: + return nil, errors.Trace(dbterror.ErrBinlogUnsafeSystemFunction.GenWithStackByArgs()) + } + } + } + } + + tableCharset, tableCollate, err := ResolveCharsetCollation(ctx.GetSessionVars(), + ast.CharsetOpt{Chs: t.Meta().Charset, Col: t.Meta().Collate}, + ast.CharsetOpt{Chs: schema.Charset, Col: schema.Collate}, + ) + if err != nil { + return nil, errors.Trace(err) + } + // Ignore table constraints now, they will be checked later. + // We use length(t.Cols()) as the default offset firstly, we will change the column's offset later. + col, _, err := buildColumnAndConstraint( + ctx, + len(t.Cols()), + specNewColumn, + nil, + tableCharset, + tableCollate, + ) + if err != nil { + return nil, errors.Trace(err) + } + + originDefVal, err := generateOriginDefaultValue(col.ToInfo(), ctx) + if err != nil { + return nil, errors.Trace(err) + } + + err = col.SetOriginDefaultValue(originDefVal) + return col, err +} + +// buildColumnAndConstraint builds table.Column and ast.Constraint from the parameters. +// outPriKeyConstraint is the primary key constraint out of column definition. For example: +// `create table t1 (id int , age int, primary key(id));` +func buildColumnAndConstraint( + ctx sessionctx.Context, + offset int, + colDef *ast.ColumnDef, + outPriKeyConstraint *ast.Constraint, + tblCharset string, + tblCollate string, +) (*table.Column, []*ast.Constraint, error) { + if colName := colDef.Name.Name.L; colName == model.ExtraHandleName.L { + return nil, nil, dbterror.ErrWrongColumnName.GenWithStackByArgs(colName) + } + + // specifiedCollate refers to the last collate specified in colDef.Options. + chs, coll, err := getCharsetAndCollateInColumnDef(ctx.GetSessionVars(), colDef) + if err != nil { + return nil, nil, errors.Trace(err) + } + chs, coll, err = ResolveCharsetCollation(ctx.GetSessionVars(), + ast.CharsetOpt{Chs: chs, Col: coll}, + ast.CharsetOpt{Chs: tblCharset, Col: tblCollate}, + ) + chs, coll = OverwriteCollationWithBinaryFlag(ctx.GetSessionVars(), colDef, chs, coll) + if err != nil { + return nil, nil, errors.Trace(err) + } + + if err := setCharsetCollationFlenDecimal(colDef.Tp, colDef.Name.Name.O, chs, coll, ctx.GetSessionVars()); err != nil { + return nil, nil, errors.Trace(err) + } + decodeEnumSetBinaryLiteralToUTF8(colDef.Tp, chs) + col, cts, err := columnDefToCol(ctx, offset, colDef, outPriKeyConstraint) + if err != nil { + return nil, nil, errors.Trace(err) + } + return col, cts, nil +} + +// OverwriteCollationWithBinaryFlag is used to handle the case like +// +// CREATE TABLE t (a VARCHAR(255) BINARY) CHARSET utf8 COLLATE utf8_general_ci; +// +// The 'BINARY' sets the column collation to *_bin according to the table charset. +func OverwriteCollationWithBinaryFlag(sessVars *variable.SessionVars, colDef *ast.ColumnDef, chs, coll string) (newChs string, newColl string) { + ignoreBinFlag := colDef.Tp.GetCharset() != "" && (colDef.Tp.GetCollate() != "" || containsColumnOption(colDef, ast.ColumnOptionCollate)) + if ignoreBinFlag { + return chs, coll + } + needOverwriteBinColl := types.IsString(colDef.Tp.GetType()) && mysql.HasBinaryFlag(colDef.Tp.GetFlag()) + if needOverwriteBinColl { + newColl, err := GetDefaultCollation(sessVars, chs) + if err != nil { + return chs, coll + } + return chs, newColl + } + return chs, coll +} + +func setCharsetCollationFlenDecimal(tp *types.FieldType, colName, colCharset, colCollate string, sessVars *variable.SessionVars) error { + var err error + if typesNeedCharset(tp.GetType()) { + tp.SetCharset(colCharset) + tp.SetCollate(colCollate) + } else { + tp.SetCharset(charset.CharsetBin) + tp.SetCollate(charset.CharsetBin) + } + + // Use default value for flen or decimal when they are unspecified. + defaultFlen, defaultDecimal := mysql.GetDefaultFieldLengthAndDecimal(tp.GetType()) + if tp.GetDecimal() == types.UnspecifiedLength { + tp.SetDecimal(defaultDecimal) + } + if tp.GetFlen() == types.UnspecifiedLength { + tp.SetFlen(defaultFlen) + if mysql.HasUnsignedFlag(tp.GetFlag()) && tp.GetType() != mysql.TypeLonglong && mysql.IsIntegerType(tp.GetType()) { + // Issue #4684: the flen of unsigned integer(except bigint) is 1 digit shorter than signed integer + // because it has no prefix "+" or "-" character. + tp.SetFlen(tp.GetFlen() - 1) + } + } else { + // Adjust the field type for blob/text types if the flen is set. + if err = adjustBlobTypesFlen(tp, colCharset); err != nil { + return err + } + } + return checkTooBigFieldLengthAndTryAutoConvert(tp, colName, sessVars) +} + +func decodeEnumSetBinaryLiteralToUTF8(tp *types.FieldType, chs string) { + if tp.GetType() != mysql.TypeEnum && tp.GetType() != mysql.TypeSet { + return + } + enc := charset.FindEncoding(chs) + for i, elem := range tp.GetElems() { + if !tp.GetElemIsBinaryLit(i) { + continue + } + s, err := enc.Transform(nil, hack.Slice(elem), charset.OpDecodeReplace) + if err != nil { + logutil.DDLLogger().Warn("decode enum binary literal to utf-8 failed", zap.Error(err)) + } + tp.SetElem(i, string(hack.String(s))) + } + tp.CleanElemIsBinaryLit() +} + +func typesNeedCharset(tp byte) bool { + switch tp { + case mysql.TypeString, mysql.TypeVarchar, mysql.TypeVarString, + mysql.TypeBlob, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, + mysql.TypeEnum, mysql.TypeSet: + return true + } + return false +} + +// checkTooBigFieldLengthAndTryAutoConvert will check whether the field length is too big +// in non-strict mode and varchar column. If it is, will try to adjust to blob or text, see issue #30328 +func checkTooBigFieldLengthAndTryAutoConvert(tp *types.FieldType, colName string, sessVars *variable.SessionVars) error { + if sessVars != nil && !sessVars.SQLMode.HasStrictMode() && tp.GetType() == mysql.TypeVarchar { + err := types.IsVarcharTooBigFieldLength(tp.GetFlen(), colName, tp.GetCharset()) + if err != nil && terror.ErrorEqual(types.ErrTooBigFieldLength, err) { + tp.SetType(mysql.TypeBlob) + if err = adjustBlobTypesFlen(tp, tp.GetCharset()); err != nil { + return err + } + if tp.GetCharset() == charset.CharsetBin { + sessVars.StmtCtx.AppendWarning(dbterror.ErrAutoConvert.FastGenByArgs(colName, "VARBINARY", "BLOB")) + } else { + sessVars.StmtCtx.AppendWarning(dbterror.ErrAutoConvert.FastGenByArgs(colName, "VARCHAR", "TEXT")) + } + } + } + return nil +} + +// columnDefToCol converts ColumnDef to Col and TableConstraints. +// outPriKeyConstraint is the primary key constraint out of column definition. such as: create table t1 (id int , age int, primary key(id)); +func columnDefToCol(ctx sessionctx.Context, offset int, colDef *ast.ColumnDef, outPriKeyConstraint *ast.Constraint) (*table.Column, []*ast.Constraint, error) { + var constraints = make([]*ast.Constraint, 0) + col := table.ToColumn(&model.ColumnInfo{ + Offset: offset, + Name: colDef.Name.Name, + FieldType: *colDef.Tp, + // TODO: remove this version field after there is no old version. + Version: model.CurrLatestColumnInfoVersion, + }) + + if !isExplicitTimeStamp() { + // Check and set TimestampFlag, OnUpdateNowFlag and NotNullFlag. + if col.GetType() == mysql.TypeTimestamp { + col.AddFlag(mysql.TimestampFlag | mysql.OnUpdateNowFlag | mysql.NotNullFlag) + } + } + var err error + setOnUpdateNow := false + hasDefaultValue := false + hasNullFlag := false + if colDef.Options != nil { + length := types.UnspecifiedLength + + keys := []*ast.IndexPartSpecification{ + { + Column: colDef.Name, + Length: length, + }, + } + + var sb strings.Builder + restoreFlags := format.RestoreStringSingleQuotes | format.RestoreKeyWordLowercase | format.RestoreNameBackQuotes | + format.RestoreSpacesAroundBinaryOperation | format.RestoreWithoutSchemaName | format.RestoreWithoutTableName + restoreCtx := format.NewRestoreCtx(restoreFlags, &sb) + + for _, v := range colDef.Options { + switch v.Tp { + case ast.ColumnOptionNotNull: + col.AddFlag(mysql.NotNullFlag) + case ast.ColumnOptionNull: + col.DelFlag(mysql.NotNullFlag) + removeOnUpdateNowFlag(col) + hasNullFlag = true + case ast.ColumnOptionAutoIncrement: + col.AddFlag(mysql.AutoIncrementFlag | mysql.NotNullFlag) + case ast.ColumnOptionPrimaryKey: + // Check PriKeyFlag first to avoid extra duplicate constraints. + if col.GetFlag()&mysql.PriKeyFlag == 0 { + constraint := &ast.Constraint{Tp: ast.ConstraintPrimaryKey, Keys: keys, + Option: &ast.IndexOption{PrimaryKeyTp: v.PrimaryKeyTp}} + constraints = append(constraints, constraint) + col.AddFlag(mysql.PriKeyFlag) + // Add NotNullFlag early so that processColumnFlags() can see it. + col.AddFlag(mysql.NotNullFlag) + } + case ast.ColumnOptionUniqKey: + // Check UniqueFlag first to avoid extra duplicate constraints. + if col.GetFlag()&mysql.UniqueFlag == 0 { + constraint := &ast.Constraint{Tp: ast.ConstraintUniqKey, Keys: keys} + constraints = append(constraints, constraint) + col.AddFlag(mysql.UniqueKeyFlag) + } + case ast.ColumnOptionDefaultValue: + hasDefaultValue, err = SetDefaultValue(ctx, col, v) + if err != nil { + return nil, nil, errors.Trace(err) + } + removeOnUpdateNowFlag(col) + case ast.ColumnOptionOnUpdate: + // TODO: Support other time functions. + if !(col.GetType() == mysql.TypeTimestamp || col.GetType() == mysql.TypeDatetime) { + return nil, nil, dbterror.ErrInvalidOnUpdate.GenWithStackByArgs(col.Name) + } + if !expression.IsValidCurrentTimestampExpr(v.Expr, colDef.Tp) { + return nil, nil, dbterror.ErrInvalidOnUpdate.GenWithStackByArgs(col.Name) + } + col.AddFlag(mysql.OnUpdateNowFlag) + setOnUpdateNow = true + case ast.ColumnOptionComment: + err := setColumnComment(ctx, col, v) + if err != nil { + return nil, nil, errors.Trace(err) + } + case ast.ColumnOptionGenerated: + sb.Reset() + err = v.Expr.Restore(restoreCtx) + if err != nil { + return nil, nil, errors.Trace(err) + } + col.GeneratedExprString = sb.String() + col.GeneratedStored = v.Stored + _, dependColNames, err := findDependedColumnNames(model.NewCIStr(""), model.NewCIStr(""), colDef) + if err != nil { + return nil, nil, errors.Trace(err) + } + col.Dependences = dependColNames + case ast.ColumnOptionCollate: + if field_types.HasCharset(colDef.Tp) { + col.FieldType.SetCollate(v.StrValue) + } + case ast.ColumnOptionFulltext: + ctx.GetSessionVars().StmtCtx.AppendWarning(dbterror.ErrTableCantHandleFt.FastGenByArgs()) + case ast.ColumnOptionCheck: + if !variable.EnableCheckConstraint.Load() { + ctx.GetSessionVars().StmtCtx.AppendWarning(errCheckConstraintIsOff) + } else { + // Check the column CHECK constraint dependency lazily, after fill all the name. + // Extract column constraint from column option. + constraint := &ast.Constraint{ + Tp: ast.ConstraintCheck, + Expr: v.Expr, + Enforced: v.Enforced, + Name: v.ConstraintName, + InColumn: true, + InColumnName: colDef.Name.Name.O, + } + constraints = append(constraints, constraint) + } + } + } + } + + if err = processAndCheckDefaultValueAndColumn(ctx, col, outPriKeyConstraint, hasDefaultValue, setOnUpdateNow, hasNullFlag); err != nil { + return nil, nil, errors.Trace(err) + } + return col, constraints, nil +} + +// isExplicitTimeStamp is used to check if explicit_defaults_for_timestamp is on or off. +// Check out this link for more details. +// https://dev.mysql.com/doc/refman/5.7/en/server-system-variables.html#sysvar_explicit_defaults_for_timestamp +func isExplicitTimeStamp() bool { + // TODO: implement the behavior as MySQL when explicit_defaults_for_timestamp = off, then this function could return false. + return true +} + +// SetDefaultValue sets the default value of the column. +func SetDefaultValue(ctx sessionctx.Context, col *table.Column, option *ast.ColumnOption) (hasDefaultValue bool, err error) { + var value any + var isSeqExpr bool + value, isSeqExpr, err = getDefaultValue( + exprctx.CtxWithHandleTruncateErrLevel(ctx.GetExprCtx(), errctx.LevelError), + col, option, + ) + if err != nil { + return false, errors.Trace(err) + } + if isSeqExpr { + if err := checkSequenceDefaultValue(col); err != nil { + return false, errors.Trace(err) + } + col.DefaultIsExpr = isSeqExpr + } + + // When the default value is expression, we skip check and convert. + if !col.DefaultIsExpr { + if hasDefaultValue, value, err = checkColumnDefaultValue(ctx.GetExprCtx(), col, value); err != nil { + return hasDefaultValue, errors.Trace(err) + } + value, err = convertTimestampDefaultValToUTC(ctx, value, col) + if err != nil { + return hasDefaultValue, errors.Trace(err) + } + } else { + hasDefaultValue = true + } + err = setDefaultValueWithBinaryPadding(col, value) + if err != nil { + return hasDefaultValue, errors.Trace(err) + } + return hasDefaultValue, nil +} + +// getFuncCallDefaultValue gets the default column value of function-call expression. +func getFuncCallDefaultValue(col *table.Column, option *ast.ColumnOption, expr *ast.FuncCallExpr) (any, bool, error) { + switch expr.FnName.L { + case ast.CurrentTimestamp, ast.CurrentDate: // CURRENT_TIMESTAMP() and CURRENT_DATE() + tp, fsp := col.FieldType.GetType(), col.FieldType.GetDecimal() + if tp == mysql.TypeTimestamp || tp == mysql.TypeDatetime { + defaultFsp := 0 + if len(expr.Args) == 1 { + if val := expr.Args[0].(*driver.ValueExpr); val != nil { + defaultFsp = int(val.GetInt64()) + } + } + if defaultFsp != fsp { + return nil, false, dbterror.ErrInvalidDefaultValue.GenWithStackByArgs(col.Name.O) + } + } + return nil, false, nil + case ast.NextVal: + // handle default next value of sequence. (keep the expr string) + str, err := getSequenceDefaultValue(option) + if err != nil { + return nil, false, errors.Trace(err) + } + return str, true, nil + case ast.Rand, ast.UUID, ast.UUIDToBin: // RAND(), UUID() and UUID_TO_BIN() + if err := expression.VerifyArgsWrapper(expr.FnName.L, len(expr.Args)); err != nil { + return nil, false, errors.Trace(err) + } + str, err := restoreFuncCall(expr) + if err != nil { + return nil, false, errors.Trace(err) + } + col.DefaultIsExpr = true + return str, false, nil + case ast.DateFormat: // DATE_FORMAT() + if err := expression.VerifyArgsWrapper(expr.FnName.L, len(expr.Args)); err != nil { + return nil, false, errors.Trace(err) + } + // Support DATE_FORMAT(NOW(),'%Y-%m'), DATE_FORMAT(NOW(),'%Y-%m-%d'), + // DATE_FORMAT(NOW(),'%Y-%m-%d %H.%i.%s'), DATE_FORMAT(NOW(),'%Y-%m-%d %H:%i:%s'). + nowFunc, ok := expr.Args[0].(*ast.FuncCallExpr) + if ok && nowFunc.FnName.L == ast.Now { + if err := expression.VerifyArgsWrapper(nowFunc.FnName.L, len(nowFunc.Args)); err != nil { + return nil, false, errors.Trace(err) + } + valExpr, isValue := expr.Args[1].(ast.ValueExpr) + if !isValue || (valExpr.GetString() != "%Y-%m" && valExpr.GetString() != "%Y-%m-%d" && + valExpr.GetString() != "%Y-%m-%d %H.%i.%s" && valExpr.GetString() != "%Y-%m-%d %H:%i:%s") { + return nil, false, dbterror.ErrDefValGeneratedNamedFunctionIsNotAllowed.GenWithStackByArgs(col.Name.String(), valExpr) + } + str, err := restoreFuncCall(expr) + if err != nil { + return nil, false, errors.Trace(err) + } + col.DefaultIsExpr = true + return str, false, nil + } + return nil, false, dbterror.ErrDefValGeneratedNamedFunctionIsNotAllowed.GenWithStackByArgs(col.Name.String(), + fmt.Sprintf("%s with disallowed args", expr.FnName.String())) + case ast.Replace: + if err := expression.VerifyArgsWrapper(expr.FnName.L, len(expr.Args)); err != nil { + return nil, false, errors.Trace(err) + } + funcCall := expr.Args[0] + // Support REPLACE(CONVERT(UPPER(UUID()) USING UTF8MB4), '-', '')) + if convertFunc, ok := funcCall.(*ast.FuncCallExpr); ok && convertFunc.FnName.L == ast.Convert { + if err := expression.VerifyArgsWrapper(convertFunc.FnName.L, len(convertFunc.Args)); err != nil { + return nil, false, errors.Trace(err) + } + funcCall = convertFunc.Args[0] + } + // Support REPLACE(UPPER(UUID()), '-', ''). + if upperFunc, ok := funcCall.(*ast.FuncCallExpr); ok && upperFunc.FnName.L == ast.Upper { + if err := expression.VerifyArgsWrapper(upperFunc.FnName.L, len(upperFunc.Args)); err != nil { + return nil, false, errors.Trace(err) + } + if uuidFunc, ok := upperFunc.Args[0].(*ast.FuncCallExpr); ok && uuidFunc.FnName.L == ast.UUID { + if err := expression.VerifyArgsWrapper(uuidFunc.FnName.L, len(uuidFunc.Args)); err != nil { + return nil, false, errors.Trace(err) + } + str, err := restoreFuncCall(expr) + if err != nil { + return nil, false, errors.Trace(err) + } + col.DefaultIsExpr = true + return str, false, nil + } + } + return nil, false, dbterror.ErrDefValGeneratedNamedFunctionIsNotAllowed.GenWithStackByArgs(col.Name.String(), + fmt.Sprintf("%s with disallowed args", expr.FnName.String())) + case ast.Upper: + if err := expression.VerifyArgsWrapper(expr.FnName.L, len(expr.Args)); err != nil { + return nil, false, errors.Trace(err) + } + // Support UPPER(SUBSTRING_INDEX(USER(), '@', 1)). + if substringIndexFunc, ok := expr.Args[0].(*ast.FuncCallExpr); ok && substringIndexFunc.FnName.L == ast.SubstringIndex { + if err := expression.VerifyArgsWrapper(substringIndexFunc.FnName.L, len(substringIndexFunc.Args)); err != nil { + return nil, false, errors.Trace(err) + } + if userFunc, ok := substringIndexFunc.Args[0].(*ast.FuncCallExpr); ok && userFunc.FnName.L == ast.User { + if err := expression.VerifyArgsWrapper(userFunc.FnName.L, len(userFunc.Args)); err != nil { + return nil, false, errors.Trace(err) + } + valExpr, isValue := substringIndexFunc.Args[1].(ast.ValueExpr) + if !isValue || valExpr.GetString() != "@" { + return nil, false, dbterror.ErrDefValGeneratedNamedFunctionIsNotAllowed.GenWithStackByArgs(col.Name.String(), valExpr) + } + str, err := restoreFuncCall(expr) + if err != nil { + return nil, false, errors.Trace(err) + } + col.DefaultIsExpr = true + return str, false, nil + } + } + return nil, false, dbterror.ErrDefValGeneratedNamedFunctionIsNotAllowed.GenWithStackByArgs(col.Name.String(), + fmt.Sprintf("%s with disallowed args", expr.FnName.String())) + case ast.StrToDate: // STR_TO_DATE() + if err := expression.VerifyArgsWrapper(expr.FnName.L, len(expr.Args)); err != nil { + return nil, false, errors.Trace(err) + } + // Support STR_TO_DATE('1980-01-01', '%Y-%m-%d'). + if _, ok1 := expr.Args[0].(ast.ValueExpr); ok1 { + if _, ok2 := expr.Args[1].(ast.ValueExpr); ok2 { + str, err := restoreFuncCall(expr) + if err != nil { + return nil, false, errors.Trace(err) + } + col.DefaultIsExpr = true + return str, false, nil + } + } + return nil, false, dbterror.ErrDefValGeneratedNamedFunctionIsNotAllowed.GenWithStackByArgs(col.Name.String(), + fmt.Sprintf("%s with disallowed args", expr.FnName.String())) + case ast.JSONObject, ast.JSONArray, ast.JSONQuote: // JSON_OBJECT(), JSON_ARRAY(), JSON_QUOTE() + if err := expression.VerifyArgsWrapper(expr.FnName.L, len(expr.Args)); err != nil { + return nil, false, errors.Trace(err) + } + str, err := restoreFuncCall(expr) + if err != nil { + return nil, false, errors.Trace(err) + } + col.DefaultIsExpr = true + return str, false, nil + + default: + return nil, false, dbterror.ErrDefValGeneratedNamedFunctionIsNotAllowed.GenWithStackByArgs(col.Name.String(), expr.FnName.String()) + } +} + +// getDefaultValue will get the default value for column. +// 1: get the expr restored string for the column which uses sequence next value as default value. +// 2: get specific default value for the other column. +func getDefaultValue(ctx exprctx.BuildContext, col *table.Column, option *ast.ColumnOption) (any, bool, error) { + // handle default value with function call + tp, fsp := col.FieldType.GetType(), col.FieldType.GetDecimal() + if x, ok := option.Expr.(*ast.FuncCallExpr); ok { + val, isSeqExpr, err := getFuncCallDefaultValue(col, option, x) + if val != nil || isSeqExpr || err != nil { + return val, isSeqExpr, err + } + // If the function call is ast.CurrentTimestamp, it needs to be continuously processed. + } + + if tp == mysql.TypeTimestamp || tp == mysql.TypeDatetime || tp == mysql.TypeDate { + vd, err := expression.GetTimeValue(ctx, option.Expr, tp, fsp, nil) + value := vd.GetValue() + if err != nil { + return nil, false, dbterror.ErrInvalidDefaultValue.GenWithStackByArgs(col.Name.O) + } + + // Value is nil means `default null`. + if value == nil { + return nil, false, nil + } + + // If value is types.Time, convert it to string. + if vv, ok := value.(types.Time); ok { + return vv.String(), false, nil + } + + return value, false, nil + } + + // evaluate the non-function-call expr to a certain value. + v, err := expression.EvalSimpleAst(ctx, option.Expr) + if err != nil { + return nil, false, errors.Trace(err) + } + + if v.IsNull() { + return nil, false, nil + } + + if v.Kind() == types.KindBinaryLiteral || v.Kind() == types.KindMysqlBit { + if types.IsTypeBlob(tp) || tp == mysql.TypeJSON { + // BLOB/TEXT/JSON column cannot have a default value. + // Skip the unnecessary decode procedure. + return v.GetString(), false, err + } + if tp == mysql.TypeBit || tp == mysql.TypeString || tp == mysql.TypeVarchar || + tp == mysql.TypeVarString || tp == mysql.TypeEnum || tp == mysql.TypeSet { + // For BinaryLiteral or bit fields, we decode the default value to utf8 string. + str, err := v.GetBinaryStringDecoded(types.StrictFlags, col.GetCharset()) + if err != nil { + // Overwrite the decoding error with invalid default value error. + err = dbterror.ErrInvalidDefaultValue.GenWithStackByArgs(col.Name.O) + } + return str, false, err + } + // For other kind of fields (e.g. INT), we supply its integer as string value. + value, err := v.GetBinaryLiteral().ToInt(ctx.GetEvalCtx().TypeCtx()) + if err != nil { + return nil, false, err + } + return strconv.FormatUint(value, 10), false, nil + } + + switch tp { + case mysql.TypeSet: + val, err := getSetDefaultValue(v, col) + return val, false, err + case mysql.TypeEnum: + val, err := getEnumDefaultValue(v, col) + return val, false, err + case mysql.TypeDuration, mysql.TypeDate: + if v, err = v.ConvertTo(ctx.GetEvalCtx().TypeCtx(), &col.FieldType); err != nil { + return "", false, errors.Trace(err) + } + case mysql.TypeBit: + if v.Kind() == types.KindInt64 || v.Kind() == types.KindUint64 { + // For BIT fields, convert int into BinaryLiteral. + return types.NewBinaryLiteralFromUint(v.GetUint64(), -1).ToString(), false, nil + } + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeFloat, mysql.TypeDouble: + // For these types, convert it to standard format firstly. + // like integer fields, convert it into integer string literals. like convert "1.25" into "1" and "2.8" into "3". + // if raise a error, we will use original expression. We will handle it in check phase + if temp, err := v.ConvertTo(ctx.GetEvalCtx().TypeCtx(), &col.FieldType); err == nil { + v = temp + } + } + + val, err := v.ToString() + return val, false, err +} + +func getSequenceDefaultValue(c *ast.ColumnOption) (expr string, err error) { + var sb strings.Builder + restoreFlags := format.RestoreStringSingleQuotes | format.RestoreKeyWordLowercase | format.RestoreNameBackQuotes | + format.RestoreSpacesAroundBinaryOperation + restoreCtx := format.NewRestoreCtx(restoreFlags, &sb) + if err := c.Expr.Restore(restoreCtx); err != nil { + return "", err + } + return sb.String(), nil +} + +func setDefaultValueWithBinaryPadding(col *table.Column, value any) error { + err := col.SetDefaultValue(value) + if err != nil { + return err + } + // https://dev.mysql.com/doc/refman/8.0/en/binary-varbinary.html + // Set the default value for binary type should append the paddings. + if value != nil { + if col.GetType() == mysql.TypeString && types.IsBinaryStr(&col.FieldType) && len(value.(string)) < col.GetFlen() { + padding := make([]byte, col.GetFlen()-len(value.(string))) + col.DefaultValue = string(append([]byte(col.DefaultValue.(string)), padding...)) + } + } + return nil +} + +func setColumnComment(ctx sessionctx.Context, col *table.Column, option *ast.ColumnOption) error { + value, err := expression.EvalSimpleAst(ctx.GetExprCtx(), option.Expr) + if err != nil { + return errors.Trace(err) + } + if col.Comment, err = value.ToString(); err != nil { + return errors.Trace(err) + } + + sessionVars := ctx.GetSessionVars() + col.Comment, err = validateCommentLength(sessionVars.StmtCtx.ErrCtx(), sessionVars.SQLMode, col.Name.L, &col.Comment, dbterror.ErrTooLongFieldComment) + return errors.Trace(err) +} + +func processAndCheckDefaultValueAndColumn(ctx sessionctx.Context, col *table.Column, + outPriKeyConstraint *ast.Constraint, hasDefaultValue, setOnUpdateNow, hasNullFlag bool) error { + processDefaultValue(col, hasDefaultValue, setOnUpdateNow) + processColumnFlags(col) + + err := checkPriKeyConstraint(col, hasDefaultValue, hasNullFlag, outPriKeyConstraint) + if err != nil { + return errors.Trace(err) + } + if err = checkColumnValueConstraint(col, col.GetCollate()); err != nil { + return errors.Trace(err) + } + if err = checkDefaultValue(ctx.GetExprCtx(), col, hasDefaultValue); err != nil { + return errors.Trace(err) + } + if err = checkColumnFieldLength(col); err != nil { + return errors.Trace(err) + } + return nil +} + +func restoreFuncCall(expr *ast.FuncCallExpr) (string, error) { + var sb strings.Builder + restoreFlags := format.RestoreStringSingleQuotes | format.RestoreKeyWordLowercase | format.RestoreNameBackQuotes | + format.RestoreSpacesAroundBinaryOperation + restoreCtx := format.NewRestoreCtx(restoreFlags, &sb) + if err := expr.Restore(restoreCtx); err != nil { + return "", err + } + return sb.String(), nil +} + +// getSetDefaultValue gets the default value for the set type. See https://dev.mysql.com/doc/refman/5.7/en/set.html. +func getSetDefaultValue(v types.Datum, col *table.Column) (string, error) { + if v.Kind() == types.KindInt64 { + setCnt := len(col.GetElems()) + maxLimit := int64(1< maxLimit { + return "", dbterror.ErrInvalidDefaultValue.GenWithStackByArgs(col.Name.O) + } + setVal, err := types.ParseSetValue(col.GetElems(), uint64(val)) + if err != nil { + return "", errors.Trace(err) + } + v.SetMysqlSet(setVal, col.GetCollate()) + return v.ToString() + } + + str, err := v.ToString() + if err != nil { + return "", errors.Trace(err) + } + if str == "" { + return str, nil + } + setVal, err := types.ParseSetName(col.GetElems(), str, col.GetCollate()) + if err != nil { + return "", dbterror.ErrInvalidDefaultValue.GenWithStackByArgs(col.Name.O) + } + v.SetMysqlSet(setVal, col.GetCollate()) + + return v.ToString() +} + +// getEnumDefaultValue gets the default value for the enum type. See https://dev.mysql.com/doc/refman/5.7/en/enum.html. +func getEnumDefaultValue(v types.Datum, col *table.Column) (string, error) { + if v.Kind() == types.KindInt64 { + val := v.GetInt64() + if val < 1 || val > int64(len(col.GetElems())) { + return "", dbterror.ErrInvalidDefaultValue.GenWithStackByArgs(col.Name.O) + } + enumVal, err := types.ParseEnumValue(col.GetElems(), uint64(val)) + if err != nil { + return "", errors.Trace(err) + } + v.SetMysqlEnum(enumVal, col.GetCollate()) + return v.ToString() + } + str, err := v.ToString() + if err != nil { + return "", errors.Trace(err) + } + // Ref: https://dev.mysql.com/doc/refman/8.0/en/enum.html + // Trailing spaces are automatically deleted from ENUM member values in the table definition when a table is created. + str = strings.TrimRight(str, " ") + enumVal, err := types.ParseEnumName(col.GetElems(), str, col.GetCollate()) + if err != nil { + return "", dbterror.ErrInvalidDefaultValue.GenWithStackByArgs(col.Name.O) + } + v.SetMysqlEnum(enumVal, col.GetCollate()) + + return v.ToString() +} + +func removeOnUpdateNowFlag(c *table.Column) { + // For timestamp Col, if it is set null or default value, + // OnUpdateNowFlag should be removed. + if mysql.HasTimestampFlag(c.GetFlag()) { + c.DelFlag(mysql.OnUpdateNowFlag) + } +} + +func processDefaultValue(c *table.Column, hasDefaultValue bool, setOnUpdateNow bool) { + setTimestampDefaultValue(c, hasDefaultValue, setOnUpdateNow) + + setYearDefaultValue(c, hasDefaultValue) + + // Set `NoDefaultValueFlag` if this field doesn't have a default value and + // it is `not null` and not an `AUTO_INCREMENT` field or `TIMESTAMP` field. + setNoDefaultValueFlag(c, hasDefaultValue) +} + +func setYearDefaultValue(c *table.Column, hasDefaultValue bool) { + if hasDefaultValue { + return + } + + if c.GetType() == mysql.TypeYear && mysql.HasNotNullFlag(c.GetFlag()) { + if err := c.SetDefaultValue("0000"); err != nil { + logutil.DDLLogger().Error("set default value failed", zap.Error(err)) + } + } +} + +func setTimestampDefaultValue(c *table.Column, hasDefaultValue bool, setOnUpdateNow bool) { + if hasDefaultValue { + return + } + + // For timestamp Col, if is not set default value or not set null, use current timestamp. + if mysql.HasTimestampFlag(c.GetFlag()) && mysql.HasNotNullFlag(c.GetFlag()) { + if setOnUpdateNow { + if err := c.SetDefaultValue(types.ZeroDatetimeStr); err != nil { + logutil.DDLLogger().Error("set default value failed", zap.Error(err)) + } + } else { + if err := c.SetDefaultValue(strings.ToUpper(ast.CurrentTimestamp)); err != nil { + logutil.DDLLogger().Error("set default value failed", zap.Error(err)) + } + } + } +} + +func setNoDefaultValueFlag(c *table.Column, hasDefaultValue bool) { + if hasDefaultValue { + return + } + + if !mysql.HasNotNullFlag(c.GetFlag()) { + return + } + + // Check if it is an `AUTO_INCREMENT` field or `TIMESTAMP` field. + if !mysql.HasAutoIncrementFlag(c.GetFlag()) && !mysql.HasTimestampFlag(c.GetFlag()) { + c.AddFlag(mysql.NoDefaultValueFlag) + } +} + +func checkDefaultValue(ctx exprctx.BuildContext, c *table.Column, hasDefaultValue bool) (err error) { + if !hasDefaultValue { + return nil + } + + if c.GetDefaultValue() != nil { + if c.DefaultIsExpr { + if mysql.HasAutoIncrementFlag(c.GetFlag()) { + return types.ErrInvalidDefault.GenWithStackByArgs(c.Name) + } + return nil + } + _, err = table.GetColDefaultValue( + exprctx.CtxWithHandleTruncateErrLevel(ctx, errctx.LevelError), + c.ToInfo(), + ) + if err != nil { + return types.ErrInvalidDefault.GenWithStackByArgs(c.Name) + } + return nil + } + // Primary key default null is invalid. + if mysql.HasPriKeyFlag(c.GetFlag()) { + return dbterror.ErrPrimaryCantHaveNull + } + + // Set not null but default null is invalid. + if mysql.HasNotNullFlag(c.GetFlag()) { + return types.ErrInvalidDefault.GenWithStackByArgs(c.Name) + } + + return nil +} + +// checkPriKeyConstraint check all parts of a PRIMARY KEY must be NOT NULL +func checkPriKeyConstraint(col *table.Column, hasDefaultValue, hasNullFlag bool, outPriKeyConstraint *ast.Constraint) error { + // Primary key should not be null. + if mysql.HasPriKeyFlag(col.GetFlag()) && hasDefaultValue && col.GetDefaultValue() == nil { + return types.ErrInvalidDefault.GenWithStackByArgs(col.Name) + } + // Set primary key flag for outer primary key constraint. + // Such as: create table t1 (id int , age int, primary key(id)) + if !mysql.HasPriKeyFlag(col.GetFlag()) && outPriKeyConstraint != nil { + for _, key := range outPriKeyConstraint.Keys { + if key.Expr == nil && key.Column.Name.L != col.Name.L { + continue + } + col.AddFlag(mysql.PriKeyFlag) + break + } + } + // Primary key should not be null. + if mysql.HasPriKeyFlag(col.GetFlag()) && hasNullFlag { + return dbterror.ErrPrimaryCantHaveNull + } + return nil +} + +func checkColumnValueConstraint(col *table.Column, collation string) error { + if col.GetType() != mysql.TypeEnum && col.GetType() != mysql.TypeSet { + return nil + } + valueMap := make(map[string]bool, len(col.GetElems())) + ctor := collate.GetCollator(collation) + enumLengthLimit := config.GetGlobalConfig().EnableEnumLengthLimit + desc, err := charset.GetCharsetInfo(col.GetCharset()) + if err != nil { + return errors.Trace(err) + } + for i := range col.GetElems() { + val := string(ctor.Key(col.GetElems()[i])) + // According to MySQL 8.0 Refman: + // The maximum supported length of an individual ENUM element is M <= 255 and (M x w) <= 1020, + // where M is the element literal length and w is the number of bytes required for the maximum-length character in the character set. + // See https://dev.mysql.com/doc/refman/8.0/en/string-type-syntax.html for more details. + if enumLengthLimit && (len(val) > 255 || len(val)*desc.Maxlen > 1020) { + return dbterror.ErrTooLongValueForType.GenWithStackByArgs(col.Name) + } + if _, ok := valueMap[val]; ok { + tpStr := "ENUM" + if col.GetType() == mysql.TypeSet { + tpStr = "SET" + } + return types.ErrDuplicatedValueInType.GenWithStackByArgs(col.Name, col.GetElems()[i], tpStr) + } + valueMap[val] = true + } + return nil +} + +// checkColumnDefaultValue checks the default value of the column. +// In non-strict SQL mode, if the default value of the column is an empty string, the default value can be ignored. +// In strict SQL mode, TEXT/BLOB/JSON can't have not null default values. +// In NO_ZERO_DATE SQL mode, TIMESTAMP/DATE/DATETIME type can't have zero date like '0000-00-00' or '0000-00-00 00:00:00'. +func checkColumnDefaultValue(ctx exprctx.BuildContext, col *table.Column, value any) (bool, any, error) { + hasDefaultValue := true + if value != nil && (col.GetType() == mysql.TypeJSON || + col.GetType() == mysql.TypeTinyBlob || col.GetType() == mysql.TypeMediumBlob || + col.GetType() == mysql.TypeLongBlob || col.GetType() == mysql.TypeBlob) { + // In non-strict SQL mode. + if !ctx.GetEvalCtx().SQLMode().HasStrictMode() && value == "" { + if col.GetType() == mysql.TypeBlob || col.GetType() == mysql.TypeLongBlob { + // The TEXT/BLOB default value can be ignored. + hasDefaultValue = false + } + // In non-strict SQL mode, if the column type is json and the default value is null, it is initialized to an empty array. + if col.GetType() == mysql.TypeJSON { + value = `null` + } + ctx.GetEvalCtx().AppendWarning(dbterror.ErrBlobCantHaveDefault.FastGenByArgs(col.Name.O)) + return hasDefaultValue, value, nil + } + // In strict SQL mode or default value is not an empty string. + return hasDefaultValue, value, dbterror.ErrBlobCantHaveDefault.GenWithStackByArgs(col.Name.O) + } + if value != nil && ctx.GetEvalCtx().SQLMode().HasNoZeroDateMode() && + ctx.GetEvalCtx().SQLMode().HasStrictMode() && types.IsTypeTime(col.GetType()) { + if vv, ok := value.(string); ok { + timeValue, err := expression.GetTimeValue(ctx, vv, col.GetType(), col.GetDecimal(), nil) + if err != nil { + return hasDefaultValue, value, errors.Trace(err) + } + if timeValue.GetMysqlTime().CoreTime() == types.ZeroCoreTime { + return hasDefaultValue, value, types.ErrInvalidDefault.GenWithStackByArgs(col.Name.O) + } + } + } + return hasDefaultValue, value, nil +} + +func checkSequenceDefaultValue(col *table.Column) error { + if mysql.IsIntegerType(col.GetType()) { + return nil + } + return dbterror.ErrColumnTypeUnsupportedNextValue.GenWithStackByArgs(col.ColumnInfo.Name.O) +} + +func convertTimestampDefaultValToUTC(ctx sessionctx.Context, defaultVal any, col *table.Column) (any, error) { + if defaultVal == nil || col.GetType() != mysql.TypeTimestamp { + return defaultVal, nil + } + if vv, ok := defaultVal.(string); ok { + if vv != types.ZeroDatetimeStr && !strings.EqualFold(vv, ast.CurrentTimestamp) { + t, err := types.ParseTime(ctx.GetSessionVars().StmtCtx.TypeCtx(), vv, col.GetType(), col.GetDecimal()) + if err != nil { + return defaultVal, errors.Trace(err) + } + err = t.ConvertTimeZone(ctx.GetSessionVars().Location(), time.UTC) + if err != nil { + return defaultVal, errors.Trace(err) + } + defaultVal = t.String() + } + } + return defaultVal, nil +} + +// processColumnFlags is used by columnDefToCol and processColumnOptions. It is intended to unify behaviors on `create/add` and `modify/change` statements. Check tidb#issue#19342. +func processColumnFlags(col *table.Column) { + if col.FieldType.EvalType().IsStringKind() { + if col.GetCharset() == charset.CharsetBin { + col.AddFlag(mysql.BinaryFlag) + } else { + col.DelFlag(mysql.BinaryFlag) + } + } + if col.GetType() == mysql.TypeBit { + // For BIT field, it's charset is binary but does not have binary flag. + col.DelFlag(mysql.BinaryFlag) + col.AddFlag(mysql.UnsignedFlag) + } + if col.GetType() == mysql.TypeYear { + // For Year field, it's charset is binary but does not have binary flag. + col.DelFlag(mysql.BinaryFlag) + col.AddFlag(mysql.ZerofillFlag) + } + + // If you specify ZEROFILL for a numeric column, MySQL automatically adds the UNSIGNED attribute to the column. + // See https://dev.mysql.com/doc/refman/5.7/en/numeric-type-overview.html for more details. + // But some types like bit and year, won't show its unsigned flag in `show create table`. + if mysql.HasZerofillFlag(col.GetFlag()) { + col.AddFlag(mysql.UnsignedFlag) + } +} + +func adjustBlobTypesFlen(tp *types.FieldType, colCharset string) error { + cs, err := charset.GetCharsetInfo(colCharset) + // when we meet the unsupported charset, we do not adjust. + if err != nil { + return err + } + l := tp.GetFlen() * cs.Maxlen + if tp.GetType() == mysql.TypeBlob { + if l <= tinyBlobMaxLength { + logutil.DDLLogger().Info(fmt.Sprintf("Automatically convert BLOB(%d) to TINYBLOB", tp.GetFlen())) + tp.SetFlen(tinyBlobMaxLength) + tp.SetType(mysql.TypeTinyBlob) + } else if l <= blobMaxLength { + tp.SetFlen(blobMaxLength) + } else if l <= mediumBlobMaxLength { + logutil.DDLLogger().Info(fmt.Sprintf("Automatically convert BLOB(%d) to MEDIUMBLOB", tp.GetFlen())) + tp.SetFlen(mediumBlobMaxLength) + tp.SetType(mysql.TypeMediumBlob) + } else if l <= longBlobMaxLength { + logutil.DDLLogger().Info(fmt.Sprintf("Automatically convert BLOB(%d) to LONGBLOB", tp.GetFlen())) + tp.SetFlen(longBlobMaxLength) + tp.SetType(mysql.TypeLongBlob) + } + } + return nil +} diff --git a/pkg/ddl/create_table.go b/pkg/ddl/create_table.go index 6cc4baf0d76dd..7b87859169b9a 100644 --- a/pkg/ddl/create_table.go +++ b/pkg/ddl/create_table.go @@ -18,12 +18,16 @@ import ( "context" "fmt" "math" + "strings" + "unicode/utf8" "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/config" + "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/parser/format" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" field_types "github.com/pingcap/tidb/pkg/parser/types" @@ -62,6 +66,82 @@ func buildTableInfoWithCheck(ctx sessionctx.Context, s *ast.CreateTableStmt, dbC return tbInfo, nil } +// CheckTableInfoValidWithStmt exposes checkTableInfoValidWithStmt to SchemaTracker. Maybe one day we can delete it. +func CheckTableInfoValidWithStmt(ctx sessionctx.Context, tbInfo *model.TableInfo, s *ast.CreateTableStmt) (err error) { + return checkTableInfoValidWithStmt(ctx, tbInfo, s) +} + +func checkTableInfoValidWithStmt(ctx sessionctx.Context, tbInfo *model.TableInfo, s *ast.CreateTableStmt) (err error) { + // All of these rely on the AST structure of expressions, which were + // lost in the model (got serialized into strings). + if err := checkGeneratedColumn(ctx, s.Table.Schema, tbInfo.Name, s.Cols); err != nil { + return errors.Trace(err) + } + + // Check if table has a primary key if required. + if !ctx.GetSessionVars().InRestrictedSQL && ctx.GetSessionVars().PrimaryKeyRequired && len(tbInfo.GetPkName().String()) == 0 { + return infoschema.ErrTableWithoutPrimaryKey + } + if tbInfo.Partition != nil { + if err := checkPartitionDefinitionConstraints(ctx, tbInfo); err != nil { + return errors.Trace(err) + } + if s.Partition != nil { + if err := checkPartitionFuncType(ctx, s.Partition.Expr, s.Table.Schema.O, tbInfo); err != nil { + return errors.Trace(err) + } + if err := checkPartitioningKeysConstraints(ctx, s, tbInfo); err != nil { + return errors.Trace(err) + } + } + } + if tbInfo.TTLInfo != nil { + if err := checkTTLInfoValid(ctx, s.Table.Schema, tbInfo); err != nil { + return errors.Trace(err) + } + } + + return nil +} + +// checkTableInfoValidExtra is like checkTableInfoValid, but also assumes the +// table info comes from untrusted source and performs further checks such as +// name length and column count. +// (checkTableInfoValid is also used in repairing objects which don't perform +// these checks. Perhaps the two functions should be merged together regardless?) +func checkTableInfoValidExtra(tbInfo *model.TableInfo) error { + if err := checkTooLongTable(tbInfo.Name); err != nil { + return err + } + + if err := checkDuplicateColumn(tbInfo.Columns); err != nil { + return err + } + if err := checkTooLongColumns(tbInfo.Columns); err != nil { + return err + } + if err := checkTooManyColumns(tbInfo.Columns); err != nil { + return errors.Trace(err) + } + if err := checkTooManyIndexes(tbInfo.Indices); err != nil { + return errors.Trace(err) + } + if err := checkColumnsAttributes(tbInfo.Columns); err != nil { + return errors.Trace(err) + } + + // FIXME: perform checkConstraintNames + if err := checkCharsetAndCollation(tbInfo.Charset, tbInfo.Collate); err != nil { + return errors.Trace(err) + } + + oldState := tbInfo.State + tbInfo.State = model.StatePublic + err := checkTableInfoValid(tbInfo) + tbInfo.State = oldState + return err +} + // BuildSessionTemporaryTableInfo builds model.TableInfo from a SQL statement. func BuildSessionTemporaryTableInfo(ctx sessionctx.Context, is infoschema.InfoSchema, s *ast.CreateTableStmt, dbCharset, dbCollate string, placementPolicyRef *model.PolicyRefInfo) (*model.TableInfo, error) { ident := ast.Ident{Schema: s.Table.Schema, Name: s.Table.Name} @@ -670,6 +750,114 @@ func BuildTableInfo( return tbInfo, err } +func precheckBuildHiddenColumnInfo( + indexPartSpecifications []*ast.IndexPartSpecification, + indexName model.CIStr, +) error { + for i, idxPart := range indexPartSpecifications { + if idxPart.Expr == nil { + continue + } + name := fmt.Sprintf("%s_%s_%d", expressionIndexPrefix, indexName, i) + if utf8.RuneCountInString(name) > mysql.MaxColumnNameLength { + // TODO: Refine the error message. + return dbterror.ErrTooLongIdent.GenWithStackByArgs("hidden column") + } + // TODO: Refine the error message. + if err := checkIllegalFn4Generated(indexName.L, typeIndex, idxPart.Expr); err != nil { + return errors.Trace(err) + } + } + return nil +} + +func buildHiddenColumnInfoWithCheck(ctx sessionctx.Context, indexPartSpecifications []*ast.IndexPartSpecification, indexName model.CIStr, tblInfo *model.TableInfo, existCols []*table.Column) ([]*model.ColumnInfo, error) { + if err := precheckBuildHiddenColumnInfo(indexPartSpecifications, indexName); err != nil { + return nil, err + } + return BuildHiddenColumnInfo(ctx, indexPartSpecifications, indexName, tblInfo, existCols) +} + +// BuildHiddenColumnInfo builds hidden column info. +func BuildHiddenColumnInfo(ctx sessionctx.Context, indexPartSpecifications []*ast.IndexPartSpecification, indexName model.CIStr, tblInfo *model.TableInfo, existCols []*table.Column) ([]*model.ColumnInfo, error) { + hiddenCols := make([]*model.ColumnInfo, 0, len(indexPartSpecifications)) + for i, idxPart := range indexPartSpecifications { + if idxPart.Expr == nil { + continue + } + idxPart.Column = &ast.ColumnName{Name: model.NewCIStr(fmt.Sprintf("%s_%s_%d", expressionIndexPrefix, indexName, i))} + // Check whether the hidden columns have existed. + col := table.FindCol(existCols, idxPart.Column.Name.L) + if col != nil { + // TODO: Use expression index related error. + return nil, infoschema.ErrColumnExists.GenWithStackByArgs(col.Name.String()) + } + idxPart.Length = types.UnspecifiedLength + // The index part is an expression, prepare a hidden column for it. + + var sb strings.Builder + restoreFlags := format.RestoreStringSingleQuotes | format.RestoreKeyWordLowercase | format.RestoreNameBackQuotes | + format.RestoreSpacesAroundBinaryOperation | format.RestoreWithoutSchemaName | format.RestoreWithoutTableName + restoreCtx := format.NewRestoreCtx(restoreFlags, &sb) + sb.Reset() + err := idxPart.Expr.Restore(restoreCtx) + if err != nil { + return nil, errors.Trace(err) + } + expr, err := expression.BuildSimpleExpr(ctx.GetExprCtx(), idxPart.Expr, + expression.WithTableInfo(ctx.GetSessionVars().CurrentDB, tblInfo), + expression.WithAllowCastArray(true), + ) + if err != nil { + // TODO: refine the error message. + return nil, err + } + if _, ok := expr.(*expression.Column); ok { + return nil, dbterror.ErrFunctionalIndexOnField + } + + colInfo := &model.ColumnInfo{ + Name: idxPart.Column.Name, + GeneratedExprString: sb.String(), + GeneratedStored: false, + Version: model.CurrLatestColumnInfoVersion, + Dependences: make(map[string]struct{}), + Hidden: true, + FieldType: *expr.GetType(ctx.GetExprCtx().GetEvalCtx()), + } + // Reset some flag, it may be caused by wrong type infer. But it's not easy to fix them all, so reset them here for safety. + colInfo.DelFlag(mysql.PriKeyFlag | mysql.UniqueKeyFlag | mysql.AutoIncrementFlag) + + if colInfo.GetType() == mysql.TypeDatetime || colInfo.GetType() == mysql.TypeDate || colInfo.GetType() == mysql.TypeTimestamp || colInfo.GetType() == mysql.TypeDuration { + if colInfo.FieldType.GetDecimal() == types.UnspecifiedLength { + colInfo.FieldType.SetDecimal(types.MaxFsp) + } + } + // For an array, the collation is set to "binary". The collation has no effect on the array itself (as it's usually + // regarded as a JSON), but will influence how TiKV handles the index value. + if colInfo.FieldType.IsArray() { + colInfo.SetCharset("binary") + colInfo.SetCollate("binary") + } + checkDependencies := make(map[string]struct{}) + for _, colName := range FindColumnNamesInExpr(idxPart.Expr) { + colInfo.Dependences[colName.Name.L] = struct{}{} + checkDependencies[colName.Name.L] = struct{}{} + } + if err = checkDependedColExist(checkDependencies, existCols); err != nil { + return nil, errors.Trace(err) + } + if !ctx.GetSessionVars().EnableAutoIncrementInGenerated { + if err = checkExpressionIndexAutoIncrement(indexName.O, colInfo.Dependences, tblInfo); err != nil { + return nil, errors.Trace(err) + } + } + idxPart.Expr = nil + hiddenCols = append(hiddenCols, colInfo) + } + return hiddenCols, nil +} + // addIndexForForeignKey uses to auto create an index for the foreign key if the table doesn't have any index cover the // foreign key columns. func addIndexForForeignKey(ctx sessionctx.Context, tbInfo *model.TableInfo) error { diff --git a/pkg/ddl/executor.go b/pkg/ddl/executor.go index f6f8092e6c1c6..4227dc9702dbe 100644 --- a/pkg/ddl/executor.go +++ b/pkg/ddl/executor.go @@ -45,7 +45,6 @@ import ( rg "github.com/pingcap/tidb/pkg/domain/resourcegroup" "github.com/pingcap/tidb/pkg/errctx" "github.com/pingcap/tidb/pkg/expression" - exprctx "github.com/pingcap/tidb/pkg/expression/context" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta" @@ -68,7 +67,6 @@ import ( "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/types" - driver "github.com/pingcap/tidb/pkg/types/parser_driver" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/collate" @@ -76,7 +74,6 @@ import ( "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/hack" "github.com/pingcap/tidb/pkg/util/intest" "github.com/pingcap/tidb/pkg/util/mathutil" "github.com/pingcap/tidb/pkg/util/set" @@ -947,872 +944,6 @@ func ResolveCharsetCollation(sessVars *variable.SessionVars, charsetOpts ...ast. return chs, coll, nil } -// OverwriteCollationWithBinaryFlag is used to handle the case like -// -// CREATE TABLE t (a VARCHAR(255) BINARY) CHARSET utf8 COLLATE utf8_general_ci; -// -// The 'BINARY' sets the column collation to *_bin according to the table charset. -func OverwriteCollationWithBinaryFlag(sessVars *variable.SessionVars, colDef *ast.ColumnDef, chs, coll string) (newChs string, newColl string) { - ignoreBinFlag := colDef.Tp.GetCharset() != "" && (colDef.Tp.GetCollate() != "" || containsColumnOption(colDef, ast.ColumnOptionCollate)) - if ignoreBinFlag { - return chs, coll - } - needOverwriteBinColl := types.IsString(colDef.Tp.GetType()) && mysql.HasBinaryFlag(colDef.Tp.GetFlag()) - if needOverwriteBinColl { - newColl, err := GetDefaultCollation(sessVars, chs) - if err != nil { - return chs, coll - } - return chs, newColl - } - return chs, coll -} - -func typesNeedCharset(tp byte) bool { - switch tp { - case mysql.TypeString, mysql.TypeVarchar, mysql.TypeVarString, - mysql.TypeBlob, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, - mysql.TypeEnum, mysql.TypeSet: - return true - } - return false -} - -func setCharsetCollationFlenDecimal(tp *types.FieldType, colName, colCharset, colCollate string, sessVars *variable.SessionVars) error { - var err error - if typesNeedCharset(tp.GetType()) { - tp.SetCharset(colCharset) - tp.SetCollate(colCollate) - } else { - tp.SetCharset(charset.CharsetBin) - tp.SetCollate(charset.CharsetBin) - } - - // Use default value for flen or decimal when they are unspecified. - defaultFlen, defaultDecimal := mysql.GetDefaultFieldLengthAndDecimal(tp.GetType()) - if tp.GetDecimal() == types.UnspecifiedLength { - tp.SetDecimal(defaultDecimal) - } - if tp.GetFlen() == types.UnspecifiedLength { - tp.SetFlen(defaultFlen) - if mysql.HasUnsignedFlag(tp.GetFlag()) && tp.GetType() != mysql.TypeLonglong && mysql.IsIntegerType(tp.GetType()) { - // Issue #4684: the flen of unsigned integer(except bigint) is 1 digit shorter than signed integer - // because it has no prefix "+" or "-" character. - tp.SetFlen(tp.GetFlen() - 1) - } - } else { - // Adjust the field type for blob/text types if the flen is set. - if err = adjustBlobTypesFlen(tp, colCharset); err != nil { - return err - } - } - return checkTooBigFieldLengthAndTryAutoConvert(tp, colName, sessVars) -} - -func decodeEnumSetBinaryLiteralToUTF8(tp *types.FieldType, chs string) { - if tp.GetType() != mysql.TypeEnum && tp.GetType() != mysql.TypeSet { - return - } - enc := charset.FindEncoding(chs) - for i, elem := range tp.GetElems() { - if !tp.GetElemIsBinaryLit(i) { - continue - } - s, err := enc.Transform(nil, hack.Slice(elem), charset.OpDecodeReplace) - if err != nil { - logutil.DDLLogger().Warn("decode enum binary literal to utf-8 failed", zap.Error(err)) - } - tp.SetElem(i, string(hack.String(s))) - } - tp.CleanElemIsBinaryLit() -} - -// buildColumnAndConstraint builds table.Column and ast.Constraint from the parameters. -// outPriKeyConstraint is the primary key constraint out of column definition. For example: -// `create table t1 (id int , age int, primary key(id));` -func buildColumnAndConstraint( - ctx sessionctx.Context, - offset int, - colDef *ast.ColumnDef, - outPriKeyConstraint *ast.Constraint, - tblCharset string, - tblCollate string, -) (*table.Column, []*ast.Constraint, error) { - if colName := colDef.Name.Name.L; colName == model.ExtraHandleName.L { - return nil, nil, dbterror.ErrWrongColumnName.GenWithStackByArgs(colName) - } - - // specifiedCollate refers to the last collate specified in colDef.Options. - chs, coll, err := getCharsetAndCollateInColumnDef(ctx.GetSessionVars(), colDef) - if err != nil { - return nil, nil, errors.Trace(err) - } - chs, coll, err = ResolveCharsetCollation(ctx.GetSessionVars(), - ast.CharsetOpt{Chs: chs, Col: coll}, - ast.CharsetOpt{Chs: tblCharset, Col: tblCollate}, - ) - chs, coll = OverwriteCollationWithBinaryFlag(ctx.GetSessionVars(), colDef, chs, coll) - if err != nil { - return nil, nil, errors.Trace(err) - } - - if err := setCharsetCollationFlenDecimal(colDef.Tp, colDef.Name.Name.O, chs, coll, ctx.GetSessionVars()); err != nil { - return nil, nil, errors.Trace(err) - } - decodeEnumSetBinaryLiteralToUTF8(colDef.Tp, chs) - col, cts, err := columnDefToCol(ctx, offset, colDef, outPriKeyConstraint) - if err != nil { - return nil, nil, errors.Trace(err) - } - return col, cts, nil -} - -// checkColumnDefaultValue checks the default value of the column. -// In non-strict SQL mode, if the default value of the column is an empty string, the default value can be ignored. -// In strict SQL mode, TEXT/BLOB/JSON can't have not null default values. -// In NO_ZERO_DATE SQL mode, TIMESTAMP/DATE/DATETIME type can't have zero date like '0000-00-00' or '0000-00-00 00:00:00'. -func checkColumnDefaultValue(ctx exprctx.BuildContext, col *table.Column, value any) (bool, any, error) { - hasDefaultValue := true - if value != nil && (col.GetType() == mysql.TypeJSON || - col.GetType() == mysql.TypeTinyBlob || col.GetType() == mysql.TypeMediumBlob || - col.GetType() == mysql.TypeLongBlob || col.GetType() == mysql.TypeBlob) { - // In non-strict SQL mode. - if !ctx.GetEvalCtx().SQLMode().HasStrictMode() && value == "" { - if col.GetType() == mysql.TypeBlob || col.GetType() == mysql.TypeLongBlob { - // The TEXT/BLOB default value can be ignored. - hasDefaultValue = false - } - // In non-strict SQL mode, if the column type is json and the default value is null, it is initialized to an empty array. - if col.GetType() == mysql.TypeJSON { - value = `null` - } - ctx.GetEvalCtx().AppendWarning(dbterror.ErrBlobCantHaveDefault.FastGenByArgs(col.Name.O)) - return hasDefaultValue, value, nil - } - // In strict SQL mode or default value is not an empty string. - return hasDefaultValue, value, dbterror.ErrBlobCantHaveDefault.GenWithStackByArgs(col.Name.O) - } - if value != nil && ctx.GetEvalCtx().SQLMode().HasNoZeroDateMode() && - ctx.GetEvalCtx().SQLMode().HasStrictMode() && types.IsTypeTime(col.GetType()) { - if vv, ok := value.(string); ok { - timeValue, err := expression.GetTimeValue(ctx, vv, col.GetType(), col.GetDecimal(), nil) - if err != nil { - return hasDefaultValue, value, errors.Trace(err) - } - if timeValue.GetMysqlTime().CoreTime() == types.ZeroCoreTime { - return hasDefaultValue, value, types.ErrInvalidDefault.GenWithStackByArgs(col.Name.O) - } - } - } - return hasDefaultValue, value, nil -} - -func checkSequenceDefaultValue(col *table.Column) error { - if mysql.IsIntegerType(col.GetType()) { - return nil - } - return dbterror.ErrColumnTypeUnsupportedNextValue.GenWithStackByArgs(col.ColumnInfo.Name.O) -} - -func convertTimestampDefaultValToUTC(ctx sessionctx.Context, defaultVal any, col *table.Column) (any, error) { - if defaultVal == nil || col.GetType() != mysql.TypeTimestamp { - return defaultVal, nil - } - if vv, ok := defaultVal.(string); ok { - if vv != types.ZeroDatetimeStr && !strings.EqualFold(vv, ast.CurrentTimestamp) { - t, err := types.ParseTime(ctx.GetSessionVars().StmtCtx.TypeCtx(), vv, col.GetType(), col.GetDecimal()) - if err != nil { - return defaultVal, errors.Trace(err) - } - err = t.ConvertTimeZone(ctx.GetSessionVars().Location(), time.UTC) - if err != nil { - return defaultVal, errors.Trace(err) - } - defaultVal = t.String() - } - } - return defaultVal, nil -} - -// isExplicitTimeStamp is used to check if explicit_defaults_for_timestamp is on or off. -// Check out this link for more details. -// https://dev.mysql.com/doc/refman/5.7/en/server-system-variables.html#sysvar_explicit_defaults_for_timestamp -func isExplicitTimeStamp() bool { - // TODO: implement the behavior as MySQL when explicit_defaults_for_timestamp = off, then this function could return false. - return true -} - -// processColumnFlags is used by columnDefToCol and processColumnOptions. It is intended to unify behaviors on `create/add` and `modify/change` statements. Check tidb#issue#19342. -func processColumnFlags(col *table.Column) { - if col.FieldType.EvalType().IsStringKind() { - if col.GetCharset() == charset.CharsetBin { - col.AddFlag(mysql.BinaryFlag) - } else { - col.DelFlag(mysql.BinaryFlag) - } - } - if col.GetType() == mysql.TypeBit { - // For BIT field, it's charset is binary but does not have binary flag. - col.DelFlag(mysql.BinaryFlag) - col.AddFlag(mysql.UnsignedFlag) - } - if col.GetType() == mysql.TypeYear { - // For Year field, it's charset is binary but does not have binary flag. - col.DelFlag(mysql.BinaryFlag) - col.AddFlag(mysql.ZerofillFlag) - } - - // If you specify ZEROFILL for a numeric column, MySQL automatically adds the UNSIGNED attribute to the column. - // See https://dev.mysql.com/doc/refman/5.7/en/numeric-type-overview.html for more details. - // But some types like bit and year, won't show its unsigned flag in `show create table`. - if mysql.HasZerofillFlag(col.GetFlag()) { - col.AddFlag(mysql.UnsignedFlag) - } -} - -func adjustBlobTypesFlen(tp *types.FieldType, colCharset string) error { - cs, err := charset.GetCharsetInfo(colCharset) - // when we meet the unsupported charset, we do not adjust. - if err != nil { - return err - } - l := tp.GetFlen() * cs.Maxlen - if tp.GetType() == mysql.TypeBlob { - if l <= tinyBlobMaxLength { - logutil.DDLLogger().Info(fmt.Sprintf("Automatically convert BLOB(%d) to TINYBLOB", tp.GetFlen())) - tp.SetFlen(tinyBlobMaxLength) - tp.SetType(mysql.TypeTinyBlob) - } else if l <= blobMaxLength { - tp.SetFlen(blobMaxLength) - } else if l <= mediumBlobMaxLength { - logutil.DDLLogger().Info(fmt.Sprintf("Automatically convert BLOB(%d) to MEDIUMBLOB", tp.GetFlen())) - tp.SetFlen(mediumBlobMaxLength) - tp.SetType(mysql.TypeMediumBlob) - } else if l <= longBlobMaxLength { - logutil.DDLLogger().Info(fmt.Sprintf("Automatically convert BLOB(%d) to LONGBLOB", tp.GetFlen())) - tp.SetFlen(longBlobMaxLength) - tp.SetType(mysql.TypeLongBlob) - } - } - return nil -} - -// columnDefToCol converts ColumnDef to Col and TableConstraints. -// outPriKeyConstraint is the primary key constraint out of column definition. such as: create table t1 (id int , age int, primary key(id)); -func columnDefToCol(ctx sessionctx.Context, offset int, colDef *ast.ColumnDef, outPriKeyConstraint *ast.Constraint) (*table.Column, []*ast.Constraint, error) { - var constraints = make([]*ast.Constraint, 0) - col := table.ToColumn(&model.ColumnInfo{ - Offset: offset, - Name: colDef.Name.Name, - FieldType: *colDef.Tp, - // TODO: remove this version field after there is no old version. - Version: model.CurrLatestColumnInfoVersion, - }) - - if !isExplicitTimeStamp() { - // Check and set TimestampFlag, OnUpdateNowFlag and NotNullFlag. - if col.GetType() == mysql.TypeTimestamp { - col.AddFlag(mysql.TimestampFlag | mysql.OnUpdateNowFlag | mysql.NotNullFlag) - } - } - var err error - setOnUpdateNow := false - hasDefaultValue := false - hasNullFlag := false - if colDef.Options != nil { - length := types.UnspecifiedLength - - keys := []*ast.IndexPartSpecification{ - { - Column: colDef.Name, - Length: length, - }, - } - - var sb strings.Builder - restoreFlags := format.RestoreStringSingleQuotes | format.RestoreKeyWordLowercase | format.RestoreNameBackQuotes | - format.RestoreSpacesAroundBinaryOperation | format.RestoreWithoutSchemaName | format.RestoreWithoutTableName - restoreCtx := format.NewRestoreCtx(restoreFlags, &sb) - - for _, v := range colDef.Options { - switch v.Tp { - case ast.ColumnOptionNotNull: - col.AddFlag(mysql.NotNullFlag) - case ast.ColumnOptionNull: - col.DelFlag(mysql.NotNullFlag) - removeOnUpdateNowFlag(col) - hasNullFlag = true - case ast.ColumnOptionAutoIncrement: - col.AddFlag(mysql.AutoIncrementFlag | mysql.NotNullFlag) - case ast.ColumnOptionPrimaryKey: - // Check PriKeyFlag first to avoid extra duplicate constraints. - if col.GetFlag()&mysql.PriKeyFlag == 0 { - constraint := &ast.Constraint{Tp: ast.ConstraintPrimaryKey, Keys: keys, - Option: &ast.IndexOption{PrimaryKeyTp: v.PrimaryKeyTp}} - constraints = append(constraints, constraint) - col.AddFlag(mysql.PriKeyFlag) - // Add NotNullFlag early so that processColumnFlags() can see it. - col.AddFlag(mysql.NotNullFlag) - } - case ast.ColumnOptionUniqKey: - // Check UniqueFlag first to avoid extra duplicate constraints. - if col.GetFlag()&mysql.UniqueFlag == 0 { - constraint := &ast.Constraint{Tp: ast.ConstraintUniqKey, Keys: keys} - constraints = append(constraints, constraint) - col.AddFlag(mysql.UniqueKeyFlag) - } - case ast.ColumnOptionDefaultValue: - hasDefaultValue, err = SetDefaultValue(ctx, col, v) - if err != nil { - return nil, nil, errors.Trace(err) - } - removeOnUpdateNowFlag(col) - case ast.ColumnOptionOnUpdate: - // TODO: Support other time functions. - if !(col.GetType() == mysql.TypeTimestamp || col.GetType() == mysql.TypeDatetime) { - return nil, nil, dbterror.ErrInvalidOnUpdate.GenWithStackByArgs(col.Name) - } - if !expression.IsValidCurrentTimestampExpr(v.Expr, colDef.Tp) { - return nil, nil, dbterror.ErrInvalidOnUpdate.GenWithStackByArgs(col.Name) - } - col.AddFlag(mysql.OnUpdateNowFlag) - setOnUpdateNow = true - case ast.ColumnOptionComment: - err := setColumnComment(ctx, col, v) - if err != nil { - return nil, nil, errors.Trace(err) - } - case ast.ColumnOptionGenerated: - sb.Reset() - err = v.Expr.Restore(restoreCtx) - if err != nil { - return nil, nil, errors.Trace(err) - } - col.GeneratedExprString = sb.String() - col.GeneratedStored = v.Stored - _, dependColNames, err := findDependedColumnNames(model.NewCIStr(""), model.NewCIStr(""), colDef) - if err != nil { - return nil, nil, errors.Trace(err) - } - col.Dependences = dependColNames - case ast.ColumnOptionCollate: - if field_types.HasCharset(colDef.Tp) { - col.FieldType.SetCollate(v.StrValue) - } - case ast.ColumnOptionFulltext: - ctx.GetSessionVars().StmtCtx.AppendWarning(dbterror.ErrTableCantHandleFt.FastGenByArgs()) - case ast.ColumnOptionCheck: - if !variable.EnableCheckConstraint.Load() { - ctx.GetSessionVars().StmtCtx.AppendWarning(errCheckConstraintIsOff) - } else { - // Check the column CHECK constraint dependency lazily, after fill all the name. - // Extract column constraint from column option. - constraint := &ast.Constraint{ - Tp: ast.ConstraintCheck, - Expr: v.Expr, - Enforced: v.Enforced, - Name: v.ConstraintName, - InColumn: true, - InColumnName: colDef.Name.Name.O, - } - constraints = append(constraints, constraint) - } - } - } - } - - if err = processAndCheckDefaultValueAndColumn(ctx, col, outPriKeyConstraint, hasDefaultValue, setOnUpdateNow, hasNullFlag); err != nil { - return nil, nil, errors.Trace(err) - } - return col, constraints, nil -} - -func restoreFuncCall(expr *ast.FuncCallExpr) (string, error) { - var sb strings.Builder - restoreFlags := format.RestoreStringSingleQuotes | format.RestoreKeyWordLowercase | format.RestoreNameBackQuotes | - format.RestoreSpacesAroundBinaryOperation - restoreCtx := format.NewRestoreCtx(restoreFlags, &sb) - if err := expr.Restore(restoreCtx); err != nil { - return "", err - } - return sb.String(), nil -} - -// getFuncCallDefaultValue gets the default column value of function-call expression. -func getFuncCallDefaultValue(col *table.Column, option *ast.ColumnOption, expr *ast.FuncCallExpr) (any, bool, error) { - switch expr.FnName.L { - case ast.CurrentTimestamp, ast.CurrentDate: // CURRENT_TIMESTAMP() and CURRENT_DATE() - tp, fsp := col.FieldType.GetType(), col.FieldType.GetDecimal() - if tp == mysql.TypeTimestamp || tp == mysql.TypeDatetime { - defaultFsp := 0 - if len(expr.Args) == 1 { - if val := expr.Args[0].(*driver.ValueExpr); val != nil { - defaultFsp = int(val.GetInt64()) - } - } - if defaultFsp != fsp { - return nil, false, dbterror.ErrInvalidDefaultValue.GenWithStackByArgs(col.Name.O) - } - } - return nil, false, nil - case ast.NextVal: - // handle default next value of sequence. (keep the expr string) - str, err := getSequenceDefaultValue(option) - if err != nil { - return nil, false, errors.Trace(err) - } - return str, true, nil - case ast.Rand, ast.UUID, ast.UUIDToBin: // RAND(), UUID() and UUID_TO_BIN() - if err := expression.VerifyArgsWrapper(expr.FnName.L, len(expr.Args)); err != nil { - return nil, false, errors.Trace(err) - } - str, err := restoreFuncCall(expr) - if err != nil { - return nil, false, errors.Trace(err) - } - col.DefaultIsExpr = true - return str, false, nil - case ast.DateFormat: // DATE_FORMAT() - if err := expression.VerifyArgsWrapper(expr.FnName.L, len(expr.Args)); err != nil { - return nil, false, errors.Trace(err) - } - // Support DATE_FORMAT(NOW(),'%Y-%m'), DATE_FORMAT(NOW(),'%Y-%m-%d'), - // DATE_FORMAT(NOW(),'%Y-%m-%d %H.%i.%s'), DATE_FORMAT(NOW(),'%Y-%m-%d %H:%i:%s'). - nowFunc, ok := expr.Args[0].(*ast.FuncCallExpr) - if ok && nowFunc.FnName.L == ast.Now { - if err := expression.VerifyArgsWrapper(nowFunc.FnName.L, len(nowFunc.Args)); err != nil { - return nil, false, errors.Trace(err) - } - valExpr, isValue := expr.Args[1].(ast.ValueExpr) - if !isValue || (valExpr.GetString() != "%Y-%m" && valExpr.GetString() != "%Y-%m-%d" && - valExpr.GetString() != "%Y-%m-%d %H.%i.%s" && valExpr.GetString() != "%Y-%m-%d %H:%i:%s") { - return nil, false, dbterror.ErrDefValGeneratedNamedFunctionIsNotAllowed.GenWithStackByArgs(col.Name.String(), valExpr) - } - str, err := restoreFuncCall(expr) - if err != nil { - return nil, false, errors.Trace(err) - } - col.DefaultIsExpr = true - return str, false, nil - } - return nil, false, dbterror.ErrDefValGeneratedNamedFunctionIsNotAllowed.GenWithStackByArgs(col.Name.String(), - fmt.Sprintf("%s with disallowed args", expr.FnName.String())) - case ast.Replace: - if err := expression.VerifyArgsWrapper(expr.FnName.L, len(expr.Args)); err != nil { - return nil, false, errors.Trace(err) - } - funcCall := expr.Args[0] - // Support REPLACE(CONVERT(UPPER(UUID()) USING UTF8MB4), '-', '')) - if convertFunc, ok := funcCall.(*ast.FuncCallExpr); ok && convertFunc.FnName.L == ast.Convert { - if err := expression.VerifyArgsWrapper(convertFunc.FnName.L, len(convertFunc.Args)); err != nil { - return nil, false, errors.Trace(err) - } - funcCall = convertFunc.Args[0] - } - // Support REPLACE(UPPER(UUID()), '-', ''). - if upperFunc, ok := funcCall.(*ast.FuncCallExpr); ok && upperFunc.FnName.L == ast.Upper { - if err := expression.VerifyArgsWrapper(upperFunc.FnName.L, len(upperFunc.Args)); err != nil { - return nil, false, errors.Trace(err) - } - if uuidFunc, ok := upperFunc.Args[0].(*ast.FuncCallExpr); ok && uuidFunc.FnName.L == ast.UUID { - if err := expression.VerifyArgsWrapper(uuidFunc.FnName.L, len(uuidFunc.Args)); err != nil { - return nil, false, errors.Trace(err) - } - str, err := restoreFuncCall(expr) - if err != nil { - return nil, false, errors.Trace(err) - } - col.DefaultIsExpr = true - return str, false, nil - } - } - return nil, false, dbterror.ErrDefValGeneratedNamedFunctionIsNotAllowed.GenWithStackByArgs(col.Name.String(), - fmt.Sprintf("%s with disallowed args", expr.FnName.String())) - case ast.Upper: - if err := expression.VerifyArgsWrapper(expr.FnName.L, len(expr.Args)); err != nil { - return nil, false, errors.Trace(err) - } - // Support UPPER(SUBSTRING_INDEX(USER(), '@', 1)). - if substringIndexFunc, ok := expr.Args[0].(*ast.FuncCallExpr); ok && substringIndexFunc.FnName.L == ast.SubstringIndex { - if err := expression.VerifyArgsWrapper(substringIndexFunc.FnName.L, len(substringIndexFunc.Args)); err != nil { - return nil, false, errors.Trace(err) - } - if userFunc, ok := substringIndexFunc.Args[0].(*ast.FuncCallExpr); ok && userFunc.FnName.L == ast.User { - if err := expression.VerifyArgsWrapper(userFunc.FnName.L, len(userFunc.Args)); err != nil { - return nil, false, errors.Trace(err) - } - valExpr, isValue := substringIndexFunc.Args[1].(ast.ValueExpr) - if !isValue || valExpr.GetString() != "@" { - return nil, false, dbterror.ErrDefValGeneratedNamedFunctionIsNotAllowed.GenWithStackByArgs(col.Name.String(), valExpr) - } - str, err := restoreFuncCall(expr) - if err != nil { - return nil, false, errors.Trace(err) - } - col.DefaultIsExpr = true - return str, false, nil - } - } - return nil, false, dbterror.ErrDefValGeneratedNamedFunctionIsNotAllowed.GenWithStackByArgs(col.Name.String(), - fmt.Sprintf("%s with disallowed args", expr.FnName.String())) - case ast.StrToDate: // STR_TO_DATE() - if err := expression.VerifyArgsWrapper(expr.FnName.L, len(expr.Args)); err != nil { - return nil, false, errors.Trace(err) - } - // Support STR_TO_DATE('1980-01-01', '%Y-%m-%d'). - if _, ok1 := expr.Args[0].(ast.ValueExpr); ok1 { - if _, ok2 := expr.Args[1].(ast.ValueExpr); ok2 { - str, err := restoreFuncCall(expr) - if err != nil { - return nil, false, errors.Trace(err) - } - col.DefaultIsExpr = true - return str, false, nil - } - } - return nil, false, dbterror.ErrDefValGeneratedNamedFunctionIsNotAllowed.GenWithStackByArgs(col.Name.String(), - fmt.Sprintf("%s with disallowed args", expr.FnName.String())) - case ast.JSONObject, ast.JSONArray, ast.JSONQuote: // JSON_OBJECT(), JSON_ARRAY(), JSON_QUOTE() - if err := expression.VerifyArgsWrapper(expr.FnName.L, len(expr.Args)); err != nil { - return nil, false, errors.Trace(err) - } - str, err := restoreFuncCall(expr) - if err != nil { - return nil, false, errors.Trace(err) - } - col.DefaultIsExpr = true - return str, false, nil - - default: - return nil, false, dbterror.ErrDefValGeneratedNamedFunctionIsNotAllowed.GenWithStackByArgs(col.Name.String(), expr.FnName.String()) - } -} - -// getDefaultValue will get the default value for column. -// 1: get the expr restored string for the column which uses sequence next value as default value. -// 2: get specific default value for the other column. -func getDefaultValue(ctx exprctx.BuildContext, col *table.Column, option *ast.ColumnOption) (any, bool, error) { - // handle default value with function call - tp, fsp := col.FieldType.GetType(), col.FieldType.GetDecimal() - if x, ok := option.Expr.(*ast.FuncCallExpr); ok { - val, isSeqExpr, err := getFuncCallDefaultValue(col, option, x) - if val != nil || isSeqExpr || err != nil { - return val, isSeqExpr, err - } - // If the function call is ast.CurrentTimestamp, it needs to be continuously processed. - } - - if tp == mysql.TypeTimestamp || tp == mysql.TypeDatetime || tp == mysql.TypeDate { - vd, err := expression.GetTimeValue(ctx, option.Expr, tp, fsp, nil) - value := vd.GetValue() - if err != nil { - return nil, false, dbterror.ErrInvalidDefaultValue.GenWithStackByArgs(col.Name.O) - } - - // Value is nil means `default null`. - if value == nil { - return nil, false, nil - } - - // If value is types.Time, convert it to string. - if vv, ok := value.(types.Time); ok { - return vv.String(), false, nil - } - - return value, false, nil - } - - // evaluate the non-function-call expr to a certain value. - v, err := expression.EvalSimpleAst(ctx, option.Expr) - if err != nil { - return nil, false, errors.Trace(err) - } - - if v.IsNull() { - return nil, false, nil - } - - if v.Kind() == types.KindBinaryLiteral || v.Kind() == types.KindMysqlBit { - if types.IsTypeBlob(tp) || tp == mysql.TypeJSON { - // BLOB/TEXT/JSON column cannot have a default value. - // Skip the unnecessary decode procedure. - return v.GetString(), false, err - } - if tp == mysql.TypeBit || tp == mysql.TypeString || tp == mysql.TypeVarchar || - tp == mysql.TypeVarString || tp == mysql.TypeEnum || tp == mysql.TypeSet { - // For BinaryLiteral or bit fields, we decode the default value to utf8 string. - str, err := v.GetBinaryStringDecoded(types.StrictFlags, col.GetCharset()) - if err != nil { - // Overwrite the decoding error with invalid default value error. - err = dbterror.ErrInvalidDefaultValue.GenWithStackByArgs(col.Name.O) - } - return str, false, err - } - // For other kind of fields (e.g. INT), we supply its integer as string value. - value, err := v.GetBinaryLiteral().ToInt(ctx.GetEvalCtx().TypeCtx()) - if err != nil { - return nil, false, err - } - return strconv.FormatUint(value, 10), false, nil - } - - switch tp { - case mysql.TypeSet: - val, err := getSetDefaultValue(v, col) - return val, false, err - case mysql.TypeEnum: - val, err := getEnumDefaultValue(v, col) - return val, false, err - case mysql.TypeDuration, mysql.TypeDate: - if v, err = v.ConvertTo(ctx.GetEvalCtx().TypeCtx(), &col.FieldType); err != nil { - return "", false, errors.Trace(err) - } - case mysql.TypeBit: - if v.Kind() == types.KindInt64 || v.Kind() == types.KindUint64 { - // For BIT fields, convert int into BinaryLiteral. - return types.NewBinaryLiteralFromUint(v.GetUint64(), -1).ToString(), false, nil - } - case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeFloat, mysql.TypeDouble: - // For these types, convert it to standard format firstly. - // like integer fields, convert it into integer string literals. like convert "1.25" into "1" and "2.8" into "3". - // if raise a error, we will use original expression. We will handle it in check phase - if temp, err := v.ConvertTo(ctx.GetEvalCtx().TypeCtx(), &col.FieldType); err == nil { - v = temp - } - } - - val, err := v.ToString() - return val, false, err -} - -func getSequenceDefaultValue(c *ast.ColumnOption) (expr string, err error) { - var sb strings.Builder - restoreFlags := format.RestoreStringSingleQuotes | format.RestoreKeyWordLowercase | format.RestoreNameBackQuotes | - format.RestoreSpacesAroundBinaryOperation - restoreCtx := format.NewRestoreCtx(restoreFlags, &sb) - if err := c.Expr.Restore(restoreCtx); err != nil { - return "", err - } - return sb.String(), nil -} - -// getSetDefaultValue gets the default value for the set type. See https://dev.mysql.com/doc/refman/5.7/en/set.html. -func getSetDefaultValue(v types.Datum, col *table.Column) (string, error) { - if v.Kind() == types.KindInt64 { - setCnt := len(col.GetElems()) - maxLimit := int64(1< maxLimit { - return "", dbterror.ErrInvalidDefaultValue.GenWithStackByArgs(col.Name.O) - } - setVal, err := types.ParseSetValue(col.GetElems(), uint64(val)) - if err != nil { - return "", errors.Trace(err) - } - v.SetMysqlSet(setVal, col.GetCollate()) - return v.ToString() - } - - str, err := v.ToString() - if err != nil { - return "", errors.Trace(err) - } - if str == "" { - return str, nil - } - setVal, err := types.ParseSetName(col.GetElems(), str, col.GetCollate()) - if err != nil { - return "", dbterror.ErrInvalidDefaultValue.GenWithStackByArgs(col.Name.O) - } - v.SetMysqlSet(setVal, col.GetCollate()) - - return v.ToString() -} - -// getEnumDefaultValue gets the default value for the enum type. See https://dev.mysql.com/doc/refman/5.7/en/enum.html. -func getEnumDefaultValue(v types.Datum, col *table.Column) (string, error) { - if v.Kind() == types.KindInt64 { - val := v.GetInt64() - if val < 1 || val > int64(len(col.GetElems())) { - return "", dbterror.ErrInvalidDefaultValue.GenWithStackByArgs(col.Name.O) - } - enumVal, err := types.ParseEnumValue(col.GetElems(), uint64(val)) - if err != nil { - return "", errors.Trace(err) - } - v.SetMysqlEnum(enumVal, col.GetCollate()) - return v.ToString() - } - str, err := v.ToString() - if err != nil { - return "", errors.Trace(err) - } - // Ref: https://dev.mysql.com/doc/refman/8.0/en/enum.html - // Trailing spaces are automatically deleted from ENUM member values in the table definition when a table is created. - str = strings.TrimRight(str, " ") - enumVal, err := types.ParseEnumName(col.GetElems(), str, col.GetCollate()) - if err != nil { - return "", dbterror.ErrInvalidDefaultValue.GenWithStackByArgs(col.Name.O) - } - v.SetMysqlEnum(enumVal, col.GetCollate()) - - return v.ToString() -} - -func removeOnUpdateNowFlag(c *table.Column) { - // For timestamp Col, if it is set null or default value, - // OnUpdateNowFlag should be removed. - if mysql.HasTimestampFlag(c.GetFlag()) { - c.DelFlag(mysql.OnUpdateNowFlag) - } -} - -func processDefaultValue(c *table.Column, hasDefaultValue bool, setOnUpdateNow bool) { - setTimestampDefaultValue(c, hasDefaultValue, setOnUpdateNow) - - setYearDefaultValue(c, hasDefaultValue) - - // Set `NoDefaultValueFlag` if this field doesn't have a default value and - // it is `not null` and not an `AUTO_INCREMENT` field or `TIMESTAMP` field. - setNoDefaultValueFlag(c, hasDefaultValue) -} - -func setYearDefaultValue(c *table.Column, hasDefaultValue bool) { - if hasDefaultValue { - return - } - - if c.GetType() == mysql.TypeYear && mysql.HasNotNullFlag(c.GetFlag()) { - if err := c.SetDefaultValue("0000"); err != nil { - logutil.DDLLogger().Error("set default value failed", zap.Error(err)) - } - } -} - -func setTimestampDefaultValue(c *table.Column, hasDefaultValue bool, setOnUpdateNow bool) { - if hasDefaultValue { - return - } - - // For timestamp Col, if is not set default value or not set null, use current timestamp. - if mysql.HasTimestampFlag(c.GetFlag()) && mysql.HasNotNullFlag(c.GetFlag()) { - if setOnUpdateNow { - if err := c.SetDefaultValue(types.ZeroDatetimeStr); err != nil { - logutil.DDLLogger().Error("set default value failed", zap.Error(err)) - } - } else { - if err := c.SetDefaultValue(strings.ToUpper(ast.CurrentTimestamp)); err != nil { - logutil.DDLLogger().Error("set default value failed", zap.Error(err)) - } - } - } -} - -func setNoDefaultValueFlag(c *table.Column, hasDefaultValue bool) { - if hasDefaultValue { - return - } - - if !mysql.HasNotNullFlag(c.GetFlag()) { - return - } - - // Check if it is an `AUTO_INCREMENT` field or `TIMESTAMP` field. - if !mysql.HasAutoIncrementFlag(c.GetFlag()) && !mysql.HasTimestampFlag(c.GetFlag()) { - c.AddFlag(mysql.NoDefaultValueFlag) - } -} - -func checkDefaultValue(ctx exprctx.BuildContext, c *table.Column, hasDefaultValue bool) (err error) { - if !hasDefaultValue { - return nil - } - - if c.GetDefaultValue() != nil { - if c.DefaultIsExpr { - if mysql.HasAutoIncrementFlag(c.GetFlag()) { - return types.ErrInvalidDefault.GenWithStackByArgs(c.Name) - } - return nil - } - _, err = table.GetColDefaultValue( - exprctx.CtxWithHandleTruncateErrLevel(ctx, errctx.LevelError), - c.ToInfo(), - ) - if err != nil { - return types.ErrInvalidDefault.GenWithStackByArgs(c.Name) - } - return nil - } - // Primary key default null is invalid. - if mysql.HasPriKeyFlag(c.GetFlag()) { - return dbterror.ErrPrimaryCantHaveNull - } - - // Set not null but default null is invalid. - if mysql.HasNotNullFlag(c.GetFlag()) { - return types.ErrInvalidDefault.GenWithStackByArgs(c.Name) - } - - return nil -} - -// checkPriKeyConstraint check all parts of a PRIMARY KEY must be NOT NULL -func checkPriKeyConstraint(col *table.Column, hasDefaultValue, hasNullFlag bool, outPriKeyConstraint *ast.Constraint) error { - // Primary key should not be null. - if mysql.HasPriKeyFlag(col.GetFlag()) && hasDefaultValue && col.GetDefaultValue() == nil { - return types.ErrInvalidDefault.GenWithStackByArgs(col.Name) - } - // Set primary key flag for outer primary key constraint. - // Such as: create table t1 (id int , age int, primary key(id)) - if !mysql.HasPriKeyFlag(col.GetFlag()) && outPriKeyConstraint != nil { - for _, key := range outPriKeyConstraint.Keys { - if key.Expr == nil && key.Column.Name.L != col.Name.L { - continue - } - col.AddFlag(mysql.PriKeyFlag) - break - } - } - // Primary key should not be null. - if mysql.HasPriKeyFlag(col.GetFlag()) && hasNullFlag { - return dbterror.ErrPrimaryCantHaveNull - } - return nil -} - -func checkColumnValueConstraint(col *table.Column, collation string) error { - if col.GetType() != mysql.TypeEnum && col.GetType() != mysql.TypeSet { - return nil - } - valueMap := make(map[string]bool, len(col.GetElems())) - ctor := collate.GetCollator(collation) - enumLengthLimit := config.GetGlobalConfig().EnableEnumLengthLimit - desc, err := charset.GetCharsetInfo(col.GetCharset()) - if err != nil { - return errors.Trace(err) - } - for i := range col.GetElems() { - val := string(ctor.Key(col.GetElems()[i])) - // According to MySQL 8.0 Refman: - // The maximum supported length of an individual ENUM element is M <= 255 and (M x w) <= 1020, - // where M is the element literal length and w is the number of bytes required for the maximum-length character in the character set. - // See https://dev.mysql.com/doc/refman/8.0/en/string-type-syntax.html for more details. - if enumLengthLimit && (len(val) > 255 || len(val)*desc.Maxlen > 1020) { - return dbterror.ErrTooLongValueForType.GenWithStackByArgs(col.Name) - } - if _, ok := valueMap[val]; ok { - tpStr := "ENUM" - if col.GetType() == mysql.TypeSet { - tpStr = "SET" - } - return types.ErrDuplicatedValueInType.GenWithStackByArgs(col.Name, col.GetElems()[i], tpStr) - } - valueMap[val] = true - } - return nil -} - func checkDuplicateColumn(cols []*model.ColumnInfo) error { colNames := set.StringSet{} for _, col := range cols { @@ -2080,82 +1211,6 @@ func checkInvisibleIndexOnPK(tblInfo *model.TableInfo) error { return nil } -// checkTableInfoValidExtra is like checkTableInfoValid, but also assumes the -// table info comes from untrusted source and performs further checks such as -// name length and column count. -// (checkTableInfoValid is also used in repairing objects which don't perform -// these checks. Perhaps the two functions should be merged together regardless?) -func checkTableInfoValidExtra(tbInfo *model.TableInfo) error { - if err := checkTooLongTable(tbInfo.Name); err != nil { - return err - } - - if err := checkDuplicateColumn(tbInfo.Columns); err != nil { - return err - } - if err := checkTooLongColumns(tbInfo.Columns); err != nil { - return err - } - if err := checkTooManyColumns(tbInfo.Columns); err != nil { - return errors.Trace(err) - } - if err := checkTooManyIndexes(tbInfo.Indices); err != nil { - return errors.Trace(err) - } - if err := checkColumnsAttributes(tbInfo.Columns); err != nil { - return errors.Trace(err) - } - - // FIXME: perform checkConstraintNames - if err := checkCharsetAndCollation(tbInfo.Charset, tbInfo.Collate); err != nil { - return errors.Trace(err) - } - - oldState := tbInfo.State - tbInfo.State = model.StatePublic - err := checkTableInfoValid(tbInfo) - tbInfo.State = oldState - return err -} - -// CheckTableInfoValidWithStmt exposes checkTableInfoValidWithStmt to SchemaTracker. Maybe one day we can delete it. -func CheckTableInfoValidWithStmt(ctx sessionctx.Context, tbInfo *model.TableInfo, s *ast.CreateTableStmt) (err error) { - return checkTableInfoValidWithStmt(ctx, tbInfo, s) -} - -func checkTableInfoValidWithStmt(ctx sessionctx.Context, tbInfo *model.TableInfo, s *ast.CreateTableStmt) (err error) { - // All of these rely on the AST structure of expressions, which were - // lost in the model (got serialized into strings). - if err := checkGeneratedColumn(ctx, s.Table.Schema, tbInfo.Name, s.Cols); err != nil { - return errors.Trace(err) - } - - // Check if table has a primary key if required. - if !ctx.GetSessionVars().InRestrictedSQL && ctx.GetSessionVars().PrimaryKeyRequired && len(tbInfo.GetPkName().String()) == 0 { - return infoschema.ErrTableWithoutPrimaryKey - } - if tbInfo.Partition != nil { - if err := checkPartitionDefinitionConstraints(ctx, tbInfo); err != nil { - return errors.Trace(err) - } - if s.Partition != nil { - if err := checkPartitionFuncType(ctx, s.Partition.Expr, s.Table.Schema.O, tbInfo); err != nil { - return errors.Trace(err) - } - if err := checkPartitioningKeysConstraints(ctx, s, tbInfo); err != nil { - return errors.Trace(err) - } - } - } - if tbInfo.TTLInfo != nil { - if err := checkTTLInfoValid(ctx, s.Table.Schema, tbInfo); err != nil { - return errors.Trace(err) - } - } - - return nil -} - func checkPartitionDefinitionConstraints(ctx sessionctx.Context, tbInfo *model.TableInfo) error { var err error if err = checkPartitionNameUnique(tbInfo.Partition); err != nil { @@ -3891,120 +2946,6 @@ func checkUnsupportedColumnConstraint(col *ast.ColumnDef, ti ast.Ident) error { return nil } -func checkAndCreateNewColumn(ctx sessionctx.Context, ti ast.Ident, schema *model.DBInfo, spec *ast.AlterTableSpec, t table.Table, specNewColumn *ast.ColumnDef) (*table.Column, error) { - err := checkUnsupportedColumnConstraint(specNewColumn, ti) - if err != nil { - return nil, errors.Trace(err) - } - - colName := specNewColumn.Name.Name.O - // Check whether added column has existed. - col := table.FindCol(t.Cols(), colName) - if col != nil { - err = infoschema.ErrColumnExists.GenWithStackByArgs(colName) - if spec.IfNotExists { - ctx.GetSessionVars().StmtCtx.AppendNote(err) - return nil, nil - } - return nil, err - } - if err = checkColumnAttributes(colName, specNewColumn.Tp); err != nil { - return nil, errors.Trace(err) - } - if utf8.RuneCountInString(colName) > mysql.MaxColumnNameLength { - return nil, dbterror.ErrTooLongIdent.GenWithStackByArgs(colName) - } - - return CreateNewColumn(ctx, schema, spec, t, specNewColumn) -} - -// CreateNewColumn creates a new column according to the column information. -func CreateNewColumn(ctx sessionctx.Context, schema *model.DBInfo, spec *ast.AlterTableSpec, t table.Table, specNewColumn *ast.ColumnDef) (*table.Column, error) { - // If new column is a generated column, do validation. - // NOTE: we do check whether the column refers other generated - // columns occurring later in a table, but we don't handle the col offset. - for _, option := range specNewColumn.Options { - if option.Tp == ast.ColumnOptionGenerated { - if err := checkIllegalFn4Generated(specNewColumn.Name.Name.L, typeColumn, option.Expr); err != nil { - return nil, errors.Trace(err) - } - - if option.Stored { - return nil, dbterror.ErrUnsupportedOnGeneratedColumn.GenWithStackByArgs("Adding generated stored column through ALTER TABLE") - } - - _, dependColNames, err := findDependedColumnNames(schema.Name, t.Meta().Name, specNewColumn) - if err != nil { - return nil, errors.Trace(err) - } - if !ctx.GetSessionVars().EnableAutoIncrementInGenerated { - if err := checkAutoIncrementRef(specNewColumn.Name.Name.L, dependColNames, t.Meta()); err != nil { - return nil, errors.Trace(err) - } - } - duplicateColNames := make(map[string]struct{}, len(dependColNames)) - for k := range dependColNames { - duplicateColNames[k] = struct{}{} - } - cols := t.Cols() - - if err := checkDependedColExist(dependColNames, cols); err != nil { - return nil, errors.Trace(err) - } - - if err := verifyColumnGenerationSingle(duplicateColNames, cols, spec.Position); err != nil { - return nil, errors.Trace(err) - } - } - // Specially, since sequence has been supported, if a newly added column has a - // sequence nextval function as it's default value option, it won't fill the - // known rows with specific sequence next value under current add column logic. - // More explanation can refer: TestSequenceDefaultLogic's comment in sequence_test.go - if option.Tp == ast.ColumnOptionDefaultValue { - if f, ok := option.Expr.(*ast.FuncCallExpr); ok { - switch f.FnName.L { - case ast.NextVal: - if _, err := getSequenceDefaultValue(option); err != nil { - return nil, errors.Trace(err) - } - return nil, errors.Trace(dbterror.ErrAddColumnWithSequenceAsDefault.GenWithStackByArgs(specNewColumn.Name.Name.O)) - case ast.Rand, ast.UUID, ast.UUIDToBin, ast.Replace, ast.Upper: - return nil, errors.Trace(dbterror.ErrBinlogUnsafeSystemFunction.GenWithStackByArgs()) - } - } - } - } - - tableCharset, tableCollate, err := ResolveCharsetCollation(ctx.GetSessionVars(), - ast.CharsetOpt{Chs: t.Meta().Charset, Col: t.Meta().Collate}, - ast.CharsetOpt{Chs: schema.Charset, Col: schema.Collate}, - ) - if err != nil { - return nil, errors.Trace(err) - } - // Ignore table constraints now, they will be checked later. - // We use length(t.Cols()) as the default offset firstly, we will change the column's offset later. - col, _, err := buildColumnAndConstraint( - ctx, - len(t.Cols()), - specNewColumn, - nil, - tableCharset, - tableCollate, - ) - if err != nil { - return nil, errors.Trace(err) - } - - originDefVal, err := generateOriginDefaultValue(col.ToInfo(), ctx) - if err != nil { - return nil, errors.Trace(err) - } - - err = col.SetOriginDefaultValue(originDefVal) - return col, err -} - // 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] @@ -5142,94 +4083,6 @@ func checkModifyCharsetAndCollation(toCharset, toCollate, origCharset, origColla return nil } -// SetDefaultValue sets the default value of the column. -func SetDefaultValue(ctx sessionctx.Context, col *table.Column, option *ast.ColumnOption) (hasDefaultValue bool, err error) { - var value any - var isSeqExpr bool - value, isSeqExpr, err = getDefaultValue( - exprctx.CtxWithHandleTruncateErrLevel(ctx.GetExprCtx(), errctx.LevelError), - col, option, - ) - if err != nil { - return false, errors.Trace(err) - } - if isSeqExpr { - if err := checkSequenceDefaultValue(col); err != nil { - return false, errors.Trace(err) - } - col.DefaultIsExpr = isSeqExpr - } - - // When the default value is expression, we skip check and convert. - if !col.DefaultIsExpr { - if hasDefaultValue, value, err = checkColumnDefaultValue(ctx.GetExprCtx(), col, value); err != nil { - return hasDefaultValue, errors.Trace(err) - } - value, err = convertTimestampDefaultValToUTC(ctx, value, col) - if err != nil { - return hasDefaultValue, errors.Trace(err) - } - } else { - hasDefaultValue = true - } - err = setDefaultValueWithBinaryPadding(col, value) - if err != nil { - return hasDefaultValue, errors.Trace(err) - } - return hasDefaultValue, nil -} - -func setDefaultValueWithBinaryPadding(col *table.Column, value any) error { - err := col.SetDefaultValue(value) - if err != nil { - return err - } - // https://dev.mysql.com/doc/refman/8.0/en/binary-varbinary.html - // Set the default value for binary type should append the paddings. - if value != nil { - if col.GetType() == mysql.TypeString && types.IsBinaryStr(&col.FieldType) && len(value.(string)) < col.GetFlen() { - padding := make([]byte, col.GetFlen()-len(value.(string))) - col.DefaultValue = string(append([]byte(col.DefaultValue.(string)), padding...)) - } - } - return nil -} - -func setColumnComment(ctx sessionctx.Context, col *table.Column, option *ast.ColumnOption) error { - value, err := expression.EvalSimpleAst(ctx.GetExprCtx(), option.Expr) - if err != nil { - return errors.Trace(err) - } - if col.Comment, err = value.ToString(); err != nil { - return errors.Trace(err) - } - - sessionVars := ctx.GetSessionVars() - col.Comment, err = validateCommentLength(sessionVars.StmtCtx.ErrCtx(), sessionVars.SQLMode, col.Name.L, &col.Comment, dbterror.ErrTooLongFieldComment) - return errors.Trace(err) -} - -func processAndCheckDefaultValueAndColumn(ctx sessionctx.Context, col *table.Column, - outPriKeyConstraint *ast.Constraint, hasDefaultValue, setOnUpdateNow, hasNullFlag bool) error { - processDefaultValue(col, hasDefaultValue, setOnUpdateNow) - processColumnFlags(col) - - err := checkPriKeyConstraint(col, hasDefaultValue, hasNullFlag, outPriKeyConstraint) - if err != nil { - return errors.Trace(err) - } - if err = checkColumnValueConstraint(col, col.GetCollate()); err != nil { - return errors.Trace(err) - } - if err = checkDefaultValue(ctx.GetExprCtx(), col, hasDefaultValue); err != nil { - return errors.Trace(err) - } - if err = checkColumnFieldLength(col); err != nil { - return errors.Trace(err) - } - return nil -} - func (e *executor) getModifiableColumnJob(ctx context.Context, sctx sessionctx.Context, ident ast.Ident, originalColName model.CIStr, spec *ast.AlterTableSpec) (*model.Job, error) { is := e.infoCache.GetLatest() @@ -5245,23 +4098,6 @@ func (e *executor) getModifiableColumnJob(ctx context.Context, sctx sessionctx.C return GetModifiableColumnJob(ctx, sctx, is, ident, originalColName, schema, t, spec) } -func isClusteredPKColumn(col *table.Column, tblInfo *model.TableInfo) bool { - switch { - case tblInfo.PKIsHandle: - return mysql.HasPriKeyFlag(col.GetFlag()) - case tblInfo.IsCommonHandle: - pk := tables.FindPrimaryIndex(tblInfo) - for _, c := range pk.Columns { - if c.Name.L == col.Name.L { - return true - } - } - return false - default: - return false - } -} - // 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. @@ -6527,114 +5363,6 @@ func (e *executor) CreatePrimaryKey(ctx sessionctx.Context, ti ast.Ident, indexN return errors.Trace(err) } -func precheckBuildHiddenColumnInfo( - indexPartSpecifications []*ast.IndexPartSpecification, - indexName model.CIStr, -) error { - for i, idxPart := range indexPartSpecifications { - if idxPart.Expr == nil { - continue - } - name := fmt.Sprintf("%s_%s_%d", expressionIndexPrefix, indexName, i) - if utf8.RuneCountInString(name) > mysql.MaxColumnNameLength { - // TODO: Refine the error message. - return dbterror.ErrTooLongIdent.GenWithStackByArgs("hidden column") - } - // TODO: Refine the error message. - if err := checkIllegalFn4Generated(indexName.L, typeIndex, idxPart.Expr); err != nil { - return errors.Trace(err) - } - } - return nil -} - -func buildHiddenColumnInfoWithCheck(ctx sessionctx.Context, indexPartSpecifications []*ast.IndexPartSpecification, indexName model.CIStr, tblInfo *model.TableInfo, existCols []*table.Column) ([]*model.ColumnInfo, error) { - if err := precheckBuildHiddenColumnInfo(indexPartSpecifications, indexName); err != nil { - return nil, err - } - return BuildHiddenColumnInfo(ctx, indexPartSpecifications, indexName, tblInfo, existCols) -} - -// BuildHiddenColumnInfo builds hidden column info. -func BuildHiddenColumnInfo(ctx sessionctx.Context, indexPartSpecifications []*ast.IndexPartSpecification, indexName model.CIStr, tblInfo *model.TableInfo, existCols []*table.Column) ([]*model.ColumnInfo, error) { - hiddenCols := make([]*model.ColumnInfo, 0, len(indexPartSpecifications)) - for i, idxPart := range indexPartSpecifications { - if idxPart.Expr == nil { - continue - } - idxPart.Column = &ast.ColumnName{Name: model.NewCIStr(fmt.Sprintf("%s_%s_%d", expressionIndexPrefix, indexName, i))} - // Check whether the hidden columns have existed. - col := table.FindCol(existCols, idxPart.Column.Name.L) - if col != nil { - // TODO: Use expression index related error. - return nil, infoschema.ErrColumnExists.GenWithStackByArgs(col.Name.String()) - } - idxPart.Length = types.UnspecifiedLength - // The index part is an expression, prepare a hidden column for it. - - var sb strings.Builder - restoreFlags := format.RestoreStringSingleQuotes | format.RestoreKeyWordLowercase | format.RestoreNameBackQuotes | - format.RestoreSpacesAroundBinaryOperation | format.RestoreWithoutSchemaName | format.RestoreWithoutTableName - restoreCtx := format.NewRestoreCtx(restoreFlags, &sb) - sb.Reset() - err := idxPart.Expr.Restore(restoreCtx) - if err != nil { - return nil, errors.Trace(err) - } - expr, err := expression.BuildSimpleExpr(ctx.GetExprCtx(), idxPart.Expr, - expression.WithTableInfo(ctx.GetSessionVars().CurrentDB, tblInfo), - expression.WithAllowCastArray(true), - ) - if err != nil { - // TODO: refine the error message. - return nil, err - } - if _, ok := expr.(*expression.Column); ok { - return nil, dbterror.ErrFunctionalIndexOnField - } - - colInfo := &model.ColumnInfo{ - Name: idxPart.Column.Name, - GeneratedExprString: sb.String(), - GeneratedStored: false, - Version: model.CurrLatestColumnInfoVersion, - Dependences: make(map[string]struct{}), - Hidden: true, - FieldType: *expr.GetType(ctx.GetExprCtx().GetEvalCtx()), - } - // Reset some flag, it may be caused by wrong type infer. But it's not easy to fix them all, so reset them here for safety. - colInfo.DelFlag(mysql.PriKeyFlag | mysql.UniqueKeyFlag | mysql.AutoIncrementFlag) - - if colInfo.GetType() == mysql.TypeDatetime || colInfo.GetType() == mysql.TypeDate || colInfo.GetType() == mysql.TypeTimestamp || colInfo.GetType() == mysql.TypeDuration { - if colInfo.FieldType.GetDecimal() == types.UnspecifiedLength { - colInfo.FieldType.SetDecimal(types.MaxFsp) - } - } - // For an array, the collation is set to "binary". The collation has no effect on the array itself (as it's usually - // regarded as a JSON), but will influence how TiKV handles the index value. - if colInfo.FieldType.IsArray() { - colInfo.SetCharset("binary") - colInfo.SetCollate("binary") - } - checkDependencies := make(map[string]struct{}) - for _, colName := range FindColumnNamesInExpr(idxPart.Expr) { - colInfo.Dependences[colName.Name.L] = struct{}{} - checkDependencies[colName.Name.L] = struct{}{} - } - if err = checkDependedColExist(checkDependencies, existCols); err != nil { - return nil, errors.Trace(err) - } - if !ctx.GetSessionVars().EnableAutoIncrementInGenerated { - if err = checkExpressionIndexAutoIncrement(indexName.O, colInfo.Dependences, tblInfo); err != nil { - return nil, errors.Trace(err) - } - } - idxPart.Expr = nil - hiddenCols = append(hiddenCols, colInfo) - } - return hiddenCols, 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, model.NewCIStr(stmt.IndexName), @@ -8285,26 +7013,6 @@ func (e *executor) AlterTableNoCache(ctx sessionctx.Context, ti ast.Ident) (err return e.DoDDLJob(ctx, job) } -// checkTooBigFieldLengthAndTryAutoConvert will check whether the field length is too big -// in non-strict mode and varchar column. If it is, will try to adjust to blob or text, see issue #30328 -func checkTooBigFieldLengthAndTryAutoConvert(tp *types.FieldType, colName string, sessVars *variable.SessionVars) error { - if sessVars != nil && !sessVars.SQLMode.HasStrictMode() && tp.GetType() == mysql.TypeVarchar { - err := types.IsVarcharTooBigFieldLength(tp.GetFlen(), colName, tp.GetCharset()) - if err != nil && terror.ErrorEqual(types.ErrTooBigFieldLength, err) { - tp.SetType(mysql.TypeBlob) - if err = adjustBlobTypesFlen(tp, tp.GetCharset()); err != nil { - return err - } - if tp.GetCharset() == charset.CharsetBin { - sessVars.StmtCtx.AppendWarning(dbterror.ErrAutoConvert.FastGenByArgs(colName, "VARBINARY", "BLOB")) - } else { - sessVars.StmtCtx.AppendWarning(dbterror.ErrAutoConvert.FastGenByArgs(colName, "VARCHAR", "TEXT")) - } - } - } - return nil -} - func (e *executor) CreateCheckConstraint(ctx sessionctx.Context, ti ast.Ident, constrName model.CIStr, constr *ast.Constraint) error { schema, t, err := e.getSchemaAndTableByIdent(ti) if err != nil { diff --git a/pkg/ddl/modify_column.go b/pkg/ddl/modify_column.go index 413616c779619..0a54f06bfa78e 100644 --- a/pkg/ddl/modify_column.go +++ b/pkg/ddl/modify_column.go @@ -607,6 +607,23 @@ func checkAutoRandom(tableInfo *model.TableInfo, originCol *table.Column, specNe return newShardBits, nil } +func isClusteredPKColumn(col *table.Column, tblInfo *model.TableInfo) bool { + switch { + case tblInfo.PKIsHandle: + return mysql.HasPriKeyFlag(col.GetFlag()) + case tblInfo.IsCommonHandle: + pk := tables.FindPrimaryIndex(tblInfo) + for _, c := range pk.Columns { + if c.Name.L == col.Name.L { + return true + } + } + return false + default: + return false + } +} + func rangeBitsIsChanged(oldBits, newBits uint64) bool { if oldBits == 0 { oldBits = autoid.AutoRandomRangeBitsDefault From f159291866669fc6a57066a4230a603c09876d1c Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Thu, 1 Aug 2024 17:34:46 +0800 Subject: [PATCH 04/13] change --- pkg/ddl/add_column.go | 10 ++ pkg/ddl/create_table.go | 206 ++++++++++++++++++++++++++++++++++++++ pkg/ddl/executor.go | 215 ---------------------------------------- 3 files changed, 216 insertions(+), 215 deletions(-) diff --git a/pkg/ddl/add_column.go b/pkg/ddl/add_column.go index 0413bb12fbf2e..6050d5f78c444 100644 --- a/pkg/ddl/add_column.go +++ b/pkg/ddl/add_column.go @@ -956,6 +956,16 @@ func checkDefaultValue(ctx exprctx.BuildContext, c *table.Column, hasDefaultValu return nil } +func checkColumnFieldLength(col *table.Column) error { + if col.GetType() == mysql.TypeVarchar { + if err := types.IsVarcharTooBigFieldLength(col.GetFlen(), col.Name.O, col.GetCharset()); err != nil { + return errors.Trace(err) + } + } + + return nil +} + // checkPriKeyConstraint check all parts of a PRIMARY KEY must be NOT NULL func checkPriKeyConstraint(col *table.Column, hasDefaultValue, hasNullFlag bool, outPriKeyConstraint *ast.Constraint) error { // Primary key should not be null. diff --git a/pkg/ddl/create_table.go b/pkg/ddl/create_table.go index 7b87859169b9a..51199e27cfb26 100644 --- a/pkg/ddl/create_table.go +++ b/pkg/ddl/create_table.go @@ -19,6 +19,7 @@ import ( "fmt" "math" "strings" + "sync/atomic" "unicode/utf8" "github.com/pingcap/errors" @@ -39,6 +40,7 @@ import ( driver "github.com/pingcap/tidb/pkg/types/parser_driver" "github.com/pingcap/tidb/pkg/util/dbterror" "github.com/pingcap/tidb/pkg/util/mock" + "github.com/pingcap/tidb/pkg/util/set" ) // BuildTableInfoFromAST builds model.TableInfo from a SQL statement. @@ -104,6 +106,59 @@ func checkTableInfoValidWithStmt(ctx sessionctx.Context, tbInfo *model.TableInfo return nil } +func checkGeneratedColumn(ctx sessionctx.Context, schemaName model.CIStr, tableName model.CIStr, colDefs []*ast.ColumnDef) error { + var colName2Generation = make(map[string]columnGenerationInDDL, len(colDefs)) + var exists bool + var autoIncrementColumn string + for i, colDef := range colDefs { + for _, option := range colDef.Options { + if option.Tp == ast.ColumnOptionGenerated { + if err := checkIllegalFn4Generated(colDef.Name.Name.L, typeColumn, option.Expr); err != nil { + return errors.Trace(err) + } + } + } + if containsColumnOption(colDef, ast.ColumnOptionAutoIncrement) { + exists, autoIncrementColumn = true, colDef.Name.Name.L + } + generated, depCols, err := findDependedColumnNames(schemaName, tableName, colDef) + if err != nil { + return errors.Trace(err) + } + if !generated { + colName2Generation[colDef.Name.Name.L] = columnGenerationInDDL{ + position: i, + generated: false, + } + } else { + colName2Generation[colDef.Name.Name.L] = columnGenerationInDDL{ + position: i, + generated: true, + dependences: depCols, + } + } + } + + // Check whether the generated column refers to any auto-increment columns + if exists { + if !ctx.GetSessionVars().EnableAutoIncrementInGenerated { + for colName, generated := range colName2Generation { + if _, found := generated.dependences[autoIncrementColumn]; found { + return dbterror.ErrGeneratedColumnRefAutoInc.GenWithStackByArgs(colName) + } + } + } + } + + for _, colDef := range colDefs { + colName := colDef.Name.Name.L + if err := verifyColumnGeneration(colName2Generation, colName); err != nil { + return errors.Trace(err) + } + } + return nil +} + // checkTableInfoValidExtra is like checkTableInfoValid, but also assumes the // table info comes from untrusted source and performs further checks such as // name length and column count. @@ -142,6 +197,66 @@ func checkTableInfoValidExtra(tbInfo *model.TableInfo) error { return err } +func checkDuplicateColumn(cols []*model.ColumnInfo) error { + colNames := set.StringSet{} + for _, col := range cols { + colName := col.Name + if colNames.Exist(colName.L) { + return infoschema.ErrColumnExists.GenWithStackByArgs(colName.O) + } + colNames.Insert(colName.L) + } + return nil +} + +func checkTooLongColumns(cols []*model.ColumnInfo) error { + for _, col := range cols { + if err := checkTooLongColumn(col.Name); err != nil { + return err + } + } + return nil +} + +func checkTooManyColumns(colDefs []*model.ColumnInfo) error { + if uint32(len(colDefs)) > atomic.LoadUint32(&config.GetGlobalConfig().TableColumnCountLimit) { + return dbterror.ErrTooManyFields + } + return nil +} + +func checkTooManyIndexes(idxDefs []*model.IndexInfo) error { + if len(idxDefs) > config.GetGlobalConfig().IndexLimit { + return dbterror.ErrTooManyKeys.GenWithStackByArgs(config.GetGlobalConfig().IndexLimit) + } + return nil +} + +// checkColumnsAttributes checks attributes for multiple columns. +func checkColumnsAttributes(colDefs []*model.ColumnInfo) error { + for _, colDef := range colDefs { + if err := checkColumnAttributes(colDef.Name.O, &colDef.FieldType); err != nil { + return errors.Trace(err) + } + } + return nil +} + +// checkColumnAttributes check attributes for single column. +func checkColumnAttributes(colName string, tp *types.FieldType) error { + switch tp.GetType() { + case mysql.TypeNewDecimal, mysql.TypeDouble, mysql.TypeFloat: + if tp.GetFlen() < tp.GetDecimal() { + return types.ErrMBiggerThanD.GenWithStackByArgs(colName) + } + case mysql.TypeDatetime, mysql.TypeDuration, mysql.TypeTimestamp: + if tp.GetDecimal() != types.UnspecifiedFsp && (tp.GetDecimal() < types.MinFsp || tp.GetDecimal() > types.MaxFsp) { + return types.ErrTooBigPrecision.GenWithStackByArgs(tp.GetDecimal(), colName, types.MaxFsp) + } + } + return nil +} + // BuildSessionTemporaryTableInfo builds model.TableInfo from a SQL statement. func BuildSessionTemporaryTableInfo(ctx sessionctx.Context, is infoschema.InfoSchema, s *ast.CreateTableStmt, dbCharset, dbCollate string, placementPolicyRef *model.PolicyRefInfo) (*model.TableInfo, error) { ident := ast.Ident{Schema: s.Table.Schema, Name: s.Table.Name} @@ -278,6 +393,15 @@ func setTableAutoRandomBits(ctx sessionctx.Context, tbInfo *model.TableInfo, col return nil } +func containsColumnOption(colDef *ast.ColumnDef, opTp ast.ColumnOptionType) bool { + for _, option := range colDef.Options { + if option.Tp == opTp { + return true + } + } + return false +} + func extractAutoRandomBitsFromColDef(colDef *ast.ColumnDef) (shardBits, rangeBits uint64, err error) { for _, op := range colDef.Options { if op.Tp == ast.ColumnOptionAutoRandom { @@ -439,6 +563,88 @@ func buildColumnsAndConstraints( return cols, constraints, nil } +func setEmptyConstraintName(namesMap map[string]bool, constr *ast.Constraint) { + if constr.Name == "" && len(constr.Keys) > 0 { + var colName string + for _, keyPart := range constr.Keys { + if keyPart.Expr != nil { + colName = "expression_index" + } + } + if colName == "" { + colName = constr.Keys[0].Column.Name.O + } + constrName := colName + i := 2 + if strings.EqualFold(constrName, mysql.PrimaryKeyName) { + constrName = fmt.Sprintf("%s_%d", constrName, 2) + i = 3 + } + for namesMap[constrName] { + // We loop forever until we find constrName that haven't been used. + constrName = fmt.Sprintf("%s_%d", colName, i) + i++ + } + constr.Name = constrName + namesMap[constrName] = true + } +} + +func checkConstraintNames(tableName model.CIStr, constraints []*ast.Constraint) error { + constrNames := map[string]bool{} + fkNames := map[string]bool{} + + // Check not empty constraint name whether is duplicated. + for _, constr := range constraints { + if constr.Tp == ast.ConstraintForeignKey { + err := checkDuplicateConstraint(fkNames, constr.Name, constr.Tp) + if err != nil { + return errors.Trace(err) + } + } else { + err := checkDuplicateConstraint(constrNames, constr.Name, constr.Tp) + if err != nil { + return errors.Trace(err) + } + } + } + + // Set empty constraint names. + checkConstraints := make([]*ast.Constraint, 0, len(constraints)) + for _, constr := range constraints { + if constr.Tp != ast.ConstraintForeignKey { + setEmptyConstraintName(constrNames, constr) + } + if constr.Tp == ast.ConstraintCheck { + checkConstraints = append(checkConstraints, constr) + } + } + // Set check constraint name under its order. + if len(checkConstraints) > 0 { + setEmptyCheckConstraintName(tableName.L, constrNames, checkConstraints) + } + return nil +} + +func checkDuplicateConstraint(namesMap map[string]bool, name string, constraintType ast.ConstraintType) error { + if name == "" { + return nil + } + nameLower := strings.ToLower(name) + if namesMap[nameLower] { + switch constraintType { + case ast.ConstraintForeignKey: + return dbterror.ErrFkDupName.GenWithStackByArgs(name) + case ast.ConstraintCheck: + return dbterror.ErrCheckConstraintDupName.GenWithStackByArgs(name) + default: + return dbterror.ErrDupKeyName.GenWithStackByArgs(name) + } + } + namesMap[nameLower] = true + return nil +} + func setColumnFlagWithConstraint(colMap map[string]*table.Column, v *ast.Constraint) { switch v.Tp { case ast.ConstraintPrimaryKey: diff --git a/pkg/ddl/executor.go b/pkg/ddl/executor.go index 4227dc9702dbe..0d3ea6d47bbdc 100644 --- a/pkg/ddl/executor.go +++ b/pkg/ddl/executor.go @@ -76,7 +76,6 @@ import ( "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/set" "github.com/pingcap/tidb/pkg/util/sqlkiller" "github.com/pingcap/tidb/pkg/util/stringutil" tikv "github.com/tikv/client-go/v2/kv" @@ -944,27 +943,6 @@ func ResolveCharsetCollation(sessVars *variable.SessionVars, charsetOpts ...ast. return chs, coll, nil } -func checkDuplicateColumn(cols []*model.ColumnInfo) error { - colNames := set.StringSet{} - for _, col := range cols { - colName := col.Name - if colNames.Exist(colName.L) { - return infoschema.ErrColumnExists.GenWithStackByArgs(colName.O) - } - colNames.Insert(colName.L) - } - return nil -} - -func containsColumnOption(colDef *ast.ColumnDef, opTp ast.ColumnOptionType) bool { - for _, option := range colDef.Options { - if option.Tp == opTp { - return true - } - } - return false -} - // 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() { @@ -983,136 +961,6 @@ func IsAutoRandomColumnID(tblInfo *model.TableInfo, colID int64) bool { return false } -func checkGeneratedColumn(ctx sessionctx.Context, schemaName model.CIStr, tableName model.CIStr, colDefs []*ast.ColumnDef) error { - var colName2Generation = make(map[string]columnGenerationInDDL, len(colDefs)) - var exists bool - var autoIncrementColumn string - for i, colDef := range colDefs { - for _, option := range colDef.Options { - if option.Tp == ast.ColumnOptionGenerated { - if err := checkIllegalFn4Generated(colDef.Name.Name.L, typeColumn, option.Expr); err != nil { - return errors.Trace(err) - } - } - } - if containsColumnOption(colDef, ast.ColumnOptionAutoIncrement) { - exists, autoIncrementColumn = true, colDef.Name.Name.L - } - generated, depCols, err := findDependedColumnNames(schemaName, tableName, colDef) - if err != nil { - return errors.Trace(err) - } - if !generated { - colName2Generation[colDef.Name.Name.L] = columnGenerationInDDL{ - position: i, - generated: false, - } - } else { - colName2Generation[colDef.Name.Name.L] = columnGenerationInDDL{ - position: i, - generated: true, - dependences: depCols, - } - } - } - - // Check whether the generated column refers to any auto-increment columns - if exists { - if !ctx.GetSessionVars().EnableAutoIncrementInGenerated { - for colName, generated := range colName2Generation { - if _, found := generated.dependences[autoIncrementColumn]; found { - return dbterror.ErrGeneratedColumnRefAutoInc.GenWithStackByArgs(colName) - } - } - } - } - - for _, colDef := range colDefs { - colName := colDef.Name.Name.L - if err := verifyColumnGeneration(colName2Generation, colName); err != nil { - return errors.Trace(err) - } - } - return nil -} - -func checkTooLongColumns(cols []*model.ColumnInfo) error { - for _, col := range cols { - if err := checkTooLongColumn(col.Name); err != nil { - return err - } - } - return nil -} - -func checkTooManyColumns(colDefs []*model.ColumnInfo) error { - if uint32(len(colDefs)) > atomic.LoadUint32(&config.GetGlobalConfig().TableColumnCountLimit) { - return dbterror.ErrTooManyFields - } - return nil -} - -func checkTooManyIndexes(idxDefs []*model.IndexInfo) error { - if len(idxDefs) > config.GetGlobalConfig().IndexLimit { - return dbterror.ErrTooManyKeys.GenWithStackByArgs(config.GetGlobalConfig().IndexLimit) - } - return nil -} - -// checkColumnsAttributes checks attributes for multiple columns. -func checkColumnsAttributes(colDefs []*model.ColumnInfo) error { - for _, colDef := range colDefs { - if err := checkColumnAttributes(colDef.Name.O, &colDef.FieldType); err != nil { - return errors.Trace(err) - } - } - return nil -} - -func checkColumnFieldLength(col *table.Column) error { - if col.GetType() == mysql.TypeVarchar { - if err := types.IsVarcharTooBigFieldLength(col.GetFlen(), col.Name.O, col.GetCharset()); err != nil { - return errors.Trace(err) - } - } - - return nil -} - -// checkColumnAttributes check attributes for single column. -func checkColumnAttributes(colName string, tp *types.FieldType) error { - switch tp.GetType() { - case mysql.TypeNewDecimal, mysql.TypeDouble, mysql.TypeFloat: - if tp.GetFlen() < tp.GetDecimal() { - return types.ErrMBiggerThanD.GenWithStackByArgs(colName) - } - case mysql.TypeDatetime, mysql.TypeDuration, mysql.TypeTimestamp: - if tp.GetDecimal() != types.UnspecifiedFsp && (tp.GetDecimal() < types.MinFsp || tp.GetDecimal() > types.MaxFsp) { - return types.ErrTooBigPrecision.GenWithStackByArgs(tp.GetDecimal(), colName, types.MaxFsp) - } - } - return nil -} - -func checkDuplicateConstraint(namesMap map[string]bool, name string, constraintType ast.ConstraintType) error { - if name == "" { - return nil - } - nameLower := strings.ToLower(name) - if namesMap[nameLower] { - switch constraintType { - case ast.ConstraintForeignKey: - return dbterror.ErrFkDupName.GenWithStackByArgs(name) - case ast.ConstraintCheck: - return dbterror.ErrCheckConstraintDupName.GenWithStackByArgs(name) - default: - return dbterror.ErrDupKeyName.GenWithStackByArgs(name) - } - } - namesMap[nameLower] = true - return nil -} - func setEmptyCheckConstraintName(tableLowerName string, namesMap map[string]bool, constrs []*ast.Constraint) { cnt := 1 constraintPrefix := tableLowerName + "_chk_" @@ -1133,69 +981,6 @@ func setEmptyCheckConstraintName(tableLowerName string, namesMap map[string]bool } } -func setEmptyConstraintName(namesMap map[string]bool, constr *ast.Constraint) { - if constr.Name == "" && len(constr.Keys) > 0 { - var colName string - for _, keyPart := range constr.Keys { - if keyPart.Expr != nil { - colName = "expression_index" - } - } - if colName == "" { - colName = constr.Keys[0].Column.Name.O - } - constrName := colName - i := 2 - if strings.EqualFold(constrName, mysql.PrimaryKeyName) { - constrName = fmt.Sprintf("%s_%d", constrName, 2) - i = 3 - } - for namesMap[constrName] { - // We loop forever until we find constrName that haven't been used. - constrName = fmt.Sprintf("%s_%d", colName, i) - i++ - } - constr.Name = constrName - namesMap[constrName] = true - } -} - -func checkConstraintNames(tableName model.CIStr, constraints []*ast.Constraint) error { - constrNames := map[string]bool{} - fkNames := map[string]bool{} - - // Check not empty constraint name whether is duplicated. - for _, constr := range constraints { - if constr.Tp == ast.ConstraintForeignKey { - err := checkDuplicateConstraint(fkNames, constr.Name, constr.Tp) - if err != nil { - return errors.Trace(err) - } - } else { - err := checkDuplicateConstraint(constrNames, constr.Name, constr.Tp) - if err != nil { - return errors.Trace(err) - } - } - } - - // Set empty constraint names. - checkConstraints := make([]*ast.Constraint, 0, len(constraints)) - for _, constr := range constraints { - if constr.Tp != ast.ConstraintForeignKey { - setEmptyConstraintName(constrNames, constr) - } - if constr.Tp == ast.ConstraintCheck { - checkConstraints = append(checkConstraints, constr) - } - } - // Set check constraint name under its order. - if len(checkConstraints) > 0 { - setEmptyCheckConstraintName(tableName.L, constrNames, checkConstraints) - } - return nil -} - // 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, From 596478ff866b8935230d67b3fb28c29092c7865c Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Thu, 1 Aug 2024 17:49:35 +0800 Subject: [PATCH 05/13] split --- pkg/ddl/executor.go | 297 +----------------------------------- pkg/ddl/placement_policy.go | 142 +++++++++++++++++ pkg/ddl/resource_group.go | 157 +++++++++++++++++++ 3 files changed, 301 insertions(+), 295 deletions(-) diff --git a/pkg/ddl/executor.go b/pkg/ddl/executor.go index 0d3ea6d47bbdc..6c1f5e9a3eb06 100644 --- a/pkg/ddl/executor.go +++ b/pkg/ddl/executor.go @@ -23,7 +23,6 @@ import ( "context" "fmt" "math" - "slices" "strconv" "strings" "sync" @@ -80,7 +79,6 @@ import ( "github.com/pingcap/tidb/pkg/util/stringutil" tikv "github.com/tikv/client-go/v2/kv" "github.com/tikv/client-go/v2/oracle" - kvutil "github.com/tikv/client-go/v2/util" clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" ) @@ -669,25 +667,6 @@ func (e *executor) AlterTablePlacement(ctx sessionctx.Context, ident ast.Ident, return errors.Trace(err) } -func checkAndNormalizePlacementPolicy(ctx sessionctx.Context, placementPolicyRef *model.PolicyRefInfo) (*model.PolicyRefInfo, error) { - if placementPolicyRef == nil { - return nil, nil - } - - if placementPolicyRef.Name.L == defaultPlacementPolicyName { - // When policy name is 'default', it means to remove the placement settings - return nil, nil - } - - policy, ok := sessiontxn.GetTxnManager(ctx).GetTxnInfoSchema().PolicyByName(placementPolicyRef.Name) - if !ok { - return nil, errors.Trace(infoschema.ErrPlacementPolicyNotExists.GenWithStackByArgs(placementPolicyRef.Name)) - } - - placementPolicyRef.ID = policy.ID - return placementPolicyRef, nil -} - func checkMultiSchemaSpecs(_ sessionctx.Context, specs []*ast.DatabaseOption) error { hasSetTiFlashReplica := false if len(specs) == 1 { @@ -1895,139 +1874,6 @@ func SetDirectPlacementOpt(placementSettings *model.PlacementSettings, placement return nil } -// SetDirectResourceGroupSettings tries to set the ResourceGroupSettings. -func SetDirectResourceGroupSettings(groupInfo *model.ResourceGroupInfo, opt *ast.ResourceGroupOption) error { - resourceGroupSettings := groupInfo.ResourceGroupSettings - switch opt.Tp { - case ast.ResourceRURate: - return SetDirectResourceGroupRUSecondOption(resourceGroupSettings, opt.UintValue, opt.BoolValue) - case ast.ResourcePriority: - resourceGroupSettings.Priority = opt.UintValue - case ast.ResourceUnitCPU: - resourceGroupSettings.CPULimiter = opt.StrValue - case ast.ResourceUnitIOReadBandwidth: - resourceGroupSettings.IOReadBandwidth = opt.StrValue - case ast.ResourceUnitIOWriteBandwidth: - resourceGroupSettings.IOWriteBandwidth = opt.StrValue - case ast.ResourceBurstableOpiton: - // Some about BurstLimit(b): - // - If b == 0, that means the limiter is unlimited capacity. default use in resource controller (burst with a rate within a unlimited capacity). - // - If b < 0, that means the limiter is unlimited capacity and fillrate(r) is ignored, can be seen as r == Inf (burst with a inf rate within a unlimited capacity). - // - If b > 0, that means the limiter is limited capacity. (current not used). - limit := int64(0) - if opt.BoolValue { - limit = -1 - } - resourceGroupSettings.BurstLimit = limit - case ast.ResourceGroupRunaway: - if len(opt.RunawayOptionList) == 0 { - resourceGroupSettings.Runaway = nil - } - for _, opt := range opt.RunawayOptionList { - if err := SetDirectResourceGroupRunawayOption(resourceGroupSettings, opt); err != nil { - return err - } - } - case ast.ResourceGroupBackground: - if groupInfo.Name.L != rg.DefaultResourceGroupName { - // FIXME: this is a temporary restriction, so we don't add a error-code for it. - return errors.New("unsupported operation. Currently, only the default resource group support change background settings") - } - if len(opt.BackgroundOptions) == 0 { - resourceGroupSettings.Background = nil - } - for _, opt := range opt.BackgroundOptions { - if err := SetDirectResourceGroupBackgroundOption(resourceGroupSettings, opt); err != nil { - return err - } - } - default: - return errors.Trace(errors.New("unknown resource unit type")) - } - return nil -} - -// SetDirectResourceGroupRUSecondOption tries to set ru second part of the ResourceGroupSettings. -func SetDirectResourceGroupRUSecondOption(resourceGroupSettings *model.ResourceGroupSettings, intVal uint64, unlimited bool) error { - if unlimited { - resourceGroupSettings.RURate = uint64(math.MaxInt32) - resourceGroupSettings.BurstLimit = -1 - } else { - resourceGroupSettings.RURate = intVal - } - return nil -} - -// SetDirectResourceGroupRunawayOption tries to set runaway part of the ResourceGroupSettings. -func SetDirectResourceGroupRunawayOption(resourceGroupSettings *model.ResourceGroupSettings, opt *ast.ResourceGroupRunawayOption) error { - if resourceGroupSettings.Runaway == nil { - resourceGroupSettings.Runaway = &model.ResourceGroupRunawaySettings{} - } - settings := resourceGroupSettings.Runaway - switch opt.Tp { - case ast.RunawayRule: - // because execute time won't be too long, we use `time` pkg which does not support to parse unit 'd'. - dur, err := time.ParseDuration(opt.RuleOption.ExecElapsed) - if err != nil { - return err - } - settings.ExecElapsedTimeMs = uint64(dur.Milliseconds()) - case ast.RunawayAction: - settings.Action = opt.ActionOption.Type - case ast.RunawayWatch: - settings.WatchType = opt.WatchOption.Type - if dur := opt.WatchOption.Duration; len(dur) > 0 { - dur, err := time.ParseDuration(dur) - if err != nil { - return err - } - settings.WatchDurationMs = dur.Milliseconds() - } else { - settings.WatchDurationMs = 0 - } - default: - return errors.Trace(errors.New("unknown runaway option type")) - } - return nil -} - -// SetDirectResourceGroupBackgroundOption set background configs of the ResourceGroupSettings. -func SetDirectResourceGroupBackgroundOption(resourceGroupSettings *model.ResourceGroupSettings, opt *ast.ResourceGroupBackgroundOption) error { - if resourceGroupSettings.Background == nil { - resourceGroupSettings.Background = &model.ResourceGroupBackgroundSettings{} - } - switch opt.Type { - case ast.BackgroundOptionTaskNames: - jobTypes, err := parseBackgroundJobTypes(opt.StrValue) - if err != nil { - return err - } - resourceGroupSettings.Background.JobTypes = jobTypes - default: - return errors.Trace(errors.New("unknown background option type")) - } - return nil -} - -func parseBackgroundJobTypes(t string) ([]string, error) { - if len(t) == 0 { - return []string{}, nil - } - - segs := strings.Split(t, ",") - res := make([]string, 0, len(segs)) - for _, s := range segs { - ty := strings.ToLower(strings.TrimSpace(s)) - if len(ty) > 0 { - if !slices.Contains(kvutil.ExplicitTypeList, ty) { - return nil, infoschema.ErrResourceGroupInvalidBackgroundTaskName.GenWithStackByArgs(ty) - } - res = append(res, ty) - } - } - return res, nil -} - func shardingBits(tblInfo *model.TableInfo) uint64 { if tblInfo.ShardRowIDBits > 0 { return tblInfo.ShardRowIDBits @@ -6310,125 +6156,6 @@ func (e *executor) AlterTablePartitionPlacement(ctx sessionctx.Context, tableIde return errors.Trace(err) } -func buildPolicyInfo(name model.CIStr, options []*ast.PlacementOption) (*model.PolicyInfo, error) { - policyInfo := &model.PolicyInfo{PlacementSettings: &model.PlacementSettings{}} - policyInfo.Name = name - for _, opt := range options { - err := SetDirectPlacementOpt(policyInfo.PlacementSettings, opt.Tp, opt.StrValue, opt.UintValue) - if err != nil { - return nil, err - } - } - return policyInfo, nil -} - -func removeTablePlacement(tbInfo *model.TableInfo) bool { - hasPlacementSettings := false - if tbInfo.PlacementPolicyRef != nil { - tbInfo.PlacementPolicyRef = nil - hasPlacementSettings = true - } - - if removePartitionPlacement(tbInfo.Partition) { - hasPlacementSettings = true - } - - return hasPlacementSettings -} - -func removePartitionPlacement(partInfo *model.PartitionInfo) bool { - if partInfo == nil { - return false - } - - hasPlacementSettings := false - for i := range partInfo.Definitions { - def := &partInfo.Definitions[i] - if def.PlacementPolicyRef != nil { - def.PlacementPolicyRef = nil - hasPlacementSettings = true - } - } - return hasPlacementSettings -} - -func handleDatabasePlacement(ctx sessionctx.Context, dbInfo *model.DBInfo) error { - if dbInfo.PlacementPolicyRef == nil { - return nil - } - - sessVars := ctx.GetSessionVars() - if sessVars.PlacementMode == variable.PlacementModeIgnore { - dbInfo.PlacementPolicyRef = nil - sessVars.StmtCtx.AppendNote( - errors.NewNoStackErrorf("Placement is ignored when TIDB_PLACEMENT_MODE is '%s'", variable.PlacementModeIgnore), - ) - return nil - } - - var err error - dbInfo.PlacementPolicyRef, err = checkAndNormalizePlacementPolicy(ctx, dbInfo.PlacementPolicyRef) - return err -} - -func handleTablePlacement(ctx sessionctx.Context, tbInfo *model.TableInfo) error { - sessVars := ctx.GetSessionVars() - if sessVars.PlacementMode == variable.PlacementModeIgnore && removeTablePlacement(tbInfo) { - sessVars.StmtCtx.AppendNote( - errors.NewNoStackErrorf("Placement is ignored when TIDB_PLACEMENT_MODE is '%s'", variable.PlacementModeIgnore), - ) - return nil - } - - var err error - tbInfo.PlacementPolicyRef, err = checkAndNormalizePlacementPolicy(ctx, tbInfo.PlacementPolicyRef) - if err != nil { - return err - } - - if tbInfo.Partition != nil { - for i := range tbInfo.Partition.Definitions { - partition := &tbInfo.Partition.Definitions[i] - partition.PlacementPolicyRef, err = checkAndNormalizePlacementPolicy(ctx, partition.PlacementPolicyRef) - if err != nil { - return err - } - } - } - return nil -} - -func handlePartitionPlacement(ctx sessionctx.Context, partInfo *model.PartitionInfo) error { - sessVars := ctx.GetSessionVars() - if sessVars.PlacementMode == variable.PlacementModeIgnore && removePartitionPlacement(partInfo) { - sessVars.StmtCtx.AppendNote( - errors.NewNoStackErrorf("Placement is ignored when TIDB_PLACEMENT_MODE is '%s'", variable.PlacementModeIgnore), - ) - return nil - } - - var err error - for i := range partInfo.Definitions { - partition := &partInfo.Definitions[i] - partition.PlacementPolicyRef, err = checkAndNormalizePlacementPolicy(ctx, partition.PlacementPolicyRef) - if err != nil { - return err - } - } - return nil -} - -func checkIgnorePlacementDDL(ctx sessionctx.Context) bool { - sessVars := ctx.GetSessionVars() - if sessVars.PlacementMode == variable.PlacementModeIgnore { - sessVars.StmtCtx.AppendNote( - errors.NewNoStackErrorf("Placement is ignored when TIDB_PLACEMENT_MODE is '%s'", variable.PlacementModeIgnore), - ) - return true - } - return false -} - // AddResourceGroup implements the DDL interface, creates a resource group. func (e *executor) AddResourceGroup(ctx sessionctx.Context, stmt *ast.CreateResourceGroupStmt) (err error) { groupName := stmt.ResourceGroupName @@ -6447,7 +6174,7 @@ func (e *executor) AddResourceGroup(ctx sessionctx.Context, stmt *ast.CreateReso return infoschema.ErrResourceGroupExists.GenWithStackByArgs(groupName) } - if err := e.checkResourceGroupValidation(groupInfo); err != nil { + if err := checkResourceGroupValidation(groupInfo); err != nil { return err } @@ -6473,11 +6200,6 @@ func (e *executor) AddResourceGroup(ctx sessionctx.Context, stmt *ast.CreateReso return err } -func (*executor) checkResourceGroupValidation(groupInfo *model.ResourceGroupInfo) error { - _, err := resourcegroup.NewGroupFromOptions(groupInfo.Name.L, groupInfo.ResourceGroupSettings) - return err -} - // DropResourceGroup implements the DDL interface. func (e *executor) DropResourceGroup(ctx sessionctx.Context, stmt *ast.DropResourceGroupStmt) (err error) { groupName := stmt.ResourceGroupName @@ -6523,21 +6245,6 @@ func (e *executor) DropResourceGroup(ctx sessionctx.Context, stmt *ast.DropResou return err } -func buildResourceGroup(oldGroup *model.ResourceGroupInfo, options []*ast.ResourceGroupOption) (*model.ResourceGroupInfo, error) { - groupInfo := &model.ResourceGroupInfo{Name: oldGroup.Name, ID: oldGroup.ID, ResourceGroupSettings: model.NewResourceGroupSettings()} - if oldGroup.ResourceGroupSettings != nil { - *groupInfo.ResourceGroupSettings = *oldGroup.ResourceGroupSettings - } - for _, opt := range options { - err := SetDirectResourceGroupSettings(groupInfo, opt) - if err != nil { - return nil, err - } - } - groupInfo.ResourceGroupSettings.Adjust() - return groupInfo, nil -} - // AlterResourceGroup implements the DDL interface. func (e *executor) AlterResourceGroup(ctx sessionctx.Context, stmt *ast.AlterResourceGroupStmt) (err error) { groupName := stmt.ResourceGroupName @@ -6557,7 +6264,7 @@ func (e *executor) AlterResourceGroup(ctx sessionctx.Context, stmt *ast.AlterRes return errors.Trace(err) } - if err := e.checkResourceGroupValidation(newGroupInfo); err != nil { + if err := checkResourceGroupValidation(newGroupInfo); err != nil { return err } diff --git a/pkg/ddl/placement_policy.go b/pkg/ddl/placement_policy.go index 2277ab3199fe2..49d42806fd983 100644 --- a/pkg/ddl/placement_policy.go +++ b/pkg/ddl/placement_policy.go @@ -24,7 +24,11 @@ import ( "github.com/pingcap/tidb/pkg/domain/infosync" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/util/dbterror" ) @@ -489,3 +493,141 @@ func GetRangePlacementPolicyName(ctx context.Context, rangeBundleID string) (str } return "", nil } + +func buildPolicyInfo(name model.CIStr, options []*ast.PlacementOption) (*model.PolicyInfo, error) { + policyInfo := &model.PolicyInfo{PlacementSettings: &model.PlacementSettings{}} + policyInfo.Name = name + for _, opt := range options { + err := SetDirectPlacementOpt(policyInfo.PlacementSettings, opt.Tp, opt.StrValue, opt.UintValue) + if err != nil { + return nil, err + } + } + return policyInfo, nil +} + +func removeTablePlacement(tbInfo *model.TableInfo) bool { + hasPlacementSettings := false + if tbInfo.PlacementPolicyRef != nil { + tbInfo.PlacementPolicyRef = nil + hasPlacementSettings = true + } + + if removePartitionPlacement(tbInfo.Partition) { + hasPlacementSettings = true + } + + return hasPlacementSettings +} + +func removePartitionPlacement(partInfo *model.PartitionInfo) bool { + if partInfo == nil { + return false + } + + hasPlacementSettings := false + for i := range partInfo.Definitions { + def := &partInfo.Definitions[i] + if def.PlacementPolicyRef != nil { + def.PlacementPolicyRef = nil + hasPlacementSettings = true + } + } + return hasPlacementSettings +} + +func handleDatabasePlacement(ctx sessionctx.Context, dbInfo *model.DBInfo) error { + if dbInfo.PlacementPolicyRef == nil { + return nil + } + + sessVars := ctx.GetSessionVars() + if sessVars.PlacementMode == variable.PlacementModeIgnore { + dbInfo.PlacementPolicyRef = nil + sessVars.StmtCtx.AppendNote( + errors.NewNoStackErrorf("Placement is ignored when TIDB_PLACEMENT_MODE is '%s'", variable.PlacementModeIgnore), + ) + return nil + } + + var err error + dbInfo.PlacementPolicyRef, err = checkAndNormalizePlacementPolicy(ctx, dbInfo.PlacementPolicyRef) + return err +} + +func handleTablePlacement(ctx sessionctx.Context, tbInfo *model.TableInfo) error { + sessVars := ctx.GetSessionVars() + if sessVars.PlacementMode == variable.PlacementModeIgnore && removeTablePlacement(tbInfo) { + sessVars.StmtCtx.AppendNote( + errors.NewNoStackErrorf("Placement is ignored when TIDB_PLACEMENT_MODE is '%s'", variable.PlacementModeIgnore), + ) + return nil + } + + var err error + tbInfo.PlacementPolicyRef, err = checkAndNormalizePlacementPolicy(ctx, tbInfo.PlacementPolicyRef) + if err != nil { + return err + } + + if tbInfo.Partition != nil { + for i := range tbInfo.Partition.Definitions { + partition := &tbInfo.Partition.Definitions[i] + partition.PlacementPolicyRef, err = checkAndNormalizePlacementPolicy(ctx, partition.PlacementPolicyRef) + if err != nil { + return err + } + } + } + return nil +} + +func handlePartitionPlacement(ctx sessionctx.Context, partInfo *model.PartitionInfo) error { + sessVars := ctx.GetSessionVars() + if sessVars.PlacementMode == variable.PlacementModeIgnore && removePartitionPlacement(partInfo) { + sessVars.StmtCtx.AppendNote( + errors.NewNoStackErrorf("Placement is ignored when TIDB_PLACEMENT_MODE is '%s'", variable.PlacementModeIgnore), + ) + return nil + } + + var err error + for i := range partInfo.Definitions { + partition := &partInfo.Definitions[i] + partition.PlacementPolicyRef, err = checkAndNormalizePlacementPolicy(ctx, partition.PlacementPolicyRef) + if err != nil { + return err + } + } + return nil +} + +func checkAndNormalizePlacementPolicy(ctx sessionctx.Context, placementPolicyRef *model.PolicyRefInfo) (*model.PolicyRefInfo, error) { + if placementPolicyRef == nil { + return nil, nil + } + + if placementPolicyRef.Name.L == defaultPlacementPolicyName { + // When policy name is 'default', it means to remove the placement settings + return nil, nil + } + + policy, ok := sessiontxn.GetTxnManager(ctx).GetTxnInfoSchema().PolicyByName(placementPolicyRef.Name) + if !ok { + return nil, errors.Trace(infoschema.ErrPlacementPolicyNotExists.GenWithStackByArgs(placementPolicyRef.Name)) + } + + placementPolicyRef.ID = policy.ID + return placementPolicyRef, nil +} + +func checkIgnorePlacementDDL(ctx sessionctx.Context) bool { + sessVars := ctx.GetSessionVars() + if sessVars.PlacementMode == variable.PlacementModeIgnore { + sessVars.StmtCtx.AppendNote( + errors.NewNoStackErrorf("Placement is ignored when TIDB_PLACEMENT_MODE is '%s'", variable.PlacementModeIgnore), + ) + return true + } + return false +} diff --git a/pkg/ddl/resource_group.go b/pkg/ddl/resource_group.go index 39265a46dee6a..fcf2bcabcde61 100644 --- a/pkg/ddl/resource_group.go +++ b/pkg/ddl/resource_group.go @@ -16,6 +16,8 @@ package ddl import ( "context" + "math" + "slices" "strings" "time" @@ -26,8 +28,10 @@ import ( rg "github.com/pingcap/tidb/pkg/domain/resourcegroup" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/util/dbterror" + kvutil "github.com/tikv/client-go/v2/util" "go.uber.org/zap" ) @@ -172,3 +176,156 @@ func onDropResourceGroup(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ } return ver, errors.Trace(err) } + +func buildResourceGroup(oldGroup *model.ResourceGroupInfo, options []*ast.ResourceGroupOption) (*model.ResourceGroupInfo, error) { + groupInfo := &model.ResourceGroupInfo{Name: oldGroup.Name, ID: oldGroup.ID, ResourceGroupSettings: model.NewResourceGroupSettings()} + if oldGroup.ResourceGroupSettings != nil { + *groupInfo.ResourceGroupSettings = *oldGroup.ResourceGroupSettings + } + for _, opt := range options { + err := SetDirectResourceGroupSettings(groupInfo, opt) + if err != nil { + return nil, err + } + } + groupInfo.ResourceGroupSettings.Adjust() + return groupInfo, nil +} + +// SetDirectResourceGroupSettings tries to set the ResourceGroupSettings. +func SetDirectResourceGroupSettings(groupInfo *model.ResourceGroupInfo, opt *ast.ResourceGroupOption) error { + resourceGroupSettings := groupInfo.ResourceGroupSettings + switch opt.Tp { + case ast.ResourceRURate: + return SetDirectResourceGroupRUSecondOption(resourceGroupSettings, opt.UintValue, opt.BoolValue) + case ast.ResourcePriority: + resourceGroupSettings.Priority = opt.UintValue + case ast.ResourceUnitCPU: + resourceGroupSettings.CPULimiter = opt.StrValue + case ast.ResourceUnitIOReadBandwidth: + resourceGroupSettings.IOReadBandwidth = opt.StrValue + case ast.ResourceUnitIOWriteBandwidth: + resourceGroupSettings.IOWriteBandwidth = opt.StrValue + case ast.ResourceBurstableOpiton: + // Some about BurstLimit(b): + // - If b == 0, that means the limiter is unlimited capacity. default use in resource controller (burst with a rate within a unlimited capacity). + // - If b < 0, that means the limiter is unlimited capacity and fillrate(r) is ignored, can be seen as r == Inf (burst with a inf rate within a unlimited capacity). + // - If b > 0, that means the limiter is limited capacity. (current not used). + limit := int64(0) + if opt.BoolValue { + limit = -1 + } + resourceGroupSettings.BurstLimit = limit + case ast.ResourceGroupRunaway: + if len(opt.RunawayOptionList) == 0 { + resourceGroupSettings.Runaway = nil + } + for _, opt := range opt.RunawayOptionList { + if err := SetDirectResourceGroupRunawayOption(resourceGroupSettings, opt); err != nil { + return err + } + } + case ast.ResourceGroupBackground: + if groupInfo.Name.L != rg.DefaultResourceGroupName { + // FIXME: this is a temporary restriction, so we don't add a error-code for it. + return errors.New("unsupported operation. Currently, only the default resource group support change background settings") + } + if len(opt.BackgroundOptions) == 0 { + resourceGroupSettings.Background = nil + } + for _, opt := range opt.BackgroundOptions { + if err := SetDirectResourceGroupBackgroundOption(resourceGroupSettings, opt); err != nil { + return err + } + } + default: + return errors.Trace(errors.New("unknown resource unit type")) + } + return nil +} + +// SetDirectResourceGroupRUSecondOption tries to set ru second part of the ResourceGroupSettings. +func SetDirectResourceGroupRUSecondOption(resourceGroupSettings *model.ResourceGroupSettings, intVal uint64, unlimited bool) error { + if unlimited { + resourceGroupSettings.RURate = uint64(math.MaxInt32) + resourceGroupSettings.BurstLimit = -1 + } else { + resourceGroupSettings.RURate = intVal + } + return nil +} + +// SetDirectResourceGroupRunawayOption tries to set runaway part of the ResourceGroupSettings. +func SetDirectResourceGroupRunawayOption(resourceGroupSettings *model.ResourceGroupSettings, opt *ast.ResourceGroupRunawayOption) error { + if resourceGroupSettings.Runaway == nil { + resourceGroupSettings.Runaway = &model.ResourceGroupRunawaySettings{} + } + settings := resourceGroupSettings.Runaway + switch opt.Tp { + case ast.RunawayRule: + // because execute time won't be too long, we use `time` pkg which does not support to parse unit 'd'. + dur, err := time.ParseDuration(opt.RuleOption.ExecElapsed) + if err != nil { + return err + } + settings.ExecElapsedTimeMs = uint64(dur.Milliseconds()) + case ast.RunawayAction: + settings.Action = opt.ActionOption.Type + case ast.RunawayWatch: + settings.WatchType = opt.WatchOption.Type + if dur := opt.WatchOption.Duration; len(dur) > 0 { + dur, err := time.ParseDuration(dur) + if err != nil { + return err + } + settings.WatchDurationMs = dur.Milliseconds() + } else { + settings.WatchDurationMs = 0 + } + default: + return errors.Trace(errors.New("unknown runaway option type")) + } + return nil +} + +// SetDirectResourceGroupBackgroundOption set background configs of the ResourceGroupSettings. +func SetDirectResourceGroupBackgroundOption(resourceGroupSettings *model.ResourceGroupSettings, opt *ast.ResourceGroupBackgroundOption) error { + if resourceGroupSettings.Background == nil { + resourceGroupSettings.Background = &model.ResourceGroupBackgroundSettings{} + } + switch opt.Type { + case ast.BackgroundOptionTaskNames: + jobTypes, err := parseBackgroundJobTypes(opt.StrValue) + if err != nil { + return err + } + resourceGroupSettings.Background.JobTypes = jobTypes + default: + return errors.Trace(errors.New("unknown background option type")) + } + return nil +} + +func parseBackgroundJobTypes(t string) ([]string, error) { + if len(t) == 0 { + return []string{}, nil + } + + segs := strings.Split(t, ",") + res := make([]string, 0, len(segs)) + for _, s := range segs { + ty := strings.ToLower(strings.TrimSpace(s)) + if len(ty) > 0 { + if !slices.Contains(kvutil.ExplicitTypeList, ty) { + return nil, infoschema.ErrResourceGroupInvalidBackgroundTaskName.GenWithStackByArgs(ty) + } + res = append(res, ty) + } + } + return res, nil +} + +func checkResourceGroupValidation(groupInfo *model.ResourceGroupInfo) error { + _, err := resourcegroup.NewGroupFromOptions(groupInfo.Name.L, groupInfo.ResourceGroupSettings) + return err +} From 81b8516432c25ba57ec789a2f1eeace3e55769b1 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Thu, 1 Aug 2024 18:09:43 +0800 Subject: [PATCH 06/13] split --- pkg/ddl/add_column.go | 143 ++++++++++++++ pkg/ddl/column.go | 94 --------- pkg/ddl/create_table.go | 374 ++++++++++++++++++++++++++++++++++++ pkg/ddl/executor.go | 101 ---------- pkg/ddl/placement_policy.go | 33 ++++ pkg/ddl/table.go | 345 --------------------------------- 6 files changed, 550 insertions(+), 540 deletions(-) diff --git a/pkg/ddl/add_column.go b/pkg/ddl/add_column.go index 6050d5f78c444..54f519b7731af 100644 --- a/pkg/ddl/add_column.go +++ b/pkg/ddl/add_column.go @@ -22,12 +22,15 @@ import ( "unicode/utf8" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/ddl/logutil" "github.com/pingcap/tidb/pkg/errctx" "github.com/pingcap/tidb/pkg/expression" exprctx "github.com/pingcap/tidb/pkg/expression/context" "github.com/pingcap/tidb/pkg/infoschema" + "github.com/pingcap/tidb/pkg/meta" + "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/format" @@ -37,6 +40,7 @@ import ( field_types "github.com/pingcap/tidb/pkg/parser/types" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" + statsutil "github.com/pingcap/tidb/pkg/statistics/handle/util" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/types" driver "github.com/pingcap/tidb/pkg/types/parser_driver" @@ -46,6 +50,100 @@ import ( "go.uber.org/zap" ) +func onAddColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { + // Handle the rolling back job. + if job.IsRollingback() { + ver, err = onDropColumn(d, t, job) + if err != nil { + return ver, errors.Trace(err) + } + return ver, nil + } + + failpoint.Inject("errorBeforeDecodeArgs", func(val failpoint.Value) { + //nolint:forcetypeassert + if val.(bool) { + failpoint.Return(ver, errors.New("occur an error before decode args")) + } + }) + + tblInfo, columnInfo, colFromArgs, pos, ifNotExists, err := checkAddColumn(t, job) + if err != nil { + if ifNotExists && infoschema.ErrColumnExists.Equal(err) { + job.Warning = toTError(err) + job.State = model.JobStateDone + return ver, nil + } + return ver, errors.Trace(err) + } + if columnInfo == nil { + columnInfo = InitAndAddColumnToTable(tblInfo, colFromArgs) + logutil.DDLLogger().Info("run add column job", zap.Stringer("job", job), zap.Reflect("columnInfo", *columnInfo)) + if err = checkAddColumnTooManyColumns(len(tblInfo.Columns)); err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + } + + originalState := columnInfo.State + switch columnInfo.State { + case model.StateNone: + // none -> delete only + columnInfo.State = model.StateDeleteOnly + ver, err = updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, originalState != columnInfo.State) + if err != nil { + return ver, errors.Trace(err) + } + job.SchemaState = model.StateDeleteOnly + case model.StateDeleteOnly: + // delete only -> write only + columnInfo.State = model.StateWriteOnly + ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, originalState != columnInfo.State) + if err != nil { + return ver, errors.Trace(err) + } + // Update the job state when all affairs done. + job.SchemaState = model.StateWriteOnly + case model.StateWriteOnly: + // write only -> reorganization + columnInfo.State = model.StateWriteReorganization + ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, originalState != columnInfo.State) + if err != nil { + return ver, errors.Trace(err) + } + // Update the job state when all affairs done. + job.SchemaState = model.StateWriteReorganization + job.MarkNonRevertible() + case model.StateWriteReorganization: + // reorganization -> public + // Adjust table column offset. + failpoint.InjectCall("onAddColumnStateWriteReorg") + offset, err := LocateOffsetToMove(columnInfo.Offset, pos, tblInfo) + if err != nil { + return ver, errors.Trace(err) + } + tblInfo.MoveColumnInfo(columnInfo.Offset, offset) + columnInfo.State = model.StatePublic + ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, originalState != columnInfo.State) + if err != nil { + return ver, errors.Trace(err) + } + + // Finish this job. + job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo) + addColumnEvent := statsutil.NewAddColumnEvent( + job.SchemaID, + tblInfo, + []*model.ColumnInfo{columnInfo}, + ) + asyncNotifyEvent(d, addColumnEvent) + default: + err = dbterror.ErrInvalidDDLState.GenWithStackByArgs("column", columnInfo.State) + } + + return ver, errors.Trace(err) +} + func checkAndCreateNewColumn(ctx sessionctx.Context, ti ast.Ident, schema *model.DBInfo, spec *ast.AlterTableSpec, t table.Table, specNewColumn *ast.ColumnDef) (*table.Column, error) { err := checkUnsupportedColumnConstraint(specNewColumn, ti) if err != nil { @@ -73,6 +171,24 @@ func checkAndCreateNewColumn(ctx sessionctx.Context, ti ast.Ident, schema *model return CreateNewColumn(ctx, schema, spec, t, specNewColumn) } +func checkUnsupportedColumnConstraint(col *ast.ColumnDef, ti ast.Ident) error { + for _, constraint := range col.Options { + switch constraint.Tp { + case ast.ColumnOptionAutoIncrement: + return dbterror.ErrUnsupportedAddColumn.GenWithStack("unsupported add column '%s' constraint AUTO_INCREMENT when altering '%s.%s'", col.Name, ti.Schema, ti.Name) + case ast.ColumnOptionPrimaryKey: + return dbterror.ErrUnsupportedAddColumn.GenWithStack("unsupported add column '%s' constraint PRIMARY KEY when altering '%s.%s'", col.Name, ti.Schema, ti.Name) + case ast.ColumnOptionUniqKey: + return dbterror.ErrUnsupportedAddColumn.GenWithStack("unsupported add column '%s' constraint UNIQUE KEY when altering '%s.%s'", col.Name, ti.Schema, ti.Name) + case ast.ColumnOptionAutoRandom: + errMsg := fmt.Sprintf(autoid.AutoRandomAlterAddColumn, col.Name, ti.Schema, ti.Name) + return dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(errMsg) + } + } + + return nil +} + // CreateNewColumn creates a new column according to the column information. func CreateNewColumn(ctx sessionctx.Context, schema *model.DBInfo, spec *ast.AlterTableSpec, t table.Table, specNewColumn *ast.ColumnDef) (*table.Column, error) { // If new column is a generated column, do validation. @@ -200,6 +316,33 @@ func buildColumnAndConstraint( return col, cts, nil } +// getCharsetAndCollateInColumnDef will iterate collate in the options, validate it by checking the charset +// of column definition. If there's no collate in the option, the default collate of column's charset will be used. +func getCharsetAndCollateInColumnDef(sessVars *variable.SessionVars, def *ast.ColumnDef) (chs, coll string, err error) { + chs = def.Tp.GetCharset() + coll = def.Tp.GetCollate() + if chs != "" && coll == "" { + if coll, err = GetDefaultCollation(sessVars, chs); err != nil { + return "", "", errors.Trace(err) + } + } + for _, opt := range def.Options { + if opt.Tp == ast.ColumnOptionCollate { + info, err := collate.GetCollationByName(opt.StrValue) + if err != nil { + return "", "", errors.Trace(err) + } + if chs == "" { + chs = info.CharsetName + } else if chs != info.CharsetName { + return "", "", dbterror.ErrCollationCharsetMismatch.GenWithStackByArgs(info.Name, chs) + } + coll = info.Name + } + } + return +} + // OverwriteCollationWithBinaryFlag is used to handle the case like // // CREATE TABLE t (a VARCHAR(255) BINARY) CHARSET utf8 COLLATE utf8_general_ci; diff --git a/pkg/ddl/column.go b/pkg/ddl/column.go index 713af458c4328..f58e784cf6c64 100644 --- a/pkg/ddl/column.go +++ b/pkg/ddl/column.go @@ -104,100 +104,6 @@ func checkAddColumn(t *meta.Meta, job *model.Job) (*model.TableInfo, *model.Colu return tblInfo, columnInfo, col, pos, false, nil } -func onAddColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { - // Handle the rolling back job. - if job.IsRollingback() { - ver, err = onDropColumn(d, t, job) - if err != nil { - return ver, errors.Trace(err) - } - return ver, nil - } - - failpoint.Inject("errorBeforeDecodeArgs", func(val failpoint.Value) { - //nolint:forcetypeassert - if val.(bool) { - failpoint.Return(ver, errors.New("occur an error before decode args")) - } - }) - - tblInfo, columnInfo, colFromArgs, pos, ifNotExists, err := checkAddColumn(t, job) - if err != nil { - if ifNotExists && infoschema.ErrColumnExists.Equal(err) { - job.Warning = toTError(err) - job.State = model.JobStateDone - return ver, nil - } - return ver, errors.Trace(err) - } - if columnInfo == nil { - columnInfo = InitAndAddColumnToTable(tblInfo, colFromArgs) - logutil.DDLLogger().Info("run add column job", zap.Stringer("job", job), zap.Reflect("columnInfo", *columnInfo)) - if err = checkAddColumnTooManyColumns(len(tblInfo.Columns)); err != nil { - job.State = model.JobStateCancelled - return ver, errors.Trace(err) - } - } - - originalState := columnInfo.State - switch columnInfo.State { - case model.StateNone: - // none -> delete only - columnInfo.State = model.StateDeleteOnly - ver, err = updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, originalState != columnInfo.State) - if err != nil { - return ver, errors.Trace(err) - } - job.SchemaState = model.StateDeleteOnly - case model.StateDeleteOnly: - // delete only -> write only - columnInfo.State = model.StateWriteOnly - ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, originalState != columnInfo.State) - if err != nil { - return ver, errors.Trace(err) - } - // Update the job state when all affairs done. - job.SchemaState = model.StateWriteOnly - case model.StateWriteOnly: - // write only -> reorganization - columnInfo.State = model.StateWriteReorganization - ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, originalState != columnInfo.State) - if err != nil { - return ver, errors.Trace(err) - } - // Update the job state when all affairs done. - job.SchemaState = model.StateWriteReorganization - job.MarkNonRevertible() - case model.StateWriteReorganization: - // reorganization -> public - // Adjust table column offset. - failpoint.InjectCall("onAddColumnStateWriteReorg") - offset, err := LocateOffsetToMove(columnInfo.Offset, pos, tblInfo) - if err != nil { - return ver, errors.Trace(err) - } - tblInfo.MoveColumnInfo(columnInfo.Offset, offset) - columnInfo.State = model.StatePublic - ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, originalState != columnInfo.State) - if err != nil { - return ver, errors.Trace(err) - } - - // Finish this job. - job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo) - addColumnEvent := statsutil.NewAddColumnEvent( - job.SchemaID, - tblInfo, - []*model.ColumnInfo{columnInfo}, - ) - asyncNotifyEvent(d, addColumnEvent) - default: - err = dbterror.ErrInvalidDDLState.GenWithStackByArgs("column", columnInfo.State) - } - - return ver, errors.Trace(err) -} - // CheckAfterPositionExists makes sure the column specified in AFTER clause is exists. // For example, ALTER TABLE t ADD COLUMN c3 INT AFTER c1. func CheckAfterPositionExists(tblInfo *model.TableInfo, pos *ast.ColumnPosition) error { diff --git a/pkg/ddl/create_table.go b/pkg/ddl/create_table.go index 51199e27cfb26..6e6e475970db1 100644 --- a/pkg/ddl/create_table.go +++ b/pkg/ddl/create_table.go @@ -23,9 +23,14 @@ import ( "unicode/utf8" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/config" + "github.com/pingcap/tidb/pkg/ddl/logutil" + "github.com/pingcap/tidb/pkg/ddl/placement" + "github.com/pingcap/tidb/pkg/domain/infosync" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/infoschema" + "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/format" @@ -34,6 +39,7 @@ import ( field_types "github.com/pingcap/tidb/pkg/parser/types" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" + statsutil "github.com/pingcap/tidb/pkg/statistics/handle/util" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/types" @@ -41,8 +47,354 @@ import ( "github.com/pingcap/tidb/pkg/util/dbterror" "github.com/pingcap/tidb/pkg/util/mock" "github.com/pingcap/tidb/pkg/util/set" + "go.uber.org/zap" ) +// DANGER: it is an internal function used by onCreateTable and onCreateTables, for reusing code. Be careful. +// 1. it expects the argument of job has been deserialized. +// 2. it won't call updateSchemaVersion, FinishTableJob and asyncNotifyEvent. +func createTable(d *ddlCtx, t *meta.Meta, job *model.Job, fkCheck bool) (*model.TableInfo, error) { + schemaID := job.SchemaID + tbInfo := job.Args[0].(*model.TableInfo) + + tbInfo.State = model.StateNone + err := checkTableNotExists(d, schemaID, tbInfo.Name.L) + if err != nil { + if infoschema.ErrDatabaseNotExists.Equal(err) || infoschema.ErrTableExists.Equal(err) { + job.State = model.JobStateCancelled + } + return tbInfo, errors.Trace(err) + } + + err = checkConstraintNamesNotExists(t, schemaID, tbInfo.Constraints) + if err != nil { + if infoschema.ErrCheckConstraintDupName.Equal(err) { + job.State = model.JobStateCancelled + } + return tbInfo, errors.Trace(err) + } + + retryable, err := checkTableForeignKeyValidInOwner(d, t, job, tbInfo, fkCheck) + if err != nil { + if !retryable { + job.State = model.JobStateCancelled + } + return tbInfo, errors.Trace(err) + } + // Allocate foreign key ID. + for _, fkInfo := range tbInfo.ForeignKeys { + fkInfo.ID = allocateFKIndexID(tbInfo) + fkInfo.State = model.StatePublic + } + switch tbInfo.State { + case model.StateNone: + // none -> public + tbInfo.State = model.StatePublic + tbInfo.UpdateTS = t.StartTS + err = createTableOrViewWithCheck(t, job, schemaID, tbInfo) + if err != nil { + return tbInfo, errors.Trace(err) + } + + failpoint.Inject("checkOwnerCheckAllVersionsWaitTime", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(tbInfo, errors.New("mock create table error")) + } + }) + + // build table & partition bundles if any. + if err = checkAllTablePlacementPoliciesExistAndCancelNonExistJob(t, job, tbInfo); err != nil { + return tbInfo, errors.Trace(err) + } + + if tbInfo.TiFlashReplica != nil { + replicaInfo := tbInfo.TiFlashReplica + if pi := tbInfo.GetPartitionInfo(); pi != nil { + logutil.DDLLogger().Info("Set TiFlash replica pd rule for partitioned table when creating", zap.Int64("tableID", tbInfo.ID)) + if e := infosync.ConfigureTiFlashPDForPartitions(false, &pi.Definitions, replicaInfo.Count, &replicaInfo.LocationLabels, tbInfo.ID); e != nil { + job.State = model.JobStateCancelled + return tbInfo, errors.Trace(e) + } + // Partitions that in adding mid-state. They have high priorities, so we should set accordingly pd rules. + if e := infosync.ConfigureTiFlashPDForPartitions(true, &pi.AddingDefinitions, replicaInfo.Count, &replicaInfo.LocationLabels, tbInfo.ID); e != nil { + job.State = model.JobStateCancelled + return tbInfo, errors.Trace(e) + } + } else { + logutil.DDLLogger().Info("Set TiFlash replica pd rule when creating", zap.Int64("tableID", tbInfo.ID)) + if e := infosync.ConfigureTiFlashPDForTable(tbInfo.ID, replicaInfo.Count, &replicaInfo.LocationLabels); e != nil { + job.State = model.JobStateCancelled + return tbInfo, errors.Trace(e) + } + } + } + + bundles, err := placement.NewFullTableBundles(t, tbInfo) + if err != nil { + job.State = model.JobStateCancelled + return tbInfo, errors.Trace(err) + } + + // Send the placement bundle to PD. + err = infosync.PutRuleBundlesWithDefaultRetry(context.TODO(), bundles) + if err != nil { + job.State = model.JobStateCancelled + return tbInfo, errors.Wrapf(err, "failed to notify PD the placement rules") + } + + return tbInfo, nil + default: + return tbInfo, dbterror.ErrInvalidDDLState.GenWithStackByArgs("table", tbInfo.State) + } +} + +func onCreateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { + failpoint.Inject("mockExceedErrorLimit", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(ver, errors.New("mock do job error")) + } + }) + + // just decode, createTable will use it as Args[0] + tbInfo := &model.TableInfo{} + fkCheck := false + if err := job.DecodeArgs(tbInfo, &fkCheck); err != nil { + // Invalid arguments, cancel this job. + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + + if len(tbInfo.ForeignKeys) > 0 { + return createTableWithForeignKeys(d, t, job, tbInfo, fkCheck) + } + + tbInfo, err := createTable(d, t, job, fkCheck) + if err != nil { + return ver, errors.Trace(err) + } + + ver, err = updateSchemaVersion(d, t, job) + if err != nil { + return ver, errors.Trace(err) + } + + // Finish this job. + job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tbInfo) + createTableEvent := statsutil.NewCreateTableEvent( + job.SchemaID, + tbInfo, + ) + asyncNotifyEvent(d, createTableEvent) + return ver, errors.Trace(err) +} + +func createTableWithForeignKeys(d *ddlCtx, t *meta.Meta, job *model.Job, tbInfo *model.TableInfo, fkCheck bool) (ver int64, err error) { + switch tbInfo.State { + case model.StateNone, model.StatePublic: + // create table in non-public or public state. The function `createTable` will always reset + // the `tbInfo.State` with `model.StateNone`, so it's fine to just call the `createTable` with + // public state. + // when `br` restores table, the state of `tbInfo` will be public. + tbInfo, err = createTable(d, t, job, fkCheck) + if err != nil { + return ver, errors.Trace(err) + } + tbInfo.State = model.StateWriteOnly + ver, err = updateVersionAndTableInfo(d, t, job, tbInfo, true) + if err != nil { + return ver, errors.Trace(err) + } + job.SchemaState = model.StateWriteOnly + case model.StateWriteOnly: + tbInfo.State = model.StatePublic + ver, err = updateVersionAndTableInfo(d, t, job, tbInfo, true) + if err != nil { + return ver, errors.Trace(err) + } + job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tbInfo) + createTableEvent := statsutil.NewCreateTableEvent( + job.SchemaID, + tbInfo, + ) + asyncNotifyEvent(d, createTableEvent) + return ver, nil + default: + return ver, errors.Trace(dbterror.ErrInvalidDDLJob.GenWithStackByArgs("table", tbInfo.State)) + } + return ver, errors.Trace(err) +} + +func onCreateTables(d *ddlCtx, t *meta.Meta, job *model.Job) (int64, error) { + var ver int64 + + var args []*model.TableInfo + fkCheck := false + err := job.DecodeArgs(&args, &fkCheck) + if err != nil { + // Invalid arguments, cancel this job. + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + + // We don't construct jobs for every table, but only tableInfo + // The following loop creates a stub job for every table + // + // it clones a stub job from the ActionCreateTables job + stubJob := job.Clone() + stubJob.Args = make([]any, 1) + for i := range args { + stubJob.TableID = args[i].ID + stubJob.Args[0] = args[i] + if args[i].Sequence != nil { + err := createSequenceWithCheck(t, stubJob, args[i]) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + } else { + tbInfo, err := createTable(d, t, stubJob, fkCheck) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + args[i] = tbInfo + } + } + + ver, err = updateSchemaVersion(d, t, job) + if err != nil { + return ver, errors.Trace(err) + } + + job.State = model.JobStateDone + job.SchemaState = model.StatePublic + job.BinlogInfo.SetTableInfos(ver, args) + + for i := range args { + createTableEvent := statsutil.NewCreateTableEvent( + job.SchemaID, + args[i], + ) + asyncNotifyEvent(d, createTableEvent) + } + + return ver, errors.Trace(err) +} + +func createTableOrViewWithCheck(t *meta.Meta, job *model.Job, schemaID int64, tbInfo *model.TableInfo) error { + err := checkTableInfoValid(tbInfo) + if err != nil { + job.State = model.JobStateCancelled + return errors.Trace(err) + } + return t.CreateTableOrView(schemaID, tbInfo) +} + +func onCreateView(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { + schemaID := job.SchemaID + tbInfo := &model.TableInfo{} + var orReplace bool + var _placeholder int64 // oldTblInfoID + if err := job.DecodeArgs(tbInfo, &orReplace, &_placeholder); err != nil { + // Invalid arguments, cancel this job. + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + tbInfo.State = model.StateNone + + oldTableID, err := findTableIDByName(d, t, schemaID, tbInfo.Name.L) + if infoschema.ErrTableNotExists.Equal(err) { + err = nil + } + failpoint.InjectCall("onDDLCreateView", job) + if err != nil { + if infoschema.ErrDatabaseNotExists.Equal(err) { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } else if !infoschema.ErrTableExists.Equal(err) { + return ver, errors.Trace(err) + } + if !orReplace { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + } + ver, err = updateSchemaVersion(d, t, job) + if err != nil { + return ver, errors.Trace(err) + } + switch tbInfo.State { + case model.StateNone: + // none -> public + tbInfo.State = model.StatePublic + tbInfo.UpdateTS = t.StartTS + if oldTableID > 0 && orReplace { + err = t.DropTableOrView(schemaID, oldTableID) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + err = t.GetAutoIDAccessors(schemaID, oldTableID).Del() + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + } + err = createTableOrViewWithCheck(t, job, schemaID, tbInfo) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + // Finish this job. + job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tbInfo) + return ver, nil + default: + return ver, dbterror.ErrInvalidDDLState.GenWithStackByArgs("table", tbInfo.State) + } +} + +func findTableIDByName(d *ddlCtx, t *meta.Meta, schemaID int64, tableName string) (int64, error) { + // Try to use memory schema info to check first. + currVer, err := t.GetSchemaVersion() + if err != nil { + return 0, err + } + is := d.infoCache.GetLatest() + if is != nil && is.SchemaMetaVersion() == currVer { + return findTableIDFromInfoSchema(is, schemaID, tableName) + } + + return findTableIDFromStore(t, schemaID, tableName) +} + +func findTableIDFromInfoSchema(is infoschema.InfoSchema, schemaID int64, tableName string) (int64, error) { + schema, ok := is.SchemaByID(schemaID) + if !ok { + return 0, infoschema.ErrDatabaseNotExists.GenWithStackByArgs("") + } + tbl, err := is.TableByName(context.Background(), schema.Name, model.NewCIStr(tableName)) + if err != nil { + return 0, err + } + return tbl.Meta().ID, nil +} + +func findTableIDFromStore(t *meta.Meta, schemaID int64, tableName string) (int64, error) { + tbls, err := t.ListSimpleTables(schemaID) + if err != nil { + if meta.ErrDBNotExists.Equal(err) { + return 0, infoschema.ErrDatabaseNotExists.GenWithStackByArgs("") + } + return 0, errors.Trace(err) + } + for _, tbl := range tbls { + if tbl.Name.L == tableName { + return tbl.ID, nil + } + } + return 0, infoschema.ErrTableNotExists.FastGenByArgs(tableName) +} + // BuildTableInfoFromAST builds model.TableInfo from a SQL statement. // Note: TableID and PartitionID are left as uninitialized value. func BuildTableInfoFromAST(s *ast.CreateTableStmt) (*model.TableInfo, error) { @@ -197,6 +549,15 @@ func checkTableInfoValidExtra(tbInfo *model.TableInfo) error { return err } +// checkTableInfoValid uses to check table info valid. This is used to validate table info. +func checkTableInfoValid(tblInfo *model.TableInfo) error { + _, err := tables.TableFromMeta(autoid.NewAllocators(false), tblInfo) + if err != nil { + return err + } + return checkInvisibleIndexOnPK(tblInfo) +} + func checkDuplicateColumn(cols []*model.ColumnInfo) error { colNames := set.StringSet{} for _, col := range cols { @@ -1131,3 +1492,16 @@ func ShouldBuildClusteredIndex(ctx sessionctx.Context, opt *ast.IndexOption, isS } return opt.PrimaryKeyTp == model.PrimaryKeyTypeClustered } + +// BuildViewInfo builds a ViewInfo structure from an ast.CreateViewStmt. +func BuildViewInfo(s *ast.CreateViewStmt) (*model.ViewInfo, error) { + // Always Use `format.RestoreNameBackQuotes` to restore `SELECT` statement despite the `ANSI_QUOTES` SQL Mode is enabled or not. + restoreFlag := format.RestoreStringSingleQuotes | format.RestoreKeyWordUppercase | format.RestoreNameBackQuotes + var sb strings.Builder + if err := s.Select.Restore(format.NewRestoreCtx(restoreFlag, &sb)); err != nil { + return nil, err + } + + return &model.ViewInfo{Definer: s.Definer, Algorithm: s.Algorithm, + Security: s.Security, SelectStmt: sb.String(), CheckOption: s.CheckOption, Cols: nil}, nil +} diff --git a/pkg/ddl/executor.go b/pkg/ddl/executor.go index 6c1f5e9a3eb06..7d1a8cb537a8d 100644 --- a/pkg/ddl/executor.go +++ b/pkg/ddl/executor.go @@ -52,7 +52,6 @@ import ( "github.com/pingcap/tidb/pkg/owner" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" - "github.com/pingcap/tidb/pkg/parser/format" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" @@ -1001,15 +1000,6 @@ func checkPartitionDefinitionConstraints(ctx sessionctx.Context, tbInfo *model.T return errors.Trace(err) } -// checkTableInfoValid uses to check table info valid. This is used to validate table info. -func checkTableInfoValid(tblInfo *model.TableInfo) error { - _, err := tables.TableFromMeta(autoid.NewAllocators(false), tblInfo) - if err != nil { - return err - } - return checkInvisibleIndexOnPK(tblInfo) -} - func (e *executor) assignPartitionIDs(defs []model.PartitionDefinition) error { genIDs, err := e.genGlobalIDs(len(defs)) if err != nil { @@ -1624,19 +1614,6 @@ func (e *executor) CreateView(ctx sessionctx.Context, s *ast.CreateViewStmt) (er return e.CreateTableWithInfo(ctx, s.ViewName.Schema, tbInfo, nil, WithOnExist(onExist)) } -// BuildViewInfo builds a ViewInfo structure from an ast.CreateViewStmt. -func BuildViewInfo(s *ast.CreateViewStmt) (*model.ViewInfo, error) { - // Always Use `format.RestoreNameBackQuotes` to restore `SELECT` statement despite the `ANSI_QUOTES` SQL Mode is enabled or not. - restoreFlag := format.RestoreStringSingleQuotes | format.RestoreKeyWordUppercase | format.RestoreNameBackQuotes - var sb strings.Builder - if err := s.Select.Restore(format.NewRestoreCtx(restoreFlag, &sb)); err != nil { - return nil, err - } - - return &model.ViewInfo{Definer: s.Definer, Algorithm: s.Algorithm, - Security: s.Security, SelectStmt: sb.String(), CheckOption: s.CheckOption, Cols: nil}, nil -} - func checkPartitionByHash(ctx sessionctx.Context, tbInfo *model.TableInfo) error { return checkNoHashPartitions(ctx, tbInfo.Partition.Num) } @@ -1841,39 +1818,6 @@ func (e *executor) getAutoIDRequirement() autoid.Requirement { } } -// SetDirectPlacementOpt tries to make the PlacementSettings assignments generic for Schema/Table/Partition -func SetDirectPlacementOpt(placementSettings *model.PlacementSettings, placementOptionType ast.PlacementOptionType, stringVal string, uintVal uint64) error { - switch placementOptionType { - case ast.PlacementOptionPrimaryRegion: - placementSettings.PrimaryRegion = stringVal - case ast.PlacementOptionRegions: - placementSettings.Regions = stringVal - case ast.PlacementOptionFollowerCount: - placementSettings.Followers = uintVal - case ast.PlacementOptionVoterCount: - placementSettings.Voters = uintVal - case ast.PlacementOptionLearnerCount: - placementSettings.Learners = uintVal - case ast.PlacementOptionSchedule: - placementSettings.Schedule = stringVal - case ast.PlacementOptionConstraints: - placementSettings.Constraints = stringVal - case ast.PlacementOptionLeaderConstraints: - placementSettings.LeaderConstraints = stringVal - case ast.PlacementOptionLearnerConstraints: - placementSettings.LearnerConstraints = stringVal - case ast.PlacementOptionFollowerConstraints: - placementSettings.FollowerConstraints = stringVal - case ast.PlacementOptionVoterConstraints: - placementSettings.VoterConstraints = stringVal - case ast.PlacementOptionSurvivalPreferences: - placementSettings.SurvivalPreferences = stringVal - default: - return errors.Trace(errors.New("unknown placement policy option")) - } - return nil -} - func shardingBits(tblInfo *model.TableInfo) uint64 { if tblInfo.ShardRowIDBits > 0 { return tblInfo.ShardRowIDBits @@ -1888,33 +1832,6 @@ func isIgnorableSpec(tp ast.AlterTableType) bool { return tp == ast.AlterTableLock || tp == ast.AlterTableAlgorithm } -// getCharsetAndCollateInColumnDef will iterate collate in the options, validate it by checking the charset -// of column definition. If there's no collate in the option, the default collate of column's charset will be used. -func getCharsetAndCollateInColumnDef(sessVars *variable.SessionVars, def *ast.ColumnDef) (chs, coll string, err error) { - chs = def.Tp.GetCharset() - coll = def.Tp.GetCollate() - if chs != "" && coll == "" { - if coll, err = GetDefaultCollation(sessVars, chs); err != nil { - return "", "", errors.Trace(err) - } - } - for _, opt := range def.Options { - if opt.Tp == ast.ColumnOptionCollate { - info, err := collate.GetCollationByName(opt.StrValue) - if err != nil { - return "", "", errors.Trace(err) - } - if chs == "" { - chs = info.CharsetName - } else if chs != info.CharsetName { - return "", "", dbterror.ErrCollationCharsetMismatch.GenWithStackByArgs(info.Name, chs) - } - coll = info.Name - } - } - return -} - // 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. @@ -2559,24 +2476,6 @@ func (e *executor) getSchemaAndTableByIdent(tableIdent ast.Ident) (dbInfo *model return schema, t, nil } -func checkUnsupportedColumnConstraint(col *ast.ColumnDef, ti ast.Ident) error { - for _, constraint := range col.Options { - switch constraint.Tp { - case ast.ColumnOptionAutoIncrement: - return dbterror.ErrUnsupportedAddColumn.GenWithStack("unsupported add column '%s' constraint AUTO_INCREMENT when altering '%s.%s'", col.Name, ti.Schema, ti.Name) - case ast.ColumnOptionPrimaryKey: - return dbterror.ErrUnsupportedAddColumn.GenWithStack("unsupported add column '%s' constraint PRIMARY KEY when altering '%s.%s'", col.Name, ti.Schema, ti.Name) - case ast.ColumnOptionUniqKey: - return dbterror.ErrUnsupportedAddColumn.GenWithStack("unsupported add column '%s' constraint UNIQUE KEY when altering '%s.%s'", col.Name, ti.Schema, ti.Name) - case ast.ColumnOptionAutoRandom: - errMsg := fmt.Sprintf(autoid.AutoRandomAlterAddColumn, col.Name, ti.Schema, ti.Name) - return dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(errMsg) - } - } - - return 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] diff --git a/pkg/ddl/placement_policy.go b/pkg/ddl/placement_policy.go index 49d42806fd983..adbfe0902e653 100644 --- a/pkg/ddl/placement_policy.go +++ b/pkg/ddl/placement_policy.go @@ -631,3 +631,36 @@ func checkIgnorePlacementDDL(ctx sessionctx.Context) bool { } return false } + +// SetDirectPlacementOpt tries to make the PlacementSettings assignments generic for Schema/Table/Partition +func SetDirectPlacementOpt(placementSettings *model.PlacementSettings, placementOptionType ast.PlacementOptionType, stringVal string, uintVal uint64) error { + switch placementOptionType { + case ast.PlacementOptionPrimaryRegion: + placementSettings.PrimaryRegion = stringVal + case ast.PlacementOptionRegions: + placementSettings.Regions = stringVal + case ast.PlacementOptionFollowerCount: + placementSettings.Followers = uintVal + case ast.PlacementOptionVoterCount: + placementSettings.Voters = uintVal + case ast.PlacementOptionLearnerCount: + placementSettings.Learners = uintVal + case ast.PlacementOptionSchedule: + placementSettings.Schedule = stringVal + case ast.PlacementOptionConstraints: + placementSettings.Constraints = stringVal + case ast.PlacementOptionLeaderConstraints: + placementSettings.LeaderConstraints = stringVal + case ast.PlacementOptionLearnerConstraints: + placementSettings.LearnerConstraints = stringVal + case ast.PlacementOptionFollowerConstraints: + placementSettings.FollowerConstraints = stringVal + case ast.PlacementOptionVoterConstraints: + placementSettings.VoterConstraints = stringVal + case ast.PlacementOptionSurvivalPreferences: + placementSettings.SurvivalPreferences = stringVal + default: + return errors.Trace(errors.New("unknown placement policy option")) + } + return nil +} diff --git a/pkg/ddl/table.go b/pkg/ddl/table.go index 8723dc78c8843..4d4696b43339c 100644 --- a/pkg/ddl/table.go +++ b/pkg/ddl/table.go @@ -50,246 +50,6 @@ import ( const tiflashCheckTiDBHTTPAPIHalfInterval = 2500 * time.Millisecond -// DANGER: it is an internal function used by onCreateTable and onCreateTables, for reusing code. Be careful. -// 1. it expects the argument of job has been deserialized. -// 2. it won't call updateSchemaVersion, FinishTableJob and asyncNotifyEvent. -func createTable(d *ddlCtx, t *meta.Meta, job *model.Job, fkCheck bool) (*model.TableInfo, error) { - schemaID := job.SchemaID - tbInfo := job.Args[0].(*model.TableInfo) - - tbInfo.State = model.StateNone - err := checkTableNotExists(d, schemaID, tbInfo.Name.L) - if err != nil { - if infoschema.ErrDatabaseNotExists.Equal(err) || infoschema.ErrTableExists.Equal(err) { - job.State = model.JobStateCancelled - } - return tbInfo, errors.Trace(err) - } - - err = checkConstraintNamesNotExists(t, schemaID, tbInfo.Constraints) - if err != nil { - if infoschema.ErrCheckConstraintDupName.Equal(err) { - job.State = model.JobStateCancelled - } - return tbInfo, errors.Trace(err) - } - - retryable, err := checkTableForeignKeyValidInOwner(d, t, job, tbInfo, fkCheck) - if err != nil { - if !retryable { - job.State = model.JobStateCancelled - } - return tbInfo, errors.Trace(err) - } - // Allocate foreign key ID. - for _, fkInfo := range tbInfo.ForeignKeys { - fkInfo.ID = allocateFKIndexID(tbInfo) - fkInfo.State = model.StatePublic - } - switch tbInfo.State { - case model.StateNone: - // none -> public - tbInfo.State = model.StatePublic - tbInfo.UpdateTS = t.StartTS - err = createTableOrViewWithCheck(t, job, schemaID, tbInfo) - if err != nil { - return tbInfo, errors.Trace(err) - } - - failpoint.Inject("checkOwnerCheckAllVersionsWaitTime", func(val failpoint.Value) { - if val.(bool) { - failpoint.Return(tbInfo, errors.New("mock create table error")) - } - }) - - // build table & partition bundles if any. - if err = checkAllTablePlacementPoliciesExistAndCancelNonExistJob(t, job, tbInfo); err != nil { - return tbInfo, errors.Trace(err) - } - - if tbInfo.TiFlashReplica != nil { - replicaInfo := tbInfo.TiFlashReplica - if pi := tbInfo.GetPartitionInfo(); pi != nil { - logutil.DDLLogger().Info("Set TiFlash replica pd rule for partitioned table when creating", zap.Int64("tableID", tbInfo.ID)) - if e := infosync.ConfigureTiFlashPDForPartitions(false, &pi.Definitions, replicaInfo.Count, &replicaInfo.LocationLabels, tbInfo.ID); e != nil { - job.State = model.JobStateCancelled - return tbInfo, errors.Trace(e) - } - // Partitions that in adding mid-state. They have high priorities, so we should set accordingly pd rules. - if e := infosync.ConfigureTiFlashPDForPartitions(true, &pi.AddingDefinitions, replicaInfo.Count, &replicaInfo.LocationLabels, tbInfo.ID); e != nil { - job.State = model.JobStateCancelled - return tbInfo, errors.Trace(e) - } - } else { - logutil.DDLLogger().Info("Set TiFlash replica pd rule when creating", zap.Int64("tableID", tbInfo.ID)) - if e := infosync.ConfigureTiFlashPDForTable(tbInfo.ID, replicaInfo.Count, &replicaInfo.LocationLabels); e != nil { - job.State = model.JobStateCancelled - return tbInfo, errors.Trace(e) - } - } - } - - bundles, err := placement.NewFullTableBundles(t, tbInfo) - if err != nil { - job.State = model.JobStateCancelled - return tbInfo, errors.Trace(err) - } - - // Send the placement bundle to PD. - err = infosync.PutRuleBundlesWithDefaultRetry(context.TODO(), bundles) - if err != nil { - job.State = model.JobStateCancelled - return tbInfo, errors.Wrapf(err, "failed to notify PD the placement rules") - } - - return tbInfo, nil - default: - return tbInfo, dbterror.ErrInvalidDDLState.GenWithStackByArgs("table", tbInfo.State) - } -} - -func onCreateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { - failpoint.Inject("mockExceedErrorLimit", func(val failpoint.Value) { - if val.(bool) { - failpoint.Return(ver, errors.New("mock do job error")) - } - }) - - // just decode, createTable will use it as Args[0] - tbInfo := &model.TableInfo{} - fkCheck := false - if err := job.DecodeArgs(tbInfo, &fkCheck); err != nil { - // Invalid arguments, cancel this job. - job.State = model.JobStateCancelled - return ver, errors.Trace(err) - } - - if len(tbInfo.ForeignKeys) > 0 { - return createTableWithForeignKeys(d, t, job, tbInfo, fkCheck) - } - - tbInfo, err := createTable(d, t, job, fkCheck) - if err != nil { - return ver, errors.Trace(err) - } - - ver, err = updateSchemaVersion(d, t, job) - if err != nil { - return ver, errors.Trace(err) - } - - // Finish this job. - job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tbInfo) - createTableEvent := statsutil.NewCreateTableEvent( - job.SchemaID, - tbInfo, - ) - asyncNotifyEvent(d, createTableEvent) - return ver, errors.Trace(err) -} - -func createTableWithForeignKeys(d *ddlCtx, t *meta.Meta, job *model.Job, tbInfo *model.TableInfo, fkCheck bool) (ver int64, err error) { - switch tbInfo.State { - case model.StateNone, model.StatePublic: - // create table in non-public or public state. The function `createTable` will always reset - // the `tbInfo.State` with `model.StateNone`, so it's fine to just call the `createTable` with - // public state. - // when `br` restores table, the state of `tbInfo` will be public. - tbInfo, err = createTable(d, t, job, fkCheck) - if err != nil { - return ver, errors.Trace(err) - } - tbInfo.State = model.StateWriteOnly - ver, err = updateVersionAndTableInfo(d, t, job, tbInfo, true) - if err != nil { - return ver, errors.Trace(err) - } - job.SchemaState = model.StateWriteOnly - case model.StateWriteOnly: - tbInfo.State = model.StatePublic - ver, err = updateVersionAndTableInfo(d, t, job, tbInfo, true) - if err != nil { - return ver, errors.Trace(err) - } - job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tbInfo) - createTableEvent := statsutil.NewCreateTableEvent( - job.SchemaID, - tbInfo, - ) - asyncNotifyEvent(d, createTableEvent) - return ver, nil - default: - return ver, errors.Trace(dbterror.ErrInvalidDDLJob.GenWithStackByArgs("table", tbInfo.State)) - } - return ver, errors.Trace(err) -} - -func onCreateTables(d *ddlCtx, t *meta.Meta, job *model.Job) (int64, error) { - var ver int64 - - var args []*model.TableInfo - fkCheck := false - err := job.DecodeArgs(&args, &fkCheck) - if err != nil { - // Invalid arguments, cancel this job. - job.State = model.JobStateCancelled - return ver, errors.Trace(err) - } - - // We don't construct jobs for every table, but only tableInfo - // The following loop creates a stub job for every table - // - // it clones a stub job from the ActionCreateTables job - stubJob := job.Clone() - stubJob.Args = make([]any, 1) - for i := range args { - stubJob.TableID = args[i].ID - stubJob.Args[0] = args[i] - if args[i].Sequence != nil { - err := createSequenceWithCheck(t, stubJob, args[i]) - if err != nil { - job.State = model.JobStateCancelled - return ver, errors.Trace(err) - } - } else { - tbInfo, err := createTable(d, t, stubJob, fkCheck) - if err != nil { - job.State = model.JobStateCancelled - return ver, errors.Trace(err) - } - args[i] = tbInfo - } - } - - ver, err = updateSchemaVersion(d, t, job) - if err != nil { - return ver, errors.Trace(err) - } - - job.State = model.JobStateDone - job.SchemaState = model.StatePublic - job.BinlogInfo.SetTableInfos(ver, args) - - for i := range args { - createTableEvent := statsutil.NewCreateTableEvent( - job.SchemaID, - args[i], - ) - asyncNotifyEvent(d, createTableEvent) - } - - return ver, errors.Trace(err) -} - -func createTableOrViewWithCheck(t *meta.Meta, job *model.Job, schemaID int64, tbInfo *model.TableInfo) error { - err := checkTableInfoValid(tbInfo) - if err != nil { - job.State = model.JobStateCancelled - return errors.Trace(err) - } - return t.CreateTableOrView(schemaID, tbInfo) -} - func repairTableOrViewWithCheck(t *meta.Meta, job *model.Job, schemaID int64, tbInfo *model.TableInfo) error { err := checkTableInfoValid(tbInfo) if err != nil { @@ -299,69 +59,6 @@ func repairTableOrViewWithCheck(t *meta.Meta, job *model.Job, schemaID int64, tb return t.UpdateTable(schemaID, tbInfo) } -func onCreateView(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { - schemaID := job.SchemaID - tbInfo := &model.TableInfo{} - var orReplace bool - var _placeholder int64 // oldTblInfoID - if err := job.DecodeArgs(tbInfo, &orReplace, &_placeholder); err != nil { - // Invalid arguments, cancel this job. - job.State = model.JobStateCancelled - return ver, errors.Trace(err) - } - tbInfo.State = model.StateNone - - oldTableID, err := findTableIDByName(d, t, schemaID, tbInfo.Name.L) - if infoschema.ErrTableNotExists.Equal(err) { - err = nil - } - failpoint.InjectCall("onDDLCreateView", job) - if err != nil { - if infoschema.ErrDatabaseNotExists.Equal(err) { - job.State = model.JobStateCancelled - return ver, errors.Trace(err) - } else if !infoschema.ErrTableExists.Equal(err) { - return ver, errors.Trace(err) - } - if !orReplace { - job.State = model.JobStateCancelled - return ver, errors.Trace(err) - } - } - ver, err = updateSchemaVersion(d, t, job) - if err != nil { - return ver, errors.Trace(err) - } - switch tbInfo.State { - case model.StateNone: - // none -> public - tbInfo.State = model.StatePublic - tbInfo.UpdateTS = t.StartTS - if oldTableID > 0 && orReplace { - err = t.DropTableOrView(schemaID, oldTableID) - if err != nil { - job.State = model.JobStateCancelled - return ver, errors.Trace(err) - } - err = t.GetAutoIDAccessors(schemaID, oldTableID).Del() - if err != nil { - job.State = model.JobStateCancelled - return ver, errors.Trace(err) - } - } - err = createTableOrViewWithCheck(t, job, schemaID, tbInfo) - if err != nil { - job.State = model.JobStateCancelled - return ver, errors.Trace(err) - } - // Finish this job. - job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tbInfo) - return ver, nil - default: - return ver, dbterror.ErrInvalidDDLState.GenWithStackByArgs("table", tbInfo.State) - } -} - func onDropTableOrView(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { tblInfo, err := checkTableExistAndCancelNonExistJob(t, job, job.SchemaID) if err != nil { @@ -1573,48 +1270,6 @@ func checkTableNotExistsFromInfoSchema(is infoschema.InfoSchema, schemaID int64, return nil } -func findTableIDByName(d *ddlCtx, t *meta.Meta, schemaID int64, tableName string) (int64, error) { - // Try to use memory schema info to check first. - currVer, err := t.GetSchemaVersion() - if err != nil { - return 0, err - } - is := d.infoCache.GetLatest() - if is != nil && is.SchemaMetaVersion() == currVer { - return findTableIDFromInfoSchema(is, schemaID, tableName) - } - - return findTableIDFromStore(t, schemaID, tableName) -} - -func findTableIDFromInfoSchema(is infoschema.InfoSchema, schemaID int64, tableName string) (int64, error) { - schema, ok := is.SchemaByID(schemaID) - if !ok { - return 0, infoschema.ErrDatabaseNotExists.GenWithStackByArgs("") - } - tbl, err := is.TableByName(context.Background(), schema.Name, model.NewCIStr(tableName)) - if err != nil { - return 0, err - } - return tbl.Meta().ID, nil -} - -func findTableIDFromStore(t *meta.Meta, schemaID int64, tableName string) (int64, error) { - tbls, err := t.ListSimpleTables(schemaID) - if err != nil { - if meta.ErrDBNotExists.Equal(err) { - return 0, infoschema.ErrDatabaseNotExists.GenWithStackByArgs("") - } - return 0, errors.Trace(err) - } - for _, tbl := range tbls { - if tbl.Name.L == tableName { - return tbl.ID, nil - } - } - return 0, infoschema.ErrTableNotExists.FastGenByArgs(tableName) -} - // updateVersionAndTableInfoWithCheck checks table info validate and updates the schema version and the table information func updateVersionAndTableInfoWithCheck(d *ddlCtx, t *meta.Meta, job *model.Job, tblInfo *model.TableInfo, shouldUpdateVer bool, multiInfos ...schemaIDAndTableInfo) ( ver int64, err error) { From 76eac4dd56372d116a64de5984e4f24b1361fa22 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Thu, 1 Aug 2024 18:25:25 +0800 Subject: [PATCH 07/13] split --- pkg/ddl/create_table.go | 20 ++++ pkg/ddl/executor.go | 220 ---------------------------------------- pkg/ddl/partition.go | 199 ++++++++++++++++++++++++++++++++++++ 3 files changed, 219 insertions(+), 220 deletions(-) diff --git a/pkg/ddl/create_table.go b/pkg/ddl/create_table.go index 6e6e475970db1..68d686ddc35dc 100644 --- a/pkg/ddl/create_table.go +++ b/pkg/ddl/create_table.go @@ -1006,6 +1006,26 @@ func checkDuplicateConstraint(namesMap map[string]bool, name string, constraintT return nil } +func setEmptyCheckConstraintName(tableLowerName string, namesMap map[string]bool, constrs []*ast.Constraint) { + cnt := 1 + constraintPrefix := tableLowerName + "_chk_" + for _, constr := range constrs { + if constr.Name == "" { + constrName := fmt.Sprintf("%s%d", constraintPrefix, cnt) + for { + // loop until find constrName that haven't been used. + if !namesMap[constrName] { + namesMap[constrName] = true + break + } + cnt++ + constrName = fmt.Sprintf("%s%d", constraintPrefix, cnt) + } + constr.Name = constrName + } + } +} + func setColumnFlagWithConstraint(colMap map[string]*table.Column, v *ast.Constraint) { switch v.Tp { case ast.ConstraintPrimaryKey: diff --git a/pkg/ddl/executor.go b/pkg/ddl/executor.go index 7d1a8cb537a8d..8db859a586669 100644 --- a/pkg/ddl/executor.go +++ b/pkg/ddl/executor.go @@ -55,7 +55,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" - field_types "github.com/pingcap/tidb/pkg/parser/types" "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" @@ -66,7 +65,6 @@ import ( "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" - "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/collate" "github.com/pingcap/tidb/pkg/util/dbterror" "github.com/pingcap/tidb/pkg/util/dbterror/exeerrors" @@ -74,7 +72,6 @@ import ( "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/sqlkiller" "github.com/pingcap/tidb/pkg/util/stringutil" tikv "github.com/tikv/client-go/v2/kv" "github.com/tikv/client-go/v2/oracle" @@ -939,26 +936,6 @@ func IsAutoRandomColumnID(tblInfo *model.TableInfo, colID int64) bool { return false } -func setEmptyCheckConstraintName(tableLowerName string, namesMap map[string]bool, constrs []*ast.Constraint) { - cnt := 1 - constraintPrefix := tableLowerName + "_chk_" - for _, constr := range constrs { - if constr.Name == "" { - constrName := fmt.Sprintf("%s%d", constraintPrefix, cnt) - for { - // loop until find constrName that haven't been used. - if !namesMap[constrName] { - namesMap[constrName] = true - break - } - cnt++ - constrName = fmt.Sprintf("%s%d", constraintPrefix, cnt) - } - constr.Name = constrName - } - } -} - // 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, @@ -974,32 +951,6 @@ func checkInvisibleIndexOnPK(tblInfo *model.TableInfo) error { return nil } -func checkPartitionDefinitionConstraints(ctx sessionctx.Context, tbInfo *model.TableInfo) error { - var err error - if err = checkPartitionNameUnique(tbInfo.Partition); err != nil { - return errors.Trace(err) - } - if err = checkAddPartitionTooManyPartitions(uint64(len(tbInfo.Partition.Definitions))); err != nil { - return err - } - if err = checkAddPartitionOnTemporaryMode(tbInfo); err != nil { - return err - } - if err = checkPartitionColumnsUnique(tbInfo); err != nil { - return err - } - - switch tbInfo.Partition.Type { - case model.PartitionTypeRange: - err = checkPartitionByRange(ctx, tbInfo) - case model.PartitionTypeHash, model.PartitionTypeKey: - err = checkPartitionByHash(ctx, tbInfo) - case model.PartitionTypeList: - err = checkPartitionByList(ctx, tbInfo) - } - return errors.Trace(err) -} - func (e *executor) assignPartitionIDs(defs []model.PartitionDefinition) error { genIDs, err := e.genGlobalIDs(len(defs)) if err != nil { @@ -1614,177 +1565,6 @@ func (e *executor) CreateView(ctx sessionctx.Context, s *ast.CreateViewStmt) (er return e.CreateTableWithInfo(ctx, s.ViewName.Schema, tbInfo, nil, WithOnExist(onExist)) } -func checkPartitionByHash(ctx sessionctx.Context, tbInfo *model.TableInfo) error { - return checkNoHashPartitions(ctx, tbInfo.Partition.Num) -} - -// checkPartitionByRange checks validity of a "BY RANGE" partition. -func checkPartitionByRange(ctx sessionctx.Context, tbInfo *model.TableInfo) error { - failpoint.Inject("CheckPartitionByRangeErr", func() { - ctx.GetSessionVars().SQLKiller.SendKillSignal(sqlkiller.QueryMemoryExceeded) - panic(ctx.GetSessionVars().SQLKiller.HandleSignal()) - }) - pi := tbInfo.Partition - - if len(pi.Columns) == 0 { - return checkRangePartitionValue(ctx, tbInfo) - } - - return checkRangeColumnsPartitionValue(ctx, tbInfo) -} - -// checkPartitionByList checks validity of a "BY LIST" partition. -func checkPartitionByList(ctx sessionctx.Context, tbInfo *model.TableInfo) error { - return checkListPartitionValue(ctx.GetExprCtx(), tbInfo) -} - -func isValidKeyPartitionColType(fieldType types.FieldType) bool { - switch fieldType.GetType() { - case mysql.TypeBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeJSON, mysql.TypeGeometry, mysql.TypeTiDBVectorFloat32: - return false - default: - return true - } -} - -func isColTypeAllowedAsPartitioningCol(partType model.PartitionType, fieldType types.FieldType) bool { - // For key partition, the permitted partition field types can be all field types except - // BLOB, JSON, Geometry - if partType == model.PartitionTypeKey { - return isValidKeyPartitionColType(fieldType) - } - // The permitted data types are shown in the following list: - // All integer types - // DATE and DATETIME - // CHAR, VARCHAR, BINARY, and VARBINARY - // See https://dev.mysql.com/doc/mysql-partitioning-excerpt/5.7/en/partitioning-columns.html - // Note that also TIME is allowed in MySQL. Also see https://bugs.mysql.com/bug.php?id=84362 - switch fieldType.GetType() { - case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong: - case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeDuration: - case mysql.TypeVarchar, mysql.TypeString: - default: - return false - } - return true -} - -func checkColumnsPartitionType(tbInfo *model.TableInfo) error { - for _, col := range tbInfo.Partition.Columns { - colInfo := tbInfo.FindPublicColumnByName(col.L) - if colInfo == nil { - return errors.Trace(dbterror.ErrFieldNotFoundPart) - } - if !isColTypeAllowedAsPartitioningCol(tbInfo.Partition.Type, colInfo.FieldType) { - return dbterror.ErrNotAllowedTypeInPartition.GenWithStackByArgs(col.O) - } - } - return nil -} - -func checkRangeColumnsPartitionValue(ctx sessionctx.Context, tbInfo *model.TableInfo) error { - // Range columns partition key supports multiple data types with integer、datetime、string. - pi := tbInfo.Partition - defs := pi.Definitions - if len(defs) < 1 { - return ast.ErrPartitionsMustBeDefined.GenWithStackByArgs("RANGE") - } - - curr := &defs[0] - if len(curr.LessThan) != len(pi.Columns) { - return errors.Trace(ast.ErrPartitionColumnList) - } - var prev *model.PartitionDefinition - for i := 1; i < len(defs); i++ { - prev, curr = curr, &defs[i] - succ, err := checkTwoRangeColumns(ctx, curr, prev, pi, tbInfo) - if err != nil { - return err - } - if !succ { - return errors.Trace(dbterror.ErrRangeNotIncreasing) - } - } - return nil -} - -func checkTwoRangeColumns(ctx sessionctx.Context, curr, prev *model.PartitionDefinition, pi *model.PartitionInfo, tbInfo *model.TableInfo) (bool, error) { - if len(curr.LessThan) != len(pi.Columns) { - return false, errors.Trace(ast.ErrPartitionColumnList) - } - for i := 0; i < len(pi.Columns); i++ { - // Special handling for MAXVALUE. - if strings.EqualFold(curr.LessThan[i], partitionMaxValue) && !strings.EqualFold(prev.LessThan[i], partitionMaxValue) { - // If current is maxvalue, it certainly >= previous. - return true, nil - } - if strings.EqualFold(prev.LessThan[i], partitionMaxValue) { - // Current is not maxvalue, and previous is maxvalue. - return false, nil - } - - // The tuples of column values used to define the partitions are strictly increasing: - // PARTITION p0 VALUES LESS THAN (5,10,'ggg') - // PARTITION p1 VALUES LESS THAN (10,20,'mmm') - // PARTITION p2 VALUES LESS THAN (15,30,'sss') - colInfo := findColumnByName(pi.Columns[i].L, tbInfo) - cmp, err := parseAndEvalBoolExpr(ctx.GetExprCtx(), curr.LessThan[i], prev.LessThan[i], colInfo, tbInfo) - if err != nil { - return false, err - } - - if cmp > 0 { - return true, nil - } - - if cmp < 0 { - return false, nil - } - } - return false, nil -} - -// equal, return 0 -// greater, return 1 -// less, return -1 -func parseAndEvalBoolExpr(ctx expression.BuildContext, l, r string, colInfo *model.ColumnInfo, tbInfo *model.TableInfo) (int64, error) { - lexpr, err := expression.ParseSimpleExpr(ctx, l, expression.WithTableInfo("", tbInfo), expression.WithCastExprTo(&colInfo.FieldType)) - if err != nil { - return 0, err - } - rexpr, err := expression.ParseSimpleExpr(ctx, r, expression.WithTableInfo("", tbInfo), expression.WithCastExprTo(&colInfo.FieldType)) - if err != nil { - return 0, err - } - - e, err := expression.NewFunctionBase(ctx, ast.EQ, field_types.NewFieldType(mysql.TypeLonglong), lexpr, rexpr) - if err != nil { - return 0, err - } - e.SetCharsetAndCollation(colInfo.GetCharset(), colInfo.GetCollate()) - res, _, err1 := e.EvalInt(ctx.GetEvalCtx(), chunk.Row{}) - if err1 != nil { - return 0, err1 - } - if res == 1 { - return 0, nil - } - - e, err = expression.NewFunctionBase(ctx, ast.GT, field_types.NewFieldType(mysql.TypeLonglong), lexpr, rexpr) - if err != nil { - return 0, err - } - e.SetCharsetAndCollation(colInfo.GetCharset(), colInfo.GetCollate()) - res, _, err1 = e.EvalInt(ctx.GetEvalCtx(), chunk.Row{}) - if err1 != nil { - return 0, err1 - } - if res > 0 { - return 1, nil - } - return -1, nil -} - func checkCharsetAndCollation(cs string, co string) error { if !charset.ValidCharsetAndCollation(cs, co) { return dbterror.ErrUnknownCharacterSet.GenWithStackByArgs(cs) diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index a3402ab93c5ac..a0ee830c0e870 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -45,6 +45,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/opcode" "github.com/pingcap/tidb/pkg/parser/terror" + field_types "github.com/pingcap/tidb/pkg/parser/types" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" statsutil "github.com/pingcap/tidb/pkg/statistics/handle/util" @@ -61,6 +62,7 @@ import ( "github.com/pingcap/tidb/pkg/util/mathutil" decoder "github.com/pingcap/tidb/pkg/util/rowDecoder" "github.com/pingcap/tidb/pkg/util/slice" + "github.com/pingcap/tidb/pkg/util/sqlkiller" "github.com/pingcap/tidb/pkg/util/stringutil" "github.com/tikv/client-go/v2/tikv" kvutil "github.com/tikv/client-go/v2/util" @@ -659,6 +661,50 @@ func getPartitionColSlices(sctx expression.BuildContext, tblInfo *model.TableInf return nil, errors.Errorf("Table partition metadata not correct, neither partition expression or list of partition columns") } +func checkColumnsPartitionType(tbInfo *model.TableInfo) error { + for _, col := range tbInfo.Partition.Columns { + colInfo := tbInfo.FindPublicColumnByName(col.L) + if colInfo == nil { + return errors.Trace(dbterror.ErrFieldNotFoundPart) + } + if !isColTypeAllowedAsPartitioningCol(tbInfo.Partition.Type, colInfo.FieldType) { + return dbterror.ErrNotAllowedTypeInPartition.GenWithStackByArgs(col.O) + } + } + return nil +} + +func isValidKeyPartitionColType(fieldType types.FieldType) bool { + switch fieldType.GetType() { + case mysql.TypeBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeJSON, mysql.TypeGeometry, mysql.TypeTiDBVectorFloat32: + return false + default: + return true + } +} + +func isColTypeAllowedAsPartitioningCol(partType model.PartitionType, fieldType types.FieldType) bool { + // For key partition, the permitted partition field types can be all field types except + // BLOB, JSON, Geometry + if partType == model.PartitionTypeKey { + return isValidKeyPartitionColType(fieldType) + } + // The permitted data types are shown in the following list: + // All integer types + // DATE and DATETIME + // CHAR, VARCHAR, BINARY, and VARBINARY + // See https://dev.mysql.com/doc/mysql-partitioning-excerpt/5.7/en/partitioning-columns.html + // Note that also TIME is allowed in MySQL. Also see https://bugs.mysql.com/bug.php?id=84362 + switch fieldType.GetType() { + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong: + case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeDuration: + case mysql.TypeVarchar, mysql.TypeString: + default: + return false + } + return true +} + // getPartitionIntervalFromTable checks if a partitioned table matches a generated INTERVAL partitioned scheme // will return nil if error occurs, i.e. not an INTERVAL partitioned table func getPartitionIntervalFromTable(ctx expression.BuildContext, tbInfo *model.TableInfo) *ast.PartitionInterval { @@ -4721,3 +4767,156 @@ func generatePartValuesWithTp(partVal types.Datum, tp types.FieldType) (string, return "", dbterror.ErrWrongTypeColumnValue.GenWithStackByArgs() } + +func checkPartitionDefinitionConstraints(ctx sessionctx.Context, tbInfo *model.TableInfo) error { + var err error + if err = checkPartitionNameUnique(tbInfo.Partition); err != nil { + return errors.Trace(err) + } + if err = checkAddPartitionTooManyPartitions(uint64(len(tbInfo.Partition.Definitions))); err != nil { + return err + } + if err = checkAddPartitionOnTemporaryMode(tbInfo); err != nil { + return err + } + if err = checkPartitionColumnsUnique(tbInfo); err != nil { + return err + } + + switch tbInfo.Partition.Type { + case model.PartitionTypeRange: + err = checkPartitionByRange(ctx, tbInfo) + case model.PartitionTypeHash, model.PartitionTypeKey: + err = checkPartitionByHash(ctx, tbInfo) + case model.PartitionTypeList: + err = checkPartitionByList(ctx, tbInfo) + } + return errors.Trace(err) +} + +func checkPartitionByHash(ctx sessionctx.Context, tbInfo *model.TableInfo) error { + return checkNoHashPartitions(ctx, tbInfo.Partition.Num) +} + +// checkPartitionByRange checks validity of a "BY RANGE" partition. +func checkPartitionByRange(ctx sessionctx.Context, tbInfo *model.TableInfo) error { + failpoint.Inject("CheckPartitionByRangeErr", func() { + ctx.GetSessionVars().SQLKiller.SendKillSignal(sqlkiller.QueryMemoryExceeded) + panic(ctx.GetSessionVars().SQLKiller.HandleSignal()) + }) + pi := tbInfo.Partition + + if len(pi.Columns) == 0 { + return checkRangePartitionValue(ctx, tbInfo) + } + + return checkRangeColumnsPartitionValue(ctx, tbInfo) +} + +func checkRangeColumnsPartitionValue(ctx sessionctx.Context, tbInfo *model.TableInfo) error { + // Range columns partition key supports multiple data types with integer、datetime、string. + pi := tbInfo.Partition + defs := pi.Definitions + if len(defs) < 1 { + return ast.ErrPartitionsMustBeDefined.GenWithStackByArgs("RANGE") + } + + curr := &defs[0] + if len(curr.LessThan) != len(pi.Columns) { + return errors.Trace(ast.ErrPartitionColumnList) + } + var prev *model.PartitionDefinition + for i := 1; i < len(defs); i++ { + prev, curr = curr, &defs[i] + succ, err := checkTwoRangeColumns(ctx, curr, prev, pi, tbInfo) + if err != nil { + return err + } + if !succ { + return errors.Trace(dbterror.ErrRangeNotIncreasing) + } + } + return nil +} + +func checkTwoRangeColumns(ctx sessionctx.Context, curr, prev *model.PartitionDefinition, pi *model.PartitionInfo, tbInfo *model.TableInfo) (bool, error) { + if len(curr.LessThan) != len(pi.Columns) { + return false, errors.Trace(ast.ErrPartitionColumnList) + } + for i := 0; i < len(pi.Columns); i++ { + // Special handling for MAXVALUE. + if strings.EqualFold(curr.LessThan[i], partitionMaxValue) && !strings.EqualFold(prev.LessThan[i], partitionMaxValue) { + // If current is maxvalue, it certainly >= previous. + return true, nil + } + if strings.EqualFold(prev.LessThan[i], partitionMaxValue) { + // Current is not maxvalue, and previous is maxvalue. + return false, nil + } + + // The tuples of column values used to define the partitions are strictly increasing: + // PARTITION p0 VALUES LESS THAN (5,10,'ggg') + // PARTITION p1 VALUES LESS THAN (10,20,'mmm') + // PARTITION p2 VALUES LESS THAN (15,30,'sss') + colInfo := findColumnByName(pi.Columns[i].L, tbInfo) + cmp, err := parseAndEvalBoolExpr(ctx.GetExprCtx(), curr.LessThan[i], prev.LessThan[i], colInfo, tbInfo) + if err != nil { + return false, err + } + + if cmp > 0 { + return true, nil + } + + if cmp < 0 { + return false, nil + } + } + return false, nil +} + +// equal, return 0 +// greater, return 1 +// less, return -1 +func parseAndEvalBoolExpr(ctx expression.BuildContext, l, r string, colInfo *model.ColumnInfo, tbInfo *model.TableInfo) (int64, error) { + lexpr, err := expression.ParseSimpleExpr(ctx, l, expression.WithTableInfo("", tbInfo), expression.WithCastExprTo(&colInfo.FieldType)) + if err != nil { + return 0, err + } + rexpr, err := expression.ParseSimpleExpr(ctx, r, expression.WithTableInfo("", tbInfo), expression.WithCastExprTo(&colInfo.FieldType)) + if err != nil { + return 0, err + } + + e, err := expression.NewFunctionBase(ctx, ast.EQ, field_types.NewFieldType(mysql.TypeLonglong), lexpr, rexpr) + if err != nil { + return 0, err + } + e.SetCharsetAndCollation(colInfo.GetCharset(), colInfo.GetCollate()) + res, _, err1 := e.EvalInt(ctx.GetEvalCtx(), chunk.Row{}) + if err1 != nil { + return 0, err1 + } + if res == 1 { + return 0, nil + } + + e, err = expression.NewFunctionBase(ctx, ast.GT, field_types.NewFieldType(mysql.TypeLonglong), lexpr, rexpr) + if err != nil { + return 0, err + } + e.SetCharsetAndCollation(colInfo.GetCharset(), colInfo.GetCollate()) + res, _, err1 = e.EvalInt(ctx.GetEvalCtx(), chunk.Row{}) + if err1 != nil { + return 0, err1 + } + if res > 0 { + return 1, nil + } + return -1, nil +} + +// checkPartitionByList checks validity of a "BY LIST" partition. +func checkPartitionByList(ctx sessionctx.Context, tbInfo *model.TableInfo) error { + return checkListPartitionValue(ctx.GetExprCtx(), tbInfo) +} From 4a4ea866b904bb7948a870bb054798250146f257 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Thu, 1 Aug 2024 18:37:48 +0800 Subject: [PATCH 08/13] split --- pkg/ddl/executor.go | 622 ------------------------------------ pkg/ddl/job_submitter.go | 665 +++++++++++++++++++++++++++++++++++++++ 2 files changed, 665 insertions(+), 622 deletions(-) create mode 100644 pkg/ddl/job_submitter.go diff --git a/pkg/ddl/executor.go b/pkg/ddl/executor.go index 8db859a586669..778fb3a031692 100644 --- a/pkg/ddl/executor.go +++ b/pkg/ddl/executor.go @@ -23,7 +23,6 @@ import ( "context" "fmt" "math" - "strconv" "strings" "sync" "sync/atomic" @@ -32,7 +31,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/ddl/label" "github.com/pingcap/tidb/pkg/ddl/logutil" @@ -73,7 +71,6 @@ import ( "github.com/pingcap/tidb/pkg/util/intest" "github.com/pingcap/tidb/pkg/util/mathutil" "github.com/pingcap/tidb/pkg/util/stringutil" - tikv "github.com/tikv/client-go/v2/kv" "github.com/tikv/client-go/v2/oracle" clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" @@ -1295,77 +1292,6 @@ func (e *executor) BatchCreateTableWithInfo(ctx sessionctx.Context, return nil } -// buildQueryStringFromJobs takes a slice of Jobs and concatenates their -// queries into a single query string. -// Each query is separated by a semicolon and a space. -// Trailing spaces are removed from each query, and a semicolon is appended -// if it's not already present. -func buildQueryStringFromJobs(jobs []*JobWrapper) string { - var queryBuilder strings.Builder - for i, job := range jobs { - q := strings.TrimSpace(job.Query) - if !strings.HasSuffix(q, ";") { - q += ";" - } - queryBuilder.WriteString(q) - - if i < len(jobs)-1 { - queryBuilder.WriteString(" ") - } - } - return queryBuilder.String() -} - -// mergeCreateTableJobsOfSameSchema combine CreateTableJobs to BatchCreateTableJob. -func mergeCreateTableJobsOfSameSchema(jobWs []*JobWrapper) (*model.Job, error) { - if len(jobWs) == 0 { - return nil, errors.Trace(fmt.Errorf("expect non-empty jobs")) - } - - var combinedJob *model.Job - - args := make([]*model.TableInfo, 0, len(jobWs)) - involvingSchemaInfo := make([]model.InvolvingSchemaInfo, 0, len(jobWs)) - var foreignKeyChecks bool - - // if there is any duplicated table name - duplication := make(map[string]struct{}) - for _, job := range jobWs { - if combinedJob == nil { - combinedJob = job.Clone() - combinedJob.Type = model.ActionCreateTables - combinedJob.Args = combinedJob.Args[:0] - foreignKeyChecks = job.Args[1].(bool) - } - // append table job args - info, ok := job.Args[0].(*model.TableInfo) - if !ok { - return nil, errors.Trace(fmt.Errorf("expect model.TableInfo, but got %T", job.Args[0])) - } - args = append(args, info) - - if _, ok := duplication[info.Name.L]; ok { - // return err even if create table if not exists - return nil, infoschema.ErrTableExists.FastGenByArgs("can not batch create tables with same name") - } - - duplication[info.Name.L] = struct{}{} - - involvingSchemaInfo = append(involvingSchemaInfo, - model.InvolvingSchemaInfo{ - Database: job.SchemaName, - Table: info.Name.L, - }) - } - - combinedJob.Args = append(combinedJob.Args, args) - combinedJob.Args = append(combinedJob.Args, foreignKeyChecks) - combinedJob.InvolvingSchemaInfo = involvingSchemaInfo - combinedJob.Query = buildQueryStringFromJobs(jobWs) - - return combinedJob, nil -} - func (e *executor) CreatePlacementPolicyWithInfo(ctx sessionctx.Context, policy *model.PolicyInfo, onExist OnExist) error { if checkIgnorePlacementDDL(ctx) { return nil @@ -6540,497 +6466,11 @@ func (e *executor) initJobDoneCh(jobID int64) { e.ddlJobDoneChMap.Store(jobID, make(chan struct{}, 1)) } -func (e *executor) limitDDLJobs() { - defer util.Recover(metrics.LabelDDL, "limitDDLJobs", nil, true) - - jobWs := make([]*JobWrapper, 0, batchAddingJobs) - ch := e.limitJobCh - for { - select { - // the channel is never closed - case jobW := <-ch: - jobWs = jobWs[:0] - failpoint.InjectCall("afterGetJobFromLimitCh", ch) - jobLen := len(ch) - jobWs = append(jobWs, jobW) - for i := 0; i < jobLen; i++ { - jobWs = append(jobWs, <-ch) - } - e.addBatchDDLJobs(jobWs) - case <-e.ctx.Done(): - return - } - } -} - -func (e *executor) notifyNewJobSubmitted(ch chan struct{}, etcdPath string, jobID int64, jobType string) { - // If the workers don't run, we needn't notify workers. - // TODO: It does not affect informing the backfill worker. - if !config.GetGlobalConfig().Instance.TiDBEnableDDL.Load() { - return - } - if e.ownerManager.IsOwner() { - asyncNotify(ch) - } else { - e.notifyNewJobByEtcd(etcdPath, jobID, jobType) - } -} - -func (e *executor) notifyNewJobByEtcd(etcdPath string, jobID int64, jobType string) { - if e.etcdCli == nil { - return - } - - jobIDStr := strconv.FormatInt(jobID, 10) - timeStart := time.Now() - err := ddlutil.PutKVToEtcd(e.ctx, e.etcdCli, 1, etcdPath, jobIDStr) - if err != nil { - logutil.DDLLogger().Info("notify handling DDL job failed", - zap.String("etcdPath", etcdPath), - zap.Int64("jobID", jobID), - zap.String("type", jobType), - zap.Error(err)) - } - metrics.DDLWorkerHistogram.WithLabelValues(metrics.WorkerNotifyDDLJob, jobType, metrics.RetLabel(err)).Observe(time.Since(timeStart).Seconds()) -} - func (e *executor) deliverJobTask(task *JobWrapper) { // TODO this might block forever, as the consumer part considers context cancel. e.limitJobCh <- task } -// addBatchDDLJobs gets global job IDs and puts the DDL jobs in the DDL queue. -func (e *executor) addBatchDDLJobs(jobWs []*JobWrapper) { - startTime := time.Now() - var ( - err error - newWs []*JobWrapper - ) - // DDLForce2Queue is a flag to tell DDL worker to always push the job to the DDL queue. - toTable := !variable.DDLForce2Queue.Load() - fastCreate := variable.EnableFastCreateTable.Load() - if toTable { - if fastCreate { - newWs, err = mergeCreateTableJobs(jobWs) - if err != nil { - logutil.DDLLogger().Warn("failed to merge create table jobs", zap.Error(err)) - } else { - jobWs = newWs - } - } - err = e.addBatchDDLJobs2Table(jobWs) - } else { - err = e.addBatchDDLJobs2Queue(jobWs) - } - var jobs string - for _, jobW := range jobWs { - if err == nil { - err = jobW.cacheErr - } - jobW.NotifyResult(err) - jobs += jobW.Job.String() + "; " - metrics.DDLWorkerHistogram.WithLabelValues(metrics.WorkerAddDDLJob, jobW.Job.Type.String(), - metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) - } - if err != nil { - logutil.DDLLogger().Warn("add DDL jobs failed", zap.String("jobs", jobs), zap.Error(err)) - } else { - logutil.DDLLogger().Info("add DDL jobs", - zap.Int("batch count", len(jobWs)), - zap.String("jobs", jobs), - zap.Bool("table", toTable), - zap.Bool("fast_create", fastCreate)) - } -} - -func (e *executor) addBatchDDLJobs2Queue(jobWs []*JobWrapper) error { - ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) - // lock to reduce conflict - e.globalIDLock.Lock() - defer e.globalIDLock.Unlock() - return kv.RunInNewTxn(ctx, e.store, true, func(_ context.Context, txn kv.Transaction) error { - t := meta.NewMeta(txn) - - count := getRequiredGIDCount(jobWs) - ids, err := t.GenGlobalIDs(count) - if err != nil { - return errors.Trace(err) - } - assignGIDsForJobs(jobWs, ids) - - if err := e.checkFlashbackJobInQueue(t); err != nil { - return errors.Trace(err) - } - - for _, jobW := range jobWs { - job := jobW.Job - job.Version = currentVersion - job.StartTS = txn.StartTS() - setJobStateToQueueing(job) - if err = buildJobDependence(t, job); err != nil { - return errors.Trace(err) - } - jobListKey := meta.DefaultJobListKey - if job.MayNeedReorg() { - jobListKey = meta.AddIndexJobListKey - } - if err = t.EnQueueDDLJob(job, jobListKey); err != nil { - return errors.Trace(err) - } - } - failpoint.Inject("mockAddBatchDDLJobsErr", func(val failpoint.Value) { - if val.(bool) { - failpoint.Return(errors.Errorf("mockAddBatchDDLJobsErr")) - } - }) - return nil - }) -} - -func (*executor) checkFlashbackJobInQueue(t *meta.Meta) error { - jobs, err := t.GetAllDDLJobsInQueue(meta.DefaultJobListKey) - if err != nil { - return errors.Trace(err) - } - for _, job := range jobs { - if job.Type == model.ActionFlashbackCluster { - return errors.Errorf("Can't add ddl job, have flashback cluster job") - } - } - return nil -} - -// addBatchDDLJobs2Table gets global job IDs and puts the DDL jobs in the DDL job table. -func (e *executor) addBatchDDLJobs2Table(jobWs []*JobWrapper) error { - var err error - - if len(jobWs) == 0 { - return nil - } - - ctx := kv.WithInternalSourceType(e.ctx, kv.InternalTxnDDL) - se, err := e.sessPool.Get() - if err != nil { - return errors.Trace(err) - } - defer e.sessPool.Put(se) - found, err := e.sysTblMgr.HasFlashbackClusterJob(ctx, e.minJobIDRefresher.GetCurrMinJobID()) - if err != nil { - return errors.Trace(err) - } - if found { - return errors.Errorf("Can't add ddl job, have flashback cluster job") - } - - var ( - startTS = uint64(0) - bdrRole = string(ast.BDRRoleNone) - ) - - err = kv.RunInNewTxn(ctx, e.store, true, func(_ context.Context, txn kv.Transaction) error { - t := meta.NewMeta(txn) - - bdrRole, err = t.GetBDRRole() - if err != nil { - return errors.Trace(err) - } - startTS = txn.StartTS() - - if variable.DDLForce2Queue.Load() { - if err := e.checkFlashbackJobInQueue(t); err != nil { - return err - } - } - - return nil - }) - if err != nil { - return errors.Trace(err) - } - - for _, jobW := range jobWs { - job := jobW.Job - job.Version = currentVersion - job.StartTS = startTS - job.BDRRole = bdrRole - - // BDR mode only affects the DDL not from CDC - if job.CDCWriteSource == 0 && bdrRole != string(ast.BDRRoleNone) { - if job.Type == model.ActionMultiSchemaChange && job.MultiSchemaInfo != nil { - for _, subJob := range job.MultiSchemaInfo.SubJobs { - if ast.DeniedByBDR(ast.BDRRole(bdrRole), subJob.Type, job) { - return dbterror.ErrBDRRestrictedDDL.FastGenByArgs(bdrRole) - } - } - } else if ast.DeniedByBDR(ast.BDRRole(bdrRole), job.Type, job) { - return dbterror.ErrBDRRestrictedDDL.FastGenByArgs(bdrRole) - } - } - - setJobStateToQueueing(job) - - if e.stateSyncer.IsUpgradingState() && !hasSysDB(job) { - if err = pauseRunningJob(sess.NewSession(se), job, model.AdminCommandBySystem); err != nil { - logutil.DDLUpgradingLogger().Warn("pause user DDL by system failed", zap.Stringer("job", job), zap.Error(err)) - jobW.cacheErr = err - continue - } - logutil.DDLUpgradingLogger().Info("pause user DDL by system successful", zap.Stringer("job", job)) - } - } - - se.GetSessionVars().SetDiskFullOpt(kvrpcpb.DiskFullOpt_AllowedOnAlmostFull) - ddlSe := sess.NewSession(se) - if err = GenGIDAndInsertJobsWithRetry(ctx, ddlSe, jobWs); err != nil { - return errors.Trace(err) - } - for _, jobW := range jobWs { - e.initJobDoneCh(jobW.ID) - } - - return nil -} - -// GenGIDAndInsertJobsWithRetry generate job related global ID and inserts DDL jobs to the DDL job -// table with retry. job id allocation and job insertion are in the same transaction, -// as we want to make sure DDL jobs are inserted in id order, then we can query from -// a min job ID when scheduling DDL jobs to mitigate https://github.com/pingcap/tidb/issues/52905. -// so this function has side effect, it will set table/db/job id of 'jobs'. -func GenGIDAndInsertJobsWithRetry(ctx context.Context, ddlSe *sess.Session, jobWs []*JobWrapper) error { - count := getRequiredGIDCount(jobWs) - return genGIDAndCallWithRetry(ctx, ddlSe, count, func(ids []int64) error { - failpoint.Inject("mockGenGlobalIDFail", func(val failpoint.Value) { - if val.(bool) { - failpoint.Return(errors.New("gofail genGlobalIDs error")) - } - }) - assignGIDsForJobs(jobWs, ids) - injectModifyJobArgFailPoint(jobWs) - return insertDDLJobs2Table(ctx, ddlSe, jobWs...) - }) -} - -// getRequiredGIDCount returns the count of required global IDs for the jobs. it's calculated -// as: the count of jobs + the count of IDs for the jobs which do NOT have pre-allocated ID. -func getRequiredGIDCount(jobWs []*JobWrapper) int { - count := len(jobWs) - idCountForTable := func(info *model.TableInfo) int { - c := 1 - if partitionInfo := info.GetPartitionInfo(); partitionInfo != nil { - c += len(partitionInfo.Definitions) - } - return c - } - for _, jobW := range jobWs { - if jobW.IDAllocated { - continue - } - switch jobW.Type { - case model.ActionCreateView, model.ActionCreateSequence, model.ActionCreateTable: - info := jobW.Args[0].(*model.TableInfo) - count += idCountForTable(info) - case model.ActionCreateTables: - infos := jobW.Args[0].([]*model.TableInfo) - for _, info := range infos { - count += idCountForTable(info) - } - case model.ActionCreateSchema: - count++ - } - // TODO support other type of jobs - } - return count -} - -// assignGIDsForJobs should be used with getRequiredGIDCount, and len(ids) must equal -// what getRequiredGIDCount returns. -func assignGIDsForJobs(jobWs []*JobWrapper, ids []int64) { - idx := 0 - - assignIDsForTable := func(info *model.TableInfo) { - info.ID = ids[idx] - idx++ - if partitionInfo := info.GetPartitionInfo(); partitionInfo != nil { - for i := range partitionInfo.Definitions { - partitionInfo.Definitions[i].ID = ids[idx] - idx++ - } - } - } - for _, jobW := range jobWs { - switch jobW.Type { - case model.ActionCreateView, model.ActionCreateSequence, model.ActionCreateTable: - info := jobW.Args[0].(*model.TableInfo) - if !jobW.IDAllocated { - assignIDsForTable(info) - } - jobW.TableID = info.ID - case model.ActionCreateTables: - if !jobW.IDAllocated { - infos := jobW.Args[0].([]*model.TableInfo) - for _, info := range infos { - assignIDsForTable(info) - } - } - case model.ActionCreateSchema: - dbInfo := jobW.Args[0].(*model.DBInfo) - if !jobW.IDAllocated { - dbInfo.ID = ids[idx] - idx++ - } - jobW.SchemaID = dbInfo.ID - } - // TODO support other type of jobs - jobW.ID = ids[idx] - idx++ - } -} - -// genGIDAndCallWithRetry generates global IDs and calls the function with retry. -// generate ID and call function runs in the same transaction. -func genGIDAndCallWithRetry(ctx context.Context, ddlSe *sess.Session, count int, fn func(ids []int64) error) error { - var resErr error - for i := uint(0); i < kv.MaxRetryCnt; i++ { - resErr = func() (err error) { - if err := ddlSe.Begin(ctx); err != nil { - return errors.Trace(err) - } - defer func() { - if err != nil { - ddlSe.Rollback() - } - }() - txn, err := ddlSe.Txn() - if err != nil { - return errors.Trace(err) - } - txn.SetOption(kv.Pessimistic, true) - forUpdateTS, err := lockGlobalIDKey(ctx, ddlSe, txn) - if err != nil { - return errors.Trace(err) - } - txn.GetSnapshot().SetOption(kv.SnapshotTS, forUpdateTS) - - m := meta.NewMeta(txn) - ids, err := m.GenGlobalIDs(count) - if err != nil { - return errors.Trace(err) - } - if err = fn(ids); err != nil { - return errors.Trace(err) - } - return ddlSe.Commit(ctx) - }() - - if resErr != nil && kv.IsTxnRetryableError(resErr) { - logutil.DDLLogger().Warn("insert job meet retryable error", zap.Error(resErr)) - kv.BackOff(i) - continue - } - break - } - return resErr -} - -// lockGlobalIDKey locks the global ID key in the meta store. it keeps trying if -// meet write conflict, we cannot have a fixed retry count for this error, see this -// https://github.com/pingcap/tidb/issues/27197#issuecomment-2216315057. -// this part is same as how we implement pessimistic + repeatable read isolation -// level in SQL executor, see doLockKeys. -// NextGlobalID is a meta key, so we cannot use "select xx for update", if we store -// it into a table row or using advisory lock, we will depends on a system table -// that is created by us, cyclic. although we can create a system table without using -// DDL logic, we will only consider change it when we have data dictionary and keep -// it this way now. -// TODO maybe we can unify the lock mechanism with SQL executor in the future, or -// implement it inside TiKV client-go. -func lockGlobalIDKey(ctx context.Context, ddlSe *sess.Session, txn kv.Transaction) (uint64, error) { - var ( - iteration uint - forUpdateTs = txn.StartTS() - ver kv.Version - err error - ) - waitTime := ddlSe.GetSessionVars().LockWaitTimeout - m := meta.NewMeta(txn) - idKey := m.GlobalIDKey() - for { - lockCtx := tikv.NewLockCtx(forUpdateTs, waitTime, time.Now()) - err = txn.LockKeys(ctx, lockCtx, idKey) - if err == nil || !terror.ErrorEqual(kv.ErrWriteConflict, err) { - break - } - // ErrWriteConflict contains a conflict-commit-ts in most case, but it cannot - // be used as forUpdateTs, see comments inside handleAfterPessimisticLockError - ver, err = ddlSe.GetStore().CurrentVersion(oracle.GlobalTxnScope) - if err != nil { - break - } - forUpdateTs = ver.Ver - - kv.BackOff(iteration) - // avoid it keep growing and overflow. - iteration = min(iteration+1, math.MaxInt) - } - return forUpdateTs, err -} - -// mergeCreateTableJobs merges CreateTable jobs to CreateTables. -func mergeCreateTableJobs(jobWs []*JobWrapper) ([]*JobWrapper, error) { - if len(jobWs) <= 1 { - return jobWs, nil - } - resJobWs := make([]*JobWrapper, 0, len(jobWs)) - mergeableJobWs := make(map[string][]*JobWrapper, len(jobWs)) - for _, jobW := range jobWs { - // we don't merge jobs with ID pre-allocated. - if jobW.Type != model.ActionCreateTable || jobW.IDAllocated { - resJobWs = append(resJobWs, jobW) - continue - } - // ActionCreateTables doesn't support foreign key now. - tbInfo, ok := jobW.Args[0].(*model.TableInfo) - if !ok || len(tbInfo.ForeignKeys) > 0 { - resJobWs = append(resJobWs, jobW) - continue - } - // CreateTables only support tables of same schema now. - mergeableJobWs[jobW.Job.SchemaName] = append(mergeableJobWs[jobW.Job.SchemaName], jobW) - } - - for schema, jobs := range mergeableJobWs { - total := len(jobs) - if total <= 1 { - resJobWs = append(resJobWs, jobs...) - continue - } - const maxBatchSize = 8 - batchCount := (total + maxBatchSize - 1) / maxBatchSize - start := 0 - for _, batchSize := range mathutil.Divide2Batches(total, batchCount) { - batch := jobs[start : start+batchSize] - job, err := mergeCreateTableJobsOfSameSchema(batch) - if err != nil { - return nil, err - } - start += batchSize - logutil.DDLLogger().Info("merge create table jobs", zap.String("schema", schema), - zap.Int("total", total), zap.Int("batch_size", batchSize)) - - newJobW := &JobWrapper{ - Job: job, - ResultCh: make([]chan jobSubmitResult, 0, batchSize), - } - // merge the result channels. - for _, j := range batch { - newJobW.ResultCh = append(newJobW.ResultCh, j.ResultCh...) - } - resJobWs = append(resJobWs, newJobW) - } - } - return resJobWs, nil -} - func updateTickerInterval(ticker *time.Ticker, lease time.Duration, action model.ActionType, i int) *time.Ticker { interval, changed := getJobCheckInterval(action, i) if !changed { @@ -7098,68 +6538,6 @@ func getJobCheckInterval(action model.ActionType, i int) (time.Duration, bool) { } } -// TODO this failpoint is only checking how job scheduler handle -// corrupted job args, we should test it there by UT, not here. -func injectModifyJobArgFailPoint(jobWs []*JobWrapper) { - failpoint.Inject("MockModifyJobArg", func(val failpoint.Value) { - if val.(bool) { - for _, jobW := range jobWs { - job := jobW.Job - // Corrupt the DDL job argument. - if job.Type == model.ActionMultiSchemaChange { - if len(job.MultiSchemaInfo.SubJobs) > 0 && len(job.MultiSchemaInfo.SubJobs[0].Args) > 0 { - job.MultiSchemaInfo.SubJobs[0].Args[0] = 1 - } - } else if len(job.Args) > 0 { - job.Args[0] = 1 - } - } - } - }) -} - -func setJobStateToQueueing(job *model.Job) { - if job.Type == model.ActionMultiSchemaChange && job.MultiSchemaInfo != nil { - for _, sub := range job.MultiSchemaInfo.SubJobs { - sub.State = model.JobStateQueueing - } - } - job.State = model.JobStateQueueing -} - -// buildJobDependence sets the curjob's dependency-ID. -// The dependency-job's ID must less than the current job's ID, and we need the largest one in the list. -func buildJobDependence(t *meta.Meta, curJob *model.Job) error { - // Jobs in the same queue are ordered. If we want to find a job's dependency-job, we need to look for - // it from the other queue. So if the job is "ActionAddIndex" job, we need find its dependency-job from DefaultJobList. - jobListKey := meta.DefaultJobListKey - if !curJob.MayNeedReorg() { - jobListKey = meta.AddIndexJobListKey - } - jobs, err := t.GetAllDDLJobsInQueue(jobListKey) - if err != nil { - return errors.Trace(err) - } - - for _, job := range jobs { - if curJob.ID < job.ID { - continue - } - isDependent, err := curJob.IsDependentOn(job) - if err != nil { - return errors.Trace(err) - } - if isDependent { - logutil.DDLLogger().Info("current DDL job depends on other job", - zap.Stringer("currentJob", curJob), - zap.Stringer("dependentJob", job)) - curJob.DependencyID = job.ID - break - } - } - return nil -} - // NewDDLReorgMeta create a DDL ReorgMeta. func NewDDLReorgMeta(ctx sessionctx.Context) *model.DDLReorgMeta { tzName, tzOffset := ddlutil.GetTimeZone(ctx) diff --git a/pkg/ddl/job_submitter.go b/pkg/ddl/job_submitter.go new file mode 100644 index 0000000000000..c45f8faac2887 --- /dev/null +++ b/pkg/ddl/job_submitter.go @@ -0,0 +1,665 @@ +// 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 ddl + +import ( + "context" + "fmt" + "math" + "strconv" + "strings" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/tidb/pkg/config" + "github.com/pingcap/tidb/pkg/ddl/logutil" + sess "github.com/pingcap/tidb/pkg/ddl/session" + ddlutil "github.com/pingcap/tidb/pkg/ddl/util" + "github.com/pingcap/tidb/pkg/infoschema" + "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/meta" + "github.com/pingcap/tidb/pkg/metrics" + "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/terror" + "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/util" + "github.com/pingcap/tidb/pkg/util/dbterror" + "github.com/pingcap/tidb/pkg/util/mathutil" + tikv "github.com/tikv/client-go/v2/kv" + "github.com/tikv/client-go/v2/oracle" + "go.uber.org/zap" +) + +func (e *executor) limitDDLJobs() { + defer util.Recover(metrics.LabelDDL, "limitDDLJobs", nil, true) + + jobWs := make([]*JobWrapper, 0, batchAddingJobs) + ch := e.limitJobCh + for { + select { + // the channel is never closed + case jobW := <-ch: + jobWs = jobWs[:0] + failpoint.InjectCall("afterGetJobFromLimitCh", ch) + jobLen := len(ch) + jobWs = append(jobWs, jobW) + for i := 0; i < jobLen; i++ { + jobWs = append(jobWs, <-ch) + } + e.addBatchDDLJobs(jobWs) + case <-e.ctx.Done(): + return + } + } +} + +// addBatchDDLJobs gets global job IDs and puts the DDL jobs in the DDL queue. +func (e *executor) addBatchDDLJobs(jobWs []*JobWrapper) { + startTime := time.Now() + var ( + err error + newWs []*JobWrapper + ) + // DDLForce2Queue is a flag to tell DDL worker to always push the job to the DDL queue. + toTable := !variable.DDLForce2Queue.Load() + fastCreate := variable.EnableFastCreateTable.Load() + if toTable { + if fastCreate { + newWs, err = mergeCreateTableJobs(jobWs) + if err != nil { + logutil.DDLLogger().Warn("failed to merge create table jobs", zap.Error(err)) + } else { + jobWs = newWs + } + } + err = e.addBatchDDLJobs2Table(jobWs) + } else { + err = e.addBatchDDLJobs2Queue(jobWs) + } + var jobs string + for _, jobW := range jobWs { + if err == nil { + err = jobW.cacheErr + } + jobW.NotifyResult(err) + jobs += jobW.Job.String() + "; " + metrics.DDLWorkerHistogram.WithLabelValues(metrics.WorkerAddDDLJob, jobW.Job.Type.String(), + metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) + } + if err != nil { + logutil.DDLLogger().Warn("add DDL jobs failed", zap.String("jobs", jobs), zap.Error(err)) + } else { + logutil.DDLLogger().Info("add DDL jobs", + zap.Int("batch count", len(jobWs)), + zap.String("jobs", jobs), + zap.Bool("table", toTable), + zap.Bool("fast_create", fastCreate)) + } +} + +// mergeCreateTableJobs merges CreateTable jobs to CreateTables. +func mergeCreateTableJobs(jobWs []*JobWrapper) ([]*JobWrapper, error) { + if len(jobWs) <= 1 { + return jobWs, nil + } + resJobWs := make([]*JobWrapper, 0, len(jobWs)) + mergeableJobWs := make(map[string][]*JobWrapper, len(jobWs)) + for _, jobW := range jobWs { + // we don't merge jobs with ID pre-allocated. + if jobW.Type != model.ActionCreateTable || jobW.IDAllocated { + resJobWs = append(resJobWs, jobW) + continue + } + // ActionCreateTables doesn't support foreign key now. + tbInfo, ok := jobW.Args[0].(*model.TableInfo) + if !ok || len(tbInfo.ForeignKeys) > 0 { + resJobWs = append(resJobWs, jobW) + continue + } + // CreateTables only support tables of same schema now. + mergeableJobWs[jobW.Job.SchemaName] = append(mergeableJobWs[jobW.Job.SchemaName], jobW) + } + + for schema, jobs := range mergeableJobWs { + total := len(jobs) + if total <= 1 { + resJobWs = append(resJobWs, jobs...) + continue + } + const maxBatchSize = 8 + batchCount := (total + maxBatchSize - 1) / maxBatchSize + start := 0 + for _, batchSize := range mathutil.Divide2Batches(total, batchCount) { + batch := jobs[start : start+batchSize] + job, err := mergeCreateTableJobsOfSameSchema(batch) + if err != nil { + return nil, err + } + start += batchSize + logutil.DDLLogger().Info("merge create table jobs", zap.String("schema", schema), + zap.Int("total", total), zap.Int("batch_size", batchSize)) + + newJobW := &JobWrapper{ + Job: job, + ResultCh: make([]chan jobSubmitResult, 0, batchSize), + } + // merge the result channels. + for _, j := range batch { + newJobW.ResultCh = append(newJobW.ResultCh, j.ResultCh...) + } + resJobWs = append(resJobWs, newJobW) + } + } + return resJobWs, nil +} + +// buildQueryStringFromJobs takes a slice of Jobs and concatenates their +// queries into a single query string. +// Each query is separated by a semicolon and a space. +// Trailing spaces are removed from each query, and a semicolon is appended +// if it's not already present. +func buildQueryStringFromJobs(jobs []*JobWrapper) string { + var queryBuilder strings.Builder + for i, job := range jobs { + q := strings.TrimSpace(job.Query) + if !strings.HasSuffix(q, ";") { + q += ";" + } + queryBuilder.WriteString(q) + + if i < len(jobs)-1 { + queryBuilder.WriteString(" ") + } + } + return queryBuilder.String() +} + +// mergeCreateTableJobsOfSameSchema combine CreateTableJobs to BatchCreateTableJob. +func mergeCreateTableJobsOfSameSchema(jobWs []*JobWrapper) (*model.Job, error) { + if len(jobWs) == 0 { + return nil, errors.Trace(fmt.Errorf("expect non-empty jobs")) + } + + var combinedJob *model.Job + + args := make([]*model.TableInfo, 0, len(jobWs)) + involvingSchemaInfo := make([]model.InvolvingSchemaInfo, 0, len(jobWs)) + var foreignKeyChecks bool + + // if there is any duplicated table name + duplication := make(map[string]struct{}) + for _, job := range jobWs { + if combinedJob == nil { + combinedJob = job.Clone() + combinedJob.Type = model.ActionCreateTables + combinedJob.Args = combinedJob.Args[:0] + foreignKeyChecks = job.Args[1].(bool) + } + // append table job args + info, ok := job.Args[0].(*model.TableInfo) + if !ok { + return nil, errors.Trace(fmt.Errorf("expect model.TableInfo, but got %T", job.Args[0])) + } + args = append(args, info) + + if _, ok := duplication[info.Name.L]; ok { + // return err even if create table if not exists + return nil, infoschema.ErrTableExists.FastGenByArgs("can not batch create tables with same name") + } + + duplication[info.Name.L] = struct{}{} + + involvingSchemaInfo = append(involvingSchemaInfo, + model.InvolvingSchemaInfo{ + Database: job.SchemaName, + Table: info.Name.L, + }) + } + + combinedJob.Args = append(combinedJob.Args, args) + combinedJob.Args = append(combinedJob.Args, foreignKeyChecks) + combinedJob.InvolvingSchemaInfo = involvingSchemaInfo + combinedJob.Query = buildQueryStringFromJobs(jobWs) + + return combinedJob, nil +} + +// addBatchDDLJobs2Table gets global job IDs and puts the DDL jobs in the DDL job table. +func (e *executor) addBatchDDLJobs2Table(jobWs []*JobWrapper) error { + var err error + + if len(jobWs) == 0 { + return nil + } + + ctx := kv.WithInternalSourceType(e.ctx, kv.InternalTxnDDL) + se, err := e.sessPool.Get() + if err != nil { + return errors.Trace(err) + } + defer e.sessPool.Put(se) + found, err := e.sysTblMgr.HasFlashbackClusterJob(ctx, e.minJobIDRefresher.GetCurrMinJobID()) + if err != nil { + return errors.Trace(err) + } + if found { + return errors.Errorf("Can't add ddl job, have flashback cluster job") + } + + var ( + startTS = uint64(0) + bdrRole = string(ast.BDRRoleNone) + ) + + err = kv.RunInNewTxn(ctx, e.store, true, func(_ context.Context, txn kv.Transaction) error { + t := meta.NewMeta(txn) + + bdrRole, err = t.GetBDRRole() + if err != nil { + return errors.Trace(err) + } + startTS = txn.StartTS() + + if variable.DDLForce2Queue.Load() { + if err := e.checkFlashbackJobInQueue(t); err != nil { + return err + } + } + + return nil + }) + if err != nil { + return errors.Trace(err) + } + + for _, jobW := range jobWs { + job := jobW.Job + job.Version = currentVersion + job.StartTS = startTS + job.BDRRole = bdrRole + + // BDR mode only affects the DDL not from CDC + if job.CDCWriteSource == 0 && bdrRole != string(ast.BDRRoleNone) { + if job.Type == model.ActionMultiSchemaChange && job.MultiSchemaInfo != nil { + for _, subJob := range job.MultiSchemaInfo.SubJobs { + if ast.DeniedByBDR(ast.BDRRole(bdrRole), subJob.Type, job) { + return dbterror.ErrBDRRestrictedDDL.FastGenByArgs(bdrRole) + } + } + } else if ast.DeniedByBDR(ast.BDRRole(bdrRole), job.Type, job) { + return dbterror.ErrBDRRestrictedDDL.FastGenByArgs(bdrRole) + } + } + + setJobStateToQueueing(job) + + if e.stateSyncer.IsUpgradingState() && !hasSysDB(job) { + if err = pauseRunningJob(sess.NewSession(se), job, model.AdminCommandBySystem); err != nil { + logutil.DDLUpgradingLogger().Warn("pause user DDL by system failed", zap.Stringer("job", job), zap.Error(err)) + jobW.cacheErr = err + continue + } + logutil.DDLUpgradingLogger().Info("pause user DDL by system successful", zap.Stringer("job", job)) + } + } + + se.GetSessionVars().SetDiskFullOpt(kvrpcpb.DiskFullOpt_AllowedOnAlmostFull) + ddlSe := sess.NewSession(se) + if err = GenGIDAndInsertJobsWithRetry(ctx, ddlSe, jobWs); err != nil { + return errors.Trace(err) + } + for _, jobW := range jobWs { + e.initJobDoneCh(jobW.ID) + } + + return nil +} + +func (e *executor) addBatchDDLJobs2Queue(jobWs []*JobWrapper) error { + ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) + // lock to reduce conflict + e.globalIDLock.Lock() + defer e.globalIDLock.Unlock() + return kv.RunInNewTxn(ctx, e.store, true, func(_ context.Context, txn kv.Transaction) error { + t := meta.NewMeta(txn) + + count := getRequiredGIDCount(jobWs) + ids, err := t.GenGlobalIDs(count) + if err != nil { + return errors.Trace(err) + } + assignGIDsForJobs(jobWs, ids) + + if err := e.checkFlashbackJobInQueue(t); err != nil { + return errors.Trace(err) + } + + for _, jobW := range jobWs { + job := jobW.Job + job.Version = currentVersion + job.StartTS = txn.StartTS() + setJobStateToQueueing(job) + if err = buildJobDependence(t, job); err != nil { + return errors.Trace(err) + } + jobListKey := meta.DefaultJobListKey + if job.MayNeedReorg() { + jobListKey = meta.AddIndexJobListKey + } + if err = t.EnQueueDDLJob(job, jobListKey); err != nil { + return errors.Trace(err) + } + } + failpoint.Inject("mockAddBatchDDLJobsErr", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(errors.Errorf("mockAddBatchDDLJobsErr")) + } + }) + return nil + }) +} + +func (*executor) checkFlashbackJobInQueue(t *meta.Meta) error { + jobs, err := t.GetAllDDLJobsInQueue(meta.DefaultJobListKey) + if err != nil { + return errors.Trace(err) + } + for _, job := range jobs { + if job.Type == model.ActionFlashbackCluster { + return errors.Errorf("Can't add ddl job, have flashback cluster job") + } + } + return nil +} + +// GenGIDAndInsertJobsWithRetry generate job related global ID and inserts DDL jobs to the DDL job +// table with retry. job id allocation and job insertion are in the same transaction, +// as we want to make sure DDL jobs are inserted in id order, then we can query from +// a min job ID when scheduling DDL jobs to mitigate https://github.com/pingcap/tidb/issues/52905. +// so this function has side effect, it will set table/db/job id of 'jobs'. +func GenGIDAndInsertJobsWithRetry(ctx context.Context, ddlSe *sess.Session, jobWs []*JobWrapper) error { + count := getRequiredGIDCount(jobWs) + return genGIDAndCallWithRetry(ctx, ddlSe, count, func(ids []int64) error { + failpoint.Inject("mockGenGlobalIDFail", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(errors.New("gofail genGlobalIDs error")) + } + }) + assignGIDsForJobs(jobWs, ids) + injectModifyJobArgFailPoint(jobWs) + return insertDDLJobs2Table(ctx, ddlSe, jobWs...) + }) +} + +// getRequiredGIDCount returns the count of required global IDs for the jobs. it's calculated +// as: the count of jobs + the count of IDs for the jobs which do NOT have pre-allocated ID. +func getRequiredGIDCount(jobWs []*JobWrapper) int { + count := len(jobWs) + idCountForTable := func(info *model.TableInfo) int { + c := 1 + if partitionInfo := info.GetPartitionInfo(); partitionInfo != nil { + c += len(partitionInfo.Definitions) + } + return c + } + for _, jobW := range jobWs { + if jobW.IDAllocated { + continue + } + switch jobW.Type { + case model.ActionCreateView, model.ActionCreateSequence, model.ActionCreateTable: + info := jobW.Args[0].(*model.TableInfo) + count += idCountForTable(info) + case model.ActionCreateTables: + infos := jobW.Args[0].([]*model.TableInfo) + for _, info := range infos { + count += idCountForTable(info) + } + case model.ActionCreateSchema: + count++ + } + // TODO support other type of jobs + } + return count +} + +// assignGIDsForJobs should be used with getRequiredGIDCount, and len(ids) must equal +// what getRequiredGIDCount returns. +func assignGIDsForJobs(jobWs []*JobWrapper, ids []int64) { + idx := 0 + + assignIDsForTable := func(info *model.TableInfo) { + info.ID = ids[idx] + idx++ + if partitionInfo := info.GetPartitionInfo(); partitionInfo != nil { + for i := range partitionInfo.Definitions { + partitionInfo.Definitions[i].ID = ids[idx] + idx++ + } + } + } + for _, jobW := range jobWs { + switch jobW.Type { + case model.ActionCreateView, model.ActionCreateSequence, model.ActionCreateTable: + info := jobW.Args[0].(*model.TableInfo) + if !jobW.IDAllocated { + assignIDsForTable(info) + } + jobW.TableID = info.ID + case model.ActionCreateTables: + if !jobW.IDAllocated { + infos := jobW.Args[0].([]*model.TableInfo) + for _, info := range infos { + assignIDsForTable(info) + } + } + case model.ActionCreateSchema: + dbInfo := jobW.Args[0].(*model.DBInfo) + if !jobW.IDAllocated { + dbInfo.ID = ids[idx] + idx++ + } + jobW.SchemaID = dbInfo.ID + } + // TODO support other type of jobs + jobW.ID = ids[idx] + idx++ + } +} + +// genGIDAndCallWithRetry generates global IDs and calls the function with retry. +// generate ID and call function runs in the same transaction. +func genGIDAndCallWithRetry(ctx context.Context, ddlSe *sess.Session, count int, fn func(ids []int64) error) error { + var resErr error + for i := uint(0); i < kv.MaxRetryCnt; i++ { + resErr = func() (err error) { + if err := ddlSe.Begin(ctx); err != nil { + return errors.Trace(err) + } + defer func() { + if err != nil { + ddlSe.Rollback() + } + }() + txn, err := ddlSe.Txn() + if err != nil { + return errors.Trace(err) + } + txn.SetOption(kv.Pessimistic, true) + forUpdateTS, err := lockGlobalIDKey(ctx, ddlSe, txn) + if err != nil { + return errors.Trace(err) + } + txn.GetSnapshot().SetOption(kv.SnapshotTS, forUpdateTS) + + m := meta.NewMeta(txn) + ids, err := m.GenGlobalIDs(count) + if err != nil { + return errors.Trace(err) + } + if err = fn(ids); err != nil { + return errors.Trace(err) + } + return ddlSe.Commit(ctx) + }() + + if resErr != nil && kv.IsTxnRetryableError(resErr) { + logutil.DDLLogger().Warn("insert job meet retryable error", zap.Error(resErr)) + kv.BackOff(i) + continue + } + break + } + return resErr +} + +// lockGlobalIDKey locks the global ID key in the meta store. it keeps trying if +// meet write conflict, we cannot have a fixed retry count for this error, see this +// https://github.com/pingcap/tidb/issues/27197#issuecomment-2216315057. +// this part is same as how we implement pessimistic + repeatable read isolation +// level in SQL executor, see doLockKeys. +// NextGlobalID is a meta key, so we cannot use "select xx for update", if we store +// it into a table row or using advisory lock, we will depends on a system table +// that is created by us, cyclic. although we can create a system table without using +// DDL logic, we will only consider change it when we have data dictionary and keep +// it this way now. +// TODO maybe we can unify the lock mechanism with SQL executor in the future, or +// implement it inside TiKV client-go. +func lockGlobalIDKey(ctx context.Context, ddlSe *sess.Session, txn kv.Transaction) (uint64, error) { + var ( + iteration uint + forUpdateTs = txn.StartTS() + ver kv.Version + err error + ) + waitTime := ddlSe.GetSessionVars().LockWaitTimeout + m := meta.NewMeta(txn) + idKey := m.GlobalIDKey() + for { + lockCtx := tikv.NewLockCtx(forUpdateTs, waitTime, time.Now()) + err = txn.LockKeys(ctx, lockCtx, idKey) + if err == nil || !terror.ErrorEqual(kv.ErrWriteConflict, err) { + break + } + // ErrWriteConflict contains a conflict-commit-ts in most case, but it cannot + // be used as forUpdateTs, see comments inside handleAfterPessimisticLockError + ver, err = ddlSe.GetStore().CurrentVersion(oracle.GlobalTxnScope) + if err != nil { + break + } + forUpdateTs = ver.Ver + + kv.BackOff(iteration) + // avoid it keep growing and overflow. + iteration = min(iteration+1, math.MaxInt) + } + return forUpdateTs, err +} + +// TODO this failpoint is only checking how job scheduler handle +// corrupted job args, we should test it there by UT, not here. +func injectModifyJobArgFailPoint(jobWs []*JobWrapper) { + failpoint.Inject("MockModifyJobArg", func(val failpoint.Value) { + if val.(bool) { + for _, jobW := range jobWs { + job := jobW.Job + // Corrupt the DDL job argument. + if job.Type == model.ActionMultiSchemaChange { + if len(job.MultiSchemaInfo.SubJobs) > 0 && len(job.MultiSchemaInfo.SubJobs[0].Args) > 0 { + job.MultiSchemaInfo.SubJobs[0].Args[0] = 1 + } + } else if len(job.Args) > 0 { + job.Args[0] = 1 + } + } + } + }) +} + +func setJobStateToQueueing(job *model.Job) { + if job.Type == model.ActionMultiSchemaChange && job.MultiSchemaInfo != nil { + for _, sub := range job.MultiSchemaInfo.SubJobs { + sub.State = model.JobStateQueueing + } + } + job.State = model.JobStateQueueing +} + +// buildJobDependence sets the curjob's dependency-ID. +// The dependency-job's ID must less than the current job's ID, and we need the largest one in the list. +func buildJobDependence(t *meta.Meta, curJob *model.Job) error { + // Jobs in the same queue are ordered. If we want to find a job's dependency-job, we need to look for + // it from the other queue. So if the job is "ActionAddIndex" job, we need find its dependency-job from DefaultJobList. + jobListKey := meta.DefaultJobListKey + if !curJob.MayNeedReorg() { + jobListKey = meta.AddIndexJobListKey + } + jobs, err := t.GetAllDDLJobsInQueue(jobListKey) + if err != nil { + return errors.Trace(err) + } + + for _, job := range jobs { + if curJob.ID < job.ID { + continue + } + isDependent, err := curJob.IsDependentOn(job) + if err != nil { + return errors.Trace(err) + } + if isDependent { + logutil.DDLLogger().Info("current DDL job depends on other job", + zap.Stringer("currentJob", curJob), + zap.Stringer("dependentJob", job)) + curJob.DependencyID = job.ID + break + } + } + return nil +} + +func (e *executor) notifyNewJobSubmitted(ch chan struct{}, etcdPath string, jobID int64, jobType string) { + // If the workers don't run, we needn't notify workers. + // TODO: It does not affect informing the backfill worker. + if !config.GetGlobalConfig().Instance.TiDBEnableDDL.Load() { + return + } + if e.ownerManager.IsOwner() { + asyncNotify(ch) + } else { + e.notifyNewJobByEtcd(etcdPath, jobID, jobType) + } +} + +func (e *executor) notifyNewJobByEtcd(etcdPath string, jobID int64, jobType string) { + if e.etcdCli == nil { + return + } + + jobIDStr := strconv.FormatInt(jobID, 10) + timeStart := time.Now() + err := ddlutil.PutKVToEtcd(e.ctx, e.etcdCli, 1, etcdPath, jobIDStr) + if err != nil { + logutil.DDLLogger().Info("notify handling DDL job failed", + zap.String("etcdPath", etcdPath), + zap.Int64("jobID", jobID), + zap.String("type", jobType), + zap.Error(err)) + } + metrics.DDLWorkerHistogram.WithLabelValues(metrics.WorkerNotifyDDLJob, jobType, metrics.RetLabel(err)).Observe(time.Since(timeStart).Seconds()) +} From 1f6b5215005b45609c21a88d3898f5f5316cca80 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Thu, 1 Aug 2024 18:38:47 +0800 Subject: [PATCH 09/13] change --- pkg/ddl/BUILD.bazel | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pkg/ddl/BUILD.bazel b/pkg/ddl/BUILD.bazel index bbf067dabbc18..0918151c0681b 100644 --- a/pkg/ddl/BUILD.bazel +++ b/pkg/ddl/BUILD.bazel @@ -11,6 +11,7 @@ package_group( go_library( name = "ddl", srcs = [ + "add_column.go", "backfilling.go", "backfilling_clean_s3.go", "backfilling_dist_executor.go", @@ -26,6 +27,7 @@ go_library( "column.go", "constant.go", "constraint.go", + "create_table.go", "ddl.go", "ddl_algorithm.go", "ddl_history.go", @@ -43,8 +45,10 @@ go_library( "index_cop.go", "index_merge_tmp.go", "job_scheduler.go", + "job_submitter.go", "job_worker.go", "mock.go", + "modify_column.go", "multi_schema_change.go", "options.go", "partition.go", From d95b5f9733c767223bd228b0fdd94df73da6d381 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Thu, 1 Aug 2024 18:46:29 +0800 Subject: [PATCH 10/13] change --- pkg/ddl/column.go | 679 -------------------------------------- pkg/ddl/modify_column.go | 683 +++++++++++++++++++++++++++++++++++++++ 2 files changed, 683 insertions(+), 679 deletions(-) diff --git a/pkg/ddl/column.go b/pkg/ddl/column.go index f58e784cf6c64..5b889c8f73464 100644 --- a/pkg/ddl/column.go +++ b/pkg/ddl/column.go @@ -28,25 +28,20 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/ddl/logutil" - sess "github.com/pingcap/tidb/pkg/ddl/session" "github.com/pingcap/tidb/pkg/expression" - exprctx "github.com/pingcap/tidb/pkg/expression/context" "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/metrics" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" - statsutil "github.com/pingcap/tidb/pkg/statistics/handle/util" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/types" - "github.com/pingcap/tidb/pkg/util" contextutil "github.com/pingcap/tidb/pkg/util/context" "github.com/pingcap/tidb/pkg/util/dbterror" decoder "github.com/pingcap/tidb/pkg/util/rowDecoder" @@ -299,260 +294,6 @@ func onSetDefaultValue(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ er return updateColumnDefaultValue(d, t, job, newCol, &newCol.Name) } -func needChangeColumnData(oldCol, newCol *model.ColumnInfo) bool { - toUnsigned := mysql.HasUnsignedFlag(newCol.GetFlag()) - originUnsigned := mysql.HasUnsignedFlag(oldCol.GetFlag()) - needTruncationOrToggleSign := func() bool { - return (newCol.GetFlen() > 0 && (newCol.GetFlen() < oldCol.GetFlen() || newCol.GetDecimal() < oldCol.GetDecimal())) || - (toUnsigned != originUnsigned) - } - // Ignore the potential max display length represented by integer's flen, use default flen instead. - defaultOldColFlen, _ := mysql.GetDefaultFieldLengthAndDecimal(oldCol.GetType()) - defaultNewColFlen, _ := mysql.GetDefaultFieldLengthAndDecimal(newCol.GetType()) - needTruncationOrToggleSignForInteger := func() bool { - return (defaultNewColFlen > 0 && defaultNewColFlen < defaultOldColFlen) || (toUnsigned != originUnsigned) - } - - // Deal with the same type. - if oldCol.GetType() == newCol.GetType() { - switch oldCol.GetType() { - case mysql.TypeNewDecimal: - // Since type decimal will encode the precision, frac, negative(signed) and wordBuf into storage together, there is no short - // cut to eliminate data reorg change for column type change between decimal. - return oldCol.GetFlen() != newCol.GetFlen() || oldCol.GetDecimal() != newCol.GetDecimal() || toUnsigned != originUnsigned - case mysql.TypeEnum, mysql.TypeSet: - return IsElemsChangedToModifyColumn(oldCol.GetElems(), newCol.GetElems()) - case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong: - return toUnsigned != originUnsigned - case mysql.TypeString: - // Due to the behavior of padding \x00 at binary type, always change column data when binary length changed - if types.IsBinaryStr(&oldCol.FieldType) { - return newCol.GetFlen() != oldCol.GetFlen() - } - } - - return needTruncationOrToggleSign() - } - - if ConvertBetweenCharAndVarchar(oldCol.GetType(), newCol.GetType()) { - return true - } - - // Deal with the different type. - switch oldCol.GetType() { - case mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString, mysql.TypeBlob, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob: - switch newCol.GetType() { - case mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString, mysql.TypeBlob, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob: - return needTruncationOrToggleSign() - } - case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong: - switch newCol.GetType() { - case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong: - return needTruncationOrToggleSignForInteger() - } - // conversion between float and double needs reorganization, see issue #31372 - } - - return true -} - -// ConvertBetweenCharAndVarchar check whether column converted between char and varchar -// TODO: it is used for plugins. so change plugin's using and remove it. -func ConvertBetweenCharAndVarchar(oldCol, newCol byte) bool { - return types.ConvertBetweenCharAndVarchar(oldCol, newCol) -} - -// IsElemsChangedToModifyColumn check elems changed -func IsElemsChangedToModifyColumn(oldElems, newElems []string) bool { - if len(newElems) < len(oldElems) { - return true - } - for index, oldElem := range oldElems { - newElem := newElems[index] - if oldElem != newElem { - return true - } - } - return false -} - -type modifyingColInfo struct { - newCol *model.ColumnInfo - oldColName *model.CIStr - modifyColumnTp byte - updatedAutoRandomBits uint64 - changingCol *model.ColumnInfo - changingIdxs []*model.IndexInfo - pos *ast.ColumnPosition - removedIdxs []int64 -} - -func getModifyColumnInfo(t *meta.Meta, job *model.Job) (*model.DBInfo, *model.TableInfo, *model.ColumnInfo, *modifyingColInfo, error) { - modifyInfo := &modifyingColInfo{pos: &ast.ColumnPosition{}} - err := job.DecodeArgs(&modifyInfo.newCol, &modifyInfo.oldColName, modifyInfo.pos, &modifyInfo.modifyColumnTp, - &modifyInfo.updatedAutoRandomBits, &modifyInfo.changingCol, &modifyInfo.changingIdxs, &modifyInfo.removedIdxs) - if err != nil { - job.State = model.JobStateCancelled - return nil, nil, nil, modifyInfo, errors.Trace(err) - } - - dbInfo, err := checkSchemaExistAndCancelNotExistJob(t, job) - if err != nil { - return nil, nil, nil, modifyInfo, errors.Trace(err) - } - - tblInfo, err := GetTableInfoAndCancelFaultJob(t, job, job.SchemaID) - if err != nil { - return nil, nil, nil, modifyInfo, errors.Trace(err) - } - - oldCol := model.FindColumnInfo(tblInfo.Columns, modifyInfo.oldColName.L) - if oldCol == nil || oldCol.State != model.StatePublic { - job.State = model.JobStateCancelled - return nil, nil, nil, modifyInfo, errors.Trace(infoschema.ErrColumnNotExists.GenWithStackByArgs(*(modifyInfo.oldColName), tblInfo.Name)) - } - - return dbInfo, tblInfo, oldCol, modifyInfo, errors.Trace(err) -} - -// GetOriginDefaultValueForModifyColumn gets the original default value for modifying column. -// Since column type change is implemented as adding a new column then substituting the old one. -// Case exists when update-where statement fetch a NULL for not-null column without any default data, -// it will errors. -// So we set original default value here to prevent this error. If the oldCol has the original default value, we use it. -// Otherwise we set the zero value as original default value. -// Besides, in insert & update records, we have already implement using the casted value of relative column to insert -// rather than the original default value. -func GetOriginDefaultValueForModifyColumn(ctx exprctx.BuildContext, changingCol, oldCol *model.ColumnInfo) (any, error) { - var err error - originDefVal := oldCol.GetOriginDefaultValue() - if originDefVal != nil { - odv, err := table.CastColumnValue(ctx, types.NewDatum(originDefVal), changingCol, false, false) - if err != nil { - logutil.DDLLogger().Info("cast origin default value failed", zap.Error(err)) - } - if !odv.IsNull() { - if originDefVal, err = odv.ToString(); err != nil { - originDefVal = nil - logutil.DDLLogger().Info("convert default value to string failed", zap.Error(err)) - } - } - } - if originDefVal == nil { - originDefVal, err = generateOriginDefaultValue(changingCol, nil) - if err != nil { - return nil, errors.Trace(err) - } - } - return originDefVal, nil -} - -func (w *worker) onModifyColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { - dbInfo, tblInfo, oldCol, modifyInfo, err := getModifyColumnInfo(t, job) - if err != nil { - return ver, err - } - - if job.IsRollingback() { - // For those column-type-change jobs which don't reorg the data. - if !needChangeColumnData(oldCol, modifyInfo.newCol) { - return rollbackModifyColumnJob(d, t, tblInfo, job, modifyInfo.newCol, oldCol, modifyInfo.modifyColumnTp) - } - // For those column-type-change jobs which reorg the data. - return rollbackModifyColumnJobWithData(d, t, tblInfo, job, oldCol, modifyInfo) - } - - // If we want to rename the column name, we need to check whether it already exists. - if modifyInfo.newCol.Name.L != modifyInfo.oldColName.L { - c := model.FindColumnInfo(tblInfo.Columns, modifyInfo.newCol.Name.L) - if c != nil { - job.State = model.JobStateCancelled - return ver, errors.Trace(infoschema.ErrColumnExists.GenWithStackByArgs(modifyInfo.newCol.Name)) - } - } - - failpoint.Inject("uninitializedOffsetAndState", func(val failpoint.Value) { - //nolint:forcetypeassert - if val.(bool) { - if modifyInfo.newCol.State != model.StatePublic { - failpoint.Return(ver, errors.New("the column state is wrong")) - } - } - }) - - err = checkAndApplyAutoRandomBits(d, t, dbInfo, tblInfo, oldCol, modifyInfo.newCol, modifyInfo.updatedAutoRandomBits) - if err != nil { - job.State = model.JobStateCancelled - return ver, errors.Trace(err) - } - - if !needChangeColumnData(oldCol, modifyInfo.newCol) { - return w.doModifyColumn(d, t, job, dbInfo, tblInfo, modifyInfo.newCol, oldCol, modifyInfo.pos) - } - - if err = isGeneratedRelatedColumn(tblInfo, modifyInfo.newCol, oldCol); err != nil { - job.State = model.JobStateCancelled - return ver, errors.Trace(err) - } - if tblInfo.Partition != nil { - job.State = model.JobStateCancelled - return ver, errors.Trace(dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("table is partition table")) - } - - changingCol := modifyInfo.changingCol - if changingCol == nil { - newColName := model.NewCIStr(genChangingColumnUniqueName(tblInfo, oldCol)) - if mysql.HasPriKeyFlag(oldCol.GetFlag()) { - job.State = model.JobStateCancelled - msg := "this column has primary key flag" - return ver, dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs(msg) - } - - changingCol = modifyInfo.newCol.Clone() - changingCol.Name = newColName - changingCol.ChangeStateInfo = &model.ChangeStateInfo{DependencyColumnOffset: oldCol.Offset} - - originDefVal, err := GetOriginDefaultValueForModifyColumn(newReorgExprCtx(), changingCol, oldCol) - if err != nil { - return ver, errors.Trace(err) - } - if err = changingCol.SetOriginDefaultValue(originDefVal); err != nil { - return ver, errors.Trace(err) - } - - InitAndAddColumnToTable(tblInfo, changingCol) - indexesToChange := FindRelatedIndexesToChange(tblInfo, oldCol.Name) - for _, info := range indexesToChange { - newIdxID := AllocateIndexID(tblInfo) - if !info.isTemp { - // We create a temp index for each normal index. - tmpIdx := info.IndexInfo.Clone() - tmpIdxName := genChangingIndexUniqueName(tblInfo, info.IndexInfo) - setIdxIDName(tmpIdx, newIdxID, model.NewCIStr(tmpIdxName)) - SetIdxColNameOffset(tmpIdx.Columns[info.Offset], changingCol) - tblInfo.Indices = append(tblInfo.Indices, tmpIdx) - } else { - // The index is a temp index created by previous modify column job(s). - // We can overwrite it to reduce reorg cost, because it will be dropped eventually. - tmpIdx := info.IndexInfo - oldTempIdxID := tmpIdx.ID - setIdxIDName(tmpIdx, newIdxID, tmpIdx.Name /* unchanged */) - SetIdxColNameOffset(tmpIdx.Columns[info.Offset], changingCol) - modifyInfo.removedIdxs = append(modifyInfo.removedIdxs, oldTempIdxID) - } - } - } else { - changingCol = model.FindColumnInfoByID(tblInfo.Columns, modifyInfo.changingCol.ID) - if changingCol == nil { - logutil.DDLLogger().Error("the changing column has been removed", zap.Error(err)) - job.State = model.JobStateCancelled - return ver, errors.Trace(infoschema.ErrColumnNotExists.GenWithStackByArgs(oldCol.Name, tblInfo.Name)) - } - } - - return w.doModifyColumnTypeWithData(d, t, job, dbInfo, tblInfo, changingCol, oldCol, modifyInfo.newCol.Name, modifyInfo.pos, modifyInfo.removedIdxs) -} - func setIdxIDName(idxInfo *model.IndexInfo, newID int64, newName model.CIStr) { idxInfo.ID = newID idxInfo.Name = newName @@ -568,32 +309,6 @@ func SetIdxColNameOffset(idxCol *model.IndexColumn, changingCol *model.ColumnInf } } -// rollbackModifyColumnJobWithData is used to rollback modify-column job which need to reorg the data. -func rollbackModifyColumnJobWithData(d *ddlCtx, t *meta.Meta, tblInfo *model.TableInfo, job *model.Job, oldCol *model.ColumnInfo, modifyInfo *modifyingColInfo) (ver int64, err error) { - // If the not-null change is included, we should clean the flag info in oldCol. - if modifyInfo.modifyColumnTp == mysql.TypeNull { - // Reset NotNullFlag flag. - tblInfo.Columns[oldCol.Offset].SetFlag(oldCol.GetFlag() &^ mysql.NotNullFlag) - // Reset PreventNullInsertFlag flag. - tblInfo.Columns[oldCol.Offset].SetFlag(oldCol.GetFlag() &^ mysql.PreventNullInsertFlag) - } - var changingIdxIDs []int64 - if modifyInfo.changingCol != nil { - changingIdxIDs = buildRelatedIndexIDs(tblInfo, modifyInfo.changingCol.ID) - // The job is in the middle state. The appended changingCol and changingIndex should - // be removed from the tableInfo as well. - removeChangingColAndIdxs(tblInfo, modifyInfo.changingCol.ID) - } - ver, err = updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, true) - if err != nil { - return ver, errors.Trace(err) - } - job.FinishTableJob(model.JobStateRollbackDone, model.StateNone, ver, tblInfo) - // Reconstruct the job args to add the temporary index ids into delete range table. - job.Args = []any{changingIdxIDs, getPartitionIDs(tblInfo)} - return ver, nil -} - func removeChangingColAndIdxs(tblInfo *model.TableInfo, changingColID int64) { restIdx := tblInfo.Indices[:0] for _, idx := range tblInfo.Indices { @@ -612,234 +327,6 @@ func removeChangingColAndIdxs(tblInfo *model.TableInfo, changingColID int64) { tblInfo.Columns = restCols } -func (w *worker) doModifyColumnTypeWithData( - d *ddlCtx, t *meta.Meta, job *model.Job, - dbInfo *model.DBInfo, tblInfo *model.TableInfo, changingCol, oldCol *model.ColumnInfo, - colName model.CIStr, pos *ast.ColumnPosition, rmIdxIDs []int64) (ver int64, _ error) { - var err error - originalState := changingCol.State - targetCol := changingCol.Clone() - targetCol.Name = colName - changingIdxs := buildRelatedIndexInfos(tblInfo, changingCol.ID) - switch changingCol.State { - case model.StateNone: - // Column from null to not null. - if !mysql.HasNotNullFlag(oldCol.GetFlag()) && mysql.HasNotNullFlag(changingCol.GetFlag()) { - // Introduce the `mysql.PreventNullInsertFlag` flag to prevent users from inserting or updating null values. - err := modifyColsFromNull2NotNull(w, dbInfo, tblInfo, []*model.ColumnInfo{oldCol}, targetCol, oldCol.GetType() != changingCol.GetType()) - if err != nil { - if dbterror.ErrWarnDataTruncated.Equal(err) || dbterror.ErrInvalidUseOfNull.Equal(err) { - job.State = model.JobStateRollingback - } - return ver, err - } - } - // none -> delete only - updateChangingObjState(changingCol, changingIdxs, model.StateDeleteOnly) - failpoint.Inject("mockInsertValueAfterCheckNull", func(val failpoint.Value) { - if valStr, ok := val.(string); ok { - var sctx sessionctx.Context - sctx, err := w.sessPool.Get() - if err != nil { - failpoint.Return(ver, err) - } - defer w.sessPool.Put(sctx) - - ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) - //nolint:forcetypeassert - _, _, err = sctx.GetRestrictedSQLExecutor().ExecRestrictedSQL(ctx, nil, valStr) - if err != nil { - job.State = model.JobStateCancelled - failpoint.Return(ver, err) - } - } - }) - ver, err = updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, originalState != changingCol.State) - if err != nil { - return ver, errors.Trace(err) - } - // Make sure job args change after `updateVersionAndTableInfoWithCheck`, otherwise, the job args will - // be updated in `updateDDLJob` even if it meets an error in `updateVersionAndTableInfoWithCheck`. - job.SchemaState = model.StateDeleteOnly - metrics.GetBackfillProgressByLabel(metrics.LblModifyColumn, job.SchemaName, tblInfo.Name.String()).Set(0) - job.Args = append(job.Args, changingCol, changingIdxs, rmIdxIDs) - case model.StateDeleteOnly: - // Column from null to not null. - if !mysql.HasNotNullFlag(oldCol.GetFlag()) && mysql.HasNotNullFlag(changingCol.GetFlag()) { - // Introduce the `mysql.PreventNullInsertFlag` flag to prevent users from inserting or updating null values. - err := modifyColsFromNull2NotNull(w, dbInfo, tblInfo, []*model.ColumnInfo{oldCol}, targetCol, oldCol.GetType() != changingCol.GetType()) - if err != nil { - if dbterror.ErrWarnDataTruncated.Equal(err) || dbterror.ErrInvalidUseOfNull.Equal(err) { - job.State = model.JobStateRollingback - } - return ver, err - } - } - // delete only -> write only - updateChangingObjState(changingCol, changingIdxs, model.StateWriteOnly) - ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, originalState != changingCol.State) - if err != nil { - return ver, errors.Trace(err) - } - job.SchemaState = model.StateWriteOnly - failpoint.InjectCall("afterModifyColumnStateDeleteOnly", job.ID) - case model.StateWriteOnly: - // write only -> reorganization - updateChangingObjState(changingCol, changingIdxs, model.StateWriteReorganization) - ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, originalState != changingCol.State) - if err != nil { - return ver, errors.Trace(err) - } - // Initialize SnapshotVer to 0 for later reorganization check. - job.SnapshotVer = 0 - job.SchemaState = model.StateWriteReorganization - case model.StateWriteReorganization: - tbl, err := getTable(d.getAutoIDRequirement(), dbInfo.ID, tblInfo) - if err != nil { - return ver, errors.Trace(err) - } - - var done bool - if job.MultiSchemaInfo != nil { - done, ver, err = doReorgWorkForModifyColumnMultiSchema(w, d, t, job, tbl, oldCol, changingCol, changingIdxs) - } else { - done, ver, err = doReorgWorkForModifyColumn(w, d, t, job, tbl, oldCol, changingCol, changingIdxs) - } - if !done { - return ver, err - } - - rmIdxIDs = append(buildRelatedIndexIDs(tblInfo, oldCol.ID), rmIdxIDs...) - - err = adjustTableInfoAfterModifyColumnWithData(tblInfo, pos, oldCol, changingCol, colName, changingIdxs) - if err != nil { - job.State = model.JobStateRollingback - return ver, errors.Trace(err) - } - - updateChangingObjState(changingCol, changingIdxs, model.StatePublic) - ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, originalState != changingCol.State) - if err != nil { - return ver, errors.Trace(err) - } - - // Finish this job. - job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo) - // Refactor the job args to add the old index ids into delete range table. - job.Args = []any{rmIdxIDs, getPartitionIDs(tblInfo)} - modifyColumnEvent := statsutil.NewModifyColumnEvent( - job.SchemaID, - tblInfo, - []*model.ColumnInfo{changingCol}, - ) - asyncNotifyEvent(d, modifyColumnEvent) - default: - err = dbterror.ErrInvalidDDLState.GenWithStackByArgs("column", changingCol.State) - } - - return ver, errors.Trace(err) -} - -func doReorgWorkForModifyColumnMultiSchema(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, tbl table.Table, - oldCol, changingCol *model.ColumnInfo, changingIdxs []*model.IndexInfo) (done bool, ver int64, err error) { - if job.MultiSchemaInfo.Revertible { - done, ver, err = doReorgWorkForModifyColumn(w, d, t, job, tbl, oldCol, changingCol, changingIdxs) - if done { - // We need another round to wait for all the others sub-jobs to finish. - job.MarkNonRevertible() - } - // We need another round to run the reorg process. - return false, ver, err - } - // Non-revertible means all the sub jobs finished. - return true, ver, err -} - -func doReorgWorkForModifyColumn(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, tbl table.Table, - oldCol, changingCol *model.ColumnInfo, changingIdxs []*model.IndexInfo) (done bool, ver int64, err error) { - job.ReorgMeta.ReorgTp = model.ReorgTypeTxn - sctx, err1 := w.sessPool.Get() - if err1 != nil { - err = errors.Trace(err1) - return - } - defer w.sessPool.Put(sctx) - rh := newReorgHandler(sess.NewSession(sctx)) - dbInfo, err := t.GetDatabase(job.SchemaID) - if err != nil { - return false, ver, errors.Trace(err) - } - reorgInfo, err := getReorgInfo(d.jobContext(job.ID, job.ReorgMeta), - d, rh, job, dbInfo, tbl, BuildElements(changingCol, changingIdxs), false) - if err != nil || reorgInfo == nil || reorgInfo.first { - // If we run reorg firstly, we should update the job snapshot version - // and then run the reorg next time. - return false, ver, errors.Trace(err) - } - - // Inject a failpoint so that we can pause here and do verification on other components. - // With a failpoint-enabled version of TiDB, you can trigger this failpoint by the following command: - // enable: curl -X PUT -d "pause" "http://127.0.0.1:10080/fail/github.com/pingcap/tidb/pkg/ddl/mockDelayInModifyColumnTypeWithData". - // disable: curl -X DELETE "http://127.0.0.1:10080/fail/github.com/pingcap/tidb/pkg/ddl/mockDelayInModifyColumnTypeWithData" - failpoint.Inject("mockDelayInModifyColumnTypeWithData", func() {}) - err = w.runReorgJob(reorgInfo, tbl.Meta(), d.lease, func() (addIndexErr error) { - defer util.Recover(metrics.LabelDDL, "onModifyColumn", - func() { - addIndexErr = dbterror.ErrCancelledDDLJob.GenWithStack("modify table `%v` column `%v` panic", tbl.Meta().Name, oldCol.Name) - }, false) - // Use old column name to generate less confusing error messages. - changingColCpy := changingCol.Clone() - changingColCpy.Name = oldCol.Name - return w.updateCurrentElement(tbl, reorgInfo) - }) - if err != nil { - if dbterror.ErrPausedDDLJob.Equal(err) { - return false, ver, nil - } - - if dbterror.ErrWaitReorgTimeout.Equal(err) { - // If timeout, we should return, check for the owner and re-wait job done. - return false, ver, nil - } - if kv.IsTxnRetryableError(err) || dbterror.ErrNotOwner.Equal(err) { - return false, ver, errors.Trace(err) - } - if err1 := rh.RemoveDDLReorgHandle(job, reorgInfo.elements); err1 != nil { - logutil.DDLLogger().Warn("run modify column job failed, RemoveDDLReorgHandle failed, can't convert job to rollback", - zap.String("job", job.String()), zap.Error(err1)) - } - logutil.DDLLogger().Warn("run modify column job failed, convert job to rollback", zap.Stringer("job", job), zap.Error(err)) - job.State = model.JobStateRollingback - return false, ver, errors.Trace(err) - } - return true, ver, nil -} - -func adjustTableInfoAfterModifyColumnWithData(tblInfo *model.TableInfo, pos *ast.ColumnPosition, - oldCol, changingCol *model.ColumnInfo, newName model.CIStr, changingIdxs []*model.IndexInfo) (err error) { - if pos != nil && pos.RelativeColumn != nil && oldCol.Name.L == pos.RelativeColumn.Name.L { - // For cases like `modify column b after b`, it should report this error. - return errors.Trace(infoschema.ErrColumnNotExists.GenWithStackByArgs(oldCol.Name, tblInfo.Name)) - } - internalColName := changingCol.Name - changingCol = replaceOldColumn(tblInfo, oldCol, changingCol, newName) - if len(changingIdxs) > 0 { - updateNewIdxColsNameOffset(changingIdxs, internalColName, changingCol) - indexesToRemove := filterIndexesToRemove(changingIdxs, newName, tblInfo) - replaceOldIndexes(tblInfo, indexesToRemove) - } - if tblInfo.TTLInfo != nil { - updateTTLInfoWhenModifyColumn(tblInfo, oldCol.Name, changingCol.Name) - } - // Move the new column to a correct offset. - destOffset, err := LocateOffsetToMove(changingCol.Offset, pos, tblInfo) - if err != nil { - return errors.Trace(err) - } - tblInfo.MoveColumnInfo(changingCol.Offset, destOffset) - return nil -} - func replaceOldColumn(tblInfo *model.TableInfo, oldCol, changingCol *model.ColumnInfo, newName model.CIStr) *model.ColumnInfo { tblInfo.MoveColumnInfo(changingCol.Offset, len(tblInfo.Columns)-1) @@ -892,30 +379,6 @@ func updateNewIdxColsNameOffset(changingIdxs []*model.IndexInfo, } } -func updateFKInfoWhenModifyColumn(tblInfo *model.TableInfo, oldCol, newCol model.CIStr) { - if oldCol.L == newCol.L { - return - } - for _, fk := range tblInfo.ForeignKeys { - for i := range fk.Cols { - if fk.Cols[i].L == oldCol.L { - fk.Cols[i] = newCol - } - } - } -} - -func updateTTLInfoWhenModifyColumn(tblInfo *model.TableInfo, oldCol, newCol model.CIStr) { - if oldCol.L == newCol.L { - return - } - if tblInfo.TTLInfo != nil { - if tblInfo.TTLInfo.ColumnName.L == oldCol.L { - tblInfo.TTLInfo.ColumnName = newCol - } - } -} - // filterIndexesToRemove filters out the indexes that can be removed. func filterIndexesToRemove(changingIdxs []*model.IndexInfo, colName model.CIStr, tblInfo *model.TableInfo) []*model.IndexInfo { indexesToRemove := make([]*model.IndexInfo, 0, len(changingIdxs)) @@ -1432,129 +895,6 @@ func updateChangingObjState(changingCol *model.ColumnInfo, changingIdxs []*model } } -// doModifyColumn updates the column information and reorders all columns. It does not support modifying column data. -func (w *worker) doModifyColumn( - d *ddlCtx, t *meta.Meta, job *model.Job, dbInfo *model.DBInfo, tblInfo *model.TableInfo, - newCol, oldCol *model.ColumnInfo, pos *ast.ColumnPosition) (ver int64, _ error) { - if oldCol.ID != newCol.ID { - job.State = model.JobStateRollingback - return ver, dbterror.ErrColumnInChange.GenWithStackByArgs(oldCol.Name, newCol.ID) - } - // Column from null to not null. - if !mysql.HasNotNullFlag(oldCol.GetFlag()) && mysql.HasNotNullFlag(newCol.GetFlag()) { - noPreventNullFlag := !mysql.HasPreventNullInsertFlag(oldCol.GetFlag()) - - // lease = 0 means it's in an integration test. In this case we don't delay so the test won't run too slowly. - // We need to check after the flag is set - if d.lease > 0 && !noPreventNullFlag { - delayForAsyncCommit() - } - - // Introduce the `mysql.PreventNullInsertFlag` flag to prevent users from inserting or updating null values. - err := modifyColsFromNull2NotNull(w, dbInfo, tblInfo, []*model.ColumnInfo{oldCol}, newCol, oldCol.GetType() != newCol.GetType()) - if err != nil { - if dbterror.ErrWarnDataTruncated.Equal(err) || dbterror.ErrInvalidUseOfNull.Equal(err) { - job.State = model.JobStateRollingback - } - return ver, err - } - // The column should get into prevent null status first. - if noPreventNullFlag { - return updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, true) - } - } - - if job.MultiSchemaInfo != nil && job.MultiSchemaInfo.Revertible { - job.MarkNonRevertible() - // Store the mark and enter the next DDL handling loop. - return updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, false) - } - - if err := adjustTableInfoAfterModifyColumn(tblInfo, newCol, oldCol, pos); err != nil { - job.State = model.JobStateRollingback - return ver, errors.Trace(err) - } - - childTableInfos, err := adjustForeignKeyChildTableInfoAfterModifyColumn(d, t, job, tblInfo, newCol, oldCol) - if err != nil { - return ver, errors.Trace(err) - } - ver, err = updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, true, childTableInfos...) - if err != nil { - // Modified the type definition of 'null' to 'not null' before this, so rollBack the job when an error occurs. - job.State = model.JobStateRollingback - return ver, errors.Trace(err) - } - - job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo) - // For those column-type-change type which doesn't need reorg data, we should also mock the job args for delete range. - job.Args = []any{[]int64{}, []int64{}} - return ver, nil -} - -func adjustTableInfoAfterModifyColumn( - tblInfo *model.TableInfo, newCol, oldCol *model.ColumnInfo, pos *ast.ColumnPosition) error { - // We need the latest column's offset and state. This information can be obtained from the store. - newCol.Offset = oldCol.Offset - newCol.State = oldCol.State - if pos != nil && pos.RelativeColumn != nil && oldCol.Name.L == pos.RelativeColumn.Name.L { - // For cases like `modify column b after b`, it should report this error. - return errors.Trace(infoschema.ErrColumnNotExists.GenWithStackByArgs(oldCol.Name, tblInfo.Name)) - } - destOffset, err := LocateOffsetToMove(oldCol.Offset, pos, tblInfo) - if err != nil { - return errors.Trace(infoschema.ErrColumnNotExists.GenWithStackByArgs(oldCol.Name, tblInfo.Name)) - } - tblInfo.Columns[oldCol.Offset] = newCol - tblInfo.MoveColumnInfo(oldCol.Offset, destOffset) - updateNewIdxColsNameOffset(tblInfo.Indices, oldCol.Name, newCol) - updateFKInfoWhenModifyColumn(tblInfo, oldCol.Name, newCol.Name) - updateTTLInfoWhenModifyColumn(tblInfo, oldCol.Name, newCol.Name) - return nil -} - -func adjustForeignKeyChildTableInfoAfterModifyColumn(d *ddlCtx, t *meta.Meta, job *model.Job, tblInfo *model.TableInfo, newCol, oldCol *model.ColumnInfo) ([]schemaIDAndTableInfo, error) { - if !variable.EnableForeignKey.Load() || newCol.Name.L == oldCol.Name.L { - return nil, nil - } - is, err := getAndCheckLatestInfoSchema(d, t) - if err != nil { - return nil, err - } - referredFKs := is.GetTableReferredForeignKeys(job.SchemaName, tblInfo.Name.L) - if len(referredFKs) == 0 { - return nil, nil - } - fkh := newForeignKeyHelper() - fkh.addLoadedTable(job.SchemaName, tblInfo.Name.L, job.SchemaID, tblInfo) - for _, referredFK := range referredFKs { - info, err := fkh.getTableFromStorage(is, t, referredFK.ChildSchema, referredFK.ChildTable) - if err != nil { - if infoschema.ErrTableNotExists.Equal(err) || infoschema.ErrDatabaseNotExists.Equal(err) { - continue - } - return nil, err - } - fkInfo := model.FindFKInfoByName(info.tblInfo.ForeignKeys, referredFK.ChildFKName.L) - if fkInfo == nil { - continue - } - for i := range fkInfo.RefCols { - if fkInfo.RefCols[i].L == oldCol.Name.L { - fkInfo.RefCols[i] = newCol.Name - } - } - } - infoList := make([]schemaIDAndTableInfo, 0, len(fkh.loaded)) - for _, info := range fkh.loaded { - if info.tblInfo.ID == tblInfo.ID { - continue - } - infoList = append(infoList, info) - } - return infoList, nil -} - func checkAndApplyAutoRandomBits(d *ddlCtx, m *meta.Meta, dbInfo *model.DBInfo, tblInfo *model.TableInfo, oldCol *model.ColumnInfo, newCol *model.ColumnInfo, newAutoRandBits uint64) error { if newAutoRandBits == 0 { @@ -1806,25 +1146,6 @@ func checkAddColumnTooManyColumns(colNum int) error { return nil } -// rollbackModifyColumnJob rollbacks the job when an error occurs. -func rollbackModifyColumnJob(d *ddlCtx, t *meta.Meta, tblInfo *model.TableInfo, job *model.Job, newCol, oldCol *model.ColumnInfo, modifyColumnTp byte) (ver int64, _ error) { - var err error - if oldCol.ID == newCol.ID && modifyColumnTp == mysql.TypeNull { - // field NotNullFlag flag reset. - tblInfo.Columns[oldCol.Offset].SetFlag(oldCol.GetFlag() &^ mysql.NotNullFlag) - // field PreventNullInsertFlag flag reset. - tblInfo.Columns[oldCol.Offset].SetFlag(oldCol.GetFlag() &^ mysql.PreventNullInsertFlag) - ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, true) - if err != nil { - return ver, errors.Trace(err) - } - } - job.FinishTableJob(model.JobStateRollbackDone, model.StateNone, ver, tblInfo) - // For those column-type-change type which doesn't need reorg data, we should also mock the job args for delete range. - job.Args = []any{[]int64{}, []int64{}} - return ver, nil -} - // modifyColsFromNull2NotNull modifies the type definitions of 'null' to 'not null'. // Introduce the `mysql.PreventNullInsertFlag` flag to prevent users from inserting or updating null values. func modifyColsFromNull2NotNull(w *worker, dbInfo *model.DBInfo, tblInfo *model.TableInfo, cols []*model.ColumnInfo, newCol *model.ColumnInfo, isDataTruncated bool) error { diff --git a/pkg/ddl/modify_column.go b/pkg/ddl/modify_column.go index 0a54f06bfa78e..09a654d133a26 100644 --- a/pkg/ddl/modify_column.go +++ b/pkg/ddl/modify_column.go @@ -21,12 +21,17 @@ import ( "strings" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/pkg/ddl/logutil" + sess "github.com/pingcap/tidb/pkg/ddl/session" "github.com/pingcap/tidb/pkg/errctx" "github.com/pingcap/tidb/pkg/expression" exprctx "github.com/pingcap/tidb/pkg/expression/context" "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/metrics" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" @@ -34,12 +39,613 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/variable" + statsutil "github.com/pingcap/tidb/pkg/statistics/handle/util" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/types" + "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/dbterror" + "go.uber.org/zap" ) +type modifyingColInfo struct { + newCol *model.ColumnInfo + oldColName *model.CIStr + modifyColumnTp byte + updatedAutoRandomBits uint64 + changingCol *model.ColumnInfo + changingIdxs []*model.IndexInfo + pos *ast.ColumnPosition + removedIdxs []int64 +} + +func (w *worker) onModifyColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { + dbInfo, tblInfo, oldCol, modifyInfo, err := getModifyColumnInfo(t, job) + if err != nil { + return ver, err + } + + if job.IsRollingback() { + // For those column-type-change jobs which don't reorg the data. + if !needChangeColumnData(oldCol, modifyInfo.newCol) { + return rollbackModifyColumnJob(d, t, tblInfo, job, modifyInfo.newCol, oldCol, modifyInfo.modifyColumnTp) + } + // For those column-type-change jobs which reorg the data. + return rollbackModifyColumnJobWithData(d, t, tblInfo, job, oldCol, modifyInfo) + } + + // If we want to rename the column name, we need to check whether it already exists. + if modifyInfo.newCol.Name.L != modifyInfo.oldColName.L { + c := model.FindColumnInfo(tblInfo.Columns, modifyInfo.newCol.Name.L) + if c != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(infoschema.ErrColumnExists.GenWithStackByArgs(modifyInfo.newCol.Name)) + } + } + + failpoint.Inject("uninitializedOffsetAndState", func(val failpoint.Value) { + //nolint:forcetypeassert + if val.(bool) { + if modifyInfo.newCol.State != model.StatePublic { + failpoint.Return(ver, errors.New("the column state is wrong")) + } + } + }) + + err = checkAndApplyAutoRandomBits(d, t, dbInfo, tblInfo, oldCol, modifyInfo.newCol, modifyInfo.updatedAutoRandomBits) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + + if !needChangeColumnData(oldCol, modifyInfo.newCol) { + return w.doModifyColumn(d, t, job, dbInfo, tblInfo, modifyInfo.newCol, oldCol, modifyInfo.pos) + } + + if err = isGeneratedRelatedColumn(tblInfo, modifyInfo.newCol, oldCol); err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + if tblInfo.Partition != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("table is partition table")) + } + + changingCol := modifyInfo.changingCol + if changingCol == nil { + newColName := model.NewCIStr(genChangingColumnUniqueName(tblInfo, oldCol)) + if mysql.HasPriKeyFlag(oldCol.GetFlag()) { + job.State = model.JobStateCancelled + msg := "this column has primary key flag" + return ver, dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs(msg) + } + + changingCol = modifyInfo.newCol.Clone() + changingCol.Name = newColName + changingCol.ChangeStateInfo = &model.ChangeStateInfo{DependencyColumnOffset: oldCol.Offset} + + originDefVal, err := GetOriginDefaultValueForModifyColumn(newReorgExprCtx(), changingCol, oldCol) + if err != nil { + return ver, errors.Trace(err) + } + if err = changingCol.SetOriginDefaultValue(originDefVal); err != nil { + return ver, errors.Trace(err) + } + + InitAndAddColumnToTable(tblInfo, changingCol) + indexesToChange := FindRelatedIndexesToChange(tblInfo, oldCol.Name) + for _, info := range indexesToChange { + newIdxID := AllocateIndexID(tblInfo) + if !info.isTemp { + // We create a temp index for each normal index. + tmpIdx := info.IndexInfo.Clone() + tmpIdxName := genChangingIndexUniqueName(tblInfo, info.IndexInfo) + setIdxIDName(tmpIdx, newIdxID, model.NewCIStr(tmpIdxName)) + SetIdxColNameOffset(tmpIdx.Columns[info.Offset], changingCol) + tblInfo.Indices = append(tblInfo.Indices, tmpIdx) + } else { + // The index is a temp index created by previous modify column job(s). + // We can overwrite it to reduce reorg cost, because it will be dropped eventually. + tmpIdx := info.IndexInfo + oldTempIdxID := tmpIdx.ID + setIdxIDName(tmpIdx, newIdxID, tmpIdx.Name /* unchanged */) + SetIdxColNameOffset(tmpIdx.Columns[info.Offset], changingCol) + modifyInfo.removedIdxs = append(modifyInfo.removedIdxs, oldTempIdxID) + } + } + } else { + changingCol = model.FindColumnInfoByID(tblInfo.Columns, modifyInfo.changingCol.ID) + if changingCol == nil { + logutil.DDLLogger().Error("the changing column has been removed", zap.Error(err)) + job.State = model.JobStateCancelled + return ver, errors.Trace(infoschema.ErrColumnNotExists.GenWithStackByArgs(oldCol.Name, tblInfo.Name)) + } + } + + return w.doModifyColumnTypeWithData(d, t, job, dbInfo, tblInfo, changingCol, oldCol, modifyInfo.newCol.Name, modifyInfo.pos, modifyInfo.removedIdxs) +} + +// rollbackModifyColumnJob rollbacks the job when an error occurs. +func rollbackModifyColumnJob(d *ddlCtx, t *meta.Meta, tblInfo *model.TableInfo, job *model.Job, newCol, oldCol *model.ColumnInfo, modifyColumnTp byte) (ver int64, _ error) { + var err error + if oldCol.ID == newCol.ID && modifyColumnTp == mysql.TypeNull { + // field NotNullFlag flag reset. + tblInfo.Columns[oldCol.Offset].SetFlag(oldCol.GetFlag() &^ mysql.NotNullFlag) + // field PreventNullInsertFlag flag reset. + tblInfo.Columns[oldCol.Offset].SetFlag(oldCol.GetFlag() &^ mysql.PreventNullInsertFlag) + ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, true) + if err != nil { + return ver, errors.Trace(err) + } + } + job.FinishTableJob(model.JobStateRollbackDone, model.StateNone, ver, tblInfo) + // For those column-type-change type which doesn't need reorg data, we should also mock the job args for delete range. + job.Args = []any{[]int64{}, []int64{}} + return ver, nil +} + +func getModifyColumnInfo(t *meta.Meta, job *model.Job) (*model.DBInfo, *model.TableInfo, *model.ColumnInfo, *modifyingColInfo, error) { + modifyInfo := &modifyingColInfo{pos: &ast.ColumnPosition{}} + err := job.DecodeArgs(&modifyInfo.newCol, &modifyInfo.oldColName, modifyInfo.pos, &modifyInfo.modifyColumnTp, + &modifyInfo.updatedAutoRandomBits, &modifyInfo.changingCol, &modifyInfo.changingIdxs, &modifyInfo.removedIdxs) + if err != nil { + job.State = model.JobStateCancelled + return nil, nil, nil, modifyInfo, errors.Trace(err) + } + + dbInfo, err := checkSchemaExistAndCancelNotExistJob(t, job) + if err != nil { + return nil, nil, nil, modifyInfo, errors.Trace(err) + } + + tblInfo, err := GetTableInfoAndCancelFaultJob(t, job, job.SchemaID) + if err != nil { + return nil, nil, nil, modifyInfo, errors.Trace(err) + } + + oldCol := model.FindColumnInfo(tblInfo.Columns, modifyInfo.oldColName.L) + if oldCol == nil || oldCol.State != model.StatePublic { + job.State = model.JobStateCancelled + return nil, nil, nil, modifyInfo, errors.Trace(infoschema.ErrColumnNotExists.GenWithStackByArgs(*(modifyInfo.oldColName), tblInfo.Name)) + } + + return dbInfo, tblInfo, oldCol, modifyInfo, errors.Trace(err) +} + +// GetOriginDefaultValueForModifyColumn gets the original default value for modifying column. +// Since column type change is implemented as adding a new column then substituting the old one. +// Case exists when update-where statement fetch a NULL for not-null column without any default data, +// it will errors. +// So we set original default value here to prevent this error. If the oldCol has the original default value, we use it. +// Otherwise we set the zero value as original default value. +// Besides, in insert & update records, we have already implement using the casted value of relative column to insert +// rather than the original default value. +func GetOriginDefaultValueForModifyColumn(ctx exprctx.BuildContext, changingCol, oldCol *model.ColumnInfo) (any, error) { + var err error + originDefVal := oldCol.GetOriginDefaultValue() + if originDefVal != nil { + odv, err := table.CastColumnValue(ctx, types.NewDatum(originDefVal), changingCol, false, false) + if err != nil { + logutil.DDLLogger().Info("cast origin default value failed", zap.Error(err)) + } + if !odv.IsNull() { + if originDefVal, err = odv.ToString(); err != nil { + originDefVal = nil + logutil.DDLLogger().Info("convert default value to string failed", zap.Error(err)) + } + } + } + if originDefVal == nil { + originDefVal, err = generateOriginDefaultValue(changingCol, nil) + if err != nil { + return nil, errors.Trace(err) + } + } + return originDefVal, nil +} + +// rollbackModifyColumnJobWithData is used to rollback modify-column job which need to reorg the data. +func rollbackModifyColumnJobWithData(d *ddlCtx, t *meta.Meta, tblInfo *model.TableInfo, job *model.Job, oldCol *model.ColumnInfo, modifyInfo *modifyingColInfo) (ver int64, err error) { + // If the not-null change is included, we should clean the flag info in oldCol. + if modifyInfo.modifyColumnTp == mysql.TypeNull { + // Reset NotNullFlag flag. + tblInfo.Columns[oldCol.Offset].SetFlag(oldCol.GetFlag() &^ mysql.NotNullFlag) + // Reset PreventNullInsertFlag flag. + tblInfo.Columns[oldCol.Offset].SetFlag(oldCol.GetFlag() &^ mysql.PreventNullInsertFlag) + } + var changingIdxIDs []int64 + if modifyInfo.changingCol != nil { + changingIdxIDs = buildRelatedIndexIDs(tblInfo, modifyInfo.changingCol.ID) + // The job is in the middle state. The appended changingCol and changingIndex should + // be removed from the tableInfo as well. + removeChangingColAndIdxs(tblInfo, modifyInfo.changingCol.ID) + } + ver, err = updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, true) + if err != nil { + return ver, errors.Trace(err) + } + job.FinishTableJob(model.JobStateRollbackDone, model.StateNone, ver, tblInfo) + // Reconstruct the job args to add the temporary index ids into delete range table. + job.Args = []any{changingIdxIDs, getPartitionIDs(tblInfo)} + return ver, nil +} + +// doModifyColumn updates the column information and reorders all columns. It does not support modifying column data. +func (w *worker) doModifyColumn( + d *ddlCtx, t *meta.Meta, job *model.Job, dbInfo *model.DBInfo, tblInfo *model.TableInfo, + newCol, oldCol *model.ColumnInfo, pos *ast.ColumnPosition) (ver int64, _ error) { + if oldCol.ID != newCol.ID { + job.State = model.JobStateRollingback + return ver, dbterror.ErrColumnInChange.GenWithStackByArgs(oldCol.Name, newCol.ID) + } + // Column from null to not null. + if !mysql.HasNotNullFlag(oldCol.GetFlag()) && mysql.HasNotNullFlag(newCol.GetFlag()) { + noPreventNullFlag := !mysql.HasPreventNullInsertFlag(oldCol.GetFlag()) + + // lease = 0 means it's in an integration test. In this case we don't delay so the test won't run too slowly. + // We need to check after the flag is set + if d.lease > 0 && !noPreventNullFlag { + delayForAsyncCommit() + } + + // Introduce the `mysql.PreventNullInsertFlag` flag to prevent users from inserting or updating null values. + err := modifyColsFromNull2NotNull(w, dbInfo, tblInfo, []*model.ColumnInfo{oldCol}, newCol, oldCol.GetType() != newCol.GetType()) + if err != nil { + if dbterror.ErrWarnDataTruncated.Equal(err) || dbterror.ErrInvalidUseOfNull.Equal(err) { + job.State = model.JobStateRollingback + } + return ver, err + } + // The column should get into prevent null status first. + if noPreventNullFlag { + return updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, true) + } + } + + if job.MultiSchemaInfo != nil && job.MultiSchemaInfo.Revertible { + job.MarkNonRevertible() + // Store the mark and enter the next DDL handling loop. + return updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, false) + } + + if err := adjustTableInfoAfterModifyColumn(tblInfo, newCol, oldCol, pos); err != nil { + job.State = model.JobStateRollingback + return ver, errors.Trace(err) + } + + childTableInfos, err := adjustForeignKeyChildTableInfoAfterModifyColumn(d, t, job, tblInfo, newCol, oldCol) + if err != nil { + return ver, errors.Trace(err) + } + ver, err = updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, true, childTableInfos...) + if err != nil { + // Modified the type definition of 'null' to 'not null' before this, so rollBack the job when an error occurs. + job.State = model.JobStateRollingback + return ver, errors.Trace(err) + } + + job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo) + // For those column-type-change type which doesn't need reorg data, we should also mock the job args for delete range. + job.Args = []any{[]int64{}, []int64{}} + return ver, nil +} + +func adjustTableInfoAfterModifyColumn( + tblInfo *model.TableInfo, newCol, oldCol *model.ColumnInfo, pos *ast.ColumnPosition) error { + // We need the latest column's offset and state. This information can be obtained from the store. + newCol.Offset = oldCol.Offset + newCol.State = oldCol.State + if pos != nil && pos.RelativeColumn != nil && oldCol.Name.L == pos.RelativeColumn.Name.L { + // For cases like `modify column b after b`, it should report this error. + return errors.Trace(infoschema.ErrColumnNotExists.GenWithStackByArgs(oldCol.Name, tblInfo.Name)) + } + destOffset, err := LocateOffsetToMove(oldCol.Offset, pos, tblInfo) + if err != nil { + return errors.Trace(infoschema.ErrColumnNotExists.GenWithStackByArgs(oldCol.Name, tblInfo.Name)) + } + tblInfo.Columns[oldCol.Offset] = newCol + tblInfo.MoveColumnInfo(oldCol.Offset, destOffset) + updateNewIdxColsNameOffset(tblInfo.Indices, oldCol.Name, newCol) + updateFKInfoWhenModifyColumn(tblInfo, oldCol.Name, newCol.Name) + updateTTLInfoWhenModifyColumn(tblInfo, oldCol.Name, newCol.Name) + return nil +} + +func updateFKInfoWhenModifyColumn(tblInfo *model.TableInfo, oldCol, newCol model.CIStr) { + if oldCol.L == newCol.L { + return + } + for _, fk := range tblInfo.ForeignKeys { + for i := range fk.Cols { + if fk.Cols[i].L == oldCol.L { + fk.Cols[i] = newCol + } + } + } +} + +func updateTTLInfoWhenModifyColumn(tblInfo *model.TableInfo, oldCol, newCol model.CIStr) { + if oldCol.L == newCol.L { + return + } + if tblInfo.TTLInfo != nil { + if tblInfo.TTLInfo.ColumnName.L == oldCol.L { + tblInfo.TTLInfo.ColumnName = newCol + } + } +} + +func adjustForeignKeyChildTableInfoAfterModifyColumn(d *ddlCtx, t *meta.Meta, job *model.Job, tblInfo *model.TableInfo, newCol, oldCol *model.ColumnInfo) ([]schemaIDAndTableInfo, error) { + if !variable.EnableForeignKey.Load() || newCol.Name.L == oldCol.Name.L { + return nil, nil + } + is, err := getAndCheckLatestInfoSchema(d, t) + if err != nil { + return nil, err + } + referredFKs := is.GetTableReferredForeignKeys(job.SchemaName, tblInfo.Name.L) + if len(referredFKs) == 0 { + return nil, nil + } + fkh := newForeignKeyHelper() + fkh.addLoadedTable(job.SchemaName, tblInfo.Name.L, job.SchemaID, tblInfo) + for _, referredFK := range referredFKs { + info, err := fkh.getTableFromStorage(is, t, referredFK.ChildSchema, referredFK.ChildTable) + if err != nil { + if infoschema.ErrTableNotExists.Equal(err) || infoschema.ErrDatabaseNotExists.Equal(err) { + continue + } + return nil, err + } + fkInfo := model.FindFKInfoByName(info.tblInfo.ForeignKeys, referredFK.ChildFKName.L) + if fkInfo == nil { + continue + } + for i := range fkInfo.RefCols { + if fkInfo.RefCols[i].L == oldCol.Name.L { + fkInfo.RefCols[i] = newCol.Name + } + } + } + infoList := make([]schemaIDAndTableInfo, 0, len(fkh.loaded)) + for _, info := range fkh.loaded { + if info.tblInfo.ID == tblInfo.ID { + continue + } + infoList = append(infoList, info) + } + return infoList, nil +} + +func (w *worker) doModifyColumnTypeWithData( + d *ddlCtx, t *meta.Meta, job *model.Job, + dbInfo *model.DBInfo, tblInfo *model.TableInfo, changingCol, oldCol *model.ColumnInfo, + colName model.CIStr, pos *ast.ColumnPosition, rmIdxIDs []int64) (ver int64, _ error) { + var err error + originalState := changingCol.State + targetCol := changingCol.Clone() + targetCol.Name = colName + changingIdxs := buildRelatedIndexInfos(tblInfo, changingCol.ID) + switch changingCol.State { + case model.StateNone: + // Column from null to not null. + if !mysql.HasNotNullFlag(oldCol.GetFlag()) && mysql.HasNotNullFlag(changingCol.GetFlag()) { + // Introduce the `mysql.PreventNullInsertFlag` flag to prevent users from inserting or updating null values. + err := modifyColsFromNull2NotNull(w, dbInfo, tblInfo, []*model.ColumnInfo{oldCol}, targetCol, oldCol.GetType() != changingCol.GetType()) + if err != nil { + if dbterror.ErrWarnDataTruncated.Equal(err) || dbterror.ErrInvalidUseOfNull.Equal(err) { + job.State = model.JobStateRollingback + } + return ver, err + } + } + // none -> delete only + updateChangingObjState(changingCol, changingIdxs, model.StateDeleteOnly) + failpoint.Inject("mockInsertValueAfterCheckNull", func(val failpoint.Value) { + if valStr, ok := val.(string); ok { + var sctx sessionctx.Context + sctx, err := w.sessPool.Get() + if err != nil { + failpoint.Return(ver, err) + } + defer w.sessPool.Put(sctx) + + ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) + //nolint:forcetypeassert + _, _, err = sctx.GetRestrictedSQLExecutor().ExecRestrictedSQL(ctx, nil, valStr) + if err != nil { + job.State = model.JobStateCancelled + failpoint.Return(ver, err) + } + } + }) + ver, err = updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, originalState != changingCol.State) + if err != nil { + return ver, errors.Trace(err) + } + // Make sure job args change after `updateVersionAndTableInfoWithCheck`, otherwise, the job args will + // be updated in `updateDDLJob` even if it meets an error in `updateVersionAndTableInfoWithCheck`. + job.SchemaState = model.StateDeleteOnly + metrics.GetBackfillProgressByLabel(metrics.LblModifyColumn, job.SchemaName, tblInfo.Name.String()).Set(0) + job.Args = append(job.Args, changingCol, changingIdxs, rmIdxIDs) + case model.StateDeleteOnly: + // Column from null to not null. + if !mysql.HasNotNullFlag(oldCol.GetFlag()) && mysql.HasNotNullFlag(changingCol.GetFlag()) { + // Introduce the `mysql.PreventNullInsertFlag` flag to prevent users from inserting or updating null values. + err := modifyColsFromNull2NotNull(w, dbInfo, tblInfo, []*model.ColumnInfo{oldCol}, targetCol, oldCol.GetType() != changingCol.GetType()) + if err != nil { + if dbterror.ErrWarnDataTruncated.Equal(err) || dbterror.ErrInvalidUseOfNull.Equal(err) { + job.State = model.JobStateRollingback + } + return ver, err + } + } + // delete only -> write only + updateChangingObjState(changingCol, changingIdxs, model.StateWriteOnly) + ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, originalState != changingCol.State) + if err != nil { + return ver, errors.Trace(err) + } + job.SchemaState = model.StateWriteOnly + failpoint.InjectCall("afterModifyColumnStateDeleteOnly", job.ID) + case model.StateWriteOnly: + // write only -> reorganization + updateChangingObjState(changingCol, changingIdxs, model.StateWriteReorganization) + ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, originalState != changingCol.State) + if err != nil { + return ver, errors.Trace(err) + } + // Initialize SnapshotVer to 0 for later reorganization check. + job.SnapshotVer = 0 + job.SchemaState = model.StateWriteReorganization + case model.StateWriteReorganization: + tbl, err := getTable(d.getAutoIDRequirement(), dbInfo.ID, tblInfo) + if err != nil { + return ver, errors.Trace(err) + } + + var done bool + if job.MultiSchemaInfo != nil { + done, ver, err = doReorgWorkForModifyColumnMultiSchema(w, d, t, job, tbl, oldCol, changingCol, changingIdxs) + } else { + done, ver, err = doReorgWorkForModifyColumn(w, d, t, job, tbl, oldCol, changingCol, changingIdxs) + } + if !done { + return ver, err + } + + rmIdxIDs = append(buildRelatedIndexIDs(tblInfo, oldCol.ID), rmIdxIDs...) + + err = adjustTableInfoAfterModifyColumnWithData(tblInfo, pos, oldCol, changingCol, colName, changingIdxs) + if err != nil { + job.State = model.JobStateRollingback + return ver, errors.Trace(err) + } + + updateChangingObjState(changingCol, changingIdxs, model.StatePublic) + ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, originalState != changingCol.State) + if err != nil { + return ver, errors.Trace(err) + } + + // Finish this job. + job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo) + // Refactor the job args to add the old index ids into delete range table. + job.Args = []any{rmIdxIDs, getPartitionIDs(tblInfo)} + modifyColumnEvent := statsutil.NewModifyColumnEvent( + job.SchemaID, + tblInfo, + []*model.ColumnInfo{changingCol}, + ) + asyncNotifyEvent(d, modifyColumnEvent) + default: + err = dbterror.ErrInvalidDDLState.GenWithStackByArgs("column", changingCol.State) + } + + return ver, errors.Trace(err) +} + +func doReorgWorkForModifyColumnMultiSchema(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, tbl table.Table, + oldCol, changingCol *model.ColumnInfo, changingIdxs []*model.IndexInfo) (done bool, ver int64, err error) { + if job.MultiSchemaInfo.Revertible { + done, ver, err = doReorgWorkForModifyColumn(w, d, t, job, tbl, oldCol, changingCol, changingIdxs) + if done { + // We need another round to wait for all the others sub-jobs to finish. + job.MarkNonRevertible() + } + // We need another round to run the reorg process. + return false, ver, err + } + // Non-revertible means all the sub jobs finished. + return true, ver, err +} + +func doReorgWorkForModifyColumn(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, tbl table.Table, + oldCol, changingCol *model.ColumnInfo, changingIdxs []*model.IndexInfo) (done bool, ver int64, err error) { + job.ReorgMeta.ReorgTp = model.ReorgTypeTxn + sctx, err1 := w.sessPool.Get() + if err1 != nil { + err = errors.Trace(err1) + return + } + defer w.sessPool.Put(sctx) + rh := newReorgHandler(sess.NewSession(sctx)) + dbInfo, err := t.GetDatabase(job.SchemaID) + if err != nil { + return false, ver, errors.Trace(err) + } + reorgInfo, err := getReorgInfo(d.jobContext(job.ID, job.ReorgMeta), + d, rh, job, dbInfo, tbl, BuildElements(changingCol, changingIdxs), false) + if err != nil || reorgInfo == nil || reorgInfo.first { + // If we run reorg firstly, we should update the job snapshot version + // and then run the reorg next time. + return false, ver, errors.Trace(err) + } + + // Inject a failpoint so that we can pause here and do verification on other components. + // With a failpoint-enabled version of TiDB, you can trigger this failpoint by the following command: + // enable: curl -X PUT -d "pause" "http://127.0.0.1:10080/fail/github.com/pingcap/tidb/pkg/ddl/mockDelayInModifyColumnTypeWithData". + // disable: curl -X DELETE "http://127.0.0.1:10080/fail/github.com/pingcap/tidb/pkg/ddl/mockDelayInModifyColumnTypeWithData" + failpoint.Inject("mockDelayInModifyColumnTypeWithData", func() {}) + err = w.runReorgJob(reorgInfo, tbl.Meta(), d.lease, func() (addIndexErr error) { + defer util.Recover(metrics.LabelDDL, "onModifyColumn", + func() { + addIndexErr = dbterror.ErrCancelledDDLJob.GenWithStack("modify table `%v` column `%v` panic", tbl.Meta().Name, oldCol.Name) + }, false) + // Use old column name to generate less confusing error messages. + changingColCpy := changingCol.Clone() + changingColCpy.Name = oldCol.Name + return w.updateCurrentElement(tbl, reorgInfo) + }) + if err != nil { + if dbterror.ErrPausedDDLJob.Equal(err) { + return false, ver, nil + } + + if dbterror.ErrWaitReorgTimeout.Equal(err) { + // If timeout, we should return, check for the owner and re-wait job done. + return false, ver, nil + } + if kv.IsTxnRetryableError(err) || dbterror.ErrNotOwner.Equal(err) { + return false, ver, errors.Trace(err) + } + if err1 := rh.RemoveDDLReorgHandle(job, reorgInfo.elements); err1 != nil { + logutil.DDLLogger().Warn("run modify column job failed, RemoveDDLReorgHandle failed, can't convert job to rollback", + zap.String("job", job.String()), zap.Error(err1)) + } + logutil.DDLLogger().Warn("run modify column job failed, convert job to rollback", zap.Stringer("job", job), zap.Error(err)) + job.State = model.JobStateRollingback + return false, ver, errors.Trace(err) + } + return true, ver, nil +} + +func adjustTableInfoAfterModifyColumnWithData(tblInfo *model.TableInfo, pos *ast.ColumnPosition, + oldCol, changingCol *model.ColumnInfo, newName model.CIStr, changingIdxs []*model.IndexInfo) (err error) { + if pos != nil && pos.RelativeColumn != nil && oldCol.Name.L == pos.RelativeColumn.Name.L { + // For cases like `modify column b after b`, it should report this error. + return errors.Trace(infoschema.ErrColumnNotExists.GenWithStackByArgs(oldCol.Name, tblInfo.Name)) + } + internalColName := changingCol.Name + changingCol = replaceOldColumn(tblInfo, oldCol, changingCol, newName) + if len(changingIdxs) > 0 { + updateNewIdxColsNameOffset(changingIdxs, internalColName, changingCol) + indexesToRemove := filterIndexesToRemove(changingIdxs, newName, tblInfo) + replaceOldIndexes(tblInfo, indexesToRemove) + } + if tblInfo.TTLInfo != nil { + updateTTLInfoWhenModifyColumn(tblInfo, oldCol.Name, changingCol.Name) + } + // Move the new column to a correct offset. + destOffset, err := LocateOffsetToMove(changingCol.Offset, pos, tblInfo) + if err != nil { + return errors.Trace(err) + } + tblInfo.MoveColumnInfo(changingCol.Offset, destOffset) + return nil +} + func checkModifyColumnWithGeneratedColumnsConstraint(allCols []*table.Column, oldColName model.CIStr) error { for _, col := range allCols { if col.GeneratedExpr == nil { @@ -327,6 +933,83 @@ func GetModifiableColumnJob( return job, nil } +func needChangeColumnData(oldCol, newCol *model.ColumnInfo) bool { + toUnsigned := mysql.HasUnsignedFlag(newCol.GetFlag()) + originUnsigned := mysql.HasUnsignedFlag(oldCol.GetFlag()) + needTruncationOrToggleSign := func() bool { + return (newCol.GetFlen() > 0 && (newCol.GetFlen() < oldCol.GetFlen() || newCol.GetDecimal() < oldCol.GetDecimal())) || + (toUnsigned != originUnsigned) + } + // Ignore the potential max display length represented by integer's flen, use default flen instead. + defaultOldColFlen, _ := mysql.GetDefaultFieldLengthAndDecimal(oldCol.GetType()) + defaultNewColFlen, _ := mysql.GetDefaultFieldLengthAndDecimal(newCol.GetType()) + needTruncationOrToggleSignForInteger := func() bool { + return (defaultNewColFlen > 0 && defaultNewColFlen < defaultOldColFlen) || (toUnsigned != originUnsigned) + } + + // Deal with the same type. + if oldCol.GetType() == newCol.GetType() { + switch oldCol.GetType() { + case mysql.TypeNewDecimal: + // Since type decimal will encode the precision, frac, negative(signed) and wordBuf into storage together, there is no short + // cut to eliminate data reorg change for column type change between decimal. + return oldCol.GetFlen() != newCol.GetFlen() || oldCol.GetDecimal() != newCol.GetDecimal() || toUnsigned != originUnsigned + case mysql.TypeEnum, mysql.TypeSet: + return IsElemsChangedToModifyColumn(oldCol.GetElems(), newCol.GetElems()) + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong: + return toUnsigned != originUnsigned + case mysql.TypeString: + // Due to the behavior of padding \x00 at binary type, always change column data when binary length changed + if types.IsBinaryStr(&oldCol.FieldType) { + return newCol.GetFlen() != oldCol.GetFlen() + } + } + + return needTruncationOrToggleSign() + } + + if ConvertBetweenCharAndVarchar(oldCol.GetType(), newCol.GetType()) { + return true + } + + // Deal with the different type. + switch oldCol.GetType() { + case mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString, mysql.TypeBlob, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob: + switch newCol.GetType() { + case mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString, mysql.TypeBlob, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob: + return needTruncationOrToggleSign() + } + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong: + switch newCol.GetType() { + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong: + return needTruncationOrToggleSignForInteger() + } + // conversion between float and double needs reorganization, see issue #31372 + } + + return true +} + +// ConvertBetweenCharAndVarchar check whether column converted between char and varchar +// TODO: it is used for plugins. so change plugin's using and remove it. +func ConvertBetweenCharAndVarchar(oldCol, newCol byte) bool { + return types.ConvertBetweenCharAndVarchar(oldCol, newCol) +} + +// IsElemsChangedToModifyColumn check elems changed +func IsElemsChangedToModifyColumn(oldElems, newElems []string) bool { + if len(newElems) < len(oldElems) { + return true + } + for index, oldElem := range oldElems { + newElem := newElems[index] + if oldElem != newElem { + return true + } + } + return false +} + // ProcessColumnCharsetAndCollation process column charset and collation func ProcessColumnCharsetAndCollation(sctx sessionctx.Context, col *table.Column, newCol *table.Column, meta *model.TableInfo, specNewColumn *ast.ColumnDef, schema *model.DBInfo) error { var chs, coll string From d7c6a80bfc79d427f65ae480a02db034c1a3c4e4 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Thu, 1 Aug 2024 19:04:10 +0800 Subject: [PATCH 11/13] change --- pkg/ddl/ddl.go | 15 +++++++------ pkg/ddl/executor.go | 13 +---------- pkg/ddl/job_submitter.go | 48 ++++++++++++++++++++++------------------ 3 files changed, 35 insertions(+), 41 deletions(-) diff --git a/pkg/ddl/ddl.go b/pkg/ddl/ddl.go index 3c4d3ea7235aa..182501cc1699b 100644 --- a/pkg/ddl/ddl.go +++ b/pkg/ddl/ddl.go @@ -250,7 +250,9 @@ type ddl struct { delRangeMgr delRangeManager enableTiFlashPoll *atomicutil.Bool // get notification if any DDL job submitted or finished. - ddlJobNotifyCh chan struct{} + ddlJobNotifyCh chan struct{} + sysTblMgr systable.Manager + minJobIDRefresher *systable.MinJobIDRefresher // globalIDLock locks global id to reduce write conflict. globalIDLock sync.Mutex @@ -702,7 +704,6 @@ func newDDL(ctx context.Context, options ...Option) (*ddl, *executor) { ddlJobNotifyCh: d.ddlJobNotifyCh, mu: &d.mu, globalIDLock: &d.globalIDLock, - stateSyncer: d.stateSyncer, } d.executor = e @@ -738,13 +739,13 @@ func (d *ddl) Start(ctxPool *pools.ResourcePool) error { d.sessPool = sess.NewSessionPool(ctxPool) d.executor.sessPool = d.sessPool - d.executor.sysTblMgr = systable.NewManager(d.sessPool) - d.executor.minJobIDRefresher = systable.NewMinJobIDRefresher(d.executor.sysTblMgr) + d.sysTblMgr = systable.NewManager(d.sessPool) + d.minJobIDRefresher = systable.NewMinJobIDRefresher(d.sysTblMgr) d.wg.Run(func() { - d.executor.limitDDLJobs() + d.limitDDLJobs() }) d.wg.Run(func() { - d.executor.minJobIDRefresher.Start(d.ctx) + d.minJobIDRefresher.Start(d.ctx) }) d.delRangeMgr = d.newDeleteRangeManager(ctxPool == nil) @@ -913,7 +914,7 @@ func (d *ddl) SetHook(h Callback) { } func (d *ddl) GetMinJobIDRefresher() *systable.MinJobIDRefresher { - return d.executor.minJobIDRefresher + return d.minJobIDRefresher } func (d *ddl) startCleanDeadTableLock() { diff --git a/pkg/ddl/executor.go b/pkg/ddl/executor.go index 778fb3a031692..39a8354fae326 100644 --- a/pkg/ddl/executor.go +++ b/pkg/ddl/executor.go @@ -36,8 +36,6 @@ import ( "github.com/pingcap/tidb/pkg/ddl/logutil" "github.com/pingcap/tidb/pkg/ddl/resourcegroup" sess "github.com/pingcap/tidb/pkg/ddl/session" - "github.com/pingcap/tidb/pkg/ddl/syncer" - "github.com/pingcap/tidb/pkg/ddl/systable" ddlutil "github.com/pingcap/tidb/pkg/ddl/util" rg "github.com/pingcap/tidb/pkg/domain/resourcegroup" "github.com/pingcap/tidb/pkg/errctx" @@ -174,6 +172,7 @@ type ExecutorForTest interface { // all fields are shared with ddl now. type executor struct { statsHandle *handle.Handle + sessPool *sess.Pool ctx context.Context uuid string @@ -189,12 +188,6 @@ type executor struct { ddlJobNotifyCh chan struct{} mu *hookStruct // TODO remove it. globalIDLock *sync.Mutex - stateSyncer syncer.StateSyncer - - // those fields are initialized on ddl.Start - sessPool *sess.Pool - sysTblMgr systable.Manager - minJobIDRefresher *systable.MinJobIDRefresher } var _ Executor = (*executor)(nil) @@ -6462,10 +6455,6 @@ func (e *executor) delJobDoneCh(jobID int64) { e.ddlJobDoneChMap.Delete(jobID) } -func (e *executor) initJobDoneCh(jobID int64) { - e.ddlJobDoneChMap.Store(jobID, make(chan struct{}, 1)) -} - func (e *executor) deliverJobTask(task *JobWrapper) { // TODO this might block forever, as the consumer part considers context cancel. e.limitJobCh <- task diff --git a/pkg/ddl/job_submitter.go b/pkg/ddl/job_submitter.go index c45f8faac2887..da78b30bc5a3e 100644 --- a/pkg/ddl/job_submitter.go +++ b/pkg/ddl/job_submitter.go @@ -45,11 +45,11 @@ import ( "go.uber.org/zap" ) -func (e *executor) limitDDLJobs() { +func (d *ddl) limitDDLJobs() { defer util.Recover(metrics.LabelDDL, "limitDDLJobs", nil, true) jobWs := make([]*JobWrapper, 0, batchAddingJobs) - ch := e.limitJobCh + ch := d.limitJobCh for { select { // the channel is never closed @@ -61,15 +61,15 @@ func (e *executor) limitDDLJobs() { for i := 0; i < jobLen; i++ { jobWs = append(jobWs, <-ch) } - e.addBatchDDLJobs(jobWs) - case <-e.ctx.Done(): + d.addBatchDDLJobs(jobWs) + case <-d.ctx.Done(): return } } } // addBatchDDLJobs gets global job IDs and puts the DDL jobs in the DDL queue. -func (e *executor) addBatchDDLJobs(jobWs []*JobWrapper) { +func (d *ddl) addBatchDDLJobs(jobWs []*JobWrapper) { startTime := time.Now() var ( err error @@ -87,9 +87,9 @@ func (e *executor) addBatchDDLJobs(jobWs []*JobWrapper) { jobWs = newWs } } - err = e.addBatchDDLJobs2Table(jobWs) + err = d.addBatchDDLJobs2Table(jobWs) } else { - err = e.addBatchDDLJobs2Queue(jobWs) + err = d.addBatchDDLJobs2Queue(jobWs) } var jobs string for _, jobW := range jobWs { @@ -240,20 +240,20 @@ func mergeCreateTableJobsOfSameSchema(jobWs []*JobWrapper) (*model.Job, error) { } // addBatchDDLJobs2Table gets global job IDs and puts the DDL jobs in the DDL job table. -func (e *executor) addBatchDDLJobs2Table(jobWs []*JobWrapper) error { +func (d *ddl) addBatchDDLJobs2Table(jobWs []*JobWrapper) error { var err error if len(jobWs) == 0 { return nil } - ctx := kv.WithInternalSourceType(e.ctx, kv.InternalTxnDDL) - se, err := e.sessPool.Get() + ctx := kv.WithInternalSourceType(d.ctx, kv.InternalTxnDDL) + se, err := d.sessPool.Get() if err != nil { return errors.Trace(err) } - defer e.sessPool.Put(se) - found, err := e.sysTblMgr.HasFlashbackClusterJob(ctx, e.minJobIDRefresher.GetCurrMinJobID()) + defer d.sessPool.Put(se) + found, err := d.sysTblMgr.HasFlashbackClusterJob(ctx, d.minJobIDRefresher.GetCurrMinJobID()) if err != nil { return errors.Trace(err) } @@ -266,7 +266,7 @@ func (e *executor) addBatchDDLJobs2Table(jobWs []*JobWrapper) error { bdrRole = string(ast.BDRRoleNone) ) - err = kv.RunInNewTxn(ctx, e.store, true, func(_ context.Context, txn kv.Transaction) error { + err = kv.RunInNewTxn(ctx, d.store, true, func(_ context.Context, txn kv.Transaction) error { t := meta.NewMeta(txn) bdrRole, err = t.GetBDRRole() @@ -276,7 +276,7 @@ func (e *executor) addBatchDDLJobs2Table(jobWs []*JobWrapper) error { startTS = txn.StartTS() if variable.DDLForce2Queue.Load() { - if err := e.checkFlashbackJobInQueue(t); err != nil { + if err := d.checkFlashbackJobInQueue(t); err != nil { return err } } @@ -308,7 +308,7 @@ func (e *executor) addBatchDDLJobs2Table(jobWs []*JobWrapper) error { setJobStateToQueueing(job) - if e.stateSyncer.IsUpgradingState() && !hasSysDB(job) { + if d.stateSyncer.IsUpgradingState() && !hasSysDB(job) { if err = pauseRunningJob(sess.NewSession(se), job, model.AdminCommandBySystem); err != nil { logutil.DDLUpgradingLogger().Warn("pause user DDL by system failed", zap.Stringer("job", job), zap.Error(err)) jobW.cacheErr = err @@ -324,18 +324,22 @@ func (e *executor) addBatchDDLJobs2Table(jobWs []*JobWrapper) error { return errors.Trace(err) } for _, jobW := range jobWs { - e.initJobDoneCh(jobW.ID) + d.initJobDoneCh(jobW.ID) } return nil } -func (e *executor) addBatchDDLJobs2Queue(jobWs []*JobWrapper) error { +func (d *ddl) initJobDoneCh(jobID int64) { + d.ddlJobDoneChMap.Store(jobID, make(chan struct{}, 1)) +} + +func (d *ddl) addBatchDDLJobs2Queue(jobWs []*JobWrapper) error { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) // lock to reduce conflict - e.globalIDLock.Lock() - defer e.globalIDLock.Unlock() - return kv.RunInNewTxn(ctx, e.store, true, func(_ context.Context, txn kv.Transaction) error { + d.globalIDLock.Lock() + defer d.globalIDLock.Unlock() + return kv.RunInNewTxn(ctx, d.store, true, func(_ context.Context, txn kv.Transaction) error { t := meta.NewMeta(txn) count := getRequiredGIDCount(jobWs) @@ -345,7 +349,7 @@ func (e *executor) addBatchDDLJobs2Queue(jobWs []*JobWrapper) error { } assignGIDsForJobs(jobWs, ids) - if err := e.checkFlashbackJobInQueue(t); err != nil { + if err := d.checkFlashbackJobInQueue(t); err != nil { return errors.Trace(err) } @@ -374,7 +378,7 @@ func (e *executor) addBatchDDLJobs2Queue(jobWs []*JobWrapper) error { }) } -func (*executor) checkFlashbackJobInQueue(t *meta.Meta) error { +func (*ddl) checkFlashbackJobInQueue(t *meta.Meta) error { jobs, err := t.GetAllDDLJobsInQueue(meta.DefaultJobListKey) if err != nil { return errors.Trace(err) From 7416e3d0ea0a405cc4b5f29fb5b3d32fe9df2484 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Thu, 1 Aug 2024 19:05:10 +0800 Subject: [PATCH 12/13] change --- pkg/ddl/executor.go | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/pkg/ddl/executor.go b/pkg/ddl/executor.go index 39a8354fae326..706d7c094faed 100644 --- a/pkg/ddl/executor.go +++ b/pkg/ddl/executor.go @@ -171,8 +171,8 @@ type ExecutorForTest interface { // all fields are shared with ddl now. type executor struct { - statsHandle *handle.Handle sessPool *sess.Pool + statsHandle *handle.Handle ctx context.Context uuid string @@ -1382,13 +1382,13 @@ func (e *executor) FlashbackCluster(ctx sessionctx.Context, flashbackTS uint64) Args: []any{ flashbackTS, map[string]any{}, - true, /* tidb_gc_enable */ - variable.On, /* tidb_enable_auto_analyze */ - variable.Off, /* tidb_super_read_only */ - 0, /* totalRegions */ - 0, /* startTS */ - 0, /* commitTS */ - variable.On, /* tidb_ttl_job_enable */ + true, /* tidb_gc_enable */ + variable.On, /* tidb_enable_auto_analyze */ + variable.Off, /* tidb_super_read_only */ + 0, /* totalRegions */ + 0, /* startTS */ + 0, /* commitTS */ + variable.On, /* tidb_ttl_job_enable */ []kv.KeyRange{} /* flashback key_ranges */}, CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, // FLASHBACK CLUSTER affects all schemas and tables. From 957a140f2e75e687c5c69976fd6fddfdfe706550 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Thu, 1 Aug 2024 19:14:05 +0800 Subject: [PATCH 13/13] lint --- pkg/ddl/executor.go | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/pkg/ddl/executor.go b/pkg/ddl/executor.go index 706d7c094faed..eeb03c47d7229 100644 --- a/pkg/ddl/executor.go +++ b/pkg/ddl/executor.go @@ -1382,13 +1382,13 @@ func (e *executor) FlashbackCluster(ctx sessionctx.Context, flashbackTS uint64) Args: []any{ flashbackTS, map[string]any{}, - true, /* tidb_gc_enable */ - variable.On, /* tidb_enable_auto_analyze */ - variable.Off, /* tidb_super_read_only */ - 0, /* totalRegions */ - 0, /* startTS */ - 0, /* commitTS */ - variable.On, /* tidb_ttl_job_enable */ + true, /* tidb_gc_enable */ + variable.On, /* tidb_enable_auto_analyze */ + variable.Off, /* tidb_super_read_only */ + 0, /* totalRegions */ + 0, /* startTS */ + 0, /* commitTS */ + variable.On, /* tidb_ttl_job_enable */ []kv.KeyRange{} /* flashback key_ranges */}, CDCWriteSource: ctx.GetSessionVars().CDCWriteSource, // FLASHBACK CLUSTER affects all schemas and tables.