From 1fc9149d8a663ccde087ab1af621532a1a52db90 Mon Sep 17 00:00:00 2001 From: Nick Travers Date: Tue, 16 Aug 2022 15:48:44 -0700 Subject: [PATCH 1/7] eventpb: add storage event types; log periodically per-store 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. --- Makefile | 1 + docs/generated/eventlog.md | 83 ++++ pkg/kv/kvserver/BUILD.bazel | 1 + pkg/kv/kvserver/store.go | 20 + pkg/storage/BUILD.bazel | 1 + pkg/storage/engine.go | 60 +++ pkg/util/log/eventpb/BUILD.bazel | 1 + pkg/util/log/eventpb/PROTOS.bzl | 1 + pkg/util/log/eventpb/event_test.go | 7 + .../eventpb/eventlog_channels_generated.go | 6 + pkg/util/log/eventpb/eventpbgen/gen.go | 12 + pkg/util/log/eventpb/json_encode_generated.go | 443 ++++++++++++++++++ pkg/util/log/eventpb/storage_events.proto | 167 +++++++ 13 files changed, 803 insertions(+) create mode 100644 pkg/util/log/eventpb/storage_events.proto diff --git a/Makefile b/Makefile index 069a66d050ad..0c1a24edfea0 100644 --- a/Makefile +++ b/Makefile @@ -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) diff --git a/docs/generated/eventlog.md b/docs/generated/eventlog.md index e43d665e45d6..251aad6e22d8 100644 --- a/docs/generated/eventlog.md +++ b/docs/generated/eventlog.md @@ -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 | diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index bec1040c5d1d..37adbb41f394 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -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", diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 502c7b4939e1..fc676f7f4c9e 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -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" @@ -125,6 +126,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, @@ -3350,6 +3358,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 } diff --git a/pkg/storage/BUILD.bazel b/pkg/storage/BUILD.bazel index 3c95c3a90f4f..76cc03f108dc 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -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", diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index 971b2cddc366..911226d14471 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -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" @@ -1024,6 +1025,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. diff --git a/pkg/util/log/eventpb/BUILD.bazel b/pkg/util/log/eventpb/BUILD.bazel index 9f8c711a7fd2..e3c94b65e7a4 100644 --- a/pkg/util/log/eventpb/BUILD.bazel +++ b/pkg/util/log/eventpb/BUILD.bazel @@ -59,6 +59,7 @@ proto_library( "role_events.proto", "session_events.proto", "sql_audit_events.proto", + "storage_events.proto", "telemetry.proto", "zone_events.proto", ], diff --git a/pkg/util/log/eventpb/PROTOS.bzl b/pkg/util/log/eventpb/PROTOS.bzl index b820c7ffaa4d..ea30af8c87e7 100644 --- a/pkg/util/log/eventpb/PROTOS.bzl +++ b/pkg/util/log/eventpb/PROTOS.bzl @@ -17,6 +17,7 @@ EVENTPB_PROTOS = [ "cluster_events.proto", "job_events.proto", "health_events.proto", + "storage_events.proto", "telemetry.proto", ] diff --git a/pkg/util/log/eventpb/event_test.go b/pkg/util/log/eventpb/event_test.go index d8aa15eb0b15..2c792e3d9677 100644 --- a/pkg/util/log/eventpb/event_test.go +++ b/pkg/util/log/eventpb/event_test.go @@ -58,6 +58,13 @@ func TestEventJSON(t *testing.T) { // `includeempty` annotation, so nothing is emitted, despite the presence of // zero values. {&SchemaSnapshotMetadata{SnapshotID: "", NumRecords: 0}, ""}, + + // Primitive fields with an `includeempty` annotation will emit their zero + // value. + { + &StoreStats{Levels: []LevelStats{{Level: 0, NumFiles: 1}, {Level: 6, NumFiles: 2}}}, + `"Levels":[{"Level":0,"NumFiles":1},{"Level":6,"NumFiles":2}]`, + }, } for _, tc := range testCases { diff --git a/pkg/util/log/eventpb/eventlog_channels_generated.go b/pkg/util/log/eventpb/eventlog_channels_generated.go index 3e6ae8996aa8..d48ec4cf0430 100644 --- a/pkg/util/log/eventpb/eventlog_channels_generated.go +++ b/pkg/util/log/eventpb/eventlog_channels_generated.go @@ -286,6 +286,12 @@ func (m *GrantRole) LoggingChannel() logpb.Channel { return logpb.Channel_USER_A // LoggingChannel implements the EventPayload interface. func (m *PasswordHashConverted) LoggingChannel() logpb.Channel { return logpb.Channel_USER_ADMIN } +// LoggingChannel implements the EventPayload interface. +func (m *LevelStats) LoggingChannel() logpb.Channel { return logpb.Channel_TELEMETRY } + +// LoggingChannel implements the EventPayload interface. +func (m *StoreStats) LoggingChannel() logpb.Channel { return logpb.Channel_TELEMETRY } + // LoggingChannel implements the EventPayload interface. func (m *CapturedIndexUsageStats) LoggingChannel() logpb.Channel { return logpb.Channel_TELEMETRY } diff --git a/pkg/util/log/eventpb/eventpbgen/gen.go b/pkg/util/log/eventpb/eventpbgen/gen.go index 4898815cf311..a9640d2902a5 100644 --- a/pkg/util/log/eventpb/eventpbgen/gen.go +++ b/pkg/util/log/eventpb/eventpbgen/gen.go @@ -659,6 +659,18 @@ func (m *{{.GoType}}) AppendJSONFields(printComma bool, b redact.RedactableBytes {{ if not .AllowZeroValue -}} } {{- end }} + {{- else if eq .FieldType "array_of_LevelStats"}} + if len(m.{{.FieldName}}) > 0 { + if printComma { b = append(b, ',')}; printComma = true + b = append(b, "\"{{.FieldName}}\":["...) + for i, l := range m.{{.FieldName}} { + if i > 0 { b = append(b, ',') } + b = append(b, '{') + printComma, b = l.AppendJSONFields(false, b) + b = append(b, '}') + } + b = append(b, ']') + } {{- else if eq .FieldType "protobuf"}} if m.{{.FieldName}} != nil { if printComma { b = append(b, ',')}; printComma = true diff --git a/pkg/util/log/eventpb/json_encode_generated.go b/pkg/util/log/eventpb/json_encode_generated.go index e5ccee19e33d..6a07498a5d0b 100644 --- a/pkg/util/log/eventpb/json_encode_generated.go +++ b/pkg/util/log/eventpb/json_encode_generated.go @@ -2860,6 +2860,145 @@ func (m *LargeRowInternal) AppendJSONFields(printComma bool, b redact.Redactable return printComma, b } +// AppendJSONFields implements the EventPayload interface. +func (m *LevelStats) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { + + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"Level\":"...) + b = strconv.AppendUint(b, uint64(m.Level), 10) + + if m.NumFiles != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"NumFiles\":"...) + b = strconv.AppendInt(b, int64(m.NumFiles), 10) + } + + if m.SizeBytes != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"SizeBytes\":"...) + b = strconv.AppendInt(b, int64(m.SizeBytes), 10) + } + + if m.Score != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"Score\":"...) + b = strconv.AppendFloat(b, float64(m.Score), 'f', -1, 32) + } + + if m.BytesIn != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"BytesIn\":"...) + b = strconv.AppendUint(b, uint64(m.BytesIn), 10) + } + + if m.BytesIngested != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"BytesIngested\":"...) + b = strconv.AppendUint(b, uint64(m.BytesIngested), 10) + } + + if m.BytesMoved != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"BytesMoved\":"...) + b = strconv.AppendUint(b, uint64(m.BytesMoved), 10) + } + + if m.BytesRead != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"BytesRead\":"...) + b = strconv.AppendUint(b, uint64(m.BytesRead), 10) + } + + if m.BytesCompacted != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"BytesCompacted\":"...) + b = strconv.AppendUint(b, uint64(m.BytesCompacted), 10) + } + + if m.BytesFlushed != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"BytesFlushed\":"...) + b = strconv.AppendUint(b, uint64(m.BytesFlushed), 10) + } + + if m.TablesCompacted != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"TablesCompacted\":"...) + b = strconv.AppendUint(b, uint64(m.TablesCompacted), 10) + } + + if m.TablesFlushed != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"TablesFlushed\":"...) + b = strconv.AppendUint(b, uint64(m.TablesFlushed), 10) + } + + if m.TablesIngested != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"TablesIngested\":"...) + b = strconv.AppendUint(b, uint64(m.TablesIngested), 10) + } + + if m.TablesMoved != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"TablesMoved\":"...) + b = strconv.AppendUint(b, uint64(m.TablesMoved), 10) + } + + if m.NumSublevels != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"NumSublevels\":"...) + b = strconv.AppendInt(b, int64(m.NumSublevels), 10) + } + + return printComma, b +} + // AppendJSONFields implements the EventPayload interface. func (m *NodeDecommissioned) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { @@ -4280,6 +4419,310 @@ func (m *SlowQueryInternal) AppendJSONFields(printComma bool, b redact.Redactabl return printComma, b } +// AppendJSONFields implements the EventPayload interface. +func (m *StoreStats) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { + + printComma, b = m.CommonEventDetails.AppendJSONFields(printComma, b) + + if m.NodeId != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"NodeId\":"...) + b = strconv.AppendInt(b, int64(m.NodeId), 10) + } + + if m.StoreId != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"StoreId\":"...) + b = strconv.AppendInt(b, int64(m.StoreId), 10) + } + + if len(m.Levels) > 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"Levels\":["...) + for i, l := range m.Levels { + if i > 0 { + b = append(b, ',') + } + b = append(b, '{') + printComma, b = l.AppendJSONFields(false, b) + b = append(b, '}') + } + b = append(b, ']') + } + + if m.CacheSize != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"CacheSize\":"...) + b = strconv.AppendInt(b, int64(m.CacheSize), 10) + } + + if m.CacheCount != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"CacheCount\":"...) + b = strconv.AppendInt(b, int64(m.CacheCount), 10) + } + + if m.CacheHits != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"CacheHits\":"...) + b = strconv.AppendInt(b, int64(m.CacheHits), 10) + } + + if m.CacheMisses != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"CacheMisses\":"...) + b = strconv.AppendInt(b, int64(m.CacheMisses), 10) + } + + if m.CompactionCountDefault != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"CompactionCountDefault\":"...) + b = strconv.AppendInt(b, int64(m.CompactionCountDefault), 10) + } + + if m.CompactionCountDeleteOnly != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"CompactionCountDeleteOnly\":"...) + b = strconv.AppendInt(b, int64(m.CompactionCountDeleteOnly), 10) + } + + if m.CompactionCountElisionOnly != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"CompactionCountElisionOnly\":"...) + b = strconv.AppendInt(b, int64(m.CompactionCountElisionOnly), 10) + } + + if m.CompactionCountMove != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"CompactionCountMove\":"...) + b = strconv.AppendInt(b, int64(m.CompactionCountMove), 10) + } + + if m.CompactionCountRead != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"CompactionCountRead\":"...) + b = strconv.AppendInt(b, int64(m.CompactionCountRead), 10) + } + + if m.CompactionCountRewrite != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"CompactionCountRewrite\":"...) + b = strconv.AppendInt(b, int64(m.CompactionCountRewrite), 10) + } + + if m.CompactionNumInProgress != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"CompactionNumInProgress\":"...) + b = strconv.AppendInt(b, int64(m.CompactionNumInProgress), 10) + } + + if m.CompactionMarkedFiles != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"CompactionMarkedFiles\":"...) + b = strconv.AppendInt(b, int64(m.CompactionMarkedFiles), 10) + } + + if m.FlushCount != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"FlushCount\":"...) + b = strconv.AppendInt(b, int64(m.FlushCount), 10) + } + + if m.MemtableSize != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"MemtableSize\":"...) + b = strconv.AppendUint(b, uint64(m.MemtableSize), 10) + } + + if m.MemtableCount != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"MemtableCount\":"...) + b = strconv.AppendInt(b, int64(m.MemtableCount), 10) + } + + if m.MemtableZombieCount != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"MemtableZombieCount\":"...) + b = strconv.AppendInt(b, int64(m.MemtableZombieCount), 10) + } + + if m.MemtableZombieSize != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"MemtableZombieSize\":"...) + b = strconv.AppendUint(b, uint64(m.MemtableZombieSize), 10) + } + + if m.WalLiveCount != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"WalLiveCount\":"...) + b = strconv.AppendInt(b, int64(m.WalLiveCount), 10) + } + + if m.WalLiveSize != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"WalLiveSize\":"...) + b = strconv.AppendUint(b, uint64(m.WalLiveSize), 10) + } + + if m.WalObsoleteCount != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"WalObsoleteCount\":"...) + b = strconv.AppendInt(b, int64(m.WalObsoleteCount), 10) + } + + if m.WalObsoleteSize != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"WalObsoleteSize\":"...) + b = strconv.AppendUint(b, uint64(m.WalObsoleteSize), 10) + } + + if m.WalPhysicalSize != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"WalPhysicalSize\":"...) + b = strconv.AppendUint(b, uint64(m.WalPhysicalSize), 10) + } + + if m.WalBytesIn != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"WalBytesIn\":"...) + b = strconv.AppendUint(b, uint64(m.WalBytesIn), 10) + } + + if m.WalBytesWritten != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"WalBytesWritten\":"...) + b = strconv.AppendUint(b, uint64(m.WalBytesWritten), 10) + } + + if m.TableObsoleteCount != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"TableObsoleteCount\":"...) + b = strconv.AppendInt(b, int64(m.TableObsoleteCount), 10) + } + + if m.TableObsoleteSize != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"TableObsoleteSize\":"...) + b = strconv.AppendUint(b, uint64(m.TableObsoleteSize), 10) + } + + if m.TableZombieCount != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"TableZombieCount\":"...) + b = strconv.AppendInt(b, int64(m.TableZombieCount), 10) + } + + if m.TableZombieSize != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"TableZombieSize\":"...) + b = strconv.AppendUint(b, uint64(m.TableZombieSize), 10) + } + + if m.RangeKeySetsCount != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"RangeKeySetsCount\":"...) + b = strconv.AppendUint(b, uint64(m.RangeKeySetsCount), 10) + } + + return printComma, b +} + // AppendJSONFields implements the EventPayload interface. func (m *TruncateTable) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { diff --git a/pkg/util/log/eventpb/storage_events.proto b/pkg/util/log/eventpb/storage_events.proto new file mode 100644 index 000000000000..1ba77d6e99b6 --- /dev/null +++ b/pkg/util/log/eventpb/storage_events.proto @@ -0,0 +1,167 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +syntax = "proto3"; +package cockroach.util.log.eventpb; +option go_package = "eventpb"; + +import "gogoproto/gogo.proto"; +import "util/log/logpb/event.proto"; + +// Category: Storage telemetry events +// Channel: TELEMETRY + +// StoreStats 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. +message StoreStats { + CommonEventDetails common = 1 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + + // node_id is the ID of the node. + int32 node_id = 2 [(gogoproto.jsontag) = ",omitempty"]; + // store_id is the ID of the store. + int32 store_id = 3 [(gogoproto.jsontag) = ",omitempty"]; + + // levels is a nested message containing per-level statistics. + repeated LevelStats levels = 4 [(gogoproto.nullable) = false, (gogoproto.jsontag) = ""]; + + // Cache metrics. + + // cache_size is the size of the cache for the store, in bytes (gauge). + int64 cache_size = 5 [(gogoproto.jsontag) = ",omitempty"]; + // cache_count is the number of items in the cache (gauge). + int64 cache_count = 6 [(gogoproto.jsontag) = ",omitempty"]; + // cache_hits is the number of cache hits (counter). + int64 cache_hits = 7 [(gogoproto.jsontag) = ",omitempty"]; + // cache_misses is the number of cache misses (counter). + int64 cache_misses = 8 [(gogoproto.jsontag) = ",omitempty"]; + + // Compaction stats. + + // compaction_count_default is the count of default compactions (counter). + int64 compaction_count_default = 9 [(gogoproto.jsontag) = ",omitempty"]; + // compaction_count_delete_only is the count of delete-only compactions + // (counter). + int64 compaction_count_delete_only = 10 [(gogoproto.jsontag) = ",omitempty"]; + // compaction_count_elision_only is the count of elision-only compactions + // (counter). + int64 compaction_count_elision_only = 11 [(gogoproto.jsontag) = ",omitempty"]; + // compaction_count_move is the count of move-compactions (counter). + int64 compaction_count_move = 12 [(gogoproto.jsontag) = ",omitempty"]; + // compaction_count_read is the count of read-compactions (counter). + int64 compaction_count_read = 13 [(gogoproto.jsontag) = ",omitempty"]; + // compaction_count_rewrite is the count of rewrite-compactions (counter). + int64 compaction_count_rewrite = 14 [(gogoproto.jsontag) = ",omitempty"]; + // compactions_num_in_progress is the number of compactions in progress + // (gauge). + int64 compaction_num_in_progress = 15 [(gogoproto.jsontag) = ",omitempty"]; + // compaction_marked_files is the count of files marked for compaction + // (gauge). + int64 compaction_marked_files = 16 [(gogoproto.jsontag) = ",omitempty"]; + + // Flush stats. + + // flush_count is the number of flushes (counter). + int64 flush_count = 17 [(gogoproto.jsontag) = ",omitempty"]; + + // Memtable stats. + + // memtable_size is the total size allocated to all memtables and (large) + // batches, in bytes (gauge). + uint64 memtable_size = 18 [(gogoproto.jsontag) = ",omitempty"]; + // memtable_count is the count of memtables (gauge). + int64 memtable_count = 19 [(gogoproto.jsontag) = ",omitempty"]; + // memtable_zombie_count is the count of memtables no longer referenced by the + // current DB state, but still in use by an iterator (gauge). + int64 memtable_zombie_count = 20 [(gogoproto.jsontag) = ",omitempty"]; + // memtable_zombie_size is the size, in bytes, of all zombie memtables + // (gauge). + uint64 memtable_zombie_size = 21 [(gogoproto.jsontag) = ",omitempty"]; + + // WAL stats. + + // wal_live_count is the count of live WAL files (gauge). + int64 wal_live_count = 22 [(gogoproto.jsontag) = ",omitempty"]; + // 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). + uint64 wal_live_size = 23 [(gogoproto.jsontag) = ",omitempty"]; + // wal_obsolete_count is the count of obsolete WAL files (gauge). + int64 wal_obsolete_count = 24 [(gogoproto.jsontag) = ",omitempty"]; + // wal_obsolete_size is the size of obsolete WAL files, in bytes (gauge). + uint64 wal_obsolete_size = 25 [(gogoproto.jsontag) = ",omitempty"]; + // wal_physical_size is the size, in bytes, of the WAL files on disk (gauge). + uint64 wal_physical_size = 26 [(gogoproto.jsontag) = ",omitempty"]; + // wal_bytes_in is the number of logical bytes written to the WAL (counter). + uint64 wal_bytes_in = 27 [(gogoproto.jsontag) = ",omitempty"]; + // wal_bytes_written is the number of bytes written to the WAL (counter). + uint64 wal_bytes_written = 28 [(gogoproto.jsontag) = ",omitempty"]; + + // Table stats. + + // table_obsolete_count is the number of tables which are no longer referenced + // by the current DB state or any open iterators (gauge). + int64 table_obsolete_count = 29 [(gogoproto.jsontag) = ",omitempty"]; + // table_obsolete_size is the size, in bytes, of obsolete tables (gauge). + uint64 table_obsolete_size = 30 [(gogoproto.jsontag) = ",omitempty"]; + // 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). + int64 table_zombie_count = 31 [(gogoproto.jsontag) = ",omitempty"]; + // table_zombie_size is the size, in bytes, of zombie tables (gauge). + uint64 table_zombie_size = 32 [(gogoproto.jsontag) = ",omitempty"]; + + // Keys. + + // range_key_sets_count is the approximate count of internal range key sets in + // the store. + uint64 range_key_sets_count = 33 [(gogoproto.jsontag) = ",omitempty"]; +} + +// LevelStats contains per-level statistics for an LSM. +message LevelStats { + // level is the level ID in a LSM (e.g. level(L0) == 0, etc.) + uint32 level = 1 [(gogoproto.jsontag) = ",includeempty"]; + // num_files is the number of files in the level (gauge). + int64 num_files = 2 [(gogoproto.jsontag) = ",omitempty"]; + // size_bytes is the size of the level, in bytes (gauge). + int64 size_bytes = 3 [(gogoproto.jsontag) = ",omitempty"]; + // score is the compaction score of the level (gauge). + float score = 4 [(gogoproto.jsontag) = ",omitempty"]; + // bytes_in is the number of bytes written to this level (counter). + uint64 bytes_in = 5 [(gogoproto.jsontag) = ",omitempty"]; + // bytes_ingested is the number of bytes ingested into this level (counter). + uint64 bytes_ingested = 6 [(gogoproto.jsontag) = ",omitempty"]; + // bytes_moved is the number of bytes moved into this level via a + // move-compaction (counter). + uint64 bytes_moved = 7 [(gogoproto.jsontag) = ",omitempty"]; + // bytes_read is the number of bytes read from this level, during compactions + // (counter). + uint64 bytes_read = 8 [(gogoproto.jsontag) = ",omitempty"]; + // bytes_compacted is the number of bytes written to this level during + // compactions (counter). + uint64 bytes_compacted = 9 [(gogoproto.jsontag) = ",omitempty"]; + // bytes flushed is the number of bytes flushed to this level. This value is + // always zero for levels other than L0 (counter). + uint64 bytes_flushed = 10 [(gogoproto.jsontag) = ",omitempty"]; + // tables_compacted is the count of tables compacted into this level + // (counter). + uint64 tables_compacted = 11 [(gogoproto.jsontag) = ",omitempty"]; + // tables_flushed is the count of tables flushed into this level (counter). + uint64 tables_flushed = 12 [(gogoproto.jsontag) = ",omitempty"]; + // tables_ingested is the count of tables ingested into this level (counter). + uint64 tables_ingested = 13 [(gogoproto.jsontag) = ",omitempty"]; + // tables_moved is the count of tables moved into this level via + // move-compactions (counter). + uint64 tables_moved = 14 [(gogoproto.jsontag) = ",omitempty"]; + // num_sublevel is the count of sublevels for the level. This value is always + // zero for levels other than L0 (gauge). + int32 num_sublevels = 15 [(gogoproto.jsontag) = ",omitempty"]; +} From 11cdc9a2daf92c5c6437357aa21f745b21a0224a Mon Sep 17 00:00:00 2001 From: Pavel Kalinnikov Date: Wed, 21 Sep 2022 12:10:56 +0100 Subject: [PATCH 2/7] kvserver: align Raft recv/send queue sizes 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. --- pkg/base/config.go | 4 ++++ pkg/kv/kvserver/store.go | 8 +++++--- pkg/kv/kvserver/store_raft.go | 15 +++++++++------ pkg/kv/kvserver/store_raft_test.go | 6 +++--- 4 files changed, 21 insertions(+), 12 deletions(-) diff --git a/pkg/base/config.go b/pkg/base/config.go index abb4af017aed..e7a8a6b9a238 100644 --- a/pkg/base/config.go +++ b/pkg/base/config.go @@ -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 diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index bba930e7aa83..ca9905acf4ce 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -98,9 +98,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 diff --git a/pkg/kv/kvserver/store_raft.go b/pkg/kv/kvserver/store_raft.go index 10796a5eca12..257ab1f8acce 100644 --- a/pkg/kv/kvserver/store_raft.go +++ b/pkg/kv/kvserver/store_raft.go @@ -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. @@ -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 { @@ -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 @@ -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 diff --git a/pkg/kv/kvserver/store_raft_test.go b/pkg/kv/kvserver/store_raft_test.go index 84a150d05301..9b84ac77d44a 100644 --- a/pkg/kv/kvserver/store_raft_test.go +++ b/pkg/kv/kvserver/store_raft_test.go @@ -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) } @@ -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()) From 8c8bc33281064d4d6cbb28414757afdce3606931 Mon Sep 17 00:00:00 2001 From: Faizan Qazi Date: Thu, 25 Aug 2022 21:19:31 +0000 Subject: [PATCH 3/7] workload/mixed-version/schemachanger: ensure libGEOS libraries are deployed Previously, the schemachanger mixed version workload did not deploy the libGEOS libraries causing operations to fail. To address, this patch will update the frontend to upload these binaries. Release justification: no real risk improves test coverage Release note: None merge back a: --- pkg/cmd/roachtest/tests/mixed_version_schemachange.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pkg/cmd/roachtest/tests/mixed_version_schemachange.go b/pkg/cmd/roachtest/tests/mixed_version_schemachange.go index d67433fae2c2..65f37362aef8 100644 --- a/pkg/cmd/roachtest/tests/mixed_version_schemachange.go +++ b/pkg/cmd/roachtest/tests/mixed_version_schemachange.go @@ -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 From f860c460a5fdada184be35bb64ee38241263858e Mon Sep 17 00:00:00 2001 From: Faizan Qazi Date: Fri, 26 Aug 2022 03:54:43 +0000 Subject: [PATCH 4/7] workload/schemachanger: disable trigram operations in a mixed version state Previously, the schema changer workload in a mixed version state attempted to use trigram indexes against 22.1, which is unsupported. This patch adds code to detect a mixed version state and expects the appropriate error when this occurs. Release justification: no risk improves test coverage Release note: None --- .../schemachange/operation_generator.go | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/pkg/workload/schemachange/operation_generator.go b/pkg/workload/schemachange/operation_generator.go index f4ca24d89ad6..692f5a905580 100644 --- a/pkg/workload/schemachange/operation_generator.go +++ b/pkg/workload/schemachange/operation_generator.go @@ -1183,6 +1183,29 @@ func (og *operationGenerator) createTable(ctx context.Context, tx pgx.Tx) (*opSt stmt := randgen.RandCreateTableWithColumnIndexNumberGenerator(og.params.rng, "table", tableIdx, databaseHasMultiRegion, og.newUniqueSeqNum) stmt.Table = *tableName stmt.IfNotExists = og.randIntn(2) == 0 + trigramIsNotSupported, err := isClusterVersionLessThan( + ctx, + tx, + clusterversion.ByKey(clusterversion.TrigramInvertedIndexes)) + if err != nil { + return nil, err + } + hasTrigramIdxUnsupported := func() bool { + if !trigramIsNotSupported { + return false + } + // Check if any of the indexes have trigrams involved. + for _, def := range stmt.Defs { + if idx, ok := def.(*tree.IndexTableDef); ok && idx.Inverted { + lastColumn := idx.Columns[len(idx.Columns)-1] + switch lastColumn.OpClass { + case "gin_trgm_ops", "gist_trgm_ops": + return true + } + } + } + return false + }() tableExists, err := og.tableExists(ctx, tx, tableName) if err != nil { @@ -1197,6 +1220,11 @@ func (og *operationGenerator) createTable(ctx context.Context, tx pgx.Tx) (*opSt {code: pgcode.DuplicateRelation, condition: tableExists && !stmt.IfNotExists}, {code: pgcode.UndefinedSchema, condition: !schemaExists}, }.add(opStmt.expectedExecErrors) + // Compatibility errors aren't guaranteed since the cluster version update is not + // fully transaction aware. + codesWithConditions{ + {code: pgcode.FeatureNotSupported, condition: hasTrigramIdxUnsupported}, + }.add(opStmt.potentialExecErrors) opStmt.sql = tree.Serialize(stmt) return opStmt, nil } From dd954ff1dcfe5918245f6e52a4b2de1a49df73e7 Mon Sep 17 00:00:00 2001 From: Faizan Qazi Date: Fri, 26 Aug 2022 18:00:38 +0000 Subject: [PATCH 5/7] workload/schemachange: disable inserts when in 22.1 mixed version state Previously, if we ran in a mixed version state with the schema changer workload we could run into an optimizer bug (#80820). To address this, this patch in a mixed version workload disables the insert portion of the workload. Release justification: improves test coverage by enabling the mixed version test Release note: None --- .../roachtest/tests/mixed_version_schemachange.go | 6 ------ pkg/workload/schemachange/operation_generator.go | 12 ++++++++++++ 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/pkg/cmd/roachtest/tests/mixed_version_schemachange.go b/pkg/cmd/roachtest/tests/mixed_version_schemachange.go index 65f37362aef8..63ba5cb8fae7 100644 --- a/pkg/cmd/roachtest/tests/mixed_version_schemachange.go +++ b/pkg/cmd/roachtest/tests/mixed_version_schemachange.go @@ -57,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), diff --git a/pkg/workload/schemachange/operation_generator.go b/pkg/workload/schemachange/operation_generator.go index 692f5a905580..5a94ccecc7c7 100644 --- a/pkg/workload/schemachange/operation_generator.go +++ b/pkg/workload/schemachange/operation_generator.go @@ -2373,6 +2373,18 @@ func (og *operationGenerator) insertRow(ctx context.Context, tx pgx.Tx) (stmt *o if err != nil { return nil, err } + // If we aren't on 22.2 then disable the insert plugin, since 21.X + // can have schema instrospection queries fail due to an optimizer bug. + skipInserts, err := isClusterVersionLessThan(ctx, tx, clusterversion.ByKey(clusterversion.Start22_2)) + if err != nil { + return nil, err + } + // If inserts are to be skipped, we will intentionally, target the insert towards + // a non-existent table, so that they become no-ops. + if skipInserts { + tableExists = false + tableName.SchemaName = "InvalidObjectName" + } if !tableExists { return makeOpStmtForSingleError(OpStmtDML, fmt.Sprintf( From 6deea40dece097d1288f67d0bbef1e9d08a3daac Mon Sep 17 00:00:00 2001 From: Xin Hao Zhang Date: Mon, 19 Sep 2022 17:26:53 -0400 Subject: [PATCH 6/7] ui/cluster-ui: fix no most recent stmt for active txns 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. --- .../activeStatementUtils.spec.ts | 304 ++++++++++-------- .../activeExecutions/activeStatementUtils.ts | 32 +- .../activeTransactionsTable.tsx | 21 +- .../cluster-ui/src/activeExecutions/types.ts | 6 +- .../activeTransactionDetails.tsx | 5 - 5 files changed, 202 insertions(+), 166 deletions(-) diff --git a/pkg/ui/workspaces/cluster-ui/src/activeExecutions/activeStatementUtils.spec.ts b/pkg/ui/workspaces/cluster-ui/src/activeExecutions/activeStatementUtils.spec.ts index e9009f039488..5398091083db 100644 --- a/pkg/ui/workspaces/cluster-ui/src/activeExecutions/activeStatementUtils.spec.ts +++ b/pkg/ui/workspaces/cluster-ui/src/activeExecutions/activeStatementUtils.spec.ts @@ -85,7 +85,6 @@ function makeActiveTxn( statementID: defaultActiveStatement.statementID, retries: 3, lastAutoRetryReason: null, - isFullScan: defaultActiveStatement.isFullScan, priority: "Normal", statementCount: 5, status: "Executing", @@ -184,67 +183,186 @@ describe("test activeStatementUtils", () => { }); describe("getActiveExecutionsFromSessions", () => { - const activeQueries = [1, 2, 3, 4].map(num => - makeActiveQuery({ id: num.toString() }), - ); + it("should convert sessions response to active statements result", () => { + const sessionsResponse: SessionsResponse = { + sessions: [ + { + id: new Uint8Array(), + username: "bar", + application_name: "application", + client_address: "clientAddress", + active_queries: [makeActiveQuery({ id: "1" })], + }, + { + id: new Uint8Array(), + username: "foo", + application_name: "application2", + client_address: "clientAddress2", + active_queries: [makeActiveQuery({ id: "2" })], + }, + { + id: new Uint8Array(), + username: "closed", + status: SessionStatusType.CLOSED, + // Closed sessions should not appear in active stmts. + application_name: "application2", + client_address: "clientAddress2", + active_queries: [makeActiveQuery({ id: "3" })], + }, + ], + errors: [], + internal_app_name_prefix: INTERNAL_APP_NAME_PREFIX, + toJSON: () => ({}), + }; + + const statements = getActiveExecutionsFromSessions( + sessionsResponse, + LAST_UPDATED, + ).statements; + + expect(statements.length).toBe(2); + + statements.forEach(stmt => { + if (stmt.user === "bar") { + expect(stmt.application).toBe("application"); + expect(stmt.clientAddress).toBe("clientAddress"); + } else if (stmt.user === "foo") { + expect(stmt.application).toBe("application2"); + expect(stmt.clientAddress).toBe("clientAddress2"); + } else { + fail(`stmt user should be foo or bar, got ${stmt.user}`); + } + // expect(stmt.transactionID).toBe(defaultActiveStatement.transactionID); + expect(stmt.status).toBe("Executing"); + expect(stmt.elapsedTimeMillis).toBe( + LAST_UPDATED.diff(MOCK_START_TIME, "ms"), + ); + expect(stmt.start.unix()).toBe( + TimestampToMoment(defaultActiveQuery.start).unix(), + ); + expect(stmt.query).toBe(defaultActiveStatement.query); + }); + }); - const sessionsResponse: SessionsResponse = { - sessions: [ - { - id: new Uint8Array(), - username: "bar", - application_name: "application", - client_address: "clientAddress", - active_queries: activeQueries, - }, + it("should convert sessions response to active transactions result", () => { + const txns = [ { id: new Uint8Array(), - username: "foo", - application_name: "application2", - client_address: "clientAddress2", - active_queries: activeQueries, + start: new Timestamp({ + seconds: Long.fromNumber(MOCK_START_TIME.unix()), + }), + num_auto_retries: 3, + num_statements_executed: 4, }, { id: new Uint8Array(), - username: "foo", - status: SessionStatusType.CLOSED, - application_name: "application2", - client_address: "clientAddress2", - active_queries: activeQueries, + start: new Timestamp({ + seconds: Long.fromNumber(MOCK_START_TIME.unix()), + }), + num_auto_retries: 4, + num_statements_executed: 3, }, - ], - errors: [], - internal_app_name_prefix: INTERNAL_APP_NAME_PREFIX, - toJSON: () => ({}), - }; - - const statements = getActiveExecutionsFromSessions( - sessionsResponse, - LAST_UPDATED, - ).statements; - - expect(statements.length).toBe(activeQueries.length * 2); - - statements.forEach(stmt => { - if (stmt.user === "bar") { - expect(stmt.application).toBe("application"); - expect(stmt.clientAddress).toBe("clientAddress"); - } else if (stmt.user === "foo") { - expect(stmt.application).toBe("application2"); - expect(stmt.clientAddress).toBe("clientAddress2"); - } else { - fail(`stmt user should be foo or bar, got ${stmt.user}`); - } - // expect(stmt.transactionID).toBe(defaultActiveStatement.transactionID); - expect(stmt.status).toBe("Executing"); - expect(stmt.elapsedTimeMillis).toBe( - LAST_UPDATED.diff(MOCK_START_TIME, "ms"), - ); - expect(stmt.start.unix()).toBe( - TimestampToMoment(defaultActiveQuery.start).unix(), + ]; + + const sessionsResponse: SessionsResponse = { + sessions: [ + { + id: new Uint8Array(), + username: "bar", + application_name: "application", + client_address: "clientAddress", + active_queries: [makeActiveQuery()], + active_txn: txns[0], + }, + { + id: new Uint8Array(), + username: "foo", + application_name: "application2", + client_address: "clientAddress2", + active_queries: [makeActiveQuery()], + active_txn: txns[1], + }, + { + id: new Uint8Array(), + username: "foo", + status: SessionStatusType.CLOSED, + application_name: "closed_application", + client_address: "clientAddress2", + active_queries: [makeActiveQuery()], + active_txn: txns[1], + }, + ], + errors: [], + internal_app_name_prefix: INTERNAL_APP_NAME_PREFIX, + toJSON: () => ({}), + }; + + const activeTransactions = getActiveExecutionsFromSessions( + sessionsResponse, + LAST_UPDATED, + ).transactions; + + // Should filter out the txn from closed session. + expect(activeTransactions.length).toBe(2); + + activeTransactions.forEach((txn: ActiveTransaction, i) => { + expect(txn.application).toBe( + sessionsResponse.sessions[i].application_name, + ); + expect(txn.elapsedTimeMillis).toBe( + LAST_UPDATED.diff(MOCK_START_TIME, "ms"), + ); + expect(txn.status).toBe("Executing"); + expect(txn.query).toBeTruthy(); + expect(txn.start.unix()).toBe( + TimestampToMoment(defaultActiveQuery.start).unix(), + ); + }); + }); + + it("should populate txn latest query when there is no active stmt for txns with at least 1 stmt", () => { + const lastActiveQueryText = "SELECT 1"; + const sessionsResponse: SessionsResponse = { + sessions: [ + { + id: new Uint8Array(), + last_active_query: lastActiveQueryText, + active_queries: [], + active_txn: { + id: new Uint8Array(), + start: new Timestamp({ + seconds: Long.fromNumber(MOCK_START_TIME.unix()), + }), + num_auto_retries: 0, + num_statements_executed: 1, + }, + }, + { + id: new Uint8Array(), + last_active_query: lastActiveQueryText, + active_queries: [], + active_txn: { + id: new Uint8Array(), + start: new Timestamp({ + seconds: Long.fromNumber(MOCK_START_TIME.unix()), + }), + num_auto_retries: 0, + num_statements_executed: 0, + }, + }, + ], + errors: [], + internal_app_name_prefix: INTERNAL_APP_NAME_PREFIX, + toJSON: () => ({}), + }; + + const activeExecs = getActiveExecutionsFromSessions( + sessionsResponse, + LAST_UPDATED, ); - // expect(stmt.sessionID).toBe(defaultActiveStatement.sessionID); - expect(stmt.query).toBe(defaultActiveStatement.query); + + expect(activeExecs.transactions[0].query).toBe(lastActiveQueryText); + expect(activeExecs.transactions[1].query).toBeFalsy(); }); }); @@ -262,84 +380,6 @@ describe("test activeStatementUtils", () => { expect(apps).toEqual(["app1", "app2", "app3", "app4"]); }); - describe("getActiveExecutionsFromSessions transactions result", () => { - const txns = [ - { - id: new Uint8Array(), - start: new Timestamp({ - seconds: Long.fromNumber(MOCK_START_TIME.unix()), - }), - num_auto_retries: 3, - num_statements_executed: 4, - }, - { - id: new Uint8Array(), - start: new Timestamp({ - seconds: Long.fromNumber(MOCK_START_TIME.unix()), - }), - num_auto_retries: 4, - num_statements_executed: 3, - }, - ]; - - const sessionsResponse: SessionsResponse = { - sessions: [ - { - id: new Uint8Array(), - username: "bar", - application_name: "application", - client_address: "clientAddress", - active_queries: [makeActiveQuery()], - active_txn: txns[0], - }, - { - id: new Uint8Array(), - username: "foo", - application_name: "application2", - client_address: "clientAddress2", - active_queries: [makeActiveQuery()], - active_txn: txns[1], - }, - { - id: new Uint8Array(), - username: "foo", - status: SessionStatusType.CLOSED, - application_name: "closed_application", - client_address: "clientAddress2", - active_queries: [makeActiveQuery()], - active_txn: txns[1], - }, - ], - errors: [], - internal_app_name_prefix: INTERNAL_APP_NAME_PREFIX, - toJSON: () => ({}), - }; - - const activeTransactions = getActiveExecutionsFromSessions( - sessionsResponse, - LAST_UPDATED, - ).transactions; - - // Should filter out the txn from closed session. - expect(activeTransactions.length).toBe(2); - - expect(activeTransactions.length).toBe(txns.length); - - activeTransactions.forEach((txn: ActiveTransaction, i) => { - expect(txn.application).toBe( - sessionsResponse.sessions[i].application_name, - ); - expect(txn.elapsedTimeMillis).toBe( - LAST_UPDATED.diff(MOCK_START_TIME, "ms"), - ); - expect(txn.status).toBe("Executing"); - expect(txn.query).toBeTruthy(); - expect(txn.start.unix()).toBe( - TimestampToMoment(defaultActiveQuery.start).unix(), - ); - }); - }); - describe("filterActiveTransactions", () => { it("should filter out txns that do not match filters", () => { const txns: ActiveTransaction[] = [ diff --git a/pkg/ui/workspaces/cluster-ui/src/activeExecutions/activeStatementUtils.ts b/pkg/ui/workspaces/cluster-ui/src/activeExecutions/activeStatementUtils.ts index ef8f276d80bb..fe00e78fb538 100644 --- a/pkg/ui/workspaces/cluster-ui/src/activeExecutions/activeStatementUtils.ts +++ b/pkg/ui/workspaces/cluster-ui/src/activeExecutions/activeStatementUtils.ts @@ -89,7 +89,6 @@ export function getActiveExecutionsFromSessions( return { statements: [], transactions: [] }; const time = lastUpdated || moment.utc(); - const activeStmtByTxnID: Record = {}; const statements: ActiveStatement[] = []; const transactions: ActiveTransaction[] = []; @@ -102,9 +101,12 @@ export function getActiveExecutionsFromSessions( .forEach(session => { const sessionID = byteArrayToUuid(session.id); - session.active_queries.forEach(query => { + let activeStmt: ActiveStatement = null; + if (session.active_queries.length) { + // There will only ever be one query in this array. + const query = session.active_queries[0]; const queryTxnID = byteArrayToUuid(query.txn_id); - const stmt: ActiveStatement = { + activeStmt = { statementID: query.id, transactionID: queryTxnID, sessionID, @@ -122,9 +124,8 @@ export function getActiveExecutionsFromSessions( isFullScan: query.is_full_scan || false, // Or here is for conversion in case the field is null. }; - statements.push(stmt); - activeStmtByTxnID[queryTxnID] = stmt; - }); + statements.push(activeStmt); + } const activeTxn = session.active_txn; if (!activeTxn) return; @@ -132,30 +133,23 @@ export function getActiveExecutionsFromSessions( transactions.push({ transactionID: byteArrayToUuid(activeTxn.id), sessionID, - query: null, - statementID: null, + query: + activeStmt?.query ?? + (activeTxn.num_statements_executed + ? session.last_active_query + : null), + statementID: activeStmt?.statementID, status: "Executing" as ExecutionStatus, start: TimestampToMoment(activeTxn.start), elapsedTimeMillis: time.diff(TimestampToMoment(activeTxn.start), "ms"), application: session.application_name, retries: activeTxn.num_auto_retries, statementCount: activeTxn.num_statements_executed, - isFullScan: session.active_queries.some(query => query.is_full_scan), lastAutoRetryReason: activeTxn.last_auto_retry_reason, priority: activeTxn.priority, }); }); - // Find most recent statement for each txn. - transactions.map(txn => { - const mostRecentStmt = activeStmtByTxnID[txn.transactionID]; - if (!mostRecentStmt) return txn; - txn.query = mostRecentStmt.query; - txn.statementID = mostRecentStmt.statementID; - txn.isFullScan = mostRecentStmt.isFullScan; - return txn; - }); - return { transactions, statements, diff --git a/pkg/ui/workspaces/cluster-ui/src/activeExecutions/activeTransactionsTable/activeTransactionsTable.tsx b/pkg/ui/workspaces/cluster-ui/src/activeExecutions/activeTransactionsTable/activeTransactionsTable.tsx index 419a86deaee2..a55590f6556c 100644 --- a/pkg/ui/workspaces/cluster-ui/src/activeExecutions/activeTransactionsTable/activeTransactionsTable.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/activeExecutions/activeTransactionsTable/activeTransactionsTable.tsx @@ -31,13 +31,20 @@ export function makeActiveTransactionsColumns( { name: "mostRecentStatement", title: executionsTableTitles.mostRecentStatement(execType), - cell: (item: ActiveTransaction) => ( - - - {limitText(item.query || "", 70)} - - - ), + cell: (item: ActiveTransaction) => { + const queryText = limitText(item.query || "", 70); + return ( + + {item.statementID ? ( + + {queryText} + + ) : ( + queryText + )} + + ); + }, sort: (item: ActiveTransaction) => item.query, }, activeTransactionColumnsFromCommon.status, diff --git a/pkg/ui/workspaces/cluster-ui/src/activeExecutions/types.ts b/pkg/ui/workspaces/cluster-ui/src/activeExecutions/types.ts index a575d6548081..d5045a677346 100644 --- a/pkg/ui/workspaces/cluster-ui/src/activeExecutions/types.ts +++ b/pkg/ui/workspaces/cluster-ui/src/activeExecutions/types.ts @@ -25,22 +25,22 @@ export const SessionStatusType = protos.cockroach.server.serverpb.Session.Status; export interface ActiveExecution { - statementID?: string; // This may not be present for a transaction. + statementID?: string; // Empty for transactions not currently executing a statement. transactionID: string; sessionID: string; status: ExecutionStatus; start: Moment; elapsedTimeMillis: number; application: string; - query?: string; // Possibly empty for a transaction. + query?: string; // For transactions, this is the latest query executed. timeSpentWaiting?: moment.Duration; - isFullScan: boolean; } export type ActiveStatement = ActiveExecution & Required> & { user: string; clientAddress: string; + isFullScan: boolean; }; export type ActiveTransaction = ActiveExecution & { diff --git a/pkg/ui/workspaces/cluster-ui/src/transactionDetails/activeTransactionDetails.tsx b/pkg/ui/workspaces/cluster-ui/src/transactionDetails/activeTransactionDetails.tsx index acb445a5805b..20b4e196eb0e 100644 --- a/pkg/ui/workspaces/cluster-ui/src/transactionDetails/activeTransactionDetails.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/transactionDetails/activeTransactionDetails.tsx @@ -60,7 +60,6 @@ export const ActiveTxnInsightsLabels = { LAST_STATEMENT_EXEC_ID: "Most Recent Statement Execution ID", SESSION_ID: "Session ID", PRIORITY: "Priority", - FULL_SCAN: "Full Scan", }; export const RECENT_STATEMENT_NOT_FOUND_MESSAGE = @@ -143,10 +142,6 @@ export const ActiveTransactionDetails: React.FC< label={ActiveTxnInsightsLabels.PRIORITY} value={capitalize(transaction.priority)} /> - From e397faf3918d5757fd22b21a6a0ee76e72074ea0 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Mon, 12 Sep 2022 15:37:09 -0400 Subject: [PATCH 7/7] schedulerlatency: export Go scheduling latency metric MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 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 --- pkg/server/server.go | 6 +- pkg/server/status/recorder.go | 2 +- pkg/ts/catalog/chart_catalog.go | 6 + .../admission/scheduler_latency_listener.go | 4 - pkg/util/metric/metric.go | 152 ++-- pkg/util/schedulerlatency/BUILD.bazel | 12 +- pkg/util/schedulerlatency/histogram.go | 263 ++++++ pkg/util/schedulerlatency/histogram_test.go | 227 +++++ pkg/util/schedulerlatency/sampler.go | 123 ++- .../scheduler_latency_test.go | 25 +- .../testdata/histogram_buckets | 853 ++++++++++++++++++ .../testdata/runtime_histogram | 100 ++ 12 files changed, 1686 insertions(+), 87 deletions(-) create mode 100644 pkg/util/schedulerlatency/histogram.go create mode 100644 pkg/util/schedulerlatency/histogram_test.go create mode 100644 pkg/util/schedulerlatency/testdata/histogram_buckets create mode 100644 pkg/util/schedulerlatency/testdata/runtime_histogram diff --git a/pkg/server/server.go b/pkg/server/server.go index a25b70ab119f..c763582314fc 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -1399,7 +1399,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 } @@ -1465,7 +1467,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, diff --git a/pkg/server/status/recorder.go b/pkg/server/status/recorder.go index 2aee2ee6001e..dd9bee86b9e8 100644 --- a/pkg/server/status/recorder.go +++ b/pkg/server/status/recorder.go @@ -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 diff --git a/pkg/ts/catalog/chart_catalog.go b/pkg/ts/catalog/chart_catalog.go index 3db5d36cd397..f5cbfb180260 100644 --- a/pkg/ts/catalog/chart_catalog.go +++ b/pkg/ts/catalog/chart_catalog.go @@ -3515,6 +3515,12 @@ var charts = []sectionDescription{ "admission.scheduler_latency_listener.p99_nanos", }, }, + { + Title: "Scheduler Latency", + Metrics: []string{ + "go.scheduler_latency", + }, + }, { Title: "Elastic CPU Durations", Metrics: []string{ diff --git a/pkg/util/admission/scheduler_latency_listener.go b/pkg/util/admission/scheduler_latency_listener.go index 93da60e61721..a2e4ac1cfb14 100644 --- a/pkg/util/admission/scheduler_latency_listener.go +++ b/pkg/util/admission/scheduler_latency_listener.go @@ -25,10 +25,6 @@ import ( var _ metric.Struct = &schedulerLatencyListenerMetrics{} -// TODO(irfansharif): There’s some discrepancy between what this struct observes -// as p99 scheduling latencies and what prometheus/client_golang computes. Worth -// investigating. - type schedulerLatencyListener struct { ctx context.Context elasticCPULimiter elasticCPULimiter diff --git a/pkg/util/metric/metric.go b/pkg/util/metric/metric.go index da77e41b4844..1341faad4489 100644 --- a/pkg/util/metric/metric.go +++ b/pkg/util/metric/metric.go @@ -22,7 +22,7 @@ import ( "github.com/gogo/protobuf/proto" "github.com/prometheus/client_golang/prometheus" prometheusgo "github.com/prometheus/client_model/go" - metrics "github.com/rcrowley/go-metrics" + "github.com/rcrowley/go-metrics" ) const ( @@ -68,8 +68,8 @@ type PrometheusExportable interface { } // PrometheusIterable is an extension of PrometheusExportable to indicate that -// this metric is comprised of children metrics which augment the parent's -// label values. +// this metric comprises children metrics which augment the parent's label +// values. // // The motivating use-case for this interface is the existence of tenants. We'd // like to capture per-tenant metrics and expose them to prometheus while not @@ -82,6 +82,22 @@ type PrometheusIterable interface { Each([]*prometheusgo.LabelPair, func(metric *prometheusgo.Metric)) } +// WindowedHistogram represents a histogram with data over recent window of +// time. It's used primarily to record histogram data into CRDB's internal +// time-series database, which does not know how to encode cumulative +// histograms. What it does instead is scrape off sample count, sum of values, +// and values at specific quantiles from "windowed" histograms and record that +// data directly. These windows could be arbitrary and overlapping. +type WindowedHistogram interface { + // TotalCountWindowed returns the number of samples in the current window. + TotalCountWindowed() int64 + // TotalSumWindowed returns the number of samples in the current window. + TotalSumWindowed() float64 + // ValueAtQuantileWindowed takes a quantile value [0,100] and returns the + // interpolated value at that quantile for the windowed histogram. + ValueAtQuantileWindowed(q float64) float64 +} + // GetName returns the metric's name. func (m *Metadata) GetName() string { return m.Name @@ -160,10 +176,6 @@ func maybeTick(m periodic) { } } -// TODO(irfansharif): Figure out how to export runtime scheduler latencies as a -// prometheus histogram? Can we use a functional histogram? And maintain deltas -// underneath? What does prometheus/client_golang do? - // NewHistogram is a prometheus-backed histogram. Depending on the value of // opts.Buckets, this is suitable for recording any kind of quantity. Common // sensible choices are {IO,Network}LatencyBuckets. @@ -192,6 +204,7 @@ func NewHistogram(meta Metadata, windowDuration time.Duration, buckets []float64 var _ periodic = (*Histogram)(nil) var _ PrometheusExportable = (*Histogram)(nil) +var _ WindowedHistogram = (*Histogram)(nil) // Histogram is a prometheus-backed histogram. It collects observed values by // keeping bucketed counts. For convenience, internally two sets of buckets are @@ -296,7 +309,7 @@ func (h *Histogram) TotalCount() int64 { return int64(h.ToPrometheusMetric().Histogram.GetSampleCount()) } -// TotalCountWindowed returns the number of samples in the current window. +// TotalCountWindowed implements the WindowedHistogram interface. func (h *Histogram) TotalCountWindowed() int64 { return int64(h.ToPrometheusMetricWindowed().Histogram.GetSampleCount()) } @@ -306,7 +319,7 @@ func (h *Histogram) TotalSum() float64 { return h.ToPrometheusMetric().Histogram.GetSampleSum() } -// TotalSumWindowed returns the number of samples in the current window. +// TotalSumWindowed implements the WindowedHistogram interface. func (h *Histogram) TotalSumWindowed() float64 { return h.ToPrometheusMetricWindowed().Histogram.GetSampleSum() } @@ -316,10 +329,9 @@ func (h *Histogram) Mean() float64 { return h.TotalSum() / float64(h.TotalCount()) } -// ValueAtQuantileWindowed takes a quantile value [0,100] and returns the -// interpolated value at that quantile for the windowed histogram. +// ValueAtQuantileWindowed implements the WindowedHistogram interface. // -// https://github.com/prometheus/prometheus/blob/d91621890a2ccb3191a6d74812cc1827dd4093bf/promql/quantile.go#L75 +// https://github.com/prometheus/prometheus/blob/d9162189/promql/quantile.go#L75 // This function is mostly taken from a prometheus internal function that // does the same thing. There are a few differences for our use case: // 1. As a user of the prometheus go client library, we don't have access @@ -328,40 +340,7 @@ func (h *Histogram) Mean() float64 { // 2. Since the prometheus client library ensures buckets are in a strictly // increasing order at creation, we do not sort them. func (h *Histogram) ValueAtQuantileWindowed(q float64) float64 { - m := h.ToPrometheusMetricWindowed() - - buckets := m.Histogram.Bucket - n := float64(*m.Histogram.SampleCount) - if n == 0 { - return 0 - } - - rank := uint64(((q / 100) * n) + 0.5) - b := sort.Search(len(buckets)-1, func(i int) bool { return *buckets[i].CumulativeCount >= rank }) - - var ( - bucketStart float64 - bucketEnd = *buckets[b].UpperBound - count = *buckets[b].CumulativeCount - ) - - // Calculate the linearly interpolated value within the bucket - if b > 0 { - bucketStart = *buckets[b-1].UpperBound - count -= *buckets[b-1].CumulativeCount - rank -= *buckets[b-1].CumulativeCount - } - val := bucketStart + (bucketEnd-bucketStart)*(float64(rank)/float64(count)) - if math.IsNaN(val) || math.IsInf(val, -1) { - return 0 - } - - // should not extrapolate past the upper bound of the largest bucket - if val > *buckets[len(buckets)-1].UpperBound { - return *buckets[len(buckets)-1].UpperBound - } - - return val + return ValueAtQuantileWindowed(h.ToPrometheusMetricWindowed().Histogram, q) } // A Counter holds a single mutable atomic value. @@ -493,20 +472,11 @@ func (g *Gauge) GetMetadata() Metadata { type GaugeFloat64 struct { Metadata bits *uint64 - fn func() float64 } // NewGaugeFloat64 creates a GaugeFloat64. func NewGaugeFloat64(metadata Metadata) *GaugeFloat64 { - return &GaugeFloat64{metadata, new(uint64), nil} -} - -// NewFunctionalGaugeFloat64 creates a GaugeFloat64 metric whose value is -// determined when asked for by calling the provided function. -// Note that Update, Inc, and Dec should NOT be called on a Gauge returned -// from NewFunctionalGaugeFloat64. -func NewFunctionalGaugeFloat64(metadata Metadata, f func() float64) *GaugeFloat64 { - return &GaugeFloat64{metadata, nil, f} + return &GaugeFloat64{metadata, new(uint64)} } // Snapshot returns a read-only copy of the gauge. @@ -521,9 +491,6 @@ func (g *GaugeFloat64) Update(v float64) { // Value returns the gauge's current value. func (g *GaugeFloat64) Value() float64 { - if g.fn != nil { - return g.fn() - } return math.Float64frombits(atomic.LoadUint64(g.bits)) } @@ -570,3 +537,70 @@ func (g *GaugeFloat64) GetMetadata() Metadata { baseMetadata.MetricType = prometheusgo.MetricType_GAUGE return baseMetadata } + +// ValueAtQuantileWindowed takes a quantile value [0,100] and returns the +// interpolated value at that quantile for the given histogram. +func ValueAtQuantileWindowed(histogram *prometheusgo.Histogram, q float64) float64 { + buckets := histogram.Bucket + n := float64(*histogram.SampleCount) + if n == 0 { + return 0 + } + + // NB: The 0.5 is added for rounding purposes; it helps in cases where + // SampleCount is small. + rank := uint64(((q / 100) * n) + 0.5) + + // Since we are missing the +Inf bucket, CumulativeCounts may never exceed + // rank. By omitting the highest bucket we have from the search, the failed + // search will land on that last bucket and we don't have to do any special + // checks regarding landing on a non-existent bucket. + b := sort.Search(len(buckets)-1, func(i int) bool { return *buckets[i].CumulativeCount >= rank }) + + var ( + bucketStart float64 // defaults to 0, which we assume is the lower bound of the smallest bucket + bucketEnd = *buckets[b].UpperBound + count = *buckets[b].CumulativeCount + ) + + // Calculate the linearly interpolated value within the bucket. + if b > 0 { + bucketStart = *buckets[b-1].UpperBound + count -= *buckets[b-1].CumulativeCount + rank -= *buckets[b-1].CumulativeCount + } + val := bucketStart + (bucketEnd-bucketStart)*(float64(rank)/float64(count)) + if math.IsNaN(val) || math.IsInf(val, -1) { + return 0 + } + + // Should not extrapolate past the upper bound of the largest bucket. + // + // NB: SampleCount includes the implicit +Inf bucket but the + // buckets[len(buckets)-1].UpperBound refers to the largest bucket defined + // by us -- the client library doesn't give us access to the +Inf bucket + // which Prometheus uses under the hood. With a high enough quantile, the + // val computed further below surpasses the upper bound of the largest + // bucket. Using that interpolated value feels wrong since we'd be + // extrapolating. Also, for specific metrics if we see our q99 values to be + // hitting the top-most bucket boundary, that's an indication for us to + // choose better buckets for more accuracy. It's also worth noting that the + // prometheus client library does the same thing when the resulting value is + // in the +Inf bucket, whereby they return the upper bound of the second + // last bucket -- see [1]. + // + // [1]: https://github.com/prometheus/prometheus/blob/d9162189/promql/quantile.go#L103. + // the buckets to provide a more accurate histogram. FWIW the Prometheus client + // library does the same when the resulting value is in the +Inf bucket and + // returns the upper bound of the second last bucket: + // It is cleaner/easier for them since they have access to the +Inf bucket + // internally. + // The 0.5 was added for rounding purposes. I went back and forth on whether to + // have it at all but thought it made sense for smaller SampleCount cases like + // in this test: + if val > *buckets[len(buckets)-1].UpperBound { + return *buckets[len(buckets)-1].UpperBound + } + + return val +} diff --git a/pkg/util/schedulerlatency/BUILD.bazel b/pkg/util/schedulerlatency/BUILD.bazel index 9213a5bc9641..5369aad2368e 100644 --- a/pkg/util/schedulerlatency/BUILD.bazel +++ b/pkg/util/schedulerlatency/BUILD.bazel @@ -5,6 +5,7 @@ go_library( name = "schedulerlatency", srcs = [ "callbacks.go", + "histogram.go", "sampler.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/util/schedulerlatency", @@ -12,23 +13,32 @@ go_library( deps = [ "//pkg/settings", "//pkg/settings/cluster", + "//pkg/util/metric", "//pkg/util/ring", "//pkg/util/stop", "//pkg/util/syncutil", + "@com_github_gogo_protobuf//proto", + "@com_github_prometheus_client_model//go", ], ) go_test( name = "schedulerlatency_test", - srcs = ["scheduler_latency_test.go"], + srcs = [ + "histogram_test.go", + "scheduler_latency_test.go", + ], args = ["-test.timeout=295s"], + data = glob(["testdata/**"]), embed = [":schedulerlatency"], deps = [ "//pkg/settings/cluster", "//pkg/testutils", "//pkg/testutils/skip", + "//pkg/util/metric", "//pkg/util/stop", "//pkg/util/syncutil", + "@com_github_cockroachdb_datadriven//:datadriven", "@com_github_stretchr_testify//require", ], ) diff --git a/pkg/util/schedulerlatency/histogram.go b/pkg/util/schedulerlatency/histogram.go new file mode 100644 index 000000000000..48639272a637 --- /dev/null +++ b/pkg/util/schedulerlatency/histogram.go @@ -0,0 +1,263 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package schedulerlatency + +import ( + "math" + "runtime/metrics" + "time" + + "github.com/cockroachdb/cockroach/pkg/util/metric" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/gogo/protobuf/proto" + prometheusgo "github.com/prometheus/client_model/go" +) + +// runtimeHistogram is a histogram that's used to export histograms generated by +// the Go runtime; it's mutable and is updated in batches. The code here is +// adapted from [1][2]: +// +// [1]: github.com/prometheus/client_golang/blob/5b7e8b2e/prometheus/go_collector_latest.go +// [2]: github.com/prometheus/client_golang/blob/5b7e8b2e/prometheus/internal/go_runtime_metrics.go +type runtimeHistogram struct { + metric.Metadata + mu struct { + syncutil.Mutex + buckets []float64 // inclusive lower bounds, like runtime/metrics + counts []uint64 + } + mult float64 // multiplier to apply to each bucket boundary, used when translating across units +} + +var _ metric.Iterable = &runtimeHistogram{} +var _ metric.PrometheusExportable = &runtimeHistogram{} +var _ metric.WindowedHistogram = (*runtimeHistogram)(nil) + +// newRuntimeHistogram creates a histogram with the given metadata configured +// with the given buckets. The buckets must be a strict subset of what this +// histogram is updated with and follow the same conventions as those in +// runtime/metrics. +func newRuntimeHistogram(metadata metric.Metadata, buckets []float64) *runtimeHistogram { + // We need to remove -Inf values. runtime/metrics keeps them around. + // But -Inf bucket should not be allowed for prometheus histograms. + if buckets[0] == math.Inf(-1) { + buckets = buckets[1:] + } + h := &runtimeHistogram{ + Metadata: metadata, + // Go runtime histograms as of go1.19 are always in seconds whereas + // CRDB's histograms are in nanoseconds. Hardcode the conversion factor + // between the two, use it when translating to the prometheus exportable + // form (also used when writing to CRDB's internal TSDB). + mult: float64(time.Second.Nanoseconds()), + } + h.mu.buckets = buckets + // Because buckets follows runtime/metrics conventions, there's + // one more value in the buckets list than there are buckets represented, + // because in runtime/metrics, the bucket values represent boundaries, + // and non-Inf boundaries are inclusive lower bounds for that bucket. + h.mu.counts = make([]uint64, len(buckets)-1) + return h +} + +// update the histogram from a runtime/metrics histogram. +func (h *runtimeHistogram) update(his *metrics.Float64Histogram) { + h.mu.Lock() + defer h.mu.Unlock() + counts, buckets := his.Counts, his.Buckets + + for i := range h.mu.counts { + h.mu.counts[i] = 0 // clear buckets + } + var j int + for i, count := range counts { // copy and reduce buckets + h.mu.counts[j] += count + if buckets[i+1] == h.mu.buckets[j+1] { + j++ + } + } +} + +// write serializes the underlying histogram state into the form prometheus +// expects. +func (h *runtimeHistogram) write(out *prometheusgo.Metric) { + h.mu.Lock() + defer h.mu.Unlock() + + sum := float64(0) + dtoBuckets := make([]*prometheusgo.Bucket, 0, len(h.mu.counts)) + totalCount := uint64(0) + for i, count := range h.mu.counts { + totalCount += count + if count != 0 { + // N.B. this computed sum is an underestimate since we're using the + // lower bound of the bucket. + sum += h.mu.buckets[i] * h.mult * float64(count) + } + + // Skip the +Inf bucket, but only for the bucket list. It must still + // count for sum and totalCount. + if math.IsInf(h.mu.buckets[i+1]*h.mult, 1) { + break + } + // Float64Histogram's upper bound is exclusive, so make it inclusive by + // obtaining the next float64 value down, in order. + upperBound := math.Nextafter(h.mu.buckets[i+1], h.mu.buckets[i]) * h.mult + dtoBuckets = append(dtoBuckets, &prometheusgo.Bucket{ + CumulativeCount: proto.Uint64(totalCount), + UpperBound: proto.Float64(upperBound), + }) + } + out.Histogram = &prometheusgo.Histogram{ + Bucket: dtoBuckets, + SampleCount: proto.Uint64(totalCount), + SampleSum: proto.Float64(sum), + } +} + +// GetType is part of the PrometheusExportable interface. +func (h *runtimeHistogram) GetType() *prometheusgo.MetricType { + return prometheusgo.MetricType_HISTOGRAM.Enum() +} + +// ToPrometheusMetric is part of the PrometheusExportable interface. +func (h *runtimeHistogram) ToPrometheusMetric() *prometheusgo.Metric { + m := &prometheusgo.Metric{} + h.write(m) + return m +} + +// GetMetadata is part of the PrometheusExportable interface. +func (h *runtimeHistogram) GetMetadata() metric.Metadata { + return h.Metadata +} + +// Inspect is part of the Iterable interface. +func (h *runtimeHistogram) Inspect(f func(interface{})) { f(h) } + +// TotalCountWindowed implements the WindowedHistogram interface. +func (h *runtimeHistogram) TotalCountWindowed() int64 { + return int64(h.ToPrometheusMetric().Histogram.GetSampleCount()) +} + +// TotalSumWindowed implements the WindowedHistogram interface. +func (h *runtimeHistogram) TotalSumWindowed() float64 { + return h.ToPrometheusMetric().Histogram.GetSampleSum() +} + +// ValueAtQuantileWindowed implements the WindowedHistogram interface. +func (h *runtimeHistogram) ValueAtQuantileWindowed(q float64) float64 { + return metric.ValueAtQuantileWindowed(h.ToPrometheusMetric().Histogram, q) +} + +// reBucketExpAndTrim takes a list of bucket boundaries (lower bound inclusive) +// and down samples the buckets to those a multiple of base apart. The end +// result is a roughly exponential (in many cases, perfectly exponential) +// bucketing scheme. It also trims the bucket range to the specified min and max +// values -- everything outside the range is merged into (-Inf, ..] and [.., +// +Inf) buckets. The following example shows how it works, lifted from +// testdata/histogram_buckets. +// +// rebucket base=10 min=0ns max=100000h +// ---- +// bucket[ 0] width=0s boundary=[-Inf, 0s) +// bucket[ 1] width=1ns boundary=[0s, 1ns) +// bucket[ 2] width=9ns boundary=[1ns, 10ns) +// bucket[ 3] width=90ns boundary=[10ns, 100ns) +// bucket[ 4] width=924ns boundary=[100ns, 1.024µs) +// bucket[ 5] width=9.216µs boundary=[1.024µs, 10.24µs) +// bucket[ 6] width=92.16µs boundary=[10.24µs, 102.4µs) +// bucket[ 7] width=946.176µs boundary=[102.4µs, 1.048576ms) +// bucket[ 8] width=9.437184ms boundary=[1.048576ms, 10.48576ms) +func reBucketExpAndTrim(buckets []float64, base, min, max float64) []float64 { + // Re-bucket as powers of the given base. + b := reBucketExp(buckets, base) + + // Merge all buckets greater than the max value into the +Inf bucket. + for i := range b { + if i == 0 { + continue + } + if b[i-1] <= max { + continue + } + + // We're looking at the boundary after the first time we've crossed the + // max limit. Since we expect recordings near the max value, we don't + // want that bucket to end at +Inf, so we merge the bucket after. + b[i] = math.Inf(1) + b = b[:i+1] + break + } + + // Merge all buckets less than the min value into the -Inf bucket. + j := 0 + for i := range b { + if b[i] > min { + j = i + break + } + } + // b[j] > min and is the lower-bound of the j-th bucket. The min must be + // contained in the (j-1)-th bucket or earlier. We want to merge 0th bucket + // until the (j-2)-th one. + if j <= 2 { + // Nothing to do (we either have one or no buckets to merge together). + } else { + // We want trim the bucket list to start at (j-2)-th bucket, so just + // have bucket before the one containing the min. + b = b[j-2:] + // b[0] now refers the lower bound of what was previously the (j-2)-th + // bucket. We make it start at -Inf. + b[0] = math.Inf(-1) + } + + return b +} + +// reBucketExp is like reBucketExpAndTrim but without the trimming logic. +func reBucketExp(buckets []float64, base float64) []float64 { + bucket := buckets[0] + var newBuckets []float64 + // We may see -Inf here, in which case, add it and continue the rebucketing + // scheme from the next one it since we risk producing NaNs otherwise. We + // need to preserve -Inf values to maintain runtime/metrics conventions + if bucket == math.Inf(-1) { + newBuckets = append(newBuckets, bucket) + buckets = buckets[1:] + bucket = buckets[0] + } + + // From now on, bucket should always have a non-Inf value because Infs are + // only ever at the ends of the bucket lists, so arithmetic operations on it + // are non-NaN. + for i := 1; i < len(buckets); i++ { + // bucket is the lower bound of the lowest bucket that has not been + // added to newBuckets. We will add it to newBuckets, but we wait to add + // it until we find the next bucket that is >= bucket*base. + + if bucket >= 0 && buckets[i] < bucket*base { + // The next bucket we want to include is at least bucket*base. + continue + } else if bucket < 0 && buckets[i] < bucket/base { + // In this case the bucket we're targeting is negative, and since + // we're ascending through buckets here, we need to divide to get + // closer to zero exponentially. + continue + } + newBuckets = append(newBuckets, bucket) + bucket = buckets[i] + } + + // The +Inf bucket will always be the last one, and we'll always + // end up including it here. + return append(newBuckets, bucket) +} diff --git a/pkg/util/schedulerlatency/histogram_test.go b/pkg/util/schedulerlatency/histogram_test.go new file mode 100644 index 000000000000..a638c59ea379 --- /dev/null +++ b/pkg/util/schedulerlatency/histogram_test.go @@ -0,0 +1,227 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package schedulerlatency + +import ( + "fmt" + "math" + "runtime/metrics" + "strconv" + "strings" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/util/metric" + "github.com/cockroachdb/datadriven" + "github.com/stretchr/testify/require" +) + +// TestHistogramBuckets is a datadriven test that's used to generate +// prometheus buckets to be used with the scheduler latency histogram exported +// by this package. It comes with the following commands. +// +// - "buckets" +// Print out the histogram bucket boundaries maintained by Go when +// collecting data for scheduling latencies. +// +// - "rebucket" base= min= max=duration +// Rebucket the default set of bucket boundaries such that they're a +// multiple of base apart. It also trims the bucket range to the specified +// min/max values; everything outside the range is merged into (-Inf, ..] +// and [.., +Inf) buckets. +func TestHistogramBuckets(t *testing.T) { + buckets := sample().Buckets + datadriven.RunTest(t, testutils.TestDataPath(t, "histogram_buckets"), + func(t *testing.T, d *datadriven.TestData) string { + switch d.Cmd { + case "buckets": + return printBuckets(buckets) + + case "rebucket": + base := parseFloat(t, d, "base") + min, max := parseDuration(t, d, "min"), parseDuration(t, d, "max") + rebucketed := reBucketExpAndTrim(buckets, base, min.Seconds(), max.Seconds()) + require.Subset(t, buckets, rebucketed) + return printBuckets(rebucketed) + + default: + return fmt.Sprintf("unknown command: %s", d.Cmd) + } + }, + ) +} + +// TestRuntimeHistogram is a datadriven test for the runtimeHistogram type. It +// comes with the following commands. +// +// - "init" +// bucket=[,) +// bucket=[,) +// +// - "update" +// bucket=[,) count= +// bucket=[,) count= +// ... +// +// - "print" +// +// NB: is also allowed to be "-inf" or "+inf". +func TestRuntimeHistogram(t *testing.T) { + var rh *runtimeHistogram + datadriven.RunTest(t, testutils.TestDataPath(t, "runtime_histogram"), + func(t *testing.T, d *datadriven.TestData) string { + switch d.Cmd { + case "init": + buckets := parseBuckets(t, d.Input) + rh = newRuntimeHistogram(metric.Metadata{}, buckets) + rh.mult = 1.0 + return "" + + case "update": + his := &metrics.Float64Histogram{ + Counts: parseCounts(t, d.Input), + Buckets: parseBuckets(t, d.Input), + } + require.True(t, len(his.Buckets) == len(his.Counts)+1) + rh.update(his) + return "" + + case "print": + var buf strings.Builder + buf.WriteString(fmt.Sprintf("count=%d sum=%0.2f\n", + rh.TotalCountWindowed(), + rh.TotalSumWindowed(), + )) + hist := rh.ToPrometheusMetric().GetHistogram() + require.NotNil(t, hist) + buf.WriteString("buckets:\n") + for _, bucket := range hist.Bucket { + buf.WriteString(fmt.Sprintf(" upper-bound=%0.2f cumulative-count=%d\n", + *bucket.UpperBound, + *bucket.CumulativeCount, + )) + } + return buf.String() + + default: + return fmt.Sprintf("unknown command: %s", d.Cmd) + } + }, + ) +} + +// parseBuckets parses out the list of bucket boundaries when the given input is +// of the form: +// +// bucket=[,) ... +// bucket=[,) ... +// +// NB: is also allowed to be "-inf" or "+inf". +func parseBuckets(t *testing.T, input string) []float64 { + var buckets []float64 + for _, line := range strings.Split(input, "\n") { + token := strings.Fields(line)[0] // bucket=[,) + token = strings.TrimPrefix(strings.TrimSpace(token), "bucket=") // [,) + token = strings.TrimPrefix(strings.TrimSpace(token), "[") // ,) + token = strings.TrimSuffix(strings.TrimSpace(token), ")") // , + + parts := strings.Split(token, ",") + var start, end float64 + var err error + + if parts[0] == "-inf" { + start = math.Inf(-1) + } else if parts[0] == "+inf" { + start = math.Inf(1) + } else { + start, err = strconv.ParseFloat(parts[0], 64) + require.NoError(t, err) + } + + if parts[1] == "-inf" { + end = math.Inf(-1) + } else if parts[1] == "+inf" { + end = math.Inf(1) + } else { + end, err = strconv.ParseFloat(parts[1], 64) + require.NoError(t, err) + } + + if len(buckets) == 0 { + buckets = append(buckets, start) + } else { + require.Equalf(t, buckets[len(buckets)-1], start, + "expected end of last bucket to be equal to start of next (around line: %q)", line) + } + buckets = append(buckets, end) + } + return buckets +} + +// parseCounts parses out the list of bucket boundaries when the given input is +// of the form: +// +// ... count= +// ... count= +func parseCounts(t *testing.T, input string) []uint64 { + var counts []uint64 + for _, line := range strings.Split(input, "\n") { + fields := strings.Fields(line) // ... count= + for _, field := range fields { + if !strings.HasPrefix(field, "count=") { + continue + } + token := field // count= + token = strings.TrimPrefix(strings.TrimSpace(token), "count=") // + count, err := strconv.ParseUint(token, 10, 64) + require.NoError(t, err) + counts = append(counts, count) + } + } + return counts +} + +func parseFloat(t *testing.T, d *datadriven.TestData, key string) float64 { + var floatStr string + d.ScanArgs(t, key, &floatStr) + f, err := strconv.ParseFloat(floatStr, 64) + require.NoError(t, err) + return f +} + +func parseDuration(t *testing.T, d *datadriven.TestData, key string) time.Duration { + var durationStr string + d.ScanArgs(t, key, &durationStr) + duration, err := time.ParseDuration(durationStr) + require.NoError(t, err) + return duration +} + +func printBuckets(buckets []float64) string { + var buf strings.Builder + for i := 0; i < len(buckets)-1; i++ { + sd := time.Duration(buckets[i] * float64(time.Second.Nanoseconds())) + ed := time.Duration(buckets[i+1] * float64(time.Second.Nanoseconds())) + s, e := sd.String(), ed.String() + d := time.Duration(ed.Nanoseconds() - sd.Nanoseconds()).String() + if math.IsInf(buckets[i], -1) { + s = "-Inf" + d = e + } + if math.IsInf(buckets[i+1], +1) { + e = "+Inf" + d = "Inf" + } + buf.WriteString(fmt.Sprintf("bucket[%3d] width=%-18s boundary=[%s, %s)\n", i, d, s, e)) + } + return buf.String() +} diff --git a/pkg/util/schedulerlatency/sampler.go b/pkg/util/schedulerlatency/sampler.go index 125c5b7f8420..9bbe426ac53e 100644 --- a/pkg/util/schedulerlatency/sampler.go +++ b/pkg/util/schedulerlatency/sampler.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/ring" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -58,9 +59,22 @@ var sampleDuration = settings.RegisterDurationSetting( }, ) +var schedulerLatency = metric.Metadata{ + Name: "go.scheduler_latency", + Help: "Go scheduling latency", + Measurement: "Nanoseconds", + Unit: metric.Unit_NANOSECONDS, +} + // StartSampler spawn a goroutine to periodically sample the scheduler latencies // and invoke all registered callbacks. -func StartSampler(ctx context.Context, st *cluster.Settings, stopper *stop.Stopper) error { +func StartSampler( + ctx context.Context, + st *cluster.Settings, + stopper *stop.Stopper, + registry *metric.Registry, + statsInterval time.Duration, +) error { return stopper.RunAsyncTask(ctx, "scheduler-latency-sampler", func(ctx context.Context) { settingsValuesMu := struct { syncutil.Mutex @@ -69,10 +83,46 @@ func StartSampler(ctx context.Context, st *cluster.Settings, stopper *stop.Stopp settingsValuesMu.period = samplePeriod.Get(&st.SV) settingsValuesMu.duration = sampleDuration.Get(&st.SV) - ticker := time.NewTicker(settingsValuesMu.period) - defer ticker.Stop() s := newSampler(settingsValuesMu.period, settingsValuesMu.duration) + _ = stopper.RunAsyncTask(ctx, "export-scheduler-stats", func(ctx context.Context) { + // cpuSchedulerLatencyBuckets are prometheus histogram buckets + // suitable for a histogram that records a (second-denominated) + // quantity where measurements correspond to delays in scheduling + // goroutines onto processors, i.e. are in the {micro,milli}-second + // range during normal operation. See TestHistogramBuckets for more + // details. + cpuSchedulerLatencyBuckets := reBucketExpAndTrim( + sample().Buckets, // original buckets + 1.1, // base + (50 * time.Microsecond).Seconds(), // min + (100 * time.Millisecond).Seconds(), // max + ) + + schedulerLatencyHistogram := newRuntimeHistogram(schedulerLatency, cpuSchedulerLatencyBuckets) + registry.AddMetric(schedulerLatencyHistogram) + + ticker := time.NewTicker(statsInterval) // compute periodic stats + defer ticker.Stop() + for { + select { + case <-ctx.Done(): + return + case <-stopper.ShouldQuiesce(): + return + case <-ticker.C: + lastIntervalHistogram := s.lastIntervalHistogram() + if lastIntervalHistogram == nil { + continue + } + + schedulerLatencyHistogram.update(lastIntervalHistogram) + } + } + }) + + ticker := time.NewTicker(settingsValuesMu.period) + defer ticker.Stop() samplePeriod.SetOnChange(&st.SV, func(ctx context.Context) { period := samplePeriod.Get(&st.SV) settingsValuesMu.Lock() @@ -109,7 +159,8 @@ func StartSampler(ctx context.Context, st *cluster.Settings, stopper *stop.Stopp type sampler struct { mu struct { syncutil.Mutex - ringBuffer ring.Buffer // contains *metrics.Float64Histogram + ringBuffer ring.Buffer // contains *metrics.Float64Histogram + lastIntervalHistogram *metrics.Float64Histogram } } @@ -128,34 +179,35 @@ func (s *sampler) setPeriodAndDuration(period, duration time.Duration) { numSamples = 1 // we need at least one sample to compare (also safeguards against integer division) } s.mu.ringBuffer.Resize(numSamples) + s.mu.lastIntervalHistogram = nil s.mu.Unlock() } // sampleOnTickAndInvokeCallbacks samples scheduler latency stats as the ticker // has ticked. It invokes all callbacks registered with this package. func (s *sampler) sampleOnTickAndInvokeCallbacks(period time.Duration) { + s.mu.Lock() + defer s.mu.Unlock() + latestCumulative := sample() - oldestCumulative, ok := s.record(latestCumulative) + oldestCumulative, ok := s.recordLocked(latestCumulative) if !ok { return } - interval := sub(latestCumulative, oldestCumulative) - latency := time.Duration(int64(percentile(interval, 0.99) * float64(time.Second.Nanoseconds()))) + s.mu.lastIntervalHistogram = sub(latestCumulative, oldestCumulative) + p99 := time.Duration(int64(percentile(s.mu.lastIntervalHistogram, 0.99) * float64(time.Second.Nanoseconds()))) globallyRegisteredCallbacks.mu.Lock() defer globallyRegisteredCallbacks.mu.Unlock() cbs := globallyRegisteredCallbacks.mu.callbacks for i := range cbs { - cbs[i].cb(latency, period) + cbs[i].cb(p99, period) } } -func (s *sampler) record( +func (s *sampler) recordLocked( sample *metrics.Float64Histogram, ) (oldest *metrics.Float64Histogram, ok bool) { - s.mu.Lock() - defer s.mu.Unlock() - if s.mu.ringBuffer.Len() == s.mu.ringBuffer.Cap() { // no more room, clear out the oldest oldest = s.mu.ringBuffer.GetLast().(*metrics.Float64Histogram) s.mu.ringBuffer.RemoveLast() @@ -164,6 +216,12 @@ func (s *sampler) record( return oldest, oldest != nil } +func (s *sampler) lastIntervalHistogram() *metrics.Float64Histogram { + s.mu.Lock() + defer s.mu.Unlock() + return s.mu.lastIntervalHistogram +} + // sample the cumulative (since process start) scheduler latency histogram from // the go runtime. func sample() *metrics.Float64Histogram { @@ -204,17 +262,37 @@ func sub(a, b *metrics.Float64Histogram) *metrics.Float64Histogram { } // percentile computes a specific percentile value of the given histogram. +// +// TODO(irfansharif): Deduplicate this with the quantile computation in +// util/metrics? Here we're using the raw histogram at the highest resolution +// and with zero translation between types; there we rebucket the histogram and +// translate to another type. func percentile(h *metrics.Float64Histogram, p float64) float64 { // Counts contains the number of occurrences for each histogram bucket. // Given N buckets, Count[n] is the number of occurrences in the range // [bucket[n], bucket[n+1]), for 0 <= n < N. // - // TODO(irfansharif): Consider maintaining the total count in the runtime - // itself to make this cheaper if we're calling this at a high frequency. - // // TODO(irfansharif): Consider adjusting the default bucket count in the // runtime to make this cheaper and with a more appropriate amount of - // resolution. + // resolution. The defaults can be seen through TestHistogramBuckets: + // + // 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) + // var total uint64 // total count across all buckets for i := range h.Counts { total += h.Counts[i] @@ -253,5 +331,16 @@ func percentile(h *metrics.Float64Histogram, p float64) float64 { break // we've found the bucket where the cumulative count until that point is p% of the total } } - return (start + end) / 2 // linear interpolate within the bucket + + return (start + end) / 2 // grab the mid-point within the bucket + + // TODO(irfansharif): Implement proper linear interpolation instead and then + // write test comparing against it against metric.ValueAtQuantileWindowed. + // If pmax, we'll return the bucket max. If pmin, we'll return the bucket + // min. If the 90th and 100th percentile values lie within some bucket, and + // we're looking for 99.9th percentile, we'll interpolate to the 99% point + // between bucket start and end. Because we're doing this naive mid-point + // thing, it makes for a confusing difference when comparing the p99 + // computed off of go.scheduler_latency vs. + // admission.scheduler_latency_listener.p99_nanos. } diff --git a/pkg/util/schedulerlatency/scheduler_latency_test.go b/pkg/util/schedulerlatency/scheduler_latency_test.go index 33a0e93b53aa..e46c5a4c3593 100644 --- a/pkg/util/schedulerlatency/scheduler_latency_test.go +++ b/pkg/util/schedulerlatency/scheduler_latency_test.go @@ -22,12 +22,16 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" + "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/stretchr/testify/require" ) -func TestSchedulerLatencyCallbacks(t *testing.T) { +// TestSchedulerLatencySampler is an integration test for the scheduler latency +// sampler -- it verifies that scheduling latencies are measured, registered +// callbacks are invoked, and that the prometheus metrics emitted are non-empty. +func TestSchedulerLatencySampler(t *testing.T) { skip.UnderStress(t) skip.UnderShort(t) @@ -71,14 +75,29 @@ func TestSchedulerLatencyCallbacks(t *testing.T) { }) defer UnregisterCallback(slcbID) - require.NoError(t, StartSampler(ctx, st, stopper)) + reg := metric.NewRegistry() + require.NoError(t, StartSampler(ctx, st, stopper, reg, 10*time.Second)) testutils.SucceedsSoon(t, func() error { mu.Lock() defer mu.Unlock() if mu.p99.Nanoseconds() == 0 { return fmt.Errorf("expected non-zero p99 scheduling latency") } - return nil + + var err error + reg.Each(func(name string, mtr interface{}) { + wh := mtr.(metric.WindowedHistogram) + count := float64(wh.TotalCountWindowed()) + avg := wh.TotalSumWindowed() / count + if math.IsNaN(avg) || math.IsInf(avg, +1) || math.IsInf(avg, -1) { + avg = 0 + } + + if wh.ValueAtQuantileWindowed(99) == 0 || count == 0 || avg == 0 { + err = fmt.Errorf("expected non-zero p99 scheduling latency metrics") + } + }) + return err }) } diff --git a/pkg/util/schedulerlatency/testdata/histogram_buckets b/pkg/util/schedulerlatency/testdata/histogram_buckets new file mode 100644 index 000000000000..a99b1bf57223 --- /dev/null +++ b/pkg/util/schedulerlatency/testdata/histogram_buckets @@ -0,0 +1,853 @@ +# Explore how the rebucketing procedure works. Buckets are picked along +# multiples of the given base. +rebucket base=10 min=0ns max=100000h +---- +bucket[ 0] width=0s boundary=[-Inf, 0s) +bucket[ 1] width=1ns boundary=[0s, 1ns) +bucket[ 2] width=9ns boundary=[1ns, 10ns) +bucket[ 3] width=90ns boundary=[10ns, 100ns) +bucket[ 4] width=924ns boundary=[100ns, 1.024µs) +bucket[ 5] width=9.216µs boundary=[1.024µs, 10.24µs) +bucket[ 6] width=92.16µs boundary=[10.24µs, 102.4µs) +bucket[ 7] width=946.176µs boundary=[102.4µs, 1.048576ms) +bucket[ 8] width=9.437184ms boundary=[1.048576ms, 10.48576ms) +bucket[ 9] width=94.37184ms boundary=[10.48576ms, 104.8576ms) +bucket[ 10] width=968.884224ms boundary=[104.8576ms, 1.073741824s) +bucket[ 11] width=9.663676416s boundary=[1.073741824s, 10.73741824s) +bucket[ 12] width=1m36.63676416s boundary=[10.73741824s, 1m47.3741824s) +bucket[ 13] width=16m32.137445376s boundary=[1m47.3741824s, 18m19.511627776s) +bucket[ 14] width=2h44m55.604649984s boundary=[18m19.511627776s, 3h3m15.11627776s) +bucket[ 15] width=27h29m16.04649984s boundary=[3h3m15.11627776s, 30h32m31.1627776s) +bucket[ 16] width=Inf boundary=[30h32m31.1627776s, +Inf) + + +# Buckets are clamped at the max specified. +rebucket base=10 min=0ns max=1s +---- +bucket[ 0] width=0s boundary=[-Inf, 0s) +bucket[ 1] width=1ns boundary=[0s, 1ns) +bucket[ 2] width=9ns boundary=[1ns, 10ns) +bucket[ 3] width=90ns boundary=[10ns, 100ns) +bucket[ 4] width=924ns boundary=[100ns, 1.024µs) +bucket[ 5] width=9.216µs boundary=[1.024µs, 10.24µs) +bucket[ 6] width=92.16µs boundary=[10.24µs, 102.4µs) +bucket[ 7] width=946.176µs boundary=[102.4µs, 1.048576ms) +bucket[ 8] width=9.437184ms boundary=[1.048576ms, 10.48576ms) +bucket[ 9] width=94.37184ms boundary=[10.48576ms, 104.8576ms) +bucket[ 10] width=968.884224ms boundary=[104.8576ms, 1.073741824s) +bucket[ 11] width=Inf boundary=[1.073741824s, +Inf) + +# Buckets are clamped at the min specified, picking the first bucket the houses +# the minimum and starting the base multiplier after that point. +rebucket base=10 min=1h max=10000h +---- +bucket[ 0] width=18m19.511627776s boundary=[-Inf, 18m19.511627776s) +bucket[ 1] width=2h44m55.604649984s boundary=[18m19.511627776s, 3h3m15.11627776s) +bucket[ 2] width=27h29m16.04649984s boundary=[3h3m15.11627776s, 30h32m31.1627776s) +bucket[ 3] width=Inf boundary=[30h32m31.1627776s, +Inf) + +# Clamping along both min/max also works. +rebucket base=10 min=1h max=3h +---- +bucket[ 0] width=18m19.511627776s boundary=[-Inf, 18m19.511627776s) +bucket[ 1] width=2h44m55.604649984s boundary=[18m19.511627776s, 3h3m15.11627776s) +bucket[ 2] width=Inf boundary=[3h3m15.11627776s, +Inf) + +# As for the bucket values used in production. +rebucket base=1.1 min=50us max=100ms +---- +bucket[ 0] width=47.104µs boundary=[-Inf, 47.104µs) +bucket[ 1] width=6.144µs boundary=[47.104µs, 53.248µs) +bucket[ 2] width=6.144µs boundary=[53.248µs, 59.392µs) +bucket[ 3] width=6.144µs boundary=[59.392µs, 65.536µs) +bucket[ 4] width=8.192µs boundary=[65.536µs, 73.728µs) +bucket[ 5] width=8.192µs boundary=[73.728µs, 81.92µs) +bucket[ 6] width=12.288µs boundary=[81.92µs, 94.208µs) +bucket[ 7] width=12.288µs boundary=[94.208µs, 106.496µs) +bucket[ 8] width=12.288µs boundary=[106.496µs, 118.784µs) +bucket[ 9] width=12.288µs boundary=[118.784µs, 131.072µs) +bucket[ 10] width=16.384µs boundary=[131.072µs, 147.456µs) +bucket[ 11] width=16.384µs boundary=[147.456µs, 163.84µs) +bucket[ 12] width=24.576µs boundary=[163.84µs, 188.416µs) +bucket[ 13] width=24.576µs boundary=[188.416µs, 212.992µs) +bucket[ 14] width=24.576µs boundary=[212.992µs, 237.568µs) +bucket[ 15] width=24.576µs boundary=[237.568µs, 262.144µs) +bucket[ 16] width=32.768µs boundary=[262.144µs, 294.912µs) +bucket[ 17] width=32.768µs boundary=[294.912µs, 327.68µs) +bucket[ 18] width=49.152µs boundary=[327.68µs, 376.832µs) +bucket[ 19] width=49.152µs boundary=[376.832µs, 425.984µs) +bucket[ 20] width=49.152µs boundary=[425.984µs, 475.136µs) +bucket[ 21] width=49.152µs boundary=[475.136µs, 524.288µs) +bucket[ 22] width=65.536µs boundary=[524.288µs, 589.824µs) +bucket[ 23] width=65.536µs boundary=[589.824µs, 655.36µs) +bucket[ 24] width=98.304µs boundary=[655.36µs, 753.664µs) +bucket[ 25] width=98.304µs boundary=[753.664µs, 851.968µs) +bucket[ 26] width=98.304µs boundary=[851.968µs, 950.272µs) +bucket[ 27] width=98.304µs boundary=[950.272µs, 1.048576ms) +bucket[ 28] width=131.072µs boundary=[1.048576ms, 1.179648ms) +bucket[ 29] width=131.072µs boundary=[1.179648ms, 1.31072ms) +bucket[ 30] width=196.608µs boundary=[1.31072ms, 1.507328ms) +bucket[ 31] width=196.608µs boundary=[1.507328ms, 1.703936ms) +bucket[ 32] width=196.608µs boundary=[1.703936ms, 1.900544ms) +bucket[ 33] width=196.608µs boundary=[1.900544ms, 2.097152ms) +bucket[ 34] width=262.144µs boundary=[2.097152ms, 2.359296ms) +bucket[ 35] width=262.144µs boundary=[2.359296ms, 2.62144ms) +bucket[ 36] width=393.216µs boundary=[2.62144ms, 3.014656ms) +bucket[ 37] width=393.216µs boundary=[3.014656ms, 3.407872ms) +bucket[ 38] width=393.216µs boundary=[3.407872ms, 3.801088ms) +bucket[ 39] width=393.216µs boundary=[3.801088ms, 4.194304ms) +bucket[ 40] width=524.288µs boundary=[4.194304ms, 4.718592ms) +bucket[ 41] width=524.288µs boundary=[4.718592ms, 5.24288ms) +bucket[ 42] width=786.432µs boundary=[5.24288ms, 6.029312ms) +bucket[ 43] width=786.432µs boundary=[6.029312ms, 6.815744ms) +bucket[ 44] width=786.432µs boundary=[6.815744ms, 7.602176ms) +bucket[ 45] width=786.432µs boundary=[7.602176ms, 8.388608ms) +bucket[ 46] width=1.048576ms boundary=[8.388608ms, 9.437184ms) +bucket[ 47] width=1.048576ms boundary=[9.437184ms, 10.48576ms) +bucket[ 48] width=1.572864ms boundary=[10.48576ms, 12.058624ms) +bucket[ 49] width=1.572864ms boundary=[12.058624ms, 13.631488ms) +bucket[ 50] width=1.572864ms boundary=[13.631488ms, 15.204352ms) +bucket[ 51] width=1.572864ms boundary=[15.204352ms, 16.777216ms) +bucket[ 52] width=2.097152ms boundary=[16.777216ms, 18.874368ms) +bucket[ 53] width=2.097152ms boundary=[18.874368ms, 20.97152ms) +bucket[ 54] width=3.145728ms boundary=[20.97152ms, 24.117248ms) +bucket[ 55] width=3.145728ms boundary=[24.117248ms, 27.262976ms) +bucket[ 56] width=3.145728ms boundary=[27.262976ms, 30.408704ms) +bucket[ 57] width=3.145728ms boundary=[30.408704ms, 33.554432ms) +bucket[ 58] width=4.194304ms boundary=[33.554432ms, 37.748736ms) +bucket[ 59] width=4.194304ms boundary=[37.748736ms, 41.94304ms) +bucket[ 60] width=6.291456ms boundary=[41.94304ms, 48.234496ms) +bucket[ 61] width=6.291456ms boundary=[48.234496ms, 54.525952ms) +bucket[ 62] width=6.291456ms boundary=[54.525952ms, 60.817408ms) +bucket[ 63] width=6.291456ms boundary=[60.817408ms, 67.108864ms) +bucket[ 64] width=8.388608ms boundary=[67.108864ms, 75.497472ms) +bucket[ 65] width=8.388608ms boundary=[75.497472ms, 83.88608ms) +bucket[ 66] width=12.582912ms boundary=[83.88608ms, 96.468992ms) +bucket[ 67] width=12.582912ms boundary=[96.468992ms, 109.051904ms) +bucket[ 68] width=Inf boundary=[109.051904ms, +Inf) + +# The full set of bucket boundaries (everything above is a subset of this +# list). +buckets +---- +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[ 5] width=1ns boundary=[4ns, 5ns) +bucket[ 6] width=1ns boundary=[5ns, 6ns) +bucket[ 7] width=1ns boundary=[6ns, 7ns) +bucket[ 8] width=1ns boundary=[7ns, 8ns) +bucket[ 9] width=1ns boundary=[8ns, 9ns) +bucket[ 10] width=1ns boundary=[9ns, 10ns) +bucket[ 11] width=1ns boundary=[10ns, 11ns) +bucket[ 12] width=1ns boundary=[11ns, 12ns) +bucket[ 13] width=1ns boundary=[12ns, 13ns) +bucket[ 14] width=1ns boundary=[13ns, 14ns) +bucket[ 15] width=0s boundary=[14ns, 14ns) +bucket[ 16] width=2ns boundary=[14ns, 16ns) +bucket[ 17] width=1ns boundary=[16ns, 17ns) +bucket[ 18] width=1ns boundary=[17ns, 18ns) +bucket[ 19] width=1ns boundary=[18ns, 19ns) +bucket[ 20] width=1ns boundary=[19ns, 20ns) +bucket[ 21] width=1ns boundary=[20ns, 21ns) +bucket[ 22] width=1ns boundary=[21ns, 22ns) +bucket[ 23] width=1ns boundary=[22ns, 23ns) +bucket[ 24] width=1ns boundary=[23ns, 24ns) +bucket[ 25] width=1ns boundary=[24ns, 25ns) +bucket[ 26] width=1ns boundary=[25ns, 26ns) +bucket[ 27] width=1ns boundary=[26ns, 27ns) +bucket[ 28] width=1ns boundary=[27ns, 28ns) +bucket[ 29] width=1ns boundary=[28ns, 29ns) +bucket[ 30] width=0s boundary=[29ns, 29ns) +bucket[ 31] width=2ns boundary=[29ns, 31ns) +bucket[ 32] width=1ns boundary=[31ns, 32ns) +bucket[ 33] width=2ns boundary=[32ns, 34ns) +bucket[ 34] width=2ns boundary=[34ns, 36ns) +bucket[ 35] width=2ns boundary=[36ns, 38ns) +bucket[ 36] width=2ns boundary=[38ns, 40ns) +bucket[ 37] width=2ns boundary=[40ns, 42ns) +bucket[ 38] width=2ns boundary=[42ns, 44ns) +bucket[ 39] width=2ns boundary=[44ns, 46ns) +bucket[ 40] width=2ns boundary=[46ns, 48ns) +bucket[ 41] width=2ns boundary=[48ns, 50ns) +bucket[ 42] width=2ns boundary=[50ns, 52ns) +bucket[ 43] width=2ns boundary=[52ns, 54ns) +bucket[ 44] width=2ns boundary=[54ns, 56ns) +bucket[ 45] width=2ns boundary=[56ns, 58ns) +bucket[ 46] width=1ns boundary=[58ns, 59ns) +bucket[ 47] width=3ns boundary=[59ns, 62ns) +bucket[ 48] width=2ns boundary=[62ns, 64ns) +bucket[ 49] width=4ns boundary=[64ns, 68ns) +bucket[ 50] width=4ns boundary=[68ns, 72ns) +bucket[ 51] width=4ns boundary=[72ns, 76ns) +bucket[ 52] width=4ns boundary=[76ns, 80ns) +bucket[ 53] width=4ns boundary=[80ns, 84ns) +bucket[ 54] width=4ns boundary=[84ns, 88ns) +bucket[ 55] width=4ns boundary=[88ns, 92ns) +bucket[ 56] width=4ns boundary=[92ns, 96ns) +bucket[ 57] width=4ns boundary=[96ns, 100ns) +bucket[ 58] width=4ns boundary=[100ns, 104ns) +bucket[ 59] width=4ns boundary=[104ns, 108ns) +bucket[ 60] width=4ns boundary=[108ns, 112ns) +bucket[ 61] width=4ns boundary=[112ns, 116ns) +bucket[ 62] width=3ns boundary=[116ns, 119ns) +bucket[ 63] width=5ns boundary=[119ns, 124ns) +bucket[ 64] width=4ns boundary=[124ns, 128ns) +bucket[ 65] width=8ns boundary=[128ns, 136ns) +bucket[ 66] width=8ns boundary=[136ns, 144ns) +bucket[ 67] width=8ns boundary=[144ns, 152ns) +bucket[ 68] width=8ns boundary=[152ns, 160ns) +bucket[ 69] width=8ns boundary=[160ns, 168ns) +bucket[ 70] width=8ns boundary=[168ns, 176ns) +bucket[ 71] width=8ns boundary=[176ns, 184ns) +bucket[ 72] width=8ns boundary=[184ns, 192ns) +bucket[ 73] width=8ns boundary=[192ns, 200ns) +bucket[ 74] width=8ns boundary=[200ns, 208ns) +bucket[ 75] width=8ns boundary=[208ns, 216ns) +bucket[ 76] width=8ns boundary=[216ns, 224ns) +bucket[ 77] width=8ns boundary=[224ns, 232ns) +bucket[ 78] width=7ns boundary=[232ns, 239ns) +bucket[ 79] width=9ns boundary=[239ns, 248ns) +bucket[ 80] width=8ns boundary=[248ns, 256ns) +bucket[ 81] width=16ns boundary=[256ns, 272ns) +bucket[ 82] width=16ns boundary=[272ns, 288ns) +bucket[ 83] width=16ns boundary=[288ns, 304ns) +bucket[ 84] width=16ns boundary=[304ns, 320ns) +bucket[ 85] width=16ns boundary=[320ns, 336ns) +bucket[ 86] width=16ns boundary=[336ns, 352ns) +bucket[ 87] width=16ns boundary=[352ns, 368ns) +bucket[ 88] width=16ns boundary=[368ns, 384ns) +bucket[ 89] width=16ns boundary=[384ns, 400ns) +bucket[ 90] width=16ns boundary=[400ns, 416ns) +bucket[ 91] width=16ns boundary=[416ns, 432ns) +bucket[ 92] width=16ns boundary=[432ns, 448ns) +bucket[ 93] width=16ns boundary=[448ns, 464ns) +bucket[ 94] width=15ns boundary=[464ns, 479ns) +bucket[ 95] width=17ns boundary=[479ns, 496ns) +bucket[ 96] width=16ns boundary=[496ns, 512ns) +bucket[ 97] width=32ns boundary=[512ns, 544ns) +bucket[ 98] width=32ns boundary=[544ns, 576ns) +bucket[ 99] width=32ns boundary=[576ns, 608ns) +bucket[100] width=32ns boundary=[608ns, 640ns) +bucket[101] width=32ns boundary=[640ns, 672ns) +bucket[102] width=32ns boundary=[672ns, 704ns) +bucket[103] width=32ns boundary=[704ns, 736ns) +bucket[104] width=32ns boundary=[736ns, 768ns) +bucket[105] width=32ns boundary=[768ns, 800ns) +bucket[106] width=32ns boundary=[800ns, 832ns) +bucket[107] width=32ns boundary=[832ns, 864ns) +bucket[108] width=32ns boundary=[864ns, 896ns) +bucket[109] width=32ns boundary=[896ns, 928ns) +bucket[110] width=31ns boundary=[928ns, 959ns) +bucket[111] width=33ns boundary=[959ns, 992ns) +bucket[112] width=32ns boundary=[992ns, 1.024µs) +bucket[113] width=64ns boundary=[1.024µs, 1.088µs) +bucket[114] width=64ns boundary=[1.088µs, 1.152µs) +bucket[115] width=64ns boundary=[1.152µs, 1.216µs) +bucket[116] width=64ns boundary=[1.216µs, 1.28µs) +bucket[117] width=64ns boundary=[1.28µs, 1.344µs) +bucket[118] width=64ns boundary=[1.344µs, 1.408µs) +bucket[119] width=64ns boundary=[1.408µs, 1.472µs) +bucket[120] width=64ns boundary=[1.472µs, 1.536µs) +bucket[121] width=64ns boundary=[1.536µs, 1.6µs) +bucket[122] width=64ns boundary=[1.6µs, 1.664µs) +bucket[123] width=64ns boundary=[1.664µs, 1.728µs) +bucket[124] width=64ns boundary=[1.728µs, 1.792µs) +bucket[125] width=64ns boundary=[1.792µs, 1.856µs) +bucket[126] width=63ns boundary=[1.856µs, 1.919µs) +bucket[127] width=65ns boundary=[1.919µs, 1.984µs) +bucket[128] width=64ns boundary=[1.984µs, 2.048µs) +bucket[129] width=128ns boundary=[2.048µs, 2.176µs) +bucket[130] width=128ns boundary=[2.176µs, 2.304µs) +bucket[131] width=128ns boundary=[2.304µs, 2.432µs) +bucket[132] width=128ns boundary=[2.432µs, 2.56µs) +bucket[133] width=128ns boundary=[2.56µs, 2.688µs) +bucket[134] width=128ns boundary=[2.688µs, 2.816µs) +bucket[135] width=128ns boundary=[2.816µs, 2.944µs) +bucket[136] width=128ns boundary=[2.944µs, 3.072µs) +bucket[137] width=128ns boundary=[3.072µs, 3.2µs) +bucket[138] width=128ns boundary=[3.2µs, 3.328µs) +bucket[139] width=128ns boundary=[3.328µs, 3.456µs) +bucket[140] width=128ns boundary=[3.456µs, 3.584µs) +bucket[141] width=128ns boundary=[3.584µs, 3.712µs) +bucket[142] width=127ns boundary=[3.712µs, 3.839µs) +bucket[143] width=129ns boundary=[3.839µs, 3.968µs) +bucket[144] width=128ns boundary=[3.968µs, 4.096µs) +bucket[145] width=256ns boundary=[4.096µs, 4.352µs) +bucket[146] width=256ns boundary=[4.352µs, 4.608µs) +bucket[147] width=256ns boundary=[4.608µs, 4.864µs) +bucket[148] width=256ns boundary=[4.864µs, 5.12µs) +bucket[149] width=256ns boundary=[5.12µs, 5.376µs) +bucket[150] width=256ns boundary=[5.376µs, 5.632µs) +bucket[151] width=256ns boundary=[5.632µs, 5.888µs) +bucket[152] width=256ns boundary=[5.888µs, 6.144µs) +bucket[153] width=256ns boundary=[6.144µs, 6.4µs) +bucket[154] width=256ns boundary=[6.4µs, 6.656µs) +bucket[155] width=256ns boundary=[6.656µs, 6.912µs) +bucket[156] width=256ns boundary=[6.912µs, 7.168µs) +bucket[157] width=256ns boundary=[7.168µs, 7.424µs) +bucket[158] width=255ns boundary=[7.424µs, 7.679µs) +bucket[159] width=257ns boundary=[7.679µs, 7.936µs) +bucket[160] width=256ns boundary=[7.936µs, 8.192µs) +bucket[161] width=512ns boundary=[8.192µs, 8.704µs) +bucket[162] width=512ns boundary=[8.704µs, 9.216µs) +bucket[163] width=512ns boundary=[9.216µs, 9.728µs) +bucket[164] width=512ns boundary=[9.728µs, 10.24µs) +bucket[165] width=512ns boundary=[10.24µs, 10.752µs) +bucket[166] width=512ns boundary=[10.752µs, 11.264µs) +bucket[167] width=512ns boundary=[11.264µs, 11.776µs) +bucket[168] width=512ns boundary=[11.776µs, 12.288µs) +bucket[169] width=512ns boundary=[12.288µs, 12.8µs) +bucket[170] width=512ns boundary=[12.8µs, 13.312µs) +bucket[171] width=512ns boundary=[13.312µs, 13.824µs) +bucket[172] width=512ns boundary=[13.824µs, 14.336µs) +bucket[173] width=512ns boundary=[14.336µs, 14.848µs) +bucket[174] width=511ns boundary=[14.848µs, 15.359µs) +bucket[175] width=513ns boundary=[15.359µs, 15.872µs) +bucket[176] width=512ns boundary=[15.872µs, 16.384µs) +bucket[177] width=1.024µs boundary=[16.384µs, 17.408µs) +bucket[178] width=1.024µs boundary=[17.408µs, 18.432µs) +bucket[179] width=1.024µs boundary=[18.432µs, 19.456µs) +bucket[180] width=1.024µs boundary=[19.456µs, 20.48µs) +bucket[181] width=1.024µs boundary=[20.48µs, 21.504µs) +bucket[182] width=1.024µs boundary=[21.504µs, 22.528µs) +bucket[183] width=1.024µs boundary=[22.528µs, 23.552µs) +bucket[184] width=1.024µs boundary=[23.552µs, 24.576µs) +bucket[185] width=1.024µs boundary=[24.576µs, 25.6µs) +bucket[186] width=1.024µs boundary=[25.6µs, 26.624µs) +bucket[187] width=1.024µs boundary=[26.624µs, 27.648µs) +bucket[188] width=1.024µs boundary=[27.648µs, 28.672µs) +bucket[189] width=1.024µs boundary=[28.672µs, 29.696µs) +bucket[190] width=1.023µs boundary=[29.696µs, 30.719µs) +bucket[191] width=1.025µs boundary=[30.719µs, 31.744µs) +bucket[192] width=1.024µs boundary=[31.744µs, 32.768µs) +bucket[193] width=2.048µs boundary=[32.768µs, 34.816µs) +bucket[194] width=2.048µs boundary=[34.816µs, 36.864µs) +bucket[195] width=2.048µs boundary=[36.864µs, 38.912µs) +bucket[196] width=2.048µs boundary=[38.912µs, 40.96µs) +bucket[197] width=2.048µs boundary=[40.96µs, 43.008µs) +bucket[198] width=2.048µs boundary=[43.008µs, 45.056µs) +bucket[199] width=2.048µs boundary=[45.056µs, 47.104µs) +bucket[200] width=2.048µs boundary=[47.104µs, 49.152µs) +bucket[201] width=2.048µs boundary=[49.152µs, 51.2µs) +bucket[202] width=2.048µs boundary=[51.2µs, 53.248µs) +bucket[203] width=2.048µs boundary=[53.248µs, 55.296µs) +bucket[204] width=2.048µs boundary=[55.296µs, 57.344µs) +bucket[205] width=2.048µs boundary=[57.344µs, 59.392µs) +bucket[206] width=2.047µs boundary=[59.392µs, 61.439µs) +bucket[207] width=2.049µs boundary=[61.439µs, 63.488µs) +bucket[208] width=2.048µs boundary=[63.488µs, 65.536µs) +bucket[209] width=4.096µs boundary=[65.536µs, 69.632µs) +bucket[210] width=4.096µs boundary=[69.632µs, 73.728µs) +bucket[211] width=4.096µs boundary=[73.728µs, 77.824µs) +bucket[212] width=4.096µs boundary=[77.824µs, 81.92µs) +bucket[213] width=4.096µs boundary=[81.92µs, 86.016µs) +bucket[214] width=4.096µs boundary=[86.016µs, 90.112µs) +bucket[215] width=4.096µs boundary=[90.112µs, 94.208µs) +bucket[216] width=4.096µs boundary=[94.208µs, 98.304µs) +bucket[217] width=4.096µs boundary=[98.304µs, 102.4µs) +bucket[218] width=4.096µs boundary=[102.4µs, 106.496µs) +bucket[219] width=4.096µs boundary=[106.496µs, 110.592µs) +bucket[220] width=4.096µs boundary=[110.592µs, 114.688µs) +bucket[221] width=4.096µs boundary=[114.688µs, 118.784µs) +bucket[222] width=4.095µs boundary=[118.784µs, 122.879µs) +bucket[223] width=4.097µs boundary=[122.879µs, 126.976µs) +bucket[224] width=4.096µs boundary=[126.976µs, 131.072µs) +bucket[225] width=8.192µs boundary=[131.072µs, 139.264µs) +bucket[226] width=8.192µs boundary=[139.264µs, 147.456µs) +bucket[227] width=8.192µs boundary=[147.456µs, 155.648µs) +bucket[228] width=8.192µs boundary=[155.648µs, 163.84µs) +bucket[229] width=8.192µs boundary=[163.84µs, 172.032µs) +bucket[230] width=8.192µs boundary=[172.032µs, 180.224µs) +bucket[231] width=8.192µs boundary=[180.224µs, 188.416µs) +bucket[232] width=8.192µs boundary=[188.416µs, 196.608µs) +bucket[233] width=8.192µs boundary=[196.608µs, 204.8µs) +bucket[234] width=8.192µs boundary=[204.8µs, 212.992µs) +bucket[235] width=8.192µs boundary=[212.992µs, 221.184µs) +bucket[236] width=8.192µs boundary=[221.184µs, 229.376µs) +bucket[237] width=8.192µs boundary=[229.376µs, 237.568µs) +bucket[238] width=8.191µs boundary=[237.568µs, 245.759µs) +bucket[239] width=8.193µs boundary=[245.759µs, 253.952µs) +bucket[240] width=8.192µs boundary=[253.952µs, 262.144µs) +bucket[241] width=16.384µs boundary=[262.144µs, 278.528µs) +bucket[242] width=16.384µs boundary=[278.528µs, 294.912µs) +bucket[243] width=16.384µs boundary=[294.912µs, 311.296µs) +bucket[244] width=16.384µs boundary=[311.296µs, 327.68µs) +bucket[245] width=16.384µs boundary=[327.68µs, 344.064µs) +bucket[246] width=16.384µs boundary=[344.064µs, 360.448µs) +bucket[247] width=16.384µs boundary=[360.448µs, 376.832µs) +bucket[248] width=16.384µs boundary=[376.832µs, 393.216µs) +bucket[249] width=16.384µs boundary=[393.216µs, 409.6µs) +bucket[250] width=16.384µs boundary=[409.6µs, 425.984µs) +bucket[251] width=16.384µs boundary=[425.984µs, 442.368µs) +bucket[252] width=16.384µs boundary=[442.368µs, 458.752µs) +bucket[253] width=16.384µs boundary=[458.752µs, 475.136µs) +bucket[254] width=16.383µs boundary=[475.136µs, 491.519µs) +bucket[255] width=16.385µs boundary=[491.519µs, 507.904µs) +bucket[256] width=16.384µs boundary=[507.904µs, 524.288µs) +bucket[257] width=32.768µs boundary=[524.288µs, 557.056µs) +bucket[258] width=32.768µs boundary=[557.056µs, 589.824µs) +bucket[259] width=32.768µs boundary=[589.824µs, 622.592µs) +bucket[260] width=32.768µs boundary=[622.592µs, 655.36µs) +bucket[261] width=32.768µs boundary=[655.36µs, 688.128µs) +bucket[262] width=32.768µs boundary=[688.128µs, 720.896µs) +bucket[263] width=32.768µs boundary=[720.896µs, 753.664µs) +bucket[264] width=32.768µs boundary=[753.664µs, 786.432µs) +bucket[265] width=32.768µs boundary=[786.432µs, 819.2µs) +bucket[266] width=32.768µs boundary=[819.2µs, 851.968µs) +bucket[267] width=32.768µs boundary=[851.968µs, 884.736µs) +bucket[268] width=32.768µs boundary=[884.736µs, 917.504µs) +bucket[269] width=32.768µs boundary=[917.504µs, 950.272µs) +bucket[270] width=32.767µs boundary=[950.272µs, 983.039µs) +bucket[271] width=32.769µs boundary=[983.039µs, 1.015808ms) +bucket[272] width=32.768µs boundary=[1.015808ms, 1.048576ms) +bucket[273] width=65.536µs boundary=[1.048576ms, 1.114112ms) +bucket[274] width=65.536µs boundary=[1.114112ms, 1.179648ms) +bucket[275] width=65.536µs boundary=[1.179648ms, 1.245184ms) +bucket[276] width=65.536µs boundary=[1.245184ms, 1.31072ms) +bucket[277] width=65.536µs boundary=[1.31072ms, 1.376256ms) +bucket[278] width=65.536µs boundary=[1.376256ms, 1.441792ms) +bucket[279] width=65.536µs boundary=[1.441792ms, 1.507328ms) +bucket[280] width=65.536µs boundary=[1.507328ms, 1.572864ms) +bucket[281] width=65.536µs boundary=[1.572864ms, 1.6384ms) +bucket[282] width=65.536µs boundary=[1.6384ms, 1.703936ms) +bucket[283] width=65.536µs boundary=[1.703936ms, 1.769472ms) +bucket[284] width=65.536µs boundary=[1.769472ms, 1.835008ms) +bucket[285] width=65.536µs boundary=[1.835008ms, 1.900544ms) +bucket[286] width=65.535µs boundary=[1.900544ms, 1.966079ms) +bucket[287] width=65.537µs boundary=[1.966079ms, 2.031616ms) +bucket[288] width=65.536µs boundary=[2.031616ms, 2.097152ms) +bucket[289] width=131.072µs boundary=[2.097152ms, 2.228224ms) +bucket[290] width=131.072µs boundary=[2.228224ms, 2.359296ms) +bucket[291] width=131.072µs boundary=[2.359296ms, 2.490368ms) +bucket[292] width=131.072µs boundary=[2.490368ms, 2.62144ms) +bucket[293] width=131.072µs boundary=[2.62144ms, 2.752512ms) +bucket[294] width=131.072µs boundary=[2.752512ms, 2.883584ms) +bucket[295] width=131.072µs boundary=[2.883584ms, 3.014656ms) +bucket[296] width=131.072µs boundary=[3.014656ms, 3.145728ms) +bucket[297] width=131.072µs boundary=[3.145728ms, 3.2768ms) +bucket[298] width=131.072µs boundary=[3.2768ms, 3.407872ms) +bucket[299] width=131.072µs boundary=[3.407872ms, 3.538944ms) +bucket[300] width=131.072µs boundary=[3.538944ms, 3.670016ms) +bucket[301] width=131.072µs boundary=[3.670016ms, 3.801088ms) +bucket[302] width=131.071µs boundary=[3.801088ms, 3.932159ms) +bucket[303] width=131.073µs boundary=[3.932159ms, 4.063232ms) +bucket[304] width=131.072µs boundary=[4.063232ms, 4.194304ms) +bucket[305] width=262.144µs boundary=[4.194304ms, 4.456448ms) +bucket[306] width=262.144µs boundary=[4.456448ms, 4.718592ms) +bucket[307] width=262.144µs boundary=[4.718592ms, 4.980736ms) +bucket[308] width=262.144µs boundary=[4.980736ms, 5.24288ms) +bucket[309] width=262.144µs boundary=[5.24288ms, 5.505024ms) +bucket[310] width=262.144µs boundary=[5.505024ms, 5.767168ms) +bucket[311] width=262.144µs boundary=[5.767168ms, 6.029312ms) +bucket[312] width=262.144µs boundary=[6.029312ms, 6.291456ms) +bucket[313] width=262.144µs boundary=[6.291456ms, 6.5536ms) +bucket[314] width=262.144µs boundary=[6.5536ms, 6.815744ms) +bucket[315] width=262.144µs boundary=[6.815744ms, 7.077888ms) +bucket[316] width=262.144µs boundary=[7.077888ms, 7.340032ms) +bucket[317] width=262.144µs boundary=[7.340032ms, 7.602176ms) +bucket[318] width=262.143µs boundary=[7.602176ms, 7.864319ms) +bucket[319] width=262.145µs boundary=[7.864319ms, 8.126464ms) +bucket[320] width=262.144µs boundary=[8.126464ms, 8.388608ms) +bucket[321] width=524.288µs boundary=[8.388608ms, 8.912896ms) +bucket[322] width=524.288µs boundary=[8.912896ms, 9.437184ms) +bucket[323] width=524.288µs boundary=[9.437184ms, 9.961472ms) +bucket[324] width=524.288µs boundary=[9.961472ms, 10.48576ms) +bucket[325] width=524.288µs boundary=[10.48576ms, 11.010048ms) +bucket[326] width=524.288µs boundary=[11.010048ms, 11.534336ms) +bucket[327] width=524.288µs boundary=[11.534336ms, 12.058624ms) +bucket[328] width=524.288µs boundary=[12.058624ms, 12.582912ms) +bucket[329] width=524.288µs boundary=[12.582912ms, 13.1072ms) +bucket[330] width=524.288µs boundary=[13.1072ms, 13.631488ms) +bucket[331] width=524.288µs boundary=[13.631488ms, 14.155776ms) +bucket[332] width=524.288µs boundary=[14.155776ms, 14.680064ms) +bucket[333] width=524.288µs boundary=[14.680064ms, 15.204352ms) +bucket[334] width=524.287µs boundary=[15.204352ms, 15.728639ms) +bucket[335] width=524.289µs boundary=[15.728639ms, 16.252928ms) +bucket[336] width=524.288µs boundary=[16.252928ms, 16.777216ms) +bucket[337] width=1.048576ms boundary=[16.777216ms, 17.825792ms) +bucket[338] width=1.048576ms boundary=[17.825792ms, 18.874368ms) +bucket[339] width=1.048576ms boundary=[18.874368ms, 19.922944ms) +bucket[340] width=1.048576ms boundary=[19.922944ms, 20.97152ms) +bucket[341] width=1.048576ms boundary=[20.97152ms, 22.020096ms) +bucket[342] width=1.048576ms boundary=[22.020096ms, 23.068672ms) +bucket[343] width=1.048576ms boundary=[23.068672ms, 24.117248ms) +bucket[344] width=1.048576ms boundary=[24.117248ms, 25.165824ms) +bucket[345] width=1.048576ms boundary=[25.165824ms, 26.2144ms) +bucket[346] width=1.048576ms boundary=[26.2144ms, 27.262976ms) +bucket[347] width=1.048576ms boundary=[27.262976ms, 28.311552ms) +bucket[348] width=1.048576ms boundary=[28.311552ms, 29.360128ms) +bucket[349] width=1.048576ms boundary=[29.360128ms, 30.408704ms) +bucket[350] width=1.048575ms boundary=[30.408704ms, 31.457279ms) +bucket[351] width=1.048577ms boundary=[31.457279ms, 32.505856ms) +bucket[352] width=1.048576ms boundary=[32.505856ms, 33.554432ms) +bucket[353] width=2.097152ms boundary=[33.554432ms, 35.651584ms) +bucket[354] width=2.097152ms boundary=[35.651584ms, 37.748736ms) +bucket[355] width=2.097152ms boundary=[37.748736ms, 39.845888ms) +bucket[356] width=2.097152ms boundary=[39.845888ms, 41.94304ms) +bucket[357] width=2.097152ms boundary=[41.94304ms, 44.040192ms) +bucket[358] width=2.097152ms boundary=[44.040192ms, 46.137344ms) +bucket[359] width=2.097152ms boundary=[46.137344ms, 48.234496ms) +bucket[360] width=2.097152ms boundary=[48.234496ms, 50.331648ms) +bucket[361] width=2.097152ms boundary=[50.331648ms, 52.4288ms) +bucket[362] width=2.097152ms boundary=[52.4288ms, 54.525952ms) +bucket[363] width=2.097152ms boundary=[54.525952ms, 56.623104ms) +bucket[364] width=2.097152ms boundary=[56.623104ms, 58.720256ms) +bucket[365] width=2.097152ms boundary=[58.720256ms, 60.817408ms) +bucket[366] width=2.097151ms boundary=[60.817408ms, 62.914559ms) +bucket[367] width=2.097153ms boundary=[62.914559ms, 65.011712ms) +bucket[368] width=2.097152ms boundary=[65.011712ms, 67.108864ms) +bucket[369] width=4.194304ms boundary=[67.108864ms, 71.303168ms) +bucket[370] width=4.194304ms boundary=[71.303168ms, 75.497472ms) +bucket[371] width=4.194304ms boundary=[75.497472ms, 79.691776ms) +bucket[372] width=4.194304ms boundary=[79.691776ms, 83.88608ms) +bucket[373] width=4.194304ms boundary=[83.88608ms, 88.080384ms) +bucket[374] width=4.194304ms boundary=[88.080384ms, 92.274688ms) +bucket[375] width=4.194304ms boundary=[92.274688ms, 96.468992ms) +bucket[376] width=4.194304ms boundary=[96.468992ms, 100.663296ms) +bucket[377] width=4.194304ms boundary=[100.663296ms, 104.8576ms) +bucket[378] width=4.194304ms boundary=[104.8576ms, 109.051904ms) +bucket[379] width=4.194304ms boundary=[109.051904ms, 113.246208ms) +bucket[380] width=4.194304ms boundary=[113.246208ms, 117.440512ms) +bucket[381] width=4.194304ms boundary=[117.440512ms, 121.634816ms) +bucket[382] width=4.194303ms boundary=[121.634816ms, 125.829119ms) +bucket[383] width=4.194305ms boundary=[125.829119ms, 130.023424ms) +bucket[384] width=4.194304ms boundary=[130.023424ms, 134.217728ms) +bucket[385] width=8.388608ms boundary=[134.217728ms, 142.606336ms) +bucket[386] width=8.388608ms boundary=[142.606336ms, 150.994944ms) +bucket[387] width=8.388608ms boundary=[150.994944ms, 159.383552ms) +bucket[388] width=8.388608ms boundary=[159.383552ms, 167.77216ms) +bucket[389] width=8.388608ms boundary=[167.77216ms, 176.160768ms) +bucket[390] width=8.388608ms boundary=[176.160768ms, 184.549376ms) +bucket[391] width=8.388608ms boundary=[184.549376ms, 192.937984ms) +bucket[392] width=8.388608ms boundary=[192.937984ms, 201.326592ms) +bucket[393] width=8.388608ms boundary=[201.326592ms, 209.7152ms) +bucket[394] width=8.388608ms boundary=[209.7152ms, 218.103808ms) +bucket[395] width=8.388608ms boundary=[218.103808ms, 226.492416ms) +bucket[396] width=8.388608ms boundary=[226.492416ms, 234.881024ms) +bucket[397] width=8.388608ms boundary=[234.881024ms, 243.269632ms) +bucket[398] width=8.388607ms boundary=[243.269632ms, 251.658239ms) +bucket[399] width=8.388609ms boundary=[251.658239ms, 260.046848ms) +bucket[400] width=8.388608ms boundary=[260.046848ms, 268.435456ms) +bucket[401] width=16.777216ms boundary=[268.435456ms, 285.212672ms) +bucket[402] width=16.777216ms boundary=[285.212672ms, 301.989888ms) +bucket[403] width=16.777216ms boundary=[301.989888ms, 318.767104ms) +bucket[404] width=16.777216ms boundary=[318.767104ms, 335.54432ms) +bucket[405] width=16.777216ms boundary=[335.54432ms, 352.321536ms) +bucket[406] width=16.777216ms boundary=[352.321536ms, 369.098752ms) +bucket[407] width=16.777216ms boundary=[369.098752ms, 385.875968ms) +bucket[408] width=16.777216ms boundary=[385.875968ms, 402.653184ms) +bucket[409] width=16.777216ms boundary=[402.653184ms, 419.4304ms) +bucket[410] width=16.777216ms boundary=[419.4304ms, 436.207616ms) +bucket[411] width=16.777216ms boundary=[436.207616ms, 452.984832ms) +bucket[412] width=16.777216ms boundary=[452.984832ms, 469.762048ms) +bucket[413] width=16.777216ms boundary=[469.762048ms, 486.539264ms) +bucket[414] width=16.777215ms boundary=[486.539264ms, 503.316479ms) +bucket[415] width=16.777217ms boundary=[503.316479ms, 520.093696ms) +bucket[416] width=16.777216ms boundary=[520.093696ms, 536.870912ms) +bucket[417] width=33.554432ms boundary=[536.870912ms, 570.425344ms) +bucket[418] width=33.554432ms boundary=[570.425344ms, 603.979776ms) +bucket[419] width=33.554432ms boundary=[603.979776ms, 637.534208ms) +bucket[420] width=33.554432ms boundary=[637.534208ms, 671.08864ms) +bucket[421] width=33.554432ms boundary=[671.08864ms, 704.643072ms) +bucket[422] width=33.554432ms boundary=[704.643072ms, 738.197504ms) +bucket[423] width=33.554432ms boundary=[738.197504ms, 771.751936ms) +bucket[424] width=33.554432ms boundary=[771.751936ms, 805.306368ms) +bucket[425] width=33.554432ms boundary=[805.306368ms, 838.8608ms) +bucket[426] width=33.554432ms boundary=[838.8608ms, 872.415232ms) +bucket[427] width=33.554432ms boundary=[872.415232ms, 905.969664ms) +bucket[428] width=33.554432ms boundary=[905.969664ms, 939.524096ms) +bucket[429] width=33.554432ms boundary=[939.524096ms, 973.078528ms) +bucket[430] width=33.554431ms boundary=[973.078528ms, 1.006632959s) +bucket[431] width=33.554433ms boundary=[1.006632959s, 1.040187392s) +bucket[432] width=33.554432ms boundary=[1.040187392s, 1.073741824s) +bucket[433] width=67.108864ms boundary=[1.073741824s, 1.140850688s) +bucket[434] width=67.108864ms boundary=[1.140850688s, 1.207959552s) +bucket[435] width=67.108864ms boundary=[1.207959552s, 1.275068416s) +bucket[436] width=67.108864ms boundary=[1.275068416s, 1.34217728s) +bucket[437] width=67.108864ms boundary=[1.34217728s, 1.409286144s) +bucket[438] width=67.108864ms boundary=[1.409286144s, 1.476395008s) +bucket[439] width=67.108864ms boundary=[1.476395008s, 1.543503872s) +bucket[440] width=67.108864ms boundary=[1.543503872s, 1.610612736s) +bucket[441] width=67.108864ms boundary=[1.610612736s, 1.6777216s) +bucket[442] width=67.108864ms boundary=[1.6777216s, 1.744830464s) +bucket[443] width=67.108864ms boundary=[1.744830464s, 1.811939328s) +bucket[444] width=67.108864ms boundary=[1.811939328s, 1.879048192s) +bucket[445] width=67.108864ms boundary=[1.879048192s, 1.946157056s) +bucket[446] width=67.108863ms boundary=[1.946157056s, 2.013265919s) +bucket[447] width=67.108865ms boundary=[2.013265919s, 2.080374784s) +bucket[448] width=67.108864ms boundary=[2.080374784s, 2.147483648s) +bucket[449] width=134.217728ms boundary=[2.147483648s, 2.281701376s) +bucket[450] width=134.217728ms boundary=[2.281701376s, 2.415919104s) +bucket[451] width=134.217728ms boundary=[2.415919104s, 2.550136832s) +bucket[452] width=134.217728ms boundary=[2.550136832s, 2.68435456s) +bucket[453] width=134.217728ms boundary=[2.68435456s, 2.818572288s) +bucket[454] width=134.217728ms boundary=[2.818572288s, 2.952790016s) +bucket[455] width=134.217728ms boundary=[2.952790016s, 3.087007744s) +bucket[456] width=134.217728ms boundary=[3.087007744s, 3.221225472s) +bucket[457] width=134.217728ms boundary=[3.221225472s, 3.3554432s) +bucket[458] width=134.217728ms boundary=[3.3554432s, 3.489660928s) +bucket[459] width=134.217728ms boundary=[3.489660928s, 3.623878656s) +bucket[460] width=134.217728ms boundary=[3.623878656s, 3.758096384s) +bucket[461] width=134.217728ms boundary=[3.758096384s, 3.892314112s) +bucket[462] width=134.217727ms boundary=[3.892314112s, 4.026531839s) +bucket[463] width=134.217729ms boundary=[4.026531839s, 4.160749568s) +bucket[464] width=134.217728ms boundary=[4.160749568s, 4.294967296s) +bucket[465] width=268.435456ms boundary=[4.294967296s, 4.563402752s) +bucket[466] width=268.435456ms boundary=[4.563402752s, 4.831838208s) +bucket[467] width=268.435456ms boundary=[4.831838208s, 5.100273664s) +bucket[468] width=268.435456ms boundary=[5.100273664s, 5.36870912s) +bucket[469] width=268.435456ms boundary=[5.36870912s, 5.637144576s) +bucket[470] width=268.435456ms boundary=[5.637144576s, 5.905580032s) +bucket[471] width=268.435456ms boundary=[5.905580032s, 6.174015488s) +bucket[472] width=268.435456ms boundary=[6.174015488s, 6.442450944s) +bucket[473] width=268.435456ms boundary=[6.442450944s, 6.7108864s) +bucket[474] width=268.435456ms boundary=[6.7108864s, 6.979321856s) +bucket[475] width=268.435456ms boundary=[6.979321856s, 7.247757312s) +bucket[476] width=268.435456ms boundary=[7.247757312s, 7.516192768s) +bucket[477] width=268.435456ms boundary=[7.516192768s, 7.784628224s) +bucket[478] width=268.435455ms boundary=[7.784628224s, 8.053063679s) +bucket[479] width=268.435457ms boundary=[8.053063679s, 8.321499136s) +bucket[480] width=268.435456ms boundary=[8.321499136s, 8.589934592s) +bucket[481] width=536.870912ms boundary=[8.589934592s, 9.126805504s) +bucket[482] width=536.870912ms boundary=[9.126805504s, 9.663676416s) +bucket[483] width=536.870912ms boundary=[9.663676416s, 10.200547328s) +bucket[484] width=536.870912ms boundary=[10.200547328s, 10.73741824s) +bucket[485] width=536.870912ms boundary=[10.73741824s, 11.274289152s) +bucket[486] width=536.870912ms boundary=[11.274289152s, 11.811160064s) +bucket[487] width=536.870912ms boundary=[11.811160064s, 12.348030976s) +bucket[488] width=536.870912ms boundary=[12.348030976s, 12.884901888s) +bucket[489] width=536.870912ms boundary=[12.884901888s, 13.4217728s) +bucket[490] width=536.870912ms boundary=[13.4217728s, 13.958643712s) +bucket[491] width=536.870912ms boundary=[13.958643712s, 14.495514624s) +bucket[492] width=536.870912ms boundary=[14.495514624s, 15.032385536s) +bucket[493] width=536.870912ms boundary=[15.032385536s, 15.569256448s) +bucket[494] width=536.870911ms boundary=[15.569256448s, 16.106127359s) +bucket[495] width=536.870913ms boundary=[16.106127359s, 16.642998272s) +bucket[496] width=536.870912ms boundary=[16.642998272s, 17.179869184s) +bucket[497] width=1.073741824s boundary=[17.179869184s, 18.253611008s) +bucket[498] width=1.073741824s boundary=[18.253611008s, 19.327352832s) +bucket[499] width=1.073741824s boundary=[19.327352832s, 20.401094656s) +bucket[500] width=1.073741824s boundary=[20.401094656s, 21.47483648s) +bucket[501] width=1.073741824s boundary=[21.47483648s, 22.548578304s) +bucket[502] width=1.073741824s boundary=[22.548578304s, 23.622320128s) +bucket[503] width=1.073741824s boundary=[23.622320128s, 24.696061952s) +bucket[504] width=1.073741824s boundary=[24.696061952s, 25.769803776s) +bucket[505] width=1.073741824s boundary=[25.769803776s, 26.8435456s) +bucket[506] width=1.073741824s boundary=[26.8435456s, 27.917287424s) +bucket[507] width=1.073741824s boundary=[27.917287424s, 28.991029248s) +bucket[508] width=1.073741824s boundary=[28.991029248s, 30.064771072s) +bucket[509] width=1.073741824s boundary=[30.064771072s, 31.138512896s) +bucket[510] width=1.073741823s boundary=[31.138512896s, 32.212254719s) +bucket[511] width=1.073741825s boundary=[32.212254719s, 33.285996544s) +bucket[512] width=1.073741824s boundary=[33.285996544s, 34.359738368s) +bucket[513] width=2.147483648s boundary=[34.359738368s, 36.507222016s) +bucket[514] width=2.147483648s boundary=[36.507222016s, 38.654705664s) +bucket[515] width=2.147483648s boundary=[38.654705664s, 40.802189312s) +bucket[516] width=2.147483648s boundary=[40.802189312s, 42.94967296s) +bucket[517] width=2.147483648s boundary=[42.94967296s, 45.097156608s) +bucket[518] width=2.147483648s boundary=[45.097156608s, 47.244640256s) +bucket[519] width=2.147483648s boundary=[47.244640256s, 49.392123904s) +bucket[520] width=2.147483648s boundary=[49.392123904s, 51.539607552s) +bucket[521] width=2.147483648s boundary=[51.539607552s, 53.6870912s) +bucket[522] width=2.147483648s boundary=[53.6870912s, 55.834574848s) +bucket[523] width=2.147483648s boundary=[55.834574848s, 57.982058496s) +bucket[524] width=2.147483648s boundary=[57.982058496s, 1m0.129542144s) +bucket[525] width=2.147483648s boundary=[1m0.129542144s, 1m2.277025792s) +bucket[526] width=2.147483647s boundary=[1m2.277025792s, 1m4.424509439s) +bucket[527] width=2.147483649s boundary=[1m4.424509439s, 1m6.571993088s) +bucket[528] width=2.147483648s boundary=[1m6.571993088s, 1m8.719476736s) +bucket[529] width=4.294967296s boundary=[1m8.719476736s, 1m13.014444032s) +bucket[530] width=4.294967296s boundary=[1m13.014444032s, 1m17.309411328s) +bucket[531] width=4.294967296s boundary=[1m17.309411328s, 1m21.604378624s) +bucket[532] width=4.294967296s boundary=[1m21.604378624s, 1m25.89934592s) +bucket[533] width=4.294967296s boundary=[1m25.89934592s, 1m30.194313216s) +bucket[534] width=4.294967296s boundary=[1m30.194313216s, 1m34.489280512s) +bucket[535] width=4.294967296s boundary=[1m34.489280512s, 1m38.784247808s) +bucket[536] width=4.294967296s boundary=[1m38.784247808s, 1m43.079215104s) +bucket[537] width=4.294967296s boundary=[1m43.079215104s, 1m47.3741824s) +bucket[538] width=4.294967296s boundary=[1m47.3741824s, 1m51.669149696s) +bucket[539] width=4.294967296s boundary=[1m51.669149696s, 1m55.964116992s) +bucket[540] width=4.294967296s boundary=[1m55.964116992s, 2m0.259084288s) +bucket[541] width=4.294967296s boundary=[2m0.259084288s, 2m4.554051584s) +bucket[542] width=4.294967295s boundary=[2m4.554051584s, 2m8.849018879s) +bucket[543] width=4.294967297s boundary=[2m8.849018879s, 2m13.143986176s) +bucket[544] width=4.294967296s boundary=[2m13.143986176s, 2m17.438953472s) +bucket[545] width=8.589934592s boundary=[2m17.438953472s, 2m26.028888064s) +bucket[546] width=8.589934592s boundary=[2m26.028888064s, 2m34.618822656s) +bucket[547] width=8.589934592s boundary=[2m34.618822656s, 2m43.208757248s) +bucket[548] width=8.589934592s boundary=[2m43.208757248s, 2m51.79869184s) +bucket[549] width=8.589934592s boundary=[2m51.79869184s, 3m0.388626432s) +bucket[550] width=8.589934592s boundary=[3m0.388626432s, 3m8.978561024s) +bucket[551] width=8.589934592s boundary=[3m8.978561024s, 3m17.568495616s) +bucket[552] width=8.589934592s boundary=[3m17.568495616s, 3m26.158430208s) +bucket[553] width=8.589934592s boundary=[3m26.158430208s, 3m34.7483648s) +bucket[554] width=8.589934592s boundary=[3m34.7483648s, 3m43.338299392s) +bucket[555] width=8.589934592s boundary=[3m43.338299392s, 3m51.928233984s) +bucket[556] width=8.589934592s boundary=[3m51.928233984s, 4m0.518168576s) +bucket[557] width=8.589934592s boundary=[4m0.518168576s, 4m9.108103168s) +bucket[558] width=8.589934591s boundary=[4m9.108103168s, 4m17.698037759s) +bucket[559] width=8.589934593s boundary=[4m17.698037759s, 4m26.287972352s) +bucket[560] width=8.589934592s boundary=[4m26.287972352s, 4m34.877906944s) +bucket[561] width=17.179869184s boundary=[4m34.877906944s, 4m52.057776128s) +bucket[562] width=17.179869184s boundary=[4m52.057776128s, 5m9.237645312s) +bucket[563] width=17.179869184s boundary=[5m9.237645312s, 5m26.417514496s) +bucket[564] width=17.179869184s boundary=[5m26.417514496s, 5m43.59738368s) +bucket[565] width=17.179869184s boundary=[5m43.59738368s, 6m0.777252864s) +bucket[566] width=17.179869184s boundary=[6m0.777252864s, 6m17.957122048s) +bucket[567] width=17.179869184s boundary=[6m17.957122048s, 6m35.136991232s) +bucket[568] width=17.179869184s boundary=[6m35.136991232s, 6m52.316860416s) +bucket[569] width=17.179869184s boundary=[6m52.316860416s, 7m9.4967296s) +bucket[570] width=17.179869184s boundary=[7m9.4967296s, 7m26.676598784s) +bucket[571] width=17.179869184s boundary=[7m26.676598784s, 7m43.856467968s) +bucket[572] width=17.179869184s boundary=[7m43.856467968s, 8m1.036337152s) +bucket[573] width=17.179869184s boundary=[8m1.036337152s, 8m18.216206336s) +bucket[574] width=17.179869183s boundary=[8m18.216206336s, 8m35.396075519s) +bucket[575] width=17.179869185s boundary=[8m35.396075519s, 8m52.575944704s) +bucket[576] width=17.179869184s boundary=[8m52.575944704s, 9m9.755813888s) +bucket[577] width=34.359738368s boundary=[9m9.755813888s, 9m44.115552256s) +bucket[578] width=34.359738368s boundary=[9m44.115552256s, 10m18.475290624s) +bucket[579] width=34.359738368s boundary=[10m18.475290624s, 10m52.835028992s) +bucket[580] width=34.359738368s boundary=[10m52.835028992s, 11m27.19476736s) +bucket[581] width=34.359738368s boundary=[11m27.19476736s, 12m1.554505728s) +bucket[582] width=34.359738368s boundary=[12m1.554505728s, 12m35.914244096s) +bucket[583] width=34.359738368s boundary=[12m35.914244096s, 13m10.273982464s) +bucket[584] width=34.359738368s boundary=[13m10.273982464s, 13m44.633720832s) +bucket[585] width=34.359738368s boundary=[13m44.633720832s, 14m18.9934592s) +bucket[586] width=34.359738368s boundary=[14m18.9934592s, 14m53.353197568s) +bucket[587] width=34.359738368s boundary=[14m53.353197568s, 15m27.712935936s) +bucket[588] width=34.359738368s boundary=[15m27.712935936s, 16m2.072674304s) +bucket[589] width=34.359738368s boundary=[16m2.072674304s, 16m36.432412672s) +bucket[590] width=34.359738367s boundary=[16m36.432412672s, 17m10.792151039s) +bucket[591] width=34.359738369s boundary=[17m10.792151039s, 17m45.151889408s) +bucket[592] width=34.359738368s boundary=[17m45.151889408s, 18m19.511627776s) +bucket[593] width=1m8.719476736s boundary=[18m19.511627776s, 19m28.231104512s) +bucket[594] width=1m8.719476736s boundary=[19m28.231104512s, 20m36.950581248s) +bucket[595] width=1m8.719476736s boundary=[20m36.950581248s, 21m45.670057984s) +bucket[596] width=1m8.719476736s boundary=[21m45.670057984s, 22m54.38953472s) +bucket[597] width=1m8.719476736s boundary=[22m54.38953472s, 24m3.109011456s) +bucket[598] width=1m8.719476736s boundary=[24m3.109011456s, 25m11.828488192s) +bucket[599] width=1m8.719476736s boundary=[25m11.828488192s, 26m20.547964928s) +bucket[600] width=1m8.719476736s boundary=[26m20.547964928s, 27m29.267441664s) +bucket[601] width=1m8.719476736s boundary=[27m29.267441664s, 28m37.9869184s) +bucket[602] width=1m8.719476736s boundary=[28m37.9869184s, 29m46.706395136s) +bucket[603] width=1m8.719476736s boundary=[29m46.706395136s, 30m55.425871872s) +bucket[604] width=1m8.719476736s boundary=[30m55.425871872s, 32m4.145348608s) +bucket[605] width=1m8.719476736s boundary=[32m4.145348608s, 33m12.864825344s) +bucket[606] width=1m8.719476735s boundary=[33m12.864825344s, 34m21.584302079s) +bucket[607] width=1m8.719476737s boundary=[34m21.584302079s, 35m30.303778816s) +bucket[608] width=1m8.719476736s boundary=[35m30.303778816s, 36m39.023255552s) +bucket[609] width=2m17.438953472s boundary=[36m39.023255552s, 38m56.462209024s) +bucket[610] width=2m17.438953472s boundary=[38m56.462209024s, 41m13.901162496s) +bucket[611] width=2m17.438953472s boundary=[41m13.901162496s, 43m31.340115968s) +bucket[612] width=2m17.438953472s boundary=[43m31.340115968s, 45m48.77906944s) +bucket[613] width=2m17.438953472s boundary=[45m48.77906944s, 48m6.218022912s) +bucket[614] width=2m17.438953472s boundary=[48m6.218022912s, 50m23.656976384s) +bucket[615] width=2m17.438953472s boundary=[50m23.656976384s, 52m41.095929856s) +bucket[616] width=2m17.438953472s boundary=[52m41.095929856s, 54m58.534883328s) +bucket[617] width=2m17.438953472s boundary=[54m58.534883328s, 57m15.9738368s) +bucket[618] width=2m17.438953472s boundary=[57m15.9738368s, 59m33.412790272s) +bucket[619] width=2m17.438953472s boundary=[59m33.412790272s, 1h1m50.851743744s) +bucket[620] width=2m17.438953472s boundary=[1h1m50.851743744s, 1h4m8.290697216s) +bucket[621] width=2m17.438953472s boundary=[1h4m8.290697216s, 1h6m25.729650688s) +bucket[622] width=2m17.438953471s boundary=[1h6m25.729650688s, 1h8m43.168604159s) +bucket[623] width=2m17.438953473s boundary=[1h8m43.168604159s, 1h11m0.607557632s) +bucket[624] width=2m17.438953472s boundary=[1h11m0.607557632s, 1h13m18.046511104s) +bucket[625] width=4m34.877906944s boundary=[1h13m18.046511104s, 1h17m52.924418048s) +bucket[626] width=4m34.877906944s boundary=[1h17m52.924418048s, 1h22m27.802324992s) +bucket[627] width=4m34.877906944s boundary=[1h22m27.802324992s, 1h27m2.680231936s) +bucket[628] width=4m34.877906944s boundary=[1h27m2.680231936s, 1h31m37.55813888s) +bucket[629] width=4m34.877906944s boundary=[1h31m37.55813888s, 1h36m12.436045824s) +bucket[630] width=4m34.877906944s boundary=[1h36m12.436045824s, 1h40m47.313952768s) +bucket[631] width=4m34.877906944s boundary=[1h40m47.313952768s, 1h45m22.191859712s) +bucket[632] width=4m34.877906944s boundary=[1h45m22.191859712s, 1h49m57.069766656s) +bucket[633] width=4m34.877906944s boundary=[1h49m57.069766656s, 1h54m31.9476736s) +bucket[634] width=4m34.877906944s boundary=[1h54m31.9476736s, 1h59m6.825580544s) +bucket[635] width=4m34.877906944s boundary=[1h59m6.825580544s, 2h3m41.703487488s) +bucket[636] width=4m34.877906944s boundary=[2h3m41.703487488s, 2h8m16.581394432s) +bucket[637] width=4m34.877906944s boundary=[2h8m16.581394432s, 2h12m51.459301376s) +bucket[638] width=4m34.877906943s boundary=[2h12m51.459301376s, 2h17m26.337208319s) +bucket[639] width=4m34.877906945s boundary=[2h17m26.337208319s, 2h22m1.215115264s) +bucket[640] width=4m34.877906944s boundary=[2h22m1.215115264s, 2h26m36.093022208s) +bucket[641] width=9m9.755813888s boundary=[2h26m36.093022208s, 2h35m45.848836096s) +bucket[642] width=9m9.755813888s boundary=[2h35m45.848836096s, 2h44m55.604649984s) +bucket[643] width=9m9.755813888s boundary=[2h44m55.604649984s, 2h54m5.360463872s) +bucket[644] width=9m9.755813888s boundary=[2h54m5.360463872s, 3h3m15.11627776s) +bucket[645] width=9m9.755813888s boundary=[3h3m15.11627776s, 3h12m24.872091648s) +bucket[646] width=9m9.755813888s boundary=[3h12m24.872091648s, 3h21m34.627905536s) +bucket[647] width=9m9.755813888s boundary=[3h21m34.627905536s, 3h30m44.383719424s) +bucket[648] width=9m9.755813888s boundary=[3h30m44.383719424s, 3h39m54.139533312s) +bucket[649] width=9m9.755813888s boundary=[3h39m54.139533312s, 3h49m3.8953472s) +bucket[650] width=9m9.755813888s boundary=[3h49m3.8953472s, 3h58m13.651161088s) +bucket[651] width=9m9.755813888s boundary=[3h58m13.651161088s, 4h7m23.406974976s) +bucket[652] width=9m9.755813888s boundary=[4h7m23.406974976s, 4h16m33.162788864s) +bucket[653] width=9m9.755813888s boundary=[4h16m33.162788864s, 4h25m42.918602752s) +bucket[654] width=9m9.755813887s boundary=[4h25m42.918602752s, 4h34m52.674416639s) +bucket[655] width=9m9.755813889s boundary=[4h34m52.674416639s, 4h44m2.430230528s) +bucket[656] width=9m9.755813888s boundary=[4h44m2.430230528s, 4h53m12.186044416s) +bucket[657] width=18m19.511627776s boundary=[4h53m12.186044416s, 5h11m31.697672192s) +bucket[658] width=18m19.511627776s boundary=[5h11m31.697672192s, 5h29m51.209299968s) +bucket[659] width=18m19.511627776s boundary=[5h29m51.209299968s, 5h48m10.720927744s) +bucket[660] width=18m19.511627776s boundary=[5h48m10.720927744s, 6h6m30.23255552s) +bucket[661] width=18m19.511627776s boundary=[6h6m30.23255552s, 6h24m49.744183296s) +bucket[662] width=18m19.511627776s boundary=[6h24m49.744183296s, 6h43m9.255811072s) +bucket[663] width=18m19.511627776s boundary=[6h43m9.255811072s, 7h1m28.767438848s) +bucket[664] width=18m19.511627776s boundary=[7h1m28.767438848s, 7h19m48.279066624s) +bucket[665] width=18m19.511627776s boundary=[7h19m48.279066624s, 7h38m7.7906944s) +bucket[666] width=18m19.511627776s boundary=[7h38m7.7906944s, 7h56m27.302322176s) +bucket[667] width=18m19.511627776s boundary=[7h56m27.302322176s, 8h14m46.813949952s) +bucket[668] width=18m19.511627776s boundary=[8h14m46.813949952s, 8h33m6.325577728s) +bucket[669] width=18m19.511627776s boundary=[8h33m6.325577728s, 8h51m25.837205504s) +bucket[670] width=18m19.511627775s boundary=[8h51m25.837205504s, 9h9m45.348833279s) +bucket[671] width=18m19.511627777s boundary=[9h9m45.348833279s, 9h28m4.860461056s) +bucket[672] width=18m19.511627776s boundary=[9h28m4.860461056s, 9h46m24.372088832s) +bucket[673] width=36m39.023255552s boundary=[9h46m24.372088832s, 10h23m3.395344384s) +bucket[674] width=36m39.023255552s boundary=[10h23m3.395344384s, 10h59m42.418599936s) +bucket[675] width=36m39.023255552s boundary=[10h59m42.418599936s, 11h36m21.441855488s) +bucket[676] width=36m39.023255552s boundary=[11h36m21.441855488s, 12h13m0.46511104s) +bucket[677] width=36m39.023255552s boundary=[12h13m0.46511104s, 12h49m39.488366592s) +bucket[678] width=36m39.023255552s boundary=[12h49m39.488366592s, 13h26m18.511622144s) +bucket[679] width=36m39.023255552s boundary=[13h26m18.511622144s, 14h2m57.534877696s) +bucket[680] width=36m39.023255552s boundary=[14h2m57.534877696s, 14h39m36.558133248s) +bucket[681] width=36m39.023255552s boundary=[14h39m36.558133248s, 15h16m15.5813888s) +bucket[682] width=36m39.023255552s boundary=[15h16m15.5813888s, 15h52m54.604644352s) +bucket[683] width=36m39.023255552s boundary=[15h52m54.604644352s, 16h29m33.627899904s) +bucket[684] width=36m39.023255552s boundary=[16h29m33.627899904s, 17h6m12.651155456s) +bucket[685] width=36m39.023255552s boundary=[17h6m12.651155456s, 17h42m51.674411008s) +bucket[686] width=36m39.023255551s boundary=[17h42m51.674411008s, 18h19m30.697666559s) +bucket[687] width=36m39.023255553s boundary=[18h19m30.697666559s, 18h56m9.720922112s) +bucket[688] width=36m39.023255552s boundary=[18h56m9.720922112s, 19h32m48.744177664s) +bucket[689] width=1h13m18.046511104s boundary=[19h32m48.744177664s, 20h46m6.790688768s) +bucket[690] width=1h13m18.046511104s boundary=[20h46m6.790688768s, 21h59m24.837199872s) +bucket[691] width=1h13m18.046511104s boundary=[21h59m24.837199872s, 23h12m42.883710976s) +bucket[692] width=1h13m18.046511104s boundary=[23h12m42.883710976s, 24h26m0.93022208s) +bucket[693] width=1h13m18.046511104s boundary=[24h26m0.93022208s, 25h39m18.976733184s) +bucket[694] width=1h13m18.046511104s boundary=[25h39m18.976733184s, 26h52m37.023244288s) +bucket[695] width=1h13m18.046511104s boundary=[26h52m37.023244288s, 28h5m55.069755392s) +bucket[696] width=1h13m18.046511104s boundary=[28h5m55.069755392s, 29h19m13.116266496s) +bucket[697] width=1h13m18.046511104s boundary=[29h19m13.116266496s, 30h32m31.1627776s) +bucket[698] width=1h13m18.046511104s boundary=[30h32m31.1627776s, 31h45m49.209288704s) +bucket[699] width=1h13m18.046511104s boundary=[31h45m49.209288704s, 32h59m7.255799808s) +bucket[700] width=1h13m18.046511104s boundary=[32h59m7.255799808s, 34h12m25.302310912s) +bucket[701] width=1h13m18.046511104s boundary=[34h12m25.302310912s, 35h25m43.348822016s) +bucket[702] width=1h13m18.046511103s boundary=[35h25m43.348822016s, 36h39m1.395333119s) +bucket[703] width=1h13m18.046511105s boundary=[36h39m1.395333119s, 37h52m19.441844224s) +bucket[704] width=1h13m18.046511104s boundary=[37h52m19.441844224s, 39h5m37.488355328s) +bucket[705] width=2h26m36.093022208s boundary=[39h5m37.488355328s, 41h32m13.581377536s) +bucket[706] width=2h26m36.093022208s boundary=[41h32m13.581377536s, 43h58m49.674399744s) +bucket[707] width=2h26m36.093022208s boundary=[43h58m49.674399744s, 46h25m25.767421952s) +bucket[708] width=2h26m36.093022208s boundary=[46h25m25.767421952s, 48h52m1.86044416s) +bucket[709] width=2h26m36.093022208s boundary=[48h52m1.86044416s, 51h18m37.953466368s) +bucket[710] width=2h26m36.093022208s boundary=[51h18m37.953466368s, 53h45m14.046488576s) +bucket[711] width=2h26m36.093022208s boundary=[53h45m14.046488576s, 56h11m50.139510784s) +bucket[712] width=2h26m36.093022208s boundary=[56h11m50.139510784s, 58h38m26.232532992s) +bucket[713] width=2h26m36.093022208s boundary=[58h38m26.232532992s, 61h5m2.3255552s) +bucket[714] width=2h26m36.093022208s boundary=[61h5m2.3255552s, 63h31m38.418577408s) +bucket[715] width=2h26m36.093022208s boundary=[63h31m38.418577408s, 65h58m14.511599616s) +bucket[716] width=2h26m36.093022208s boundary=[65h58m14.511599616s, 68h24m50.604621824s) +bucket[717] width=2h26m36.093022208s boundary=[68h24m50.604621824s, 70h51m26.697644032s) +bucket[718] width=2h26m36.093022207s boundary=[70h51m26.697644032s, 73h18m2.790666239s) +bucket[719] width=2h26m36.093022209s boundary=[73h18m2.790666239s, 75h44m38.883688448s) +bucket[720] width=Inf boundary=[75h44m38.883688448s, +Inf) diff --git a/pkg/util/schedulerlatency/testdata/runtime_histogram b/pkg/util/schedulerlatency/testdata/runtime_histogram new file mode 100644 index 000000000000..f4c8151f5e4b --- /dev/null +++ b/pkg/util/schedulerlatency/testdata/runtime_histogram @@ -0,0 +1,100 @@ +# Explore how the histogram type that wraps around runtime/metrics behaves. +# Initialize it to buckets that have a: +# - width of 1.0 apart in the range [0, 5) +# - width of 5.0 apart in the range [5, 20) +# - width of 10.0 apart in the range [20, 50) +init +bucket=[-inf,0) +bucket=[0,1) +bucket=[1,2) +bucket=[2,3) +bucket=[3,4) +bucket=[4,5) +bucket=[5,10) +bucket=[10,15) +bucket=[15,20) +bucket=[20,30) +bucket=[30,40) +bucket=[40,50) +bucket=[50,+inf) +---- + +# We should see the right cumulative buckets, trimming out +/- inf. +print +---- +count=0 sum=0.00 +buckets: + upper-bound=1.00 cumulative-count=0 + upper-bound=2.00 cumulative-count=0 + upper-bound=3.00 cumulative-count=0 + upper-bound=4.00 cumulative-count=0 + upper-bound=5.00 cumulative-count=0 + upper-bound=10.00 cumulative-count=0 + upper-bound=15.00 cumulative-count=0 + upper-bound=20.00 cumulative-count=0 + upper-bound=30.00 cumulative-count=0 + upper-bound=40.00 cumulative-count=0 + upper-bound=50.00 cumulative-count=0 + +# If updating within a bucket (the histogram is initialized with a subset of +# the buckets updates can have), the right buckets are incremented. We'll +# increment the buckets between [3, 4) by 9 across two sub-increments. +# We'll also increment [10, 15) using data that lies within the range. +update +bucket=[0,1) count=0 +bucket=[1,2) count=0 +bucket=[2,3) count=0 +bucket=[3,3.5) count=5 +bucket=[3.5,4) count=4 +bucket=[4,5) count=0 +bucket=[5,10) count=0 +bucket=[10,11) count=1 +bucket=[11,15) count=0 +---- + +# Observe how the right buckets are incremented and reduced into. +print +---- +count=10 sum=37.00 +buckets: + upper-bound=1.00 cumulative-count=0 + upper-bound=2.00 cumulative-count=0 + upper-bound=3.00 cumulative-count=0 + upper-bound=4.00 cumulative-count=9 + upper-bound=5.00 cumulative-count=9 + upper-bound=10.00 cumulative-count=9 + upper-bound=15.00 cumulative-count=10 + upper-bound=20.00 cumulative-count=10 + upper-bound=30.00 cumulative-count=10 + upper-bound=40.00 cumulative-count=10 + upper-bound=50.00 cumulative-count=10 + +# Since it's a batch histogram, updating it again resets all counts. +update +bucket=[0,1) count=0 +bucket=[1,2) count=0 +bucket=[2,3) count=0 +bucket=[3,4) count=0 +bucket=[4,5) count=0 +bucket=[5,10) count=0 +bucket=[10,15) count=2 +bucket=[15,20) count=1 +---- + +# Observe how the count reflects the cumulative value, while sum is an +# underestimate, computing using start of bucket boundaries: 2*10 + 1*15. +print +---- +count=3 sum=35.00 +buckets: + upper-bound=1.00 cumulative-count=0 + upper-bound=2.00 cumulative-count=0 + upper-bound=3.00 cumulative-count=0 + upper-bound=4.00 cumulative-count=0 + upper-bound=5.00 cumulative-count=0 + upper-bound=10.00 cumulative-count=0 + upper-bound=15.00 cumulative-count=2 + upper-bound=20.00 cumulative-count=3 + upper-bound=30.00 cumulative-count=3 + upper-bound=40.00 cumulative-count=3 + upper-bound=50.00 cumulative-count=3