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.
 
 

941 lines
31 KiB

// Copyright 2019 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 stmtsummary
import (
"bytes"
"container/list"
"fmt"
"sort"
"strings"
"sync"
"sync/atomic"
"time"
"github.com/pingcap/parser/auth"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/hack"
"github.com/pingcap/tidb/util/kvcache"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/plancodec"
"go.uber.org/zap"
)
// stmtSummaryByDigestKey defines key for stmtSummaryByDigestMap.summaryMap.
type stmtSummaryByDigestKey struct {
// Same statements may appear in different schema, but they refer to different tables.
schemaName string
digest string
// The digest of the previous statement.
prevDigest string
// The digest of the plan of this SQL.
planDigest string
// `hash` is the hash value of this object.
hash []byte
}
// Hash implements SimpleLRUCache.Key.
// Only when current SQL is `commit` do we record `prevSQL`. Otherwise, `prevSQL` is empty.
// `prevSQL` is included in the key To distinguish different transactions.
func (key *stmtSummaryByDigestKey) Hash() []byte {
if len(key.hash) == 0 {
key.hash = make([]byte, 0, len(key.schemaName)+len(key.digest)+len(key.prevDigest)+len(key.planDigest))
key.hash = append(key.hash, hack.Slice(key.digest)...)
key.hash = append(key.hash, hack.Slice(key.schemaName)...)
key.hash = append(key.hash, hack.Slice(key.prevDigest)...)
key.hash = append(key.hash, hack.Slice(key.planDigest)...)
}
return key.hash
}
// stmtSummaryByDigestMap is a LRU cache that stores statement summaries.
type stmtSummaryByDigestMap struct {
// It's rare to read concurrently, so RWMutex is not needed.
sync.Mutex
summaryMap *kvcache.SimpleLRUCache
// beginTimeForCurInterval is the begin time for current summary.
beginTimeForCurInterval int64
// sysVars encapsulates system variables needed to control statement summary.
sysVars *systemVars
}
// StmtSummaryByDigestMap is a global map containing all statement summaries.
var StmtSummaryByDigestMap = newStmtSummaryByDigestMap()
// stmtSummaryByDigest is the summary for each type of statements.
type stmtSummaryByDigest struct {
// It's rare to read concurrently, so RWMutex is not needed.
// Mutex is only used to lock `history`.
sync.Mutex
initialized bool
// Each element in history is a summary in one interval.
history *list.List
// Following fields are common for each summary element.
// They won't change once this object is created, so locking is not needed.
schemaName string
digest string
planDigest string
stmtType string
normalizedSQL string
tableNames string
isInternal bool
}
// stmtSummaryByDigestElement is the summary for each type of statements in current interval.
type stmtSummaryByDigestElement struct {
sync.Mutex
// Each summary is summarized between [beginTime, endTime).
beginTime int64
endTime int64
// basic
sampleSQL string
prevSQL string
samplePlan string
indexNames []string
execCount int64
sumErrors int
sumWarnings int
// latency
sumLatency time.Duration
maxLatency time.Duration
minLatency time.Duration
sumParseLatency time.Duration
maxParseLatency time.Duration
sumCompileLatency time.Duration
maxCompileLatency time.Duration
// coprocessor
sumNumCopTasks int64
maxCopProcessTime time.Duration
maxCopProcessAddress string
maxCopWaitTime time.Duration
maxCopWaitAddress string
// TiKV
sumProcessTime time.Duration
maxProcessTime time.Duration
sumWaitTime time.Duration
maxWaitTime time.Duration
sumBackoffTime time.Duration
maxBackoffTime time.Duration
sumTotalKeys int64
maxTotalKeys int64
sumProcessedKeys int64
maxProcessedKeys int64
// txn
commitCount int64
sumGetCommitTsTime time.Duration
maxGetCommitTsTime time.Duration
sumPrewriteTime time.Duration
maxPrewriteTime time.Duration
sumCommitTime time.Duration
maxCommitTime time.Duration
sumLocalLatchTime time.Duration
maxLocalLatchTime time.Duration
sumCommitBackoffTime int64
maxCommitBackoffTime int64
sumResolveLockTime int64
maxResolveLockTime int64
sumWriteKeys int64
maxWriteKeys int
sumWriteSize int64
maxWriteSize int
sumPrewriteRegionNum int64
maxPrewriteRegionNum int32
sumTxnRetry int64
maxTxnRetry int
sumExecRetryCount int64
sumExecRetryTime time.Duration
sumBackoffTimes int64
backoffTypes map[fmt.Stringer]int
authUsers map[string]struct{}
// other
sumMem int64
maxMem int64
sumDisk int64
maxDisk int64
sumAffectedRows uint64
sumKVTotal time.Duration
sumPDTotal time.Duration
sumBackoffTotal time.Duration
sumWriteSQLRespTotal time.Duration
prepared bool
// The first time this type of SQL executes.
firstSeen time.Time
// The last time this type of SQL executes.
lastSeen time.Time
// plan cache
planInCache bool
planCacheHits int64
// pessimistic execution retry information.
execRetryCount uint
execRetryTime time.Duration
}
// StmtExecInfo records execution information of each statement.
type StmtExecInfo struct {
SchemaName string
OriginalSQL string
NormalizedSQL string
Digest string
PrevSQL string
PrevSQLDigest string
PlanGenerator func() string
PlanDigest string
PlanDigestGen func() string
User string
TotalLatency time.Duration
ParseLatency time.Duration
CompileLatency time.Duration
StmtCtx *stmtctx.StatementContext
CopTasks *stmtctx.CopTasksDetails
ExecDetail *execdetails.ExecDetails
MemMax int64
DiskMax int64
StartTime time.Time
IsInternal bool
Succeed bool
PlanInCache bool
ExecRetryCount uint
ExecRetryTime time.Duration
execdetails.StmtExecDetails
Prepared bool
}
// newStmtSummaryByDigestMap creates an empty stmtSummaryByDigestMap.
func newStmtSummaryByDigestMap() *stmtSummaryByDigestMap {
sysVars := newSysVars()
maxStmtCount := uint(sysVars.getVariable(typeMaxStmtCount))
return &stmtSummaryByDigestMap{
summaryMap: kvcache.NewSimpleLRUCache(maxStmtCount, 0, 0),
sysVars: sysVars,
}
}
// AddStatement adds a statement to StmtSummaryByDigestMap.
func (ssMap *stmtSummaryByDigestMap) AddStatement(sei *StmtExecInfo) {
// All times are counted in seconds.
now := time.Now().Unix()
intervalSeconds := ssMap.refreshInterval()
historySize := ssMap.historySize()
key := &stmtSummaryByDigestKey{
schemaName: sei.SchemaName,
digest: sei.Digest,
prevDigest: sei.PrevSQLDigest,
planDigest: sei.PlanDigest,
}
// Calculate hash value in advance, to reduce the time holding the lock.
key.Hash()
// Enclose the block in a function to ensure the lock will always be released.
summary, beginTime := func() (*stmtSummaryByDigest, int64) {
ssMap.Lock()
defer ssMap.Unlock()
// Check again. Statements could be added before disabling the flag and after Clear().
if !ssMap.Enabled() {
return nil, 0
}
if sei.IsInternal && !ssMap.EnabledInternal() {
return nil, 0
}
if ssMap.beginTimeForCurInterval+intervalSeconds <= now {
// `beginTimeForCurInterval` is a multiple of intervalSeconds, so that when the interval is a multiple
// of 60 (or 600, 1800, 3600, etc), begin time shows 'XX:XX:00', not 'XX:XX:01'~'XX:XX:59'.
ssMap.beginTimeForCurInterval = now / intervalSeconds * intervalSeconds
}
beginTime := ssMap.beginTimeForCurInterval
value, ok := ssMap.summaryMap.Get(key)
var summary *stmtSummaryByDigest
if !ok {
// Lazy initialize it to release ssMap.mutex ASAP.
summary = new(stmtSummaryByDigest)
ssMap.summaryMap.Put(key, summary)
} else {
summary = value.(*stmtSummaryByDigest)
}
summary.isInternal = summary.isInternal && sei.IsInternal
return summary, beginTime
}()
// Lock a single entry, not the whole cache.
if summary != nil {
summary.add(sei, beginTime, intervalSeconds, historySize)
}
}
// Clear removes all statement summaries.
func (ssMap *stmtSummaryByDigestMap) Clear() {
ssMap.Lock()
defer ssMap.Unlock()
ssMap.summaryMap.DeleteAll()
ssMap.beginTimeForCurInterval = 0
}
// clearInternal removes all statement summaries which are internal summaries.
func (ssMap *stmtSummaryByDigestMap) clearInternal() {
ssMap.Lock()
defer ssMap.Unlock()
for _, key := range ssMap.summaryMap.Keys() {
summary, ok := ssMap.summaryMap.Get(key)
if !ok {
continue
}
if summary.(*stmtSummaryByDigest).isInternal {
ssMap.summaryMap.Delete(key)
}
}
}
// ToCurrentDatum converts current statement summaries to datum.
func (ssMap *stmtSummaryByDigestMap) ToCurrentDatum(user *auth.UserIdentity, isSuper bool) [][]types.Datum {
ssMap.Lock()
values := ssMap.summaryMap.Values()
beginTime := ssMap.beginTimeForCurInterval
ssMap.Unlock()
rows := make([][]types.Datum, 0, len(values))
for _, value := range values {
record := value.(*stmtSummaryByDigest).toCurrentDatum(beginTime, user, isSuper)
if record != nil {
rows = append(rows, record)
}
}
return rows
}
// ToHistoryDatum converts history statements summaries to datum.
func (ssMap *stmtSummaryByDigestMap) ToHistoryDatum(user *auth.UserIdentity, isSuper bool) [][]types.Datum {
ssMap.Lock()
values := ssMap.summaryMap.Values()
ssMap.Unlock()
historySize := ssMap.historySize()
rows := make([][]types.Datum, 0, len(values)*historySize)
for _, value := range values {
records := value.(*stmtSummaryByDigest).toHistoryDatum(historySize, user, isSuper)
rows = append(rows, records...)
}
return rows
}
// GetMoreThanOnceBindableStmt gets users' select/update/delete SQLs that occurred more than once.
func (ssMap *stmtSummaryByDigestMap) GetMoreThanOnceBindableStmt() ([]string, []string) {
ssMap.Lock()
values := ssMap.summaryMap.Values()
ssMap.Unlock()
schemas := make([]string, 0, len(values))
sqls := make([]string, 0, len(values))
for _, value := range values {
ssbd := value.(*stmtSummaryByDigest)
func() {
ssbd.Lock()
defer ssbd.Unlock()
if ssbd.initialized && (ssbd.stmtType == "Select" || ssbd.stmtType == "Delete" || ssbd.stmtType == "Update" || ssbd.stmtType == "Insert" || ssbd.stmtType == "Replace") {
if ssbd.history.Len() > 0 {
ssElement := ssbd.history.Back().Value.(*stmtSummaryByDigestElement)
ssElement.Lock()
// Empty auth users means that it is an internal queries.
if len(ssElement.authUsers) > 0 && (ssbd.history.Len() > 1 || ssElement.execCount > 1) {
schemas = append(schemas, ssbd.schemaName)
sqls = append(sqls, ssElement.sampleSQL)
}
ssElement.Unlock()
}
}
}()
}
return schemas, sqls
}
// SetEnabled enables or disables statement summary in global(cluster) or session(server) scope.
func (ssMap *stmtSummaryByDigestMap) SetEnabled(value string, inSession bool) error {
if err := ssMap.sysVars.setVariable(typeEnable, value, inSession); err != nil {
return err
}
// Clear all summaries once statement summary is disabled.
if ssMap.sysVars.getVariable(typeEnable) == 0 {
ssMap.Clear()
}
return nil
}
// Enabled returns whether statement summary is enabled.
func (ssMap *stmtSummaryByDigestMap) Enabled() bool {
return ssMap.sysVars.getVariable(typeEnable) > 0
}
// SetEnabledInternalQuery enables or disables internal statement summary in global(cluster) or session(server) scope.
func (ssMap *stmtSummaryByDigestMap) SetEnabledInternalQuery(value string, inSession bool) error {
if err := ssMap.sysVars.setVariable(typeEnableInternalQuery, value, inSession); err != nil {
return err
}
// Clear all summaries once statement summary is disabled.
if ssMap.sysVars.getVariable(typeEnableInternalQuery) == 0 {
ssMap.clearInternal()
}
return nil
}
// EnabledInternal returns whether internal statement summary is enabled.
func (ssMap *stmtSummaryByDigestMap) EnabledInternal() bool {
return ssMap.sysVars.getVariable(typeEnableInternalQuery) > 0
}
// SetRefreshInterval sets refreshing interval in ssMap.sysVars.
func (ssMap *stmtSummaryByDigestMap) SetRefreshInterval(value string, inSession bool) error {
return ssMap.sysVars.setVariable(typeRefreshInterval, value, inSession)
}
// refreshInterval gets the refresh interval for summaries.
func (ssMap *stmtSummaryByDigestMap) refreshInterval() int64 {
return ssMap.sysVars.getVariable(typeRefreshInterval)
}
// SetHistorySize sets the history size for all summaries.
func (ssMap *stmtSummaryByDigestMap) SetHistorySize(value string, inSession bool) error {
return ssMap.sysVars.setVariable(typeHistorySize, value, inSession)
}
// historySize gets the history size for summaries.
func (ssMap *stmtSummaryByDigestMap) historySize() int {
return int(ssMap.sysVars.getVariable(typeHistorySize))
}
// SetHistorySize sets the history size for all summaries.
func (ssMap *stmtSummaryByDigestMap) SetMaxStmtCount(value string, inSession bool) error {
if err := ssMap.sysVars.setVariable(typeMaxStmtCount, value, inSession); err != nil {
return err
}
capacity := ssMap.sysVars.getVariable(typeMaxStmtCount)
ssMap.Lock()
defer ssMap.Unlock()
return ssMap.summaryMap.SetCapacity(uint(capacity))
}
func (ssMap *stmtSummaryByDigestMap) maxStmtCount() int {
return int(ssMap.sysVars.getVariable(typeMaxStmtCount))
}
// SetHistorySize sets the history size for all summaries.
func (ssMap *stmtSummaryByDigestMap) SetMaxSQLLength(value string, inSession bool) error {
return ssMap.sysVars.setVariable(typeMaxSQLLength, value, inSession)
}
func (ssMap *stmtSummaryByDigestMap) maxSQLLength() int {
return int(ssMap.sysVars.getVariable(typeMaxSQLLength))
}
// newStmtSummaryByDigest creates a stmtSummaryByDigest from StmtExecInfo.
func (ssbd *stmtSummaryByDigest) init(sei *StmtExecInfo, beginTime int64, intervalSeconds int64, historySize int) {
// Use "," to separate table names to support FIND_IN_SET.
var buffer bytes.Buffer
for i, value := range sei.StmtCtx.Tables {
// In `create database` statement, DB name is not empty but table name is empty.
if len(value.Table) == 0 {
continue
}
buffer.WriteString(strings.ToLower(value.DB))
buffer.WriteString(".")
buffer.WriteString(strings.ToLower(value.Table))
if i < len(sei.StmtCtx.Tables)-1 {
buffer.WriteString(",")
}
}
tableNames := buffer.String()
planDigest := sei.PlanDigest
if sei.PlanDigestGen != nil && len(planDigest) == 0 {
// It comes here only when the plan is 'Point_Get'.
planDigest = sei.PlanDigestGen()
}
ssbd.schemaName = sei.SchemaName
ssbd.digest = sei.Digest
ssbd.planDigest = planDigest
ssbd.stmtType = sei.StmtCtx.StmtType
ssbd.normalizedSQL = formatSQL(sei.NormalizedSQL)
ssbd.tableNames = tableNames
ssbd.history = list.New()
ssbd.initialized = true
}
func (ssbd *stmtSummaryByDigest) add(sei *StmtExecInfo, beginTime int64, intervalSeconds int64, historySize int) {
// Enclose this block in a function to ensure the lock will always be released.
ssElement, isElementNew := func() (*stmtSummaryByDigestElement, bool) {
ssbd.Lock()
defer ssbd.Unlock()
if !ssbd.initialized {
ssbd.init(sei, beginTime, intervalSeconds, historySize)
}
var ssElement *stmtSummaryByDigestElement
isElementNew := true
if ssbd.history.Len() > 0 {
lastElement := ssbd.history.Back().Value.(*stmtSummaryByDigestElement)
if lastElement.beginTime >= beginTime {
ssElement = lastElement
isElementNew = false
} else {
// The last elements expires to the history.
lastElement.onExpire(intervalSeconds)
}
}
if isElementNew {
// If the element is new created, `ssElement.add(sei)` should be done inside the lock of `ssbd`.
ssElement = newStmtSummaryByDigestElement(sei, beginTime, intervalSeconds)
ssbd.history.PushBack(ssElement)
}
// `historySize` might be modified anytime, so check expiration every time.
// Even if history is set to 0, current summary is still needed.
for ssbd.history.Len() > historySize && ssbd.history.Len() > 1 {
ssbd.history.Remove(ssbd.history.Front())
}
return ssElement, isElementNew
}()
// Lock a single entry, not the whole `ssbd`.
if !isElementNew {
ssElement.add(sei, intervalSeconds)
}
}
func (ssbd *stmtSummaryByDigest) toCurrentDatum(beginTimeForCurInterval int64, user *auth.UserIdentity, isSuper bool) []types.Datum {
var ssElement *stmtSummaryByDigestElement
ssbd.Lock()
if ssbd.initialized && ssbd.history.Len() > 0 {
ssElement = ssbd.history.Back().Value.(*stmtSummaryByDigestElement)
}
ssbd.Unlock()
// `ssElement` is lazy expired, so expired elements could also be read.
// `beginTime` won't change since `ssElement` is created, so locking is not needed here.
isAuthed := true
if user != nil && !isSuper {
_, isAuthed = ssElement.authUsers[user.Username]
}
if ssElement == nil || ssElement.beginTime < beginTimeForCurInterval || !isAuthed {
return nil
}
return ssElement.toDatum(ssbd)
}
func (ssbd *stmtSummaryByDigest) toHistoryDatum(historySize int, user *auth.UserIdentity, isSuper bool) [][]types.Datum {
// Collect all history summaries to an array.
ssElements := ssbd.collectHistorySummaries(historySize)
rows := make([][]types.Datum, 0, len(ssElements))
for _, ssElement := range ssElements {
isAuthed := true
if user != nil && !isSuper {
_, isAuthed = ssElement.authUsers[user.Username]
}
if isAuthed {
rows = append(rows, ssElement.toDatum(ssbd))
}
}
return rows
}
// collectHistorySummaries puts at most `historySize` summaries to an array.
func (ssbd *stmtSummaryByDigest) collectHistorySummaries(historySize int) []*stmtSummaryByDigestElement {
ssbd.Lock()
defer ssbd.Unlock()
if !ssbd.initialized {
return nil
}
ssElements := make([]*stmtSummaryByDigestElement, 0, ssbd.history.Len())
for listElement := ssbd.history.Front(); listElement != nil && len(ssElements) < historySize; listElement = listElement.Next() {
ssElement := listElement.Value.(*stmtSummaryByDigestElement)
ssElements = append(ssElements, ssElement)
}
return ssElements
}
func newStmtSummaryByDigestElement(sei *StmtExecInfo, beginTime int64, intervalSeconds int64) *stmtSummaryByDigestElement {
// sampleSQL / authUsers(sampleUser) / samplePlan / prevSQL / indexNames store the values shown at the first time,
// because it compacts performance to update every time.
ssElement := &stmtSummaryByDigestElement{
beginTime: beginTime,
sampleSQL: formatSQL(sei.OriginalSQL),
// PrevSQL is already truncated to cfg.Log.QueryLogMaxLen.
prevSQL: sei.PrevSQL,
// samplePlan needs to be decoded so it can't be truncated.
samplePlan: sei.PlanGenerator(),
indexNames: sei.StmtCtx.IndexNames,
minLatency: sei.TotalLatency,
firstSeen: sei.StartTime,
lastSeen: sei.StartTime,
backoffTypes: make(map[fmt.Stringer]int),
authUsers: make(map[string]struct{}),
planInCache: false,
planCacheHits: 0,
prepared: sei.Prepared,
}
ssElement.add(sei, intervalSeconds)
return ssElement
}
// onExpire is called when this element expires to history.
func (ssElement *stmtSummaryByDigestElement) onExpire(intervalSeconds int64) {
ssElement.Lock()
defer ssElement.Unlock()
// refreshInterval may change anytime, so we need to update endTime.
if ssElement.beginTime+intervalSeconds > ssElement.endTime {
// // If interval changes to a bigger value, update endTime to beginTime + interval.
ssElement.endTime = ssElement.beginTime + intervalSeconds
} else if ssElement.beginTime+intervalSeconds < ssElement.endTime {
now := time.Now().Unix()
// If interval changes to a smaller value and now > beginTime + interval, update endTime to current time.
if now > ssElement.beginTime+intervalSeconds {
ssElement.endTime = now
}
}
}
func (ssElement *stmtSummaryByDigestElement) add(sei *StmtExecInfo, intervalSeconds int64) {
ssElement.Lock()
defer ssElement.Unlock()
// add user to auth users set
if len(sei.User) > 0 {
ssElement.authUsers[sei.User] = struct{}{}
}
// refreshInterval may change anytime, update endTime ASAP.
ssElement.endTime = ssElement.beginTime + intervalSeconds
ssElement.execCount++
if !sei.Succeed {
ssElement.sumErrors += 1
}
ssElement.sumWarnings += int(sei.StmtCtx.WarningCount())
// latency
ssElement.sumLatency += sei.TotalLatency
if sei.TotalLatency > ssElement.maxLatency {
ssElement.maxLatency = sei.TotalLatency
}
if sei.TotalLatency < ssElement.minLatency {
ssElement.minLatency = sei.TotalLatency
}
ssElement.sumParseLatency += sei.ParseLatency
if sei.ParseLatency > ssElement.maxParseLatency {
ssElement.maxParseLatency = sei.ParseLatency
}
ssElement.sumCompileLatency += sei.CompileLatency
if sei.CompileLatency > ssElement.maxCompileLatency {
ssElement.maxCompileLatency = sei.CompileLatency
}
// coprocessor
numCopTasks := int64(sei.CopTasks.NumCopTasks)
ssElement.sumNumCopTasks += numCopTasks
if sei.CopTasks.MaxProcessTime > ssElement.maxCopProcessTime {
ssElement.maxCopProcessTime = sei.CopTasks.MaxProcessTime
ssElement.maxCopProcessAddress = sei.CopTasks.MaxProcessAddress
}
if sei.CopTasks.MaxWaitTime > ssElement.maxCopWaitTime {
ssElement.maxCopWaitTime = sei.CopTasks.MaxWaitTime
ssElement.maxCopWaitAddress = sei.CopTasks.MaxWaitAddress
}
// TiKV
ssElement.sumProcessTime += sei.ExecDetail.ProcessTime
if sei.ExecDetail.ProcessTime > ssElement.maxProcessTime {
ssElement.maxProcessTime = sei.ExecDetail.ProcessTime
}
ssElement.sumWaitTime += sei.ExecDetail.WaitTime
if sei.ExecDetail.WaitTime > ssElement.maxWaitTime {
ssElement.maxWaitTime = sei.ExecDetail.WaitTime
}
ssElement.sumBackoffTime += sei.ExecDetail.BackoffTime
if sei.ExecDetail.BackoffTime > ssElement.maxBackoffTime {
ssElement.maxBackoffTime = sei.ExecDetail.BackoffTime
}
ssElement.sumTotalKeys += sei.ExecDetail.TotalKeys
if sei.ExecDetail.TotalKeys > ssElement.maxTotalKeys {
ssElement.maxTotalKeys = sei.ExecDetail.TotalKeys
}
ssElement.sumProcessedKeys += sei.ExecDetail.ProcessedKeys
if sei.ExecDetail.ProcessedKeys > ssElement.maxProcessedKeys {
ssElement.maxProcessedKeys = sei.ExecDetail.ProcessedKeys
}
// txn
commitDetails := sei.ExecDetail.CommitDetail
if commitDetails != nil {
ssElement.commitCount++
ssElement.sumPrewriteTime += commitDetails.PrewriteTime
if commitDetails.PrewriteTime > ssElement.maxPrewriteTime {
ssElement.maxPrewriteTime = commitDetails.PrewriteTime
}
ssElement.sumCommitTime += commitDetails.CommitTime
if commitDetails.CommitTime > ssElement.maxCommitTime {
ssElement.maxCommitTime = commitDetails.CommitTime
}
ssElement.sumGetCommitTsTime += commitDetails.GetCommitTsTime
if commitDetails.GetCommitTsTime > ssElement.maxGetCommitTsTime {
ssElement.maxGetCommitTsTime = commitDetails.GetCommitTsTime
}
commitBackoffTime := atomic.LoadInt64(&commitDetails.CommitBackoffTime)
ssElement.sumCommitBackoffTime += commitBackoffTime
if commitBackoffTime > ssElement.maxCommitBackoffTime {
ssElement.maxCommitBackoffTime = commitBackoffTime
}
resolveLockTime := atomic.LoadInt64(&commitDetails.ResolveLockTime)
ssElement.sumResolveLockTime += resolveLockTime
if resolveLockTime > ssElement.maxResolveLockTime {
ssElement.maxResolveLockTime = resolveLockTime
}
ssElement.sumLocalLatchTime += commitDetails.LocalLatchTime
if commitDetails.LocalLatchTime > ssElement.maxLocalLatchTime {
ssElement.maxLocalLatchTime = commitDetails.LocalLatchTime
}
ssElement.sumWriteKeys += int64(commitDetails.WriteKeys)
if commitDetails.WriteKeys > ssElement.maxWriteKeys {
ssElement.maxWriteKeys = commitDetails.WriteKeys
}
ssElement.sumWriteSize += int64(commitDetails.WriteSize)
if commitDetails.WriteSize > ssElement.maxWriteSize {
ssElement.maxWriteSize = commitDetails.WriteSize
}
prewriteRegionNum := atomic.LoadInt32(&commitDetails.PrewriteRegionNum)
ssElement.sumPrewriteRegionNum += int64(prewriteRegionNum)
if prewriteRegionNum > ssElement.maxPrewriteRegionNum {
ssElement.maxPrewriteRegionNum = prewriteRegionNum
}
ssElement.sumTxnRetry += int64(commitDetails.TxnRetry)
if commitDetails.TxnRetry > ssElement.maxTxnRetry {
ssElement.maxTxnRetry = commitDetails.TxnRetry
}
commitDetails.Mu.Lock()
ssElement.sumBackoffTimes += int64(len(commitDetails.Mu.BackoffTypes))
for _, backoffType := range commitDetails.Mu.BackoffTypes {
ssElement.backoffTypes[backoffType] += 1
}
commitDetails.Mu.Unlock()
}
//plan cache
if sei.PlanInCache {
ssElement.planInCache = true
ssElement.planCacheHits += 1
} else {
ssElement.planInCache = false
}
// other
ssElement.sumAffectedRows += sei.StmtCtx.AffectedRows()
ssElement.sumMem += sei.MemMax
if sei.MemMax > ssElement.maxMem {
ssElement.maxMem = sei.MemMax
}
ssElement.sumDisk += sei.DiskMax
if sei.DiskMax > ssElement.maxDisk {
ssElement.maxDisk = sei.DiskMax
}
if sei.StartTime.Before(ssElement.firstSeen) {
ssElement.firstSeen = sei.StartTime
}
if ssElement.lastSeen.Before(sei.StartTime) {
ssElement.lastSeen = sei.StartTime
}
if sei.ExecRetryCount > 0 {
ssElement.execRetryCount += sei.ExecRetryCount
ssElement.execRetryTime += sei.ExecRetryTime
}
ssElement.sumKVTotal += time.Duration(atomic.LoadInt64(&sei.StmtExecDetails.WaitKVRespDuration))
ssElement.sumPDTotal += time.Duration(atomic.LoadInt64(&sei.StmtExecDetails.WaitPDRespDuration))
ssElement.sumBackoffTotal += time.Duration(atomic.LoadInt64(&sei.StmtExecDetails.BackoffDuration))
ssElement.sumWriteSQLRespTotal += sei.StmtExecDetails.WriteSQLRespDuration
}
func (ssElement *stmtSummaryByDigestElement) toDatum(ssbd *stmtSummaryByDigest) []types.Datum {
ssElement.Lock()
defer ssElement.Unlock()
plan, err := plancodec.DecodePlan(ssElement.samplePlan)
if err != nil {
logutil.BgLogger().Error("decode plan in statement summary failed", zap.String("plan", ssElement.samplePlan), zap.Error(err))
plan = ""
}
sampleUser := ""
for key := range ssElement.authUsers {
sampleUser = key
break
}
// Actually, there's a small chance that endTime is out of date, but it's hard to keep it up to date all the time.
return types.MakeDatums(
types.NewTime(types.FromGoTime(time.Unix(ssElement.beginTime, 0)), mysql.TypeTimestamp, 0),
types.NewTime(types.FromGoTime(time.Unix(ssElement.endTime, 0)), mysql.TypeTimestamp, 0),
ssbd.stmtType,
ssbd.schemaName,
ssbd.digest,
ssbd.normalizedSQL,
convertEmptyToNil(ssbd.tableNames),
convertEmptyToNil(strings.Join(ssElement.indexNames, ",")),
convertEmptyToNil(sampleUser),
ssElement.execCount,
ssElement.sumErrors,
ssElement.sumWarnings,
int64(ssElement.sumLatency),
int64(ssElement.maxLatency),
int64(ssElement.minLatency),
avgInt(int64(ssElement.sumLatency), ssElement.execCount),
avgInt(int64(ssElement.sumParseLatency), ssElement.execCount),
int64(ssElement.maxParseLatency),
avgInt(int64(ssElement.sumCompileLatency), ssElement.execCount),
int64(ssElement.maxCompileLatency),
ssElement.sumNumCopTasks,
int64(ssElement.maxCopProcessTime),
convertEmptyToNil(ssElement.maxCopProcessAddress),
int64(ssElement.maxCopWaitTime),
convertEmptyToNil(ssElement.maxCopWaitAddress),
avgInt(int64(ssElement.sumProcessTime), ssElement.execCount),
int64(ssElement.maxProcessTime),
avgInt(int64(ssElement.sumWaitTime), ssElement.execCount),
int64(ssElement.maxWaitTime),
avgInt(int64(ssElement.sumBackoffTime), ssElement.execCount),
int64(ssElement.maxBackoffTime),
avgInt(ssElement.sumTotalKeys, ssElement.execCount),
ssElement.maxTotalKeys,
avgInt(ssElement.sumProcessedKeys, ssElement.execCount),
ssElement.maxProcessedKeys,
avgInt(int64(ssElement.sumPrewriteTime), ssElement.commitCount),
int64(ssElement.maxPrewriteTime),
avgInt(int64(ssElement.sumCommitTime), ssElement.commitCount),
int64(ssElement.maxCommitTime),
avgInt(int64(ssElement.sumGetCommitTsTime), ssElement.commitCount),
int64(ssElement.maxGetCommitTsTime),
avgInt(ssElement.sumCommitBackoffTime, ssElement.commitCount),
ssElement.maxCommitBackoffTime,
avgInt(ssElement.sumResolveLockTime, ssElement.commitCount),
ssElement.maxResolveLockTime,
avgInt(int64(ssElement.sumLocalLatchTime), ssElement.commitCount),
int64(ssElement.maxLocalLatchTime),
avgFloat(ssElement.sumWriteKeys, ssElement.commitCount),
ssElement.maxWriteKeys,
avgFloat(ssElement.sumWriteSize, ssElement.commitCount),
ssElement.maxWriteSize,
avgFloat(ssElement.sumPrewriteRegionNum, ssElement.commitCount),
int(ssElement.maxPrewriteRegionNum),
avgFloat(ssElement.sumTxnRetry, ssElement.commitCount),
ssElement.maxTxnRetry,
int(ssElement.execRetryCount),
int64(ssElement.execRetryTime),
ssElement.sumBackoffTimes,
formatBackoffTypes(ssElement.backoffTypes),
avgInt(ssElement.sumMem, ssElement.execCount),
ssElement.maxMem,
avgInt(ssElement.sumDisk, ssElement.execCount),
ssElement.maxDisk,
avgInt(int64(ssElement.sumKVTotal), ssElement.commitCount),
avgInt(int64(ssElement.sumPDTotal), ssElement.commitCount),
avgInt(int64(ssElement.sumBackoffTotal), ssElement.commitCount),
avgInt(int64(ssElement.sumWriteSQLRespTotal), ssElement.commitCount),
ssElement.prepared,
avgFloat(int64(ssElement.sumAffectedRows), ssElement.execCount),
types.NewTime(types.FromGoTime(ssElement.firstSeen), mysql.TypeTimestamp, 0),
types.NewTime(types.FromGoTime(ssElement.lastSeen), mysql.TypeTimestamp, 0),
ssElement.planInCache,
ssElement.planCacheHits,
ssElement.sampleSQL,
ssElement.prevSQL,
ssbd.planDigest,
plan,
)
}
// Truncate SQL to maxSQLLength.
func formatSQL(sql string) string {
maxSQLLength := StmtSummaryByDigestMap.maxSQLLength()
length := len(sql)
if length > maxSQLLength {
sql = fmt.Sprintf("%.*s(len:%d)", maxSQLLength, sql, length)
}
return sql
}
// Format the backoffType map to a string or nil.
func formatBackoffTypes(backoffMap map[fmt.Stringer]int) interface{} {
type backoffStat struct {
backoffType fmt.Stringer
count int
}
size := len(backoffMap)
if size == 0 {
return nil
}
backoffArray := make([]backoffStat, 0, len(backoffMap))
for backoffType, count := range backoffMap {
backoffArray = append(backoffArray, backoffStat{backoffType, count})
}
sort.Slice(backoffArray, func(i, j int) bool {
return backoffArray[i].count > backoffArray[j].count
})
var buffer bytes.Buffer
for index, stat := range backoffArray {
if _, err := fmt.Fprintf(&buffer, "%v:%d", stat.backoffType, stat.count); err != nil {
return "FORMAT ERROR"
}
if index < len(backoffArray)-1 {
buffer.WriteString(",")
}
}
return buffer.String()
}
func avgInt(sum int64, count int64) int64 {
if count > 0 {
return sum / count
}
return 0
}
func avgFloat(sum int64, count int64) float64 {
if count > 0 {
return float64(sum) / float64(count)
}
return 0
}
func convertEmptyToNil(str string) interface{} {
if str == "" {
return nil
}
return str
}