Skip to content

Commit

Permalink
add tidb_index_usage table and cluster_tidb_index_usage table
Browse files Browse the repository at this point in the history
Signed-off-by: Yang Keao <yangkeao@chunibyo.icu>
  • Loading branch information
YangKeao committed Feb 2, 2024
1 parent 0250675 commit 3b7b564
Show file tree
Hide file tree
Showing 6 changed files with 197 additions and 2 deletions.
4 changes: 3 additions & 1 deletion pkg/executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -2111,7 +2111,9 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) exec.Ex
strings.ToLower(infoschema.TableRunawayWatches),
strings.ToLower(infoschema.TableCheckConstraints),
strings.ToLower(infoschema.TableTiDBCheckConstraints),
strings.ToLower(infoschema.TableKeywords):
strings.ToLower(infoschema.TableKeywords),
strings.ToLower(infoschema.TableTiDBIndexUsage),
strings.ToLower(infoschema.ClusterTableTiDBIndexUsage):
return &MemTableReaderExec{
BaseExecutor: exec.NewBaseExecutor(b.ctx, v.Schema(), v.ID()),
table: v.Table,
Expand Down
56 changes: 56 additions & 0 deletions pkg/executor/infoschema_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -201,6 +201,10 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex
err = e.setDataFromTiDBCheckConstraints(sctx, dbs)
case infoschema.TableKeywords:
err = e.setDataFromKeywords()
case infoschema.TableTiDBIndexUsage:
e.setDataFromIndexUsage(sctx, dbs)
case infoschema.ClusterTableTiDBIndexUsage:
err = e.setDataForClusterIndexUsage(sctx, dbs)
}
if err != nil {
return nil, err
Expand Down Expand Up @@ -3453,6 +3457,58 @@ func (e *memtableRetriever) setDataFromKeywords() error {
return nil
}

func (e *memtableRetriever) setDataFromIndexUsage(ctx sessionctx.Context, schemas []*model.DBInfo) {
dom := domain.GetDomain(ctx)
rows := make([][]types.Datum, 0, 100)
checker := privilege.GetPrivilegeManager(ctx)

for _, schema := range schemas {
for _, tbl := range schema.Tables {
allowed := checker == nil || checker.RequestVerification(
ctx.GetSessionVars().ActiveRoles,
schema.Name.L, tbl.Name.L, "", mysql.AllPrivMask)
if !allowed {
continue
}

for _, idx := range tbl.Indices {
row := make([]types.Datum, 0, 14)

usage := dom.StatsHandle().GetIndexUsage(tbl.ID, idx.ID)
row = append(row, types.NewStringDatum(schema.Name.O))
row = append(row, types.NewStringDatum(tbl.Name.O))
row = append(row, types.NewStringDatum(idx.Name.O))
row = append(row, types.NewIntDatum(int64(usage.QueryTotal)))
row = append(row, types.NewIntDatum(int64(usage.KvReqTotal)))
row = append(row, types.NewIntDatum(int64(usage.RowAccessTotal)))
for _, percentage := range usage.PercentageAccess {
row = append(row, types.NewIntDatum(int64(percentage)))
}
lastUsedAt := types.Datum{}
lastUsedAt.SetNull()
if !usage.LastUsedAt.IsZero() {
t := types.NewTime(types.FromGoTime(usage.LastUsedAt), mysql.TypeTimestamp, 0)
lastUsedAt = types.NewTimeDatum(t)
}
row = append(row, lastUsedAt)
rows = append(rows, row)
}
}
}

e.rows = rows
}

func (e *memtableRetriever) setDataForClusterIndexUsage(ctx sessionctx.Context, schemas []*model.DBInfo) error {
e.setDataFromIndexUsage(ctx, schemas)
rows, err := infoschema.AppendHostInfoToRows(ctx, e.rows)
if err != nil {
return err
}
e.rows = rows
return nil
}

func checkRule(rule *label.Rule) (dbName, tableName string, partitionName string, err error) {
s := strings.Split(rule.ID, "/")
if len(s) < 3 {
Expand Down
3 changes: 3 additions & 0 deletions pkg/infoschema/cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,8 @@ const (
ClusterTableMemoryUsage = "CLUSTER_MEMORY_USAGE"
// ClusterTableMemoryUsageOpsHistory is the memory control operators history of tidb cluster.
ClusterTableMemoryUsageOpsHistory = "CLUSTER_MEMORY_USAGE_OPS_HISTORY"
// ClusterTableTiDBIndexUsage is a table to show the usage stats of indexes across the whole cluster.
ClusterTableTiDBIndexUsage = "CLUSTER_TIDB_INDEX_USAGE"
)

// memTableToAllTiDBClusterTables means add memory table to cluster table that will send cop request to all TiDB nodes.
Expand All @@ -66,6 +68,7 @@ var memTableToAllTiDBClusterTables = map[string]string{
TableTrxSummary: ClusterTableTrxSummary,
TableMemoryUsage: ClusterTableMemoryUsage,
TableMemoryUsageOpsHistory: ClusterTableMemoryUsageOpsHistory,
TableTiDBIndexUsage: ClusterTableTiDBIndexUsage,
}

// memTableToDDLOwnerClusterTables means add memory table to cluster table that will send cop request to DDL owner node.
Expand Down
22 changes: 22 additions & 0 deletions pkg/infoschema/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -213,6 +213,8 @@ const (
TableTiDBCheckConstraints = "TIDB_CHECK_CONSTRAINTS"
// TableKeywords is the list of keywords.
TableKeywords = "KEYWORDS"
// TableTiDBIndexUsage is a table to show the usage stats of indexes in the current instance.
TableTiDBIndexUsage = "TIDB_INDEX_USAGE"
)

const (
Expand Down Expand Up @@ -324,6 +326,8 @@ var tableIDMap = map[string]int64{
TableCheckConstraints: autoid.InformationSchemaDBID + 90,
TableTiDBCheckConstraints: autoid.InformationSchemaDBID + 91,
TableKeywords: autoid.InformationSchemaDBID + 92,
TableTiDBIndexUsage: autoid.InformationSchemaDBID + 93,
ClusterTableTiDBIndexUsage: autoid.InformationSchemaDBID + 94,
}

// columnInfo represents the basic column information of all kinds of INFORMATION_SCHEMA tables
Expand Down Expand Up @@ -1670,6 +1674,23 @@ var tableKeywords = []columnInfo{
{name: "RESERVED", tp: mysql.TypeLong, size: 11},
}

var tableTiDBIndexUsage = []columnInfo{
{name: "TABLE_SCHEMA", tp: mysql.TypeVarchar, size: 64},
{name: "TABLE_NAME", tp: mysql.TypeVarchar, size: 64},
{name: "INDEX_NAME", tp: mysql.TypeVarchar, size: 64},
{name: "QUERY_TOTAL", tp: mysql.TypeLonglong, size: 21},
{name: "KV_REQ_TOTAL", tp: mysql.TypeLonglong, size: 21},
{name: "ROWS_ACCESS_TOTAL", tp: mysql.TypeLonglong, size: 21},
{name: "PERCENTAGE_ACCESS_0", tp: mysql.TypeLonglong, size: 21},
{name: "PERCENTAGE_ACCESS_0_1", tp: mysql.TypeLonglong, size: 21},
{name: "PERCENTAGE_ACCESS_1_10", tp: mysql.TypeLonglong, size: 21},
{name: "PERCENTAGE_ACCESS_10_20", tp: mysql.TypeLonglong, size: 21},
{name: "PERCENTAGE_ACCESS_20_50", tp: mysql.TypeLonglong, size: 21},
{name: "PERCENTAGE_ACCESS_50_100", tp: mysql.TypeLonglong, size: 21},
{name: "PERCENTAGE_ACCESS_100", tp: mysql.TypeLonglong, size: 21},
{name: "LAST_ACCESS_TIME", tp: mysql.TypeDatetime, size: 21},
}

// GetShardingInfo returns a nil or description string for the sharding information of given TableInfo.
// The returned description string may be:
// - "NOT_SHARDED": for tables that SHARD_ROW_ID_BITS is not specified.
Expand Down Expand Up @@ -2210,6 +2231,7 @@ var tableNameToColumns = map[string][]columnInfo{
TableCheckConstraints: tableCheckConstraintsCols,
TableTiDBCheckConstraints: tableTiDBCheckConstraintsCols,
TableKeywords: tableKeywords,
TableTiDBIndexUsage: tableTiDBIndexUsage,
}

func createInfoSchemaTable(_ autoid.Allocators, meta *model.TableInfo) (table.Table, error) {
Expand Down
4 changes: 3 additions & 1 deletion pkg/infoschema/test/clustertablestest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@ go_test(
"tables_test.go",
],
flaky = True,
shard_count = 47,
shard_count = 49,
deps = [
"//pkg/config",
"//pkg/domain",
Expand Down Expand Up @@ -40,6 +40,7 @@ go_test(
"//pkg/util",
"//pkg/util/dbterror/exeerrors",
"//pkg/util/gctuner",
"//pkg/util/logutil",
"//pkg/util/memory",
"//pkg/util/resourcegrouptag",
"//pkg/util/set",
Expand All @@ -56,5 +57,6 @@ go_test(
"@com_github_tikv_pd_client//http",
"@org_golang_google_grpc//:grpc",
"@org_uber_go_goleak//:goleak",
"@org_uber_go_zap//:zap",
],
)
110 changes: 110 additions & 0 deletions pkg/infoschema/test/clustertablestest/cluster_tables_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,13 +50,15 @@ import (
"github.com/pingcap/tidb/pkg/testkit/external"
"github.com/pingcap/tidb/pkg/util"
"github.com/pingcap/tidb/pkg/util/dbterror/exeerrors"
"github.com/pingcap/tidb/pkg/util/logutil"
"github.com/pingcap/tidb/pkg/util/resourcegrouptag"
"github.com/pingcap/tidb/pkg/util/set"
"github.com/pingcap/tidb/pkg/util/stmtsummary"
"github.com/pingcap/tipb/go-tipb"
"github.com/stretchr/testify/require"
"github.com/tikv/client-go/v2/testutils"
pd "github.com/tikv/pd/client/http"
"go.uber.org/zap"
"google.golang.org/grpc"
)

Expand Down Expand Up @@ -1548,3 +1550,111 @@ func TestCreateBindingForPrepareToken(t *testing.T) {
tk.MustExec(fmt.Sprintf("create binding from history using plan digest '%s'", planDigest[0][0]))
}
}

func testIndexUsageTable(t *testing.T, clusterTable bool) {
var tk *testkit.TestKit
var tableName string

if clusterTable {
s := new(clusterTablesSuite)
s.store, s.dom = testkit.CreateMockStoreAndDomain(t)
s.rpcserver, s.listenAddr = s.setUpRPCService(t, "127.0.0.1:0", nil)
s.httpServer, s.mockAddr = s.setUpMockPDHTTPServer()
s.startTime = time.Now()
defer s.httpServer.Close()
defer s.rpcserver.Stop()
tk = s.newTestKitWithRoot(t)
tableName = infoschema.ClusterTableTiDBIndexUsage
} else {
store := testkit.CreateMockStore(t)
tk = testkit.NewTestKit(t, store)
tableName = infoschema.TableTiDBIndexUsage
}

tk.MustExec("use test")
tk.MustExec("create table t1(id1 int unique, id2 int unique)")
tk.MustExec("create table t2(id1 int unique, id2 int unique)")

for i := 0; i < 100; i++ {
for j := 1; j <= 2; j++ {
tk.MustExec(fmt.Sprintf("insert into t%d values (?, ?)", j), i, i)
}
}
tk.MustExec("analyze table t1, t2")
tk.RefreshSession()
tk.MustExec("use test")
// range scan 0-10 through t1 id1
tk.MustQuery("select * from t1 use index(id1) where id1 >= 0 and id1 < 10")
// range scan 10-30 through t1 id2
tk.MustQuery("select * from t1 use index(id2) where id2 >= 10 and id2 < 30")
// range scan 30-60 through t2 id1
tk.MustQuery("select * from t2 use index(id1) where id1 >= 30 and id1 < 60")
// range scan 60-100 through t2 id2
tk.MustQuery("select * from t2 use index(id2) where id2 >= 60 and id2 < 100")
tk.RefreshSession()

require.Eventually(t, func() bool {
result := tk.MustQuery(fmt.Sprintf(`select
query_total,
rows_access_total,
percentage_access_0,
percentage_access_0_1,
percentage_access_1_10,
percentage_access_10_20,
percentage_access_20_50,
percentage_access_50_100,
percentage_access_100
from information_schema.%s
where table_schema='test' and
(table_name='t1' or table_name='t2') and
(index_name='id1' or index_name='id2') and
last_access_time is not null
order by table_name, index_name;`, tableName))
expectedResult := testkit.Rows(
"1 10 0 0 0 1 0 0 0",
"1 20 0 0 0 0 1 0 0",
"1 30 0 0 0 0 1 0 0",
"1 40 0 0 0 0 1 0 0")
if !result.Equal(expectedResult) {
logutil.BgLogger().Warn("result not equal", zap.Any("rows", result.Rows()))
return false
}
return true
}, time.Second*5, time.Millisecond*100)

// use another less-privileged user to select
tk.MustExec("create user test_user")
tk.MustExec("grant all privileges on test.t1 to test_user")
tk.RefreshSession()
require.NoError(t, tk.Session().Auth(&auth.UserIdentity{
Username: "test_user",
Hostname: "127.0.0.1",
}, nil, nil, nil))
// `test_user` cannot see table `t2`.
tk.MustQuery(fmt.Sprintf(`select
query_total,
rows_access_total,
percentage_access_0,
percentage_access_0_1,
percentage_access_1_10,
percentage_access_10_20,
percentage_access_20_50,
percentage_access_50_100,
percentage_access_100
from information_schema.%s
where table_schema='test' and
(table_name='t1' or table_name='t2') and
(index_name='id1' or index_name='id2') and
last_access_time is not null
order by table_name, index_name;`, tableName)).Check(testkit.Rows(
"1 10 0 0 0 1 0 0 0",
"1 20 0 0 0 0 1 0 0"))
}

func TestIndexUsageTable(t *testing.T) {
testIndexUsageTable(t, false)
}

func TestClusterIndexUsageTable(t *testing.T) {
testIndexUsageTable(t, true)
}

0 comments on commit 3b7b564

Please sign in to comment.