diff --git a/Makefile b/Makefile index fdbb29b5a8ca..edf48f4ae428 100644 --- a/Makefile +++ b/Makefile @@ -1629,6 +1629,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/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index 0ecbdb630aa5..b7b48274ed8c 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -186,6 +186,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 b11490877a89..965f9b6268e0 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, @@ -3343,6 +3351,17 @@ 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 tick 1 for the same reasons as above. + if logcrash.DiagnosticsReportingEnabled.Get(&s.ClusterSettings().SV) && + tick%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 0b94700f0f49..fc0809d598d2 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -69,6 +69,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 700fa1fa360c..571ddc93d933 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" @@ -1026,6 +1027,64 @@ 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, + } + 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 a62871f3850b..c704ef163f21 100644 --- a/pkg/util/log/eventpb/BUILD.bazel +++ b/pkg/util/log/eventpb/BUILD.bazel @@ -58,6 +58,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 dc68f18aa665..9872cdd0b9f0 100644 --- a/pkg/util/log/eventpb/eventpbgen/gen.go +++ b/pkg/util/log/eventpb/eventpbgen/gen.go @@ -660,6 +660,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 0287b457c364..4c70d3e3516f 100644 --- a/pkg/util/log/eventpb/json_encode_generated.go +++ b/pkg/util/log/eventpb/json_encode_generated.go @@ -2840,6 +2840,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) { @@ -4189,6 +4328,301 @@ 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) + } + + 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..2c24d38a6fd0 --- /dev/null +++ b/pkg/util/log/eventpb/storage_events.proto @@ -0,0 +1,161 @@ +// 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"]; +} + +// 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"]; +}