Skip to content

Commit

Permalink
Merge pull request #5 from Deardrops/4.0-itai-invisible-index
Browse files Browse the repository at this point in the history
ddl: support invisible index
  • Loading branch information
bb7133 authored May 9, 2020
2 parents 44a44da + fe95944 commit 63dc364
Show file tree
Hide file tree
Showing 22 changed files with 440 additions and 29 deletions.
35 changes: 34 additions & 1 deletion bindinfo/bind_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -503,7 +503,7 @@ func (s *testSuite) TestCapturePlanBaseline(c *C) {
tk.MustExec(" set @@tidb_capture_plan_baselines = off")
}()
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("drop table if exists t, t1")
tk.MustExec("create table t(a int)")
s.domain.BindHandle().CaptureBaselines()
tk.MustQuery("show global bindings").Check(testkit.Rows())
Expand Down Expand Up @@ -1096,3 +1096,36 @@ func (s *testSuite) TestReCreateBindAfterEvolvePlan(c *C) {
tk.MustQuery("select * from t where a >= 4 and b >= 1")
c.Assert(tk.Se.GetSessionVars().StmtCtx.IndexNames[0], Equals, "t:idx_b")
}

func (s *testSuite) TestInvisibleIndex(c *C) {
tk := testkit.NewTestKit(c, s.store)
s.cleanBindingEnv(tk)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, b int, unique idx_a(a), index idx_b(b) invisible)")
tk.MustGetErrMsg(
"create global binding for select * from t using select * from t use index(idx_b) ",
"[planner:1176]Key 'idx_b' doesn't exist in table 't'")

// Create bind using index
tk.MustExec("create global binding for select * from t using select * from t use index(idx_a) ")

tk.MustQuery("select * from t")
c.Assert(tk.Se.GetSessionVars().StmtCtx.IndexNames[0], Equals, "t:idx_a")
c.Assert(tk.MustUseIndex("select * from t", "idx_a(a)"), IsTrue)

tk.MustExec(`prepare stmt1 from 'select * from t'`)
tk.MustExec("execute stmt1")
c.Assert(len(tk.Se.GetSessionVars().StmtCtx.IndexNames), Equals, 1)
c.Assert(tk.Se.GetSessionVars().StmtCtx.IndexNames[0], Equals, "t:idx_a")

// And then make this index invisible
tk.MustExec("alter table t alter index idx_a invisible")
tk.MustQuery("select * from t")
c.Assert(len(tk.Se.GetSessionVars().StmtCtx.IndexNames), Equals, 0)

tk.MustExec("execute stmt1")
c.Assert(len(tk.Se.GetSessionVars().StmtCtx.IndexNames), Equals, 0)

tk.MustExec("drop binding for select * from t")
}
4 changes: 2 additions & 2 deletions bindinfo/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -638,10 +638,10 @@ func (h *BindHandle) CaptureBaselines() {
}

func getHintsForSQL(sctx sessionctx.Context, sql string) (string, error) {
oriVals := sctx.GetSessionVars().UsePlanBaselines
origVals := sctx.GetSessionVars().UsePlanBaselines
sctx.GetSessionVars().UsePlanBaselines = false
recordSets, err := sctx.(sqlexec.SQLExecutor).ExecuteInternal(context.TODO(), fmt.Sprintf("explain format='hint' %s", sql))
sctx.GetSessionVars().UsePlanBaselines = oriVals
sctx.GetSessionVars().UsePlanBaselines = origVals
if len(recordSets) > 0 {
defer terror.Log(recordSets[0].Close())
}
Expand Down
43 changes: 43 additions & 0 deletions ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2157,3 +2157,46 @@ func (s *testIntegrationSuite3) TestCreateTableWithAutoIdCache(c *C) {
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "table option auto_id_cache overflows int64")
}

func (s *testIntegrationSuite4) TestAlterIndexVisibility(c *C) {
config.GetGlobalConfig().Experimental.AllowsExpressionIndex = true
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("create database if not exists alter_index_test")
tk.MustExec("USE alter_index_test;")
tk.MustExec("drop table if exists t, t1, t2, t3;")

tk.MustExec("create table t(a int NOT NULL, b int, key(a), unique(b) invisible)")
queryIndexOnTable := func(tableName string) string {
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)
}
query := queryIndexOnTable("t")
tk.MustQuery(query).Check(testkit.Rows("a YES", "b NO"))

