Skip to content

Commit

Permalink
save some unnecessary WithLabelValues
Browse files Browse the repository at this point in the history
Signed-off-by: ekexium <eke@fastmail.com>
  • Loading branch information
ekexium committed Aug 15, 2022
1 parent 6fb3e06 commit 402b5c4
Show file tree
Hide file tree
Showing 5 changed files with 125 additions and 78 deletions.
102 changes: 67 additions & 35 deletions executor/adapter.go
Expand Up @@ -988,46 +988,78 @@ var (
execCommitWaitLatch = metrics.ExecPhaseDuration.WithLabelValues(phaseCommitWaitLatch, "0")
execCommitWaitBinlog = metrics.ExecPhaseDuration.WithLabelValues(phaseCommitWaitBinlog, "0")
execWriteResponse = metrics.ExecPhaseDuration.WithLabelValues(phaseWriteResponse, "0")
execUnknown = metrics.ExecPhaseDuration.WithLabelValues("unknown", "0")

// pre-define observers for internal queries
execBuildLockingInternal = metrics.ExecPhaseDuration.WithLabelValues(phaseBuildLocking, "1")
execOpenLockingInternal = metrics.ExecPhaseDuration.WithLabelValues(phaseOpenLocking, "1")
execNextLockingInternal = metrics.ExecPhaseDuration.WithLabelValues(phaseNextLocking, "1")
execLockLockingInternal = metrics.ExecPhaseDuration.WithLabelValues(phaseLockLocking, "1")
execBuildFinalInternal = metrics.ExecPhaseDuration.WithLabelValues(phaseBuildFinal, "1")
execOpenFinalInternal = metrics.ExecPhaseDuration.WithLabelValues(phaseOpenFinal, "1")
execNextFinalInternal = metrics.ExecPhaseDuration.WithLabelValues(phaseNextFinal, "1")
execLockFinalInternal = metrics.ExecPhaseDuration.WithLabelValues(phaseLockFinal, "1")
execCommitPrewriteInternal = metrics.ExecPhaseDuration.WithLabelValues(phaseCommitPrewrite, "1")
execCommitCommitInternal = metrics.ExecPhaseDuration.WithLabelValues(phaseCommitCommit, "1")
execCommitWaitCommitTSInternal = metrics.ExecPhaseDuration.WithLabelValues(phaseCommitWaitCommitTS, "1")
execCommitWaitLatestTSInternal = metrics.ExecPhaseDuration.WithLabelValues(phaseCommitWaitLatestTS, "1")
execCommitWaitLatchInternal = metrics.ExecPhaseDuration.WithLabelValues(phaseCommitWaitLatch, "1")
execCommitWaitBinlogInternal = metrics.ExecPhaseDuration.WithLabelValues(phaseCommitWaitBinlog, "1")
execWriteResponseInternal = metrics.ExecPhaseDuration.WithLabelValues(phaseWriteResponse, "1")
execUnknownInternal = metrics.ExecPhaseDuration.WithLabelValues("unknown", "1")
)

var phaseDurationObserverMap map[string]prometheus.Observer
var phaseDurationObserverMapInternal map[string]prometheus.Observer

func init() {
phaseDurationObserverMap = map[string]prometheus.Observer{
phaseBuildLocking: execBuildLocking,
phaseOpenLocking: execOpenLocking,
phaseNextLocking: execNextLocking,
phaseLockLocking: execLockLocking,
phaseBuildFinal: execBuildFinal,
phaseOpenFinal: execOpenFinal,
phaseNextFinal: execNextFinal,
phaseLockFinal: execLockFinal,
phaseCommitPrewrite: execCommitPrewrite,
phaseCommitCommit: execCommitCommit,
phaseCommitWaitCommitTS: execCommitWaitCommitTS,
phaseCommitWaitLatestTS: execCommitWaitLatestTS,
phaseCommitWaitLatch: execCommitWaitLatch,
phaseCommitWaitBinlog: execCommitWaitBinlog,
phaseWriteResponse: execWriteResponse,
}
phaseDurationObserverMapInternal = map[string]prometheus.Observer{
phaseBuildLocking: execBuildLockingInternal,
phaseOpenLocking: execOpenLockingInternal,
phaseNextLocking: execNextLockingInternal,
phaseLockLocking: execLockLockingInternal,
phaseBuildFinal: execBuildFinalInternal,
phaseOpenFinal: execOpenFinalInternal,
phaseNextFinal: execNextFinalInternal,
phaseLockFinal: execLockFinalInternal,
phaseCommitPrewrite: execCommitPrewriteInternal,
phaseCommitCommit: execCommitCommitInternal,
phaseCommitWaitCommitTS: execCommitWaitCommitTSInternal,
phaseCommitWaitLatestTS: execCommitWaitLatestTSInternal,
phaseCommitWaitLatch: execCommitWaitLatchInternal,
phaseCommitWaitBinlog: execCommitWaitBinlogInternal,
phaseWriteResponse: execWriteResponseInternal,
}
}

