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 @@
APPLICATION | jobs.key_visualizer.resume_completed | Number of key_visualizer jobs which successfully resumed to completion | jobs | COUNTER | COUNT | AVG | NON_NEGATIVE_DERIVATIVE |
APPLICATION | jobs.key_visualizer.resume_failed | Number of key_visualizer jobs which failed with a non-retriable error | jobs | COUNTER | COUNT | AVG | NON_NEGATIVE_DERIVATIVE |
APPLICATION | jobs.key_visualizer.resume_retry_error | Number of key_visualizer jobs which failed with a retriable error | jobs | COUNTER | COUNT | AVG | NON_NEGATIVE_DERIVATIVE |
-APPLICATION | jobs.metrics.task_failed | Number of metrics sql activity updater tasks that failed | errors | COUNTER | COUNT | AVG | NON_NEGATIVE_DERIVATIVE |
+APPLICATION | jobs.metrics.task_failed | Number of metrics poller tasks that failed | errors | COUNTER | COUNT | AVG | NON_NEGATIVE_DERIVATIVE |
APPLICATION | jobs.migration.currently_idle | Number of migration jobs currently considered Idle and can be freely shut down | jobs | GAUGE | COUNT | AVG | NONE |
APPLICATION | jobs.migration.currently_paused | Number of migration jobs currently considered Paused | jobs | GAUGE | COUNT | AVG | NONE |
APPLICATION | jobs.migration.currently_running | Number of migration jobs currently running in Resume or OnFailOrCancel state | jobs | GAUGE | COUNT | AVG | NONE |
@@ -1464,11 +1464,16 @@
APPLICATION | sql.service.latency.internal | Latency of SQL request execution (internal queries) | SQL Internal Statements | HISTOGRAM | NANOSECONDS | AVG | NONE |
APPLICATION | sql.statements.active | Number of currently active user SQL statements | Active Statements | GAUGE | COUNT | AVG | NONE |
APPLICATION | sql.statements.active.internal | Number of currently active user SQL statements (internal queries) | SQL Internal Statements | GAUGE | COUNT | AVG | NONE |
+APPLICATION | sql.stats.activity.update.latency | The latency of updates made by the SQL activity updater job. Includes failed update attempts | Nanoseconds | HISTOGRAM | NANOSECONDS | AVG | NONE |
+APPLICATION | sql.stats.activity.updates.failed | Number of update attempts made by the SQL activity updater job that failed with errors | failed updatesgi | COUNTER | COUNT | AVG | NON_NEGATIVE_DERIVATIVE |
+APPLICATION | sql.stats.activity.updates.successful | Number of successful updates made by the SQL activity updater job | successful updates | COUNTER | COUNT | AVG | NON_NEGATIVE_DERIVATIVE |
APPLICATION | sql.stats.cleanup.rows_removed | Number of stale statistics rows that are removed | SQL Stats Cleanup | COUNTER | COUNT | AVG | NON_NEGATIVE_DERIVATIVE |
APPLICATION | sql.stats.discarded.current | Number of fingerprint statistics being discarded | Discarded SQL Stats | COUNTER | COUNT | AVG | NON_NEGATIVE_DERIVATIVE |
APPLICATION | sql.stats.flush.count | Number of times SQL Stats are flushed to persistent storage | SQL Stats Flush | COUNTER | COUNT | AVG | NON_NEGATIVE_DERIVATIVE |
+APPLICATION | sql.stats.flush.done_signals.ignored | Number of times the SQL Stats activity update job ignored the signal sent to it indicating a flush has completed | flush done signals ignored | COUNTER | COUNT | AVG | NON_NEGATIVE_DERIVATIVE |
APPLICATION | sql.stats.flush.duration | Time took to in nanoseconds to complete SQL Stats flush | SQL Stats Flush | HISTOGRAM | NANOSECONDS | AVG | NONE |
APPLICATION | sql.stats.flush.error | Number of errors encountered when flushing SQL Stats | SQL Stats Flush | COUNTER | COUNT | AVG | NON_NEGATIVE_DERIVATIVE |
+APPLICATION | sql.stats.flush.fingerprint.count | The number of unique statement and transaction fingerprints included in the SQL Stats flush | statement & transaction fingerprints | COUNTER | COUNT | AVG | NON_NEGATIVE_DERIVATIVE |
APPLICATION | sql.stats.mem.current | Current memory usage for fingerprint storage | Memory | GAUGE | BYTES | AVG | NONE |
APPLICATION | sql.stats.mem.max | Memory usage for fingerprint storage | Memory | HISTOGRAM | BYTES | AVG | NONE |
APPLICATION | sql.stats.reported.mem.current | Current memory usage for reported fingerprint storage | Memory | GAUGE | BYTES | AVG | NONE |
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")
}
}