Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
86277: eventpb: add storage event types r=jbowens,sumeerbhola a=nicktrav

Add the `StoreStats` event type, a per-store event emitted to the
`TELEMETRY` logging channel. This event type will be computed from the
Pebble metrics for each store.

Emit a `StoreStats` event periodically, by default, once per hour, per
store.

Touches #85589.

Release note: None.

Release justification: low risk, high benefit changes to existing
functionality.

87142: workload/mixed-version/schemachanger: re-enable mixed version workload r=fqazi a=fqazi

Fixes: #58489 #87477

Previously the mixed version schema changer workload was disabled because of the lack of version gates. These changes will do the following:

- Start reporting errors on this workload again.
- Disable trigrams in a mixed version state.
- Disable the insert part of the workload in a mixed version state (there is an optimizer on 22.1 that can cause some of the queries to fail)

Release justification: low risk only extends test coverage

87883: schedulerlatency: export Go scheduling latency metric r=irfansharif a=irfansharif

And record data into CRDB's internal time-series database. Informs
\#82743 and #87823. To export scheduling latencies to prometheus, we
choose an exponential bucketing scheme with base multiple of 1.1, and
the output range bounded to [50us, 100ms). This makes for ~70 buckets.
It's worth noting that the default histogram buckets used in Go are
not fit for our purposes. If we care about improving it, we could
consider patching the runtime.

```
  bucket[  0] width=0s boundary=[-Inf, 0s)
  bucket[  1] width=1ns boundary=[0s, 1ns)
  bucket[  2] width=1ns boundary=[1ns, 2ns)
  bucket[  3] width=1ns boundary=[2ns, 3ns)
  bucket[  4] width=1ns boundary=[3ns, 4ns)
  ...
  bucket[270] width=16.384µs boundary=[737.28µs, 753.664µs)
  bucket[271] width=16.384µs boundary=[753.664µs, 770.048µs)
  bucket[272] width=278.528µs boundary=[770.048µs, 1.048576ms)
  bucket[273] width=32.768µs boundary=[1.048576ms, 1.081344ms)
  bucket[274] width=32.768µs boundary=[1.081344ms, 1.114112ms)
  ...
  bucket[717] width=1h13m18.046511104s boundary=[53h45m14.046488576s, 54h58m32.09299968s)
  bucket[718] width=1h13m18.046511104s boundary=[54h58m32.09299968s, 56h11m50.139510784s)
  bucket[719] width=1h13m18.046511104s boundary=[56h11m50.139510784s, 57h25m8.186021888s)
  bucket[720] width=57h25m8.186021888s boundary=[57h25m8.186021888s, +Inf)
```

Release note: None
Release justification: observability-only PR, low-risk high-benefit; would help understand admission control out in the wild

88179: ui/cluster-ui: fix no most recent stmt for active txns r=xinhaoz a=xinhaoz

Fixes #87738

Previously, active txns could have an empty 'Most Recent Statement' column, even if their executed statement count was non-zero. This was due to the most recent query text being populated by the active stmt, which could be empty at the time of querying. This commit populates the last statement text for a txn even when it is not currently executing a query.

This commit also removes the `isFullScan` field from active txn pages, as we cannot fill this field out without all stmts in the txn.

Release note (ui change): Full scan field is removed from active txn details page.

Release note (bug fix): active txns with non-zero
executed statement count now always have populated stmt text, even when no stmt is being executed.

88334: kvserver: align Raft recv/send queue sizes r=erikgrinaker a=pavelkalinnikov

Fixes #87465

Release justification: performance fix
Release note: Made sending and receiving Raft queue sizes match. Previously the receiver could unnecessarily drop messages in situations when the sending queue is bigger than the receiving one.

Co-authored-by: Nick Travers <travers@cockroachlabs.com>
Co-authored-by: Faizan Qazi <faizan@cockroachlabs.com>
Co-authored-by: irfan sharif <irfanmahmoudsharif@gmail.com>
Co-authored-by: Xin Hao Zhang <xzhang@cockroachlabs.com>
Co-authored-by: Pavel Kalinnikov <pavel@cockroachlabs.com>
  • Loading branch information
