Skip to content

Commit

Permalink
plannercore: ignore invisible indexes in optimizer (pingcap#15421)
Browse files Browse the repository at this point in the history
  • Loading branch information
Deardrops authored May 7, 2020
1 parent d367a01 commit 26e946d
Show file tree
Hide file tree
Showing 16 changed files with 228 additions and 8 deletions.
34 changes: 33 additions & 1 deletion bindinfo/bind_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -544,7 +544,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 @@ -1137,3 +1137,35 @@ 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'")
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")

// TODO: Add these test
//tk.MustExec("alter table t alter index idx_a invisible")
//tk.MustQuery("select * from t where a > 3")
//c.Assert(tk.Se.GetSessionVars().StmtCtx.IndexNames[0], Equals, "t:idx_a")
//c.Assert(tk.MustUseIndex("select * from t where a > 3", "idx_a(a)"), IsTrue)
//
//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
50 changes: 49 additions & 1 deletion ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1104,6 +1104,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 @@ -1323,7 +1364,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 @@ -1427,6 +1471,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
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
2 changes: 1 addition & 1 deletion ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -610,7 +610,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
86 changes: 86 additions & 0 deletions ddl/serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/pingcap/tidb/ddl"
ddlutil "github.com/pingcap/tidb/ddl/util"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta"
Expand Down Expand Up @@ -1012,3 +1013,88 @@ func (s *testSerialSuite) TestForbidUnsupportedCollations(c *C) {
// mustGetUnsupportedCollation("create database ucd collate utf8mb4_unicode_ci", errMsgUnsupportedUnicodeCI)
// mustGetUnsupportedCollation("alter table t convert to collate utf8mb4_unicode_ci", "utf8mb4_unicode_ci")
}

func (s *testSerialSuite) TestInvisibleIndex(c *C) {
tk := testkit.NewTestKit(c, s.store)

tk.MustExec("use test")
tk.MustExec("drop table if exists t,t1,t2,t3,t4,t5,t6")

// The DDL statement related to invisible index.
showIndexes := "select index_name, is_visible from information_schema.statistics where table_schema = 'test' and table_name = 't'"
// 1. Create table with invisible index
tk.MustExec("create table t (a int, b int, unique (a) invisible)")
tk.MustQuery(showIndexes).Check(testkit.Rows("a NO"))
tk.MustExec("insert into t values (1, 2)")
tk.MustQuery("select * from t").Check(testkit.Rows("1 2"))
// 2. Drop invisible index
tk.MustGetErrMsg("alter table t drop column a", "[ddl:8200]can't drop column a with index covered now")
tk.MustExec("alter table t drop index a")
tk.MustQuery(showIndexes).Check(testkit.Rows())
tk.MustExec("insert into t values (3, 4)")
tk.MustQuery("select * from t").Check(testkit.Rows("1 2", "3 4"))
// 3. Add an invisible index
tk.MustExec("alter table t add index (b) invisible")
tk.MustQuery(showIndexes).Check(testkit.Rows("b NO"))
tk.MustExec("insert into t values (5, 6)")
tk.MustQuery("select * from t").Check(testkit.Rows("1 2", "3 4", "5 6"))
// 4. Drop it
tk.MustExec("alter table t drop index b")
tk.MustQuery(showIndexes).Check(testkit.Rows())
tk.MustExec("insert into t values (7, 8)")
tk.MustQuery("select * from t").Check(testkit.Rows("1 2", "3 4", "5 6", "7 8"))
// 5. Create a multiple-column invisible index
tk.MustExec("alter table t add index a_b(a, b) invisible")
tk.MustQuery(showIndexes).Check(testkit.Rows("a_b NO", "a_b NO"))
tk.MustExec("insert into t values (9, 10)")
tk.MustQuery("select * from t").Check(testkit.Rows("1 2", "3 4", "5 6", "7 8", "9 10"))
// 6. Drop it
tk.MustExec("alter table t drop index a_b")
tk.MustQuery(showIndexes).Check(testkit.Rows())
tk.MustExec("insert into t values (11, 12)")
tk.MustQuery("select * from t").Check(testkit.Rows("1 2", "3 4", "5 6", "7 8", "9 10", "11 12"))

cfg := config.GetGlobalConfig()
newCfg := *cfg
orignalAlterPrimaryKey := newCfg.AlterPrimaryKey
newCfg.AlterPrimaryKey = true
config.StoreGlobalConfig(&newCfg)
defer func() {
newCfg.AlterPrimaryKey = orignalAlterPrimaryKey
config.StoreGlobalConfig(&newCfg)
}()

// Limitation: Primary key cannot be invisible index
tk.MustGetErrCode("create table t1 (a int, primary key (a) invisible)", errno.ErrPKIndexCantBeInvisible)
tk.MustGetErrCode("create table t1 (a int, b int, primary key (a, b) invisible)", errno.ErrPKIndexCantBeInvisible)
tk.MustExec("create table t1 (a int, b int)")
tk.MustGetErrCode("alter table t1 add primary key(a) invisible", errno.ErrPKIndexCantBeInvisible)
tk.MustGetErrCode("alter table t1 add primary key(a, b) invisible", errno.ErrPKIndexCantBeInvisible)

// Implicit primary key cannot be invisible index
// Create a implicit primary key
tk.MustGetErrCode("create table t2(a int not null, unique (a) invisible)", errno.ErrPKIndexCantBeInvisible)
// Column `a` become implicit primary key after DDL statement on itself
tk.MustExec("create table t2(a int not null)")
tk.MustGetErrCode("alter table t2 add unique (a) invisible", errno.ErrPKIndexCantBeInvisible)
tk.MustExec("create table t3(a int, unique index (a) invisible)")
tk.MustGetErrCode("alter table t3 modify column a int not null", errno.ErrPKIndexCantBeInvisible)
// Only first unique column can be implicit primary
tk.MustExec("create table t4(a int not null, b int not null, unique (a), unique (b) invisible)")
showIndexes = "select index_name, is_visible from information_schema.statistics where table_schema = 'test' and table_name = 't4'"
tk.MustQuery(showIndexes).Check(testkit.Rows("a YES", "b NO"))
tk.MustExec("insert into t4 values (1, 2)")
tk.MustQuery("select * from t4").Check(testkit.Rows("1 2"))
tk.MustGetErrCode("create table t5(a int not null, b int not null, unique (b) invisible, unique (a))", errno.ErrPKIndexCantBeInvisible)
// Column `b` become implicit primary key after DDL statement on other columns
tk.MustExec("create table t5(a int not null, b int not null, unique (a), unique (b) invisible)")
tk.MustGetErrCode("alter table t5 drop index a", errno.ErrPKIndexCantBeInvisible)
tk.MustGetErrCode("alter table t5 modify column a int null", errno.ErrPKIndexCantBeInvisible)
// If these is a explicit primary key, no key will become implicit primary key
tk.MustExec("create table t6 (a int not null, b int, unique (a) invisible, primary key(b))")
showIndexes = "select index_name, is_visible from information_schema.statistics where table_schema = 'test' and table_name = 't6'"
tk.MustQuery(showIndexes).Check(testkit.Rows("a NO", "PRIMARY YES"))
tk.MustExec("insert into t6 values (1, 2)")
tk.MustQuery("select * from t6").Check(testkit.Rows("1 2"))
tk.MustGetErrCode("alter table t6 drop primary key", errno.ErrPKIndexCantBeInvisible)
}
1 change: 0 additions & 1 deletion ddl/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -913,7 +913,6 @@ func updateVersionAndTableInfoWithCheck(t *meta.Meta, job *model.Job, tblInfo *m
return ver, errors.Trace(err)
}
return updateVersionAndTableInfo(t, job, tblInfo, shouldUpdateVer)

}

