Skip to content

Commit

Permalink
ucp: migrate dataForProcesslist & dataForClusterProcesslist
Browse files Browse the repository at this point in the history
  • Loading branch information
hsqlu committed Mar 19, 2020
1 parent d052f5c commit 918df7d
Show file tree
Hide file tree
Showing 4 changed files with 54 additions and 47 deletions.
2 changes: 2 additions & 0 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1432,6 +1432,8 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo
strings.ToLower(infoschema.TableUserPrivileges),
strings.ToLower(infoschema.TableMetricTables),
strings.ToLower(infoschema.TableCollationCharacterSetApplicability),
strings.ToLower(infoschema.TableProcesslist),
strings.ToLower(infoschema.ClusterTableProcesslist),
strings.ToLower(infoschema.TableTiKVRegionPeers),
strings.ToLower(infoschema.TableTiDBHotRegions),
strings.ToLower(infoschema.TableSessionVar),
Expand Down
45 changes: 45 additions & 0 deletions executor/infoschema_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -88,6 +88,10 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex
e.setDataForMetricTables(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:
Expand Down Expand Up @@ -793,6 +797,47 @@ func (e *memtableRetriever) setDataFromKeyColumnUsage(ctx sessionctx.Context, sc
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()
Expand Down
10 changes: 2 additions & 8 deletions infoschema/cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ import (

"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/domain/infosync"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util"
)
Expand All @@ -30,13 +29,13 @@ import (
const (
// ClusterTableSlowLog is the string constant of cluster slow query memory table.
ClusterTableSlowLog = "CLUSTER_SLOW_QUERY"
clusterTableProcesslist = "CLUSTER_PROCESSLIST"
ClusterTableProcesslist = "CLUSTER_PROCESSLIST"
)

// memTableToClusterTables means add memory table to cluster table.
var memTableToClusterTables = map[string]string{
TableSlowQuery: ClusterTableSlowLog,
tableProcesslist: clusterTableProcesslist,
TableProcesslist: ClusterTableProcesslist,
}

func init() {
Expand Down Expand Up @@ -72,11 +71,6 @@ func isClusterTableByName(dbName, tableName string) bool {
return false
}

func dataForClusterProcesslist(ctx sessionctx.Context) (rows [][]types.Datum, err error) {
rows = dataForProcesslist(ctx)
return AppendHostInfoToRows(rows)
}

// AppendHostInfoToRows appends host info to the rows.
func AppendHostInfoToRows(rows [][]types.Datum) ([][]types.Datum, error) {
serverInfo, err := infosync.GetServerInfo()
Expand Down
44 changes: 5 additions & 39 deletions infoschema/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -93,7 +93,8 @@ const (
tableTableSpaces = "TABLESPACES"
// TableCollationCharacterSetApplicability is the string constant of infoschema memory table.
TableCollationCharacterSetApplicability = "COLLATION_CHARACTER_SET_APPLICABILITY"
tableProcesslist = "PROCESSLIST"
// TableProcesslist is the string constant of infoschema table.
TableProcesslist = "PROCESSLIST"
// TableTiDBIndexes is the string constant of infoschema table
TableTiDBIndexes = "TIDB_INDEXES"
// TableTiDBHotRegions is the string constant of infoschema table
Expand Down Expand Up @@ -166,7 +167,7 @@ var tableIDMap = map[string]int64{
tableOptimizerTrace: autoid.InformationSchemaDBID + 30,
tableTableSpaces: autoid.InformationSchemaDBID + 31,
TableCollationCharacterSetApplicability: autoid.InformationSchemaDBID + 32,
tableProcesslist: autoid.InformationSchemaDBID + 33,
TableProcesslist: autoid.InformationSchemaDBID + 33,
TableTiDBIndexes: autoid.InformationSchemaDBID + 34,
TableSlowQuery: autoid.InformationSchemaDBID + 35,
TableTiDBHotRegions: autoid.InformationSchemaDBID + 36,
Expand All @@ -180,7 +181,7 @@ var tableIDMap = map[string]int64{
TableClusterLoad: autoid.InformationSchemaDBID + 44,
tableTiFlashReplica: autoid.InformationSchemaDBID + 45,
ClusterTableSlowLog: autoid.InformationSchemaDBID + 46,
clusterTableProcesslist: autoid.InformationSchemaDBID + 47,
ClusterTableProcesslist: autoid.InformationSchemaDBID + 47,
TableClusterLog: autoid.InformationSchemaDBID + 48,
TableClusterHardware: autoid.InformationSchemaDBID + 49,
TableClusterSystemInfo: autoid.InformationSchemaDBID + 50,
Expand Down Expand Up @@ -1080,36 +1081,6 @@ func dataForTiKVStoreStatus(ctx sessionctx.Context) (records [][]types.Datum, er
return records, nil
}

func dataForProcesslist(ctx sessionctx.Context) [][]types.Datum {
sm := ctx.GetSessionManager()
if sm == nil {
return nil
}

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)
}
return records
}

// GetShardingInfo returns a nil or description string for the sharding information of given TableInfo.
// The returned description string may be:
// - "NOT_SHARDED": for tables that SHARD_ROW_ID_BITS is not specified.
Expand Down Expand Up @@ -1556,7 +1527,7 @@ var tableNameToColumns = map[string][]columnInfo{
tableOptimizerTrace: tableOptimizerTraceCols,
tableTableSpaces: tableTableSpacesCols,
TableCollationCharacterSetApplicability: tableCollationCharacterSetApplicabilityCols,
tableProcesslist: tableProcesslistCols,
TableProcesslist: tableProcesslistCols,
TableTiDBIndexes: tableTiDBIndexesCols,
TableSlowQuery: slowQueryCols,
TableTiDBHotRegions: TableTiDBHotRegionsCols,
Expand Down Expand Up @@ -1639,8 +1610,6 @@ func (it *infoschemaTable) getRows(ctx sessionctx.Context, cols []*table.Column)
case tableSessionStatus:
case tableOptimizerTrace:
case tableTableSpaces:
case tableProcesslist:
fullRows = dataForProcesslist(ctx)
case tableTiKVStoreStatus:
fullRows, err = dataForTiKVStoreStatus(ctx)
case tableAnalyzeStatus:
Expand All @@ -1651,9 +1620,6 @@ func (it *infoschemaTable) getRows(ctx sessionctx.Context, cols []*table.Column)
fullRows, err = dataForServersInfo()
case tableTiFlashReplica:
fullRows = dataForTableTiFlashReplica(ctx, dbs)
// Data for cluster processlist memory table.
case clusterTableProcesslist:
fullRows, err = dataForClusterProcesslist(ctx)
}
if err != nil {
return nil, err
Expand Down

0 comments on commit 918df7d

Please sign in to comment.