diff --git a/pkg/kv/kvserver/metrics.go b/pkg/kv/kvserver/metrics.go index 9adfaf4e4020..01eb6ebc9f5d 100644 --- a/pkg/kv/kvserver/metrics.go +++ b/pkg/kv/kvserver/metrics.go @@ -489,6 +489,12 @@ var ( Measurement: "Events", Unit: metric.Unit_COUNT, } + metaRdbWriteStallNanos = metric.Metadata{ + Name: "storage.write-stall-nanos", + Help: "Total write stall duration in nanos", + Measurement: "Nanoseconds", + Unit: metric.Unit_NANOSECONDS, + } // Disk health metrics. metaDiskSlow = metric.Metadata{ @@ -1554,6 +1560,7 @@ type StoreMetrics struct { RdbL0NumFiles *metric.Gauge RdbBytesIngested [7]*metric.Gauge // idx = level RdbWriteStalls *metric.Gauge + RdbWriteStallNanos *metric.Gauge // Disk health metrics. DiskSlow *metric.Gauge @@ -2022,6 +2029,7 @@ func newStoreMetrics(histogramWindow time.Duration) *StoreMetrics { RdbL0NumFiles: metric.NewGauge(metaRdbL0NumFiles), RdbBytesIngested: rdbBytesIngested, RdbWriteStalls: metric.NewGauge(metaRdbWriteStalls), + RdbWriteStallNanos: metric.NewGauge(metaRdbWriteStallNanos), // Disk health metrics. DiskSlow: metric.NewGauge(metaDiskSlow), @@ -2279,6 +2287,7 @@ func (sm *StoreMetrics) updateEngineMetrics(m storage.Metrics) { sm.RdbMarkedForCompactionFiles.Update(int64(m.Compact.MarkedFiles)) sm.RdbNumSSTables.Update(m.NumSSTables()) sm.RdbWriteStalls.Update(m.WriteStallCount) + sm.RdbWriteStallNanos.Update(m.WriteStallDuration.Nanoseconds()) sm.DiskSlow.Update(m.DiskSlowCount) sm.DiskStalled.Update(m.DiskStallCount) diff --git a/pkg/server/node.go b/pkg/server/node.go index 4714a8617d44..9eee5705c646 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -785,8 +785,12 @@ func (n *Node) GetPebbleMetrics() []admission.StoreMetrics { var metrics []admission.StoreMetrics _ = n.stores.VisitStores(func(store *kvserver.Store) error { m := store.Engine().GetMetrics() - metrics = append( - metrics, admission.StoreMetrics{StoreID: int32(store.StoreID()), Metrics: m.Metrics}) + im := store.Engine().GetInternalIntervalMetrics() + metrics = append(metrics, admission.StoreMetrics{ + StoreID: int32(store.StoreID()), + Metrics: m.Metrics, + WriteStallCount: m.WriteStallCount, + InternalIntervalMetrics: im}) return nil }) return metrics diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index f266c2544b47..409ecd08ca91 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -940,6 +940,12 @@ type Engine interface { // MinVersionIsAtLeastTargetVersion returns whether the engine's recorded // storage min version is at least the target version. MinVersionIsAtLeastTargetVersion(target roachpb.Version) (bool, error) + + // GetInternalIntervalMetrics returns low-level metrics from Pebble, that + // are reset at every interval, where an interval is defined over successive + // calls to this method. Hence, this should be used with care, with only one + // caller, which is currently the admission control subsystem. + GetInternalIntervalMetrics() *pebble.InternalIntervalMetrics } // Batch is the interface for batch specific operations. @@ -978,7 +984,8 @@ type Metrics struct { // // We do not split this metric across these two reasons, but they can be // distinguished in the pebble logs. - WriteStallCount int64 + WriteStallCount int64 + WriteStallDuration time.Duration // DiskSlowCount counts the number of times Pebble records disk slowness. DiskSlowCount int64 // DiskStallCount counts the number of times Pebble observes slow writes diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index 5cf0bde91f48..a1863f11c08a 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -43,6 +43,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" "github.com/cockroachdb/errors/oserror" @@ -632,9 +633,11 @@ type Pebble struct { // Stats updated by pebble.EventListener invocations, and returned in // GetMetrics. Updated and retrieved atomically. - writeStallCount int64 - diskSlowCount int64 - diskStallCount int64 + writeStallCount int64 + writeStallDuration time.Duration + writeStallStartNanos int64 + diskSlowCount int64 + diskStallCount int64 // Relevant options copied over from pebble.Options. fs vfs.FS @@ -905,6 +908,22 @@ func (p *Pebble) makeMetricEtcEventListener(ctx context.Context) pebble.EventLis return pebble.EventListener{ WriteStallBegin: func(info pebble.WriteStallBeginInfo) { atomic.AddInt64(&p.writeStallCount, 1) + startNanos := timeutil.Now().UnixNano() + atomic.StoreInt64(&p.writeStallStartNanos, startNanos) + }, + WriteStallEnd: func() { + startNanos := atomic.SwapInt64(&p.writeStallStartNanos, 0) + if startNanos == 0 { + // Should not happen since these callbacks are registered when Pebble + // is opened, but just in case we miss the WriteStallBegin, lets not + // corrupt the metric. + return + } + stallDuration := timeutil.Now().UnixNano() - startNanos + if stallDuration < 0 { + return + } + atomic.AddInt64((*int64)(&p.writeStallDuration), stallDuration) }, DiskSlow: func(info pebble.DiskSlowInfo) { maxSyncDuration := maxSyncDurationDefault @@ -1486,13 +1505,19 @@ func (p *Pebble) Flush() error { func (p *Pebble) GetMetrics() Metrics { m := p.db.Metrics() return Metrics{ - Metrics: m, - WriteStallCount: atomic.LoadInt64(&p.writeStallCount), - DiskSlowCount: atomic.LoadInt64(&p.diskSlowCount), - DiskStallCount: atomic.LoadInt64(&p.diskStallCount), + Metrics: m, + WriteStallCount: atomic.LoadInt64(&p.writeStallCount), + WriteStallDuration: time.Duration(atomic.LoadInt64((*int64)(&p.writeStallDuration))), + DiskSlowCount: atomic.LoadInt64(&p.diskSlowCount), + DiskStallCount: atomic.LoadInt64(&p.diskStallCount), } } +// GetInternalIntervalMetrics implements the Engine interface. +func (p *Pebble) GetInternalIntervalMetrics() *pebble.InternalIntervalMetrics { + return p.db.InternalIntervalMetrics() +} + // GetEncryptionRegistries implements the Engine interface. func (p *Pebble) GetEncryptionRegistries() (*EncryptionRegistries, error) { rv := &EncryptionRegistries{} diff --git a/pkg/ts/catalog/chart_catalog.go b/pkg/ts/catalog/chart_catalog.go index 92e095d466c2..8603d26a9707 100644 --- a/pkg/ts/catalog/chart_catalog.go +++ b/pkg/ts/catalog/chart_catalog.go @@ -2823,6 +2823,11 @@ var charts = []sectionDescription{ Title: "Stalls", Metrics: []string{"storage.write-stalls"}, }, + { + Title: "Stall Duration", + Metrics: []string{"storage.write-stall-nanos"}, + AxisLabel: "Duration (nanos)", + }, }, }, { diff --git a/pkg/util/admission/admissionpb/io_threshold.go b/pkg/util/admission/admissionpb/io_threshold.go index 0de6648e1d9b..f4089fa47b8e 100644 --- a/pkg/util/admission/admissionpb/io_threshold.go +++ b/pkg/util/admission/admissionpb/io_threshold.go @@ -19,10 +19,18 @@ import ( ) // 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). +// considering the Store overloaded wrt compaction of L0. 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. +// +// TODO(sumeer): consider whether we need to enhance this to incorporate +// overloading via flush bandwidth. I suspect we can get away without +// incorporating flush bandwidth since typically chronic overload will be due +// to compactions falling behind (though that may change if we increase the +// max number of compactions). And we will need to incorporate overload due to +// disk bandwidth bottleneck. func (iot IOThreshold) Score() (float64, bool) { if iot == (IOThreshold{}) { return 0, false @@ -42,7 +50,7 @@ func (iot IOThreshold) SafeFormat(s interfaces.SafePrinter, _ rune) { sc, overload := iot.Score() s.Printf("%.3f", redact.SafeFloat(sc)) if overload { - s.Printf("[overload]") + s.Printf("[L0-overload]") } } diff --git a/pkg/util/admission/granter.go b/pkg/util/admission/granter.go index 1cf568a7e229..74cf522857c9 100644 --- a/pkg/util/admission/granter.go +++ b/pkg/util/admission/granter.go @@ -66,6 +66,32 @@ var EnabledSoftSlotGranting = settings.RegisterBoolSetting( true, ) +// MinFlushUtilizationFraction is a lower-bound on the dynamically adjusted +// flush utilization target fraction that attempts to reduce write stalls. Set +// it to a high fraction (>>1, e.g. 10), to effectively disable flush based +// tokens. +// +// The target fraction is used to multiply the (measured) peak flush rate, to +// compute the flush tokens. For example, if the dynamic target fraction (for +// which this setting provides a lower bound) is currently 0.75, then +// 0.75*peak-flush-rate will be used to set the flush tokens. The lower bound +// of 0.5 should not need to be tuned, and should not be tuned without +// consultation with a domain expert. If the storage.write-stall-nanos +// indicates significant write stalls, and the granter logs show that the +// dynamic target fraction has already reached the lower bound, one can +// consider lowering it slightly and then observe the effect. +var MinFlushUtilizationFraction = settings.RegisterFloatSetting( + settings.SystemOnly, + "admission.min_flush_util_fraction", + "when computing flush tokens, this fraction is a lower bound on the dynamically "+ + "adjusted flush utilization target fraction that attempts to reduce write stalls. Set "+ + "it to a high fraction (>>1, e.g. 10), to disable flush based tokens. The dynamic "+ + "target fraction is used to multiply the (measured) peak flush rate, to compute the flush "+ + "tokens. If the storage.write-stall-nanos indicates significant write stalls, and the granter "+ + "logs show that the dynamic target fraction has already reached the lower bound, one can "+ + "consider lowering it slightly (after consultation with domain experts)", 0.5, + settings.PositiveFloat) + // grantChainID is the ID for a grant chain. See continueGrantChain for // details. type grantChainID uint64 @@ -576,7 +602,11 @@ type kvStoreTokenGranter struct { requester requester // There is no rate limiting in granting these tokens. That is, they are all // burst tokens. - availableIOTokens int64 + availableIOTokens int64 + // startingIOTokens is the number of tokens set by + // setAvailableIOTokensLocked. It is used to compute the tokens used, by + // computing startingIOTokens-availableIOTokens. + startingIOTokens int64 ioTokensExhaustedDurationMetric *metric.Counter exhaustedStart time.Time } @@ -643,7 +673,8 @@ func (sg *kvStoreTokenGranter) continueGrantChain(grantChainID grantChainID) { sg.coord.continueGrantChain(KVWork, grantChainID) } -func (sg *kvStoreTokenGranter) setAvailableIOTokensLocked(tokens int64) { +func (sg *kvStoreTokenGranter) setAvailableIOTokensLocked(tokens int64) (tokensUsed int64) { + tokensUsed = sg.startingIOTokens - sg.availableIOTokens // It is possible for availableIOTokens to be negative because of // tookWithoutPermission or because tryGet will satisfy requests until // availableIOTokens become <= 0. We want to remember this previous @@ -653,6 +684,8 @@ func (sg *kvStoreTokenGranter) setAvailableIOTokensLocked(tokens int64) { // Clamp to tokens. sg.availableIOTokens = tokens } + sg.startingIOTokens = tokens + return tokensUsed } // GrantCoordinator is the top-level object that coordinates grants across @@ -997,7 +1030,7 @@ func appendMetricStructsForQueues(ms []metric.Struct, coord *GrantCoordinator) [ // pebbleMetricsTick is called every adjustmentInterval seconds and passes // through to the ioLoadListener, so that it can adjust the plan for future IO // token allocations. -func (coord *GrantCoordinator) pebbleMetricsTick(ctx context.Context, m *pebble.Metrics) { +func (coord *GrantCoordinator) pebbleMetricsTick(ctx context.Context, m StoreMetrics) { coord.ioLoadListener.pebbleMetricsTick(ctx, m) } @@ -1009,10 +1042,8 @@ func (coord *GrantCoordinator) allocateIOTokensTick() { if !coord.grantChainActive { coord.tryGrant() } - // Else, let the grant chain finish. We could terminate it, but token - // replenishment occurs at 1s granularity which is coarse enough to not - // bother. Also, in production we turn off grant chains on the - // GrantCoordinators used for IO, so we will always call tryGrant. + // Else, let the grant chain finish. NB: we turn off grant chains on the + // GrantCoordinators used for IO, so the if-condition is always true. } // testingTryGrant is only for unit tests, since they sometimes cut out @@ -1378,7 +1409,7 @@ func (sgc *StoreGrantCoordinators) SetPebbleMetricsProvider( if !loaded { sgc.numStores++ } - gc.pebbleMetricsTick(startupCtx, m.Metrics) + gc.pebbleMetricsTick(startupCtx, m) gc.allocateIOTokensTick() } if sgc.disableTickerForTesting { @@ -1389,13 +1420,13 @@ func (sgc *StoreGrantCoordinators) SetPebbleMetricsProvider( go func() { var ticks int64 - ticker := time.NewTicker(time.Second) + ticker := time.NewTicker(ioTokenTickDuration) done := false for !done { select { case <-ticker.C: ticks++ - if ticks%adjustmentInterval == 0 { + if ticks%ticksInAdjustmentInterval == 0 { metrics := sgc.pebbleMetricsProvider.GetPebbleMetrics() if len(metrics) != sgc.numStores { log.Warningf(ctx, @@ -1404,7 +1435,7 @@ func (sgc *StoreGrantCoordinators) SetPebbleMetricsProvider( for _, m := range metrics { if unsafeGc, ok := sgc.gcMap.Load(int64(m.StoreID)); ok { gc := (*GrantCoordinator)(unsafeGc) - gc.pebbleMetricsTick(ctx, m.Metrics) + gc.pebbleMetricsTick(ctx, m) iotc.UpdateIOThreshold(roachpb.StoreID(m.StoreID), gc.ioLoadListener.ioThreshold) } else { log.Warningf(ctx, @@ -1745,6 +1776,8 @@ type IOThresholdConsumer interface { type StoreMetrics struct { StoreID int32 *pebble.Metrics + WriteStallCount int64 + *pebble.InternalIntervalMetrics } // granterWithIOTokens is used to abstract kvStoreTokenGranter for testing. @@ -1754,8 +1787,11 @@ type granterWithIOTokens interface { // tight bound when the callee has negative available tokens, due to the use // of granter.tookWithoutPermission, since in that the case the callee // increments that negative value with the value provided by tokens. This - // method needs to be called periodically. - setAvailableIOTokensLocked(tokens int64) + // method needs to be called periodically. The return value is the number of + // used tokens in the interval since the prior call to this method. Note + // that tokensUsed can be negative, though that will be rare, since it is + // possible for tokens to be returned. + setAvailableIOTokensLocked(tokens int64) (tokensUsed int64) } // storeAdmissionStats are stats maintained by a storeRequester. The non-test @@ -1819,6 +1855,8 @@ type ioLoadListenerState struct { cumL0AddedBytes uint64 // Gauge. curL0Bytes int64 + // Cumulative. + cumWriteStallCount int64 // Exponentially smoothed per interval values. @@ -1829,13 +1867,25 @@ type ioLoadListenerState struct { // is determined when the work is done, as then the L0 fraction is known // precisely. smoothedIntIngestedAccountedL0BytesFraction float64 // '1.0' means: all ingested bytes went to L0 - smoothedTotalNumByteTokens float64 // smoothed history of token bucket sizes + // Smoothed history of byte tokens calculated based on compactions out of L0. + smoothedCompactionByteTokens float64 + + // Smoothed history of flush tokens calculated based on memtable flushes, + // before multiplying by target fraction. + smoothedNumFlushTokens float64 + // The target fraction to be used for the effective flush tokens. It is in + // the interval + // [MinFlushUtilizationFraction,maxFlushUtilTargetFraction]. + flushUtilTargetFraction float64 // totalNumByteTokens represents the tokens to give out until the next call to // adjustTokens. They are parceled out in small intervals. tokensAllocated // represents what has been given out. totalNumByteTokens int64 tokensAllocated int64 + // Used tokens can be negative if some tokens taken in one interval were + // returned in another, but that will be extremely rare. + tokensUsed int64 } // ioLoadListener adjusts tokens in kvStoreTokenGranter for IO, specifically due to @@ -1863,12 +1913,12 @@ const unlimitedTokens = math.MaxInt64 // Token changes are made at a coarse time granularity of 15s since // compactions can take ~10s to complete. The totalNumByteTokens to give out over -// the 15s interval are given out in a smoothed manner, at 1s intervals. +// the 15s interval are given out in a smoothed manner, at 250ms intervals. // This has similarities with the following kinds of token buckets: // - Zero replenishment rate and a burst value that is changed every 15s. We // explicitly don't want a huge burst every 15s. -// - A replenishment rate equal to totalNumByteTokens/15, with a burst capped at -// totalNumByteTokens/15. The only difference with the code here is that if +// - A replenishment rate equal to totalNumByteTokens/60, with a burst capped at +// totalNumByteTokens/60. The only difference with the code here is that if // totalNumByteTokens is small, the integer rounding effects are compensated for. // // In an experiment with extreme overload using KV0 with block size 64KB, @@ -1898,18 +1948,35 @@ const unlimitedTokens = math.MaxInt64 // reasonable to simply use metrics that are updated when compactions // complete (as opposed to also tracking progress in bytes of on-going // compactions). +// +// The 250ms interval to hand out the computed tokens is due to the needs of +// flush tokens. For compaction tokens, a 1s interval is fine-grained enough. +// +// If flushing a memtable take 100ms, then 10 memtables can be sustainably +// flushed in 1s. If we dole out flush tokens in 1s intervals, then there are +// enough tokens to create 10 memtables at the very start of a 1s interval, +// which will cause a write stall. Intuitively, the faster it is to flush a +// memtable, the smaller the interval for doling out these tokens. We have +// observed flushes taking ~0.5s, so we picked a 250ms interval for doling out +// these tokens. We could use a value smaller than 250ms, but we've observed +// CPU utilization issues at lower intervals (see the discussion in +// runnable.go). const adjustmentInterval = 15 +const ticksInAdjustmentInterval = 60 +const ioTokenTickDuration = 250 * time.Millisecond // pebbleMetricsTicks is called every adjustmentInterval seconds, and decides // the token allocations until the next call. -func (io *ioLoadListener) pebbleMetricsTick(ctx context.Context, m *pebble.Metrics) { +func (io *ioLoadListener) pebbleMetricsTick(ctx context.Context, metrics StoreMetrics) { + m := metrics.Metrics if !io.statsInitialized { io.statsInitialized = true io.adjustTokensResult = adjustTokensResult{ ioLoadListenerState: ioLoadListenerState{ - cumAdmissionStats: io.kvRequester.getStoreAdmissionStats(), - cumL0AddedBytes: m.Levels[0].BytesFlushed + m.Levels[0].BytesIngested, - curL0Bytes: m.Levels[0].Size, + cumAdmissionStats: io.kvRequester.getStoreAdmissionStats(), + cumL0AddedBytes: m.Levels[0].BytesFlushed + m.Levels[0].BytesIngested, + curL0Bytes: m.Levels[0].Size, + cumWriteStallCount: metrics.WriteStallCount, // Reasonable starting fraction until we see some ingests. smoothedIntIngestedAccountedL0BytesFraction: 0.5, // No initial limit, i.e, the first interval is unlimited. @@ -1926,21 +1993,21 @@ func (io *ioLoadListener) pebbleMetricsTick(ctx context.Context, m *pebble.Metri } return } - io.adjustTokens(ctx, m) + io.adjustTokens(ctx, metrics) } -// allocateTokensTick gives out 1/adjustmentInterval of the totalNumByteTokens every -// 1s. +// allocateTokensTick gives out 1/ticksInAdjustmentInterval of the +// totalNumByteTokens every 250ms. func (io *ioLoadListener) allocateTokensTick() { var toAllocate int64 // unlimitedTokens==MaxInt64, so avoid overflow in the rounding up // calculation. - if io.totalNumByteTokens >= unlimitedTokens-(adjustmentInterval-1) { - toAllocate = io.totalNumByteTokens / adjustmentInterval + if io.totalNumByteTokens >= unlimitedTokens-(ticksInAdjustmentInterval-1) { + toAllocate = io.totalNumByteTokens / ticksInAdjustmentInterval } else { // Round up so that we don't accumulate tokens to give in a burst on the // last tick. - toAllocate = (io.totalNumByteTokens + adjustmentInterval - 1) / adjustmentInterval + toAllocate = (io.totalNumByteTokens + ticksInAdjustmentInterval - 1) / ticksInAdjustmentInterval if toAllocate < 0 { panic(errors.AssertionFailedf("toAllocate is negative %d", toAllocate)) } @@ -1955,25 +2022,39 @@ func (io *ioLoadListener) allocateTokensTick() { if io.tokensAllocated < 0 { panic(errors.AssertionFailedf("tokens allocated is negative %d", io.tokensAllocated)) } - io.mu.kvGranter.setAvailableIOTokensLocked(toAllocate) + io.tokensUsed += io.mu.kvGranter.setAvailableIOTokensLocked(toAllocate) } // adjustTokens computes a new value of totalNumByteTokens (and resets // tokensAllocated). The new value, when overloaded, is based on comparing how // many bytes are being moved out of L0 via compactions with the average // number of bytes being added to L0 per KV work. We want the former to be -// (significantly) larger so that L0 returns to a healthy state. -func (io *ioLoadListener) adjustTokens(ctx context.Context, m *pebble.Metrics) { - res := io.adjustTokensInner(ctx, io.ioLoadListenerState, io.kvRequester.getStoreAdmissionStats(), m.Levels[0], - L0FileCountOverloadThreshold.Get(&io.settings.SV), L0SubLevelCountOverloadThreshold.Get(&io.settings.SV), +// (significantly) larger so that L0 returns to a healthy state. The byte +// token computation also takes into account the flush throughput, since an +// inability to flush fast enough can result in write stalls due to high +// memtable counts, which we want to avoid as it can cause latency hiccups of +// 100+ms for all write traffic. +func (io *ioLoadListener) adjustTokens(ctx context.Context, metrics StoreMetrics) { + res := io.adjustTokensInner(ctx, io.ioLoadListenerState, io.kvRequester.getStoreAdmissionStats(), + metrics.Levels[0], metrics.WriteStallCount, metrics.InternalIntervalMetrics, + L0FileCountOverloadThreshold.Get(&io.settings.SV), + L0SubLevelCountOverloadThreshold.Get(&io.settings.SV), + MinFlushUtilizationFraction.Get(&io.settings.SV), ) io.adjustTokensResult = res io.kvRequester.setStoreRequestEstimates(res.requestEstimates) - if _, overloaded := res.ioThreshold.Score(); overloaded { + if _, overloaded := res.ioThreshold.Score(); overloaded || res.aux.doLogFlush { log.Infof(logtags.AddTag(ctx, "s", io.storeID), "IO overload: %s", res) } } +type tokenKind int8 + +const ( + compactionTokenKind tokenKind = iota + flushTokenKind +) + type adjustTokensAuxComputations struct { intL0AddedBytes int64 intL0CompactedBytes int64 @@ -1987,6 +2068,15 @@ type adjustTokensAuxComputations struct { intPerWorkUnaccountedL0Bytes float64 l0BytesIngestFraction float64 + + intFlushTokens float64 + intFlushUtilization float64 + intWriteStalls int64 + + prevTokensUsed int64 + tokenKind tokenKind + + doLogFlush bool } func (*ioLoadListener) adjustTokensInner( @@ -1994,7 +2084,10 @@ func (*ioLoadListener) adjustTokensInner( prev ioLoadListenerState, cumAdmissionStats storeAdmissionStats, l0Metrics pebble.LevelMetrics, + cumWriteStallCount int64, + im *pebble.InternalIntervalMetrics, threshNumFiles, threshNumSublevels int64, + minFlushUtilTargetFraction float64, ) adjustTokensResult { ioThreshold := &admissionpb.IOThreshold{ L0NumFiles: l0Metrics.NumFiles, @@ -2106,9 +2199,192 @@ func (*ioLoadListener) adjustTokensInner( smoothedIntIngestedAccountedL0BytesFraction = prev.smoothedIntIngestedAccountedL0BytesFraction } - // We constrain admission if the store is over the threshold. + // Flush tokens: + // + // Write stalls happen when flushing of memtables is a bottleneck. + // + // Computing Flush Tokens: + // Flush can go from not being the bottleneck in one 15s interval + // (adjustmentInterval) to being the bottleneck in the next 15s interval + // (e.g. when L0 falls below the unhealthy threshold and compaction tokens + // become unlimited). So the flush token limit has to react quickly (cannot + // afford to wait for multiple 15s intervals). We've observed that if we + // normalize the flush rate based on flush loop utilization (the PeakRate + // computation below), and use that to compute flush tokens, the token + // counts are quite stable. Here are two examples, showing this steady token + // count computed using PeakRate of the flush ThroughputMetric, despite + // changes in flush loop utilization (the util number below). + // + // Example 1: Case where IO bandwidth was not a bottleneck + // flush: tokens: 2312382401, util: 0.90 + // flush: tokens: 2345477107, util: 0.31 + // flush: tokens: 2317829891, util: 0.29 + // flush: tokens: 2428387843, util: 0.17 + // + // Example 2: Case where IO bandwidth became a bottleneck (and mean fsync + // latency was fluctuating between 1ms and 4ms in the low util to high util + // cases). + // + // flush: tokens: 1406132615, util: 1.00 + // flush: tokens: 1356476227, util: 0.64 + // flush: tokens: 1374880806, util: 0.24 + // flush: tokens: 1328578534, util: 0.96 + // + // Hence, using PeakRate as a basis for computing flush tokens seems sound. + // The other important question is what fraction of PeakRate avoids write + // stalls. It is likely less than 100% since while a flush is ongoing, + // memtables can accumulate and cause a stall. For example, we have observed + // write stalls at 80% of PeakRate. The fraction depends on configuration + // parameters like MemTableStopWritesThreshold (defaults to 4 in + // CockroachDB), and environmental and workload factors like how long a + // flush takes to flush a single 64MB memtable. Instead of trying to measure + // and adjust for these, we use a simple multiplier, + // flushUtilTargetFraction. By default, flushUtilTargetFraction ranges + // between 0.5 and 1.5. The lower bound is configurable via + // admission.min_flush_util_percent and if configured above the upper bound, + // the upper bound will be ignored and the target fraction will not be + // dynamically adjusted. The dynamic adjustment logic uses an additive step + // size of flushUtilTargetFractionIncrement (0.025), with the following + // logic: + // - Reduce the fraction if there is a write-stall. The reduction may use a + // small multiple of flushUtilTargetFractionIncrement. This is so that + // this probing spends more time below the threshold where write stalls + // occur. + // - Increase fraction if no write-stall and flush tokens were almost all + // used. + // + // This probing unfortunately cannot eliminate write stalls altogether. + // Future improvements could use more history to settle on a good + // flushUtilTargetFraction for longer, or use some measure of how close we + // are to a write-stall to stop the increase. + // + // Ingestion and flush tokens: + // + // Ingested sstables do not utilize any flush capacity. Consider 2 cases: + // - sstable ingested into L0: there was either data overlap with L0, or + // file boundary overlap with L0-L6. To be conservative, lets assume there + // was data overlap, and that this data overlap extended into the memtable + // at the time of ingestion. Memtable(s) would have been force flushed to + // handle such overlap. The cost of flushing a memtable is based on how + // much of the allocated memtable capacity is used, so an early flush + // seems harmless. However, write stalls are based on allocated memtable + // capacity, so there is a potential negative interaction of these forced + // flushes since they cause additional memtable capacity allocation. + // - sstable ingested into L1-L6: there was no data overlap with L0, which + // implies that there was no reason to flush memtables. + // + // Since there is some interaction noted in bullet 1, and because it + // simplifies the admission control token behavior, we use flush tokens in + // an identical manner as compaction tokens -- to be consumed by all data + // flowing into L0. Some of this conservative choice will be compensated for + // by flushUtilTargetFraction (when the mix of ingestion and actual flushes + // are stable). Another thing to note is that compactions out of L0 are + // typically the more persistent bottleneck than flushes for the following + // reason: + // There is a dedicated flush thread. With a maximum compaction concurrency + // of C, we have up to C threads dedicated to handling the write-amp of W + // (caused by rewriting the same data). So C/(W-1) threads on average are + // reading the original data (that will be rewritten W-1 times). Since L0 + // can have multiple overlapping files, and intra-L0 compactions are usually + // avoided, we can assume (at best) that the original data (in L0) is being + // read only when compacting to levels lower than L0. That is, C/(W-1) + // threads are reading from L0 to compact to levels lower than L0. Since W + // can be 20+ and C defaults to 3 (we plan to dynamically adjust C but one + // can expect C to be <= 10), C/(W-1) < 1. So the main reason we are + // considering flush tokens is transient flush bottlenecks, and workloads + // where W is small. + + // Compute flush utilization for this interval. A very low flush utilization + // will cause flush tokens to be unlimited. + intFlushUtilization := float64(0) + if im.Flush.WriteThroughput.WorkDuration > 0 { + intFlushUtilization = float64(im.Flush.WriteThroughput.WorkDuration) / + float64(im.Flush.WriteThroughput.WorkDuration+im.Flush.WriteThroughput.IdleDuration) + } + // Compute flush tokens for this interval that would cause 100% utilization. + intFlushTokens := float64(im.Flush.WriteThroughput.PeakRate()) * adjustmentInterval + intWriteStalls := cumWriteStallCount - prev.cumWriteStallCount + + // Ensure flushUtilTargetFraction is in the configured bounds. This also + // does lazy initialization. + const maxFlushUtilTargetFraction = 1.5 + flushUtilTargetFraction := prev.flushUtilTargetFraction + if flushUtilTargetFraction == 0 { + // Initialization: use the maximum configured fraction. + flushUtilTargetFraction = minFlushUtilTargetFraction + if flushUtilTargetFraction < maxFlushUtilTargetFraction { + flushUtilTargetFraction = maxFlushUtilTargetFraction + } + } else if flushUtilTargetFraction < minFlushUtilTargetFraction { + // The min can be changed in a running system, so we bump up to conform to + // the min. + flushUtilTargetFraction = minFlushUtilTargetFraction + } + numFlushTokens := int64(unlimitedTokens) + // doLogFlush becomes true if something interesting is done here. + doLogFlush := false + smoothedNumFlushTokens := prev.smoothedNumFlushTokens + const flushUtilIgnoreThreshold = 0.05 + if intFlushUtilization > flushUtilIgnoreThreshold { + if smoothedNumFlushTokens == 0 { + // Initialization. + smoothedNumFlushTokens = intFlushTokens + } else { + smoothedNumFlushTokens = alpha*intFlushTokens + (1-alpha)*prev.smoothedNumFlushTokens + } + const flushUtilTargetFractionIncrement = 0.025 + // Have we used, over the last (15s) cycle, more than 90% of the tokens we + // would give out for the next cycle? If yes, highTokenUsage is true. + highTokenUsage := + float64(prev.tokensUsed) >= 0.9*smoothedNumFlushTokens*flushUtilTargetFraction + if intWriteStalls > 0 { + // Try decrease since there were write-stalls. + numDecreaseSteps := 1 + // These constants of 5, 3, 2, 2 were found to work reasonably well, + // without causing large decreases. We need better benchmarking to tune + // such constants. + if intWriteStalls >= 5 { + numDecreaseSteps = 3 + } else if intWriteStalls >= 2 { + numDecreaseSteps = 2 + } + for i := 0; i < numDecreaseSteps; i++ { + if flushUtilTargetFraction >= minFlushUtilTargetFraction+flushUtilTargetFractionIncrement { + flushUtilTargetFraction -= flushUtilTargetFractionIncrement + doLogFlush = true + } else { + break + } + } + } else if flushUtilTargetFraction < maxFlushUtilTargetFraction-flushUtilTargetFractionIncrement && + intWriteStalls == 0 && highTokenUsage { + // No write-stalls, and token usage was high, so give out more tokens. + flushUtilTargetFraction += flushUtilTargetFractionIncrement + doLogFlush = true + } + if highTokenUsage { + doLogFlush = true + } + flushTokensFloat := flushUtilTargetFraction * smoothedNumFlushTokens + if flushTokensFloat < float64(math.MaxInt64) { + numFlushTokens = int64(flushTokensFloat) + } + // Else avoid overflow by using the previously set unlimitedTokens. This + // should not really happen. + } + // Else intFlushUtilization is too low. We don't want to make token + // determination based on a very low utilization, so we hand out unlimited + // tokens. Note that flush utilization has been observed to fluctuate from + // 0.16 to 0.9 in a single interval, when compaction tokens are not limited, + // hence we have set flushUtilIgnoreThreshold to a very low value. If we've + // erred towards it being too low, we run the risk of computing incorrect + // tokens. If we've erred towards being too high, we run the risk of giving + // out unlimitedTokens and causing write stalls. + + // We constrain admission based on compactions, if the store is over the L0 + // threshold. var totalNumByteTokens int64 - var smoothedTotalNumByteTokens float64 + var smoothedCompactionByteTokens float64 _, overloaded := ioThreshold.Score() if overloaded { @@ -2121,12 +2397,12 @@ func (*ioLoadListener) adjustTokensInner( // Smooth it. This may seem peculiar since we are already using // smoothedIntL0CompactedBytes, but the else clause below uses a different // computation so we also want the history of smoothedTotalNumByteTokens. - smoothedTotalNumByteTokens = alpha*fTotalNumByteTokens + (1-alpha)*prev.smoothedTotalNumByteTokens - if float64(math.MaxInt64) < smoothedTotalNumByteTokens { + smoothedCompactionByteTokens = alpha*fTotalNumByteTokens + (1-alpha)*prev.smoothedCompactionByteTokens + if float64(math.MaxInt64) < smoothedCompactionByteTokens { // Avoid overflow. This should not really happen. totalNumByteTokens = math.MaxInt64 } else { - totalNumByteTokens = int64(smoothedTotalNumByteTokens) + totalNumByteTokens = int64(smoothedCompactionByteTokens) } } else { // Under the threshold. Maintain a smoothedTotalNumByteTokens based on what was @@ -2135,21 +2411,32 @@ func (*ioLoadListener) adjustTokensInner( // we've seen extreme situations with alternating 15s intervals of above // and below the threshold. numTokens := intL0CompactedBytes - smoothedTotalNumByteTokens = alpha*float64(numTokens) + (1-alpha)*prev.smoothedTotalNumByteTokens + smoothedCompactionByteTokens = alpha*float64(numTokens) + (1-alpha)*prev.smoothedCompactionByteTokens totalNumByteTokens = unlimitedTokens } + // Use the minimum of the token count calculated using compactions and + // flushes. + tokenKind := compactionTokenKind + if totalNumByteTokens > numFlushTokens { + totalNumByteTokens = numFlushTokens + tokenKind = flushTokenKind + } // Install the latest cumulative stats. return adjustTokensResult{ ioLoadListenerState: ioLoadListenerState{ cumAdmissionStats: cumAdmissionStats, cumL0AddedBytes: cumL0AddedBytes, curL0Bytes: curL0Bytes, + cumWriteStallCount: cumWriteStallCount, smoothedIntL0CompactedBytes: smoothedIntL0CompactedBytes, smoothedIntPerWorkUnaccountedL0Bytes: smoothedIntPerWorkUnaccountedL0Bytes, smoothedIntIngestedAccountedL0BytesFraction: smoothedIntIngestedAccountedL0BytesFraction, - smoothedTotalNumByteTokens: smoothedTotalNumByteTokens, + smoothedCompactionByteTokens: smoothedCompactionByteTokens, + smoothedNumFlushTokens: smoothedNumFlushTokens, + flushUtilTargetFraction: flushUtilTargetFraction, totalNumByteTokens: totalNumByteTokens, tokensAllocated: 0, + tokensUsed: 0, }, requestEstimates: storeRequestEstimates{ workByteAddition: max(1, int64(smoothedIntPerWorkUnaccountedL0Bytes)), @@ -2166,6 +2453,12 @@ func (*ioLoadListener) adjustTokensInner( intUnaccountedL0Bytes: intUnaccountedL0Bytes, intPerWorkUnaccountedL0Bytes: intPerWorkUnaccountedL0Bytes, l0BytesIngestFraction: intIngestedAccountedL0BytesFraction, + intFlushTokens: intFlushTokens, + intFlushUtilization: intFlushUtilization, + intWriteStalls: intWriteStalls, + prevTokensUsed: prev.tokensUsed, + tokenKind: tokenKind, + doLogFlush: doLogFlush, }, ioThreshold: ioThreshold, } @@ -2188,7 +2481,7 @@ func max(i, j int64) int64 { func (res adjustTokensResult) SafeFormat(p redact.SafePrinter, _ rune) { ib := humanizeutil.IBytes // NB: "≈" indicates smoothed quantities. - p.Printf("score %v (%d ssts, %d sub-levels), ", res.ioThreshold, res.ioThreshold.L0NumFiles, res.ioThreshold.L0NumSubLevels) + p.Printf("compaction 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". @@ -2202,14 +2495,19 @@ func (res adjustTokensResult) SafeFormat(p redact.SafePrinter, _ rune) { p.Printf("%s unacc [≈%s/req, n=%d], ", ib(res.aux.intUnaccountedL0Bytes), ib(int64(res.smoothedIntPerWorkUnaccountedL0Bytes)), res.aux.intAdmittedCount) // How much got compacted out of L0 recently. - p.Printf("compacted %s [≈%s]; ", ib(res.aux.intL0CompactedBytes), ib(res.smoothedIntL0CompactedBytes)) - + p.Printf("compacted %s [≈%s], ", ib(res.aux.intL0CompactedBytes), ib(res.smoothedIntL0CompactedBytes)) + p.Printf("flushed %s [≈%s]; ", ib(int64(res.aux.intFlushTokens)), + ib(int64(res.smoothedNumFlushTokens))) p.Printf("admitting ") if n := res.ioLoadListenerState.totalNumByteTokens; n < unlimitedTokens { - p.Printf("%s", ib(n)) - if n := res.ioLoadListenerState.tokensAllocated; 0 < n && n < unlimitedTokens/adjustmentInterval { - p.Printf("(used %s)", ib(n)) + p.Printf("%s (rate %s/s)", ib(n), ib(n/adjustmentInterval)) + switch res.aux.tokenKind { + case compactionTokenKind: + p.Printf(" due to L0 growth") + case flushTokenKind: + p.Printf(" due to memtable flush (multiplier %.3f)", res.flushUtilTargetFraction) } + p.Printf(" (used %s)", ib(res.aux.prevTokensUsed)) p.Printf(" with L0 penalty: +%s/req, *%.2f/ingest", ib(res.requestEstimates.workByteAddition), res.requestEstimates.fractionOfIngestIntoL0, ) diff --git a/pkg/util/admission/granter_test.go b/pkg/util/admission/granter_test.go index dd19ae6fe243..c473f5e10caa 100644 --- a/pkg/util/admission/granter_test.go +++ b/pkg/util/admission/granter_test.go @@ -448,15 +448,20 @@ func (r *testRequesterForIOLL) setStoreRequestEstimates(estimates storeRequestEs } type testGranterWithIOTokens struct { - buf strings.Builder + buf strings.Builder + allTokensUsed bool } -func (g *testGranterWithIOTokens) setAvailableIOTokensLocked(tokens int64) { - fmt.Fprintf(&g.buf, "setAvailableIOTokens: %s", tokensFor1sToString(tokens)) +func (g *testGranterWithIOTokens) setAvailableIOTokensLocked(tokens int64) (tokensUsed int64) { + fmt.Fprintf(&g.buf, "setAvailableIOTokens: %s", tokensForTokenTickDurationToString(tokens)) + if g.allTokensUsed { + return tokens * 2 + } + return 0 } -func tokensFor1sToString(tokens int64) string { - if tokens >= unlimitedTokens/adjustmentInterval { +func tokensForTokenTickDurationToString(tokens int64) string { + if tokens >= unlimitedTokens/ticksInAdjustmentInterval { return "unlimited" } return fmt.Sprintf("%d", tokens) @@ -467,7 +472,7 @@ type rawTokenResult adjustTokensResult // TestIOLoadListener is a datadriven test with the following command that // sets the state for token calculation and then ticks adjustmentInterval // times to cause tokens to be set in the testGranterWithIOTokens: -// set-state admitted= l0-bytes= l0-added= l0-files= l0-sublevels= +// set-state admitted= l0-bytes= l0-added= l0-files= l0-sublevels= ... func TestIOLoadListener(t *testing.T) { req := &testRequesterForIOLL{} kvGranter := &testGranterWithIOTokens{} @@ -510,6 +515,12 @@ func TestIOLoadListener(t *testing.T) { } return fmt.Sprintf("%+v", req.stats) + case "set-min-flush-util": + var percent int + d.ScanArgs(t, "percent", &percent) + MinFlushUtilizationFraction.Override(ctx, &st.SV, float64(percent)/100) + return "" + case "set-state": // Setup state used as input for token adjustment. var metrics pebble.Metrics @@ -526,8 +537,38 @@ func TestIOLoadListener(t *testing.T) { var l0SubLevels int d.ScanArgs(t, "l0-sublevels", &l0SubLevels) metrics.Levels[0].Sublevels = int32(l0SubLevels) + var flushBytes, flushWorkSec, flushIdleSec int + if d.HasArg("flush-bytes") { + d.ScanArgs(t, "flush-bytes", &flushBytes) + d.ScanArgs(t, "flush-work-sec", &flushWorkSec) + d.ScanArgs(t, "flush-idle-sec", &flushIdleSec) + } + flushMetric := pebble.ThroughputMetric{ + Bytes: int64(flushBytes), + WorkDuration: time.Duration(flushWorkSec) * time.Second, + IdleDuration: time.Duration(flushIdleSec) * time.Second, + } + im := &pebble.InternalIntervalMetrics{} + im.Flush.WriteThroughput = flushMetric + var writeStallCount int + if d.HasArg("write-stall-count") { + d.ScanArgs(t, "write-stall-count", &writeStallCount) + } + var allTokensUsed bool + if d.HasArg("all-tokens-used") { + d.ScanArgs(t, "all-tokens-used", &allTokensUsed) + } + kvGranter.allTokensUsed = allTokensUsed + var printOnlyFirstTick bool + if d.HasArg("print-only-first-tick") { + d.ScanArgs(t, "print-only-first-tick", &printOnlyFirstTick) + } + ioll.pebbleMetricsTick(ctx, StoreMetrics{ + Metrics: &metrics, + WriteStallCount: int64(writeStallCount), + InternalIntervalMetrics: im, + }) var buf strings.Builder - ioll.pebbleMetricsTick(ctx, &metrics) // Do the ticks until just before next adjustment. res := ioll.adjustTokensResult fmt.Fprintln(&buf, redact.StringWithoutMarkers(&res)) @@ -537,9 +578,11 @@ func TestIOLoadListener(t *testing.T) { fmt.Fprintf(&buf, "%s\n", req.buf.String()) req.buf.Reset() } - for i := 0; i < adjustmentInterval; i++ { + for i := 0; i < ticksInAdjustmentInterval; i++ { ioll.allocateTokensTick() - fmt.Fprintf(&buf, "tick: %d, %s\n", i, kvGranter.buf.String()) + if i == 0 || !printOnlyFirstTick { + fmt.Fprintf(&buf, "tick: %d, %s\n", i, kvGranter.buf.String()) + } kvGranter.buf.Reset() } return buf.String() @@ -566,7 +609,7 @@ func TestIOLoadListenerOverflow(t *testing.T) { // Override the totalNumByteTokens manually to trigger the overflow bug. ioll.totalNumByteTokens = math.MaxInt64 - i ioll.tokensAllocated = 0 - for j := 0; j < adjustmentInterval; j++ { + for j := 0; j < ticksInAdjustmentInterval; j++ { ioll.allocateTokensTick() } } @@ -576,8 +619,10 @@ func TestIOLoadListenerOverflow(t *testing.T) { Sublevels: 100, NumFiles: 10000, } - ioll.pebbleMetricsTick(ctx, &m) - ioll.pebbleMetricsTick(ctx, &m) + ioll.pebbleMetricsTick(ctx, + StoreMetrics{Metrics: &m, InternalIntervalMetrics: &pebble.InternalIntervalMetrics{}}) + ioll.pebbleMetricsTick(ctx, + StoreMetrics{Metrics: &m, InternalIntervalMetrics: &pebble.InternalIntervalMetrics{}}) ioll.allocateTokensTick() } @@ -585,8 +630,9 @@ type testGranterNonNegativeTokens struct { t *testing.T } -func (g *testGranterNonNegativeTokens) setAvailableIOTokensLocked(tokens int64) { +func (g *testGranterNonNegativeTokens) setAvailableIOTokensLocked(tokens int64) (tokensUsed int64) { require.LessOrEqual(g.t, int64(0), tokens) + return 0 } func TestAdjustTokensInnerAndLogging(t *testing.T) { @@ -622,7 +668,7 @@ func TestAdjustTokensInnerAndLogging(t *testing.T) { smoothedIntL0CompactedBytes: 47 * mb, smoothedIntPerWorkUnaccountedL0Bytes: 2204, // 2kb smoothedIntIngestedAccountedL0BytesFraction: 0.3, - smoothedTotalNumByteTokens: 201 * mb, + smoothedCompactionByteTokens: 201 * mb, totalNumByteTokens: int64(201 * mb), }, admissionStats: admStats, @@ -640,7 +686,8 @@ func TestAdjustTokensInnerAndLogging(t *testing.T) { for _, tt := range tests { buf.Printf("%s:\n", tt.name) res := (*ioLoadListener)(nil).adjustTokensInner( - ctx, tt.prev, tt.admissionStats, tt.l0Metrics, 100, 10) + ctx, tt.prev, tt.admissionStats, tt.l0Metrics, 0, + &pebble.InternalIntervalMetrics{}, 100, 10, 0.50) buf.Printf("%s\n", res) } echotest.Require(t, string(redact.Sprint(buf)), filepath.Join(testutils.TestDataPath(t, "format_adjust_tokens_stats.txt"))) @@ -680,13 +727,16 @@ func TestBadIOLoadListenerStats(t *testing.T) { ioll.mu.kvGranter = kvGranter for i := 0; i < 100; i++ { randomValues() - ioll.pebbleMetricsTick(ctx, &m) - for j := 0; j < adjustmentInterval; j++ { + ioll.pebbleMetricsTick(ctx, StoreMetrics{ + Metrics: &m, + InternalIntervalMetrics: &pebble.InternalIntervalMetrics{}, + }) + for j := 0; j < ticksInAdjustmentInterval; j++ { ioll.allocateTokensTick() require.LessOrEqual(t, int64(0), ioll.smoothedIntL0CompactedBytes) require.LessOrEqual(t, float64(0), ioll.smoothedIntPerWorkUnaccountedL0Bytes) require.LessOrEqual(t, float64(0), ioll.smoothedIntIngestedAccountedL0BytesFraction) - require.LessOrEqual(t, float64(0), ioll.smoothedTotalNumByteTokens) + require.LessOrEqual(t, float64(0), ioll.smoothedCompactionByteTokens) require.LessOrEqual(t, int64(0), ioll.totalNumByteTokens) require.LessOrEqual(t, int64(0), ioll.tokensAllocated) } diff --git a/pkg/util/admission/testdata/format_adjust_tokens_stats.txt b/pkg/util/admission/testdata/format_adjust_tokens_stats.txt index 70a2f3315147..5cc10c658b7c 100644 --- a/pkg/util/admission/testdata/format_adjust_tokens_stats.txt +++ b/pkg/util/admission/testdata/format_adjust_tokens_stats.txt @@ -1,6 +1,6 @@ echo ---- zero: -score 0.000 (0 ssts, 0 sub-levels), L0 growth 0 B: 0 B acc-write + 0 B acc-ingest + 0 B unacc [≈0 B/req, n=1], compacted 0 B [≈0 B]; admitting all +compaction score 0.000 (0 ssts, 0 sub-levels), L0 growth 0 B: 0 B acc-write + 0 B acc-ingest + 0 B unacc [≈0 B/req, n=1], compacted 0 B [≈0 B], flushed 0 B [≈0 B]; admitting all real-numbers: -score 2.700[overload] (195 ssts, 27 sub-levels), L0 growth 577 MiB: 178 MiB acc-write + 73 MiB acc-ingest + 326 MiB unacc [≈270 KiB/req, n=621], compacted 77 MiB [≈62 MiB]; admitting 116 MiB with L0 penalty: +270 KiB/req, *0.34/ingest +compaction score 2.700[L0-overload] (195 ssts, 27 sub-levels), L0 growth 577 MiB: 178 MiB acc-write + 73 MiB acc-ingest + 326 MiB unacc [≈270 KiB/req, n=621], compacted 77 MiB [≈62 MiB], flushed 0 B [≈0 B]; admitting 116 MiB (rate 7.7 MiB/s) due to L0 growth (used 0 B) with L0 penalty: +270 KiB/req, *0.34/ingest diff --git a/pkg/util/admission/testdata/granter b/pkg/util/admission/testdata/granter index a8127ea2e084..c37f76c0619f 100644 --- a/pkg/util/admission/testdata/granter +++ b/pkg/util/admission/testdata/granter @@ -378,14 +378,14 @@ sql-sql-response: avail: -1 sql-leaf-start: used: 0, total: 2 sql-root-start: us init-store-grant-coordinator ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 614891469123651720 +(chain: id: 0 active: false index: 5) io-avail: 153722867280912930 # Initial tokens are effectively unlimited. try-get work=kv v=10000 ---- kv: tryGet(10000) returned true GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 614891469123641720 +(chain: id: 0 active: false index: 5) io-avail: 153722867280902930 # Set the io tokens to a smaller value. set-io-tokens tokens=500 diff --git a/pkg/util/admission/testdata/io_load_listener b/pkg/util/admission/testdata/io_load_listener index c929e78a30fe..7b0ab669d39b 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 ---- -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:} +compaction 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], flushed 0 B [≈0 B]; admitting all +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:0 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:0 smoothedIntPerWorkUnaccountedL0Bytes:0 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 tokensAllocated:0 tokensUsed: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 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 doLogFlush:false} ioThreshold:} tick: 0, setAvailableIOTokens: unlimited tick: 1, setAvailableIOTokens: unlimited tick: 2, setAvailableIOTokens: unlimited @@ -28,6 +28,51 @@ tick: 11, setAvailableIOTokens: unlimited tick: 12, setAvailableIOTokens: unlimited tick: 13, setAvailableIOTokens: unlimited tick: 14, setAvailableIOTokens: unlimited +tick: 15, setAvailableIOTokens: unlimited +tick: 16, setAvailableIOTokens: unlimited +tick: 17, setAvailableIOTokens: unlimited +tick: 18, setAvailableIOTokens: unlimited +tick: 19, setAvailableIOTokens: unlimited +tick: 20, setAvailableIOTokens: unlimited +tick: 21, setAvailableIOTokens: unlimited +tick: 22, setAvailableIOTokens: unlimited +tick: 23, setAvailableIOTokens: unlimited +tick: 24, setAvailableIOTokens: unlimited +tick: 25, setAvailableIOTokens: unlimited +tick: 26, setAvailableIOTokens: unlimited +tick: 27, setAvailableIOTokens: unlimited +tick: 28, setAvailableIOTokens: unlimited +tick: 29, setAvailableIOTokens: unlimited +tick: 30, setAvailableIOTokens: unlimited +tick: 31, setAvailableIOTokens: unlimited +tick: 32, setAvailableIOTokens: unlimited +tick: 33, setAvailableIOTokens: unlimited +tick: 34, setAvailableIOTokens: unlimited +tick: 35, setAvailableIOTokens: unlimited +tick: 36, setAvailableIOTokens: unlimited +tick: 37, setAvailableIOTokens: unlimited +tick: 38, setAvailableIOTokens: unlimited +tick: 39, setAvailableIOTokens: unlimited +tick: 40, setAvailableIOTokens: unlimited +tick: 41, setAvailableIOTokens: unlimited +tick: 42, setAvailableIOTokens: unlimited +tick: 43, setAvailableIOTokens: unlimited +tick: 44, setAvailableIOTokens: unlimited +tick: 45, setAvailableIOTokens: unlimited +tick: 46, setAvailableIOTokens: unlimited +tick: 47, setAvailableIOTokens: unlimited +tick: 48, setAvailableIOTokens: unlimited +tick: 49, setAvailableIOTokens: unlimited +tick: 50, setAvailableIOTokens: unlimited +tick: 51, setAvailableIOTokens: unlimited +tick: 52, setAvailableIOTokens: unlimited +tick: 53, setAvailableIOTokens: unlimited +tick: 54, setAvailableIOTokens: unlimited +tick: 55, setAvailableIOTokens: unlimited +tick: 56, setAvailableIOTokens: unlimited +tick: 57, setAvailableIOTokens: unlimited +tick: 58, setAvailableIOTokens: unlimited +tick: 59, setAvailableIOTokens: unlimited prep-admission-stats admitted=10000 ---- @@ -39,24 +84,69 @@ 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 ---- -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:} +compaction score 1.050[L0-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], flushed 0 B [≈0 B]; admitting 12 KiB (rate 833 B/s) due to L0 growth (used 0 B) 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 cumWriteStallCount:0 smoothedIntL0CompactedBytes:50000 smoothedIntPerWorkUnaccountedL0Bytes:10 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedCompactionByteTokens:12500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:12500 tokensAllocated:0 tokensUsed: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 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 doLogFlush:false} ioThreshold:} store-request-estimates: fractionOfIngestIntoL0: 0.50, workByteAddition: 10 -tick: 0, setAvailableIOTokens: 834 -tick: 1, setAvailableIOTokens: 834 -tick: 2, setAvailableIOTokens: 834 -tick: 3, setAvailableIOTokens: 834 -tick: 4, setAvailableIOTokens: 834 -tick: 5, setAvailableIOTokens: 834 -tick: 6, setAvailableIOTokens: 834 -tick: 7, setAvailableIOTokens: 834 -tick: 8, setAvailableIOTokens: 834 -tick: 9, setAvailableIOTokens: 834 -tick: 10, setAvailableIOTokens: 834 -tick: 11, setAvailableIOTokens: 834 -tick: 12, setAvailableIOTokens: 834 -tick: 13, setAvailableIOTokens: 834 -tick: 14, setAvailableIOTokens: 824 +tick: 0, setAvailableIOTokens: 209 +tick: 1, setAvailableIOTokens: 209 +tick: 2, setAvailableIOTokens: 209 +tick: 3, setAvailableIOTokens: 209 +tick: 4, setAvailableIOTokens: 209 +tick: 5, setAvailableIOTokens: 209 +tick: 6, setAvailableIOTokens: 209 +tick: 7, setAvailableIOTokens: 209 +tick: 8, setAvailableIOTokens: 209 +tick: 9, setAvailableIOTokens: 209 +tick: 10, setAvailableIOTokens: 209 +tick: 11, setAvailableIOTokens: 209 +tick: 12, setAvailableIOTokens: 209 +tick: 13, setAvailableIOTokens: 209 +tick: 14, setAvailableIOTokens: 209 +tick: 15, setAvailableIOTokens: 209 +tick: 16, setAvailableIOTokens: 209 +tick: 17, setAvailableIOTokens: 209 +tick: 18, setAvailableIOTokens: 209 +tick: 19, setAvailableIOTokens: 209 +tick: 20, setAvailableIOTokens: 209 +tick: 21, setAvailableIOTokens: 209 +tick: 22, setAvailableIOTokens: 209 +tick: 23, setAvailableIOTokens: 209 +tick: 24, setAvailableIOTokens: 209 +tick: 25, setAvailableIOTokens: 209 +tick: 26, setAvailableIOTokens: 209 +tick: 27, setAvailableIOTokens: 209 +tick: 28, setAvailableIOTokens: 209 +tick: 29, setAvailableIOTokens: 209 +tick: 30, setAvailableIOTokens: 209 +tick: 31, setAvailableIOTokens: 209 +tick: 32, setAvailableIOTokens: 209 +tick: 33, setAvailableIOTokens: 209 +tick: 34, setAvailableIOTokens: 209 +tick: 35, setAvailableIOTokens: 209 +tick: 36, setAvailableIOTokens: 209 +tick: 37, setAvailableIOTokens: 209 +tick: 38, setAvailableIOTokens: 209 +tick: 39, setAvailableIOTokens: 209 +tick: 40, setAvailableIOTokens: 209 +tick: 41, setAvailableIOTokens: 209 +tick: 42, setAvailableIOTokens: 209 +tick: 43, setAvailableIOTokens: 209 +tick: 44, setAvailableIOTokens: 209 +tick: 45, setAvailableIOTokens: 209 +tick: 46, setAvailableIOTokens: 209 +tick: 47, setAvailableIOTokens: 209 +tick: 48, setAvailableIOTokens: 209 +tick: 49, setAvailableIOTokens: 209 +tick: 50, setAvailableIOTokens: 209 +tick: 51, setAvailableIOTokens: 209 +tick: 52, setAvailableIOTokens: 209 +tick: 53, setAvailableIOTokens: 209 +tick: 54, setAvailableIOTokens: 209 +tick: 55, setAvailableIOTokens: 209 +tick: 56, setAvailableIOTokens: 209 +tick: 57, setAvailableIOTokens: 209 +tick: 58, setAvailableIOTokens: 209 +tick: 59, setAvailableIOTokens: 169 prep-admission-stats admitted=20000 ---- @@ -65,46 +155,77 @@ 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 ---- -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:} +compaction score 1.050[L0-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], flushed 0 B [≈0 B]; admitting 24 KiB (rate 1.6 KiB/s) due to L0 growth (used 0 B) 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 cumWriteStallCount:0 smoothedIntL0CompactedBytes:75000 smoothedIntPerWorkUnaccountedL0Bytes:10 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:25000 tokensAllocated:0 tokensUsed: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 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 doLogFlush:false} ioThreshold:} store-request-estimates: fractionOfIngestIntoL0: 0.50, workByteAddition: 10 -tick: 0, setAvailableIOTokens: 1667 -tick: 1, setAvailableIOTokens: 1667 -tick: 2, setAvailableIOTokens: 1667 -tick: 3, setAvailableIOTokens: 1667 -tick: 4, setAvailableIOTokens: 1667 -tick: 5, setAvailableIOTokens: 1667 -tick: 6, setAvailableIOTokens: 1667 -tick: 7, setAvailableIOTokens: 1667 -tick: 8, setAvailableIOTokens: 1667 -tick: 9, setAvailableIOTokens: 1667 -tick: 10, setAvailableIOTokens: 1667 -tick: 11, setAvailableIOTokens: 1667 -tick: 12, setAvailableIOTokens: 1667 -tick: 13, setAvailableIOTokens: 1667 -tick: 14, setAvailableIOTokens: 1662 +tick: 0, setAvailableIOTokens: 417 +tick: 1, setAvailableIOTokens: 417 +tick: 2, setAvailableIOTokens: 417 +tick: 3, setAvailableIOTokens: 417 +tick: 4, setAvailableIOTokens: 417 +tick: 5, setAvailableIOTokens: 417 +tick: 6, setAvailableIOTokens: 417 +tick: 7, setAvailableIOTokens: 417 +tick: 8, setAvailableIOTokens: 417 +tick: 9, setAvailableIOTokens: 417 +tick: 10, setAvailableIOTokens: 417 +tick: 11, setAvailableIOTokens: 417 +tick: 12, setAvailableIOTokens: 417 +tick: 13, setAvailableIOTokens: 417 +tick: 14, setAvailableIOTokens: 417 +tick: 15, setAvailableIOTokens: 417 +tick: 16, setAvailableIOTokens: 417 +tick: 17, setAvailableIOTokens: 417 +tick: 18, setAvailableIOTokens: 417 +tick: 19, setAvailableIOTokens: 417 +tick: 20, setAvailableIOTokens: 417 +tick: 21, setAvailableIOTokens: 417 +tick: 22, setAvailableIOTokens: 417 +tick: 23, setAvailableIOTokens: 417 +tick: 24, setAvailableIOTokens: 417 +tick: 25, setAvailableIOTokens: 417 +tick: 26, setAvailableIOTokens: 417 +tick: 27, setAvailableIOTokens: 417 +tick: 28, setAvailableIOTokens: 417 +tick: 29, setAvailableIOTokens: 417 +tick: 30, setAvailableIOTokens: 417 +tick: 31, setAvailableIOTokens: 417 +tick: 32, setAvailableIOTokens: 417 +tick: 33, setAvailableIOTokens: 417 +tick: 34, setAvailableIOTokens: 417 +tick: 35, setAvailableIOTokens: 417 +tick: 36, setAvailableIOTokens: 417 +tick: 37, setAvailableIOTokens: 417 +tick: 38, setAvailableIOTokens: 417 +tick: 39, setAvailableIOTokens: 417 +tick: 40, setAvailableIOTokens: 417 +tick: 41, setAvailableIOTokens: 417 +tick: 42, setAvailableIOTokens: 417 +tick: 43, setAvailableIOTokens: 417 +tick: 44, setAvailableIOTokens: 417 +tick: 45, setAvailableIOTokens: 417 +tick: 46, setAvailableIOTokens: 417 +tick: 47, setAvailableIOTokens: 417 +tick: 48, setAvailableIOTokens: 417 +tick: 49, setAvailableIOTokens: 417 +tick: 50, setAvailableIOTokens: 417 +tick: 51, setAvailableIOTokens: 417 +tick: 52, setAvailableIOTokens: 417 +tick: 53, setAvailableIOTokens: 417 +tick: 54, setAvailableIOTokens: 417 +tick: 55, setAvailableIOTokens: 417 +tick: 56, setAvailableIOTokens: 417 +tick: 57, setAvailableIOTokens: 417 +tick: 58, setAvailableIOTokens: 417 +tick: 59, setAvailableIOTokens: 397 # No delta. This used to trigger an overflow bug. -set-state l0-bytes=10000 l0-added=201000 l0-files=21 l0-sublevels=21 +set-state l0-bytes=10000 l0-added=201000 l0-files=21 l0-sublevels=21 print-only-first-tick=true ---- -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:} +compaction score 1.050[L0-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], flushed 0 B [≈0 B]; admitting 21 KiB (rate 1.4 KiB/s) due to L0 growth (used 0 B) 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 cumWriteStallCount:0 smoothedIntL0CompactedBytes:37500 smoothedIntPerWorkUnaccountedL0Bytes:10 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedCompactionByteTokens:21875 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:21875 tokensAllocated:0 tokensUsed: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 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 doLogFlush:false} ioThreshold:} store-request-estimates: fractionOfIngestIntoL0: 0.50, workByteAddition: 10 -tick: 0, setAvailableIOTokens: 1459 -tick: 1, setAvailableIOTokens: 1459 -tick: 2, setAvailableIOTokens: 1459 -tick: 3, setAvailableIOTokens: 1459 -tick: 4, setAvailableIOTokens: 1459 -tick: 5, setAvailableIOTokens: 1459 -tick: 6, setAvailableIOTokens: 1459 -tick: 7, setAvailableIOTokens: 1459 -tick: 8, setAvailableIOTokens: 1459 -tick: 9, setAvailableIOTokens: 1459 -tick: 10, setAvailableIOTokens: 1459 -tick: 11, setAvailableIOTokens: 1459 -tick: 12, setAvailableIOTokens: 1459 -tick: 13, setAvailableIOTokens: 1459 -tick: 14, setAvailableIOTokens: 1449 +tick: 0, setAvailableIOTokens: 365 prep-admission-stats admitted=30000 ---- @@ -112,26 +233,12 @@ prep-admission-stats admitted=30000 # l0-sublevels drops below threshold. We calculate the smoothed values, but # don't limit the tokens. -set-state l0-bytes=10000 l0-added=501000 l0-files=21 l0-sublevels=20 +set-state l0-bytes=10000 l0-added=501000 l0-files=21 l0-sublevels=20 print-only-first-tick=true ---- -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:} +compaction 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], flushed 0 B [≈0 B]; admitting all +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:30000 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:168750 smoothedIntPerWorkUnaccountedL0Bytes:20 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedCompactionByteTokens:160937.5 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:9223372036854775807 tokensAllocated:0 tokensUsed: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 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 doLogFlush:false} ioThreshold:} store-request-estimates: fractionOfIngestIntoL0: 0.50, workByteAddition: 20 tick: 0, setAvailableIOTokens: unlimited -tick: 1, setAvailableIOTokens: unlimited -tick: 2, setAvailableIOTokens: unlimited -tick: 3, setAvailableIOTokens: unlimited -tick: 4, setAvailableIOTokens: unlimited -tick: 5, setAvailableIOTokens: unlimited -tick: 6, setAvailableIOTokens: unlimited -tick: 7, setAvailableIOTokens: unlimited -tick: 8, setAvailableIOTokens: unlimited -tick: 9, setAvailableIOTokens: unlimited -tick: 10, setAvailableIOTokens: unlimited -tick: 11, setAvailableIOTokens: unlimited -tick: 12, setAvailableIOTokens: unlimited -tick: 13, setAvailableIOTokens: unlimited -tick: 14, setAvailableIOTokens: unlimited # Test cases with more information in storeAdmissionStats. init @@ -141,25 +248,11 @@ prep-admission-stats admitted=0 ---- {admittedCount:0 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} -set-state l0-bytes=1000 l0-added=1000 l0-files=21 l0-sublevels=21 +set-state l0-bytes=1000 l0-added=1000 l0-files=21 l0-sublevels=21 print-only-first-tick=true ---- -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:} +compaction 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], flushed 0 B [≈0 B]; admitting all +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:0 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:1000 curL0Bytes:1000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:0 smoothedIntPerWorkUnaccountedL0Bytes:0 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 tokensAllocated:0 tokensUsed: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 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 doLogFlush:false} ioThreshold:} tick: 0, setAvailableIOTokens: unlimited -tick: 1, setAvailableIOTokens: unlimited -tick: 2, setAvailableIOTokens: unlimited -tick: 3, setAvailableIOTokens: unlimited -tick: 4, setAvailableIOTokens: unlimited -tick: 5, setAvailableIOTokens: unlimited -tick: 6, setAvailableIOTokens: unlimited -tick: 7, setAvailableIOTokens: unlimited -tick: 8, setAvailableIOTokens: unlimited -tick: 9, setAvailableIOTokens: unlimited -tick: 10, setAvailableIOTokens: unlimited -tick: 11, setAvailableIOTokens: unlimited -tick: 12, setAvailableIOTokens: unlimited -tick: 13, setAvailableIOTokens: unlimited -tick: 14, setAvailableIOTokens: unlimited # L0 will see an addition of 200,000 bytes. 180,000 bytes were mentioned by # the admitted requests, but 30,000 went into levels below L0. So 150,000 are @@ -168,52 +261,24 @@ prep-admission-stats admitted=10 admitted-bytes=180000 ingested-bytes=50000 inge ---- {admittedCount:10 admittedWithBytesCount:0 admittedAccountedBytes:180000 ingestedAccountedBytes:50000 ingestedAccountedL0Bytes:20000} -set-state l0-bytes=1000 l0-added=201000 l0-files=21 l0-sublevels=21 +set-state l0-bytes=1000 l0-added=201000 l0-files=21 l0-sublevels=21 print-only-first-tick=true ---- -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:} +compaction score 1.050[L0-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], flushed 0 B [≈0 B]; admitting 24 KiB (rate 1.6 KiB/s) due to L0 growth (used 0 B) 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 cumWriteStallCount:0 smoothedIntL0CompactedBytes:100000 smoothedIntPerWorkUnaccountedL0Bytes:5000 smoothedIntIngestedAccountedL0BytesFraction:0.45 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:25000 tokensAllocated:0 tokensUsed: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 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 doLogFlush:false} ioThreshold:} store-request-estimates: fractionOfIngestIntoL0: 0.45, workByteAddition: 5000 -tick: 0, setAvailableIOTokens: 1667 -tick: 1, setAvailableIOTokens: 1667 -tick: 2, setAvailableIOTokens: 1667 -tick: 3, setAvailableIOTokens: 1667 -tick: 4, setAvailableIOTokens: 1667 -tick: 5, setAvailableIOTokens: 1667 -tick: 6, setAvailableIOTokens: 1667 -tick: 7, setAvailableIOTokens: 1667 -tick: 8, setAvailableIOTokens: 1667 -tick: 9, setAvailableIOTokens: 1667 -tick: 10, setAvailableIOTokens: 1667 -tick: 11, setAvailableIOTokens: 1667 -tick: 12, setAvailableIOTokens: 1667 -tick: 13, setAvailableIOTokens: 1667 -tick: 14, setAvailableIOTokens: 1662 +tick: 0, setAvailableIOTokens: 417 # L0 will see an addition of 20,000 bytes, all of which are accounted for. prep-admission-stats admitted=20 admitted-bytes=200000 ingested-bytes=50000 ingested-into-l0=20000 ---- {admittedCount:20 admittedWithBytesCount:0 admittedAccountedBytes:200000 ingestedAccountedBytes:50000 ingestedAccountedL0Bytes:20000} -set-state l0-bytes=1000 l0-added=221000 l0-files=21 l0-sublevels=21 +set-state l0-bytes=1000 l0-added=221000 l0-files=21 l0-sublevels=21 print-only-first-tick=true ---- -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:} +compaction score 1.050[L0-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], flushed 0 B [≈0 B]; admitting 27 KiB (rate 1.8 KiB/s) due to L0 growth (used 0 B) 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 cumWriteStallCount:0 smoothedIntL0CompactedBytes:60000 smoothedIntPerWorkUnaccountedL0Bytes:2500 smoothedIntIngestedAccountedL0BytesFraction:0.45 smoothedCompactionByteTokens:27500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:27500 tokensAllocated:0 tokensUsed: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 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 doLogFlush:false} ioThreshold:} store-request-estimates: fractionOfIngestIntoL0: 0.45, workByteAddition: 2500 -tick: 0, setAvailableIOTokens: 1834 -tick: 1, setAvailableIOTokens: 1834 -tick: 2, setAvailableIOTokens: 1834 -tick: 3, setAvailableIOTokens: 1834 -tick: 4, setAvailableIOTokens: 1834 -tick: 5, setAvailableIOTokens: 1834 -tick: 6, setAvailableIOTokens: 1834 -tick: 7, setAvailableIOTokens: 1834 -tick: 8, setAvailableIOTokens: 1834 -tick: 9, setAvailableIOTokens: 1834 -tick: 10, setAvailableIOTokens: 1834 -tick: 11, setAvailableIOTokens: 1834 -tick: 12, setAvailableIOTokens: 1834 -tick: 13, setAvailableIOTokens: 1834 -tick: 14, setAvailableIOTokens: 1824 +tick: 0, setAvailableIOTokens: 459 # L0 will see an addition of 20,000 bytes, but we think we have added 100,000 # bytes to L0. We don't let unaccounted bytes become negative. @@ -221,23 +286,165 @@ prep-admission-stats admitted=30 admitted-bytes=300000 ingested-bytes=50000 inge ---- {admittedCount:30 admittedWithBytesCount:0 admittedAccountedBytes:300000 ingestedAccountedBytes:50000 ingestedAccountedL0Bytes:20000} -set-state l0-bytes=1000 l0-added=241000 l0-files=21 l0-sublevels=21 +set-state l0-bytes=1000 l0-added=241000 l0-files=21 l0-sublevels=21 print-only-first-tick=true ---- -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:} +compaction score 1.050[L0-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], flushed 0 B [≈0 B]; admitting 23 KiB (rate 1.5 KiB/s) due to L0 growth (used 0 B) 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 cumWriteStallCount:0 smoothedIntL0CompactedBytes:40000 smoothedIntPerWorkUnaccountedL0Bytes:1250 smoothedIntIngestedAccountedL0BytesFraction:0.45 smoothedCompactionByteTokens:23750 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:23750 tokensAllocated:0 tokensUsed: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 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 doLogFlush:false} ioThreshold:} store-request-estimates: fractionOfIngestIntoL0: 0.45, workByteAddition: 1250 -tick: 0, setAvailableIOTokens: 1584 -tick: 1, setAvailableIOTokens: 1584 -tick: 2, setAvailableIOTokens: 1584 -tick: 3, setAvailableIOTokens: 1584 -tick: 4, setAvailableIOTokens: 1584 -tick: 5, setAvailableIOTokens: 1584 -tick: 6, setAvailableIOTokens: 1584 -tick: 7, setAvailableIOTokens: 1584 -tick: 8, setAvailableIOTokens: 1584 -tick: 9, setAvailableIOTokens: 1584 -tick: 10, setAvailableIOTokens: 1584 -tick: 11, setAvailableIOTokens: 1584 -tick: 12, setAvailableIOTokens: 1584 -tick: 13, setAvailableIOTokens: 1584 -tick: 14, setAvailableIOTokens: 1574 +tick: 0, setAvailableIOTokens: 396 + +# Test case with flush tokens. +init +---- + +prep-admission-stats admitted=0 +---- +{admittedCount:0 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} + +set-state l0-bytes=10000 l0-added=1000 l0-files=1 l0-sublevels=1 print-only-first-tick=true +---- +compaction score 0.000 (1 ssts, 1 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], flushed 0 B [≈0 B]; admitting all +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:0 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:0 smoothedIntPerWorkUnaccountedL0Bytes:0 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 tokensAllocated:0 tokensUsed: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 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 doLogFlush:false} ioThreshold:} +tick: 0, setAvailableIOTokens: unlimited + +# Flush loop utilization is too low for the interval flush tokens to +# contribute to the smoothed value, or for tokens to become limited. +set-state l0-bytes=10000 l0-added=11000 l0-files=1 l0-sublevels=1 flush-bytes=1000 flush-work-sec=2 flush-idle-sec=100 print-only-first-tick=true +---- +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 9.8 KiB: 0 B acc-write + 0 B acc-ingest + 9.8 KiB unacc [≈0 B/req, n=1], compacted 9.8 KiB [≈4.9 KiB], flushed 7.3 KiB [≈0 B]; admitting all +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:0 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:5000 smoothedIntPerWorkUnaccountedL0Bytes:0 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedCompactionByteTokens:5000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:9223372036854775807 tokensAllocated:0 tokensUsed:0} requestEstimates:{fractionOfIngestIntoL0:0.5 workByteAddition:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intAdmittedCount:1 intAdmittedBytes:0 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:0 intUnaccountedL0Bytes:10000 intPerWorkUnaccountedL0Bytes:10000 l0BytesIngestFraction:0 intFlushTokens:7500 intFlushUtilization:0.0196078431372549 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 doLogFlush:false} ioThreshold:} +store-request-estimates: fractionOfIngestIntoL0: 0.50, workByteAddition: 1 +tick: 0, setAvailableIOTokens: unlimited + +# Flush loop utilization is high enough, so we compute flush tokens for limiting admission. +set-state l0-bytes=10000 l0-added=11000 l0-files=1 l0-sublevels=1 flush-bytes=1000 flush-work-sec=2 flush-idle-sec=10 print-only-first-tick=true +---- +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B: 0 B acc-write + 0 B acc-ingest + 0 B unacc [≈0 B/req, n=1], compacted 0 B [≈2.4 KiB], flushed 7.3 KiB [≈7.3 KiB]; admitting 11 KiB (rate 750 B/s) due to memtable flush (multiplier 1.500) (used 0 B) with L0 penalty: +1 B/req, *0.50/ingest +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:0 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:2500 smoothedIntPerWorkUnaccountedL0Bytes:0 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedCompactionByteTokens:2500 smoothedNumFlushTokens:7500 flushUtilTargetFraction:1.5 totalNumByteTokens:11250 tokensAllocated:0 tokensUsed:0} requestEstimates:{fractionOfIngestIntoL0:0.5 workByteAddition:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intAdmittedCount:1 intAdmittedBytes:0 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:0 intUnaccountedL0Bytes:0 intPerWorkUnaccountedL0Bytes:0 l0BytesIngestFraction:0 intFlushTokens:7500 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:0 tokenKind:1 doLogFlush:false} ioThreshold:} +store-request-estimates: fractionOfIngestIntoL0: 0.50, workByteAddition: 1 +tick: 0, setAvailableIOTokens: 188 + +# Write stalls are happening, so decrease the flush utilization target +# fraction from 1.5 to 1.475. But the peak flush rate has also increased since +# now we flushed 10x the bytes, so the overall tokens increase. +set-state l0-bytes=10000 l0-added=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=1 print-only-first-tick=true +---- +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B: 0 B acc-write + 0 B acc-ingest + 0 B unacc [≈0 B/req, n=1], compacted 0 B [≈1.2 KiB], flushed 73 KiB [≈40 KiB]; admitting 59 KiB (rate 4.0 KiB/s) due to memtable flush (multiplier 1.475) (used 0 B) with L0 penalty: +1 B/req, *0.50/ingest +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:0 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:1 smoothedIntL0CompactedBytes:1250 smoothedIntPerWorkUnaccountedL0Bytes:0 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedCompactionByteTokens:1250 smoothedNumFlushTokens:41250 flushUtilTargetFraction:1.475 totalNumByteTokens:60843 tokensAllocated:0 tokensUsed:0} requestEstimates:{fractionOfIngestIntoL0:0.5 workByteAddition:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intAdmittedCount:1 intAdmittedBytes:0 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:0 intUnaccountedL0Bytes:0 intPerWorkUnaccountedL0Bytes:0 l0BytesIngestFraction:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 tokenKind:1 doLogFlush:true} ioThreshold:} +store-request-estimates: fractionOfIngestIntoL0: 0.50, workByteAddition: 1 +tick: 0, setAvailableIOTokens: 1015 + +# Two write stalls happened, so decrease the flush utilization target fraction +# by a bigger step, from 1.475 to 1.425. Since the smoothed peak flush rate is +# increasing, the overall flush tokens continue to increase. +set-state l0-bytes=10000 l0-added=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=3 print-only-first-tick=true +---- +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B: 0 B acc-write + 0 B acc-ingest + 0 B unacc [≈0 B/req, n=1], compacted 0 B [≈625 B], flushed 73 KiB [≈57 KiB]; admitting 81 KiB (rate 5.4 KiB/s) due to memtable flush (multiplier 1.425) (used 0 B) with L0 penalty: +1 B/req, *0.50/ingest +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:0 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:3 smoothedIntL0CompactedBytes:625 smoothedIntPerWorkUnaccountedL0Bytes:0 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedCompactionByteTokens:625 smoothedNumFlushTokens:58125 flushUtilTargetFraction:1.4250000000000003 totalNumByteTokens:82828 tokensAllocated:0 tokensUsed:0} requestEstimates:{fractionOfIngestIntoL0:0.5 workByteAddition:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intAdmittedCount:1 intAdmittedBytes:0 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:0 intUnaccountedL0Bytes:0 intPerWorkUnaccountedL0Bytes:0 l0BytesIngestFraction:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:2 prevTokensUsed:0 tokenKind:1 doLogFlush:true} ioThreshold:} +store-request-estimates: fractionOfIngestIntoL0: 0.50, workByteAddition: 1 +tick: 0, setAvailableIOTokens: 1381 + +# Five more write stalls, so the the flush utilization target fraction is +# decreased to 1.35. The smoothed peak flush rate continues to increase. +set-state l0-bytes=10000 l0-added=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=8 print-only-first-tick=true +---- +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B: 0 B acc-write + 0 B acc-ingest + 0 B unacc [≈0 B/req, n=1], compacted 0 B [≈312 B], flushed 73 KiB [≈65 KiB]; admitting 88 KiB (rate 5.8 KiB/s) due to memtable flush (multiplier 1.350) (used 0 B) with L0 penalty: +1 B/req, *0.50/ingest +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:0 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:8 smoothedIntL0CompactedBytes:312 smoothedIntPerWorkUnaccountedL0Bytes:0 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedCompactionByteTokens:312.5 smoothedNumFlushTokens:66562.5 flushUtilTargetFraction:1.3500000000000005 totalNumByteTokens:89859 tokensAllocated:0 tokensUsed:0} requestEstimates:{fractionOfIngestIntoL0:0.5 workByteAddition:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intAdmittedCount:1 intAdmittedBytes:0 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:0 intUnaccountedL0Bytes:0 intPerWorkUnaccountedL0Bytes:0 l0BytesIngestFraction:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:5 prevTokensUsed:0 tokenKind:1 doLogFlush:true} ioThreshold:} +store-request-estimates: fractionOfIngestIntoL0: 0.50, workByteAddition: 1 +tick: 0, setAvailableIOTokens: 1498 + +# Another write stall, and the flush utilization target fraction drops to 1.325. +set-state l0-bytes=10000 l0-added=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=9 print-only-first-tick=true +---- +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B: 0 B acc-write + 0 B acc-ingest + 0 B unacc [≈0 B/req, n=1], compacted 0 B [≈156 B], flushed 73 KiB [≈69 KiB]; admitting 92 KiB (rate 6.1 KiB/s) due to memtable flush (multiplier 1.325) (used 0 B) with L0 penalty: +1 B/req, *0.50/ingest +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:0 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:9 smoothedIntL0CompactedBytes:156 smoothedIntPerWorkUnaccountedL0Bytes:0 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedCompactionByteTokens:156.25 smoothedNumFlushTokens:70781.25 flushUtilTargetFraction:1.3250000000000006 totalNumByteTokens:93785 tokensAllocated:0 tokensUsed:0} requestEstimates:{fractionOfIngestIntoL0:0.5 workByteAddition:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intAdmittedCount:1 intAdmittedBytes:0 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:0 intUnaccountedL0Bytes:0 intPerWorkUnaccountedL0Bytes:0 l0BytesIngestFraction:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 tokenKind:1 doLogFlush:true} ioThreshold:} +store-request-estimates: fractionOfIngestIntoL0: 0.50, workByteAddition: 1 +tick: 0, setAvailableIOTokens: 1564 + +# Set a lower bound of 1.3 on the flush utilization target fraction. +set-min-flush-util percent=130 +---- + +# Another write stall causes the flush utilization target fraction to decrease +# to 1.3, which is also the lower bound. +set-state l0-bytes=10000 l0-added=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=10 print-only-first-tick=true +---- +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B: 0 B acc-write + 0 B acc-ingest + 0 B unacc [≈0 B/req, n=1], compacted 0 B [≈78 B], flushed 73 KiB [≈71 KiB]; admitting 92 KiB (rate 6.2 KiB/s) due to memtable flush (multiplier 1.300) (used 0 B) with L0 penalty: +1 B/req, *0.50/ingest +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:0 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:10 smoothedIntL0CompactedBytes:78 smoothedIntPerWorkUnaccountedL0Bytes:0 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedCompactionByteTokens:78.125 smoothedNumFlushTokens:72890.625 flushUtilTargetFraction:1.3000000000000007 totalNumByteTokens:94757 tokensAllocated:0 tokensUsed:0} requestEstimates:{fractionOfIngestIntoL0:0.5 workByteAddition:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intAdmittedCount:1 intAdmittedBytes:0 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:0 intUnaccountedL0Bytes:0 intPerWorkUnaccountedL0Bytes:0 l0BytesIngestFraction:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 tokenKind:1 doLogFlush:true} ioThreshold:} +store-request-estimates: fractionOfIngestIntoL0: 0.50, workByteAddition: 1 +tick: 0, setAvailableIOTokens: 1580 + +# Despite another write stall, the flush utilization target fraction does not +# decrease since it is already at the lower bound. +set-state l0-bytes=10000 l0-added=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=11 print-only-first-tick=true +---- +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B: 0 B acc-write + 0 B acc-ingest + 0 B unacc [≈0 B/req, n=1], compacted 0 B [≈39 B], flushed 73 KiB [≈72 KiB]; admitting 94 KiB (rate 6.3 KiB/s) due to memtable flush (multiplier 1.300) (used 0 B) with L0 penalty: +1 B/req, *0.50/ingest +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:0 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:11 smoothedIntL0CompactedBytes:39 smoothedIntPerWorkUnaccountedL0Bytes:0 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedCompactionByteTokens:39.0625 smoothedNumFlushTokens:73945.3125 flushUtilTargetFraction:1.3000000000000007 totalNumByteTokens:96128 tokensAllocated:0 tokensUsed:0} requestEstimates:{fractionOfIngestIntoL0:0.5 workByteAddition:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intAdmittedCount:1 intAdmittedBytes:0 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:0 intUnaccountedL0Bytes:0 intPerWorkUnaccountedL0Bytes:0 l0BytesIngestFraction:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 tokenKind:1 doLogFlush:false} ioThreshold:} +store-request-estimates: fractionOfIngestIntoL0: 0.50, workByteAddition: 1 +tick: 0, setAvailableIOTokens: 1603 + +# Bump up the lower bound to 1.35, which is greater than the current flush +# utilization target fraction. +set-min-flush-util percent=135 +---- + +# Despite another write stall, the flush utilization target fraction +# increases to the new lower bound. +set-state l0-bytes=10000 l0-added=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=12 print-only-first-tick=true +---- +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B: 0 B acc-write + 0 B acc-ingest + 0 B unacc [≈0 B/req, n=1], compacted 0 B [≈19 B], flushed 73 KiB [≈73 KiB]; admitting 98 KiB (rate 6.5 KiB/s) due to memtable flush (multiplier 1.350) (used 0 B) with L0 penalty: +1 B/req, *0.50/ingest +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:0 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:12 smoothedIntL0CompactedBytes:19 smoothedIntPerWorkUnaccountedL0Bytes:0 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedCompactionByteTokens:19.53125 smoothedNumFlushTokens:74472.65625 flushUtilTargetFraction:1.35 totalNumByteTokens:100538 tokensAllocated:0 tokensUsed:0} requestEstimates:{fractionOfIngestIntoL0:0.5 workByteAddition:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intAdmittedCount:1 intAdmittedBytes:0 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:0 intUnaccountedL0Bytes:0 intPerWorkUnaccountedL0Bytes:0 l0BytesIngestFraction:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 tokenKind:1 doLogFlush:false} ioThreshold:} +store-request-estimates: fractionOfIngestIntoL0: 0.50, workByteAddition: 1 +tick: 0, setAvailableIOTokens: 1676 + +# The flush utilization is too low, so there is no limit on flush tokens. +set-state l0-bytes=10000 l0-added=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=100 write-stall-count=13 print-only-first-tick=true +---- +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B: 0 B acc-write + 0 B acc-ingest + 0 B unacc [≈0 B/req, n=1], compacted 0 B [≈9 B], flushed 73 KiB [≈73 KiB]; admitting all +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:0 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 smoothedIntL0CompactedBytes:9 smoothedIntPerWorkUnaccountedL0Bytes:0 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedCompactionByteTokens:9.765625 smoothedNumFlushTokens:74472.65625 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 tokensAllocated:0 tokensUsed:0} requestEstimates:{fractionOfIngestIntoL0:0.5 workByteAddition:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intAdmittedCount:1 intAdmittedBytes:0 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:0 intUnaccountedL0Bytes:0 intPerWorkUnaccountedL0Bytes:0 l0BytesIngestFraction:0 intFlushTokens:75000 intFlushUtilization:0.0196078431372549 intWriteStalls:1 prevTokensUsed:0 tokenKind:0 doLogFlush:false} ioThreshold:} +store-request-estimates: fractionOfIngestIntoL0: 0.50, workByteAddition: 1 +tick: 0, setAvailableIOTokens: unlimited + +# Flush utilization is high enough, so flush tokens are again limited. +set-state l0-bytes=10000 l0-added=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=13 print-only-first-tick=true +---- +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B: 0 B acc-write + 0 B acc-ingest + 0 B unacc [≈0 B/req, n=1], compacted 0 B [≈4 B], flushed 73 KiB [≈73 KiB]; admitting 98 KiB (rate 6.6 KiB/s) due to memtable flush (multiplier 1.350) (used 0 B) with L0 penalty: +1 B/req, *0.50/ingest +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:0 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 smoothedIntL0CompactedBytes:4 smoothedIntPerWorkUnaccountedL0Bytes:0 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedCompactionByteTokens:4.8828125 smoothedNumFlushTokens:74736.328125 flushUtilTargetFraction:1.35 totalNumByteTokens:100894 tokensAllocated:0 tokensUsed:0} requestEstimates:{fractionOfIngestIntoL0:0.5 workByteAddition:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intAdmittedCount:1 intAdmittedBytes:0 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:0 intUnaccountedL0Bytes:0 intPerWorkUnaccountedL0Bytes:0 l0BytesIngestFraction:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:0 tokenKind:1 doLogFlush:false} ioThreshold:} +store-request-estimates: fractionOfIngestIntoL0: 0.50, workByteAddition: 1 +tick: 0, setAvailableIOTokens: 1682 + +# No write stalls, and token utilization is high, which will have an effect +# in the next pebbleMetricsTick. +set-state l0-bytes=10000 l0-added=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=13 all-tokens-used=true print-only-first-tick=true +---- +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B: 0 B acc-write + 0 B acc-ingest + 0 B unacc [≈0 B/req, n=1], compacted 0 B [≈2 B], flushed 73 KiB [≈73 KiB]; admitting 99 KiB (rate 6.6 KiB/s) due to memtable flush (multiplier 1.350) (used 0 B) with L0 penalty: +1 B/req, *0.50/ingest +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:0 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 smoothedIntL0CompactedBytes:2 smoothedIntPerWorkUnaccountedL0Bytes:0 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedCompactionByteTokens:2.44140625 smoothedNumFlushTokens:74868.1640625 flushUtilTargetFraction:1.35 totalNumByteTokens:101072 tokensAllocated:0 tokensUsed:0} requestEstimates:{fractionOfIngestIntoL0:0.5 workByteAddition:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intAdmittedCount:1 intAdmittedBytes:0 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:0 intUnaccountedL0Bytes:0 intPerWorkUnaccountedL0Bytes:0 l0BytesIngestFraction:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:0 tokenKind:1 doLogFlush:false} ioThreshold:} +store-request-estimates: fractionOfIngestIntoL0: 0.50, workByteAddition: 1 +tick: 0, setAvailableIOTokens: 1685 + +# No write stalls, and token utilization was high, so flush utilization +# target fraction is increased to 1.375. +set-state l0-bytes=10000 l0-added=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=13 all-tokens-used=true print-only-first-tick=true +---- +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B: 0 B acc-write + 0 B acc-ingest + 0 B unacc [≈0 B/req, n=1], compacted 0 B [≈1 B], flushed 73 KiB [≈73 KiB]; admitting 101 KiB (rate 6.7 KiB/s) due to memtable flush (multiplier 1.375) (used 197 KiB) with L0 penalty: +1 B/req, *0.50/ingest +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:0 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 smoothedIntL0CompactedBytes:1 smoothedIntPerWorkUnaccountedL0Bytes:0 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedCompactionByteTokens:1.220703125 smoothedNumFlushTokens:74934.08203125 flushUtilTargetFraction:1.375 totalNumByteTokens:103034 tokensAllocated:0 tokensUsed:0} requestEstimates:{fractionOfIngestIntoL0:0.5 workByteAddition:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intAdmittedCount:1 intAdmittedBytes:0 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:0 intUnaccountedL0Bytes:0 intPerWorkUnaccountedL0Bytes:0 l0BytesIngestFraction:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:202144 tokenKind:1 doLogFlush:true} ioThreshold:} +store-request-estimates: fractionOfIngestIntoL0: 0.50, workByteAddition: 1 +tick: 0, setAvailableIOTokens: 1718 + +# No write stalls, and token utilization was high, so flush utilization +# target fraction is increased to 1.4. +set-state l0-bytes=10000 l0-added=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=13 all-tokens-used=true print-only-first-tick=true +---- +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B: 0 B acc-write + 0 B acc-ingest + 0 B unacc [≈0 B/req, n=1], compacted 0 B [≈0 B], flushed 73 KiB [≈73 KiB]; admitting 102 KiB (rate 6.8 KiB/s) due to memtable flush (multiplier 1.400) (used 201 KiB) with L0 penalty: +1 B/req, *0.50/ingest +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:0 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 smoothedIntL0CompactedBytes:0 smoothedIntPerWorkUnaccountedL0Bytes:0 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedCompactionByteTokens:0.6103515625 smoothedNumFlushTokens:74967.041015625 flushUtilTargetFraction:1.4 totalNumByteTokens:104953 tokensAllocated:0 tokensUsed:0} requestEstimates:{fractionOfIngestIntoL0:0.5 workByteAddition:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intAdmittedCount:1 intAdmittedBytes:0 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:0 intUnaccountedL0Bytes:0 intPerWorkUnaccountedL0Bytes:0 l0BytesIngestFraction:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:206068 tokenKind:1 doLogFlush:true} ioThreshold:} +store-request-estimates: fractionOfIngestIntoL0: 0.50, workByteAddition: 1 +tick: 0, setAvailableIOTokens: 1750 + +# There is a write stall, so even though token utilization is high, we +# decrease flush utilization target fraction to 1.375. +set-state l0-bytes=10000 l0-added=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=14 all-tokens-used=true print-only-first-tick=true +---- +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B: 0 B acc-write + 0 B acc-ingest + 0 B unacc [≈0 B/req, n=1], compacted 0 B [≈0 B], flushed 73 KiB [≈73 KiB]; admitting 101 KiB (rate 6.7 KiB/s) due to memtable flush (multiplier 1.375) (used 205 KiB) with L0 penalty: +1 B/req, *0.50/ingest +{ioLoadListenerState:{cumAdmissionStats:{admittedCount:0 admittedWithBytesCount:0 admittedAccountedBytes:0 ingestedAccountedBytes:0 ingestedAccountedL0Bytes:0} cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:14 smoothedIntL0CompactedBytes:0 smoothedIntPerWorkUnaccountedL0Bytes:0 smoothedIntIngestedAccountedL0BytesFraction:0.5 smoothedCompactionByteTokens:0.30517578125 smoothedNumFlushTokens:74983.5205078125 flushUtilTargetFraction:1.375 totalNumByteTokens:103102 tokensAllocated:0 tokensUsed:0} requestEstimates:{fractionOfIngestIntoL0:0.5 workByteAddition:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intAdmittedCount:1 intAdmittedBytes:0 intIngestedBytes:0 intIngestedAccountedL0Bytes:0 intAccountedL0Bytes:0 intUnaccountedL0Bytes:0 intPerWorkUnaccountedL0Bytes:0 l0BytesIngestFraction:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:209906 tokenKind:1 doLogFlush:true} ioThreshold:} +store-request-estimates: fractionOfIngestIntoL0: 0.50, workByteAddition: 1 +tick: 0, setAvailableIOTokens: 1719