Skip to content

Commit

Permalink
storage: export per-store disk metrics
Browse files Browse the repository at this point in the history
Currently, disk metrics are computed and aggregated at a node level.
However, a cockroach node can run multiple stores. This commit
adds new timeseries disk metrics computed per-store.

Epic: None.
Release note (general change): The following metrics were added for
observability of per-store disk events:
- storage.disk.read.count
- storage.disk.read.bytes
- storage.disk.read.time
- storage.disk.write.count
- storage.disk.write.bytes
- storage.disk.write.time
- storage.disk.io.time
- storage.disk.weightedio.time
- storage.disk.iopsinprogress

The metrics match the definitions of the sys.host.disk.* system metrics.
  • Loading branch information
CheranMahalingam committed Mar 18, 2024
1 parent 425133f commit f33445b
Show file tree
Hide file tree
Showing 5 changed files with 109 additions and 5 deletions.
11 changes: 10 additions & 1 deletion docs/generated/metrics/metrics.html
Original file line number Diff line number Diff line change
Expand Up @@ -630,6 +630,15 @@
<tr><td>STORAGE</td><td>storage.compactions.keys.pinned.count</td><td>Cumulative count of storage engine KVs written to sstables during flushes and compactions due to open LSM snapshots.<br/><br/>Various subsystems of CockroachDB take LSM snapshots to maintain a consistent view<br/>of the database over an extended duration. In order to maintain the consistent view,<br/>flushes and compactions within the storage engine must preserve keys that otherwise<br/>would have been dropped. This increases write amplification, and introduces keys<br/>that must be skipped during iteration. This metric records the cumulative count of<br/>KVs preserved during flushes and compactions over the lifetime of the process.<br/></td><td>Keys</td><td>GAUGE</td><td>COUNT</td><td>AVG</td><td>NONE</td></tr>
<tr><td>STORAGE</td><td>storage.disk-slow</td><td>Number of instances of disk operations taking longer than 10s</td><td>Events</td><td>GAUGE</td><td>COUNT</td><td>AVG</td><td>NONE</td></tr>
<tr><td>STORAGE</td><td>storage.disk-stalled</td><td>Number of instances of disk operations taking longer than 20s</td><td>Events</td><td>GAUGE</td><td>COUNT</td><td>AVG</td><td>NONE</td></tr>
<tr><td>STORAGE</td><td>storage.disk.io.time</td><td>Time spent reading from or writing to the store&#39;s disk since this process started (as reported by the OS)</td><td>Time</td><td>GAUGE</td><td>NANOSECONDS</td><td>AVG</td><td>NONE</td></tr>
<tr><td>STORAGE</td><td>storage.disk.iopsinprogress</td><td>IO operations currently in progress on the store&#39;s disk (as reported by the OS)</td><td>Operations</td><td>GAUGE</td><td>COUNT</td><td>AVG</td><td>NONE</td></tr>
<tr><td>STORAGE</td><td>storage.disk.read.bytes</td><td>Bytes read from the store&#39;s disk since this process started (as reported by the OS)</td><td>Bytes</td><td>GAUGE</td><td>BYTES</td><td>AVG</td><td>NONE</td></tr>
<tr><td>STORAGE</td><td>storage.disk.read.count</td><td>Disk read operations on the store&#39;s disk since this process started (as reported by the OS)</td><td>Operations</td><td>GAUGE</td><td>COUNT</td><td>AVG</td><td>NONE</td></tr>
<tr><td>STORAGE</td><td>storage.disk.read.time</td><td>Time spent reading from the store&#39;s disk since this process started (as reported by the OS)</td><td>Time</td><td>GAUGE</td><td>NANOSECONDS</td><td>AVG</td><td>NONE</td></tr>
<tr><td>STORAGE</td><td>storage.disk.weightedio.time</td><td>Weighted time spent reading from or writing to the store&#39;s disk since this process started (as reported by the OS)</td><td>Time</td><td>GAUGE</td><td>NANOSECONDS</td><td>AVG</td><td>NONE</td></tr>
<tr><td>STORAGE</td><td>storage.disk.write.bytes</td><td>Bytes written to the store&#39;s disk since this process started (as reported by the OS)</td><td>Bytes</td><td>GAUGE</td><td>BYTES</td><td>AVG</td><td>NONE</td></tr>
<tr><td>STORAGE</td><td>storage.disk.write.count</td><td>Disk write operations on the store&#39;s disk since this process started (as reported by the OS)</td><td>Operations</td><td>GAUGE</td><td>COUNT</td><td>AVG</td><td>NONE</td></tr>
<tr><td>STORAGE</td><td>storage.disk.write.time</td><td>Time spent writing to the store&#39;s disks since this process started (as reported by the OS)</td><td>Time</td><td>GAUGE</td><td>NANOSECONDS</td><td>AVG</td><td>NONE</td></tr>
<tr><td>STORAGE</td><td>storage.flush.ingest.count</td><td>Flushes performing an ingest (flushable ingestions)</td><td>Flushes</td><td>GAUGE</td><td>COUNT</td><td>AVG</td><td>NONE</td></tr>
<tr><td>STORAGE</td><td>storage.flush.ingest.table.bytes</td><td>Bytes ingested via flushes (flushable ingestions)</td><td>Bytes</td><td>GAUGE</td><td>BYTES</td><td>AVG</td><td>NONE</td></tr>
<tr><td>STORAGE</td><td>storage.flush.ingest.table.count</td><td>Tables ingested via flushes (flushable ingestions)</td><td>Tables</td><td>GAUGE</td><td>COUNT</td><td>AVG</td><td>NONE</td></tr>
Expand Down Expand Up @@ -1590,7 +1599,7 @@
<tr><td>SERVER</td><td>sys.host.disk.read.bytes</td><td>Bytes read from all disks since this process started (as reported by the OS)</td><td>Bytes</td><td>GAUGE</td><td>BYTES</td><td>AVG</td><td>NONE</td></tr>
<tr><td>SERVER</td><td>sys.host.disk.read.count</td><td>Disk read operations across all disks since this process started (as reported by the OS)</td><td>Operations</td><td>GAUGE</td><td>COUNT</td><td>AVG</td><td>NONE</td></tr>
<tr><td>SERVER</td><td>sys.host.disk.read.time</td><td>Time spent reading from all disks since this process started (as reported by the OS)</td><td>Time</td><td>GAUGE</td><td>NANOSECONDS</td><td>AVG</td><td>NONE</td></tr>
<tr><td>SERVER</td><td>sys.host.disk.weightedio.time</td><td>Weighted time spent reading from or writing to to all disks since this process started (as reported by the OS)</td><td>Time</td><td>GAUGE</td><td>NANOSECONDS</td><td>AVG</td><td>NONE</td></tr>
<tr><td>SERVER</td><td>sys.host.disk.weightedio.time</td><td>Weighted time spent reading from or writing to all disks since this process started (as reported by the OS)</td><td>Time</td><td>GAUGE</td><td>NANOSECONDS</td><td>AVG</td><td>NONE</td></tr>
<tr><td>SERVER</td><td>sys.host.disk.write.bytes</td><td>Bytes written to all disks since this process started (as reported by the OS)</td><td>Bytes</td><td>GAUGE</td><td>BYTES</td><td>AVG</td><td>NONE</td></tr>
<tr><td>SERVER</td><td>sys.host.disk.write.count</td><td>Disk write operations across all disks since this process started (as reported by the OS)</td><td>Operations</td><td>GAUGE</td><td>COUNT</td><td>AVG</td><td>NONE</td></tr>
<tr><td>SERVER</td><td>sys.host.disk.write.time</td><td>Time spent writing to all disks since this process started (as reported by the OS)</td><td>Time</td><td>GAUGE</td><td>NANOSECONDS</td><td>AVG</td><td>NONE</td></tr>
Expand Down
88 changes: 88 additions & 0 deletions pkg/kv/kvserver/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/raft/raftpb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/disk"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/storage/fs"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -2355,6 +2356,60 @@ Note that the measurement does not include the duration for replicating the eval
Measurement: "Batches",
Unit: metric.Unit_COUNT,
}
metaDiskReadCount = metric.Metadata{
Name: "storage.disk.read.count",
Unit: metric.Unit_COUNT,
Measurement: "Operations",
Help: "Disk read operations on the store's disk since this process started (as reported by the OS)",
}
metaDiskReadBytes = metric.Metadata{
Name: "storage.disk.read.bytes",
Unit: metric.Unit_BYTES,
Measurement: "Bytes",
Help: "Bytes read from the store's disk since this process started (as reported by the OS)",
}
metaDiskReadTime = metric.Metadata{
Name: "storage.disk.read.time",
Unit: metric.Unit_NANOSECONDS,
Measurement: "Time",
Help: "Time spent reading from the store's disk since this process started (as reported by the OS)",
}
metaDiskWriteCount = metric.Metadata{
Name: "storage.disk.write.count",
Unit: metric.Unit_COUNT,
Measurement: "Operations",
Help: "Disk write operations on the store's disk since this process started (as reported by the OS)",
}
metaDiskWriteBytes = metric.Metadata{
Name: "storage.disk.write.bytes",
Unit: metric.Unit_BYTES,
Measurement: "Bytes",
Help: "Bytes written to the store's disk since this process started (as reported by the OS)",
}
metaDiskWriteTime = metric.Metadata{
Name: "storage.disk.write.time",
Unit: metric.Unit_NANOSECONDS,
Measurement: "Time",
Help: "Time spent writing to the store's disks since this process started (as reported by the OS)",
}
metaDiskIOTime = metric.Metadata{
Name: "storage.disk.io.time",
Unit: metric.Unit_NANOSECONDS,
Measurement: "Time",
Help: "Time spent reading from or writing to the store's disk since this process started (as reported by the OS)",
}
metaDiskWeightedIOTime = metric.Metadata{
Name: "storage.disk.weightedio.time",
Unit: metric.Unit_NANOSECONDS,
Measurement: "Time",
Help: "Weighted time spent reading from or writing to the store's disk since this process started (as reported by the OS)",
}
metaIopsInProgress = metric.Metadata{
Name: "storage.disk.iopsinprogress",
Unit: metric.Unit_COUNT,
Measurement: "Operations",
Help: "IO operations currently in progress on the store's disk (as reported by the OS)",
}
)