func getPhaseDurationObserver(phase string, internal bool) prometheus.Observer {
if internal {
return metrics.ExecPhaseDuration.WithLabelValues(phase, "1")
}
switch phase {
case phaseBuildLocking:
return execBuildLocking
case phaseOpenLocking:
return execOpenLocking
case phaseNextLocking:
return execNextLocking
case phaseLockLocking:
return execLockLocking
case phaseBuildFinal:
return execBuildFinal
case phaseOpenFinal:
return execOpenFinal
case phaseNextFinal:
return execNextFinal
case phaseLockFinal:
return execLockFinal
case phaseCommitPrewrite:
return execCommitPrewrite
case phaseCommitCommit:
return execCommitCommit
case phaseCommitWaitCommitTS:
return execCommitWaitCommitTS
case phaseCommitWaitLatestTS:
return execCommitWaitLatestTS
case phaseCommitWaitLatch:
return execCommitWaitLatch
case phaseCommitWaitBinlog:
return execCommitWaitBinlog
case phaseWriteResponse:
return execWriteResponse
default:
return metrics.ExecPhaseDuration.WithLabelValues(phase, "0")
if ob, found := phaseDurationObserverMapInternal[phase]; found {
return ob
}
return execUnknownInternal
}
if ob, found := phaseDurationObserverMap[phase]; found {
return ob
}
return execUnknown
}

