Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
16 changes: 5 additions & 11 deletions pkg/ddl/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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 {
Expand All @@ -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
Expand Down Expand Up @@ -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
}

Expand Down Expand Up @@ -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
}

Expand Down
45 changes: 44 additions & 1 deletion pkg/ddl/index_modify_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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")
Expand Down Expand Up @@ -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)
Expand Down
9 changes: 9 additions & 0 deletions pkg/ddl/ingest/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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) {
Expand Down
27 changes: 7 additions & 20 deletions pkg/ddl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
Expand Down Expand Up @@ -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
}
}
Expand All @@ -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
}
}
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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)
Expand Down
128 changes: 93 additions & 35 deletions tests/integrationtest/r/globalindex/ddl.result
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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
Expand All @@ -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
10 changes: 10 additions & 0 deletions tests/integrationtest/r/globalindex/expression_index.result
Original file line number Diff line number Diff line change
Expand Up @@ -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
17 changes: 17 additions & 0 deletions tests/integrationtest/r/globalindex/insert.result
Original file line number Diff line number Diff line change
Expand Up @@ -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
Loading