6 people committed Sep 21, 2022
6 parents 6450a9d + 1fc9149 + dd954ff + e397faf + 6deea40 + 11cdc9a commit 766b62d
Show file tree
Hide file tree
Showing 35 changed files with 2,754 additions and 272 deletions.
1 change: 1 addition & 0 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -1633,6 +1633,7 @@ EVENTPB_PROTOS = \
pkg/util/log/eventpb/cluster_events.proto \
pkg/util/log/eventpb/job_events.proto \
pkg/util/log/eventpb/health_events.proto \
pkg/util/log/eventpb/storage_events.proto \
pkg/util/log/eventpb/telemetry.proto

EVENTLOG_PROTOS = pkg/util/log/logpb/event.proto $(EVENTPB_PROTOS)
Expand Down
83 changes: 83 additions & 0 deletions docs/generated/eventlog.md
Original file line number Diff line number Diff line change
Expand Up @@ -2438,6 +2438,89 @@ are automatically converted server-side.
| `NewMethod` | The new hash method. | no |


#### Common fields

| Field | Description | Sensitive |
|--|--|--|
| `Timestamp` | The timestamp of the event. Expressed as nanoseconds since the Unix epoch. | no |
| `EventType` | The type of the event. | no |

## Storage telemetry events



Events in this category are logged to the `TELEMETRY` channel.


### `level_stats`

An event of type `level_stats` contains per-level statistics for an LSM.


| Field | Description | Sensitive |
|--|--|--|
| `Level` | level is the level ID in a LSM (e.g. level(L0) == 0, etc.) | no |
| `NumFiles` | num_files is the number of files in the level (gauge). | no |
| `SizeBytes` | size_bytes is the size of the level, in bytes (gauge). | no |
| `Score` | score is the compaction score of the level (gauge). | no |
| `BytesIn` | bytes_in is the number of bytes written to this level (counter). | no |
| `BytesIngested` | bytes_ingested is the number of bytes ingested into this level (counter). | no |
| `BytesMoved` | bytes_moved is the number of bytes moved into this level via a move-compaction (counter). | no |
| `BytesRead` | bytes_read is the number of bytes read from this level, during compactions (counter). | no |
| `BytesCompacted` | bytes_compacted is the number of bytes written to this level during compactions (counter). | no |
| `BytesFlushed` | bytes flushed is the number of bytes flushed to this level. This value is always zero for levels other than L0 (counter). | no |
| `TablesCompacted` | tables_compacted is the count of tables compacted into this level (counter). | no |
| `TablesFlushed` | tables_flushed is the count of tables flushed into this level (counter). | no |
| `TablesIngested` | tables_ingested is the count of tables ingested into this level (counter). | no |
| `TablesMoved` | tables_moved is the count of tables moved into this level via move-compactions (counter). | no |
| `NumSublevels` | num_sublevel is the count of sublevels for the level. This value is always zero for levels other than L0 (gauge). | no |



### `store_stats`

An event of type `store_stats` contains per store stats.

Note that because stats are scoped to the lifetime of the process, counters
(and certain gauges) will be reset across node restarts.