tk.MustExec("alter table t alter index a invisible")
tk.MustQuery(query).Check(testkit.Rows("a NO", "b NO"))

tk.MustExec("alter table t alter index b visible")
tk.MustQuery(query).Check(testkit.Rows("a NO", "b YES"))

tk.MustExec("alter table t alter index b invisible")
tk.MustQuery(query).Check(testkit.Rows("a NO", "b NO"))

tk.MustGetErrMsg("alter table t alter index non_exists_idx visible", "[schema:1176]Key 'non_exists_idx' doesn't exist in table 't'")

// Alter implicit primary key to invisible index should throw error
tk.MustExec("create table t1(a int NOT NULL, unique(a))")
tk.MustGetErrMsg("alter table t1 alter index a invisible", "[ddl:3522]A primary key index cannot be invisible")

// Alter explicit primary key to invisible index should throw error
tk.MustExec("create table t2(a int, primary key(a))")
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" `)

// Alter expression index
tk.MustExec("create table t3(a int NOT NULL, b int)")
tk.MustExec("alter table t3 add index idx((a+b));")
query = queryIndexOnTable("t3")
tk.MustQuery(query).Check(testkit.Rows("idx YES"))

tk.MustExec("alter table t3 alter index idx invisible")
tk.MustQuery(query).Check(testkit.Rows("idx NO"))
}
85 changes: 84 additions & 1 deletion ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1131,6 +1131,47 @@ func checkConstraintNames(constraints []*ast.Constraint) error {
return nil
}

// checkInvisibleIndexOnPK check if primary key is invisible index.
func checkInvisibleIndexOnPK(tblInfo *model.TableInfo) error {
pk := getPrimaryKey(tblInfo)
if pk != nil && pk.Invisible {
return ErrPKIndexCantBeInvisible
}
return nil
}

// getPrimaryKey extract the primary key in a table and return `IndexInfo`
// The returned primary key could be explicit or implicit.
// If there is no explicit primary key in table,
// the first UNIQUE INDEX on NOT NULL columns will be the implicit primary key.
// For more information about implicit primary key, see
// https://dev.mysql.com/doc/refman/8.0/en/invisible-indexes.html
func getPrimaryKey(tblInfo *model.TableInfo) *model.IndexInfo {
var implicitPK *model.IndexInfo

for _, key := range tblInfo.Indices {
if key.Primary {
// table has explicit primary key
return key
}
// find the first unique key with NOT NULL columns
if implicitPK == nil && key.Unique {
// ensure all columns in unique key have NOT NULL flag
allColNotNull := true
for _, idxCol := range key.Columns {
col := model.FindColumnInfo(tblInfo.Cols(), idxCol.Name.L)
if !mysql.HasNotNullFlag(col.Flag) {
allColNotNull = false
}
}
if allColNotNull {
implicitPK = key
}
}
}
return implicitPK
}

func setTableAutoRandomBits(ctx sessionctx.Context, tbInfo *model.TableInfo, colDefs []*ast.ColumnDef) error {
allowAutoRandom := config.GetGlobalConfig().Experimental.AllowAutoRandom
pkColName := tbInfo.GetPkName()
Expand Down Expand Up @@ -1350,7 +1391,10 @@ func checkTableInfoValidWithStmt(ctx sessionctx.Context, tbInfo *model.TableInfo
// checkTableInfoValid uses to check table info valid. This is used to validate table info.
func checkTableInfoValid(tblInfo *model.TableInfo) error {
_, err := tables.TableFromMeta(nil, tblInfo)
return err
if err != nil {
return err
}
return checkInvisibleIndexOnPK(tblInfo)
}

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

if err = checkInvisibleIndexOnPK(tbInfo); err != nil {
return nil, errors.Trace(err)
}

return tbInfo, nil
}

Expand Down Expand Up @@ -2206,6 +2254,8 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A
err = d.AlterTableSetTiFlashReplica(ctx, ident, spec.TiFlashReplica)
case ast.AlterTableOrderByColumns:
err = d.OrderByColumns(ctx, ident)
case ast.AlterTableIndexInvisible:
err = d.AlterIndexVisibility(ctx, ident, spec.IndexName, spec.Visibility)
default:
// Nothing to do now.
}
Expand Down Expand Up @@ -4555,3 +4605,36 @@ func (d *ddl) DropSequence(ctx sessionctx.Context, ti ast.Ident, ifExists bool)
err = d.callHookOnChanged(err)
return errors.Trace(err)
}

