From 723526708ddcdf63a40b264ce1f63eaec6885001 Mon Sep 17 00:00:00 2001 From: lionelee Date: Sat, 4 Mar 2023 00:03:10 +0800 Subject: [PATCH] metrics: attach const label keyspace_id (#41693) close pingcap/tidb#41698 --- br/cmd/br/BUILD.bazel | 1 + br/cmd/br/backup.go | 5 + br/cmd/br/restore.go | 5 + domain/BUILD.bazel | 1 + domain/historical_stats.go | 11 +- domain/metrics/BUILD.bazel | 12 ++ domain/metrics/metrics.go | 52 +++++ domain/plan_replayer.go | 14 +- domain/plan_replayer_dump.go | 11 +- executor/BUILD.bazel | 1 + executor/adapter.go | 199 ++++-------------- executor/analyze_fast.go | 14 +- executor/builder.go | 37 ++-- executor/metrics/BUILD.bazel | 12 ++ executor/metrics/metrics.go | 249 +++++++++++++++++++++++ executor/simple.go | 17 +- infoschema/BUILD.bazel | 2 +- infoschema/cache.go | 24 +-- infoschema/metrics/BUILD.bazel | 12 ++ infoschema/metrics/metrics.go | 46 +++++ metrics/BUILD.bazel | 1 + metrics/bindinfo.go | 15 +- metrics/ddl.go | 78 ++++--- metrics/distsql.go | 32 ++- metrics/domain.go | 58 ++++-- metrics/executor.go | 27 ++- metrics/gc_worker.go | 27 ++- metrics/log_backup.go | 33 ++- metrics/meta.go | 21 +- metrics/metrics.go | 57 ++++-- metrics/owner.go | 29 ++- metrics/resourcemanager.go | 16 +- metrics/server.go | 125 ++++++++---- metrics/session.go | 77 +++++-- metrics/sli.go | 14 +- metrics/stats.go | 64 ++++-- metrics/telemetry.go | 113 +++++++--- metrics/topsql.go | 15 +- metrics/ttl.go | 29 ++- metrics/wrapper.go | 77 +++++++ planner/core/BUILD.bazel | 1 + planner/core/common_plans.go | 4 - planner/core/logical_plan_builder.go | 13 +- planner/core/metrics/BUILD.bazel | 12 ++ planner/core/metrics/metrics.go | 41 ++++ planner/core/plan_cache.go | 7 +- server/BUILD.bazel | 1 + server/conn.go | 72 ++----- server/metrics/BUILD.bazel | 13 ++ server/metrics/metrics.go | 95 +++++++++ server/packetio.go | 13 +- session/BUILD.bazel | 2 +- session/metrics/BUILD.bazel | 12 ++ session/metrics/metrics.go | 146 +++++++++++++ session/nontransactional.go | 15 +- session/session.go | 157 +++++--------- session/tidb.go | 9 +- session/txninfo/txn_info.go | 68 ++++--- sessiontxn/isolation/BUILD.bazel | 2 +- sessiontxn/isolation/metrics/BUILD.bazel | 12 ++ sessiontxn/isolation/metrics/metrics.go | 36 ++++ sessiontxn/isolation/readcommitted.go | 11 +- statistics/handle/BUILD.bazel | 2 +- statistics/handle/dump.go | 11 +- statistics/handle/handle.go | 16 +- statistics/handle/lru_cache.go | 30 +-- statistics/handle/metrics/BUILD.bazel | 12 ++ statistics/handle/metrics/metrics.go | 62 ++++++ store/copr/BUILD.bazel | 1 + store/copr/coprocessor.go | 14 +- store/copr/metrics/BUILD.bazel | 12 ++ store/copr/metrics/metrics.go | 38 ++++ tidb-server/BUILD.bazel | 2 +- tidb-server/main.go | 15 +- ttl/metrics/metrics.go | 34 +++- util/metricsutil/BUILD.bazel | 31 +++ util/metricsutil/common.go | 144 +++++++++++++ util/topsql/reporter/BUILD.bazel | 3 +- util/topsql/reporter/datamodel.go | 5 +- util/topsql/reporter/metrics.go | 36 ---- util/topsql/reporter/metrics/BUILD.bazel | 12 ++ util/topsql/reporter/metrics/metrics.go | 64 ++++++ util/topsql/reporter/pubsub.go | 25 +-- util/topsql/reporter/reporter.go | 7 +- util/topsql/reporter/single_target.go | 25 +-- 85 files changed, 2097 insertions(+), 879 deletions(-) create mode 100644 domain/metrics/BUILD.bazel create mode 100644 domain/metrics/metrics.go create mode 100644 executor/metrics/BUILD.bazel create mode 100644 executor/metrics/metrics.go create mode 100644 infoschema/metrics/BUILD.bazel create mode 100644 infoschema/metrics/metrics.go create mode 100644 metrics/wrapper.go create mode 100644 planner/core/metrics/BUILD.bazel create mode 100644 planner/core/metrics/metrics.go create mode 100644 server/metrics/BUILD.bazel create mode 100644 server/metrics/metrics.go create mode 100644 session/metrics/BUILD.bazel create mode 100644 session/metrics/metrics.go create mode 100644 sessiontxn/isolation/metrics/BUILD.bazel create mode 100644 sessiontxn/isolation/metrics/metrics.go create mode 100644 statistics/handle/metrics/BUILD.bazel create mode 100644 statistics/handle/metrics/metrics.go create mode 100644 store/copr/metrics/BUILD.bazel create mode 100644 store/copr/metrics/metrics.go create mode 100644 util/metricsutil/BUILD.bazel create mode 100644 util/metricsutil/common.go delete mode 100644 util/topsql/reporter/metrics.go create mode 100644 util/topsql/reporter/metrics/BUILD.bazel create mode 100644 util/topsql/reporter/metrics/metrics.go diff --git a/br/cmd/br/BUILD.bazel b/br/cmd/br/BUILD.bazel index 7202ee66a0419..a95909dc84f03 100644 --- a/br/cmd/br/BUILD.bazel +++ b/br/cmd/br/BUILD.bazel @@ -34,6 +34,7 @@ go_library( "//session", "//util", "//util/logutil", + "//util/metricsutil", "@com_github_gogo_protobuf//proto", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_kvproto//pkg/brpb", diff --git a/br/cmd/br/backup.go b/br/cmd/br/backup.go index f8ebb2c4578b5..003bde2a2794b 100644 --- a/br/cmd/br/backup.go +++ b/br/cmd/br/backup.go @@ -13,6 +13,7 @@ import ( "github.com/pingcap/tidb/br/pkg/version/build" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/util/metricsutil" "github.com/spf13/cobra" "go.uber.org/zap" "sourcegraph.com/sourcegraph/appdash" @@ -25,6 +26,10 @@ func runBackupCommand(command *cobra.Command, cmdName string) error { return errors.Trace(err) } + if err := metricsutil.RegisterMetricsForBR(cfg.PD, cfg.KeyspaceName); err != nil { + return errors.Trace(err) + } + ctx := GetDefaultContext() if cfg.EnableOpenTracing { var store *appdash.MemoryStore diff --git a/br/cmd/br/restore.go b/br/cmd/br/restore.go index e826df0e59e77..de54dbc37726d 100644 --- a/br/cmd/br/restore.go +++ b/br/cmd/br/restore.go @@ -15,6 +15,7 @@ import ( "github.com/pingcap/tidb/br/pkg/utils" "github.com/pingcap/tidb/br/pkg/version/build" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/util/metricsutil" "github.com/spf13/cobra" "go.uber.org/zap" "sourcegraph.com/sourcegraph/appdash" @@ -27,6 +28,10 @@ func runRestoreCommand(command *cobra.Command, cmdName string) error { return errors.Trace(err) } + if err := metricsutil.RegisterMetricsForBR(cfg.PD, cfg.KeyspaceName); err != nil { + return errors.Trace(err) + } + if task.IsStreamRestore(cmdName) { if err := cfg.ParseStreamRestoreFlags(command.Flags()); err != nil { return errors.Trace(err) diff --git a/domain/BUILD.bazel b/domain/BUILD.bazel index f2deb8676df0c..0c2afa2f7e184 100644 --- a/domain/BUILD.bazel +++ b/domain/BUILD.bazel @@ -30,6 +30,7 @@ go_library( "//ddl/util", "//domain/globalconfigsync", "//domain/infosync", + "//domain/metrics", "//errno", "//infoschema", "//infoschema/perfschema", diff --git a/domain/historical_stats.go b/domain/historical_stats.go index 6d4125b75f5d7..38aeb0c46b0b3 100644 --- a/domain/historical_stats.go +++ b/domain/historical_stats.go @@ -17,7 +17,7 @@ package domain import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/metrics" + domain_metrics "github.com/pingcap/tidb/domain/metrics" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/statistics/handle" @@ -25,11 +25,6 @@ import ( "go.uber.org/zap" ) -var ( - generateHistoricalStatsSuccessCounter = metrics.HistoricalStatsCounter.WithLabelValues("generate", "success") - generateHistoricalStatsFailedCounter = metrics.HistoricalStatsCounter.WithLabelValues("generate", "fail") -) - // HistoricalStatsWorker indicates for dump historical stats type HistoricalStatsWorker struct { tblCH chan int64 @@ -85,10 +80,10 @@ func (w *HistoricalStatsWorker) DumpHistoricalStats(tableID int64, statsHandle * return errors.Errorf("cannot get DBInfo by TableID %d", tableID) } if _, err := statsHandle.RecordHistoricalStatsToStorage(dbInfo.Name.O, tblInfo, tableID, isPartition); err != nil { - generateHistoricalStatsFailedCounter.Inc() + domain_metrics.GenerateHistoricalStatsFailedCounter.Inc() return errors.Errorf("record table %s.%s's historical stats failed, err:%v", dbInfo.Name.O, tblInfo.Name.O, err) } - generateHistoricalStatsSuccessCounter.Inc() + domain_metrics.GenerateHistoricalStatsSuccessCounter.Inc() return nil } diff --git a/domain/metrics/BUILD.bazel b/domain/metrics/BUILD.bazel new file mode 100644 index 0000000000000..c0d3843193791 --- /dev/null +++ b/domain/metrics/BUILD.bazel @@ -0,0 +1,12 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "metrics", + srcs = ["metrics.go"], + importpath = "github.com/pingcap/tidb/domain/metrics", + visibility = ["//visibility:public"], + deps = [ + "//metrics", + "@com_github_prometheus_client_golang//prometheus", + ], +) diff --git a/domain/metrics/metrics.go b/domain/metrics/metrics.go new file mode 100644 index 0000000000000..08ddabdaeae9a --- /dev/null +++ b/domain/metrics/metrics.go @@ -0,0 +1,52 @@ +// Copyright 2023 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package metrics + +import ( + "github.com/pingcap/tidb/metrics" + "github.com/prometheus/client_golang/prometheus" +) + +// domain metrics vars +var ( + GenerateHistoricalStatsSuccessCounter prometheus.Counter + GenerateHistoricalStatsFailedCounter prometheus.Counter + + PlanReplayerDumpTaskSuccess prometheus.Counter + PlanReplayerDumpTaskFailed prometheus.Counter + + PlanReplayerCaptureTaskSendCounter prometheus.Counter + PlanReplayerCaptureTaskDiscardCounter prometheus.Counter + + PlanReplayerRegisterTaskGauge prometheus.Gauge +) + +func init() { + InitMetricsVars() +} + +// InitMetricsVars init domain metrics vars. +func InitMetricsVars() { + GenerateHistoricalStatsSuccessCounter = metrics.HistoricalStatsCounter.WithLabelValues("generate", "success") + GenerateHistoricalStatsFailedCounter = metrics.HistoricalStatsCounter.WithLabelValues("generate", "fail") + + PlanReplayerDumpTaskSuccess = metrics.PlanReplayerTaskCounter.WithLabelValues("dump", "success") + PlanReplayerDumpTaskFailed = metrics.PlanReplayerTaskCounter.WithLabelValues("dump", "fail") + + PlanReplayerCaptureTaskSendCounter = metrics.PlanReplayerTaskCounter.WithLabelValues("capture", "send") + PlanReplayerCaptureTaskDiscardCounter = metrics.PlanReplayerTaskCounter.WithLabelValues("capture", "discard") + + PlanReplayerRegisterTaskGauge = metrics.PlanReplayerRegisterTaskGauge +} diff --git a/domain/plan_replayer.go b/domain/plan_replayer.go index bb36048f02b26..535838fff375e 100644 --- a/domain/plan_replayer.go +++ b/domain/plan_replayer.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/bindinfo" "github.com/pingcap/tidb/domain/infosync" + domain_metrics "github.com/pingcap/tidb/domain/metrics" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser" @@ -179,13 +180,6 @@ func insertPlanReplayerSuccessStatusRecord(ctx context.Context, sctx sessionctx. } } -var ( - planReplayerCaptureTaskSendCounter = metrics.PlanReplayerTaskCounter.WithLabelValues("capture", "send") - planReplayerCaptureTaskDiscardCounter = metrics.PlanReplayerTaskCounter.WithLabelValues("capture", "discard") - - planReplayerRegisterTaskGauge = metrics.PlanReplayerRegisterTaskGauge -) - type planReplayerHandle struct { *planReplayerTaskCollectorHandle *planReplayerTaskDumpHandle @@ -200,10 +194,10 @@ func (h *planReplayerHandle) SendTask(task *PlanReplayerDumpTask) bool { if !task.IsContinuesCapture { h.planReplayerTaskCollectorHandle.removeTask(task.PlanReplayerTaskKey) } - planReplayerCaptureTaskSendCounter.Inc() + domain_metrics.PlanReplayerCaptureTaskSendCounter.Inc() return true default: - planReplayerCaptureTaskDiscardCounter.Inc() + domain_metrics.PlanReplayerCaptureTaskDiscardCounter.Inc() // directly discard the task if the task channel is full in order not to block the query process logutil.BgLogger().Warn("discard one plan replayer dump task", zap.String("sql-digest", task.SQLDigest), zap.String("plan-digest", task.PlanDigest)) @@ -241,7 +235,7 @@ func (h *planReplayerTaskCollectorHandle) CollectPlanReplayerTask() error { } } h.setupTasks(tasks) - planReplayerRegisterTaskGauge.Set(float64(len(tasks))) + domain_metrics.PlanReplayerRegisterTaskGauge.Set(float64(len(tasks))) return nil } diff --git a/domain/plan_replayer_dump.go b/domain/plan_replayer_dump.go index 920e4ba61813f..1e661a8ccdb74 100644 --- a/domain/plan_replayer_dump.go +++ b/domain/plan_replayer_dump.go @@ -28,8 +28,8 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/bindinfo" "github.com/pingcap/tidb/config" + domain_metrics "github.com/pingcap/tidb/domain/metrics" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx" @@ -151,11 +151,6 @@ func (tne *tableNameExtractor) handleIsView(t *ast.TableName) (bool, error) { return true, nil } -var ( - planReplayerDumpTaskSuccess = metrics.PlanReplayerTaskCounter.WithLabelValues("dump", "success") - planReplayerDumpTaskFailed = metrics.PlanReplayerTaskCounter.WithLabelValues("dump", "fail") -) - // DumpPlanReplayerInfo will dump the information about sqls. // The files will be organized into the following format: /* @@ -224,9 +219,9 @@ func DumpPlanReplayerInfo(ctx context.Context, sctx sessionctx.Context, zap.Strings("sqls", sqls)) } errMsg = err.Error() - planReplayerDumpTaskFailed.Inc() + domain_metrics.PlanReplayerDumpTaskFailed.Inc() } else { - planReplayerDumpTaskSuccess.Inc() + domain_metrics.PlanReplayerDumpTaskSuccess.Inc() } err1 := zw.Close() if err1 != nil { diff --git a/executor/BUILD.bazel b/executor/BUILD.bazel index da282a4dc7435..dd45fe8d6ee9b 100644 --- a/executor/BUILD.bazel +++ b/executor/BUILD.bazel @@ -114,6 +114,7 @@ go_library( "//domain/infosync", "//errno", "//executor/aggfuncs", + "//executor/metrics", "//expression", "//expression/aggregation", "//infoschema", diff --git a/executor/adapter.go b/executor/adapter.go index 133e54518757e..2f399a6ee63b3 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl/placement" "github.com/pingcap/tidb/domain" + executor_metrics "github.com/pingcap/tidb/executor/metrics" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" @@ -74,32 +75,6 @@ import ( "go.uber.org/zap/zapcore" ) -// metrics option -var ( - totalQueryProcHistogramGeneral = metrics.TotalQueryProcHistogram.WithLabelValues(metrics.LblGeneral) - totalCopProcHistogramGeneral = metrics.TotalCopProcHistogram.WithLabelValues(metrics.LblGeneral) - totalCopWaitHistogramGeneral = metrics.TotalCopWaitHistogram.WithLabelValues(metrics.LblGeneral) - totalQueryProcHistogramInternal = metrics.TotalQueryProcHistogram.WithLabelValues(metrics.LblInternal) - totalCopProcHistogramInternal = metrics.TotalCopProcHistogram.WithLabelValues(metrics.LblInternal) - totalCopWaitHistogramInternal = metrics.TotalCopWaitHistogram.WithLabelValues(metrics.LblInternal) - - selectForUpdateFirstAttemptDuration = metrics.PessimisticDMLDurationByAttempt.WithLabelValues("select-for-update", "first-attempt") - selectForUpdateRetryDuration = metrics.PessimisticDMLDurationByAttempt.WithLabelValues("select-for-update", "retry") - dmlFirstAttemptDuration = metrics.PessimisticDMLDurationByAttempt.WithLabelValues("dml", "first-attempt") - dmlRetryDuration = metrics.PessimisticDMLDurationByAttempt.WithLabelValues("dml", "retry") - - // aggressiveLockingTxnUsedCount counts transactions where at least one statement has aggressive locking enabled. - aggressiveLockingTxnUsedCount = metrics.AggressiveLockingUsageCount.WithLabelValues(metrics.LblAggressiveLockingTxnUsed) - // aggressiveLockingStmtUsedCount counts statements that have aggressive locking enabled. - aggressiveLockingStmtUsedCount = metrics.AggressiveLockingUsageCount.WithLabelValues(metrics.LblAggressiveLockingStmtUsed) - // aggressiveLockingTxnUsedCount counts transactions where at least one statement has aggressive locking enabled, - // and it takes effect (which is determined according to whether lock-with-conflict has occurred during execution). - aggressiveLockingTxnEffectiveCount = metrics.AggressiveLockingUsageCount.WithLabelValues(metrics.LblAggressiveLockingTxnEffective) - // aggressiveLockingTxnUsedCount counts statements where at least one statement has aggressive locking enabled, - // and it takes effect (which is determined according to whether lock-with-conflict has occurred during execution). - aggressiveLockingStmtEffectiveCount = metrics.AggressiveLockingUsageCount.WithLabelValues(metrics.LblAggressiveLockingStmtEffective) -) - // processinfoSetter is the interface use to set current running process info. type processinfoSetter interface { SetProcessInfo(string, time.Time, byte, uint64) @@ -909,10 +884,10 @@ func (a *ExecStmt) handlePessimisticSelectForUpdate(ctx context.Context, e Execu rs, err := a.runPessimisticSelectForUpdate(ctx, e) if isFirstAttempt { - selectForUpdateFirstAttemptDuration.Observe(time.Since(startTime).Seconds()) + executor_metrics.SelectForUpdateFirstAttemptDuration.Observe(time.Since(startTime).Seconds()) isFirstAttempt = false } else { - selectForUpdateRetryDuration.Observe(time.Since(startTime).Seconds()) + executor_metrics.SelectForUpdateRetryDuration.Observe(time.Since(startTime).Seconds()) } e, err = a.handlePessimisticLockError(ctx, err) @@ -1044,10 +1019,10 @@ func (a *ExecStmt) handlePessimisticDML(ctx context.Context, e Executor) (err er } if isFirstAttempt { - dmlFirstAttemptDuration.Observe(time.Since(startTime).Seconds()) + executor_metrics.DmlFirstAttemptDuration.Observe(time.Since(startTime).Seconds()) isFirstAttempt = false } else { - dmlRetryDuration.Observe(time.Since(startTime).Seconds()) + executor_metrics.DmlRetryDuration.Observe(time.Since(startTime).Seconds()) } if err != nil { @@ -1286,117 +1261,17 @@ func FormatSQL(sql string) stringutil.StringerFunc { } } -const ( - phaseBuildLocking = "build:locking" - phaseOpenLocking = "open:locking" - phaseNextLocking = "next:locking" - phaseLockLocking = "lock:locking" - phaseBuildFinal = "build:final" - phaseOpenFinal = "open:final" - phaseNextFinal = "next:final" - phaseLockFinal = "lock:final" - phaseCommitPrewrite = "commit:prewrite" - phaseCommitCommit = "commit:commit" - phaseCommitWaitCommitTS = "commit:wait:commit-ts" - phaseCommitWaitLatestTS = "commit:wait:latest-ts" - phaseCommitWaitLatch = "commit:wait:local-latch" - phaseCommitWaitBinlog = "commit:wait:prewrite-binlog" - phaseWriteResponse = "write-response" -) - -var ( - sessionExecuteRunDurationInternal = metrics.SessionExecuteRunDuration.WithLabelValues(metrics.LblInternal) - sessionExecuteRunDurationGeneral = metrics.SessionExecuteRunDuration.WithLabelValues(metrics.LblGeneral) - totalTiFlashQuerySuccCounter = metrics.TiFlashQueryTotalCounter.WithLabelValues("", metrics.LblOK) - - // pre-define observers for non-internal queries - execBuildLocking = metrics.ExecPhaseDuration.WithLabelValues(phaseBuildLocking, "0") - execOpenLocking = metrics.ExecPhaseDuration.WithLabelValues(phaseOpenLocking, "0") - execNextLocking = metrics.ExecPhaseDuration.WithLabelValues(phaseNextLocking, "0") - execLockLocking = metrics.ExecPhaseDuration.WithLabelValues(phaseLockLocking, "0") - execBuildFinal = metrics.ExecPhaseDuration.WithLabelValues(phaseBuildFinal, "0") - execOpenFinal = metrics.ExecPhaseDuration.WithLabelValues(phaseOpenFinal, "0") - execNextFinal = metrics.ExecPhaseDuration.WithLabelValues(phaseNextFinal, "0") - execLockFinal = metrics.ExecPhaseDuration.WithLabelValues(phaseLockFinal, "0") - execCommitPrewrite = metrics.ExecPhaseDuration.WithLabelValues(phaseCommitPrewrite, "0") - execCommitCommit = metrics.ExecPhaseDuration.WithLabelValues(phaseCommitCommit, "0") - execCommitWaitCommitTS = metrics.ExecPhaseDuration.WithLabelValues(phaseCommitWaitCommitTS, "0") - execCommitWaitLatestTS = metrics.ExecPhaseDuration.WithLabelValues(phaseCommitWaitLatestTS, "0") - 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 { - if ob, found := phaseDurationObserverMapInternal[phase]; found { + if ob, found := executor_metrics.PhaseDurationObserverMapInternal[phase]; found { return ob } - return execUnknownInternal + return executor_metrics.ExecUnknownInternal } - if ob, found := phaseDurationObserverMap[phase]; found { + if ob, found := executor_metrics.PhaseDurationObserverMap[phase]; found { return ob } - return execUnknown + return executor_metrics.ExecUnknown } func (a *ExecStmt) observePhaseDurations(internal bool, commitDetails *util.CommitDetails) { @@ -1404,14 +1279,14 @@ func (a *ExecStmt) observePhaseDurations(internal bool, commitDetails *util.Comm duration time.Duration phase string }{ - {a.phaseBuildDurations[0], phaseBuildFinal}, - {a.phaseBuildDurations[1], phaseBuildLocking}, - {a.phaseOpenDurations[0], phaseOpenFinal}, - {a.phaseOpenDurations[1], phaseOpenLocking}, - {a.phaseNextDurations[0], phaseNextFinal}, - {a.phaseNextDurations[1], phaseNextLocking}, - {a.phaseLockDurations[0], phaseLockFinal}, - {a.phaseLockDurations[1], phaseLockLocking}, + {a.phaseBuildDurations[0], executor_metrics.PhaseBuildFinal}, + {a.phaseBuildDurations[1], executor_metrics.PhaseBuildLocking}, + {a.phaseOpenDurations[0], executor_metrics.PhaseOpenFinal}, + {a.phaseOpenDurations[1], executor_metrics.PhaseOpenLocking}, + {a.phaseNextDurations[0], executor_metrics.PhaseNextFinal}, + {a.phaseNextDurations[1], executor_metrics.PhaseNextLocking}, + {a.phaseLockDurations[0], executor_metrics.PhaseLockFinal}, + {a.phaseLockDurations[1], executor_metrics.PhaseLockLocking}, } { if it.duration > 0 { getPhaseDurationObserver(it.phase, internal).Observe(it.duration.Seconds()) @@ -1422,12 +1297,12 @@ func (a *ExecStmt) observePhaseDurations(internal bool, commitDetails *util.Comm duration time.Duration phase string }{ - {commitDetails.PrewriteTime, phaseCommitPrewrite}, - {commitDetails.CommitTime, phaseCommitCommit}, - {commitDetails.GetCommitTsTime, phaseCommitWaitCommitTS}, - {commitDetails.GetLatestTsTime, phaseCommitWaitLatestTS}, - {commitDetails.LocalLatchTime, phaseCommitWaitLatch}, - {commitDetails.WaitPrewriteBinlogTime, phaseCommitWaitBinlog}, + {commitDetails.PrewriteTime, executor_metrics.PhaseCommitPrewrite}, + {commitDetails.CommitTime, executor_metrics.PhaseCommitCommit}, + {commitDetails.GetCommitTsTime, executor_metrics.PhaseCommitWaitCommitTS}, + {commitDetails.GetLatestTsTime, executor_metrics.PhaseCommitWaitLatestTS}, + {commitDetails.LocalLatchTime, executor_metrics.PhaseCommitWaitLatch}, + {commitDetails.WaitPrewriteBinlogTime, executor_metrics.PhaseCommitWaitBinlog}, } { if it.duration > 0 { getPhaseDurationObserver(it.phase, internal).Observe(it.duration.Seconds()) @@ -1437,7 +1312,7 @@ func (a *ExecStmt) observePhaseDurations(internal bool, commitDetails *util.Comm if stmtDetailsRaw := a.GoCtx.Value(execdetails.StmtExecDetailKey); stmtDetailsRaw != nil { d := stmtDetailsRaw.(*execdetails.StmtExecDetails).WriteSQLRespDuration if d > 0 { - getPhaseDurationObserver(phaseWriteResponse, internal).Observe(d.Seconds()) + getPhaseDurationObserver(executor_metrics.PhaseWriteResponse, internal).Observe(d.Seconds()) } } } @@ -1484,7 +1359,7 @@ func (a *ExecStmt) FinishExecuteStmt(txnTS uint64, err error, hasMoreResults boo a.observeStmtFinishedForTopSQL() if sessVars.StmtCtx.IsTiFlash.Load() { if succ { - totalTiFlashQuerySuccCounter.Inc() + executor_metrics.TotalTiFlashQuerySuccCounter.Inc() } else { metrics.TiFlashQueryTotalCounter.WithLabelValues(metrics.ExecuteErrorToLabel(err), metrics.LblError).Inc() } @@ -1494,9 +1369,9 @@ func (a *ExecStmt) FinishExecuteStmt(txnTS uint64, err error, hasMoreResults boo a.observePhaseDurations(sessVars.InRestrictedSQL, execDetail.CommitDetail) executeDuration := time.Since(sessVars.StartTime) - sessVars.DurationCompile if sessVars.InRestrictedSQL { - sessionExecuteRunDurationInternal.Observe(executeDuration.Seconds()) + executor_metrics.SessionExecuteRunDurationInternal.Observe(executeDuration.Seconds()) } else { - sessionExecuteRunDurationGeneral.Observe(executeDuration.Seconds()) + executor_metrics.SessionExecuteRunDurationGeneral.Observe(executeDuration.Seconds()) } // Reset DurationParse due to the next statement may not need to be parsed (not a text protocol query). sessVars.DurationParse = 0 @@ -1514,22 +1389,22 @@ func (a *ExecStmt) FinishExecuteStmt(txnTS uint64, err error, hasMoreResults boo // Update aggressive locking related counters by stmt if execDetail.LockKeysDetail != nil { if execDetail.LockKeysDetail.AggressiveLockNewCount > 0 || execDetail.LockKeysDetail.AggressiveLockDerivedCount > 0 { - aggressiveLockingStmtUsedCount.Inc() + executor_metrics.AggressiveLockingStmtUsedCount.Inc() // If this statement is finished when some of the keys are locked with conflict in the last retry, or // some of the keys are derived from the previous retry, we consider the optimization of aggressive locking // takes effect on this statement. if execDetail.LockKeysDetail.LockedWithConflictCount > 0 || execDetail.LockKeysDetail.AggressiveLockDerivedCount > 0 { - aggressiveLockingStmtEffectiveCount.Inc() + executor_metrics.AggressiveLockingStmtEffectiveCount.Inc() } } } // If the transaction is committed, update aggressive locking related counters by txn if execDetail.CommitDetail != nil { if sessVars.TxnCtx.AggressiveLockingUsed { - aggressiveLockingTxnUsedCount.Inc() + executor_metrics.AggressiveLockingTxnUsedCount.Inc() } if sessVars.TxnCtx.AggressiveLockingEffective { - aggressiveLockingTxnEffectiveCount.Inc() + executor_metrics.AggressiveLockingTxnEffectiveCount.Inc() } } } @@ -1687,13 +1562,13 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) { logutil.SlowQueryLogger.Warn(slowLog) if costTime >= threshold { if sessVars.InRestrictedSQL { - totalQueryProcHistogramInternal.Observe(costTime.Seconds()) - totalCopProcHistogramInternal.Observe(execDetail.TimeDetail.ProcessTime.Seconds()) - totalCopWaitHistogramInternal.Observe(execDetail.TimeDetail.WaitTime.Seconds()) + executor_metrics.TotalQueryProcHistogramInternal.Observe(costTime.Seconds()) + executor_metrics.TotalCopProcHistogramInternal.Observe(execDetail.TimeDetail.ProcessTime.Seconds()) + executor_metrics.TotalCopWaitHistogramInternal.Observe(execDetail.TimeDetail.WaitTime.Seconds()) } else { - totalQueryProcHistogramGeneral.Observe(costTime.Seconds()) - totalCopProcHistogramGeneral.Observe(execDetail.TimeDetail.ProcessTime.Seconds()) - totalCopWaitHistogramGeneral.Observe(execDetail.TimeDetail.WaitTime.Seconds()) + executor_metrics.TotalQueryProcHistogramGeneral.Observe(costTime.Seconds()) + executor_metrics.TotalCopProcHistogramGeneral.Observe(execDetail.TimeDetail.ProcessTime.Seconds()) + executor_metrics.TotalCopWaitHistogramGeneral.Observe(execDetail.TimeDetail.WaitTime.Seconds()) } var userString string if sessVars.User != nil { diff --git a/executor/analyze_fast.go b/executor/analyze_fast.go index b9dcc5d55ee97..b33fcf6b062cc 100644 --- a/executor/analyze_fast.go +++ b/executor/analyze_fast.go @@ -27,8 +27,8 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/domain" + executor_metrics "github.com/pingcap/tidb/executor/metrics" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" @@ -46,12 +46,6 @@ import ( "github.com/tikv/client-go/v2/tikv" ) -var ( - fastAnalyzeHistogramSample = metrics.FastAnalyzeHistogram.WithLabelValues(metrics.LblGeneral, "sample") - fastAnalyzeHistogramAccessRegions = metrics.FastAnalyzeHistogram.WithLabelValues(metrics.LblGeneral, "access_regions") - fastAnalyzeHistogramScanKeys = metrics.FastAnalyzeHistogram.WithLabelValues(metrics.LblGeneral, "scan_keys") -) - func analyzeFastExec(exec *AnalyzeFastExec) *statistics.AnalyzeResults { hists, cms, topNs, fms, err := exec.buildStats() if err != nil { @@ -238,7 +232,7 @@ func (e *AnalyzeFastExec) buildSampTask() (err error) { break } } - fastAnalyzeHistogramAccessRegions.Observe(float64(accessRegionsCounter)) + executor_metrics.FastAnalyzeHistogramAccessRegions.Observe(float64(accessRegionsCounter)) return nil } @@ -460,7 +454,7 @@ func (e *AnalyzeFastExec) handleSampTasks(workID int, step uint32, err *error) { return } } - fastAnalyzeHistogramSample.Observe(float64(len(kvMap))) + executor_metrics.FastAnalyzeHistogramSample.Observe(float64(len(kvMap))) *err = e.handleBatchSeekResponse(kvMap) if *err != nil { @@ -561,7 +555,7 @@ func (e *AnalyzeFastExec) runTasks() ([]*statistics.Histogram, []*statistics.CMS } scanKeysSize, err := e.handleScanTasks(bo) - fastAnalyzeHistogramScanKeys.Observe(float64(scanKeysSize)) + executor_metrics.FastAnalyzeHistogramScanKeys.Observe(float64(scanKeysSize)) if err != nil { return nil, nil, nil, nil, err } diff --git a/executor/builder.go b/executor/builder.go index b38e5d85148bb..27fe25c087ed5 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -35,11 +35,11 @@ import ( "github.com/pingcap/tidb/distsql" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor/aggfuncs" + executor_metrics "github.com/pingcap/tidb/executor/metrics" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" "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/model" "github.com/pingcap/tidb/parser/mysql" @@ -75,19 +75,6 @@ import ( "golang.org/x/exp/slices" ) -var ( - executorCounterMergeJoinExec = metrics.ExecutorCounter.WithLabelValues("MergeJoinExec") - executorCountHashJoinExec = metrics.ExecutorCounter.WithLabelValues("HashJoinExec") - executorCounterHashAggExec = metrics.ExecutorCounter.WithLabelValues("HashAggExec") - executorStreamAggExec = metrics.ExecutorCounter.WithLabelValues("StreamAggExec") - executorCounterSortExec = metrics.ExecutorCounter.WithLabelValues("SortExec") - executorCounterTopNExec = metrics.ExecutorCounter.WithLabelValues("TopNExec") - executorCounterNestedLoopApplyExec = metrics.ExecutorCounter.WithLabelValues("NestedLoopApplyExec") - executorCounterIndexLookUpJoin = metrics.ExecutorCounter.WithLabelValues("IndexLookUpJoin") - executorCounterIndexLookUpExecutor = metrics.ExecutorCounter.WithLabelValues("IndexLookUpExecutor") - executorCounterIndexMergeReaderExecutor = metrics.ExecutorCounter.WithLabelValues("IndexMergeReaderExecutor") -) - // executorBuilder builds an Executor from a Plan. // The InfoSchema must not change during execution. type executorBuilder struct { @@ -1413,7 +1400,7 @@ func (b *executorBuilder) buildMergeJoin(v *plannercore.PhysicalMergeJoin) Execu return nil } - executorCounterMergeJoinExec.Inc() + executor_metrics.ExecutorCounterMergeJoinExec.Inc() return e } @@ -1520,7 +1507,7 @@ func (b *executorBuilder) buildHashJoin(v *plannercore.PhysicalHashJoin) Executo } e.buildWorker.buildKeyColIdx, e.buildWorker.buildNAKeyColIdx, e.buildWorker.buildSideExec, e.buildWorker.hashJoinCtx = buildKeyColIdx, buildNAKeyColIdx, buildSideExec, e.hashJoinCtx e.hashJoinCtx.isNullAware = isNAJoin - executorCountHashJoinExec.Inc() + executor_metrics.ExecutorCountHashJoinExec.Inc() // We should use JoinKey to construct the type information using by hashing, instead of using the child's schema directly. // When a hybrid type column is hashed multiple times, we need to distinguish what field types are used. @@ -1650,7 +1637,7 @@ func (b *executorBuilder) buildHashAgg(v *plannercore.PhysicalHashAgg) Executor } } - executorCounterHashAggExec.Inc() + executor_metrics.ExecutorCounterHashAggExec.Inc() return e } @@ -1682,7 +1669,7 @@ func (b *executorBuilder) buildStreamAgg(v *plannercore.PhysicalStreamAgg) Execu } } - executorStreamAggExec.Inc() + executor_metrics.ExecutorStreamAggExec.Inc() return e } @@ -2073,7 +2060,7 @@ func (b *executorBuilder) buildSort(v *plannercore.PhysicalSort) Executor { ByItems: v.ByItems, schema: v.Schema(), } - executorCounterSortExec.Inc() + executor_metrics.ExecutorCounterSortExec.Inc() return &sortExec } @@ -2087,7 +2074,7 @@ func (b *executorBuilder) buildTopN(v *plannercore.PhysicalTopN) Executor { ByItems: v.ByItems, schema: v.Schema(), } - executorCounterTopNExec.Inc() + executor_metrics.ExecutorCounterTopNExec.Inc() return &TopNExec{ SortExec: sortExec, limit: &plannercore.PhysicalLimit{Count: v.Count, Offset: v.Offset}, @@ -2143,7 +2130,7 @@ func (b *executorBuilder) buildApply(v *plannercore.PhysicalApply) Executor { ctx: b.ctx, canUseCache: v.CanUseCache, } - executorCounterNestedLoopApplyExec.Inc() + executor_metrics.ExecutorCounterNestedLoopApplyExec.Inc() // try parallel mode if v.Concurrency > 1 { @@ -3203,7 +3190,7 @@ func (b *executorBuilder) buildIndexLookUpJoin(v *plannercore.PhysicalIndexJoin) e.innerCtx.hashCollators = hashCollators e.joinResult = tryNewCacheChunk(e) - executorCounterIndexLookUpJoin.Inc() + executor_metrics.ExecutorCounterIndexLookUpJoin.Inc() return e } @@ -3257,7 +3244,7 @@ func (b *executorBuilder) buildIndexLookUpMergeJoin(v *plannercore.PhysicalIndex innerKeyCols[i] = v.InnerJoinKeys[i].Index keyCollators = append(keyCollators, collate.GetCollator(v.InnerJoinKeys[i].RetType.GetCollate())) } - executorCounterIndexLookUpJoin.Inc() + executor_metrics.ExecutorCounterIndexLookUpJoin.Inc() readerBuilder, err := b.newDataReaderBuilder(innerPlan) if err != nil { @@ -3906,7 +3893,7 @@ func (b *executorBuilder) buildIndexLookUpReader(v *plannercore.PhysicalIndexLoo ts := v.TablePlans[0].(*plannercore.PhysicalTableScan) ret.ranges = is.Ranges - executorCounterIndexLookUpExecutor.Inc() + executor_metrics.ExecutorCounterIndexLookUpExecutor.Inc() sctx := b.ctx.GetSessionVars().StmtCtx sctx.IndexNames = append(sctx.IndexNames, is.Table.Name.O+":"+is.Index.Name.O) @@ -4068,7 +4055,7 @@ func (b *executorBuilder) buildIndexMergeReader(v *plannercore.PhysicalIndexMerg } } sctx.TableIDs = append(sctx.TableIDs, ts.Table.ID) - executorCounterIndexMergeReaderExecutor.Inc() + executor_metrics.ExecutorCounterIndexMergeReaderExecutor.Inc() if !b.ctx.GetSessionVars().StmtCtx.UseDynamicPartitionPrune() { return ret diff --git a/executor/metrics/BUILD.bazel b/executor/metrics/BUILD.bazel new file mode 100644 index 0000000000000..8fd4ab43550a7 --- /dev/null +++ b/executor/metrics/BUILD.bazel @@ -0,0 +1,12 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "metrics", + srcs = ["metrics.go"], + importpath = "github.com/pingcap/tidb/executor/metrics", + visibility = ["//visibility:public"], + deps = [ + "//metrics", + "@com_github_prometheus_client_golang//prometheus", + ], +) diff --git a/executor/metrics/metrics.go b/executor/metrics/metrics.go new file mode 100644 index 0000000000000..108c06f786879 --- /dev/null +++ b/executor/metrics/metrics.go @@ -0,0 +1,249 @@ +// Copyright 2023 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package metrics + +import ( + "github.com/pingcap/tidb/metrics" + "github.com/prometheus/client_golang/prometheus" +) + +// phases +const ( + PhaseBuildLocking = "build:locking" + PhaseOpenLocking = "open:locking" + PhaseNextLocking = "next:locking" + PhaseLockLocking = "lock:locking" + PhaseBuildFinal = "build:final" + PhaseOpenFinal = "open:final" + PhaseNextFinal = "next:final" + PhaseLockFinal = "lock:final" + PhaseCommitPrewrite = "commit:prewrite" + PhaseCommitCommit = "commit:commit" + PhaseCommitWaitCommitTS = "commit:wait:commit-ts" + PhaseCommitWaitLatestTS = "commit:wait:latest-ts" + PhaseCommitWaitLatch = "commit:wait:local-latch" + PhaseCommitWaitBinlog = "commit:wait:prewrite-binlog" + PhaseWriteResponse = "write-response" +) + +// executor metrics vars +var ( + TotalQueryProcHistogramGeneral prometheus.Observer + TotalCopProcHistogramGeneral prometheus.Observer + TotalCopWaitHistogramGeneral prometheus.Observer + TotalQueryProcHistogramInternal prometheus.Observer + TotalCopProcHistogramInternal prometheus.Observer + TotalCopWaitHistogramInternal prometheus.Observer + + SelectForUpdateFirstAttemptDuration prometheus.Observer + SelectForUpdateRetryDuration prometheus.Observer + DmlFirstAttemptDuration prometheus.Observer + DmlRetryDuration prometheus.Observer + + // AggressiveLockingTxnUsedCount counts transactions where at least one statement has aggressive locking enabled. + AggressiveLockingTxnUsedCount prometheus.Counter + // AggressiveLockingStmtUsedCount counts statements that have aggressive locking enabled. + AggressiveLockingStmtUsedCount prometheus.Counter + // AggressiveLockingTxnUsedCount counts transactions where at least one statement has aggressive locking enabled, + // and it takes effect (which is determined according to whether lock-with-conflict has occurred during execution). + AggressiveLockingTxnEffectiveCount prometheus.Counter + // AggressiveLockingTxnUsedCount counts statements where at least one statement has aggressive locking enabled, + // and it takes effect (which is determined according to whether lock-with-conflict has occurred during execution). + AggressiveLockingStmtEffectiveCount prometheus.Counter + + FastAnalyzeHistogramSample prometheus.Observer + FastAnalyzeHistogramAccessRegions prometheus.Observer + FastAnalyzeHistogramScanKeys prometheus.Observer + + ExecutorCounterMergeJoinExec prometheus.Counter + ExecutorCountHashJoinExec prometheus.Counter + ExecutorCounterHashAggExec prometheus.Counter + ExecutorStreamAggExec prometheus.Counter + ExecutorCounterSortExec prometheus.Counter + ExecutorCounterTopNExec prometheus.Counter + ExecutorCounterNestedLoopApplyExec prometheus.Counter + ExecutorCounterIndexLookUpJoin prometheus.Counter + ExecutorCounterIndexLookUpExecutor prometheus.Counter + ExecutorCounterIndexMergeReaderExecutor prometheus.Counter + + SessionExecuteRunDurationInternal prometheus.Observer + SessionExecuteRunDurationGeneral prometheus.Observer + TotalTiFlashQuerySuccCounter prometheus.Counter + + // pre-define observers for non-internal queries + ExecBuildLocking prometheus.Observer + ExecOpenLocking prometheus.Observer + ExecNextLocking prometheus.Observer + ExecLockLocking prometheus.Observer + ExecBuildFinal prometheus.Observer + ExecOpenFinal prometheus.Observer + ExecNextFinal prometheus.Observer + ExecLockFinal prometheus.Observer + ExecCommitPrewrite prometheus.Observer + ExecCommitCommit prometheus.Observer + ExecCommitWaitCommitTS prometheus.Observer + ExecCommitWaitLatestTS prometheus.Observer + ExecCommitWaitLatch prometheus.Observer + ExecCommitWaitBinlog prometheus.Observer + ExecWriteResponse prometheus.Observer + ExecUnknown prometheus.Observer + + // pre-define observers for internal queries + ExecBuildLockingInternal prometheus.Observer + ExecOpenLockingInternal prometheus.Observer + ExecNextLockingInternal prometheus.Observer + ExecLockLockingInternal prometheus.Observer + ExecBuildFinalInternal prometheus.Observer + ExecOpenFinalInternal prometheus.Observer + ExecNextFinalInternal prometheus.Observer + ExecLockFinalInternal prometheus.Observer + ExecCommitPrewriteInternal prometheus.Observer + ExecCommitCommitInternal prometheus.Observer + ExecCommitWaitCommitTSInternal prometheus.Observer + ExecCommitWaitLatestTSInternal prometheus.Observer + ExecCommitWaitLatchInternal prometheus.Observer + ExecCommitWaitBinlogInternal prometheus.Observer + ExecWriteResponseInternal prometheus.Observer + ExecUnknownInternal prometheus.Observer + + TransactionDurationPessimisticRollbackInternal prometheus.Observer + TransactionDurationPessimisticRollbackGeneral prometheus.Observer + TransactionDurationOptimisticRollbackInternal prometheus.Observer + TransactionDurationOptimisticRollbackGeneral prometheus.Observer + + PhaseDurationObserverMap map[string]prometheus.Observer + PhaseDurationObserverMapInternal map[string]prometheus.Observer +) + +func init() { + InitMetricsVars() + InitPhaseDurationObserverMap() +} + +// InitMetricsVars init executor metrics vars. +func InitMetricsVars() { + TotalQueryProcHistogramGeneral = metrics.TotalQueryProcHistogram.WithLabelValues(metrics.LblGeneral) + TotalCopProcHistogramGeneral = metrics.TotalCopProcHistogram.WithLabelValues(metrics.LblGeneral) + TotalCopWaitHistogramGeneral = metrics.TotalCopWaitHistogram.WithLabelValues(metrics.LblGeneral) + TotalQueryProcHistogramInternal = metrics.TotalQueryProcHistogram.WithLabelValues(metrics.LblInternal) + TotalCopProcHistogramInternal = metrics.TotalCopProcHistogram.WithLabelValues(metrics.LblInternal) + TotalCopWaitHistogramInternal = metrics.TotalCopWaitHistogram.WithLabelValues(metrics.LblInternal) + + SelectForUpdateFirstAttemptDuration = metrics.PessimisticDMLDurationByAttempt.WithLabelValues("select-for-update", "first-attempt") + SelectForUpdateRetryDuration = metrics.PessimisticDMLDurationByAttempt.WithLabelValues("select-for-update", "retry") + DmlFirstAttemptDuration = metrics.PessimisticDMLDurationByAttempt.WithLabelValues("dml", "first-attempt") + DmlRetryDuration = metrics.PessimisticDMLDurationByAttempt.WithLabelValues("dml", "retry") + + AggressiveLockingTxnUsedCount = metrics.AggressiveLockingUsageCount.WithLabelValues(metrics.LblAggressiveLockingTxnUsed) + AggressiveLockingStmtUsedCount = metrics.AggressiveLockingUsageCount.WithLabelValues(metrics.LblAggressiveLockingStmtUsed) + AggressiveLockingTxnEffectiveCount = metrics.AggressiveLockingUsageCount.WithLabelValues(metrics.LblAggressiveLockingTxnEffective) + AggressiveLockingStmtEffectiveCount = metrics.AggressiveLockingUsageCount.WithLabelValues(metrics.LblAggressiveLockingStmtEffective) + + FastAnalyzeHistogramSample = metrics.FastAnalyzeHistogram.WithLabelValues(metrics.LblGeneral, "sample") + FastAnalyzeHistogramAccessRegions = metrics.FastAnalyzeHistogram.WithLabelValues(metrics.LblGeneral, "access_regions") + FastAnalyzeHistogramScanKeys = metrics.FastAnalyzeHistogram.WithLabelValues(metrics.LblGeneral, "scan_keys") + + ExecutorCounterMergeJoinExec = metrics.ExecutorCounter.WithLabelValues("MergeJoinExec") + ExecutorCountHashJoinExec = metrics.ExecutorCounter.WithLabelValues("HashJoinExec") + ExecutorCounterHashAggExec = metrics.ExecutorCounter.WithLabelValues("HashAggExec") + ExecutorStreamAggExec = metrics.ExecutorCounter.WithLabelValues("StreamAggExec") + ExecutorCounterSortExec = metrics.ExecutorCounter.WithLabelValues("SortExec") + ExecutorCounterTopNExec = metrics.ExecutorCounter.WithLabelValues("TopNExec") + ExecutorCounterNestedLoopApplyExec = metrics.ExecutorCounter.WithLabelValues("NestedLoopApplyExec") + ExecutorCounterIndexLookUpJoin = metrics.ExecutorCounter.WithLabelValues("IndexLookUpJoin") + ExecutorCounterIndexLookUpExecutor = metrics.ExecutorCounter.WithLabelValues("IndexLookUpExecutor") + ExecutorCounterIndexMergeReaderExecutor = metrics.ExecutorCounter.WithLabelValues("IndexMergeReaderExecutor") + + SessionExecuteRunDurationInternal = metrics.SessionExecuteRunDuration.WithLabelValues(metrics.LblInternal) + SessionExecuteRunDurationGeneral = metrics.SessionExecuteRunDuration.WithLabelValues(metrics.LblGeneral) + TotalTiFlashQuerySuccCounter = metrics.TiFlashQueryTotalCounter.WithLabelValues("", metrics.LblOK) + + ExecBuildLocking = metrics.ExecPhaseDuration.WithLabelValues(PhaseBuildLocking, "0") + ExecOpenLocking = metrics.ExecPhaseDuration.WithLabelValues(PhaseOpenLocking, "0") + ExecNextLocking = metrics.ExecPhaseDuration.WithLabelValues(PhaseNextLocking, "0") + ExecLockLocking = metrics.ExecPhaseDuration.WithLabelValues(PhaseLockLocking, "0") + ExecBuildFinal = metrics.ExecPhaseDuration.WithLabelValues(PhaseBuildFinal, "0") + ExecOpenFinal = metrics.ExecPhaseDuration.WithLabelValues(PhaseOpenFinal, "0") + ExecNextFinal = metrics.ExecPhaseDuration.WithLabelValues(PhaseNextFinal, "0") + ExecLockFinal = metrics.ExecPhaseDuration.WithLabelValues(PhaseLockFinal, "0") + ExecCommitPrewrite = metrics.ExecPhaseDuration.WithLabelValues(PhaseCommitPrewrite, "0") + ExecCommitCommit = metrics.ExecPhaseDuration.WithLabelValues(PhaseCommitCommit, "0") + ExecCommitWaitCommitTS = metrics.ExecPhaseDuration.WithLabelValues(PhaseCommitWaitCommitTS, "0") + ExecCommitWaitLatestTS = metrics.ExecPhaseDuration.WithLabelValues(PhaseCommitWaitLatestTS, "0") + ExecCommitWaitLatch = metrics.ExecPhaseDuration.WithLabelValues(PhaseCommitWaitLatch, "0") + ExecCommitWaitBinlog = metrics.ExecPhaseDuration.WithLabelValues(PhaseCommitWaitBinlog, "0") + ExecWriteResponse = metrics.ExecPhaseDuration.WithLabelValues(PhaseWriteResponse, "0") + ExecUnknown = metrics.ExecPhaseDuration.WithLabelValues("unknown", "0") + + 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") + + TransactionDurationPessimisticRollbackInternal = metrics.TransactionDuration.WithLabelValues(metrics.LblPessimistic, metrics.LblRollback, metrics.LblInternal) + TransactionDurationPessimisticRollbackGeneral = metrics.TransactionDuration.WithLabelValues(metrics.LblPessimistic, metrics.LblRollback, metrics.LblGeneral) + TransactionDurationOptimisticRollbackInternal = metrics.TransactionDuration.WithLabelValues(metrics.LblOptimistic, metrics.LblRollback, metrics.LblInternal) + TransactionDurationOptimisticRollbackGeneral = metrics.TransactionDuration.WithLabelValues(metrics.LblOptimistic, metrics.LblRollback, metrics.LblGeneral) +} + +// InitPhaseDurationObserverMap init observer map +func InitPhaseDurationObserverMap() { + 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, + } +} diff --git a/executor/simple.go b/executor/simple.go index 392007135ed37..3e0bb573fe96a 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -32,10 +32,10 @@ import ( "github.com/pingcap/tidb/distsql" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/errno" + executor_metrics "github.com/pingcap/tidb/executor/metrics" "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/auth" "github.com/pingcap/tidb/parser/model" @@ -65,13 +65,6 @@ import ( const notSpecified = -1 -var ( - transactionDurationPessimisticRollbackInternal = metrics.TransactionDuration.WithLabelValues(metrics.LblPessimistic, metrics.LblRollback, metrics.LblInternal) - transactionDurationPessimisticRollbackGeneral = metrics.TransactionDuration.WithLabelValues(metrics.LblPessimistic, metrics.LblRollback, metrics.LblGeneral) - transactionDurationOptimisticRollbackInternal = metrics.TransactionDuration.WithLabelValues(metrics.LblOptimistic, metrics.LblRollback, metrics.LblInternal) - transactionDurationOptimisticRollbackGeneral = metrics.TransactionDuration.WithLabelValues(metrics.LblOptimistic, metrics.LblRollback, metrics.LblGeneral) -) - // SimpleExec represents simple statement executor. // For statements do simple execution. // includes `UseStmt`, 'SetStmt`, `DoStmt`, @@ -821,13 +814,13 @@ func (e *SimpleExec) executeRollback(s *ast.RollbackStmt) error { isInternal = true } if isInternal && sessVars.TxnCtx.IsPessimistic { - transactionDurationPessimisticRollbackInternal.Observe(duration) + executor_metrics.TransactionDurationPessimisticRollbackInternal.Observe(duration) } else if isInternal && !sessVars.TxnCtx.IsPessimistic { - transactionDurationOptimisticRollbackInternal.Observe(duration) + executor_metrics.TransactionDurationOptimisticRollbackInternal.Observe(duration) } else if !isInternal && sessVars.TxnCtx.IsPessimistic { - transactionDurationPessimisticRollbackGeneral.Observe(duration) + executor_metrics.TransactionDurationPessimisticRollbackGeneral.Observe(duration) } else if !isInternal && !sessVars.TxnCtx.IsPessimistic { - transactionDurationOptimisticRollbackGeneral.Observe(duration) + executor_metrics.TransactionDurationOptimisticRollbackGeneral.Observe(duration) } sessVars.TxnCtx.ClearDelta() return txn.Rollback() diff --git a/infoschema/BUILD.bazel b/infoschema/BUILD.bazel index 20a4f150e4a27..5240f2afe4e94 100644 --- a/infoschema/BUILD.bazel +++ b/infoschema/BUILD.bazel @@ -20,10 +20,10 @@ go_library( "//ddl/resourcegroup", "//domain/infosync", "//errno", + "//infoschema/metrics", "//kv", "//meta", "//meta/autoid", - "//metrics", "//parser/charset", "//parser/model", "//parser/mysql", diff --git a/infoschema/cache.go b/infoschema/cache.go index 34cc08eca2231..24c16f88b7a32 100644 --- a/infoschema/cache.go +++ b/infoschema/cache.go @@ -18,17 +18,7 @@ import ( "sort" "sync" - "github.com/pingcap/tidb/metrics" -) - -var ( - getLatestCounter = metrics.InfoCacheCounters.WithLabelValues("get", "latest") - getTSCounter = metrics.InfoCacheCounters.WithLabelValues("get", "ts") - getVersionCounter = metrics.InfoCacheCounters.WithLabelValues("get", "version") - - hitLatestCounter = metrics.InfoCacheCounters.WithLabelValues("hit", "latest") - hitTSCounter = metrics.InfoCacheCounters.WithLabelValues("hit", "ts") - hitVersionCounter = metrics.InfoCacheCounters.WithLabelValues("hit", "version") + infoschema_metrics "github.com/pingcap/tidb/infoschema/metrics" ) // InfoCache handles information schema, including getting and setting. @@ -58,9 +48,9 @@ func (h *InfoCache) Reset(capacity int) { func (h *InfoCache) GetLatest() InfoSchema { h.mu.RLock() defer h.mu.RUnlock() - getLatestCounter.Inc() + infoschema_metrics.GetLatestCounter.Inc() if len(h.cache) > 0 { - hitLatestCounter.Inc() + infoschema_metrics.HitLatestCounter.Inc() return h.cache[0] } return nil @@ -70,7 +60,7 @@ func (h *InfoCache) GetLatest() InfoSchema { func (h *InfoCache) GetByVersion(version int64) InfoSchema { h.mu.RLock() defer h.mu.RUnlock() - getVersionCounter.Inc() + infoschema_metrics.GetVersionCounter.Inc() i := sort.Search(len(h.cache), func(i int) bool { return h.cache[i].SchemaMetaVersion() <= version }) @@ -94,7 +84,7 @@ func (h *InfoCache) GetByVersion(version int64) InfoSchema { // ``` if i < len(h.cache) && (i != 0 || h.cache[i].SchemaMetaVersion() == version) { - hitVersionCounter.Inc() + infoschema_metrics.HitVersionCounter.Inc() return h.cache[i] } return nil @@ -107,10 +97,10 @@ func (h *InfoCache) GetBySnapshotTS(snapshotTS uint64) InfoSchema { h.mu.RLock() defer h.mu.RUnlock() - getTSCounter.Inc() + infoschema_metrics.GetTSCounter.Inc() if snapshotTS >= h.maxUpdatedSnapshotTS { if len(h.cache) > 0 { - hitTSCounter.Inc() + infoschema_metrics.HitTSCounter.Inc() return h.cache[0] } } diff --git a/infoschema/metrics/BUILD.bazel b/infoschema/metrics/BUILD.bazel new file mode 100644 index 0000000000000..7a6812a20032a --- /dev/null +++ b/infoschema/metrics/BUILD.bazel @@ -0,0 +1,12 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "metrics", + srcs = ["metrics.go"], + importpath = "github.com/pingcap/tidb/infoschema/metrics", + visibility = ["//visibility:public"], + deps = [ + "//metrics", + "@com_github_prometheus_client_golang//prometheus", + ], +) diff --git a/infoschema/metrics/metrics.go b/infoschema/metrics/metrics.go new file mode 100644 index 0000000000000..994a28a228f44 --- /dev/null +++ b/infoschema/metrics/metrics.go @@ -0,0 +1,46 @@ +// Copyright 2023 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package metrics + +import ( + "github.com/pingcap/tidb/metrics" + "github.com/prometheus/client_golang/prometheus" +) + +// infoschema metrics vars +var ( + GetLatestCounter prometheus.Counter + GetTSCounter prometheus.Counter + GetVersionCounter prometheus.Counter + + HitLatestCounter prometheus.Counter + HitTSCounter prometheus.Counter + HitVersionCounter prometheus.Counter +) + +func init() { + InitMetricsVars() +} + +// InitMetricsVars init infoschema metrics vars. +func InitMetricsVars() { + GetLatestCounter = metrics.InfoCacheCounters.WithLabelValues("get", "latest") + GetTSCounter = metrics.InfoCacheCounters.WithLabelValues("get", "ts") + GetVersionCounter = metrics.InfoCacheCounters.WithLabelValues("get", "version") + + HitLatestCounter = metrics.InfoCacheCounters.WithLabelValues("hit", "latest") + HitTSCounter = metrics.InfoCacheCounters.WithLabelValues("hit", "ts") + HitVersionCounter = metrics.InfoCacheCounters.WithLabelValues("hit", "version") +} diff --git a/metrics/BUILD.bazel b/metrics/BUILD.bazel index c414e04d9c907..3457da50b9a08 100644 --- a/metrics/BUILD.bazel +++ b/metrics/BUILD.bazel @@ -21,6 +21,7 @@ go_library( "telemetry.go", "topsql.go", "ttl.go", + "wrapper.go", ], importpath = "github.com/pingcap/tidb/metrics", visibility = ["//visibility:public"], diff --git a/metrics/bindinfo.go b/metrics/bindinfo.go index 47722b1502322..0c579e4857076 100644 --- a/metrics/bindinfo.go +++ b/metrics/bindinfo.go @@ -18,7 +18,14 @@ import "github.com/prometheus/client_golang/prometheus" // bindinfo metrics. var ( - BindUsageCounter = prometheus.NewCounterVec( + BindUsageCounter *prometheus.CounterVec + BindTotalGauge *prometheus.GaugeVec + BindMemoryUsage *prometheus.GaugeVec +) + +// InitBindInfoMetrics initializes bindinfo metrics. +func InitBindInfoMetrics() { + BindUsageCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "bindinfo", @@ -26,7 +33,7 @@ var ( Help: "Counter of query using sql bind", }, []string{LabelScope}) - BindTotalGauge = prometheus.NewGaugeVec( + BindTotalGauge = NewGaugeVec( prometheus.GaugeOpts{ Namespace: "tidb", Subsystem: "bindinfo", @@ -34,11 +41,11 @@ var ( Help: "Total number of sql bind", }, []string{LabelScope, LblType}) - BindMemoryUsage = prometheus.NewGaugeVec( + BindMemoryUsage = NewGaugeVec( prometheus.GaugeOpts{ Namespace: "tidb", Subsystem: "bindinfo", Name: "bind_memory_usage", Help: "Memory usage of sql bind", }, []string{LabelScope, LblType}) -) +} diff --git a/metrics/ddl.go b/metrics/ddl.go index c97077ed0aea0..1289957524f35 100644 --- a/metrics/ddl.go +++ b/metrics/ddl.go @@ -22,7 +22,44 @@ import ( // Metrics for the DDL package. var ( - JobsGauge = prometheus.NewGaugeVec( + JobsGauge *prometheus.GaugeVec + HandleJobHistogram *prometheus.HistogramVec + BatchAddIdxHistogram *prometheus.HistogramVec + + SyncerInit = "init" + SyncerRestart = "restart" + SyncerClear = "clear" + SyncerRewatch = "rewatch" + + DeploySyncerHistogram *prometheus.HistogramVec + UpdateSelfVersionHistogram *prometheus.HistogramVec + + OwnerUpdateGlobalVersion = "update_global_version" + OwnerCheckAllVersions = "check_all_versions" + OwnerHandleSyncerHistogram *prometheus.HistogramVec + + // Metrics for ddl_worker.go. + WorkerNotifyDDLJob = "notify_job" + WorkerAddDDLJob = "add_job" + WorkerRunDDLJob = "run_job" + WorkerFinishDDLJob = "finish_job" + WorkerWaitSchemaChanged = "wait_schema_changed" + DDLWorkerHistogram *prometheus.HistogramVec + + CreateDDLInstance = "create_ddl_instance" + CreateDDL = "create_ddl" + DDLOwner = "owner" + DDLCounter *prometheus.CounterVec + + BackfillTotalCounter *prometheus.CounterVec + BackfillProgressGauge *prometheus.GaugeVec + DDLJobTableDuration *prometheus.HistogramVec + DDLRunningJobCount *prometheus.GaugeVec +) + +// InitDDLMetrics initializes defines DDL metrics. +func InitDDLMetrics() { + JobsGauge = NewGaugeVec( prometheus.GaugeOpts{ Namespace: "tidb", Subsystem: "ddl", @@ -30,7 +67,7 @@ var ( Help: "Gauge of jobs.", }, []string{LblType}) - HandleJobHistogram = prometheus.NewHistogramVec( + HandleJobHistogram = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "ddl", @@ -39,7 +76,7 @@ var ( Buckets: prometheus.ExponentialBuckets(0.01, 2, 24), // 10ms ~ 24hours }, []string{LblType, LblResult}) - BatchAddIdxHistogram = prometheus.NewHistogramVec( + BatchAddIdxHistogram = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "ddl", @@ -48,11 +85,7 @@ var ( Buckets: prometheus.ExponentialBuckets(0.001, 2, 28), // 1ms ~ 1.5days }, []string{LblType}) - SyncerInit = "init" - SyncerRestart = "restart" - SyncerClear = "clear" - SyncerRewatch = "rewatch" - DeploySyncerHistogram = prometheus.NewHistogramVec( + DeploySyncerHistogram = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "ddl", @@ -61,7 +94,7 @@ var ( Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 524s }, []string{LblType, LblResult}) - UpdateSelfVersionHistogram = prometheus.NewHistogramVec( + UpdateSelfVersionHistogram = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "ddl", @@ -70,9 +103,7 @@ var ( Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 524s }, []string{LblResult}) - OwnerUpdateGlobalVersion = "update_global_version" - OwnerCheckAllVersions = "check_all_versions" - OwnerHandleSyncerHistogram = prometheus.NewHistogramVec( + OwnerHandleSyncerHistogram = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "ddl", @@ -81,13 +112,7 @@ var ( Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 524s }, []string{LblType, LblResult}) - // Metrics for ddl_worker.go. - WorkerNotifyDDLJob = "notify_job" - WorkerAddDDLJob = "add_job" - WorkerRunDDLJob = "run_job" - WorkerFinishDDLJob = "finish_job" - WorkerWaitSchemaChanged = "wait_schema_changed" - DDLWorkerHistogram = prometheus.NewHistogramVec( + DDLWorkerHistogram = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "ddl", @@ -96,10 +121,7 @@ var ( Buckets: prometheus.ExponentialBuckets(0.001, 2, 28), // 1ms ~ 1.5days }, []string{LblType, LblAction, LblResult}) - CreateDDLInstance = "create_ddl_instance" - CreateDDL = "create_ddl" - DDLOwner = "owner" - DDLCounter = prometheus.NewCounterVec( + DDLCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "ddl", @@ -107,7 +129,7 @@ var ( Help: "Counter of creating ddl/worker and isowner.", }, []string{LblType}) - BackfillTotalCounter = prometheus.NewCounterVec( + BackfillTotalCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "ddl", @@ -115,7 +137,7 @@ var ( Help: "Speed of add index", }, []string{LblType}) - BackfillProgressGauge = prometheus.NewGaugeVec( + BackfillProgressGauge = NewGaugeVec( prometheus.GaugeOpts{ Namespace: "tidb", Subsystem: "ddl", @@ -123,7 +145,7 @@ var ( Help: "Percentage progress of backfill", }, []string{LblType}) - DDLJobTableDuration = prometheus.NewHistogramVec(prometheus.HistogramOpts{ + DDLJobTableDuration = NewHistogramVec(prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "ddl", Name: "job_table_duration_seconds", @@ -131,14 +153,14 @@ var ( Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 524s }, []string{LblType}) - DDLRunningJobCount = prometheus.NewGaugeVec( + DDLRunningJobCount = NewGaugeVec( prometheus.GaugeOpts{ Namespace: "tidb", Subsystem: "ddl", Name: "running_job_count", Help: "Running DDL jobs count", }, []string{LblType}) -) +} // Label constants. const ( diff --git a/metrics/distsql.go b/metrics/distsql.go index aecc70791b483..b161d5d0dfa32 100644 --- a/metrics/distsql.go +++ b/metrics/distsql.go @@ -20,7 +20,18 @@ import ( // distsql metrics. var ( - DistSQLQueryHistogram = prometheus.NewHistogramVec( + DistSQLQueryHistogram *prometheus.HistogramVec + DistSQLScanKeysPartialHistogram prometheus.Histogram + DistSQLScanKeysHistogram prometheus.Histogram + DistSQLPartialCountHistogram prometheus.Histogram + DistSQLCoprCacheCounter *prometheus.CounterVec + DistSQLCoprClosestReadCounter *prometheus.CounterVec + DistSQLCoprRespBodySize *prometheus.HistogramVec +) + +// InitDistSQLMetrics initializes distsql metrics. +func InitDistSQLMetrics() { + DistSQLQueryHistogram = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "distsql", @@ -29,7 +40,7 @@ var ( Buckets: prometheus.ExponentialBuckets(0.0005, 2, 29), // 0.5ms ~ 1.5days }, []string{LblType, LblSQLType, LblCoprType}) - DistSQLScanKeysPartialHistogram = prometheus.NewHistogram( + DistSQLScanKeysPartialHistogram = NewHistogram( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "distsql", @@ -37,7 +48,8 @@ var ( Help: "number of scanned keys for each partial result.", }, ) - DistSQLScanKeysHistogram = prometheus.NewHistogram( + + DistSQLScanKeysHistogram = NewHistogram( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "distsql", @@ -45,7 +57,8 @@ var ( Help: "number of scanned keys for each query.", }, ) - DistSQLPartialCountHistogram = prometheus.NewHistogram( + + DistSQLPartialCountHistogram = NewHistogram( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "distsql", @@ -53,21 +66,24 @@ var ( Help: "number of partial results for each query.", }, ) - DistSQLCoprCacheCounter = prometheus.NewCounterVec( + + DistSQLCoprCacheCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "distsql", Name: "copr_cache", Help: "coprocessor cache hit, evict and miss number", }, []string{LblType}) - DistSQLCoprClosestReadCounter = prometheus.NewCounterVec( + + DistSQLCoprClosestReadCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "distsql", Name: "copr_closest_read", Help: "counter of total copr read local read hit.", }, []string{LblType}) - DistSQLCoprRespBodySize = prometheus.NewHistogramVec( + + DistSQLCoprRespBodySize = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "distsql", @@ -75,4 +91,4 @@ var ( Help: "copr task response data size in bytes.", Buckets: prometheus.ExponentialBuckets(1024, 2, 20), }, []string{LblStore}) -) +} diff --git a/metrics/domain.go b/metrics/domain.go index ab6d9df5ae0d8..fe92653dcf96f 100644 --- a/metrics/domain.go +++ b/metrics/domain.go @@ -21,7 +21,37 @@ import ( // Metrics for the domain package. var ( // LoadSchemaCounter records the counter of load schema. - LoadSchemaCounter = prometheus.NewCounterVec( + LoadSchemaCounter *prometheus.CounterVec + + // LoadSchemaDuration records the duration of load schema. + LoadSchemaDuration prometheus.Histogram + + // InfoCacheCounters are the counters of get/hit. + InfoCacheCounters *prometheus.CounterVec + + // InfoCacheCounterGet is the total number of getting entry. + InfoCacheCounterGet = "get" + // InfoCacheCounterHit is the cache hit numbers for get. + InfoCacheCounterHit = "hit" + + // LoadPrivilegeCounter records the counter of load privilege. + LoadPrivilegeCounter *prometheus.CounterVec + + // LoadSysVarCacheCounter records the counter of loading sysvars + LoadSysVarCacheCounter *prometheus.CounterVec + + SchemaValidatorStop = "stop" + SchemaValidatorRestart = "restart" + SchemaValidatorReset = "reset" + SchemaValidatorCacheEmpty = "cache_empty" + SchemaValidatorCacheMiss = "cache_miss" + // HandleSchemaValidate records the counter of handling schema validate. + HandleSchemaValidate *prometheus.CounterVec +) + +// InitDomainMetrics initializes domain metrics. +func InitDomainMetrics() { + LoadSchemaCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "domain", @@ -29,8 +59,7 @@ var ( Help: "Counter of load schema", }, []string{LblType}) - // LoadSchemaDuration records the duration of load schema. - LoadSchemaDuration = prometheus.NewHistogram( + LoadSchemaDuration = NewHistogram( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "domain", @@ -39,21 +68,15 @@ var ( Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 524s }) - // InfoCacheCounters are the counters of get/hit. - InfoCacheCounters = prometheus.NewCounterVec( + InfoCacheCounters = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "domain", Name: "infocache_counters", Help: "Counters of infoCache: get/hit.", }, []string{LblAction, LblType}) - // InfoCacheCounterGet is the total number of getting entry. - InfoCacheCounterGet = "get" - // InfoCacheCounterHit is the cache hit numbers for get. - InfoCacheCounterHit = "hit" - // LoadPrivilegeCounter records the counter of load privilege. - LoadPrivilegeCounter = prometheus.NewCounterVec( + LoadPrivilegeCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "domain", @@ -61,8 +84,7 @@ var ( Help: "Counter of load privilege", }, []string{LblType}) - // LoadSysVarCacheCounter records the counter of loading sysvars - LoadSysVarCacheCounter = prometheus.NewCounterVec( + LoadSysVarCacheCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "domain", @@ -70,17 +92,11 @@ var ( Help: "Counter of load sysvar cache", }, []string{LblType}) - SchemaValidatorStop = "stop" - SchemaValidatorRestart = "restart" - SchemaValidatorReset = "reset" - SchemaValidatorCacheEmpty = "cache_empty" - SchemaValidatorCacheMiss = "cache_miss" - // HandleSchemaValidate records the counter of handling schema validate. - HandleSchemaValidate = prometheus.NewCounterVec( + HandleSchemaValidate = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "domain", Name: "handle_schema_validate", Help: "Counter of handle schema validate", }, []string{LblType}) -) +} diff --git a/metrics/executor.go b/metrics/executor.go index 6edd6e003caae..8ad3604b8b013 100644 --- a/metrics/executor.go +++ b/metrics/executor.go @@ -20,7 +20,21 @@ import ( var ( // ExecutorCounter records the number of expensive executors. - ExecutorCounter = prometheus.NewCounterVec( + ExecutorCounter *prometheus.CounterVec + + // StmtNodeCounter records the number of statement with the same type. + StmtNodeCounter *prometheus.CounterVec + + // DbStmtNodeCounter records the number of statement with the same type and db. + DbStmtNodeCounter *prometheus.CounterVec + + // ExecPhaseDuration records the duration of each execution phase. + ExecPhaseDuration *prometheus.SummaryVec +) + +// InitExecutorMetrics initializes excutor metrics. +func InitExecutorMetrics() { + ExecutorCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "executor", @@ -29,8 +43,7 @@ var ( }, []string{LblType}, ) - // StmtNodeCounter records the number of statement with the same type. - StmtNodeCounter = prometheus.NewCounterVec( + StmtNodeCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "executor", @@ -38,8 +51,7 @@ var ( Help: "Counter of StmtNode.", }, []string{LblType, LblDb}) - // DbStmtNodeCounter records the number of statement with the same type and db. - DbStmtNodeCounter = prometheus.NewCounterVec( + DbStmtNodeCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "executor", @@ -47,12 +59,11 @@ var ( Help: "Counter of StmtNode by Database.", }, []string{LblDb, LblType}) - // ExecPhaseDuration records the duration of each execution phase. - ExecPhaseDuration = prometheus.NewSummaryVec( + ExecPhaseDuration = NewSummaryVec( prometheus.SummaryOpts{ Namespace: "tidb", Subsystem: "executor", Name: "phase_duration_seconds", Help: "Summary of each execution phase duration.", }, []string{LblPhase, LblInternal}) -) +} diff --git a/metrics/gc_worker.go b/metrics/gc_worker.go index 7c136401ede6e..ef3bc21185364 100644 --- a/metrics/gc_worker.go +++ b/metrics/gc_worker.go @@ -21,7 +21,18 @@ import ( // Metrics for the GC worker. var ( - GCWorkerCounter = prometheus.NewCounterVec( + GCWorkerCounter *prometheus.CounterVec + GCHistogram *prometheus.HistogramVec + GCConfigGauge *prometheus.GaugeVec + GCJobFailureCounter *prometheus.CounterVec + GCActionRegionResultCounter *prometheus.CounterVec + GCRegionTooManyLocksCounter prometheus.Counter + GCUnsafeDestroyRangeFailuresCounterVec *prometheus.CounterVec +) + +// InitGCWorkerMetrics initializes GC worker metrics. +func InitGCWorkerMetrics() { + GCWorkerCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "tikvclient", @@ -29,7 +40,7 @@ var ( Help: "Counter of gc worker actions.", }, []string{"type"}) - GCHistogram = prometheus.NewHistogramVec( + GCHistogram = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "tikvclient", @@ -38,7 +49,7 @@ var ( Buckets: prometheus.ExponentialBuckets(1, 2, 20), // 1s ~ 6days }, []string{"stage"}) - GCConfigGauge = prometheus.NewGaugeVec( + GCConfigGauge = NewGaugeVec( prometheus.GaugeOpts{ Namespace: "tidb", Subsystem: "tikvclient", @@ -46,7 +57,7 @@ var ( Help: "Gauge of GC configs.", }, []string{"type"}) - GCJobFailureCounter = prometheus.NewCounterVec( + GCJobFailureCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "tikvclient", @@ -54,7 +65,7 @@ var ( Help: "Counter of gc job failure.", }, []string{"type"}) - GCActionRegionResultCounter = prometheus.NewCounterVec( + GCActionRegionResultCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "tikvclient", @@ -62,16 +73,14 @@ var ( Help: "Counter of gc action result on region level.", }, []string{"type"}) - GCRegionTooManyLocksCounter = prometheus.NewCounter( + GCRegionTooManyLocksCounter = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "tikvclient", Name: "gc_region_too_many_locks", Help: "Counter of gc scan lock request more than once in the same region.", }) - - GCUnsafeDestroyRangeFailuresCounterVec *prometheus.CounterVec -) +} func init() { GCUnsafeDestroyRangeFailuresCounterVec = metrics.TiKVUnsafeDestroyRangeFailuresCounterVec diff --git a/metrics/log_backup.go b/metrics/log_backup.go index d9fc17d42d74c..e11c82285b54e 100644 --- a/metrics/log_backup.go +++ b/metrics/log_backup.go @@ -21,50 +21,67 @@ import ( // log backup metrics. // see the `Help` field for details. var ( - LastCheckpoint = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + LastCheckpoint *prometheus.GaugeVec + AdvancerOwner prometheus.Gauge + AdvancerTickDuration *prometheus.HistogramVec + GetCheckpointBatchSize *prometheus.HistogramVec + RegionCheckpointRequest *prometheus.CounterVec + RegionCheckpointFailure *prometheus.CounterVec + RegionCheckpointSubscriptionEvent *prometheus.HistogramVec +) + +// InitLogBackupMetrics initializes log backup metrics. +func InitLogBackupMetrics() { + LastCheckpoint = NewGaugeVec(prometheus.GaugeOpts{ Namespace: "tidb", Subsystem: "log_backup", Name: "last_checkpoint", Help: "The last global checkpoint of log backup.", }, []string{"task"}) - AdvancerOwner = prometheus.NewGauge(prometheus.GaugeOpts{ + + AdvancerOwner = NewGauge(prometheus.GaugeOpts{ Namespace: "tidb", Subsystem: "log_backup", Name: "advancer_owner", Help: "If the node is the owner of advancers, set this to `1`, otherwise `0`.", ConstLabels: map[string]string{}, }) - AdvancerTickDuration = prometheus.NewHistogramVec(prometheus.HistogramOpts{ + + AdvancerTickDuration = NewHistogramVec(prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "log_backup", Name: "advancer_tick_duration_sec", Help: "The time cost of each step during advancer ticking.", Buckets: prometheus.ExponentialBuckets(0.01, 3.0, 8), }, []string{"step"}) - GetCheckpointBatchSize = prometheus.NewHistogramVec(prometheus.HistogramOpts{ + + GetCheckpointBatchSize = NewHistogramVec(prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "log_backup", Name: "advancer_batch_size", Help: "The batch size of scanning region or get region checkpoint.", Buckets: prometheus.ExponentialBuckets(1, 2.0, 12), }, []string{"type"}) - RegionCheckpointRequest = prometheus.NewCounterVec(prometheus.CounterOpts{ + + RegionCheckpointRequest = NewCounterVec(prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "log_backup", Name: "region_request", Help: "The failure / success stat requesting region checkpoints.", }, []string{"result"}) - RegionCheckpointFailure = prometheus.NewCounterVec(prometheus.CounterOpts{ + + RegionCheckpointFailure = NewCounterVec(prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "log_backup", Name: "region_request_failure", Help: "The failure reasons of requesting region checkpoints.", }, []string{"reason"}) - RegionCheckpointSubscriptionEvent = prometheus.NewHistogramVec(prometheus.HistogramOpts{ + + RegionCheckpointSubscriptionEvent = NewHistogramVec(prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "log_backup", Name: "region_checkpoint_event", Help: "The region flush event size.", Buckets: prometheus.ExponentialBuckets(8, 2.0, 12), }, []string{"store"}) -) +} diff --git a/metrics/meta.go b/metrics/meta.go index af967fe48a3bb..45cbaf2559ddd 100644 --- a/metrics/meta.go +++ b/metrics/meta.go @@ -23,7 +23,18 @@ var ( GlobalAutoID = "global" TableAutoIDAlloc = "alloc" TableAutoIDRebase = "rebase" - AutoIDHistogram = prometheus.NewHistogramVec( + AutoIDHistogram *prometheus.HistogramVec + + GetSchemaDiff = "get_schema_diff" + SetSchemaDiff = "set_schema_diff" + GetHistoryDDLJob = "get_history_ddl_job" + + MetaHistogram *prometheus.HistogramVec +) + +// InitMetaMetrics initializes meta metrics. +func InitMetaMetrics() { + AutoIDHistogram = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "autoid", @@ -32,11 +43,7 @@ var ( Buckets: prometheus.ExponentialBuckets(0.0005, 2, 29), // 0.5ms ~ 1.5days }, []string{LblType, LblResult}) - GetSchemaDiff = "get_schema_diff" - SetSchemaDiff = "set_schema_diff" - GetHistoryDDLJob = "get_history_ddl_job" - - MetaHistogram = prometheus.NewHistogramVec( + MetaHistogram = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "meta", @@ -44,4 +51,4 @@ var ( Help: "Bucketed histogram of processing time (s) of tidb meta data operations.", Buckets: prometheus.ExponentialBuckets(0.0005, 2, 29), // 0.5ms ~ 1.5days }, []string{LblType, LblResult}) -) +} diff --git a/metrics/metrics.go b/metrics/metrics.go index f63af624a3a0c..4c09bc153c405 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -26,22 +26,10 @@ import ( var ( // PanicCounter measures the count of panics. - PanicCounter = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Namespace: "tidb", - Subsystem: "server", - Name: "panic_total", - Help: "Counter of panic.", - }, []string{LblType}) + PanicCounter *prometheus.CounterVec // MemoryUsage measures the usage gauge of memory. - MemoryUsage = prometheus.NewGaugeVec( - prometheus.GaugeOpts{ - Namespace: "tidb", - Subsystem: "server", - Name: "memory_usage", - Help: "Memory Usage", - }, []string{LblModule, LblType}) + MemoryUsage *prometheus.GaugeVec ) // metrics labels. @@ -79,6 +67,47 @@ func RetLabel(err error) string { return opFailed } +func init() { + InitMetrics() +} + +// InitMetrics is used to initialize metrics. +func InitMetrics() { + InitBindInfoMetrics() + InitDDLMetrics() + InitDistSQLMetrics() + InitDomainMetrics() + InitExecutorMetrics() + InitGCWorkerMetrics() + InitLogBackupMetrics() + InitMetaMetrics() + InitOwnerMetrics() + InitResourceManagerMetrics() + InitServerMetrics() + InitSessionMetrics() + InitSliMetrics() + InitStatsMetrics() + InitTelemetryMetrics() + InitTopSQLMetrics() + InitTTLMetrics() + + PanicCounter = NewCounterVec( + prometheus.CounterOpts{ + Namespace: "tidb", + Subsystem: "server", + Name: "panic_total", + Help: "Counter of panic.", + }, []string{LblType}) + + MemoryUsage = NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "tidb", + Subsystem: "server", + Name: "memory_usage", + Help: "Memory Usage", + }, []string{LblModule, LblType}) +} + // RegisterMetrics registers the metrics which are ONLY used in TiDB server. func RegisterMetrics() { // use new go collector diff --git a/metrics/owner.go b/metrics/owner.go index 7f00b5816db92..0ddc26d321683 100644 --- a/metrics/owner.go +++ b/metrics/owner.go @@ -20,7 +20,22 @@ import ( // Metrics var ( - NewSessionHistogram = prometheus.NewHistogramVec( + NewSessionHistogram *prometheus.HistogramVec + + WatcherClosed = "watcher_closed" + Cancelled = "cancelled" + Deleted = "deleted" + SessionDone = "session_done" + CtxDone = "context_done" + WatchOwnerCounter *prometheus.CounterVec + + NoLongerOwner = "no_longer_owner" + CampaignOwnerCounter *prometheus.CounterVec +) + +// InitOwnerMetrics initializes owner metrics. +func InitOwnerMetrics() { + NewSessionHistogram = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "owner", @@ -29,12 +44,7 @@ var ( Buckets: prometheus.ExponentialBuckets(0.0005, 2, 22), // 0.5ms ~ 1048s }, []string{LblType, LblResult}) - WatcherClosed = "watcher_closed" - Cancelled = "cancelled" - Deleted = "deleted" - SessionDone = "session_done" - CtxDone = "context_done" - WatchOwnerCounter = prometheus.NewCounterVec( + WatchOwnerCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "owner", @@ -42,12 +52,11 @@ var ( Help: "Counter of watch owner.", }, []string{LblType, LblResult}) - NoLongerOwner = "no_longer_owner" - CampaignOwnerCounter = prometheus.NewCounterVec( + CampaignOwnerCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "owner", Name: "campaign_owner_total", Help: "Counter of campaign owner.", }, []string{LblType, LblResult}) -) +} diff --git a/metrics/resourcemanager.go b/metrics/resourcemanager.go index c6715512500d4..9a3b439454028 100644 --- a/metrics/resourcemanager.go +++ b/metrics/resourcemanager.go @@ -18,18 +18,26 @@ import "github.com/prometheus/client_golang/prometheus" var ( // EMACPUUsageGauge means exponential moving average of CPU usage - EMACPUUsageGauge = prometheus.NewGauge(prometheus.GaugeOpts{ + EMACPUUsageGauge prometheus.Gauge + + // PoolConcurrencyCounter means how much concurrency in the pool + PoolConcurrencyCounter *prometheus.GaugeVec +) + +// InitResourceManagerMetrics initializes resource manager metrics. +func InitResourceManagerMetrics() { + EMACPUUsageGauge = NewGauge(prometheus.GaugeOpts{ Namespace: "tidb", Subsystem: "rm", Name: "ema_cpu_usage", Help: "exponential moving average of CPU usage", }) - // PoolConcurrencyCounter means how much concurrency in the pool - PoolConcurrencyCounter = prometheus.NewGaugeVec( + + PoolConcurrencyCounter = NewGaugeVec( prometheus.GaugeOpts{ Namespace: "tidb", Subsystem: "rm", Name: "pool_concurrency", Help: "How many concurrency in the pool", }, []string{LblType}) -) +} diff --git a/metrics/server.go b/metrics/server.go index 5ff220ea9c612..bf6dbeed0517c 100644 --- a/metrics/server.go +++ b/metrics/server.go @@ -27,7 +27,53 @@ var ( // Metrics var ( - PacketIOCounter = prometheus.NewCounterVec( + PacketIOCounter *prometheus.CounterVec + QueryDurationHistogram *prometheus.HistogramVec + QueryTotalCounter *prometheus.CounterVec + AffectedRowsCounter *prometheus.CounterVec + ConnGauge prometheus.Gauge + DisconnectionCounter *prometheus.CounterVec + PreparedStmtGauge prometheus.Gauge + ExecuteErrorCounter *prometheus.CounterVec + CriticalErrorCounter prometheus.Counter + + EventStart = "start" + EventGracefulDown = "graceful_shutdown" + // Eventkill occurs when the server.Kill() function is called. + EventKill = "kill" + EventClose = "close" + + ServerEventCounter *prometheus.CounterVec + TimeJumpBackCounter prometheus.Counter + PlanCacheCounter *prometheus.CounterVec + PlanCacheMissCounter *prometheus.CounterVec + PlanCacheInstanceMemoryUsage *prometheus.GaugeVec + PlanCacheInstancePlanNumCounter *prometheus.GaugeVec + ReadFromTableCacheCounter prometheus.Counter + HandShakeErrorCounter prometheus.Counter + GetTokenDurationHistogram prometheus.Histogram + NumOfMultiQueryHistogram prometheus.Histogram + TotalQueryProcHistogram *prometheus.HistogramVec + TotalCopProcHistogram *prometheus.HistogramVec + TotalCopWaitHistogram *prometheus.HistogramVec + MaxProcs prometheus.Gauge + GOGC prometheus.Gauge + ConnIdleDurationHistogram *prometheus.HistogramVec + ServerInfo *prometheus.GaugeVec + TokenGauge prometheus.Gauge + ConfigStatus *prometheus.GaugeVec + TiFlashQueryTotalCounter *prometheus.CounterVec + TiFlashFailedMPPStoreState *prometheus.GaugeVec + PDAPIExecutionHistogram *prometheus.HistogramVec + PDAPIRequestCounter *prometheus.CounterVec + CPUProfileCounter prometheus.Counter + LoadTableCacheDurationHistogram prometheus.Histogram + RCCheckTSWriteConfilictCounter *prometheus.CounterVec +) + +// InitServerMetrics initializes server metrics. +func InitServerMetrics() { + PacketIOCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "server", @@ -35,7 +81,7 @@ var ( Help: "Counters of packet IO bytes.", }, []string{LblType}) - QueryDurationHistogram = prometheus.NewHistogramVec( + QueryDurationHistogram = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "server", @@ -44,7 +90,7 @@ var ( Buckets: prometheus.ExponentialBuckets(0.0005, 2, 29), // 0.5ms ~ 1.5days }, []string{LblSQLType, LblDb}) - QueryTotalCounter = prometheus.NewCounterVec( + QueryTotalCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "server", @@ -52,7 +98,7 @@ var ( Help: "Counter of queries.", }, []string{LblType, LblResult}) - AffectedRowsCounter = prometheus.NewCounterVec( + AffectedRowsCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "server", @@ -60,7 +106,7 @@ var ( Help: "Counters of server affected rows.", }, []string{LblSQLType}) - ConnGauge = prometheus.NewGauge( + ConnGauge = NewGauge( prometheus.GaugeOpts{ Namespace: "tidb", Subsystem: "server", @@ -68,7 +114,7 @@ var ( Help: "Number of connections.", }) - DisconnectionCounter = prometheus.NewCounterVec( + DisconnectionCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "server", @@ -76,14 +122,14 @@ var ( Help: "Counter of connections disconnected.", }, []string{LblResult}) - PreparedStmtGauge = prometheus.NewGauge(prometheus.GaugeOpts{ + PreparedStmtGauge = NewGauge(prometheus.GaugeOpts{ Namespace: "tidb", Subsystem: "server", Name: "prepared_stmts", Help: "number of prepared statements.", }) - ExecuteErrorCounter = prometheus.NewCounterVec( + ExecuteErrorCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "server", @@ -91,7 +137,7 @@ var ( Help: "Counter of execute errors.", }, []string{LblType, LblDb}) - CriticalErrorCounter = prometheus.NewCounter( + CriticalErrorCounter = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "server", @@ -99,12 +145,7 @@ var ( Help: "Counter of critical errors.", }) - EventStart = "start" - EventGracefulDown = "graceful_shutdown" - // Eventkill occurs when the server.Kill() function is called. - EventKill = "kill" - EventClose = "close" - ServerEventCounter = prometheus.NewCounterVec( + ServerEventCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "server", @@ -112,7 +153,7 @@ var ( Help: "Counter of tidb-server event.", }, []string{LblType}) - TimeJumpBackCounter = prometheus.NewCounter( + TimeJumpBackCounter = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "monitor", @@ -120,7 +161,7 @@ var ( Help: "Counter of system time jumps backward.", }) - PlanCacheCounter = prometheus.NewCounterVec( + PlanCacheCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "server", @@ -128,7 +169,7 @@ var ( Help: "Counter of query using plan cache.", }, []string{LblType}) - PlanCacheMissCounter = prometheus.NewCounterVec( + PlanCacheMissCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "server", @@ -136,7 +177,7 @@ var ( Help: "Counter of plan cache miss.", }, []string{LblType}) - PlanCacheInstanceMemoryUsage = prometheus.NewGaugeVec( + PlanCacheInstanceMemoryUsage = NewGaugeVec( prometheus.GaugeOpts{ Namespace: "tidb", Subsystem: "server", @@ -144,7 +185,7 @@ var ( Help: "Total plan cache memory usage of all sessions in a instance", }, []string{LblType}) - PlanCacheInstancePlanNumCounter = prometheus.NewGaugeVec( + PlanCacheInstancePlanNumCounter = NewGaugeVec( prometheus.GaugeOpts{ Namespace: "tidb", Subsystem: "server", @@ -152,7 +193,7 @@ var ( Help: "Counter of plan of all prepared plan cache in a instance", }, []string{LblType}) - ReadFromTableCacheCounter = prometheus.NewCounter( + ReadFromTableCacheCounter = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "server", @@ -161,7 +202,7 @@ var ( }, ) - HandShakeErrorCounter = prometheus.NewCounter( + HandShakeErrorCounter = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "server", @@ -170,7 +211,7 @@ var ( }, ) - GetTokenDurationHistogram = prometheus.NewHistogram( + GetTokenDurationHistogram = NewHistogram( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "server", @@ -179,7 +220,7 @@ var ( Buckets: prometheus.ExponentialBuckets(1, 2, 30), // 1us ~ 528s }) - NumOfMultiQueryHistogram = prometheus.NewHistogram( + NumOfMultiQueryHistogram = NewHistogram( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "server", @@ -188,7 +229,7 @@ var ( Buckets: prometheus.ExponentialBuckets(1, 2, 20), // 1 ~ 1048576 }) - TotalQueryProcHistogram = prometheus.NewHistogramVec( + TotalQueryProcHistogram = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "server", @@ -196,7 +237,8 @@ var ( Help: "Bucketed histogram of processing time (s) of of slow queries.", Buckets: prometheus.ExponentialBuckets(0.001, 2, 28), // 1ms ~ 1.5days }, []string{LblSQLType}) - TotalCopProcHistogram = prometheus.NewHistogramVec( + + TotalCopProcHistogram = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "server", @@ -204,7 +246,8 @@ var ( Help: "Bucketed histogram of all cop processing time (s) of of slow queries.", Buckets: prometheus.ExponentialBuckets(0.001, 2, 28), // 1ms ~ 1.5days }, []string{LblSQLType}) - TotalCopWaitHistogram = prometheus.NewHistogramVec( + + TotalCopWaitHistogram = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "server", @@ -213,7 +256,7 @@ var ( Buckets: prometheus.ExponentialBuckets(0.001, 2, 28), // 1ms ~ 1.5days }, []string{LblSQLType}) - MaxProcs = prometheus.NewGauge( + MaxProcs = NewGauge( prometheus.GaugeOpts{ Namespace: "tidb", Subsystem: "server", @@ -221,7 +264,7 @@ var ( Help: "The value of GOMAXPROCS.", }) - GOGC = prometheus.NewGauge( + GOGC = NewGauge( prometheus.GaugeOpts{ Namespace: "tidb", Subsystem: "server", @@ -229,7 +272,7 @@ var ( Help: "The value of GOGC", }) - ConnIdleDurationHistogram = prometheus.NewHistogramVec( + ConnIdleDurationHistogram = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "server", @@ -238,7 +281,7 @@ var ( Buckets: prometheus.ExponentialBuckets(0.0005, 2, 29), // 0.5ms ~ 1.5days }, []string{LblInTxn}) - ServerInfo = prometheus.NewGaugeVec( + ServerInfo = NewGaugeVec( prometheus.GaugeOpts{ Namespace: "tidb", Subsystem: "server", @@ -246,7 +289,7 @@ var ( Help: "Indicate the tidb server info, and the value is the start timestamp (s).", }, []string{LblVersion, LblHash}) - TokenGauge = prometheus.NewGauge( + TokenGauge = NewGauge( prometheus.GaugeOpts{ Namespace: "tidb", Subsystem: "server", @@ -255,7 +298,7 @@ var ( }, ) - ConfigStatus = prometheus.NewGaugeVec( + ConfigStatus = NewGaugeVec( prometheus.GaugeOpts{ Namespace: "tidb", Subsystem: "config", @@ -263,7 +306,7 @@ var ( Help: "Status of the TiDB server configurations.", }, []string{LblType}) - TiFlashQueryTotalCounter = prometheus.NewCounterVec( + TiFlashQueryTotalCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "server", @@ -271,7 +314,7 @@ var ( Help: "Counter of TiFlash queries.", }, []string{LblType, LblResult}) - TiFlashFailedMPPStoreState = prometheus.NewGaugeVec( + TiFlashFailedMPPStoreState = NewGaugeVec( prometheus.GaugeOpts{ Namespace: "tidb", Subsystem: "server", @@ -279,7 +322,7 @@ var ( Help: "Statues of failed tiflash mpp store,-1 means detector heartbeat,0 means reachable,1 means abnormal.", }, []string{LblAddress}) - PDAPIExecutionHistogram = prometheus.NewHistogramVec( + PDAPIExecutionHistogram = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "server", @@ -288,7 +331,7 @@ var ( Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 524s }, []string{LblType}) - PDAPIRequestCounter = prometheus.NewCounterVec( + PDAPIRequestCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "server", @@ -296,7 +339,7 @@ var ( Help: "Counter of the pd http api requests", }, []string{LblType, LblResult}) - CPUProfileCounter = prometheus.NewCounter( + CPUProfileCounter = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "server", @@ -304,7 +347,7 @@ var ( Help: "Counter of cpu profiling", }) - LoadTableCacheDurationHistogram = prometheus.NewHistogram( + LoadTableCacheDurationHistogram = NewHistogram( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "server", @@ -313,14 +356,14 @@ var ( Buckets: prometheus.ExponentialBuckets(1, 2, 30), // 1us ~ 528s }) - RCCheckTSWriteConfilictCounter = prometheus.NewCounterVec( + RCCheckTSWriteConfilictCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "server", Name: "rc_check_ts_conflict_total", Help: "Counter of WriteConflict caused by RCCheckTS.", }, []string{LblType}) -) +} // ExecuteErrorToLabel converts an execute error to label. func ExecuteErrorToLabel(err error) string { diff --git a/metrics/session.go b/metrics/session.go index 208092c11b087..eed4c49801696 100644 --- a/metrics/session.go +++ b/metrics/session.go @@ -18,7 +18,32 @@ import "github.com/prometheus/client_golang/prometheus" // Session metrics. var ( - AutoIDReqDuration = prometheus.NewHistogram( + AutoIDReqDuration prometheus.Histogram + SessionExecuteParseDuration *prometheus.HistogramVec + SessionExecuteCompileDuration *prometheus.HistogramVec + SessionExecuteRunDuration *prometheus.HistogramVec + SchemaLeaseErrorCounter *prometheus.CounterVec + SessionRetry *prometheus.HistogramVec + SessionRetryErrorCounter *prometheus.CounterVec + SessionRestrictedSQLCounter prometheus.Counter + StatementPerTransaction *prometheus.HistogramVec + TransactionDuration *prometheus.HistogramVec + StatementDeadlockDetectDuration prometheus.Histogram + StatementPessimisticRetryCount prometheus.Histogram + StatementLockKeysCount prometheus.Histogram + ValidateReadTSFromPDCount prometheus.Counter + NonTransactionalDMLCount *prometheus.CounterVec + TxnStatusEnteringCounter *prometheus.CounterVec + TxnDurationHistogram *prometheus.HistogramVec + LazyPessimisticUniqueCheckSetCount prometheus.Counter + PessimisticDMLDurationByAttempt *prometheus.HistogramVec + ResourceGroupQueryTotalCounter *prometheus.CounterVec + AggressiveLockingUsageCount *prometheus.CounterVec +) + +// InitSessionMetrics initializes session metrics. +func InitSessionMetrics() { + AutoIDReqDuration = NewHistogram( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "meta", @@ -27,7 +52,7 @@ var ( Buckets: prometheus.ExponentialBuckets(0.00004, 2, 28), // 40us ~ 1.5h }) - SessionExecuteParseDuration = prometheus.NewHistogramVec( + SessionExecuteParseDuration = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "session", @@ -35,7 +60,8 @@ var ( Help: "Bucketed histogram of processing time (s) in parse SQL.", Buckets: prometheus.ExponentialBuckets(0.00004, 2, 28), // 40us ~ 1.5h }, []string{LblSQLType}) - SessionExecuteCompileDuration = prometheus.NewHistogramVec( + + SessionExecuteCompileDuration = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "session", @@ -44,7 +70,8 @@ var ( // Build plan may execute the statement, or allocate table ID, so it might take a long time. Buckets: prometheus.ExponentialBuckets(0.00004, 2, 28), // 40us ~ 1.5h }, []string{LblSQLType}) - SessionExecuteRunDuration = prometheus.NewHistogramVec( + + SessionExecuteRunDuration = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "session", @@ -52,14 +79,16 @@ var ( Help: "Bucketed histogram of processing time (s) in running executor.", Buckets: prometheus.ExponentialBuckets(0.0001, 2, 30), // 100us ~ 15h }, []string{LblSQLType}) - SchemaLeaseErrorCounter = prometheus.NewCounterVec( + + SchemaLeaseErrorCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "session", Name: "schema_lease_error_total", Help: "Counter of schema lease error", }, []string{LblType}) - SessionRetry = prometheus.NewHistogramVec( + + SessionRetry = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "session", @@ -67,7 +96,8 @@ var ( Help: "Bucketed histogram of session retry count.", Buckets: prometheus.LinearBuckets(0, 1, 21), // 0 ~ 20 }, []string{LblScope}) - SessionRetryErrorCounter = prometheus.NewCounterVec( + + SessionRetryErrorCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "session", @@ -75,7 +105,7 @@ var ( Help: "Counter of session retry error.", }, []string{LblSQLType, LblType}) - SessionRestrictedSQLCounter = prometheus.NewCounter( + SessionRestrictedSQLCounter = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "session", @@ -83,7 +113,7 @@ var ( Help: "Counter of internal restricted sql.", }) - StatementPerTransaction = prometheus.NewHistogramVec( + StatementPerTransaction = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "session", @@ -92,7 +122,7 @@ var ( Buckets: prometheus.ExponentialBuckets(1, 2, 16), // 1 ~ 32768 }, []string{LblTxnMode, LblType, LblScope}) - TransactionDuration = prometheus.NewHistogramVec( + TransactionDuration = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "session", @@ -101,7 +131,7 @@ var ( Buckets: prometheus.ExponentialBuckets(0.001, 2, 28), // 1ms ~ 1.5days }, []string{LblTxnMode, LblType, LblScope}) - StatementDeadlockDetectDuration = prometheus.NewHistogram( + StatementDeadlockDetectDuration = NewHistogram( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "session", @@ -111,7 +141,7 @@ var ( }, ) - StatementPessimisticRetryCount = prometheus.NewHistogram( + StatementPessimisticRetryCount = NewHistogram( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "session", @@ -120,7 +150,7 @@ var ( Buckets: prometheus.ExponentialBuckets(1, 2, 16), // 1 ~ 32768 }) - StatementLockKeysCount = prometheus.NewHistogram( + StatementLockKeysCount = NewHistogram( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "session", @@ -129,7 +159,7 @@ var ( Buckets: prometheus.ExponentialBuckets(1, 2, 21), // 1 ~ 1048576 }) - ValidateReadTSFromPDCount = prometheus.NewCounter( + ValidateReadTSFromPDCount = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "session", @@ -137,7 +167,7 @@ var ( Help: "Counter of validating read ts by getting a timestamp from PD", }) - NonTransactionalDMLCount = prometheus.NewCounterVec( + NonTransactionalDMLCount = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "session", @@ -145,7 +175,8 @@ var ( Help: "Counter of non-transactional delete", }, []string{LblType}, ) - TxnStatusEnteringCounter = prometheus.NewCounterVec( + + TxnStatusEnteringCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "session", @@ -153,7 +184,8 @@ var ( Help: "How many times transactions enter this state", }, []string{LblType}, ) - TxnDurationHistogram = prometheus.NewHistogramVec( + + TxnDurationHistogram = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "session", @@ -161,7 +193,8 @@ var ( Help: "Bucketed histogram of different states of a transaction.", Buckets: prometheus.ExponentialBuckets(0.0005, 2, 29), // 0.5ms ~ 1.5days }, []string{LblType, LblHasLock}) - LazyPessimisticUniqueCheckSetCount = prometheus.NewCounter( + + LazyPessimisticUniqueCheckSetCount = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "session", @@ -170,7 +203,7 @@ var ( }, ) - PessimisticDMLDurationByAttempt = prometheus.NewHistogramVec( + PessimisticDMLDurationByAttempt = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "session", @@ -179,7 +212,7 @@ var ( Buckets: prometheus.ExponentialBuckets(0.001, 2, 28), // 1ms ~ 1.5days }, []string{LblType, LblPhase}) - ResourceGroupQueryTotalCounter = prometheus.NewCounterVec( + ResourceGroupQueryTotalCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "session", @@ -187,14 +220,14 @@ var ( Help: "Counter of the total number of queries for the resource group", }, []string{LblName}) - AggressiveLockingUsageCount = prometheus.NewCounterVec( + AggressiveLockingUsageCount = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "server", Name: "transaction_aggressive_locking_usage", Help: "The counter of statements and transactions in which aggressive locking is used or takes effect", }, []string{LblType}) -) +} // Label constants. const ( diff --git a/metrics/sli.go b/metrics/sli.go index 381be0208957c..fd5cf1f62fe42 100644 --- a/metrics/sli.go +++ b/metrics/sli.go @@ -20,7 +20,14 @@ import ( var ( // SmallTxnWriteDuration uses to collect small transaction write duration. - SmallTxnWriteDuration = prometheus.NewHistogram( + SmallTxnWriteDuration prometheus.Histogram + // TxnWriteThroughput uses to collect transaction write throughput which transaction is not small. + TxnWriteThroughput prometheus.Histogram +) + +// InitSliMetrics initializes sli metrics. +func InitSliMetrics() { + SmallTxnWriteDuration = NewHistogram( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "sli", @@ -29,8 +36,7 @@ var ( Buckets: prometheus.ExponentialBuckets(0.001, 2, 28), // 1ms ~ 74h }) - // TxnWriteThroughput uses to collect transaction write throughput which transaction is not small. - TxnWriteThroughput = prometheus.NewHistogram( + TxnWriteThroughput = NewHistogram( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "sli", @@ -38,4 +44,4 @@ var ( Help: "Bucketed histogram of transaction write throughput (bytes/second).", Buckets: prometheus.ExponentialBuckets(64, 1.3, 40), // 64 bytes/s ~ 2.3MB/s }) -) +} diff --git a/metrics/stats.go b/metrics/stats.go index 5d73753f5669c..784b95a04c284 100644 --- a/metrics/stats.go +++ b/metrics/stats.go @@ -20,7 +20,31 @@ import ( // Stats metrics. var ( - AutoAnalyzeHistogram = prometheus.NewHistogram( + AutoAnalyzeHistogram prometheus.Histogram + AutoAnalyzeCounter *prometheus.CounterVec + StatsInaccuracyRate prometheus.Histogram + PseudoEstimation *prometheus.CounterVec + DumpFeedbackCounter *prometheus.CounterVec + UpdateStatsCounter *prometheus.CounterVec + StoreQueryFeedbackCounter *prometheus.CounterVec + SignificantFeedbackCounter prometheus.Counter + FastAnalyzeHistogram *prometheus.HistogramVec + SyncLoadCounter prometheus.Counter + SyncLoadTimeoutCounter prometheus.Counter + SyncLoadHistogram prometheus.Histogram + ReadStatsHistogram prometheus.Histogram + StatsCacheLRUCounter *prometheus.CounterVec + StatsCacheLRUGauge *prometheus.GaugeVec + StatsHealthyGauge *prometheus.GaugeVec + + HistoricalStatsCounter *prometheus.CounterVec + PlanReplayerTaskCounter *prometheus.CounterVec + PlanReplayerRegisterTaskGauge prometheus.Gauge +) + +// InitStatsMetrics initializes stats metrics. +func InitStatsMetrics() { + AutoAnalyzeHistogram = NewHistogram( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "statistics", @@ -29,7 +53,7 @@ var ( Buckets: prometheus.ExponentialBuckets(0.01, 2, 24), // 10ms ~ 24h }) - AutoAnalyzeCounter = prometheus.NewCounterVec( + AutoAnalyzeCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "statistics", @@ -37,7 +61,7 @@ var ( Help: "Counter of auto analyze.", }, []string{LblType}) - StatsInaccuracyRate = prometheus.NewHistogram( + StatsInaccuracyRate = NewHistogram( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "statistics", @@ -46,7 +70,7 @@ var ( Buckets: prometheus.ExponentialBuckets(0.01, 2, 14), }) - PseudoEstimation = prometheus.NewCounterVec( + PseudoEstimation = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "statistics", @@ -54,7 +78,7 @@ var ( Help: "Counter of pseudo estimation caused by outdated stats.", }, []string{LblType}) - DumpFeedbackCounter = prometheus.NewCounterVec( + DumpFeedbackCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "statistics", @@ -62,7 +86,7 @@ var ( Help: "Counter of dumping feedback.", }, []string{LblType}) - UpdateStatsCounter = prometheus.NewCounterVec( + UpdateStatsCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "statistics", @@ -70,7 +94,7 @@ var ( Help: "Counter of updating stats using feedback.", }, []string{LblType}) - StoreQueryFeedbackCounter = prometheus.NewCounterVec( + StoreQueryFeedbackCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "statistics", @@ -78,7 +102,7 @@ var ( Help: "Counter of storing query feedback.", }, []string{LblType}) - SignificantFeedbackCounter = prometheus.NewCounter( + SignificantFeedbackCounter = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "statistics", @@ -86,7 +110,7 @@ var ( Help: "Counter of query feedback whose actual count is much different than calculated by current statistics", }) - FastAnalyzeHistogram = prometheus.NewHistogramVec( + FastAnalyzeHistogram = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "statistics", @@ -95,7 +119,7 @@ var ( Buckets: prometheus.ExponentialBuckets(1, 2, 16), }, []string{LblSQLType, LblType}) - SyncLoadCounter = prometheus.NewCounter( + SyncLoadCounter = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "statistics", @@ -103,7 +127,7 @@ var ( Help: "Counter of sync load.", }) - SyncLoadTimeoutCounter = prometheus.NewCounter( + SyncLoadTimeoutCounter = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "statistics", @@ -111,7 +135,7 @@ var ( Help: "Counter of sync load timeout.", }) - SyncLoadHistogram = prometheus.NewHistogram( + SyncLoadHistogram = NewHistogram( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "statistics", @@ -120,7 +144,7 @@ var ( Buckets: prometheus.ExponentialBuckets(1, 2, 22), // 1ms ~ 1h }) - ReadStatsHistogram = prometheus.NewHistogram( + ReadStatsHistogram = NewHistogram( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "statistics", @@ -129,7 +153,7 @@ var ( Buckets: prometheus.ExponentialBuckets(1, 2, 22), // 1ms ~ 1h }) - StatsCacheLRUCounter = prometheus.NewCounterVec( + StatsCacheLRUCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "statistics", @@ -137,38 +161,38 @@ var ( Help: "Counter of lru for statsCache operation", }, []string{LblType}) - StatsCacheLRUGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + StatsCacheLRUGauge = NewGaugeVec(prometheus.GaugeOpts{ Namespace: "tidb", Subsystem: "statistics", Name: "stats_cache_lru_val", Help: "gauge of stats cache lru value", }, []string{LblType}) - StatsHealthyGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + StatsHealthyGauge = NewGaugeVec(prometheus.GaugeOpts{ Namespace: "tidb", Subsystem: "statistics", Name: "stats_healthy", Help: "Gauge of stats healthy", }, []string{LblType}) - HistoricalStatsCounter = prometheus.NewCounterVec(prometheus.CounterOpts{ + HistoricalStatsCounter = NewCounterVec(prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "statistics", Name: "historical_stats", Help: "counter of the historical stats operation", }, []string{LblType, LblResult}) - PlanReplayerTaskCounter = prometheus.NewCounterVec(prometheus.CounterOpts{ + PlanReplayerTaskCounter = NewCounterVec(prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "plan_replayer", Name: "task", Help: "counter of plan replayer captured task", }, []string{LblType, LblResult}) - PlanReplayerRegisterTaskGauge = prometheus.NewGauge(prometheus.GaugeOpts{ + PlanReplayerRegisterTaskGauge = NewGauge(prometheus.GaugeOpts{ Namespace: "tidb", Subsystem: "plan_replayer", Name: "register_task", Help: "gauge of plan replayer registered task", }) -) +} diff --git a/metrics/telemetry.go b/metrics/telemetry.go index f59ed87a07682..096478c6ace31 100644 --- a/metrics/telemetry.go +++ b/metrics/telemetry.go @@ -22,196 +22,253 @@ import ( // Metrics var ( - TelemetrySQLCTECnt = prometheus.NewCounterVec( + TelemetrySQLCTECnt *prometheus.CounterVec + TelemetryMultiSchemaChangeCnt prometheus.Counter + TelemetryTablePartitionCnt prometheus.Counter + TelemetryTablePartitionListCnt prometheus.Counter + TelemetryTablePartitionRangeCnt prometheus.Counter + TelemetryTablePartitionHashCnt prometheus.Counter + TelemetryTablePartitionRangeColumnsCnt prometheus.Counter + TelemetryTablePartitionRangeColumnsGt1Cnt prometheus.Counter + TelemetryTablePartitionRangeColumnsGt2Cnt prometheus.Counter + TelemetryTablePartitionRangeColumnsGt3Cnt prometheus.Counter + TelemetryTablePartitionListColumnsCnt prometheus.Counter + TelemetryTablePartitionMaxPartitionsCnt prometheus.Counter + TelemetryAccountLockCnt *prometheus.CounterVec + TelemetryTablePartitionCreateIntervalPartitionsCnt prometheus.Counter + TelemetryTablePartitionAddIntervalPartitionsCnt prometheus.Counter + TelemetryTablePartitionDropIntervalPartitionsCnt prometheus.Counter + TelemetryExchangePartitionCnt prometheus.Counter + TelemetryAddIndexIngestCnt prometheus.Counter + TelemetryFlashbackClusterCnt prometheus.Counter + TelemetryIndexMergeUsage prometheus.Counter + TelemetryCompactPartitionCnt prometheus.Counter + TelemetryReorganizePartitionCnt prometheus.Counter + TelemetryDistReorgCnt prometheus.Counter + TelemetryStoreBatchedQueryCnt prometheus.Counter + TelemetryBatchedQueryTaskCnt prometheus.Counter + TelemetryStoreBatchedCnt prometheus.Counter + TelemetryStoreBatchedFallbackCnt prometheus.Counter +) + +// InitTelemetryMetrics initializes telemetry metrics. +func InitTelemetryMetrics() { + TelemetrySQLCTECnt = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "telemetry", Name: "non_recursive_cte_usage", Help: "Counter of usage of CTE", }, []string{LblCTEType}) - TelemetryMultiSchemaChangeCnt = prometheus.NewCounter( + + TelemetryMultiSchemaChangeCnt = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "telemetry", Name: "multi_schema_change_usage", Help: "Counter of usage of multi-schema change", }) - TelemetryTablePartitionCnt = prometheus.NewCounter( + + TelemetryTablePartitionCnt = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "telemetry", Name: "table_partition_usage", Help: "Counter of CREATE TABLE which includes of table partitioning", }) - TelemetryTablePartitionListCnt = prometheus.NewCounter( + + TelemetryTablePartitionListCnt = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "telemetry", Name: "table_partition_list_usage", Help: "Counter of CREATE TABLE which includes LIST partitioning", }) - TelemetryTablePartitionRangeCnt = prometheus.NewCounter( + + TelemetryTablePartitionRangeCnt = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "telemetry", Name: "table_partition_range_usage", Help: "Counter of CREATE TABLE which includes RANGE partitioning", }) - TelemetryTablePartitionHashCnt = prometheus.NewCounter( + + TelemetryTablePartitionHashCnt = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "telemetry", Name: "table_partition_hash_usage", Help: "Counter of CREATE TABLE which includes HASH partitioning", }) - TelemetryTablePartitionRangeColumnsCnt = prometheus.NewCounter( + + TelemetryTablePartitionRangeColumnsCnt = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "telemetry", Name: "table_partition_range_columns_usage", Help: "Counter of CREATE TABLE which includes RANGE COLUMNS partitioning", }) - TelemetryTablePartitionRangeColumnsGt1Cnt = prometheus.NewCounter( + + TelemetryTablePartitionRangeColumnsGt1Cnt = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "telemetry", Name: "table_partition_range_multi_columns_usage", Help: "Counter of CREATE TABLE which includes RANGE COLUMNS partitioning with more than one partitioning column", }) - TelemetryTablePartitionRangeColumnsGt2Cnt = prometheus.NewCounter( + + TelemetryTablePartitionRangeColumnsGt2Cnt = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "telemetry", Name: "table_partition_range_multi_columns_usage", Help: "Counter of CREATE TABLE which includes RANGE COLUMNS partitioning with more than two partitioning columns", }) - TelemetryTablePartitionRangeColumnsGt3Cnt = prometheus.NewCounter( + + TelemetryTablePartitionRangeColumnsGt3Cnt = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "telemetry", Name: "table_partition_range_multi_columns_usage", Help: "Counter of CREATE TABLE which includes RANGE COLUMNS partitioning with more than three partitioning columns", }) - TelemetryTablePartitionListColumnsCnt = prometheus.NewCounter( + + TelemetryTablePartitionListColumnsCnt = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "telemetry", Name: "table_partition_list_columns_usage", Help: "Counter of CREATE TABLE which includes LIST COLUMNS partitioning", }) - TelemetryTablePartitionMaxPartitionsCnt = prometheus.NewCounter( + + TelemetryTablePartitionMaxPartitionsCnt = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "telemetry", Name: "table_partition_max_partition_usage", Help: "Counter of partitions created by CREATE TABLE statements", }) - TelemetryAccountLockCnt = prometheus.NewCounterVec( + + TelemetryAccountLockCnt = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "telemetry", Name: "account_lock_usage", Help: "Counter of locked/unlocked users", }, []string{LblAccountLock}) - TelemetryTablePartitionCreateIntervalPartitionsCnt = prometheus.NewCounter( + + TelemetryTablePartitionCreateIntervalPartitionsCnt = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "telemetry", Name: "table_partition_create_interval_partition_usage", Help: "Counter of partitions created by CREATE TABLE INTERVAL statements", }) - TelemetryTablePartitionAddIntervalPartitionsCnt = prometheus.NewCounter( + + TelemetryTablePartitionAddIntervalPartitionsCnt = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "telemetry", Name: "table_partition_add_interval_partition_usage", Help: "Counter of partitions added by ALTER TABLE LAST PARTITION statements", }) - TelemetryTablePartitionDropIntervalPartitionsCnt = prometheus.NewCounter( + + TelemetryTablePartitionDropIntervalPartitionsCnt = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "telemetry", Name: "table_partition_drop_interval_partition_usage", Help: "Counter of partitions added by ALTER TABLE FIRST PARTITION statements", }) - TelemetryExchangePartitionCnt = prometheus.NewCounter( + + TelemetryExchangePartitionCnt = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "telemetry", Name: "exchange_partition_usage", Help: "Counter of usage of exchange partition statements", }) - TelemetryAddIndexIngestCnt = prometheus.NewCounter( + + TelemetryAddIndexIngestCnt = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "telemetry", Name: "add_index_ingest_usage", Help: "Counter of usage of add index acceleration solution", }) - TelemetryFlashbackClusterCnt = prometheus.NewCounter( + + TelemetryFlashbackClusterCnt = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "telemetry", Name: "flashback_cluster_usage", Help: "Counter of usage of flashback cluster", }) - TelemetryIndexMergeUsage = prometheus.NewCounter( + + TelemetryIndexMergeUsage = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "telemetry", Name: "index_merge_usage", Help: "Counter of usage of index merge", }) - TelemetryCompactPartitionCnt = prometheus.NewCounter( + + TelemetryCompactPartitionCnt = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "telemetry", Name: "compact_partition_usage", Help: "Counter of compact table partition", }) - TelemetryReorganizePartitionCnt = prometheus.NewCounter( + + TelemetryReorganizePartitionCnt = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "telemetry", Name: "reorganize_partition_usage", Help: "Counter of alter table reorganize partition", }) - TelemetryDistReorgCnt = prometheus.NewCounter( + + TelemetryDistReorgCnt = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "telemetry", Name: "distributed_reorg_count", Help: "Counter of usage of distributed reorg DDL tasks count", }) - TelemetryStoreBatchedQueryCnt = prometheus.NewCounter( + + TelemetryStoreBatchedQueryCnt = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "telemetry", Name: "store_batched_query", Help: "Counter of queries which use store batched coprocessor tasks", }) - TelemetryBatchedQueryTaskCnt = prometheus.NewCounter( + + TelemetryBatchedQueryTaskCnt = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "telemetry", Name: "batched_query_task", Help: "Counter of coprocessor tasks in batched queries", }) - TelemetryStoreBatchedCnt = prometheus.NewCounter( + + TelemetryStoreBatchedCnt = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "telemetry", Name: "store_batched", Help: "Counter of store batched coprocessor tasks", }) - TelemetryStoreBatchedFallbackCnt = prometheus.NewCounter( + + TelemetryStoreBatchedFallbackCnt = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "telemetry", Name: "store_batched_fallback", Help: "Counter of store batched fallback coprocessor tasks", }) -) +} // readCounter reads the value of a prometheus.Counter. // Returns -1 when failing to read the value. diff --git a/metrics/topsql.go b/metrics/topsql.go index 7ee6e55ae3f71..65b67ff6c7137 100644 --- a/metrics/topsql.go +++ b/metrics/topsql.go @@ -18,7 +18,14 @@ import "github.com/prometheus/client_golang/prometheus" // Top SQL metrics. var ( - TopSQLIgnoredCounter = prometheus.NewCounterVec( + TopSQLIgnoredCounter *prometheus.CounterVec + TopSQLReportDurationHistogram *prometheus.HistogramVec + TopSQLReportDataHistogram *prometheus.HistogramVec +) + +// InitTopSQLMetrics initializes top-sql metrics. +func InitTopSQLMetrics() { + TopSQLIgnoredCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "topsql", @@ -26,7 +33,7 @@ var ( Help: "Counter of ignored top-sql metrics (register-sql, register-plan, collect-data and report-data), normally it should be 0.", }, []string{LblType}) - TopSQLReportDurationHistogram = prometheus.NewHistogramVec( + TopSQLReportDurationHistogram = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "topsql", @@ -35,7 +42,7 @@ var ( Buckets: prometheus.ExponentialBuckets(0.001, 2, 24), // 1ms ~ 2.3h }, []string{LblType, LblResult}) - TopSQLReportDataHistogram = prometheus.NewHistogramVec( + TopSQLReportDataHistogram = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "topsql", @@ -43,4 +50,4 @@ var ( Help: "Bucket histogram of reporting records/sql/plan count to the top-sql agent.", Buckets: prometheus.ExponentialBuckets(1, 2, 20), // 1 ~ 524288 }, []string{LblType}) -) +} diff --git a/metrics/ttl.go b/metrics/ttl.go index 52782b08dba76..87ee9d6e5f45f 100644 --- a/metrics/ttl.go +++ b/metrics/ttl.go @@ -18,7 +18,22 @@ import "github.com/prometheus/client_golang/prometheus" // TTL metrics var ( - TTLQueryDuration = prometheus.NewHistogramVec( + TTLQueryDuration *prometheus.HistogramVec + + TTLProcessedExpiredRowsCounter *prometheus.CounterVec + + TTLJobStatus *prometheus.GaugeVec + + TTLTaskStatus *prometheus.GaugeVec + + TTLPhaseTime *prometheus.CounterVec + + TTLInsertRowsCount prometheus.Counter +) + +// InitTTLMetrics initializes ttl metrics. +func InitTTLMetrics() { + TTLQueryDuration = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "server", @@ -27,7 +42,7 @@ var ( Buckets: prometheus.ExponentialBuckets(0.01, 2, 20), // 10ms ~ 1.45hour }, []string{LblSQLType, LblResult}) - TTLProcessedExpiredRowsCounter = prometheus.NewCounterVec( + TTLProcessedExpiredRowsCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "server", @@ -35,7 +50,7 @@ var ( Help: "The count of expired rows processed in TTL jobs", }, []string{LblSQLType, LblResult}) - TTLJobStatus = prometheus.NewGaugeVec( + TTLJobStatus = NewGaugeVec( prometheus.GaugeOpts{ Namespace: "tidb", Subsystem: "server", @@ -43,7 +58,7 @@ var ( Help: "The jobs count in the specified status", }, []string{LblType}) - TTLTaskStatus = prometheus.NewGaugeVec( + TTLTaskStatus = NewGaugeVec( prometheus.GaugeOpts{ Namespace: "tidb", Subsystem: "server", @@ -51,7 +66,7 @@ var ( Help: "The tasks count in the specified status", }, []string{LblType}) - TTLPhaseTime = prometheus.NewCounterVec( + TTLPhaseTime = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "server", @@ -59,11 +74,11 @@ var ( Help: "The time spent in each phase", }, []string{LblType, LblPhase}) - TTLInsertRowsCount = prometheus.NewCounter( + TTLInsertRowsCount = NewCounter( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "server", Name: "ttl_insert_rows", Help: "The count of TTL rows inserted", }) -) +} diff --git a/metrics/wrapper.go b/metrics/wrapper.go new file mode 100644 index 0000000000000..e1bcd21243c3d --- /dev/null +++ b/metrics/wrapper.go @@ -0,0 +1,77 @@ +// Copyright 2023 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package metrics + +import ( + "fmt" + "strings" + + "github.com/prometheus/client_golang/prometheus" +) + +var constLabels prometheus.Labels + +// SetConstLabels sets constant labels for metrics. +func SetConstLabels(kv ...string) { + if len(kv)%2 == 1 { + panic(fmt.Sprintf("got the odd number of inputs for const labels: %d", len(kv))) + } + constLabels = make(prometheus.Labels, len(kv)/2) + for i := 0; i < len(kv); i += 2 { + constLabels[strings.ToLower(kv[i])] = kv[i+1] + } +} + +// NewCounter wraps a prometheus.NewCounter. +func NewCounter(opts prometheus.CounterOpts) prometheus.Counter { + opts.ConstLabels = constLabels + return prometheus.NewCounter(opts) +} + +// NewCounterVec wraps a prometheus.NewCounterVec. +func NewCounterVec(opts prometheus.CounterOpts, labelNames []string) *prometheus.CounterVec { + opts.ConstLabels = constLabels + return prometheus.NewCounterVec(opts, labelNames) +} + +// NewGauge wraps a prometheus.NewGauge. +func NewGauge(opts prometheus.GaugeOpts) prometheus.Gauge { + opts.ConstLabels = constLabels + return prometheus.NewGauge(opts) +} + +// NewGaugeVec wraps a prometheus.NewGaugeVec. +func NewGaugeVec(opts prometheus.GaugeOpts, labelNames []string) *prometheus.GaugeVec { + opts.ConstLabels = constLabels + return prometheus.NewGaugeVec(opts, labelNames) +} + +// NewHistogram wraps a prometheus.NewHistogram. +func NewHistogram(opts prometheus.HistogramOpts) prometheus.Histogram { + opts.ConstLabels = constLabels + return prometheus.NewHistogram(opts) +} + +// NewHistogramVec wraps a prometheus.NewHistogramVec. +func NewHistogramVec(opts prometheus.HistogramOpts, labelNames []string) *prometheus.HistogramVec { + opts.ConstLabels = constLabels + return prometheus.NewHistogramVec(opts, labelNames) +} + +// NewSummaryVec wraps a prometheus.NewSummaryVec. +func NewSummaryVec(opts prometheus.SummaryOpts, labelNames []string) *prometheus.SummaryVec { + opts.ConstLabels = constLabels + return prometheus.NewSummaryVec(opts, labelNames) +} diff --git a/planner/core/BUILD.bazel b/planner/core/BUILD.bazel index 3e6a454cbc1c5..8b6c7dcafba52 100644 --- a/planner/core/BUILD.bazel +++ b/planner/core/BUILD.bazel @@ -98,6 +98,7 @@ go_library( "//parser/terror", "//parser/types", "//planner/core/internal", + "//planner/core/metrics", "//planner/funcdep", "//planner/property", "//planner/util", diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 8945447a8559b..0b805e1ed0821 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" @@ -44,9 +43,6 @@ import ( "github.com/pingcap/tipb/go-tipb" ) -var planCacheCounter = metrics.PlanCacheCounter.WithLabelValues("prepare") -var planCacheMissCounter = metrics.PlanCacheMissCounter.WithLabelValues("cache_miss") - // ShowDDL is for showing DDL information. type ShowDDL struct { baseSchemaProducer diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index aae357559cab4..b5b0b4dec4c33 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -32,7 +32,6 @@ import ( "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/charset" @@ -41,6 +40,7 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/opcode" "github.com/pingcap/tidb/parser/terror" + core_metrics "github.com/pingcap/tidb/planner/core/metrics" fd "github.com/pingcap/tidb/planner/funcdep" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/planner/util" @@ -4296,11 +4296,6 @@ func (ds *DataSource) AddExtraPhysTblIDColumn() *expression.Column { return pidCol } -var ( - pseudoEstimationNotAvailable = metrics.PseudoEstimation.WithLabelValues("nodata") - pseudoEstimationOutdate = metrics.PseudoEstimation.WithLabelValues("outdate") -) - // getStatsTable gets statistics information for a table specified by "tableID". // A pseudo statistics table is returned in any of the following scenario: // 1. tidb-server started and statistics handle has not been initialized. @@ -4323,7 +4318,7 @@ func getStatsTable(ctx sessionctx.Context, tblInfo *model.TableInfo, pid int64) // 2. table row count from statistics is zero. if statsTbl.Count == 0 { - pseudoEstimationNotAvailable.Inc() + core_metrics.PseudoEstimationNotAvailable.Inc() return statistics.PseudoTable(tblInfo) } @@ -4335,9 +4330,9 @@ func getStatsTable(ctx sessionctx.Context, tblInfo *model.TableInfo, pid int64) tbl.Pseudo = true statsTbl = &tbl if pseudoStatsForUninitialized { - pseudoEstimationNotAvailable.Inc() + core_metrics.PseudoEstimationNotAvailable.Inc() } else { - pseudoEstimationOutdate.Inc() + core_metrics.PseudoEstimationOutdate.Inc() } } diff --git a/planner/core/metrics/BUILD.bazel b/planner/core/metrics/BUILD.bazel new file mode 100644 index 0000000000000..86070d1901b66 --- /dev/null +++ b/planner/core/metrics/BUILD.bazel @@ -0,0 +1,12 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "metrics", + srcs = ["metrics.go"], + importpath = "github.com/pingcap/tidb/planner/core/metrics", + visibility = ["//visibility:public"], + deps = [ + "//metrics", + "@com_github_prometheus_client_golang//prometheus", + ], +) diff --git a/planner/core/metrics/metrics.go b/planner/core/metrics/metrics.go new file mode 100644 index 0000000000000..1e8e108d5f43c --- /dev/null +++ b/planner/core/metrics/metrics.go @@ -0,0 +1,41 @@ +// Copyright 2023 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package metrics + +import ( + "github.com/pingcap/tidb/metrics" + "github.com/prometheus/client_golang/prometheus" +) + +// planner core metrics vars +var ( + PlanCacheCounter prometheus.Counter + PlanCacheMissCounter prometheus.Counter + + PseudoEstimationNotAvailable prometheus.Counter + PseudoEstimationOutdate prometheus.Counter +) + +func init() { + InitMetricsVars() +} + +// InitMetricsVars init planner core metrics vars. +func InitMetricsVars() { + PlanCacheCounter = metrics.PlanCacheCounter.WithLabelValues("prepare") + PlanCacheMissCounter = metrics.PlanCacheMissCounter.WithLabelValues("cache_miss") + PseudoEstimationNotAvailable = metrics.PseudoEstimation.WithLabelValues("nodata") + PseudoEstimationOutdate = metrics.PseudoEstimation.WithLabelValues("outdate") +} diff --git a/planner/core/plan_cache.go b/planner/core/plan_cache.go index aa0d45961fe80..f416edbd1381a 100644 --- a/planner/core/plan_cache.go +++ b/planner/core/plan_cache.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/mysql" + core_metrics "github.com/pingcap/tidb/planner/core/metrics" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" @@ -206,7 +207,7 @@ func getCachedPointPlan(stmt *ast.Prepared, sessVars *variable.SessionVars, stmt if metrics.ResettablePlanCacheCounterFortTest { metrics.PlanCacheCounter.WithLabelValues("prepare").Inc() } else { - planCacheCounter.Inc() + core_metrics.PlanCacheCounter.Inc() } sessVars.FoundInPlanCache = true stmtCtx.PointExec = true @@ -247,7 +248,7 @@ func getCachedPlan(sctx sessionctx.Context, isNonPrepared bool, cacheKey kvcache if metrics.ResettablePlanCacheCounterFortTest { metrics.PlanCacheCounter.WithLabelValues("prepare").Inc() } else { - planCacheCounter.Inc() + core_metrics.PlanCacheCounter.Inc() } stmtCtx.SetPlanDigest(stmt.NormalizedPlan, stmt.PlanDigest) return cachedVal.Plan, cachedVal.OutPutNames, true, nil @@ -262,7 +263,7 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isNonPrepared sessVars := sctx.GetSessionVars() stmtCtx := sessVars.StmtCtx - planCacheMissCounter.Inc() + core_metrics.PlanCacheMissCounter.Inc() sctx.GetSessionVars().StmtCtx.InPreparedPlanBuilding = true p, names, err := OptimizeAstNode(ctx, sctx, stmtAst.Stmt, is) sctx.GetSessionVars().StmtCtx.InPreparedPlanBuilding = false diff --git a/server/BUILD.bazel b/server/BUILD.bazel index d264f44d576ab..cd47779b89e25 100644 --- a/server/BUILD.bazel +++ b/server/BUILD.bazel @@ -51,6 +51,7 @@ go_library( "//plugin", "//privilege", "//privilege/privileges", + "//server/metrics", "//session", "//session/txninfo", "//sessionctx", diff --git a/server/conn.go b/server/conn.go index 8de0dbd1287b7..13a9bee4b15ee 100644 --- a/server/conn.go +++ b/server/conn.go @@ -72,6 +72,7 @@ import ( plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/plugin" "github.com/pingcap/tidb/privilege" + server_metrics "github.com/pingcap/tidb/server/metrics" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" @@ -101,51 +102,6 @@ const ( connStatusWaitShutdown // Notified by server to close. ) -var ( - queryTotalCountOk = [...]prometheus.Counter{ - mysql.ComSleep: metrics.QueryTotalCounter.WithLabelValues("Sleep", "OK"), - mysql.ComQuit: metrics.QueryTotalCounter.WithLabelValues("Quit", "OK"), - mysql.ComInitDB: metrics.QueryTotalCounter.WithLabelValues("InitDB", "OK"), - mysql.ComQuery: metrics.QueryTotalCounter.WithLabelValues("Query", "OK"), - mysql.ComPing: metrics.QueryTotalCounter.WithLabelValues("Ping", "OK"), - mysql.ComFieldList: metrics.QueryTotalCounter.WithLabelValues("FieldList", "OK"), - mysql.ComStmtPrepare: metrics.QueryTotalCounter.WithLabelValues("StmtPrepare", "OK"), - mysql.ComStmtExecute: metrics.QueryTotalCounter.WithLabelValues("StmtExecute", "OK"), - mysql.ComStmtFetch: metrics.QueryTotalCounter.WithLabelValues("StmtFetch", "OK"), - mysql.ComStmtClose: metrics.QueryTotalCounter.WithLabelValues("StmtClose", "OK"), - mysql.ComStmtSendLongData: metrics.QueryTotalCounter.WithLabelValues("StmtSendLongData", "OK"), - mysql.ComStmtReset: metrics.QueryTotalCounter.WithLabelValues("StmtReset", "OK"), - mysql.ComSetOption: metrics.QueryTotalCounter.WithLabelValues("SetOption", "OK"), - } - queryTotalCountErr = [...]prometheus.Counter{ - mysql.ComSleep: metrics.QueryTotalCounter.WithLabelValues("Sleep", "Error"), - mysql.ComQuit: metrics.QueryTotalCounter.WithLabelValues("Quit", "Error"), - mysql.ComInitDB: metrics.QueryTotalCounter.WithLabelValues("InitDB", "Error"), - mysql.ComQuery: metrics.QueryTotalCounter.WithLabelValues("Query", "Error"), - mysql.ComPing: metrics.QueryTotalCounter.WithLabelValues("Ping", "Error"), - mysql.ComFieldList: metrics.QueryTotalCounter.WithLabelValues("FieldList", "Error"), - mysql.ComStmtPrepare: metrics.QueryTotalCounter.WithLabelValues("StmtPrepare", "Error"), - mysql.ComStmtExecute: metrics.QueryTotalCounter.WithLabelValues("StmtExecute", "Error"), - mysql.ComStmtFetch: metrics.QueryTotalCounter.WithLabelValues("StmtFetch", "Error"), - mysql.ComStmtClose: metrics.QueryTotalCounter.WithLabelValues("StmtClose", "Error"), - mysql.ComStmtSendLongData: metrics.QueryTotalCounter.WithLabelValues("StmtSendLongData", "Error"), - mysql.ComStmtReset: metrics.QueryTotalCounter.WithLabelValues("StmtReset", "Error"), - mysql.ComSetOption: metrics.QueryTotalCounter.WithLabelValues("SetOption", "Error"), - } - - disconnectNormal = metrics.DisconnectionCounter.WithLabelValues(metrics.LblOK) - disconnectByClientWithError = metrics.DisconnectionCounter.WithLabelValues(metrics.LblError) - disconnectErrorUndetermined = metrics.DisconnectionCounter.WithLabelValues("undetermined") - - connIdleDurationHistogramNotInTxn = metrics.ConnIdleDurationHistogram.WithLabelValues("0") - connIdleDurationHistogramInTxn = metrics.ConnIdleDurationHistogram.WithLabelValues("1") - - affectedRowsCounterInsert = metrics.AffectedRowsCounter.WithLabelValues("Insert") - affectedRowsCounterUpdate = metrics.AffectedRowsCounter.WithLabelValues("Update") - affectedRowsCounterDelete = metrics.AffectedRowsCounter.WithLabelValues("Delete") - affectedRowsCounterReplace = metrics.AffectedRowsCounter.WithLabelValues("Replace") -) - // newClientConn creates a *clientConn object. func newClientConn(s *Server) *clientConn { return &clientConn{ @@ -1132,7 +1088,7 @@ func (cc *clientConn) Run(ctx context.Context) { } } } - disconnectByClientWithError.Inc() + server_metrics.DisconnectByClientWithError.Inc() return } @@ -1148,11 +1104,11 @@ func (cc *clientConn) Run(ctx context.Context) { cc.audit(plugin.Error) // tell the plugin API there was a dispatch error if terror.ErrorEqual(err, io.EOF) { cc.addMetrics(data[0], startTime, nil) - disconnectNormal.Inc() + server_metrics.DisconnectNormal.Inc() return } else if terror.ErrResultUndetermined.Equal(err) { logutil.Logger(ctx).Error("result undetermined, close this connection", zap.Error(err)) - disconnectErrorUndetermined.Inc() + server_metrics.DisconnectErrorUndetermined.Inc() return } else if terror.ErrCritical.Equal(err) { metrics.CriticalErrorCounter.Add(1) @@ -1233,10 +1189,10 @@ func (cc *clientConn) addMetrics(cmd byte, startTime time.Time, err error) { } var counter prometheus.Counter - if err != nil && int(cmd) < len(queryTotalCountErr) { - counter = queryTotalCountErr[cmd] - } else if err == nil && int(cmd) < len(queryTotalCountOk) { - counter = queryTotalCountOk[cmd] + if err != nil && int(cmd) < len(server_metrics.QueryTotalCountErr) { + counter = server_metrics.QueryTotalCountErr[cmd] + } else if err == nil && int(cmd) < len(server_metrics.QueryTotalCountOk) { + counter = server_metrics.QueryTotalCountOk[cmd] } if counter != nil { counter.Inc() @@ -1267,13 +1223,13 @@ func (cc *clientConn) addMetrics(cmd byte, startTime time.Time, err error) { switch sqlType { case "Insert": - affectedRowsCounterInsert.Add(float64(affectedRows)) + server_metrics.AffectedRowsCounterInsert.Add(float64(affectedRows)) case "Replace": - affectedRowsCounterReplace.Add(float64(affectedRows)) + server_metrics.AffectedRowsCounterReplace.Add(float64(affectedRows)) case "Delete": - affectedRowsCounterDelete.Add(float64(affectedRows)) + server_metrics.AffectedRowsCounterDelete.Add(float64(affectedRows)) case "Update": - affectedRowsCounterUpdate.Add(float64(affectedRows)) + server_metrics.AffectedRowsCounterUpdate.Add(float64(affectedRows)) } metrics.QueryDurationHistogram.WithLabelValues(sqlType, dbName).Observe(cost.Seconds()) @@ -1289,9 +1245,9 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { }() t := time.Now() if (cc.ctx.Status() & mysql.ServerStatusInTrans) > 0 { - connIdleDurationHistogramInTxn.Observe(t.Sub(cc.lastActive).Seconds()) + server_metrics.ConnIdleDurationHistogramInTxn.Observe(t.Sub(cc.lastActive).Seconds()) } else { - connIdleDurationHistogramNotInTxn.Observe(t.Sub(cc.lastActive).Seconds()) + server_metrics.ConnIdleDurationHistogramNotInTxn.Observe(t.Sub(cc.lastActive).Seconds()) } cfg := config.GetGlobalConfig() diff --git a/server/metrics/BUILD.bazel b/server/metrics/BUILD.bazel new file mode 100644 index 0000000000000..7a3636ec5449a --- /dev/null +++ b/server/metrics/BUILD.bazel @@ -0,0 +1,13 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "metrics", + srcs = ["metrics.go"], + importpath = "github.com/pingcap/tidb/server/metrics", + visibility = ["//visibility:public"], + deps = [ + "//metrics", + "//parser/mysql", + "@com_github_prometheus_client_golang//prometheus", + ], +) diff --git a/server/metrics/metrics.go b/server/metrics/metrics.go new file mode 100644 index 0000000000000..74f327b4ca28e --- /dev/null +++ b/server/metrics/metrics.go @@ -0,0 +1,95 @@ +// Copyright 2023 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package metrics + +import ( + "github.com/pingcap/tidb/metrics" + "github.com/pingcap/tidb/parser/mysql" + "github.com/prometheus/client_golang/prometheus" +) + +// server metrics vars +var ( + QueryTotalCountOk []prometheus.Counter + QueryTotalCountErr []prometheus.Counter + + DisconnectNormal prometheus.Counter + DisconnectByClientWithError prometheus.Counter + DisconnectErrorUndetermined prometheus.Counter + + ConnIdleDurationHistogramNotInTxn prometheus.Observer + ConnIdleDurationHistogramInTxn prometheus.Observer + + AffectedRowsCounterInsert prometheus.Counter + AffectedRowsCounterUpdate prometheus.Counter + AffectedRowsCounterDelete prometheus.Counter + AffectedRowsCounterReplace prometheus.Counter + + ReadPacketBytes prometheus.Counter + WritePacketBytes prometheus.Counter +) + +func init() { + InitMetricsVars() +} + +// InitMetricsVars init server metrics vars. +func InitMetricsVars() { + QueryTotalCountOk = []prometheus.Counter{ + mysql.ComSleep: metrics.QueryTotalCounter.WithLabelValues("Sleep", "OK"), + mysql.ComQuit: metrics.QueryTotalCounter.WithLabelValues("Quit", "OK"), + mysql.ComInitDB: metrics.QueryTotalCounter.WithLabelValues("InitDB", "OK"), + mysql.ComQuery: metrics.QueryTotalCounter.WithLabelValues("Query", "OK"), + mysql.ComPing: metrics.QueryTotalCounter.WithLabelValues("Ping", "OK"), + mysql.ComFieldList: metrics.QueryTotalCounter.WithLabelValues("FieldList", "OK"), + mysql.ComStmtPrepare: metrics.QueryTotalCounter.WithLabelValues("StmtPrepare", "OK"), + mysql.ComStmtExecute: metrics.QueryTotalCounter.WithLabelValues("StmtExecute", "OK"), + mysql.ComStmtFetch: metrics.QueryTotalCounter.WithLabelValues("StmtFetch", "OK"), + mysql.ComStmtClose: metrics.QueryTotalCounter.WithLabelValues("StmtClose", "OK"), + mysql.ComStmtSendLongData: metrics.QueryTotalCounter.WithLabelValues("StmtSendLongData", "OK"), + mysql.ComStmtReset: metrics.QueryTotalCounter.WithLabelValues("StmtReset", "OK"), + mysql.ComSetOption: metrics.QueryTotalCounter.WithLabelValues("SetOption", "OK"), + } + QueryTotalCountErr = []prometheus.Counter{ + mysql.ComSleep: metrics.QueryTotalCounter.WithLabelValues("Sleep", "Error"), + mysql.ComQuit: metrics.QueryTotalCounter.WithLabelValues("Quit", "Error"), + mysql.ComInitDB: metrics.QueryTotalCounter.WithLabelValues("InitDB", "Error"), + mysql.ComQuery: metrics.QueryTotalCounter.WithLabelValues("Query", "Error"), + mysql.ComPing: metrics.QueryTotalCounter.WithLabelValues("Ping", "Error"), + mysql.ComFieldList: metrics.QueryTotalCounter.WithLabelValues("FieldList", "Error"), + mysql.ComStmtPrepare: metrics.QueryTotalCounter.WithLabelValues("StmtPrepare", "Error"), + mysql.ComStmtExecute: metrics.QueryTotalCounter.WithLabelValues("StmtExecute", "Error"), + mysql.ComStmtFetch: metrics.QueryTotalCounter.WithLabelValues("StmtFetch", "Error"), + mysql.ComStmtClose: metrics.QueryTotalCounter.WithLabelValues("StmtClose", "Error"), + mysql.ComStmtSendLongData: metrics.QueryTotalCounter.WithLabelValues("StmtSendLongData", "Error"), + mysql.ComStmtReset: metrics.QueryTotalCounter.WithLabelValues("StmtReset", "Error"), + mysql.ComSetOption: metrics.QueryTotalCounter.WithLabelValues("SetOption", "Error"), + } + + DisconnectNormal = metrics.DisconnectionCounter.WithLabelValues(metrics.LblOK) + DisconnectByClientWithError = metrics.DisconnectionCounter.WithLabelValues(metrics.LblError) + DisconnectErrorUndetermined = metrics.DisconnectionCounter.WithLabelValues("undetermined") + + ConnIdleDurationHistogramNotInTxn = metrics.ConnIdleDurationHistogram.WithLabelValues("0") + ConnIdleDurationHistogramInTxn = metrics.ConnIdleDurationHistogram.WithLabelValues("1") + + AffectedRowsCounterInsert = metrics.AffectedRowsCounter.WithLabelValues("Insert") + AffectedRowsCounterUpdate = metrics.AffectedRowsCounter.WithLabelValues("Update") + AffectedRowsCounterDelete = metrics.AffectedRowsCounter.WithLabelValues("Delete") + AffectedRowsCounterReplace = metrics.AffectedRowsCounter.WithLabelValues("Replace") + + ReadPacketBytes = metrics.PacketIOCounter.WithLabelValues("read") + WritePacketBytes = metrics.PacketIOCounter.WithLabelValues("write") +} diff --git a/server/packetio.go b/server/packetio.go index 2562d66582783..736df5e241b3c 100644 --- a/server/packetio.go +++ b/server/packetio.go @@ -41,19 +41,14 @@ import ( "time" "github.com/pingcap/errors" - "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" + server_metrics "github.com/pingcap/tidb/server/metrics" "github.com/pingcap/tidb/sessionctx/variable" ) const defaultWriterSize = 16 * 1024 -var ( - readPacketBytes = metrics.PacketIOCounter.WithLabelValues("read") - writePacketBytes = metrics.PacketIOCounter.WithLabelValues("write") -) - // packetIO is a helper to read and write data in packet format. // MySQL Packets: https://dev.mysql.com/doc/internals/en/mysql-packet.html type packetIO struct { @@ -138,7 +133,7 @@ func (p *packetIO) readPacket() ([]byte, error) { } if len(data) < mysql.MaxPayloadLen { - readPacketBytes.Add(float64(len(data))) + server_metrics.ReadPacketBytes.Add(float64(len(data))) return data, nil } @@ -156,14 +151,14 @@ func (p *packetIO) readPacket() ([]byte, error) { } } - readPacketBytes.Add(float64(len(data))) + server_metrics.ReadPacketBytes.Add(float64(len(data))) return data, nil } // writePacket writes data that already have header func (p *packetIO) writePacket(data []byte) error { length := len(data) - 4 - writePacketBytes.Add(float64(len(data))) + server_metrics.WritePacketBytes.Add(float64(len(data))) for length >= mysql.MaxPayloadLen { data[3] = p.sequence diff --git a/session/BUILD.bazel b/session/BUILD.bazel index 08f89fd9b8182..191639fc3a546 100644 --- a/session/BUILD.bazel +++ b/session/BUILD.bazel @@ -46,6 +46,7 @@ go_library( "//plugin", "//privilege", "//privilege/privileges", + "//session/metrics", "//session/txninfo", "//sessionctx", "//sessionctx/binloginfo", @@ -95,7 +96,6 @@ go_library( "@com_github_pingcap_failpoint//:failpoint", "@com_github_pingcap_kvproto//pkg/kvrpcpb", "@com_github_pingcap_tipb//go-binlog", - "@com_github_prometheus_client_golang//prometheus", "@com_github_tikv_client_go_v2//error", "@com_github_tikv_client_go_v2//kv", "@com_github_tikv_client_go_v2//oracle", diff --git a/session/metrics/BUILD.bazel b/session/metrics/BUILD.bazel new file mode 100644 index 0000000000000..9dc63b40d72ec --- /dev/null +++ b/session/metrics/BUILD.bazel @@ -0,0 +1,12 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "metrics", + srcs = ["metrics.go"], + importpath = "github.com/pingcap/tidb/session/metrics", + visibility = ["//visibility:public"], + deps = [ + "//metrics", + "@com_github_prometheus_client_golang//prometheus", + ], +) diff --git a/session/metrics/metrics.go b/session/metrics/metrics.go new file mode 100644 index 0000000000000..2c6310790c7c4 --- /dev/null +++ b/session/metrics/metrics.go @@ -0,0 +1,146 @@ +// Copyright 2023 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package metrics + +import ( + "github.com/pingcap/tidb/metrics" + "github.com/prometheus/client_golang/prometheus" +) + +// session metrics vars +var ( + NonTransactionalDeleteCount prometheus.Counter + NonTransactionalInsertCount prometheus.Counter + NonTransactionalUpdateCount prometheus.Counter + + StatementPerTransactionPessimisticOKInternal prometheus.Observer + StatementPerTransactionPessimisticOKGeneral prometheus.Observer + StatementPerTransactionPessimisticErrorInternal prometheus.Observer + StatementPerTransactionPessimisticErrorGeneral prometheus.Observer + StatementPerTransactionOptimisticOKInternal prometheus.Observer + StatementPerTransactionOptimisticOKGeneral prometheus.Observer + StatementPerTransactionOptimisticErrorInternal prometheus.Observer + StatementPerTransactionOptimisticErrorGeneral prometheus.Observer + TransactionDurationPessimisticCommitInternal prometheus.Observer + TransactionDurationPessimisticCommitGeneral prometheus.Observer + TransactionDurationPessimisticAbortInternal prometheus.Observer + TransactionDurationPessimisticAbortGeneral prometheus.Observer + TransactionDurationOptimisticCommitInternal prometheus.Observer + TransactionDurationOptimisticCommitGeneral prometheus.Observer + TransactionDurationOptimisticAbortInternal prometheus.Observer + TransactionDurationOptimisticAbortGeneral prometheus.Observer + TransactionRetryInternal prometheus.Observer + TransactionRetryGeneral prometheus.Observer + + SessionExecuteCompileDurationInternal prometheus.Observer + SessionExecuteCompileDurationGeneral prometheus.Observer + SessionExecuteParseDurationInternal prometheus.Observer + SessionExecuteParseDurationGeneral prometheus.Observer + + TelemetryCTEUsageRecurCTE prometheus.Counter + TelemetryCTEUsageNonRecurCTE prometheus.Counter + TelemetryCTEUsageNotCTE prometheus.Counter + TelemetryMultiSchemaChangeUsage prometheus.Counter + TelemetryFlashbackClusterUsage prometheus.Counter + + TelemetryTablePartitionUsage prometheus.Counter + TelemetryTablePartitionListUsage prometheus.Counter + TelemetryTablePartitionRangeUsage prometheus.Counter + TelemetryTablePartitionHashUsage prometheus.Counter + TelemetryTablePartitionRangeColumnsUsage prometheus.Counter + TelemetryTablePartitionRangeColumnsGt1Usage prometheus.Counter + TelemetryTablePartitionRangeColumnsGt2Usage prometheus.Counter + TelemetryTablePartitionRangeColumnsGt3Usage prometheus.Counter + TelemetryTablePartitionListColumnsUsage prometheus.Counter + TelemetryTablePartitionMaxPartitionsUsage prometheus.Counter + TelemetryTablePartitionCreateIntervalUsage prometheus.Counter + TelemetryTablePartitionAddIntervalUsage prometheus.Counter + TelemetryTablePartitionDropIntervalUsage prometheus.Counter + TelemetryExchangePartitionUsage prometheus.Counter + TelemetryTableCompactPartitionUsage prometheus.Counter + TelemetryReorganizePartitionUsage prometheus.Counter + + TelemetryLockUserUsage prometheus.Counter + TelemetryUnlockUserUsage prometheus.Counter + TelemetryCreateOrAlterUserUsage prometheus.Counter + + TelemetryIndexMerge prometheus.Counter + TelemetryStoreBatchedUsage prometheus.Counter +) + +func init() { + InitMetricsVars() +} + +// InitMetricsVars init session metrics vars. +func InitMetricsVars() { + NonTransactionalDeleteCount = metrics.NonTransactionalDMLCount.With(prometheus.Labels{metrics.LblType: "delete"}) + NonTransactionalInsertCount = metrics.NonTransactionalDMLCount.With(prometheus.Labels{metrics.LblType: "insert"}) + NonTransactionalUpdateCount = metrics.NonTransactionalDMLCount.With(prometheus.Labels{metrics.LblType: "update"}) + + StatementPerTransactionPessimisticOKInternal = metrics.StatementPerTransaction.WithLabelValues(metrics.LblPessimistic, metrics.LblOK, metrics.LblInternal) + StatementPerTransactionPessimisticOKGeneral = metrics.StatementPerTransaction.WithLabelValues(metrics.LblPessimistic, metrics.LblOK, metrics.LblGeneral) + StatementPerTransactionPessimisticErrorInternal = metrics.StatementPerTransaction.WithLabelValues(metrics.LblPessimistic, metrics.LblError, metrics.LblInternal) + StatementPerTransactionPessimisticErrorGeneral = metrics.StatementPerTransaction.WithLabelValues(metrics.LblPessimistic, metrics.LblError, metrics.LblGeneral) + StatementPerTransactionOptimisticOKInternal = metrics.StatementPerTransaction.WithLabelValues(metrics.LblOptimistic, metrics.LblOK, metrics.LblInternal) + StatementPerTransactionOptimisticOKGeneral = metrics.StatementPerTransaction.WithLabelValues(metrics.LblOptimistic, metrics.LblOK, metrics.LblGeneral) + StatementPerTransactionOptimisticErrorInternal = metrics.StatementPerTransaction.WithLabelValues(metrics.LblOptimistic, metrics.LblError, metrics.LblInternal) + StatementPerTransactionOptimisticErrorGeneral = metrics.StatementPerTransaction.WithLabelValues(metrics.LblOptimistic, metrics.LblError, metrics.LblGeneral) + TransactionDurationPessimisticCommitInternal = metrics.TransactionDuration.WithLabelValues(metrics.LblPessimistic, metrics.LblCommit, metrics.LblInternal) + TransactionDurationPessimisticCommitGeneral = metrics.TransactionDuration.WithLabelValues(metrics.LblPessimistic, metrics.LblCommit, metrics.LblGeneral) + TransactionDurationPessimisticAbortInternal = metrics.TransactionDuration.WithLabelValues(metrics.LblPessimistic, metrics.LblAbort, metrics.LblInternal) + TransactionDurationPessimisticAbortGeneral = metrics.TransactionDuration.WithLabelValues(metrics.LblPessimistic, metrics.LblAbort, metrics.LblGeneral) + TransactionDurationOptimisticCommitInternal = metrics.TransactionDuration.WithLabelValues(metrics.LblOptimistic, metrics.LblCommit, metrics.LblInternal) + TransactionDurationOptimisticCommitGeneral = metrics.TransactionDuration.WithLabelValues(metrics.LblOptimistic, metrics.LblCommit, metrics.LblGeneral) + TransactionDurationOptimisticAbortInternal = metrics.TransactionDuration.WithLabelValues(metrics.LblOptimistic, metrics.LblAbort, metrics.LblInternal) + TransactionDurationOptimisticAbortGeneral = metrics.TransactionDuration.WithLabelValues(metrics.LblOptimistic, metrics.LblAbort, metrics.LblGeneral) + TransactionRetryInternal = metrics.SessionRetry.WithLabelValues(metrics.LblInternal) + TransactionRetryGeneral = metrics.SessionRetry.WithLabelValues(metrics.LblGeneral) + + SessionExecuteCompileDurationInternal = metrics.SessionExecuteCompileDuration.WithLabelValues(metrics.LblInternal) + SessionExecuteCompileDurationGeneral = metrics.SessionExecuteCompileDuration.WithLabelValues(metrics.LblGeneral) + SessionExecuteParseDurationInternal = metrics.SessionExecuteParseDuration.WithLabelValues(metrics.LblInternal) + SessionExecuteParseDurationGeneral = metrics.SessionExecuteParseDuration.WithLabelValues(metrics.LblGeneral) + + TelemetryCTEUsageRecurCTE = metrics.TelemetrySQLCTECnt.WithLabelValues("recurCTE") + TelemetryCTEUsageNonRecurCTE = metrics.TelemetrySQLCTECnt.WithLabelValues("nonRecurCTE") + TelemetryCTEUsageNotCTE = metrics.TelemetrySQLCTECnt.WithLabelValues("notCTE") + TelemetryMultiSchemaChangeUsage = metrics.TelemetryMultiSchemaChangeCnt + TelemetryFlashbackClusterUsage = metrics.TelemetryFlashbackClusterCnt + + TelemetryTablePartitionUsage = metrics.TelemetryTablePartitionCnt + TelemetryTablePartitionListUsage = metrics.TelemetryTablePartitionListCnt + TelemetryTablePartitionRangeUsage = metrics.TelemetryTablePartitionRangeCnt + TelemetryTablePartitionHashUsage = metrics.TelemetryTablePartitionHashCnt + TelemetryTablePartitionRangeColumnsUsage = metrics.TelemetryTablePartitionRangeColumnsCnt + TelemetryTablePartitionRangeColumnsGt1Usage = metrics.TelemetryTablePartitionRangeColumnsGt1Cnt + TelemetryTablePartitionRangeColumnsGt2Usage = metrics.TelemetryTablePartitionRangeColumnsGt2Cnt + TelemetryTablePartitionRangeColumnsGt3Usage = metrics.TelemetryTablePartitionRangeColumnsGt3Cnt + TelemetryTablePartitionListColumnsUsage = metrics.TelemetryTablePartitionListColumnsCnt + TelemetryTablePartitionMaxPartitionsUsage = metrics.TelemetryTablePartitionMaxPartitionsCnt + TelemetryTablePartitionCreateIntervalUsage = metrics.TelemetryTablePartitionCreateIntervalPartitionsCnt + TelemetryTablePartitionAddIntervalUsage = metrics.TelemetryTablePartitionAddIntervalPartitionsCnt + TelemetryTablePartitionDropIntervalUsage = metrics.TelemetryTablePartitionDropIntervalPartitionsCnt + TelemetryExchangePartitionUsage = metrics.TelemetryExchangePartitionCnt + TelemetryTableCompactPartitionUsage = metrics.TelemetryCompactPartitionCnt + TelemetryReorganizePartitionUsage = metrics.TelemetryReorganizePartitionCnt + + TelemetryLockUserUsage = metrics.TelemetryAccountLockCnt.WithLabelValues("lockUser") + TelemetryUnlockUserUsage = metrics.TelemetryAccountLockCnt.WithLabelValues("unlockUser") + TelemetryCreateOrAlterUserUsage = metrics.TelemetryAccountLockCnt.WithLabelValues("createOrAlterUser") + + TelemetryIndexMerge = metrics.TelemetryIndexMergeUsage + TelemetryStoreBatchedUsage = metrics.TelemetryStoreBatchedQueryCnt +} diff --git a/session/nontransactional.go b/session/nontransactional.go index 83a660f827ab6..f9b2dbcc52f49 100644 --- a/session/nontransactional.go +++ b/session/nontransactional.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/errno" - "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/format" @@ -32,6 +31,7 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/opcode" "github.com/pingcap/tidb/planner/core" + session_metrics "github.com/pingcap/tidb/session/metrics" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" @@ -43,19 +43,12 @@ import ( "github.com/pingcap/tidb/util/mathutil" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/sqlexec" - "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" ) // ErrNonTransactionalJobFailure is the error when a non-transactional job fails. The error is returned and following jobs are canceled. var ErrNonTransactionalJobFailure = dbterror.ClassSession.NewStd(errno.ErrNonTransactionalJobFailure) -var ( - nonTransactionalDeleteCount = metrics.NonTransactionalDMLCount.With(prometheus.Labels{metrics.LblType: "delete"}) - nonTransactionalInsertCount = metrics.NonTransactionalDMLCount.With(prometheus.Labels{metrics.LblType: "insert"}) - nonTransactionalUpdateCount = metrics.NonTransactionalDMLCount.With(prometheus.Labels{metrics.LblType: "update"}) -) - // job: handle keys in [start, end] type job struct { start types.Datum @@ -211,7 +204,7 @@ func checkConstraint(stmt *ast.NonTransactionalDMLStmt, se Session) error { if err := checkReadClauses(s.Limit, s.Order); err != nil { return err } - nonTransactionalDeleteCount.Inc() + session_metrics.NonTransactionalDeleteCount.Inc() case *ast.UpdateStmt: if err := checkTableRef(s.TableRefs, true); err != nil { return err @@ -219,7 +212,7 @@ func checkConstraint(stmt *ast.NonTransactionalDMLStmt, se Session) error { if err := checkReadClauses(s.Limit, s.Order); err != nil { return err } - nonTransactionalUpdateCount.Inc() + session_metrics.NonTransactionalUpdateCount.Inc() case *ast.InsertStmt: if s.Select == nil { return errors.New("Non-transactional insert supports insert select stmt only") @@ -234,7 +227,7 @@ func checkConstraint(stmt *ast.NonTransactionalDMLStmt, se Session) error { if err := checkReadClauses(selectStmt.Limit, selectStmt.OrderBy); err != nil { return err } - nonTransactionalInsertCount.Inc() + session_metrics.NonTransactionalInsertCount.Inc() default: return errors.New("Unsupported DML type for non-transactional DML") } diff --git a/session/session.go b/session/session.go index 1544e4edc4211..d6fd6b83ec855 100644 --- a/session/session.go +++ b/session/session.go @@ -67,6 +67,7 @@ import ( "github.com/pingcap/tidb/plugin" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/privilege/privileges" + session_metrics "github.com/pingcap/tidb/session/metrics" "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" @@ -112,62 +113,6 @@ import ( "go.uber.org/zap" ) -var ( - statementPerTransactionPessimisticOKInternal = metrics.StatementPerTransaction.WithLabelValues(metrics.LblPessimistic, metrics.LblOK, metrics.LblInternal) - statementPerTransactionPessimisticOKGeneral = metrics.StatementPerTransaction.WithLabelValues(metrics.LblPessimistic, metrics.LblOK, metrics.LblGeneral) - statementPerTransactionPessimisticErrorInternal = metrics.StatementPerTransaction.WithLabelValues(metrics.LblPessimistic, metrics.LblError, metrics.LblInternal) - statementPerTransactionPessimisticErrorGeneral = metrics.StatementPerTransaction.WithLabelValues(metrics.LblPessimistic, metrics.LblError, metrics.LblGeneral) - statementPerTransactionOptimisticOKInternal = metrics.StatementPerTransaction.WithLabelValues(metrics.LblOptimistic, metrics.LblOK, metrics.LblInternal) - statementPerTransactionOptimisticOKGeneral = metrics.StatementPerTransaction.WithLabelValues(metrics.LblOptimistic, metrics.LblOK, metrics.LblGeneral) - statementPerTransactionOptimisticErrorInternal = metrics.StatementPerTransaction.WithLabelValues(metrics.LblOptimistic, metrics.LblError, metrics.LblInternal) - statementPerTransactionOptimisticErrorGeneral = metrics.StatementPerTransaction.WithLabelValues(metrics.LblOptimistic, metrics.LblError, metrics.LblGeneral) - transactionDurationPessimisticCommitInternal = metrics.TransactionDuration.WithLabelValues(metrics.LblPessimistic, metrics.LblCommit, metrics.LblInternal) - transactionDurationPessimisticCommitGeneral = metrics.TransactionDuration.WithLabelValues(metrics.LblPessimistic, metrics.LblCommit, metrics.LblGeneral) - transactionDurationPessimisticAbortInternal = metrics.TransactionDuration.WithLabelValues(metrics.LblPessimistic, metrics.LblAbort, metrics.LblInternal) - transactionDurationPessimisticAbortGeneral = metrics.TransactionDuration.WithLabelValues(metrics.LblPessimistic, metrics.LblAbort, metrics.LblGeneral) - transactionDurationOptimisticCommitInternal = metrics.TransactionDuration.WithLabelValues(metrics.LblOptimistic, metrics.LblCommit, metrics.LblInternal) - transactionDurationOptimisticCommitGeneral = metrics.TransactionDuration.WithLabelValues(metrics.LblOptimistic, metrics.LblCommit, metrics.LblGeneral) - transactionDurationOptimisticAbortInternal = metrics.TransactionDuration.WithLabelValues(metrics.LblOptimistic, metrics.LblAbort, metrics.LblInternal) - transactionDurationOptimisticAbortGeneral = metrics.TransactionDuration.WithLabelValues(metrics.LblOptimistic, metrics.LblAbort, metrics.LblGeneral) - transactionRetryInternal = metrics.SessionRetry.WithLabelValues(metrics.LblInternal) - transactionRetryGeneral = metrics.SessionRetry.WithLabelValues(metrics.LblGeneral) - - sessionExecuteCompileDurationInternal = metrics.SessionExecuteCompileDuration.WithLabelValues(metrics.LblInternal) - sessionExecuteCompileDurationGeneral = metrics.SessionExecuteCompileDuration.WithLabelValues(metrics.LblGeneral) - sessionExecuteParseDurationInternal = metrics.SessionExecuteParseDuration.WithLabelValues(metrics.LblInternal) - sessionExecuteParseDurationGeneral = metrics.SessionExecuteParseDuration.WithLabelValues(metrics.LblGeneral) - - telemetryCTEUsageRecurCTE = metrics.TelemetrySQLCTECnt.WithLabelValues("recurCTE") - telemetryCTEUsageNonRecurCTE = metrics.TelemetrySQLCTECnt.WithLabelValues("nonRecurCTE") - telemetryCTEUsageNotCTE = metrics.TelemetrySQLCTECnt.WithLabelValues("notCTE") - telemetryMultiSchemaChangeUsage = metrics.TelemetryMultiSchemaChangeCnt - telemetryFlashbackClusterUsage = metrics.TelemetryFlashbackClusterCnt - - telemetryTablePartitionUsage = metrics.TelemetryTablePartitionCnt - telemetryTablePartitionListUsage = metrics.TelemetryTablePartitionListCnt - telemetryTablePartitionRangeUsage = metrics.TelemetryTablePartitionRangeCnt - telemetryTablePartitionHashUsage = metrics.TelemetryTablePartitionHashCnt - telemetryTablePartitionRangeColumnsUsage = metrics.TelemetryTablePartitionRangeColumnsCnt - telemetryTablePartitionRangeColumnsGt1Usage = metrics.TelemetryTablePartitionRangeColumnsGt1Cnt - telemetryTablePartitionRangeColumnsGt2Usage = metrics.TelemetryTablePartitionRangeColumnsGt2Cnt - telemetryTablePartitionRangeColumnsGt3Usage = metrics.TelemetryTablePartitionRangeColumnsGt3Cnt - telemetryTablePartitionListColumnsUsage = metrics.TelemetryTablePartitionListColumnsCnt - telemetryTablePartitionMaxPartitionsUsage = metrics.TelemetryTablePartitionMaxPartitionsCnt - telemetryTablePartitionCreateIntervalUsage = metrics.TelemetryTablePartitionCreateIntervalPartitionsCnt - telemetryTablePartitionAddIntervalUsage = metrics.TelemetryTablePartitionAddIntervalPartitionsCnt - telemetryTablePartitionDropIntervalUsage = metrics.TelemetryTablePartitionDropIntervalPartitionsCnt - telemetryExchangePartitionUsage = metrics.TelemetryExchangePartitionCnt - telemetryTableCompactPartitionUsage = metrics.TelemetryCompactPartitionCnt - telemetryReorganizePartitionUsage = metrics.TelemetryReorganizePartitionCnt - - telemetryLockUserUsage = metrics.TelemetryAccountLockCnt.WithLabelValues("lockUser") - telemetryUnlockUserUsage = metrics.TelemetryAccountLockCnt.WithLabelValues("unlockUser") - telemetryCreateOrAlterUserUsage = metrics.TelemetryAccountLockCnt.WithLabelValues("createOrAlterUser") - - telemetryIndexMerge = metrics.TelemetryIndexMergeUsage - telemetryStoreBatchedUsage = metrics.TelemetryStoreBatchedQueryCnt -) - // Session context, it is consistent with the lifecycle of a client connection. type Session interface { sessionctx.Context @@ -1224,9 +1169,9 @@ func (s *session) retry(ctx context.Context, maxCnt uint) (err error) { s.sessionVars.RetryInfo.Retrying = false // retryCnt only increments on retryable error, so +1 here. if s.sessionVars.InRestrictedSQL { - transactionRetryInternal.Observe(float64(retryCnt + 1)) + session_metrics.TransactionRetryInternal.Observe(float64(retryCnt + 1)) } else { - transactionRetryGeneral.Observe(float64(retryCnt + 1)) + session_metrics.TransactionRetryGeneral.Observe(float64(retryCnt + 1)) } s.sessionVars.SetInTxn(false) if err != nil { @@ -1740,9 +1685,9 @@ func (s *session) Parse(ctx context.Context, sql string) ([]ast.StmtNode, error) s.GetSessionVars().DurationParse = durParse isInternal := s.isInternal() if isInternal { - sessionExecuteParseDurationInternal.Observe(durParse.Seconds()) + session_metrics.SessionExecuteParseDurationInternal.Observe(durParse.Seconds()) } else { - sessionExecuteParseDurationGeneral.Observe(durParse.Seconds()) + session_metrics.SessionExecuteParseDurationGeneral.Observe(durParse.Seconds()) } for _, warn := range warns { s.sessionVars.StmtCtx.AppendWarning(util.SyntaxWarn(warn)) @@ -1792,9 +1737,9 @@ func (s *session) ParseWithParams(ctx context.Context, sql string, args ...inter } durParse := time.Since(parseStartTime) if internal { - sessionExecuteParseDurationInternal.Observe(durParse.Seconds()) + session_metrics.SessionExecuteParseDurationInternal.Observe(durParse.Seconds()) } else { - sessionExecuteParseDurationGeneral.Observe(durParse.Seconds()) + session_metrics.SessionExecuteParseDurationGeneral.Observe(durParse.Seconds()) } for _, warn := range warns { s.sessionVars.StmtCtx.AppendWarning(util.SyntaxWarn(warn)) @@ -2215,9 +2160,9 @@ func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlex durCompile := time.Since(s.sessionVars.StartTime) s.GetSessionVars().DurationCompile = durCompile if s.isInternal() { - sessionExecuteCompileDurationInternal.Observe(durCompile.Seconds()) + session_metrics.SessionExecuteCompileDurationInternal.Observe(durCompile.Seconds()) } else { - sessionExecuteCompileDurationGeneral.Observe(durCompile.Seconds()) + session_metrics.SessionExecuteCompileDurationGeneral.Observe(durCompile.Seconds()) } s.currentPlan = stmt.Plan if execStmt, ok := stmtNode.(*ast.ExecuteStmt); ok { @@ -3890,37 +3835,37 @@ func (s *session) recordOnTransactionExecution(err error, counter int, duration if s.sessionVars.TxnCtx.IsPessimistic { if err != nil { if isInternal { - transactionDurationPessimisticAbortInternal.Observe(duration) - statementPerTransactionPessimisticErrorInternal.Observe(float64(counter)) + session_metrics.TransactionDurationPessimisticAbortInternal.Observe(duration) + session_metrics.StatementPerTransactionPessimisticErrorInternal.Observe(float64(counter)) } else { - transactionDurationPessimisticAbortGeneral.Observe(duration) - statementPerTransactionPessimisticErrorGeneral.Observe(float64(counter)) + session_metrics.TransactionDurationPessimisticAbortGeneral.Observe(duration) + session_metrics.StatementPerTransactionPessimisticErrorGeneral.Observe(float64(counter)) } } else { if isInternal { - transactionDurationPessimisticCommitInternal.Observe(duration) - statementPerTransactionPessimisticOKInternal.Observe(float64(counter)) + session_metrics.TransactionDurationPessimisticCommitInternal.Observe(duration) + session_metrics.StatementPerTransactionPessimisticOKInternal.Observe(float64(counter)) } else { - transactionDurationPessimisticCommitGeneral.Observe(duration) - statementPerTransactionPessimisticOKGeneral.Observe(float64(counter)) + session_metrics.TransactionDurationPessimisticCommitGeneral.Observe(duration) + session_metrics.StatementPerTransactionPessimisticOKGeneral.Observe(float64(counter)) } } } else { if err != nil { if isInternal { - transactionDurationOptimisticAbortInternal.Observe(duration) - statementPerTransactionOptimisticErrorInternal.Observe(float64(counter)) + session_metrics.TransactionDurationOptimisticAbortInternal.Observe(duration) + session_metrics.StatementPerTransactionOptimisticErrorInternal.Observe(float64(counter)) } else { - transactionDurationOptimisticAbortGeneral.Observe(duration) - statementPerTransactionOptimisticErrorGeneral.Observe(float64(counter)) + session_metrics.TransactionDurationOptimisticAbortGeneral.Observe(duration) + session_metrics.StatementPerTransactionOptimisticErrorGeneral.Observe(float64(counter)) } } else { if isInternal { - transactionDurationOptimisticCommitInternal.Observe(duration) - statementPerTransactionOptimisticOKInternal.Observe(float64(counter)) + session_metrics.TransactionDurationOptimisticCommitInternal.Observe(duration) + session_metrics.StatementPerTransactionOptimisticOKInternal.Observe(float64(counter)) } else { - transactionDurationOptimisticCommitGeneral.Observe(duration) - statementPerTransactionOptimisticOKGeneral.Observe(float64(counter)) + session_metrics.TransactionDurationOptimisticCommitGeneral.Observe(duration) + session_metrics.StatementPerTransactionOptimisticOKGeneral.Observe(float64(counter)) } } } @@ -4053,83 +3998,83 @@ func (s *session) updateTelemetryMetric(es *executor.ExecStmt) { ti := es.Ti if ti.UseRecursive { - telemetryCTEUsageRecurCTE.Inc() + session_metrics.TelemetryCTEUsageRecurCTE.Inc() } else if ti.UseNonRecursive { - telemetryCTEUsageNonRecurCTE.Inc() + session_metrics.TelemetryCTEUsageNonRecurCTE.Inc() } else { - telemetryCTEUsageNotCTE.Inc() + session_metrics.TelemetryCTEUsageNotCTE.Inc() } if ti.UseIndexMerge { - telemetryIndexMerge.Inc() + session_metrics.TelemetryIndexMerge.Inc() } if ti.UseMultiSchemaChange { - telemetryMultiSchemaChangeUsage.Inc() + session_metrics.TelemetryMultiSchemaChangeUsage.Inc() } if ti.UseFlashbackToCluster { - telemetryFlashbackClusterUsage.Inc() + session_metrics.TelemetryFlashbackClusterUsage.Inc() } if ti.UseExchangePartition { - telemetryExchangePartitionUsage.Inc() + session_metrics.TelemetryExchangePartitionUsage.Inc() } if ti.PartitionTelemetry != nil { if ti.PartitionTelemetry.UseTablePartition { - telemetryTablePartitionUsage.Inc() - telemetryTablePartitionMaxPartitionsUsage.Add(float64(ti.PartitionTelemetry.TablePartitionMaxPartitionsNum)) + session_metrics.TelemetryTablePartitionUsage.Inc() + session_metrics.TelemetryTablePartitionMaxPartitionsUsage.Add(float64(ti.PartitionTelemetry.TablePartitionMaxPartitionsNum)) } if ti.PartitionTelemetry.UseTablePartitionList { - telemetryTablePartitionListUsage.Inc() + session_metrics.TelemetryTablePartitionListUsage.Inc() } if ti.PartitionTelemetry.UseTablePartitionRange { - telemetryTablePartitionRangeUsage.Inc() + session_metrics.TelemetryTablePartitionRangeUsage.Inc() } if ti.PartitionTelemetry.UseTablePartitionHash { - telemetryTablePartitionHashUsage.Inc() + session_metrics.TelemetryTablePartitionHashUsage.Inc() } if ti.PartitionTelemetry.UseTablePartitionRangeColumns { - telemetryTablePartitionRangeColumnsUsage.Inc() + session_metrics.TelemetryTablePartitionRangeColumnsUsage.Inc() } if ti.PartitionTelemetry.UseTablePartitionRangeColumnsGt1 { - telemetryTablePartitionRangeColumnsGt1Usage.Inc() + session_metrics.TelemetryTablePartitionRangeColumnsGt1Usage.Inc() } if ti.PartitionTelemetry.UseTablePartitionRangeColumnsGt2 { - telemetryTablePartitionRangeColumnsGt2Usage.Inc() + session_metrics.TelemetryTablePartitionRangeColumnsGt2Usage.Inc() } if ti.PartitionTelemetry.UseTablePartitionRangeColumnsGt3 { - telemetryTablePartitionRangeColumnsGt3Usage.Inc() + session_metrics.TelemetryTablePartitionRangeColumnsGt3Usage.Inc() } if ti.PartitionTelemetry.UseTablePartitionListColumns { - telemetryTablePartitionListColumnsUsage.Inc() + session_metrics.TelemetryTablePartitionListColumnsUsage.Inc() } if ti.PartitionTelemetry.UseCreateIntervalPartition { - telemetryTablePartitionCreateIntervalUsage.Inc() + session_metrics.TelemetryTablePartitionCreateIntervalUsage.Inc() } if ti.PartitionTelemetry.UseAddIntervalPartition { - telemetryTablePartitionAddIntervalUsage.Inc() + session_metrics.TelemetryTablePartitionAddIntervalUsage.Inc() } if ti.PartitionTelemetry.UseDropIntervalPartition { - telemetryTablePartitionDropIntervalUsage.Inc() + session_metrics.TelemetryTablePartitionDropIntervalUsage.Inc() } if ti.PartitionTelemetry.UseCompactTablePartition { - telemetryTableCompactPartitionUsage.Inc() + session_metrics.TelemetryTableCompactPartitionUsage.Inc() } if ti.PartitionTelemetry.UseReorganizePartition { - telemetryReorganizePartitionUsage.Inc() + session_metrics.TelemetryReorganizePartitionUsage.Inc() } } if ti.AccountLockTelemetry != nil { - telemetryLockUserUsage.Add(float64(ti.AccountLockTelemetry.LockUser)) - telemetryUnlockUserUsage.Add(float64(ti.AccountLockTelemetry.UnlockUser)) - telemetryCreateOrAlterUserUsage.Add(float64(ti.AccountLockTelemetry.CreateOrAlterUser)) + session_metrics.TelemetryLockUserUsage.Add(float64(ti.AccountLockTelemetry.LockUser)) + session_metrics.TelemetryUnlockUserUsage.Add(float64(ti.AccountLockTelemetry.UnlockUser)) + session_metrics.TelemetryCreateOrAlterUserUsage.Add(float64(ti.AccountLockTelemetry.CreateOrAlterUser)) } if ti.UseTableLookUp.Load() && s.sessionVars.StoreBatchSize > 0 { - telemetryStoreBatchedUsage.Inc() + session_metrics.TelemetryStoreBatchedUsage.Inc() } } diff --git a/session/tidb.go b/session/tidb.go index c5b95b6999afb..b589f5fa88905 100644 --- a/session/tidb.go +++ b/session/tidb.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" + session_metrics "github.com/pingcap/tidb/session/metrics" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/sessiontxn" @@ -224,15 +225,15 @@ func recordAbortTxnDuration(sessVars *variable.SessionVars, isInternal bool) { duration := time.Since(sessVars.TxnCtx.CreateTime).Seconds() if sessVars.TxnCtx.IsPessimistic { if isInternal { - transactionDurationPessimisticAbortInternal.Observe(duration) + session_metrics.TransactionDurationPessimisticAbortInternal.Observe(duration) } else { - transactionDurationPessimisticAbortGeneral.Observe(duration) + session_metrics.TransactionDurationPessimisticAbortGeneral.Observe(duration) } } else { if isInternal { - transactionDurationOptimisticAbortInternal.Observe(duration) + session_metrics.TransactionDurationOptimisticAbortInternal.Observe(duration) } else { - transactionDurationOptimisticAbortGeneral.Observe(duration) + session_metrics.TransactionDurationOptimisticAbortGeneral.Observe(duration) } } } diff --git a/session/txninfo/txn_info.go b/session/txninfo/txn_info.go index 31e4d338eb623..5e19fa15fe676 100644 --- a/session/txninfo/txn_info.go +++ b/session/txninfo/txn_info.go @@ -47,38 +47,48 @@ const ( TxnStateCounter ) -var txnDurationHistogramForState [][]prometheus.Observer = [][]prometheus.Observer{ - { - metrics.TxnDurationHistogram.WithLabelValues("idle", "false"), - metrics.TxnDurationHistogram.WithLabelValues("idle", "true"), - }, - { - metrics.TxnDurationHistogram.WithLabelValues("executing_sql", "false"), - metrics.TxnDurationHistogram.WithLabelValues("executing_sql", "true"), - }, - { - metrics.TxnDurationHistogram.WithLabelValues("acquiring_lock", "false"), - metrics.TxnDurationHistogram.WithLabelValues("acquiring_lock", "true"), - }, - { - metrics.TxnDurationHistogram.WithLabelValues("committing", "false"), - metrics.TxnDurationHistogram.WithLabelValues("committing", "true"), - }, - { - metrics.TxnDurationHistogram.WithLabelValues("rolling_back", "false"), - metrics.TxnDurationHistogram.WithLabelValues("rolling_back", "true"), - }, -} +var ( + txnDurationHistogramForState [][]prometheus.Observer + txnStatusEnteringCounterForState []prometheus.Counter +) -var txnStatusEnteringCounterForState []prometheus.Counter = []prometheus.Counter{ - metrics.TxnStatusEnteringCounter.WithLabelValues("idle"), - metrics.TxnStatusEnteringCounter.WithLabelValues("executing_sql"), - metrics.TxnStatusEnteringCounter.WithLabelValues("acquiring_lock"), - metrics.TxnStatusEnteringCounter.WithLabelValues("committing"), - metrics.TxnStatusEnteringCounter.WithLabelValues("rolling_back"), +func init() { + InitMetricsVars() } -func init() { +// InitMetricsVars init transaction metrics vars. +func InitMetricsVars() { + txnDurationHistogramForState = [][]prometheus.Observer{ + { + metrics.TxnDurationHistogram.WithLabelValues("idle", "false"), + metrics.TxnDurationHistogram.WithLabelValues("idle", "true"), + }, + { + metrics.TxnDurationHistogram.WithLabelValues("executing_sql", "false"), + metrics.TxnDurationHistogram.WithLabelValues("executing_sql", "true"), + }, + { + metrics.TxnDurationHistogram.WithLabelValues("acquiring_lock", "false"), + metrics.TxnDurationHistogram.WithLabelValues("acquiring_lock", "true"), + }, + { + metrics.TxnDurationHistogram.WithLabelValues("committing", "false"), + metrics.TxnDurationHistogram.WithLabelValues("committing", "true"), + }, + { + metrics.TxnDurationHistogram.WithLabelValues("rolling_back", "false"), + metrics.TxnDurationHistogram.WithLabelValues("rolling_back", "true"), + }, + } + + txnStatusEnteringCounterForState = []prometheus.Counter{ + metrics.TxnStatusEnteringCounter.WithLabelValues("idle"), + metrics.TxnStatusEnteringCounter.WithLabelValues("executing_sql"), + metrics.TxnStatusEnteringCounter.WithLabelValues("acquiring_lock"), + metrics.TxnStatusEnteringCounter.WithLabelValues("committing"), + metrics.TxnStatusEnteringCounter.WithLabelValues("rolling_back"), + } + if len(txnDurationHistogramForState) != int(TxnStateCounter) { panic("len(txnDurationHistogramForState) != TxnStateCounter") } diff --git a/sessiontxn/isolation/BUILD.bazel b/sessiontxn/isolation/BUILD.bazel index de291d289074f..f4fe40db4cfce 100644 --- a/sessiontxn/isolation/BUILD.bazel +++ b/sessiontxn/isolation/BUILD.bazel @@ -15,7 +15,6 @@ go_library( "//config", "//infoschema", "//kv", - "//metrics", "//parser/ast", "//parser/mysql", "//parser/terror", @@ -24,6 +23,7 @@ go_library( "//sessionctx/variable", "//sessiontxn", "//sessiontxn/internal", + "//sessiontxn/isolation/metrics", "//sessiontxn/staleread", "//table/temptable", "//util/logutil", diff --git a/sessiontxn/isolation/metrics/BUILD.bazel b/sessiontxn/isolation/metrics/BUILD.bazel new file mode 100644 index 0000000000000..fcb4bcf68b783 --- /dev/null +++ b/sessiontxn/isolation/metrics/BUILD.bazel @@ -0,0 +1,12 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "metrics", + srcs = ["metrics.go"], + importpath = "github.com/pingcap/tidb/sessiontxn/isolation/metrics", + visibility = ["//visibility:public"], + deps = [ + "//metrics", + "@com_github_prometheus_client_golang//prometheus", + ], +) diff --git a/sessiontxn/isolation/metrics/metrics.go b/sessiontxn/isolation/metrics/metrics.go new file mode 100644 index 0000000000000..8c95bdf8461f1 --- /dev/null +++ b/sessiontxn/isolation/metrics/metrics.go @@ -0,0 +1,36 @@ +// Copyright 2023 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package isolation + +import ( + "github.com/pingcap/tidb/metrics" + "github.com/prometheus/client_golang/prometheus" +) + +// isolation metrics vars +var ( + RcReadCheckTSWriteConfilictCounter prometheus.Counter + RcWriteCheckTSWriteConfilictCounter prometheus.Counter +) + +func init() { + InitMetricsVars() +} + +// InitMetricsVars init isolation metrics vars. +func InitMetricsVars() { + RcReadCheckTSWriteConfilictCounter = metrics.RCCheckTSWriteConfilictCounter.WithLabelValues(metrics.LblRCReadCheckTS) + RcWriteCheckTSWriteConfilictCounter = metrics.RCCheckTSWriteConfilictCounter.WithLabelValues(metrics.LblRCWriteCheckTS) +} diff --git a/sessiontxn/isolation/readcommitted.go b/sessiontxn/isolation/readcommitted.go index 80cfcc4e4f5d8..33fd82dce454d 100644 --- a/sessiontxn/isolation/readcommitted.go +++ b/sessiontxn/isolation/readcommitted.go @@ -21,24 +21,19 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/terror" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/sessiontxn" + isolation_metrics "github.com/pingcap/tidb/sessiontxn/isolation/metrics" "github.com/pingcap/tidb/util/logutil" tikverr "github.com/tikv/client-go/v2/error" "github.com/tikv/client-go/v2/oracle" "go.uber.org/zap" ) -var ( - rcReadCheckTSWriteConfilictCounter = metrics.RCCheckTSWriteConfilictCounter.WithLabelValues(metrics.LblRCReadCheckTS) - rcWriteCheckTSWriteConfilictCounter = metrics.RCCheckTSWriteConfilictCounter.WithLabelValues(metrics.LblRCWriteCheckTS) -) - type stmtState struct { stmtTS uint64 stmtTSFuture oracle.Future @@ -208,7 +203,7 @@ func (p *PessimisticRCTxnContextProvider) handleAfterQueryError(queryErr error) return sessiontxn.NoIdea() } - rcReadCheckTSWriteConfilictCounter.Inc() + isolation_metrics.RcReadCheckTSWriteConfilictCounter.Inc() logutil.Logger(p.ctx).Info("RC read with ts checking has failed, retry RC read", zap.String("sql", sessVars.StmtCtx.OriginalSQL), zap.Error(queryErr)) @@ -240,7 +235,7 @@ func (p *PessimisticRCTxnContextProvider) handleAfterPessimisticLockError(ctx co zap.String("err", lockErr.Error())) retryable = true if p.checkTSInWriteStmt { - rcWriteCheckTSWriteConfilictCounter.Inc() + isolation_metrics.RcWriteCheckTSWriteConfilictCounter.Inc() } } diff --git a/statistics/handle/BUILD.bazel b/statistics/handle/BUILD.bazel index 34fa9b315644e..50f011f8e648e 100644 --- a/statistics/handle/BUILD.bazel +++ b/statistics/handle/BUILD.bazel @@ -31,6 +31,7 @@ go_library( "//sessionctx/variable", "//sessiontxn", "//statistics", + "//statistics/handle/metrics", "//table", "//types", "//util", @@ -49,7 +50,6 @@ go_library( "@com_github_pingcap_failpoint//:failpoint", "@com_github_pingcap_log//:log", "@com_github_pingcap_tipb//go-tipb", - "@com_github_prometheus_client_golang//prometheus", "@com_github_tikv_client_go_v2//oracle", "@org_golang_x_exp//slices", "@org_uber_go_atomic//:atomic", diff --git a/statistics/handle/dump.go b/statistics/handle/dump.go index 016fc858857ca..69c3370c38b53 100644 --- a/statistics/handle/dump.go +++ b/statistics/handle/dump.go @@ -23,13 +23,13 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" + handle_metrics "github.com/pingcap/tidb/statistics/handle/metrics" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/sqlexec" @@ -135,11 +135,6 @@ func (h *Handle) DumpStatsToJSON(dbName string, tableInfo *model.TableInfo, return h.DumpStatsToJSONBySnapshot(dbName, tableInfo, snapshot, dumpPartitionStats) } -var ( - dumpHistoricalStatsSuccessCounter = metrics.HistoricalStatsCounter.WithLabelValues("dump", "success") - dumpHistoricalStatsFailedCounter = metrics.HistoricalStatsCounter.WithLabelValues("dump", "fail") -) - // DumpHistoricalStatsBySnapshot dumped json tables from mysql.stats_meta_history and mysql.stats_history func (h *Handle) DumpHistoricalStatsBySnapshot(dbName string, tableInfo *model.TableInfo, snapshot uint64) (jt *JSONTable, err error) { historicalStatsEnabled, err := h.CheckHistoricalStatsEnable() @@ -152,9 +147,9 @@ func (h *Handle) DumpHistoricalStatsBySnapshot(dbName string, tableInfo *model.T defer func() { if err == nil { - dumpHistoricalStatsSuccessCounter.Inc() + handle_metrics.DumpHistoricalStatsSuccessCounter.Inc() } else { - dumpHistoricalStatsFailedCounter.Inc() + handle_metrics.DumpHistoricalStatsFailedCounter.Inc() } }() pi := tableInfo.GetPartitionInfo() diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index b2c472c2d2c20..c028066a3cd02 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -31,7 +31,6 @@ import ( ddlUtil "github.com/pingcap/tidb/ddl/util" "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/model" "github.com/pingcap/tidb/parser/mysql" @@ -39,6 +38,7 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" + handle_metrics "github.com/pingcap/tidb/statistics/handle/metrics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" @@ -48,7 +48,6 @@ import ( "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/syncutil" - "github.com/prometheus/client_golang/prometheus" "github.com/tikv/client-go/v2/oracle" atomic2 "go.uber.org/atomic" "go.uber.org/zap" @@ -530,15 +529,6 @@ func DurationToTS(d time.Duration) uint64 { return oracle.ComposeTS(d.Nanoseconds()/int64(time.Millisecond), 0) } -var statsHealthyGauges = []prometheus.Gauge{ - metrics.StatsHealthyGauge.WithLabelValues("[0,50)"), - metrics.StatsHealthyGauge.WithLabelValues("[50,80)"), - metrics.StatsHealthyGauge.WithLabelValues("[80,100)"), - metrics.StatsHealthyGauge.WithLabelValues("[100,100]"), - // [0,100] should always be the last - metrics.StatsHealthyGauge.WithLabelValues("[0,100]"), -} - // UpdateStatsHealthyMetrics updates stats healthy distribution metrics according to stats cache. func (h *Handle) UpdateStatsHealthyMetrics() { v := h.statsCache.Load() @@ -564,7 +554,7 @@ func (h *Handle) UpdateStatsHealthyMetrics() { distribution[4] += 1 } for i, val := range distribution { - statsHealthyGauges[i].Set(float64(val)) + handle_metrics.StatsHealthyGauges[i].Set(float64(val)) } } @@ -1026,7 +1016,7 @@ func (h *Handle) updateStatsCache(newCache statsCache) (updated bool) { } h.statsCache.Unlock() if updated && enableQuota { - costGauge.Set(float64(newCost)) + handle_metrics.CostGauge.Set(float64(newCost)) } return } diff --git a/statistics/handle/lru_cache.go b/statistics/handle/lru_cache.go index df36912631264..701b0f1168a77 100644 --- a/statistics/handle/lru_cache.go +++ b/statistics/handle/lru_cache.go @@ -19,18 +19,8 @@ import ( "math" "sync" - "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/statistics" -) - -var ( - missCounter = metrics.StatsCacheLRUCounter.WithLabelValues("miss") - hitCounter = metrics.StatsCacheLRUCounter.WithLabelValues("hit") - updateCounter = metrics.StatsCacheLRUCounter.WithLabelValues("update") - delCounter = metrics.StatsCacheLRUCounter.WithLabelValues("del") - evictCounter = metrics.StatsCacheLRUCounter.WithLabelValues("evict") - costGauge = metrics.StatsCacheLRUGauge.WithLabelValues("track") - capacityGauge = metrics.StatsCacheLRUGauge.WithLabelValues("capacity") + handle_metrics "github.com/pingcap/tidb/statistics/handle/metrics" ) type statsInnerCache struct { @@ -62,7 +52,7 @@ func newInnerLruCache(c int64) *innerItemLruCache { if c < 1 { c = math.MaxInt64 } - capacityGauge.Set(float64(c)) + handle_metrics.CapacityGauge.Set(float64(c)) return &innerItemLruCache{ capacity: c, cache: list.New(), @@ -337,20 +327,20 @@ func (s *statsInnerCache) capacity() int64 { func (c *innerItemLruCache) get(tblID, id int64, isIndex bool) (*lruCacheItem, bool) { v, ok := c.elements[tblID] if !ok { - missCounter.Inc() + handle_metrics.MissCounter.Inc() return nil, false } isIndexSet, ok := v[isIndex] if !ok { - missCounter.Inc() + handle_metrics.MissCounter.Inc() return nil, false } ele, ok := isIndexSet[id] if !ok { - missCounter.Inc() + handle_metrics.MissCounter.Inc() return nil, false } - hitCounter.Inc() + handle_metrics.HitCounter.Inc() c.cache.MoveToFront(ele) return ele.Value.(*lruCacheItem), true } @@ -368,7 +358,7 @@ func (c *innerItemLruCache) del(tblID, id int64, isIndex bool) { if !ok { return } - delCounter.Inc() + handle_metrics.DelCounter.Inc() memUsage := c.elements[tblID][isIndex][id].Value.(*lruCacheItem).innerMemUsage delete(c.elements[tblID][isIndex], id) c.cache.Remove(ele) @@ -382,7 +372,7 @@ func (c *innerItemLruCache) del(tblID, id int64, isIndex bool) { func (c *innerItemLruCache) put(tblID, id int64, isIndex bool, item statistics.TableCacheItem, itemMem statistics.CacheItemMemoryUsage, needEvict, needMove bool) { defer func() { - updateCounter.Inc() + handle_metrics.UpdateCounter.Inc() if needEvict { c.evictIfNeeded() } @@ -431,7 +421,7 @@ func (c *innerItemLruCache) put(tblID, id int64, isIndex bool, item statistics.T func (c *innerItemLruCache) evictIfNeeded() { curr := c.cache.Back() for c.trackingCost > c.capacity && curr != nil { - evictCounter.Inc() + handle_metrics.EvictCounter.Inc() prev := curr.Prev() item := curr.Value.(*lruCacheItem) oldMem := item.innerMemUsage @@ -473,6 +463,6 @@ func (c *innerItemLruCache) setCapacity(capacity int64) { capacity = math.MaxInt64 } c.capacity = capacity - capacityGauge.Set(float64(c.capacity)) + handle_metrics.CapacityGauge.Set(float64(c.capacity)) c.evictIfNeeded() } diff --git a/statistics/handle/metrics/BUILD.bazel b/statistics/handle/metrics/BUILD.bazel new file mode 100644 index 0000000000000..0c12cf8d2ee2d --- /dev/null +++ b/statistics/handle/metrics/BUILD.bazel @@ -0,0 +1,12 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "metrics", + srcs = ["metrics.go"], + importpath = "github.com/pingcap/tidb/statistics/handle/metrics", + visibility = ["//visibility:public"], + deps = [ + "//metrics", + "@com_github_prometheus_client_golang//prometheus", + ], +) diff --git a/statistics/handle/metrics/metrics.go b/statistics/handle/metrics/metrics.go new file mode 100644 index 0000000000000..9e06ccf1aa120 --- /dev/null +++ b/statistics/handle/metrics/metrics.go @@ -0,0 +1,62 @@ +// Copyright 2023 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package metrics + +import ( + "github.com/pingcap/tidb/metrics" + "github.com/prometheus/client_golang/prometheus" +) + +// statistics metrics vars +var ( + MissCounter prometheus.Counter + HitCounter prometheus.Counter + UpdateCounter prometheus.Counter + DelCounter prometheus.Counter + EvictCounter prometheus.Counter + CostGauge prometheus.Gauge + CapacityGauge prometheus.Gauge + StatsHealthyGauges []prometheus.Gauge + + DumpHistoricalStatsSuccessCounter prometheus.Counter + DumpHistoricalStatsFailedCounter prometheus.Counter +) + +func init() { + InitMetricsVars() +} + +// InitMetricsVars init statistics metrics vars. +func InitMetricsVars() { + MissCounter = metrics.StatsCacheLRUCounter.WithLabelValues("miss") + HitCounter = metrics.StatsCacheLRUCounter.WithLabelValues("hit") + UpdateCounter = metrics.StatsCacheLRUCounter.WithLabelValues("update") + DelCounter = metrics.StatsCacheLRUCounter.WithLabelValues("del") + EvictCounter = metrics.StatsCacheLRUCounter.WithLabelValues("evict") + CostGauge = metrics.StatsCacheLRUGauge.WithLabelValues("track") + CapacityGauge = metrics.StatsCacheLRUGauge.WithLabelValues("capacity") + + StatsHealthyGauges = []prometheus.Gauge{ + metrics.StatsHealthyGauge.WithLabelValues("[0,50)"), + metrics.StatsHealthyGauge.WithLabelValues("[50,80)"), + metrics.StatsHealthyGauge.WithLabelValues("[80,100)"), + metrics.StatsHealthyGauge.WithLabelValues("[100,100]"), + // [0,100] should always be the last + metrics.StatsHealthyGauge.WithLabelValues("[0,100]"), + } + + DumpHistoricalStatsSuccessCounter = metrics.HistoricalStatsCounter.WithLabelValues("dump", "success") + DumpHistoricalStatsFailedCounter = metrics.HistoricalStatsCounter.WithLabelValues("dump", "fail") +} diff --git a/store/copr/BUILD.bazel b/store/copr/BUILD.bazel index f970fa578222f..bb5f1d2d29c62 100644 --- a/store/copr/BUILD.bazel +++ b/store/copr/BUILD.bazel @@ -22,6 +22,7 @@ go_library( "//kv", "//metrics", "//parser/terror", + "//store/copr/metrics", "//store/driver/backoff", "//store/driver/error", "//store/driver/options", diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index 08716210c24c5..7c9bc95a70b54 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -37,6 +37,7 @@ import ( "github.com/pingcap/tidb/kv" tidbmetrics "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser/terror" + copr_metrics "github.com/pingcap/tidb/store/copr/metrics" "github.com/pingcap/tidb/store/driver/backoff" derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/driver/options" @@ -57,13 +58,6 @@ import ( "go.uber.org/zap" ) -var coprCacheCounterEvict = tidbmetrics.DistSQLCoprCacheCounter.WithLabelValues("evict") - -var ( - coprCacheCounterHit = tidbmetrics.DistSQLCoprCacheCounter.WithLabelValues("hit") - coprCacheCounterMiss = tidbmetrics.DistSQLCoprCacheCounter.WithLabelValues("miss") -) - // Maximum total sleep time(in ms) for kv/cop commands. const ( copBuildTaskMaxBackoff = 5000 @@ -1106,7 +1100,7 @@ func (worker *copIteratorWorker) handleTask(ctx context.Context, task *copTask, } } if worker.store.coprCache != nil && worker.store.coprCache.cache.Metrics != nil { - coprCacheCounterEvict.Add(float64(worker.store.coprCache.cache.Metrics.KeysEvicted())) + copr_metrics.CoprCacheCounterEvict.Add(float64(worker.store.coprCache.cache.Metrics.KeysEvicted())) } } @@ -1552,7 +1546,7 @@ func (worker *copIteratorWorker) handleCopCache(task *copTask, resp *copResponse if cacheValue == nil { return errors.New("Internal error: received illegal TiKV response") } - coprCacheCounterHit.Add(1) + copr_metrics.CoprCacheCounterHit.Add(1) // Cache hit and is valid: use cached data as response data and we don't update the cache. data := make([]byte, len(cacheValue.Data)) copy(data, cacheValue.Data) @@ -1580,7 +1574,7 @@ func (worker *copIteratorWorker) handleCopCache(task *copTask, resp *copResponse resp.detail.CoprCacheHit = true return nil } - coprCacheCounterMiss.Add(1) + copr_metrics.CoprCacheCounterMiss.Add(1) // Cache not hit or cache hit but not valid: update the cache if the response can be cached. if cacheKey != nil && resp.pbResp.CanBeCached && resp.pbResp.CacheLastVersion > 0 { if resp.detail != nil { diff --git a/store/copr/metrics/BUILD.bazel b/store/copr/metrics/BUILD.bazel new file mode 100644 index 0000000000000..fca7dcf58b045 --- /dev/null +++ b/store/copr/metrics/BUILD.bazel @@ -0,0 +1,12 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "metrics", + srcs = ["metrics.go"], + importpath = "github.com/pingcap/tidb/store/copr/metrics", + visibility = ["//visibility:public"], + deps = [ + "//metrics", + "@com_github_prometheus_client_golang//prometheus", + ], +) diff --git a/store/copr/metrics/metrics.go b/store/copr/metrics/metrics.go new file mode 100644 index 0000000000000..92b59ecbc10dd --- /dev/null +++ b/store/copr/metrics/metrics.go @@ -0,0 +1,38 @@ +// Copyright 2023 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package copr + +import ( + "github.com/pingcap/tidb/metrics" + "github.com/prometheus/client_golang/prometheus" +) + +// copr metrics vars +var ( + CoprCacheCounterEvict prometheus.Counter + CoprCacheCounterHit prometheus.Counter + CoprCacheCounterMiss prometheus.Counter +) + +func init() { + InitMetricsVars() +} + +// InitMetricsVars init copr metrics vars. +func InitMetricsVars() { + CoprCacheCounterEvict = metrics.DistSQLCoprCacheCounter.WithLabelValues("evict") + CoprCacheCounterHit = metrics.DistSQLCoprCacheCounter.WithLabelValues("hit") + CoprCacheCounterMiss = metrics.DistSQLCoprCacheCounter.WithLabelValues("miss") +} diff --git a/tidb-server/BUILD.bazel b/tidb-server/BUILD.bazel index 13a974e19dde7..9893f0dccb606 100644 --- a/tidb-server/BUILD.bazel +++ b/tidb-server/BUILD.bazel @@ -33,7 +33,6 @@ go_library( "//store/copr", "//store/driver", "//store/mockstore", - "//store/mockstore/unistore/metrics", "//tidb-binlog/pump_client", "//util", "//util/chunk", @@ -44,6 +43,7 @@ go_library( "//util/kvcache", "//util/logutil", "//util/memory", + "//util/metricsutil", "//util/printer", "//util/sem", "//util/signal", diff --git a/tidb-server/main.go b/tidb-server/main.go index 41296980e7ec3..0b69c0b577f78 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -58,7 +58,6 @@ import ( "github.com/pingcap/tidb/store/copr" "github.com/pingcap/tidb/store/driver" "github.com/pingcap/tidb/store/mockstore" - uni_metrics "github.com/pingcap/tidb/store/mockstore/unistore/metrics" pumpcli "github.com/pingcap/tidb/tidb-binlog/pump_client" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" @@ -69,6 +68,7 @@ import ( "github.com/pingcap/tidb/util/kvcache" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" + "github.com/pingcap/tidb/util/metricsutil" "github.com/pingcap/tidb/util/printer" "github.com/pingcap/tidb/util/sem" "github.com/pingcap/tidb/util/signal" @@ -195,7 +195,9 @@ func main() { os.Exit(0) } registerStores() - registerMetrics() + err := metricsutil.RegisterMetrics() + terror.MustNil(err) + if variable.EnableTmpStorageOnOOM.Load() { config.GetGlobalConfig().UpdateTempStoragePath() err := disk.InitializeTempDir() @@ -206,7 +208,7 @@ func main() { setupExtensions() setupStmtSummary() - err := cpuprofile.StartCPUProfiler() + err = cpuprofile.StartCPUProfiler() terror.MustNil(err) if config.GetGlobalConfig().DisaggregatedTiFlash && config.GetGlobalConfig().UseAutoScaler { @@ -326,13 +328,6 @@ func registerStores() { terror.MustNil(err) } -func registerMetrics() { - metrics.RegisterMetrics() - if config.GetGlobalConfig().Store == "unistore" { - uni_metrics.RegisterMetrics() - } -} - func createStoreAndDomain(keyspaceName string) (kv.Storage, *domain.Domain) { cfg := config.GetGlobalConfig() var fullPath string diff --git a/ttl/metrics/metrics.go b/ttl/metrics/metrics.go index da9549cdb023d..3389cf16f5d7f 100644 --- a/ttl/metrics/metrics.go +++ b/ttl/metrics/metrics.go @@ -37,21 +37,43 @@ var ( // TTL metrics var ( + SelectSuccessDuration prometheus.Observer + SelectErrorDuration prometheus.Observer + DeleteSuccessDuration prometheus.Observer + DeleteErrorDuration prometheus.Observer + + ScannedExpiredRows prometheus.Counter + DeleteSuccessExpiredRows prometheus.Counter + DeleteErrorExpiredRows prometheus.Counter + + RunningJobsCnt prometheus.Gauge + CancellingJobsCnt prometheus.Gauge + + ScanningTaskCnt prometheus.Gauge + DeletingTaskCnt prometheus.Gauge +) + +func init() { + InitMetricsVars() +} + +// InitMetricsVars init ttl metrics vars vars. +func InitMetricsVars() { SelectSuccessDuration = metrics.TTLQueryDuration.With(prometheus.Labels{metrics.LblSQLType: "select", metrics.LblResult: metrics.LblOK}) - SelectErrorDuration = metrics.TTLQueryDuration.With(prometheus.Labels{metrics.LblSQLType: "select", metrics.LblResult: metrics.LblError}) + SelectErrorDuration = metrics.TTLQueryDuration.With(prometheus.Labels{metrics.LblSQLType: "select", metrics.LblResult: metrics.LblError}) DeleteSuccessDuration = metrics.TTLQueryDuration.With(prometheus.Labels{metrics.LblSQLType: "delete", metrics.LblResult: metrics.LblOK}) - DeleteErrorDuration = metrics.TTLQueryDuration.With(prometheus.Labels{metrics.LblSQLType: "delete", metrics.LblResult: metrics.LblError}) + DeleteErrorDuration = metrics.TTLQueryDuration.With(prometheus.Labels{metrics.LblSQLType: "delete", metrics.LblResult: metrics.LblError}) - ScannedExpiredRows = metrics.TTLProcessedExpiredRowsCounter.With(prometheus.Labels{metrics.LblSQLType: "select", metrics.LblResult: metrics.LblOK}) + ScannedExpiredRows = metrics.TTLProcessedExpiredRowsCounter.With(prometheus.Labels{metrics.LblSQLType: "select", metrics.LblResult: metrics.LblOK}) DeleteSuccessExpiredRows = metrics.TTLProcessedExpiredRowsCounter.With(prometheus.Labels{metrics.LblSQLType: "delete", metrics.LblResult: metrics.LblOK}) - DeleteErrorExpiredRows = metrics.TTLProcessedExpiredRowsCounter.With(prometheus.Labels{metrics.LblSQLType: "delete", metrics.LblResult: metrics.LblError}) + DeleteErrorExpiredRows = metrics.TTLProcessedExpiredRowsCounter.With(prometheus.Labels{metrics.LblSQLType: "delete", metrics.LblResult: metrics.LblError}) - RunningJobsCnt = metrics.TTLJobStatus.With(prometheus.Labels{metrics.LblType: "running"}) + RunningJobsCnt = metrics.TTLJobStatus.With(prometheus.Labels{metrics.LblType: "running"}) CancellingJobsCnt = metrics.TTLJobStatus.With(prometheus.Labels{metrics.LblType: "cancelling"}) ScanningTaskCnt = metrics.TTLTaskStatus.With(prometheus.Labels{metrics.LblType: "scanning"}) DeletingTaskCnt = metrics.TTLTaskStatus.With(prometheus.Labels{metrics.LblType: "deleting"}) -) +} func initWorkerPhases(workerType string) map[string]prometheus.Counter { return map[string]prometheus.Counter{ diff --git a/util/metricsutil/BUILD.bazel b/util/metricsutil/BUILD.bazel new file mode 100644 index 0000000000000..0d01a684a2746 --- /dev/null +++ b/util/metricsutil/BUILD.bazel @@ -0,0 +1,31 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "metricsutil", + srcs = ["common.go"], + importpath = "github.com/pingcap/tidb/util/metricsutil", + visibility = ["//visibility:public"], + deps = [ + "//config", + "//domain/metrics", + "//executor/metrics", + "//infoschema/metrics", + "//keyspace", + "//metrics", + "//planner/core/metrics", + "//server/metrics", + "//session/metrics", + "//session/txninfo", + "//sessiontxn/isolation/metrics", + "//statistics/handle/metrics", + "//store", + "//store/copr/metrics", + "//store/mockstore/unistore/metrics", + "//ttl/metrics", + "//util", + "//util/topsql/reporter/metrics", + "@com_github_pingcap_kvproto//pkg/keyspacepb", + "@com_github_tikv_client_go_v2//config", + "@com_github_tikv_pd_client//:client", + ], +) diff --git a/util/metricsutil/common.go b/util/metricsutil/common.go new file mode 100644 index 0000000000000..e62233754272a --- /dev/null +++ b/util/metricsutil/common.go @@ -0,0 +1,144 @@ +// Copyright 2023 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package metricsutil + +import ( + "context" + "fmt" + "strings" + "time" + + "github.com/pingcap/kvproto/pkg/keyspacepb" + "github.com/pingcap/tidb/config" + domain_metrics "github.com/pingcap/tidb/domain/metrics" + executor_metrics "github.com/pingcap/tidb/executor/metrics" + infoschema_metrics "github.com/pingcap/tidb/infoschema/metrics" + "github.com/pingcap/tidb/keyspace" + "github.com/pingcap/tidb/metrics" + plannercore "github.com/pingcap/tidb/planner/core/metrics" + server_metrics "github.com/pingcap/tidb/server/metrics" + session_metrics "github.com/pingcap/tidb/session/metrics" + txninfo "github.com/pingcap/tidb/session/txninfo" + isolation_metrics "github.com/pingcap/tidb/sessiontxn/isolation/metrics" + statshandler_metrics "github.com/pingcap/tidb/statistics/handle/metrics" + kvstore "github.com/pingcap/tidb/store" + copr_metrics "github.com/pingcap/tidb/store/copr/metrics" + unimetrics "github.com/pingcap/tidb/store/mockstore/unistore/metrics" + ttlmetrics "github.com/pingcap/tidb/ttl/metrics" + "github.com/pingcap/tidb/util" + topsqlreporter_metrics "github.com/pingcap/tidb/util/topsql/reporter/metrics" + tikvconfig "github.com/tikv/client-go/v2/config" + pd "github.com/tikv/pd/client" +) + +// RegisterMetrics register metrics with const label 'keyspace_id' if keyspaceName set. +func RegisterMetrics() error { + cfg := config.GetGlobalConfig() + if keyspace.IsKeyspaceNameEmpty(cfg.KeyspaceName) || strings.ToLower(cfg.Store) != "tikv" { + return registerMetrics(nil) // register metrics without label 'keyspace_id'. + } + + pdAddrs, _, _, err := tikvconfig.ParsePath("tikv://" + cfg.Path) + if err != nil { + return err + } + + timeoutSec := time.Duration(cfg.PDClient.PDServerTimeout) * time.Second + pdCli, err := pd.NewClient(pdAddrs, pd.SecurityOption{ + CAPath: cfg.Security.ClusterSSLCA, + CertPath: cfg.Security.ClusterSSLCert, + KeyPath: cfg.Security.ClusterSSLKey, + }, pd.WithCustomTimeoutOption(timeoutSec)) + if err != nil { + return err + } + defer pdCli.Close() + + keyspaceMeta, err := getKeyspaceMeta(pdCli, cfg.KeyspaceName) + if err != nil { + return err + } + + return registerMetrics(keyspaceMeta) +} + +// RegisterMetricsForBR register metrics with const label keyspace_id for BR. +func RegisterMetricsForBR(pdAddrs []string, keyspaceName string) error { + if keyspace.IsKeyspaceNameEmpty(keyspaceName) { + return registerMetrics(nil) // register metrics without label 'keyspace_id'. + } + + timeoutSec := 10 * time.Second + pdCli, err := pd.NewClient(pdAddrs, pd.SecurityOption{}, + pd.WithCustomTimeoutOption(timeoutSec)) + if err != nil { + return err + } + defer pdCli.Close() + + keyspaceMeta, err := getKeyspaceMeta(pdCli, keyspaceName) + if err != nil { + return err + } + + return registerMetrics(keyspaceMeta) +} + +func registerMetrics(keyspaceMeta *keyspacepb.KeyspaceMeta) error { + if keyspaceMeta != nil { + metrics.SetConstLabels("keyspace_id", fmt.Sprint(keyspaceMeta.GetId())) + } + + metrics.InitMetrics() + metrics.RegisterMetrics() + + copr_metrics.InitMetricsVars() + domain_metrics.InitMetricsVars() + executor_metrics.InitMetricsVars() + infoschema_metrics.InitMetricsVars() + isolation_metrics.InitMetricsVars() + plannercore.InitMetricsVars() + server_metrics.InitMetricsVars() + session_metrics.InitMetricsVars() + statshandler_metrics.InitMetricsVars() + topsqlreporter_metrics.InitMetricsVars() + ttlmetrics.InitMetricsVars() + txninfo.InitMetricsVars() + + if config.GetGlobalConfig().Store == "unistore" { + unimetrics.RegisterMetrics() + } + return nil +} + +func getKeyspaceMeta(pdCli pd.Client, keyspaceName string) (*keyspacepb.KeyspaceMeta, error) { + // Load Keyspace meta with retry. + var keyspaceMeta *keyspacepb.KeyspaceMeta + err := util.RunWithRetry(util.DefaultMaxRetries, util.RetryInterval, func() (bool, error) { + var errInner error + keyspaceMeta, errInner = pdCli.LoadKeyspace(context.TODO(), keyspaceName) + // Retry when pd not bootstrapped or if keyspace not exists. + if kvstore.IsNotBootstrappedError(errInner) || kvstore.IsKeyspaceNotExistError(errInner) { + return true, errInner + } + // Do not retry when success or encountered unexpected error. + return false, errInner + }) + if err != nil { + return nil, err + } + + return keyspaceMeta, nil +} diff --git a/util/topsql/reporter/BUILD.bazel b/util/topsql/reporter/BUILD.bazel index 98a83a0d00910..d910cc5f425cf 100644 --- a/util/topsql/reporter/BUILD.bazel +++ b/util/topsql/reporter/BUILD.bazel @@ -5,7 +5,6 @@ go_library( srcs = [ "datamodel.go", "datasink.go", - "metrics.go", "pubsub.go", "reporter.go", "single_target.go", @@ -14,11 +13,11 @@ go_library( visibility = ["//visibility:public"], deps = [ "//config", - "//metrics", "//util", "//util/hack", "//util/logutil", "//util/topsql/collector", + "//util/topsql/reporter/metrics", "//util/topsql/state", "//util/topsql/stmtstats", "@com_github_pingcap_errors//:errors", diff --git a/util/topsql/reporter/datamodel.go b/util/topsql/reporter/datamodel.go index e02ff9763ceab..edf5538288e12 100644 --- a/util/topsql/reporter/datamodel.go +++ b/util/topsql/reporter/datamodel.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/topsql/collector" + reporter_metrics "github.com/pingcap/tidb/util/topsql/reporter/metrics" topsqlstate "github.com/pingcap/tidb/util/topsql/state" "github.com/pingcap/tidb/util/topsql/stmtstats" "github.com/pingcap/tipb/go-tipb" @@ -617,7 +618,7 @@ func newNormalizedSQLMap() *normalizedSQLMap { // If the internal map size exceeds the limit, the relationship will be discarded. func (m *normalizedSQLMap) register(sqlDigest []byte, normalizedSQL string, isInternal bool) { if m.length.Load() >= topsqlstate.GlobalState.MaxCollect.Load() { - ignoreExceedSQLCounter.Inc() + reporter_metrics.IgnoreExceedSQLCounter.Inc() return } data := m.data.Load().(*sync.Map) @@ -681,7 +682,7 @@ func newNormalizedPlanMap() *normalizedPlanMap { // If the internal map size exceeds the limit, the relationship will be discarded. func (m *normalizedPlanMap) register(planDigest []byte, normalizedPlan string, isLarge bool) { if m.length.Load() >= topsqlstate.GlobalState.MaxCollect.Load() { - ignoreExceedPlanCounter.Inc() + reporter_metrics.IgnoreExceedPlanCounter.Inc() return } data := m.data.Load().(*sync.Map) diff --git a/util/topsql/reporter/metrics.go b/util/topsql/reporter/metrics.go deleted file mode 100644 index 0314956a6c795..0000000000000 --- a/util/topsql/reporter/metrics.go +++ /dev/null @@ -1,36 +0,0 @@ -// Copyright 2021 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, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package reporter - -import "github.com/pingcap/tidb/metrics" - -var ( - ignoreExceedSQLCounter = metrics.TopSQLIgnoredCounter.WithLabelValues("ignore_exceed_sql") - ignoreExceedPlanCounter = metrics.TopSQLIgnoredCounter.WithLabelValues("ignore_exceed_plan") - ignoreCollectChannelFullCounter = metrics.TopSQLIgnoredCounter.WithLabelValues("ignore_collect_channel_full") - ignoreCollectStmtChannelFullCounter = metrics.TopSQLIgnoredCounter.WithLabelValues("ignore_collect_stmt_channel_full") - ignoreReportChannelFullCounter = metrics.TopSQLIgnoredCounter.WithLabelValues("ignore_report_channel_full") - reportAllDurationSuccHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("all", metrics.LblOK) - reportAllDurationFailedHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("all", metrics.LblError) - reportRecordDurationSuccHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("record", metrics.LblOK) - reportRecordDurationFailedHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("record", metrics.LblError) - reportSQLDurationSuccHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("sql", metrics.LblOK) - reportSQLDurationFailedHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("sql", metrics.LblError) - reportPlanDurationSuccHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("plan", metrics.LblOK) - reportPlanDurationFailedHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("plan", metrics.LblError) - topSQLReportRecordCounterHistogram = metrics.TopSQLReportDataHistogram.WithLabelValues("record") - topSQLReportSQLCountHistogram = metrics.TopSQLReportDataHistogram.WithLabelValues("sql") - topSQLReportPlanCountHistogram = metrics.TopSQLReportDataHistogram.WithLabelValues("plan") -) diff --git a/util/topsql/reporter/metrics/BUILD.bazel b/util/topsql/reporter/metrics/BUILD.bazel new file mode 100644 index 0000000000000..cb37cd3dcb3c4 --- /dev/null +++ b/util/topsql/reporter/metrics/BUILD.bazel @@ -0,0 +1,12 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "metrics", + srcs = ["metrics.go"], + importpath = "github.com/pingcap/tidb/util/topsql/reporter/metrics", + visibility = ["//visibility:public"], + deps = [ + "//metrics", + "@com_github_prometheus_client_golang//prometheus", + ], +) diff --git a/util/topsql/reporter/metrics/metrics.go b/util/topsql/reporter/metrics/metrics.go new file mode 100644 index 0000000000000..325b2827946c4 --- /dev/null +++ b/util/topsql/reporter/metrics/metrics.go @@ -0,0 +1,64 @@ +// Copyright 2021 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package reporter + +import ( + "github.com/pingcap/tidb/metrics" + "github.com/prometheus/client_golang/prometheus" +) + +// reporter metrics vars +var ( + IgnoreExceedSQLCounter prometheus.Counter + IgnoreExceedPlanCounter prometheus.Counter + IgnoreCollectChannelFullCounter prometheus.Counter + IgnoreCollectStmtChannelFullCounter prometheus.Counter + IgnoreReportChannelFullCounter prometheus.Counter + ReportAllDurationSuccHistogram prometheus.Observer + ReportAllDurationFailedHistogram prometheus.Observer + ReportRecordDurationSuccHistogram prometheus.Observer + ReportRecordDurationFailedHistogram prometheus.Observer + ReportSQLDurationSuccHistogram prometheus.Observer + ReportSQLDurationFailedHistogram prometheus.Observer + ReportPlanDurationSuccHistogram prometheus.Observer + ReportPlanDurationFailedHistogram prometheus.Observer + TopSQLReportRecordCounterHistogram prometheus.Observer + TopSQLReportSQLCountHistogram prometheus.Observer + TopSQLReportPlanCountHistogram prometheus.Observer +) + +func init() { + InitMetricsVars() +} + +// InitMetricsVars init topsql reporter metrics vars. +func InitMetricsVars() { + IgnoreExceedSQLCounter = metrics.TopSQLIgnoredCounter.WithLabelValues("ignore_exceed_sql") + IgnoreExceedPlanCounter = metrics.TopSQLIgnoredCounter.WithLabelValues("ignore_exceed_plan") + IgnoreCollectChannelFullCounter = metrics.TopSQLIgnoredCounter.WithLabelValues("ignore_collect_channel_full") + IgnoreCollectStmtChannelFullCounter = metrics.TopSQLIgnoredCounter.WithLabelValues("ignore_collect_stmt_channel_full") + IgnoreReportChannelFullCounter = metrics.TopSQLIgnoredCounter.WithLabelValues("ignore_report_channel_full") + ReportAllDurationSuccHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("all", metrics.LblOK) + ReportAllDurationFailedHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("all", metrics.LblError) + ReportRecordDurationSuccHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("record", metrics.LblOK) + ReportRecordDurationFailedHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("record", metrics.LblError) + ReportSQLDurationSuccHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("sql", metrics.LblOK) + ReportSQLDurationFailedHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("sql", metrics.LblError) + ReportPlanDurationSuccHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("plan", metrics.LblOK) + ReportPlanDurationFailedHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("plan", metrics.LblError) + TopSQLReportRecordCounterHistogram = metrics.TopSQLReportDataHistogram.WithLabelValues("record") + TopSQLReportSQLCountHistogram = metrics.TopSQLReportDataHistogram.WithLabelValues("sql") + TopSQLReportPlanCountHistogram = metrics.TopSQLReportDataHistogram.WithLabelValues("plan") +} diff --git a/util/topsql/reporter/pubsub.go b/util/topsql/reporter/pubsub.go index c18c50f648ab1..cdf93c7c2aa3a 100644 --- a/util/topsql/reporter/pubsub.go +++ b/util/topsql/reporter/pubsub.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" + reporter_metrics "github.com/pingcap/tidb/util/topsql/reporter/metrics" "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" ) @@ -85,7 +86,7 @@ func (ds *pubSubDataSink) TrySend(data *ReportData, deadline time.Time) error { case <-ds.ctx.Done(): return ds.ctx.Err() default: - ignoreReportChannelFullCounter.Inc() + reporter_metrics.IgnoreReportChannelFullCounter.Inc() return errors.New("the channel of pubsub dataSink is full") } } @@ -112,9 +113,9 @@ func (ds *pubSubDataSink) run() error { err = ds.doSend(ctx, task.data) if err != nil { - reportAllDurationFailedHistogram.Observe(time.Since(start).Seconds()) + reporter_metrics.ReportAllDurationFailedHistogram.Observe(time.Since(start).Seconds()) } else { - reportAllDurationSuccHistogram.Observe(time.Since(start).Seconds()) + reporter_metrics.ReportAllDurationSuccHistogram.Observe(time.Since(start).Seconds()) } }, nil) @@ -163,11 +164,11 @@ func (ds *pubSubDataSink) sendTopSQLRecords(ctx context.Context, records []tipb. start := time.Now() sentCount := 0 defer func() { - topSQLReportRecordCounterHistogram.Observe(float64(sentCount)) + reporter_metrics.TopSQLReportRecordCounterHistogram.Observe(float64(sentCount)) if err != nil { - reportRecordDurationFailedHistogram.Observe(time.Since(start).Seconds()) + reporter_metrics.ReportRecordDurationFailedHistogram.Observe(time.Since(start).Seconds()) } else { - reportRecordDurationSuccHistogram.Observe(time.Since(start).Seconds()) + reporter_metrics.ReportRecordDurationSuccHistogram.Observe(time.Since(start).Seconds()) } }() @@ -200,11 +201,11 @@ func (ds *pubSubDataSink) sendSQLMeta(ctx context.Context, sqlMetas []tipb.SQLMe start := time.Now() sentCount := 0 defer func() { - topSQLReportSQLCountHistogram.Observe(float64(sentCount)) + reporter_metrics.TopSQLReportSQLCountHistogram.Observe(float64(sentCount)) if err != nil { - reportSQLDurationFailedHistogram.Observe(time.Since(start).Seconds()) + reporter_metrics.ReportSQLDurationFailedHistogram.Observe(time.Since(start).Seconds()) } else { - reportSQLDurationSuccHistogram.Observe(time.Since(start).Seconds()) + reporter_metrics.ReportSQLDurationSuccHistogram.Observe(time.Since(start).Seconds()) } }() @@ -237,11 +238,11 @@ func (ds *pubSubDataSink) sendPlanMeta(ctx context.Context, planMetas []tipb.Pla start := time.Now() sentCount := 0 defer func() { - topSQLReportPlanCountHistogram.Observe(float64(sentCount)) + reporter_metrics.TopSQLReportPlanCountHistogram.Observe(float64(sentCount)) if err != nil { - reportPlanDurationFailedHistogram.Observe(time.Since(start).Seconds()) + reporter_metrics.ReportPlanDurationFailedHistogram.Observe(time.Since(start).Seconds()) } else { - reportPlanDurationSuccHistogram.Observe(time.Since(start).Seconds()) + reporter_metrics.ReportPlanDurationSuccHistogram.Observe(time.Since(start).Seconds()) } }() diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index 1a6dcb83b1247..c21f0a67417aa 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/topsql/collector" + reporter_metrics "github.com/pingcap/tidb/util/topsql/reporter/metrics" topsqlstate "github.com/pingcap/tidb/util/topsql/state" "github.com/pingcap/tidb/util/topsql/stmtstats" "go.uber.org/zap" @@ -123,7 +124,7 @@ func (tsr *RemoteTopSQLReporter) Collect(data []collector.SQLCPUTimeRecord) { case tsr.collectCPUTimeChan <- data: default: // ignore if chan blocked - ignoreCollectChannelFullCounter.Inc() + reporter_metrics.IgnoreCollectChannelFullCounter.Inc() } } @@ -139,7 +140,7 @@ func (tsr *RemoteTopSQLReporter) CollectStmtStatsMap(data stmtstats.StatementSta case tsr.collectStmtStatsChan <- data: default: // ignore if chan blocked - ignoreCollectStmtChannelFullCounter.Inc() + reporter_metrics.IgnoreCollectStmtChannelFullCounter.Inc() } } @@ -249,7 +250,7 @@ func (tsr *RemoteTopSQLReporter) takeDataAndSendToReportChan() { }: default: // ignore if chan blocked - ignoreReportChannelFullCounter.Inc() + reporter_metrics.IgnoreReportChannelFullCounter.Inc() } } diff --git a/util/topsql/reporter/single_target.go b/util/topsql/reporter/single_target.go index 16c88956453cd..7c5c6de64fa93 100644 --- a/util/topsql/reporter/single_target.go +++ b/util/topsql/reporter/single_target.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/util/logutil" + reporter_metrics "github.com/pingcap/tidb/util/topsql/reporter/metrics" "github.com/pingcap/tipb/go-tipb" "go.uber.org/atomic" "go.uber.org/zap" @@ -159,7 +160,7 @@ func (ds *SingleTargetDataSink) TrySend(data *ReportData, deadline time.Time) er case <-ds.ctx.Done(): return ds.ctx.Err() default: - ignoreReportChannelFullCounter.Inc() + reporter_metrics.IgnoreReportChannelFullCounter.Inc() return errors.New("the channel of single target dataSink is full") } } @@ -189,9 +190,9 @@ func (ds *SingleTargetDataSink) doSend(addr string, task sendTask) { defer func() { if err != nil { logutil.BgLogger().Warn("[top-sql] single target data sink failed to send data to receiver", zap.Error(err)) - reportAllDurationFailedHistogram.Observe(time.Since(start).Seconds()) + reporter_metrics.ReportAllDurationFailedHistogram.Observe(time.Since(start).Seconds()) } else { - reportAllDurationSuccHistogram.Observe(time.Since(start).Seconds()) + reporter_metrics.ReportAllDurationSuccHistogram.Observe(time.Since(start).Seconds()) } }() @@ -236,11 +237,11 @@ func (ds *SingleTargetDataSink) sendBatchTopSQLRecord(ctx context.Context, recor start := time.Now() sentCount := 0 defer func() { - topSQLReportRecordCounterHistogram.Observe(float64(sentCount)) + reporter_metrics.TopSQLReportRecordCounterHistogram.Observe(float64(sentCount)) if err != nil { - reportRecordDurationFailedHistogram.Observe(time.Since(start).Seconds()) + reporter_metrics.ReportRecordDurationFailedHistogram.Observe(time.Since(start).Seconds()) } else { - reportRecordDurationSuccHistogram.Observe(time.Since(start).Seconds()) + reporter_metrics.ReportRecordDurationSuccHistogram.Observe(time.Since(start).Seconds()) } }() @@ -270,11 +271,11 @@ func (ds *SingleTargetDataSink) sendBatchSQLMeta(ctx context.Context, sqlMetas [ start := time.Now() sentCount := 0 defer func() { - topSQLReportSQLCountHistogram.Observe(float64(sentCount)) + reporter_metrics.TopSQLReportSQLCountHistogram.Observe(float64(sentCount)) if err != nil { - reportSQLDurationFailedHistogram.Observe(time.Since(start).Seconds()) + reporter_metrics.ReportSQLDurationFailedHistogram.Observe(time.Since(start).Seconds()) } else { - reportSQLDurationSuccHistogram.Observe(time.Since(start).Seconds()) + reporter_metrics.ReportSQLDurationSuccHistogram.Observe(time.Since(start).Seconds()) } }() @@ -305,11 +306,11 @@ func (ds *SingleTargetDataSink) sendBatchPlanMeta(ctx context.Context, planMetas start := time.Now() sentCount := 0 defer func() { - topSQLReportPlanCountHistogram.Observe(float64(sentCount)) + reporter_metrics.TopSQLReportPlanCountHistogram.Observe(float64(sentCount)) if err != nil { - reportPlanDurationFailedHistogram.Observe(time.Since(start).Seconds()) + reporter_metrics.ReportPlanDurationFailedHistogram.Observe(time.Since(start).Seconds()) } else { - reportPlanDurationSuccHistogram.Observe(time.Since(start).Seconds()) + reporter_metrics.ReportPlanDurationSuccHistogram.Observe(time.Since(start).Seconds()) } }()