From 6412ae868aac3591e0583ad65ce413c43da1892e Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Fri, 1 Jul 2022 12:51:04 -0400 Subject: [PATCH] admission,kvserver: broadcast per-store IO overload status The plan for #79215 is to have stores not send to followers that are experiencing IO overload, unless this is necessary for quorum. This commit teaches admission control to relay the current status of the IO overload signals to the Store, which in turn gossips it. We add a type `IOThreshold` that encapsulates the input signals to I/O admission control to present them as a unified score, which hopefully leads to less ad-hoc use of the underlying signals. Together, this paves the way for using the signal to inform distribution decisions (#83490) and (bluntly) shape raft traffic (#79215). Touches #79215. Touches #77604. Touches #82611. Touches #83490. Release note: None --- pkg/gen/protobuf.bzl | 1 + pkg/kv/kvserver/store.go | 19 +++++- pkg/roachpb/BUILD.bazel | 2 + pkg/roachpb/metadata.go | 4 +- pkg/roachpb/metadata.proto | 2 + pkg/server/BUILD.bazel | 1 + pkg/server/node.go | 11 +++ pkg/server/server.go | 2 +- pkg/util/admission/admissionpb/BUILD.bazel | 25 +++++++ .../admission/admissionpb/io_threshold.go | 51 ++++++++++++++ .../admission/admissionpb/io_threshold.proto | 26 +++++++ pkg/util/admission/granter.go | 67 ++++++++++++------- pkg/util/admission/granter_test.go | 19 ++++-- pkg/util/admission/testdata/io_load_listener | 36 +++++----- 14 files changed, 213 insertions(+), 53 deletions(-) create mode 100644 pkg/util/admission/admissionpb/io_threshold.go create mode 100644 pkg/util/admission/admissionpb/io_threshold.proto diff --git a/pkg/gen/protobuf.bzl b/pkg/gen/protobuf.bzl index 698dcc59aa97..b2ff1c497ee2 100644 --- a/pkg/gen/protobuf.bzl +++ b/pkg/gen/protobuf.bzl @@ -54,6 +54,7 @@ PROTOBUF_SRCS = [ "//pkg/testutils/grpcutils:grpcutils_go_proto", "//pkg/ts/catalog:catalog_go_proto", "//pkg/ts/tspb:tspb_go_proto", + "//pkg/util/admission/admissionpb:admissionpb_go_proto", "//pkg/util/duration:duration_go_proto", "//pkg/util/hlc:hlc_go_proto", "//pkg/util/log/eventpb:eventpb_go_proto", diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 928061c0d873..cd389d2a9771 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -938,6 +938,10 @@ type Store struct { syncutil.Mutex roachpb.StoreCapacity } + ioThreshold struct { + syncutil.Mutex + t *admissionpb.IOThreshold // never nil + } counts struct { // Number of placeholders removed due to error. Not a good fit for meaningful @@ -1170,6 +1174,7 @@ func NewStore( metrics: newStoreMetrics(cfg.HistogramWindowInterval), ctSender: cfg.ClosedTimestampSender, } + s.ioThreshold.t = &admissionpb.IOThreshold{} if cfg.RPCContext != nil { s.allocator = allocatorimpl.MakeAllocator(cfg.StorePool, cfg.RPCContext.RemoteClocks.Latency, cfg.TestingKnobs.AllocatorKnobs) } else { @@ -2509,6 +2514,13 @@ func (s *Store) asyncGossipStore(ctx context.Context, reason string, useCached b } } +// UpdateIOThreshold updates the IOThreshold reported in the StoreDescriptor. +func (s *Store) UpdateIOThreshold(ioThreshold *admissionpb.IOThreshold) { + s.ioThreshold.Lock() + defer s.ioThreshold.Unlock() + s.ioThreshold.t = ioThreshold +} + // GossipStore broadcasts the store on the gossip network. func (s *Store) GossipStore(ctx context.Context, useCached bool) error { // Temporarily indicate that we're gossiping the store capacity to avoid @@ -2942,7 +2954,7 @@ func (s *Store) Capacity(ctx context.Context, useCached bool) (roachpb.StoreCapa capacity, err := s.engine.Capacity() if err != nil { - return capacity, err + return roachpb.StoreCapacity{}, err } now := s.cfg.Clock.NowAsClockTimestamp() @@ -2993,6 +3005,11 @@ func (s *Store) Capacity(ctx context.Context, useCached bool) (roachpb.StoreCapa capacity.QueriesPerSecond = totalQueriesPerSecond capacity.WritesPerSecond = totalWritesPerSecond capacity.L0Sublevels = l0SublevelsMax + { + s.ioThreshold.Lock() + capacity.IOThreshold = *s.ioThreshold.t + s.ioThreshold.Unlock() + } capacity.BytesPerReplica = roachpb.PercentilesFromData(bytesPerReplica) capacity.WritesPerReplica = roachpb.PercentilesFromData(writesPerReplica) s.recordNewPerSecondStats(totalQueriesPerSecond, totalWritesPerSecond) diff --git a/pkg/roachpb/BUILD.bazel b/pkg/roachpb/BUILD.bazel index 8ec29dc0ed1b..c4ba10ee9878 100644 --- a/pkg/roachpb/BUILD.bazel +++ b/pkg/roachpb/BUILD.bazel @@ -163,6 +163,7 @@ proto_library( "//pkg/settings:settings_proto", "//pkg/storage/enginepb:enginepb_proto", "//pkg/util:util_proto", + "//pkg/util/admission/admissionpb:admissionpb_proto", "//pkg/util/hlc:hlc_proto", "//pkg/util/tracing/tracingpb:tracingpb_proto", "@com_github_cockroachdb_errors//errorspb:errorspb_proto", @@ -184,6 +185,7 @@ go_proto_library( "//pkg/settings", "//pkg/storage/enginepb", "//pkg/util", + "//pkg/util/admission/admissionpb", "//pkg/util/hlc", "//pkg/util/tracing/tracingpb", "@com_github_cockroachdb_errors//errorspb", diff --git a/pkg/roachpb/metadata.go b/pkg/roachpb/metadata.go index 9f74164bd272..226391d1eb2e 100644 --- a/pkg/roachpb/metadata.go +++ b/pkg/roachpb/metadata.go @@ -577,11 +577,11 @@ func (sc StoreCapacity) String() string { func (sc StoreCapacity) SafeFormat(w redact.SafePrinter, _ rune) { w.Printf("disk (capacity=%s, available=%s, used=%s, logicalBytes=%s), "+ "ranges=%d, leases=%d, queries=%.2f, writes=%.2f, "+ - "l0Sublevels=%d, bytesPerReplica={%s}, writesPerReplica={%s}", + "l0Sublevels=%d, ioThreshold={%v} bytesPerReplica={%s}, writesPerReplica={%s}", humanizeutil.IBytes(sc.Capacity), humanizeutil.IBytes(sc.Available), humanizeutil.IBytes(sc.Used), humanizeutil.IBytes(sc.LogicalBytes), sc.RangeCount, sc.LeaseCount, sc.QueriesPerSecond, sc.WritesPerSecond, - sc.L0Sublevels, sc.BytesPerReplica, sc.WritesPerReplica) + sc.L0Sublevels, sc.IOThreshold, sc.BytesPerReplica, sc.WritesPerReplica) } // FractionUsed computes the fraction of storage capacity that is in use. diff --git a/pkg/roachpb/metadata.proto b/pkg/roachpb/metadata.proto index 7d2e7d865457..8198e63f79c9 100644 --- a/pkg/roachpb/metadata.proto +++ b/pkg/roachpb/metadata.proto @@ -14,6 +14,7 @@ option go_package = "roachpb"; import "util/unresolved_addr.proto"; import "util/hlc/timestamp.proto"; +import "util/admission/admissionpb/io_threshold.proto"; import "gogoproto/gogo.proto"; // Attributes specifies a list of arbitrary strings describing @@ -335,6 +336,7 @@ message StoreCapacity { // instances where overlapping node-binary versions within a cluster result // in this this field missing. optional int64 l0_sublevels = 12 [(gogoproto.nullable) = false]; + optional cockroach.util.admission.admissionpb.IOThreshold io_threshold = 13 [(gogoproto.nullable) = false, (gogoproto.customname) = "IOThreshold" ]; // bytes_per_replica and writes_per_replica contain percentiles for the // number of bytes and writes-per-second to each replica in the store. // This information can be used for rebalancing decisions. diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 376044bf0068..b0d5c8291461 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -219,6 +219,7 @@ go_library( "//pkg/upgrade/upgrademanager", "//pkg/util", "//pkg/util/admission", + "//pkg/util/admission/admissionpb", "//pkg/util/buildutil", "//pkg/util/contextutil", "//pkg/util/envutil", diff --git a/pkg/server/node.go b/pkg/server/node.go index 79ba0966f9a4..4714a8617d44 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -44,6 +44,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/admission" + "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/grpcutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -769,6 +770,16 @@ func (n *Node) computePeriodicMetrics(ctx context.Context, tick int) error { }) } +// UpdateIOThreshold relays the supplied IOThreshold to the same method on the +// designated Store. +func (n *Node) UpdateIOThreshold(id roachpb.StoreID, threshold *admissionpb.IOThreshold) { + s, err := n.stores.GetStore(id) + if err != nil { + log.Errorf(n.AnnotateCtx(context.Background()), "%v", err) + } + s.UpdateIOThreshold(threshold) +} + // GetPebbleMetrics implements admission.PebbleMetricsProvider. func (n *Node) GetPebbleMetrics() []admission.StoreMetrics { var metrics []admission.StoreMetrics diff --git a/pkg/server/server.go b/pkg/server/server.go index b75c0b10efb2..09b9d057a938 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -1448,7 +1448,7 @@ func (s *Server) PreStart(ctx context.Context) error { // existing stores shouldn’t be able to acquire leases yet. Although, below // Raft commands like log application and snapshot application may be able // to bypass admission control. - s.storeGrantCoords.SetPebbleMetricsProvider(ctx, s.node) + s.storeGrantCoords.SetPebbleMetricsProvider(ctx, s.node, s.node) // Once all stores are initialized, check if offline storage recovery // was done prior to start and record any actions appropriately. diff --git a/pkg/util/admission/admissionpb/BUILD.bazel b/pkg/util/admission/admissionpb/BUILD.bazel index d8121dfb4b72..e2dd52c5f1d6 100644 --- a/pkg/util/admission/admissionpb/BUILD.bazel +++ b/pkg/util/admission/admissionpb/BUILD.bazel @@ -1,3 +1,5 @@ +load("@rules_proto//proto:defs.bzl", "proto_library") +load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( @@ -5,7 +7,30 @@ go_library( srcs = [ "admissionpb.go", "doc.go", + "io_threshold.go", ], + embed = [":admissionpb_go_proto"], importpath = "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb", visibility = ["//visibility:public"], + deps = [ + "@com_github_cockroachdb_redact//:redact", + "@com_github_cockroachdb_redact//interfaces", + ], +) + +proto_library( + name = "admissionpb_proto", + srcs = ["io_threshold.proto"], + strip_import_prefix = "/pkg", + visibility = ["//visibility:public"], + deps = ["@com_github_gogo_protobuf//gogoproto:gogo_proto"], +) + +go_proto_library( + name = "admissionpb_go_proto", + compilers = ["//pkg/cmd/protoc-gen-gogoroach:protoc-gen-gogoroach_compiler"], + importpath = "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb", + proto = ":admissionpb_proto", + visibility = ["//visibility:public"], + deps = ["@com_github_gogo_protobuf//gogoproto"], ) diff --git a/pkg/util/admission/admissionpb/io_threshold.go b/pkg/util/admission/admissionpb/io_threshold.go new file mode 100644 index 000000000000..0de6648e1d9b --- /dev/null +++ b/pkg/util/admission/admissionpb/io_threshold.go @@ -0,0 +1,51 @@ +// 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 admissionpb + +import ( + "math" + + "github.com/cockroachdb/redact" + "github.com/cockroachdb/redact/interfaces" +) + +// Score returns, as the second return value, whether IO admission control is +// considering the Store overloaded. The first return value is a 1-normalized +// float (i.e. 1.0 is the threshold at which the second value flips to true). +// +// The zero value returns (0, false). Use of the nil pointer is not allowed. +func (iot IOThreshold) Score() (float64, bool) { + if iot == (IOThreshold{}) { + return 0, false + } + f := math.Max( + float64(iot.L0NumFiles)/float64(iot.L0NumFilesThreshold), + float64(iot.L0NumSubLevels)/float64(iot.L0NumSubLevelsThreshold), + ) + return f, f > 1.0 +} + +// SafeFormat implements redact.SafeFormatter. +func (iot IOThreshold) SafeFormat(s interfaces.SafePrinter, _ rune) { + if iot == (IOThreshold{}) { + s.Printf("N/A") + } + sc, overload := iot.Score() + s.Printf("%.3f", redact.SafeFloat(sc)) + if overload { + s.Printf("[overload]") + } +} + +func (iot IOThreshold) String() string { + return redact.StringWithoutMarkers(iot) +} diff --git a/pkg/util/admission/admissionpb/io_threshold.proto b/pkg/util/admission/admissionpb/io_threshold.proto new file mode 100644 index 000000000000..9e9bde2c3cff --- /dev/null +++ b/pkg/util/admission/admissionpb/io_threshold.proto @@ -0,0 +1,26 @@ +// 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.admission.admissionpb; +option go_package = "admissionpb"; + +import "gogoproto/gogo.proto"; + +// IOThreshold wraps the raw signals that IO admission control utilizes to determine +// when to introduce queueing. +message IOThreshold { + option (gogoproto.goproto_stringer) = false; + + int64 l0_num_sub_levels = 1; + int64 l0_num_sub_levels_threshold = 2; + int64 l0_num_files = 3; + int64 l0_num_files_threshold = 4; +} diff --git a/pkg/util/admission/granter.go b/pkg/util/admission/granter.go index 7118806fa7b4..9ebfb5db411d 100644 --- a/pkg/util/admission/granter.go +++ b/pkg/util/admission/granter.go @@ -17,8 +17,10 @@ import ( "unsafe" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/metric" @@ -1263,7 +1265,7 @@ type StoreGrantCoordinators struct { // SetPebbleMetricsProvider sets a PebbleMetricsProvider and causes the load // on the various storage engines to be used for admission control. func (sgc *StoreGrantCoordinators) SetPebbleMetricsProvider( - startupCtx context.Context, pmp PebbleMetricsProvider, + startupCtx context.Context, pmp PebbleMetricsProvider, iotc IOThresholdConsumer, ) { if sgc.pebbleMetricsProvider != nil { panic(errors.AssertionFailedf("SetPebbleMetricsProvider called more than once")) @@ -1307,6 +1309,7 @@ func (sgc *StoreGrantCoordinators) SetPebbleMetricsProvider( if unsafeGc, ok := sgc.gcMap.Load(int64(m.StoreID)); ok { gc := (*GrantCoordinator)(unsafeGc) gc.pebbleMetricsTick(ctx, m.Metrics) + iotc.UpdateIOThreshold(roachpb.StoreID(m.StoreID), gc.ioLoadListener.ioThreshold) } else { log.Warningf(ctx, "seeing metrics for unknown storeID %d", m.StoreID) @@ -1522,6 +1525,11 @@ type PebbleMetricsProvider interface { GetPebbleMetrics() []StoreMetrics } +// IOThresholdConsumer is informed about updated IOThresholds. +type IOThresholdConsumer interface { + UpdateIOThreshold(roachpb.StoreID, *admissionpb.IOThreshold) +} + // StoreMetrics are the metrics for a store. type StoreMetrics struct { StoreID int32 @@ -1686,14 +1694,24 @@ const adjustmentInterval = 15 func (io *ioLoadListener) pebbleMetricsTick(ctx context.Context, m *pebble.Metrics) { if !io.statsInitialized { io.statsInitialized = true - io.ioLoadListenerState = ioLoadListenerState{ - cumAdmissionStats: io.kvRequester.getStoreAdmissionStats(), - cumL0AddedBytes: m.Levels[0].BytesFlushed + m.Levels[0].BytesIngested, - curL0Bytes: m.Levels[0].Size, - // Reasonable starting fraction until we see some ingests. - smoothedIntIngestedAccountedL0BytesFraction: 0.5, - // No initial limit, i.e, the first interval is unlimited. - totalNumByteTokens: unlimitedTokens, + io.adjustTokensResult = adjustTokensResult{ + ioLoadListenerState: ioLoadListenerState{ + cumAdmissionStats: io.kvRequester.getStoreAdmissionStats(), + cumL0AddedBytes: m.Levels[0].BytesFlushed + m.Levels[0].BytesIngested, + curL0Bytes: m.Levels[0].Size, + // Reasonable starting fraction until we see some ingests. + smoothedIntIngestedAccountedL0BytesFraction: 0.5, + // No initial limit, i.e, the first interval is unlimited. + totalNumByteTokens: unlimitedTokens, + }, + requestEstimates: storeRequestEstimates{}, + aux: adjustTokensAuxComputations{}, + ioThreshold: &admissionpb.IOThreshold{ + L0NumSubLevels: int64(m.Levels[0].Sublevels), + L0NumSubLevelsThreshold: math.MaxInt64, + L0NumFiles: m.Levels[0].NumFiles, + L0NumFilesThreshold: math.MaxInt64, + }, } return } @@ -1740,15 +1758,12 @@ func (io *ioLoadListener) adjustTokens(ctx context.Context, m *pebble.Metrics) { ) io.adjustTokensResult = res io.kvRequester.setStoreRequestEstimates(res.requestEstimates) - if res.aux.shouldLog { + if _, overloaded := res.ioThreshold.Score(); overloaded { log.Infof(logtags.AddTag(ctx, "s", io.storeID), "IO overload: %s", res) } } type adjustTokensAuxComputations struct { - shouldLog bool - curL0NumFiles, curL0NumSublevels int64 - intL0AddedBytes int64 intL0CompactedBytes int64 @@ -1770,6 +1785,13 @@ func (*ioLoadListener) adjustTokensInner( l0Metrics pebble.LevelMetrics, threshNumFiles, threshNumSublevels int64, ) adjustTokensResult { + ioThreshold := &admissionpb.IOThreshold{ + L0NumFiles: l0Metrics.NumFiles, + L0NumFilesThreshold: threshNumFiles, + L0NumSubLevels: int64(l0Metrics.Sublevels), + L0NumSubLevelsThreshold: threshNumSublevels, + } + curL0Bytes := l0Metrics.Size cumL0AddedBytes := l0Metrics.BytesFlushed + l0Metrics.BytesIngested // L0 growth over the last interval. @@ -1795,13 +1817,8 @@ func (*ioLoadListener) adjustTokensInner( // intAdmittedCount is the number of requests admitted since the last token adjustment. intAdmittedCount := cumAdmissionStats.admittedCount - prev.cumAdmissionStats.admittedCount - doLog := true if intAdmittedCount <= 0 { intAdmittedCount = 1 - // Admission control is likely disabled, given there was no KVWork - // admitted for 15s. And even if it is enabled, this is not an interesting - // situation. - doLog = false } // intAdmittedBytes are the bytes admitted since the last token adjustment. This @@ -1881,9 +1898,9 @@ func (*ioLoadListener) adjustTokensInner( // We constrain admission if the store is over the threshold. var totalNumByteTokens int64 var smoothedTotalNumByteTokens float64 - curL0NumFiles := l0Metrics.NumFiles - curL0NumSublevels := int64(l0Metrics.Sublevels) - if curL0NumFiles > threshNumFiles || curL0NumSublevels > threshNumSublevels { + + _, overloaded := ioThreshold.Score() + if overloaded { // Don't admit more byte work than we can remove via compactions. totalNumByteTokens // tracks our goal for admission. // Scale down since we want to get under the thresholds over time. This @@ -1909,7 +1926,6 @@ func (*ioLoadListener) adjustTokensInner( numTokens := intL0CompactedBytes smoothedTotalNumByteTokens = alpha*float64(numTokens) + (1-alpha)*prev.smoothedTotalNumByteTokens totalNumByteTokens = unlimitedTokens - doLog = false } // Install the latest cumulative stats. return adjustTokensResult{ @@ -1929,9 +1945,6 @@ func (*ioLoadListener) adjustTokensInner( fractionOfIngestIntoL0: smoothedIntIngestedAccountedL0BytesFraction, }, aux: adjustTokensAuxComputations{ - shouldLog: doLog, - curL0NumFiles: curL0NumFiles, - curL0NumSublevels: curL0NumSublevels, intL0AddedBytes: intL0AddedBytes, intL0CompactedBytes: intL0CompactedBytes, intAdmittedCount: intAdmittedCount, @@ -1943,6 +1956,7 @@ func (*ioLoadListener) adjustTokensInner( intPerWorkUnaccountedL0Bytes: intPerWorkUnaccountedL0Bytes, l0BytesIngestFraction: intIngestedAccountedL0BytesFraction, }, + ioThreshold: ioThreshold, } } @@ -1950,6 +1964,7 @@ type adjustTokensResult struct { ioLoadListenerState requestEstimates storeRequestEstimates aux adjustTokensAuxComputations + ioThreshold *admissionpb.IOThreshold // never nil } func max(i, j int64) int64 { @@ -1962,7 +1977,7 @@ func max(i, j int64) int64 { func (res adjustTokensResult) SafeFormat(p redact.SafePrinter, _ rune) { ib := humanizeutil.IBytes // NB: "≈" indicates smoothed quantities. - p.Printf("%d ssts, %d sub-levels, ", res.aux.curL0NumFiles, res.aux.curL0NumSublevels) + p.Printf("score %v (%d ssts, %d sub-levels), ", res.ioThreshold, res.ioThreshold.L0NumFiles, res.ioThreshold.L0NumSubLevels) p.Printf("L0 growth %s: ", ib(res.aux.intL0AddedBytes)) // Writes to L0 that we expected because requests asked admission control for them. // This is the "happy path". diff --git a/pkg/util/admission/granter_test.go b/pkg/util/admission/granter_test.go index 72ae85572118..4ccebda20006 100644 --- a/pkg/util/admission/granter_test.go +++ b/pkg/util/admission/granter_test.go @@ -22,9 +22,11 @@ import ( "time" "unsafe" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/echotest" + "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -182,7 +184,7 @@ func TestGranterBasic(t *testing.T) { } var testMetricsProvider testMetricsProvider testMetricsProvider.setMetricsForStores([]int32{1}, pebble.Metrics{}) - storeCoordinators.SetPebbleMetricsProvider(context.Background(), &testMetricsProvider) + storeCoordinators.SetPebbleMetricsProvider(context.Background(), &testMetricsProvider, &testMetricsProvider) unsafeGranter, ok := storeCoordinators.gcMap.Load(int64(1)) require.True(t, ok) coord = (*GrantCoordinator)(unsafeGranter) @@ -286,6 +288,11 @@ func (m *testMetricsProvider) GetPebbleMetrics() []StoreMetrics { return m.metrics } +func (m *testMetricsProvider) UpdateIOThreshold( + id roachpb.StoreID, threshold *admissionpb.IOThreshold, +) { +} + func (m *testMetricsProvider) setMetricsForStores(stores []int32, metrics pebble.Metrics) { m.metrics = m.metrics[:0] for _, s := range stores { @@ -342,7 +349,7 @@ func TestStoreCoordinators(t *testing.T) { mp.setMetricsForStores([]int32{10, 20}, metrics) // Setting the metrics provider will cause the initialization of two // GrantCoordinators for the two stores. - storeCoords.SetPebbleMetricsProvider(context.Background(), &mp) + storeCoords.SetPebbleMetricsProvider(context.Background(), &mp, &mp) // Now we have 1+2 = 3 KVWork requesters. require.Equal(t, 3, len(requesters)) // Confirm that the store IDs are as expected. @@ -477,11 +484,13 @@ func TestIOLoadListener(t *testing.T) { var l0SubLevels int d.ScanArgs(t, "l0-sublevels", &l0SubLevels) metrics.Levels[0].Sublevels = int32(l0SubLevels) + var buf strings.Builder ioll.pebbleMetricsTick(ctx, &metrics) // Do the ticks until just before next adjustment. - var buf strings.Builder - fmt.Fprintln(&buf, redact.StringWithoutMarkers(&ioll.adjustTokensResult)) - fmt.Fprintf(&buf, "%+v\n", (rawTokenResult)(ioll.adjustTokensResult)) + res := ioll.adjustTokensResult + fmt.Fprintln(&buf, redact.StringWithoutMarkers(&res)) + res.ioThreshold = nil // avoid nondeterminism + fmt.Fprintf(&buf, "%+v\n", (rawTokenResult)(res)) if req.buf.Len() > 0 { fmt.Fprintf(&buf, "%s\n", req.buf.String()) req.buf.Reset() diff --git a/pkg/util/admission/testdata/io_load_listener b/pkg/util/admission/testdata/io_load_listener index 853c24cb09af..c929e78a30fe 100644 --- a/pkg/util/admission/testdata/io_load_listener +++ b/pkg/util/admission/testdata/io_load_listener @@ -11,8 +11,8 @@ prep-admission-stats admitted=0 # Even though above the threshold, the first 15 ticks don't limit the tokens. set-state l0-bytes=10000 l0-added=1000 l0-files=21 l0-sublevels=21 ---- -0 ssts, 0 sub-levels, L0 growth 0 B: 0 B acc-write + 0 B acc-ingest + 0 B unacc [≈0 B/req, n=0], compacted 0 B [≈0 B]; admitting all -{ioLoadListenerState:{cumAdmissionStats:{admittedCount:0 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:1000 curL0Bytes:10000 smoothedIntL0CompactedBytes:0 smoothedIntPerWorkUnaccountedL0Bytes:0 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedTotalNumByteTokens:0 totalNumByteTokens:9223372036854775807 tokensAllocated:0} requestEstimates:{fractionOfIngestIntoL0:0 workByteAddition:0} aux:{shouldLog:false curL0NumFiles:0 curL0NumSublevels:0 intL0AddedBytes:0 intL0CompactedBytes:0 intAdmittedCount:0 intAdmittedBytes:0 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:0 intUnaccountedL0Bytes:0 intPerWorkUnaccountedL0Bytes:0 l0BytesIngestFraction:0}} +score 0.000 (21 ssts, 21 sub-levels), L0 growth 0 B: 0 B acc-write + 0 B acc-ingest + 0 B unacc [≈0 B/req, n=0], compacted 0 B [≈0 B]; admitting all +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:0 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:1000 curL0Bytes:10000 smoothedIntL0CompactedBytes:0 smoothedIntPerWorkUnaccountedL0Bytes:0 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedTotalNumByteTokens:0 totalNumByteTokens:9223372036854775807 tokensAllocated:0} requestEstimates:{fractionOfIngestIntoL0:0 workByteAddition:0} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intAdmittedCount:0 intAdmittedBytes:0 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:0 intUnaccountedL0Bytes:0 intPerWorkUnaccountedL0Bytes:0 l0BytesIngestFraction:0} ioThreshold:} tick: 0, setAvailableIOTokens: unlimited tick: 1, setAvailableIOTokens: unlimited tick: 2, setAvailableIOTokens: unlimited @@ -39,8 +39,8 @@ prep-admission-stats admitted=10000 # removed), but smoothing it drops the tokens to 12,500. set-state l0-bytes=10000 l0-added=101000 l0-files=21 l0-sublevels=21 ---- -21 ssts, 21 sub-levels, L0 growth 98 KiB: 0 B acc-write + 0 B acc-ingest + 98 KiB unacc [≈10 B/req, n=10000], compacted 98 KiB [≈49 KiB]; admitting 12 KiB with L0 penalty: +10 B/req, *0.50/ingest -{ioLoadListenerState:{cumAdmissionStats:{admittedCount:10000 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:101000 curL0Bytes:10000 smoothedIntL0CompactedBytes:50000 smoothedIntPerWorkUnaccountedL0Bytes:10 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedTotalNumByteTokens:12500 totalNumByteTokens:12500 tokensAllocated:0} requestEstimates:{fractionOfIngestIntoL0:0.5 workByteAddition:10} aux:{shouldLog:true curL0NumFiles:21 curL0NumSublevels:21 intL0AddedBytes:100000 intL0CompactedBytes:100000 intAdmittedCount:10000 intAdmittedBytes:0 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:0 intUnaccountedL0Bytes:100000 intPerWorkUnaccountedL0Bytes:10 l0BytesIngestFraction:0}} +score 1.050[overload] (21 ssts, 21 sub-levels), L0 growth 98 KiB: 0 B acc-write + 0 B acc-ingest + 98 KiB unacc [≈10 B/req, n=10000], compacted 98 KiB [≈49 KiB]; admitting 12 KiB with L0 penalty: +10 B/req, *0.50/ingest +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:10000 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:101000 curL0Bytes:10000 smoothedIntL0CompactedBytes:50000 smoothedIntPerWorkUnaccountedL0Bytes:10 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedTotalNumByteTokens:12500 totalNumByteTokens:12500 tokensAllocated:0} requestEstimates:{fractionOfIngestIntoL0:0.5 workByteAddition:10} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intAdmittedCount:10000 intAdmittedBytes:0 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:0 intUnaccountedL0Bytes:100000 intPerWorkUnaccountedL0Bytes:10 l0BytesIngestFraction:0} ioThreshold:} store-request-estimates: fractionOfIngestIntoL0: 0.50, workByteAddition: 10 tick: 0, setAvailableIOTokens: 834 tick: 1, setAvailableIOTokens: 834 @@ -65,8 +65,8 @@ prep-admission-stats admitted=20000 # Same delta as previous but smoothing bumps up the tokens to 25,000. set-state l0-bytes=10000 l0-added=201000 l0-files=21 l0-sublevels=21 ---- -21 ssts, 21 sub-levels, L0 growth 98 KiB: 0 B acc-write + 0 B acc-ingest + 98 KiB unacc [≈10 B/req, n=10000], compacted 98 KiB [≈73 KiB]; admitting 24 KiB with L0 penalty: +10 B/req, *0.50/ingest -{ioLoadListenerState:{cumAdmissionStats:{admittedCount:20000 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:201000 curL0Bytes:10000 smoothedIntL0CompactedBytes:75000 smoothedIntPerWorkUnaccountedL0Bytes:10 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedTotalNumByteTokens:25000 totalNumByteTokens:25000 tokensAllocated:0} requestEstimates:{fractionOfIngestIntoL0:0.5 workByteAddition:10} aux:{shouldLog:true curL0NumFiles:21 curL0NumSublevels:21 intL0AddedBytes:100000 intL0CompactedBytes:100000 intAdmittedCount:10000 intAdmittedBytes:0 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:0 intUnaccountedL0Bytes:100000 intPerWorkUnaccountedL0Bytes:10 l0BytesIngestFraction:0}} +score 1.050[overload] (21 ssts, 21 sub-levels), L0 growth 98 KiB: 0 B acc-write + 0 B acc-ingest + 98 KiB unacc [≈10 B/req, n=10000], compacted 98 KiB [≈73 KiB]; admitting 24 KiB with L0 penalty: +10 B/req, *0.50/ingest +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:20000 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:201000 curL0Bytes:10000 smoothedIntL0CompactedBytes:75000 smoothedIntPerWorkUnaccountedL0Bytes:10 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedTotalNumByteTokens:25000 totalNumByteTokens:25000 tokensAllocated:0} requestEstimates:{fractionOfIngestIntoL0:0.5 workByteAddition:10} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intAdmittedCount:10000 intAdmittedBytes:0 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:0 intUnaccountedL0Bytes:100000 intPerWorkUnaccountedL0Bytes:10 l0BytesIngestFraction:0} ioThreshold:} store-request-estimates: fractionOfIngestIntoL0: 0.50, workByteAddition: 10 tick: 0, setAvailableIOTokens: 1667 tick: 1, setAvailableIOTokens: 1667 @@ -87,8 +87,8 @@ tick: 14, setAvailableIOTokens: 1662 # No delta. This used to trigger an overflow bug. set-state l0-bytes=10000 l0-added=201000 l0-files=21 l0-sublevels=21 ---- -21 ssts, 21 sub-levels, L0 growth 0 B: 0 B acc-write + 0 B acc-ingest + 0 B unacc [≈10 B/req, n=1], compacted 0 B [≈37 KiB]; admitting 21 KiB with L0 penalty: +10 B/req, *0.50/ingest -{ioLoadListenerState:{cumAdmissionStats:{admittedCount:20000 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:201000 curL0Bytes:10000 smoothedIntL0CompactedBytes:37500 smoothedIntPerWorkUnaccountedL0Bytes:10 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedTotalNumByteTokens:21875 totalNumByteTokens:21875 tokensAllocated:0} requestEstimates:{fractionOfIngestIntoL0:0.5 workByteAddition:10} aux:{shouldLog:false curL0NumFiles:21 curL0NumSublevels:21 intL0AddedBytes:0 intL0CompactedBytes:0 intAdmittedCount:1 intAdmittedBytes:0 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:0 intUnaccountedL0Bytes:0 intPerWorkUnaccountedL0Bytes:0 l0BytesIngestFraction:0}} +score 1.050[overload] (21 ssts, 21 sub-levels), L0 growth 0 B: 0 B acc-write + 0 B acc-ingest + 0 B unacc [≈10 B/req, n=1], compacted 0 B [≈37 KiB]; admitting 21 KiB with L0 penalty: +10 B/req, *0.50/ingest +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:20000 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:201000 curL0Bytes:10000 smoothedIntL0CompactedBytes:37500 smoothedIntPerWorkUnaccountedL0Bytes:10 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedTotalNumByteTokens:21875 totalNumByteTokens:21875 tokensAllocated:0} requestEstimates:{fractionOfIngestIntoL0:0.5 workByteAddition:10} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intAdmittedCount:1 intAdmittedBytes:0 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:0 intUnaccountedL0Bytes:0 intPerWorkUnaccountedL0Bytes:0 l0BytesIngestFraction:0} ioThreshold:} store-request-estimates: fractionOfIngestIntoL0: 0.50, workByteAddition: 10 tick: 0, setAvailableIOTokens: 1459 tick: 1, setAvailableIOTokens: 1459 @@ -114,8 +114,8 @@ prep-admission-stats admitted=30000 # don't limit the tokens. set-state l0-bytes=10000 l0-added=501000 l0-files=21 l0-sublevels=20 ---- -21 ssts, 20 sub-levels, L0 growth 293 KiB: 0 B acc-write + 0 B acc-ingest + 293 KiB unacc [≈20 B/req, n=10000], compacted 293 KiB [≈165 KiB]; admitting all -{ioLoadListenerState:{cumAdmissionStats:{admittedCount:30000 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:501000 curL0Bytes:10000 smoothedIntL0CompactedBytes:168750 smoothedIntPerWorkUnaccountedL0Bytes:20 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedTotalNumByteTokens:160937.5 totalNumByteTokens:9223372036854775807 tokensAllocated:0} requestEstimates:{fractionOfIngestIntoL0:0.5 workByteAddition:20} aux:{shouldLog:false curL0NumFiles:21 curL0NumSublevels:20 intL0AddedBytes:300000 intL0CompactedBytes:300000 intAdmittedCount:10000 intAdmittedBytes:0 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:0 intUnaccountedL0Bytes:300000 intPerWorkUnaccountedL0Bytes:30 l0BytesIngestFraction:0}} +score 1.000 (21 ssts, 20 sub-levels), L0 growth 293 KiB: 0 B acc-write + 0 B acc-ingest + 293 KiB unacc [≈20 B/req, n=10000], compacted 293 KiB [≈165 KiB]; admitting all +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:30000 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:501000 curL0Bytes:10000 smoothedIntL0CompactedBytes:168750 smoothedIntPerWorkUnaccountedL0Bytes:20 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedTotalNumByteTokens:160937.5 totalNumByteTokens:9223372036854775807 tokensAllocated:0} requestEstimates:{fractionOfIngestIntoL0:0.5 workByteAddition:20} aux:{intL0AddedBytes:300000 intL0CompactedBytes:300000 intAdmittedCount:10000 intAdmittedBytes:0 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:0 intUnaccountedL0Bytes:300000 intPerWorkUnaccountedL0Bytes:30 l0BytesIngestFraction:0} ioThreshold:} store-request-estimates: fractionOfIngestIntoL0: 0.50, workByteAddition: 20 tick: 0, setAvailableIOTokens: unlimited tick: 1, setAvailableIOTokens: unlimited @@ -143,8 +143,8 @@ prep-admission-stats admitted=0 set-state l0-bytes=1000 l0-added=1000 l0-files=21 l0-sublevels=21 ---- -0 ssts, 0 sub-levels, L0 growth 0 B: 0 B acc-write + 0 B acc-ingest + 0 B unacc [≈0 B/req, n=0], compacted 0 B [≈0 B]; admitting all -{ioLoadListenerState:{cumAdmissionStats:{admittedCount:0 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:1000 curL0Bytes:1000 smoothedIntL0CompactedBytes:0 smoothedIntPerWorkUnaccountedL0Bytes:0 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedTotalNumByteTokens:0 totalNumByteTokens:9223372036854775807 tokensAllocated:0} requestEstimates:{fractionOfIngestIntoL0:0 workByteAddition:0} aux:{shouldLog:false curL0NumFiles:0 curL0NumSublevels:0 intL0AddedBytes:0 intL0CompactedBytes:0 intAdmittedCount:0 intAdmittedBytes:0 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:0 intUnaccountedL0Bytes:0 intPerWorkUnaccountedL0Bytes:0 l0BytesIngestFraction:0}} +score 0.000 (21 ssts, 21 sub-levels), L0 growth 0 B: 0 B acc-write + 0 B acc-ingest + 0 B unacc [≈0 B/req, n=0], compacted 0 B [≈0 B]; admitting all +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:0 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:1000 curL0Bytes:1000 smoothedIntL0CompactedBytes:0 smoothedIntPerWorkUnaccountedL0Bytes:0 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedTotalNumByteTokens:0 totalNumByteTokens:9223372036854775807 tokensAllocated:0} requestEstimates:{fractionOfIngestIntoL0:0 workByteAddition:0} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intAdmittedCount:0 intAdmittedBytes:0 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:0 intUnaccountedL0Bytes:0 intPerWorkUnaccountedL0Bytes:0 l0BytesIngestFraction:0} ioThreshold:} tick: 0, setAvailableIOTokens: unlimited tick: 1, setAvailableIOTokens: unlimited tick: 2, setAvailableIOTokens: unlimited @@ -170,8 +170,8 @@ prep-admission-stats admitted=10 admitted-bytes=180000 ingested-bytes=50000 inge set-state l0-bytes=1000 l0-added=201000 l0-files=21 l0-sublevels=21 ---- -21 ssts, 21 sub-levels, L0 growth 195 KiB: 127 KiB acc-write + 20 KiB acc-ingest + 49 KiB unacc [≈4.9 KiB/req, n=10], compacted 195 KiB [≈98 KiB]; admitting 24 KiB with L0 penalty: +4.9 KiB/req, *0.45/ingest -{ioLoadListenerState:{cumAdmissionStats:{admittedCount:10 admittedWithBytesCount:0 admittedAccountedBytes:180000 ingestedAccountedBytes:50000 ingestedAccountedL0Bytes:20000} cumL0AddedBytes:201000 curL0Bytes:1000 smoothedIntL0CompactedBytes:100000 smoothedIntPerWorkUnaccountedL0Bytes:5000 smoothedIntIngestedAccountedL0BytesFraction:0.45 smoothedTotalNumByteTokens:25000 totalNumByteTokens:25000 tokensAllocated:0} requestEstimates:{fractionOfIngestIntoL0:0.45 workByteAddition:5000} aux:{shouldLog:true curL0NumFiles:21 curL0NumSublevels:21 intL0AddedBytes:200000 intL0CompactedBytes:200000 intAdmittedCount:10 intAdmittedBytes:180000 intIngestedBytes:50000 intIngestedAccountedL0Bytes:20000 intAccountedL0Bytes:150000 intUnaccountedL0Bytes:50000 intPerWorkUnaccountedL0Bytes:5000 l0BytesIngestFraction:0.4}} +score 1.050[overload] (21 ssts, 21 sub-levels), L0 growth 195 KiB: 127 KiB acc-write + 20 KiB acc-ingest + 49 KiB unacc [≈4.9 KiB/req, n=10], compacted 195 KiB [≈98 KiB]; admitting 24 KiB with L0 penalty: +4.9 KiB/req, *0.45/ingest +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:10 admittedWithBytesCount:0 admittedAccountedBytes:180000 ingestedAccountedBytes:50000 ingestedAccountedL0Bytes:20000} cumL0AddedBytes:201000 curL0Bytes:1000 smoothedIntL0CompactedBytes:100000 smoothedIntPerWorkUnaccountedL0Bytes:5000 smoothedIntIngestedAccountedL0BytesFraction:0.45 smoothedTotalNumByteTokens:25000 totalNumByteTokens:25000 tokensAllocated:0} requestEstimates:{fractionOfIngestIntoL0:0.45 workByteAddition:5000} aux:{intL0AddedBytes:200000 intL0CompactedBytes:200000 intAdmittedCount:10 intAdmittedBytes:180000 intIngestedBytes:50000 intIngestedAccountedL0Bytes:20000 intAccountedL0Bytes:150000 intUnaccountedL0Bytes:50000 intPerWorkUnaccountedL0Bytes:5000 l0BytesIngestFraction:0.4} ioThreshold:} store-request-estimates: fractionOfIngestIntoL0: 0.45, workByteAddition: 5000 tick: 0, setAvailableIOTokens: 1667 tick: 1, setAvailableIOTokens: 1667 @@ -196,8 +196,8 @@ prep-admission-stats admitted=20 admitted-bytes=200000 ingested-bytes=50000 inge set-state l0-bytes=1000 l0-added=221000 l0-files=21 l0-sublevels=21 ---- -21 ssts, 21 sub-levels, L0 growth 20 KiB: 20 KiB acc-write + 0 B acc-ingest + 0 B unacc [≈2.4 KiB/req, n=10], compacted 20 KiB [≈59 KiB]; admitting 27 KiB with L0 penalty: +2.4 KiB/req, *0.45/ingest -{ioLoadListenerState:{cumAdmissionStats:{admittedCount:20 admittedWithBytesCount:0 admittedAccountedBytes:200000 ingestedAccountedBytes:50000 ingestedAccountedL0Bytes:20000} cumL0AddedBytes:221000 curL0Bytes:1000 smoothedIntL0CompactedBytes:60000 smoothedIntPerWorkUnaccountedL0Bytes:2500 smoothedIntIngestedAccountedL0BytesFraction:0.45 smoothedTotalNumByteTokens:27500 totalNumByteTokens:27500 tokensAllocated:0} requestEstimates:{fractionOfIngestIntoL0:0.45 workByteAddition:2500} aux:{shouldLog:true curL0NumFiles:21 curL0NumSublevels:21 intL0AddedBytes:20000 intL0CompactedBytes:20000 intAdmittedCount:10 intAdmittedBytes:20000 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:20000 intUnaccountedL0Bytes:0 intPerWorkUnaccountedL0Bytes:0 l0BytesIngestFraction:0}} +score 1.050[overload] (21 ssts, 21 sub-levels), L0 growth 20 KiB: 20 KiB acc-write + 0 B acc-ingest + 0 B unacc [≈2.4 KiB/req, n=10], compacted 20 KiB [≈59 KiB]; admitting 27 KiB with L0 penalty: +2.4 KiB/req, *0.45/ingest +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:20 admittedWithBytesCount:0 admittedAccountedBytes:200000 ingestedAccountedBytes:50000 ingestedAccountedL0Bytes:20000} cumL0AddedBytes:221000 curL0Bytes:1000 smoothedIntL0CompactedBytes:60000 smoothedIntPerWorkUnaccountedL0Bytes:2500 smoothedIntIngestedAccountedL0BytesFraction:0.45 smoothedTotalNumByteTokens:27500 totalNumByteTokens:27500 tokensAllocated:0} requestEstimates:{fractionOfIngestIntoL0:0.45 workByteAddition:2500} aux:{intL0AddedBytes:20000 intL0CompactedBytes:20000 intAdmittedCount:10 intAdmittedBytes:20000 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:20000 intUnaccountedL0Bytes:0 intPerWorkUnaccountedL0Bytes:0 l0BytesIngestFraction:0} ioThreshold:} store-request-estimates: fractionOfIngestIntoL0: 0.45, workByteAddition: 2500 tick: 0, setAvailableIOTokens: 1834 tick: 1, setAvailableIOTokens: 1834 @@ -223,8 +223,8 @@ prep-admission-stats admitted=30 admitted-bytes=300000 ingested-bytes=50000 inge set-state l0-bytes=1000 l0-added=241000 l0-files=21 l0-sublevels=21 ---- -21 ssts, 21 sub-levels, L0 growth 20 KiB: 98 KiB acc-write + 0 B acc-ingest + 0 B unacc [≈1.2 KiB/req, n=10], compacted 20 KiB [≈39 KiB]; admitting 23 KiB with L0 penalty: +1.2 KiB/req, *0.45/ingest -{ioLoadListenerState:{cumAdmissionStats:{admittedCount:30 admittedWithBytesCount:0 admittedAccountedBytes:300000 ingestedAccountedBytes:50000 ingestedAccountedL0Bytes:20000} cumL0AddedBytes:241000 curL0Bytes:1000 smoothedIntL0CompactedBytes:40000 smoothedIntPerWorkUnaccountedL0Bytes:1250 smoothedIntIngestedAccountedL0BytesFraction:0.45 smoothedTotalNumByteTokens:23750 totalNumByteTokens:23750 tokensAllocated:0} requestEstimates:{fractionOfIngestIntoL0:0.45 workByteAddition:1250} aux:{shouldLog:true curL0NumFiles:21 curL0NumSublevels:21 intL0AddedBytes:20000 intL0CompactedBytes:20000 intAdmittedCount:10 intAdmittedBytes:100000 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:100000 intUnaccountedL0Bytes:0 intPerWorkUnaccountedL0Bytes:0 l0BytesIngestFraction:0}} +score 1.050[overload] (21 ssts, 21 sub-levels), L0 growth 20 KiB: 98 KiB acc-write + 0 B acc-ingest + 0 B unacc [≈1.2 KiB/req, n=10], compacted 20 KiB [≈39 KiB]; admitting 23 KiB with L0 penalty: +1.2 KiB/req, *0.45/ingest +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:30 admittedWithBytesCount:0 admittedAccountedBytes:300000 ingestedAccountedBytes:50000 ingestedAccountedL0Bytes:20000} cumL0AddedBytes:241000 curL0Bytes:1000 smoothedIntL0CompactedBytes:40000 smoothedIntPerWorkUnaccountedL0Bytes:1250 smoothedIntIngestedAccountedL0BytesFraction:0.45 smoothedTotalNumByteTokens:23750 totalNumByteTokens:23750 tokensAllocated:0} requestEstimates:{fractionOfIngestIntoL0:0.45 workByteAddition:1250} aux:{intL0AddedBytes:20000 intL0CompactedBytes:20000 intAdmittedCount:10 intAdmittedBytes:100000 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:100000 intUnaccountedL0Bytes:0 intPerWorkUnaccountedL0Bytes:0 l0BytesIngestFraction:0} ioThreshold:} store-request-estimates: fractionOfIngestIntoL0: 0.45, workByteAddition: 1250 tick: 0, setAvailableIOTokens: 1584 tick: 1, setAvailableIOTokens: 1584