Skip to content

Commit

Permalink
ddl: Global index more restrictions (#55440)
Browse files Browse the repository at this point in the history
close #55424
  • Loading branch information
mjonss authored Aug 15, 2024
1 parent 881eedd commit 2045418
Show file tree
Hide file tree
Showing 8 changed files with 234 additions and 44 deletions.
7 changes: 5 additions & 2 deletions pkg/ddl/create_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -414,7 +414,7 @@ func buildTableInfoWithCheck(ctx sessionctx.Context, s *ast.CreateTableStmt, dbC
if err = checkTableInfoValidWithStmt(ctx, tbInfo, s); err != nil {
return nil, err
}
if err = checkTableInfoValidExtra(tbInfo); err != nil {
if err = checkTableInfoValidExtra(ctx, tbInfo); err != nil {
return nil, err
}
return tbInfo, nil
Expand Down Expand Up @@ -516,7 +516,7 @@ func checkGeneratedColumn(ctx sessionctx.Context, schemaName model.CIStr, tableN
// name length and column count.
// (checkTableInfoValid is also used in repairing objects which don't perform
// these checks. Perhaps the two functions should be merged together regardless?)
func checkTableInfoValidExtra(tbInfo *model.TableInfo) error {
func checkTableInfoValidExtra(ctx sessionctx.Context, tbInfo *model.TableInfo) error {
if err := checkTooLongTable(tbInfo.Name); err != nil {
return err
}
Expand All @@ -536,6 +536,9 @@ func checkTableInfoValidExtra(tbInfo *model.TableInfo) error {
if err := checkColumnsAttributes(tbInfo.Columns); err != nil {
return errors.Trace(err)
}
if err := checkGlobalIndexes(ctx, tbInfo); err != nil {
return errors.Trace(err)
}

// FIXME: perform checkConstraintNames
if err := checkCharsetAndCollation(tbInfo.Charset, tbInfo.Collate); err != nil {
Expand Down
61 changes: 59 additions & 2 deletions pkg/ddl/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -940,6 +940,47 @@ func checkInvisibleIndexOnPK(tblInfo *model.TableInfo) error {
return nil
}

// checkGlobalIndex check if the index is allowed to have global index
func checkGlobalIndex(ctx sessionctx.Context, tblInfo *model.TableInfo, indexInfo *model.IndexInfo) error {
pi := tblInfo.GetPartitionInfo()
isPartitioned := pi != nil && pi.Type != model.PartitionTypeNone
if indexInfo.Global {
if !isPartitioned {
// Makes no sense with LOCAL/GLOBAL index for non-partitioned tables, since we don't support
// partitioning an index differently from the table partitioning.
return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("Global Index on non-partitioned table")
}
if !ctx.GetSessionVars().EnableGlobalIndex {
return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("GLOBAL IndexOption when tidb_enable_global_index is disabled")
}
// TODO: remove limitation
if !indexInfo.Unique {
return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("GLOBAL IndexOption on non-unique index")
}
// TODO: remove limitation
// check that not all partitioned columns are included.
inAllPartitionColumns, err := checkPartitionKeysConstraint(pi, indexInfo.Columns, tblInfo)
if err != nil {
return errors.Trace(err)
}
if inAllPartitionColumns {
return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("Global Index including all columns in the partitioning expression")
}
}
return nil
}

// checkGlobalIndexes check if global index is supported.
func checkGlobalIndexes(ctx sessionctx.Context, tblInfo *model.TableInfo) error {
for _, indexInfo := range tblInfo.Indices {
err := checkGlobalIndex(ctx, tblInfo, indexInfo)
if err != nil {
return err
}
}
return nil
}

func (e *executor) assignPartitionIDs(defs []model.PartitionDefinition) error {
genIDs, err := e.genGlobalIDs(len(defs))
if err != nil {
Expand Down Expand Up @@ -1048,7 +1089,7 @@ func (e *executor) createTableWithInfoJob(
}
}

if err := checkTableInfoValidExtra(tbInfo); err != nil {
if err := checkTableInfoValidExtra(ctx, tbInfo); err != nil {
return nil, err
}

Expand Down Expand Up @@ -4650,6 +4691,19 @@ func (e *executor) createIndex(ctx sessionctx.Context, ti ast.Ident, keyType ast
return errors.Trace(err)
}

globalIndex := false
if indexOption != nil && indexOption.Global {
globalIndex = true
}
if globalIndex {
if tblInfo.GetPartitionInfo() == nil {
return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("Global Index on non-partitioned table")
}
if !unique {
// TODO: remove this limitation
return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("Global IndexOption on non-unique index")
}
}
if unique && tblInfo.GetPartitionInfo() != nil {
ck, err := checkPartitionKeysConstraint(tblInfo.GetPartitionInfo(), indexColumns, tblInfo)
if err != nil {
Expand All @@ -4660,9 +4714,12 @@ func (e *executor) createIndex(ctx sessionctx.Context, ti ast.Ident, keyType ast
return dbterror.ErrUniqueKeyNeedAllFieldsInPf.GenWithStackByArgs("UNIQUE INDEX")
}
// index columns does not contain all partition columns, must set global
if indexOption == nil || !indexOption.Global {
if !globalIndex {
return dbterror.ErrGlobalIndexNotExplicitlySet.GenWithStackByArgs(indexName.O)
}
} else if globalIndex {
// TODO: remove this restriction
return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("Global IndexOption on index including all columns in the partitioning expression")
}
}
// May be truncate comment here, when index comment too long and sql_mode is't strict.
Expand Down
2 changes: 1 addition & 1 deletion pkg/ddl/tests/partition/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@ go_test(
"main_test.go",
],
flaky = True,
shard_count = 50,
shard_count = 49,
deps = [
"//pkg/config",
"//pkg/ddl",
Expand Down
36 changes: 0 additions & 36 deletions pkg/ddl/tests/partition/db_partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3733,39 +3733,3 @@ func checkGlobalAndPK(t *testing.T, tk *testkit.TestKit, name string, indexes in
require.True(t, idxInfo.Primary)
}
}
func TestGlobalIndexExplicitOption(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("set tidb_enable_global_index=OFF")
defer func() {
tk.MustExec("set tidb_enable_global_index=default")
}()
tk.MustContainErrMsg(`create table t3(a int not null, b int, primary key(a) nonclustered, unique idx_b(b) global) partition by hash(a) partitions 3`, "[ddl:1503]A UNIQUE INDEX must include all columns in the table's partitioning function")

tk.MustContainErrMsg(`create table t (a int primary key nonclustered, b int) partition by hash(b) partitions 3`, "[ddl:1503]A PRIMARY KEY must include all columns in the table's partitioning function")
tk.MustExec(`create table t (a int, b int, unique key (a)) partition by hash(a) partitions 3`)
tk.MustContainErrMsg(`alter table t partition by hash(b) partitions 3`, "[ddl:1503]A UNIQUE INDEX must include all columns in the table's partitioning function")
tk.MustContainErrMsg(`alter table t partition by hash(b) partitions 3 update indexes (a global)`, "[ddl:1503]A UNIQUE INDEX must include all columns in the table's partitioning function")
tk.MustExec(`drop table t`)

tk.MustExec("set tidb_enable_global_index=ON")
tk.MustExec(`create table t (a int not null, b int, primary key(a) nonclustered, unique idx_b(b) global) partition by hash(a) partitions 3`)
tk.MustExec(`drop table t`)
tk.MustContainErrMsg(`create table t (a int key global, b int) partition by hash(b) partitions 3`, "[ddl:1503]A CLUSTERED INDEX must include all columns in the table's partitioning function")
tk.MustContainErrMsg(`create table t (a int unique, b int) partition by hash(b) partitions 3`, "[ddl:8264]Global Index is needed for index 'a', since the unique index is not including all partitioning columns, and GLOBAL is not given as IndexOption")
tk.MustContainErrMsg(`create table t (a int unique key, b int) partition by hash(b) partitions 3`, "[ddl:8264]Global Index is needed for index 'a', since the unique index is not including all partitioning columns, and GLOBAL is not given as IndexOption")
tk.MustContainErrMsg(`create table t (a int primary key nonclustered, b int) partition by hash(b) partitions 3`, "[ddl:8264]Global Index is needed for index 'PRIMARY', since the unique index is not including all partitioning columns, and GLOBAL is not given as IndexOption")
createTable := "CREATE TABLE `t` (\n" +
" `a` int(11) NOT NULL,\n" +
" `b` int(11) DEFAULT NULL,\n" +
" PRIMARY KEY (`a`) /*T![clustered_index] NONCLUSTERED */ /*T![global_index] GLOBAL */\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" +
"PARTITION BY HASH (`b`) PARTITIONS 3"
tk.MustExec(createTable)
tk.MustQuery(`show create table t`).Check(testkit.Rows("t " + createTable))
tk.MustExec(`drop table t`)
tk.MustExec(`create table t (a int, b int, unique key (a)) partition by hash(a) partitions 3`)
tk.MustContainErrMsg(`alter table t partition by hash(b) partitions 3`, "[ddl:8264]Global Index is needed for index 'a', since the unique index is not including all partitioning columns, and GLOBAL is not given as IndexOption")
tk.MustExec(`alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a GLOBAL)`)
}
6 changes: 5 additions & 1 deletion tests/integrationtest/r/ddl/db_partition.result
Original file line number Diff line number Diff line change
Expand Up @@ -3220,9 +3220,13 @@ create table pt14 (id int not null, lname varchar(30), fname varchar(100) genera
create table nt14 (id int not null, lname varchar(30), fname varchar(100) generated always as (concat(lname, ' ')) virtual);
alter table pt14 exchange partition p0 with table nt14;
## unique index
create table pt15 (id int not null, unique index uk_id (id) global) partition by hash(id) partitions 1;
create table pt15 (id int not null, unique index uk_id (id)) partition by hash(id) partitions 1;
create table nt15 (id int not null, index uk_id (id));
alter table pt15 exchange partition p0 with table nt15;
Error 1736 (HY000): Tables have different definitions
create table pt15b (id int not null, a int, unique index uk_id (a) global) partition by hash(id) partitions 1;
create table nt15b (id int not null, a int, unique index uk_id (a));
alter table pt15b exchange partition p0 with table nt15b;
Error 1731 (HY000): Non matching attribute 'global index: uk_id' between partition and table
## auto_increment
create table pt16 (id int not null primary key auto_increment) partition by hash(id) partitions 1;
Expand Down
82 changes: 82 additions & 0 deletions tests/integrationtest/r/globalindex/ddl.result
Original file line number Diff line number Diff line change
@@ -0,0 +1,82 @@
set tidb_enable_global_index=OFF;
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, unique index idx(a) global) partition by hash(b) partitions 3;
Error 1503 (HY000): A UNIQUE INDEX must include all columns in the table's partitioning function
create table t (a int, b int, index idx(a) global) partition by hash(b) partitions 3;
Error 8200 (HY000): Unsupported GLOBAL IndexOption when tidb_enable_global_index is disabled
create table t3(a int not null, b int, primary key(a) nonclustered, unique idx_b(b) global) partition by hash(a) partitions 3;
Error 1503 (HY000): A UNIQUE INDEX must include all columns in the table's partitioning function
create table t (a int primary key nonclustered, b int) partition by hash(b) partitions 3;
Error 1503 (HY000): A PRIMARY KEY must include all columns in the table's partitioning function
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 1503 (HY000): A UNIQUE INDEX must include all columns in the table's partitioning function
alter table t partition by hash(b) partitions 3 update indexes (a global);
Error 1503 (HY000): A UNIQUE INDEX must include all columns in the table's partitioning function
alter table t add index idxErr (b) global;
Error 8200 (HY000): Unsupported Global IndexOption on non-unique index
alter table t add unique index idxErr (b) global;
Error 1503 (HY000): A UNIQUE INDEX must include all columns in the table's partitioning function
create index idxErr on t (b) global;
Error 8200 (HY000): Unsupported Global IndexOption on non-unique index
create unique index idxErr on t (b) global;
Error 1503 (HY000): A UNIQUE INDEX must include all columns in the table's partitioning function
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;
set tidb_enable_global_index=ON;
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 1503 (HY000): A CLUSTERED INDEX must include all columns in the table's partitioning function
create table t (a int unique, b int) 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
create table t (a int unique key, b int) 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
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(11) NOT NULL,
`b` int(11) DEFAULT NULL,
PRIMARY KEY (`a`) /*T![clustered_index] NONCLUSTERED */ /*T![global_index] GLOBAL */
) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin
PARTITION BY HASH (`b`) PARTITIONS 3;
show create table t;
Table Create Table
t CREATE TABLE `t` (
`a` int(11) NOT NULL,
`b` int(11) DEFAULT NULL,
PRIMARY KEY (`a`) /*T![clustered_index] NONCLUSTERED */ /*T![global_index] GLOBAL */
) 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)) 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 (a) global;
alter table t add unique index idxErr (b) global;
Error 8200 (HY000): Unsupported Global IndexOption on index including all columns in the partitioning expression
create index idxErr on t (b) global;
Error 8200 (HY000): Unsupported Global IndexOption on non-unique index
create unique index idxOK2 on t (a) global;
create unique index idxErr on t (b) global;
Error 8200 (HY000): Unsupported Global IndexOption on index including all columns in the partitioning expression
8 changes: 6 additions & 2 deletions tests/integrationtest/t/ddl/db_partition.test
Original file line number Diff line number Diff line change
Expand Up @@ -2168,10 +2168,14 @@ create table nt14 (id int not null, lname varchar(30), fname varchar(100) genera
alter table pt14 exchange partition p0 with table nt14;

--echo ## unique index
create table pt15 (id int not null, unique index uk_id (id) global) partition by hash(id) partitions 1;
create table pt15 (id int not null, unique index uk_id (id)) partition by hash(id) partitions 1;
create table nt15 (id int not null, index uk_id (id));
-- error 1731
-- error 1736
alter table pt15 exchange partition p0 with table nt15;
create table pt15b (id int not null, a int, unique index uk_id (a) global) partition by hash(id) partitions 1;
create table nt15b (id int not null, a int, unique index uk_id (a));
-- error 1731
alter table pt15b exchange partition p0 with table nt15b;

--echo ## auto_increment
create table pt16 (id int not null primary key auto_increment) partition by hash(id) partitions 1;
Expand Down
76 changes: 76 additions & 0 deletions tests/integrationtest/t/globalindex/ddl.test
Original file line number Diff line number Diff line change
@@ -0,0 +1,76 @@
set tidb_enable_global_index=OFF;
-- 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 1503
create table t (a int, b int, unique index idx(a) global) partition by hash(b) partitions 3;
-- error 8200
create table t (a int, b int, index idx(a) global) partition by hash(b) partitions 3;
-- error 1503
create table t3(a int not null, b int, primary key(a) nonclustered, unique idx_b(b) global) partition by hash(a) partitions 3;
-- error 1503
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 1503
alter table t partition by hash(b) partitions 3;
-- error 1503
alter table t partition by hash(b) partitions 3 update indexes (a global);
-- error 8200
alter table t add index idxErr (b) global;
-- error 1503
alter table t add unique index idxErr (b) global;
-- error 8200
create index idxErr on t (b) global;
-- error 1503
create unique index idxErr 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;

set tidb_enable_global_index=ON;
-- 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 1503
create table t (a int key global, b int) partition by hash(b) partitions 3;
-- error 8264
create table t (a int unique, b int) partition by hash(b) partitions 3;
-- error 8264
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,
PRIMARY KEY (`a`) /*T![clustered_index] NONCLUSTERED */ /*T![global_index] GLOBAL */
) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin
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);
-- error 8200
alter table t add index idxErr (b) global;
alter table t add unique index idxOK (a) global;
-- error 8200
alter table t add unique index idxErr (b) global;
-- error 8200
create index idxErr on t (b) global;
create unique index idxOK2 on t (a) global;
-- error 8200
create unique index idxErr on t (b) global;

0 comments on commit 2045418

Please sign in to comment.