Skip to content

Commit

Permalink
*: Deprecated and always set to ON tidb_enable_global_index (#56169)
Browse files Browse the repository at this point in the history
ref #45133
  • Loading branch information
mjonss authored Sep 24, 2024
1 parent 1787505 commit 0442c68
Show file tree
Hide file tree
Showing 54 changed files with 132 additions and 318 deletions.
6 changes: 3 additions & 3 deletions pkg/ddl/create_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -405,7 +405,7 @@ func buildTableInfoWithCheck(ctx sessionctx.Context, s *ast.CreateTableStmt, dbC
if err = checkTableInfoValidWithStmt(ctx, tbInfo, s); err != nil {
return nil, err
}
if err = checkTableInfoValidExtra(ctx, tbInfo); err != nil {
if err = checkTableInfoValidExtra(tbInfo); err != nil {
return nil, err
}
return tbInfo, nil
Expand Down Expand Up @@ -507,7 +507,7 @@ func checkGeneratedColumn(ctx sessionctx.Context, schemaName pmodel.CIStr, table
// 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(ctx sessionctx.Context, tbInfo *model.TableInfo) error {
func checkTableInfoValidExtra(tbInfo *model.TableInfo) error {
if err := checkTooLongTable(tbInfo.Name); err != nil {
return err
}
Expand All @@ -527,7 +527,7 @@ func checkTableInfoValidExtra(ctx sessionctx.Context, tbInfo *model.TableInfo) e
if err := checkColumnsAttributes(tbInfo.Columns); err != nil {
return errors.Trace(err)
}
if err := checkGlobalIndexes(ctx, tbInfo); err != nil {
if err := checkGlobalIndexes(tbInfo); err != nil {
return errors.Trace(err)
}

Expand Down
50 changes: 23 additions & 27 deletions pkg/ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2385,35 +2385,31 @@ func TestDuplicateErrorMessage(t *testing.T) {

for _, newCollate := range []bool{false, true} {
collate.SetNewCollationEnabledForTest(newCollate)
for _, globalIndex := range []bool{false, true} {
tk.MustExec(fmt.Sprintf("set tidb_enable_global_index=%t", globalIndex))
for _, clusteredIndex := range []variable.ClusteredIndexDefMode{variable.ClusteredIndexDefModeOn, variable.ClusteredIndexDefModeOff, variable.ClusteredIndexDefModeIntOnly} {
tk.Session().GetSessionVars().EnableClusteredIndex = clusteredIndex
for _, t := range tests {
tk.MustExec("drop table if exists t;")
fields := make([]string, len(t.types))

for i, tp := range t.types {
fields[i] = fmt.Sprintf("a%d %s", i, tp)
}
tk.MustExec("create table t (id1 int, id2 varchar(10), " + strings.Join(fields, ",") + ",primary key(id1, id2)) " +
"collate utf8mb4_general_ci " +
"partition by range (id1) (partition p1 values less than (2), partition p2 values less than (maxvalue))")

vals := strings.Join(t.values, ",")
tk.MustExec(fmt.Sprintf("insert into t values (1, 'asd', %s), (1, 'dsa', %s)", vals, vals))
for i := range t.types {
fields[i] = fmt.Sprintf("a%d", i)
}
index := strings.Join(fields, ",")
for i, val := range t.values {
fields[i] = strings.Replace(val, "'", "", -1)
}
tk.MustGetErrMsg("alter table t add unique index t_idx(id1,"+index+")",
fmt.Sprintf("[kv:1062]Duplicate entry '1-%s' for key 't.t_idx'", strings.Join(fields, "-")))
for _, clusteredIndex := range []variable.ClusteredIndexDefMode{variable.ClusteredIndexDefModeOn, variable.ClusteredIndexDefModeOff, variable.ClusteredIndexDefModeIntOnly} {
tk.Session().GetSessionVars().EnableClusteredIndex = clusteredIndex
for _, t := range tests {
tk.MustExec("drop table if exists t;")
fields := make([]string, len(t.types))

for i, tp := range t.types {
fields[i] = fmt.Sprintf("a%d %s", i, tp)
}
tk.MustExec("create table t (id1 int, id2 varchar(10), " + strings.Join(fields, ",") + ",primary key(id1, id2)) " +
"collate utf8mb4_general_ci " +
"partition by range (id1) (partition p1 values less than (2), partition p2 values less than (maxvalue))")

vals := strings.Join(t.values, ",")
tk.MustExec(fmt.Sprintf("insert into t values (1, 'asd', %s), (1, 'dsa', %s)", vals, vals))
for i := range t.types {
fields[i] = fmt.Sprintf("a%d", i)
}
index := strings.Join(fields, ",")
for i, val := range t.values {
fields[i] = strings.Replace(val, "'", "", -1)
}
tk.MustGetErrMsg("alter table t add unique index t_idx(id1,"+index+")",
fmt.Sprintf("[kv:1062]Duplicate entry '1-%s' for key 't.t_idx'", strings.Join(fields, "-")))
}
tk.MustExec("set tidb_enable_global_index=default")
}
}
}
Expand Down
21 changes: 6 additions & 15 deletions pkg/ddl/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -952,7 +952,7 @@ func checkInvisibleIndexOnPK(tblInfo *model.TableInfo) error {
}

// checkGlobalIndex check if the index is allowed to have global index
func checkGlobalIndex(ctx sessionctx.Context, tblInfo *model.TableInfo, indexInfo *model.IndexInfo) error {
func checkGlobalIndex(tblInfo *model.TableInfo, indexInfo *model.IndexInfo) error {
pi := tblInfo.GetPartitionInfo()
isPartitioned := pi != nil && pi.Type != pmodel.PartitionTypeNone
if indexInfo.Global {
Expand All @@ -961,9 +961,6 @@ func checkGlobalIndex(ctx sessionctx.Context, tblInfo *model.TableInfo, indexInf
// 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")
Expand All @@ -982,9 +979,9 @@ func checkGlobalIndex(ctx sessionctx.Context, tblInfo *model.TableInfo, indexInf
}

// checkGlobalIndexes check if global index is supported.
func checkGlobalIndexes(ctx sessionctx.Context, tblInfo *model.TableInfo) error {
func checkGlobalIndexes(tblInfo *model.TableInfo) error {
for _, indexInfo := range tblInfo.Indices {
err := checkGlobalIndex(ctx, tblInfo, indexInfo)
err := checkGlobalIndex(tblInfo, indexInfo)
if err != nil {
return err
}
Expand Down Expand Up @@ -1093,7 +1090,7 @@ func (e *executor) createTableWithInfoJob(
}
}

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

Expand Down Expand Up @@ -4556,10 +4553,7 @@ func (e *executor) CreatePrimaryKey(ctx sessionctx.Context, ti ast.Ident, indexN
return err
}
if !ck {
if !ctx.GetSessionVars().EnableGlobalIndex {
return dbterror.ErrUniqueKeyNeedAllFieldsInPf.GenWithStackByArgs("PRIMARY")
}
// index columns does not contain all partition columns, must set global
// index columns does not contain all partition columns, must be global
if indexOption == nil || !indexOption.Global {
return dbterror.ErrGlobalIndexNotExplicitlySet.GenWithStackByArgs("PRIMARY")
}
Expand Down Expand Up @@ -4718,10 +4712,7 @@ func (e *executor) createIndex(ctx sessionctx.Context, ti ast.Ident, keyType ast
return err
}
if !ck {
if !ctx.GetSessionVars().EnableGlobalIndex {
return dbterror.ErrUniqueKeyNeedAllFieldsInPf.GenWithStackByArgs("UNIQUE INDEX")
}
// index columns does not contain all partition columns, must set global
// index columns does not contain all partition columns, must be global
if !globalIndex {
return dbterror.ErrGlobalIndexNotExplicitlySet.GenWithStackByArgs(indexName.O)
}
Expand Down
4 changes: 0 additions & 4 deletions pkg/ddl/index_modify_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -681,10 +681,6 @@ func TestAddGlobalIndex(t *testing.T) {
store := testkit.CreateMockStoreWithSchemaLease(t, indexModifyLease)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("set tidb_enable_global_index=true")
defer func() {
tk.MustExec("set tidb_enable_global_index=default")
}()
tk.MustExec("create table test_t1 (a int, b int) partition by range (b)" +
" (partition p0 values less than (10), " +
" partition p1 values less than (maxvalue));")
Expand Down
35 changes: 7 additions & 28 deletions pkg/ddl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -643,21 +643,8 @@ func buildTablePartitionInfo(ctx sessionctx.Context, s *ast.PartitionOptions, tb
return err
}
if !ck {
indexTp := ""
if !ctx.GetSessionVars().EnableGlobalIndex {
if index.Primary {
indexTp = "PRIMARY KEY"
if tbInfo.IsCommonHandle {
indexTp = "CLUSTERED INDEX"
}
} else {
indexTp = "UNIQUE INDEX"
}
} else if index.Primary && tbInfo.IsCommonHandle {
indexTp = "CLUSTERED INDEX"
}
if indexTp != "" {
return dbterror.ErrUniqueKeyNeedAllFieldsInPf.GenWithStackByArgs(indexTp)
if index.Primary && tbInfo.IsCommonHandle {
return dbterror.ErrUniqueKeyNeedAllFieldsInPf.GenWithStackByArgs("CLUSTERED INDEX")
}
if !index.Global {
return dbterror.ErrGlobalIndexNotExplicitlySet.GenWithStackByArgs(index.Name.O)
Expand Down Expand Up @@ -4312,17 +4299,9 @@ func checkPartitioningKeysConstraints(sctx sessionctx.Context, s *ast.CreateTabl
// See https://dev.mysql.com/doc/refman/5.7/en/partitioning-limitations-partitioning-keys-unique-keys.html
for _, index := range tblInfo.Indices {
if index.Unique && !checkUniqueKeyIncludePartKey(partCols, index.Columns) {
if index.Primary {
if index.Primary && tblInfo.IsCommonHandle {
// global index does not support clustered index
if tblInfo.IsCommonHandle {
return dbterror.ErrUniqueKeyNeedAllFieldsInPf.GenWithStackByArgs("CLUSTERED INDEX")
}
if !sctx.GetSessionVars().EnableGlobalIndex {
return dbterror.ErrUniqueKeyNeedAllFieldsInPf.GenWithStackByArgs("PRIMARY KEY")
}
}
if !sctx.GetSessionVars().EnableGlobalIndex {
return dbterror.ErrUniqueKeyNeedAllFieldsInPf.GenWithStackByArgs("UNIQUE INDEX")
return dbterror.ErrUniqueKeyNeedAllFieldsInPf.GenWithStackByArgs("CLUSTERED INDEX")
}
}
}
Expand Down Expand Up @@ -4366,10 +4345,10 @@ func checkPartitionKeysConstraint(pi *model.PartitionInfo, indexColumns []*model
}
}

// In MySQL, every unique key on the table must use every column in the table's partitioning expression.(This
// also includes the table's primary key.)
// In TiDB, global index will be built when this constraint is not satisfied and EnableGlobalIndex is set.
// In MySQL, 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
// TiDB can remove this limitation with Global Index
return checkUniqueKeyIncludePartKey(columnInfoSlice(partCols), indexColumns), nil
}

Expand Down
Loading

0 comments on commit 0442c68

Please sign in to comment.