From f036eecc9dab4e3c703337f9dab62359079675af Mon Sep 17 00:00:00 2001 From: wjHuang Date: Tue, 7 Jan 2020 10:29:39 +0800 Subject: [PATCH] ddl: support create/drop expression index (#14117) --- ddl/column.go | 3 + ddl/db_change_test.go | 10 ++++ ddl/db_integration_test.go | 102 +++++++++++++++++++++++++++++++- ddl/db_test.go | 30 +++++++--- ddl/ddl.go | 6 ++ ddl/ddl_api.go | 115 +++++++++++++++++++++++++++++++++---- ddl/generated_column.go | 6 +- ddl/index.go | 104 +++++++++++++++++++++++++-------- ddl/partition.go | 4 +- ddl/rollingback.go | 16 +++--- executor/show.go | 11 +++- executor/show_test.go | 13 +++++ go.mod | 1 - go.sum | 12 +++- planner/core/preprocess.go | 16 ++++-- util/admin/admin.go | 9 ++- util/admin/admin_test.go | 2 +- 17 files changed, 390 insertions(+), 70 deletions(-) diff --git a/ddl/column.go b/ddl/column.go index 90d8703d9c219..f38a774d57634 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -77,6 +77,9 @@ func adjustColumnInfoInDropColumn(tblInfo *model.TableInfo, offset int) { oldCols[i].Offset = i - 1 } oldCols[offset].Offset = len(oldCols) - 1 + // For expression index, we drop hidden columns and index simultaneously. + // So we need to change the offset of expression index. + offsetChanged[offset] = len(oldCols) - 1 // Update index column offset info. // TODO: There may be some corner cases for index column offsets, we may check this later. for _, idx := range tblInfo.Indices { diff --git a/ddl/db_change_test.go b/ddl/db_change_test.go index b4da6d65905f8..6b3fc8b4e431f 100644 --- a/ddl/db_change_test.go +++ b/ddl/db_change_test.go @@ -773,6 +773,16 @@ func (s *testStateChangeSuite) TestParallelAlterAddIndex(c *C) { s.testControlParallelExecSQL(c, sql1, sql2, f) } +func (s *testStateChangeSuite) TestParallelAlterAddExpressionIndex(c *C) { + sql1 := "ALTER TABLE t add index expr_index_b((b+1));" + sql2 := "CREATE INDEX expr_index_b ON t ((c+1));" + f := func(c *C, err1, err2 error) { + c.Assert(err1, IsNil) + c.Assert(err2.Error(), Equals, "[ddl:1061]index already exist expr_index_b") + } + s.testControlParallelExecSQL(c, sql1, sql2, f) +} + func (s *testStateChangeSuite) TestParallelAddPrimaryKey(c *C) { sql1 := "ALTER TABLE t add primary key index_b(b);" sql2 := "ALTER TABLE t add primary key index_b(c);" diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index cb416f0865dbe..9c239f1ad7367 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -51,6 +51,7 @@ var _ = Suite(&testIntegrationSuite2{&testIntegrationSuite{}}) var _ = Suite(&testIntegrationSuite3{&testIntegrationSuite{}}) var _ = Suite(&testIntegrationSuite4{&testIntegrationSuite{}}) var _ = Suite(&testIntegrationSuite5{&testIntegrationSuite{}}) +var _ = Suite(&testIntegrationSuite6{&testIntegrationSuite{}}) type testIntegrationSuite struct { lease time.Duration @@ -121,6 +122,7 @@ func (s *testIntegrationSuite2) TearDownTest(c *C) { type testIntegrationSuite3 struct{ *testIntegrationSuite } type testIntegrationSuite4 struct{ *testIntegrationSuite } type testIntegrationSuite5 struct{ *testIntegrationSuite } +type testIntegrationSuite6 struct{ *testIntegrationSuite } func (s *testIntegrationSuite5) TestNoZeroDateMode(c *C) { tk := testkit.NewTestKit(c, s.store) @@ -1010,7 +1012,7 @@ func (s *testIntegrationSuite5) TestBackwardCompatibility(c *C) { unique := false indexName := model.NewCIStr("idx_b") - idxColName := &ast.IndexPartSpecification{ + indexPartSpecification := &ast.IndexPartSpecification{ Column: &ast.ColumnName{ Schema: schemaName, Table: tableName, @@ -1018,14 +1020,14 @@ func (s *testIntegrationSuite5) TestBackwardCompatibility(c *C) { }, Length: types.UnspecifiedLength, } - idxColNames := []*ast.IndexPartSpecification{idxColName} + indexPartSpecifications := []*ast.IndexPartSpecification{indexPartSpecification} var indexOption *ast.IndexOption job := &model.Job{ SchemaID: schema.ID, TableID: tbl.Meta().ID, Type: model.ActionAddIndex, BinlogInfo: &model.HistoryInfo{}, - Args: []interface{}{unique, indexName, idxColNames, indexOption}, + Args: []interface{}{unique, indexName, indexPartSpecifications, indexOption}, } txn, err := s.store.Begin() c.Assert(err, IsNil) @@ -1896,3 +1898,97 @@ func (s *testIntegrationSuite3) TestParserIssue284(c *C) { tk.MustExec("drop table test.t_parser_issue_284") tk.MustExec("drop table test.t_parser_issue_284_2") } + +func (s *testIntegrationSuite6) TestAddExpressionIndex(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + + tk.MustGetErrCode("create table t(a int, b int, index((a+b)));", mysql.ErrNotSupportedYet) + + tk.MustExec("create table t (a int, b real);") + tk.MustExec("insert into t values (1, 2.1);") + tk.MustExec("alter table t add index idx((a+b));") + + tblInfo, err := s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + columns := tblInfo.Meta().Columns + c.Assert(len(columns), Equals, 3) + c.Assert(columns[2].Hidden, IsTrue) + + tk.MustQuery("select * from t;").Check(testkit.Rows("1 2.1")) + + tk.MustExec("alter table t add index idx_multi((a+b),(a+1), b);") + tblInfo, err = s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + columns = tblInfo.Meta().Columns + c.Assert(len(columns), Equals, 5) + c.Assert(columns[3].Hidden, IsTrue) + c.Assert(columns[4].Hidden, IsTrue) + + tk.MustQuery("select * from t;").Check(testkit.Rows("1 2.1")) + + tk.MustExec("alter table t drop index idx;") + tblInfo, err = s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + columns = tblInfo.Meta().Columns + c.Assert(len(columns), Equals, 4) + + tk.MustQuery("select * from t;").Check(testkit.Rows("1 2.1")) + + tk.MustExec("alter table t drop index idx_multi;") + tblInfo, err = s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + columns = tblInfo.Meta().Columns + c.Assert(len(columns), Equals, 2) + + tk.MustQuery("select * from t;").Check(testkit.Rows("1 2.1")) + + // Test for error + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a int, b real);") + tk.MustGetErrCode("alter table t add primary key ((a+b));", mysql.ErrFunctionalIndexPrimaryKey) + tk.MustGetErrCode("alter table t add index ((rand()));", mysql.ErrGeneratedColumnFunctionIsNotAllowed) + tk.MustGetErrCode("alter table t add index ((now()+1));", mysql.ErrGeneratedColumnFunctionIsNotAllowed) + + tk.MustExec("alter table t add column (_V$_idx_0 int);") + tk.MustGetErrCode("alter table t add index idx((a+1));", mysql.ErrDupFieldName) + tk.MustExec("alter table t drop column _V$_idx_0;") + tk.MustExec("alter table t add index idx((a+1));") + tk.MustGetErrCode("alter table t add column (_V$_idx_0 int);", mysql.ErrDupFieldName) + tk.MustExec("alter table t drop index idx;") + tk.MustExec("alter table t add column (_V$_idx_0 int);") + + tk.MustExec("alter table t add column (_V$_expression_index_0 int);") + tk.MustGetErrCode("alter table t add index ((a+1));", mysql.ErrDupFieldName) + tk.MustExec("alter table t drop column _V$_expression_index_0;") + tk.MustExec("alter table t add index ((a+1));") + tk.MustGetErrCode("alter table t drop column _V$_expression_index_0;", mysql.ErrCantDropFieldOrKey) + tk.MustGetErrCode("alter table t add column e int as (_V$_expression_index_0 + 1);", mysql.ErrBadField) +} + +func (s *testIntegrationSuite6) TestAddExpressionIndexOnPartition(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + s.tk.MustExec(`create table t( + a int, + b varchar(100), + c int, + PARTITION BY RANGE ( a ) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11), + PARTITION p2 VALUES LESS THAN (16), + PARTITION p3 VALUES LESS THAN (21) + );`) + tk.MustExec("insert into t values (1, 'test', 2), (12, 'test', 3), (15, 'test', 10), (20, 'test', 20);") + tk.MustExec("alter table t add index idx((a+c));") + + tblInfo, err := s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + columns := tblInfo.Meta().Columns + c.Assert(len(columns), Equals, 4) + c.Assert(columns[3].Hidden, IsTrue) + + tk.MustQuery("select * from t;").Check(testkit.Rows("1 'test' 2", "12 'test' 3", "15 'test' 10", "20 'test' 20")) +} diff --git a/ddl/db_test.go b/ddl/db_test.go index 44e254376c155..48dd6102135a9 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -66,6 +66,7 @@ var _ = Suite(&testDBSuite2{&testDBSuite{}}) var _ = Suite(&testDBSuite3{&testDBSuite{}}) var _ = Suite(&testDBSuite4{&testDBSuite{}}) var _ = Suite(&testDBSuite5{&testDBSuite{}}) +var _ = Suite(&testDBSuite6{&testDBSuite{}}) const defaultBatchSize = 1024 @@ -132,6 +133,7 @@ type testDBSuite2 struct{ *testDBSuite } type testDBSuite3 struct{ *testDBSuite } type testDBSuite4 struct{ *testDBSuite } type testDBSuite5 struct{ *testDBSuite } +type testDBSuite6 struct{ *testDBSuite } func (s *testDBSuite4) TestAddIndexWithPK(c *C) { s.tk = testkit.NewTestKit(c, s.store) @@ -227,7 +229,7 @@ func (s *testDBSuite5) TestAddPrimaryKeyRollback1(c *C) { idxName := "PRIMARY" addIdxSQL := "alter table t1 add primary key c3_index (c3);" errMsg := "[kv:1062]Duplicate entry '' for key 'PRIMARY'" - testAddIndexRollback(c, s.store, s.lease, idxName, addIdxSQL, errMsg, hasNullValsInKey) + testAddIndexRollback(c, s.store, s.lease, idxName, addIdxSQL, errMsg, hasNullValsInKey, false) } // TestAddPrimaryKeyRollback2 is used to test scenarios that will roll back when a null primary key is encountered. @@ -236,7 +238,7 @@ func (s *testDBSuite1) TestAddPrimaryKeyRollback2(c *C) { idxName := "PRIMARY" addIdxSQL := "alter table t1 add primary key c3_index (c3);" errMsg := "[ddl:1138]Invalid use of NULL value" - testAddIndexRollback(c, s.store, s.lease, idxName, addIdxSQL, errMsg, hasNullValsInKey) + testAddIndexRollback(c, s.store, s.lease, idxName, addIdxSQL, errMsg, hasNullValsInKey, false) } func (s *testDBSuite2) TestAddUniqueIndexRollback(c *C) { @@ -244,7 +246,17 @@ func (s *testDBSuite2) TestAddUniqueIndexRollback(c *C) { idxName := "c3_index" addIdxSQL := "create unique index c3_index on t1 (c3)" errMsg := "[kv:1062]Duplicate entry '' for key 'c3_index'" - testAddIndexRollback(c, s.store, s.lease, idxName, addIdxSQL, errMsg, hasNullValsInKey) + testAddIndexRollback(c, s.store, s.lease, idxName, addIdxSQL, errMsg, hasNullValsInKey, false) +} + +func (s *testDBSuite6) TestAddExpressionIndexRollback(c *C) { + // TODO: This test may cause a bug which has been fixed in following PR, uncomment these code + // in that PR @wjhuang2016 + //hasNullValsInKey := false + //idxName := "expr_idx" + //addIdxSQL := "alter table t1 add index expr_idx ((pow(c1, c2)));" + //errMsg := "[ddl:8202]Cannot decode index value, because [types:1690]DOUBLE value is out of range in 'pow(144, 144)'" + //testAddIndexRollback(c, s.store, s.lease, idxName, addIdxSQL, errMsg, hasNullValsInKey, true) } func batchInsert(tk *testkit.TestKit, tbl string, start, end int) { @@ -258,8 +270,7 @@ func batchInsert(tk *testkit.TestKit, tbl string, start, end int) { tk.MustExec(dml) } -func testAddIndexRollback(c *C, store kv.Storage, lease time.Duration, - idxName, addIdxSQL, errMsg string, hasNullValsInKey bool) { +func testAddIndexRollback(c *C, store kv.Storage, lease time.Duration, idxName, addIdxSQL, errMsg string, hasNullValsInKey bool, isExpressionIndex bool) { tk := testkit.NewTestKit(c, store) tk.MustExec("use test_db") tk.MustExec("drop table if exists t1") @@ -320,7 +331,9 @@ LOOP: for i := base - 10; i < base; i++ { tk.MustExec("delete from t1 where c1 = ?", i+10) } - sessionExec(c, store, addIdxSQL) + if !isExpressionIndex { + sessionExec(c, store, addIdxSQL) + } tk.MustExec("drop table t1") } @@ -520,9 +533,10 @@ func testCancelDropIndex(c *C, store kv.Storage, d ddl.DDL, idxName, addIdxSQL, cancelSucc bool }{ // model.JobStateNone means the jobs is canceled before the first run. + // if we cancel successfully, we need to set needAddIndex to false in the next test case. Otherwise, set needAddIndex to true. {true, model.JobStateNone, model.StateNone, true}, - {false, model.JobStateRunning, model.StateWriteOnly, true}, - {false, model.JobStateRunning, model.StateDeleteOnly, false}, + {false, model.JobStateRunning, model.StateWriteOnly, false}, + {true, model.JobStateRunning, model.StateDeleteOnly, false}, {true, model.JobStateRunning, model.StateDeleteReorganization, false}, } var checkErr error diff --git a/ddl/ddl.go b/ddl/ddl.go index 3124249594a9b..f1fed9081b923 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -226,6 +226,10 @@ var ( ErrFieldNotFoundPart = terror.ClassDDL.New(mysql.ErrFieldNotFoundPart, mysql.MySQLErrName[mysql.ErrFieldNotFoundPart]) // ErrWrongTypeColumnValue returns 'Partition column values of incorrect type' ErrWrongTypeColumnValue = terror.ClassDDL.New(mysql.ErrWrongTypeColumnValue, mysql.MySQLErrName[mysql.ErrWrongTypeColumnValue]) + // ErrFunctionalIndexPrimaryKey returns 'The primary key cannot be a functional index' + ErrFunctionalIndexPrimaryKey = terror.ClassDDL.New(mysql.ErrFunctionalIndexPrimaryKey, mysql.MySQLErrName[mysql.ErrFunctionalIndexPrimaryKey]) + // ErrFunctionalIndexOnField returns 'Functional index on a column is not supported. Consider using a regular index instead' + 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]) ) @@ -669,6 +673,7 @@ func init() { mysql.ErrFieldNotFoundPart: mysql.ErrFieldNotFoundPart, mysql.ErrFieldTypeNotAllowedAsPartitionField: mysql.ErrFieldTypeNotAllowedAsPartitionField, mysql.ErrFileNotFound: mysql.ErrFileNotFound, + mysql.ErrFunctionalIndexPrimaryKey: mysql.ErrFunctionalIndexPrimaryKey, mysql.ErrGeneratedColumnFunctionIsNotAllowed: mysql.ErrGeneratedColumnFunctionIsNotAllowed, mysql.ErrGeneratedColumnNonPrior: mysql.ErrGeneratedColumnNonPrior, mysql.ErrGeneratedColumnRefAutoInc: mysql.ErrGeneratedColumnRefAutoInc, @@ -733,6 +738,7 @@ func init() { mysql.ErrWrongTableName: mysql.ErrWrongTableName, mysql.ErrWrongTypeColumnValue: mysql.ErrWrongTypeColumnValue, mysql.WarnDataTruncated: mysql.WarnDataTruncated, + mysql.ErrFunctionalIndexOnField: mysql.ErrFunctionalIndexOnField, mysql.ErrFkColumnCannotDrop: mysql.ErrFkColumnCannotDrop, mysql.ErrFKIncompatibleColumns: mysql.ErrFKIncompatibleColumns, } diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 644ba98ab29aa..d1a54d349e598 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -54,6 +54,8 @@ import ( "go.uber.org/zap" ) +const expressionIndexPrefix = "_V$" + func (d *ddl) CreateSchema(ctx sessionctx.Context, schema model.CIStr, charsetInfo *ast.CharsetOpt) (err error) { is := d.GetInfoSchemaWithInterceptor(ctx) _, ok := is.SchemaByName(schema) @@ -3490,7 +3492,7 @@ func getAnonymousIndex(t table.Table, colName model.CIStr) model.CIStr { } func (d *ddl) CreatePrimaryKey(ctx sessionctx.Context, ti ast.Ident, indexName model.CIStr, - idxColNames []*ast.IndexPartSpecification, indexOption *ast.IndexOption) error { + indexPartSpecifications []*ast.IndexPartSpecification, indexOption *ast.IndexOption) error { if !config.GetGlobalConfig().AlterPrimaryKey { return ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported add primary key, alter-primary-key is false") } @@ -3509,22 +3511,30 @@ func (d *ddl) CreatePrimaryKey(ctx sessionctx.Context, ti ast.Ident, indexName m return infoschema.ErrMultiplePriKey } + // Primary keys cannot include expression index parts. A primary key requires the generated column to be stored, + // but expression index parts are implemented as virtual generated columns, not stored generated columns. + for _, idxPart := range indexPartSpecifications { + if idxPart.Expr != nil { + return ErrFunctionalIndexPrimaryKey + } + } + tblInfo := t.Meta() // Check before the job is put to the queue. // This check is redundant, but useful. If DDL check fail before the job is put // to job queue, the fail path logic is super fast. // After DDL job is put to the queue, and if the check fail, TiDB will run the DDL cancel logic. // The recover step causes DDL wait a few seconds, makes the unit test painfully slow. - _, err = buildIndexColumns(tblInfo.Columns, idxColNames) + _, err = buildIndexColumns(tblInfo.Columns, indexPartSpecifications) if err != nil { return errors.Trace(err) } - if _, err = checkPKOnGeneratedColumn(tblInfo, idxColNames); err != nil { + if _, err = checkPKOnGeneratedColumn(tblInfo, indexPartSpecifications); err != nil { return err } if tblInfo.GetPartitionInfo() != nil { - if err := checkPartitionKeysConstraint(tblInfo.GetPartitionInfo(), idxColNames, tblInfo, true); err != nil { + if err := checkPartitionKeysConstraint(tblInfo.GetPartitionInfo(), indexPartSpecifications, tblInfo, true); err != nil { return err } } @@ -3542,7 +3552,7 @@ func (d *ddl) CreatePrimaryKey(ctx sessionctx.Context, ti ast.Ident, indexName m SchemaName: schema.Name.L, Type: model.ActionAddPrimaryKey, BinlogInfo: &model.HistoryInfo{}, - Args: []interface{}{unique, indexName, idxColNames, indexOption, sqlMode}, + Args: []interface{}{unique, indexName, indexPartSpecifications, indexOption, sqlMode}, Priority: ctx.GetSessionVars().DDLReorgPriority, } @@ -3551,9 +3561,77 @@ func (d *ddl) CreatePrimaryKey(ctx sessionctx.Context, ti ast.Ident, indexName m return errors.Trace(err) } -func (d *ddl) CreateIndex(ctx sessionctx.Context, ti ast.Ident, keyType ast.IndexKeyType, indexName model.CIStr, - idxColNames []*ast.IndexPartSpecification, indexOption *ast.IndexOption, ifNotExists bool) error { +func buildHiddenColumnInfo(ctx sessionctx.Context, t table.Table, indexPartSpecifications []*ast.IndexPartSpecification, indexName model.CIStr) ([]*model.ColumnInfo, error) { + tblInfo := t.Meta() + 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(t.Cols(), 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. + if len(idxPart.Column.Name.L) > mysql.MaxColumnNameLength { + // TODO: Refine the error message. + return nil, ErrTooLongIdent.GenWithStackByArgs("hidden column") + } + // TODO: refine the error message. + if err := checkIllegalFn4GeneratedColumn("expression index", idxPart.Expr); err != nil { + return nil, errors.Trace(err) + } + var sb strings.Builder + restoreFlags := format.RestoreStringSingleQuotes | format.RestoreKeyWordLowercase | format.RestoreNameBackQuotes | + format.RestoreSpacesAroundBinaryOperation + restoreCtx := format.NewRestoreCtx(restoreFlags, &sb) + sb.Reset() + err := idxPart.Expr.Restore(restoreCtx) + if err != nil { + return nil, errors.Trace(err) + } + expr, err := expression.RewriteSimpleExprWithTableInfo(ctx, tblInfo, idxPart.Expr) + if err != nil { + // TODO: refine the error message. + return nil, err + } + if _, ok := expr.(*expression.Column); ok { + return nil, 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(), + } + checkDependencies := make(map[string]struct{}) + for _, colName := range findColumnNamesInExpr(idxPart.Expr) { + colInfo.Dependences[colName.Name.O] = struct{}{} + checkDependencies[colName.Name.O] = struct{}{} + } + if err = checkDependedColExist(checkDependencies, t.Cols()); err != nil { + return nil, errors.Trace(err) + } + if err = checkAutoIncrementRef("", colInfo.Dependences, tblInfo); err != nil { + return nil, errors.Trace(err) + } + idxPart.Expr = nil + hiddenCols = append(hiddenCols, colInfo) + } + return hiddenCols, nil +} + +func (d *ddl) CreateIndex(ctx sessionctx.Context, ti ast.Ident, keyType ast.IndexKeyType, indexName model.CIStr, + indexPartSpecifications []*ast.IndexPartSpecification, indexOption *ast.IndexOption, ifNotExists bool) error { // not support Spatial and FullText index if keyType == ast.IndexKeyTypeFullText || keyType == ast.IndexKeyTypeSpatial { return errUnsupportedIndexType.GenWithStack("FULLTEXT and SPATIAL index is not supported") @@ -3566,7 +3644,11 @@ func (d *ddl) CreateIndex(ctx sessionctx.Context, ti ast.Ident, keyType ast.Inde // Deal with anonymous index. if len(indexName.L) == 0 { - indexName = getAnonymousIndex(t, idxColNames[0].Column.Name) + colName := model.NewCIStr("expression_index") + if indexPartSpecifications[0].Column != nil { + colName = indexPartSpecifications[0].Column.Name + } + indexName = getAnonymousIndex(t, colName) } if indexInfo := t.Meta().FindIndexByName(indexName.L); indexInfo != nil { @@ -3583,17 +3665,27 @@ func (d *ddl) CreateIndex(ctx sessionctx.Context, ti ast.Ident, keyType ast.Inde } tblInfo := t.Meta() + + // Build hidden columns if necessary. + hiddenCols, err := buildHiddenColumnInfo(ctx, t, indexPartSpecifications, indexName) + if err != nil { + return err + } + if err = checkAddColumnTooManyColumns(len(t.Cols()) + len(hiddenCols)); err != nil { + return errors.Trace(err) + } + // Check before the job is put to the queue. // This check is redundant, but useful. If DDL check fail before the job is put // to job queue, the fail path logic is super fast. // After DDL job is put to the queue, and if the check fail, TiDB will run the DDL cancel logic. // The recover step causes DDL wait a few seconds, makes the unit test painfully slow. - _, err = buildIndexColumns(tblInfo.Columns, idxColNames) + _, err = buildIndexColumns(append(tblInfo.Columns, hiddenCols...), indexPartSpecifications) if err != nil { return errors.Trace(err) } if unique && tblInfo.GetPartitionInfo() != nil { - if err := checkPartitionKeysConstraint(tblInfo.GetPartitionInfo(), idxColNames, tblInfo, false); err != nil { + if err := checkPartitionKeysConstraint(tblInfo.GetPartitionInfo(), indexPartSpecifications, tblInfo, false); err != nil { return err } } @@ -3601,14 +3693,13 @@ func (d *ddl) CreateIndex(ctx sessionctx.Context, ti ast.Ident, keyType ast.Inde if _, err = validateCommentLength(ctx.GetSessionVars(), indexName.String(), indexOption); err != nil { return errors.Trace(err) } - job := &model.Job{ SchemaID: schema.ID, TableID: t.Meta().ID, SchemaName: schema.Name.L, Type: model.ActionAddIndex, BinlogInfo: &model.HistoryInfo{}, - Args: []interface{}{unique, indexName, idxColNames, indexOption}, + Args: []interface{}{unique, indexName, indexPartSpecifications, indexOption, hiddenCols}, Priority: ctx.GetSessionVars().DDLReorgPriority, } diff --git a/ddl/generated_column.go b/ddl/generated_column.go index 6feab0e59e49b..0ea41d8691c2e 100644 --- a/ddl/generated_column.go +++ b/ddl/generated_column.go @@ -69,11 +69,13 @@ func verifyColumnGenerationSingle(dependColNames map[string]struct{}, cols []*ta return nil } -// checkDependedColExist ensure all depended columns exist. +// checkDependedColExist ensure all depended columns exist and not hidden. // NOTE: this will MODIFY parameter `dependCols`. func checkDependedColExist(dependCols map[string]struct{}, cols []*table.Column) error { for _, col := range cols { - delete(dependCols, col.Name.L) + if !col.Hidden { + delete(dependCols, col.Name.L) + } } if len(dependCols) != 0 { for arbitraryCol := range dependCols { diff --git a/ddl/index.go b/ddl/index.go index 75074962543d1..6cf5d4d0729cc 100755 --- a/ddl/index.go +++ b/ddl/index.go @@ -54,23 +54,24 @@ const ( MaxCommentLength = 1024 ) -func buildIndexColumns(columns []*model.ColumnInfo, idxColNames []*ast.IndexPartSpecification) ([]*model.IndexColumn, error) { +func buildIndexColumns(columns []*model.ColumnInfo, indexPartSpecifications []*ast.IndexPartSpecification) ([]*model.IndexColumn, error) { // Build offsets. - idxColumns := make([]*model.IndexColumn, 0, len(idxColNames)) + idxParts := make([]*model.IndexColumn, 0, len(indexPartSpecifications)) + var col *model.ColumnInfo // The sum of length of all index columns. sumLength := 0 - for _, ic := range idxColNames { - col := model.FindColumnInfo(columns, ic.Column.Name.L) + for _, ip := range indexPartSpecifications { + col = model.FindColumnInfo(columns, ip.Column.Name.L) if col == nil { - return nil, errKeyColumnDoesNotExits.GenWithStack("column does not exist: %s", ic.Column.Name) + return nil, errKeyColumnDoesNotExits.GenWithStack("column does not exist: %s", ip.Column.Name) } - if err := checkIndexColumn(col, ic); err != nil { + if err := checkIndexColumn(col, ip); err != nil { return nil, err } - indexColumnLength, err := getIndexColumnLength(col, ic.Length) + indexColumnLength, err := getIndexColumnLength(col, ip.Length) if err != nil { return nil, err } @@ -81,19 +82,19 @@ func buildIndexColumns(columns []*model.ColumnInfo, idxColNames []*ast.IndexPart return nil, errors.Trace(errTooLongKey) } - idxColumns = append(idxColumns, &model.IndexColumn{ + idxParts = append(idxParts, &model.IndexColumn{ Name: col.Name, Offset: col.Offset, - Length: ic.Length, + Length: ip.Length, }) } - return idxColumns, nil + return idxParts, nil } -func checkPKOnGeneratedColumn(tblInfo *model.TableInfo, idxColNames []*ast.IndexPartSpecification) (*model.ColumnInfo, error) { +func checkPKOnGeneratedColumn(tblInfo *model.TableInfo, indexPartSpecifications []*ast.IndexPartSpecification) (*model.ColumnInfo, error) { var lastCol *model.ColumnInfo - for _, colName := range idxColNames { + for _, colName := range indexPartSpecifications { lastCol = getColumnInfoByName(tblInfo, colName.Column.Name.L) if lastCol == nil { return nil, errKeyColumnDoesNotExits.GenWithStackByArgs(colName.Column.Name) @@ -204,12 +205,12 @@ func calcBytesLengthForDecimal(m int) int { return (m / 9 * 4) + ((m%9)+1)/2 } -func buildIndexInfo(tblInfo *model.TableInfo, indexName model.CIStr, idxColNames []*ast.IndexPartSpecification, state model.SchemaState) (*model.IndexInfo, error) { +func buildIndexInfo(tblInfo *model.TableInfo, indexName model.CIStr, indexPartSpecifications []*ast.IndexPartSpecification, state model.SchemaState) (*model.IndexInfo, error) { if err := checkTooLongIndex(indexName); err != nil { return nil, errors.Trace(err) } - idxColumns, err := buildIndexColumns(tblInfo.Columns, idxColNames) + idxColumns, err := buildIndexColumns(tblInfo.Columns, indexPartSpecifications) if err != nil { return nil, errors.Trace(err) } @@ -337,6 +338,14 @@ func checkPrimaryKeyNotNull(w *worker, sqlMode mysql.SQLMode, t *meta.Meta, job return nil, err } +func updateHiddenColumns(tblInfo *model.TableInfo, idxInfo *model.IndexInfo, state model.SchemaState) { + for _, col := range idxInfo.Columns { + if tblInfo.Columns[col.Offset].Hidden { + tblInfo.Columns[col.Offset].State = state + } + } +} + func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job, isPK bool) (ver int64, err error) { // Handle the rolling back job. if job.IsRollingback() { @@ -355,18 +364,19 @@ func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job, isPK boo } var ( - unique bool - indexName model.CIStr - idxColNames []*ast.IndexPartSpecification - indexOption *ast.IndexOption - sqlMode mysql.SQLMode - warnings []string + unique bool + indexName model.CIStr + indexPartSpecifications []*ast.IndexPartSpecification + indexOption *ast.IndexOption + sqlMode mysql.SQLMode + warnings []string + hiddenCols []*model.ColumnInfo ) if isPK { // Notice: sqlMode and warnings is used to support non-strict mode. - err = job.DecodeArgs(&unique, &indexName, &idxColNames, &indexOption, &sqlMode, &warnings) + err = job.DecodeArgs(&unique, &indexName, &indexPartSpecifications, &indexOption, &sqlMode, &warnings) } else { - err = job.DecodeArgs(&unique, &indexName, &idxColNames, &indexOption) + err = job.DecodeArgs(&unique, &indexName, &indexPartSpecifications, &indexOption, &hiddenCols) } if err != nil { job.State = model.JobStateCancelled @@ -382,9 +392,32 @@ func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job, isPK boo } return ver, err } + for _, hiddenCol := range hiddenCols { + columnInfo := model.FindColumnInfo(tblInfo.Columns, hiddenCol.Name.L) + if columnInfo != nil && columnInfo.State == model.StatePublic { + // We already have a column with the same column name. + job.State = model.JobStateCancelled + // TODO: refine the error message + return ver, infoschema.ErrColumnExists.GenWithStackByArgs(hiddenCol.Name) + } + } if indexInfo == nil { - indexInfo, err = buildIndexInfo(tblInfo, indexName, idxColNames, model.StateNone) + if len(hiddenCols) > 0 { + pos := &ast.ColumnPosition{Tp: ast.ColumnPositionNone} + for _, hiddenCol := range hiddenCols { + _, _, err = createColumnInfo(tblInfo, hiddenCol, pos) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + } + } + if err = checkAddColumnTooManyColumns(len(tblInfo.Columns)); err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + indexInfo, err = buildIndexInfo(tblInfo, indexName, indexPartSpecifications, model.StateNone) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) @@ -403,7 +436,7 @@ func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job, isPK boo } indexInfo.Primary = false if isPK { - if _, err = checkPKOnGeneratedColumn(tblInfo, idxColNames); err != nil { + if _, err = checkPKOnGeneratedColumn(tblInfo, indexPartSpecifications); err != nil { job.State = model.JobStateCancelled return ver, err } @@ -420,12 +453,14 @@ func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job, isPK boo // none -> delete only job.SchemaState = model.StateDeleteOnly indexInfo.State = model.StateDeleteOnly + updateHiddenColumns(tblInfo, indexInfo, model.StateDeleteOnly) ver, err = updateVersionAndTableInfoWithCheck(t, job, tblInfo, originalState != indexInfo.State) metrics.AddIndexProgress.Set(0) case model.StateDeleteOnly: // delete only -> write only job.SchemaState = model.StateWriteOnly indexInfo.State = model.StateWriteOnly + updateHiddenColumns(tblInfo, indexInfo, model.StateWriteOnly) _, err = checkPrimaryKeyNotNull(w, sqlMode, t, job, tblInfo, indexInfo) if err != nil { break @@ -435,6 +470,7 @@ func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job, isPK boo // write only -> reorganization job.SchemaState = model.StateWriteReorganization indexInfo.State = model.StateWriteReorganization + updateHiddenColumns(tblInfo, indexInfo, model.StateWriteReorganization) _, err = checkPrimaryKeyNotNull(w, sqlMode, t, job, tblInfo, indexInfo) if err != nil { break @@ -444,6 +480,7 @@ func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job, isPK boo ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexInfo.State) case model.StateWriteReorganization: // reorganization -> public + updateHiddenColumns(tblInfo, indexInfo, model.StatePublic) tbl, err := getTable(d.store, schemaID, tblInfo) if err != nil { return ver, errors.Trace(err) @@ -511,22 +548,39 @@ func onDropIndex(t *meta.Meta, job *model.Job) (ver int64, _ error) { return ver, errors.Trace(err) } + dependentHiddenCols := make([]*model.ColumnInfo, 0) + for _, indexColumn := range indexInfo.Columns { + if tblInfo.Columns[indexColumn.Offset].Hidden { + dependentHiddenCols = append(dependentHiddenCols, tblInfo.Columns[indexColumn.Offset]) + } + } + originalState := indexInfo.State switch indexInfo.State { case model.StatePublic: // public -> write only job.SchemaState = model.StateWriteOnly indexInfo.State = model.StateWriteOnly + if len(dependentHiddenCols) > 0 { + firstHiddenOffset := dependentHiddenCols[0].Offset + for i := 0; i < len(dependentHiddenCols); i++ { + tblInfo.Columns[firstHiddenOffset].State = model.StateWriteOnly + // Set this column's offset to the last and reset all following columns' offsets. + adjustColumnInfoInDropColumn(tblInfo, firstHiddenOffset) + } + } ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexInfo.State) case model.StateWriteOnly: // write only -> delete only job.SchemaState = model.StateDeleteOnly indexInfo.State = model.StateDeleteOnly + updateHiddenColumns(tblInfo, indexInfo, model.StateDeleteOnly) ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexInfo.State) case model.StateDeleteOnly: // delete only -> reorganization job.SchemaState = model.StateDeleteReorganization indexInfo.State = model.StateDeleteReorganization + updateHiddenColumns(tblInfo, indexInfo, model.StateDeleteReorganization) ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexInfo.State) case model.StateDeleteReorganization: // reorganization -> absent @@ -540,6 +594,8 @@ func onDropIndex(t *meta.Meta, job *model.Job) (ver int64, _ error) { // Set column index flag. dropIndexColumnFlag(tblInfo, indexInfo) + tblInfo.Columns = tblInfo.Columns[:len(tblInfo.Columns)-len(dependentHiddenCols)] + ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != model.StateNone) if err != nil { return ver, errors.Trace(err) diff --git a/ddl/partition.go b/ddl/partition.go index 7de4c536e37da..b4362c8059d01 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -677,7 +677,7 @@ func checkRangePartitioningKeysConstraints(sctx sessionctx.Context, s *ast.Creat return nil } -func checkPartitionKeysConstraint(pi *model.PartitionInfo, idxColNames []*ast.IndexPartSpecification, tblInfo *model.TableInfo, isPK bool) error { +func checkPartitionKeysConstraint(pi *model.PartitionInfo, indexPartSpecifications []*ast.IndexPartSpecification, tblInfo *model.TableInfo, isPK bool) error { var ( partCols []*model.ColumnInfo err error @@ -704,7 +704,7 @@ func checkPartitionKeysConstraint(pi *model.PartitionInfo, idxColNames []*ast.In // Every unique key on the table must use every column in the table's partitioning expression.(This // also includes the table's primary key.) // See https://dev.mysql.com/doc/refman/5.7/en/partitioning-limitations-partitioning-keys-unique-keys.html - if !checkUniqueKeyIncludePartKey(columnInfoSlice(partCols), idxColNames) { + if !checkUniqueKeyIncludePartKey(columnInfoSlice(partCols), indexPartSpecifications) { if isPK { return ErrUniqueKeyNeedAllFieldsInPf.GenWithStackByArgs("PRIMARY") } diff --git a/ddl/rollingback.go b/ddl/rollingback.go index 3969805e4eb06..b3d0367cc2c54 100644 --- a/ddl/rollingback.go +++ b/ddl/rollingback.go @@ -59,6 +59,8 @@ func convertAddIdxJob2RollbackJob(t *meta.Meta, job *model.Job, tblInfo *model.T // So the next state is delete only state. originalState := indexInfo.State indexInfo.State = model.StateDeleteOnly + // Change dependent hidden columns if necessary. + updateHiddenColumns(tblInfo, indexInfo, model.StateDeleteOnly) job.SchemaState = model.StateDeleteOnly ver, err1 := updateVersionAndTableInfo(t, job, tblInfo, originalState != indexInfo.State) if err1 != nil { @@ -82,12 +84,12 @@ func convertNotStartAddIdxJob2RollbackJob(t *meta.Meta, job *model.Job, occuredE } var ( - unique bool - indexName model.CIStr - idxColNames []*ast.IndexPartSpecification - indexOption *ast.IndexOption + unique bool + indexName model.CIStr + indexPartSpecifications []*ast.IndexPartSpecification + indexOption *ast.IndexOption ) - err = job.DecodeArgs(&unique, &indexName, &idxColNames, &indexOption) + err = job.DecodeArgs(&unique, &indexName, &indexPartSpecifications, &indexOption) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) @@ -150,12 +152,12 @@ func rollingbackDropIndex(t *meta.Meta, job *model.Job) (ver int64, err error) { originalState := indexInfo.State switch indexInfo.State { - case model.StateDeleteOnly, model.StateDeleteReorganization, model.StateNone: + case model.StateWriteOnly, model.StateDeleteOnly, model.StateDeleteReorganization, model.StateNone: // We can not rollback now, so just continue to drop index. // Normally won't fetch here, because there is check when cancel ddl jobs. see function: isJobRollbackable. job.State = model.JobStateRunning return ver, nil - case model.StatePublic, model.StateWriteOnly: + case model.StatePublic: job.State = model.JobStateRollbackDone indexInfo.State = model.StatePublic default: diff --git a/executor/show.go b/executor/show.go index d94e28e136953..70e75421add10 100644 --- a/executor/show.go +++ b/executor/show.go @@ -787,10 +787,15 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T } cols := make([]string, 0, len(idxInfo.Columns)) + var colInfo string for _, c := range idxInfo.Columns { - colInfo := escape(c.Name, sqlMode) - if c.Length != types.UnspecifiedLength { - colInfo = fmt.Sprintf("%s(%s)", colInfo, strconv.Itoa(c.Length)) + if tableInfo.Columns[c.Offset].Hidden { + colInfo = fmt.Sprintf("(%s)", tableInfo.Columns[c.Offset].GeneratedExprString) + } else { + colInfo = escape(c.Name, sqlMode) + if c.Length != types.UnspecifiedLength { + colInfo = fmt.Sprintf("%s(%s)", colInfo, strconv.Itoa(c.Length)) + } } cols = append(cols, colInfo) } diff --git a/executor/show_test.go b/executor/show_test.go index f8e3467c53196..8e50c8184b45c 100644 --- a/executor/show_test.go +++ b/executor/show_test.go @@ -609,6 +609,19 @@ func (s *testSuite5) TestShowCreateTable(c *C) { " `a` varchar(123) DEFAULT NULL\n"+ ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin", )) + + // for expression index + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int, b real);") + tk.MustExec("alter table t add index expr_idx((a*b+1));") + tk.MustQuery("show create table t;").Check(testutil.RowsWithSep("|", + ""+ + "t CREATE TABLE `t` (\n"+ + " `a` int(11) DEFAULT NULL,\n"+ + " `b` double DEFAULT NULL,\n"+ + " KEY `expr_idx` ((`a` * `b` + 1))\n"+ + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin", + )) } func (s *testAutoRandomSuite) TestShowCreateTableAutoRandom(c *C) { diff --git a/go.mod b/go.mod index 1842e4176789f..839ecb7b65329 100644 --- a/go.mod +++ b/go.mod @@ -52,7 +52,6 @@ require ( github.com/sirupsen/logrus v1.2.0 github.com/soheilhy/cmux v0.1.4 github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 - github.com/struCoder/pidusage v0.1.2 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 github.com/uber-go/atomic v1.3.2 github.com/uber/jaeger-client-go v2.15.0+incompatible diff --git a/go.sum b/go.sum index 34c70ba833b7d..643a165f26eb7 100644 --- a/go.sum +++ b/go.sum @@ -1,8 +1,11 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= +github.com/OneOfOne/xxhash v1.2.2 h1:KMrpdQIwFcEqXDklaen+P1axHaj9BSKzvpUUfnHldSE= github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= +github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d h1:G0m3OIz70MZUWq3EgK3CesDbo8upS2Vm9/P3FtgI+Jk= github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= +github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc h1:cAKDfWh5VpdgMhJosfJnn5/FoN2SRZ4p7fJNX58YPaU= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= @@ -44,11 +47,13 @@ github.com/dgraph-io/ristretto v0.0.1/go.mod h1:T40EBc7CJke8TkpiYfGGKAeFjSaxuFXh github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2 h1:tdlZCpZ/P9DhczCTSixgIKmwPv6+wP5DGjqLYw5SUiA= github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= +github.com/docker/go-units v0.4.0 h1:3uh0PgVws3nIA0Q+MwDC8yjEPf9zjRfZZWXZYDct3Tw= github.com/docker/go-units v0.4.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= github.com/dustin/go-humanize v0.0.0-20180421182945-02af3965c54e/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385/go.mod h1:0vRUJqYpeSZifjYj7uP3BG/gKcuzL9xWVV/Y+cK33KM= +github.com/elazarl/go-bindata-assetfs v1.0.0 h1:G/bYguwHIzWq9ZoyUQqrjTmJbbYn3j3CKKpKinvZLFk= github.com/elazarl/go-bindata-assetfs v1.0.0/go.mod h1:v+YaWX3bdea5J/mo8dSETolEo7R71Vk1u8bnjau5yw4= github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= @@ -58,6 +63,7 @@ github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeME github.com/ghodss/yaml v1.0.1-0.20190212211648-25d852aebe32/go.mod h1:GIjDIg/heH5DOkXY3YJ/wNhfHsQHoXGjl8G8amsYQ1I= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= +github.com/go-ole/go-ole v1.2.4 h1:nNBDSCOigTSiarFpYE9J/KtEA1IOW4CNeqT9TQDqCxI= github.com/go-ole/go-ole v1.2.4/go.mod h1:XCwSNxSkXRo4vlyPy93sltvi/qJq0jqQhjqQNIwKuxM= github.com/go-playground/overalls v0.0.0-20180201144345-22ec1a223b7c/go.mod h1:UqxAgEOt89sCiXlrc/ycnx00LVvUO/eS8tMUkWX4R7w= github.com/go-sql-driver/mysql v0.0.0-20170715192408-3955978caca4 h1:3DFRjZdCDhzvxDf0U6/1qAryzOqD7Y5iAj0DJRRl1bs= @@ -121,6 +127,7 @@ github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+o github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5 h1:2U0HzY8BJ8hVwDKIzp7y4voR9CX/nvcfymLmg2UiOio= github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/konsorten/go-windows-terminal-sequences v1.0.2 h1:DB17ag19krx9CFsz4o3enTrPXyIXCl+2iCXH/aMAp9s= github.com/konsorten/go-windows-terminal-sequences v1.0.2/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= @@ -215,6 +222,7 @@ github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFR github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= github.com/shirou/gopsutil v2.19.10+incompatible h1:lA4Pi29JEVIQIgATSeftHSY0rMGI9CLrl2ZvDLiahto= github.com/shirou/gopsutil v2.19.10+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= +github.com/shirou/w32 v0.0.0-20160930032740-bb4de0191aa4 h1:udFKJ0aHUL60LboW/A+DfgoHVedieIzIXE8uylPue0U= github.com/shirou/w32 v0.0.0-20160930032740-bb4de0191aa4/go.mod h1:qsXQc7+bwAM3Q1u/4XEfrquwF8Lw7D7y5cD8CuHnfIc= github.com/shurcooL/httpfs v0.0.0-20171119174359-809beceb2371/go.mod h1:ZY1cvUeJuFPAdZ/B6v7RHavJWZn2YPVFQ1OSXhCGOkg= github.com/shurcooL/vfsgen v0.0.0-20181020040650-a97a25d856ca/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw= @@ -233,7 +241,6 @@ github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+ github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= -github.com/struCoder/pidusage v0.1.2/go.mod h1:pWBlW3YuSwRl6h7R5KbvA4N8oOqe9LjaKW5CwT1SPjI= github.com/syndtr/goleveldb v0.0.0-20180815032940-ae2bd5eed72d/go.mod h1:Z4AUp2Km+PwemOoO/VB5AOx9XSsIItzFjoJlOSiYmn0= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= @@ -249,6 +256,7 @@ github.com/ugorji/go/codec v0.0.0-20190204201341-e444a5086c43/go.mod h1:iT03XoTw github.com/unrolled/render v0.0.0-20171102162132-65450fb6b2d3/go.mod h1:tu82oB5W2ykJRVioYsB+IQKcft7ryBr7w12qMBUPyXg= github.com/unrolled/render v0.0.0-20180914162206-b9786414de4d/go.mod h1:tu82oB5W2ykJRVioYsB+IQKcft7ryBr7w12qMBUPyXg= github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA= +github.com/urfave/negroni v0.3.0 h1:PaXOb61mWeZJxc1Ji2xJjpVg9QfPo0rrB+lHyBxGNSU= github.com/urfave/negroni v0.3.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= github.com/yookoala/realpath v1.0.0/go.mod h1:gJJMA9wuX7AcqLy1+ffPatSCySA1FQ2S8Ya9AIoYBpE= @@ -262,6 +270,7 @@ go.uber.org/atomic v1.5.0 h1:OI5t8sDa1Or+q8AeE+yKeB/SDYioSHAgcVljj9JIETY= go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/automaxprocs v1.2.0 h1:+RUihKM+nmYUoB9w0D0Ov5TJ2PpFO2FgenTxMJiZBZA= go.uber.org/automaxprocs v1.2.0/go.mod h1:YfO3fm683kQpzETxlTGZhGIVmXAhaw3gxeBADbpZtnU= +go.uber.org/goleak v0.10.0 h1:G3eWbSNIskeRqtsN/1uI5B+eP73y3JUuBsv9AZjehb4= go.uber.org/goleak v0.10.0/go.mod h1:VCZuO8V8mFPlL0F5J5GK1rtHV3DrFcQ1R8ryq7FK0aI= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= @@ -368,6 +377,7 @@ gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.1-2019.2.3 h1:3JgtbtFHMiCmsznwGVTUWbgGov+pVqnlf1dEJTNAXeM= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o= sourcegraph.com/sourcegraph/appdash v0.0.0-20180531100431-4c381bd170b4 h1:VO9oZbbkvTwqLimlQt15QNdOOBArT2dw/bvzsMZBiqQ= diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index e6536a7e35d2b..eb6b5f6dddcd9 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -398,6 +398,12 @@ func (p *preprocessor) checkCreateTableGrammar(stmt *ast.CreateTableStmt) { } } for _, constraint := range stmt.Constraints { + for _, spec := range constraint.Keys { + if spec.Expr != nil { + p.err = ErrNotSupportedYet.GenWithStackByArgs("create table with expression index") + return + } + } switch tp := constraint.Tp; tp { case ast.ConstraintKey, ast.ConstraintIndex, ast.ConstraintUniq, ast.ConstraintUniqKey, ast.ConstraintUniqIndex: err := checkIndexInfo(constraint.Name, constraint.Keys) @@ -595,11 +601,13 @@ func (p *preprocessor) checkAlterTableGrammar(stmt *ast.AlterTableStmt) { func checkDuplicateColumnName(IndexPartSpecifications []*ast.IndexPartSpecification) error { colNames := make(map[string]struct{}, len(IndexPartSpecifications)) for _, IndexColNameWithExpr := range IndexPartSpecifications { - name := IndexColNameWithExpr.Column.Name - if _, ok := colNames[name.L]; ok { - return infoschema.ErrColumnExists.GenWithStackByArgs(name) + if IndexColNameWithExpr.Column != nil { + name := IndexColNameWithExpr.Column.Name + if _, ok := colNames[name.L]; ok { + return infoschema.ErrColumnExists.GenWithStackByArgs(name) + } + colNames[name.L] = struct{}{} } - colNames[name.L] = struct{}{} } return nil } diff --git a/util/admin/admin.go b/util/admin/admin.go index d8cf0f7c243ba..c5407f84c1951 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -87,9 +87,14 @@ func GetDDLInfo(txn kv.Transaction) (*DDLInfo, error) { func IsJobRollbackable(job *model.Job) bool { switch job.Type { case model.ActionDropIndex, model.ActionDropPrimaryKey: - // We can't cancel if index current state is in StateDeleteOnly or StateDeleteReorganization, otherwise will cause inconsistent between record and index. + // We can't cancel if index current state is in StateDeleteOnly or StateDeleteReorganization or StateWriteOnly, otherwise there will be an inconsistent issue between record and index. + // In WriteOnly state, we can rollback for normal index but can't rollback for expression index(need to drop hidden column). Since we can't + // know the type of index here, we consider all indices except primary index as non-rollbackable. + // TODO: distinguish normal index and expression index so that we can rollback `DropIndex` for normal index in WriteOnly state. + // TODO: make DropPrimaryKey rollbackable in WriteOnly, it need to deal with some tests. if job.SchemaState == model.StateDeleteOnly || - job.SchemaState == model.StateDeleteReorganization { + job.SchemaState == model.StateDeleteReorganization || + job.SchemaState == model.StateWriteOnly { return false } case model.ActionDropSchema, model.ActionDropTable: diff --git a/util/admin/admin_test.go b/util/admin/admin_test.go index d40c0f4b9cd9b..d8d18bbcb757b 100644 --- a/util/admin/admin_test.go +++ b/util/admin/admin_test.go @@ -234,7 +234,7 @@ func (s *testSuite) TestCancelJobs(c *C) { // test can't cancelable job. job.Type = model.ActionDropIndex - job.SchemaState = model.StateDeleteOnly + job.SchemaState = model.StateWriteOnly job.State = model.JobStateRunning job.ID = 101 err = t.EnQueueDDLJob(job)