diff --git a/pkg/ddl/executor.go b/pkg/ddl/executor.go index da51fc979de5b..b7906ad09af8a 100644 --- a/pkg/ddl/executor.go +++ b/pkg/ddl/executor.go @@ -958,10 +958,6 @@ func checkGlobalIndex(ec errctx.Context, tblInfo *model.TableInfo, indexInfo *mo // partitioning an index differently from the table partitioning. return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("Global Index on non-partitioned table") } - // TODO: remove limitation - if !indexInfo.Unique { - return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("GLOBAL IndexOption on non-unique index") - } validateGlobalIndexWithGeneratedColumns(ec, tblInfo, indexInfo.Name.O, indexInfo.Columns) } return nil @@ -4543,15 +4539,11 @@ func GetName4AnonymousIndex(t table.Table, colName pmodel.CIStr, idxName pmodel. return indexName } -func checkCreateUniqueGlobalIndex(ec errctx.Context, tblInfo *model.TableInfo, indexName string, indexColumns []*model.IndexColumn, isUnique bool, isGlobal bool) error { +func checkCreateGlobalIndex(ec errctx.Context, tblInfo *model.TableInfo, indexName string, indexColumns []*model.IndexColumn, isUnique bool, isGlobal bool) error { pi := tblInfo.GetPartitionInfo() if isGlobal && pi == nil { return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("Global Index on non-partitioned table") } - if isGlobal && !isUnique { - // TODO: remove this limitation - return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("Global IndexOption on non-unique index") - } if isUnique && pi != nil { ck, err := checkPartitionKeysConstraint(tblInfo.GetPartitionInfo(), indexColumns, tblInfo) if err != nil { @@ -4561,6 +4553,8 @@ func checkCreateUniqueGlobalIndex(ec errctx.Context, tblInfo *model.TableInfo, i // index columns does not contain all partition columns, must be global return dbterror.ErrGlobalIndexNotExplicitlySet.GenWithStackByArgs(indexName) } + } + if isGlobal { validateGlobalIndexWithGeneratedColumns(ec, tblInfo, indexName, indexColumns) } return nil @@ -4611,7 +4605,7 @@ func (e *executor) CreatePrimaryKey(ctx sessionctx.Context, ti ast.Ident, indexN return err } - if err = checkCreateUniqueGlobalIndex(ctx.GetSessionVars().StmtCtx.ErrCtx(), tblInfo, "PRIMARY", indexColumns, true, indexOption != nil && indexOption.Global); err != nil { + if err = checkCreateGlobalIndex(ctx.GetSessionVars().StmtCtx.ErrCtx(), tblInfo, "PRIMARY", indexColumns, true, indexOption != nil && indexOption.Global); err != nil { return err } @@ -4879,7 +4873,7 @@ func (e *executor) createIndex(ctx sessionctx.Context, ti ast.Ident, keyType ast return errors.Trace(err) } - if err = checkCreateUniqueGlobalIndex(ctx.GetSessionVars().StmtCtx.ErrCtx(), tblInfo, indexName.O, indexColumns, unique, indexOption != nil && indexOption.Global); err != nil { + if err = checkCreateGlobalIndex(ctx.GetSessionVars().StmtCtx.ErrCtx(), tblInfo, indexName.O, indexColumns, unique, indexOption != nil && indexOption.Global); err != nil { return err } diff --git a/pkg/ddl/index_modify_test.go b/pkg/ddl/index_modify_test.go index f7b503ca7d15a..9bb9c6f859af2 100644 --- a/pkg/ddl/index_modify_test.go +++ b/pkg/ddl/index_modify_test.go @@ -749,6 +749,39 @@ func TestAddGlobalIndex(t *testing.T) { require.NoError(t, txn.Commit(context.Background())) + // Test add non-unqiue global index + tk.MustExec("drop table if exists test_t2") + tk.MustExec("create table test_t2 (a int, b int) partition by range (b)" + + " (partition p0 values less than (10), " + + " partition p1 values less than (maxvalue));") + tk.MustExec("insert test_t2 values (2, 1)") + tk.MustExec("alter table test_t2 add key p_a (a) global") + tk.MustExec("insert test_t2 values (1, 11)") + tbl = external.GetTableByName(t, tk, "test", "test_t2") + tblInfo = tbl.Meta() + indexInfo = tblInfo.FindIndexByName("p_a") + require.NotNil(t, indexInfo) + require.True(t, indexInfo.Global) + require.False(t, indexInfo.Unique) + + require.NoError(t, sessiontxn.NewTxn(context.Background(), tk.Session())) + txn, err = tk.Session().Txn(true) + require.NoError(t, err) + + // check row 1 + pid = tblInfo.Partition.Definitions[0].ID + idxVals = []types.Datum{types.NewDatum(2)} + rowVals = []types.Datum{types.NewDatum(2), types.NewDatum(1)} + checkGlobalIndexRow(t, tk.Session(), tblInfo, indexInfo, pid, idxVals, rowVals) + + // check row 2 + pid = tblInfo.Partition.Definitions[1].ID + idxVals = []types.Datum{types.NewDatum(1)} + rowVals = []types.Datum{types.NewDatum(1), types.NewDatum(11)} + checkGlobalIndexRow(t, tk.Session(), tblInfo, indexInfo, pid, idxVals, rowVals) + + require.NoError(t, txn.Commit(context.Background())) + // `sanity_check.go` will check the del_range numbers are correct or not. // normal index tk.MustExec("drop table if exists t") @@ -801,7 +834,17 @@ func checkGlobalIndexRow( require.NoError(t, err) key := tablecodec.EncodeIndexSeekKey(tblInfo.ID, indexInfo.ID, encodedValue) require.NoError(t, err) - value, err := txn.Get(context.Background(), key) + var value []byte + if indexInfo.Unique { + value, err = txn.Get(context.Background(), key) + } else { + var iter kv.Iterator + iter, err = txn.Iter(key, key.PrefixNext()) + require.NoError(t, err) + require.True(t, iter.Valid()) + key = iter.Key() + value = iter.Value() + } require.NoError(t, err) idxColInfos := tables.BuildRowcodecColInfoForIndexColumns(indexInfo, tblInfo) colVals, err := tablecodec.DecodeIndexKV(key, value, len(indexInfo.Columns), tablecodec.HandleDefault, idxColInfos) diff --git a/pkg/ddl/ingest/integration_test.go b/pkg/ddl/ingest/integration_test.go index 2aba4f018354e..c2a3dd5c4ad8e 100644 --- a/pkg/ddl/ingest/integration_test.go +++ b/pkg/ddl/ingest/integration_test.go @@ -494,6 +494,15 @@ func TestAddGlobalIndexInIngest(t *testing.T) { require.Greater(t, len(rsGlobalIndex1.Rows()), len(rsGlobalIndex.Rows())) require.Equal(t, rsGlobalIndex1.String(), rsTable.String()) require.Equal(t, rsGlobalIndex1.String(), rsGlobalIndex2.String()) + + // for non-unique global idnexes + tk.MustExec("alter table t add index idx_7(b) global, add index idx_8(b) global") + rsNonUniqueGlobalIndex1 := tk.MustQuery("select * from t use index(idx_7)").Sort() + rsTable = tk.MustQuery("select * from t use index()").Sort() + rsNonUniqueGlobalIndex2 := tk.MustQuery("select * from t use index(idx_8)").Sort() + require.Greater(t, len(rsNonUniqueGlobalIndex1.Rows()), len(rsGlobalIndex.Rows())) + require.Equal(t, rsNonUniqueGlobalIndex1.String(), rsTable.String()) + require.Equal(t, rsNonUniqueGlobalIndex1.String(), rsNonUniqueGlobalIndex2.String()) } func TestAddGlobalIndexInIngestWithUpdate(t *testing.T) { diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index 80902fe4a37cd..bf3159e5805ad 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -3244,28 +3244,18 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver // When removing partitioning, set all indexes to 'local' since it will become a non-partitioned table! newGlobal = false } - if !index.Unique { - // for now, only unique index can be global, non-unique indexes are 'local' - // TODO: For the future loosen this restriction and allow non-unique global indexes - if newGlobal { - job.State = model.JobStateCancelled - return ver, dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs(fmt.Sprintf("PARTITION BY, index '%v' is not unique, but has Global Index set", index.Name.O)) - } + if !index.Global && !newGlobal { continue } inAllPartitionColumns, err := checkPartitionKeysConstraint(partInfo, index.Columns, tblInfo) if err != nil { return ver, errors.Trace(err) } - // Currently only support Explicit Global indexes. - if !inAllPartitionColumns && !newGlobal { + // Currently only support Explicit Global indexes for unique index. + if !inAllPartitionColumns && !newGlobal && index.Unique { job.State = model.JobStateCancelled return ver, dbterror.ErrGlobalIndexNotExplicitlySet.GenWithStackByArgs(index.Name.O) } - if !index.Global && !newGlobal { - // still local index, no need to duplicate index. - continue - } if tblInfo.Partition.DDLChangedIndex == nil { tblInfo.Partition.DDLChangedIndex = make(map[int64]bool) } @@ -3400,7 +3390,7 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver } for i := range tblInfo.Indices { - if tblInfo.Indices[i].Unique && tblInfo.Indices[i].State == model.StateDeleteOnly { + if tblInfo.Indices[i].State == model.StateDeleteOnly { tblInfo.Indices[i].State = model.StateWriteOnly } } @@ -3420,7 +3410,7 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver // so that new data will be updated in both old and new partitions when reorganizing. job.SnapshotVer = 0 for i := range tblInfo.Indices { - if tblInfo.Indices[i].Unique && tblInfo.Indices[i].State == model.StateWriteOnly { + if tblInfo.Indices[i].State == model.StateWriteOnly { tblInfo.Indices[i].State = model.StateWriteReorganization } } @@ -3462,9 +3452,6 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver }) for _, index := range tblInfo.Indices { - if !index.Unique { - continue - } isNew, ok := tblInfo.Partition.DDLChangedIndex[index.ID] if !ok { continue @@ -3564,8 +3551,8 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver var dropIndices []*model.IndexInfo for _, indexInfo := range tblInfo.Indices { - if indexInfo.Unique && indexInfo.State == model.StateDeleteOnly { - // Drop the old unique (possible global) index, see onDropIndex + if indexInfo.State == model.StateDeleteOnly { + // Drop the old indexes, see onDropIndex indexInfo.State = model.StateNone DropIndexColumnFlag(tblInfo, indexInfo) RemoveDependentHiddenColumns(tblInfo, indexInfo) diff --git a/tests/integrationtest/r/globalindex/ddl.result b/tests/integrationtest/r/globalindex/ddl.result index 735d7856b1a03..0b4bbdac32471 100644 --- a/tests/integrationtest/r/globalindex/ddl.result +++ b/tests/integrationtest/r/globalindex/ddl.result @@ -5,39 +5,11 @@ Error 8200 (HY000): Unsupported Global Index on non-partitioned table create table t (a int, b int, unique index idx(a) global) partition by hash(b) partitions 3; drop table t; create table t (a int, b int, index idx(a) global) partition by hash(b) partitions 3; -Error 8200 (HY000): Unsupported GLOBAL IndexOption on non-unique index +drop table t; create table t3(a int not null, b int, primary key(a) nonclustered, unique idx_b(b) global) partition by hash(a) partitions 3; drop table t3; create table t (a int primary key nonclustered, b int) partition by hash(b) partitions 3; Error 8264 (HY000): Global Index is needed for index 'PRIMARY', since the unique index is not including all partitioning columns, and GLOBAL is not given as IndexOption -create table t (a int, b int, unique key (a)) partition by hash(a) partitions 3; -alter table t partition by hash(b) partitions 3; -Error 8264 (HY000): Global Index is needed for index 'a', since the unique index is not including all partitioning columns, and GLOBAL is not given as IndexOption -alter table t partition by hash(b) partitions 3 update indexes (a global); -alter table t add index idxErr (b) global; -Error 8200 (HY000): Unsupported Global IndexOption on non-unique index -alter table t add unique index idxOK (b) global; -create index idxErr on t (b) global; -Error 8200 (HY000): Unsupported Global IndexOption on non-unique index -create unique index idxOK2 on t (b) global; -alter table t remove partitioning; -alter table t add index idxErr (b) global; -Error 8200 (HY000): Unsupported Global Index on non-partitioned table -alter table t add unique index idxErr (b) global; -Error 8200 (HY000): Unsupported Global Index on non-partitioned table -create index idxErr on t (b) global; -Error 8200 (HY000): Unsupported Global Index on non-partitioned table -create unique index idxErr on t (b) global; -Error 8200 (HY000): Unsupported Global Index on non-partitioned table -drop table t; -create table t (a int, b int, unique index idx(a) global); -Error 8200 (HY000): Unsupported Global Index on non-partitioned table -create table t (a int, b int, index idx(a) global); -Error 8200 (HY000): Unsupported Global Index on non-partitioned table -create table t (a int, b int, index idx(a) global) partition by hash(b) partitions 3; -Error 8200 (HY000): Unsupported GLOBAL IndexOption on non-unique index -create table t (a int not null, b int, primary key(a) nonclustered, unique idx_b(b) global) partition by hash(a) partitions 3; -drop table t; create table t (a int key global, b int) partition by hash(b) partitions 3; Error 8200 (HY000): Unsupported create an index that is both a global index and a clustered index create table t (a int unique, b int) partition by hash(b) partitions 3; @@ -65,14 +37,34 @@ create table t (a int, b int, unique key (a)) partition by hash(a) partitions 3; alter table t partition by hash(b) partitions 3; Error 8264 (HY000): Global Index is needed for index 'a', since the unique index is not including all partitioning columns, and GLOBAL is not given as IndexOption alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a GLOBAL); +alter table t add index idxOK (b) global; +alter table t add unique index idxOK2 (a) global; +alter table t add unique index idxOK3 (b) global; +create index idxOK4 on t (b) global; +create unique index idxOK5 on t (a) global; +create unique index idxOK6 on t (b) global; +alter table t remove partitioning; +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int DEFAULT NULL, + `b` int DEFAULT NULL, + UNIQUE KEY `a` (`a`), + KEY `idxOK` (`b`), + UNIQUE KEY `idxOK2` (`a`), + UNIQUE KEY `idxOK3` (`b`), + KEY `idxOK4` (`b`), + UNIQUE KEY `idxOK5` (`a`), + UNIQUE KEY `idxOK6` (`b`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin alter table t add index idxErr (b) global; -Error 8200 (HY000): Unsupported Global IndexOption on non-unique index -alter table t add unique index idxOK (a) global; -alter table t add unique index idxOK2 (b) global; +Error 8200 (HY000): Unsupported Global Index on non-partitioned table +alter table t add unique index idxErr (b) global; +Error 8200 (HY000): Unsupported Global Index on non-partitioned table create index idxErr on t (b) global; -Error 8200 (HY000): Unsupported Global IndexOption on non-unique index -create unique index idxOK3 on t (a) global; -create unique index idxOK4 on t (b) global; +Error 8200 (HY000): Unsupported Global Index on non-partitioned table +create unique index idxErr on t (b) global; +Error 8200 (HY000): Unsupported Global Index on non-partitioned table drop table t; create table t(a int, b int, primary key (a) nonclustered global); Error 8200 (HY000): Unsupported Global Index on non-partitioned table @@ -82,6 +74,72 @@ create table t(a int, b int, primary key (a) global) partition by hash(a) partit Error 8200 (HY000): Unsupported create an index that is both a global index and a clustered index create table t(a int, b int, primary key (b) global) partition by hash(a) partitions 5; Error 8200 (HY000): Unsupported create an index that is both a global index and a clustered index +create table t(a int, b int, key(a), key(b)) partition by hash(a) partitions 4; +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a GLOBAL, b LOCAL); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int DEFAULT NULL, + `b` int DEFAULT NULL, + KEY `b` (`b`), + KEY `a` (`a`) /*T![global_index] GLOBAL */ +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +PARTITION BY HASH (`b`) PARTITIONS 3 +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a GLOBAL, b GLOBAL); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int DEFAULT NULL, + `b` int DEFAULT NULL, + KEY `b` (`b`) /*T![global_index] GLOBAL */, + KEY `a` (`a`) /*T![global_index] GLOBAL */ +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +PARTITION BY HASH (`b`) PARTITIONS 3 +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a LOCAL); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int DEFAULT NULL, + `b` int DEFAULT NULL, + KEY `b` (`b`) /*T![global_index] GLOBAL */, + KEY `a` (`a`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +PARTITION BY HASH (`b`) PARTITIONS 3 +drop table t; +create table t(a int, b int, unique key(a), unique key(b) global) partition by hash(a) partitions 4; +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a GLOBAL, b LOCAL); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int DEFAULT NULL, + `b` int DEFAULT NULL, + UNIQUE KEY `a` (`a`) /*T![global_index] GLOBAL */, + UNIQUE KEY `b` (`b`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +PARTITION BY HASH (`b`) PARTITIONS 3 +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a GLOBAL, b GLOBAL); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int DEFAULT NULL, + `b` int DEFAULT NULL, + UNIQUE KEY `a` (`a`) /*T![global_index] GLOBAL */, + UNIQUE KEY `b` (`b`) /*T![global_index] GLOBAL */ +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +PARTITION BY HASH (`b`) PARTITIONS 3 +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a LOCAL); +Error 8264 (HY000): Global Index is needed for index 'a', since the unique index is not including all partitioning columns, and GLOBAL is not given as IndexOption +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (b LOCAL); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int DEFAULT NULL, + `b` int DEFAULT NULL, + UNIQUE KEY `a` (`a`) /*T![global_index] GLOBAL */, + UNIQUE KEY `b` (`b`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +PARTITION BY HASH (`b`) PARTITIONS 3 +drop table t; create table t(a int, b int); alter table t add primary key (a) global; Error 8200 (HY000): Unsupported Global Index on non-partitioned table diff --git a/tests/integrationtest/r/globalindex/expression_index.result b/tests/integrationtest/r/globalindex/expression_index.result index d8460fd4d771b..ec2a175682e79 100644 --- a/tests/integrationtest/r/globalindex/expression_index.result +++ b/tests/integrationtest/r/globalindex/expression_index.result @@ -40,3 +40,13 @@ Projection 3333.33 root NULL globalindex__expression_index.t.a, globalindex__exp select * from t partition(p0) use index(idx) where lower(b) > 'c'; a b 5 x +drop table if exists t; +CREATE TABLE `t` ( +`a` int DEFAULT NULL, +`b` char DEFAULT NULL, +KEY `idx` ((lower(`b`))) global +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +PARTITION BY HASH (`a`) PARTITIONS 5; +show warnings; +Level Code Message +Warning 8265 Auto analyze is not effective for index 'idx', need analyze manually diff --git a/tests/integrationtest/r/globalindex/insert.result b/tests/integrationtest/r/globalindex/insert.result index e873815ea20ff..94bf8a12a5359 100644 --- a/tests/integrationtest/r/globalindex/insert.result +++ b/tests/integrationtest/r/globalindex/insert.result @@ -10,3 +10,20 @@ select * from t use index (idx1) order by a desc; a b 2 5 1 3 +drop table if exists t; +create table t(a int, b int, index idx(a) global) partition by hash(b) partitions 5; +insert into t values (1, 1), (1, 2), (2, 2); +select * from t use index (idx); +a b +1 1 +1 2 +2 2 +alter table t add index idx1(b) global; +insert into t values (2, 4), (3, 4); +select * from t use index (idx1) order by a desc, b; +a b +3 4 +2 2 +2 4 +1 1 +1 2 diff --git a/tests/integrationtest/r/globalindex/update.result b/tests/integrationtest/r/globalindex/update.result index 6a138c912af1c..8ae79d047ba9f 100644 --- a/tests/integrationtest/r/globalindex/update.result +++ b/tests/integrationtest/r/globalindex/update.result @@ -60,3 +60,14 @@ replace into t values ('',826536 ); select * from t; a b 826536 +drop table t; +create table t(a int, b int, index idx(a) global) partition by hash(b) partitions 5; +insert into t values (1, 2), (1, 3), (1, 4); +replace into t values (2, 3); +update t set a = 3, b = 4 where a = 1; +select * from t; +a b +2 3 +3 4 +3 4 +3 4 diff --git a/tests/integrationtest/t/globalindex/ddl.test b/tests/integrationtest/t/globalindex/ddl.test index 3fcbb99f503aa..6807370d228d8 100644 --- a/tests/integrationtest/t/globalindex/ddl.test +++ b/tests/integrationtest/t/globalindex/ddl.test @@ -4,41 +4,13 @@ create table t (a int, b int, unique index idx(a) global); create table t (a int, b int, index idx(a) global); create table t (a int, b int, unique index idx(a) global) partition by hash(b) partitions 3; drop table t; --- error 8200 create table t (a int, b int, index idx(a) global) partition by hash(b) partitions 3; +drop table t; create table t3(a int not null, b int, primary key(a) nonclustered, unique idx_b(b) global) partition by hash(a) partitions 3; drop table t3; -- error 8264 create table t (a int primary key nonclustered, b int) partition by hash(b) partitions 3; -create table t (a int, b int, unique key (a)) partition by hash(a) partitions 3; --- error 8264 -alter table t partition by hash(b) partitions 3; -alter table t partition by hash(b) partitions 3 update indexes (a global); --- error 8200 -alter table t add index idxErr (b) global; -alter table t add unique index idxOK (b) global; --- error 8200 -create index idxErr on t (b) global; -create unique index idxOK2 on t (b) global; -alter table t remove partitioning; --- error 8200 -alter table t add index idxErr (b) global; --- error 8200 -alter table t add unique index idxErr (b) global; --- error 8200 -create index idxErr on t (b) global; --- error 8200 -create unique index idxErr on t (b) global; -drop table t; --- error 8200 -create table t (a int, b int, unique index idx(a) global); --- error 8200 -create table t (a int, b int, index idx(a) global); --- error 8200 -create table t (a int, b int, index idx(a) global) partition by hash(b) partitions 3; -create table t (a int not null, b int, primary key(a) nonclustered, unique idx_b(b) global) partition by hash(a) partitions 3; -drop table t; -- error 8200 create table t (a int key global, b int) partition by hash(b) partitions 3; -- error 8264 @@ -47,6 +19,7 @@ create table t (a int unique, b int) partition by hash(b) partitions 3; create table t (a int unique key, b int) partition by hash(b) partitions 3; -- error 8264 create table t (a int primary key nonclustered, b int) partition by hash(b) partitions 3; + CREATE TABLE `t` ( `a` int(11) NOT NULL, `b` int(11) DEFAULT NULL, @@ -55,20 +28,30 @@ CREATE TABLE `t` ( PARTITION BY HASH (`b`) PARTITIONS 3; show create table t; drop table t; + create table t (a int, b int, unique key (a)) partition by hash(a) partitions 3; -- error 8264 alter table t partition by hash(b) partitions 3; alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a GLOBAL); +alter table t add index idxOK (b) global; +alter table t add unique index idxOK2 (a) global; +alter table t add unique index idxOK3 (b) global; +create index idxOK4 on t (b) global; +create unique index idxOK5 on t (a) global; +create unique index idxOK6 on t (b) global; + +alter table t remove partitioning; +show create table t; -- error 8200 alter table t add index idxErr (b) global; -alter table t add unique index idxOK (a) global; -alter table t add unique index idxOK2 (b) global; +-- error 8200 +alter table t add unique index idxErr (b) global; -- error 8200 create index idxErr on t (b) global; -create unique index idxOK3 on t (a) global; -create unique index idxOK4 on t (b) global; - +-- error 8200 +create unique index idxErr on t (b) global; drop table t; + --error 8200 create table t(a int, b int, primary key (a) nonclustered global); --error 8200 @@ -78,6 +61,26 @@ create table t(a int, b int, primary key (a) global) partition by hash(a) partit --error 8200 create table t(a int, b int, primary key (b) global) partition by hash(a) partitions 5; +create table t(a int, b int, key(a), key(b)) partition by hash(a) partitions 4; +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a GLOBAL, b LOCAL); +show create table t; +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a GLOBAL, b GLOBAL); +show create table t; +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a LOCAL); +show create table t; +drop table t; + +create table t(a int, b int, unique key(a), unique key(b) global) partition by hash(a) partitions 4; +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a GLOBAL, b LOCAL); +show create table t; +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a GLOBAL, b GLOBAL); +show create table t; +--error 8264 +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a LOCAL); +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (b LOCAL); +show create table t; +drop table t; + create table t(a int, b int); --error 8200 alter table t add primary key (a) global; diff --git a/tests/integrationtest/t/globalindex/expression_index.test b/tests/integrationtest/t/globalindex/expression_index.test index de5d36dc1ab2f..52ec1761502ba 100644 --- a/tests/integrationtest/t/globalindex/expression_index.test +++ b/tests/integrationtest/t/globalindex/expression_index.test @@ -23,3 +23,15 @@ select * from t use index(idx) where lower(b) > 'c' order by lower(b); explain format='brief' select * from t partition(p0) use index(idx) where lower(b) > 'c'; select * from t partition(p0) use index(idx) where lower(b) > 'c'; + +# For global index + non-unique index +drop table if exists t; +CREATE TABLE `t` ( + `a` int DEFAULT NULL, + `b` char DEFAULT NULL, + KEY `idx` ((lower(`b`))) global +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +PARTITION BY HASH (`a`) PARTITIONS 5; + +show warnings; + diff --git a/tests/integrationtest/t/globalindex/insert.test b/tests/integrationtest/t/globalindex/insert.test index 9e641354a341c..cb7457680fee7 100644 --- a/tests/integrationtest/t/globalindex/insert.test +++ b/tests/integrationtest/t/globalindex/insert.test @@ -8,3 +8,12 @@ insert into t values (2, 4), (3, 4) on duplicate key update a=2, b=5; select * from t use index (idx1) order by a desc; +# Test global index + non-unique index +drop table if exists t; +create table t(a int, b int, index idx(a) global) partition by hash(b) partitions 5; +insert into t values (1, 1), (1, 2), (2, 2); +select * from t use index (idx); + +alter table t add index idx1(b) global; +insert into t values (2, 4), (3, 4); +select * from t use index (idx1) order by a desc, b; diff --git a/tests/integrationtest/t/globalindex/update.test b/tests/integrationtest/t/globalindex/update.test index e383f37f589d6..8a026ba854316 100644 --- a/tests/integrationtest/t/globalindex/update.test +++ b/tests/integrationtest/t/globalindex/update.test @@ -43,3 +43,14 @@ create table t(a varchar(70), b mediumint(9), unique index idx_a(a) global, uniq insert into t values ('',826534 ); replace into t values ('',826536 ); select * from t; + +# For global index + non-unique index +drop table t; +create table t(a int, b int, index idx(a) global) partition by hash(b) partitions 5; +insert into t values (1, 2), (1, 3), (1, 4); + +# Replace will not affects, `idx` is a non-unique index. +replace into t values (2, 3); +update t set a = 3, b = 4 where a = 1; +--sorted_result +select * from t;