Skip to content

Commit 63dc364

Browse files
authored
Merge pull request pingcap#5 from Deardrops/4.0-itai-invisible-index
ddl: support invisible index
2 parents 44a44da + fe95944 commit 63dc364

22 files changed

+440
-29
lines changed

bindinfo/bind_test.go

Lines changed: 34 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -503,7 +503,7 @@ func (s *testSuite) TestCapturePlanBaseline(c *C) {
503503
tk.MustExec(" set @@tidb_capture_plan_baselines = off")
504504
}()
505505
tk.MustExec("use test")
506-
tk.MustExec("drop table if exists t")
506+
tk.MustExec("drop table if exists t, t1")
507507
tk.MustExec("create table t(a int)")
508508
s.domain.BindHandle().CaptureBaselines()
509509
tk.MustQuery("show global bindings").Check(testkit.Rows())
@@ -1096,3 +1096,36 @@ func (s *testSuite) TestReCreateBindAfterEvolvePlan(c *C) {
10961096
tk.MustQuery("select * from t where a >= 4 and b >= 1")
10971097
c.Assert(tk.Se.GetSessionVars().StmtCtx.IndexNames[0], Equals, "t:idx_b")
10981098
}
1099+
1100+
func (s *testSuite) TestInvisibleIndex(c *C) {
1101+
tk := testkit.NewTestKit(c, s.store)
1102+
s.cleanBindingEnv(tk)
1103+
tk.MustExec("use test")
1104+
tk.MustExec("drop table if exists t")
1105+
tk.MustExec("create table t(a int, b int, unique idx_a(a), index idx_b(b) invisible)")
1106+
tk.MustGetErrMsg(
1107+
"create global binding for select * from t using select * from t use index(idx_b) ",
1108+
"[planner:1176]Key 'idx_b' doesn't exist in table 't'")
1109+
1110+
// Create bind using index
1111+
tk.MustExec("create global binding for select * from t using select * from t use index(idx_a) ")
1112+
1113+
tk.MustQuery("select * from t")
1114+
c.Assert(tk.Se.GetSessionVars().StmtCtx.IndexNames[0], Equals, "t:idx_a")
1115+
c.Assert(tk.MustUseIndex("select * from t", "idx_a(a)"), IsTrue)
1116+
1117+
tk.MustExec(`prepare stmt1 from 'select * from t'`)
1118+
tk.MustExec("execute stmt1")
1119+
c.Assert(len(tk.Se.GetSessionVars().StmtCtx.IndexNames), Equals, 1)
1120+
c.Assert(tk.Se.GetSessionVars().StmtCtx.IndexNames[0], Equals, "t:idx_a")
1121+
1122+
// And then make this index invisible
1123+
tk.MustExec("alter table t alter index idx_a invisible")
1124+
tk.MustQuery("select * from t")
1125+
c.Assert(len(tk.Se.GetSessionVars().StmtCtx.IndexNames), Equals, 0)
1126+
1127+
tk.MustExec("execute stmt1")
1128+
c.Assert(len(tk.Se.GetSessionVars().StmtCtx.IndexNames), Equals, 0)
1129+
1130+
tk.MustExec("drop binding for select * from t")
1131+
}

bindinfo/handle.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -638,10 +638,10 @@ func (h *BindHandle) CaptureBaselines() {
638638
}
639639