func (d *ddl) AlterIndexVisibility(ctx sessionctx.Context, ident ast.Ident, indexName model.CIStr, visibility ast.IndexVisibility) error {
schema, tb, err := d.getSchemaAndTableByIdent(ctx, ident)
if err != nil {
return err
}

invisible := false
if visibility == ast.IndexVisibilityInvisible {
invisible = true
}

skip, err := validateAlterIndexVisibility(indexName, invisible, tb.Meta())
if err != nil {
return errors.Trace(err)
}
if skip {
return nil
}

job := &model.Job{
SchemaID: schema.ID,
TableID: tb.Meta().ID,
SchemaName: schema.Name.L,
Type: model.ActionAlterIndexVisibility,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{indexName, invisible},
}

err = d.doDDLJob(ctx, job)
err = d.callHookOnChanged(err)
return errors.Trace(err)
}
2 changes: 2 additions & 0 deletions ddl/ddl_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -653,6 +653,8 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64,
ver, err = onUpdateFlashReplicaStatus(t, job)
case model.ActionCreateSequence:
ver, err = onCreateSequence(d, t, job)
case model.ActionAlterIndexVisibility:
ver, err = onAlterIndexVisibility(t, job)
default:
// Invalid job, cancel it.
job.State = model.JobStateCancelled
Expand Down
41 changes: 39 additions & 2 deletions ddl/ddl_worker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ import (
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/admin"
"github.com/pingcap/tidb/util/mock"
Expand Down Expand Up @@ -452,6 +453,9 @@ func buildCancelJobTests(firstID int64) []testCancelJob {
{act: model.ActionDropPrimaryKey, jobIDs: []int64{firstID + 37}, cancelRetErrs: []error{admin.ErrCannotCancelDDLJob.GenWithStackByArgs(firstID + 37)}, cancelState: model.StateDeleteOnly},

{act: model.ActionRenameDatabase, jobIDs: []int64{firstID + 38}, cancelRetErrs: noErrs, cancelState: model.StateNone},

{act: model.ActionAlterIndexVisibility, jobIDs: []int64{firstID + 40}, cancelRetErrs: noErrs, cancelState: model.StateNone},
{act: model.ActionAlterIndexVisibility, jobIDs: []int64{firstID + 41}, cancelRetErrs: []error{admin.ErrCancelFinishedDDLJob.GenWithStackByArgs(firstID + 47)}, cancelState: model.StatePublic},
}

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

func (s *testDDLSuite) TestCancelJob(c *C) {
func checkIdxVisibility(changedTable table.Table, idxName string, expected bool) bool {
for _, idxInfo := range changedTable.Meta().Indices {
if idxInfo.Name.O == idxName && idxInfo.Invisible == expected {
return true
}
}
return false
}

func (s *testDDLSuite) TestCancelJob1(c *C) {
store := testCreateStore(c, "test_cancel_job")
defer store.Close()
d := newDDL(
Expand Down Expand Up @@ -855,10 +868,34 @@ func (s *testDDLSuite) TestCancelJob(c *C) {
s.checkDropIdx(c, d, dbInfo.ID, tblInfo.ID, idxOrigName, true)

// for rename database
updateTest(&tests[33])
updateTest(&tests[34])
doDDLJobErrWithSchemaState(ctx, d, c, dbInfo1.ID, 0, model.ActionRenameDatabase, []interface{}{"newDB"}, &cancelState)
c.Check(checkErr, IsNil)
testCheckSchemaState(c, d, dbInfo, model.StatePublic)

// test alter index visibility failed caused by canceled.
indexName := "idx_c3"
testCreateIndex(c, ctx, d, dbInfo, tblInfo, false, indexName, "c3")
c.Check(errors.ErrorStack(checkErr), Equals, "")
txn, err = ctx.Txn(true)
c.Assert(err, IsNil)
c.Assert(txn.Commit(context.Background()), IsNil)
s.checkAddIdx(c, d, dbInfo.ID, tblInfo.ID, indexName, true)

updateTest(&tests[35])
alterIndexVisibility := []interface{}{model.NewCIStr(indexName), true}
doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, test.act, alterIndexVisibility, &test.cancelState)
c.Check(checkErr, IsNil)
changedTable = testGetTable(c, d, dbInfo.ID, tblInfo.ID)
c.Assert(checkIdxVisibility(changedTable, indexName, false), IsTrue)

// cancel alter index visibility successfully
updateTest(&tests[36])
alterIndexVisibility = []interface{}{model.NewCIStr(indexName), true}
doDDLJobSuccess(ctx, d, c, dbInfo.ID, tblInfo.ID, test.act, alterIndexVisibility)
c.Check(checkErr, IsNil)
changedTable = testGetTable(c, d, dbInfo.ID, tblInfo.ID)
c.Assert(checkIdxVisibility(changedTable, indexName, true), IsTrue)
}

func (s *testDDLSuite) TestIgnorableSpec(c *C) {
Expand Down
2 changes: 2 additions & 0 deletions ddl/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -98,6 +98,8 @@ var (
// ErrUnsupportedModifyPrimaryKey returns an error when add or drop the primary key.
// It's exported for testing.
ErrUnsupportedModifyPrimaryKey = terror.ClassDDL.New(mysql.ErrUnsupportedDDLOperation, fmt.Sprintf(mysql.MySQLErrName[mysql.ErrUnsupportedDDLOperation], "%s primary key"))
// ErrPKIndexCantBeInvisible return an error when primary key is invisible index
ErrPKIndexCantBeInvisible = terror.ClassDDL.New(mysql.ErrPKIndexCantBeInvisible, mysql.MySQLErrName[mysql.ErrPKIndexCantBeInvisible])

// ErrColumnBadNull returns for a bad null value.
ErrColumnBadNull = terror.ClassDDL.New(mysql.ErrBadNull, mysql.MySQLErrName[mysql.ErrBadNull])
Expand Down
56 changes: 54 additions & 2 deletions ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -296,7 +296,7 @@ func validateRenameIndex(from, to model.CIStr, tbl *model.TableInfo) (ignore boo

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

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

func validateAlterIndexVisibility(indexName model.CIStr, invisible bool, tbl *model.TableInfo) (bool, error) {
if idx := tbl.FindIndexByName(indexName.L); idx == nil {
return false, errors.Trace(infoschema.ErrKeyNotExists.GenWithStackByArgs(indexName.O, tbl.Name))
} else if idx.Invisible == invisible {
return true, nil
}
return false, nil
}

func onAlterIndexVisibility(t *meta.Meta, job *model.Job) (ver int64, _ error) {
tblInfo, from, invisible, err := checkAlterIndexVisibility(t, job)
if err != nil || tblInfo == nil {
return ver, errors.Trace(err)
}
idx := tblInfo.FindIndexByName(from.L)
idx.Invisible = invisible
if ver, err = updateVersionAndTableInfoWithCheck(t, job, tblInfo, true); err != nil {
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
}
job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo)
return ver, nil
}

func getNullColInfos(tblInfo *model.TableInfo, indexInfo *model.IndexInfo) ([]*model.ColumnInfo, error) {
nullCols := make([]*model.ColumnInfo, 0, len(indexInfo.Columns))
for _, colName := range indexInfo.Columns {
Expand Down Expand Up @@ -610,7 +634,7 @@ func onDropIndex(t *meta.Meta, job *model.Job) (ver int64, _ error) {

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

ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != model.StateNone)
ver, err = updateVersionAndTableInfoWithCheck(t, job, tblInfo, originalState != model.StateNone)
if err != nil {
return ver, errors.Trace(err)
}
Expand Down Expand Up @@ -712,6 +736,34 @@ func checkRenameIndex(t *meta.Meta, job *model.Job) (*model.TableInfo, model.CIS
return tblInfo, from, to, errors.Trace(err)
}

func checkAlterIndexVisibility(t *meta.Meta, job *model.Job) (*model.TableInfo, model.CIStr, bool, error) {
var (
indexName model.CIStr
invisible bool
)

schemaID := job.SchemaID
tblInfo, err := getTableInfoAndCancelFaultJob(t, job, schemaID)
if err != nil {
return nil, indexName, invisible, errors.Trace(err)
}

if err := job.DecodeArgs(&indexName, &invisible); err != nil {
job.State = model.JobStateCancelled
return nil, indexName, invisible, errors.Trace(err)
}

skip, err := validateAlterIndexVisibility(indexName, invisible, tblInfo)
if err != nil {
job.State = model.JobStateCancelled
return nil, indexName, invisible, errors.Trace(err)
}
if skip {
return nil, indexName, invisible, nil
}
return tblInfo, indexName, invisible, nil
}

const (
// DefaultTaskHandleCnt is default batch size of adding indices.
DefaultTaskHandleCnt = 128
Expand Down
Loading

0 comments on commit 63dc364

Please sign in to comment.