func (a *ExecStmt) observePhaseDurations(internal bool, commitDetails *util.CommitDetails) {
Expand Down
13 changes: 2 additions & 11 deletions planner/core/plan_cache.go
Expand Up @@ -23,7 +23,6 @@ import (
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/parser/ast"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/privilege"
Expand Down Expand Up @@ -198,11 +197,7 @@ func getPointQueryPlan(stmt *ast.Prepared, sessVars *variable.SessionVars, stmtC
logutil.BgLogger().Debug("rebuild range failed", zap.Error(err))
return nil, nil, false, nil
}
if metrics.ResettablePlanCacheCounterFortTest {
metrics.PlanCacheCounter.WithLabelValues("prepare").Inc()
} else {
planCacheCounter.Inc()
}
planCacheCounter.Inc()
sessVars.FoundInPlanCache = true
stmtCtx.PointExec = true
return plan, names, true, nil
Expand Down Expand Up @@ -245,11 +240,7 @@ func getGeneralPlan(sctx sessionctx.Context, cacheKey kvcache.Key, bindSQL strin
// So we need to record this.
sessVars.FoundInBinding = true
}
if metrics.ResettablePlanCacheCounterFortTest {
metrics.PlanCacheCounter.WithLabelValues("prepare").Inc()
} else {
planCacheCounter.Inc()
}
planCacheCounter.Inc()
stmtCtx.SetPlanDigest(stmt.NormalizedPlan, stmt.PlanDigest)
return cachedVal.Plan, cachedVal.OutPutNames, true, nil
}
Expand Down
10 changes: 6 additions & 4 deletions session/session.go
Expand Up @@ -122,7 +122,9 @@ var (
sessionExecuteParseDurationInternal = metrics.SessionExecuteParseDuration.WithLabelValues(metrics.LblInternal)
sessionExecuteParseDurationGeneral = metrics.SessionExecuteParseDuration.WithLabelValues(metrics.LblGeneral)

telemetryCTEUsage = metrics.TelemetrySQLCTECnt
telemetryCTEUsageRecurCTE = metrics.TelemetrySQLCTECnt.WithLabelValues("recursive_cte")
telemetryCTEUsageNonRecurCTE = metrics.TelemetrySQLCTECnt.WithLabelValues("nonRecurCTE")
telemetryCTEUsageNotCTE = metrics.TelemetrySQLCTECnt.WithLabelValues("notCTE")
telemetryMultiSchemaChangeUsage = metrics.TelemetryMultiSchemaChangeCnt
telemetryTablePartitionUsage = metrics.TelemetryTablePartitionCnt
telemetryTablePartitionListUsage = metrics.TelemetryTablePartitionListCnt
Expand Down Expand Up @@ -3332,11 +3334,11 @@ func (s *session) updateTelemetryMetric(es *executor.ExecStmt) {

ti := es.Ti
if ti.UseRecursive {
telemetryCTEUsage.WithLabelValues("recurCTE").Inc()
telemetryCTEUsageRecurCTE.Inc()
} else if ti.UseNonRecursive {
telemetryCTEUsage.WithLabelValues("nonRecurCTE").Inc()
telemetryCTEUsageNonRecurCTE.Inc()
} else {
telemetryCTEUsage.WithLabelValues("notCTE").Inc()
telemetryCTEUsageNotCTE.Inc()
}

if ti.UseMultiSchemaChange {
Expand Down
25 changes: 7 additions & 18 deletions session/txn.go
Expand Up @@ -28,7 +28,6 @@ import (
"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/parser/terror"
"github.com/pingcap/tidb/session/txninfo"
Expand Down Expand Up @@ -95,13 +94,10 @@ func (txn *LazyTxn) updateState(state txninfo.TxnRunningState) {
txn.mu.TxnInfo.State = state
txn.mu.TxnInfo.LastStateChangeTime = time.Now()
if !lastStateChangeTime.IsZero() {
hasLockLbl := "false"
if !txn.mu.TxnInfo.BlockStartTime.IsZero() {
hasLockLbl = "true"
}
metrics.TxnDurationHistogram.WithLabelValues(txninfo.StateLabel(lastState), hasLockLbl).Observe(time.Since(lastStateChangeTime).Seconds())
hasLockLbl := !txn.mu.TxnInfo.BlockStartTime.IsZero()
txninfo.TxnDurationHistogram(lastState, hasLockLbl).Observe(time.Since(lastStateChangeTime).Seconds())
}
metrics.TxnStatusEnteringCounter.WithLabelValues(txninfo.StateLabel(state)).Inc()
txninfo.TxnStatusEnteringCounter(state).Inc()
}
}

Expand Down Expand Up @@ -157,19 +153,16 @@ func (txn *LazyTxn) resetTxnInfo(
) {
if !txn.mu.LastStateChangeTime.IsZero() {
lastState := txn.mu.State
hasLockLbl := "false"
if !txn.mu.TxnInfo.BlockStartTime.IsZero() {
hasLockLbl = "true"
}
metrics.TxnDurationHistogram.WithLabelValues(txninfo.StateLabel(lastState), hasLockLbl).Observe(time.Since(txn.mu.TxnInfo.LastStateChangeTime).Seconds())
hasLockLbl := !txn.mu.BlockStartTime.IsZero()
txninfo.TxnDurationHistogram(lastState, hasLockLbl).Observe(time.Since(txn.mu.TxnInfo.LastStateChangeTime).Seconds())
}
if txn.mu.TxnInfo.StartTS != 0 {
txninfo.Recorder.OnTrxEnd(&txn.mu.TxnInfo)
}
txn.mu.TxnInfo = txninfo.TxnInfo{}
txn.mu.TxnInfo.StartTS = startTS
txn.mu.TxnInfo.State = state
metrics.TxnStatusEnteringCounter.WithLabelValues(txninfo.StateLabel(state)).Inc()
txninfo.TxnStatusEnteringCounter(state).Inc()
txn.mu.TxnInfo.LastStateChangeTime = time.Now()
txn.mu.TxnInfo.EntriesCount = entriesCount
txn.mu.TxnInfo.EntriesSize = entriesSize
Expand Down Expand Up @@ -294,11 +287,7 @@ func (txn *LazyTxn) changeToInvalid() {
txn.mu.TxnInfo = txninfo.TxnInfo{}
txn.mu.Unlock()
if !lastStateChangeTime.IsZero() {
hasLockLbl := "false"
if hasLock {
hasLockLbl = "true"
}
metrics.TxnDurationHistogram.WithLabelValues(txninfo.StateLabel(lastState), hasLockLbl).Observe(time.Since(lastStateChangeTime).Seconds())
txninfo.TxnDurationHistogram(lastState, hasLock).Observe(time.Since(lastStateChangeTime).Seconds())
}
}

Expand Down
53 changes: 43 additions & 10 deletions session/txninfo/txn_info.go
Expand Up @@ -18,9 +18,11 @@ import (
"encoding/json"
"time"

"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/logutil"
"github.com/prometheus/client_golang/prometheus"
"github.com/tikv/client-go/v2/oracle"
"go.uber.org/zap"
)
Expand All @@ -41,18 +43,49 @@ const (
TxnRollingBack
)

// StateLabel is used to translate TxnRunningState to its prometheus label name.
var stateLabel map[TxnRunningState]string = map[TxnRunningState]string{
TxnIdle: "idle",
TxnRunning: "executing_sql",
TxnLockAcquiring: "acquiring_lock",
TxnCommitting: "committing",
TxnRollingBack: "rolling_back",
var txnDurationHistogramForState map[TxnRunningState][]prometheus.Observer = map[TxnRunningState][]prometheus.Observer{
TxnIdle: []prometheus.Observer{
metrics.TxnDurationHistogram.WithLabelValues("idle", "false"),
metrics.TxnDurationHistogram.WithLabelValues("idle", "true"),
},
TxnRunning: []prometheus.Observer{
metrics.TxnDurationHistogram.WithLabelValues("executing_sql", "false"),
metrics.TxnDurationHistogram.WithLabelValues("executing_sql", "true"),
},
TxnLockAcquiring: []prometheus.Observer{
metrics.TxnDurationHistogram.WithLabelValues("acquiring_lock", "false"),
metrics.TxnDurationHistogram.WithLabelValues("acquiring_lock", "true"),
},
TxnCommitting: []prometheus.Observer{
metrics.TxnDurationHistogram.WithLabelValues("committing", "false"),
metrics.TxnDurationHistogram.WithLabelValues("committing", "true"),
},
TxnRollingBack: []prometheus.Observer{
metrics.TxnDurationHistogram.WithLabelValues("rolling_back", "false"),
metrics.TxnDurationHistogram.WithLabelValues("rolling_back", "true"),
},
}

var txnStatusEnteringCounterForState map[TxnRunningState]prometheus.Counter = map[TxnRunningState]prometheus.Counter{
TxnIdle: metrics.TxnStatusEnteringCounter.WithLabelValues("idle"),
TxnRunning: metrics.TxnStatusEnteringCounter.WithLabelValues("executing_sql"),
TxnLockAcquiring: metrics.TxnStatusEnteringCounter.WithLabelValues("acquiring_lock"),
TxnCommitting: metrics.TxnStatusEnteringCounter.WithLabelValues("committing"),
TxnRollingBack: metrics.TxnStatusEnteringCounter.WithLabelValues("rolling_back"),
}

// TxnDurationHistogram returns the observer for the given state and hasLock type.
func TxnDurationHistogram(state TxnRunningState, hasLock bool) prometheus.Observer {
hasLockInt := 0
if hasLock {
hasLockInt = 1
}
return txnDurationHistogramForState[state][hasLockInt]
}

// StateLabel is used to translate TxnRunningState to its prometheus label name.
func StateLabel(state TxnRunningState) string {
return stateLabel[state]
// TxnStatusEnteringCounter returns the counter for the given state.
func TxnStatusEnteringCounter(state TxnRunningState) prometheus.Counter {
return txnStatusEnteringCounterForState[state]
}

const (
Expand Down

0 comments on commit 402b5c4

Please sign in to comment.