Skip to content

Commit

Permalink
server/status: Add timeseries for IO time
Browse files Browse the repository at this point in the history
These metrics are either linux-specific or difficult to interpret so
they're not added to dashboards until we figure out what's useful.

Undoes #30488. The disk read and write time metrics aren't the most
useful, but they're what we have and they're better than nothing.

Fixes #34591

Release note: None
  • Loading branch information
bdarnell committed Mar 18, 2019
1 parent 081bb9a commit 034df0b
Show file tree
Hide file tree
Showing 3 changed files with 88 additions and 10 deletions.
5 changes: 5 additions & 0 deletions pkg/server/status/disk_counters.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ package status

import (
"context"
"time"

"github.com/shirou/gopsutil/disk"
)
Expand All @@ -34,8 +35,12 @@ func getDiskCounters(ctx context.Context) ([]diskStats, error) {
output[i] = diskStats{
readBytes: int64(counters.ReadBytes),
readCount: int64(counters.ReadCount),
readTime: time.Duration(counters.ReadTime) * time.Millisecond,
writeBytes: int64(counters.WriteBytes),
writeCount: int64(counters.WriteCount),
writeTime: time.Duration(counters.WriteTime) * time.Millisecond,
ioTime: time.Duration(counters.IoTime) * time.Millisecond,
weightedIOTime: time.Duration(counters.WeightedIO) * time.Millisecond,
iopsInProgress: int64(counters.IopsInProgress),
}
i++
Expand Down
4 changes: 4 additions & 0 deletions pkg/server/status/disk_counters_darwin.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,8 +33,12 @@ func getDiskCounters(context.Context) ([]diskStats, error) {
output[i] = diskStats{
readBytes: counters.BytesRead,
readCount: counters.NumRead,
readTime: counters.TotalReadTime,
writeBytes: counters.BytesWritten,
writeCount: counters.NumWrite,
writeTime: counters.TotalWriteTime,
ioTime: 0, // Not reported by this library.
weightedIOTime: 0, // Not reported by this library.
iopsInProgress: 0, // Not reported by this library. (#27927)
}
}
Expand Down
89 changes: 79 additions & 10 deletions pkg/server/status/runtime.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metric"
"github.com/dustin/go-humanize"
humanize "github.com/dustin/go-humanize"
"github.com/elastic/gosigar"
"github.com/shirou/gopsutil/net"
)
Expand Down Expand Up @@ -158,6 +158,12 @@ var (
Measurement: "Bytes",
Help: "Bytes read from all disks since this process started",
}
metaHostDiskReadTime = metric.Metadata{
Name: "sys.host.disk.read.time",
Unit: metric.Unit_NANOSECONDS,
Measurement: "Time",
Help: "Time spent reading from all disks since this process started",
}
metaHostDiskWriteCount = metric.Metadata{
Name: "sys.host.disk.write.count",
Unit: metric.Unit_COUNT,
Expand All @@ -170,6 +176,24 @@ var (
Measurement: "Bytes",
Help: "Bytes written to all disks since this process started",
}
metaHostDiskWriteTime = metric.Metadata{
Name: "sys.host.disk.write.time",
Unit: metric.Unit_NANOSECONDS,
Measurement: "Time",
Help: "Time spent writing to all disks since this process started",
}
metaHostDiskIOTime = metric.Metadata{
Name: "sys.host.disk.io.time",
Unit: metric.Unit_NANOSECONDS,
Measurement: "Time",
Help: "Time spent reading from or writing to all disks since this process started",
}
metaHostDiskWeightedIOTime = metric.Metadata{
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",
}
metaHostIopsInProgress = metric.Metadata{
Name: "sys.host.disk.iopsinprogress",
Unit: metric.Unit_COUNT,
Expand Down Expand Up @@ -269,15 +293,19 @@ type RuntimeStatSampler struct {
FDOpen *metric.Gauge
FDSoftLimit *metric.Gauge
// Disk and network stats.
HostDiskReadBytes *metric.Gauge
HostDiskReadCount *metric.Gauge
HostDiskWriteBytes *metric.Gauge
HostDiskWriteCount *metric.Gauge
IopsInProgress *metric.Gauge // not collected on macOS.
HostNetRecvBytes *metric.Gauge
HostNetRecvPackets *metric.Gauge
HostNetSendBytes *metric.Gauge
HostNetSendPackets *metric.Gauge
HostDiskReadBytes *metric.Gauge
HostDiskReadCount *metric.Gauge
HostDiskReadTime *metric.Gauge
HostDiskWriteBytes *metric.Gauge
HostDiskWriteCount *metric.Gauge
HostDiskWriteTime *metric.Gauge
HostDiskIOTime *metric.Gauge
HostDiskWeightedIOTime *metric.Gauge
IopsInProgress *metric.Gauge
HostNetRecvBytes *metric.Gauge
HostNetRecvPackets *metric.Gauge
HostNetSendBytes *metric.Gauge
HostNetSendPackets *metric.Gauge
// Uptime and build.
Uptime *metric.Gauge // We use a gauge to be able to call Update.
BuildTimestamp *metric.Gauge
Expand Down Expand Up @@ -338,8 +366,12 @@ func NewRuntimeStatSampler(ctx context.Context, clock *hlc.Clock) *RuntimeStatSa
Rss: metric.NewGauge(metaRSS),
HostDiskReadBytes: metric.NewGauge(metaHostDiskReadBytes),
HostDiskReadCount: metric.NewGauge(metaHostDiskReadCount),
HostDiskReadTime: metric.NewGauge(metaHostDiskReadTime),
HostDiskWriteBytes: metric.NewGauge(metaHostDiskWriteBytes),
HostDiskWriteCount: metric.NewGauge(metaHostDiskWriteCount),
HostDiskWriteTime: metric.NewGauge(metaHostDiskWriteTime),
HostDiskIOTime: metric.NewGauge(metaHostDiskIOTime),
HostDiskWeightedIOTime: metric.NewGauge(metaHostDiskWeightedIOTime),
IopsInProgress: metric.NewGauge(metaHostIopsInProgress),
HostNetRecvBytes: metric.NewGauge(metaHostNetRecvBytes),
HostNetRecvPackets: metric.NewGauge(metaHostNetRecvPackets),
Expand Down Expand Up @@ -407,8 +439,12 @@ func (rsr *RuntimeStatSampler) SampleEnvironment(ctx context.Context) {

rsr.HostDiskReadBytes.Update(diskCounters.readBytes)
rsr.HostDiskReadCount.Update(diskCounters.readCount)
rsr.HostDiskReadTime.Update(int64(diskCounters.readTime))
rsr.HostDiskWriteBytes.Update(diskCounters.writeBytes)
rsr.HostDiskWriteCount.Update(diskCounters.writeCount)
rsr.HostDiskWriteTime.Update(int64(diskCounters.writeTime))
rsr.HostDiskIOTime.Update(int64(diskCounters.ioTime))
rsr.HostDiskWeightedIOTime.Update(int64(diskCounters.weightedIOTime))
rsr.IopsInProgress.Update(diskCounters.iopsInProgress)
}

Expand Down Expand Up @@ -534,13 +570,36 @@ func (rsr *RuntimeStatSampler) GetCPUCombinedPercentNorm() float64 {
return rsr.CPUCombinedPercentNorm.Value()
}

// diskStats contains the disk statistics returned by the operating
// system. Interpretation of some of these stats varies by platform,
// although as much as possible they are normalized to the semantics
// used by linux's diskstats interface.
//
// Except for iopsInProgress, these metrics act like counters (always
// increasing, and best interpreted as a rate).
type diskStats struct {
readBytes int64
readCount int64

// readTime (and writeTime) may increase more than 1s per second if
// access to storage is parallelized.
readTime time.Duration

writeBytes int64
writeCount int64
writeTime time.Duration

// ioTime is the amount of time that iopsInProgress is non-zero (so
// its increase is capped at 1s/s). Only available on linux.
ioTime time.Duration

// weightedIOTime is a linux-specific metric that attempts to
// represent "an easy measure of both I/O completion time and the
// backlog that may be accumulating."
weightedIOTime time.Duration

// iopsInProgress is a gauge of the number of pending IO operations.
// Not available on macOS.
iopsInProgress int64
}

Expand Down Expand Up @@ -569,9 +628,14 @@ func sumDiskCounters(disksStats []diskStats) diskStats {
for _, stats := range disksStats {
output.readBytes += stats.readBytes
output.readCount += stats.readCount
output.readTime += stats.readTime

output.writeBytes += stats.writeBytes
output.writeCount += stats.writeCount
output.writeTime += stats.writeTime

output.ioTime += stats.ioTime
output.weightedIOTime += stats.weightedIOTime

output.iopsInProgress += stats.iopsInProgress
}
Expand All @@ -583,9 +647,14 @@ func sumDiskCounters(disksStats []diskStats) diskStats {
func subtractDiskCounters(from *diskStats, sub diskStats) {
from.writeCount -= sub.writeCount
from.writeBytes -= sub.writeBytes
from.writeTime -= sub.writeTime

from.readCount -= sub.readCount
from.readBytes -= sub.readBytes
from.readTime -= sub.readTime

from.ioTime -= sub.ioTime
from.weightedIOTime -= sub.weightedIOTime
}

// sumNetworkCounters returns a new net.IOCountersStat whose values are the sum of the
Expand Down

0 comments on commit 034df0b

Please sign in to comment.