Skip to content

Commit

Permalink
executor: fix the issue that the int primary key doesn't show in the …
Browse files Browse the repository at this point in the history
…index usage (#58257)

close #58256
  • Loading branch information
YangKeao authored Dec 23, 2024
1 parent 3735ed5 commit 52de47d
Show file tree
Hide file tree
Showing 6 changed files with 199 additions and 19 deletions.
1 change: 1 addition & 0 deletions pkg/executor/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -514,6 +514,7 @@ go_test(
"@org_golang_google_grpc//:grpc",
"@org_uber_go_atomic//:atomic",
"@org_uber_go_goleak//:goleak",
"@org_uber_go_zap//:zap",
"@org_uber_go_zap//zapcore",
],
)
2 changes: 1 addition & 1 deletion pkg/executor/infoschema_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -3910,7 +3910,7 @@ func (e *memtableRetriever) setDataFromIndexUsage(ctx context.Context, sctx sess
usage := dom.StatsHandle().GetIndexUsage(tbl.ID, idx.ID)
row = append(row, types.NewStringDatum(schema.O))
row = append(row, types.NewStringDatum(tbl.Name.O))
row = append(row, types.NewStringDatum(idx.Name.O))
row = append(row, types.NewStringDatum(idx.Name))
row = append(row, types.NewIntDatum(int64(usage.QueryTotal)))
row = append(row, types.NewIntDatum(int64(usage.KvReqTotal)))
row = append(row, types.NewIntDatum(int64(usage.RowAccessTotal)))
Expand Down
179 changes: 170 additions & 9 deletions pkg/executor/infoschema_reader_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,10 +35,12 @@ import (
"github.com/pingcap/tidb/pkg/store/mockstore"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/testkit/testfailpoint"
"github.com/pingcap/tidb/pkg/util/logutil"
"github.com/pingcap/tidb/pkg/util/stringutil"
"github.com/stretchr/testify/require"
"github.com/tikv/client-go/v2/tikv"
"github.com/tikv/client-go/v2/tikvrpc"
"go.uber.org/zap"
)

func TestInspectionTables(t *testing.T) {
Expand Down Expand Up @@ -631,30 +633,39 @@ func TestIndexUsageTable(t *testing.T) {
tk.MustExec("use test")
tk.MustExec("create table idt1(col_1 int primary key, col_2 int, index idx_1(col_1), index idx_2(col_2), index idx_3(col_1, col_2));")
tk.MustExec("create table idt2(col_1 int primary key, col_2 int, index idx_1(col_1), index idx_2(col_2), index idx_4(col_2, col_1));")
tk.MustExec("create table idt3(col_1 varchar(255) primary key);")
tk.MustExec("create table idt4(col_1 varchar(255) primary key NONCLUSTERED);")
tk.MustExec("create table idt5(col_1 int);")

tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage
where TABLE_SCHEMA = 'test';`).Check(
where TABLE_SCHEMA = 'test';`).Sort().Check(
testkit.RowsWithSep("|",
"test|idt1|idx_1",
"test|idt1|idx_2",
"test|idt1|idx_3",
"test|idt1|primary",
"test|idt2|idx_1",
"test|idt2|idx_2",
"test|idt2|idx_4"))
tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_NAME = 'idt1'`).Check(
"test|idt2|idx_4",
"test|idt2|primary",
"test|idt3|primary",
"test|idt4|primary"))
tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_NAME = 'idt1'`).Sort().Check(
testkit.RowsWithSep("|",
"test|idt1|idx_1",
"test|idt1|idx_2",
"test|idt1|idx_3"))
"test|idt1|idx_3",
"test|idt1|primary"))
tk.MustQuery("select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where INDEX_NAME = 'IDX_3'").Check(
testkit.RowsWithSep("|",
"test|idt1|idx_3"))
tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage
where TABLE_SCHEMA = 'test' and TABLE_NAME = 'idt1';`).Check(
where TABLE_SCHEMA = 'test' and TABLE_NAME = 'idt1';`).Sort().Check(
testkit.RowsWithSep("|",
"test|idt1|idx_1",
"test|idt1|idx_2",
"test|idt1|idx_3"))
"test|idt1|idx_3",
"test|idt1|primary"))
tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage
where TABLE_SCHEMA = 'test' and INDEX_NAME = 'idx_2';`).Sort().Check(
testkit.RowsWithSep("|",
Expand All @@ -669,13 +680,13 @@ func TestIndexUsageTable(t *testing.T) {
testkit.RowsWithSep("|",
"test|idt2|idx_4"))
tk.MustQuery(`select count(*) from information_schema.tidb_index_usage
where TABLE_SCHEMA = 'test' and TABLE_NAME in ('idt1', 'idt2');`).Check(
testkit.RowsWithSep("|", "6"))
where TABLE_SCHEMA = 'test' and TABLE_NAME in ('idt1', 'idt2');`).Check(
testkit.RowsWithSep("|", "8"))

tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage
where TABLE_SCHEMA = 'test1';`).Check(testkit.Rows())
tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage
where TABLE_NAME = 'idt3';`).Check(testkit.Rows())
where TABLE_NAME = 'idt3';`).Check(testkit.Rows("test idt3 primary"))
tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage
where INDEX_NAME = 'IDX_5';`).Check(testkit.Rows())
tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage
Expand Down Expand Up @@ -1089,3 +1100,153 @@ func TestInfoschemaTablesSpecialOptimizationCovered(t *testing.T) {
require.Equal(t, testCase.expect, covered, testCase.sql)
}
}

func TestIndexUsageWithData(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
// Some bad tests will set the global variable to 0, and they don't set it back. So even if the default value for this variable is 1,
// we'll need to set it to 1 here.
tk.MustExec("set global tidb_enable_collect_execution_info=1;")
tk.RefreshSession()

insertDataAndScanToT := func(indexName string) {
// insert 1000 rows
tk.MustExec("INSERT into t WITH RECURSIVE cte AS (select 1 as n UNION ALL select n+1 FROM cte WHERE n < 1000) select n from cte;")
tk.MustExec("ANALYZE TABLE t")

// full scan
sql := fmt.Sprintf("SELECT * FROM t use index(%s) ORDER BY a", indexName)
rows := tk.MustQuery(sql).Rows()
require.Len(t, rows, 1000)
for i, r := range rows {
require.Equal(t, r[0], strconv.Itoa(i+1))
}

logutil.BgLogger().Info("execute with plan",
zap.String("sql", sql),
zap.String("plan", tk.MustQuery("explain "+sql).String()))

// scan 1/4 of the rows
sql = fmt.Sprintf("SELECT * FROM t use index(%s) WHERE a <= 250 ORDER BY a", indexName)
rows = tk.MustQuery(sql).Rows()
require.Len(t, rows, 250)
for i, r := range rows {
require.Equal(t, r[0], strconv.Itoa(i+1))
}

logutil.BgLogger().Info("execute with plan",
zap.String("sql", sql),
zap.String("plan", tk.MustQuery("explain "+sql).String()))
}

checkIndexUsage := func(startQuery time.Time, endQuery time.Time) {
require.Eventually(t, func() bool {
rows := tk.MustQuery("select QUERY_TOTAL,PERCENTAGE_ACCESS_20_50,PERCENTAGE_ACCESS_100,LAST_ACCESS_TIME from information_schema.tidb_index_usage where table_schema = 'test'").Rows()

if len(rows) != 1 {
return false
}
if rows[0][0] != "2" || rows[0][1] != "1" || rows[0][2] != "1" {
return false
}
lastAccessTime, err := time.ParseInLocation(time.DateTime, rows[0][3].(string), time.Local)
if err != nil {
return false
}
if lastAccessTime.Unix() < startQuery.Unix() || lastAccessTime.Unix() > endQuery.Unix() {
return false
}

return true
}, 5*time.Second, 100*time.Millisecond)
}

t.Run("test index usage with normal index", func(t *testing.T) {
tk.MustExec("use test")
tk.MustExec("create table t (a int, index idx(a));")
defer tk.MustExec("drop table t")

tk.MustQuery("select * from information_schema.tidb_index_usage where table_schema = 'test'").Check(testkit.Rows(
"test t idx 0 0 0 0 0 0 0 0 0 0 <nil>",
))

startQuery := time.Now()
insertDataAndScanToT("idx")
endQuery := time.Now()

checkIndexUsage(startQuery, endQuery)
})

t.Run("test index usage with integer primary key", func(t *testing.T) {
tk.MustExec("use test")
tk.MustExec("create table t (a int primary key);")
defer tk.MustExec("drop table t")

tk.MustQuery("select * from information_schema.tidb_index_usage where table_schema = 'test'").Check(testkit.Rows(
"test t primary 0 0 0 0 0 0 0 0 0 0 <nil>",
))

startQuery := time.Now()
insertDataAndScanToT("primary")
endQuery := time.Now()

checkIndexUsage(startQuery, endQuery)
})

t.Run("test index usage with integer clustered primary key", func(t *testing.T) {
tk.MustExec("use test")
tk.MustExec("create table t (a bigint primary key clustered);")
defer tk.MustExec("drop table t")

tk.MustQuery("select * from information_schema.tidb_index_usage where table_schema = 'test'").Check(testkit.Rows(
"test t primary 0 0 0 0 0 0 0 0 0 0 <nil>",
))

startQuery := time.Now()
insertDataAndScanToT("primary")
endQuery := time.Now()

checkIndexUsage(startQuery, endQuery)
})

t.Run("test index usage with string primary key", func(t *testing.T) {
tk.MustExec("use test")
tk.MustExec("create table t (a varchar(16) primary key clustered);")
defer tk.MustExec("drop table t")

tk.MustQuery("select * from information_schema.tidb_index_usage where table_schema = 'test'").Check(testkit.Rows(
"test t primary 0 0 0 0 0 0 0 0 0 0 <nil>",
))

tk.MustExec("INSERT into t WITH RECURSIVE cte AS (select 1 as n UNION ALL select n+1 FROM cte WHERE n < 1000) select n from cte;")
tk.MustExec("ANALYZE TABLE t")

// full scan
rows := tk.MustQuery("SELECT * FROM t ORDER BY a").Rows()
require.Len(t, rows, 1000)

// scan 1/4 of the rows
startQuery := time.Now()
rows = tk.MustQuery("select * from t where a < '3'").Rows()
require.Len(t, rows, 223)
endQuery := time.Now()

checkIndexUsage(startQuery, endQuery)
})

t.Run("test index usage with nonclustered primary key", func(t *testing.T) {
tk.MustExec("use test")
tk.MustExec("create table t (a int primary key nonclustered);")
defer tk.MustExec("drop table t")

tk.MustQuery("select * from information_schema.tidb_index_usage where table_schema = 'test'").Check(testkit.Rows(
"test t primary 0 0 0 0 0 0 0 0 0 0 <nil>",
))

startQuery := time.Now()
insertDataAndScanToT("primary")
endQuery := time.Now()

checkIndexUsage(startQuery, endQuery)
})
}
1 change: 0 additions & 1 deletion pkg/executor/internal/exec/indexusage.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,6 @@ func (e *IndexUsageReporter) ReportCopIndexUsageForTable(tbl table.Table, indexI
func (e *IndexUsageReporter) ReportCopIndexUsage(tableID int64, physicalTableID int64, indexID int64, planID int) {
tableRowCount, ok := e.getTableRowCount(physicalTableID)
if !ok {
// skip if the table is empty or the stats is not valid
return
}

Expand Down
31 changes: 23 additions & 8 deletions pkg/planner/core/memtable_infoschema_extractor.go
Original file line number Diff line number Diff line change
Expand Up @@ -941,6 +941,13 @@ type InfoSchemaTiDBIndexUsageExtractor struct {
predIdxNamesInited bool
}

// IndexUsageIndexInfo is the necessary index info for information_schema.tidb_index_usage. It only includes the index name
// and ID in lower case.
type IndexUsageIndexInfo struct {
Name string
ID int64
}

// NewInfoSchemaTiDBIndexUsageExtractor creates a new InfoSchemaTiDBIndexUsageExtractor.
func NewInfoSchemaTiDBIndexUsageExtractor() *InfoSchemaTiDBIndexUsageExtractor {
e := &InfoSchemaTiDBIndexUsageExtractor{}
Expand All @@ -957,7 +964,7 @@ func NewInfoSchemaTiDBIndexUsageExtractor() *InfoSchemaTiDBIndexUsageExtractor {
// If no index found in predicate, it return all indexes.
func (e *InfoSchemaTiDBIndexUsageExtractor) ListIndexes(
tbl *model.TableInfo,
) []*model.IndexInfo {
) []IndexUsageIndexInfo {
ec := e.extractableColumns
if !e.predIdxNamesInited {
e.predIdxNames = set.NewStringSet()
Expand All @@ -970,23 +977,31 @@ func (e *InfoSchemaTiDBIndexUsageExtractor) ListIndexes(
predCol := e.predIdxNames
regexp := e.GetBase().colsRegexp[ec.indexName]

indexes := make([]IndexUsageIndexInfo, 0, len(tbl.Indices))
// Append the int primary key. The clustered index is already included in the `tbl.Indices`, but the int primary key is not.
if tbl.PKIsHandle {
indexes = append(indexes, IndexUsageIndexInfo{Name: primaryKeyName, ID: 0})
}
for _, index := range tbl.Indices {
indexes = append(indexes, IndexUsageIndexInfo{Name: index.Name.L, ID: index.ID})
}
if len(predCol) == 0 && len(regexp) == 0 {
return tbl.Indices
return indexes
}

indexes := make([]*model.IndexInfo, 0, len(predCol))
retIndexes := make([]IndexUsageIndexInfo, 0, len(indexes))
ForLoop:
for _, index := range tbl.Indices {
if len(predCol) > 0 && !predCol.Exist(index.Name.L) {
for _, index := range indexes {
if len(predCol) > 0 && !predCol.Exist(index.Name) {
continue
}
for _, re := range regexp {
if !re.DoMatch(index.Name.L) {
if !re.DoMatch(index.Name) {
continue ForLoop
}
}
indexes = append(indexes, index)
retIndexes = append(retIndexes, IndexUsageIndexInfo{Name: index.Name, ID: index.ID})
}

return indexes
return retIndexes
}
4 changes: 4 additions & 0 deletions tests/integrationtest/r/infoschema/infoschema.result
Original file line number Diff line number Diff line change
Expand Up @@ -314,14 +314,17 @@ TABLE_SCHEMA TABLE_NAME INDEX_NAME
indexusage idt1 idx_1
indexusage idt1 idx_2
indexusage idt1 idx_3
indexusage idt1 primary
indexusage idt2 idx_1
indexusage idt2 idx_2
indexusage idt2 idx_4
indexusage idt2 primary
select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_NAME = 'idt1';
TABLE_SCHEMA TABLE_NAME INDEX_NAME
indexusage idt1 idx_1
indexusage idt1 idx_2
indexusage idt1 idx_3
indexusage idt1 primary
select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where INDEX_NAME = 'IDX_3';
TABLE_SCHEMA TABLE_NAME INDEX_NAME
indexusage idt1 idx_3
Expand All @@ -330,6 +333,7 @@ TABLE_SCHEMA TABLE_NAME INDEX_NAME
indexusage idt1 idx_1
indexusage idt1 idx_2
indexusage idt1 idx_3
indexusage idt1 primary
select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_SCHEMA = 'indexusage' and INDEX_NAME = 'idx_2';
TABLE_SCHEMA TABLE_NAME INDEX_NAME
indexusage idt1 idx_2
Expand Down

0 comments on commit 52de47d

Please sign in to comment.