// StoreMetrics is the set of metrics for a given store.
Expand Down Expand Up @@ -2750,6 +2805,17 @@ type StoreMetrics struct {

FlushUtilization *metric.GaugeFloat64
FsyncLatency *metric.ManualWindowHistogram

// Disk metrics
DiskReadBytes *metric.Gauge
DiskReadCount *metric.Gauge
DiskReadTime *metric.Gauge
DiskWriteBytes *metric.Gauge
DiskWriteCount *metric.Gauge
DiskWriteTime *metric.Gauge
DiskIOTime *metric.Gauge
DiskWeightedIOTime *metric.Gauge
IopsInProgress *metric.Gauge
}

type tenantMetricsRef struct {
Expand Down Expand Up @@ -3491,6 +3557,16 @@ func newStoreMetrics(histogramWindow time.Duration) *StoreMetrics {

ReplicaReadBatchDroppedLatchesBeforeEval: metric.NewCounter(metaReplicaReadBatchDroppedLatchesBeforeEval),
ReplicaReadBatchWithoutInterleavingIter: metric.NewCounter(metaReplicaReadBatchWithoutInterleavingIter),

DiskReadBytes: metric.NewGauge(metaDiskReadBytes),
DiskReadCount: metric.NewGauge(metaDiskReadCount),
DiskReadTime: metric.NewGauge(metaDiskReadTime),
DiskWriteBytes: metric.NewGauge(metaDiskWriteBytes),
DiskWriteCount: metric.NewGauge(metaDiskWriteCount),
DiskWriteTime: metric.NewGauge(metaDiskWriteTime),
DiskIOTime: metric.NewGauge(metaDiskIOTime),
DiskWeightedIOTime: metric.NewGauge(metaDiskWeightedIOTime),
IopsInProgress: metric.NewGauge(metaIopsInProgress),
}

storeRegistry.AddMetricStruct(sm)
Expand Down Expand Up @@ -3700,6 +3776,18 @@ func (sm *StoreMetrics) updateEnvStats(stats fs.EnvStats) {
sm.EncryptionAlgorithm.Update(int64(stats.EncryptionType))
}

func (sm *StoreMetrics) updateDiskStats(stats disk.Stats) {
sm.DiskReadCount.Update(int64(stats.ReadsCount))
sm.DiskReadBytes.Update(int64(stats.BytesRead()))
sm.DiskReadTime.Update(int64(stats.ReadsDuration))
sm.DiskWriteCount.Update(int64(stats.WritesCount))
sm.DiskWriteBytes.Update(int64(stats.BytesWritten()))
sm.DiskWriteTime.Update(int64(stats.WritesDuration))
sm.DiskIOTime.Update(int64(stats.CumulativeDuration))
sm.DiskWeightedIOTime.Update(int64(stats.WeightedIODuration))
sm.IopsInProgress.Update(int64(stats.InProgressCount))
}

func (sm *StoreMetrics) handleMetricsResult(ctx context.Context, metric result.Metrics) {
sm.LeaseRequestSuccessCount.Inc(int64(metric.LeaseRequestSuccess))
metric.LeaseRequestSuccess = 0
Expand Down
9 changes: 9 additions & 0 deletions pkg/kv/kvserver/store.go
Original file line number Diff line number Diff line change
Expand Up @@ -3367,6 +3367,15 @@ func (s *Store) computeMetrics(ctx context.Context) (m storage.Metrics, err erro
s.metrics.RdbCheckpoints.Update(int64(len(dirs)))
}

// Get disk stats for the disk associated with this store.
if s.diskMonitor != nil {
diskStats, err := s.diskMonitor.CumulativeStats()
if err != nil {
return m, err
}
s.metrics.updateDiskStats(diskStats)
}

return m, nil
}

Expand Down
1 change: 1 addition & 0 deletions pkg/server/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ import (
"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble"
"github.com/cockroachdb/pebble/bloom"
"github.com/cockroachdb/pebble/vfs"
"github.com/cockroachdb/redact"
)

Expand Down
5 changes: 1 addition & 4 deletions pkg/server/status/runtime.go
Original file line number Diff line number Diff line change
Expand Up @@ -256,7 +256,7 @@ var (
Name: "sys.host.disk.weightedio.time",
Unit: metric.Unit_NANOSECONDS,
Measurement: "Time",
Help: "Weighted time spent reading from or writing to to all disks since this process started (as reported by the OS)",
Help: "Weighted time spent reading from or writing to all disks since this process started (as reported by the OS)",
}
metaHostIopsInProgress = metric.Metadata{
Name: "sys.host.disk.iopsinprogress",
Expand Down Expand Up @@ -734,13 +734,10 @@ func (rsr *RuntimeStatSampler) SampleEnvironment(ctx context.Context, cs *CGoMem
}
}

var deltaDisk DiskStats
diskCounters, err := getSummedDiskCounters(ctx)
if err != nil {
log.Ops.Warningf(ctx, "problem fetching disk stats: %s; disk stats will be empty.", err)
} else {
deltaDisk = diskCounters
subtractDiskCounters(&deltaDisk, rsr.last.disk)
rsr.last.disk = diskCounters
subtractDiskCounters(&diskCounters, rsr.initialDiskCounters)

Expand Down

0 comments on commit f33445b

Please sign in to comment.