| Field | Description | Sensitive |
|--|--|--|
| `NodeId` | node_id is the ID of the node. | no |
| `StoreId` | store_id is the ID of the store. | no |
| `Levels` | levels is a nested message containing per-level statistics. | yes |
| `CacheSize` | cache_size is the size of the cache for the store, in bytes (gauge). | no |
| `CacheCount` | cache_count is the number of items in the cache (gauge). | no |
| `CacheHits` | cache_hits is the number of cache hits (counter). | no |
| `CacheMisses` | cache_misses is the number of cache misses (counter). | no |
| `CompactionCountDefault` | compaction_count_default is the count of default compactions (counter). | no |
| `CompactionCountDeleteOnly` | compaction_count_delete_only is the count of delete-only compactions (counter). | no |
| `CompactionCountElisionOnly` | compaction_count_elision_only is the count of elision-only compactions (counter). | no |
| `CompactionCountMove` | compaction_count_move is the count of move-compactions (counter). | no |
| `CompactionCountRead` | compaction_count_read is the count of read-compactions (counter). | no |
| `CompactionCountRewrite` | compaction_count_rewrite is the count of rewrite-compactions (counter). | no |
| `CompactionNumInProgress` | compactions_num_in_progress is the number of compactions in progress (gauge). | no |
| `CompactionMarkedFiles` | compaction_marked_files is the count of files marked for compaction (gauge). | no |
| `FlushCount` | flush_count is the number of flushes (counter). | no |
| `MemtableSize` | memtable_size is the total size allocated to all memtables and (large) batches, in bytes (gauge). | no |
| `MemtableCount` | memtable_count is the count of memtables (gauge). | no |
| `MemtableZombieCount` | memtable_zombie_count is the count of memtables no longer referenced by the current DB state, but still in use by an iterator (gauge). | no |
| `MemtableZombieSize` | memtable_zombie_size is the size, in bytes, of all zombie memtables (gauge). | no |
| `WalLiveCount` | wal_live_count is the count of live WAL files (gauge). | no |
| `WalLiveSize` | wal_live_size is the size, in bytes, of live data in WAL files. With WAL recycling, this value is less than the actual on-disk size of the WAL files (gauge). | no |
| `WalObsoleteCount` | wal_obsolete_count is the count of obsolete WAL files (gauge). | no |
| `WalObsoleteSize` | wal_obsolete_size is the size of obsolete WAL files, in bytes (gauge). | no |
| `WalPhysicalSize` | wal_physical_size is the size, in bytes, of the WAL files on disk (gauge). | no |
| `WalBytesIn` | wal_bytes_in is the number of logical bytes written to the WAL (counter). | no |
| `WalBytesWritten` | wal_bytes_written is the number of bytes written to the WAL (counter). | no |
| `TableObsoleteCount` | table_obsolete_count is the number of tables which are no longer referenced by the current DB state or any open iterators (gauge). | no |
| `TableObsoleteSize` | table_obsolete_size is the size, in bytes, of obsolete tables (gauge). | no |
| `TableZombieCount` | table_zombie_count is the number of tables no longer referenced by the current DB state, but are still in use by an open iterator (gauge). | no |
| `TableZombieSize` | table_zombie_size is the size, in bytes, of zombie tables (gauge). | no |
| `RangeKeySetsCount` | range_key_sets_count is the approximate count of internal range key sets in the store. | no |


#### Common fields