640640
func getHintsForSQL(sctx sessionctx.Context, sql string) (string, error) {
641-
oriVals := sctx.GetSessionVars().UsePlanBaselines
641+
origVals := sctx.GetSessionVars().UsePlanBaselines
642642
sctx.GetSessionVars().UsePlanBaselines = false
643643
recordSets, err := sctx.(sqlexec.SQLExecutor).ExecuteInternal(context.TODO(), fmt.Sprintf("explain format='hint' %s", sql))
644-
sctx.GetSessionVars().UsePlanBaselines = oriVals
644+
sctx.GetSessionVars().UsePlanBaselines = origVals
645645
if len(recordSets) > 0 {
646646
defer terror.Log(recordSets[0].Close())
647647
}

ddl/db_integration_test.go

Lines changed: 43 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2157,3 +2157,46 @@ func (s *testIntegrationSuite3) TestCreateTableWithAutoIdCache(c *C) {
21572157
c.Assert(err, NotNil)
21582158
c.Assert(err.Error(), Equals, "table option auto_id_cache overflows int64")
21592159
}
2160+
2161+
func (s *testIntegrationSuite4) TestAlterIndexVisibility(c *C) {
2162+
config.GetGlobalConfig().Experimental.AllowsExpressionIndex = true
2163+
tk := testkit.NewTestKit(c, s.store)
2164+
tk.MustExec("create database if not exists alter_index_test")
2165+
tk.MustExec("USE alter_index_test;")
2166+
tk.MustExec("drop table if exists t, t1, t2, t3;")
2167+
2168+
tk.MustExec("create table t(a int NOT NULL, b int, key(a), unique(b) invisible)")
2169+
queryIndexOnTable := func(tableName string) string {
2170+
return fmt.Sprintf("select index_name, is_visible from information_schema.statistics where table_schema = 'alter_index_test' and table_name = '%s' order by index_name", tableName)
2171+
}
2172+
query := queryIndexOnTable("t")
2173+
tk.MustQuery(query).Check(testkit.Rows("a YES", "b NO"))
2174+
2175+
tk.MustExec("alter table t alter index a invisible")
2176+
tk.MustQuery(query).Check(testkit.Rows("a NO", "b NO"))
2177+
2178+
tk.MustExec("alter table t alter index b visible")
2179+
tk.MustQuery(query).Check(testkit.Rows("a NO", "b YES"))
2180+
2181+
tk.MustExec("alter table t alter index b invisible")
2182+
tk.MustQuery(query).Check(testkit.Rows("a NO", "b NO"))
2183+
2184+
tk.MustGetErrMsg("alter table t alter index non_exists_idx visible", "[schema:1176]Key 'non_exists_idx' doesn't exist in table 't'")
2185+
2186+
// Alter implicit primary key to invisible index should throw error
2187+
tk.MustExec("create table t1(a int NOT NULL, unique(a))")
2188+
tk.MustGetErrMsg("alter table t1 alter index a invisible", "[ddl:3522]A primary key index cannot be invisible")
2189+
2190+
// Alter explicit primary key to invisible index should throw error
2191+
tk.MustExec("create table t2(a int, primary key(a))")
2192+
tk.MustGetErrMsg("alter table t2 alter index PRIMARY invisible", `[parser:1064]You have an error in your SQL syntax; check the manual that corresponds to your TiDB version for the right syntax to use line 1 column 34 near "PRIMARY invisible" `)
2193+
2194+
// Alter expression index
2195+
tk.MustExec("create table t3(a int NOT NULL, b int)")
2196+
tk.MustExec("alter table t3 add index idx((a+b));")
2197+
query = queryIndexOnTable("t3")
2198+
tk.MustQuery(query).Check(testkit.Rows("idx YES"))
2199+
2200+
tk.MustExec("alter table t3 alter index idx invisible")
2201+
tk.MustQuery(query).Check(testkit.Rows("idx NO"))
2202+
}

ddl/ddl_api.go

Lines changed: 84 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1131,6 +1131,47 @@ func checkConstraintNames(constraints []*ast.Constraint) error {
11311131
return nil
11321132
}
11331133

1134+
// checkInvisibleIndexOnPK check if primary key is invisible index.
1135+
func checkInvisibleIndexOnPK(tblInfo *model.TableInfo) error {
1136+
pk := getPrimaryKey(tblInfo)
1137+
if pk != nil && pk.Invisible {
1138+
return ErrPKIndexCantBeInvisible
1139+
}
1140+
return nil
1141+
}
1142+
1143+
// getPrimaryKey extract the primary key in a table and return `IndexInfo`
1144+
// The returned primary key could be explicit or implicit.
1145+
// If there is no explicit primary key in table,
1146+
// the first UNIQUE INDEX on NOT NULL columns will be the implicit primary key.
1147+
// For more information about implicit primary key, see
1148+
// https://dev.mysql.com/doc/refman/8.0/en/invisible-indexes.html
1149+
func getPrimaryKey(tblInfo *model.TableInfo) *model.IndexInfo {
1150+
var implicitPK *model.IndexInfo
1151+
1152+
for _, key := range tblInfo.Indices {
1153+
if key.Primary {
1154+
// table has explicit primary key
1155+
return key
1156+
}
1157+
// find the first unique key with NOT NULL columns
1158+
if implicitPK == nil && key.Unique {
1159+
// ensure all columns in unique key have NOT NULL flag
1160+
allColNotNull := true
1161+
for _, idxCol := range key.Columns {
1162+
col := model.FindColumnInfo(tblInfo.Cols(), idxCol.Name.L)
1163+
if !mysql.HasNotNullFlag(col.Flag) {
1164+
allColNotNull = false
1165+
}
1166+
}
1167+
if allColNotNull {
1168+
implicitPK = key
1169+
}
1170+
}
1171+
}
1172+
return implicitPK
1173+
}
1174+
11341175
func setTableAutoRandomBits(ctx sessionctx.Context, tbInfo *model.TableInfo, colDefs []*ast.ColumnDef) error {
11351176
allowAutoRandom := config.GetGlobalConfig().Experimental.AllowAutoRandom
11361177
pkColName := tbInfo.GetPkName()
@@ -1350,7 +1391,10 @@ func checkTableInfoValidWithStmt(ctx sessionctx.Context, tbInfo *model.TableInfo
13501391
// checkTableInfoValid uses to check table info valid. This is used to validate table info.
13511392
func checkTableInfoValid(tblInfo *model.TableInfo) error {
13521393
_, err := tables.TableFromMeta(nil, tblInfo)
1353-
return err
1394+
if err != nil {
1395+
return err
1396+
}
1397+
return checkInvisibleIndexOnPK(tblInfo)
13541398
}
13551399

13561400
func buildTableInfoWithLike(ident ast.Ident, referTblInfo *model.TableInfo) (*model.TableInfo, error) {
@@ -1454,6 +1498,10 @@ func buildTableInfoWithStmt(ctx sessionctx.Context, s *ast.CreateTableStmt, dbCh
14541498
return nil, errors.Trace(err)
14551499
}
14561500

1501+
if err = checkInvisibleIndexOnPK(tbInfo); err != nil {
1502+
return nil, errors.Trace(err)
1503+
}
1504+
14571505
return tbInfo, nil
14581506
}
14591507

@@ -2206,6 +2254,8 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A
22062254
err = d.AlterTableSetTiFlashReplica(ctx, ident, spec.TiFlashReplica)
22072255
case ast.AlterTableOrderByColumns:
22082256
err = d.OrderByColumns(ctx, ident)
2257+
case ast.AlterTableIndexInvisible:
2258+
err = d.AlterIndexVisibility(ctx, ident, spec.IndexName, spec.Visibility)
22092259
default:
22102260
// Nothing to do now.
22112261
}
@@ -4555,3 +4605,36 @@ func (d *ddl) DropSequence(ctx sessionctx.Context, ti ast.Ident, ifExists bool)
45554605
err = d.callHookOnChanged(err)
45564606
return errors.Trace(err)
45574607
}
4608+
4609+
func (d *ddl) AlterIndexVisibility(ctx sessionctx.Context, ident ast.Ident, indexName model.CIStr, visibility ast.IndexVisibility) error {
4610+
schema, tb, err := d.getSchemaAndTableByIdent(ctx, ident)
4611+
if err != nil {
4612+
return err
4613+
}
4614+
4615+
invisible := false
4616+
if visibility == ast.IndexVisibilityInvisible {
4617+
invisible = true
4618+
}
4619+
4620+
skip, err := validateAlterIndexVisibility(indexName, invisible, tb.Meta())
4621+
if err != nil {
4622+
return errors.Trace(err)
4623+
}
4624+
if skip {
4625+
return nil
4626+
}
4627+
4628+
job := &model.Job{
4629+
SchemaID: schema.ID,
4630+
TableID: tb.Meta().ID,
4631+
SchemaName: schema.Name.L,
4632+
Type: model.ActionAlterIndexVisibility,
4633+
BinlogInfo: &model.HistoryInfo{},
4634+
Args: []interface{}{indexName, invisible},
4635+
}
4636+
4637+
err = d.doDDLJob(ctx, job)
4638+
err = d.callHookOnChanged(err)
4639+
return errors.Trace(err)
4640+
}

ddl/ddl_worker.go

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -653,6 +653,8 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64,
653653
ver, err = onUpdateFlashReplicaStatus(t, job)
654654
case model.ActionCreateSequence:
655655
ver, err = onCreateSequence(d, t, job)
656+
case model.ActionAlterIndexVisibility:
657+
ver, err = onAlterIndexVisibility(t, job)
656658
default:
657659
// Invalid job, cancel it.
658660
job.State = model.JobStateCancelled

ddl/ddl_worker_test.go

Lines changed: 39 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -28,6 +28,7 @@ import (
2828
"github.com/pingcap/tidb/kv"
2929
"github.com/pingcap/tidb/meta"
3030
"github.com/pingcap/tidb/sessionctx"
31+
"github.com/pingcap/tidb/table"
3132
"github.com/pingcap/tidb/types"
3233
"github.com/pingcap/tidb/util/admin"
3334
"github.com/pingcap/tidb/util/mock"
@@ -452,6 +453,9 @@ func buildCancelJobTests(firstID int64) []testCancelJob {
452453
{act: model.ActionDropPrimaryKey, jobIDs: []int64{firstID + 37}, cancelRetErrs: []error{admin.ErrCannotCancelDDLJob.GenWithStackByArgs(firstID + 37)}, cancelState: model.StateDeleteOnly},
453454

454455
{act: model.ActionRenameDatabase, jobIDs: []int64{firstID + 38}, cancelRetErrs: noErrs, cancelState: model.StateNone},
456+
457+
{act: model.ActionAlterIndexVisibility, jobIDs: []int64{firstID + 40}, cancelRetErrs: noErrs, cancelState: model.StateNone},
458+
{act: model.ActionAlterIndexVisibility, jobIDs: []int64{firstID + 41}, cancelRetErrs: []error{admin.ErrCancelFinishedDDLJob.GenWithStackByArgs(firstID + 47)}, cancelState: model.StatePublic},
455459
}
456460

457461
return tests
@@ -501,7 +505,16 @@ func (s *testDDLSuite) checkCancelDropColumn(c *C, d *ddl, schemaID int64, table
501505
c.Assert(notFound, Equals, success)
502506
}
503507

504-
func (s *testDDLSuite) TestCancelJob(c *C) {
508+
func checkIdxVisibility(changedTable table.Table, idxName string, expected bool) bool {
509+
for _, idxInfo := range changedTable.Meta().Indices {
510+
if idxInfo.Name.O == idxName && idxInfo.Invisible == expected {
511+
return true
512+
}
513+
}
514+
return false
515+
}
516+
517+
func (s *testDDLSuite) TestCancelJob1(c *C) {
505518
store := testCreateStore(c, "test_cancel_job")
506519
defer store.Close()
507520
d := newDDL(
@@ -855,10 +868,34 @@ func (s *testDDLSuite) TestCancelJob(c *C) {
855868
s.checkDropIdx(c, d, dbInfo.ID, tblInfo.ID, idxOrigName, true)
856869

857870
// for rename database
858-
updateTest(&tests[33])
871+
updateTest(&tests[34])
859872
doDDLJobErrWithSchemaState(ctx, d, c, dbInfo1.ID, 0, model.ActionRenameDatabase, []interface{}{"newDB"}, &cancelState)
860873
c.Check(checkErr, IsNil)
861874
testCheckSchemaState(c, d, dbInfo, model.StatePublic)
875+
876+
// test alter index visibility failed caused by canceled.
877+
indexName := "idx_c3"
878+
testCreateIndex(c, ctx, d, dbInfo, tblInfo, false, indexName, "c3")
879+
c.Check(errors.ErrorStack(checkErr), Equals, "")
880+
txn, err = ctx.Txn(true)
881+
c.Assert(err, IsNil)
882+
c.Assert(txn.Commit(context.Background()), IsNil)
883+
s.checkAddIdx(c, d, dbInfo.ID, tblInfo.ID, indexName, true)
884+
885+
updateTest(&tests[35])
886+
alterIndexVisibility := []interface{}{model.NewCIStr(indexName), true}
887+
doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, test.act, alterIndexVisibility, &test.cancelState)
888+
c.Check(checkErr, IsNil)
889+
changedTable = testGetTable(c, d, dbInfo.ID, tblInfo.ID)
890+
c.Assert(checkIdxVisibility(changedTable, indexName, false), IsTrue)
891+
892+
// cancel alter index visibility successfully
893+
updateTest(&tests[36])
894+
alterIndexVisibility = []interface{}{model.NewCIStr(indexName), true}
895+
doDDLJobSuccess(ctx, d, c, dbInfo.ID, tblInfo.ID, test.act, alterIndexVisibility)
896+
c.Check(checkErr, IsNil)
897+
changedTable = testGetTable(c, d, dbInfo.ID, tblInfo.ID)
898+
c.Assert(checkIdxVisibility(changedTable, indexName, true), IsTrue)
862899
}
863900

864901
func (s *testDDLSuite) TestIgnorableSpec(c *C) {

ddl/error.go

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -98,6 +98,8 @@ var (
9898
// ErrUnsupportedModifyPrimaryKey returns an error when add or drop the primary key.
9999
// It's exported for testing.
100100
ErrUnsupportedModifyPrimaryKey = terror.ClassDDL.New(mysql.ErrUnsupportedDDLOperation, fmt.Sprintf(mysql.MySQLErrName[mysql.ErrUnsupportedDDLOperation], "%s primary key"))
101+
// ErrPKIndexCantBeInvisible return an error when primary key is invisible index
102+
ErrPKIndexCantBeInvisible = terror.ClassDDL.New(mysql.ErrPKIndexCantBeInvisible, mysql.MySQLErrName[mysql.ErrPKIndexCantBeInvisible])
101103

102104
// ErrColumnBadNull returns for a bad null value.
103105
ErrColumnBadNull = terror.ClassDDL.New(mysql.ErrBadNull, mysql.MySQLErrName[mysql.ErrBadNull])

ddl/index.go

Lines changed: 54 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -296,7 +296,7 @@ func validateRenameIndex(from, to model.CIStr, tbl *model.TableInfo) (ignore boo
296296

297297
func onRenameIndex(t *meta.Meta, job *model.Job) (ver int64, _ error) {
298298
tblInfo, from, to, err := checkRenameIndex(t, job)
299-
if err != nil {
299+
if err != nil || tblInfo == nil {
300300
return ver, errors.Trace(err)
301301
}
302302

@@ -310,6 +310,30 @@ func onRenameIndex(t *meta.Meta, job *model.Job) (ver int64, _ error) {
310310
return ver, nil
311311
}
312312

313+
func validateAlterIndexVisibility(indexName model.CIStr, invisible bool, tbl *model.TableInfo) (bool, error) {
314+
if idx := tbl.FindIndexByName(indexName.L); idx == nil {
315+
return false, errors.Trace(infoschema.ErrKeyNotExists.GenWithStackByArgs(indexName.O, tbl.Name))
316+
} else if idx.Invisible == invisible {
317+
return true, nil
318+
}
319+
return false, nil
320+
}
321+
322+
func onAlterIndexVisibility(t *meta.Meta, job *model.Job) (ver int64, _ error) {
323+
tblInfo, from, invisible, err := checkAlterIndexVisibility(t, job)
324+
if err != nil || tblInfo == nil {
325+
return ver, errors.Trace(err)
326+
}
327+
idx := tblInfo.FindIndexByName(from.L)
328+
idx.Invisible = invisible
329+
if ver, err = updateVersionAndTableInfoWithCheck(t, job, tblInfo, true); err != nil {
330+
job.State = model.JobStateCancelled
331+
return ver, errors.Trace(err)
332+
}
333+
job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo)
334+
return ver, nil
335+
}
336+
313337
func getNullColInfos(tblInfo *model.TableInfo, indexInfo *model.IndexInfo) ([]*model.ColumnInfo, error) {
314338
nullCols := make([]*model.ColumnInfo, 0, len(indexInfo.Columns))
315339
for _, colName := range indexInfo.Columns {
@@ -610,7 +634,7 @@ func onDropIndex(t *meta.Meta, job *model.Job) (ver int64, _ error) {
610634

611635
tblInfo.Columns = tblInfo.Columns[:len(tblInfo.Columns)-len(dependentHiddenCols)]
612636

613-
ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != model.StateNone)
637+
ver, err = updateVersionAndTableInfoWithCheck(t, job, tblInfo, originalState != model.StateNone)
614638
if err != nil {
615639
return ver, errors.Trace(err)
616640
}
@@ -712,6 +736,34 @@ func checkRenameIndex(t *meta.Meta, job *model.Job) (*model.TableInfo, model.CIS
712736
return tblInfo, from, to, errors.Trace(err)
713737
}
714738

739+
func checkAlterIndexVisibility(t *meta.Meta, job *model.Job) (*model.TableInfo, model.CIStr, bool, error) {
740+
var (
741+
indexName model.CIStr
742+
invisible bool
743+
)
744+
745+
schemaID := job.SchemaID
746+
tblInfo, err := getTableInfoAndCancelFaultJob(t, job, schemaID)
747+
if err != nil {
748+
return nil, indexName, invisible, errors.Trace(err)
749+
}
750+
751+
if err := job.DecodeArgs(&indexName, &invisible); err != nil {
752+
job.State = model.JobStateCancelled
753+
return nil, indexName, invisible, errors.Trace(err)
754+
}
755+
756+
skip, err := validateAlterIndexVisibility(indexName, invisible, tblInfo)
757+
if err != nil {
758+
job.State = model.JobStateCancelled
759+
return nil, indexName, invisible, errors.Trace(err)
760+
}
761+
if skip {
762+
return nil, indexName, invisible, nil
763+
}
764+
return tblInfo, indexName, invisible, nil
765+
}
766+
715767
const (
716768
// DefaultTaskHandleCnt is default batch size of adding indices.
717769
DefaultTaskHandleCnt = 128

0 commit comments

Comments
 (0)