Skip to content

Commit

Permalink
kvserver: expose LSM snapshot storage-engine timeseries metrics
Browse files Browse the repository at this point in the history
Expose two new storage-engine metrics that provide visibility into LSM
snapshots. These two new metrics record cumulative count and size of obsolete
keys written to sstables during flushes and compactions.

Epic: None
Release note (ops change): Expose two new timeseries metrics
`storage.compactions.keys.pinned.count` and
`storage.compactions.keys.pinned.bytes` providing some observability into the
volume of keys preserved by open LSM snapshots.

Informs cockroachdb/pebble#1204.
  • Loading branch information
jbowens committed Apr 6, 2023
1 parent 14cf13d commit 55e50b8
Show file tree
Hide file tree
Showing 2 changed files with 122 additions and 80 deletions.
194 changes: 114 additions & 80 deletions pkg/kv/kvserver/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -670,6 +670,34 @@ See storage.AggregatedIteratorStats for more details.`,
Measurement: "Iterator Ops",
Unit: metric.Unit_COUNT,
}
metaStorageCompactionsKeysPinnedCount = metric.Metadata{
Name: "storage.compactions.keys.pinned.count",
Help: `Cumulative count of storage engine KVs written to sstables during flushes and compactions due to open LSM snapshots.
Various subsystems of CockroachDB take LSM snapshots to maintain a consistent view
of the database over an extended duration. In order to maintain the consistent view,
flushes and compactions within the storage engine must preserve keys that otherwise
would have been dropped. This increases write amplification, and introduces keys
that must be skipped during iteration. This metric records the cumulative count of
KVs preserved during flushes and compactions over the lifetime of the process.
`,
Measurement: "Keys",
Unit: metric.Unit_COUNT,
}
metaStorageCompactionsKeysPinnedBytes = metric.Metadata{
Name: "storage.compactions.keys.pinned.bytes",
Help: `Cumulative size of storage engine KVs written to sstables during flushes and compactions due to open LSM snapshots.
Various subsystems of CockroachDB take LSM snapshots to maintain a consistent view
of the database over an extended duration. In order to maintain the consistent view,
flushes and compactions within the storage engine must preserve keys that otherwise
would have been dropped. This increases write amplification, and introduces keys
that must be skipped during iteration. This metric records the cumulative number of
bytes preserved during flushes and compactions over the lifetime of the process.
`,
Measurement: "Bytes",
Unit: metric.Unit_BYTES,
}
metaSharedStorageBytesWritten = metric.Metadata{
Name: "storage.shared-storage.write",
Help: "Bytes written to external storage",
Expand Down Expand Up @@ -1914,46 +1942,48 @@ type StoreMetrics struct {
//
// TODO(jackson): Reconcile this mismatch so that metrics that are
// semantically counters are exported as such to Prometheus. See #99922.
RdbBlockCacheHits *metric.Gauge
RdbBlockCacheMisses *metric.Gauge
RdbBlockCacheUsage *metric.Gauge
RdbBlockCachePinnedUsage *metric.Gauge
RdbBloomFilterPrefixChecked *metric.Gauge
RdbBloomFilterPrefixUseful *metric.Gauge
RdbMemtableTotalSize *metric.Gauge
RdbFlushes *metric.Gauge
RdbFlushedBytes *metric.Gauge
RdbCompactions *metric.Gauge
RdbIngestedBytes *metric.Gauge
RdbCompactedBytesRead *metric.Gauge
RdbCompactedBytesWritten *metric.Gauge
RdbTableReadersMemEstimate *metric.Gauge
RdbReadAmplification *metric.Gauge
RdbNumSSTables *metric.Gauge
RdbPendingCompaction *metric.Gauge
RdbMarkedForCompactionFiles *metric.Gauge
RdbKeysRangeKeySets *metric.Gauge
RdbKeysTombstones *metric.Gauge
RdbL0BytesFlushed *metric.Gauge
RdbL0Sublevels *metric.Gauge
RdbL0NumFiles *metric.Gauge
RdbBytesIngested [7]*metric.Gauge // idx = level
RdbLevelSize [7]*metric.Gauge // idx = level
RdbLevelScore [7]*metric.GaugeFloat64 // idx = level
RdbWriteStalls *metric.Gauge
RdbWriteStallNanos *metric.Gauge
SharedStorageBytesRead *metric.Gauge
SharedStorageBytesWritten *metric.Gauge
IterBlockBytes *metric.Gauge
IterBlockBytesInCache *metric.Gauge
IterBlockReadDuration *metric.Gauge
IterExternalSeeks *metric.Gauge
IterExternalSteps *metric.Gauge
IterInternalSeeks *metric.Gauge
IterInternalSteps *metric.Gauge
FlushableIngestCount *metric.Gauge
FlushableIngestTableCount *metric.Gauge
FlushableIngestTableSize *metric.Gauge
RdbBlockCacheHits *metric.Gauge
RdbBlockCacheMisses *metric.Gauge
RdbBlockCacheUsage *metric.Gauge
RdbBlockCachePinnedUsage *metric.Gauge
RdbBloomFilterPrefixChecked *metric.Gauge
RdbBloomFilterPrefixUseful *metric.Gauge
RdbMemtableTotalSize *metric.Gauge
RdbFlushes *metric.Gauge
RdbFlushedBytes *metric.Gauge
RdbCompactions *metric.Gauge
RdbIngestedBytes *metric.Gauge
RdbCompactedBytesRead *metric.Gauge
RdbCompactedBytesWritten *metric.Gauge
RdbTableReadersMemEstimate *metric.Gauge
RdbReadAmplification *metric.Gauge
RdbNumSSTables *metric.Gauge
RdbPendingCompaction *metric.Gauge
RdbMarkedForCompactionFiles *metric.Gauge
RdbKeysRangeKeySets *metric.Gauge
RdbKeysTombstones *metric.Gauge
RdbL0BytesFlushed *metric.Gauge
RdbL0Sublevels *metric.Gauge
RdbL0NumFiles *metric.Gauge
RdbBytesIngested [7]*metric.Gauge // idx = level
RdbLevelSize [7]*metric.Gauge // idx = level
RdbLevelScore [7]*metric.GaugeFloat64 // idx = level
RdbWriteStalls *metric.Gauge
RdbWriteStallNanos *metric.Gauge
SharedStorageBytesRead *metric.Gauge
SharedStorageBytesWritten *metric.Gauge
StorageCompactionsPinnedKeys *metric.Gauge
StorageCompactionsPinnedBytes *metric.Gauge
IterBlockBytes *metric.Gauge
IterBlockBytesInCache *metric.Gauge
IterBlockReadDuration *metric.Gauge
IterExternalSeeks *metric.Gauge
IterExternalSteps *metric.Gauge
IterInternalSeeks *metric.Gauge
IterInternalSteps *metric.Gauge
FlushableIngestCount *metric.Gauge
FlushableIngestTableCount *metric.Gauge
FlushableIngestTableSize *metric.Gauge

RdbCheckpoints *metric.Gauge

Expand Down Expand Up @@ -2478,46 +2508,48 @@ func newStoreMetrics(histogramWindow time.Duration) *StoreMetrics {
// but the meaning of the metric itself is a counter.
// TODO(jackson): Reconcile this mismatch so that metrics that are
// semantically counters are exported as such to Prometheus. See #99922.
RdbBlockCacheHits: metric.NewGauge(metaRdbBlockCacheHits),
RdbBlockCacheMisses: metric.NewGauge(metaRdbBlockCacheMisses),
RdbBlockCacheUsage: metric.NewGauge(metaRdbBlockCacheUsage),
RdbBlockCachePinnedUsage: metric.NewGauge(metaRdbBlockCachePinnedUsage),
RdbBloomFilterPrefixChecked: metric.NewGauge(metaRdbBloomFilterPrefixChecked),
RdbBloomFilterPrefixUseful: metric.NewGauge(metaRdbBloomFilterPrefixUseful),
RdbMemtableTotalSize: metric.NewGauge(metaRdbMemtableTotalSize),
RdbFlushes: metric.NewGauge(metaRdbFlushes),
RdbFlushedBytes: metric.NewGauge(metaRdbFlushedBytes),
RdbCompactions: metric.NewGauge(metaRdbCompactions),
RdbIngestedBytes: metric.NewGauge(metaRdbIngestedBytes),
RdbCompactedBytesRead: metric.NewGauge(metaRdbCompactedBytesRead),
RdbCompactedBytesWritten: metric.NewGauge(metaRdbCompactedBytesWritten),
RdbTableReadersMemEstimate: metric.NewGauge(metaRdbTableReadersMemEstimate),
RdbReadAmplification: metric.NewGauge(metaRdbReadAmplification),
RdbNumSSTables: metric.NewGauge(metaRdbNumSSTables),
RdbPendingCompaction: metric.NewGauge(metaRdbPendingCompaction),
RdbMarkedForCompactionFiles: metric.NewGauge(metaRdbMarkedForCompactionFiles),
RdbKeysRangeKeySets: metric.NewGauge(metaRdbKeysRangeKeySets),
RdbKeysTombstones: metric.NewGauge(metaRdbKeysTombstones),
RdbL0BytesFlushed: metric.NewGauge(metaRdbL0BytesFlushed),
RdbL0Sublevels: metric.NewGauge(metaRdbL0Sublevels),
RdbL0NumFiles: metric.NewGauge(metaRdbL0NumFiles),
RdbBytesIngested: rdbBytesIngested,
RdbLevelSize: rdbLevelSize,
RdbLevelScore: rdbLevelScore,
RdbWriteStalls: metric.NewGauge(metaRdbWriteStalls),
RdbWriteStallNanos: metric.NewGauge(metaRdbWriteStallNanos),
IterBlockBytes: metric.NewGauge(metaBlockBytes),
IterBlockBytesInCache: metric.NewGauge(metaBlockBytesInCache),
IterBlockReadDuration: metric.NewGauge(metaBlockReadDuration),
IterExternalSeeks: metric.NewGauge(metaIterExternalSeeks),
IterExternalSteps: metric.NewGauge(metaIterExternalSteps),
IterInternalSeeks: metric.NewGauge(metaIterInternalSeeks),
IterInternalSteps: metric.NewGauge(metaIterInternalSteps),
SharedStorageBytesRead: metric.NewGauge(metaSharedStorageBytesRead),
SharedStorageBytesWritten: metric.NewGauge(metaSharedStorageBytesWritten),
FlushableIngestCount: metric.NewGauge(metaFlushableIngestCount),
FlushableIngestTableCount: metric.NewGauge(metaFlushableIngestTableCount),
FlushableIngestTableSize: metric.NewGauge(metaFlushableIngestTableBytes),
RdbBlockCacheHits: metric.NewGauge(metaRdbBlockCacheHits),
RdbBlockCacheMisses: metric.NewGauge(metaRdbBlockCacheMisses),
RdbBlockCacheUsage: metric.NewGauge(metaRdbBlockCacheUsage),
RdbBlockCachePinnedUsage: metric.NewGauge(metaRdbBlockCachePinnedUsage),
RdbBloomFilterPrefixChecked: metric.NewGauge(metaRdbBloomFilterPrefixChecked),
RdbBloomFilterPrefixUseful: metric.NewGauge(metaRdbBloomFilterPrefixUseful),
RdbMemtableTotalSize: metric.NewGauge(metaRdbMemtableTotalSize),
RdbFlushes: metric.NewGauge(metaRdbFlushes),
RdbFlushedBytes: metric.NewGauge(metaRdbFlushedBytes),
RdbCompactions: metric.NewGauge(metaRdbCompactions),
RdbIngestedBytes: metric.NewGauge(metaRdbIngestedBytes),
RdbCompactedBytesRead: metric.NewGauge(metaRdbCompactedBytesRead),
RdbCompactedBytesWritten: metric.NewGauge(metaRdbCompactedBytesWritten),
RdbTableReadersMemEstimate: metric.NewGauge(metaRdbTableReadersMemEstimate),
RdbReadAmplification: metric.NewGauge(metaRdbReadAmplification),
RdbNumSSTables: metric.NewGauge(metaRdbNumSSTables),
RdbPendingCompaction: metric.NewGauge(metaRdbPendingCompaction),
RdbMarkedForCompactionFiles: metric.NewGauge(metaRdbMarkedForCompactionFiles),
RdbKeysRangeKeySets: metric.NewGauge(metaRdbKeysRangeKeySets),
RdbKeysTombstones: metric.NewGauge(metaRdbKeysTombstones),
RdbL0BytesFlushed: metric.NewGauge(metaRdbL0BytesFlushed),
RdbL0Sublevels: metric.NewGauge(metaRdbL0Sublevels),
RdbL0NumFiles: metric.NewGauge(metaRdbL0NumFiles),
RdbBytesIngested: rdbBytesIngested,
RdbLevelSize: rdbLevelSize,
RdbLevelScore: rdbLevelScore,
RdbWriteStalls: metric.NewGauge(metaRdbWriteStalls),
RdbWriteStallNanos: metric.NewGauge(metaRdbWriteStallNanos),
IterBlockBytes: metric.NewGauge(metaBlockBytes),
IterBlockBytesInCache: metric.NewGauge(metaBlockBytesInCache),
IterBlockReadDuration: metric.NewGauge(metaBlockReadDuration),
IterExternalSeeks: metric.NewGauge(metaIterExternalSeeks),
IterExternalSteps: metric.NewGauge(metaIterExternalSteps),
IterInternalSeeks: metric.NewGauge(metaIterInternalSeeks),
IterInternalSteps: metric.NewGauge(metaIterInternalSteps),
SharedStorageBytesRead: metric.NewGauge(metaSharedStorageBytesRead),
SharedStorageBytesWritten: metric.NewGauge(metaSharedStorageBytesWritten),
StorageCompactionsPinnedKeys: metric.NewGauge(metaStorageCompactionsKeysPinnedCount),
StorageCompactionsPinnedBytes: metric.NewGauge(metaStorageCompactionsKeysPinnedBytes),
FlushableIngestCount: metric.NewGauge(metaFlushableIngestCount),
FlushableIngestTableCount: metric.NewGauge(metaFlushableIngestTableCount),
FlushableIngestTableSize: metric.NewGauge(metaFlushableIngestTableBytes),

RdbCheckpoints: metric.NewGauge(metaRdbCheckpoints),

Expand Down Expand Up @@ -2853,6 +2885,8 @@ func (sm *StoreMetrics) updateEngineMetrics(m storage.Metrics) {
sm.IterExternalSteps.Update(int64(m.Iterator.ExternalSteps))
sm.IterInternalSeeks.Update(int64(m.Iterator.InternalSeeks))
sm.IterInternalSteps.Update(int64(m.Iterator.InternalSteps))
sm.StorageCompactionsPinnedKeys.Update(int64(m.Snapshots.PinnedKeys))
sm.StorageCompactionsPinnedBytes.Update(int64(m.Snapshots.PinnedSize))
sm.SharedStorageBytesRead.Update(m.SharedStorageReadBytes)
sm.SharedStorageBytesWritten.Update(m.SharedStorageWriteBytes)
sm.RdbL0Sublevels.Update(int64(m.Levels[0].Sublevels))
Expand Down
8 changes: 8 additions & 0 deletions pkg/ts/catalog/chart_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -3095,6 +3095,14 @@ var charts = []sectionDescription{
Title: "Tombstone Count",
Metrics: []string{"storage.keys.tombstone.count"},
},
{
Title: "Pinned Keys Written",
Metrics: []string{"storage.compactions.keys.pinned.count"},
},
{
Title: "Pinned Key Bytes Written",
Metrics: []string{"storage.compactions.keys.pinned.bytes"},
},
},
},
{
Expand Down

0 comments on commit 55e50b8

Please sign in to comment.