| Field | Description | Sensitive |
Expand Down
4 changes: 4 additions & 0 deletions pkg/base/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -361,6 +361,10 @@ type RaftConfig struct {
// without acknowledgement. With an average entry size of 1 KB that
// translates to ~4096 commands that might be executed in the handling of a
// single raft.Ready operation.
//
// This setting is used both by sending and receiving end of Raft messages. To
// minimize dropped messages on the receiver, its size should at least match
// the sender's (being it the default size, or taken from the env variables).
RaftMaxInflightMsgs int

// Splitting a range which has a replica needing a snapshot results in two
Expand Down
9 changes: 2 additions & 7 deletions pkg/cmd/roachtest/tests/mixed_version_schemachange.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,8 @@ func registerSchemaChangeMixedVersions(r registry.Registry) {
// This tests the work done for 20.1 that made schema changes jobs and in
// addition prevented making any new schema changes on a mixed cluster in
// order to prevent bugs during upgrades.
Cluster: r.MakeClusterSpec(4),
Cluster: r.MakeClusterSpec(4),
NativeLibs: registry.LibGEOS,
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
maxOps := 100
concurrency := 5
Expand Down Expand Up @@ -56,12 +57,6 @@ func runSchemaChangeWorkloadStep(loadNode, maxOps, concurrency int) versionStep
t.L().Printf("Workload step run: %d", numFeatureRuns)
runCmd := []string{
"./workload run schemachange --verbose=1",
// The workload is still in development and occasionally discovers schema
// change errors so for now we don't fail on them but only on panics, server
// crashes, deadlocks, etc.
// TODO(spaskob): remove when https://github.com/cockroachdb/cockroach/issues/47430
// is closed.
"--tolerate-errors=true",
fmt.Sprintf("--max-ops %d", maxOps),
fmt.Sprintf("--concurrency %d", concurrency),
fmt.Sprintf("{pgurl:1-%d}", u.c.Spec().NodeCount),
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -185,6 +185,7 @@ go_library(
"//pkg/util/iterutil",
"//pkg/util/limit",
"//pkg/util/log",
"//pkg/util/log/logcrash",
"//pkg/util/metric",
"//pkg/util/metric/aggmetric",
"//pkg/util/mon",
Expand Down
28 changes: 25 additions & 3 deletions pkg/kv/kvserver/store.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/iterutil"
"github.com/cockroachdb/cockroach/pkg/util/limit"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/log/logcrash"
"github.com/cockroachdb/cockroach/pkg/util/metric"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
Expand Down Expand Up @@ -98,9 +99,11 @@ const (
// store's Raft log entry cache.
defaultRaftEntryCacheSize = 1 << 24 // 16M

// replicaRequestQueueSize specifies the maximum number of requests to queue
// for a replica.
replicaRequestQueueSize = 100
// replicaQueueExtraSize is the number of requests that a replica's incoming
// message queue can keep over RaftConfig.RaftMaxInflightMsgs. When the leader
// maxes out RaftMaxInflightMsgs, we want the receiving replica to still have
// some buffer for other messages, primarily heartbeats.
replicaQueueExtraSize = 10

defaultGossipWhenCapacityDeltaExceedsFraction = 0.01

Expand All @@ -125,6 +128,13 @@ var storeSchedulerConcurrency = envutil.EnvOrDefaultInt(
var logSSTInfoTicks = envutil.EnvOrDefaultInt(
"COCKROACH_LOG_SST_INFO_TICKS_INTERVAL", 60)

// By default, telemetry events are emitted once per hour, per store:
// (10s tick interval) * 6 * 60 = 3600s = 1h.
var logStoreTelemetryTicks = envutil.EnvOrDefaultInt(
"COCKROACH_LOG_STORE_TELEMETRY_TICKS_INTERVAL",
6*60,
)

// bulkIOWriteLimit is defined here because it is used by BulkIOWriteLimiter.
var bulkIOWriteLimit = settings.RegisterByteSizeSetting(
settings.TenantWritable,
Expand Down Expand Up @@ -3347,6 +3357,18 @@ func (s *Store) ComputeMetrics(ctx context.Context, tick int) error {
// will not contain the log prefix.
log.Infof(ctx, "\n%s", m.Metrics)
}
// Periodically emit a store stats structured event to the TELEMETRY channel,
// if reporting is enabled. These events are intended to be emitted at low
// frequency. Trigger on every (N-1)-th tick to avoid spamming the telemetry
// channel if crash-looping.
if logcrash.DiagnosticsReportingEnabled.Get(&s.ClusterSettings().SV) &&
tick%logStoreTelemetryTicks == logStoreTelemetryTicks-1 {
// The stats event is populated from a subset of the Metrics.
e := m.AsStoreStatsEvent()
e.NodeId = int32(s.NodeID())
e.StoreId = int32(s.StoreID())
log.StructuredEvent(ctx, &e)
}
return nil
}

Expand Down
15 changes: 9 additions & 6 deletions pkg/kv/kvserver/store_raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,8 @@ type raftReceiveQueue struct {
syncutil.Mutex
infos []raftRequestInfo
}
acc mon.BoundAccount
maxLen int
acc mon.BoundAccount
}

// Len returns the number of requests in the queue.
Expand Down Expand Up @@ -109,7 +110,7 @@ func (q *raftReceiveQueue) Append(
size = int64(req.Size())
q.mu.Lock()
defer q.mu.Unlock()
if q.mu.destroyed || len(q.mu.infos) >= replicaRequestQueueSize {
if q.mu.destroyed || len(q.mu.infos) >= q.maxLen {
return false, size, false
}
if q.acc.Grow(context.Background(), size) != nil {
Expand All @@ -136,13 +137,12 @@ func (qs *raftReceiveQueues) Load(rangeID roachpb.RangeID) (*raftReceiveQueue, b
}

func (qs *raftReceiveQueues) LoadOrCreate(
rangeID roachpb.RangeID,
rangeID roachpb.RangeID, maxLen int,
) (_ *raftReceiveQueue, loaded bool) {

if q, ok := qs.Load(rangeID); ok {
return q, ok // fast path
}
q := &raftReceiveQueue{}
q := &raftReceiveQueue{maxLen: maxLen}
q.acc.Init(context.Background(), qs.mon)
value, loaded := qs.m.LoadOrStore(int64(rangeID), unsafe.Pointer(q))
return (*raftReceiveQueue)(value), loaded
Expand Down Expand Up @@ -303,7 +303,10 @@ func (s *Store) HandleRaftUncoalescedRequest(
// count them.
s.metrics.RaftRcvdMessages[req.Message.Type].Inc(1)

q, _ := s.raftRecvQueues.LoadOrCreate(req.RangeID)
// NB: add a buffer for extra messages, to allow heartbeats getting through
// even if MsgApp quota is maxed out by the sender.
q, _ := s.raftRecvQueues.LoadOrCreate(req.RangeID,
s.cfg.RaftMaxInflightMsgs+replicaQueueExtraSize)
enqueue, size, appended := q.Append(req, respStream)
if !appended {
// TODO(peter): Return an error indicating the request was dropped. Note
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/store_raft_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,11 +44,11 @@ func TestRaftReceiveQueue(t *testing.T) {
qs.Load(r5)
require.Zero(t, m.AllocBytes())

q1, loaded := qs.LoadOrCreate(r1)
q1, loaded := qs.LoadOrCreate(r1, 10 /* maxLen */)
require.Zero(t, m.AllocBytes())
require.False(t, loaded)
{
q1x, loadedx := qs.LoadOrCreate(r1)
q1x, loadedx := qs.LoadOrCreate(r1, 10 /* maxLen */)
require.True(t, loadedx)
require.Equal(t, q1, q1x)
}
Expand Down Expand Up @@ -99,7 +99,7 @@ func TestRaftReceiveQueue(t *testing.T) {
}

// Now interleave creation of a second queue.
q5, loaded := qs.LoadOrCreate(r5)
q5, loaded := qs.LoadOrCreate(r5, 1 /* maxLen */)
{
require.False(t, loaded)
require.Zero(t, q5.acc.Used())
Expand Down
6 changes: 4 additions & 2 deletions pkg/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -1403,7 +1403,9 @@ func (s *Server) PreStart(ctx context.Context) error {
}
})

if err := schedulerlatency.StartSampler(ctx, s.st, s.stopper); err != nil {
if err := schedulerlatency.StartSampler(
ctx, s.st, s.stopper, s.registry, base.DefaultMetricsSampleInterval,
); err != nil {
return err
}

Expand Down Expand Up @@ -1469,7 +1471,7 @@ func (s *Server) PreStart(ctx context.Context) error {
})

// We can now add the node registry.
s.recorder.AddNode(
s.recorder.AddNode( // XXX: Has to occur before
s.registry,
s.node.Descriptor,
s.node.startedAt,
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/status/recorder.go
Original file line number Diff line number Diff line change
Expand Up @@ -528,7 +528,7 @@ type registryRecorder struct {

func extractValue(name string, mtr interface{}, fn func(string, float64)) error {
switch mtr := mtr.(type) {
case *metric.Histogram:
case metric.WindowedHistogram:
n := float64(mtr.TotalCountWindowed())
fn(name+"-count", n)
avg := mtr.TotalSumWindowed() / n
Expand Down
1 change: 1 addition & 0 deletions pkg/storage/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,7 @@ go_library(
"//pkg/util/humanizeutil",
"//pkg/util/iterutil",
"//pkg/util/log",
"//pkg/util/log/eventpb",
"//pkg/util/mon",
"//pkg/util/protoutil",
"//pkg/util/syncutil",
Expand Down
60 changes: 60 additions & 0 deletions pkg/storage/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/iterutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/log/eventpb"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -1014,6 +1015,65 @@ func (m *Metrics) CompactedBytes() (read, written uint64) {
return read, written
}

// AsStoreStatsEvent converts a Metrics struct into an eventpb.StoreStats event,
// suitable for logging to the telemetry channel.
func (m *Metrics) AsStoreStatsEvent() eventpb.StoreStats {
e := eventpb.StoreStats{
CacheSize: m.BlockCache.Size,
CacheCount: m.BlockCache.Count,
CacheHits: m.BlockCache.Hits,
CacheMisses: m.BlockCache.Misses,
CompactionCountDefault: m.Compact.DefaultCount,
CompactionCountDeleteOnly: m.Compact.DeleteOnlyCount,
CompactionCountElisionOnly: m.Compact.ElisionOnlyCount,
CompactionCountMove: m.Compact.MoveCount,
CompactionCountRead: m.Compact.ReadCount,
CompactionCountRewrite: m.Compact.RewriteCount,
CompactionNumInProgress: m.Compact.NumInProgress,
CompactionMarkedFiles: int64(m.Compact.MarkedFiles),
FlushCount: m.Flush.Count,
MemtableSize: m.MemTable.Size,
MemtableCount: m.MemTable.Count,
MemtableZombieCount: m.MemTable.ZombieCount,
MemtableZombieSize: m.MemTable.ZombieSize,
WalLiveCount: m.WAL.Files,
WalLiveSize: m.WAL.Size,
WalObsoleteCount: m.WAL.ObsoleteFiles,
WalObsoleteSize: m.WAL.ObsoletePhysicalSize,
WalPhysicalSize: m.WAL.PhysicalSize,
WalBytesIn: m.WAL.BytesIn,
WalBytesWritten: m.WAL.BytesWritten,
TableObsoleteCount: m.Table.ObsoleteCount,
TableObsoleteSize: m.Table.ObsoleteSize,
TableZombieCount: m.Table.ZombieCount,
TableZombieSize: m.Table.ZombieSize,
RangeKeySetsCount: m.Keys.RangeKeySetsCount,
}
for i, l := range m.Levels {
if l.NumFiles == 0 {
continue
}
e.Levels = append(e.Levels, eventpb.LevelStats{
Level: uint32(i),
NumFiles: l.NumFiles,
SizeBytes: l.Size,
Score: float32(l.Score),
BytesIn: l.BytesIn,
BytesIngested: l.BytesIngested,
BytesMoved: l.BytesMoved,
BytesRead: l.BytesRead,
BytesCompacted: l.BytesCompacted,
BytesFlushed: l.BytesFlushed,
TablesCompacted: l.TablesCompacted,
TablesFlushed: l.TablesFlushed,
TablesIngested: l.TablesIngested,
TablesMoved: l.TablesMoved,
NumSublevels: l.Sublevels,
})
}
return e
}

// EnvStats is a set of RocksDB env stats, including encryption status.
type EnvStats struct {
// TotalFiles is the total number of files reported by rocksdb.
Expand Down
6 changes: 6 additions & 0 deletions pkg/ts/catalog/chart_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -3521,6 +3521,12 @@ var charts = []sectionDescription{
"admission.scheduler_latency_listener.p99_nanos",
},
},
{
Title: "Scheduler Latency",
Metrics: []string{
"go.scheduler_latency",
},
},
{
Title: "Elastic CPU Durations",
Metrics: []string{
Expand Down
Loading

0 comments on commit 766b62d

Please sign in to comment.