Skip to content

Commit

Permalink
Merge branch 'master' into global_index_drop_partition
Browse files Browse the repository at this point in the history
  • Loading branch information
AilinKid authored Oct 14, 2020
2 parents ddd12da + 79711ad commit ff7f6a6
Show file tree
Hide file tree
Showing 136 changed files with 49,634 additions and 1,072 deletions.
4 changes: 1 addition & 3 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -147,10 +147,8 @@ testSuite:
@echo "testSuite"
./tools/check/check_testSuite.sh

clean:
clean: failpoint-disable
$(GO) clean -i ./...
rm -rf *.out
rm -rf parser

# Split tests for CI to run `make test` in parallel.
test: test_part_1 test_part_2
Expand Down
5 changes: 5 additions & 0 deletions cmd/explaintest/r/explain.result
Original file line number Diff line number Diff line change
Expand Up @@ -39,3 +39,8 @@ StreamAgg_8 8000.00 root group by:Column#7, funcs:group_concat(Column#5, Column
└─TableReader_15 10000.00 root data:TableFullScan_14
└─TableFullScan_14 10000.00 cop[tikv] table:t keep order:true, stats:pseudo
drop table t;
drop view if exists v;
create view v as select cast(replace(substring_index(substring_index("",',',1),':',-1),'"','') as CHAR(32)) as event_id;
desc v;
Field Type Null Key Default Extra
event_id varchar(32) YES NULL
21 changes: 11 additions & 10 deletions cmd/explaintest/r/explain_easy.result
Original file line number Diff line number Diff line change
Expand Up @@ -556,28 +556,29 @@ Projection_4 10.00 root plus(1, test.t.nb)->Column#5
explain select * from t ta left outer join t tb on ta.nb = tb.nb and ta.a > 1 where ifnull(ta.nb, 1) or ta.nb is null;
id estRows task access object operator info
HashJoin_7 8320.83 root left outer join, equal:[eq(test.t.nb, test.t.nb)], left cond:[gt(test.t.a, 1)]
├─TableReader_13(Build) 10000.00 root data:TableFullScan_12
│ └─TableFullScan_12 10000.00 cop[tikv] table:tb keep order:false, stats:pseudo
├─TableReader_14(Build) 6656.67 root data:Selection_13
│ └─Selection_13 6656.67 cop[tikv] or(test.t.nb, 0)
│ └─TableFullScan_12 10000.00 cop[tikv] table:tb keep order:false, stats:pseudo
└─TableReader_11(Probe) 6656.67 root data:Selection_10
└─Selection_10 6656.67 cop[tikv] or(test.t.nb, isnull(test.t.nb))
└─Selection_10 6656.67 cop[tikv] or(test.t.nb, 0)
└─TableFullScan_9 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo
explain select * from t ta right outer join t tb on ta.nb = tb.nb and ta.a > 1 where ifnull(tb.nb, 1) or tb.nb is null;
id estRows task access object operator info
HashJoin_7 6656.67 root right outer join, equal:[eq(test.t.nb, test.t.nb)]
├─TableReader_11(Build) 3333.33 root data:Selection_10
│ └─Selection_10 3333.33 cop[tikv] gt(test.t.a, 1)
├─TableReader_11(Build) 2218.89 root data:Selection_10
│ └─Selection_10 2218.89 cop[tikv] gt(test.t.a, 1), or(test.t.nb, 0)
│ └─TableFullScan_9 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo
└─TableReader_14(Probe) 6656.67 root data:Selection_13
└─Selection_13 6656.67 cop[tikv] or(test.t.nb, isnull(test.t.nb))
└─Selection_13 6656.67 cop[tikv] or(test.t.nb, 0)
└─TableFullScan_12 10000.00 cop[tikv] table:tb keep order:false, stats:pseudo
explain select * from t ta inner join t tb on ta.nb = tb.nb and ta.a > 1 where ifnull(tb.nb, 1) or tb.nb is null;
id estRows task access object operator info
HashJoin_9 4166.67 root inner join, equal:[eq(test.t.nb, test.t.nb)]
├─TableReader_12(Build) 3333.33 root data:Selection_11
│ └─Selection_11 3333.33 cop[tikv] gt(test.t.a, 1)
HashJoin_9 2773.61 root inner join, equal:[eq(test.t.nb, test.t.nb)]
├─TableReader_12(Build) 2218.89 root data:Selection_11
│ └─Selection_11 2218.89 cop[tikv] gt(test.t.a, 1), or(test.t.nb, 0)
│ └─TableFullScan_10 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo
└─TableReader_15(Probe) 6656.67 root data:Selection_14
└─Selection_14 6656.67 cop[tikv] or(test.t.nb, isnull(test.t.nb))
└─Selection_14 6656.67 cop[tikv] or(test.t.nb, 0)
└─TableFullScan_13 10000.00 cop[tikv] table:tb keep order:false, stats:pseudo
explain select ifnull(t.nc, 1) in (select count(*) from t s , t t1 where s.a = t.a and s.a = t1.a) from t;
id estRows task access object operator info
Expand Down
4 changes: 2 additions & 2 deletions cmd/explaintest/r/partition_pruning.result
Original file line number Diff line number Diff line change
Expand Up @@ -2333,8 +2333,8 @@ TableReader_7 250.00 root partition:all data:Selection_6
└─TableFullScan_5 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo
explain select * from t1 where a>=0 and a <= 0xFFFFFFFFFFFFFFFF;
id estRows task access object operator info
TableReader_7 250.00 root partition:all data:Selection_6
└─Selection_6 250.00 cop[tikv] ge(test.t1.a, 0), le(test.t1.a, 18446744073709551615)
TableReader_7 3323.33 root partition:all data:Selection_6
└─Selection_6 3323.33 cop[tikv] le(test.t1.a, 18446744073709551615)
└─TableFullScan_5 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo
drop table t1;
create table t1 (a bigint) partition by range(a+0) (
Expand Down
4 changes: 4 additions & 0 deletions cmd/explaintest/t/explain.test
Original file line number Diff line number Diff line change
Expand Up @@ -14,3 +14,7 @@ set session tidb_hashagg_final_concurrency = 1;
explain select group_concat(a) from t group by id;
explain select group_concat(a, b) from t group by id;
drop table t;

drop view if exists v;
create view v as select cast(replace(substring_index(substring_index("",',',1),':',-1),'"','') as CHAR(32)) as event_id;
desc v;
9 changes: 8 additions & 1 deletion config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ import (
tracing "github.com/uber/jaeger-client-go/config"

"go.uber.org/zap"
"google.golang.org/grpc/encoding/gzip"
)

// Config number limitations
Expand Down Expand Up @@ -493,6 +494,8 @@ type TiKVClient struct {
// After having pinged for keepalive check, the client waits for a duration of Timeout in seconds
// and if no activity is seen even after that the connection is closed.
GrpcKeepAliveTimeout uint `toml:"grpc-keepalive-timeout" json:"grpc-keepalive-timeout"`
// GrpcCompressionType is the compression type for gRPC channel: none or gzip.
GrpcCompressionType string `toml:"grpc-compression-type" json:"grpc-compression-type"`
// CommitTimeout is the max time which command 'commit' will wait.
CommitTimeout string `toml:"commit-timeout" json:"commit-timeout"`
AsyncCommit AsyncCommit `toml:"async-commit" json:"async-commit"`
Expand Down Expand Up @@ -693,6 +696,7 @@ var defaultConf = Config{
GrpcConnectionCount: 4,
GrpcKeepAliveTime: 10,
GrpcKeepAliveTimeout: 3,
GrpcCompressionType: "none",
CommitTimeout: "41s",
AsyncCommit: AsyncCommit{
Enable: false,
Expand Down Expand Up @@ -882,7 +886,7 @@ func (c *Config) Valid() error {
if c.Security.SkipGrantTable && !hasRootPrivilege() {
return fmt.Errorf("TiDB run with skip-grant-table need root privilege")
}
if _, ok := ValidStorage[c.Store]; !ok {
if !ValidStorage[c.Store] {
nameList := make([]string, 0, len(ValidStorage))
for k, v := range ValidStorage {
if v {
Expand Down Expand Up @@ -918,6 +922,9 @@ func (c *Config) Valid() error {
if c.TiKVClient.GrpcConnectionCount == 0 {
return fmt.Errorf("grpc-connection-count should be greater than 0")
}
if c.TiKVClient.GrpcCompressionType != "none" && c.TiKVClient.GrpcCompressionType != gzip.Name {
return fmt.Errorf("grpc-compression-type should be none or %s, but got %s", gzip.Name, c.TiKVClient.GrpcCompressionType)
}

if c.Performance.TxnTotalSizeLimit > 10<<30 {
return fmt.Errorf("txn-total-size-limit should be less than %d", 10<<30)
Expand Down
3 changes: 3 additions & 0 deletions config/config.toml.example
Original file line number Diff line number Diff line change
Expand Up @@ -353,6 +353,9 @@ grpc-keepalive-time = 10
# and if no activity is seen even after that the connection is closed.
grpc-keepalive-timeout = 3

# The compression type for gRPC channel: none or gzip.
grpc-compression-type = "none"

# Max time for commit command, must be twice bigger than raft election timeout.
commit-timeout = "41s"

Expand Down
5 changes: 3 additions & 2 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -516,17 +516,18 @@ func checkDropColumnForStatePublic(tblInfo *model.TableInfo, colInfo *model.Colu
// When the dropping column has not-null flag and it hasn't the default value, we can backfill the column value like "add column".
// NOTE: If the state of StateWriteOnly can be rollbacked, we'd better reconsider the original default value.
// And we need consider the column without not-null flag.
if colInfo.OriginDefaultValue == nil && mysql.HasNotNullFlag(colInfo.Flag) {
if colInfo.GetOriginDefaultValue() == nil && mysql.HasNotNullFlag(colInfo.Flag) {
// If the column is timestamp default current_timestamp, and DDL owner is new version TiDB that set column.Version to 1,
// then old TiDB update record in the column write only stage will uses the wrong default value of the dropping column.
// Because new version of the column default value is UTC time, but old version TiDB will think the default value is the time in system timezone.
// But currently will be ok, because we can't cancel the drop column job when the job is running,
// so the column will be dropped succeed and client will never see the wrong default value of the dropped column.
// More info about this problem, see PR#9115.
colInfo.OriginDefaultValue, err = generateOriginDefaultValue(colInfo)
originDefVal, err := generateOriginDefaultValue(colInfo)
if err != nil {
return err
}
return colInfo.SetOriginDefaultValue(originDefVal)
}
return nil
}
Expand Down
14 changes: 14 additions & 0 deletions ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1118,6 +1118,18 @@ func (s *testIntegrationSuite5) TestBitDefaultValue(c *C) {
tk.MustQuery("select c from t_bit").Check(testkit.Rows("\x19\xb9"))
tk.MustExec("update t_bit set c = b'11100000000111'")
tk.MustQuery("select c from t_bit").Check(testkit.Rows("\x38\x07"))
tk.MustExec("drop table t_bit")

tk.MustExec("create table t_bit (a int)")
tk.MustExec("insert into t_bit value (1)")
tk.MustExec("alter table t_bit add column b bit(1) default b'0';")
tk.MustExec("alter table t_bit modify column b bit(1) default b'1';")
tk.MustQuery("select b from t_bit").Check(testkit.Rows("\x00"))
tk.MustExec("drop table t_bit")

tk.MustExec("create table t_bit (a bit);")
tk.MustExec("insert into t_bit values (null);")
tk.MustQuery("select count(*) from t_bit where a is null;").Check(testkit.Rows("1"))

tk.MustExec(`create table testalltypes1 (
field_1 bit default 1,
Expand Down Expand Up @@ -2188,6 +2200,8 @@ func (s *testIntegrationSuite7) TestCreateExpressionIndexError(c *C) {
tk.MustExec("drop table if exists t;")
tk.MustGetErrCode("create table t (j json, key k (((j,j))))", errno.ErrFunctionalIndexRowValueIsNotAllowed)
tk.MustExec("create table t (j json, key k ((j+1),(j+1)))")

tk.MustGetErrCode("create table t1 (col1 int, index ((concat(''))));", errno.ErrWrongKeyColumnFunctionalIndex)
}

func (s *testIntegrationSuite7) TestAddExpressionIndexOnPartition(c *C) {
Expand Down
1 change: 1 addition & 0 deletions ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -110,6 +110,7 @@ type DDL interface {
RepairTable(ctx sessionctx.Context, table *ast.TableName, createStmt *ast.CreateTableStmt) error
CreateSequence(ctx sessionctx.Context, stmt *ast.CreateSequenceStmt) error
DropSequence(ctx sessionctx.Context, tableIdent ast.Ident, ifExists bool) (err error)
AlterSequence(ctx sessionctx.Context, stmt *ast.AlterSequenceStmt) error

// CreateSchemaWithInfo creates a database (schema) given its database info.
//
Expand Down
93 changes: 82 additions & 11 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -353,6 +353,25 @@ func ResolveCharsetCollation(charsetOpts ...ast.CharsetOpt) (string, string, err
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(colDef *ast.ColumnDef, chs, coll string) (newChs string, newColl string) {
ignoreBinFlag := colDef.Tp.Charset != "" && (colDef.Tp.Collate != "" || containsColumnOption(colDef, ast.ColumnOptionCollate))
if ignoreBinFlag {
return chs, coll
}
needOverwriteBinColl := types.IsString(colDef.Tp.Tp) && mysql.HasBinaryFlag(colDef.Tp.Flag)
if needOverwriteBinColl {
newColl, err := charset.GetDefaultCollation(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,
Expand Down Expand Up @@ -412,6 +431,7 @@ func buildColumnAndConstraint(
ast.CharsetOpt{Chs: chs, Col: coll},
ast.CharsetOpt{Chs: tblCharset, Col: tblCollate},
)
chs, coll = OverwriteCollationWithBinaryFlag(colDef, chs, coll)
if err != nil {
return nil, nil, errors.Trace(err)
}
Expand Down Expand Up @@ -504,8 +524,12 @@ func isExplicitTimeStamp() bool {

// 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() && col.Charset == charset.CharsetBin {
col.Flag |= mysql.BinaryFlag
if col.FieldType.EvalType().IsStringKind() {
if col.Charset == charset.CharsetBin {
col.Flag |= mysql.BinaryFlag
} else {
col.Flag &= ^mysql.BinaryFlag
}
}
if col.Tp == mysql.TypeBit {
// For BIT field, it's charset is binary but does not have binary flag.
Expand Down Expand Up @@ -2619,11 +2643,13 @@ func checkAndCreateNewColumn(ctx sessionctx.Context, ti ast.Ident, schema *model
return nil, errors.Trace(err)
}

col.OriginDefaultValue, err = generateOriginDefaultValue(col.ToInfo())
originDefVal, err := generateOriginDefaultValue(col.ToInfo())
if err != nil {
return nil, errors.Trace(err)
}
return col, nil

err = col.SetOriginDefaultValue(originDefVal)
return col, err
}

// AddColumn will add a new column to the table.
Expand Down Expand Up @@ -3539,12 +3565,13 @@ func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, or
// 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,
FieldType: *specNewColumn.Tp,
Name: newColName,
Version: col.Version,
Offset: col.Offset,
State: col.State,
OriginDefaultValue: col.OriginDefaultValue,
OriginDefaultValueBit: col.OriginDefaultValueBit,
FieldType: *specNewColumn.Tp,
Name: newColName,
Version: col.Version,
})

var chs, coll string
Expand All @@ -3564,6 +3591,7 @@ func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, or
ast.CharsetOpt{Chs: t.Meta().Charset, Col: t.Meta().Collate},
ast.CharsetOpt{Chs: schema.Charset, Col: schema.Collate},
)
chs, coll = OverwriteCollationWithBinaryFlag(specNewColumn, chs, coll)
if err != nil {
return nil, errors.Trace(err)
}
Expand Down Expand Up @@ -4543,7 +4571,7 @@ func buildHiddenColumnInfo(ctx sessionctx.Context, indexPartSpecifications []*as
if err = checkDependedColExist(checkDependencies, existCols); err != nil {
return nil, errors.Trace(err)
}
if err = checkAutoIncrementRef("", colInfo.Dependences, tblInfo); err != nil {
if err = checkExpressionIndexAutoIncrement(indexName.O, colInfo.Dependences, tblInfo); err != nil {
return nil, errors.Trace(err)
}
idxPart.Expr = nil
Expand Down Expand Up @@ -5291,6 +5319,49 @@ func (d *ddl) CreateSequence(ctx sessionctx.Context, stmt *ast.CreateSequenceStm
return d.CreateTableWithInfo(ctx, ident.Schema, tbInfo, onExist, false /*tryRetainID*/)
}

func (d *ddl) AlterSequence(ctx sessionctx.Context, stmt *ast.AlterSequenceStmt) error {
ident := ast.Ident{Name: stmt.Name.Name, Schema: stmt.Name.Schema}
is := d.GetInfoSchemaWithInterceptor(ctx)
// Check schema existence.
db, ok := is.SchemaByName(ident.Schema)
if !ok {
return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ident.Schema)
}
// Check table existence.
tbl, err := is.TableByName(ident.Schema, ident.Name)
if err != nil {
if stmt.IfExists {
ctx.GetSessionVars().StmtCtx.AppendNote(err)
return nil
}
return err
}
if !tbl.Meta().IsSequence() {
return ErrWrongObject.GenWithStackByArgs(ident.Schema, ident.Name, "SEQUENCE")
}

// Validate the new sequence option value in old sequenceInfo.
oldSequenceInfo := tbl.Meta().Sequence
copySequenceInfo := *oldSequenceInfo
_, _, err = alterSequenceOptions(stmt.SeqOptions, ident, &copySequenceInfo)
if err != nil {
return err
}

job := &model.Job{
SchemaID: db.ID,
TableID: tbl.Meta().ID,
SchemaName: db.Name.L,
Type: model.ActionAlterSequence,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{ident, stmt.SeqOptions},
}

err = d.doDDLJob(ctx, job)
err = d.callHookOnChanged(err)
return errors.Trace(err)
}

func (d *ddl) DropSequence(ctx sessionctx.Context, ti ast.Ident, ifExists bool) (err error) {
schema, tbl, err := d.getSchemaAndTableByIdent(ctx, ti)
if err != nil {
Expand Down
2 changes: 2 additions & 0 deletions ddl/ddl_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -685,6 +685,8 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64,
ver, err = onAlterIndexVisibility(t, job)
case model.ActionAlterTableAlterPartition:
ver, err = onAlterTablePartition(t, job)
case model.ActionAlterSequence:
ver, err = onAlterSequence(t, job)
default:
// Invalid job, cancel it.
job.State = model.JobStateCancelled
Expand Down
8 changes: 7 additions & 1 deletion ddl/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,8 @@ var (
errOnlyOnRangeListPartition = terror.ClassDDL.New(mysql.ErrOnlyOnRangeListPartition, mysql.MySQLErrName[mysql.ErrOnlyOnRangeListPartition])
// errWrongKeyColumn is for table column cannot be indexed.
errWrongKeyColumn = terror.ClassDDL.New(mysql.ErrWrongKeyColumn, mysql.MySQLErrName[mysql.ErrWrongKeyColumn])
// errWrongKeyColumnFunctionalIndex is for expression cannot be indexed.
errWrongKeyColumnFunctionalIndex = terror.ClassDDL.New(mysql.ErrWrongKeyColumnFunctionalIndex, mysql.MySQLErrName[mysql.ErrWrongKeyColumnFunctionalIndex])
// errWrongFKOptionForGeneratedColumn is for wrong foreign key reference option on generated columns.
errWrongFKOptionForGeneratedColumn = terror.ClassDDL.New(mysql.ErrWrongFKOptionForGeneratedColumn, mysql.MySQLErrName[mysql.ErrWrongFKOptionForGeneratedColumn])
// ErrUnsupportedOnGeneratedColumn is for unsupported actions on generated columns.
Expand All @@ -81,6 +83,8 @@ var (
ErrInvalidDefaultValue = terror.ClassDDL.New(mysql.ErrInvalidDefault, mysql.MySQLErrName[mysql.ErrInvalidDefault])
// ErrGeneratedColumnRefAutoInc forbids to refer generated columns to auto-increment columns .
ErrGeneratedColumnRefAutoInc = terror.ClassDDL.New(mysql.ErrGeneratedColumnRefAutoInc, mysql.MySQLErrName[mysql.ErrGeneratedColumnRefAutoInc])
// ErrExpressionIndexCanNotRefer forbids to refer expression index to auto-increment column.
ErrExpressionIndexCanNotRefer = terror.ClassDDL.New(mysql.ErrFunctionalIndexRefAutoIncrement, mysql.MySQLErrName[mysql.ErrFunctionalIndexRefAutoIncrement])
// ErrUnsupportedAddPartition returns for does not support add partitions.
ErrUnsupportedAddPartition = terror.ClassDDL.New(mysql.ErrUnsupportedDDLOperation, fmt.Sprintf(mysql.MySQLErrName[mysql.ErrUnsupportedDDLOperation], "add partitions"))
// ErrUnsupportedCoalescePartition returns for does not support coalesce partitions.
Expand Down Expand Up @@ -200,8 +204,10 @@ var (
ErrFunctionalIndexOnField = terror.ClassDDL.New(mysql.ErrFunctionalIndexOnField, mysql.MySQLErrName[mysql.ErrFunctionalIndexOnField])
// ErrInvalidAutoRandom returns when auto_random is used incorrectly.
ErrInvalidAutoRandom = terror.ClassDDL.New(mysql.ErrInvalidAutoRandom, mysql.MySQLErrName[mysql.ErrInvalidAutoRandom])
// ErrUnsupportedConstraintCheck returns when use ADD CONSTRAINT CHECK
// ErrUnsupportedConstraintCheck returns when use ADD CONSTRAINT CHECK.
ErrUnsupportedConstraintCheck = terror.ClassDDL.New(mysql.ErrUnsupportedConstraintCheck, mysql.MySQLErrName[mysql.ErrUnsupportedConstraintCheck])
// ErrDerivedMustHaveAlias returns when a sub select statement does not have a table alias.
ErrDerivedMustHaveAlias = terror.ClassDDL.New(mysql.ErrDerivedMustHaveAlias, mysql.MySQLErrName[mysql.ErrDerivedMustHaveAlias])

// ErrSequenceRunOut returns when the sequence has been run out.
ErrSequenceRunOut = terror.ClassDDL.New(mysql.ErrSequenceRunOut, mysql.MySQLErrName[mysql.ErrSequenceRunOut])
Expand Down
11 changes: 11 additions & 0 deletions ddl/generated_column.go
Original file line number Diff line number Diff line change
Expand Up @@ -331,3 +331,14 @@ func checkAutoIncrementRef(name string, dependencies map[string]struct{}, tbInfo
}
return nil
}

// checkExpressionIndexAutoIncrement checks if an generated column depends on an auto-increment column and raises an error if so.
func checkExpressionIndexAutoIncrement(name string, dependencies map[string]struct{}, tbInfo *model.TableInfo) error {
exists, autoIncrementColumn := infoschema.HasAutoIncrementColumn(tbInfo)
if exists {
if _, found := dependencies[autoIncrementColumn]; found {
return ErrExpressionIndexCanNotRefer.GenWithStackByArgs(name)
}
}
return nil
}
Loading

0 comments on commit ff7f6a6

Please sign in to comment.