From d4fdd3fe9eb1b9f0ef4b8f5fb1c09805e9d2e2ca Mon Sep 17 00:00:00 2001 From: Alex Barganier Date: Thu, 14 Mar 2024 17:03:24 -0400 Subject: [PATCH 1/4] pkg/sql: expand metrics used by the SQL stats activity update job Addresses: https://github.com/cockroachdb/cockroach/issues/119779 Currently, the SQL activity update job is lacking observability. While we have a metric for job failures, we've seen instances whe the query run by the job gets caught in a retry loop, meaning the metric is rarely incremented. Therefore, additional metrics, such as counts of successful runs, and the latency of successful runs, will be helpful to further inspect the state of the job. This patch adds metrics for both. Release note (ops change): Two new metrics have been added to track the status of the SQL activity update job, which is used to pre-aggregate top K information within the SQL stats subsytem and write the results to `system.statement_activity` and `system.transaction_activity`. The new metrics are: - `sql.stats.activity.updates.successful`: Number of successful updates made by the SQL activity updater job. - `sql.stats.activity.update.latency`: The latency of updates made by the SQL activity updater job. Includes failed update attempts. --- docs/generated/metrics/metrics.html | 2 ++ pkg/sql/sql_activity_update_job.go | 28 +++++++++++++++++++++++++++- 2 files changed, 29 insertions(+), 1 deletion(-) diff --git a/docs/generated/metrics/metrics.html b/docs/generated/metrics/metrics.html index e5a77ced1abb..ca5dff454aa0 100644 --- a/docs/generated/metrics/metrics.html +++ b/docs/generated/metrics/metrics.html @@ -1464,6 +1464,8 @@ APPLICATIONsql.service.latency.internalLatency of SQL request execution (internal queries)SQL Internal StatementsHISTOGRAMNANOSECONDSAVGNONE APPLICATIONsql.statements.activeNumber of currently active user SQL statementsActive StatementsGAUGECOUNTAVGNONE APPLICATIONsql.statements.active.internalNumber of currently active user SQL statements (internal queries)SQL Internal StatementsGAUGECOUNTAVGNONE +APPLICATIONsql.stats.activity.update.latencyThe latency of updates made by the SQL activity updater job. Includes failed update attemptsNanosecondsHISTOGRAMNANOSECONDSAVGNONE +APPLICATIONsql.stats.activity.updates.successfulNumber of successful updates made by the SQL activity updater jobsuccessful updatesCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE APPLICATIONsql.stats.cleanup.rows_removedNumber of stale statistics rows that are removedSQL Stats CleanupCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE APPLICATIONsql.stats.discarded.currentNumber of fingerprint statistics being discardedDiscarded SQL StatsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE APPLICATIONsql.stats.flush.countNumber of times SQL Stats are flushed to persistent storageSQL Stats FlushCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE diff --git a/pkg/sql/sql_activity_update_job.go b/pkg/sql/sql_activity_update_job.go index d4f360bd20a0..3390b0aa0a9d 100644 --- a/pkg/sql/sql_activity_update_job.go +++ b/pkg/sql/sql_activity_update_job.go @@ -15,6 +15,7 @@ import ( "fmt" "time" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -103,11 +104,15 @@ func (j *sqlActivityUpdateJob) Resume(ctx context.Context, execCtxI interface{}) case <-flushDoneSignal: // A flush was done. Set the timer and wait for it to complete. if sqlStatsActivityFlushEnabled.Get(&settings.SV) { + startTime := timeutil.Now().UnixNano() updater := newSqlActivityUpdater(settings, execCtx.ExecCfg().InternalDB, nil) if err := updater.TransferStatsToActivity(ctx); err != nil { log.Warningf(ctx, "error running sql activity updater job: %v", err) metrics.NumErrors.Inc(1) + } else { + metrics.NumSuccessfulUpdates.Inc(1) } + metrics.UpdateLatency.RecordValue(timeutil.Now().UnixNano() - startTime) } case <-ctx.Done(): return nil @@ -120,7 +125,9 @@ func (j *sqlActivityUpdateJob) Resume(ctx context.Context, execCtxI interface{}) // ActivityUpdaterMetrics must be public for metrics to get // registered type ActivityUpdaterMetrics struct { - NumErrors *metric.Counter + NumErrors *metric.Counter + NumSuccessfulUpdates *metric.Counter + UpdateLatency metric.IHistogram } func (m ActivityUpdaterMetrics) MetricStruct() {} @@ -134,6 +141,25 @@ func newActivityUpdaterMetrics() metric.Struct { Unit: metric.Unit_COUNT, MetricType: io_prometheus_client.MetricType_COUNTER, }), + NumSuccessfulUpdates: metric.NewCounter(metric.Metadata{ + Name: "sql.stats.activity.updates.successful", + Help: "Number of successful updates made by the SQL activity updater job", + Measurement: "successful updates", + Unit: metric.Unit_COUNT, + MetricType: io_prometheus_client.MetricType_COUNTER, + }), + UpdateLatency: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metric.Metadata{ + Name: "sql.stats.activity.update.latency", + Help: "The latency of updates made by the SQL activity updater job. Includes failed update attempts", + Measurement: "Nanoseconds", + Unit: metric.Unit_NANOSECONDS, + MetricType: io_prometheus_client.MetricType_HISTOGRAM, + }, + Duration: base.DefaultHistogramWindowInterval(), + BucketConfig: metric.LongRunning60mLatencyBuckets, + Mode: metric.HistogramModePrometheus, + }), } } From 2c4078e1a304a8d896c8f304b8ed341cc9fffceb Mon Sep 17 00:00:00 2001 From: Alex Barganier Date: Thu, 14 Mar 2024 17:26:50 -0400 Subject: [PATCH 2/4] pkg/sql/sqlstats: counter metric for ignore flush done signals Addresses: https://github.com/cockroachdb/cockroach/issues/119779 We've had escalations recently involving the SQL activity update job running for extended periods of time, such that the signal made to the job indicating a flush has completed was not received because there was no listener. While we've added a default case to prevent this from hanging the flush job, and some logging to go with it, a counter metric indicating when this occurs would also be useful to have when debugging. This patch adds such a counter. Release note (ops change): A new counter metric, `sql.stats.flush.done_signals.ignored`, has been introduced. The metric tracks the number of times the SQL Stats activity update job ignored the signal sent to it indicating a flush has completed. This may indicate that the SQL Activity update job is taking longer than expected to complete. --- docs/generated/metrics/metrics.html | 1 + pkg/sql/conn_executor.go | 19 +++++++++++-------- pkg/sql/exec_util.go | 9 +++++++++ pkg/sql/executor_statement_metrics.go | 9 +++++---- .../sqlstats/persistedsqlstats/provider.go | 8 +++++--- 5 files changed, 31 insertions(+), 15 deletions(-) diff --git a/docs/generated/metrics/metrics.html b/docs/generated/metrics/metrics.html index ca5dff454aa0..86008a8bd984 100644 --- a/docs/generated/metrics/metrics.html +++ b/docs/generated/metrics/metrics.html @@ -1469,6 +1469,7 @@ APPLICATIONsql.stats.cleanup.rows_removedNumber of stale statistics rows that are removedSQL Stats CleanupCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE APPLICATIONsql.stats.discarded.currentNumber of fingerprint statistics being discardedDiscarded SQL StatsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE APPLICATIONsql.stats.flush.countNumber of times SQL Stats are flushed to persistent storageSQL Stats FlushCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +APPLICATIONsql.stats.flush.done_signals.ignoredNumber of times the SQL Stats activity update job ignored the signal sent to it indicating a flush has completedflush done signals ignoredCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE APPLICATIONsql.stats.flush.durationTime took to in nanoseconds to complete SQL Stats flushSQL Stats FlushHISTOGRAMNANOSECONDSAVGNONE APPLICATIONsql.stats.flush.errorNumber of errors encountered when flushing SQL StatsSQL Stats FlushCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE APPLICATIONsql.stats.mem.currentCurrent memory usage for fingerprint storageMemoryGAUGEBYTESAVGNONE diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index 35a661d5a526..f3128abd4872 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -484,13 +484,14 @@ func NewServer(cfg *ExecutorConfig, pool *mon.BytesMonitor) *Server { DB: NewInternalDB( s, MemoryMetrics{}, sqlStatsInternalExecutorMonitor, ), - ClusterID: s.cfg.NodeInfo.LogicalClusterID, - SQLIDContainer: cfg.NodeInfo.NodeID, - JobRegistry: s.cfg.JobRegistry, - Knobs: cfg.SQLStatsTestingKnobs, - FlushCounter: serverMetrics.StatsMetrics.SQLStatsFlushStarted, - FailureCounter: serverMetrics.StatsMetrics.SQLStatsFlushFailure, - FlushDuration: serverMetrics.StatsMetrics.SQLStatsFlushDuration, + ClusterID: s.cfg.NodeInfo.LogicalClusterID, + SQLIDContainer: cfg.NodeInfo.NodeID, + JobRegistry: s.cfg.JobRegistry, + Knobs: cfg.SQLStatsTestingKnobs, + FlushCounter: serverMetrics.StatsMetrics.SQLStatsFlushStarted, + FlushDoneSignalsIgnored: serverMetrics.StatsMetrics.SQLStatsFlushDoneSignalsIgnored, + FailureCounter: serverMetrics.StatsMetrics.SQLStatsFlushFailure, + FlushDuration: serverMetrics.StatsMetrics.SQLStatsFlushDuration, }, memSQLStats) s.sqlStats = persistedSQLStats @@ -588,7 +589,9 @@ func makeServerMetrics(cfg *ExecutorConfig) ServerMetrics { ReportedSQLStatsMemoryCurBytesCount: metric.NewGauge(MetaReportedSQLStatsMemCurBytes), DiscardedStatsCount: metric.NewCounter(MetaDiscardedSQLStats), SQLStatsFlushStarted: metric.NewCounter(MetaSQLStatsFlushStarted), - SQLStatsFlushFailure: metric.NewCounter(MetaSQLStatsFlushFailure), + SQLStatsFlushDoneSignalsIgnored: metric.NewCounter(MetaSQLStatsFlushDoneSignalsIgnored), + + SQLStatsFlushFailure: metric.NewCounter(MetaSQLStatsFlushFailure), SQLStatsFlushDuration: metric.NewHistogram(metric.HistogramOptions{ Mode: metric.HistogramModePreferHdrLatency, Metadata: MetaSQLStatsFlushDuration, diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 7536ff992e33..84759a418956 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -125,6 +125,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" + io_prometheus_client "github.com/prometheus/client_model/go" ) func init() { @@ -1093,6 +1094,14 @@ var ( Measurement: "SQL Stats Flush", Unit: metric.Unit_COUNT, } + MetaSQLStatsFlushDoneSignalsIgnored = metric.Metadata{ + Name: "sql.stats.flush.done_signals.ignored", + Help: "Number of times the SQL Stats activity update job ignored the signal sent to it indicating " + + "a flush has completed", + Measurement: "flush done signals ignored", + Unit: metric.Unit_COUNT, + MetricType: io_prometheus_client.MetricType_COUNTER, + } MetaSQLStatsFlushFailure = metric.Metadata{ Name: "sql.stats.flush.error", Help: "Number of errors encountered when flushing SQL Stats", diff --git a/pkg/sql/executor_statement_metrics.go b/pkg/sql/executor_statement_metrics.go index 948dce70a590..cb7f8314c346 100644 --- a/pkg/sql/executor_statement_metrics.go +++ b/pkg/sql/executor_statement_metrics.go @@ -75,10 +75,11 @@ type StatsMetrics struct { DiscardedStatsCount *metric.Counter - SQLStatsFlushStarted *metric.Counter - SQLStatsFlushFailure *metric.Counter - SQLStatsFlushDuration metric.IHistogram - SQLStatsRemovedRows *metric.Counter + SQLStatsFlushStarted *metric.Counter + SQLStatsFlushDoneSignalsIgnored *metric.Counter + SQLStatsFlushFailure *metric.Counter + SQLStatsFlushDuration metric.IHistogram + SQLStatsRemovedRows *metric.Counter SQLTxnStatsCollectionOverhead metric.IHistogram } diff --git a/pkg/sql/sqlstats/persistedsqlstats/provider.go b/pkg/sql/sqlstats/persistedsqlstats/provider.go index e78b06c8bbb6..5f038501396b 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/provider.go +++ b/pkg/sql/sqlstats/persistedsqlstats/provider.go @@ -46,9 +46,10 @@ type Config struct { JobRegistry *jobs.Registry // Metrics. - FlushCounter *metric.Counter - FlushDuration metric.IHistogram - FailureCounter *metric.Counter + FlushCounter *metric.Counter + FlushDuration metric.IHistogram + FlushDoneSignalsIgnored *metric.Counter + FailureCounter *metric.Counter // Testing knobs. Knobs *sqlstats.TestingKnobs @@ -211,6 +212,7 @@ func (s *PersistedSQLStats) startSQLStatsFlushLoop(ctx context.Context, stopper // Don't block the flush loop if the sql activity update job is not // ready to receive. We should at least continue to collect and flush // stats for this node. + s.cfg.FlushDoneSignalsIgnored.Inc(1) log.Warning(ctx, "sql-stats-worker: unable to signal flush completion") } } From 83d75e8c3e10944489bb71b4241ca35e31236bc3 Mon Sep 17 00:00:00 2001 From: Alex Barganier Date: Fri, 15 Mar 2024 12:57:39 -0400 Subject: [PATCH 3/4] pkg/sql: rename metric tracking SQL activity update job failures The metric used to track failures of the SQL Activity update job didn't have a descriptive name, and the help text was grammatically incorrect. Furthermore, the metric name is the same as a metric used within the job system, meaning one of these metrics is probably clobbering the other when writing to TSDB or outputting to `/_status/vars`. This patch simply updates the metric name to better describe what it measures, and fixes the help text description. Release note (ops change): A new counter metric, `sql.stats.activity.updates.failed`, has been introduced to measure the number of update attempts made by the SQL activity updater job that failed with errors. The SQL activity update job is used to pre-aggregate top K information within the SQL stats subsystem and write the results to `system.statement_activity` and `system.transaction_activity`. --- docs/generated/metrics/metrics.html | 3 ++- pkg/sql/sql_activity_update_job.go | 12 ++++++------ 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/docs/generated/metrics/metrics.html b/docs/generated/metrics/metrics.html index 86008a8bd984..99c54a96d2fd 100644 --- a/docs/generated/metrics/metrics.html +++ b/docs/generated/metrics/metrics.html @@ -1096,7 +1096,7 @@ APPLICATIONjobs.key_visualizer.resume_completedNumber of key_visualizer jobs which successfully resumed to completionjobsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE APPLICATIONjobs.key_visualizer.resume_failedNumber of key_visualizer jobs which failed with a non-retriable errorjobsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE APPLICATIONjobs.key_visualizer.resume_retry_errorNumber of key_visualizer jobs which failed with a retriable errorjobsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE -APPLICATIONjobs.metrics.task_failedNumber of metrics sql activity updater tasks that failederrorsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +APPLICATIONjobs.metrics.task_failedNumber of metrics poller tasks that failederrorsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE APPLICATIONjobs.migration.currently_idleNumber of migration jobs currently considered Idle and can be freely shut downjobsGAUGECOUNTAVGNONE APPLICATIONjobs.migration.currently_pausedNumber of migration jobs currently considered PausedjobsGAUGECOUNTAVGNONE APPLICATIONjobs.migration.currently_runningNumber of migration jobs currently running in Resume or OnFailOrCancel statejobsGAUGECOUNTAVGNONE @@ -1465,6 +1465,7 @@ APPLICATIONsql.statements.activeNumber of currently active user SQL statementsActive StatementsGAUGECOUNTAVGNONE APPLICATIONsql.statements.active.internalNumber of currently active user SQL statements (internal queries)SQL Internal StatementsGAUGECOUNTAVGNONE APPLICATIONsql.stats.activity.update.latencyThe latency of updates made by the SQL activity updater job. Includes failed update attemptsNanosecondsHISTOGRAMNANOSECONDSAVGNONE +APPLICATIONsql.stats.activity.updates.failedNumber of update attempts made by the SQL activity updater job that failed with errorsfailed updatesgiCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE APPLICATIONsql.stats.activity.updates.successfulNumber of successful updates made by the SQL activity updater jobsuccessful updatesCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE APPLICATIONsql.stats.cleanup.rows_removedNumber of stale statistics rows that are removedSQL Stats CleanupCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE APPLICATIONsql.stats.discarded.currentNumber of fingerprint statistics being discardedDiscarded SQL StatsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE diff --git a/pkg/sql/sql_activity_update_job.go b/pkg/sql/sql_activity_update_job.go index 3390b0aa0a9d..35f899275470 100644 --- a/pkg/sql/sql_activity_update_job.go +++ b/pkg/sql/sql_activity_update_job.go @@ -108,7 +108,7 @@ func (j *sqlActivityUpdateJob) Resume(ctx context.Context, execCtxI interface{}) updater := newSqlActivityUpdater(settings, execCtx.ExecCfg().InternalDB, nil) if err := updater.TransferStatsToActivity(ctx); err != nil { log.Warningf(ctx, "error running sql activity updater job: %v", err) - metrics.NumErrors.Inc(1) + metrics.NumFailedUpdates.Inc(1) } else { metrics.NumSuccessfulUpdates.Inc(1) } @@ -125,7 +125,7 @@ func (j *sqlActivityUpdateJob) Resume(ctx context.Context, execCtxI interface{}) // ActivityUpdaterMetrics must be public for metrics to get // registered type ActivityUpdaterMetrics struct { - NumErrors *metric.Counter + NumFailedUpdates *metric.Counter NumSuccessfulUpdates *metric.Counter UpdateLatency metric.IHistogram } @@ -134,10 +134,10 @@ func (m ActivityUpdaterMetrics) MetricStruct() {} func newActivityUpdaterMetrics() metric.Struct { return ActivityUpdaterMetrics{ - NumErrors: metric.NewCounter(metric.Metadata{ - Name: "jobs.metrics.task_failed", - Help: "Number of metrics sql activity updater tasks that failed", - Measurement: "errors", + NumFailedUpdates: metric.NewCounter(metric.Metadata{ + Name: "sql.stats.activity.updates.failed", + Help: "Number of update attempts made by the SQL activity updater job that failed with errors", + Measurement: "failed updatesgi", Unit: metric.Unit_COUNT, MetricType: io_prometheus_client.MetricType_COUNTER, }), From 099d0b05186624c52aeab08585dd4946a63130a3 Mon Sep 17 00:00:00 2001 From: Alex Barganier Date: Mon, 18 Mar 2024 12:26:56 -0400 Subject: [PATCH 4/4] pkg/sql/sqlstats: counter metric for flushed fingerprint counts Addresses: https://github.com/cockroachdb/cockroach/issues/119779 The count of unique fingeprints flushed to `system.statement_statistics` and `system.transaction_statistics` is the core component that determines data cardinality within the SQL stats subsystem. Today, we don't have good metrics around this source of cardinality. As we aim to reduce cardinality by improving our fingerprinting algorithms, creating a metric to count the number of unique statement and transaction fingerprints included in each flush of the in-memory SQL stats will be a helpful measurement to benchmark cardinality reduction. This patch adds a new metric to track the # of unique fingerprints (stmt and txn) included in each flush. Release note (ops change): A new counter metric, `sql.stats.flush.fingerprint.count`, has been introduced. The metric tracks the number of unique statement and transaction fingerprints included in the SQL Stats flush. --- docs/generated/metrics/metrics.html | 1 + pkg/sql/conn_executor.go | 2 ++ pkg/sql/exec_util.go | 6 ++++++ pkg/sql/executor_statement_metrics.go | 1 + pkg/sql/sqlstats/persistedsqlstats/flush.go | 4 +++- pkg/sql/sqlstats/persistedsqlstats/provider.go | 1 + 6 files changed, 14 insertions(+), 1 deletion(-) diff --git a/docs/generated/metrics/metrics.html b/docs/generated/metrics/metrics.html index 99c54a96d2fd..322cf53ac390 100644 --- a/docs/generated/metrics/metrics.html +++ b/docs/generated/metrics/metrics.html @@ -1473,6 +1473,7 @@ APPLICATIONsql.stats.flush.done_signals.ignoredNumber of times the SQL Stats activity update job ignored the signal sent to it indicating a flush has completedflush done signals ignoredCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE APPLICATIONsql.stats.flush.durationTime took to in nanoseconds to complete SQL Stats flushSQL Stats FlushHISTOGRAMNANOSECONDSAVGNONE APPLICATIONsql.stats.flush.errorNumber of errors encountered when flushing SQL StatsSQL Stats FlushCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +APPLICATIONsql.stats.flush.fingerprint.countThe number of unique statement and transaction fingerprints included in the SQL Stats flushstatement & transaction fingerprintsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE APPLICATIONsql.stats.mem.currentCurrent memory usage for fingerprint storageMemoryGAUGEBYTESAVGNONE APPLICATIONsql.stats.mem.maxMemory usage for fingerprint storageMemoryHISTOGRAMBYTESAVGNONE APPLICATIONsql.stats.reported.mem.currentCurrent memory usage for reported fingerprint storageMemoryGAUGEBYTESAVGNONE diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index f3128abd4872..e3ee3063d2bc 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -490,6 +490,7 @@ func NewServer(cfg *ExecutorConfig, pool *mon.BytesMonitor) *Server { Knobs: cfg.SQLStatsTestingKnobs, FlushCounter: serverMetrics.StatsMetrics.SQLStatsFlushStarted, FlushDoneSignalsIgnored: serverMetrics.StatsMetrics.SQLStatsFlushDoneSignalsIgnored, + FlushedFingerprintCount: serverMetrics.StatsMetrics.SQLStatsFlushFingerprintCount, FailureCounter: serverMetrics.StatsMetrics.SQLStatsFlushFailure, FlushDuration: serverMetrics.StatsMetrics.SQLStatsFlushDuration, }, memSQLStats) @@ -590,6 +591,7 @@ func makeServerMetrics(cfg *ExecutorConfig) ServerMetrics { DiscardedStatsCount: metric.NewCounter(MetaDiscardedSQLStats), SQLStatsFlushStarted: metric.NewCounter(MetaSQLStatsFlushStarted), SQLStatsFlushDoneSignalsIgnored: metric.NewCounter(MetaSQLStatsFlushDoneSignalsIgnored), + SQLStatsFlushFingerprintCount: metric.NewCounter(MetaSQLStatsFlushFingerprintCount), SQLStatsFlushFailure: metric.NewCounter(MetaSQLStatsFlushFailure), SQLStatsFlushDuration: metric.NewHistogram(metric.HistogramOptions{ diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 84759a418956..41e6835e2f05 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -1094,6 +1094,12 @@ var ( Measurement: "SQL Stats Flush", Unit: metric.Unit_COUNT, } + MetaSQLStatsFlushFingerprintCount = metric.Metadata{ + Name: "sql.stats.flush.fingerprint.count", + Help: "The number of unique statement and transaction fingerprints included in the SQL Stats flush", + Measurement: "statement & transaction fingerprints", + Unit: metric.Unit_COUNT, + } MetaSQLStatsFlushDoneSignalsIgnored = metric.Metadata{ Name: "sql.stats.flush.done_signals.ignored", Help: "Number of times the SQL Stats activity update job ignored the signal sent to it indicating " + diff --git a/pkg/sql/executor_statement_metrics.go b/pkg/sql/executor_statement_metrics.go index cb7f8314c346..7a796e5dab4a 100644 --- a/pkg/sql/executor_statement_metrics.go +++ b/pkg/sql/executor_statement_metrics.go @@ -77,6 +77,7 @@ type StatsMetrics struct { SQLStatsFlushStarted *metric.Counter SQLStatsFlushDoneSignalsIgnored *metric.Counter + SQLStatsFlushFingerprintCount *metric.Counter SQLStatsFlushFailure *metric.Counter SQLStatsFlushDuration metric.IHistogram SQLStatsRemovedRows *metric.Counter diff --git a/pkg/sql/sqlstats/persistedsqlstats/flush.go b/pkg/sql/sqlstats/persistedsqlstats/flush.go index c14684f89678..81655ba4eb5a 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/flush.go +++ b/pkg/sql/sqlstats/persistedsqlstats/flush.go @@ -62,9 +62,11 @@ func (s *PersistedSQLStats) Flush(ctx context.Context, stopper *stop.Stopper) { return } + fingerprintCount := s.SQLStats.GetTotalFingerprintCount() + s.cfg.FlushedFingerprintCount.Inc(fingerprintCount) if log.V(1) { log.Infof(ctx, "flushing %d stmt/txn fingerprints (%d bytes) after %s", - s.SQLStats.GetTotalFingerprintCount(), s.SQLStats.GetTotalFingerprintBytes(), timeutil.Since(s.lastFlushStarted)) + fingerprintCount, s.SQLStats.GetTotalFingerprintBytes(), timeutil.Since(s.lastFlushStarted)) } s.lastFlushStarted = now diff --git a/pkg/sql/sqlstats/persistedsqlstats/provider.go b/pkg/sql/sqlstats/persistedsqlstats/provider.go index 5f038501396b..b06d467c3c96 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/provider.go +++ b/pkg/sql/sqlstats/persistedsqlstats/provider.go @@ -50,6 +50,7 @@ type Config struct { FlushDuration metric.IHistogram FlushDoneSignalsIgnored *metric.Counter FailureCounter *metric.Counter + FlushedFingerprintCount *metric.Counter // Testing knobs. Knobs *sqlstats.TestingKnobs