Skip to content

Commit

Permalink
planner: add a new sys table to store index advisor results and a new…
Browse files Browse the repository at this point in the history
… table to store kernel options (#56188)

ref #12303
  • Loading branch information
qw4990 authored Sep 23, 2024
1 parent fd41e60 commit 8106d93
Show file tree
Hide file tree
Showing 15 changed files with 164 additions and 39 deletions.
5 changes: 3 additions & 2 deletions br/pkg/restore/snap_client/systable_restore.go
Original file line number Diff line number Diff line change
Expand Up @@ -63,8 +63,9 @@ var unRecoverableTable = map[string]map[string]struct{}{
"global_variables": {},
"capture_plan_baselines_blacklist": {},
// gc info don't need to recover.
"gc_delete_range": {},
"gc_delete_range_done": {},
"gc_delete_range": {},
"gc_delete_range_done": {},
"index_advisor_results": {},

// replace into view is not supported now
"tidb_mdl_view": {},
Expand Down
2 changes: 1 addition & 1 deletion br/pkg/restore/snap_client/systable_restore_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -116,5 +116,5 @@ func TestCheckSysTableCompatibility(t *testing.T) {
//
// The above variables are in the file br/pkg/restore/systable_restore.go
func TestMonitorTheSystemTableIncremental(t *testing.T) {
require.Equal(t, int64(213), session.CurrentBootstrapVersion)
require.Equal(t, int64(214), session.CurrentBootstrapVersion)
}
4 changes: 2 additions & 2 deletions pkg/ddl/column_type_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -260,7 +260,7 @@ func TestRowFormatWithChecksums(t *testing.T) {
data, err := h.GetMvccByEncodedKey(encodedKey)
require.NoError(t, err)
// row value with checksums
expected := []byte{0x80, 0x2, 0x3, 0x0, 0x0, 0x0, 0x1, 0x2, 0x3, 0x1, 0x0, 0x4, 0x0, 0x7, 0x0, 0x1, 0x31, 0x32, 0x33, 0x31, 0x32, 0x33, 0x1, 0x2b, 0x9, 0x2d, 0x78}
expected := []byte{0x80, 0x2, 0x3, 0x0, 0x0, 0x0, 0x1, 0x2, 0x3, 0x1, 0x0, 0x4, 0x0, 0x7, 0x0, 0x1, 0x31, 0x32, 0x33, 0x31, 0x32, 0x33, 0x1, 0xa9, 0x7a, 0xf4, 0xc8}
require.Equal(t, expected, data.Info.Writes[0].ShortValue)
tk.MustExec("drop table if exists t")
}
Expand All @@ -284,7 +284,7 @@ func TestRowLevelChecksumWithMultiSchemaChange(t *testing.T) {
data, err := h.GetMvccByEncodedKey(encodedKey)
require.NoError(t, err)
// checksum skipped and with a null col vv
expected := []byte{0x80, 0x2, 0x3, 0x0, 0x1, 0x0, 0x1, 0x2, 0x4, 0x3, 0x1, 0x0, 0x4, 0x0, 0x7, 0x0, 0x1, 0x31, 0x32, 0x33, 0x31, 0x32, 0x33, 0x1, 0x69, 0x31, 0x3, 0x90}
expected := []byte{0x80, 0x2, 0x3, 0x0, 0x1, 0x0, 0x1, 0x2, 0x4, 0x3, 0x1, 0x0, 0x4, 0x0, 0x7, 0x0, 0x1, 0x31, 0x32, 0x33, 0x31, 0x32, 0x33, 0x1, 0xeb, 0x42, 0xda, 0x20}
require.Equal(t, expected, data.Info.Writes[0].ShortValue)
tk.MustExec("drop table if exists t")
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -2763,8 +2763,8 @@ func (e *RecommendIndexExec) Next(ctx context.Context, req *chunk.Chunk) error {
req.AppendString(1, r.Table)
req.AppendString(2, r.IndexName)
req.AppendString(3, strings.Join(r.IndexColumns, ","))
req.AppendString(4, fmt.Sprintf("%v", r.IndexSize))
req.AppendString(5, r.Reason)
req.AppendString(4, fmt.Sprintf("%v", r.IndexDetail.IndexSize))
req.AppendString(5, r.IndexDetail.Reason)

jData, err := json.Marshal(r.TopImpactedQueries)
if err != nil {
Expand Down
2 changes: 1 addition & 1 deletion pkg/executor/importer/importer_testkit_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -353,7 +353,7 @@ func TestProcessChunkWith(t *testing.T) {
require.Len(t, progress.GetColSize(), 3)
checksumMap := checksum.GetInnerChecksums()
require.Len(t, checksumMap, 1)
require.Equal(t, verify.MakeKVChecksum(111, 3, 13867387642099248025), *checksumMap[verify.DataKVGroupID])
require.Equal(t, verify.MakeKVChecksum(111, 3, 14585065391351463171), *checksumMap[verify.DataKVGroupID])
})
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/executor/infoschema_cluster_table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -397,7 +397,7 @@ func TestTableStorageStats(t *testing.T) {
"test 2",
))
rows := tk.MustQuery("select TABLE_NAME from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'mysql';").Rows()
result := 55
result := 57
require.Len(t, rows, result)

// More tests about the privileges.
Expand Down
34 changes: 17 additions & 17 deletions pkg/executor/infoschema_reader_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -607,7 +607,7 @@ func TestColumnTable(t *testing.T) {
testkit.RowsWithSep("|",
"test|tbl1|col_2"))
tk.MustQuery(`select count(*) from information_schema.columns;`).Check(
testkit.RowsWithSep("|", "4944"))
testkit.RowsWithSep("|", "4961"))
}

func TestIndexUsageTable(t *testing.T) {
Expand Down Expand Up @@ -654,7 +654,7 @@ func TestIndexUsageTable(t *testing.T) {
testkit.RowsWithSep("|",
"test|idt2|idx_4"))
tk.MustQuery(`select count(*) from information_schema.tidb_index_usage;`).Check(
testkit.RowsWithSep("|", "73"))
testkit.RowsWithSep("|", "77"))

tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage
where TABLE_SCHEMA = 'test1';`).Check(testkit.Rows())
Expand Down Expand Up @@ -857,22 +857,22 @@ func TestInfoSchemaDDLJobs(t *testing.T) {
tk2 := testkit.NewTestKit(t, store)
tk2.MustQuery(`SELECT JOB_ID, JOB_TYPE, SCHEMA_STATE, SCHEMA_ID, TABLE_ID, table_name, STATE
FROM information_schema.ddl_jobs WHERE table_name = "t1";`).Check(testkit.RowsWithSep("|",
"127|add index /* txn-merge */|public|120|125|t1|synced",
"126|create table|public|120|125|t1|synced",
"113|add index /* txn-merge */|public|106|111|t1|synced",
"112|create table|public|106|111|t1|synced",
"131|add index /* txn-merge */|public|124|129|t1|synced",
"130|create table|public|124|129|t1|synced",
"117|add index /* txn-merge */|public|110|115|t1|synced",
"116|create table|public|110|115|t1|synced",
))
tk2.MustQuery(`SELECT JOB_ID, JOB_TYPE, SCHEMA_STATE, SCHEMA_ID, TABLE_ID, table_name, STATE
FROM information_schema.ddl_jobs WHERE db_name = "d1" and JOB_TYPE LIKE "add index%%";`).Check(testkit.RowsWithSep("|",
"133|add index /* txn-merge */|public|120|131|t3|synced",
"130|add index /* txn-merge */|public|120|128|t2|synced",
"127|add index /* txn-merge */|public|120|125|t1|synced",
"124|add index /* txn-merge */|public|120|122|t0|synced",
"137|add index /* txn-merge */|public|124|135|t3|synced",
"134|add index /* txn-merge */|public|124|132|t2|synced",
"131|add index /* txn-merge */|public|124|129|t1|synced",
"128|add index /* txn-merge */|public|124|126|t0|synced",
))
tk2.MustQuery(`SELECT JOB_ID, JOB_TYPE, SCHEMA_STATE, SCHEMA_ID, TABLE_ID, table_name, STATE
FROM information_schema.ddl_jobs WHERE db_name = "d0" and table_name = "t3";`).Check(testkit.RowsWithSep("|",
"119|add index /* txn-merge */|public|106|117|t3|synced",
"118|create table|public|106|117|t3|synced",
"123|add index /* txn-merge */|public|110|121|t3|synced",
"122|create table|public|110|121|t3|synced",
))
tk2.MustQuery(`SELECT JOB_ID, JOB_TYPE, SCHEMA_STATE, SCHEMA_ID, TABLE_ID, table_name, STATE
FROM information_schema.ddl_jobs WHERE state = "running";`).Check(testkit.Rows())
Expand All @@ -883,15 +883,15 @@ func TestInfoSchemaDDLJobs(t *testing.T) {
if job.SchemaState == model.StateWriteOnly && loaded.CompareAndSwap(false, true) {
tk2.MustQuery(`SELECT JOB_ID, JOB_TYPE, SCHEMA_STATE, SCHEMA_ID, TABLE_ID, table_name, STATE
FROM information_schema.ddl_jobs WHERE table_name = "t0" and state = "running";`).Check(testkit.RowsWithSep("|",
"134|add index /* txn-merge */|write only|106|108|t0|running",
"138 add index /* txn-merge */ write only 110 112 t0 running",
))
tk2.MustQuery(`SELECT JOB_ID, JOB_TYPE, SCHEMA_STATE, SCHEMA_ID, TABLE_ID, table_name, STATE
FROM information_schema.ddl_jobs WHERE db_name = "d0" and state = "running";`).Check(testkit.RowsWithSep("|",
"134|add index /* txn-merge */|write only|106|108|t0|running",
"138 add index /* txn-merge */ write only 110 112 t0 running",
))
tk2.MustQuery(`SELECT JOB_ID, JOB_TYPE, SCHEMA_STATE, SCHEMA_ID, TABLE_ID, table_name, STATE
FROM information_schema.ddl_jobs WHERE state = "running";`).Check(testkit.RowsWithSep("|",
"134|add index /* txn-merge */|write only|106|108|t0|running",
"138 add index /* txn-merge */ write only 110 112 t0 running",
))
}
})
Expand All @@ -907,8 +907,8 @@ func TestInfoSchemaDDLJobs(t *testing.T) {
tk.MustExec("create table test2.t1(id int)")
tk.MustQuery(`SELECT JOB_ID, JOB_TYPE, SCHEMA_STATE, SCHEMA_ID, TABLE_ID, table_name, STATE
FROM information_schema.ddl_jobs WHERE db_name = "test2" and table_name = "t1"`).Check(testkit.RowsWithSep("|",
"143|create table|public|140|142|t1|synced",
"138|create table|public|135|137|t1|synced",
"147|create table|public|144|146|t1|synced",
"142|create table|public|139|141|t1|synced",
))

// Test explain output, since the output may change in future.
Expand Down
8 changes: 4 additions & 4 deletions pkg/expression/integration_test/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1116,17 +1116,17 @@ func TestTiDBEncodeKey(t *testing.T) {
err := tk.QueryToErr("select tidb_encode_record_key('test', 't1', 0);")
require.ErrorContains(t, err, "doesn't exist")
tk.MustQuery("select tidb_encode_record_key('test', 't', 1);").
Check(testkit.Rows("74800000000000006a5f728000000000000001"))
Check(testkit.Rows("74800000000000006e5f728000000000000001"))

tk.MustExec("alter table t add index i(b);")
err = tk.QueryToErr("select tidb_encode_index_key('test', 't', 'i1', 1);")
require.ErrorContains(t, err, "index not found")
tk.MustQuery("select tidb_encode_index_key('test', 't', 'i', 1, 1);").
Check(testkit.Rows("74800000000000006a5f698000000000000001038000000000000001038000000000000001"))
Check(testkit.Rows("74800000000000006e5f698000000000000001038000000000000001038000000000000001"))

tk.MustExec("create table t1 (a int primary key, b int) partition by hash(a) partitions 4;")
tk.MustExec("insert into t1 values (1, 1);")
tk.MustQuery("select tidb_encode_record_key('test', 't1(p1)', 1);").Check(testkit.Rows("74800000000000006f5f728000000000000001"))
tk.MustQuery("select tidb_encode_record_key('test', 't1(p1)', 1);").Check(testkit.Rows("7480000000000000735f728000000000000001"))
rs := tk.MustQuery("select tidb_mvcc_info('74800000000000006f5f728000000000000001');")
mvccInfo := rs.Rows()[0][0].(string)
require.NotEqual(t, mvccInfo, `{"info":{}}`)
Expand All @@ -1143,7 +1143,7 @@ func TestTiDBEncodeKey(t *testing.T) {
err = tk2.QueryToErr("select tidb_encode_index_key('test', 't', 'i1', 1);")
require.ErrorContains(t, err, "SELECT command denied")
tk.MustExec("grant select on test.t1 to 'alice'@'%';")
tk2.MustQuery("select tidb_encode_record_key('test', 't1(p1)', 1);").Check(testkit.Rows("74800000000000006f5f728000000000000001"))
tk2.MustQuery("select tidb_encode_record_key('test', 't1(p1)', 1);").Check(testkit.Rows("7480000000000000735f728000000000000001"))
}

func TestIssue9710(t *testing.T) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/indexadvisor/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ go_test(
"utils_test.go",
],
flaky = True,
shard_count = 33,
shard_count = 34,
deps = [
":indexadvisor",
"//pkg/parser/mysql",
Expand Down
44 changes: 41 additions & 3 deletions pkg/planner/indexadvisor/indexadvisor.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ package indexadvisor

import (
"context"
"encoding/json"
"errors"
"fmt"
"math"
Expand Down Expand Up @@ -86,6 +87,8 @@ func AdviseIndexes(ctx context.Context, sctx sessionctx.Context,
return nil, err
}

saveRecommendations(sctx, results)

return results, nil
}

Expand Down Expand Up @@ -175,6 +178,7 @@ func prepareRecommendation(indexes s.Set[Index], queries s.Set[Query], optimizer
Database: idx.SchemaName,
Table: idx.TableName,
IndexColumns: cols,
IndexDetail: new(IndexDetail),
}

// generate a graceful index name
Expand All @@ -187,7 +191,7 @@ func prepareRecommendation(indexes s.Set[Index], queries s.Set[Query], optimizer
advisorLogger().Info("show index stats failed", zap.Error(err))
return nil, err
}
indexResult.IndexSize = uint64(indexSize)
indexResult.IndexDetail.IndexSize = uint64(indexSize)

// calculate the improvements
var workloadCostBefore, workloadCostAfter float64
Expand Down Expand Up @@ -241,8 +245,8 @@ func prepareRecommendation(indexes s.Set[Index], queries s.Set[Query], optimizer

normText, _ := NormalizeDigest(indexResult.TopImpactedQueries[0].Query)
indexResult.WorkloadImpact = workloadImpact
indexResult.Reason = fmt.Sprintf(`Column %v appear in Equal or
Range Predicate clause(s) in query '%v'`, cols, normText)
indexResult.IndexDetail.Reason =
fmt.Sprintf(`Column %v appear in Equal or Range Predicate clause(s) in query: %v`, cols, normText)
results = append(results, indexResult)
}
return results, nil
Expand Down Expand Up @@ -278,3 +282,37 @@ func gracefulIndexName(opt Optimizer, schema, tableName string, cols []string) s
}
return indexName
}

func saveRecommendations(sctx sessionctx.Context, results []*Recommendation) {
for _, r := range results {
q, err := json.Marshal(r.TopImpactedQueries)
if err != nil {
advisorLogger().Error("marshal top impacted queries failed", zap.Error(err))
continue
}
w, err := json.Marshal(r.WorkloadImpact)
if err != nil {
advisorLogger().Error("marshal workload impact failed", zap.Error(err))
continue
}
d, err := json.Marshal(r.IndexDetail)
if err != nil {
advisorLogger().Error("marshal index detail failed", zap.Error(err))
continue
}

template := `insert into mysql.index_advisor_results (
created_at, updated_at, schema_name, table_name, index_name,
index_columns, index_details, top_impacted_queries, workload_impact, extra) values
(now(), now(), %?, %?, %?, %?, %?, %?, %?, null)
on duplicate key update
updated_at=now(), index_details=%?, top_impacted_queries=%?, workload_impact=%?`

if _, err := exec(sctx, template, r.Database, r.Table,
r.IndexName, strings.Join(r.IndexColumns, ","),
json.RawMessage(d), json.RawMessage(q), json.RawMessage(w),
json.RawMessage(d), json.RawMessage(q), json.RawMessage(w)); err != nil {
advisorLogger().Error("save advise result failed", zap.Error(err))
}
}
}
30 changes: 30 additions & 0 deletions pkg/planner/indexadvisor/indexadvisor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -385,3 +385,33 @@ FROM (SELECT block_number AS block_receipts
require.NoError(t, err)
require.True(t, len(r) > 0)
}

func TestIndexAdvisorStorage(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec(`use test`)
tk.MustExec(`create table t (a int, b int, c int, d varchar(32))`)
q := `select index_columns, index_details->'$.Reason' from mysql.index_advisor_results`

tk.MustQuery(`recommend index run for "select a from t where a=1"`)
tk.MustQuery(q).Sort().Check(testkit.Rows(
"a \"Column [a] appear in Equal or Range Predicate clause(s) in query: select `a` from `test` . `t` where `a` = ?\""))

tk.MustQuery(`recommend index run for "select b from t where b=1"`)
tk.MustQuery(q).Sort().Check(testkit.Rows(
"a \"Column [a] appear in Equal or Range Predicate clause(s) in query: select `a` from `test` . `t` where `a` = ?\"",
"b \"Column [b] appear in Equal or Range Predicate clause(s) in query: select `b` from `test` . `t` where `b` = ?\""))

tk.MustQuery(`recommend index run for "select d from t where d='x'"`)
tk.MustQuery(q).Sort().Check(testkit.Rows(
"a \"Column [a] appear in Equal or Range Predicate clause(s) in query: select `a` from `test` . `t` where `a` = ?\"",
"b \"Column [b] appear in Equal or Range Predicate clause(s) in query: select `b` from `test` . `t` where `b` = ?\"",
"d \"Column [d] appear in Equal or Range Predicate clause(s) in query: select `d` from `test` . `t` where `d` = ?\""))

tk.MustQuery(`recommend index run for "select c, b from t where c=1 and b=1"`)
tk.MustQuery(q).Sort().Check(testkit.Rows(
"a \"Column [a] appear in Equal or Range Predicate clause(s) in query: select `a` from `test` . `t` where `a` = ?\"",
"b \"Column [b] appear in Equal or Range Predicate clause(s) in query: select `b` from `test` . `t` where `b` = ?\"",
"b,c \"Column [b c] appear in Equal or Range Predicate clause(s) in query: select `c` , `b` from `test` . `t` where `c` = ? and `b` = ?\"",
"d \"Column [d] appear in Equal or Range Predicate clause(s) in query: select `d` from `test` . `t` where `d` = ?\""))
}
9 changes: 7 additions & 2 deletions pkg/planner/indexadvisor/model.go
Original file line number Diff line number Diff line change
Expand Up @@ -146,14 +146,19 @@ type WorkloadImpact struct {
WorkloadImprovement float64
}

// IndexDetail represents the detail of the index.
type IndexDetail struct {
Reason string // why recommend this index
IndexSize uint64 // byte
}

// Recommendation represents the result of the index advisor.
type Recommendation struct {
Database string
Table string
IndexName string
IndexColumns []string
IndexSize uint64
Reason string
IndexDetail *IndexDetail
WorkloadImpact *WorkloadImpact
TopImpactedQueries []*ImpactedQuery
}
4 changes: 2 additions & 2 deletions pkg/planner/indexadvisor/utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -530,10 +530,10 @@ func evaluateIndexSetCost(
return IndexSetCost{workloadCost, totCols, strings.Join(keys, ",")}, nil
}

func exec(sctx sessionctx.Context, sql string) (ret []chunk.Row, err error) {
func exec(sctx sessionctx.Context, sql string, args ...any) (ret []chunk.Row, err error) {
executor := sctx.(sqlexec.SQLExecutor)
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats)
result, err := executor.ExecuteInternal(ctx, sql)
result, err := executor.ExecuteInternal(ctx, sql, args...)
if err != nil {
return nil, fmt.Errorf("execute %v failed: %v", sql, err)
}
Expand Down
Loading

0 comments on commit 8106d93

Please sign in to comment.