diff --git a/docs/generated/metrics/metrics.html b/docs/generated/metrics/metrics.html index e5a77ced1abb..322cf53ac390 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 @@ -1464,11 +1464,16 @@ 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.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 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.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 35a661d5a526..e3ee3063d2bc 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -484,13 +484,15 @@ 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, + FlushedFingerprintCount: serverMetrics.StatsMetrics.SQLStatsFlushFingerprintCount, + FailureCounter: serverMetrics.StatsMetrics.SQLStatsFlushFailure, + FlushDuration: serverMetrics.StatsMetrics.SQLStatsFlushDuration, }, memSQLStats) s.sqlStats = persistedSQLStats @@ -588,7 +590,10 @@ 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), + SQLStatsFlushFingerprintCount: metric.NewCounter(MetaSQLStatsFlushFingerprintCount), + + 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..41e6835e2f05 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,20 @@ 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 " + + "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..7a796e5dab4a 100644 --- a/pkg/sql/executor_statement_metrics.go +++ b/pkg/sql/executor_statement_metrics.go @@ -75,10 +75,12 @@ type StatsMetrics struct { DiscardedStatsCount *metric.Counter - SQLStatsFlushStarted *metric.Counter - SQLStatsFlushFailure *metric.Counter - SQLStatsFlushDuration metric.IHistogram - SQLStatsRemovedRows *metric.Counter + SQLStatsFlushStarted *metric.Counter + SQLStatsFlushDoneSignalsIgnored *metric.Counter + SQLStatsFlushFingerprintCount *metric.Counter + SQLStatsFlushFailure *metric.Counter + SQLStatsFlushDuration metric.IHistogram + SQLStatsRemovedRows *metric.Counter SQLTxnStatsCollectionOverhead metric.IHistogram } diff --git a/pkg/sql/sql_activity_update_job.go b/pkg/sql/sql_activity_update_job.go index d4f360bd20a0..35f899275470 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) + metrics.NumFailedUpdates.Inc(1) + } else { + metrics.NumSuccessfulUpdates.Inc(1) } + metrics.UpdateLatency.RecordValue(timeutil.Now().UnixNano() - startTime) } case <-ctx.Done(): return nil @@ -120,20 +125,41 @@ 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 } 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, }), + 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, + }), } } 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 e78b06c8bbb6..b06d467c3c96 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/provider.go +++ b/pkg/sql/sqlstats/persistedsqlstats/provider.go @@ -46,9 +46,11 @@ 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 + FlushedFingerprintCount *metric.Counter // Testing knobs. Knobs *sqlstats.TestingKnobs @@ -211,6 +213,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") } }