Skip to content

Commit

Permalink
*: support tidb_opt_use_invisible_indexes (#50654)
Browse files Browse the repository at this point in the history
ref #50653
  • Loading branch information
hawkingrei authored Jan 25, 2024
1 parent 4abe192 commit e42d0bf
Show file tree
Hide file tree
Showing 6 changed files with 28 additions and 7 deletions.
17 changes: 17 additions & 0 deletions pkg/planner/core/casetest/index/index_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -78,3 +78,20 @@ func TestNullConditionForPrefixIndex(t *testing.T) {
" └─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#7",
" └─IndexRangeScan_16 99.90 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" -inf,\"0xfff\" +inf], keep order:false, stats:pseudo"))
}

func TestInvisibleIndex(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("CREATE TABLE t1 ( a INT, KEY( a ) INVISIBLE );")
tk.MustExec("INSERT INTO t1 VALUES (1), (2), (3), (4), (5), (6), (7), (8), (9), (10);")
tk.MustQuery(`EXPLAIN SELECT a FROM t1;`).Check(
testkit.Rows(
`TableReader_5 10000.00 root data:TableFullScan_4`,
`└─TableFullScan_4 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo`))
tk.MustExec("set session tidb_opt_use_invisible_indexes=on;")
tk.MustQuery(`EXPLAIN SELECT a FROM t1;`).Check(
testkit.Rows(
`IndexReader_7 10000.00 root index:IndexFullScan_6`,
`└─IndexFullScan_6 10000.00 cop[tikv] table:t1, index:a(a) keep order:false, stats:pseudo`))
}
4 changes: 2 additions & 2 deletions pkg/planner/core/point_get_plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -740,7 +740,7 @@ func newBatchPointGetPlan(
}
}
for _, idxInfo := range tbl.Indices {
if !idxInfo.Unique || idxInfo.State != model.StatePublic || idxInfo.Invisible || idxInfo.MVIndex ||
if !idxInfo.Unique || idxInfo.State != model.StatePublic || (idxInfo.Invisible && !ctx.GetSessionVars().OptimizerUseInvisibleIndexes) || idxInfo.MVIndex ||
!indexIsAvailableByHints(idxInfo, indexHints) {
continue
}
Expand Down Expand Up @@ -1135,7 +1135,7 @@ func checkTblIndexForPointPlan(ctx sessionctx.Context, tblName *ast.TableName, s
dbName = ctx.GetSessionVars().CurrentDB
}
for _, idxInfo := range tbl.Indices {
if !idxInfo.Unique || idxInfo.State != model.StatePublic || idxInfo.Invisible || idxInfo.MVIndex ||
if !idxInfo.Unique || idxInfo.State != model.StatePublic || (idxInfo.Invisible && !ctx.GetSessionVars().OptimizerUseInvisibleIndexes) || idxInfo.MVIndex ||
!indexIsAvailableByHints(idxInfo, tblName.IndexHints) {
continue
}
Expand Down
4 changes: 1 addition & 3 deletions pkg/session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -1957,9 +1957,7 @@ func (s *session) getInternalSession(execOption sqlexec.ExecOption) (*session, f
if cache := s.sessionVars.InspectionTableCache; cache != nil {
se.sessionVars.InspectionTableCache = cache
}
if ok := s.sessionVars.OptimizerUseInvisibleIndexes; ok {
se.sessionVars.OptimizerUseInvisibleIndexes = true
}
se.sessionVars.OptimizerUseInvisibleIndexes = s.sessionVars.OptimizerUseInvisibleIndexes

if execOption.SnapshotTS != 0 {
if err := se.sessionVars.SetSystemVar(variable.TiDBSnapshot, strconv.FormatUint(execOption.SnapshotTS, 10)); err != nil {
Expand Down
4 changes: 4 additions & 0 deletions pkg/sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -2396,6 +2396,10 @@ var defaultSysVars = []*SysVar{
s.EnableAdvancedJoinHint = TiDBOptOn(val)
return nil
}},
{Scope: ScopeSession, Name: TiDBOptUseInvisibleIndexes, Value: BoolToOnOff(false), Type: TypeBool, SetSession: func(s *SessionVars, val string) error {
s.OptimizerUseInvisibleIndexes = TiDBOptOn(val)
return nil
}},
{Scope: ScopeGlobal | ScopeSession, Name: TiDBAnalyzePartitionConcurrency, Value: strconv.FormatInt(DefTiDBAnalyzePartitionConcurrency, 10),
MinValue: 1, MaxValue: uint64(config.GetGlobalConfig().Performance.AnalyzePartitionConcurrencyQuota), SetSession: func(s *SessionVars, val string) error {
s.AnalyzePartitionConcurrency = int(TidbOptInt64(val, DefTiDBAnalyzePartitionConcurrency))
Expand Down
3 changes: 2 additions & 1 deletion pkg/sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -851,7 +851,8 @@ const (

// TiDBOptAdvancedJoinHint indicates whether the join method hint is compatible with join order hint.
TiDBOptAdvancedJoinHint = "tidb_opt_advanced_join_hint"

// TiDBOptUseInvisibleIndexes indicates whether to use invisible indexes.
TiDBOptUseInvisibleIndexes = "tidb_opt_use_invisible_indexes"
// TiDBAnalyzePartitionConcurrency indicates concurrency for save/read partitions stats in Analyze
TiDBAnalyzePartitionConcurrency = "tidb_analyze_partition_concurrency"
// TiDBMergePartitionStatsConcurrency indicates the concurrency when merge partition stats into global stats
Expand Down
3 changes: 2 additions & 1 deletion pkg/util/admin/admin.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,9 +70,10 @@ const (
// 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
originOptUseInvisibleIdx := ctx.GetSessionVars().OptimizerUseInvisibleIndexes
ctx.GetSessionVars().OptimizerUseInvisibleIndexes = true
defer func() {
ctx.GetSessionVars().OptimizerUseInvisibleIndexes = false
ctx.GetSessionVars().OptimizerUseInvisibleIndexes = originOptUseInvisibleIdx
}()

var snapshot uint64
Expand Down

0 comments on commit e42d0bf

Please sign in to comment.