Documentation
¶
Index ¶
- Constants
- Variables
- func ContextWithInitializedExecDetails(ctx context.Context) context.Context
- func FormatDuration(d time.Duration) string
- func MergeTiFlashRUConsumption(executionSummaries []*tipb.ExecutorExecutionSummary, ruDetails *util.RUDetails) error
- type BasicRuntimeStats
- func (*BasicRuntimeStats) Clone() RuntimeStats
- func (e *BasicRuntimeStats) GetActRows() int64
- func (e *BasicRuntimeStats) GetTime() int64
- func (e *BasicRuntimeStats) Merge(rs RuntimeStats)
- func (e *BasicRuntimeStats) Record(d time.Duration, rowNum int)
- func (e *BasicRuntimeStats) RecordClose(d time.Duration)
- func (e *BasicRuntimeStats) RecordOpen(d time.Duration)
- func (e *BasicRuntimeStats) SetRowNum(rowNum int64)
- func (e *BasicRuntimeStats) String() string
- func (*BasicRuntimeStats) Tp() int
- type ConcurrencyInfo
- type CopExecDetails
- type CopRuntimeStats
- type CopTasksDetails
- type CopTasksSummary
- type Duration
- type DurationWithAddr
- type ExecDetails
- type Int64
- type P90BackoffSummary
- type P90Summary
- type Percentile
- func (p *Percentile[valueType]) Add(value valueType)
- func (p *Percentile[valueType]) GetMax() valueType
- func (p *Percentile[valueType]) GetMin() valueType
- func (p *Percentile[valueType]) GetPercentile(f float64) float64
- func (p *Percentile[valueType]) MergePercentile(p2 *Percentile[valueType])
- func (p *Percentile[valueType]) Size() int
- func (p *Percentile[valueType]) Sum() float64
- type RURuntimeStats
- type RootRuntimeStats
- type RuntimeStats
- type RuntimeStatsColl
- func (e *RuntimeStatsColl) ExistsCopStats(planID int) bool
- func (e *RuntimeStatsColl) ExistsRootStats(planID int) bool
- func (e *RuntimeStatsColl) GetBasicRuntimeStats(planID int, initNewExecutorStats bool) *BasicRuntimeStats
- func (e *RuntimeStatsColl) GetCopCountAndRows(planID int) (int32, int64)
- func (e *RuntimeStatsColl) GetCopStats(planID int) *CopRuntimeStats
- func (e *RuntimeStatsColl) GetPlanActRows(planID int) int64
- func (e *RuntimeStatsColl) GetRootStats(planID int) *RootRuntimeStats
- func (e *RuntimeStatsColl) GetStmtCopRuntimeStats() StmtCopRuntimeStats
- func (e *RuntimeStatsColl) RecordCopStats(planID int, storeType kv.StoreType, scan *util.ScanDetail, ...) int
- func (e *RuntimeStatsColl) RecordOneCopTask(planID int, storeType kv.StoreType, summary *tipb.ExecutorExecutionSummary) int
- func (e *RuntimeStatsColl) RegisterStats(planID int, info RuntimeStats)
- type RuntimeStatsWithCommit
- type RuntimeStatsWithConcurrencyInfo
- func (e *RuntimeStatsWithConcurrencyInfo) Clone() RuntimeStats
- func (*RuntimeStatsWithConcurrencyInfo) Merge(RuntimeStats)
- func (e *RuntimeStatsWithConcurrencyInfo) SetConcurrencyInfo(infos ...*ConcurrencyInfo)
- func (e *RuntimeStatsWithConcurrencyInfo) String() string
- func (*RuntimeStatsWithConcurrencyInfo) Tp() int
- type StmtCopRuntimeStats
- type StmtExecDetails
- type SyncExecDetails
- func (s *SyncExecDetails) CopTasksDetails() *CopTasksDetails
- func (s *SyncExecDetails) CopTasksSummary() *CopTasksSummary
- func (s *SyncExecDetails) GetExecDetails() ExecDetails
- func (s *SyncExecDetails) MergeCopExecDetails(details *CopExecDetails, copTime time.Duration)
- func (s *SyncExecDetails) MergeExecDetails(commitDetails *util.CommitDetails)
- func (s *SyncExecDetails) MergeLockKeysExecDetails(lockKeys *util.LockKeysDetails)
- func (s *SyncExecDetails) Reset()
- type TaskTimeStats
- type TiFlashNetworkTrafficSummary
- func (networkTraffic *TiFlashNetworkTrafficSummary) Clone() TiFlashNetworkTrafficSummary
- func (networkTraffic *TiFlashNetworkTrafficSummary) Empty() bool
- func (networkTraffic *TiFlashNetworkTrafficSummary) GetInterZoneTrafficBytes() uint64
- func (networkTraffic *TiFlashNetworkTrafficSummary) Merge(other TiFlashNetworkTrafficSummary)
- func (networkTraffic *TiFlashNetworkTrafficSummary) String() string
- func (networkTraffic *TiFlashNetworkTrafficSummary) UpdateTiKVExecDetails(tikvDetails *util.ExecDetails)
- type TiFlashScanContext
- type TiFlashWaitSummary
- type TiflashStats
Constants ¶
const ( // CopTimeStr represents the sum of cop-task time spend in TiDB distSQL. CopTimeStr = "Cop_time" // WaitTimeStr means the time of all coprocessor wait. WaitTimeStr = "Wait_time" // LockKeysTimeStr means the time interval between pessimistic lock wait start and lock got obtain LockKeysTimeStr = "LockKeys_time" // RequestCountStr means the request count. RequestCountStr = "Request_count" // WaitPrewriteBinlogTimeStr means the time of waiting prewrite binlog finished when transaction committing. WaitPrewriteBinlogTimeStr = "Wait_prewrite_binlog_time" // GetCommitTSTimeStr means the time of getting commit ts. GetCommitTSTimeStr = "Get_commit_ts_time" // GetLatestTsTimeStr means the time of getting latest ts in async commit and 1pc. GetLatestTsTimeStr = "Get_latest_ts_time" // CommitBackoffTimeStr means the time of commit backoff. CommitBackoffTimeStr = "Commit_backoff_time" // BackoffTypesStr means the backoff type. BackoffTypesStr = "Backoff_types" // SlowestPrewriteRPCDetailStr means the details of the slowest RPC during the transaction 2pc prewrite process. SlowestPrewriteRPCDetailStr = "Slowest_prewrite_rpc_detail" // CommitPrimaryRPCDetailStr means the details of the slowest RPC during the transaction 2pc commit process. CommitPrimaryRPCDetailStr = "Commit_primary_rpc_detail" // ResolveLockTimeStr means the time of resolving lock. ResolveLockTimeStr = "Resolve_lock_time" // LocalLatchWaitTimeStr means the time of waiting in local latch. LocalLatchWaitTimeStr = "Local_latch_wait_time" // TxnRetryStr means the count of transaction retry. TxnRetryStr = "Txn_retry" // GetSnapshotTimeStr means the time spent on getting an engine snapshot. GetSnapshotTimeStr = "Get_snapshot_time" // RocksdbDeleteSkippedCountStr means the count of rocksdb delete skipped count. RocksdbDeleteSkippedCountStr = "Rocksdb_delete_skipped_count" // RocksdbKeySkippedCountStr means the count of rocksdb key skipped count. RocksdbKeySkippedCountStr = "Rocksdb_key_skipped_count" // RocksdbBlockCacheHitCountStr means the count of rocksdb block cache hit. RocksdbBlockCacheHitCountStr = "Rocksdb_block_cache_hit_count" // RocksdbBlockReadCountStr means the count of rocksdb block read. RocksdbBlockReadCountStr = "Rocksdb_block_read_count" // RocksdbBlockReadByteStr means the bytes of rocksdb block read. RocksdbBlockReadByteStr = "Rocksdb_block_read_byte" // RocksdbBlockReadTimeStr means the time spent on rocksdb block read. RocksdbBlockReadTimeStr = "Rocksdb_block_read_time" // ProcessTimeStr represents the sum of process time of all the coprocessor tasks. ProcessTimeStr = "Process_time" // BackoffTimeStr means the time of all back-off. BackoffTimeStr = "Backoff_time" // TotalKeysStr means the total scan keys. TotalKeysStr = "Total_keys" // ProcessKeysStr means the total processed keys. ProcessKeysStr = "Process_keys" // PreWriteTimeStr means the time of pre-write. PreWriteTimeStr = "Prewrite_time" // CommitTimeStr means the time of commit. CommitTimeStr = "Commit_time" // WriteKeysStr means the count of keys in the transaction. WriteKeysStr = "Write_keys" // WriteSizeStr means the key/value size in the transaction. WriteSizeStr = "Write_size" // PrewriteRegionStr means the count of region when pre-write. PrewriteRegionStr = "Prewrite_region" )
const ( // TpBasicRuntimeStats is the tp for BasicRuntimeStats. TpBasicRuntimeStats int = iota // TpRuntimeStatsWithCommit is the tp for RuntimeStatsWithCommit. TpRuntimeStatsWithCommit // TpRuntimeStatsWithConcurrencyInfo is the tp for RuntimeStatsWithConcurrencyInfo. TpRuntimeStatsWithConcurrencyInfo // TpSnapshotRuntimeStats is the tp for SnapshotRuntimeStats. TpSnapshotRuntimeStats // TpHashJoinRuntimeStats is the tp for HashJoinRuntimeStats. TpHashJoinRuntimeStats // TpHashJoinRuntimeStatsV2 is the tp for hashJoinRuntimeStatsV2. TpHashJoinRuntimeStatsV2 // TpIndexLookUpJoinRuntimeStats is the tp for IndexLookUpJoinRuntimeStats. TpIndexLookUpJoinRuntimeStats // TpRuntimeStatsWithSnapshot is the tp for RuntimeStatsWithSnapshot. TpRuntimeStatsWithSnapshot // TpJoinRuntimeStats is the tp for JoinRuntimeStats. TpJoinRuntimeStats // TpSelectResultRuntimeStats is the tp for SelectResultRuntimeStats. TpSelectResultRuntimeStats // TpInsertRuntimeStat is the tp for InsertRuntimeStat TpInsertRuntimeStat // TpIndexLookUpRunTimeStats is the tp for IndexLookUpRunTimeStats TpIndexLookUpRunTimeStats // TpSlowQueryRuntimeStat is the tp for SlowQueryRuntimeStat TpSlowQueryRuntimeStat // TpHashAggRuntimeStat is the tp for HashAggRuntimeStat TpHashAggRuntimeStat // TpIndexMergeRunTimeStats is the tp for IndexMergeRunTimeStats TpIndexMergeRunTimeStats // TpBasicCopRunTimeStats is the tp for BasicCopRunTimeStats TpBasicCopRunTimeStats // TpUpdateRuntimeStats is the tp for UpdateRuntimeStats TpUpdateRuntimeStats // TpFKCheckRuntimeStats is the tp for FKCheckRuntimeStats TpFKCheckRuntimeStats // TpFKCascadeRuntimeStats is the tp for FKCascadeRuntimeStats TpFKCascadeRuntimeStats // TpRURuntimeStats is the tp for RURuntimeStats TpRURuntimeStats )
const MaxDetailsNumsForOneQuery = 1000
MaxDetailsNumsForOneQuery is the max number of details to keep for P90 for one query.
Variables ¶
var StmtExecDetailKey = stmtExecDetailKeyType{}
StmtExecDetailKey used to carry StmtExecDetail info in context.Context.
Functions ¶
func ContextWithInitializedExecDetails ¶
ContextWithInitializedExecDetails returns a context with initialized stmt execution, execution and resource usage details.
func FormatDuration ¶
FormatDuration uses to format duration, this function will prune precision before format duration. Pruning precision is for human readability. The prune rule is:
- if the duration was less than 1us, return the original string.
- readable value >=10, keep 1 decimal, otherwise, keep 2 decimal. such as: 9.412345ms -> 9.41ms 10.412345ms -> 10.4ms 5.999s -> 6s 100.45µs -> 100.5µs
func MergeTiFlashRUConsumption ¶
func MergeTiFlashRUConsumption(executionSummaries []*tipb.ExecutorExecutionSummary, ruDetails *util.RUDetails) error
MergeTiFlashRUConsumption merge execution summaries from selectResponse into ruDetails.
Types ¶
type BasicRuntimeStats ¶
type BasicRuntimeStats struct {
// contains filtered or unexported fields
}
BasicRuntimeStats is the basic runtime stats.
func (*BasicRuntimeStats) Clone ¶
func (*BasicRuntimeStats) Clone() RuntimeStats
Clone implements the RuntimeStats interface. BasicRuntimeStats shouldn't implement Clone interface because all executors with the same executor_id should share the same BasicRuntimeStats, duplicated BasicRuntimeStats are easy to cause mistakes.
func (*BasicRuntimeStats) GetActRows ¶
func (e *BasicRuntimeStats) GetActRows() int64
GetActRows return total rows of BasicRuntimeStats.
func (*BasicRuntimeStats) GetTime ¶
func (e *BasicRuntimeStats) GetTime() int64
GetTime get the int64 total time
func (*BasicRuntimeStats) Merge ¶
func (e *BasicRuntimeStats) Merge(rs RuntimeStats)
Merge implements the RuntimeStats interface.
func (*BasicRuntimeStats) Record ¶
func (e *BasicRuntimeStats) Record(d time.Duration, rowNum int)
Record records executor's execution.
func (*BasicRuntimeStats) RecordClose ¶
func (e *BasicRuntimeStats) RecordClose(d time.Duration)
RecordClose records executor's close time.
func (*BasicRuntimeStats) RecordOpen ¶
func (e *BasicRuntimeStats) RecordOpen(d time.Duration)
RecordOpen records executor's open time.
func (*BasicRuntimeStats) SetRowNum ¶
func (e *BasicRuntimeStats) SetRowNum(rowNum int64)
SetRowNum sets the row num.
func (*BasicRuntimeStats) String ¶
func (e *BasicRuntimeStats) String() string
String implements the RuntimeStats interface.
func (*BasicRuntimeStats) Tp ¶
func (*BasicRuntimeStats) Tp() int
Tp implements the RuntimeStats interface.
type ConcurrencyInfo ¶
type ConcurrencyInfo struct {
// contains filtered or unexported fields
}
ConcurrencyInfo is used to save the concurrency information of the executor operator
func NewConcurrencyInfo ¶
func NewConcurrencyInfo(name string, num int) *ConcurrencyInfo
NewConcurrencyInfo creates new executor's concurrencyInfo.
type CopExecDetails ¶
type CopExecDetails struct {
ScanDetail *util.ScanDetail
TimeDetail util.TimeDetail
CalleeAddress string
BackoffTime time.Duration
BackoffSleep map[string]time.Duration
BackoffTimes map[string]int
}
CopExecDetails contains cop execution detail information.
type CopRuntimeStats ¶
type CopRuntimeStats struct {
// contains filtered or unexported fields
}
CopRuntimeStats collects cop tasks' execution info.
func (*CopRuntimeStats) GetActRows ¶
func (crs *CopRuntimeStats) GetActRows() int64
GetActRows return total rows of CopRuntimeStats.
func (*CopRuntimeStats) GetTasks ¶
func (crs *CopRuntimeStats) GetTasks() int32
GetTasks return total tasks of CopRuntimeStats
func (*CopRuntimeStats) String ¶
func (crs *CopRuntimeStats) String() string
type CopTasksDetails ¶
type CopTasksDetails struct {
NumCopTasks int
ProcessTimeStats TaskTimeStats
WaitTimeStats TaskTimeStats
BackoffTimeStatsMap map[string]TaskTimeStats
TotBackoffTimes map[string]int
}
CopTasksDetails collects some useful information of cop-tasks during execution.
func (*CopTasksDetails) ToZapFields ¶
func (d *CopTasksDetails) ToZapFields() (fields []zap.Field)
ToZapFields wraps the CopTasksDetails as zap.Fileds.
type CopTasksSummary ¶
type CopTasksSummary struct {
NumCopTasks int
MaxProcessAddress string
MaxProcessTime time.Duration
TotProcessTime time.Duration
MaxWaitAddress string
MaxWaitTime time.Duration
TotWaitTime time.Duration
}
CopTasksSummary collects some summary information of cop-tasks for statement summary.
type Duration ¶
Duration is a wrapper of time.Duration to implement the canGetFloat64 interface.
func (Duration) GetFloat64 ¶
GetFloat64 implements the canGetFloat64 interface.
type DurationWithAddr ¶
DurationWithAddr is a wrapper of time.Duration and string to implement the canGetFloat64 interface.
func (DurationWithAddr) GetFloat64 ¶
func (d DurationWithAddr) GetFloat64() float64
GetFloat64 implements the canGetFloat64 interface.
type ExecDetails ¶
type ExecDetails struct {
CopExecDetails
CommitDetail *util.CommitDetails
LockKeysDetail *util.LockKeysDetails
CopTime time.Duration
RequestCount int
}
ExecDetails contains execution detail information.
func (ExecDetails) String ¶
func (d ExecDetails) String() string
String implements the fmt.Stringer interface.
func (ExecDetails) ToZapFields ¶
func (d ExecDetails) ToZapFields() (fields []zap.Field)
ToZapFields wraps the ExecDetails as zap.Fields.
type Int64 ¶
type Int64 int64
Int64 is a wrapper of int64 to implement the canGetFloat64 interface.
func (Int64) GetFloat64 ¶
GetFloat64 implements the canGetFloat64 interface.
type P90BackoffSummary ¶
type P90BackoffSummary struct {
ReqTimes int
BackoffPercentile Percentile[DurationWithAddr]
TotBackoffTime time.Duration
TotBackoffTimes int
}
P90BackoffSummary contains execution summary for a backoff type.
type P90Summary ¶
type P90Summary struct {
NumCopTasks int
ProcessTimePercentile Percentile[DurationWithAddr]
WaitTimePercentile Percentile[DurationWithAddr]
BackoffInfo map[string]*P90BackoffSummary
}
P90Summary contains execution summary for cop tasks.
func (*P90Summary) Merge ¶
func (d *P90Summary) Merge(backoffSleep map[string]time.Duration, backoffTimes map[string]int, calleeAddress string, timeDetail util.TimeDetail)
Merge merges DetailsNeedP90 into P90Summary.
func (*P90Summary) Reset ¶
func (d *P90Summary) Reset()
Reset resets all fields in DetailsNeedP90Summary.
type Percentile ¶
type Percentile[valueType canGetFloat64] struct {
// contains filtered or unexported fields
}
Percentile is a struct to calculate the percentile of a series of values.
func (*Percentile[valueType]) Add ¶
func (p *Percentile[valueType]) Add(value valueType)
Add adds a value to calculate the percentile.
func (*Percentile[valueType]) GetMax ¶
func (p *Percentile[valueType]) GetMax() valueType
GetMax returns the max value.
func (*Percentile[valueType]) GetMin ¶
func (p *Percentile[valueType]) GetMin() valueType
GetMin returns the min value.
func (*Percentile[valueType]) GetPercentile ¶
func (p *Percentile[valueType]) GetPercentile(f float64) float64
GetPercentile returns the percentile `f` of the values.
func (*Percentile[valueType]) MergePercentile ¶
func (p *Percentile[valueType]) MergePercentile(p2 *Percentile[valueType])
MergePercentile merges two Percentile.
func (*Percentile[valueType]) Size ¶
func (p *Percentile[valueType]) Size() int
Size returns the size of the values.
func (*Percentile[valueType]) Sum ¶
func (p *Percentile[valueType]) Sum() float64
Sum returns the sum of the values.
type RURuntimeStats ¶
RURuntimeStats is a wrapper of util.RUDetails, which implements the RuntimeStats interface.
func (*RURuntimeStats) Clone ¶
func (e *RURuntimeStats) Clone() RuntimeStats
Clone implements the RuntimeStats interface.
func (*RURuntimeStats) Merge ¶
func (e *RURuntimeStats) Merge(other RuntimeStats)
Merge implements the RuntimeStats interface.
func (*RURuntimeStats) String ¶
func (e *RURuntimeStats) String() string
String implements the RuntimeStats interface.
func (*RURuntimeStats) Tp ¶
func (*RURuntimeStats) Tp() int
Tp implements the RuntimeStats interface.
type RootRuntimeStats ¶
type RootRuntimeStats struct {
// contains filtered or unexported fields
}
RootRuntimeStats is the executor runtime stats that combine with multiple runtime stats.
func NewRootRuntimeStats ¶
func NewRootRuntimeStats() *RootRuntimeStats
NewRootRuntimeStats returns a new RootRuntimeStats
func (*RootRuntimeStats) GetActRows ¶
func (e *RootRuntimeStats) GetActRows() int64
GetActRows return total rows of RootRuntimeStats.
func (*RootRuntimeStats) MergeStats ¶
func (e *RootRuntimeStats) MergeStats() (basic *BasicRuntimeStats, groups []RuntimeStats)
MergeStats merges stats in the RootRuntimeStats and return the stats suitable for display directly.
func (*RootRuntimeStats) String ¶
func (e *RootRuntimeStats) String() string
String implements the RuntimeStats interface.
type RuntimeStats ¶
type RuntimeStats interface {
String() string
Merge(RuntimeStats)
Clone() RuntimeStats
Tp() int
}
RuntimeStats is used to express the executor runtime information.
type RuntimeStatsColl ¶
type RuntimeStatsColl struct {
// contains filtered or unexported fields
}
RuntimeStatsColl collects executors's execution info.
func NewRuntimeStatsColl ¶
func NewRuntimeStatsColl(reuse *RuntimeStatsColl) *RuntimeStatsColl
NewRuntimeStatsColl creates new executor collector. Reuse the object to reduce allocation when *RuntimeStatsColl is not nil.
func (*RuntimeStatsColl) ExistsCopStats ¶
func (e *RuntimeStatsColl) ExistsCopStats(planID int) bool
ExistsCopStats checks if the planID exists in the copStats collection.
func (*RuntimeStatsColl) ExistsRootStats ¶
func (e *RuntimeStatsColl) ExistsRootStats(planID int) bool
ExistsRootStats checks if the planID exists in the rootStats collection.
func (*RuntimeStatsColl) GetBasicRuntimeStats ¶
func (e *RuntimeStatsColl) GetBasicRuntimeStats(planID int, initNewExecutorStats bool) *BasicRuntimeStats
GetBasicRuntimeStats gets basicRuntimeStats for a executor When rootStat/rootStat's basicRuntimeStats is nil, the behavior is decided by initNewExecutorStats argument: 1. If true, it created a new one, and increase basicRuntimeStats' executorCount 2. Else, it returns nil
func (*RuntimeStatsColl) GetCopCountAndRows ¶
func (e *RuntimeStatsColl) GetCopCountAndRows(planID int) (int32, int64)
GetCopCountAndRows returns the total cop-tasks count and total rows of all cop-tasks.
func (*RuntimeStatsColl) GetCopStats ¶
func (e *RuntimeStatsColl) GetCopStats(planID int) *CopRuntimeStats
GetCopStats gets the CopRuntimeStats specified by planID.
func (*RuntimeStatsColl) GetPlanActRows ¶
func (e *RuntimeStatsColl) GetPlanActRows(planID int) int64
GetPlanActRows returns the actual rows of the plan.
func (*RuntimeStatsColl) GetRootStats ¶
func (e *RuntimeStatsColl) GetRootStats(planID int) *RootRuntimeStats
GetRootStats gets execStat for a executor.
func (*RuntimeStatsColl) GetStmtCopRuntimeStats ¶
func (e *RuntimeStatsColl) GetStmtCopRuntimeStats() StmtCopRuntimeStats
GetStmtCopRuntimeStats gets execStat for a executor.
func (*RuntimeStatsColl) RecordCopStats ¶
func (e *RuntimeStatsColl) RecordCopStats(planID int, storeType kv.StoreType, scan *util.ScanDetail, time util.TimeDetail, summary *tipb.ExecutorExecutionSummary) int
RecordCopStats records a specific cop tasks's execution detail.
func (*RuntimeStatsColl) RecordOneCopTask ¶
func (e *RuntimeStatsColl) RecordOneCopTask(planID int, storeType kv.StoreType, summary *tipb.ExecutorExecutionSummary) int
RecordOneCopTask records a specific cop tasks's execution summary.
func (*RuntimeStatsColl) RegisterStats ¶
func (e *RuntimeStatsColl) RegisterStats(planID int, info RuntimeStats)
RegisterStats register execStat for a executor.
type RuntimeStatsWithCommit ¶
type RuntimeStatsWithCommit struct {
Commit *util.CommitDetails
LockKeys *util.LockKeysDetails
TxnCnt int
}
RuntimeStatsWithCommit is the RuntimeStats with commit detail.
func (*RuntimeStatsWithCommit) Clone ¶
func (e *RuntimeStatsWithCommit) Clone() RuntimeStats
Clone implements the RuntimeStats interface.
func (*RuntimeStatsWithCommit) Merge ¶
func (e *RuntimeStatsWithCommit) Merge(rs RuntimeStats)
Merge implements the RuntimeStats interface.
func (*RuntimeStatsWithCommit) MergeCommitDetails ¶
func (e *RuntimeStatsWithCommit) MergeCommitDetails(detail *util.CommitDetails)
MergeCommitDetails merges the commit details.
func (*RuntimeStatsWithCommit) String ¶
func (e *RuntimeStatsWithCommit) String() string
String implements the RuntimeStats interface.
func (*RuntimeStatsWithCommit) Tp ¶
func (*RuntimeStatsWithCommit) Tp() int
Tp implements the RuntimeStats interface.
type RuntimeStatsWithConcurrencyInfo ¶
type RuntimeStatsWithConcurrencyInfo struct {
// protect concurrency
sync.Mutex
// contains filtered or unexported fields
}
RuntimeStatsWithConcurrencyInfo is the BasicRuntimeStats with ConcurrencyInfo.
func (*RuntimeStatsWithConcurrencyInfo) Clone ¶
func (e *RuntimeStatsWithConcurrencyInfo) Clone() RuntimeStats
Clone implements the RuntimeStats interface.
func (*RuntimeStatsWithConcurrencyInfo) Merge ¶
func (*RuntimeStatsWithConcurrencyInfo) Merge(RuntimeStats)
Merge implements the RuntimeStats interface.
func (*RuntimeStatsWithConcurrencyInfo) SetConcurrencyInfo ¶
func (e *RuntimeStatsWithConcurrencyInfo) SetConcurrencyInfo(infos ...*ConcurrencyInfo)
SetConcurrencyInfo sets the concurrency informations. We must clear the concurrencyInfo first when we call the SetConcurrencyInfo. When the num <= 0, it means the exector operator is not executed parallel.
func (*RuntimeStatsWithConcurrencyInfo) String ¶
func (e *RuntimeStatsWithConcurrencyInfo) String() string
String implements the RuntimeStats interface.
func (*RuntimeStatsWithConcurrencyInfo) Tp ¶
func (*RuntimeStatsWithConcurrencyInfo) Tp() int
Tp implements the RuntimeStats interface.
type StmtCopRuntimeStats ¶
type StmtCopRuntimeStats struct {
// TiflashNetworkStats stats all mpp tasks' network traffic info, nil if no any mpp tasks' network traffic
TiflashNetworkStats *TiFlashNetworkTrafficSummary
}
StmtCopRuntimeStats stores the cop runtime stats of the total statement
type StmtExecDetails ¶
StmtExecDetails contains stmt level execution detail info.
func GetExecDetailsFromContext ¶
func GetExecDetailsFromContext(ctx context.Context) (stmtDetail StmtExecDetails, tikvExecDetail util.ExecDetails, ruDetails *util.RUDetails)
GetExecDetailsFromContext gets stmt execution, execution and resource usage details from context.
type SyncExecDetails ¶
type SyncExecDetails struct {
// contains filtered or unexported fields
}
SyncExecDetails is a synced version of `ExecDetails` and its `P90Summary`
func (*SyncExecDetails) CopTasksDetails ¶
func (s *SyncExecDetails) CopTasksDetails() *CopTasksDetails
CopTasksDetails returns some useful information of cop-tasks during execution.
func (*SyncExecDetails) CopTasksSummary ¶
func (s *SyncExecDetails) CopTasksSummary() *CopTasksSummary
CopTasksSummary returns some summary information of cop-tasks for statement summary.
func (*SyncExecDetails) GetExecDetails ¶
func (s *SyncExecDetails) GetExecDetails() ExecDetails
GetExecDetails returns the exec details inside. It's actually not safe, because the `ExecDetails` still contains some reference, which is not protected after returning outside.
func (*SyncExecDetails) MergeCopExecDetails ¶
func (s *SyncExecDetails) MergeCopExecDetails(details *CopExecDetails, copTime time.Duration)
MergeCopExecDetails merges a CopExecDetails into self.
func (*SyncExecDetails) MergeExecDetails ¶
func (s *SyncExecDetails) MergeExecDetails(commitDetails *util.CommitDetails)
MergeExecDetails merges a single region execution details into self, used to print the information in slow query log.
func (*SyncExecDetails) MergeLockKeysExecDetails ¶
func (s *SyncExecDetails) MergeLockKeysExecDetails(lockKeys *util.LockKeysDetails)
MergeLockKeysExecDetails merges lock keys execution details into self.
type TaskTimeStats ¶
type TaskTimeStats struct {
AvgTime time.Duration
P90Time time.Duration
MaxAddress string
MaxTime time.Duration
TotTime time.Duration
}
TaskTimeStats is used for recording time-related statistical metrics, including dimensions such as average values, percentile values, maximum values, etc. It is suitable for scenarios involving latency statistics, wait time analysis, and similar use cases.
func (TaskTimeStats) FormatFloatFields ¶
func (s TaskTimeStats) FormatFloatFields() (avgStr, p90Str, maxStr string)
FormatFloatFields returns the AvgTime, P90Time and MaxTime in float format.
type TiFlashNetworkTrafficSummary ¶
type TiFlashNetworkTrafficSummary struct {
// contains filtered or unexported fields
}
TiFlashNetworkTrafficSummary is used to express network traffic in tiflash
func (*TiFlashNetworkTrafficSummary) Clone ¶
func (networkTraffic *TiFlashNetworkTrafficSummary) Clone() TiFlashNetworkTrafficSummary
Clone implements the deep copy of * TiFlashNetworkTrafficSummary
func (*TiFlashNetworkTrafficSummary) Empty ¶
func (networkTraffic *TiFlashNetworkTrafficSummary) Empty() bool
Empty check whether TiFlashNetworkTrafficSummary is Empty, if no any network traffic, we regard it as empty
func (*TiFlashNetworkTrafficSummary) GetInterZoneTrafficBytes ¶
func (networkTraffic *TiFlashNetworkTrafficSummary) GetInterZoneTrafficBytes() uint64
GetInterZoneTrafficBytes returns the inter zone network traffic bytes involved between tiflash instances.
func (*TiFlashNetworkTrafficSummary) Merge ¶
func (networkTraffic *TiFlashNetworkTrafficSummary) Merge(other TiFlashNetworkTrafficSummary)
Merge make sum to merge the information in TiFlashNetworkTrafficSummary
func (*TiFlashNetworkTrafficSummary) String ¶
func (networkTraffic *TiFlashNetworkTrafficSummary) String() string
String dumps TiFlashNetworkTrafficSummary info as string
func (*TiFlashNetworkTrafficSummary) UpdateTiKVExecDetails ¶
func (networkTraffic *TiFlashNetworkTrafficSummary) UpdateTiKVExecDetails(tikvDetails *util.ExecDetails)
UpdateTiKVExecDetails update tikvDetails with TiFlashNetworkTrafficSummary's values
type TiFlashScanContext ¶
type TiFlashScanContext struct {
// contains filtered or unexported fields
}
TiFlashScanContext is used to express the table scan information in tiflash
func (*TiFlashScanContext) Clone ¶
func (context *TiFlashScanContext) Clone() TiFlashScanContext
Clone implements the deep copy of * TiFlashshScanContext
func (*TiFlashScanContext) Empty ¶
func (context *TiFlashScanContext) Empty() bool
Empty check whether TiFlashScanContext is Empty, if scan no pack and skip no pack, we regard it as empty
func (*TiFlashScanContext) Merge ¶
func (context *TiFlashScanContext) Merge(other TiFlashScanContext)
Merge make sum to merge the information in TiFlashScanContext
func (*TiFlashScanContext) String ¶
func (context *TiFlashScanContext) String() string
type TiFlashWaitSummary ¶
type TiFlashWaitSummary struct {
// contains filtered or unexported fields
}
TiFlashWaitSummary is used to express all kinds of wait information in tiflash
func (*TiFlashWaitSummary) CanBeIgnored ¶
func (waitSummary *TiFlashWaitSummary) CanBeIgnored() bool
CanBeIgnored check whether TiFlashWaitSummary can be ignored, not all tidb executors have significant tiflash wait summary
func (*TiFlashWaitSummary) Clone ¶
func (waitSummary *TiFlashWaitSummary) Clone() TiFlashWaitSummary
Clone implements the deep copy of * TiFlashWaitSummary
func (*TiFlashWaitSummary) Merge ¶
func (waitSummary *TiFlashWaitSummary) Merge(other TiFlashWaitSummary)
Merge make sum to merge the information in TiFlashWaitSummary
func (*TiFlashWaitSummary) String ¶
func (waitSummary *TiFlashWaitSummary) String() string
String dumps TiFlashWaitSummary info as string
type TiflashStats ¶
type TiflashStats struct {
// contains filtered or unexported fields
}
TiflashStats contains tiflash execution stats.