// updateVersionAndTableInfo updates the schema version and the table information.
Expand Down
1 change: 1 addition & 0 deletions errno/errcode.go
Original file line number Diff line number Diff line change
Expand Up @@ -906,6 +906,7 @@ const (
ErrUserAlreadyExists = 3163
ErrInvalidJSONPathArrayCell = 3165
ErrInvalidEncryptionOption = 3184
ErrPKIndexCantBeInvisible = 3522
ErrRoleNotGranted = 3530
ErrLockAcquireFailAndNoWaitSet = 3572
ErrWindowNoSuchWindow = 3579
Expand Down
1 change: 1 addition & 0 deletions errno/errname.go
Original file line number Diff line number Diff line change
Expand Up @@ -902,6 +902,7 @@ var MySQLErrName = map[uint16]string{
ErrUserAlreadyExists: "User %s already exists.",
ErrInvalidJSONPathArrayCell: "A path expression is not a path to a cell in an array.",
ErrInvalidEncryptionOption: "Invalid encryption option.",
ErrPKIndexCantBeInvisible: "A primary key index cannot be invisible",
ErrWindowNoSuchWindow: "Window name '%s' is not defined.",
ErrWindowCircularityInWindowGraph: "There is a circularity in the window dependency graph.",
ErrWindowNoChildPartitioning: "A window which depends on another cannot define partitioning.",
Expand Down
1 change: 1 addition & 0 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1710,6 +1710,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) {
errCount, warnCount := vars.StmtCtx.NumErrorWarnings()
vars.SysErrorCount = errCount
vars.SysWarningCount = warnCount
sc.OptimizerUseInvisibleIndexes = vars.StmtCtx.OptimizerUseInvisibleIndexes
vars.StmtCtx = sc
vars.PrevFoundInPlanCache = vars.FoundInPlanCache
vars.FoundInPlanCache = false
Expand Down
5 changes: 3 additions & 2 deletions executor/seqtest/seq_executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -286,20 +286,21 @@ func (s *seqTestSuite) TestShow(c *C) {
b int,
c int UNIQUE KEY,
d int UNIQUE KEY,
index (b) invisible,
index invisible_idx_b (b) invisible,
index (d) invisible)`)
excepted :=
"t CREATE TABLE `t` (\n" +
" `a` int(11) DEFAULT NULL,\n" +
" `b` int(11) DEFAULT NULL,\n" +
" `c` int(11) DEFAULT NULL,\n" +
" `d` int(11) DEFAULT NULL,\n" +
" KEY `b` (`b`) /*!80000 INVISIBLE */,\n" +
" KEY `invisible_idx_b` (`b`) /*!80000 INVISIBLE */,\n" +
" KEY `d` (`d`) /*!80000 INVISIBLE */,\n" +
" UNIQUE KEY `c` (`c`),\n" +
" UNIQUE KEY `d_2` (`d`)\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin"
tk.MustQuery("show create table t").Check(testkit.Rows(excepted))
tk.MustExec("drop table t")

testSQL = "SHOW VARIABLES LIKE 'character_set_results';"
result = tk.MustQuery(testSQL)
Expand Down
32 changes: 32 additions & 0 deletions planner/core/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -649,6 +649,38 @@ func (s *testIntegrationSuite) TestIndexMerge(c *C) {
}
}

func (s *testIntegrationSuite) TestInvisibleIndex(c *C) {
tk := testkit.NewTestKit(c, s.store)

tk.MustExec("use test")
tk.MustExec("drop table if exists t")

// Optimizer cannot see invisible indexes.
tk.MustExec("create table t(a int, b int, unique index i_a (a) invisible, unique index i_b(b))")
tk.MustExec("insert into t values (1,2)")

// Optimizer cannot use invisible indexes.
tk.MustQuery("select a from t order by a").Check(testkit.Rows("1"))
c.Check(tk.MustUseIndex("select a from t order by a", "i_a"), IsFalse)
tk.MustQuery("select a from t where a > 0").Check(testkit.Rows("1"))
c.Check(tk.MustUseIndex("select a from t where a > 1", "i_a"), IsFalse)

// If use invisible indexes in index hint and sql hint, throw an error.
errStr := "[planner:1176]Key 'i_a' doesn't exist in table 't'"
tk.MustGetErrMsg("select * from t use index(i_a)", errStr)
tk.MustGetErrMsg("select * from t force index(i_a)", errStr)
tk.MustGetErrMsg("select * from t ignore index(i_a)", errStr)
tk.MustQuery("select /*+ USE_INDEX(t, i_a) */ * from t")
c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 1)
c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings()[0].Err.Error(), Equals, errStr)
tk.MustQuery("select /*+ IGNORE_INDEX(t, i_a), USE_INDEX(t, i_b) */ a from t order by a")
c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 1)
c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings()[0].Err.Error(), Equals, errStr)

tk.MustExec("admin check table t")
tk.MustExec("admin check index t i_a")
}

// for issue #14822
func (s *testIntegrationSuite) TestIndexJoinTableRange(c *C) {
tk := testkit.NewTestKit(c, s.store)
Expand Down
5 changes: 5 additions & 0 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -723,8 +723,13 @@ func (b *PlanBuilder) getPossibleAccessPaths(indexHints []*ast.IndexHint, tbl ta
if tblInfo.TiFlashReplica != nil && tblInfo.TiFlashReplica.Available {
publicPaths = append(publicPaths, &util.AccessPath{IsTablePath: true, StoreType: kv.TiFlash})
}
optimizerUseInvisibleIndexes := b.ctx.GetSessionVars().StmtCtx.OptimizerUseInvisibleIndexes
for _, index := range tblInfo.Indices {
if index.State == model.StatePublic {
// Filter out invisible index, because they are not visible for optimizer
if !optimizerUseInvisibleIndexes && index.Invisible {
continue
}
publicPaths = append(publicPaths, &util.AccessPath{Index: index})
}
}
Expand Down
8 changes: 8 additions & 0 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -762,6 +762,10 @@ func (s *session) ExecRestrictedSQLWithContext(ctx context.Context, sql string)
se.sessionVars.InspectionTableCache = cache
defer func() { se.sessionVars.InspectionTableCache = nil }()
}
if ok := s.sessionVars.StmtCtx.OptimizerUseInvisibleIndexes; ok {
se.sessionVars.StmtCtx.OptimizerUseInvisibleIndexes = true
defer func() { se.sessionVars.StmtCtx.OptimizerUseInvisibleIndexes = false }()
}
defer func() {
if se != nil && se.GetSessionVars().StmtCtx.WarningCount() > 0 {
warnings := se.GetSessionVars().StmtCtx.GetWarnings()
Expand Down Expand Up @@ -822,6 +826,10 @@ func (s *session) ExecRestrictedSQLWithSnapshot(sql string) ([]chunk.Row, []*ast
se.sessionVars.SnapshotInfoschema = nil
}()
}
if ok := s.sessionVars.StmtCtx.OptimizerUseInvisibleIndexes; ok {
se.sessionVars.StmtCtx.OptimizerUseInvisibleIndexes = true
defer func() { se.sessionVars.StmtCtx.OptimizerUseInvisibleIndexes = false }()
}
return execRestrictedSQL(ctx, se, sql)
}

Expand Down
2 changes: 2 additions & 0 deletions sessionctx/stmtctx/stmtctx.go
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,8 @@ type StatementContext struct {
BatchCheck bool
InNullRejectCheck bool
AllowInvalidDate bool
// OptimizerUseInvisibleIndexes indicates whether optimizer can use invisible index
OptimizerUseInvisibleIndexes bool

// mu struct holds variables that change during execution.
mu struct {
Expand Down
2 changes: 2 additions & 0 deletions util/admin/admin.go
Original file line number Diff line number Diff line change
Expand Up @@ -309,6 +309,8 @@ const (
// It returns nil if the count from the index is equal to the count from the table columns,
// otherwise it returns an error and the corresponding index's offset.
func CheckIndicesCount(ctx sessionctx.Context, dbName, tableName string, indices []string) (byte, int, error) {
// Here we need check all indexes, includes invisible index
ctx.GetSessionVars().StmtCtx.OptimizerUseInvisibleIndexes = true
// Add `` for some names like `table name`.
sql := fmt.Sprintf("SELECT COUNT(*) FROM `%s`.`%s` USE INDEX()", dbName, tableName)
tblCnt, err := getCount(ctx, sql)
Expand Down

0 comments on commit 26e946d

Please sign in to comment.