You can not select more than 25 topics
Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
1889 lines
58 KiB
1889 lines
58 KiB
// Copyright 2020 PingCAP, Inc.
|
|
//
|
|
// Licensed under the Apache License, Version 2.0 (the "License");
|
|
// you may not use this file except in compliance with the License.
|
|
// You may obtain a copy of the License at
|
|
//
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
|
//
|
|
// Unless required by applicable law or agreed to in writing, software
|
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
// See the License for the specific language governing permissions and
|
|
// limitations under the License.
|
|
|
|
package executor
|
|
|
|
import (
|
|
"context"
|
|
"fmt"
|
|
"io/ioutil"
|
|
"net/http"
|
|
"sort"
|
|
"strconv"
|
|
"strings"
|
|
"sync"
|
|
"time"
|
|
|
|
"github.com/cznic/mathutil"
|
|
"github.com/pingcap/errors"
|
|
"github.com/pingcap/parser/charset"
|
|
"github.com/pingcap/parser/model"
|
|
"github.com/pingcap/parser/mysql"
|
|
"github.com/pingcap/parser/terror"
|
|
"github.com/pingcap/tidb/domain"
|
|
"github.com/pingcap/tidb/domain/infosync"
|
|
"github.com/pingcap/tidb/infoschema"
|
|
"github.com/pingcap/tidb/meta/autoid"
|
|
plannercore "github.com/pingcap/tidb/planner/core"
|
|
"github.com/pingcap/tidb/privilege"
|
|
"github.com/pingcap/tidb/sessionctx"
|
|
"github.com/pingcap/tidb/sessionctx/variable"
|
|
"github.com/pingcap/tidb/statistics"
|
|
"github.com/pingcap/tidb/store/helper"
|
|
"github.com/pingcap/tidb/store/tikv"
|
|
"github.com/pingcap/tidb/table"
|
|
"github.com/pingcap/tidb/types"
|
|
"github.com/pingcap/tidb/util"
|
|
"github.com/pingcap/tidb/util/chunk"
|
|
"github.com/pingcap/tidb/util/collate"
|
|
"github.com/pingcap/tidb/util/pdapi"
|
|
"github.com/pingcap/tidb/util/set"
|
|
"github.com/pingcap/tidb/util/sqlexec"
|
|
"github.com/pingcap/tidb/util/stmtsummary"
|
|
"go.etcd.io/etcd/clientv3"
|
|
)
|
|
|
|
type memtableRetriever struct {
|
|
dummyCloser
|
|
table *model.TableInfo
|
|
columns []*model.ColumnInfo
|
|
rows [][]types.Datum
|
|
rowIdx int
|
|
retrieved bool
|
|
initialized bool
|
|
}
|
|
|
|
// retrieve implements the infoschemaRetriever interface
|
|
func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Context) ([][]types.Datum, error) {
|
|
if e.retrieved {
|
|
return nil, nil
|
|
}
|
|
|
|
//Cache the ret full rows in schemataRetriever
|
|
if !e.initialized {
|
|
is := infoschema.GetInfoSchema(sctx)
|
|
dbs := is.AllSchemas()
|
|
sort.Sort(infoschema.SchemasSorter(dbs))
|
|
var err error
|
|
switch e.table.Name.O {
|
|
case infoschema.TableSchemata:
|
|
e.setDataFromSchemata(sctx, dbs)
|
|
case infoschema.TableStatistics:
|
|
e.setDataForStatistics(sctx, dbs)
|
|
case infoschema.TableTables:
|
|
err = e.setDataFromTables(sctx, dbs)
|
|
case infoschema.TableSequences:
|
|
e.setDataFromSequences(sctx, dbs)
|
|
case infoschema.TablePartitions:
|
|
err = e.setDataFromPartitions(sctx, dbs)
|
|
case infoschema.TableClusterInfo:
|
|
err = e.dataForTiDBClusterInfo(sctx)
|
|
case infoschema.TableAnalyzeStatus:
|
|
e.setDataForAnalyzeStatus(sctx)
|
|
case infoschema.TableTiDBIndexes:
|
|
e.setDataFromIndexes(sctx, dbs)
|
|
case infoschema.TableViews:
|
|
e.setDataFromViews(sctx, dbs)
|
|
case infoschema.TableEngines:
|
|
e.setDataFromEngines()
|
|
case infoschema.TableCharacterSets:
|
|
e.setDataFromCharacterSets()
|
|
case infoschema.TableCollations:
|
|
e.setDataFromCollations()
|
|
case infoschema.TableKeyColumn:
|
|
e.setDataFromKeyColumnUsage(sctx, dbs)
|
|
case infoschema.TableMetricTables:
|
|
e.setDataForMetricTables(sctx)
|
|
case infoschema.TableProfiling:
|
|
e.setDataForPseudoProfiling(sctx)
|
|
case infoschema.TableCollationCharacterSetApplicability:
|
|
e.dataForCollationCharacterSetApplicability()
|
|
case infoschema.TableProcesslist:
|
|
e.setDataForProcessList(sctx)
|
|
case infoschema.ClusterTableProcesslist:
|
|
err = e.setDataForClusterProcessList(sctx)
|
|
case infoschema.TableUserPrivileges:
|
|
e.setDataFromUserPrivileges(sctx)
|
|
case infoschema.TableTiKVRegionPeers:
|
|
err = e.setDataForTikVRegionPeers(sctx)
|
|
case infoschema.TableTiDBHotRegions:
|
|
err = e.setDataForTiDBHotRegions(sctx)
|
|
case infoschema.TableConstraints:
|
|
e.setDataFromTableConstraints(sctx, dbs)
|
|
case infoschema.TableSessionVar:
|
|
err = e.setDataFromSessionVar(sctx)
|
|
case infoschema.TableTiDBServersInfo:
|
|
err = e.setDataForServersInfo()
|
|
case infoschema.TableTiFlashReplica:
|
|
e.dataForTableTiFlashReplica(sctx, dbs)
|
|
case infoschema.TableStatementsSummary,
|
|
infoschema.TableStatementsSummaryHistory,
|
|
infoschema.ClusterTableStatementsSummary,
|
|
infoschema.ClusterTableStatementsSummaryHistory:
|
|
err = e.setDataForStatementsSummary(sctx, e.table.Name.O)
|
|
}
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
e.initialized = true
|
|
}
|
|
|
|
//Adjust the amount of each return
|
|
maxCount := 1024
|
|
retCount := maxCount
|
|
if e.rowIdx+maxCount > len(e.rows) {
|
|
retCount = len(e.rows) - e.rowIdx
|
|
e.retrieved = true
|
|
}
|
|
ret := make([][]types.Datum, retCount)
|
|
for i := e.rowIdx; i < e.rowIdx+retCount; i++ {
|
|
ret[i-e.rowIdx] = e.rows[i]
|
|
}
|
|
e.rowIdx += retCount
|
|
return adjustColumns(ret, e.columns, e.table), nil
|
|
}
|
|
|
|
func getRowCountAllTable(ctx sessionctx.Context) (map[int64]uint64, error) {
|
|
rows, _, err := ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL("select table_id, count from mysql.stats_meta")
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
rowCountMap := make(map[int64]uint64, len(rows))
|
|
for _, row := range rows {
|
|
tableID := row.GetInt64(0)
|
|
rowCnt := row.GetUint64(1)
|
|
rowCountMap[tableID] = rowCnt
|
|
}
|
|
return rowCountMap, nil
|
|
}
|
|
|
|
type tableHistID struct {
|
|
tableID int64
|
|
histID int64
|
|
}
|
|
|
|
func getColLengthAllTables(ctx sessionctx.Context) (map[tableHistID]uint64, error) {
|
|
rows, _, err := ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL("select table_id, hist_id, tot_col_size from mysql.stats_histograms where is_index = 0")
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
colLengthMap := make(map[tableHistID]uint64, len(rows))
|
|
for _, row := range rows {
|
|
tableID := row.GetInt64(0)
|
|
histID := row.GetInt64(1)
|
|
totalSize := row.GetInt64(2)
|
|
if totalSize < 0 {
|
|
totalSize = 0
|
|
}
|
|
colLengthMap[tableHistID{tableID: tableID, histID: histID}] = uint64(totalSize)
|
|
}
|
|
return colLengthMap, nil
|
|
}
|
|
|
|
func getDataAndIndexLength(info *model.TableInfo, physicalID int64, rowCount uint64, columnLengthMap map[tableHistID]uint64) (uint64, uint64) {
|
|
columnLength := make(map[string]uint64, len(info.Columns))
|
|
for _, col := range info.Columns {
|
|
if col.State != model.StatePublic {
|
|
continue
|
|
}
|
|
length := col.FieldType.StorageLength()
|
|
if length != types.VarStorageLen {
|
|
columnLength[col.Name.L] = rowCount * uint64(length)
|
|
} else {
|
|
length := columnLengthMap[tableHistID{tableID: physicalID, histID: col.ID}]
|
|
columnLength[col.Name.L] = length
|
|
}
|
|
}
|
|
dataLength, indexLength := uint64(0), uint64(0)
|
|
for _, length := range columnLength {
|
|
dataLength += length
|
|
}
|
|
for _, idx := range info.Indices {
|
|
if idx.State != model.StatePublic {
|
|
continue
|
|
}
|
|
for _, col := range idx.Columns {
|
|
if col.Length == types.UnspecifiedLength {
|
|
indexLength += columnLength[col.Name.L]
|
|
} else {
|
|
indexLength += rowCount * uint64(col.Length)
|
|
}
|
|
}
|
|
}
|
|
return dataLength, indexLength
|
|
}
|
|
|
|
type statsCache struct {
|
|
mu sync.RWMutex
|
|
modifyTime time.Time
|
|
tableRows map[int64]uint64
|
|
colLength map[tableHistID]uint64
|
|
}
|
|
|
|
var tableStatsCache = &statsCache{}
|
|
|
|
// TableStatsCacheExpiry is the expiry time for table stats cache.
|
|
var TableStatsCacheExpiry = 3 * time.Second
|
|
|
|
func (c *statsCache) get(ctx sessionctx.Context) (map[int64]uint64, map[tableHistID]uint64, error) {
|
|
c.mu.RLock()
|
|
if time.Since(c.modifyTime) < TableStatsCacheExpiry {
|
|
tableRows, colLength := c.tableRows, c.colLength
|
|
c.mu.RUnlock()
|
|
return tableRows, colLength, nil
|
|
}
|
|
c.mu.RUnlock()
|
|
|
|
c.mu.Lock()
|
|
defer c.mu.Unlock()
|
|
if time.Since(c.modifyTime) < TableStatsCacheExpiry {
|
|
return c.tableRows, c.colLength, nil
|
|
}
|
|
tableRows, err := getRowCountAllTable(ctx)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
colLength, err := getColLengthAllTables(ctx)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
c.tableRows = tableRows
|
|
c.colLength = colLength
|
|
c.modifyTime = time.Now()
|
|
return tableRows, colLength, nil
|
|
}
|
|
|
|
func getAutoIncrementID(ctx sessionctx.Context, schema *model.DBInfo, tblInfo *model.TableInfo) (int64, error) {
|
|
is := infoschema.GetInfoSchema(ctx)
|
|
tbl, err := is.TableByName(schema.Name, tblInfo.Name)
|
|
if err != nil {
|
|
return 0, err
|
|
}
|
|
return tbl.Allocators(ctx).Get(autoid.RowIDAllocType).Base() + 1, nil
|
|
}
|
|
|
|
func (e *memtableRetriever) setDataFromSchemata(ctx sessionctx.Context, schemas []*model.DBInfo) {
|
|
checker := privilege.GetPrivilegeManager(ctx)
|
|
rows := make([][]types.Datum, 0, len(schemas))
|
|
|
|
for _, schema := range schemas {
|
|
|
|
charset := mysql.DefaultCharset
|
|
collation := mysql.DefaultCollationName
|
|
|
|
if len(schema.Charset) > 0 {
|
|
charset = schema.Charset // Overwrite default
|
|
}
|
|
|
|
if len(schema.Collate) > 0 {
|
|
collation = schema.Collate // Overwrite default
|
|
}
|
|
|
|
if checker != nil && !checker.RequestVerification(ctx.GetSessionVars().ActiveRoles, schema.Name.L, "", "", mysql.AllPrivMask) {
|
|
continue
|
|
}
|
|
record := types.MakeDatums(
|
|
infoschema.CatalogVal, // CATALOG_NAME
|
|
schema.Name.O, // SCHEMA_NAME
|
|
charset, // DEFAULT_CHARACTER_SET_NAME
|
|
collation, // DEFAULT_COLLATION_NAME
|
|
nil,
|
|
)
|
|
rows = append(rows, record)
|
|
}
|
|
e.rows = rows
|
|
}
|
|
|
|
func (e *memtableRetriever) setDataForStatistics(ctx sessionctx.Context, schemas []*model.DBInfo) {
|
|
checker := privilege.GetPrivilegeManager(ctx)
|
|
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
|
|
}
|
|
e.setDataForStatisticsInTable(schema, table)
|
|
}
|
|
}
|
|
}
|
|
|
|
func (e *memtableRetriever) setDataForStatisticsInTable(schema *model.DBInfo, table *model.TableInfo) {
|
|
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
|
|
"", // INDEX_COMMENT
|
|
"YES", // IS_VISIBLE
|
|
"NULL", // Expression
|
|
)
|
|
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 = ""
|
|
}
|
|
|
|
visible := "YES"
|
|
if index.Invisible {
|
|
visible = "NO"
|
|
}
|
|
|
|
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
|
|
"", // INDEX_COMMENT
|
|
visible, // IS_VISIBLE
|
|
expression, // Expression
|
|
)
|
|
rows = append(rows, record)
|
|
}
|
|
}
|
|
e.rows = append(e.rows, rows...)
|
|
}
|
|
|
|
func (e *memtableRetriever) setDataFromTables(ctx sessionctx.Context, schemas []*model.DBInfo) error {
|
|
tableRowsMap, colLengthMap, err := tableStatsCache.get(ctx)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
checker := privilege.GetPrivilegeManager(ctx)
|
|
|
|
var rows [][]types.Datum
|
|
createTimeTp := mysql.TypeDatetime
|
|
for _, schema := range schemas {
|
|
for _, table := range schema.Tables {
|
|
collation := table.Collate
|
|
if collation == "" {
|
|
collation = mysql.DefaultCollationName
|
|
}
|
|
createTime := types.NewTime(types.FromGoTime(table.GetUpdateTime()), createTimeTp, types.DefaultFsp)
|
|
|
|
createOptions := ""
|
|
|
|
if table.IsSequence() {
|
|
continue
|
|
}
|
|
|
|
if checker != nil && !checker.RequestVerification(ctx.GetSessionVars().ActiveRoles, schema.Name.L, table.Name.L, "", mysql.AllPrivMask) {
|
|
continue
|
|
}
|
|
|
|
if !table.IsView() {
|
|
if table.GetPartitionInfo() != nil {
|
|
createOptions = "partitioned"
|
|
}
|
|
var autoIncID interface{}
|
|
hasAutoIncID, _ := infoschema.HasAutoIncrementColumn(table)
|
|
if hasAutoIncID {
|
|
autoIncID, err = getAutoIncrementID(ctx, schema, table)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
}
|
|
|
|
var rowCount, dataLength, indexLength uint64
|
|
if table.GetPartitionInfo() == nil {
|
|
rowCount = tableRowsMap[table.ID]
|
|
dataLength, indexLength = getDataAndIndexLength(table, table.ID, rowCount, colLengthMap)
|
|
} else {
|
|
for _, pi := range table.GetPartitionInfo().Definitions {
|
|
rowCount += tableRowsMap[pi.ID]
|
|
parDataLen, parIndexLen := getDataAndIndexLength(table, pi.ID, tableRowsMap[pi.ID], colLengthMap)
|
|
dataLength += parDataLen
|
|
indexLength += parIndexLen
|
|
}
|
|
}
|
|
avgRowLength := uint64(0)
|
|
if rowCount != 0 {
|
|
avgRowLength = dataLength / rowCount
|
|
}
|
|
var tableType string
|
|
switch schema.Name.L {
|
|
case util.InformationSchemaName.L, util.PerformanceSchemaName.L,
|
|
util.MetricSchemaName.L:
|
|
tableType = "SYSTEM VIEW"
|
|
default:
|
|
tableType = "BASE TABLE"
|
|
}
|
|
shardingInfo := infoschema.GetShardingInfo(schema, table)
|
|
record := types.MakeDatums(
|
|
infoschema.CatalogVal, // TABLE_CATALOG
|
|
schema.Name.O, // TABLE_SCHEMA
|
|
table.Name.O, // TABLE_NAME
|
|
tableType, // TABLE_TYPE
|
|
"InnoDB", // ENGINE
|
|
uint64(10), // VERSION
|
|
"Compact", // ROW_FORMAT
|
|
rowCount, // TABLE_ROWS
|
|
avgRowLength, // AVG_ROW_LENGTH
|
|
dataLength, // DATA_LENGTH
|
|
uint64(0), // MAX_DATA_LENGTH
|
|
indexLength, // INDEX_LENGTH
|
|
uint64(0), // DATA_FREE
|
|
autoIncID, // AUTO_INCREMENT
|
|
createTime, // CREATE_TIME
|
|
nil, // UPDATE_TIME
|
|
nil, // CHECK_TIME
|
|
collation, // TABLE_COLLATION
|
|
nil, // CHECKSUM
|
|
createOptions, // CREATE_OPTIONS
|
|
table.Comment, // TABLE_COMMENT
|
|
table.ID, // TIDB_TABLE_ID
|
|
shardingInfo, // TIDB_ROW_ID_SHARDING_INFO
|
|
)
|
|
rows = append(rows, record)
|
|
} else {
|
|
record := types.MakeDatums(
|
|
infoschema.CatalogVal, // TABLE_CATALOG
|
|
schema.Name.O, // TABLE_SCHEMA
|
|
table.Name.O, // TABLE_NAME
|
|
"VIEW", // TABLE_TYPE
|
|
nil, // ENGINE
|
|
nil, // VERSION
|
|
nil, // ROW_FORMAT
|
|
nil, // TABLE_ROWS
|
|
nil, // AVG_ROW_LENGTH
|
|
nil, // DATA_LENGTH
|
|
nil, // MAX_DATA_LENGTH
|
|
nil, // INDEX_LENGTH
|
|
nil, // DATA_FREE
|
|
nil, // AUTO_INCREMENT
|
|
createTime, // CREATE_TIME
|
|
nil, // UPDATE_TIME
|
|
nil, // CHECK_TIME
|
|
nil, // TABLE_COLLATION
|
|
nil, // CHECKSUM
|
|
nil, // CREATE_OPTIONS
|
|
"VIEW", // TABLE_COMMENT
|
|
table.ID, // TIDB_TABLE_ID
|
|
nil, // TIDB_ROW_ID_SHARDING_INFO
|
|
)
|
|
rows = append(rows, record)
|
|
}
|
|
}
|
|
}
|
|
e.rows = rows
|
|
return nil
|
|
}
|
|
|
|
func (e *hugeMemTableRetriever) setDataForColumns(ctx sessionctx.Context) error {
|
|
checker := privilege.GetPrivilegeManager(ctx)
|
|
e.rows = e.rows[:0]
|
|
batch := 1024
|
|
for ; e.dbsIdx < len(e.dbs); e.dbsIdx++ {
|
|
schema := e.dbs[e.dbsIdx]
|
|
for e.tblIdx < len(schema.Tables) {
|
|
table := schema.Tables[e.tblIdx]
|
|
e.tblIdx++
|
|
if checker != nil && !checker.RequestVerification(ctx.GetSessionVars().ActiveRoles, schema.Name.L, table.Name.L, "", mysql.AllPrivMask) {
|
|
continue
|
|
}
|
|
|
|
e.dataForColumnsInTable(schema, table)
|
|
if len(e.rows) >= batch {
|
|
return nil
|
|
}
|
|
}
|
|
e.tblIdx = 0
|
|
}
|
|
return nil
|
|
}
|
|
|
|
func (e *hugeMemTableRetriever) dataForColumnsInTable(schema *model.DBInfo, tbl *model.TableInfo) {
|
|
for i, col := range tbl.Columns {
|
|
if col.Hidden {
|
|
continue
|
|
}
|
|
var charMaxLen, charOctLen, numericPrecision, numericScale, datetimePrecision interface{}
|
|
colLen, decimal := col.Flen, col.Decimal
|
|
defaultFlen, defaultDecimal := mysql.GetDefaultFieldLengthAndDecimal(col.Tp)
|
|
if decimal == types.UnspecifiedLength {
|
|
decimal = defaultDecimal
|
|
}
|
|
if colLen == types.UnspecifiedLength {
|
|
colLen = defaultFlen
|
|
}
|
|
if col.Tp == mysql.TypeSet {
|
|
// Example: In MySQL set('a','bc','def','ghij') has length 13, because
|
|
// len('a')+len('bc')+len('def')+len('ghij')+len(ThreeComma)=13
|
|
// Reference link: https://bugs.mysql.com/bug.php?id=22613
|
|
colLen = 0
|
|
for _, ele := range col.Elems {
|
|
colLen += len(ele)
|
|
}
|
|
if len(col.Elems) != 0 {
|
|
colLen += (len(col.Elems) - 1)
|
|
}
|
|
charMaxLen = colLen
|
|
charOctLen = colLen
|
|
} else if col.Tp == mysql.TypeEnum {
|
|
// Example: In MySQL enum('a', 'ab', 'cdef') has length 4, because
|
|
// the longest string in the enum is 'cdef'
|
|
// Reference link: https://bugs.mysql.com/bug.php?id=22613
|
|
colLen = 0
|
|
for _, ele := range col.Elems {
|
|
if len(ele) > colLen {
|
|
colLen = len(ele)
|
|
}
|
|
}
|
|
charMaxLen = colLen
|
|
charOctLen = colLen
|
|
} else if types.IsString(col.Tp) {
|
|
charMaxLen = colLen
|
|
charOctLen = colLen
|
|
} else if types.IsTypeFractionable(col.Tp) {
|
|
datetimePrecision = decimal
|
|
} else if types.IsTypeNumeric(col.Tp) {
|
|
numericPrecision = colLen
|
|
if col.Tp != mysql.TypeFloat && col.Tp != mysql.TypeDouble {
|
|
numericScale = decimal
|
|
} else if decimal != -1 {
|
|
numericScale = decimal
|
|
}
|
|
}
|
|
columnType := col.FieldType.InfoSchemaStr()
|
|
columnDesc := table.NewColDesc(table.ToColumn(col))
|
|
var columnDefault interface{}
|
|
if columnDesc.DefaultValue != nil {
|
|
columnDefault = fmt.Sprintf("%v", columnDesc.DefaultValue)
|
|
}
|
|
record := types.MakeDatums(
|
|
infoschema.CatalogVal, // TABLE_CATALOG
|
|
schema.Name.O, // TABLE_SCHEMA
|
|
tbl.Name.O, // TABLE_NAME
|
|
col.Name.O, // COLUMN_NAME
|
|
i+1, // ORIGINAL_POSITION
|
|
columnDefault, // COLUMN_DEFAULT
|
|
columnDesc.Null, // IS_NULLABLE
|
|
types.TypeToStr(col.Tp, col.Charset), // DATA_TYPE
|
|
charMaxLen, // CHARACTER_MAXIMUM_LENGTH
|
|
charOctLen, // CHARACTER_OCTET_LENGTH
|
|
numericPrecision, // NUMERIC_PRECISION
|
|
numericScale, // NUMERIC_SCALE
|
|
datetimePrecision, // DATETIME_PRECISION
|
|
columnDesc.Charset, // CHARACTER_SET_NAME
|
|
columnDesc.Collation, // COLLATION_NAME
|
|
columnType, // COLUMN_TYPE
|
|
columnDesc.Key, // COLUMN_KEY
|
|
columnDesc.Extra, // EXTRA
|
|
"select,insert,update,references", // PRIVILEGES
|
|
columnDesc.Comment, // COLUMN_COMMENT
|
|
col.GeneratedExprString, // GENERATION_EXPRESSION
|
|
)
|
|
e.rows = append(e.rows, record)
|
|
}
|
|
}
|
|
|
|
func (e *memtableRetriever) setDataFromPartitions(ctx sessionctx.Context, schemas []*model.DBInfo) error {
|
|
tableRowsMap, colLengthMap, err := tableStatsCache.get(ctx)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
checker := privilege.GetPrivilegeManager(ctx)
|
|
var rows [][]types.Datum
|
|
createTimeTp := mysql.TypeDatetime
|
|
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(
|
|
infoschema.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(
|
|
infoschema.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)
|
|
}
|
|
}
|
|
}
|
|
}
|
|
e.rows = rows
|
|
return nil
|
|
}
|
|
|
|
func (e *memtableRetriever) setDataFromIndexes(ctx sessionctx.Context, schemas []*model.DBInfo) {
|
|
checker := privilege.GetPrivilegeManager(ctx)
|
|
var rows [][]types.Datum
|
|
for _, schema := range schemas {
|
|
for _, tb := range schema.Tables {
|
|
if checker != nil && !checker.RequestVerification(ctx.GetSessionVars().ActiveRoles, schema.Name.L, tb.Name.L, "", mysql.AllPrivMask) {
|
|
continue
|
|
}
|
|
|
|
if tb.PKIsHandle {
|
|
var pkCol *model.ColumnInfo
|
|
for _, col := range tb.Cols() {
|
|
if mysql.HasPriKeyFlag(col.Flag) {
|
|
pkCol = col
|
|
break
|
|
}
|
|
}
|
|
record := types.MakeDatums(
|
|
schema.Name.O, // TABLE_SCHEMA
|
|
tb.Name.O, // TABLE_NAME
|
|
0, // NON_UNIQUE
|
|
"PRIMARY", // KEY_NAME
|
|
1, // SEQ_IN_INDEX
|
|
pkCol.Name.O, // COLUMN_NAME
|
|
nil, // SUB_PART
|
|
"", // INDEX_COMMENT
|
|
"NULL", // Expression
|
|
0, // INDEX_ID
|
|
)
|
|
rows = append(rows, record)
|
|
}
|
|
for _, idxInfo := range tb.Indices {
|
|
if idxInfo.State != model.StatePublic {
|
|
continue
|
|
}
|
|
for i, col := range idxInfo.Columns {
|
|
nonUniq := 1
|
|
if idxInfo.Unique {
|
|
nonUniq = 0
|
|
}
|
|
var subPart interface{}
|
|
if col.Length != types.UnspecifiedLength {
|
|
subPart = col.Length
|
|
}
|
|
colName := col.Name.O
|
|
expression := "NULL"
|
|
tblCol := tb.Columns[col.Offset]
|
|
if tblCol.Hidden {
|
|
colName = "NULL"
|
|
expression = fmt.Sprintf("(%s)", tblCol.GeneratedExprString)
|
|
}
|
|
record := types.MakeDatums(
|
|
schema.Name.O, // TABLE_SCHEMA
|
|
tb.Name.O, // TABLE_NAME
|
|
nonUniq, // NON_UNIQUE
|
|
idxInfo.Name.O, // KEY_NAME
|
|
i+1, // SEQ_IN_INDEX
|
|
colName, // COLUMN_NAME
|
|
subPart, // SUB_PART
|
|
idxInfo.Comment, // INDEX_COMMENT
|
|
expression, // Expression
|
|
idxInfo.ID, // INDEX_ID
|
|
)
|
|
rows = append(rows, record)
|
|
}
|
|
}
|
|
}
|
|
}
|
|
e.rows = rows
|
|
}
|
|
|
|
func (e *memtableRetriever) setDataFromViews(ctx sessionctx.Context, schemas []*model.DBInfo) {
|
|
checker := privilege.GetPrivilegeManager(ctx)
|
|
var rows [][]types.Datum
|
|
for _, schema := range schemas {
|
|
for _, table := range schema.Tables {
|
|
if !table.IsView() {
|
|
continue
|
|
}
|
|
collation := table.Collate
|
|
charset := table.Charset
|
|
if collation == "" {
|
|
collation = mysql.DefaultCollationName
|
|
}
|
|
if charset == "" {
|
|
charset = mysql.DefaultCharset
|
|
}
|
|
if checker != nil && !checker.RequestVerification(ctx.GetSessionVars().ActiveRoles, schema.Name.L, table.Name.L, "", mysql.AllPrivMask) {
|
|
continue
|
|
}
|
|
record := types.MakeDatums(
|
|
infoschema.CatalogVal, // TABLE_CATALOG
|
|
schema.Name.O, // TABLE_SCHEMA
|
|
table.Name.O, // TABLE_NAME
|
|
table.View.SelectStmt, // VIEW_DEFINITION
|
|
table.View.CheckOption.String(), // CHECK_OPTION
|
|
"NO", // IS_UPDATABLE
|
|
table.View.Definer.String(), // DEFINER
|
|
table.View.Security.String(), // SECURITY_TYPE
|
|
charset, // CHARACTER_SET_CLIENT
|
|
collation, // COLLATION_CONNECTION
|
|
)
|
|
rows = append(rows, record)
|
|
}
|
|
}
|
|
e.rows = rows
|
|
}
|
|
|
|
// DDLJobsReaderExec executes DDLJobs information retrieving.
|
|
type DDLJobsReaderExec struct {
|
|
baseExecutor
|
|
DDLJobRetriever
|
|
|
|
cacheJobs []*model.Job
|
|
is infoschema.InfoSchema
|
|
}
|
|
|
|
// Open implements the Executor Next interface.
|
|
func (e *DDLJobsReaderExec) Open(ctx context.Context) error {
|
|
if err := e.baseExecutor.Open(ctx); err != nil {
|
|
return err
|
|
}
|
|
txn, err := e.ctx.Txn(true)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
e.DDLJobRetriever.is = e.is
|
|
e.activeRoles = e.ctx.GetSessionVars().ActiveRoles
|
|
err = e.DDLJobRetriever.initial(txn)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
return nil
|
|
}
|
|
|
|
// Next implements the Executor Next interface.
|
|
func (e *DDLJobsReaderExec) Next(ctx context.Context, req *chunk.Chunk) error {
|
|
req.GrowAndReset(e.maxChunkSize)
|
|
checker := privilege.GetPrivilegeManager(e.ctx)
|
|
count := 0
|
|
|
|
// Append running DDL jobs.
|
|
if e.cursor < len(e.runningJobs) {
|
|
num := mathutil.Min(req.Capacity(), len(e.runningJobs)-e.cursor)
|
|
for i := e.cursor; i < e.cursor+num; i++ {
|
|
e.appendJobToChunk(req, e.runningJobs[i], checker)
|
|
req.AppendString(11, e.runningJobs[i].Query)
|
|
}
|
|
e.cursor += num
|
|
count += num
|
|
}
|
|
var err error
|
|
|
|
// Append history DDL jobs.
|
|
if count < req.Capacity() {
|
|
e.cacheJobs, err = e.historyJobIter.GetLastJobs(req.Capacity()-count, e.cacheJobs)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
for _, job := range e.cacheJobs {
|
|
e.appendJobToChunk(req, job, checker)
|
|
req.AppendString(11, job.Query)
|
|
}
|
|
e.cursor += len(e.cacheJobs)
|
|
}
|
|
return nil
|
|
}
|
|
|
|
func (e *memtableRetriever) setDataFromEngines() {
|
|
var rows [][]types.Datum
|
|
rows = append(rows,
|
|
types.MakeDatums(
|
|
"InnoDB", // Engine
|
|
"DEFAULT", // Support
|
|
"Supports transactions, row-level locking, and foreign keys", // Comment
|
|
"YES", // Transactions
|
|
"YES", // XA
|
|
"YES", // Savepoints
|
|
),
|
|
)
|
|
e.rows = rows
|
|
}
|
|
|
|
func (e *memtableRetriever) setDataFromCharacterSets() {
|
|
var rows [][]types.Datum
|
|
charsets := charset.GetSupportedCharsets()
|
|
for _, charset := range charsets {
|
|
rows = append(rows,
|
|
types.MakeDatums(charset.Name, charset.DefaultCollation, charset.Desc, charset.Maxlen),
|
|
)
|
|
}
|
|
e.rows = rows
|
|
}
|
|
|
|
func (e *memtableRetriever) setDataFromCollations() {
|
|
var rows [][]types.Datum
|
|
collations := collate.GetSupportedCollations()
|
|
for _, collation := range collations {
|
|
isDefault := ""
|
|
if collation.IsDefault {
|
|
isDefault = "Yes"
|
|
}
|
|
rows = append(rows,
|
|
types.MakeDatums(collation.Name, collation.CharsetName, collation.ID, isDefault, "Yes", 1),
|
|
)
|
|
}
|
|
e.rows = rows
|
|
}
|
|
|
|
func (e *memtableRetriever) dataForCollationCharacterSetApplicability() {
|
|
var rows [][]types.Datum
|
|
collations := collate.GetSupportedCollations()
|
|
for _, collation := range collations {
|
|
rows = append(rows,
|
|
types.MakeDatums(collation.Name, collation.CharsetName),
|
|
)
|
|
}
|
|
e.rows = rows
|
|
}
|
|
|
|
func (e *memtableRetriever) dataForTiDBClusterInfo(ctx sessionctx.Context) error {
|
|
servers, err := infoschema.GetClusterServerInfo(ctx)
|
|
if err != nil {
|
|
e.rows = nil
|
|
return err
|
|
}
|
|
rows := make([][]types.Datum, 0, len(servers))
|
|
for _, server := range servers {
|
|
startTimeStr := ""
|
|
upTimeStr := ""
|
|
if server.StartTimestamp > 0 {
|
|
startTime := time.Unix(server.StartTimestamp, 0)
|
|
startTimeStr = startTime.Format(time.RFC3339)
|
|
upTimeStr = time.Since(startTime).String()
|
|
}
|
|
row := types.MakeDatums(
|
|
server.ServerType,
|
|
server.Address,
|
|
server.StatusAddr,
|
|
server.Version,
|
|
server.GitHash,
|
|
startTimeStr,
|
|
upTimeStr,
|
|
)
|
|
rows = append(rows, row)
|
|
}
|
|
e.rows = rows
|
|
return nil
|
|
}
|
|
|
|
func (e *memtableRetriever) setDataFromKeyColumnUsage(ctx sessionctx.Context, schemas []*model.DBInfo) {
|
|
checker := privilege.GetPrivilegeManager(ctx)
|
|
rows := make([][]types.Datum, 0, len(schemas)) // The capacity is not accurate, but it is not a big problem.
|
|
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 := keyColumnUsageInTable(schema, table)
|
|
rows = append(rows, rs...)
|
|
}
|
|
}
|
|
e.rows = rows
|
|
}
|
|
|
|
func (e *memtableRetriever) setDataForClusterProcessList(ctx sessionctx.Context) error {
|
|
e.setDataForProcessList(ctx)
|
|
rows, err := infoschema.AppendHostInfoToRows(e.rows)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
e.rows = rows
|
|
return nil
|
|
}
|
|
|
|
func (e *memtableRetriever) setDataForProcessList(ctx sessionctx.Context) {
|
|
sm := ctx.GetSessionManager()
|
|
if sm == nil {
|
|
return
|
|
}
|
|
|
|
loginUser := ctx.GetSessionVars().User
|
|
var hasProcessPriv bool
|
|
if pm := privilege.GetPrivilegeManager(ctx); pm != nil {
|
|
if pm.RequestVerification(ctx.GetSessionVars().ActiveRoles, "", "", "", mysql.ProcessPriv) {
|
|
hasProcessPriv = true
|
|
}
|
|
}
|
|
|
|
pl := sm.ShowProcessList()
|
|
|
|
records := make([][]types.Datum, 0, len(pl))
|
|
for _, pi := range pl {
|
|
// If you have the PROCESS privilege, you can see all threads.
|
|
// Otherwise, you can see only your own threads.
|
|
if !hasProcessPriv && loginUser != nil && pi.User != loginUser.Username {
|
|
continue
|
|
}
|
|
|
|
rows := pi.ToRow(ctx.GetSessionVars().StmtCtx.TimeZone)
|
|
record := types.MakeDatums(rows...)
|
|
records = append(records, record)
|
|
}
|
|
e.rows = records
|
|
}
|
|
|
|
func (e *memtableRetriever) setDataFromUserPrivileges(ctx sessionctx.Context) {
|
|
pm := privilege.GetPrivilegeManager(ctx)
|
|
e.rows = pm.UserPrivilegesTable()
|
|
}
|
|
|
|
func (e *memtableRetriever) setDataForMetricTables(ctx sessionctx.Context) {
|
|
var rows [][]types.Datum
|
|
tables := make([]string, 0, len(infoschema.MetricTableMap))
|
|
for name := range infoschema.MetricTableMap {
|
|
tables = append(tables, name)
|
|
}
|
|
sort.Strings(tables)
|
|
for _, name := range tables {
|
|
schema := infoschema.MetricTableMap[name]
|
|
record := types.MakeDatums(
|
|
name, // METRICS_NAME
|
|
schema.PromQL, // PROMQL
|
|
strings.Join(schema.Labels, ","), // LABELS
|
|
schema.Quantile, // QUANTILE
|
|
schema.Comment, // COMMENT
|
|
)
|
|
rows = append(rows, record)
|
|
}
|
|
e.rows = rows
|
|
}
|
|
|
|
func keyColumnUsageInTable(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, // CONSTRAINT_CATALOG
|
|
schema.Name.O, // CONSTRAINT_SCHEMA
|
|
infoschema.PrimaryConstraint, // CONSTRAINT_NAME
|
|
infoschema.CatalogVal, // TABLE_CATALOG
|
|
schema.Name.O, // TABLE_SCHEMA
|
|
table.Name.O, // TABLE_NAME
|
|
col.Name.O, // COLUMN_NAME
|
|
1, // ORDINAL_POSITION
|
|
1, // POSITION_IN_UNIQUE_CONSTRAINT
|
|
nil, // REFERENCED_TABLE_SCHEMA
|
|
nil, // REFERENCED_TABLE_NAME
|
|
nil, // REFERENCED_COLUMN_NAME
|
|
)
|
|
rows = append(rows, record)
|
|
break
|
|
}
|
|
}
|
|
}
|
|
nameToCol := make(map[string]*model.ColumnInfo, len(table.Columns))
|
|
for _, c := range table.Columns {
|
|
nameToCol[c.Name.L] = c
|
|
}
|
|
for _, index := range table.Indices {
|
|
var idxName string
|
|
if index.Primary {
|
|
idxName = infoschema.PrimaryConstraint
|
|
} else if index.Unique {
|
|
idxName = index.Name.O
|
|
} else {
|
|
// Only handle unique/primary key
|
|
continue
|
|
}
|
|
for i, key := range index.Columns {
|
|
col := nameToCol[key.Name.L]
|
|
record := types.MakeDatums(
|
|
infoschema.CatalogVal, // CONSTRAINT_CATALOG
|
|
schema.Name.O, // CONSTRAINT_SCHEMA
|
|
idxName, // CONSTRAINT_NAME
|
|
infoschema.CatalogVal, // TABLE_CATALOG
|
|
schema.Name.O, // TABLE_SCHEMA
|
|
table.Name.O, // TABLE_NAME
|
|
col.Name.O, // COLUMN_NAME
|
|
i+1, // ORDINAL_POSITION,
|
|
nil, // POSITION_IN_UNIQUE_CONSTRAINT
|
|
nil, // REFERENCED_TABLE_SCHEMA
|
|
nil, // REFERENCED_TABLE_NAME
|
|
nil, // REFERENCED_COLUMN_NAME
|
|
)
|
|
rows = append(rows, record)
|
|
}
|
|
}
|
|
for _, fk := range table.ForeignKeys {
|
|
fkRefCol := ""
|
|
if len(fk.RefCols) > 0 {
|
|
fkRefCol = fk.RefCols[0].O
|
|
}
|
|
for i, key := range fk.Cols {
|
|
col := nameToCol[key.L]
|
|
record := types.MakeDatums(
|
|
infoschema.CatalogVal, // CONSTRAINT_CATALOG
|
|
schema.Name.O, // CONSTRAINT_SCHEMA
|
|
fk.Name.O, // CONSTRAINT_NAME
|
|
infoschema.CatalogVal, // TABLE_CATALOG
|
|
schema.Name.O, // TABLE_SCHEMA
|
|
table.Name.O, // TABLE_NAME
|
|
col.Name.O, // COLUMN_NAME
|
|
i+1, // ORDINAL_POSITION,
|
|
1, // POSITION_IN_UNIQUE_CONSTRAINT
|
|
schema.Name.O, // REFERENCED_TABLE_SCHEMA
|
|
fk.RefTable.O, // REFERENCED_TABLE_NAME
|
|
fkRefCol, // REFERENCED_COLUMN_NAME
|
|
)
|
|
rows = append(rows, record)
|
|
}
|
|
}
|
|
return rows
|
|
}
|
|
|
|
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")
|
|
}
|
|
tikvHelper := &helper.Helper{
|
|
Store: tikvStore,
|
|
RegionCache: tikvStore.GetRegionCache(),
|
|
}
|
|
regionsInfo, err := tikvHelper.GetRegionsInfo()
|
|
if err != nil {
|
|
return err
|
|
}
|
|
for _, region := range regionsInfo.Regions {
|
|
e.setNewTiKVRegionPeersCols(®ion)
|
|
}
|
|
return nil
|
|
}
|
|
|
|
func (e *memtableRetriever) setNewTiKVRegionPeersCols(region *helper.RegionInfo) {
|
|
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)
|
|
}
|
|
e.rows = append(e.rows, records...)
|
|
}
|
|
|
|
const (
|
|
normalPeer = "NORMAL"
|
|
pendingPeer = "PENDING"
|
|
downPeer = "DOWN"
|
|
)
|
|
|
|
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")
|
|
}
|
|
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 err
|
|
}
|
|
e.setDataForHotRegionByMetrics(metrics, "read")
|
|
metrics, err = tikvHelper.ScrapeHotInfo(pdapi.HotWrite, allSchemas)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
e.setDataForHotRegionByMetrics(metrics, "write")
|
|
return nil
|
|
}
|
|
|
|
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))
|
|
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)
|
|
}
|
|
e.rows = append(e.rows, rows...)
|
|
}
|
|
|
|
// setDataFromTableConstraints constructs data for table information_schema.constraints.See https://dev.mysql.com/doc/refman/5.7/en/table-constraints-table.html
|
|
func (e *memtableRetriever) setDataFromTableConstraints(ctx sessionctx.Context, schemas []*model.DBInfo) {
|
|
checker := privilege.GetPrivilegeManager(ctx)
|
|
var rows [][]types.Datum
|
|
for _, schema := range schemas {
|
|
for _, tbl := range schema.Tables {
|
|
if checker != nil && !checker.RequestVerification(ctx.GetSessionVars().ActiveRoles, schema.Name.L, tbl.Name.L, "", mysql.AllPrivMask) {
|
|
continue
|
|
}
|
|
|
|
if tbl.PKIsHandle {
|
|
record := types.MakeDatums(
|
|
infoschema.CatalogVal, // CONSTRAINT_CATALOG
|
|
schema.Name.O, // CONSTRAINT_SCHEMA
|
|
mysql.PrimaryKeyName, // CONSTRAINT_NAME
|
|
schema.Name.O, // TABLE_SCHEMA
|
|
tbl.Name.O, // TABLE_NAME
|
|
infoschema.PrimaryKeyType, // CONSTRAINT_TYPE
|
|
)
|
|
rows = append(rows, record)
|
|
}
|
|
|
|
for _, idx := range tbl.Indices {
|
|
var cname, ctype string
|
|
if idx.Primary {
|
|
cname = mysql.PrimaryKeyName
|
|
ctype = infoschema.PrimaryKeyType
|
|
} else if idx.Unique {
|
|
cname = idx.Name.O
|
|
ctype = infoschema.UniqueKeyType
|
|
} else {
|
|
// The index has no constriant.
|
|
continue
|
|
}
|
|
record := types.MakeDatums(
|
|
infoschema.CatalogVal, // CONSTRAINT_CATALOG
|
|
schema.Name.O, // CONSTRAINT_SCHEMA
|
|
cname, // CONSTRAINT_NAME
|
|
schema.Name.O, // TABLE_SCHEMA
|
|
tbl.Name.O, // TABLE_NAME
|
|
ctype, // CONSTRAINT_TYPE
|
|
)
|
|
rows = append(rows, record)
|
|
}
|
|
}
|
|
}
|
|
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
|
|
sessionVars := ctx.GetSessionVars()
|
|
for _, v := range variable.SysVars {
|
|
var value string
|
|
value, err = variable.GetSessionSystemVar(sessionVars, v.Name)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
row := types.MakeDatums(v.Name, value)
|
|
rows = append(rows, row)
|
|
}
|
|
e.rows = rows
|
|
return nil
|
|
}
|
|
|
|
// dataForAnalyzeStatusHelper is a helper function which can be used in show_stats.go
|
|
func dataForAnalyzeStatusHelper(sctx sessionctx.Context) (rows [][]types.Datum) {
|
|
checker := privilege.GetPrivilegeManager(sctx)
|
|
for _, job := range statistics.GetAllAnalyzeJobs() {
|
|
job.Lock()
|
|
var startTime interface{}
|
|
if job.StartTime.IsZero() {
|
|
startTime = nil
|
|
} else {
|
|
startTime = types.NewTime(types.FromGoTime(job.StartTime), mysql.TypeDatetime, 0)
|
|
}
|
|
if checker == nil || checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, job.DBName, job.TableName, "", mysql.AllPrivMask) {
|
|
rows = append(rows, types.MakeDatums(
|
|
job.DBName, // TABLE_SCHEMA
|
|
job.TableName, // TABLE_NAME
|
|
job.PartitionName, // PARTITION_NAME
|
|
job.JobInfo, // JOB_INFO
|
|
job.RowCount, // ROW_COUNT
|
|
startTime, // START_TIME
|
|
job.State, // STATE
|
|
))
|
|
}
|
|
job.Unlock()
|
|
}
|
|
return
|
|
}
|
|
|
|
// setDataForAnalyzeStatus gets all the analyze jobs.
|
|
func (e *memtableRetriever) setDataForAnalyzeStatus(sctx sessionctx.Context) {
|
|
e.rows = dataForAnalyzeStatusHelper(sctx)
|
|
}
|
|
|
|
// setDataForPseudoProfiling returns pseudo data for table profiling when system variable `profiling` is set to `ON`.
|
|
func (e *memtableRetriever) setDataForPseudoProfiling(sctx sessionctx.Context) {
|
|
if v, ok := sctx.GetSessionVars().GetSystemVar("profiling"); ok && variable.TiDBOptOn(v) {
|
|
row := types.MakeDatums(
|
|
0, // QUERY_ID
|
|
0, // SEQ
|
|
"", // STATE
|
|
types.NewDecFromInt(0), // DURATION
|
|
types.NewDecFromInt(0), // CPU_USER
|
|
types.NewDecFromInt(0), // CPU_SYSTEM
|
|
0, // CONTEXT_VOLUNTARY
|
|
0, // CONTEXT_INVOLUNTARY
|
|
0, // BLOCK_OPS_IN
|
|
0, // BLOCK_OPS_OUT
|
|
0, // MESSAGES_SENT
|
|
0, // MESSAGES_RECEIVED
|
|
0, // PAGE_FAULTS_MAJOR
|
|
0, // PAGE_FAULTS_MINOR
|
|
0, // SWAPS
|
|
"", // SOURCE_FUNCTION
|
|
"", // SOURCE_FILE
|
|
0, // SOURCE_LINE
|
|
)
|
|
e.rows = append(e.rows, row)
|
|
}
|
|
}
|
|
|
|
func (e *memtableRetriever) setDataForServersInfo() error {
|
|
serversInfo, err := infosync.GetAllServerInfo(context.Background())
|
|
if err != nil {
|
|
return err
|
|
}
|
|
rows := make([][]types.Datum, 0, len(serversInfo))
|
|
for _, info := range serversInfo {
|
|
row := types.MakeDatums(
|
|
info.ID, // DDL_ID
|
|
info.IP, // IP
|
|
int(info.Port), // PORT
|
|
int(info.StatusPort), // STATUS_PORT
|
|
info.Lease, // LEASE
|
|
info.Version, // VERSION
|
|
info.GitHash, // GIT_HASH
|
|
info.BinlogStatus, // BINLOG_STATUS
|
|
)
|
|
rows = append(rows, row)
|
|
}
|
|
e.rows = rows
|
|
return nil
|
|
}
|
|
|
|
func (e *memtableRetriever) setDataFromSequences(ctx sessionctx.Context, schemas []*model.DBInfo) {
|
|
checker := privilege.GetPrivilegeManager(ctx)
|
|
var rows [][]types.Datum
|
|
for _, schema := range schemas {
|
|
for _, table := range schema.Tables {
|
|
if !table.IsSequence() {
|
|
continue
|
|
}
|
|
if checker != nil && !checker.RequestVerification(ctx.GetSessionVars().ActiveRoles, schema.Name.L, table.Name.L, "", mysql.AllPrivMask) {
|
|
continue
|
|
}
|
|
record := types.MakeDatums(
|
|
infoschema.CatalogVal, // TABLE_CATALOG
|
|
schema.Name.O, // TABLE_SCHEMA
|
|
table.Name.O, // TABLE_NAME
|
|
table.Sequence.Cache, // Cache
|
|
table.Sequence.CacheValue, // CACHE_VALUE
|
|
table.Sequence.Cycle, // CYCLE
|
|
table.Sequence.Increment, // INCREMENT
|
|
table.Sequence.MaxValue, // MAXVALUE
|
|
table.Sequence.MinValue, // MINVALUE
|
|
table.Sequence.Start, // START
|
|
table.Sequence.Comment, // COMMENT
|
|
)
|
|
rows = append(rows, record)
|
|
}
|
|
}
|
|
e.rows = rows
|
|
}
|
|
|
|
// dataForTableTiFlashReplica constructs data for table tiflash replica info.
|
|
func (e *memtableRetriever) dataForTableTiFlashReplica(ctx sessionctx.Context, schemas []*model.DBInfo) {
|
|
var rows [][]types.Datum
|
|
progressMap, err := infosync.GetTiFlashTableSyncProgress(context.Background())
|
|
if err != nil {
|
|
ctx.GetSessionVars().StmtCtx.AppendWarning(err)
|
|
}
|
|
for _, schema := range schemas {
|
|
for _, tbl := range schema.Tables {
|
|
if tbl.TiFlashReplica == nil {
|
|
continue
|
|
}
|
|
progress := 1.0
|
|
if !tbl.TiFlashReplica.Available {
|
|
if pi := tbl.GetPartitionInfo(); pi != nil && len(pi.Definitions) > 0 {
|
|
progress = 0
|
|
for _, p := range pi.Definitions {
|
|
if tbl.TiFlashReplica.IsPartitionAvailable(p.ID) {
|
|
progress += 1
|
|
} else {
|
|
progress += progressMap[p.ID]
|
|
}
|
|
}
|
|
progress = progress / float64(len(pi.Definitions))
|
|
} else {
|
|
progress = progressMap[tbl.ID]
|
|
}
|
|
}
|
|
record := types.MakeDatums(
|
|
schema.Name.O, // TABLE_SCHEMA
|
|
tbl.Name.O, // TABLE_NAME
|
|
tbl.ID, // TABLE_ID
|
|
int64(tbl.TiFlashReplica.Count), // REPLICA_COUNT
|
|
strings.Join(tbl.TiFlashReplica.LocationLabels, ","), // LOCATION_LABELS
|
|
tbl.TiFlashReplica.Available, // AVAILABLE
|
|
progress, // PROGRESS
|
|
)
|
|
rows = append(rows, record)
|
|
}
|
|
}
|
|
e.rows = rows
|
|
return
|
|
}
|
|
|
|
func (e *memtableRetriever) setDataForStatementsSummary(ctx sessionctx.Context, tableName string) error {
|
|
user := ctx.GetSessionVars().User
|
|
isSuper := false
|
|
if pm := privilege.GetPrivilegeManager(ctx); pm != nil {
|
|
isSuper = pm.RequestVerificationWithUser("", "", "", mysql.SuperPriv, user)
|
|
}
|
|
switch tableName {
|
|
case infoschema.TableStatementsSummary,
|
|
infoschema.ClusterTableStatementsSummary:
|
|
e.rows = stmtsummary.StmtSummaryByDigestMap.ToCurrentDatum(user, isSuper)
|
|
case infoschema.TableStatementsSummaryHistory,
|
|
infoschema.ClusterTableStatementsSummaryHistory:
|
|
e.rows = stmtsummary.StmtSummaryByDigestMap.ToHistoryDatum(user, isSuper)
|
|
}
|
|
switch tableName {
|
|
case infoschema.ClusterTableStatementsSummary,
|
|
infoschema.ClusterTableStatementsSummaryHistory:
|
|
rows, err := infoschema.AppendHostInfoToRows(e.rows)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
e.rows = rows
|
|
}
|
|
return nil
|
|
}
|
|
|
|
// TiFlashSystemTableRetriever is used to read system table from tiflash.
|
|
type TiFlashSystemTableRetriever struct {
|
|
dummyCloser
|
|
table *model.TableInfo
|
|
outputCols []*model.ColumnInfo
|
|
instanceCount int
|
|
instanceIdx int
|
|
instanceInfos []tiflashInstanceInfo
|
|
rowIdx int
|
|
retrieved bool
|
|
initialized bool
|
|
extractor *plannercore.TiFlashSystemTableExtractor
|
|
}
|
|
|
|
func (e *TiFlashSystemTableRetriever) retrieve(ctx context.Context, sctx sessionctx.Context) ([][]types.Datum, error) {
|
|
if e.extractor.SkipRequest || e.retrieved {
|
|
return nil, nil
|
|
}
|
|
if !e.initialized {
|
|
err := e.initialize(sctx, e.extractor.TiFlashInstances)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
}
|
|
if e.instanceCount == 0 || e.instanceIdx >= e.instanceCount {
|
|
e.retrieved = true
|
|
return nil, nil
|
|
}
|
|
|
|
for {
|
|
rows, err := e.dataForTiFlashSystemTables(sctx, e.extractor.TiDBDatabases, e.extractor.TiDBTables)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
if len(rows) > 0 || e.instanceIdx >= e.instanceCount {
|
|
return rows, nil
|
|
}
|
|
}
|
|
}
|
|
|
|
type tiflashInstanceInfo struct {
|
|
id string
|
|
url string
|
|
}
|
|
|
|
func (e *TiFlashSystemTableRetriever) initialize(sctx sessionctx.Context, tiflashInstances set.StringSet) error {
|
|
store := sctx.GetStore()
|
|
if etcd, ok := store.(tikv.EtcdBackend); ok {
|
|
var addrs []string
|
|
var err error
|
|
if addrs, err = etcd.EtcdAddrs(); err != nil {
|
|
return err
|
|
}
|
|
if addrs != nil {
|
|
domainFromCtx := domain.GetDomain(sctx)
|
|
if domainFromCtx != nil {
|
|
cli := domainFromCtx.GetEtcdClient()
|
|
prefix := "/tiflash/cluster/http_port/"
|
|
kv := clientv3.NewKV(cli)
|
|
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
|
|
resp, err := kv.Get(ctx, prefix, clientv3.WithPrefix())
|
|
cancel()
|
|
if err != nil {
|
|
return errors.Trace(err)
|
|
}
|
|
for _, ev := range resp.Kvs {
|
|
id := string(ev.Key)[len(prefix):]
|
|
if len(tiflashInstances) > 0 && !tiflashInstances.Exist(id) {
|
|
continue
|
|
}
|
|
url := fmt.Sprintf("%s://%s", util.InternalHTTPSchema(), ev.Value)
|
|
req, err := http.NewRequest(http.MethodGet, url, nil)
|
|
if err != nil {
|
|
return errors.Trace(err)
|
|
}
|
|
_, err = util.InternalHTTPClient().Do(req)
|
|
if err != nil {
|
|
sctx.GetSessionVars().StmtCtx.AppendWarning(err)
|
|
continue
|
|
}
|
|
e.instanceInfos = append(e.instanceInfos, tiflashInstanceInfo{
|
|
id: id,
|
|
url: url,
|
|
})
|
|
e.instanceCount += 1
|
|
}
|
|
e.initialized = true
|
|
return nil
|
|
}
|
|
}
|
|
return errors.Errorf("Etcd addrs not found")
|
|
}
|
|
return errors.Errorf("%T not an etcd backend", store)
|
|
}
|
|
|
|
func (e *TiFlashSystemTableRetriever) dataForTiFlashSystemTables(ctx sessionctx.Context, tidbDatabases string, tidbTables string) ([][]types.Datum, error) {
|
|
var columnNames []string
|
|
for _, c := range e.outputCols {
|
|
if c.Name.O == "TIFLASH_INSTANCE" {
|
|
continue
|
|
}
|
|
columnNames = append(columnNames, c.Name.L)
|
|
}
|
|
maxCount := 1024
|
|
targetTable := strings.ToLower(strings.Replace(e.table.Name.O, "TIFLASH", "DT", 1))
|
|
var filters []string
|
|
if len(tidbDatabases) > 0 {
|
|
filters = append(filters, fmt.Sprintf("tidb_database IN (%s)", strings.ReplaceAll(tidbDatabases, "\"", "'")))
|
|
}
|
|
if len(tidbTables) > 0 {
|
|
filters = append(filters, fmt.Sprintf("tidb_table IN (%s)", strings.ReplaceAll(tidbTables, "\"", "'")))
|
|
}
|
|
sql := fmt.Sprintf("SELECT %s FROM system.%s", strings.Join(columnNames, ","), targetTable)
|
|
if len(filters) > 0 {
|
|
sql = fmt.Sprintf("%s WHERE %s", sql, strings.Join(filters, " AND "))
|
|
}
|
|
sql = fmt.Sprintf("%s LIMIT %d, %d", sql, e.rowIdx, maxCount)
|
|
notNumber := "nan"
|
|
instanceInfo := e.instanceInfos[e.instanceIdx]
|
|
url := instanceInfo.url
|
|
req, err := http.NewRequest(http.MethodGet, url, nil)
|
|
if err != nil {
|
|
return nil, errors.Trace(err)
|
|
}
|
|
q := req.URL.Query()
|
|
q.Add("query", sql)
|
|
req.URL.RawQuery = q.Encode()
|
|
resp, err := util.InternalHTTPClient().Do(req)
|
|
if err != nil {
|
|
return nil, errors.Trace(err)
|
|
}
|
|
body, err := ioutil.ReadAll(resp.Body)
|
|
terror.Log(resp.Body.Close())
|
|
if err != nil {
|
|
return nil, errors.Trace(err)
|
|
}
|
|
records := strings.Split(string(body), "\n")
|
|
var rows [][]types.Datum
|
|
for _, record := range records {
|
|
if len(record) == 0 {
|
|
continue
|
|
}
|
|
fields := strings.Split(record, "\t")
|
|
if len(fields) < len(e.outputCols)-1 {
|
|
return nil, errors.Errorf("Record from tiflash doesn't match schema %v", fields)
|
|
}
|
|
row := make([]types.Datum, len(e.outputCols))
|
|
for index, column := range e.outputCols {
|
|
if column.Name.O == "TIFLASH_INSTANCE" {
|
|
continue
|
|
}
|
|
if column.Tp == mysql.TypeVarchar {
|
|
row[index].SetString(fields[index], mysql.DefaultCollationName)
|
|
} else if column.Tp == mysql.TypeLonglong {
|
|
if fields[index] == notNumber {
|
|
continue
|
|
}
|
|
value, err := strconv.ParseInt(fields[index], 10, 64)
|
|
if err != nil {
|
|
return nil, errors.Trace(err)
|
|
}
|
|
row[index].SetInt64(value)
|
|
} else if column.Tp == mysql.TypeDouble {
|
|
if fields[index] == notNumber {
|
|
continue
|
|
}
|
|
value, err := strconv.ParseFloat(fields[index], 64)
|
|
if err != nil {
|
|
return nil, errors.Trace(err)
|
|
}
|
|
row[index].SetFloat64(value)
|
|
} else {
|
|
return nil, errors.Errorf("Meet column of unknown type %v", column)
|
|
}
|
|
}
|
|
row[len(e.outputCols)-1].SetString(instanceInfo.id, mysql.DefaultCollationName)
|
|
rows = append(rows, row)
|
|
}
|
|
e.rowIdx += len(rows)
|
|
if len(rows) < maxCount {
|
|
e.instanceIdx += 1
|
|
e.rowIdx = 0
|
|
}
|
|
return rows, nil
|
|
}
|
|
|
|
type hugeMemTableRetriever struct {
|
|
dummyCloser
|
|
table *model.TableInfo
|
|
columns []*model.ColumnInfo
|
|
retrieved bool
|
|
initialized bool
|
|
rows [][]types.Datum
|
|
dbs []*model.DBInfo
|
|
dbsIdx int
|
|
tblIdx int
|
|
}
|
|
|
|
// retrieve implements the infoschemaRetriever interface
|
|
func (e *hugeMemTableRetriever) retrieve(ctx context.Context, sctx sessionctx.Context) ([][]types.Datum, error) {
|
|
if e.retrieved {
|
|
return nil, nil
|
|
}
|
|
|
|
if !e.initialized {
|
|
is := infoschema.GetInfoSchema(sctx)
|
|
dbs := is.AllSchemas()
|
|
sort.Sort(infoschema.SchemasSorter(dbs))
|
|
e.dbs = dbs
|
|
e.initialized = true
|
|
e.rows = make([][]types.Datum, 0, 1024)
|
|
}
|
|
|
|
var err error
|
|
switch e.table.Name.O {
|
|
case infoschema.TableColumns:
|
|
err = e.setDataForColumns(sctx)
|
|
}
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
e.retrieved = len(e.rows) == 0
|
|
|
|
return adjustColumns(e.rows, e.columns, e.table), nil
|
|
}
|
|
|
|
func adjustColumns(input [][]types.Datum, outColumns []*model.ColumnInfo, table *model.TableInfo) [][]types.Datum {
|
|
if len(outColumns) == len(table.Columns) {
|
|
return input
|
|
}
|
|
rows := make([][]types.Datum, len(input))
|
|
for i, fullRow := range input {
|
|
row := make([]types.Datum, len(outColumns))
|
|
for j, col := range outColumns {
|
|
row[j] = fullRow[col.Offset]
|
|
}
|
|
rows[i] = row
|
|
}
|
|
return rows
|
|
}
|
|
|