Skip to content

Commit

Permalink
ddl: support create/drop expression index (#14117)
Browse files Browse the repository at this point in the history
  • Loading branch information
wjhuang2016 authored and sre-bot committed Jan 7, 2020
1 parent b85ae7d commit f036eec
Show file tree
Hide file tree
Showing 17 changed files with 390 additions and 70 deletions.
3 changes: 3 additions & 0 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
10 changes: 10 additions & 0 deletions ddl/db_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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);"
Expand Down
102 changes: 99 additions & 3 deletions ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -1010,22 +1012,22 @@ 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,
Name: model.NewCIStr("b"),
},
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)
Expand Down Expand Up @@ -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"))
}
30 changes: 22 additions & 8 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -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.
Expand All @@ -236,15 +238,25 @@ 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) {
hasNullValsInKey := false
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) {
Expand All @@ -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")
Expand Down Expand Up @@ -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")
}

Expand Down Expand Up @@ -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
Expand Down
6 changes: 6 additions & 0 deletions ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -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])
)
Expand Down Expand Up @@ -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,
Expand Down Expand Up @@ -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,
}
Expand Down
Loading

0 comments on commit f036eec

Please sign in to comment.