From d9c87a2a64b6d72182c2aa27268e44dc709baf3e Mon Sep 17 00:00:00 2001 From: gauss Date: Sat, 7 Mar 2020 23:44:40 +0800 Subject: [PATCH 1/6] executor: Migrate dataForStatistics, dataForTikVRegionPeers, dataForTiDBHotRegions from package to Signed-off-by: gauss --- executor/builder.go | 5 +- executor/infoschema_reader.go | 218 ++++++++++++++++++++++++++++++ infoschema/tables.go | 245 +++------------------------------- 3 files changed, 240 insertions(+), 228 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index bcf0ae77615c1..c036566d1e3fd 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1408,6 +1408,7 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo }, } case strings.ToLower(infoschema.TableSchemata), + strings.ToLower(infoschema.TableStatistics), strings.ToLower(infoschema.TableTiDBIndexes), strings.ToLower(infoschema.TableViews), strings.ToLower(infoschema.TableEngines), @@ -1415,7 +1416,9 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo strings.ToLower(infoschema.TableCharacterSets), strings.ToLower(infoschema.TableKeyColumn), strings.ToLower(infoschema.TableUserPrivileges), - strings.ToLower(infoschema.TableCollationCharacterSetApplicability): + strings.ToLower(infoschema.TableCollationCharacterSetApplicability), + strings.ToLower(infoschema.TableTiKVRegionPeers), + strings.ToLower(infoschema.TableTiDBHotRegions): return &MemTableReaderExec{ baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()), retriever: &memtableRetriever{ diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index a89c8d810401a..052e16187aa33 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -18,13 +18,18 @@ import ( "fmt" "sort" + "github.com/pingcap/errors" "github.com/pingcap/parser/charset" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/store/helper" + "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/pdapi" + "github.com/pingcap/tidb/util/set" ) type memtableRetriever struct { @@ -52,6 +57,8 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex switch e.table.Name.O { case infoschema.TableSchemata: e.rows = dataForSchemata(sctx, dbs) + case infoschema.TableStatistics: + e.rows = dataForStatistics(sctx, dbs) case infoschema.TableTiDBIndexes: e.rows, err = dataForIndexes(sctx, dbs) case infoschema.TableViews: @@ -68,6 +75,10 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex e.rows = dataForCollationCharacterSetApplicability() case infoschema.TableUserPrivileges: e.setDataFromUserPrivileges(sctx) + case infoschema.TableTiKVRegionPeers: + e.rows, err = dataForTikVRegionPeers(sctx) + case infoschema.TableTiDBHotRegions: + e.rows, err = dataForTiDBHotRegions(sctx) } if err != nil { return nil, err @@ -133,6 +144,97 @@ func dataForSchemata(ctx sessionctx.Context, schemas []*model.DBInfo) [][]types. return rows } +func dataForStatistics(ctx sessionctx.Context, schemas []*model.DBInfo) [][]types.Datum { + checker := privilege.GetPrivilegeManager(ctx) + var rows [][]types.Datum + for _, schema := range schemas { + for _, table := range schema.Tables { + if checker != nil && !checker.RequestVerification(ctx.GetSessionVars().ActiveRoles, schema.Name.L, table.Name.L, "", mysql.AllPrivMask) { + continue + } + + rs := dataForStatisticsInTable(schema, table) + rows = append(rows, rs...) + } + } + return rows +} + +func dataForStatisticsInTable(schema *model.DBInfo, table *model.TableInfo) [][]types.Datum { + var rows [][]types.Datum + if table.PKIsHandle { + for _, col := range table.Columns { + if mysql.HasPriKeyFlag(col.Flag) { + record := types.MakeDatums( + infoschema.CatalogVal, // TABLE_CATALOG + schema.Name.O, // TABLE_SCHEMA + table.Name.O, // TABLE_NAME + "0", // NON_UNIQUE + schema.Name.O, // INDEX_SCHEMA + "PRIMARY", // INDEX_NAME + 1, // SEQ_IN_INDEX + col.Name.O, // COLUMN_NAME + "A", // COLLATION + 0, // CARDINALITY + nil, // SUB_PART + nil, // PACKED + "", // NULLABLE + "BTREE", // INDEX_TYPE + "", // COMMENT + "NULL", // Expression + "", // INDEX_COMMENT + ) + rows = append(rows, record) + } + } + } + nameToCol := make(map[string]*model.ColumnInfo, len(table.Columns)) + for _, c := range table.Columns { + nameToCol[c.Name.L] = c + } + for _, index := range table.Indices { + nonUnique := "1" + if index.Unique { + nonUnique = "0" + } + for i, key := range index.Columns { + col := nameToCol[key.Name.L] + nullable := "YES" + if mysql.HasNotNullFlag(col.Flag) { + nullable = "" + } + colName := col.Name.O + expression := "NULL" + tblCol := table.Columns[col.Offset] + if tblCol.Hidden { + colName = "NULL" + expression = fmt.Sprintf("(%s)", tblCol.GeneratedExprString) + } + record := types.MakeDatums( + infoschema.CatalogVal, // TABLE_CATALOG + schema.Name.O, // TABLE_SCHEMA + table.Name.O, // TABLE_NAME + nonUnique, // NON_UNIQUE + schema.Name.O, // INDEX_SCHEMA + index.Name.O, // INDEX_NAME + i+1, // SEQ_IN_INDEX + colName, // COLUMN_NAME + "A", // COLLATION + 0, // CARDINALITY + nil, // SUB_PART + nil, // PACKED + nullable, // NULLABLE + "BTREE", // INDEX_TYPE + "", // COMMENT + expression, // Expression + "", // INDEX_COMMENT + ) + rows = append(rows, record) + } + } + return rows +} + func dataForIndexes(ctx sessionctx.Context, schemas []*model.DBInfo) ([][]types.Datum, error) { checker := privilege.GetPrivilegeManager(ctx) var rows [][]types.Datum @@ -388,6 +490,122 @@ func keyColumnUsageInTable(schema *model.DBInfo, table *model.TableInfo) [][]typ return rows } +func dataForTikVRegionPeers(ctx sessionctx.Context) (records [][]types.Datum, err error) { + 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") + } + tikvHelper := &helper.Helper{ + Store: tikvStore, + RegionCache: tikvStore.GetRegionCache(), + } + regionsInfo, err := tikvHelper.GetRegionsInfo() + if err != nil { + return nil, err + } + for _, region := range regionsInfo.Regions { + rows := newTiKVRegionPeersCols(®ion) + records = append(records, rows...) + } + return records, nil +} + +func newTiKVRegionPeersCols(region *helper.RegionInfo) [][]types.Datum { + records := make([][]types.Datum, 0, len(region.Peers)) + pendingPeerIDSet := set.NewInt64Set() + for _, peer := range region.PendingPeers { + pendingPeerIDSet.Insert(peer.ID) + } + downPeerMap := make(map[int64]int64, len(region.DownPeers)) + for _, peerStat := range region.DownPeers { + downPeerMap[peerStat.ID] = peerStat.DownSec + } + for _, peer := range region.Peers { + row := make([]types.Datum, len(infoschema.TableTiKVRegionPeersCols)) + row[0].SetInt64(region.ID) + row[1].SetInt64(peer.ID) + row[2].SetInt64(peer.StoreID) + if peer.IsLearner { + row[3].SetInt64(1) + } else { + row[3].SetInt64(0) + } + if peer.ID == region.Leader.ID { + row[4].SetInt64(1) + } else { + row[4].SetInt64(0) + } + if pendingPeerIDSet.Exist(peer.ID) { + row[5].SetString(pendingPeer, mysql.DefaultCollationName) + } else if downSec, ok := downPeerMap[peer.ID]; ok { + row[5].SetString(downPeer, mysql.DefaultCollationName) + row[6].SetInt64(downSec) + } else { + row[5].SetString(normalPeer, mysql.DefaultCollationName) + } + records = append(records, row) + } + return records +} + +const ( + normalPeer = "NORMAL" + pendingPeer = "PENDING" + downPeer = "DOWN" +) + +func dataForTiDBHotRegions(ctx sessionctx.Context) (records [][]types.Datum, err error) { + tikvStore, ok := ctx.GetStore().(tikv.Storage) + if !ok { + return nil, errors.New("Information about hot region can be gotten only when the storage is TiKV") + } + allSchemas := ctx.GetSessionVars().TxnCtx.InfoSchema.(infoschema.InfoSchema).AllSchemas() + tikvHelper := &helper.Helper{ + Store: tikvStore, + RegionCache: tikvStore.GetRegionCache(), + } + metrics, err := tikvHelper.ScrapeHotInfo(pdapi.HotRead, allSchemas) + if err != nil { + return nil, err + } + records = append(records, dataForHotRegionByMetrics(metrics, "read")...) + metrics, err = tikvHelper.ScrapeHotInfo(pdapi.HotWrite, allSchemas) + if err != nil { + return nil, err + } + records = append(records, dataForHotRegionByMetrics(metrics, "write")...) + return records, nil +} + +func dataForHotRegionByMetrics(metrics []helper.HotTableIndex, tp string) [][]types.Datum { + rows := make([][]types.Datum, 0, len(metrics)) + for _, tblIndex := range metrics { + row := make([]types.Datum, len(infoschema.TableTiDBHotRegionsCols)) + if tblIndex.IndexName != "" { + row[1].SetInt64(tblIndex.IndexID) + row[4].SetString(tblIndex.IndexName, mysql.DefaultCollationName) + } else { + row[1].SetNull() + row[4].SetNull() + } + row[0].SetInt64(tblIndex.TableID) + row[2].SetString(tblIndex.DbName, mysql.DefaultCollationName) + row[3].SetString(tblIndex.TableName, mysql.DefaultCollationName) + row[5].SetUint64(tblIndex.RegionID) + row[6].SetString(tp, mysql.DefaultCollationName) + if tblIndex.RegionMetric == nil { + row[7].SetNull() + row[8].SetNull() + } else { + row[7].SetInt64(int64(tblIndex.RegionMetric.MaxHotDegree)) + row[8].SetInt64(int64(tblIndex.RegionMetric.Count)) + } + row[9].SetUint64(tblIndex.RegionMetric.FlowBytes) + rows = append(rows, row) + } + return rows +} + func (e *memtableRetriever) setDataFromUserPrivileges(ctx sessionctx.Context) { pm := privilege.GetPrivilegeManager(ctx) e.rows = pm.UserPrivilegesTable() diff --git a/infoschema/tables.go b/infoschema/tables.go index 1cefacc678be9..cb8582cba0a0b 100755 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -46,7 +46,6 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/pdapi" - "github.com/pingcap/tidb/util/set" "github.com/pingcap/tidb/util/sqlexec" ) @@ -56,7 +55,8 @@ const ( tableTables = "TABLES" tableColumns = "COLUMNS" tableColumnStatistics = "COLUMN_STATISTICS" - tableStatistics = "STATISTICS" + // TableStatistics is the string constant of infoschema table + TableStatistics = "STATISTICS" // TableCharacterSets is the string constant of infoschema charactersets memory table TableCharacterSets = "CHARACTER_SETS" // TableCollations is the string constant of infoschema collations memory table @@ -94,13 +94,15 @@ const ( TableCollationCharacterSetApplicability = "COLLATION_CHARACTER_SET_APPLICABILITY" tableProcesslist = "PROCESSLIST" // TableTiDBIndexes is the string constant of infoschema table - TableTiDBIndexes = "TIDB_INDEXES" - tableTiDBHotRegions = "TIDB_HOT_REGIONS" + TableTiDBIndexes = "TIDB_INDEXES" + // TableTiDBHotRegions is the string constant of infoschema table + TableTiDBHotRegions = "TIDB_HOT_REGIONS" tableTiKVStoreStatus = "TIKV_STORE_STATUS" tableAnalyzeStatus = "ANALYZE_STATUS" tableTiKVRegionStatus = "TIKV_REGION_STATUS" - tableTiKVRegionPeers = "TIKV_REGION_PEERS" - tableTiDBServersInfo = "TIDB_SERVERS_INFO" + // TableTiKVRegionPeers is the string constant of infoschema table + TableTiKVRegionPeers = "TIKV_REGION_PEERS" + tableTiDBServersInfo = "TIDB_SERVERS_INFO" // TableSlowQuery is the string constant of slow query memory table. TableSlowQuery = "SLOW_QUERY" // TableClusterInfo is the string constant of cluster info memory table. @@ -135,7 +137,7 @@ var tableIDMap = map[string]int64{ tableTables: autoid.InformationSchemaDBID + 2, tableColumns: autoid.InformationSchemaDBID + 3, tableColumnStatistics: autoid.InformationSchemaDBID + 4, - tableStatistics: autoid.InformationSchemaDBID + 5, + TableStatistics: autoid.InformationSchemaDBID + 5, TableCharacterSets: autoid.InformationSchemaDBID + 6, TableCollations: autoid.InformationSchemaDBID + 7, tableFiles: autoid.InformationSchemaDBID + 8, @@ -166,11 +168,11 @@ var tableIDMap = map[string]int64{ tableProcesslist: autoid.InformationSchemaDBID + 33, TableTiDBIndexes: autoid.InformationSchemaDBID + 34, TableSlowQuery: autoid.InformationSchemaDBID + 35, - tableTiDBHotRegions: autoid.InformationSchemaDBID + 36, + TableTiDBHotRegions: autoid.InformationSchemaDBID + 36, tableTiKVStoreStatus: autoid.InformationSchemaDBID + 37, tableAnalyzeStatus: autoid.InformationSchemaDBID + 38, tableTiKVRegionStatus: autoid.InformationSchemaDBID + 39, - tableTiKVRegionPeers: autoid.InformationSchemaDBID + 40, + TableTiKVRegionPeers: autoid.InformationSchemaDBID + 40, tableTiDBServersInfo: autoid.InformationSchemaDBID + 41, TableClusterInfo: autoid.InformationSchemaDBID + 42, TableClusterConfig: autoid.InformationSchemaDBID + 43, @@ -726,7 +728,8 @@ var slowQueryCols = []columnInfo{ {name: variable.SlowLogQuerySQLStr, tp: mysql.TypeLongBlob, size: types.UnspecifiedLength}, } -var tableTiDBHotRegionsCols = []columnInfo{ +// TableTiDBHotRegionsCols is TiDB hot region mem table columns. +var TableTiDBHotRegionsCols = []columnInfo{ {name: "TABLE_ID", tp: mysql.TypeLonglong, size: 21}, {name: "INDEX_ID", tp: mysql.TypeLonglong, size: 21}, {name: "DB_NAME", tp: mysql.TypeVarchar, size: 64}, @@ -789,7 +792,8 @@ var tableTiKVRegionStatusCols = []columnInfo{ {name: "APPROXIMATE_KEYS", tp: mysql.TypeLonglong, size: 21}, } -var tableTiKVRegionPeersCols = []columnInfo{ +// TableTiKVRegionPeersCols is TiKV region peers mem table columns. +var TableTiKVRegionPeersCols = []columnInfo{ {name: "REGION_ID", tp: mysql.TypeLonglong, size: 21}, {name: "PEER_ID", tp: mysql.TypeLonglong, size: 21}, {name: "STORE_ID", tp: mysql.TypeLonglong, size: 21}, @@ -1025,70 +1029,6 @@ func newTiKVRegionStatusCol(region *helper.RegionInfo, table *helper.TableInfo) return row } -const ( - normalPeer = "NORMAL" - pendingPeer = "PENDING" - downPeer = "DOWN" -) - -func dataForTikVRegionPeers(ctx sessionctx.Context) (records [][]types.Datum, err error) { - 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") - } - tikvHelper := &helper.Helper{ - Store: tikvStore, - RegionCache: tikvStore.GetRegionCache(), - } - regionsInfo, err := tikvHelper.GetRegionsInfo() - if err != nil { - return nil, err - } - for _, region := range regionsInfo.Regions { - rows := newTiKVRegionPeersCols(®ion) - records = append(records, rows...) - } - return records, nil -} - -func newTiKVRegionPeersCols(region *helper.RegionInfo) [][]types.Datum { - records := make([][]types.Datum, 0, len(region.Peers)) - pendingPeerIDSet := set.NewInt64Set() - for _, peer := range region.PendingPeers { - pendingPeerIDSet.Insert(peer.ID) - } - downPeerMap := make(map[int64]int64, len(region.DownPeers)) - for _, peerStat := range region.DownPeers { - downPeerMap[peerStat.ID] = peerStat.DownSec - } - for _, peer := range region.Peers { - row := make([]types.Datum, len(tableTiKVRegionPeersCols)) - row[0].SetInt64(region.ID) - row[1].SetInt64(peer.ID) - row[2].SetInt64(peer.StoreID) - if peer.IsLearner { - row[3].SetInt64(1) - } else { - row[3].SetInt64(0) - } - if peer.ID == region.Leader.ID { - row[4].SetInt64(1) - } else { - row[4].SetInt64(0) - } - if pendingPeerIDSet.Exist(peer.ID) { - row[5].SetString(pendingPeer, mysql.DefaultCollationName) - } else if downSec, ok := downPeerMap[peer.ID]; ok { - row[5].SetString(downPeer, mysql.DefaultCollationName) - row[6].SetInt64(downSec) - } else { - row[5].SetString(normalPeer, mysql.DefaultCollationName) - } - records = append(records, row) - } - return records -} - func dataForTiKVStoreStatus(ctx sessionctx.Context) (records [][]types.Datum, err error) { tikvStore, ok := ctx.GetStore().(tikv.Storage) if !ok { @@ -1557,97 +1497,6 @@ func dataForColumnsInTable(schema *model.DBInfo, tbl *model.TableInfo) [][]types return rows } -func dataForStatistics(ctx sessionctx.Context, schemas []*model.DBInfo) [][]types.Datum { - checker := privilege.GetPrivilegeManager(ctx) - var rows [][]types.Datum - for _, schema := range schemas { - for _, table := range schema.Tables { - if checker != nil && !checker.RequestVerification(ctx.GetSessionVars().ActiveRoles, schema.Name.L, table.Name.L, "", mysql.AllPrivMask) { - continue - } - - rs := dataForStatisticsInTable(schema, table) - rows = append(rows, rs...) - } - } - return rows -} - -func dataForStatisticsInTable(schema *model.DBInfo, table *model.TableInfo) [][]types.Datum { - var rows [][]types.Datum - if table.PKIsHandle { - for _, col := range table.Columns { - if mysql.HasPriKeyFlag(col.Flag) { - record := types.MakeDatums( - CatalogVal, // TABLE_CATALOG - schema.Name.O, // TABLE_SCHEMA - table.Name.O, // TABLE_NAME - "0", // NON_UNIQUE - schema.Name.O, // INDEX_SCHEMA - "PRIMARY", // INDEX_NAME - 1, // SEQ_IN_INDEX - col.Name.O, // COLUMN_NAME - "A", // COLLATION - 0, // CARDINALITY - nil, // SUB_PART - nil, // PACKED - "", // NULLABLE - "BTREE", // INDEX_TYPE - "", // COMMENT - "NULL", // Expression - "", // INDEX_COMMENT - ) - rows = append(rows, record) - } - } - } - nameToCol := make(map[string]*model.ColumnInfo, len(table.Columns)) - for _, c := range table.Columns { - nameToCol[c.Name.L] = c - } - for _, index := range table.Indices { - nonUnique := "1" - if index.Unique { - nonUnique = "0" - } - for i, key := range index.Columns { - col := nameToCol[key.Name.L] - nullable := "YES" - if mysql.HasNotNullFlag(col.Flag) { - nullable = "" - } - colName := col.Name.O - expression := "NULL" - tblCol := table.Columns[col.Offset] - if tblCol.Hidden { - colName = "NULL" - expression = fmt.Sprintf("(%s)", tblCol.GeneratedExprString) - } - record := types.MakeDatums( - CatalogVal, // TABLE_CATALOG - schema.Name.O, // TABLE_SCHEMA - table.Name.O, // TABLE_NAME - nonUnique, // NON_UNIQUE - schema.Name.O, // INDEX_SCHEMA - index.Name.O, // INDEX_NAME - i+1, // SEQ_IN_INDEX - colName, // COLUMN_NAME - "A", // COLLATION - 0, // CARDINALITY - nil, // SUB_PART - nil, // PACKED - nullable, // NULLABLE - "BTREE", // INDEX_TYPE - "", // COMMENT - expression, // Expression - "", // INDEX_COMMENT - ) - rows = append(rows, record) - } - } - return rows -} - const ( primaryKeyType = "PRIMARY KEY" // PrimaryConstraint is the string constant of PRIMARY @@ -1832,58 +1681,6 @@ func dataForPartitions(ctx sessionctx.Context, schemas []*model.DBInfo) ([][]typ return rows, nil } -func dataForTiDBHotRegions(ctx sessionctx.Context) (records [][]types.Datum, err error) { - tikvStore, ok := ctx.GetStore().(tikv.Storage) - if !ok { - return nil, errors.New("Information about hot region can be gotten only when the storage is TiKV") - } - allSchemas := ctx.GetSessionVars().TxnCtx.InfoSchema.(InfoSchema).AllSchemas() - tikvHelper := &helper.Helper{ - Store: tikvStore, - RegionCache: tikvStore.GetRegionCache(), - } - metrics, err := tikvHelper.ScrapeHotInfo(pdapi.HotRead, allSchemas) - if err != nil { - return nil, err - } - records = append(records, dataForHotRegionByMetrics(metrics, "read")...) - metrics, err = tikvHelper.ScrapeHotInfo(pdapi.HotWrite, allSchemas) - if err != nil { - return nil, err - } - records = append(records, dataForHotRegionByMetrics(metrics, "write")...) - return records, nil -} - -func dataForHotRegionByMetrics(metrics []helper.HotTableIndex, tp string) [][]types.Datum { - rows := make([][]types.Datum, 0, len(metrics)) - for _, tblIndex := range metrics { - row := make([]types.Datum, len(tableTiDBHotRegionsCols)) - if tblIndex.IndexName != "" { - row[1].SetInt64(tblIndex.IndexID) - row[4].SetString(tblIndex.IndexName, mysql.DefaultCollationName) - } else { - row[1].SetNull() - row[4].SetNull() - } - row[0].SetInt64(tblIndex.TableID) - row[2].SetString(tblIndex.DbName, mysql.DefaultCollationName) - row[3].SetString(tblIndex.TableName, mysql.DefaultCollationName) - row[5].SetUint64(tblIndex.RegionID) - row[6].SetString(tp, mysql.DefaultCollationName) - if tblIndex.RegionMetric == nil { - row[7].SetNull() - row[8].SetNull() - } else { - row[7].SetInt64(int64(tblIndex.RegionMetric.MaxHotDegree)) - row[8].SetInt64(int64(tblIndex.RegionMetric.Count)) - } - row[9].SetUint64(tblIndex.RegionMetric.FlowBytes) - rows = append(rows, row) - } - return rows -} - // DataForAnalyzeStatus gets all the analyze jobs. func DataForAnalyzeStatus(ctx sessionctx.Context) (rows [][]types.Datum) { checker := privilege.GetPrivilegeManager(ctx) @@ -2183,7 +1980,7 @@ var tableNameToColumns = map[string][]columnInfo{ tableTables: tablesCols, tableColumns: columnsCols, tableColumnStatistics: columnStatisticsCols, - tableStatistics: statisticsCols, + TableStatistics: statisticsCols, TableCharacterSets: charsetCols, TableCollations: collationsCols, tableFiles: filesCols, @@ -2213,11 +2010,11 @@ var tableNameToColumns = map[string][]columnInfo{ tableProcesslist: tableProcesslistCols, TableTiDBIndexes: tableTiDBIndexesCols, TableSlowQuery: slowQueryCols, - tableTiDBHotRegions: tableTiDBHotRegionsCols, + TableTiDBHotRegions: TableTiDBHotRegionsCols, tableTiKVStoreStatus: tableTiKVStoreStatusCols, tableAnalyzeStatus: tableAnalyzeStatusCols, tableTiKVRegionStatus: tableTiKVRegionStatusCols, - tableTiKVRegionPeers: tableTiKVRegionPeersCols, + TableTiKVRegionPeers: TableTiKVRegionPeersCols, tableTiDBServersInfo: tableTiDBServersInfoCols, TableClusterInfo: tableClusterInfoCols, TableClusterConfig: tableClusterConfigCols, @@ -2276,8 +2073,6 @@ func (it *infoschemaTable) getRows(ctx sessionctx.Context, cols []*table.Column) fullRows, err = dataForTables(ctx, dbs) case tableColumns: fullRows = dataForColumns(ctx, dbs) - case tableStatistics: - fullRows = dataForStatistics(ctx, dbs) case tableSessionVar: fullRows, err = dataForSessionVar(ctx) case tableConstraints: @@ -2305,16 +2100,12 @@ func (it *infoschemaTable) getRows(ctx sessionctx.Context, cols []*table.Column) case tableTableSpaces: case tableProcesslist: fullRows = dataForProcesslist(ctx) - case tableTiDBHotRegions: - fullRows, err = dataForTiDBHotRegions(ctx) case tableTiKVStoreStatus: fullRows, err = dataForTiKVStoreStatus(ctx) case tableAnalyzeStatus: fullRows = DataForAnalyzeStatus(ctx) case tableTiKVRegionStatus: fullRows, err = dataForTiKVRegionStatus(ctx) - case tableTiKVRegionPeers: - fullRows, err = dataForTikVRegionPeers(ctx) case tableTiDBServersInfo: fullRows, err = dataForServersInfo() case TableClusterInfo: From 4afb56042a61a11eaba91372246798f067e4dd15 Mon Sep 17 00:00:00 2001 From: gauss Date: Mon, 9 Mar 2020 10:55:59 +0800 Subject: [PATCH 2/6] fix conflict --- infoschema/tables.go | 155 ------------------------------------------- 1 file changed, 155 deletions(-) diff --git a/infoschema/tables.go b/infoschema/tables.go index d229bd74d4ec6..af0113e09f425 100755 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -45,8 +45,6 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/pdapi" - "github.com/pingcap/tidb/util/set" - "github.com/pingcap/tidb/util/sqlexec" ) const ( @@ -1340,159 +1338,6 @@ func dataForPseudoProfiling() [][]types.Datum { return rows } -func dataForPartitions(ctx sessionctx.Context, schemas []*model.DBInfo) ([][]types.Datum, error) { - tableRowsMap, colLengthMap, err := tableStatsCache.get(ctx) - if err != nil { - return nil, err - } - checker := privilege.GetPrivilegeManager(ctx) - var rows [][]types.Datum - createTimeTp := partitionsCols[18].tp - for _, schema := range schemas { - for _, table := range schema.Tables { - if checker != nil && !checker.RequestVerification(ctx.GetSessionVars().ActiveRoles, schema.Name.L, table.Name.L, "", mysql.SelectPriv) { - continue - } - createTime := types.NewTime(types.FromGoTime(table.GetUpdateTime()), createTimeTp, types.DefaultFsp) - - var rowCount, dataLength, indexLength uint64 - if table.GetPartitionInfo() == nil { - rowCount = tableRowsMap[table.ID] - dataLength, indexLength = getDataAndIndexLength(table, table.ID, rowCount, colLengthMap) - avgRowLength := uint64(0) - if rowCount != 0 { - avgRowLength = dataLength / rowCount - } - record := types.MakeDatums( - CatalogVal, // TABLE_CATALOG - schema.Name.O, // TABLE_SCHEMA - table.Name.O, // TABLE_NAME - nil, // PARTITION_NAME - nil, // SUBPARTITION_NAME - nil, // PARTITION_ORDINAL_POSITION - nil, // SUBPARTITION_ORDINAL_POSITION - nil, // PARTITION_METHOD - nil, // SUBPARTITION_METHOD - nil, // PARTITION_EXPRESSION - nil, // SUBPARTITION_EXPRESSION - nil, // PARTITION_DESCRIPTION - rowCount, // TABLE_ROWS - avgRowLength, // AVG_ROW_LENGTH - dataLength, // DATA_LENGTH - nil, // MAX_DATA_LENGTH - indexLength, // INDEX_LENGTH - nil, // DATA_FREE - createTime, // CREATE_TIME - nil, // UPDATE_TIME - nil, // CHECK_TIME - nil, // CHECKSUM - nil, // PARTITION_COMMENT - nil, // NODEGROUP - nil, // TABLESPACE_NAME - ) - rows = append(rows, record) - } else { - for i, pi := range table.GetPartitionInfo().Definitions { - rowCount = tableRowsMap[pi.ID] - dataLength, indexLength = getDataAndIndexLength(table, pi.ID, tableRowsMap[pi.ID], colLengthMap) - - avgRowLength := uint64(0) - if rowCount != 0 { - avgRowLength = dataLength / rowCount - } - - var partitionDesc string - if table.Partition.Type == model.PartitionTypeRange { - partitionDesc = pi.LessThan[0] - } - - record := types.MakeDatums( - CatalogVal, // TABLE_CATALOG - schema.Name.O, // TABLE_SCHEMA - table.Name.O, // TABLE_NAME - pi.Name.O, // PARTITION_NAME - nil, // SUBPARTITION_NAME - i+1, // PARTITION_ORDINAL_POSITION - nil, // SUBPARTITION_ORDINAL_POSITION - table.Partition.Type.String(), // PARTITION_METHOD - nil, // SUBPARTITION_METHOD - table.Partition.Expr, // PARTITION_EXPRESSION - nil, // SUBPARTITION_EXPRESSION - partitionDesc, // PARTITION_DESCRIPTION - rowCount, // TABLE_ROWS - avgRowLength, // AVG_ROW_LENGTH - dataLength, // DATA_LENGTH - uint64(0), // MAX_DATA_LENGTH - indexLength, // INDEX_LENGTH - uint64(0), // DATA_FREE - createTime, // CREATE_TIME - nil, // UPDATE_TIME - nil, // CHECK_TIME - nil, // CHECKSUM - pi.Comment, // PARTITION_COMMENT - nil, // NODEGROUP - nil, // TABLESPACE_NAME - ) - rows = append(rows, record) - } - } - } - } - return rows, nil -} - -func dataForTiDBHotRegions(ctx sessionctx.Context) (records [][]types.Datum, err error) { - tikvStore, ok := ctx.GetStore().(tikv.Storage) - if !ok { - return nil, errors.New("Information about hot region can be gotten only when the storage is TiKV") - } - allSchemas := ctx.GetSessionVars().TxnCtx.InfoSchema.(InfoSchema).AllSchemas() - tikvHelper := &helper.Helper{ - Store: tikvStore, - RegionCache: tikvStore.GetRegionCache(), - } - metrics, err := tikvHelper.ScrapeHotInfo(pdapi.HotRead, allSchemas) - if err != nil { - return nil, err - } - records = append(records, dataForHotRegionByMetrics(metrics, "read")...) - metrics, err = tikvHelper.ScrapeHotInfo(pdapi.HotWrite, allSchemas) - if err != nil { - return nil, err - } - records = append(records, dataForHotRegionByMetrics(metrics, "write")...) - return records, nil -} - -func dataForHotRegionByMetrics(metrics []helper.HotTableIndex, tp string) [][]types.Datum { - rows := make([][]types.Datum, 0, len(metrics)) - for _, tblIndex := range metrics { - row := make([]types.Datum, len(tableTiDBHotRegionsCols)) - if tblIndex.IndexName != "" { - row[1].SetInt64(tblIndex.IndexID) - row[4].SetString(tblIndex.IndexName, mysql.DefaultCollationName) - } else { - row[1].SetNull() - row[4].SetNull() - } - row[0].SetInt64(tblIndex.TableID) - row[2].SetString(tblIndex.DbName, mysql.DefaultCollationName) - row[3].SetString(tblIndex.TableName, mysql.DefaultCollationName) - row[5].SetUint64(tblIndex.RegionID) - row[6].SetString(tp, mysql.DefaultCollationName) - if tblIndex.RegionMetric == nil { - row[7].SetNull() - row[8].SetNull() - } else { - row[7].SetInt64(int64(tblIndex.RegionMetric.MaxHotDegree)) - row[8].SetInt64(int64(tblIndex.RegionMetric.Count)) - } - row[9].SetUint64(tblIndex.RegionMetric.FlowBytes) - rows = append(rows, row) - } - return rows -} - // DataForAnalyzeStatus gets all the analyze jobs. func DataForAnalyzeStatus(ctx sessionctx.Context) (rows [][]types.Datum) { checker := privilege.GetPrivilegeManager(ctx) From a5fd6a81cfb1b1ae8079bb971c4de05c40868852 Mon Sep 17 00:00:00 2001 From: gauss Date: Mon, 9 Mar 2020 16:57:54 +0800 Subject: [PATCH 3/6] fix conflict --- executor/infoschema_reader.go | 38 +++++++++++++++++------------------ 1 file changed, 19 insertions(+), 19 deletions(-) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 451f7c069727a..b75a5a17ef136 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -62,7 +62,7 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex case infoschema.TableSchemata: e.setDataFromSchemata(sctx, dbs) case infoschema.TableStatistics: - e.rows = dataForStatistics(sctx, dbs) + e.setDataForStatistics(sctx, dbs) case infoschema.TableTables: err = e.dataForTables(sctx, dbs) case infoschema.TablePartitions: @@ -84,9 +84,9 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex case infoschema.TableUserPrivileges: e.setDataFromUserPrivileges(sctx) case infoschema.TableTiKVRegionPeers: - e.rows, err = dataForTikVRegionPeers(sctx) + err = e.dataForTikVRegionPeers(sctx) case infoschema.TableTiDBHotRegions: - e.rows, err = dataForTiDBHotRegions(sctx) + err = e.dataForTiDBHotRegions(sctx) } if err != nil { return nil, err @@ -280,7 +280,7 @@ func (e *memtableRetriever) setDataFromSchemata(ctx sessionctx.Context, schemas e.rows = rows } -func dataForStatistics(ctx sessionctx.Context, schemas []*model.DBInfo) [][]types.Datum { +func (e *memtableRetriever) setDataForStatistics(ctx sessionctx.Context, schemas []*model.DBInfo) { checker := privilege.GetPrivilegeManager(ctx) var rows [][]types.Datum for _, schema := range schemas { @@ -293,7 +293,7 @@ func dataForStatistics(ctx sessionctx.Context, schemas []*model.DBInfo) [][]type rows = append(rows, rs...) } } - return rows + e.rows = rows } func dataForStatisticsInTable(schema *model.DBInfo, table *model.TableInfo) [][]types.Datum { @@ -852,10 +852,10 @@ func keyColumnUsageInTable(schema *model.DBInfo, table *model.TableInfo) [][]typ return rows } -func dataForTikVRegionPeers(ctx sessionctx.Context) (records [][]types.Datum, err error) { +func (e *memtableRetriever) dataForTikVRegionPeers(ctx sessionctx.Context) (err error) { 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") + return errors.New("Information about TiKV region status can be gotten only when the storage is TiKV") } tikvHelper := &helper.Helper{ Store: tikvStore, @@ -863,13 +863,15 @@ func dataForTikVRegionPeers(ctx sessionctx.Context) (records [][]types.Datum, er } regionsInfo, err := tikvHelper.GetRegionsInfo() if err != nil { - return nil, err + return err } + var records [][]types.Datum for _, region := range regionsInfo.Regions { rows := newTiKVRegionPeersCols(®ion) records = append(records, rows...) } - return records, nil + e.rows = records + return nil } func newTiKVRegionPeersCols(region *helper.RegionInfo) [][]types.Datum { @@ -916,10 +918,10 @@ const ( downPeer = "DOWN" ) -func dataForTiDBHotRegions(ctx sessionctx.Context) (records [][]types.Datum, err error) { +func (e *memtableRetriever) dataForTiDBHotRegions(ctx sessionctx.Context) (err error) { tikvStore, ok := ctx.GetStore().(tikv.Storage) if !ok { - return nil, errors.New("Information about hot region can be gotten only when the storage is TiKV") + return errors.New("Information about hot region can be gotten only when the storage is TiKV") } allSchemas := ctx.GetSessionVars().TxnCtx.InfoSchema.(infoschema.InfoSchema).AllSchemas() tikvHelper := &helper.Helper{ @@ -928,15 +930,18 @@ func dataForTiDBHotRegions(ctx sessionctx.Context) (records [][]types.Datum, err } metrics, err := tikvHelper.ScrapeHotInfo(pdapi.HotRead, allSchemas) if err != nil { - return nil, err + return err } + var records [][]types.Datum records = append(records, dataForHotRegionByMetrics(metrics, "read")...) metrics, err = tikvHelper.ScrapeHotInfo(pdapi.HotWrite, allSchemas) if err != nil { - return nil, err + return err } records = append(records, dataForHotRegionByMetrics(metrics, "write")...) - return records, nil + + e.rows = records + return nil } func dataForHotRegionByMetrics(metrics []helper.HotTableIndex, tp string) [][]types.Datum { @@ -967,8 +972,3 @@ func dataForHotRegionByMetrics(metrics []helper.HotTableIndex, tp string) [][]ty } return rows } - -func (e *memtableRetriever) setDataFromUserPrivileges(ctx sessionctx.Context) { - pm := privilege.GetPrivilegeManager(ctx) - e.rows = pm.UserPrivilegesTable() -} From 536009720a3b30b4d884145080e3b25bd29bd9fc Mon Sep 17 00:00:00 2001 From: gauss Date: Tue, 10 Mar 2020 13:42:18 +0800 Subject: [PATCH 4/6] remove global private function --- executor/infoschema_reader.go | 33 ++++++++++++--------------------- 1 file changed, 12 insertions(+), 21 deletions(-) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 5ed410d627645..4424fca167275 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -283,21 +283,17 @@ func (e *memtableRetriever) setDataFromSchemata(ctx sessionctx.Context, schemas func (e *memtableRetriever) setDataForStatistics(ctx sessionctx.Context, schemas []*model.DBInfo) { checker := privilege.GetPrivilegeManager(ctx) - var rows [][]types.Datum for _, schema := range schemas { for _, table := range schema.Tables { if checker != nil && !checker.RequestVerification(ctx.GetSessionVars().ActiveRoles, schema.Name.L, table.Name.L, "", mysql.AllPrivMask) { continue } - - rs := dataForStatisticsInTable(schema, table) - rows = append(rows, rs...) + e.dataForStatisticsInTable(schema, table) } } - e.rows = rows } -func dataForStatisticsInTable(schema *model.DBInfo, table *model.TableInfo) [][]types.Datum { +func (e *memtableRetriever) dataForStatisticsInTable(schema *model.DBInfo, table *model.TableInfo) { var rows [][]types.Datum if table.PKIsHandle { for _, col := range table.Columns { @@ -369,7 +365,7 @@ func dataForStatisticsInTable(schema *model.DBInfo, table *model.TableInfo) [][] rows = append(rows, record) } } - return rows + e.rows = append(e.rows, rows...) } func (e *memtableRetriever) dataForTables(ctx sessionctx.Context, schemas []*model.DBInfo) error { @@ -860,7 +856,7 @@ func keyColumnUsageInTable(schema *model.DBInfo, table *model.TableInfo) [][]typ return rows } -func (e *memtableRetriever) dataForTikVRegionPeers(ctx sessionctx.Context) (err error) { +func (e *memtableRetriever) dataForTikVRegionPeers(ctx sessionctx.Context) error { tikvStore, ok := ctx.GetStore().(tikv.Storage) if !ok { return errors.New("Information about TiKV region status can be gotten only when the storage is TiKV") @@ -873,16 +869,13 @@ func (e *memtableRetriever) dataForTikVRegionPeers(ctx sessionctx.Context) (err if err != nil { return err } - var records [][]types.Datum for _, region := range regionsInfo.Regions { - rows := newTiKVRegionPeersCols(®ion) - records = append(records, rows...) + e.newTiKVRegionPeersCols(®ion) } - e.rows = records return nil } -func newTiKVRegionPeersCols(region *helper.RegionInfo) [][]types.Datum { +func (e *memtableRetriever) newTiKVRegionPeersCols(region *helper.RegionInfo) { records := make([][]types.Datum, 0, len(region.Peers)) pendingPeerIDSet := set.NewInt64Set() for _, peer := range region.PendingPeers { @@ -917,7 +910,7 @@ func newTiKVRegionPeersCols(region *helper.RegionInfo) [][]types.Datum { } records = append(records, row) } - return records + e.rows = append(e.rows, records...) } const ( @@ -926,7 +919,7 @@ const ( downPeer = "DOWN" ) -func (e *memtableRetriever) dataForTiDBHotRegions(ctx sessionctx.Context) (err error) { +func (e *memtableRetriever) dataForTiDBHotRegions(ctx sessionctx.Context) error { tikvStore, ok := ctx.GetStore().(tikv.Storage) if !ok { return errors.New("Information about hot region can be gotten only when the storage is TiKV") @@ -940,19 +933,17 @@ func (e *memtableRetriever) dataForTiDBHotRegions(ctx sessionctx.Context) (err e if err != nil { return err } - var records [][]types.Datum - records = append(records, dataForHotRegionByMetrics(metrics, "read")...) + e.dataForHotRegionByMetrics(metrics, "read") metrics, err = tikvHelper.ScrapeHotInfo(pdapi.HotWrite, allSchemas) if err != nil { return err } - records = append(records, dataForHotRegionByMetrics(metrics, "write")...) + e.dataForHotRegionByMetrics(metrics, "write") - e.rows = records return nil } -func dataForHotRegionByMetrics(metrics []helper.HotTableIndex, tp string) [][]types.Datum { +func (e *memtableRetriever) dataForHotRegionByMetrics(metrics []helper.HotTableIndex, tp string) { rows := make([][]types.Datum, 0, len(metrics)) for _, tblIndex := range metrics { row := make([]types.Datum, len(infoschema.TableTiDBHotRegionsCols)) @@ -978,5 +969,5 @@ func dataForHotRegionByMetrics(metrics []helper.HotTableIndex, tp string) [][]ty row[9].SetUint64(tblIndex.RegionMetric.FlowBytes) rows = append(rows, row) } - return rows + e.rows = append(e.rows, rows...) } From db21755dddac2821d8a263178c118b4398d10a43 Mon Sep 17 00:00:00 2001 From: gauss Date: Wed, 11 Mar 2020 09:40:34 +0800 Subject: [PATCH 5/6] polish code --- executor/infoschema_reader.go | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 62dc32749bc96..c01cd620dd4c7 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -85,9 +85,9 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex case infoschema.TableUserPrivileges: e.setDataFromUserPrivileges(sctx) case infoschema.TableTiKVRegionPeers: - err = e.dataForTikVRegionPeers(sctx) + err = e.setDataForTikVRegionPeers(sctx) case infoschema.TableTiDBHotRegions: - err = e.dataForTiDBHotRegions(sctx) + err = e.setDataForTiDBHotRegions(sctx) case infoschema.TableConstraints: e.setDataFromTableConstraints(sctx, dbs) } @@ -850,7 +850,7 @@ func keyColumnUsageInTable(schema *model.DBInfo, table *model.TableInfo) [][]typ return rows } -func (e *memtableRetriever) dataForTikVRegionPeers(ctx sessionctx.Context) error { +func (e *memtableRetriever) setDataForTikVRegionPeers(ctx sessionctx.Context) error { tikvStore, ok := ctx.GetStore().(tikv.Storage) if !ok { return errors.New("Information about TiKV region status can be gotten only when the storage is TiKV") @@ -864,12 +864,12 @@ func (e *memtableRetriever) dataForTikVRegionPeers(ctx sessionctx.Context) error return err } for _, region := range regionsInfo.Regions { - e.newTiKVRegionPeersCols(®ion) + e.setNewTiKVRegionPeersCols(®ion) } return nil } -func (e *memtableRetriever) newTiKVRegionPeersCols(region *helper.RegionInfo) { +func (e *memtableRetriever) setNewTiKVRegionPeersCols(region *helper.RegionInfo) { records := make([][]types.Datum, 0, len(region.Peers)) pendingPeerIDSet := set.NewInt64Set() for _, peer := range region.PendingPeers { @@ -913,7 +913,7 @@ const ( downPeer = "DOWN" ) -func (e *memtableRetriever) dataForTiDBHotRegions(ctx sessionctx.Context) error { +func (e *memtableRetriever) setDataForTiDBHotRegions(ctx sessionctx.Context) error { tikvStore, ok := ctx.GetStore().(tikv.Storage) if !ok { return errors.New("Information about hot region can be gotten only when the storage is TiKV") @@ -927,17 +927,16 @@ func (e *memtableRetriever) dataForTiDBHotRegions(ctx sessionctx.Context) error if err != nil { return err } - e.dataForHotRegionByMetrics(metrics, "read") + e.setDataForHotRegionByMetrics(metrics, "read") metrics, err = tikvHelper.ScrapeHotInfo(pdapi.HotWrite, allSchemas) if err != nil { return err } - e.dataForHotRegionByMetrics(metrics, "write") - + e.setDataForHotRegionByMetrics(metrics, "write") return nil } -func (e *memtableRetriever) dataForHotRegionByMetrics(metrics []helper.HotTableIndex, tp string) { +func (e *memtableRetriever) setDataForHotRegionByMetrics(metrics []helper.HotTableIndex, tp string) { rows := make([][]types.Datum, 0, len(metrics)) for _, tblIndex := range metrics { row := make([]types.Datum, len(infoschema.TableTiDBHotRegionsCols)) From a88b372a78d364badba44fa373a1ad81225132be Mon Sep 17 00:00:00 2001 From: gauss Date: Wed, 11 Mar 2020 10:00:56 +0800 Subject: [PATCH 6/6] rename function --- executor/infoschema_reader.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index c01cd620dd4c7..9094fafb36adb 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -282,12 +282,12 @@ func (e *memtableRetriever) setDataForStatistics(ctx sessionctx.Context, schemas if checker != nil && !checker.RequestVerification(ctx.GetSessionVars().ActiveRoles, schema.Name.L, table.Name.L, "", mysql.AllPrivMask) { continue } - e.dataForStatisticsInTable(schema, table) + e.setDataForStatisticsInTable(schema, table) } } } -func (e *memtableRetriever) dataForStatisticsInTable(schema *model.DBInfo, table *model.TableInfo) { +func (e *memtableRetriever) setDataForStatisticsInTable(schema *model.DBInfo, table *model.TableInfo) { var rows [][]types.Datum if table.PKIsHandle { for _, col := range table.Columns {