From 2e4ba13f48d50226077968de97c81aad23576a6e Mon Sep 17 00:00:00 2001 From: guoni Date: Sun, 1 Mar 2020 20:41:30 +0800 Subject: [PATCH 01/33] add disk_usage table --- executor/builder.go | 9 ++ executor/infoschema_reader.go | 153 +++++++++++++++++++ executor/infoschema_reader_test.go | 25 +++ infoschema/tables.go | 11 ++ planner/core/logical_plan_builder.go | 2 + planner/core/memtable_predicate_extractor.go | 35 +++++ 6 files changed, 235 insertions(+) diff --git a/executor/builder.go b/executor/builder.go index 6d30b3ec2e8cf..e2e040eca476d 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1366,6 +1366,15 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo extractor: v.Extractor.(*plannercore.SlowQueryExtractor), }, } + case strings.ToLower(infoschema.TableDiskUsage): + return &MemTableReaderExec{ + baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()), + retriever: &diskUsageRetriever{ + table: v.Table, + outputCols: v.Columns, + extractor: v.Extractor.(*plannercore.DiskUsageExtractor), + }, + } } } tb, _ := b.is.TableByID(v.Table.ID) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index ef78bb9ce872a..71e268e2f561d 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -15,14 +15,23 @@ package executor import ( "context" + "encoding/json" + "fmt" + "net/http" + "net/url" "sort" + "github.com/pingcap/errors" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/infoschema" + plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/store/tikv" + "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/codec" ) type memtableRetriever struct { @@ -153,3 +162,147 @@ func dataForViews(ctx sessionctx.Context, schemas []*model.DBInfo) ([][]types.Da } return rows, nil } + +//slowQueryRetriever is used to read slow log data. +type diskUsageRetriever struct { + dummyCloser + table *model.TableInfo + outputCols []*model.ColumnInfo + retrieved bool + initialized bool + extractor *plannercore.DiskUsageExtractor + initialTables []*initialTable + curTable int +} + +func (e *diskUsageRetriever) retrieve(ctx context.Context, sctx sessionctx.Context) ([][]types.Datum, error) { + if e.retrieved { + return nil, nil + } + if !e.initialized { + err := e.initialize(sctx) + if err != nil { + return nil, err + } + } + if len(e.initialTables) == 0 || e.curTable >= len(e.initialTables) { + e.retrieved = true + return nil, nil + } + + rows, err := e.dataForDiskUsage(sctx) + if err != nil { + return nil, err + } + if len(e.outputCols) == len(e.table.Columns) { + return rows, nil + } + retRows := make([][]types.Datum, len(rows)) + for i, fullRow := range rows { + row := make([]types.Datum, len(e.outputCols)) + for j, col := range e.outputCols { + row[j] = fullRow[col.Offset] + } + retRows[i] = row + } + return retRows, nil +} + +type initialTable struct { + db string + tb *model.TableInfo +} + +func (e *diskUsageRetriever) initialize(sctx sessionctx.Context) error { + is := infoschema.GetInfoSchema(sctx) + dbs := is.AllSchemas() + sort.Sort(infoschema.SchemasSorter(dbs)) + schemas := e.extractor.TableSchema + tables := e.extractor.TableName + var initialTables []initialTable + if schemas != nil || tables != nil { + for _, schema := range dbs { + if schemas.Exist(schema.Name.L) { + for _, table := range schema.Tables { + if tables.Exist(table.Name.L) { + initialTable := initialTable{db: schema.Name.L, tb: table} + initialTables = append(initialTables, initialTable) + } + } + } + } + if len(initialTables) == 0 && tables == nil { + return errors.Errorf("schema or table not exist, please check the schema and table") + } + } + for _, schema := range dbs { + for _, table := range schema.Tables { + e.initialTables = append(e.initialTables, &initialTable{schema.Name.O, table}) + } + } + e.initialized = true + return nil +} + +// pdRegionStats is the json response from PD. +type pdRegionStats struct { + Count int `json:"count"` + EmptyCount int `json:"empty_count"` + StorageSize int64 `json:"storage_size"` + StoreLeaderCount map[uint64]int `json:"store_leader_count"` + StorePeerCount map[uint64]int `json:"store_peer_count"` + StoreLeaderSize map[uint64]int64 `json:"store_leader_size"` + StorePeerSize map[uint64]int64 `json:"store_peer_size"` +} + +func (e *diskUsageRetriever) dataForDiskUsage(ctx sessionctx.Context) ([][]types.Datum, error) { + rows := make([][]types.Datum, 0, 1024) + tikvStore, ok := ctx.GetStore().(tikv.Storage) + if !ok { + return nil, errors.New("Information about TiKV region status can be gotten only when the storage is TiKV") + } + var pdAddrs []string + etcd, ok := tikvStore.(tikv.EtcdBackend) + if !ok { + return nil, errors.New("not implemented") + } + pdAddrs = etcd.EtcdAddrs() + if len(pdAddrs) < 0 { + return nil, errors.New("pd unavailable") + } + count := 0 + for i := e.curTable; e.curTable <= len(e.initialTables) && count < 1024; i++ { + table := (e.initialTables)[e.curTable] + tableID := table.tb.ID + // Include table and index data, because their range located in tableID_i tableID_r + startKey := tablecodec.EncodeTablePrefix(tableID) + endKey := tablecodec.EncodeTablePrefix(tableID + 1) + startKey = codec.EncodeBytes([]byte{}, startKey) + endKey = codec.EncodeBytes([]byte{}, endKey) + + statURL := fmt.Sprintf("http://%s/pd/api/v1/stats/region?start_key=%s&end_key=%s", + pdAddrs[0], + url.QueryEscape(string(startKey)), + url.QueryEscape(string(endKey))) + + resp, err := http.Get(statURL) + if err != nil { + return nil, err + } + var stats pdRegionStats + dec := json.NewDecoder(resp.Body) + if err := dec.Decode(&stats); err != nil { + return nil, err + } + record := types.MakeDatums( + table.db, // TABLE_SCHEMA + table.tb.Name.O, // TABLE_NAME + tableID, // TABLE_ID + stats.StorageSize, //DISK_USAGE + ) + rows = append(rows, record) + count++ + e.curTable++ + } + return rows, nil +} diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index 70fe4b1083712..2b3e5305a728c 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/util/testkit" + "strconv" ) var _ = Suite(&testInfoschemaTableSuite{}) @@ -83,3 +84,27 @@ func (s *testInfoschemaTableSuite) TestViews(c *C) { tk.MustQuery("SELECT * FROM information_schema.views WHERE table_schema='test' AND table_name='v1'").Check(testkit.Rows("def test v1 SELECT 1 CASCADED NO root@localhost DEFINER utf8mb4 utf8mb4_bin")) tk.MustQuery("SELECT table_catalog, table_schema, table_name, table_type, engine, version, row_format, table_rows, avg_row_length, data_length, max_data_length, index_length, data_free, auto_increment, update_time, check_time, table_collation, checksum, create_options, table_comment FROM information_schema.tables WHERE table_schema='test' AND table_name='v1'").Check(testkit.Rows("def test v1 VIEW VIEW")) } + +func (s *testInfoschemaTableSuite) TestDiskUsage(c *C) { + tk := testkit.NewTestKit(c, s.store) + + // use both schema and table in where + tk.MustQuery("select table_schema,table_name FROM information_schema.DISK_USAGE WHERE table_schema = 'information_schema' and table_name = 'schemata'").Check( + testkit.Rows("information_schema schemata")) + //tk.MustQuery("select disk_usage FROM information_schema.DISK_USAGE WHERE table_schema = 'information_schema' and table_name = 'schemata'").Check( + // testkit.Rows("")) + diskUsage, err := strconv.Atoi(tk.MustQuery("select disk_usage FROM information_schema.DISK_USAGE WHERE table_schema = 'information_schema' and table_name = 'schemata'").Rows()[0][0].(string)) + c.Assert(err, IsNil) + c.Assert(diskUsage, Greater, 0) + + //use only table in where + tk.MustQuery("select table_schema,table_name FROM information_schema.DISK_USAGE WHERE table_name = 'schemata'").Check( + testkit.Rows("information_schema schemata")) + diskUsage, err = strconv.Atoi(tk.MustQuery("select disk_usage FROM information_schema.DISK_USAGE WHERE table_name = 'schemata'").Rows()[0][0].(string)) + c.Assert(err, IsNil) + c.Assert(diskUsage, Greater, 0) + + //use not exist table in where + err = tk.QueryToErr("select table_schema,table_name FROM information_schema.DISK_USAGE WHERE table_name = 'notExit'") + c.Assert(err, NotNil) +} diff --git a/infoschema/tables.go b/infoschema/tables.go index 66c585e25902f..288dcd77a846a 100755 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -120,6 +120,8 @@ const ( TableMetricSummaryByLabel = "METRICS_SUMMARY_BY_LABEL" // TableInspectionSummary is the string constant of inspection summary table TableInspectionSummary = "INSPECTION_SUMMARY" + // TableDiskUsage is a table that contains all tables disk usage + TableDiskUsage = "DISK_USAGE" ) var tableIDMap = map[string]int64{ @@ -178,6 +180,7 @@ var tableIDMap = map[string]int64{ TableMetricSummaryByLabel: autoid.InformationSchemaDBID + 53, TableMetricTables: autoid.InformationSchemaDBID + 54, TableInspectionSummary: autoid.InformationSchemaDBID + 55, + TableDiskUsage: autoid.InformationSchemaDBID + 56, } type columnInfo struct { @@ -956,6 +959,13 @@ var tableMetricSummaryByLabelCols = []columnInfo{ {name: "COMMENT", tp: mysql.TypeVarchar, size: 256}, } +var TableDiskUsageCols = []columnInfo{ + {name: "TABLE_SCHEMA", tp: mysql.TypeVarchar, size: 64}, + {name: "TABLE_NAME", tp: mysql.TypeVarchar, size: 64}, + {name: "TABLE_ID", tp: mysql.TypeLonglong, size: 21}, + {name: "DISK_USAGE", tp: mysql.TypeLonglong, size: 64}, +} + func dataForTiKVRegionStatus(ctx sessionctx.Context) (records [][]types.Datum, err error) { tikvStore, ok := ctx.GetStore().(tikv.Storage) if !ok { @@ -2452,6 +2462,7 @@ var tableNameToColumns = map[string][]columnInfo{ TableMetricSummaryByLabel: tableMetricSummaryByLabelCols, TableMetricTables: tableMetricTablesCols, TableInspectionSummary: tableInspectionSummaryCols, + TableDiskUsage: TableDiskUsageCols, } func createInfoSchemaTable(_ autoid.Allocators, meta *model.TableInfo) (table.Table, error) { diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 03efedf93f107..922b3993cbda1 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -2842,6 +2842,8 @@ func (b *PlanBuilder) buildMemTable(_ context.Context, dbName model.CIStr, table p.QueryTimeRange = b.timeRangeForSummaryTable() case infoschema.TableSlowQuery: p.Extractor = &SlowQueryExtractor{} + case infoschema.TableDiskUsage: + p.Extractor = &DiskUsageExtractor{} } } return p, nil diff --git a/planner/core/memtable_predicate_extractor.go b/planner/core/memtable_predicate_extractor.go index 6a775e602f239..001b65820f6e1 100644 --- a/planner/core/memtable_predicate_extractor.go +++ b/planner/core/memtable_predicate_extractor.go @@ -753,3 +753,38 @@ func (e *SlowQueryExtractor) setTimeRange(start, end int64) { e.StartTime, e.EndTime = startTime, endTime e.Enable = true } + +// SlowQueryExtractor is used to extract some predicates of `slow_query` +type DiskUsageExtractor struct { + extractHelper + // SkipRequest means the where clause always false, we don't need to request any component + SkipRequest bool + //usage set.StringSet + // TableSchema represents tableSchema applied to, and we should apply all table disk usage if there is no schema specified + // e.g: SELECT * FROM information_schema.disk_usage WHERE table_schema in ('test', 'information_schema') + TableSchema set.StringSet + // TableName represents tableName applied to, and we should apply all table disk usage if there is no table specified + // e.g: SELECT * FROM information_schema.disk_usage WHERE table in ('schemata', 'tables') + TableName set.StringSet +} + +// Extract implements the MemTablePredicateExtractor Extract interface +func (e *DiskUsageExtractor) Extract( + _ sessionctx.Context, + schema *expression.Schema, + names []*types.FieldName, + predicates []expression.Expression, +) []expression.Expression { + // Extract the `rule` columns + remained, schemaSkip, tableSchema := e.extractCol(schema, names, predicates, "table_schema", true) + remained, tableSkip, tableName := e.extractCol(schema, names, predicates, "table_name", true) + //remained, usageSkip, diskUsage := e.extractCol(schema, names, predicates, "table_schema", true) + e.SkipRequest = schemaSkip || tableSkip + if e.SkipRequest { + return nil + } + e.TableSchema = tableSchema + e.TableName = tableName + //e.usage = diskUsage + return remained +} From 07a29b2432bdbad66040636d1eeedac8784bd2f8 Mon Sep 17 00:00:00 2001 From: reafans Date: Mon, 2 Mar 2020 03:27:02 +0800 Subject: [PATCH 02/33] fix bug --- executor/infoschema_reader.go | 41 +++++++++++++++++++++--------- executor/infoschema_reader_test.go | 13 +++------- 2 files changed, 33 insertions(+), 21 deletions(-) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 71e268e2f561d..18cb46a9a241d 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -219,25 +219,42 @@ func (e *diskUsageRetriever) initialize(sctx sessionctx.Context) error { sort.Sort(infoschema.SchemasSorter(dbs)) schemas := e.extractor.TableSchema tables := e.extractor.TableName - var initialTables []initialTable - if schemas != nil || tables != nil { + //var initialTables []initialTable + if len(schemas) != 0 { for _, schema := range dbs { if schemas.Exist(schema.Name.L) { - for _, table := range schema.Tables { - if tables.Exist(table.Name.L) { - initialTable := initialTable{db: schema.Name.L, tb: table} - initialTables = append(initialTables, initialTable) + if len(tables) != 0 { + for _, table := range schema.Tables { + if tables.Exist(table.Name.L) { + e.initialTables = append(e.initialTables, &initialTable{db: schema.Name.L, tb: table}) + } + } + } else { + for _, table := range schema.Tables { + e.initialTables = append(e.initialTables, &initialTable{db: schema.Name.L, tb: table}) } } } + } - if len(initialTables) == 0 && tables == nil { + if len(e.initialTables) == 0 && tables == nil { return errors.Errorf("schema or table not exist, please check the schema and table") } - } - for _, schema := range dbs { - for _, table := range schema.Tables { - e.initialTables = append(e.initialTables, &initialTable{schema.Name.O, table}) + } else { + if len(tables) != 0 { + for _, schema := range dbs { + for _, table := range schema.Tables { + if tables.Exist(table.Name.L) { + e.initialTables = append(e.initialTables, &initialTable{db: schema.Name.L, tb: table}) + } + } + } + } else { + for _, schema := range dbs { + for _, table := range schema.Tables { + e.initialTables = append(e.initialTables, &initialTable{schema.Name.O, table}) + } + } } } e.initialized = true @@ -271,7 +288,7 @@ func (e *diskUsageRetriever) dataForDiskUsage(ctx sessionctx.Context) ([][]types return nil, errors.New("pd unavailable") } count := 0 - for i := e.curTable; e.curTable <= len(e.initialTables) && count < 1024; i++ { + for i := e.curTable; e.curTable < len(e.initialTables) && count < 1024; i++ { table := (e.initialTables)[e.curTable] tableID := table.tb.ID // Include table and index data, because their range located in tableID_i tableID_r diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index 2b3e5305a728c..21224e2d88ce9 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -89,22 +89,17 @@ func (s *testInfoschemaTableSuite) TestDiskUsage(c *C) { tk := testkit.NewTestKit(c, s.store) // use both schema and table in where - tk.MustQuery("select table_schema,table_name FROM information_schema.DISK_USAGE WHERE table_schema = 'information_schema' and table_name = 'schemata'").Check( + tk.MustQuery("SELECT table_schema,table_name FROM information_schema.disk_usage WHERE table_schema='information_schema' AND table_name='schemata'").Check( testkit.Rows("information_schema schemata")) - //tk.MustQuery("select disk_usage FROM information_schema.DISK_USAGE WHERE table_schema = 'information_schema' and table_name = 'schemata'").Check( - // testkit.Rows("")) diskUsage, err := strconv.Atoi(tk.MustQuery("select disk_usage FROM information_schema.DISK_USAGE WHERE table_schema = 'information_schema' and table_name = 'schemata'").Rows()[0][0].(string)) c.Assert(err, IsNil) c.Assert(diskUsage, Greater, 0) //use only table in where - tk.MustQuery("select table_schema,table_name FROM information_schema.DISK_USAGE WHERE table_name = 'schemata'").Check( - testkit.Rows("information_schema schemata")) - diskUsage, err = strconv.Atoi(tk.MustQuery("select disk_usage FROM information_schema.DISK_USAGE WHERE table_name = 'schemata'").Rows()[0][0].(string)) - c.Assert(err, IsNil) - c.Assert(diskUsage, Greater, 0) + err = tk.QueryToErr("select * FROM information_schema.DISK_USAGE WHERE table_name = 'schemata'") + c.Assert(err, NotNil) //use not exist table in where - err = tk.QueryToErr("select table_schema,table_name FROM information_schema.DISK_USAGE WHERE table_name = 'notExit'") + err = tk.QueryToErr("select table_schema,table_name FROM information_schema.DISK_USAGE WHERE table_schema = 'test' and table_name = 'notExit'") c.Assert(err, NotNil) } From 6fe158361097a6ff525b07b787aaa527ad3d8c26 Mon Sep 17 00:00:00 2001 From: reafans Date: Mon, 2 Mar 2020 10:51:46 +0800 Subject: [PATCH 03/33] fix bug --- executor/infoschema_reader_test.go | 20 -------------------- 1 file changed, 20 deletions(-) diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index 21224e2d88ce9..70fe4b1083712 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -19,7 +19,6 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/util/testkit" - "strconv" ) var _ = Suite(&testInfoschemaTableSuite{}) @@ -84,22 +83,3 @@ func (s *testInfoschemaTableSuite) TestViews(c *C) { tk.MustQuery("SELECT * FROM information_schema.views WHERE table_schema='test' AND table_name='v1'").Check(testkit.Rows("def test v1 SELECT 1 CASCADED NO root@localhost DEFINER utf8mb4 utf8mb4_bin")) tk.MustQuery("SELECT table_catalog, table_schema, table_name, table_type, engine, version, row_format, table_rows, avg_row_length, data_length, max_data_length, index_length, data_free, auto_increment, update_time, check_time, table_collation, checksum, create_options, table_comment FROM information_schema.tables WHERE table_schema='test' AND table_name='v1'").Check(testkit.Rows("def test v1 VIEW VIEW")) } - -func (s *testInfoschemaTableSuite) TestDiskUsage(c *C) { - tk := testkit.NewTestKit(c, s.store) - - // use both schema and table in where - tk.MustQuery("SELECT table_schema,table_name FROM information_schema.disk_usage WHERE table_schema='information_schema' AND table_name='schemata'").Check( - testkit.Rows("information_schema schemata")) - diskUsage, err := strconv.Atoi(tk.MustQuery("select disk_usage FROM information_schema.DISK_USAGE WHERE table_schema = 'information_schema' and table_name = 'schemata'").Rows()[0][0].(string)) - c.Assert(err, IsNil) - c.Assert(diskUsage, Greater, 0) - - //use only table in where - err = tk.QueryToErr("select * FROM information_schema.DISK_USAGE WHERE table_name = 'schemata'") - c.Assert(err, NotNil) - - //use not exist table in where - err = tk.QueryToErr("select table_schema,table_name FROM information_schema.DISK_USAGE WHERE table_schema = 'test' and table_name = 'notExit'") - c.Assert(err, NotNil) -} From fc0d988a80b6054e7b9797c066d8903003256739 Mon Sep 17 00:00:00 2001 From: reafans Date: Mon, 2 Mar 2020 10:57:11 +0800 Subject: [PATCH 04/33] fix lint --- infoschema/tables.go | 4 ++-- planner/core/memtable_predicate_extractor.go | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/infoschema/tables.go b/infoschema/tables.go index 288dcd77a846a..57724cae8158d 100755 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -959,7 +959,7 @@ var tableMetricSummaryByLabelCols = []columnInfo{ {name: "COMMENT", tp: mysql.TypeVarchar, size: 256}, } -var TableDiskUsageCols = []columnInfo{ +var tableDiskUsageCols = []columnInfo{ {name: "TABLE_SCHEMA", tp: mysql.TypeVarchar, size: 64}, {name: "TABLE_NAME", tp: mysql.TypeVarchar, size: 64}, {name: "TABLE_ID", tp: mysql.TypeLonglong, size: 21}, @@ -2462,7 +2462,7 @@ var tableNameToColumns = map[string][]columnInfo{ TableMetricSummaryByLabel: tableMetricSummaryByLabelCols, TableMetricTables: tableMetricTablesCols, TableInspectionSummary: tableInspectionSummaryCols, - TableDiskUsage: TableDiskUsageCols, + TableDiskUsage: tableDiskUsageCols, } func createInfoSchemaTable(_ autoid.Allocators, meta *model.TableInfo) (table.Table, error) { diff --git a/planner/core/memtable_predicate_extractor.go b/planner/core/memtable_predicate_extractor.go index 001b65820f6e1..26fc50614377d 100644 --- a/planner/core/memtable_predicate_extractor.go +++ b/planner/core/memtable_predicate_extractor.go @@ -754,7 +754,7 @@ func (e *SlowQueryExtractor) setTimeRange(start, end int64) { e.Enable = true } -// SlowQueryExtractor is used to extract some predicates of `slow_query` +// DiskUsageExtractor is used to extract some predicates of `disk_usage` type DiskUsageExtractor struct { extractHelper // SkipRequest means the where clause always false, we don't need to request any component From 48a3ccd1974078fec3bff310fb6c88a31eaea35a Mon Sep 17 00:00:00 2001 From: reafans Date: Mon, 2 Mar 2020 11:07:38 +0800 Subject: [PATCH 05/33] fix bug --- planner/core/memtable_predicate_extractor.go | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/planner/core/memtable_predicate_extractor.go b/planner/core/memtable_predicate_extractor.go index 26fc50614377d..e182650e9bbdd 100644 --- a/planner/core/memtable_predicate_extractor.go +++ b/planner/core/memtable_predicate_extractor.go @@ -775,16 +775,15 @@ func (e *DiskUsageExtractor) Extract( names []*types.FieldName, predicates []expression.Expression, ) []expression.Expression { - // Extract the `rule` columns + // Extract the `table_schema` columns remained, schemaSkip, tableSchema := e.extractCol(schema, names, predicates, "table_schema", true) - remained, tableSkip, tableName := e.extractCol(schema, names, predicates, "table_name", true) - //remained, usageSkip, diskUsage := e.extractCol(schema, names, predicates, "table_schema", true) + // Extract the `table_name` columns + remained, tableSkip, tableName := e.extractCol(schema, names, remained, "table_name", true) e.SkipRequest = schemaSkip || tableSkip if e.SkipRequest { return nil } e.TableSchema = tableSchema e.TableName = tableName - //e.usage = diskUsage return remained } From fb65feef9dd9b57faa3c7aad0fb5c574e5e5b5ec Mon Sep 17 00:00:00 2001 From: guoni Date: Tue, 10 Mar 2020 23:45:30 +0800 Subject: [PATCH 06/33] fmt --- go.sum | 2 -- 1 file changed, 2 deletions(-) diff --git a/go.sum b/go.sum index ce1488dfee690..7d87fdbca861f 100644 --- a/go.sum +++ b/go.sum @@ -278,8 +278,6 @@ github.com/pingcap/sysutil v0.0.0-20200309085538-962fd285f3bb/go.mod h1:EB/852NM github.com/pingcap/tidb-tools v4.0.0-beta.1.0.20200306084441-875bd09aa3d5+incompatible h1:84F7MFMfdAYObrznvRslmVu43aoihrlL+7mMyMlOi0o= github.com/pingcap/tidb-tools v4.0.0-beta.1.0.20200306084441-875bd09aa3d5+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= -github.com/pingcap/tipb v0.0.0-20200212061130-c4d518eb1d60 h1:aJPXrT1u4VfUSGFA2oQVwl4pOXzqe+YI6wed01cjDH4= -github.com/pingcap/tipb v0.0.0-20200212061130-c4d518eb1d60/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1 h1:iURUrRGxPUNPdy5/HRSm+Yj6okJ6UtLINN0Q9M4+h3I= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= From fbedd2e5d29954cfa251a660214e7de382aa6f5c Mon Sep 17 00:00:00 2001 From: guoni Date: Sun, 15 Mar 2020 19:00:24 +0800 Subject: [PATCH 07/33] fix the sys table name --- executor/infoschema_reader.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 496a7c85c9392..29bab6a8df14a 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -809,8 +809,8 @@ func (e *memtableRetriever) setDataFromTableConstraints(ctx sessionctx.Context, e.rows = rows } -//slowQueryRetriever is used to read slow log data. -type diskUsageRetriever struct { +//tableStorageStatsRetriever is used to read slow log data. +type tableStorageStatsRetriever struct { dummyCloser table *model.TableInfo outputCols []*model.ColumnInfo @@ -821,7 +821,7 @@ type diskUsageRetriever struct { curTable int } -func (e *diskUsageRetriever) retrieve(ctx context.Context, sctx sessionctx.Context) ([][]types.Datum, error) { +func (e *tableStorageStatsRetriever) retrieve(ctx context.Context, sctx sessionctx.Context) ([][]types.Datum, error) { if e.retrieved { return nil, nil } @@ -859,7 +859,7 @@ type initialTable struct { tb *model.TableInfo } -func (e *diskUsageRetriever) initialize(sctx sessionctx.Context) error { +func (e *tableStorageStatsRetriever) initialize(sctx sessionctx.Context) error { is := infoschema.GetInfoSchema(sctx) dbs := is.AllSchemas() sort.Sort(infoschema.SchemasSorter(dbs)) @@ -918,7 +918,7 @@ type pdRegionStats struct { StorePeerSize map[uint64]int64 `json:"store_peer_size"` } -func (e *diskUsageRetriever) dataForDiskUsage(ctx sessionctx.Context) ([][]types.Datum, error) { +func (e *tableStorageStatsRetriever) dataForDiskUsage(ctx sessionctx.Context) ([][]types.Datum, error) { rows := make([][]types.Datum, 0, 1024) tikvStore, ok := ctx.GetStore().(tikv.Storage) if !ok { @@ -930,7 +930,7 @@ func (e *diskUsageRetriever) dataForDiskUsage(ctx sessionctx.Context) ([][]types return nil, errors.New("not implemented") } pdAddrs = etcd.EtcdAddrs() - if len(pdAddrs) < 0 { + if len(pdAddrs) == 0 { return nil, errors.New("pd unavailable") } count := 0 From 43d25ba766da2a0c6ceac9e640dd647ab43578cb Mon Sep 17 00:00:00 2001 From: guoni Date: Tue, 17 Mar 2020 19:51:42 +0800 Subject: [PATCH 08/33] refine dode --- executor/builder.go | 4 ++-- executor/infoschema_reader.go | 13 ++++++------- planner/core/logical_plan_builder.go | 2 +- planner/core/memtable_predicate_extractor.go | 10 +++++++--- 4 files changed, 16 insertions(+), 13 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 9c76414dfa9b2..f40355b69ef18 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1458,10 +1458,10 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo case strings.ToLower(infoschema.TableDiskUsage): return &MemTableReaderExec{ baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()), - retriever: &diskUsageRetriever{ + retriever: &tableStorageStatsRetriever{ table: v.Table, outputCols: v.Columns, - extractor: v.Extractor.(*plannercore.DiskUsageExtractor), + extractor: v.Extractor.(*plannercore.TableStorageStatsExtractor), }, } } diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 9d46fca10edff..ff5f82211b262 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1089,18 +1089,18 @@ func (e *memtableRetriever) setDataFromTableConstraints(ctx sessionctx.Context, } //slowQueryRetriever is used to read slow log data. -type diskUsageRetriever struct { +type tableStorageStatsRetriever struct { dummyCloser table *model.TableInfo outputCols []*model.ColumnInfo retrieved bool initialized bool - extractor *plannercore.DiskUsageExtractor + extractor *plannercore.TableStorageStatsExtractor initialTables []*initialTable curTable int } -func (e *diskUsageRetriever) retrieve(ctx context.Context, sctx sessionctx.Context) ([][]types.Datum, error) { +func (e *tableStorageStatsRetriever) retrieve(ctx context.Context, sctx sessionctx.Context) ([][]types.Datum, error) { if e.retrieved { return nil, nil } @@ -1115,7 +1115,7 @@ func (e *diskUsageRetriever) retrieve(ctx context.Context, sctx sessionctx.Conte return nil, nil } - rows, err := e.dataForDiskUsage(sctx) + rows, err := e.dataForTableStorageStats(sctx) if err != nil { return nil, err } @@ -1138,13 +1138,12 @@ type initialTable struct { tb *model.TableInfo } -func (e *diskUsageRetriever) initialize(sctx sessionctx.Context) error { +func (e *tableStorageStatsRetriever) initialize(sctx sessionctx.Context) error { is := infoschema.GetInfoSchema(sctx) dbs := is.AllSchemas() sort.Sort(infoschema.SchemasSorter(dbs)) schemas := e.extractor.TableSchema tables := e.extractor.TableName - //var initialTables []initialTable if len(schemas) != 0 { for _, schema := range dbs { if schemas.Exist(schema.Name.L) { @@ -1197,7 +1196,7 @@ type pdRegionStats struct { StorePeerSize map[uint64]int64 `json:"store_peer_size"` } -func (e *diskUsageRetriever) dataForDiskUsage(ctx sessionctx.Context) ([][]types.Datum, error) { +func (e *tableStorageStatsRetriever) dataForTableStorageStats(ctx sessionctx.Context) ([][]types.Datum, error) { rows := make([][]types.Datum, 0, 1024) tikvStore, ok := ctx.GetStore().(tikv.Storage) if !ok { diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 4c4f57d4249c7..8af7df6ec0a35 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -2846,7 +2846,7 @@ func (b *PlanBuilder) buildMemTable(_ context.Context, dbName model.CIStr, table case infoschema.TableSlowQuery: p.Extractor = &SlowQueryExtractor{} case infoschema.TableDiskUsage: - p.Extractor = &DiskUsageExtractor{} + p.Extractor = &TableStorageStatsExtractor{} } } return p, nil diff --git a/planner/core/memtable_predicate_extractor.go b/planner/core/memtable_predicate_extractor.go index 5d3087028fdfc..784e23f23419f 100644 --- a/planner/core/memtable_predicate_extractor.go +++ b/planner/core/memtable_predicate_extractor.go @@ -895,8 +895,8 @@ func (e *SlowQueryExtractor) setTimeRange(start, end int64) { e.Enable = true } -// DiskUsageExtractor is used to extract some predicates of `disk_usage` -type DiskUsageExtractor struct { +// TableStorageStatsExtractor is used to extract some predicates of `disk_usage` +type TableStorageStatsExtractor struct { extractHelper // SkipRequest means the where clause always false, we don't need to request any component SkipRequest bool @@ -910,7 +910,7 @@ type DiskUsageExtractor struct { } // Extract implements the MemTablePredicateExtractor Extract interface -func (e *DiskUsageExtractor) Extract( +func (e *TableStorageStatsExtractor) Extract( _ sessionctx.Context, schema *expression.Schema, names []*types.FieldName, @@ -929,6 +929,10 @@ func (e *DiskUsageExtractor) Extract( return remained } +func (e *TableStorageStatsExtractor) explainInfo(p *PhysicalMemTable) string { + return "" +} + func (e *SlowQueryExtractor) explainInfo(p *PhysicalMemTable) string { return "" } From 29c5d2421c9fb5caa495ba8bc6b2bf9975113e6e Mon Sep 17 00:00:00 2001 From: guoni Date: Thu, 19 Mar 2020 18:49:33 +0800 Subject: [PATCH 09/33] refine code --- executor/builder.go | 2 +- infoschema/tables.go | 10 +++++----- planner/core/logical_plan_builder.go | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index f40355b69ef18..5aff1e57ad3b6 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1455,7 +1455,7 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo extractor: v.Extractor.(*plannercore.SlowQueryExtractor), }, } - case strings.ToLower(infoschema.TableDiskUsage): + case strings.ToLower(infoschema.TableStorageStats): return &MemTableReaderExec{ baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()), retriever: &tableStorageStatsRetriever{ diff --git a/infoschema/tables.go b/infoschema/tables.go index cafaeafb32751..97b541bf90c93 100755 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -130,10 +130,10 @@ const ( TableMetricSummaryByLabel = "METRICS_SUMMARY_BY_LABEL" // TableInspectionSummary is the string constant of inspection summary table. TableInspectionSummary = "INSPECTION_SUMMARY" - // TableDiskUsage is a table that contains all tables disk usage - TableDiskUsage = "DISK_USAGE" // TableInspectionRules is the string constant of currently implemented inspection and summary rules. TableInspectionRules = "INSPECTION_RULES" + // TableStorageStats is a table that contains all tables disk usage + TableStorageStats = "TABLE_STORAGE_STATS" ) var tableIDMap = map[string]int64{ @@ -193,7 +193,7 @@ var tableIDMap = map[string]int64{ TableMetricTables: autoid.InformationSchemaDBID + 54, TableInspectionSummary: autoid.InformationSchemaDBID + 55, TableInspectionRules: autoid.InformationSchemaDBID + 56, - TableDiskUsage: autoid.InformationSchemaDBID + 57, + TableStorageStats: autoid.InformationSchemaDBID + 57, } type columnInfo struct { @@ -982,7 +982,7 @@ var tableMetricSummaryByLabelCols = []columnInfo{ {name: "COMMENT", tp: mysql.TypeVarchar, size: 256}, } -var tableDiskUsageCols = []columnInfo{ +var tableStorageStatsCols = []columnInfo{ {name: "TABLE_SCHEMA", tp: mysql.TypeVarchar, size: 64}, {name: "TABLE_NAME", tp: mysql.TypeVarchar, size: 64}, {name: "TABLE_ID", tp: mysql.TypeLonglong, size: 21}, @@ -1541,7 +1541,7 @@ var tableNameToColumns = map[string][]columnInfo{ TableMetricTables: tableMetricTablesCols, TableInspectionSummary: tableInspectionSummaryCols, TableInspectionRules: tableInspectionRulesCols, - TableDiskUsage: tableDiskUsageCols, + TableStorageStats: tableStorageStatsCols, } func createInfoSchemaTable(_ autoid.Allocators, meta *model.TableInfo) (table.Table, error) { diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 8af7df6ec0a35..972b2184e3ad1 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -2845,7 +2845,7 @@ func (b *PlanBuilder) buildMemTable(_ context.Context, dbName model.CIStr, table p.QueryTimeRange = b.timeRangeForSummaryTable() case infoschema.TableSlowQuery: p.Extractor = &SlowQueryExtractor{} - case infoschema.TableDiskUsage: + case infoschema.TableStorageStats: p.Extractor = &TableStorageStatsExtractor{} } } From 2225fad99bfcb0e70da9d2f4dd3dc84b240a4e07 Mon Sep 17 00:00:00 2001 From: guoni Date: Fri, 20 Mar 2020 14:16:39 +0800 Subject: [PATCH 10/33] add test --- executor/builder.go | 1 + executor/infoschema_reader.go | 27 +++++++++++++++++---------- executor/infoschema_reader_test.go | 30 ++++++++++++++++++++++++++++++ 3 files changed, 48 insertions(+), 10 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 5aff1e57ad3b6..8f858249861b7 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1458,6 +1458,7 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo case strings.ToLower(infoschema.TableStorageStats): return &MemTableReaderExec{ baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()), + table: v.Table, retriever: &tableStorageStatsRetriever{ table: v.Table, outputCols: v.Columns, diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index c986477c98f18..f28bb4c3ecfd8 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1115,7 +1115,7 @@ func (e *tableStorageStatsRetriever) retrieve(ctx context.Context, sctx sessionc return nil, nil } - rows, err := e.dataForTableStorageStats(sctx) + rows, err := e.setDataForTableStorageStats(sctx) if err != nil { return nil, err } @@ -1187,16 +1187,14 @@ func (e *tableStorageStatsRetriever) initialize(sctx sessionctx.Context) error { // pdRegionStats is the json response from PD. type pdRegionStats struct { - Count int `json:"count"` - EmptyCount int `json:"empty_count"` - StorageSize int64 `json:"storage_size"` - StoreLeaderCount map[uint64]int `json:"store_leader_count"` - StorePeerCount map[uint64]int `json:"store_peer_count"` - StoreLeaderSize map[uint64]int64 `json:"store_leader_size"` - StorePeerSize map[uint64]int64 `json:"store_peer_size"` + Count int `json:"count"` + EmptyCount int `json:"empty_count"` + StorageSize int64 `json:"storage_size"` + StorageKeys int64 `json:"storage_keys"` + StorePeerCount map[uint64]int `json:"store_peer_count"` } -func (e *tableStorageStatsRetriever) dataForTableStorageStats(ctx sessionctx.Context) ([][]types.Datum, error) { +func (e *tableStorageStatsRetriever) setDataForTableStorageStats(ctx sessionctx.Context) ([][]types.Datum, error) { rows := make([][]types.Datum, 0, 1024) tikvStore, ok := ctx.GetStore().(tikv.Storage) if !ok { @@ -1235,11 +1233,20 @@ func (e *tableStorageStatsRetriever) dataForTableStorageStats(ctx sessionctx.Con if err := dec.Decode(&stats); err != nil { return nil, err } + peerCount := 0 + for _, v := range stats.StorePeerCount { + peerCount = v + break + } record := types.MakeDatums( table.db, // TABLE_SCHEMA table.tb.Name.O, // TABLE_NAME tableID, // TABLE_ID - stats.StorageSize, //DISK_USAGE + peerCount, // TABLE_PEER_COUNT + stats.Count, // TABLE_REGION_COUNT + stats.EmptyCount, // TABLE_EMPTY_REGION_COUNT + stats.StorageSize, // TABLE_SIZE + stats.StorageKeys, // TABLE_KEYS ) rows = append(rows, record) count++ diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index 292baa1fab1d4..f8cadf54fb971 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -721,3 +721,33 @@ func (s *testInfoschemaClusterTableSuite) TestTiDBClusterInfo(c *C) { "tikv key3.key4.nest4 n-value5", )) } + +func (s *testInfoschemaTableSuite) TestTableStorageStats(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustQuery("select TABLE_NAME from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema' and TABLE_NAME='schemata';").Check( + testkit.Rows("schemata")) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int, b int, index idx(a))") + tk.MustQuery("select TABLE_NAME, STORAGE_SIZE from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'test' and TABLE_NAME='t';").Check( + testkit.Rows("t 1")) + + //test the privilege of new user for information_schema.table_constraints + tk.MustExec("create user table_storage_tester") + tableStorageTester := testkit.NewTestKit(c, s.store) + tableStorageTester.MustExec("use information_schema") + c.Assert(tableStorageTester.Se.Auth(&auth.UserIdentity{ + Username: "table_storage_tester", + Hostname: "127.0.0.1", + }, nil, nil), IsTrue) + + // Test the privilege. + tableStorageTester.MustQuery("select * from TABLE_STORAGE_STATS;").Check([][]interface{}{}) + tk.MustExec("CREATE ROLE r_stats_meta ;") + tk.MustExec("GRANT ALL PRIVILEGES ON mysql.stats_meta TO r_stats_meta;") + tk.MustExec("GRANT r_stats_meta TO table_storage_tester;") + tableStorageTester.MustExec("set role r_stats_meta") + c.Assert(len(tableStorageTester.MustQuery("select * from TABLE_STORAGE_STATS where TABLE_NAME='stats_meta';").Rows()), Greater, 0) +} From d043aa4ccf771bf4b5439e2c9a1f51a735e76f0c Mon Sep 17 00:00:00 2001 From: guoni Date: Sat, 21 Mar 2020 15:05:58 +0800 Subject: [PATCH 11/33] refine --- executor/infoschema_reader.go | 4 +- executor/infoschema_reader_test.go | 73 ++++++++++++++++++++---------- 2 files changed, 51 insertions(+), 26 deletions(-) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index f28bb4c3ecfd8..a01ff10db72e9 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1186,7 +1186,7 @@ func (e *tableStorageStatsRetriever) initialize(sctx sessionctx.Context) error { } // pdRegionStats is the json response from PD. -type pdRegionStats struct { +type PdRegionStats struct { Count int `json:"count"` EmptyCount int `json:"empty_count"` StorageSize int64 `json:"storage_size"` @@ -1228,7 +1228,7 @@ func (e *tableStorageStatsRetriever) setDataForTableStorageStats(ctx sessionctx. if err != nil { return nil, err } - var stats pdRegionStats + var stats PdRegionStats dec := json.NewDecoder(resp.Body) if err := dec.Decode(&stats); err != nil { return nil, err diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index f8cadf54fb971..d1a97805ed94d 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -618,6 +618,16 @@ func (s *testInfoschemaClusterTableSuite) setUpMockPDHTTPServer() (*httptest.Ser router.Handle(pdapi.Config, fn.Wrap(mockConfig)) // TiDB/TiKV config router.Handle("/config", fn.Wrap(mockConfig)) + // pd region + router.Handle(pdapi.Regions, fn.Wrap(func() (*executor.PdRegionStats, error) { + return &executor.PdRegionStats{ + Count: 1, + EmptyCount: 1, + StorageSize: 1, + StorageKeys: 1, + StorePeerCount: map[uint64]int{1: 1}, + }, nil + })) return server, mockAddr } @@ -722,32 +732,47 @@ func (s *testInfoschemaClusterTableSuite) TestTiDBClusterInfo(c *C) { )) } -func (s *testInfoschemaTableSuite) TestTableStorageStats(c *C) { +func (s *testInfoschemaClusterTableSuite) TestTableStorageStats(c *C) { tk := testkit.NewTestKit(c, s.store) + //err := tk.QueryToErr("select * from information_schema.TABLE_STORAGE_STATS") + //c.Assert(err, NotNil) + mockAddr := s.mockAddr + store := &mockStore{ + s.store.(tikv.Storage), + mockAddr, + } + tk = testkit.NewTestKit(c, store) + //tidbStatusAddr := fmt.Sprintf(":%d", config.GetGlobalConfig().Status.StatusPort) + //row := func(cols ...string) string { return strings.Join(cols, " ") } + //tk.MustQuery("select type, instance, status_address, version, git_hash from information_schema.cluster_info").Check(testkit.Rows( + // row("tidb", ":4000", tidbStatusAddr, "5.7.25-TiDB-None", "None"), + // row("pd", mockAddr, mockAddr, "4.0.0-alpha", "mock-pd-githash"), + // row("tikv", "127.0.0.1:20160", mockAddr, "4.0.0-alpha", "mock-tikv-githash"), + //)) tk.MustQuery("select TABLE_NAME from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema' and TABLE_NAME='schemata';").Check( testkit.Rows("schemata")) - - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t (a int, b int, index idx(a))") - tk.MustQuery("select TABLE_NAME, STORAGE_SIZE from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'test' and TABLE_NAME='t';").Check( - testkit.Rows("t 1")) - - //test the privilege of new user for information_schema.table_constraints - tk.MustExec("create user table_storage_tester") - tableStorageTester := testkit.NewTestKit(c, s.store) - tableStorageTester.MustExec("use information_schema") - c.Assert(tableStorageTester.Se.Auth(&auth.UserIdentity{ - Username: "table_storage_tester", - Hostname: "127.0.0.1", - }, nil, nil), IsTrue) - - // Test the privilege. - tableStorageTester.MustQuery("select * from TABLE_STORAGE_STATS;").Check([][]interface{}{}) - tk.MustExec("CREATE ROLE r_stats_meta ;") - tk.MustExec("GRANT ALL PRIVILEGES ON mysql.stats_meta TO r_stats_meta;") - tk.MustExec("GRANT r_stats_meta TO table_storage_tester;") - tableStorageTester.MustExec("set role r_stats_meta") - c.Assert(len(tableStorageTester.MustQuery("select * from TABLE_STORAGE_STATS where TABLE_NAME='stats_meta';").Rows()), Greater, 0) + // + //tk.MustExec("use test") + //tk.MustExec("drop table if exists t") + //tk.MustExec("create table t (a int, b int, index idx(a))") + //tk.MustQuery("select TABLE_NAME, STORAGE_SIZE from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'test' and TABLE_NAME='t';").Check( + // testkit.Rows("t 1")) + // + ////test the privilege of new user for information_schema.table_constraints + //tk.MustExec("create user table_storage_tester") + //tableStorageTester := testkit.NewTestKit(c, s.store) + //tableStorageTester.MustExec("use information_schema") + //c.Assert(tableStorageTester.Se.Auth(&auth.UserIdentity{ + // Username: "table_storage_tester", + // Hostname: "127.0.0.1", + //}, nil, nil), IsTrue) + // + //// Test the privilege. + //tableStorageTester.MustQuery("select * from TABLE_STORAGE_STATS;").Check([][]interface{}{}) + //tk.MustExec("CREATE ROLE r_stats_meta ;") + //tk.MustExec("GRANT ALL PRIVILEGES ON mysql.stats_meta TO r_stats_meta;") + //tk.MustExec("GRANT r_stats_meta TO table_storage_tester;") + //tableStorageTester.MustExec("set role r_stats_meta") + //c.Assert(len(tableStorageTester.MustQuery("select * from TABLE_STORAGE_STATS where TABLE_NAME='stats_meta';").Rows()), Greater, 0) } From e5a12c7c995313e43c679ddbf7a6873ad428185d Mon Sep 17 00:00:00 2001 From: guoni Date: Mon, 23 Mar 2020 10:01:15 +0800 Subject: [PATCH 12/33] add ut --- executor/infoschema_reader.go | 12 ++++++++++++ executor/infoschema_reader_test.go | 23 +++++++++++++---------- 2 files changed, 25 insertions(+), 10 deletions(-) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index a01ff10db72e9..46d7fea1c0e70 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -17,6 +17,7 @@ import ( "context" "encoding/json" "fmt" + "github.com/pingcap/failpoint" "net/http" "net/url" "sort" @@ -1207,6 +1208,17 @@ func (e *tableStorageStatsRetriever) setDataForTableStorageStats(ctx sessionctx. } pdAddrs = etcd.EtcdAddrs() if len(pdAddrs) == 0 { + failpoint.Inject("mockClusterConfigServerInfo", func(val failpoint.Value) { + if s := val.(string); len(s) > 0 { + serversInfo := parseFailpointServerInfo(s) + for _,v := range serversInfo { + if v.ServerType == "pd" { + pdAddrs[0] = v.Address + break + } + } + } + }) return nil, errors.New("pd unavailable") } count := 0 diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index d1a97805ed94d..0bf41cac668a8 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -734,24 +734,27 @@ func (s *testInfoschemaClusterTableSuite) TestTiDBClusterInfo(c *C) { func (s *testInfoschemaClusterTableSuite) TestTableStorageStats(c *C) { tk := testkit.NewTestKit(c, s.store) - //err := tk.QueryToErr("select * from information_schema.TABLE_STORAGE_STATS") - //c.Assert(err, NotNil) + err := tk.QueryToErr("select * from information_schema.TABLE_STORAGE_STATS") + c.Assert(err, NotNil) mockAddr := s.mockAddr store := &mockStore{ s.store.(tikv.Storage), mockAddr, } - + // information_schema.cluster_info tk = testkit.NewTestKit(c, store) - //tidbStatusAddr := fmt.Sprintf(":%d", config.GetGlobalConfig().Status.StatusPort) - //row := func(cols ...string) string { return strings.Join(cols, " ") } - //tk.MustQuery("select type, instance, status_address, version, git_hash from information_schema.cluster_info").Check(testkit.Rows( - // row("tidb", ":4000", tidbStatusAddr, "5.7.25-TiDB-None", "None"), - // row("pd", mockAddr, mockAddr, "4.0.0-alpha", "mock-pd-githash"), - // row("tikv", "127.0.0.1:20160", mockAddr, "4.0.0-alpha", "mock-tikv-githash"), - //)) tk.MustQuery("select TABLE_NAME from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema' and TABLE_NAME='schemata';").Check( testkit.Rows("schemata")) + //var servers []string + //for _, s := range testServers { + // servers = append(servers, strings.Join([]string{s.typ, s.address, s.address}, ",")) + //} + //fpName1 := "github.com/pingcap/tidb/executor/mockClusterServerInfo" + //fpExpr := strings.Join(servers, ";") + //c.Assert(failpoint.Enable(fpName1, fmt.Sprintf(`return("%s")`, fpExpr)), IsNil) + //defer func() { c.Assert(failpoint.Disable(fpName1), IsNil) }() + + // //tk.MustExec("use test") //tk.MustExec("drop table if exists t") From 8fbc3a89a50739acd340d767170cdf6860599160 Mon Sep 17 00:00:00 2001 From: guoni Date: Mon, 23 Mar 2020 17:02:10 +0800 Subject: [PATCH 13/33] refine --- infoschema/tables.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/infoschema/tables.go b/infoschema/tables.go index f7489baab72be..9fc99d7d2d21f 100755 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -996,6 +996,10 @@ var tableStorageStatsCols = []columnInfo{ {name: "TABLE_NAME", tp: mysql.TypeVarchar, size: 64}, {name: "TABLE_ID", tp: mysql.TypeLonglong, size: 21}, {name: "DISK_USAGE", tp: mysql.TypeLonglong, size: 64}, + {name: "DISK_USAGE", tp: mysql.TypeLonglong, size: 64}, + {name: "DISK_USAGE", tp: mysql.TypeLonglong, size: 64}, + {name: "DISK_USAGE", tp: mysql.TypeLonglong, size: 64}, + {name: "DISK_USAGE", tp: mysql.TypeLonglong, size: 64}, } var tableDDLJobsCols = []columnInfo{ From 320b3df7251b4a0700ac6f63e4dcb16f981aaf13 Mon Sep 17 00:00:00 2001 From: guoni Date: Mon, 23 Mar 2020 21:41:18 +0800 Subject: [PATCH 14/33] add mock addr --- executor/infoschema_reader.go | 51 +++++++++++++++--------------- executor/infoschema_reader_test.go | 16 ++++------ 2 files changed, 32 insertions(+), 35 deletions(-) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 6ad59b70a3209..17db00bad7e5c 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1322,6 +1322,7 @@ type tableStorageStatsRetriever struct { extractor *plannercore.TableStorageStatsExtractor initialTables []*initialTable curTable int + pdAddress string } func (e *tableStorageStatsRetriever) retrieve(ctx context.Context, sctx sessionctx.Context) ([][]types.Datum, error) { @@ -1405,6 +1406,29 @@ func (e *tableStorageStatsRetriever) initialize(sctx sessionctx.Context) error { } } } + + // Cache the PD address. + pdAddr := "" + failpoint.Inject("mockClusterPDServerInfo", func(val failpoint.Value) { + if s := val.(string); len(s) > 0 { + pdAddr = s + } + }) + if pdAddr == "" { + tikvStore, ok := sctx.GetStore().(tikv.Storage) + if !ok { + return errors.New("Information about TiKV region status can be gotten only when the storage is TiKV") + } + etcd, ok := tikvStore.(tikv.EtcdBackend) + if !ok { + return errors.New("not implemented") + } + pdAddr = etcd.EtcdAddrs()[0] + if pdAddr == "" { + return errors.New("pd unavailable") + } + } + e.pdAddress = pdAddr e.initialized = true return nil } @@ -1420,30 +1444,7 @@ type PdRegionStats struct { func (e *tableStorageStatsRetriever) setDataForTableStorageStats(ctx sessionctx.Context) ([][]types.Datum, error) { rows := make([][]types.Datum, 0, 1024) - tikvStore, ok := ctx.GetStore().(tikv.Storage) - if !ok { - return nil, errors.New("Information about TiKV region status can be gotten only when the storage is TiKV") - } - var pdAddrs []string - etcd, ok := tikvStore.(tikv.EtcdBackend) - if !ok { - return nil, errors.New("not implemented") - } - pdAddrs = etcd.EtcdAddrs() - if len(pdAddrs) == 0 { - failpoint.Inject("mockClusterConfigServerInfo", func(val failpoint.Value) { - if s := val.(string); len(s) > 0 { - serversInfo := parseFailpointServerInfo(s) - for _, v := range serversInfo { - if v.ServerType == "pd" { - pdAddrs[0] = v.Address - break - } - } - } - }) - return nil, errors.New("pd unavailable") - } + count := 0 for i := e.curTable; e.curTable < len(e.initialTables) && count < 1024; i++ { table := (e.initialTables)[e.curTable] @@ -1455,7 +1456,7 @@ func (e *tableStorageStatsRetriever) setDataForTableStorageStats(ctx sessionctx. endKey = codec.EncodeBytes([]byte{}, endKey) statURL := fmt.Sprintf("http://%s/pd/api/v1/stats/region?start_key=%s&end_key=%s", - pdAddrs[0], + e.pdAddress, url.QueryEscape(string(startKey)), url.QueryEscape(string(endKey))) diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index a72f462957cd0..47a74afc47a1a 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -780,20 +780,16 @@ func (s *testInfoschemaClusterTableSuite) TestTableStorageStats(c *C) { mockAddr, } + // information_schema.TABLE_STORAGE_STATS + pdAddr := mockAddr + fpExpr := fmt.Sprintf(`return("%s")`, pdAddr) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockClusterPDInfo", fpExpr), IsNil) + defer func() { c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/mockClusterPDInfo"), IsNil) }() + // information_schema.TABLE_STORAGE_STATS tk = testkit.NewTestKit(c, store) tk.MustQuery("select TABLE_NAME from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema' and TABLE_NAME='schemata';").Check( testkit.Rows("schemata")) - //var servers []string - //for _, s := range testServers { - // servers = append(servers, strings.Join([]string{s.typ, s.address, s.address}, ",")) - //} - //fpName1 := "github.com/pingcap/tidb/executor/mockClusterServerInfo" - //fpExpr := strings.Join(servers, ";") - //c.Assert(failpoint.Enable(fpName1, fmt.Sprintf(`return("%s")`, fpExpr)), IsNil) - //defer func() { c.Assert(failpoint.Disable(fpName1), IsNil) }() - - // //tk.MustExec("use test") //tk.MustExec("drop table if exists t") //tk.MustExec("create table t (a int, b int, index idx(a))") From 94829264292539784090aab495c5166660d85128 Mon Sep 17 00:00:00 2001 From: guoni Date: Mon, 23 Mar 2020 21:47:23 +0800 Subject: [PATCH 15/33] fix bug --- executor/infoschema_reader.go | 2 +- infoschema/tables.go | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 29a183a98523f..568db3c65019c 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1434,7 +1434,7 @@ func (e *tableStorageStatsRetriever) initialize(sctx sessionctx.Context) error { return nil } -// pdRegionStats is the json response from PD. +// PdRegionStats is the json response from PD. type PdRegionStats struct { Count int `json:"count"` EmptyCount int `json:"empty_count"` diff --git a/infoschema/tables.go b/infoschema/tables.go index 9fc99d7d2d21f..fcaf659b10383 100755 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -995,11 +995,11 @@ var tableStorageStatsCols = []columnInfo{ {name: "TABLE_SCHEMA", tp: mysql.TypeVarchar, size: 64}, {name: "TABLE_NAME", tp: mysql.TypeVarchar, size: 64}, {name: "TABLE_ID", tp: mysql.TypeLonglong, size: 21}, - {name: "DISK_USAGE", tp: mysql.TypeLonglong, size: 64}, - {name: "DISK_USAGE", tp: mysql.TypeLonglong, size: 64}, - {name: "DISK_USAGE", tp: mysql.TypeLonglong, size: 64}, - {name: "DISK_USAGE", tp: mysql.TypeLonglong, size: 64}, - {name: "DISK_USAGE", tp: mysql.TypeLonglong, size: 64}, + {name: "PEER_COUNT", tp: mysql.TypeLonglong, size: 21}, + {name: "REGION_COUNT", tp: mysql.TypeLonglong, size: 21}, + {name: "EMPTY_REGION_COUNT", tp: mysql.TypeLonglong, size: 21}, + {name: "TABLE_SIZE", tp: mysql.TypeLonglong, size: 64}, + {name: "TABLE_KEYS", tp: mysql.TypeLonglong, size: 64}, } var tableDDLJobsCols = []columnInfo{ From dfc6bc37c084674467377f35981a6eb7c1da2472 Mon Sep 17 00:00:00 2001 From: guoni Date: Tue, 24 Mar 2020 00:11:48 +0800 Subject: [PATCH 16/33] add UT --- executor/infoschema_reader.go | 35 +++++++++-------------- executor/infoschema_reader_test.go | 46 ++++++++++-------------------- 2 files changed, 28 insertions(+), 53 deletions(-) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 568db3c65019c..682ec28c360ae 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -17,7 +17,6 @@ import ( "context" "encoding/json" "fmt" - "github.com/pingcap/failpoint" "net/http" "net/url" "sort" @@ -1402,34 +1401,26 @@ func (e *tableStorageStatsRetriever) initialize(sctx sessionctx.Context) error { } else { for _, schema := range dbs { for _, table := range schema.Tables { - e.initialTables = append(e.initialTables, &initialTable{schema.Name.O, table}) + e.initialTables = append(e.initialTables, &initialTable{schema.Name.L, table}) } } } } // Cache the PD address. - pdAddr := "" - failpoint.Inject("mockClusterPDServerInfo", func(val failpoint.Value) { - if s := val.(string); len(s) > 0 { - pdAddr = s - } - }) - if pdAddr == "" { - tikvStore, ok := sctx.GetStore().(tikv.Storage) - if !ok { - return errors.New("Information about TiKV region status can be gotten only when the storage is TiKV") - } - etcd, ok := tikvStore.(tikv.EtcdBackend) - if !ok { - return errors.New("not implemented") - } - pdAddr = etcd.EtcdAddrs()[0] - if pdAddr == "" { - return errors.New("pd unavailable") - } + tikvStore, ok := sctx.GetStore().(tikv.Storage) + if !ok { + return errors.New("Information about TiKV region status can be gotten only when the storage is TiKV") + } + etcd, ok := tikvStore.(tikv.EtcdBackend) + if !ok { + return errors.New("not implemented") + } + pdAddrs := etcd.EtcdAddrs() + if len(pdAddrs) == 0 { + return errors.New("pd unavailable") } - e.pdAddress = pdAddr + e.pdAddress = pdAddrs[0] e.initialized = true return nil } diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index 85be95fc49218..cbfca78b539d3 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -645,7 +645,7 @@ func (s *testInfoschemaClusterTableSuite) setUpMockPDHTTPServer() (*httptest.Ser // TiDB/TiKV config router.Handle("/config", fn.Wrap(mockConfig)) // pd region - router.Handle(pdapi.Regions, fn.Wrap(func() (*executor.PdRegionStats, error) { + router.Handle("/pd/api/v1/stats/region", fn.Wrap(func() (*executor.PdRegionStats, error) { return &executor.PdRegionStats{ Count: 1, EmptyCount: 1, @@ -768,38 +768,22 @@ func (s *testInfoschemaClusterTableSuite) TestTableStorageStats(c *C) { mockAddr, } - // information_schema.TABLE_STORAGE_STATS - pdAddr := mockAddr - fpExpr := fmt.Sprintf(`return("%s")`, pdAddr) - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockClusterPDInfo", fpExpr), IsNil) - defer func() { c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/mockClusterPDInfo"), IsNil) }() - // information_schema.TABLE_STORAGE_STATS tk = testkit.NewTestKit(c, store) - tk.MustQuery("select TABLE_NAME from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema' and TABLE_NAME='schemata';").Check( - testkit.Rows("schemata")) - //tk.MustExec("use test") - //tk.MustExec("drop table if exists t") - //tk.MustExec("create table t (a int, b int, index idx(a))") - //tk.MustQuery("select TABLE_NAME, STORAGE_SIZE from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'test' and TABLE_NAME='t';").Check( - // testkit.Rows("t 1")) - // - ////test the privilege of new user for information_schema.table_constraints - //tk.MustExec("create user table_storage_tester") - //tableStorageTester := testkit.NewTestKit(c, s.store) - //tableStorageTester.MustExec("use information_schema") - //c.Assert(tableStorageTester.Se.Auth(&auth.UserIdentity{ - // Username: "table_storage_tester", - // Hostname: "127.0.0.1", - //}, nil, nil), IsTrue) - // - //// Test the privilege. - //tableStorageTester.MustQuery("select * from TABLE_STORAGE_STATS;").Check([][]interface{}{}) - //tk.MustExec("CREATE ROLE r_stats_meta ;") - //tk.MustExec("GRANT ALL PRIVILEGES ON mysql.stats_meta TO r_stats_meta;") - //tk.MustExec("GRANT r_stats_meta TO table_storage_tester;") - //tableStorageTester.MustExec("set role r_stats_meta") - //c.Assert(len(tableStorageTester.MustQuery("select * from TABLE_STORAGE_STATS where TABLE_NAME='stats_meta';").Rows()), Greater, 0) + tk.MustQuery("select TABLE_NAME from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema' and TABLE_NAME='schemata';").Check(testkit.Rows("SCHEMATA")) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int, b int, index idx(a))") + tk.MustQuery("select TABLE_NAME, TABLE_SIZE from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'test' and TABLE_NAME='t';").Check(testkit.Rows("t 1")) + + tk.MustExec("create table t1 (a int, b int, index idx(a))") + tk.MustQuery("select TABLE_NAME, sum(TABLE_SIZE) from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'test' group by TABLE_NAME;").Sort().Check(testkit.Rows( + "t 1", + "t1 1", + )) + tk.MustQuery("select TABLE_SCHEMA, sum(TABLE_SIZE) from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'test' group by TABLE_SCHEMA;").Check(testkit.Rows( + "test 2", + )) } func (s *testInfoschemaTableSuite) TestSequences(c *C) { From c3936e1e7f116cd4e3e86d923d34aa6a594f0262 Mon Sep 17 00:00:00 2001 From: reafans Date: Thu, 16 Apr 2020 01:18:11 +0800 Subject: [PATCH 17/33] address comment --- executor/infoschema_reader.go | 52 +++++++----------------------- infoschema/tables.go | 4 +-- server/http_handler.go | 48 +++------------------------- store/helper/helper.go | 60 +++++++++++++++++++++++++++++++++++ 4 files changed, 77 insertions(+), 87 deletions(-) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index e3c232eeb2888..9917fdc7b62f4 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -17,8 +17,6 @@ import ( "context" "encoding/json" "fmt" - "net/http" - "net/url" "sort" "strings" "sync" @@ -40,12 +38,10 @@ import ( "github.com/pingcap/tidb/store/helper" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" binaryJson "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/pdapi" "github.com/pingcap/tidb/util/set" @@ -1442,6 +1438,8 @@ type tableStorageStatsRetriever struct { initialTables []*initialTable curTable int pdAddress string + helper *helper.Helper + stats helper.PdRegionStats } func (e *tableStorageStatsRetriever) retrieve(ctx context.Context, sctx sessionctx.Context) ([][]types.Datum, error) { @@ -1531,12 +1529,9 @@ func (e *tableStorageStatsRetriever) initialize(sctx sessionctx.Context) error { if !ok { return errors.New("Information about TiKV region status can be gotten only when the storage is TiKV") } - etcd, ok := tikvStore.(tikv.EtcdBackend) - if !ok { - return errors.New("not implemented") - } - pdAddrs := etcd.EtcdAddrs() - if len(pdAddrs) == 0 { + e.helper = helper.NewHelper(tikvStore) + pdAddrs, err := e.helper.GetPDAddr() + if err != nil { return errors.New("pd unavailable") } e.pdAddress = pdAddrs[0] @@ -1544,15 +1539,6 @@ func (e *tableStorageStatsRetriever) initialize(sctx sessionctx.Context) error { return nil } -// PdRegionStats is the json response from PD. -type PdRegionStats struct { - Count int `json:"count"` - EmptyCount int `json:"empty_count"` - StorageSize int64 `json:"storage_size"` - StorageKeys int64 `json:"storage_keys"` - StorePeerCount map[uint64]int `json:"store_peer_count"` -} - func (e *tableStorageStatsRetriever) setDataForTableStorageStats(ctx sessionctx.Context) ([][]types.Datum, error) { rows := make([][]types.Datum, 0, 1024) @@ -1560,28 +1546,12 @@ func (e *tableStorageStatsRetriever) setDataForTableStorageStats(ctx sessionctx. for i := e.curTable; e.curTable < len(e.initialTables) && count < 1024; i++ { table := (e.initialTables)[e.curTable] tableID := table.tb.ID - // Include table and index data, because their range located in tableID_i tableID_r - startKey := tablecodec.EncodeTablePrefix(tableID) - endKey := tablecodec.EncodeTablePrefix(tableID + 1) - startKey = codec.EncodeBytes([]byte{}, startKey) - endKey = codec.EncodeBytes([]byte{}, endKey) - - statURL := fmt.Sprintf("http://%s/pd/api/v1/stats/region?start_key=%s&end_key=%s", - e.pdAddress, - url.QueryEscape(string(startKey)), - url.QueryEscape(string(endKey))) - - resp, err := http.Get(statURL) + err := e.helper.GetPdRegionStats(tableID, &e.stats) if err != nil { return nil, err } - var stats PdRegionStats - dec := json.NewDecoder(resp.Body) - if err := dec.Decode(&stats); err != nil { - return nil, err - } peerCount := 0 - for _, v := range stats.StorePeerCount { + for _, v := range e.stats.StorePeerCount { peerCount = v break } @@ -1590,10 +1560,10 @@ func (e *tableStorageStatsRetriever) setDataForTableStorageStats(ctx sessionctx. table.tb.Name.O, // TABLE_NAME tableID, // TABLE_ID peerCount, // TABLE_PEER_COUNT - stats.Count, // TABLE_REGION_COUNT - stats.EmptyCount, // TABLE_EMPTY_REGION_COUNT - stats.StorageSize, // TABLE_SIZE - stats.StorageKeys, // TABLE_KEYS + e.stats.Count, // TABLE_REGION_COUNT + e.stats.EmptyCount, // TABLE_EMPTY_REGION_COUNT + e.stats.StorageSize, // TABLE_SIZE + e.stats.StorageKeys, // TABLE_KEYS ) rows = append(rows, record) count++ diff --git a/infoschema/tables.go b/infoschema/tables.go index 5525879a8dfe5..17477354cc71e 100755 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -1109,8 +1109,8 @@ var tableStorageStatsCols = []columnInfo{ {name: "PEER_COUNT", tp: mysql.TypeLonglong, size: 21}, {name: "REGION_COUNT", tp: mysql.TypeLonglong, size: 21}, {name: "EMPTY_REGION_COUNT", tp: mysql.TypeLonglong, size: 21}, - {name: "TABLE_SIZE", tp: mysql.TypeLonglong, size: 64}, - {name: "TABLE_KEYS", tp: mysql.TypeLonglong, size: 64}, + {name: "TABLE_SIZE", tp: mysql.TypeLonglong, size: 64, comment: "The disk usage(MB) of the table"}, + {name: "TABLE_KEYS", tp: mysql.TypeLonglong, size: 64, comment: "The count of keys of the table"}, } // GetShardingInfo returns a nil or description string for the sharding information of given TableInfo. diff --git a/server/http_handler.go b/server/http_handler.go index afa5618ba1dc8..50564a4da90de 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -882,7 +882,7 @@ func (h tableHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { case opTableRegions: h.handleRegionRequest(schema, tableVal, w, req) case opTableDiskUsage: - h.handleDiskUsageRequest(schema, tableVal, w, req) + h.handleDiskUsageRequest(tableVal, w) case opTableScatter: h.handleScatterTableRequest(schema, tableVal, w, req) case opStopTableScatter: @@ -1178,54 +1178,14 @@ func (h tableHandler) getRegionsByID(tbl table.Table, id int64, name string) (*T }, nil } -// pdRegionStats is the json response from PD. -type pdRegionStats struct { - Count int `json:"count"` - EmptyCount int `json:"empty_count"` - StorageSize int64 `json:"storage_size"` - StoreLeaderCount map[uint64]int `json:"store_leader_count"` - StorePeerCount map[uint64]int `json:"store_peer_count"` - StoreLeaderSize map[uint64]int64 `json:"store_leader_size"` - StorePeerSize map[uint64]int64 `json:"store_peer_size"` -} - -func (h tableHandler) handleDiskUsageRequest(schema infoschema.InfoSchema, tbl table.Table, w http.ResponseWriter, req *http.Request) { +func (h tableHandler) handleDiskUsageRequest(tbl table.Table, w http.ResponseWriter) { tableID := tbl.Meta().ID - pdAddrs, err := h.getPDAddr() - if err != nil { - writeError(w, err) - return - } - - // Include table and index data, because their range located in tableID_i tableID_r - startKey := tablecodec.EncodeTablePrefix(tableID) - endKey := tablecodec.EncodeTablePrefix(tableID + 1) - startKey = codec.EncodeBytes([]byte{}, startKey) - endKey = codec.EncodeBytes([]byte{}, endKey) - - statURL := fmt.Sprintf("%s://%s/pd/api/v1/stats/region?start_key=%s&end_key=%s", - util.InternalHTTPSchema(), - pdAddrs[0], - url.QueryEscape(string(startKey)), - url.QueryEscape(string(endKey))) - - resp, err := util.InternalHTTPClient().Get(statURL) + var stats helper.PdRegionStats + err := h.GetPdRegionStats(tableID, &stats) if err != nil { writeError(w, err) return } - defer func() { - if err := resp.Body.Close(); err != nil { - log.Error(err) - } - }() - - var stats pdRegionStats - dec := json.NewDecoder(resp.Body) - if err := dec.Decode(&stats); err != nil { - writeError(w, err) - return - } writeData(w, stats.StorageSize) } diff --git a/store/helper/helper.go b/store/helper/helper.go index 8033ee4421091..b831356ceeb96 100644 --- a/store/helper/helper.go +++ b/store/helper/helper.go @@ -22,6 +22,7 @@ import ( "io" "math" "net/http" + "net/url" "sort" "strconv" "strings" @@ -38,6 +39,7 @@ import ( "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/pdapi" + log "github.com/sirupsen/logrus" "go.uber.org/zap" ) @@ -718,3 +720,61 @@ func (h *Helper) GetStoresStat() (*StoresStat, error) { } return &storesStat, nil } + +func (h *Helper) GetPDAddr() ([]string, error) { + var pdAddrs []string + etcd, ok := h.Store.(tikv.EtcdBackend) + if !ok { + return nil, errors.New("not implemented") + } + pdAddrs = etcd.EtcdAddrs() + if len(pdAddrs) < 0 { + return nil, errors.New("pd unavailable") + } + return pdAddrs, nil +} + +// PdRegionStats is the json response from PD. +type PdRegionStats struct { + Count int `json:"count"` + EmptyCount int `json:"empty_count"` + StorageSize int64 `json:"storage_size"` + StorageKeys int64 `json:"storage_keys"` + StoreLeaderCount map[uint64]int `json:"store_leader_count"` + StorePeerCount map[uint64]int `json:"store_peer_count"` +} + +func (h *Helper) GetPdRegionStats(tableID int64, stats *PdRegionStats) error { + pdAddrs, err := h.GetPDAddr() + if err != nil { + return err + } + + startKey := tablecodec.EncodeTablePrefix(tableID) + endKey := tablecodec.EncodeTablePrefix(tableID + 1) + startKey = codec.EncodeBytes([]byte{}, startKey) + endKey = codec.EncodeBytes([]byte{}, endKey) + + statURL := fmt.Sprintf("%s://%s/pd/api/v1/stats/region?start_key=%s&end_key=%s", + util.InternalHTTPSchema(), + pdAddrs[0], + url.QueryEscape(string(startKey)), + url.QueryEscape(string(endKey))) + + resp, err := util.InternalHTTPClient().Get(statURL) + defer func() { + if err := resp.Body.Close(); err != nil { + log.Error(err) + } + }() + + if err != nil { + return err + } + dec := json.NewDecoder(resp.Body) + if err := dec.Decode(stats); err != nil { + return err + } + + return nil +} From 2f510d51507be8eedd0dd9edccd7aad0c9ee3d5a Mon Sep 17 00:00:00 2001 From: reafans Date: Thu, 16 Apr 2020 01:27:54 +0800 Subject: [PATCH 18/33] fmt --- executor/infoschema_reader.go | 8 ++++---- executor/infoschema_reader_test.go | 15 ++++++++------- 2 files changed, 12 insertions(+), 11 deletions(-) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 9917fdc7b62f4..eff5b454835c5 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1556,10 +1556,10 @@ func (e *tableStorageStatsRetriever) setDataForTableStorageStats(ctx sessionctx. break } record := types.MakeDatums( - table.db, // TABLE_SCHEMA - table.tb.Name.O, // TABLE_NAME - tableID, // TABLE_ID - peerCount, // TABLE_PEER_COUNT + table.db, // TABLE_SCHEMA + table.tb.Name.O, // TABLE_NAME + tableID, // TABLE_ID + peerCount, // TABLE_PEER_COUNT e.stats.Count, // TABLE_REGION_COUNT e.stats.EmptyCount, // TABLE_EMPTY_REGION_COUNT e.stats.StorageSize, // TABLE_SIZE diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index ba9ad82b76942..6e363b1778d73 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -649,13 +649,14 @@ func (s *testInfoschemaClusterTableSuite) setUpMockPDHTTPServer() (*httptest.Ser // TiDB/TiKV config router.Handle("/config", fn.Wrap(mockConfig)) // pd region - router.Handle("/pd/api/v1/stats/region", fn.Wrap(func() (*executor.PdRegionStats, error) { - return &executor.PdRegionStats{ - Count: 1, - EmptyCount: 1, - StorageSize: 1, - StorageKeys: 1, - StorePeerCount: map[uint64]int{1: 1}, + router.Handle("/pd/api/v1/stats/region", fn.Wrap(func() (*helper.PdRegionStats, error) { + return &helper.PdRegionStats{ + Count: 1, + EmptyCount: 1, + StorageSize: 1, + StorageKeys: 1, + StoreLeaderCount: map[uint64]int{1: 1}, + StorePeerCount: map[uint64]int{1: 1}, }, nil })) return server, mockAddr From 4471e233485b137ea3361b0bf820972f71ea25a8 Mon Sep 17 00:00:00 2001 From: reafans Date: Thu, 16 Apr 2020 01:36:03 +0800 Subject: [PATCH 19/33] fmt --- store/helper/helper.go | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/store/helper/helper.go b/store/helper/helper.go index b831356ceeb96..cf5d3e912f376 100644 --- a/store/helper/helper.go +++ b/store/helper/helper.go @@ -721,6 +721,7 @@ func (h *Helper) GetStoresStat() (*StoresStat, error) { return &storesStat, nil } +// GetPDAddr return the PD Address. func (h *Helper) GetPDAddr() ([]string, error) { var pdAddrs []string etcd, ok := h.Store.(tikv.EtcdBackend) @@ -744,6 +745,7 @@ type PdRegionStats struct { StorePeerCount map[uint64]int `json:"store_peer_count"` } +// GetPdRegionStats get the RegionStats by tableID. func (h *Helper) GetPdRegionStats(tableID int64, stats *PdRegionStats) error { pdAddrs, err := h.GetPDAddr() if err != nil { @@ -762,17 +764,18 @@ func (h *Helper) GetPdRegionStats(tableID int64, stats *PdRegionStats) error { url.QueryEscape(string(endKey))) resp, err := util.InternalHTTPClient().Get(statURL) + if err != nil { + return err + } + defer func() { - if err := resp.Body.Close(); err != nil { + if err = resp.Body.Close(); err != nil { log.Error(err) } }() - if err != nil { - return err - } dec := json.NewDecoder(resp.Body) - if err := dec.Decode(stats); err != nil { + if err = dec.Decode(stats); err != nil { return err } From 26d106e7d5642aec900343c0aed81d253882da5d Mon Sep 17 00:00:00 2001 From: reafans Date: Tue, 21 Apr 2020 15:35:08 +0800 Subject: [PATCH 20/33] refine --- store/helper/helper.go | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/store/helper/helper.go b/store/helper/helper.go index cf5d3e912f376..5096eafae5ba8 100644 --- a/store/helper/helper.go +++ b/store/helper/helper.go @@ -775,9 +775,6 @@ func (h *Helper) GetPdRegionStats(tableID int64, stats *PdRegionStats) error { }() dec := json.NewDecoder(resp.Body) - if err = dec.Decode(stats); err != nil { - return err - } - return nil + return dec.Decode(stats) } From 6cec231082f79c926819424fc9d7bc32b96eacbc Mon Sep 17 00:00:00 2001 From: reafans Date: Tue, 21 Apr 2020 17:52:32 +0800 Subject: [PATCH 21/33] refine comment --- infoschema/tables.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/infoschema/tables.go b/infoschema/tables.go index 2322aa97d1317..17d6111811081 100755 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -1107,10 +1107,10 @@ var tableStorageStatsCols = []columnInfo{ {name: "TABLE_NAME", tp: mysql.TypeVarchar, size: 64}, {name: "TABLE_ID", tp: mysql.TypeLonglong, size: 21}, {name: "PEER_COUNT", tp: mysql.TypeLonglong, size: 21}, - {name: "REGION_COUNT", tp: mysql.TypeLonglong, size: 21}, - {name: "EMPTY_REGION_COUNT", tp: mysql.TypeLonglong, size: 21}, - {name: "TABLE_SIZE", tp: mysql.TypeLonglong, size: 64, comment: "The disk usage(MB) of the table"}, - {name: "TABLE_KEYS", tp: mysql.TypeLonglong, size: 64, comment: "The count of keys of the table"}, + {name: "REGION_COUNT", tp: mysql.TypeLonglong, size: 21, comment: "The region count of single replica of the table"}, + {name: "EMPTY_REGION_COUNT", tp: mysql.TypeLonglong, size: 21, comment: "The region count of single replica of the table"}, + {name: "TABLE_SIZE", tp: mysql.TypeLonglong, size: 64, comment: "The disk usage(MB) of single replica of the table, if the table size is less than 1MB, it would show 1MB "}, + {name: "TABLE_KEYS", tp: mysql.TypeLonglong, size: 64, comment: "The count of keys of single replica of the table"}, } // GetShardingInfo returns a nil or description string for the sharding information of given TableInfo. From 6f3f4280bfcaa11ade32098fec032693112a5495 Mon Sep 17 00:00:00 2001 From: reafans Date: Wed, 22 Apr 2020 11:01:50 +0800 Subject: [PATCH 22/33] refine --- executor/infoschema_reader.go | 7 ++----- infoschema/tables.go | 2 +- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index eff5b454835c5..ba2019a0fba95 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1550,11 +1550,8 @@ func (e *tableStorageStatsRetriever) setDataForTableStorageStats(ctx sessionctx. if err != nil { return nil, err } - peerCount := 0 - for _, v := range e.stats.StorePeerCount { - peerCount = v - break - } + peerCount := len(e.stats.StorePeerCount) + record := types.MakeDatums( table.db, // TABLE_SCHEMA table.tb.Name.O, // TABLE_NAME diff --git a/infoschema/tables.go b/infoschema/tables.go index 17d6111811081..7356af91c3669 100755 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -1109,7 +1109,7 @@ var tableStorageStatsCols = []columnInfo{ {name: "PEER_COUNT", tp: mysql.TypeLonglong, size: 21}, {name: "REGION_COUNT", tp: mysql.TypeLonglong, size: 21, comment: "The region count of single replica of the table"}, {name: "EMPTY_REGION_COUNT", tp: mysql.TypeLonglong, size: 21, comment: "The region count of single replica of the table"}, - {name: "TABLE_SIZE", tp: mysql.TypeLonglong, size: 64, comment: "The disk usage(MB) of single replica of the table, if the table size is less than 1MB, it would show 1MB "}, + {name: "TABLE_SIZE", tp: mysql.TypeLonglong, size: 64, comment: "The disk usage(MB) of single replica of the table, if the table size is empty or less than 1MB, it would show 1MB "}, {name: "TABLE_KEYS", tp: mysql.TypeLonglong, size: 64, comment: "The count of keys of single replica of the table"}, } From a73757372c78cb165bfa701e010e389ec21ae490 Mon Sep 17 00:00:00 2001 From: reafans Date: Wed, 22 Apr 2020 21:35:07 +0800 Subject: [PATCH 23/33] address comment --- executor/infoschema_reader.go | 53 ++++++-------------- executor/infoschema_reader_test.go | 19 ++++--- planner/core/memtable_predicate_extractor.go | 19 ++++--- server/http_handler.go | 4 +- store/helper/helper.go | 8 +-- 5 files changed, 43 insertions(+), 60 deletions(-) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index ba2019a0fba95..1378708e69e98 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1427,7 +1427,7 @@ func (e *memtableRetriever) setDataFromTableConstraints(ctx sessionctx.Context, e.rows = rows } -//tableStorageStatsRetriever is used to read slow log data. +// tableStorageStatsRetriever is used to read slow log data. type tableStorageStatsRetriever struct { dummyCloser table *model.TableInfo @@ -1439,7 +1439,7 @@ type tableStorageStatsRetriever struct { curTable int pdAddress string helper *helper.Helper - stats helper.PdRegionStats + stats helper.PDRegionStats } func (e *tableStorageStatsRetriever) retrieve(ctx context.Context, sctx sessionctx.Context) ([][]types.Datum, error) { @@ -1477,7 +1477,7 @@ func (e *tableStorageStatsRetriever) retrieve(ctx context.Context, sctx sessionc type initialTable struct { db string - tb *model.TableInfo + *model.TableInfo } func (e *tableStorageStatsRetriever) initialize(sctx sessionctx.Context) error { @@ -1486,43 +1486,20 @@ func (e *tableStorageStatsRetriever) initialize(sctx sessionctx.Context) error { sort.Sort(infoschema.SchemasSorter(dbs)) schemas := e.extractor.TableSchema tables := e.extractor.TableName - if len(schemas) != 0 { - for _, schema := range dbs { - if schemas.Exist(schema.Name.L) { - if len(tables) != 0 { - for _, table := range schema.Tables { - if tables.Exist(table.Name.L) { - e.initialTables = append(e.initialTables, &initialTable{db: schema.Name.L, tb: table}) - } - } - } else { - for _, table := range schema.Tables { - e.initialTables = append(e.initialTables, &initialTable{db: schema.Name.L, tb: table}) - } - } - } - } - if len(e.initialTables) == 0 && tables == nil { - return errors.Errorf("schema or table not exist, please check the schema and table") - } - } else { - if len(tables) != 0 { - for _, schema := range dbs { - for _, table := range schema.Tables { - if tables.Exist(table.Name.L) { - e.initialTables = append(e.initialTables, &initialTable{db: schema.Name.L, tb: table}) - } - } - } - } else { - for _, schema := range dbs { - for _, table := range schema.Tables { + // Extract the tables to the initialTable. + for _, schema := range dbs { + if len(schemas) == 0 || schemas.Exist(schema.Name.L) { + for _, table := range schema.Tables { + if len(tables) == 0 || tables.Exist(table.Name.L) { e.initialTables = append(e.initialTables, &initialTable{schema.Name.L, table}) } } } } + if len(e.initialTables) == 0 { + return errors.Errorf("schema or table not exist, please check the schema and table") + } // Cache the PD address. tikvStore, ok := sctx.GetStore().(tikv.Storage) @@ -1532,7 +1509,7 @@ func (e *tableStorageStatsRetriever) initialize(sctx sessionctx.Context) error { e.helper = helper.NewHelper(tikvStore) pdAddrs, err := e.helper.GetPDAddr() if err != nil { - return errors.New("pd unavailable") + return err } e.pdAddress = pdAddrs[0] e.initialized = true @@ -1545,8 +1522,8 @@ func (e *tableStorageStatsRetriever) setDataForTableStorageStats(ctx sessionctx. count := 0 for i := e.curTable; e.curTable < len(e.initialTables) && count < 1024; i++ { table := (e.initialTables)[e.curTable] - tableID := table.tb.ID - err := e.helper.GetPdRegionStats(tableID, &e.stats) + tableID := table.ID + err := e.helper.GetPDRegionStats(tableID, &e.stats) if err != nil { return nil, err } @@ -1554,7 +1531,7 @@ func (e *tableStorageStatsRetriever) setDataForTableStorageStats(ctx sessionctx. record := types.MakeDatums( table.db, // TABLE_SCHEMA - table.tb.Name.O, // TABLE_NAME + table.Name.O, // TABLE_NAME tableID, // TABLE_ID peerCount, // TABLE_PEER_COUNT e.stats.Count, // TABLE_REGION_COUNT diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index 6e363b1778d73..4a26b0e104ae4 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -644,13 +644,13 @@ func (s *testInfoschemaClusterTableSuite) setUpMockPDHTTPServer() (*httptest.Ser } return configuration, nil } - // pd config + // PD config. router.Handle(pdapi.Config, fn.Wrap(mockConfig)) - // TiDB/TiKV config + // TiDB/TiKV config. router.Handle("/config", fn.Wrap(mockConfig)) - // pd region - router.Handle("/pd/api/v1/stats/region", fn.Wrap(func() (*helper.PdRegionStats, error) { - return &helper.PdRegionStats{ + // PD region. + router.Handle("/pd/api/v1/stats/region", fn.Wrap(func() (*helper.PDRegionStats, error) { + return &helper.PDRegionStats{ Count: 1, EmptyCount: 1, StorageSize: 1, @@ -770,7 +770,7 @@ func (s *testInfoschemaClusterTableSuite) TestTableStorageStats(c *C) { mockAddr, } - // information_schema.TABLE_STORAGE_STATS + // Test information_schema.TABLE_STORAGE_STATS. tk = testkit.NewTestKit(c, store) tk.MustQuery("select TABLE_NAME from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema' and TABLE_NAME='schemata';").Check(testkit.Rows("SCHEMATA")) tk.MustExec("use test") @@ -786,6 +786,13 @@ func (s *testInfoschemaClusterTableSuite) TestTableStorageStats(c *C) { tk.MustQuery("select TABLE_SCHEMA, sum(TABLE_SIZE) from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'test' group by TABLE_SCHEMA;").Check(testkit.Rows( "test 2", )) + + // Test information_schema.TABLE_STORAGE_STATS when the TABLE_SCHEMA or TABLE_NAME is not exist. + err = tk.QueryToErr("select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'db_not_exist';") + c.Assert(err.Error(), Equals, "schema or table not exist, please check the schema and table") + tk.MustExec("drop table if exists `tb_not_exist`") + err = tk.QueryToErr("select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'test' and TABLE_NAME = 'tb_not_exist';") + c.Assert(err.Error(), Equals, "schema or table not exist, please check the schema and table") } func (s *testInfoschemaTableSuite) TestSequences(c *C) { diff --git a/planner/core/memtable_predicate_extractor.go b/planner/core/memtable_predicate_extractor.go index 0d495e597fa53..f89d54a9ae6e6 100644 --- a/planner/core/memtable_predicate_extractor.go +++ b/planner/core/memtable_predicate_extractor.go @@ -956,30 +956,29 @@ func (e *SlowQueryExtractor) setTimeRange(start, end int64) { e.Enable = true } -// TableStorageStatsExtractor is used to extract some predicates of `disk_usage` +// TableStorageStatsExtractor is used to extract some predicates of `disk_usage`. type TableStorageStatsExtractor struct { extractHelper - // SkipRequest means the where clause always false, we don't need to request any component + // SkipRequest means the where clause always false, we don't need to request any component. SkipRequest bool - //usage set.StringSet - // TableSchema represents tableSchema applied to, and we should apply all table disk usage if there is no schema specified - // e.g: SELECT * FROM information_schema.disk_usage WHERE table_schema in ('test', 'information_schema') + // TableSchema represents tableSchema applied to, and we should apply all table disk usage if there is no schema specified. + // e.g: SELECT * FROM information_schema.disk_usage WHERE table_schema in ('test', 'information_schema'). TableSchema set.StringSet - // TableName represents tableName applied to, and we should apply all table disk usage if there is no table specified - // e.g: SELECT * FROM information_schema.disk_usage WHERE table in ('schemata', 'tables') + // TableName represents tableName applied to, and we should apply all table disk usage if there is no table specified. + // e.g: SELECT * FROM information_schema.disk_usage WHERE table in ('schemata', 'tables'). TableName set.StringSet } -// Extract implements the MemTablePredicateExtractor Extract interface +// Extract implements the MemTablePredicateExtractor Extract interface. func (e *TableStorageStatsExtractor) Extract( _ sessionctx.Context, schema *expression.Schema, names []*types.FieldName, predicates []expression.Expression, ) []expression.Expression { - // Extract the `table_schema` columns + // Extract the `table_schema` columns. remained, schemaSkip, tableSchema := e.extractCol(schema, names, predicates, "table_schema", true) - // Extract the `table_name` columns + // Extract the `table_name` columns. remained, tableSkip, tableName := e.extractCol(schema, names, remained, "table_name", true) e.SkipRequest = schemaSkip || tableSkip if e.SkipRequest { diff --git a/server/http_handler.go b/server/http_handler.go index 50564a4da90de..dbfff78f78c9c 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -1180,8 +1180,8 @@ func (h tableHandler) getRegionsByID(tbl table.Table, id int64, name string) (*T func (h tableHandler) handleDiskUsageRequest(tbl table.Table, w http.ResponseWriter) { tableID := tbl.Meta().ID - var stats helper.PdRegionStats - err := h.GetPdRegionStats(tableID, &stats) + var stats helper.PDRegionStats + err := h.GetPDRegionStats(tableID, &stats) if err != nil { writeError(w, err) return diff --git a/store/helper/helper.go b/store/helper/helper.go index 5096eafae5ba8..a3073c1ef84fd 100644 --- a/store/helper/helper.go +++ b/store/helper/helper.go @@ -735,8 +735,8 @@ func (h *Helper) GetPDAddr() ([]string, error) { return pdAddrs, nil } -// PdRegionStats is the json response from PD. -type PdRegionStats struct { +// PDRegionStats is the json response from PD. +type PDRegionStats struct { Count int `json:"count"` EmptyCount int `json:"empty_count"` StorageSize int64 `json:"storage_size"` @@ -745,8 +745,8 @@ type PdRegionStats struct { StorePeerCount map[uint64]int `json:"store_peer_count"` } -// GetPdRegionStats get the RegionStats by tableID. -func (h *Helper) GetPdRegionStats(tableID int64, stats *PdRegionStats) error { +// GetPDRegionStats get the RegionStats by tableID. +func (h *Helper) GetPDRegionStats(tableID int64, stats *PDRegionStats) error { pdAddrs, err := h.GetPDAddr() if err != nil { return err From fd72b7d6385c7dc91d2e84e36fa5fd1d37aa85b9 Mon Sep 17 00:00:00 2001 From: reafans Date: Fri, 24 Apr 2020 15:20:11 +0800 Subject: [PATCH 24/33] address comment --- executor/infoschema_reader.go | 36 +++++++++++++++++++++--------- executor/infoschema_reader_test.go | 20 ++++++++++------- store/helper/helper.go | 2 +- 3 files changed, 38 insertions(+), 20 deletions(-) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 1378708e69e98..43c2c0e172701 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1482,24 +1482,37 @@ type initialTable struct { func (e *tableStorageStatsRetriever) initialize(sctx sessionctx.Context) error { is := infoschema.GetInfoSchema(sctx) - dbs := is.AllSchemas() - sort.Sort(infoschema.SchemasSorter(dbs)) + var databases []string schemas := e.extractor.TableSchema tables := e.extractor.TableName + // If not specify the table_schema, it would traverse all schemas and their tables. + if len(schemas) == 0 { + return errors.Errorf("Please specify the 'table_schema'") + } + + // Filter the sys or memory schema. + for schema := range schemas { + if !util.IsMemOrSysDB(schema) { + databases = append(databases, schema) + } + } + // Extract the tables to the initialTable. - for _, schema := range dbs { - if len(schemas) == 0 || schemas.Exist(schema.Name.L) { - for _, table := range schema.Tables { - if len(tables) == 0 || tables.Exist(table.Name.L) { - e.initialTables = append(e.initialTables, &initialTable{schema.Name.L, table}) + for _, db := range databases { + if len(tables) == 0 { // The user didn't specified the table, extract all tables of this db to initialTable. + tbs := is.SchemaTables(model.NewCIStr(db)) + for _, tb := range tbs { + e.initialTables = append(e.initialTables, &initialTable{db, tb.Meta()}) + } + } else { // The user specified the table, extract the specified tables of this db to initialTable. + for tb, _ := range tables { + if tb, ok := is.TableByName(model.NewCIStr(db), model.NewCIStr(tb)); ok == nil { + e.initialTables = append(e.initialTables, &initialTable{db, tb.Meta()}) } } } } - if len(e.initialTables) == 0 { - return errors.Errorf("schema or table not exist, please check the schema and table") - } // Cache the PD address. tikvStore, ok := sctx.GetStore().(tikv.Storage) @@ -1520,7 +1533,8 @@ func (e *tableStorageStatsRetriever) setDataForTableStorageStats(ctx sessionctx. rows := make([][]types.Datum, 0, 1024) count := 0 - for i := e.curTable; e.curTable < len(e.initialTables) && count < 1024; i++ { + //for i := e.curTable; e.curTable < len(e.initialTables) && count < 1024; i++ { + for e.curTable < len(e.initialTables) && count < 1024 { table := (e.initialTables)[e.curTable] tableID := table.ID err := e.helper.GetPDRegionStats(tableID, &e.stats) diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index 4a26b0e104ae4..4b6ed64f55967 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -772,7 +772,18 @@ func (s *testInfoschemaClusterTableSuite) TestTableStorageStats(c *C) { // Test information_schema.TABLE_STORAGE_STATS. tk = testkit.NewTestKit(c, store) - tk.MustQuery("select TABLE_NAME from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema' and TABLE_NAME='schemata';").Check(testkit.Rows("SCHEMATA")) + + // Test not set the schema. + err = tk.QueryToErr("select * from information_schema.TABLE_STORAGE_STATS") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "Please specify the 'table_schema'") + + // Test it would get null set when get the sys schema. + tk.MustQuery("select TABLE_NAME from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema';").Check([][]interface{}{}) + tk.MustQuery("select TABLE_NAME from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'mysql';").Check([][]interface{}{}) + tk.MustQuery("select TABLE_NAME from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA in ('mysql', 'metrics_schema');").Check([][]interface{}{}) + tk.MustQuery("select TABLE_NAME from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema' and TABLE_NAME='schemata';").Check([][]interface{}{}) + tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int, b int, index idx(a))") @@ -786,13 +797,6 @@ func (s *testInfoschemaClusterTableSuite) TestTableStorageStats(c *C) { tk.MustQuery("select TABLE_SCHEMA, sum(TABLE_SIZE) from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'test' group by TABLE_SCHEMA;").Check(testkit.Rows( "test 2", )) - - // Test information_schema.TABLE_STORAGE_STATS when the TABLE_SCHEMA or TABLE_NAME is not exist. - err = tk.QueryToErr("select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'db_not_exist';") - c.Assert(err.Error(), Equals, "schema or table not exist, please check the schema and table") - tk.MustExec("drop table if exists `tb_not_exist`") - err = tk.QueryToErr("select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'test' and TABLE_NAME = 'tb_not_exist';") - c.Assert(err.Error(), Equals, "schema or table not exist, please check the schema and table") } func (s *testInfoschemaTableSuite) TestSequences(c *C) { diff --git a/store/helper/helper.go b/store/helper/helper.go index a3073c1ef84fd..d7ff18f4f5c42 100644 --- a/store/helper/helper.go +++ b/store/helper/helper.go @@ -729,7 +729,7 @@ func (h *Helper) GetPDAddr() ([]string, error) { return nil, errors.New("not implemented") } pdAddrs = etcd.EtcdAddrs() - if len(pdAddrs) < 0 { + if len(pdAddrs) == 0 { return nil, errors.New("pd unavailable") } return pdAddrs, nil From 49758c0890146a7f4e1af7f4be3d799435382798 Mon Sep 17 00:00:00 2001 From: reafans Date: Fri, 24 Apr 2020 15:39:20 +0800 Subject: [PATCH 25/33] fmt --- executor/infoschema_reader.go | 1 - 1 file changed, 1 deletion(-) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index e08b540bb7562..713641089fff9 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1537,7 +1537,6 @@ func (e *tableStorageStatsRetriever) setDataForTableStorageStats(ctx sessionctx. rows := make([][]types.Datum, 0, 1024) count := 0 - //for i := e.curTable; e.curTable < len(e.initialTables) && count < 1024; i++ { for e.curTable < len(e.initialTables) && count < 1024 { table := (e.initialTables)[e.curTable] tableID := table.ID From de815b31dfcb634986209517066039ecc57838b3 Mon Sep 17 00:00:00 2001 From: reafans Date: Fri, 24 Apr 2020 16:55:01 +0800 Subject: [PATCH 26/33] fmt --- executor/infoschema_reader.go | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 713641089fff9..16f441219d626 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1504,14 +1504,16 @@ func (e *tableStorageStatsRetriever) initialize(sctx sessionctx.Context) error { // Extract the tables to the initialTable. for _, db := range databases { - if len(tables) == 0 { // The user didn't specified the table, extract all tables of this db to initialTable. + // The user didn't specified the table, extract all tables of this db to initialTable. + if len(tables) == 0 { tbs := is.SchemaTables(model.NewCIStr(db)) for _, tb := range tbs { e.initialTables = append(e.initialTables, &initialTable{db, tb.Meta()}) } - } else { // The user specified the table, extract the specified tables of this db to initialTable. - for tb, _ := range tables { - if tb, ok := is.TableByName(model.NewCIStr(db), model.NewCIStr(tb)); ok == nil { + } else { + // The user specified the table, extract the specified tables of this db to initialTable. + for tb := range tables { + if tb, err := is.TableByName(model.NewCIStr(db), model.NewCIStr(tb)); err == nil { e.initialTables = append(e.initialTables, &initialTable{db, tb.Meta()}) } } From a4d67c2da39ef9ff45d8a1db8766fd9bac289fa2 Mon Sep 17 00:00:00 2001 From: reafans Date: Fri, 24 Apr 2020 17:18:41 +0800 Subject: [PATCH 27/33] add explainInfo --- planner/core/memtable_predicate_extractor.go | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/planner/core/memtable_predicate_extractor.go b/planner/core/memtable_predicate_extractor.go index f89d54a9ae6e6..ca9681ffe5008 100644 --- a/planner/core/memtable_predicate_extractor.go +++ b/planner/core/memtable_predicate_extractor.go @@ -990,7 +990,18 @@ func (e *TableStorageStatsExtractor) Extract( } func (e *TableStorageStatsExtractor) explainInfo(p *PhysicalMemTable) string { - return "" + if e.SkipRequest { + return "skip_request: true" + } + + r := new(bytes.Buffer) + if len(e.TableSchema) > 0 { + r.WriteString(fmt.Sprintf("Schema:%s", extractStringFromStringSet(e.TableSchema))) + } + if len(e.TableName) > 0 { + r.WriteString(fmt.Sprintf(" table:%s", extractStringFromStringSet(e.TableName))) + } + return r.String() } func (e *SlowQueryExtractor) explainInfo(p *PhysicalMemTable) string { From 6e06349f76fe78095be7e09a31ec856e49b83c48 Mon Sep 17 00:00:00 2001 From: reafans Date: Wed, 6 May 2020 11:58:38 +0800 Subject: [PATCH 28/33] address comment --- executor/explainfor_test.go | 11 +++++++++++ executor/infoschema_reader.go | 1 - executor/infoschema_reader_test.go | 4 ++-- planner/core/memtable_predicate_extractor.go | 7 +++++-- 4 files changed, 18 insertions(+), 5 deletions(-) diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index 2dbe6f1f17e3b..d2785ee328fe1 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -168,3 +168,14 @@ func (s *testPrepareSerialSuite) TestExplainForConnPlanCache(c *C) { " └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", )) } + +func (s *testSuite) TestExplainTableStorage(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustQuery(fmt.Sprintf("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema'")).Check(testkit.Rows( + fmt.Sprintf("MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS schema:[\"information_schema\"]"))) + tk.MustQuery(fmt.Sprintf("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_NAME = 'schemata'")).Check(testkit.Rows( + fmt.Sprintf("MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS table:[\"schemata\"]"))) + tk.MustQuery(fmt.Sprintf("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema' and TABLE_NAME = 'schemata'")).Check(testkit.Rows( + fmt.Sprintf("MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS schema:[\"information_schema\"], table:[\"schemata\"]"))) + +} diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 16f441219d626..e4588f687bb84 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1537,7 +1537,6 @@ func (e *tableStorageStatsRetriever) initialize(sctx sessionctx.Context) error { func (e *tableStorageStatsRetriever) setDataForTableStorageStats(ctx sessionctx.Context) ([][]types.Datum, error) { rows := make([][]types.Datum, 0, 1024) - count := 0 for e.curTable < len(e.initialTables) && count < 1024 { table := (e.initialTables)[e.curTable] diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index 4b6ed64f55967..d4a62e3438c58 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -762,8 +762,8 @@ func (s *testInfoschemaClusterTableSuite) TestTiDBClusterInfo(c *C) { func (s *testInfoschemaClusterTableSuite) TestTableStorageStats(c *C) { tk := testkit.NewTestKit(c, s.store) - err := tk.QueryToErr("select * from information_schema.TABLE_STORAGE_STATS") - c.Assert(err, NotNil) + err := tk.QueryToErr("select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'test'") + c.Assert(err.Error(), Equals, "pd unavailable") mockAddr := s.mockAddr store := &mockStore{ s.store.(tikv.Storage), diff --git a/planner/core/memtable_predicate_extractor.go b/planner/core/memtable_predicate_extractor.go index ca9681ffe5008..55cd30d0905c3 100644 --- a/planner/core/memtable_predicate_extractor.go +++ b/planner/core/memtable_predicate_extractor.go @@ -996,10 +996,13 @@ func (e *TableStorageStatsExtractor) explainInfo(p *PhysicalMemTable) string { r := new(bytes.Buffer) if len(e.TableSchema) > 0 { - r.WriteString(fmt.Sprintf("Schema:%s", extractStringFromStringSet(e.TableSchema))) + r.WriteString(fmt.Sprintf("schema:[%s]", extractStringFromStringSet(e.TableSchema))) + } + if r.Len() > 0 && len(e.TableName) > 0{ + r.WriteString(", ") } if len(e.TableName) > 0 { - r.WriteString(fmt.Sprintf(" table:%s", extractStringFromStringSet(e.TableName))) + r.WriteString(fmt.Sprintf("table:[%s]", extractStringFromStringSet(e.TableName))) } return r.String() } From 87430e69931e6ae9f86a167641178b7aee595d3a Mon Sep 17 00:00:00 2001 From: reafans Date: Wed, 6 May 2020 12:05:55 +0800 Subject: [PATCH 29/33] fmt --- executor/explainfor_test.go | 1 - planner/core/memtable_predicate_extractor.go | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index d2785ee328fe1..3fcffd84ac64b 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -177,5 +177,4 @@ func (s *testSuite) TestExplainTableStorage(c *C) { fmt.Sprintf("MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS table:[\"schemata\"]"))) tk.MustQuery(fmt.Sprintf("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema' and TABLE_NAME = 'schemata'")).Check(testkit.Rows( fmt.Sprintf("MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS schema:[\"information_schema\"], table:[\"schemata\"]"))) - } diff --git a/planner/core/memtable_predicate_extractor.go b/planner/core/memtable_predicate_extractor.go index 55cd30d0905c3..2d42bf49adefa 100644 --- a/planner/core/memtable_predicate_extractor.go +++ b/planner/core/memtable_predicate_extractor.go @@ -998,7 +998,7 @@ func (e *TableStorageStatsExtractor) explainInfo(p *PhysicalMemTable) string { if len(e.TableSchema) > 0 { r.WriteString(fmt.Sprintf("schema:[%s]", extractStringFromStringSet(e.TableSchema))) } - if r.Len() > 0 && len(e.TableName) > 0{ + if r.Len() > 0 && len(e.TableName) > 0 { r.WriteString(", ") } if len(e.TableName) > 0 { From a5247bd47463c33de35208ffae2508b2ffdd5440 Mon Sep 17 00:00:00 2001 From: reafans Date: Tue, 12 May 2020 15:44:17 +0800 Subject: [PATCH 30/33] remove useless variable --- executor/infoschema_reader.go | 15 +-------------- 1 file changed, 1 insertion(+), 14 deletions(-) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index e4588f687bb84..6bf578c3ddbf0 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1441,7 +1441,6 @@ type tableStorageStatsRetriever struct { extractor *plannercore.TableStorageStatsExtractor initialTables []*initialTable curTable int - pdAddress string helper *helper.Helper stats helper.PDRegionStats } @@ -1490,7 +1489,7 @@ func (e *tableStorageStatsRetriever) initialize(sctx sessionctx.Context) error { schemas := e.extractor.TableSchema tables := e.extractor.TableName - // If not specify the table_schema, it would traverse all schemas and their tables. + // If not specify the table_schema, return an error to avoid traverse all schemas and their tables. if len(schemas) == 0 { return errors.Errorf("Please specify the 'table_schema'") } @@ -1519,18 +1518,6 @@ func (e *tableStorageStatsRetriever) initialize(sctx sessionctx.Context) error { } } } - - // Cache the PD address. - tikvStore, ok := sctx.GetStore().(tikv.Storage) - if !ok { - return errors.New("Information about TiKV region status can be gotten only when the storage is TiKV") - } - e.helper = helper.NewHelper(tikvStore) - pdAddrs, err := e.helper.GetPDAddr() - if err != nil { - return err - } - e.pdAddress = pdAddrs[0] e.initialized = true return nil } From eb8bcdde9c4be7045054ce4a80481188bdf03a87 Mon Sep 17 00:00:00 2001 From: reafans Date: Tue, 12 May 2020 15:52:48 +0800 Subject: [PATCH 31/33] fmt --- executor/infoschema_reader.go | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 6bf578c3ddbf0..f3d03f922bef8 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1518,6 +1518,13 @@ func (e *tableStorageStatsRetriever) initialize(sctx sessionctx.Context) error { } } } + + // Cache the helper. + tikvStore, ok := sctx.GetStore().(tikv.Storage) + if !ok { + return errors.Errorf("Information about TiKV region status can be gotten only when the storage is TiKV") + } + e.helper = helper.NewHelper(tikvStore) e.initialized = true return nil } From 7db052a01da013ed8bf370f9058cd5d45f4b2084 Mon Sep 17 00:00:00 2001 From: reafans Date: Tue, 12 May 2020 16:29:00 +0800 Subject: [PATCH 32/33] refine --- executor/infoschema_reader.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index f3d03f922bef8..1722a6bc664ec 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1519,12 +1519,16 @@ func (e *tableStorageStatsRetriever) initialize(sctx sessionctx.Context) error { } } - // Cache the helper. + // Cache the helper and return an error if pd unavailable. tikvStore, ok := sctx.GetStore().(tikv.Storage) if !ok { return errors.Errorf("Information about TiKV region status can be gotten only when the storage is TiKV") } e.helper = helper.NewHelper(tikvStore) + _, err := e.helper.GetPDAddr() + if err != nil { + return err + } e.initialized = true return nil } From 1e1fdeb4d9ace1e9577466852fb66a66666d4bcc Mon Sep 17 00:00:00 2001 From: crazycs Date: Tue, 2 Jun 2020 01:04:34 +0800 Subject: [PATCH 33/33] fix ci Signed-off-by: crazycs --- executor/explainfor_test.go | 1 - 1 file changed, 1 deletion(-) diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index 210a7bcf2a0a5..be99b72e4135f 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -204,7 +204,6 @@ func (s *testPrepareSerialSuite) TestExplainDotForExplainPlan(c *C) { tk.MustQuery(fmt.Sprintf("explain format=\"dot\" for connection %s", connID)).Check(nil) } - func (s *testSuite) TestExplainTableStorage(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustQuery(fmt.Sprintf("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema'")).Check(testkit.Rows(