From adebfe7d2dd953db307ed1d98d5eff2aa0886dfe Mon Sep 17 00:00:00 2001 From: reafans <30926443+reafans@users.noreply.github.com> Date: Tue, 2 Jun 2020 23:17:46 +0800 Subject: [PATCH 1/3] cherry pick #15056 to release-4.0 Signed-off-by: ti-srebot --- executor/builder.go | 10 ++ executor/explainfor_test.go | 14 ++ executor/infoschema_reader.go | 131 +++++++++++++++++++ executor/infoschema_reader_test.go | 54 +++++++- infoschema/tables.go | 25 ++++ planner/core/logical_plan_builder.go | 5 + planner/core/memtable_predicate_extractor.go | 51 ++++++++ server/http_handler.go | 48 +------ store/helper/helper.go | 60 +++++++++ 9 files changed, 352 insertions(+), 46 deletions(-) mode change 100644 => 100755 infoschema/tables.go diff --git a/executor/builder.go b/executor/builder.go index ddc50058a8048..22f4002715f29 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1471,6 +1471,16 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo extractor: v.Extractor.(*plannercore.SlowQueryExtractor), }, } + 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, + extractor: v.Extractor.(*plannercore.TableStorageStatsExtractor), + }, + } case strings.ToLower(infoschema.TableDDLJobs): return &DDLJobsReaderExec{ baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()), diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index a97c0d125b869..cc4c5190a8fe6 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -212,7 +212,21 @@ func (s *testPrepareSerialSuite) TestExplainDotForExplainPlan(c *C) { tk.MustQuery(fmt.Sprintf("explain format=\"dot\" for connection %s", connID)).Check(nil) } +<<<<<<< HEAD func (s *testSuite) TestExplainTiFlashSystemTables(c *C) { +======= +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\"]"))) +} + +func (s *testSuite) TestInspectionSummaryTable(c *C) { +>>>>>>> cfbd75465... planner,executor,infoschema: add system table `table_storage_stats` (#15056) tk := testkit.NewTestKitWithInit(c, s.store) tiflashInstance := "192.168.1.7:3930" database := "test" diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 09bd1c7b39b06..de9189548d16d 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1317,6 +1317,137 @@ func (e *memtableRetriever) setDataFromTableConstraints(ctx sessionctx.Context, e.rows = rows } +// tableStorageStatsRetriever is used to read slow log data. +type tableStorageStatsRetriever struct { + dummyCloser + table *model.TableInfo + outputCols []*model.ColumnInfo + retrieved bool + initialized bool + extractor *plannercore.TableStorageStatsExtractor + initialTables []*initialTable + curTable int + helper *helper.Helper + stats helper.PDRegionStats +} + +func (e *tableStorageStatsRetriever) 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.setDataForTableStorageStats(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 + *model.TableInfo +} + +func (e *tableStorageStatsRetriever) initialize(sctx sessionctx.Context) error { + is := infoschema.GetInfoSchema(sctx) + var databases []string + schemas := e.extractor.TableSchema + tables := e.extractor.TableName + + // 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'") + } + + // 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 _, DB := range databases { + // 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, err := is.TableByName(model.NewCIStr(DB), model.NewCIStr(tb)); err == nil { + e.initialTables = append(e.initialTables, &initialTable{DB, tb.Meta()}) + } + } + } + } + + // 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 +} + +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] + tableID := table.ID + err := e.helper.GetPDRegionStats(tableID, &e.stats) + if err != nil { + return nil, err + } + peerCount := len(e.stats.StorePeerCount) + + record := types.MakeDatums( + table.db, // TABLE_SCHEMA + table.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 + e.stats.StorageKeys, // TABLE_KEYS + ) + rows = append(rows, record) + count++ + e.curTable++ + } + return rows, nil +} + func (e *memtableRetriever) setDataFromSessionVar(ctx sessionctx.Context) error { var rows [][]types.Datum var err error diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index be50fb0947417..993226ef65963 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -646,10 +646,21 @@ 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{ + Count: 1, + EmptyCount: 1, + StorageSize: 1, + StorageKeys: 1, + StoreLeaderCount: map[uint64]int{1: 1}, + StorePeerCount: map[uint64]int{1: 1}, + }, nil + })) return server, mockAddr } @@ -755,6 +766,45 @@ 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 where TABLE_SCHEMA = 'test'") + c.Assert(err.Error(), Equals, "pd unavailable") + mockAddr := s.mockAddr + store := &mockStore{ + s.store.(tikv.Storage), + mockAddr, + } + + // Test information_schema.TABLE_STORAGE_STATS. + tk = testkit.NewTestKit(c, store) + + // 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))") + 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) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("CREATE SEQUENCE test.seq maxvalue 10000000") diff --git a/infoschema/tables.go b/infoschema/tables.go old mode 100644 new mode 100755 index f3bcec1486d4d..9ae32436199da --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -145,10 +145,15 @@ const ( TableStatementsSummary = "STATEMENTS_SUMMARY" // TableStatementsSummaryHistory is the string constant of statements summary history table. TableStatementsSummaryHistory = "STATEMENTS_SUMMARY_HISTORY" +<<<<<<< HEAD // TableTiFlashTables is the string constant of tiflash tables table. TableTiFlashTables = "TIFLASH_TABLES" // TableTiFlashSegments is the string constant of tiflash segments table. TableTiFlashSegments = "TIFLASH_SEGMENTS" +======= + // TableStorageStats is a table that contains all tables disk usage + TableStorageStats = "TABLE_STORAGE_STATS" +>>>>>>> cfbd75465... planner,executor,infoschema: add system table `table_storage_stats` (#15056) ) var tableIDMap = map[string]int64{ @@ -214,8 +219,12 @@ var tableIDMap = map[string]int64{ TableStatementsSummaryHistory: autoid.InformationSchemaDBID + 60, ClusterTableStatementsSummary: autoid.InformationSchemaDBID + 61, ClusterTableStatementsSummaryHistory: autoid.InformationSchemaDBID + 62, +<<<<<<< HEAD TableTiFlashTables: autoid.InformationSchemaDBID + 64, TableTiFlashSegments: autoid.InformationSchemaDBID + 65, +======= + TableStorageStats: autoid.InformationSchemaDBID + 63, +>>>>>>> cfbd75465... planner,executor,infoschema: add system table `table_storage_stats` (#15056) } type columnInfo struct { @@ -1218,6 +1227,7 @@ var tableStatementsSummaryCols = []columnInfo{ {name: "PLAN", tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "Sampled execution plan"}, } +<<<<<<< HEAD var tableTableTiFlashTablesCols = []columnInfo{ {name: "DATABASE", tp: mysql.TypeVarchar, size: 64}, {name: "TABLE", tp: mysql.TypeVarchar, size: 64}, @@ -1295,6 +1305,17 @@ var tableTableTiFlashSegmentsCols = []columnInfo{ {name: "DELTA_CACHE_SIZE", tp: mysql.TypeLonglong, size: 64}, {name: "DELTA_INDEX_SIZE", tp: mysql.TypeLonglong, size: 64}, {name: "TIFLASH_INSTANCE", tp: mysql.TypeVarchar, size: 64}, +======= +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: "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 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"}, +>>>>>>> cfbd75465... planner,executor,infoschema: add system table `table_storage_stats` (#15056) } // GetShardingInfo returns a nil or description string for the sharding information of given TableInfo. @@ -1623,8 +1644,12 @@ var tableNameToColumns = map[string][]columnInfo{ TableSequences: tableSequencesCols, TableStatementsSummary: tableStatementsSummaryCols, TableStatementsSummaryHistory: tableStatementsSummaryCols, +<<<<<<< HEAD TableTiFlashTables: tableTableTiFlashTablesCols, TableTiFlashSegments: tableTableTiFlashSegmentsCols, +======= + TableStorageStats: tableStorageStatsCols, +>>>>>>> cfbd75465... planner,executor,infoschema: add system table `table_storage_stats` (#15056) } 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 ff5e68e9730e6..5e1dc97584756 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -3057,8 +3057,13 @@ func (b *PlanBuilder) buildMemTable(_ context.Context, dbName model.CIStr, table p.QueryTimeRange = b.timeRangeForSummaryTable() case infoschema.TableSlowQuery: p.Extractor = &SlowQueryExtractor{} +<<<<<<< HEAD case infoschema.TableTiFlashTables, infoschema.TableTiFlashSegments: p.Extractor = &TiFlashSystemTableExtractor{} +======= + case infoschema.TableStorageStats: + p.Extractor = &TableStorageStatsExtractor{} +>>>>>>> cfbd75465... planner,executor,infoschema: add system table `table_storage_stats` (#15056) } } return p, nil diff --git a/planner/core/memtable_predicate_extractor.go b/planner/core/memtable_predicate_extractor.go index fc5aa7d3627f7..4fc28d6da6c25 100644 --- a/planner/core/memtable_predicate_extractor.go +++ b/planner/core/memtable_predicate_extractor.go @@ -926,6 +926,57 @@ func (e *SlowQueryExtractor) setTimeRange(start, end int64) { e.Enable = true } +// 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 + // 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 *TableStorageStatsExtractor) Extract( + _ sessionctx.Context, + schema *expression.Schema, + names []*types.FieldName, + predicates []expression.Expression, +) []expression.Expression { + // Extract the `table_schema` columns. + remained, schemaSkip, tableSchema := 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 + return remained +} + +func (e *TableStorageStatsExtractor) explainInfo(p *PhysicalMemTable) string { + 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 r.Len() > 0 && len(e.TableName) > 0 { + r.WriteString(", ") + } + if len(e.TableName) > 0 { + r.WriteString(fmt.Sprintf("table:[%s]", extractStringFromStringSet(e.TableName))) + } + return r.String() +} + func (e *SlowQueryExtractor) explainInfo(p *PhysicalMemTable) string { return "" } diff --git a/server/http_handler.go b/server/http_handler.go index a4e82d0d23fdd..c99a8afd4ce94 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -965,7 +965,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: // supports partition table, only get one physical table, prevent too many scatter schedulers. ptbl, err := h.getPartition(tableVal, partitionName) @@ -1274,54 +1274,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 ffd09c5b574f2..911e403f9dffc 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" ) @@ -726,3 +728,61 @@ 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) + 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"` +} + +// GetPDRegionStats get the RegionStats by tableID. +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) + if err != nil { + return err + } + + defer func() { + if err = resp.Body.Close(); err != nil { + log.Error(err) + } + }() + + dec := json.NewDecoder(resp.Body) + + return dec.Decode(stats) +} From 77da386abc1777e9e86636629f95926aa3a13660 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Tue, 13 Oct 2020 19:34:08 +0800 Subject: [PATCH 2/3] fix conflict Signed-off-by: crazycs520 --- executor/builder.go | 2 +- executor/explainfor_test.go | 24 ++++++++++-------------- infoschema/tables.go | 16 +++------------- planner/core/logical_plan_builder.go | 3 --- store/helper/helper.go | 5 ++++- 5 files changed, 18 insertions(+), 32 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 22f4002715f29..4fea31d56af9e 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1473,7 +1473,7 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo } case strings.ToLower(infoschema.TableStorageStats): return &MemTableReaderExec{ - baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()), + baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()), table: v.Table, retriever: &tableStorageStatsRetriever{ table: v.Table, diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index cc4c5190a8fe6..9aeb96e94109c 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -212,21 +212,7 @@ func (s *testPrepareSerialSuite) TestExplainDotForExplainPlan(c *C) { tk.MustQuery(fmt.Sprintf("explain format=\"dot\" for connection %s", connID)).Check(nil) } -<<<<<<< HEAD func (s *testSuite) TestExplainTiFlashSystemTables(c *C) { -======= -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\"]"))) -} - -func (s *testSuite) TestInspectionSummaryTable(c *C) { ->>>>>>> cfbd75465... planner,executor,infoschema: add system table `table_storage_stats` (#15056) tk := testkit.NewTestKitWithInit(c, s.store) tiflashInstance := "192.168.1.7:3930" database := "test" @@ -248,3 +234,13 @@ func (s *testSuite) TestInspectionSummaryTable(c *C) { tk.MustQuery(fmt.Sprintf("desc select * from information_schema.TIFLASH_SEGMENTS where TIFLASH_INSTANCE = '%s' and TIDB_DATABASE = '%s' and TIDB_TABLE = '%s'", tiflashInstance, database, table)).Check(testkit.Rows( fmt.Sprintf("MemTableScan_5 10000.00 root table:TIFLASH_SEGMENTS tiflash_instances:[\"%s\"], tidb_databases:[\"%s\"], tidb_tables:[\"%s\"]", tiflashInstance, database, table))) } + +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/infoschema/tables.go b/infoschema/tables.go index 9ae32436199da..10a7f9d25f7e7 100755 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -145,15 +145,12 @@ const ( TableStatementsSummary = "STATEMENTS_SUMMARY" // TableStatementsSummaryHistory is the string constant of statements summary history table. TableStatementsSummaryHistory = "STATEMENTS_SUMMARY_HISTORY" -<<<<<<< HEAD // TableTiFlashTables is the string constant of tiflash tables table. TableTiFlashTables = "TIFLASH_TABLES" // TableTiFlashSegments is the string constant of tiflash segments table. TableTiFlashSegments = "TIFLASH_SEGMENTS" -======= // TableStorageStats is a table that contains all tables disk usage TableStorageStats = "TABLE_STORAGE_STATS" ->>>>>>> cfbd75465... planner,executor,infoschema: add system table `table_storage_stats` (#15056) ) var tableIDMap = map[string]int64{ @@ -219,12 +216,9 @@ var tableIDMap = map[string]int64{ TableStatementsSummaryHistory: autoid.InformationSchemaDBID + 60, ClusterTableStatementsSummary: autoid.InformationSchemaDBID + 61, ClusterTableStatementsSummaryHistory: autoid.InformationSchemaDBID + 62, -<<<<<<< HEAD + TableStorageStats: autoid.InformationSchemaDBID + 63, TableTiFlashTables: autoid.InformationSchemaDBID + 64, TableTiFlashSegments: autoid.InformationSchemaDBID + 65, -======= - TableStorageStats: autoid.InformationSchemaDBID + 63, ->>>>>>> cfbd75465... planner,executor,infoschema: add system table `table_storage_stats` (#15056) } type columnInfo struct { @@ -1227,7 +1221,6 @@ var tableStatementsSummaryCols = []columnInfo{ {name: "PLAN", tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "Sampled execution plan"}, } -<<<<<<< HEAD var tableTableTiFlashTablesCols = []columnInfo{ {name: "DATABASE", tp: mysql.TypeVarchar, size: 64}, {name: "TABLE", tp: mysql.TypeVarchar, size: 64}, @@ -1305,7 +1298,8 @@ var tableTableTiFlashSegmentsCols = []columnInfo{ {name: "DELTA_CACHE_SIZE", tp: mysql.TypeLonglong, size: 64}, {name: "DELTA_INDEX_SIZE", tp: mysql.TypeLonglong, size: 64}, {name: "TIFLASH_INSTANCE", tp: mysql.TypeVarchar, size: 64}, -======= +} + var tableStorageStatsCols = []columnInfo{ {name: "TABLE_SCHEMA", tp: mysql.TypeVarchar, size: 64}, {name: "TABLE_NAME", tp: mysql.TypeVarchar, size: 64}, @@ -1315,7 +1309,6 @@ var tableStorageStatsCols = []columnInfo{ {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 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"}, ->>>>>>> cfbd75465... planner,executor,infoschema: add system table `table_storage_stats` (#15056) } // GetShardingInfo returns a nil or description string for the sharding information of given TableInfo. @@ -1644,12 +1637,9 @@ var tableNameToColumns = map[string][]columnInfo{ TableSequences: tableSequencesCols, TableStatementsSummary: tableStatementsSummaryCols, TableStatementsSummaryHistory: tableStatementsSummaryCols, -<<<<<<< HEAD TableTiFlashTables: tableTableTiFlashTablesCols, TableTiFlashSegments: tableTableTiFlashSegmentsCols, -======= TableStorageStats: tableStorageStatsCols, ->>>>>>> cfbd75465... planner,executor,infoschema: add system table `table_storage_stats` (#15056) } 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 5e1dc97584756..7c869188fd674 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -3057,13 +3057,10 @@ func (b *PlanBuilder) buildMemTable(_ context.Context, dbName model.CIStr, table p.QueryTimeRange = b.timeRangeForSummaryTable() case infoschema.TableSlowQuery: p.Extractor = &SlowQueryExtractor{} -<<<<<<< HEAD case infoschema.TableTiFlashTables, infoschema.TableTiFlashSegments: p.Extractor = &TiFlashSystemTableExtractor{} -======= case infoschema.TableStorageStats: p.Extractor = &TableStorageStatsExtractor{} ->>>>>>> cfbd75465... planner,executor,infoschema: add system table `table_storage_stats` (#15056) } } return p, nil diff --git a/store/helper/helper.go b/store/helper/helper.go index 911e403f9dffc..5a86a2d82713f 100644 --- a/store/helper/helper.go +++ b/store/helper/helper.go @@ -736,7 +736,10 @@ func (h *Helper) GetPDAddr() ([]string, error) { if !ok { return nil, errors.New("not implemented") } - pdAddrs = etcd.EtcdAddrs() + pdAddrs, err := etcd.EtcdAddrs() + if err != nil { + return nil, err + } if len(pdAddrs) == 0 { return nil, errors.New("pd unavailable") } From 17cd6bf4a39f2f8c32312b34d0dafbff9fafc02f Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Wed, 14 Oct 2020 11:37:59 +0800 Subject: [PATCH 3/3] revert file version change Signed-off-by: crazycs520 --- infoschema/tables.go | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100755 => 100644 infoschema/tables.go diff --git a/infoschema/tables.go b/infoschema/tables.go old mode 100755 new mode 100644