From e07b0afb86aa010d72ab7fc492632a658d5b3884 Mon Sep 17 00:00:00 2001 From: Aaditya Sondhi <20070511+aadityasondhi@users.noreply.github.com> Date: Tue, 13 Aug 2024 17:16:46 -0400 Subject: [PATCH] admission: account for write-amp in disk bandwidth limiter Previously, we would calculate elastic disk bandwidth tokens using arbitrary load thresholds and an estimate on incoming bytes into the LSM through flushes and ingestions. This calculation lacked accounting for write amplification in the LSM. This patch simplifies the disk bandwidth limiter to remove the disk load watcher and simply adjust tokens using the known provisioned disk bandwidth. For token deducation, we create a write-amp model that is a relationship between incoming LSM bytes to actual disk writes. The token granting semantics are as follows: - elastic writes: deduct tokens, and wait for positive count in bucket. - regular writes: deduct tokens, but proceed even with no tokens available. The requested write bytes are "inflated" using the estimated write amplification to account for async compactions in the LSM. This patch also lays the framework for future integrations where we can account for range snapshot ingestions separately as those don't incur the same write amplification as flushed LSM bytes do. Informs: #86857 Release note: None --- pkg/util/admission/admission.go | 12 +- pkg/util/admission/disk_bandwidth.go | 367 ++++------------ pkg/util/admission/disk_bandwidth_test.go | 72 +-- pkg/util/admission/grant_coordinator.go | 8 +- pkg/util/admission/granter.go | 97 ++-- pkg/util/admission/granter_test.go | 35 +- pkg/util/admission/io_load_listener.go | 143 +++--- pkg/util/admission/io_load_listener_test.go | 50 ++- pkg/util/admission/store_token_estimation.go | 47 +- .../admission/store_token_estimation_test.go | 13 +- .../admission/testdata/disk_bandwidth_limiter | 43 +- pkg/util/admission/testdata/disk_load_watcher | 78 ---- .../testdata/format_adjust_tokens_stats.txt | 4 +- pkg/util/admission/testdata/granter | 154 +++---- pkg/util/admission/testdata/io_load_listener | 413 ++++++++++-------- .../testdata/store_per_work_token_estimator | 59 ++- 16 files changed, 747 insertions(+), 848 deletions(-) delete mode 100644 pkg/util/admission/testdata/disk_load_watcher diff --git a/pkg/util/admission/admission.go b/pkg/util/admission/admission.go index bd71b4f9c0d3..c88dce7f829d 100644 --- a/pkg/util/admission/admission.go +++ b/pkg/util/admission/admission.go @@ -274,13 +274,13 @@ type granterWithIOTokens interface { // negative, though that will be rare, since it is possible for tokens to be // returned. setAvailableTokens( - ioTokens int64, elasticIOTokens int64, elasticDiskBandwidthTokens int64, - ioTokensCapacity int64, elasticIOTokenCapacity int64, elasticDiskBandwidthTokensCapacity int64, + ioTokens int64, elasticIOTokens int64, elasticDiskWriteTokens int64, + ioTokensCapacity int64, elasticIOTokenCapacity int64, elasticDiskWriteTokensCapacity int64, lastTick bool, ) (tokensUsed int64, tokensUsedByElasticWork int64) - // getDiskTokensUsedAndReset returns the disk bandwidth tokens used - // since the last such call. - getDiskTokensUsedAndReset() [admissionpb.NumWorkClasses]int64 + // getDiskTokensUsedAndReset returns the disk bandwidth tokens used since the + // last such call. + getDiskTokensUsedAndReset() [admissionpb.NumWorkClasses]diskTokens // setLinearModels supplies the models to use when storeWriteDone or // storeReplicatedWorkAdmittedLocked is called, to adjust token consumption. // Note that these models are not used for token adjustment at admission @@ -288,7 +288,7 @@ type granterWithIOTokens interface { // granter. This asymmetry is due to the need to use all the functionality // of WorkQueue at admission time. See the long explanatory comment at the // beginning of store_token_estimation.go, regarding token estimation. - setLinearModels(l0WriteLM, l0IngestLM, ingestLM tokensLinearModel) + setLinearModels(l0WriteLM, l0IngestLM, ingestLM, writeAmpLM tokensLinearModel) } // granterWithStoreReplicatedWorkAdmitted is used to abstract diff --git a/pkg/util/admission/disk_bandwidth.go b/pkg/util/admission/disk_bandwidth.go index 63846c56bfeb..316d48ac16e9 100644 --- a/pkg/util/admission/disk_bandwidth.go +++ b/pkg/util/admission/disk_bandwidth.go @@ -11,14 +11,16 @@ package admission import ( - "context" "math" + "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" - "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/redact" ) +// TODO(aaditya): Update this comment once read and IOPS are integrated to this. +// Issue: https://github.com/cockroachdb/cockroach/issues/107623 +// // The functionality in this file is geared towards preventing chronic overload // of disk bandwidth which typically results in severely high latency for all work. // @@ -47,17 +49,14 @@ import ( // latency effects, presumably because the queue buildup is different. So it // is non-trivial to approach full utilization without risking high latency. // -// Due to these challenges, we adopt a goal of simplicity of design, and -// strong abstraction boundaries. +// Due to these challenges, we adopt a goal of simplicity of design. // -// - The disk load is abstracted using an enum, diskLoadLevel. The -// diskLoadWatcher, that maps load signals to this enum, can be evolved -// independently. +// - The disk bandwidth limiter estimates disk write byte tokens using the +// provisioned value and subtracts away reads seen in the previous interval. // -// - The approach uses easy to understand small multiplicative increase and -// large multiplicative decrease, (unlike what we do for flush and -// compaction tokens, where we try to more precisely calculate the -// sustainable rates). +// - We estimate the write amplification using a model of the incoming writes +// and actual bytes written to disk in the previous interval. We then use this +// model when deducting tokens for disk writes. // // Since we are using a simple approach that is somewhat coarse in its behavior, // we start by limiting its application to two kinds of writes (the second one @@ -79,289 +78,113 @@ import ( // // Extending this to all incoming writes is future work. -// The load level of a disk. -type diskLoadLevel int8 - -const ( - // diskLoadLow implies no need to shape anything. - diskLoadLow diskLoadLevel = iota - // diskLoadModerate implies shaping and small multiplicative increase. - diskLoadModerate - // diskLoadHigh implies shaping and hold steady. - diskLoadHigh - // diskLoadOverload implies shaping and large multiplicative decrease. - diskLoadOverload -) - -func diskLoadLevelString(level diskLoadLevel) redact.SafeString { - switch level { - case diskLoadLow: - return "low" - case diskLoadModerate: - return "moderate" - case diskLoadHigh: - return "high" - case diskLoadOverload: - return "overload" - } - return "" -} - -// diskLoadWatcher computes the diskLoadLevel based on provided stats. -type diskLoadWatcher struct { - lastInterval intervalDiskLoadInfo - lastUtil float64 - loadLevel diskLoadLevel -} - // intervalDiskLoadInfo provides disk stats over an adjustmentInterval. type intervalDiskLoadInfo struct { - // readBandwidth is the measure disk read bandwidth in bytes/s. - readBandwidth int64 - // writeBandwidth is the measured disk write bandwidth in bytes/s. - writeBandwidth int64 - // provisionedBandwidth is the aggregate (read+write) provisioned bandwidth - // in bytes/s. - provisionedBandwidth int64 -} - -// setIntervalInfo is called at the same time as ioLoadListener.pebbleMetricsTick. -func (d *diskLoadWatcher) setIntervalInfo(load intervalDiskLoadInfo) { - lastInterval := load - util := float64(load.readBandwidth+load.writeBandwidth) / float64(load.provisionedBandwidth) - // The constants and other heuristics in the following logic can seem - // extremely arbitrary: they were subject to some tuning and evolution based - // on the experiments in https://github.com/cockroachdb/cockroach/pull/82813 - // that used (a) an artificial provisioned bandwidth limit lower than the - // actual, to see how well the system stayed within that limit, (b) an - // actual provisioned bandwidth limit. The difficulty in general is that - // small changes can have outsize influence if a higher number of - // compactions start happening. - var loadLevel diskLoadLevel - const lowUtilThreshold = 0.3 - const moderateUtilThreshold = 0.7 - const highUtilThreshold = 0.95 - const highlyOverUtilizedThreshold = 2.0 - const smallDelta = 0.05 - if util < lowUtilThreshold { - // Were at moderate or lower and have not increased significantly and the - // lastUtil was also low, then we can afford to stop limiting tokens. We - // are trying to carefully narrow this case since not limiting tokens can - // blow up bandwidth. - // - // An alternative would be to never have unlimited tokens, since that - // ensures there is always some reasonable bound in place. It may mean - // that the initial tokens are insufficient and the tokens catch up to - // what is needed with some lag, and during that time there is unnecessary - // queueing. This downside could be avoided by ramping up faster. This - // alternative is worth investigating. - - if d.loadLevel <= diskLoadModerate && util < d.lastUtil+smallDelta && - d.lastUtil < lowUtilThreshold { - loadLevel = diskLoadLow - } else { - // util is increasing, or we just dropped from something higher than - // moderate. Give it more time at moderate, where we will gradually - // increase tokens. - loadLevel = diskLoadModerate - } - } else if util < moderateUtilThreshold { - // Wide band from [0.3,0.7) where we gradually increase tokens. Also, 0.7 - // is deliberately a lowish fraction since the effect on compactions can - // lag and kick in later. We are ok with accepting a lower utilization for - // elastic work to make progress. - loadLevel = diskLoadModerate - } else if util < highUtilThreshold || - (util < highlyOverUtilizedThreshold && util < d.lastUtil-smallDelta) { - // Wide band from [0.7,0.95) where we will hold the number of tokens - // steady. We don't want to overreact and decrease too early since - // compaction bandwidth usage can be lumpy. For this same reason, if we - // are trending downward, we want to hold. Note that util < 2 will always - // be true in typical configurations where one cannot actually exceed - // provisioned bandwidth -- but we also run experiments where we - // artificially constrain the provisioned bandwidth, where this is useful. - // And it is possible that some production settings may set a slightly - // lower value of provisioned bandwidth, if they want to further reduce - // the probability of hitting the real provisioned bandwidth due to - // elastic work. - loadLevel = diskLoadHigh - } else { - // Overloaded. We will reduce tokens. - loadLevel = diskLoadOverload - } - *d = diskLoadWatcher{ - lastInterval: lastInterval, - lastUtil: util, - loadLevel: loadLevel, - } - // TODO(sumeer): Use the history of fsync latency and the value in the - // current interval, and if high, increase the load level computed earlier. - // We shouldn't rely fully on syncLatencyMicros since (a) sync latency could - // arise due to an external unrelated outage, (b) some customers may set - // fsync to be a noop. As an alternative to sync latency, we could also - // consider looking at fluctuations of peak-rate that the WAL writer can - // sustain. -} - -func (d *diskLoadWatcher) getLoadLevel() diskLoadLevel { - return d.loadLevel -} - -func (d diskLoadWatcher) SafeFormat(p redact.SafePrinter, _ rune) { - p.Printf("disk bandwidth: read: %s/s, write: %s/s, provisioned: %s/s, util: %.2f", - humanizeutil.IBytes(d.lastInterval.readBandwidth), - humanizeutil.IBytes(d.lastInterval.writeBandwidth), - humanizeutil.IBytes(d.lastInterval.provisionedBandwidth), d.lastUtil) -} - -// intervalLSMInfo provides stats about the LSM over an adjustmentInterval. -type intervalLSMInfo struct { - // Flushed bytes + Ingested bytes seen by the LSM. Ingested bytes incur the - // cost of writing a sstable, even though that is done outside Pebble, so - // ingestion is similar in cost to flushing. Ingested bytes don't cause WAL - // writes, but we ignore that difference for simplicity, and just work with - // the sum of flushed and ingested bytes. - incomingBytes int64 - // regularTokensUsed and elasticTokensUsed are the byte tokens used for - // regular and elastic work respectively. Each of these includes both - // writes that will get flushed and ingested bytes. The - // regularTokensUsed+elasticTokensUsed do not need to sum up to - // incomingBytes, since these stats are produced by different sources. - regularTokensUsed int64 - elasticTokensUsed int64 + // intReadBytes represents measured disk read bytes in a given interval. + intReadBytes int64 + // intWriteBytes represents measured write bytes in a given interval. + intWriteBytes int64 + // intProvisionedDiskBytes represents the disk writes (in bytes) available in + // an adjustmentInterval. + intProvisionedDiskBytes int64 + // elasticBandwidthMaxUtil sets the maximum disk bandwidth utilization for + // elastic requests + elasticBandwidthMaxUtil float64 } +// diskBandwidthLimiterState is used as auxiliary information for logging +// purposes and keeping past state. type diskBandwidthLimiterState struct { - smoothedIncomingBytes float64 - smoothedElasticFraction float64 - elasticTokens int64 - - prevElasticTokensUsed int64 + tokens diskTokens + prevTokens diskTokens + usedTokens [admissionpb.NumWorkClasses]diskTokens + diskBWUtil float64 + diskLoad intervalDiskLoadInfo } // diskBandwidthLimiter produces tokens for elastic work. type diskBandwidthLimiter struct { - diskLoadWatcher diskLoadWatcher - state diskBandwidthLimiterState + state diskBandwidthLimiterState } -func makeDiskBandwidthLimiter() diskBandwidthLimiter { - return diskBandwidthLimiter{ - state: diskBandwidthLimiterState{ - elasticTokens: math.MaxInt64, - }, +func newDiskBandwidthLimiter() *diskBandwidthLimiter { + return &diskBandwidthLimiter{ + state: diskBandwidthLimiterState{}, } } +// diskTokens tokens represent actual bytes and IO on physical disks. Currently, +// these are used to impose disk bandwidth limits on elastic traffic, but +// regular traffic will also deduct from these buckets. +type diskTokens struct { + readByteTokens int64 + writeByteTokens int64 + readIOPSTokens int64 + writeIOPSTokens int64 +} + // computeElasticTokens is called every adjustmentInterval. func (d *diskBandwidthLimiter) computeElasticTokens( - ctx context.Context, id intervalDiskLoadInfo, il intervalLSMInfo, -) (elasticTokens int64) { - d.diskLoadWatcher.setIntervalInfo(id) + id intervalDiskLoadInfo, usedTokens [admissionpb.NumWorkClasses]diskTokens, +) diskTokens { + // We are using disk read bytes over the previous adjustment interval as a + // proxy for future reads. It is a somewhat bad proxy, but for now we are ok + // with the inaccuracy. This will be improved once we start to account for + // disk reads in AC. + // TODO(aaditya): Include calculation for read and IOPS. + // Issue: https://github.com/cockroachdb/cockroach/issues/107623 const alpha = 0.5 - prev := d.state - smoothedIncomingBytes := alpha*float64(il.incomingBytes) + (1-alpha)*prev.smoothedIncomingBytes - smoothedElasticFraction := prev.smoothedElasticFraction - var intElasticFraction float64 - if il.regularTokensUsed+il.elasticTokensUsed > 0 { - intElasticFraction = - float64(il.elasticTokensUsed) / float64(il.regularTokensUsed+il.elasticTokensUsed) - smoothedElasticFraction = alpha*intElasticFraction + (1-alpha)*prev.smoothedElasticFraction - } - intElasticBytes := int64(float64(il.incomingBytes) * intElasticFraction) - ll := d.diskLoadWatcher.getLoadLevel() - - // The constants and other heuristics in the following logic can seem - // arbitrary: they were subject to some tuning and evolution based on the - // experiments in https://github.com/cockroachdb/cockroach/pull/82813 that - // used (a) an artificial provisioned bandwidth limit lower than the actual, - // to see how well the system stayed within that limit, (b) an actual - // provisioned bandwidth limit. The difficulty in general is that small - // changes can have outsize influence if a higher number of compactions - // start happening, or the compaction backlog is cleared. - // - // TODO(sumeer): experiment with a more sophisticated controller for the - // elastic token adjustment, e.g. a PID (Proportional-Integral-Derivative) - // controller. - doLog := true - switch ll { - case diskLoadLow: - elasticTokens = math.MaxInt64 - if elasticTokens == prev.elasticTokens { - doLog = false - } - // else we stay in the common case of low bandwidth usage. - case diskLoadModerate: - tokensFullyUtilized := - // elasticTokens == MaxInt64 is also considered fully utilized since we - // can never fully utilize unlimited tokens. - prev.elasticTokens == math.MaxInt64 || - (prev.elasticTokens > 0 && float64(il.elasticTokensUsed)/float64(prev.elasticTokens) >= 0.8) - - if tokensFullyUtilized { - // Smoothed elastic bytes plus 10% of smoothedIncomingBytes is given to - // elastic work. That is, we are increasing the total incoming bytes by - // 10% (not just the elastic bytes by 10%). Note that each token - // represents 1 incoming byte. - elasticBytes := (smoothedElasticFraction + 0.1) * smoothedIncomingBytes - // Sometimes we see the tokens not increasing even though we are staying - // for multiple intervals at moderate. This is because the smoothed - // fraction and incoming bytes can be decreasing. We do want to increase - // tokens since we know there is spare capacity, so we try many ways - // (that don't look at smoothed numbers only). Also, we sometimes come - // here due to an overload=>moderate transition because compaction - // bandwidth usage can be lumpy (high when there is a backlog and then - // dropping severely) -- in that case we want to start increasing - // immediately, since we have likely decreased too much. - intBasedElasticTokens := (smoothedElasticFraction + 0.1) * float64(il.incomingBytes) - elasticBytes = math.Max(elasticBytes, intBasedElasticTokens) - elasticBytes = math.Max(elasticBytes, 1.1*float64(il.elasticTokensUsed)) - elasticTokens = int64(elasticBytes) - if elasticTokens == 0 { - // Don't get stuck in a situation where smoothedIncomingBytes are 0. - elasticTokens = math.MaxInt64 - } - } else { - // No change. - elasticTokens = prev.elasticTokens - } - case diskLoadHigh: - // No change. - elasticTokens = prev.elasticTokens - case diskLoadOverload: - // Sometimes we come here after a low => overload transition. The - // intElasticBytes will be very high because tokens were unlimited. We - // don't want to use that as the starting point of the decrease if the - // smoothed value is lower. Hence, the min logic below, to try to dampen - // the increase quickly. - elasticTokens = int64(0.5 * math.Min(float64(intElasticBytes), - smoothedElasticFraction*smoothedIncomingBytes)) + smoothedReadBytes := alpha*float64(id.intReadBytes) + alpha*float64(d.state.diskLoad.intReadBytes) + // Pessimistic approach using the max value between the smoothed and current + // reads. + intReadBytes := int64(math.Max(smoothedReadBytes, float64(id.intReadBytes))) + diskWriteTokens := int64(float64(id.intProvisionedDiskBytes)*id.elasticBandwidthMaxUtil) - intReadBytes + // TODO(aaditya): consider setting a different floor to avoid starving out + // elastic writes completely due to out-sized reads from above. + diskWriteTokens = int64(math.Max(0, float64(diskWriteTokens))) + + totalUsedTokens := sumDiskTokens(usedTokens[admissionpb.ElasticWorkClass], usedTokens[admissionpb.RegularWorkClass]) + tokens := diskTokens{ + readByteTokens: 0, + writeByteTokens: diskWriteTokens, + readIOPSTokens: 0, + writeIOPSTokens: 0, } - // We can end up with 0 elastic tokens here -- e.g. if intElasticBytes was 0 - // but we were still overloaded because of compactions. The trouble with 0 - // elastic tokens is that if we don't admit anything, we cannot correct an - // occasional poor estimate of the per-request bytes. So we decide to give - // out at least 1 token. A single elastic request should not be too big for - // this to matter. - elasticTokens = max(1, elasticTokens) + prevState := d.state d.state = diskBandwidthLimiterState{ - smoothedIncomingBytes: smoothedIncomingBytes, - smoothedElasticFraction: smoothedElasticFraction, - elasticTokens: elasticTokens, - prevElasticTokensUsed: il.elasticTokensUsed, + tokens: tokens, + prevTokens: prevState.tokens, + usedTokens: usedTokens, + diskBWUtil: float64(totalUsedTokens.writeByteTokens) / float64(prevState.tokens.writeByteTokens), + diskLoad: id, } - if doLog { - log.Infof(ctx, "%v", d) - } - return elasticTokens + return tokens } func (d *diskBandwidthLimiter) SafeFormat(p redact.SafePrinter, _ rune) { ib := humanizeutil.IBytes - level := d.diskLoadWatcher.getLoadLevel() - p.Printf("diskBandwidthLimiter %s (%v): elastic-frac: %.2f, incoming: %s, "+ - "elastic-tokens (used %s): %s", - diskLoadLevelString(level), d.diskLoadWatcher, d.state.smoothedElasticFraction, - ib(int64(d.state.smoothedIncomingBytes)), ib(d.state.prevElasticTokensUsed), - ib(d.state.elasticTokens)) + p.Printf("diskBandwidthLimiter (tokenUtilization %.2f, tokensUsed (elastic %s, "+ + "regular %s) tokens (write %s (prev %s)), writeBW %s/s, readBW %s/s, provisioned %s/s)", + d.state.diskBWUtil, + ib(d.state.usedTokens[admissionpb.ElasticWorkClass].writeByteTokens), + ib(d.state.usedTokens[admissionpb.RegularWorkClass].writeByteTokens), + ib(d.state.tokens.writeByteTokens), + ib(d.state.prevTokens.writeByteTokens), + ib(d.state.diskLoad.intWriteBytes/adjustmentInterval), + ib(d.state.diskLoad.intReadBytes/adjustmentInterval), + ib(d.state.diskLoad.intProvisionedDiskBytes/adjustmentInterval), + ) +} + +func (d *diskBandwidthLimiter) String() string { + return redact.StringWithoutMarkers(d) +} + +func sumDiskTokens(l diskTokens, r diskTokens) diskTokens { + return diskTokens{ + readByteTokens: l.readByteTokens + r.readByteTokens, + writeByteTokens: l.writeByteTokens + r.writeByteTokens, + readIOPSTokens: l.readIOPSTokens + r.readIOPSTokens, + writeIOPSTokens: l.writeIOPSTokens + r.writeIOPSTokens, + } } diff --git a/pkg/util/admission/disk_bandwidth_test.go b/pkg/util/admission/disk_bandwidth_test.go index 3967c1ae68a1..eb8c884279d6 100644 --- a/pkg/util/admission/disk_bandwidth_test.go +++ b/pkg/util/admission/disk_bandwidth_test.go @@ -11,89 +11,53 @@ package admission import ( - "context" "fmt" "testing" "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" + "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/datadriven" "github.com/cockroachdb/redact" ) -func TestDiskLoadWatcher(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - var dlw diskLoadWatcher - watcherToString := func() string { - level := dlw.getLoadLevel() - return fmt.Sprintf("%s\nload-level: %s", redact.Sprint(dlw), - diskLoadLevelString(level)) - } - - datadriven.RunTest(t, datapathutils.TestDataPath(t, "disk_load_watcher"), - func(t *testing.T, d *datadriven.TestData) string { - switch d.Cmd { - case "init": - dlw = diskLoadWatcher{} - return watcherToString() - - case "interval-info": - var readBandwidth, writeBandwidth, provisionedBandwidth int - d.ScanArgs(t, "read-bw", &readBandwidth) - d.ScanArgs(t, "write-bw", &writeBandwidth) - d.ScanArgs(t, "provisioned-bw", &provisionedBandwidth) - dlw.setIntervalInfo(intervalDiskLoadInfo{ - readBandwidth: int64(readBandwidth), - writeBandwidth: int64(writeBandwidth), - provisionedBandwidth: int64(provisionedBandwidth), - }) - return watcherToString() - - default: - return fmt.Sprintf("unknown command: %s", d.Cmd) - } - }) -} - func TestDiskBandwidthLimiter(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - var dbl diskBandwidthLimiter + var dbl *diskBandwidthLimiter dblToString := func() string { - return string(redact.Sprint(&dbl)) + return string(redact.Sprint(dbl)) } datadriven.RunTest(t, datapathutils.TestDataPath(t, "disk_bandwidth_limiter"), func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "init": - dbl = makeDiskBandwidthLimiter() + dbl = newDiskBandwidthLimiter() return dblToString() case "compute": - var readBandwidth, writeBandwidth, provisionedBandwidth int - d.ScanArgs(t, "read-bw", &readBandwidth) - d.ScanArgs(t, "write-bw", &writeBandwidth) - d.ScanArgs(t, "provisioned-bw", &provisionedBandwidth) + var readBytes, writeBytes, intProvisionedBytes int + d.ScanArgs(t, "int-read-bytes", &readBytes) + d.ScanArgs(t, "int-write-bytes", &writeBytes) + d.ScanArgs(t, "int-provisioned-bytes", &intProvisionedBytes) diskLoad := intervalDiskLoadInfo{ - readBandwidth: int64(readBandwidth), - writeBandwidth: int64(writeBandwidth), - provisionedBandwidth: int64(provisionedBandwidth), + intReadBytes: int64(readBytes), + intWriteBytes: int64(writeBytes), + intProvisionedDiskBytes: int64(intProvisionedBytes), + elasticBandwidthMaxUtil: 0.9, } - var incomingBytes, regularTokensUsed, elasticTokensUsed int - d.ScanArgs(t, "incoming-bytes", &incomingBytes) + var regularTokensUsed, elasticTokensUsed int64 d.ScanArgs(t, "regular-tokens-used", ®ularTokensUsed) d.ScanArgs(t, "elastic-tokens-used", &elasticTokensUsed) - lsmInfo := intervalLSMInfo{ - incomingBytes: int64(incomingBytes), - regularTokensUsed: int64(regularTokensUsed), - elasticTokensUsed: int64(elasticTokensUsed), + usedTokens := [admissionpb.NumWorkClasses]diskTokens{ + {writeByteTokens: regularTokensUsed}, // regular + {writeByteTokens: elasticTokensUsed}, // elastic } - dbl.computeElasticTokens(context.Background(), diskLoad, lsmInfo) + + dbl.computeElasticTokens(diskLoad, usedTokens) return dblToString() default: diff --git a/pkg/util/admission/grant_coordinator.go b/pkg/util/admission/grant_coordinator.go index 8f650ca44878..0c5d23f0e112 100644 --- a/pkg/util/admission/grant_coordinator.go +++ b/pkg/util/admission/grant_coordinator.go @@ -175,7 +175,7 @@ func (sgc *StoreGrantCoordinators) initGrantCoordinator(storeID roachpb.StoreID) } kvg.coordMu.availableIOTokens[admissionpb.RegularWorkClass] = unlimitedTokens / unloadedDuration.ticksInAdjustmentInterval() kvg.coordMu.availableIOTokens[admissionpb.ElasticWorkClass] = kvg.coordMu.availableIOTokens[admissionpb.RegularWorkClass] - kvg.coordMu.elasticDiskBWTokensAvailable = unlimitedTokens / unloadedDuration.ticksInAdjustmentInterval() + kvg.coordMu.diskTokensAvailable.writeByteTokens = unlimitedTokens / unloadedDuration.ticksInAdjustmentInterval() opts := makeWorkQueueOptions(KVWork) // This is IO work, so override the usesTokens value. @@ -214,7 +214,7 @@ func (sgc *StoreGrantCoordinators) initGrantCoordinator(storeID roachpb.StoreID) settings: sgc.settings, kvRequester: storeReq, perWorkTokenEstimator: makeStorePerWorkTokenEstimator(), - diskBandwidthLimiter: makeDiskBandwidthLimiter(), + diskBandwidthLimiter: newDiskBandwidthLimiter(), kvGranter: kvg, l0CompactedBytes: sgc.l0CompactedBytes, l0TokensProduced: sgc.l0TokensProduced, @@ -987,9 +987,9 @@ func (coord *GrantCoordinator) SafeFormat(s redact.SafePrinter, _ rune) { case *slotGranter: s.Printf("%s%s: used: %d, total: %d", curSep, kind, g.usedSlots, g.totalSlots) case *kvStoreTokenGranter: - s.Printf(" io-avail: %d(%d), elastic-disk-bw-tokens-avail: %d", g.coordMu.availableIOTokens[admissionpb.RegularWorkClass], + s.Printf(" io-avail: %d(%d), disk-write-tokens-avail: %d", g.coordMu.availableIOTokens[admissionpb.RegularWorkClass], g.coordMu.availableIOTokens[admissionpb.ElasticWorkClass], - g.coordMu.elasticDiskBWTokensAvailable) + g.coordMu.diskTokensAvailable.writeByteTokens) } case SQLStatementLeafStartWork, SQLStatementRootStartWork: if coord.granters[i] != nil { diff --git a/pkg/util/admission/granter.go b/pkg/util/admission/granter.go index 24c29f5d39e6..6a483779275c 100644 --- a/pkg/util/admission/granter.go +++ b/pkg/util/admission/granter.go @@ -303,10 +303,10 @@ type kvStoreTokenGranter struct { // blocks if availableElasticIOTokens <= 0. availableIOTokens [admissionpb.NumWorkClasses]int64 elasticIOTokensUsedByElastic int64 + // TODO(aaditya): add support for read/IOPS tokens. // Disk bandwidth tokens. - elasticDiskBWTokensAvailable int64 - - diskBWTokensUsed [admissionpb.NumWorkClasses]int64 + diskTokensAvailable diskTokens + diskTokensUsed [admissionpb.NumWorkClasses]diskTokens } ioTokensExhaustedDurationMetric [admissionpb.NumWorkClasses]*metric.Counter @@ -320,7 +320,7 @@ type kvStoreTokenGranter struct { tokensTakenMetric *metric.Counter // Estimation models. - l0WriteLM, l0IngestLM, ingestLM tokensLinearModel + l0WriteLM, l0IngestLM, ingestLM, writeAmpLM tokensLinearModel } var _ granterWithLockedCalls = &kvStoreTokenGranter{} @@ -397,21 +397,42 @@ func (sg *kvStoreTokenGranter) tryGetLocked(count int64, demuxHandle int8) grant // queue, and since the only case where tryGetLocked is called for // elasticWorkClass is when the queue is empty, this case should be rare // (and not cause a performance isolation failure). + + // NB: For disk write tokens, we apply `writeAmpLM` to account for write + // amplification. This model is used again later in + // `storeReplicatedWorkAdmittedLocked()`. There is an obvious gap in + // accounting here, since the model could change between the two calls for the + // same request. For example: + // (1) We have a request that requests 50 tokens, and write-amp LM is + // currently 10.0x + 1. We will deduct 501 tokens for disk writes. + // (2) Before we adjust the actual write bytes used by the write request, the + // write-amp model is updated to 5.0x + 1. + // (3) In `storeReplicatedWorkAdmittedLocked()`, we learn that the request + // used 200 actual bytes. Now we will apply the new LM to get 251 tokens + // initially deducted, and apply the LM for 1001 actual bytes used. We + // will now deduct 750 additional tokens from the bucket. Now we have + // deducted 1251 tokens rather than 1001. + // This can also go the other way, where we deduct fewer tokens than actually + // needed. We are generally okay with this since the model changes + // infrequently (every 15s), and the disk bandwidth limiter is designed to + // generally under admit and only pace elastic work. + adjustedDiskWriteTokens := sg.writeAmpLM.applyLinearModel(count) switch wc { case admissionpb.RegularWorkClass: if sg.coordMu.availableIOTokens[admissionpb.RegularWorkClass] > 0 { sg.subtractTokensLocked(count, count, false) - sg.coordMu.diskBWTokensUsed[wc] += count + sg.coordMu.diskTokensAvailable.writeByteTokens -= adjustedDiskWriteTokens + sg.coordMu.diskTokensUsed[wc].writeByteTokens += adjustedDiskWriteTokens return grantSuccess } case admissionpb.ElasticWorkClass: - if sg.coordMu.elasticDiskBWTokensAvailable > 0 && + if sg.coordMu.diskTokensAvailable.writeByteTokens > 0 && sg.coordMu.availableIOTokens[admissionpb.RegularWorkClass] > 0 && sg.coordMu.availableIOTokens[admissionpb.ElasticWorkClass] > 0 { - sg.coordMu.elasticDiskBWTokensAvailable -= count sg.subtractTokensLocked(count, count, false) sg.coordMu.elasticIOTokensUsedByElastic += count - sg.coordMu.diskBWTokensUsed[wc] += count + sg.coordMu.diskTokensAvailable.writeByteTokens -= adjustedDiskWriteTokens + sg.coordMu.diskTokensUsed[wc].writeByteTokens += adjustedDiskWriteTokens return grantSuccess } } @@ -428,11 +449,12 @@ func (sg *kvStoreTokenGranter) returnGrantLocked(count int64, demuxHandle int8) // Return count tokens to the "IO tokens". sg.subtractTokensLocked(-count, -count, false) if wc == admissionpb.ElasticWorkClass { - // Return count tokens to the elastic disk bandwidth tokens. - sg.coordMu.elasticDiskBWTokensAvailable += count sg.coordMu.elasticIOTokensUsedByElastic -= count } - sg.coordMu.diskBWTokensUsed[wc] -= count + // Return tokens to disk bandwidth bucket. + diskTokenCount := sg.writeAmpLM.applyLinearModel(count) + sg.coordMu.diskTokensAvailable.writeByteTokens += diskTokenCount + sg.coordMu.diskTokensUsed[wc].writeByteTokens -= diskTokenCount } func (sg *kvStoreTokenGranter) tookWithoutPermission(workClass admissionpb.WorkClass, count int64) { @@ -444,10 +466,11 @@ func (sg *kvStoreTokenGranter) tookWithoutPermissionLocked(count int64, demuxHan wc := admissionpb.WorkClass(demuxHandle) sg.subtractTokensLocked(count, count, false) if wc == admissionpb.ElasticWorkClass { - sg.coordMu.elasticDiskBWTokensAvailable -= count sg.coordMu.elasticIOTokensUsedByElastic += count } - sg.coordMu.diskBWTokensUsed[wc] += count + diskTokenCount := sg.writeAmpLM.applyLinearModel(count) + sg.coordMu.diskTokensAvailable.writeByteTokens -= diskTokenCount + sg.coordMu.diskTokensUsed[wc].writeByteTokens += diskTokenCount } // subtractTokensLocked is a helper function that subtracts count tokens (count @@ -497,7 +520,7 @@ func (sg *kvStoreTokenGranter) requesterHasWaitingRequests() bool { // tryGrantLocked implements granterWithLockedCalls. func (sg *kvStoreTokenGranter) tryGrantLocked(grantChainID grantChainID) grantResult { // First try granting to regular requester. - for wc := range sg.coordMu.diskBWTokensUsed { + for wc := range sg.coordMu.diskTokensUsed { req := sg.regularRequester if admissionpb.WorkClass(wc) == admissionpb.ElasticWorkClass { req = sg.elasticRequester @@ -534,10 +557,10 @@ func (sg *kvStoreTokenGranter) tryGrantLocked(grantChainID grantChainID) grantRe func (sg *kvStoreTokenGranter) setAvailableTokens( ioTokens int64, elasticIOTokens int64, - elasticDiskBandwidthTokens int64, + elasticDiskWriteTokens int64, ioTokenCapacity int64, elasticIOTokenCapacity int64, - elasticDiskBandwidthTokensCapacity int64, + elasticDiskWriteTokensCapacity int64, lastTick bool, ) (ioTokensUsed int64, ioTokensUsedByElasticWork int64) { sg.coord.mu.Lock() @@ -577,34 +600,40 @@ func (sg *kvStoreTokenGranter) setAvailableTokens( } sg.startingIOTokens = sg.coordMu.availableIOTokens[admissionpb.RegularWorkClass] - sg.coordMu.elasticDiskBWTokensAvailable += elasticDiskBandwidthTokens - if sg.coordMu.elasticDiskBWTokensAvailable > elasticDiskBandwidthTokensCapacity { - sg.coordMu.elasticDiskBWTokensAvailable = elasticDiskBandwidthTokensCapacity + sg.coordMu.diskTokensAvailable.writeByteTokens += elasticDiskWriteTokens + if sg.coordMu.diskTokensAvailable.writeByteTokens > elasticDiskWriteTokensCapacity { + sg.coordMu.diskTokensAvailable.writeByteTokens = elasticDiskWriteTokensCapacity } return ioTokensUsed, ioTokensUsedByElasticWork } // getDiskTokensUsedAndResetLocked implements granterWithIOTokens. -func (sg *kvStoreTokenGranter) getDiskTokensUsedAndReset() [admissionpb.NumWorkClasses]int64 { +func (sg *kvStoreTokenGranter) getDiskTokensUsedAndReset() ( + usedTokens [admissionpb.NumWorkClasses]diskTokens, +) { sg.coord.mu.Lock() defer sg.coord.mu.Unlock() - result := sg.coordMu.diskBWTokensUsed - for i := range sg.coordMu.diskBWTokensUsed { - sg.coordMu.diskBWTokensUsed[i] = 0 + for i := 0; i < int(admissionpb.NumWorkClasses); i++ { + usedTokens[i] = sg.coordMu.diskTokensUsed[i] + sg.coordMu.diskTokensUsed[i] = diskTokens{} } - return result + return usedTokens } // setAdmittedModelsLocked implements granterWithIOTokens. func (sg *kvStoreTokenGranter) setLinearModels( - l0WriteLM tokensLinearModel, l0IngestLM tokensLinearModel, ingestLM tokensLinearModel, + l0WriteLM tokensLinearModel, + l0IngestLM tokensLinearModel, + ingestLM tokensLinearModel, + writeAmpLM tokensLinearModel, ) { sg.coord.mu.Lock() defer sg.coord.mu.Unlock() sg.l0WriteLM = l0WriteLM sg.l0IngestLM = l0IngestLM sg.ingestLM = ingestLM + sg.writeAmpLM = writeAmpLM } func (sg *kvStoreTokenGranter) storeReplicatedWorkAdmittedLocked( @@ -616,7 +645,7 @@ func (sg *kvStoreTokenGranter) storeReplicatedWorkAdmittedLocked( // Reminder: coord.mu protects the state in the kvStoreTokenGranter. exhaustedFunc := func() bool { return sg.coordMu.availableIOTokens[admissionpb.RegularWorkClass] <= 0 || - (wc == admissionpb.ElasticWorkClass && (sg.coordMu.elasticDiskBWTokensAvailable <= 0 || + (wc == admissionpb.ElasticWorkClass && (sg.coordMu.diskTokensAvailable.writeByteTokens <= 0 || sg.coordMu.availableIOTokens[admissionpb.ElasticWorkClass] <= 0)) } wasExhausted := exhaustedFunc() @@ -625,14 +654,20 @@ func (sg *kvStoreTokenGranter) storeReplicatedWorkAdmittedLocked( actualL0Tokens := actualL0WriteTokens + actualL0IngestTokens additionalL0TokensNeeded := actualL0Tokens - originalTokens sg.subtractTokensLocked(additionalL0TokensNeeded, additionalL0TokensNeeded, false) - actualIngestTokens := sg.ingestLM.applyLinearModel(admittedInfo.IngestedBytes) - additionalDiskBWTokensNeeded := (actualL0WriteTokens + actualIngestTokens) - originalTokens if wc == admissionpb.ElasticWorkClass { - sg.coordMu.elasticDiskBWTokensAvailable -= additionalDiskBWTokensNeeded sg.coordMu.elasticIOTokensUsedByElastic += additionalL0TokensNeeded } - sg.coordMu.diskBWTokensUsed[wc] += additionalDiskBWTokensNeeded - if canGrantAnother && (additionalL0TokensNeeded < 0 || additionalDiskBWTokensNeeded < 0) { + + // Adjust disk write tokens. + ingestIntoLSM := sg.ingestLM.applyLinearModel(admittedInfo.IngestedBytes) + totalBytesIntoLSM := actualL0WriteTokens + ingestIntoLSM + actualDiskWriteTokens := sg.writeAmpLM.applyLinearModel(totalBytesIntoLSM) + originalDiskTokens := sg.writeAmpLM.applyLinearModel(originalTokens) + additionalDiskWriteTokens := actualDiskWriteTokens - originalDiskTokens + sg.coordMu.diskTokensAvailable.writeByteTokens -= additionalDiskWriteTokens + sg.coordMu.diskTokensUsed[wc].writeByteTokens += additionalDiskWriteTokens + + if canGrantAnother && (additionalL0TokensNeeded < 0) { isExhausted := exhaustedFunc() if (wasExhausted && !isExhausted) || sg.coord.knobs.AlwaysTryGrantWhenAdmitted { sg.coord.tryGrantLocked() diff --git a/pkg/util/admission/granter_test.go b/pkg/util/admission/granter_test.go index 71a25509660f..778891f151bc 100644 --- a/pkg/util/admission/granter_test.go +++ b/pkg/util/admission/granter_test.go @@ -43,7 +43,7 @@ import ( // continue-grant-chain work= // cpu-load runnable= procs= [infrequent=] // init-store-grant-coordinator -// set-tokens io-tokens= elastic-disk-bw-tokens= +// set-tokens io-tokens= disk-write-tokens= func TestGranterBasic(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -158,9 +158,11 @@ func TestGranterBasic(t *testing.T) { coord, ok = storeCoordinators.gcMap.Load(1) require.True(t, ok) kvStoreGranter := coord.granters[KVWork].(*kvStoreTokenGranter) - // Use the same model for all 3 kinds of models. + // Use the same model for the IO linear models. tlm := tokensLinearModel{multiplier: 0.5, constant: 50} - kvStoreGranter.setLinearModels(tlm, tlm, tlm) + // Use w-amp of 1 for the purpose of this test. + wamplm := tokensLinearModel{multiplier: 1, constant: 0} + kvStoreGranter.setLinearModels(tlm, tlm, tlm, wamplm) return flushAndReset() case "set-has-waiting-requests": @@ -231,10 +233,10 @@ func TestGranterBasic(t *testing.T) { case "set-tokens-loop": var ioTokens int - var elasticTokens int + var elasticDiskWriteTokens int var loop int d.ScanArgs(t, "io-tokens", &ioTokens) - d.ScanArgs(t, "elastic-disk-bw-tokens", &elasticTokens) + d.ScanArgs(t, "disk-write-tokens", &elasticDiskWriteTokens) d.ScanArgs(t, "loop", &loop) for loop > 0 { @@ -242,8 +244,13 @@ func TestGranterBasic(t *testing.T) { // We are not using a real ioLoadListener, and simply setting the // tokens (the ioLoadListener has its own test). coord.granters[KVWork].(*kvStoreTokenGranter).setAvailableTokens( - int64(ioTokens), int64(ioTokens), int64(elasticTokens), - int64(ioTokens*250), int64(ioTokens*250), int64(elasticTokens*250), false, + int64(ioTokens), + int64(ioTokens), + int64(elasticDiskWriteTokens), + int64(ioTokens*250), + int64(ioTokens*250), + int64(elasticDiskWriteTokens*250), + false, // lastTick ) } coord.testingTryGrant() @@ -251,10 +258,10 @@ func TestGranterBasic(t *testing.T) { case "set-tokens": var ioTokens int - var elasticTokens int + var elasticDiskWriteTokens int var tickInterval int d.ScanArgs(t, "io-tokens", &ioTokens) - d.ScanArgs(t, "elastic-disk-bw-tokens", &elasticTokens) + d.ScanArgs(t, "disk-write-tokens", &elasticDiskWriteTokens) elasticIOTokens := ioTokens if d.HasArg("elastic-io-tokens") { d.ScanArgs(t, "elastic-io-tokens", &elasticIOTokens) @@ -273,9 +280,13 @@ func TestGranterBasic(t *testing.T) { // We are not using a real ioLoadListener, and simply setting the // tokens (the ioLoadListener has its own test). coord.granters[KVWork].(*kvStoreTokenGranter).setAvailableTokens( - int64(ioTokens), int64(elasticIOTokens), int64(elasticTokens), - int64(ioTokens*burstMultiplier), int64(elasticIOTokens*burstMultiplier), - int64(elasticTokens*burstMultiplier), false, + int64(ioTokens), + int64(elasticIOTokens), + int64(elasticDiskWriteTokens), + int64(ioTokens*burstMultiplier), + int64(elasticIOTokens*burstMultiplier), + int64(elasticDiskWriteTokens*burstMultiplier), + false, // lastTick ) coord.testingTryGrant() return flushAndReset() diff --git a/pkg/util/admission/io_load_listener.go b/pkg/util/admission/io_load_listener.go index 3cded67ba568..a4f185dc1792 100644 --- a/pkg/util/admission/io_load_listener.go +++ b/pkg/util/admission/io_load_listener.go @@ -85,6 +85,15 @@ var L0SubLevelCountOverloadThreshold = settings.RegisterIntSetting( "when the L0 sub-level count exceeds this threshold, the store is considered overloaded", l0SubLevelCountOverloadThreshold, settings.PositiveInt) +// ElasticBandwidthMaxUtil sets the max utilization for disk bandwidth for elastic traffic. +var ElasticBandwidthMaxUtil = settings.RegisterFloatSetting( + settings.SystemOnly, "kvadmission.store.elastic_disk_bandwidth_max_util", + "sets the max utilization for disk bandwidth for elastic traffic", + 0.8, + settings.NonNegativeFloatWithMaximum(1.0), + settings.FloatWithMinimum(0.05), +) + // L0MinimumSizePerSubLevel is a minimum size threshold per sub-level, to // avoid over reliance on the sub-level count as a signal of overload. Pebble // sometimes has to do frequent flushes of the memtable due to ingesting @@ -194,7 +203,7 @@ type ioLoadListener struct { statsInitialized bool adjustTokensResult perWorkTokenEstimator storePerWorkTokenEstimator - diskBandwidthLimiter diskBandwidthLimiter + diskBandwidthLimiter *diskBandwidthLimiter l0CompactedBytes *metric.Counter l0TokensProduced *metric.Counter @@ -210,9 +219,8 @@ type ioLoadListenerState struct { cumFlushWriteThroughput pebble.ThroughputMetric diskBW struct { // Cumulative - bytesRead uint64 - bytesWritten uint64 - incomingLSMBytes uint64 + bytesRead uint64 + bytesWritten uint64 } cumCompactionStats cumStoreCompactionStats cumWALSecondaryWriteDuration time.Duration @@ -247,8 +255,8 @@ type ioLoadListenerState struct { // elasticDiskBWTokens represents the tokens to give out until the next call // to adjustTokens. They are parceled out in small intervals. // elasticDiskTokensAllocated represents what has been given out. - elasticDiskBWTokens int64 - elasticDiskBWTokensAllocated int64 + elasticDiskWriteTokens int64 + elasticDiskWriteTokensAllocated int64 } type cumStoreCompactionStats struct { @@ -443,14 +451,11 @@ func (t *tokenAllocationTicker) stop() { *t = tokenAllocationTicker{} } -func cumLSMWriteAndIngestedBytes( - m *pebble.Metrics, -) (writeAndIngestedBytes uint64, ingestedBytes uint64) { +func cumLSMIngestedBytes(m *pebble.Metrics) (ingestedBytes uint64) { for i := range m.Levels { - writeAndIngestedBytes += m.Levels[i].BytesIngested + m.Levels[i].BytesFlushed ingestedBytes += m.Levels[i].BytesIngested } - return writeAndIngestedBytes, ingestedBytes + return ingestedBytes } func replaceFlushThroughputBytesBySSTableWriteThroughput(m *pebble.Metrics) { @@ -467,8 +472,9 @@ func (io *ioLoadListener) pebbleMetricsTick(ctx context.Context, metrics StoreMe if !io.statsInitialized { io.statsInitialized = true sas := io.kvRequester.getStoreAdmissionStats() - cumLSMIncomingBytes, cumLSMIngestedBytes := cumLSMWriteAndIngestedBytes(metrics.Metrics) - io.perWorkTokenEstimator.updateEstimates(metrics.Levels[0], cumLSMIngestedBytes, sas) + cumIngestBytes := cumLSMIngestedBytes(metrics.Metrics) + io.perWorkTokenEstimator.updateEstimates( + metrics.Levels[0], cumIngestBytes, metrics.DiskStats.BytesWritten, sas) io.adjustTokensResult = adjustTokensResult{ ioLoadListenerState: ioLoadListenerState{ cumL0AddedBytes: m.Levels[0].BytesFlushed + m.Levels[0].BytesIngested, @@ -480,7 +486,7 @@ func (io *ioLoadListener) pebbleMetricsTick(ctx context.Context, metrics StoreMe // No initial limit, i.e, the first interval is unlimited. totalNumByteTokens: unlimitedTokens, totalNumElasticByteTokens: unlimitedTokens, - elasticDiskBWTokens: unlimitedTokens, + elasticDiskWriteTokens: unlimitedTokens, }, aux: adjustTokensAuxComputations{}, ioThreshold: &admissionpb.IOThreshold{ @@ -494,7 +500,6 @@ func (io *ioLoadListener) pebbleMetricsTick(ctx context.Context, metrics StoreMe } io.diskBW.bytesRead = metrics.DiskStats.BytesRead io.diskBW.bytesWritten = metrics.DiskStats.BytesWritten - io.diskBW.incomingLSMBytes = cumLSMIncomingBytes io.copyAuxEtcFromPerWorkEstimator() // Assume system starts off unloaded. @@ -551,15 +556,15 @@ func (io *ioLoadListener) allocateTokensTick(remainingTicks int64) { panic(errors.AssertionFailedf("toAllocateElasticByteTokens is negative %d", toAllocateElasticByteTokens)) } - toAllocateElasticDiskBWTokens := + toAllocateElasticDiskWriteTokens := allocateFunc( - io.elasticDiskBWTokens, - io.elasticDiskBWTokensAllocated, + io.elasticDiskWriteTokens, + io.elasticDiskWriteTokensAllocated, remainingTicks, ) - if toAllocateElasticDiskBWTokens < 0 { + if toAllocateElasticDiskWriteTokens < 0 { panic(errors.AssertionFailedf("toAllocateElasticDiskBWTokens is negative %d", - toAllocateElasticDiskBWTokens)) + toAllocateElasticDiskWriteTokens)) } // INVARIANT: toAllocate >= 0. io.byteTokensAllocated += toAllocateByteTokens @@ -571,7 +576,7 @@ func (io *ioLoadListener) allocateTokensTick(remainingTicks int64) { panic(errors.AssertionFailedf( "tokens allocated is negative %d", io.elasticByteTokensAllocated)) } - io.elasticDiskBWTokensAllocated += toAllocateElasticDiskBWTokens + io.elasticDiskWriteTokensAllocated += toAllocateElasticDiskWriteTokens tokensMaxCapacity := allocateFunc( io.totalNumByteTokens, 0, unloadedDuration.ticksInAdjustmentInterval(), @@ -579,12 +584,12 @@ func (io *ioLoadListener) allocateTokensTick(remainingTicks int64) { elasticTokensMaxCapacity := allocateFunc( io.totalNumElasticByteTokens, 0, unloadedDuration.ticksInAdjustmentInterval()) diskBWTokenMaxCapacity := allocateFunc( - io.elasticDiskBWTokens, 0, unloadedDuration.ticksInAdjustmentInterval(), + io.elasticDiskWriteTokens, 0, unloadedDuration.ticksInAdjustmentInterval(), ) tokensUsed, tokensUsedByElasticWork := io.kvGranter.setAvailableTokens( toAllocateByteTokens, toAllocateElasticByteTokens, - toAllocateElasticDiskBWTokens, + toAllocateElasticDiskWriteTokens, tokensMaxCapacity, elasticTokensMaxCapacity, diskBWTokenMaxCapacity, @@ -595,12 +600,13 @@ func (io *ioLoadListener) allocateTokensTick(remainingTicks int64) { } func computeIntervalDiskLoadInfo( - prevCumBytesRead uint64, prevCumBytesWritten uint64, diskStats DiskStats, + prevCumBytesRead uint64, prevCumBytesWritten uint64, diskStats DiskStats, elasticBWUtil float64, ) intervalDiskLoadInfo { return intervalDiskLoadInfo{ - readBandwidth: int64((diskStats.BytesRead - prevCumBytesRead) / adjustmentInterval), - writeBandwidth: int64((diskStats.BytesWritten - prevCumBytesWritten) / adjustmentInterval), - provisionedBandwidth: diskStats.ProvisionedBandwidth, + intReadBytes: int64(diskStats.BytesRead - prevCumBytesRead), + intWriteBytes: int64(diskStats.BytesWritten - prevCumBytesWritten), + intProvisionedDiskBytes: diskStats.ProvisionedBandwidth * adjustmentInterval, + elasticBandwidthMaxUtil: elasticBWUtil, } } @@ -616,7 +622,7 @@ func computeIntervalDiskLoadInfo( func (io *ioLoadListener) adjustTokens(ctx context.Context, metrics StoreMetrics) { sas := io.kvRequester.getStoreAdmissionStats() // Copy the cumulative disk bandwidth values for later use. - cumDiskBW := io.ioLoadListenerState.diskBW + cumDiskBW := io.diskBW wt := metrics.Flush.WriteThroughput wt.Subtract(io.cumFlushWriteThroughput) if wt.Bytes < 0 { @@ -634,38 +640,34 @@ func (io *ioLoadListener) adjustTokens(ctx context.Context, metrics StoreMetrics MinFlushUtilizationFraction.Get(&io.settings.SV), ) io.adjustTokensResult = res - cumLSMIncomingBytes, cumLSMIngestedBytes := cumLSMWriteAndIngestedBytes(metrics.Metrics) - { - // Disk Bandwidth tokens. - io.aux.diskBW.intervalDiskLoadInfo = computeIntervalDiskLoadInfo( - cumDiskBW.bytesRead, cumDiskBW.bytesWritten, metrics.DiskStats) - diskTokensUsed := io.kvGranter.getDiskTokensUsedAndReset() - io.aux.diskBW.intervalLSMInfo = intervalLSMInfo{ - incomingBytes: int64(cumLSMIncomingBytes) - int64(cumDiskBW.incomingLSMBytes), - regularTokensUsed: diskTokensUsed[admissionpb.RegularWorkClass], - elasticTokensUsed: diskTokensUsed[admissionpb.ElasticWorkClass], - } - if metrics.DiskStats.ProvisionedBandwidth > 0 { - io.elasticDiskBWTokens = io.diskBandwidthLimiter.computeElasticTokens(ctx, - io.aux.diskBW.intervalDiskLoadInfo, io.aux.diskBW.intervalLSMInfo) - io.elasticDiskBWTokensAllocated = 0 - } - if metrics.DiskStats.ProvisionedBandwidth == 0 || - !DiskBandwidthTokensForElasticEnabled.Get(&io.settings.SV) { - io.elasticDiskBWTokens = unlimitedTokens - } - io.diskBW.bytesRead = metrics.DiskStats.BytesRead - io.diskBW.bytesWritten = metrics.DiskStats.BytesWritten - io.diskBW.incomingLSMBytes = cumLSMIncomingBytes + cumIngestedBytes := cumLSMIngestedBytes(metrics.Metrics) + + // Disk Bandwidth tokens. + elasticBWMaxUtil := ElasticBandwidthMaxUtil.Get(&io.settings.SV) + intDiskLoadInfo := computeIntervalDiskLoadInfo( + cumDiskBW.bytesRead, cumDiskBW.bytesWritten, metrics.DiskStats, elasticBWMaxUtil) + diskTokensUsed := io.kvGranter.getDiskTokensUsedAndReset() + if metrics.DiskStats.ProvisionedBandwidth > 0 { + tokens := io.diskBandwidthLimiter.computeElasticTokens( + intDiskLoadInfo, diskTokensUsed) + io.elasticDiskWriteTokens = tokens.writeByteTokens + io.elasticDiskWriteTokensAllocated = 0 + } + if metrics.DiskStats.ProvisionedBandwidth == 0 || + !DiskBandwidthTokensForElasticEnabled.Get(&io.settings.SV) { + io.elasticDiskWriteTokens = unlimitedTokens } - io.perWorkTokenEstimator.updateEstimates(metrics.Levels[0], cumLSMIngestedBytes, sas) + io.diskBW.bytesRead = metrics.DiskStats.BytesRead + io.diskBW.bytesWritten = metrics.DiskStats.BytesWritten + + io.perWorkTokenEstimator.updateEstimates(metrics.Levels[0], cumIngestedBytes, metrics.DiskStats.BytesWritten, sas) io.copyAuxEtcFromPerWorkEstimator() requestEstimates := io.perWorkTokenEstimator.getStoreRequestEstimatesAtAdmission() io.kvRequester.setStoreRequestEstimates(requestEstimates) - l0WriteLM, l0IngestLM, ingestLM := io.perWorkTokenEstimator.getModelsAtDone() - io.kvGranter.setLinearModels(l0WriteLM, l0IngestLM, ingestLM) - if io.aux.doLogFlush || io.elasticDiskBWTokens != unlimitedTokens || log.V(1) { - log.Infof(ctx, "IO overload: %s", io.adjustTokensResult) + l0WriteLM, l0IngestLM, ingestLM, writeAmpLM := io.perWorkTokenEstimator.getModelsAtDone() + io.kvGranter.setLinearModels(l0WriteLM, l0IngestLM, ingestLM, writeAmpLM) + if io.aux.doLogFlush || io.diskBandwidthLimiter.state.diskBWUtil > 0.8 || log.V(1) { + log.Infof(ctx, "IO overload: %s; %s", io.adjustTokensResult, io.diskBandwidthLimiter) } } @@ -678,10 +680,11 @@ func (io *ioLoadListener) copyAuxEtcFromPerWorkEstimator() { io.adjustTokensResult.aux.perWorkTokensAux = io.perWorkTokenEstimator.aux requestEstimates := io.perWorkTokenEstimator.getStoreRequestEstimatesAtAdmission() io.adjustTokensResult.requestEstimates = requestEstimates - l0WriteLM, l0IngestLM, ingestLM := io.perWorkTokenEstimator.getModelsAtDone() + l0WriteLM, l0IngestLM, ingestLM, writeAmpLM := io.perWorkTokenEstimator.getModelsAtDone() io.adjustTokensResult.l0WriteLM = l0WriteLM io.adjustTokensResult.l0IngestLM = l0IngestLM io.adjustTokensResult.ingestLM = ingestLM + io.adjustTokensResult.writeAmpLM = writeAmpLM } type tokenKind int8 @@ -710,11 +713,6 @@ type adjustTokensAuxComputations struct { perWorkTokensAux perWorkTokensAux doLogFlush bool - - diskBW struct { - intervalDiskLoadInfo intervalDiskLoadInfo - intervalLSMInfo intervalLSMInfo - } } // adjustTokensInner is used for computing tokens based on compaction and @@ -1159,6 +1157,7 @@ type adjustTokensResult struct { l0WriteLM tokensLinearModel l0IngestLM tokensLinearModel ingestLM tokensLinearModel + writeAmpLM tokensLinearModel aux adjustTokensAuxComputations ioThreshold *admissionpb.IOThreshold // never nil } @@ -1186,6 +1185,10 @@ func (res adjustTokensResult) SafeFormat(p redact.SafePrinter, _ rune) { p.Printf("%s acc-ingest (%s bypassed) + ", ib(res.aux.perWorkTokensAux.intIngestedAccountedBytes), ib(res.aux.perWorkTokensAux.intIngestedBypassedAccountedBytes)) + // Adjusted LSM writes and disk writes that were used for w-amp estimation. + p.Printf("%s adjusted-LSM-writes + %s adjusted-disk-writes + ", + ib(res.aux.perWorkTokensAux.intAdjustedLSMWrites), + ib(res.aux.perWorkTokensAux.intAdjustedDiskWriteBytes)) // The models we are fitting to compute tokens based on the reported size of // the write and ingest. p.Printf("write-model %.2fx+%s (smoothed %.2fx+%s) + ", @@ -1196,6 +1199,10 @@ func (res adjustTokensResult) SafeFormat(p redact.SafePrinter, _ rune) { res.aux.perWorkTokensAux.intL0IngestedLinearModel.multiplier, ib(res.aux.perWorkTokensAux.intL0IngestedLinearModel.constant), res.l0IngestLM.multiplier, ib(res.l0IngestLM.constant)) + p.Printf("write-amp-model %.2fx+%s (smoothed %.2fx+%s) + ", + res.aux.perWorkTokensAux.intWriteAmpLinearModel.multiplier, + ib(res.aux.perWorkTokensAux.intWriteAmpLinearModel.constant), + res.writeAmpLM.multiplier, ib(res.writeAmpLM.constant)) // The tokens used per request at admission time, when no size information // is known. p.Printf("at-admission-tokens %s, ", ib(res.requestEstimates.writeTokens)) @@ -1233,18 +1240,6 @@ func (res adjustTokensResult) SafeFormat(p redact.SafePrinter, _ rune) { } else { p.SafeString("all") } - if res.elasticDiskBWTokens != unlimitedTokens { - p.Printf("; elastic-disk-bw tokens %s (used %s, regular used %s): "+ - "write model %.2fx+%s ingest model %.2fx+%s, ", - ib(res.elasticDiskBWTokens), ib(res.aux.diskBW.intervalLSMInfo.elasticTokensUsed), - ib(res.aux.diskBW.intervalLSMInfo.regularTokensUsed), - res.l0WriteLM.multiplier, ib(res.l0WriteLM.constant), - res.ingestLM.multiplier, ib(res.ingestLM.constant)) - p.Printf("disk bw read %s write %s provisioned %s", - ib(res.aux.diskBW.intervalDiskLoadInfo.readBandwidth), - ib(res.aux.diskBW.intervalDiskLoadInfo.writeBandwidth), - ib(res.aux.diskBW.intervalDiskLoadInfo.provisionedBandwidth)) - } p.Printf("; write stalls %d", res.aux.intWriteStalls) } diff --git a/pkg/util/admission/io_load_listener_test.go b/pkg/util/admission/io_load_listener_test.go index a923344559ff..0df001f10be8 100644 --- a/pkg/util/admission/io_load_listener_test.go +++ b/pkg/util/admission/io_load_listener_test.go @@ -55,11 +55,12 @@ func TestIOLoadListener(t *testing.T) { case "init": L0MinimumSizePerSubLevel.Override(ctx, &st.SV, 0) walFailoverUnlimitedTokens.Override(ctx, &st.SV, false) + ElasticBandwidthMaxUtil.Override(ctx, &st.SV, 1) ioll = &ioLoadListener{ settings: st, kvRequester: req, perWorkTokenEstimator: makeStorePerWorkTokenEstimator(), - diskBandwidthLimiter: makeDiskBandwidthLimiter(), + diskBandwidthLimiter: newDiskBandwidthLimiter(), l0CompactedBytes: metric.NewCounter(l0CompactedBytes), l0TokensProduced: metric.NewCounter(l0TokensProduced), } @@ -201,14 +202,14 @@ func TestIOLoadListener(t *testing.T) { if d.HasArg("bytes-written") { d.ScanArgs(t, "bytes-written", &bytesWritten) } - if d.HasArg("disk-bw-tokens-used") { - var regularTokensUsed, elasticTokensUsed int - d.ScanArgs(t, "disk-bw-tokens-used", ®ularTokensUsed, &elasticTokensUsed) - kvGranter.diskBandwidthTokensUsed[admissionpb.RegularWorkClass] = int64(regularTokensUsed) - kvGranter.diskBandwidthTokensUsed[admissionpb.ElasticWorkClass] = int64(elasticTokensUsed) + if d.HasArg("disk-write-tokens-used") { + var regularTokensUsed, elasticTokensUsed int64 + d.ScanArgs(t, "disk-write-tokens-used", ®ularTokensUsed, &elasticTokensUsed) + kvGranter.diskBandwidthTokensUsed[admissionpb.RegularWorkClass] = diskTokens{writeByteTokens: regularTokensUsed} + kvGranter.diskBandwidthTokensUsed[admissionpb.ElasticWorkClass] = diskTokens{writeByteTokens: elasticTokensUsed} } else { - kvGranter.diskBandwidthTokensUsed[admissionpb.RegularWorkClass] = 0 - kvGranter.diskBandwidthTokensUsed[admissionpb.ElasticWorkClass] = 0 + kvGranter.diskBandwidthTokensUsed[admissionpb.RegularWorkClass] = diskTokens{} + kvGranter.diskBandwidthTokensUsed[admissionpb.ElasticWorkClass] = diskTokens{} } var printOnlyFirstTick bool if d.HasArg("print-only-first-tick") { @@ -232,6 +233,7 @@ func TestIOLoadListener(t *testing.T) { // Do the ticks until just before next adjustment. res := ioll.adjustTokensResult fmt.Fprintln(&buf, redact.StringWithoutMarkers(&res)) + fmt.Fprintln(&buf, redact.StringWithoutMarkers(ioll.diskBandwidthLimiter)) res.ioThreshold = nil // avoid nondeterminism fmt.Fprintf(&buf, "%+v\n", (rawTokenResult)(res)) if req.buf.Len() > 0 { @@ -368,7 +370,7 @@ func TestBadIOLoadListenerStats(t *testing.T) { settings: st, kvRequester: req, perWorkTokenEstimator: makeStorePerWorkTokenEstimator(), - diskBandwidthLimiter: makeDiskBandwidthLimiter(), + diskBandwidthLimiter: newDiskBandwidthLimiter(), l0CompactedBytes: metric.NewCounter(l0CompactedBytes), l0TokensProduced: metric.NewCounter(l0TokensProduced), } @@ -387,8 +389,8 @@ func TestBadIOLoadListenerStats(t *testing.T) { require.LessOrEqual(t, float64(0), ioll.flushUtilTargetFraction) require.LessOrEqual(t, int64(0), ioll.totalNumByteTokens) require.LessOrEqual(t, int64(0), ioll.byteTokensAllocated) - require.LessOrEqual(t, int64(0), ioll.elasticDiskBWTokens) - require.LessOrEqual(t, int64(0), ioll.elasticDiskBWTokensAllocated) + require.LessOrEqual(t, int64(0), ioll.elasticDiskWriteTokens) + require.LessOrEqual(t, int64(0), ioll.elasticDiskWriteTokensAllocated) } } } @@ -417,7 +419,7 @@ func (r *testRequesterForIOLL) setStoreRequestEstimates(estimates storeRequestEs type testGranterWithIOTokens struct { buf strings.Builder allTokensUsed bool - diskBandwidthTokensUsed [admissionpb.NumWorkClasses]int64 + diskBandwidthTokensUsed [admissionpb.NumWorkClasses]diskTokens } var _ granterWithIOTokens = &testGranterWithIOTokens{} @@ -447,12 +449,17 @@ func (g *testGranterWithIOTokens) setAvailableTokens( return 0, 0 } -func (g *testGranterWithIOTokens) getDiskTokensUsedAndReset() [admissionpb.NumWorkClasses]int64 { +func (g *testGranterWithIOTokens) getDiskTokensUsedAndReset() ( + usedTokens [admissionpb.NumWorkClasses]diskTokens, +) { return g.diskBandwidthTokensUsed } func (g *testGranterWithIOTokens) setLinearModels( - l0WriteLM tokensLinearModel, l0IngestLM tokensLinearModel, ingestLM tokensLinearModel, + l0WriteLM tokensLinearModel, + l0IngestLM tokensLinearModel, + ingestLM tokensLinearModel, + writeAmpLM tokensLinearModel, ) { fmt.Fprintf(&g.buf, "setAdmittedDoneModelsLocked: l0-write-lm: ") printLinearModel(&g.buf, l0WriteLM) @@ -460,6 +467,8 @@ func (g *testGranterWithIOTokens) setLinearModels( printLinearModel(&g.buf, l0IngestLM) fmt.Fprintf(&g.buf, " ingest-lm: ") printLinearModel(&g.buf, ingestLM) + fmt.Fprintf(&g.buf, " write-amp-lm: ") + printLinearModel(&g.buf, writeAmpLM) fmt.Fprintf(&g.buf, "\n") } @@ -493,12 +502,17 @@ func (g *testGranterNonNegativeTokens) setAvailableTokens( return 0, 0 } -func (g *testGranterNonNegativeTokens) getDiskTokensUsedAndReset() [admissionpb.NumWorkClasses]int64 { - return [admissionpb.NumWorkClasses]int64{} +func (g *testGranterNonNegativeTokens) getDiskTokensUsedAndReset() ( + usedTokens [admissionpb.NumWorkClasses]diskTokens, +) { + return [admissionpb.NumWorkClasses]diskTokens{} } func (g *testGranterNonNegativeTokens) setLinearModels( - l0WriteLM tokensLinearModel, l0IngestLM tokensLinearModel, ingestLM tokensLinearModel, + l0WriteLM tokensLinearModel, + l0IngestLM tokensLinearModel, + ingestLM tokensLinearModel, + writeAmpLM tokensLinearModel, ) { require.LessOrEqual(g.t, 0.5, l0WriteLM.multiplier) require.LessOrEqual(g.t, int64(0), l0WriteLM.constant) @@ -506,6 +520,8 @@ func (g *testGranterNonNegativeTokens) setLinearModels( require.LessOrEqual(g.t, int64(0), l0IngestLM.constant) require.LessOrEqual(g.t, 0.5, ingestLM.multiplier) require.LessOrEqual(g.t, int64(0), ingestLM.constant) + require.LessOrEqual(g.t, 1.0, writeAmpLM.multiplier) + require.LessOrEqual(g.t, int64(0), writeAmpLM.constant) } // Tests if the tokenAllocationTicker produces correct adjustment interval diff --git a/pkg/util/admission/store_token_estimation.go b/pkg/util/admission/store_token_estimation.go index 2f7c9b70cbf6..a78176401781 100644 --- a/pkg/util/admission/store_token_estimation.go +++ b/pkg/util/admission/store_token_estimation.go @@ -102,6 +102,11 @@ import "github.com/cockroachdb/pebble" // modeled via the a.x term, and not via the b term, since workloads are // likely (at least for regular writes) to vary significantly in x. +// In addition to the models above, we have one for estimating write +// amplification. writeAmpLM maps the incoming writes to the LSM (L0 writes + +// ingests) to actual disk writes. We use this model to deduct from disk write +// tokens from disk_bandwidth.go. + // See the comment above for the justification of these constants. const l0WriteMultiplierMin = 0.5 const l0WriteMultiplierMax = 3.0 @@ -109,6 +114,8 @@ const l0IngestMultiplierMin = 0.001 const l0IngestMultiplierMax = 1.5 const ingestMultiplierMin = 0.5 const ingestMultiplierMax = 1.5 +const writeAmpMultiplierMin = 1.0 +const writeAmpMultiplierMax = 100.0 type storePerWorkTokenEstimator struct { atAdmissionWorkTokens int64 @@ -121,11 +128,18 @@ type storePerWorkTokenEstimator struct { // Unlike the models above that model bytes into L0, this model computes all // ingested bytes into the LSM. atDoneIngestTokensLinearModel tokensLinearModelFitter + // This model is used to estimate the write amplification due to asynchronous + // compactions after bytes are written to L0. It models the relationship + // between ingests (not including range snapshots) plus incoming L0 bytes and + // total disk write throughput in a given interval. We ignore range snapshots + // here, since they land into lower levels (usually L6) of the LSM. + atDoneWriteAmpLinearModel tokensLinearModelFitter cumStoreAdmissionStats storeAdmissionStats cumL0WriteBytes uint64 cumL0IngestedBytes uint64 cumLSMIngestedBytes uint64 + cumDiskWrites uint64 // Tracked for logging and copied out of here. aux perWorkTokensAux @@ -143,6 +157,7 @@ type perWorkTokensAux struct { intL0WriteLinearModel tokensLinearModel intL0IngestedLinearModel tokensLinearModel intIngestedLinearModel tokensLinearModel + intWriteAmpLinearModel tokensLinearModel // The bypassed count and bytes are also included in the overall interval // stats. @@ -158,6 +173,13 @@ type perWorkTokensAux struct { // intLSMIngestedBytes, and may even be higher than that value because these // are from a different source. intL0IgnoredIngestedBytes int64 + + // These are used for write amplification estimation. intAdjustedLSMWrites + // represent the accounted LSM writes (ingestion + regular writes). + // intAdjustedDiskWriteBytes represent the total write bytes for the interval, + // excluding ignored bytes. + intAdjustedLSMWrites int64 + intAdjustedDiskWriteBytes int64 } func makeStorePerWorkTokenEstimator() storePerWorkTokenEstimator { @@ -169,18 +191,24 @@ func makeStorePerWorkTokenEstimator() storePerWorkTokenEstimator { l0IngestMultiplierMin, l0IngestMultiplierMax, true), atDoneIngestTokensLinearModel: makeTokensLinearModelFitter( ingestMultiplierMin, ingestMultiplierMax, false), + atDoneWriteAmpLinearModel: makeTokensLinearModelFitter( + writeAmpMultiplierMin, writeAmpMultiplierMax, false), } } // NB: first call to updateEstimates only initializes the cumulative values. func (e *storePerWorkTokenEstimator) updateEstimates( - l0Metrics pebble.LevelMetrics, cumLSMIngestedBytes uint64, admissionStats storeAdmissionStats, + l0Metrics pebble.LevelMetrics, + cumLSMIngestedBytes uint64, + cumDiskWrite uint64, + admissionStats storeAdmissionStats, ) { if e.cumL0WriteBytes == 0 { e.cumStoreAdmissionStats = admissionStats e.cumL0WriteBytes = l0Metrics.BytesFlushed e.cumL0IngestedBytes = l0Metrics.BytesIngested e.cumLSMIngestedBytes = cumLSMIngestedBytes + e.cumDiskWrites = cumDiskWrite return } intL0WriteBytes := int64(l0Metrics.BytesFlushed) - int64(e.cumL0WriteBytes) @@ -221,6 +249,15 @@ func (e *storePerWorkTokenEstimator) updateEstimates( e.atDoneIngestTokensLinearModel.updateModelUsingIntervalStats( intIngestedAccountedBytes, adjustedIntLSMIngestedBytes, intWorkCount) + // Write amplification model. + intDiskWrite := int64(cumDiskWrite - e.cumDiskWrites) + adjustedIntLSMWrites := adjustedIntL0WriteBytes + adjustedIntLSMIngestedBytes + adjustedIntDiskWrites := intDiskWrite - intIgnoredIngestedBytes - intL0IgnoredWriteBytes + if adjustedIntDiskWrites < 0 { + adjustedIntDiskWrites = 0 + } + e.atDoneWriteAmpLinearModel.updateModelUsingIntervalStats(adjustedIntLSMWrites, adjustedIntDiskWrites, intWorkCount) + intL0TotalBytes := adjustedIntL0WriteBytes + adjustedIntL0IngestedBytes intAboveRaftWorkCount := int64(admissionStats.aboveRaftStats.workCount) - int64(e.cumStoreAdmissionStats.aboveRaftStats.workCount) @@ -265,6 +302,7 @@ func (e *storePerWorkTokenEstimator) updateEstimates( intL0WriteLinearModel: e.atDoneL0WriteTokensLinearModel.intLinearModel, intL0IngestedLinearModel: e.atDoneL0IngestTokensLinearModel.intLinearModel, intIngestedLinearModel: e.atDoneIngestTokensLinearModel.intLinearModel, + intWriteAmpLinearModel: e.atDoneWriteAmpLinearModel.intLinearModel, intBypassedWorkCount: int64(admissionStats.aux.bypassedCount) - int64(e.cumStoreAdmissionStats.aux.bypassedCount), intL0WriteBypassedAccountedBytes: int64(admissionStats.aux.writeBypassedAccountedBytes) - @@ -273,12 +311,15 @@ func (e *storePerWorkTokenEstimator) updateEstimates( int64(e.cumStoreAdmissionStats.aux.ingestedBypassedAccountedBytes), intL0IgnoredWriteBytes: intL0IgnoredWriteBytes, intL0IgnoredIngestedBytes: intL0IgnoredIngestedBytes, + intAdjustedDiskWriteBytes: adjustedIntDiskWrites, + intAdjustedLSMWrites: adjustedIntLSMWrites, } // Store the latest cumulative values. e.cumStoreAdmissionStats = admissionStats e.cumL0WriteBytes = l0Metrics.BytesFlushed e.cumL0IngestedBytes = l0Metrics.BytesIngested e.cumLSMIngestedBytes = cumLSMIngestedBytes + e.cumDiskWrites = cumDiskWrite } func (e *storePerWorkTokenEstimator) getStoreRequestEstimatesAtAdmission() storeRequestEstimates { @@ -289,8 +330,10 @@ func (e *storePerWorkTokenEstimator) getModelsAtDone() ( l0WriteLM tokensLinearModel, l0IngestLM tokensLinearModel, ingestLM tokensLinearModel, + writeAmpLM tokensLinearModel, ) { return e.atDoneL0WriteTokensLinearModel.smoothedLinearModel, e.atDoneL0IngestTokensLinearModel.smoothedLinearModel, - e.atDoneIngestTokensLinearModel.smoothedLinearModel + e.atDoneIngestTokensLinearModel.smoothedLinearModel, + e.atDoneWriteAmpLinearModel.smoothedLinearModel } diff --git a/pkg/util/admission/store_token_estimation_test.go b/pkg/util/admission/store_token_estimation_test.go index 59762063bf48..cebd15b89cb8 100644 --- a/pkg/util/admission/store_token_estimation_test.go +++ b/pkg/util/admission/store_token_estimation_test.go @@ -31,6 +31,7 @@ func TestStorePerWorkTokenEstimator(t *testing.T) { var l0Metrics pebble.LevelMetrics var admissionStats storeAdmissionStats var cumLSMIngestedBytes uint64 + var cumDiskWrites uint64 datadriven.RunTest(t, datapathutils.TestDataPath(t, "store_per_work_token_estimator"), func(t *testing.T, d *datadriven.TestData) string { @@ -94,11 +95,17 @@ func TestStorePerWorkTokenEstimator(t *testing.T) { d.ScanArgs(t, "ignored-written", &ignoredWritten) admissionStats.statsToIgnore.writeBytes += uint64(ignoredWritten) } - estimator.updateEstimates(l0Metrics, cumLSMIngestedBytes, admissionStats) - wL0lm, iL0lm, ilm := estimator.getModelsAtDone() + if d.HasArg("disk-writes") { + var diskWrites int + d.ScanArgs(t, "disk-writes", &diskWrites) + cumDiskWrites += uint64(diskWrites) + } + estimator.updateEstimates(l0Metrics, cumLSMIngestedBytes, cumDiskWrites, admissionStats) + wL0lm, iL0lm, ilm, wamplm := estimator.getModelsAtDone() require.Equal(t, wL0lm, estimator.atDoneL0WriteTokensLinearModel.smoothedLinearModel) require.Equal(t, iL0lm, estimator.atDoneL0IngestTokensLinearModel.smoothedLinearModel) require.Equal(t, ilm, estimator.atDoneIngestTokensLinearModel.smoothedLinearModel) + require.Equal(t, wamplm, estimator.atDoneWriteAmpLinearModel.smoothedLinearModel) var b strings.Builder fmt.Fprintf(&b, "interval state: %+v\n", estimator.aux) fmt.Fprintf(&b, "at-admission-tokens: %d\n", @@ -109,6 +116,8 @@ func TestStorePerWorkTokenEstimator(t *testing.T) { printLinearModelFitter(&b, estimator.atDoneL0IngestTokensLinearModel) fmt.Fprintf(&b, "ingest-tokens: ") printLinearModelFitter(&b, estimator.atDoneIngestTokensLinearModel) + fmt.Fprintf(&b, "write-amp: ") + printLinearModelFitter(&b, estimator.atDoneWriteAmpLinearModel) return b.String() default: diff --git a/pkg/util/admission/testdata/disk_bandwidth_limiter b/pkg/util/admission/testdata/disk_bandwidth_limiter index 011f295f7afc..971111974055 100644 --- a/pkg/util/admission/testdata/disk_bandwidth_limiter +++ b/pkg/util/admission/testdata/disk_bandwidth_limiter @@ -1,41 +1,18 @@ init ---- -diskBandwidthLimiter low (disk bandwidth: read: 0 B/s, write: 0 B/s, provisioned: 0 B/s, util: 0.00): elastic-frac: 0.00, incoming: 0 B, elastic-tokens (used 0 B): 8.0 EiB +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -# Elastic fraction for the interval is 0.25, and exponential smoothing causes -# it to rise to 0.12. elastic-tokens is 22 since the max logic picks the -# interval incoming bytes and a multiplier of elastic-fr+0.1, which means -# 100*(0.12+0.1)=22 -compute read-bw=100 write-bw=300 provisioned-bw=1000 incoming-bytes=100 regular-tokens-used=30 elastic-tokens-used=10 +# Since token utilization is from the previous estimation loop, we expect that to be +Inf in the first iteration. +compute int-read-bytes=50 int-write-bytes=100 int-provisioned-bytes=1000 regular-tokens-used=60 elastic-tokens-used=20 ---- -diskBandwidthLimiter moderate (disk bandwidth: read: 100 B/s, write: 300 B/s, provisioned: 1000 B/s, util: 0.40): elastic-frac: 0.12, incoming: 50 B, elastic-tokens (used 10 B): 22 B +diskBandwidthLimiter (tokenUtilization +Inf, tokensUsed (elastic 20 B, regular 60 B) tokens (write 850 B (prev 0 B)), writeBW 6 B/s, readBW 3 B/s, provisioned 66 B/s) -# Repeat. No change in elastic tokens since the 22B of elastic tokens were -# underutilized. -compute read-bw=100 write-bw=300 provisioned-bw=1000 incoming-bytes=100 regular-tokens-used=30 elastic-tokens-used=10 +# Utilization is now not Inf. For token calculation, we adjust using the max of smoothed read bytes and the current value. Here we use 80. +compute int-read-bytes=80 int-write-bytes=150 int-provisioned-bytes=800 regular-tokens-used=100 elastic-tokens-used=30 ---- -diskBandwidthLimiter moderate (disk bandwidth: read: 100 B/s, write: 300 B/s, provisioned: 1000 B/s, util: 0.40): elastic-frac: 0.19, incoming: 75 B, elastic-tokens (used 10 B): 22 B +diskBandwidthLimiter (tokenUtilization 0.15, tokensUsed (elastic 30 B, regular 100 B) tokens (write 640 B (prev 850 B)), writeBW 10 B/s, readBW 5 B/s, provisioned 53 B/s) -compute read-bw=100 write-bw=300 provisioned-bw=1000 incoming-bytes=100 regular-tokens-used=30 elastic-tokens-used=21 +# Now we use 65 for read bandwidth adjustment. +compute int-read-bytes=50 int-write-bytes=100 int-provisioned-bytes=800 regular-tokens-used=10 elastic-tokens-used=30 ---- -diskBandwidthLimiter moderate (disk bandwidth: read: 100 B/s, write: 300 B/s, provisioned: 1000 B/s, util: 0.40): elastic-frac: 0.30, incoming: 87 B, elastic-tokens (used 21 B): 39 B - -# No change in elastic tokens because again underutilized. -compute read-bw=100 write-bw=300 provisioned-bw=2000 incoming-bytes=100 regular-tokens-used=30 elastic-tokens-used=21 ----- -diskBandwidthLimiter moderate (disk bandwidth: read: 100 B/s, write: 300 B/s, provisioned: 2.0 KiB/s, util: 0.20): elastic-frac: 0.36, incoming: 93 B, elastic-tokens (used 21 B): 39 B - -# Disk load is low, so unlimited tokens. -compute read-bw=100 write-bw=300 provisioned-bw=2000 incoming-bytes=100 regular-tokens-used=30 elastic-tokens-used=21 ----- -diskBandwidthLimiter low (disk bandwidth: read: 100 B/s, write: 300 B/s, provisioned: 2.0 KiB/s, util: 0.20): elastic-frac: 0.38, incoming: 96 B, elastic-tokens (used 21 B): 8.0 EiB - -# Overloaded. Note that the min logic makes the elastic tokens much lower than 1000*0.5=500. -compute read-bw=100 write-bw=2000 provisioned-bw=2000 incoming-bytes=1000 regular-tokens-used=30 elastic-tokens-used=1000 ----- -diskBandwidthLimiter overload (disk bandwidth: read: 100 B/s, write: 2.0 KiB/s, provisioned: 2.0 KiB/s, util: 1.05): elastic-frac: 0.68, incoming: 548 B, elastic-tokens (used 1000 B): 185 B - -# Disk load is high, so no change to tokens. -compute read-bw=100 write-bw=1600 provisioned-bw=2000 incoming-bytes=1000 regular-tokens-used=30 elastic-tokens-used=1000 ----- -diskBandwidthLimiter high (disk bandwidth: read: 100 B/s, write: 1.6 KiB/s, provisioned: 2.0 KiB/s, util: 0.85): elastic-frac: 0.82, incoming: 774 B, elastic-tokens (used 1000 B): 185 B +diskBandwidthLimiter (tokenUtilization 0.06, tokensUsed (elastic 30 B, regular 10 B) tokens (write 655 B (prev 640 B)), writeBW 6 B/s, readBW 3 B/s, provisioned 53 B/s) diff --git a/pkg/util/admission/testdata/disk_load_watcher b/pkg/util/admission/testdata/disk_load_watcher deleted file mode 100644 index d83bc88d4c42..000000000000 --- a/pkg/util/admission/testdata/disk_load_watcher +++ /dev/null @@ -1,78 +0,0 @@ -init ----- -disk bandwidth: read: 0 B/s, write: 0 B/s, provisioned: 0 B/s, util: 0.00 -load-level: low - -# 40% utilization means moderate load. -interval-info read-bw=10 write-bw=10 provisioned-bw=50 ----- -disk bandwidth: read: 10 B/s, write: 10 B/s, provisioned: 50 B/s, util: 0.40 -load-level: moderate - -# 15% utilization ought to be low load, but we wait until one more round of -# low load before making the change. -interval-info read-bw=10 write-bw=20 provisioned-bw=200 ----- -disk bandwidth: read: 10 B/s, write: 20 B/s, provisioned: 200 B/s, util: 0.15 -load-level: moderate - -# Same 15% utilization, and now it is low load. -interval-info read-bw=10 write-bw=20 provisioned-bw=200 ----- -disk bandwidth: read: 10 B/s, write: 20 B/s, provisioned: 200 B/s, util: 0.15 -load-level: low - -# 95% utilization, so overloaded. -interval-info read-bw=90 write-bw=100 provisioned-bw=200 ----- -disk bandwidth: read: 90 B/s, write: 100 B/s, provisioned: 200 B/s, util: 0.95 -load-level: overload - -# Utilization can exceed 100%. -interval-info read-bw=190 write-bw=100 provisioned-bw=200 ----- -disk bandwidth: read: 190 B/s, write: 100 B/s, provisioned: 200 B/s, util: 1.45 -load-level: overload - -# Utilization exceeds 100% but decreased sufficiently to count as high load. -interval-info read-bw=150 write-bw=100 provisioned-bw=200 ----- -disk bandwidth: read: 150 B/s, write: 100 B/s, provisioned: 200 B/s, util: 1.25 -load-level: high - -# Same utilization, and since did not decrease, it is considered overloaded -# again. -interval-info read-bw=150 write-bw=100 provisioned-bw=200 ----- -disk bandwidth: read: 150 B/s, write: 100 B/s, provisioned: 200 B/s, util: 1.25 -load-level: overload - -# Small decrease and still above 95% so overloaded. -interval-info read-bw=150 write-bw=92 provisioned-bw=200 ----- -disk bandwidth: read: 150 B/s, write: 92 B/s, provisioned: 200 B/s, util: 1.21 -load-level: overload - -# Larger decrease, so even though above 95% it is high load. -interval-info read-bw=150 write-bw=81 provisioned-bw=200 ----- -disk bandwidth: read: 150 B/s, write: 81 B/s, provisioned: 200 B/s, util: 1.16 -load-level: high - -# Utilization < 70% so moderate load. -interval-info read-bw=120 write-bw=19 provisioned-bw=200 ----- -disk bandwidth: read: 120 B/s, write: 19 B/s, provisioned: 200 B/s, util: 0.69 -load-level: moderate - -# 85% utilization, so high load. -interval-info read-bw=70 write-bw=100 provisioned-bw=200 ----- -disk bandwidth: read: 70 B/s, write: 100 B/s, provisioned: 200 B/s, util: 0.85 -load-level: high - -# 68% utilization so considered moderate load. -interval-info read-bw=35 write-bw=100 provisioned-bw=200 ----- -disk bandwidth: read: 35 B/s, write: 100 B/s, provisioned: 200 B/s, util: 0.68 -load-level: moderate diff --git a/pkg/util/admission/testdata/format_adjust_tokens_stats.txt b/pkg/util/admission/testdata/format_adjust_tokens_stats.txt index cdfe7c868ed2..67fb804d3f14 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: -compaction score 0.000 (0 ssts, 0 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 0.00x+0 B) + ingested-model 0.00x+0 B (smoothed 0.00x+0 B) + at-admission-tokens 0 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 1.00); admitting all; elastic-disk-bw tokens 0 B (used 0 B, regular used 0 B): write model 0.00x+0 B ingest model 0.00x+0 B, disk bw read 0 B write 0 B provisioned 0 B; write stalls 12 +compaction score 0.000 (0 ssts, 0 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 0.00x+0 B) + ingested-model 0.00x+0 B (smoothed 0.00x+0 B) + write-amp-model 0.00x+0 B (smoothed 0.00x+0 B) + at-admission-tokens 0 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 1.00); admitting all; write stalls 12 real-numbers: -compaction score 2.700[L0-overload] (195 ssts, 27 sub-levels), L0 growth 577 MiB (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 0.00x+0 B) + ingested-model 0.00x+0 B (smoothed 0.00x+0 B) + at-admission-tokens 0 B, compacted 77 MiB [≈62 MiB], flushed 0 B [≈0 B] (mult 1.00); admitting 116 MiB (rate 7.7 MiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); elastic-disk-bw tokens 0 B (used 0 B, regular used 0 B): write model 0.00x+0 B ingest model 0.00x+0 B, disk bw read 0 B write 0 B provisioned 0 B; write stalls 2 +compaction score 2.700[L0-overload] (195 ssts, 27 sub-levels), L0 growth 577 MiB (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 0.00x+0 B) + ingested-model 0.00x+0 B (smoothed 0.00x+0 B) + write-amp-model 0.00x+0 B (smoothed 0.00x+0 B) + at-admission-tokens 0 B, compacted 77 MiB [≈62 MiB], flushed 0 B [≈0 B] (mult 1.00); admitting 116 MiB (rate 7.7 MiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 2 diff --git a/pkg/util/admission/testdata/granter b/pkg/util/admission/testdata/granter index 0d6cf98803c7..97a2ad92f357 100644 --- a/pkg/util/admission/testdata/granter +++ b/pkg/util/admission/testdata/granter @@ -383,52 +383,52 @@ 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: 153722867280912930(153722867280912930), elastic-disk-bw-tokens-avail: 153722867280912930 +(chain: id: 0 active: false index: 5) io-avail: 153722867280912930(153722867280912930), disk-write-tokens-avail: 153722867280912930 # Set tokens to a large value that permits all request sizes in this file. # Set elastic tokens to a large value that permits all request sizes. -set-tokens io-tokens=100000 elastic-disk-bw-tokens=100000 tick-interval=250 +set-tokens io-tokens=100000 disk-write-tokens=100000 tick-interval=250 ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 100000(100000), elastic-disk-bw-tokens-avail: 100000 +(chain: id: 0 active: false index: 5) io-avail: 100000(100000), disk-write-tokens-avail: 100000 # Initial tokens are effectively unlimited. try-get work=kv v=10000 ---- kv-regular: tryGet(10000) returned true GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 90000(90000), elastic-disk-bw-tokens-avail: 100000 +(chain: id: 0 active: false index: 5) io-avail: 90000(90000), disk-write-tokens-avail: 90000 # Set the io tokens to a smaller value. -set-tokens io-tokens=500 elastic-disk-bw-tokens=100000 tick-interval=250 +set-tokens io-tokens=500 disk-write-tokens=100000 tick-interval=250 ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 500(500), elastic-disk-bw-tokens-avail: 100000 +(chain: id: 0 active: false index: 5) io-avail: 500(500), disk-write-tokens-avail: 100000 -# Subtract 100 tokens for elastic work. Note that elastic-disk-bw-tokens-avail also decreases by 100. +# Subtract 100 tokens for elastic work. Note that disk-write-tokens-avail also decreases by 100. took-without-permission work=kv-elastic v=100 ---- kv-elastic: tookWithoutPermission(100) GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 400(400), elastic-disk-bw-tokens-avail: 99900 +(chain: id: 0 active: false index: 5) io-avail: 400(400), disk-write-tokens-avail: 99900 # Add 200 tokens. return-grant work=kv v=200 ---- kv-regular: returnGrant(200) GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 600(600), elastic-disk-bw-tokens-avail: 99900 +(chain: id: 0 active: false index: 5) io-avail: 600(600), disk-write-tokens-avail: 100100 # Setup waiting requests that want 400 tokens each. set-has-waiting-requests work=kv v=true ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 600(600), elastic-disk-bw-tokens-avail: 99900 +(chain: id: 0 active: false index: 5) io-avail: 600(600), disk-write-tokens-avail: 100100 set-return-value-from-granted work=kv v=400 ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 600(600), elastic-disk-bw-tokens-avail: 99900 +(chain: id: 0 active: false index: 5) io-avail: 600(600), disk-write-tokens-avail: 100100 # Returning tokens triggers granting and 2 requests will be granted until the # tokens become <= 0. @@ -438,25 +438,25 @@ kv-regular: returnGrant(100) kv-regular: granted in chain 0, and returning 400 kv-regular: granted in chain 0, and returning 400 GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: -100(-100), elastic-disk-bw-tokens-avail: 99900 +(chain: id: 0 active: false index: 5) io-avail: -100(-100), disk-write-tokens-avail: 99400 set-return-value-from-granted work=kv v=100 ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: -100(-100), elastic-disk-bw-tokens-avail: 99900 +(chain: id: 0 active: false index: 5) io-avail: -100(-100), disk-write-tokens-avail: 99400 # No tokens to give. try-get work=kv ---- kv-regular: tryGet(1) returned false GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: -100(-100), elastic-disk-bw-tokens-avail: 99900 +(chain: id: 0 active: false index: 5) io-avail: -100(-100), disk-write-tokens-avail: 99400 # Increment by 50 tokens. -set-tokens io-tokens=50 elastic-disk-bw-tokens=99900 tick-interval=250 +set-tokens io-tokens=50 disk-write-tokens=99900 tick-interval=250 ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: -50(-50), elastic-disk-bw-tokens-avail: 99900 +(chain: id: 0 active: false index: 5) io-avail: -50(-50), disk-write-tokens-avail: 99900 # Return another 50 tokens. Since the number of tokens is 0, there is no # grant. @@ -464,7 +464,7 @@ return-grant work=kv v=50 ---- kv-regular: returnGrant(50) GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 0(0), elastic-disk-bw-tokens-avail: 99900 +(chain: id: 0 active: false index: 5) io-avail: 0(0), disk-write-tokens-avail: 99950 # As soon as the tokens > 0, it will grant. return-grant work=kv v=1 @@ -472,19 +472,19 @@ return-grant work=kv v=1 kv-regular: returnGrant(1) kv-regular: granted in chain 0, and returning 100 GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: -99(-99), elastic-disk-bw-tokens-avail: 99900 +(chain: id: 0 active: false index: 5) io-avail: -99(-99), disk-write-tokens-avail: 99851 # Have waiting requests for kv-elastic too. set-has-waiting-requests work=kv-elastic v=true ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: -99(-99), elastic-disk-bw-tokens-avail: 99900 +(chain: id: 0 active: false index: 5) io-avail: -99(-99), disk-write-tokens-avail: 99851 # The kv-elastic waiting requests need 200 tokens each. set-return-value-from-granted work=kv-elastic v=200 ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: -99(-99), elastic-disk-bw-tokens-avail: 99900 +(chain: id: 0 active: false index: 5) io-avail: -99(-99), disk-write-tokens-avail: 99851 # Since there are regular requests waiting, those are granted first. return-grant work=kv-elastic v=400 @@ -495,122 +495,124 @@ kv-regular: granted in chain 0, and returning 100 kv-regular: granted in chain 0, and returning 100 kv-regular: granted in chain 0, and returning 100 GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: -99(-99), elastic-disk-bw-tokens-avail: 100300 +(chain: id: 0 active: false index: 5) io-avail: -99(-99), disk-write-tokens-avail: 99851 # No more regular requests waiting. set-has-waiting-requests work=kv v=false ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: -99(-99), elastic-disk-bw-tokens-avail: 100300 +(chain: id: 0 active: false index: 5) io-avail: -99(-99), disk-write-tokens-avail: 99851 # kv-elastic is granted. -set-tokens io-tokens=100 elastic-disk-bw-tokens=100300 tick-interval=250 +set-tokens io-tokens=100 disk-write-tokens=100300 tick-interval=250 ---- kv-elastic: granted in chain 0, and returning 200 GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: -199(-199), elastic-disk-bw-tokens-avail: 100100 +(chain: id: 0 active: false index: 5) io-avail: -199(-199), disk-write-tokens-avail: 100100 # Nothing is granted. -set-tokens io-tokens=0 elastic-disk-bw-tokens=50 tick-interval=250 +set-tokens io-tokens=0 disk-write-tokens=50 tick-interval=250 ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: -199(-199), elastic-disk-bw-tokens-avail: 50 +(chain: id: 0 active: false index: 5) io-avail: -199(-199), disk-write-tokens-avail: 50 # Both kinds of tokens are decremented and become negative. -set-tokens io-tokens=200 elastic-disk-bw-tokens=50 tick-interval=250 +set-tokens io-tokens=200 disk-write-tokens=50 tick-interval=250 ---- kv-elastic: granted in chain 0, and returning 200 GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: -199(-199), elastic-disk-bw-tokens-avail: -150 +(chain: id: 0 active: false index: 5) io-avail: -199(-199), disk-write-tokens-avail: -150 # IO tokens become positive. But no grant to elastic work since # elastic-disk-bw tokens are negative. -set-tokens io-tokens=300 elastic-disk-bw-tokens=0 tick-interval=250 +set-tokens io-tokens=300 disk-write-tokens=0 tick-interval=250 ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 101(101), elastic-disk-bw-tokens-avail: -150 +(chain: id: 0 active: false index: 5) io-avail: 101(101), disk-write-tokens-avail: -150 # Regular kv work can get tokens. try-get work=kv v=10 ---- kv-regular: tryGet(10) returned true GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 91(91), elastic-disk-bw-tokens-avail: -150 +(chain: id: 0 active: false index: 5) io-avail: 91(91), disk-write-tokens-avail: -160 # Elastic kv work cannot get tokens. try-get work=kv-elastic v=10 ---- kv-elastic: tryGet(10) returned false GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 91(91), elastic-disk-bw-tokens-avail: -150 +(chain: id: 0 active: false index: 5) io-avail: 91(91), disk-write-tokens-avail: -160 -# Still negative. Add elastic-disk-bw-tokens, but don't change io tokens. -set-tokens io-tokens=91 elastic-disk-bw-tokens=50 tick-interval=250 +# Still negative. Add disk-write-tokens, but don't change io tokens. +set-tokens io-tokens=91 disk-write-tokens=50 tick-interval=250 ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 91(91), elastic-disk-bw-tokens-avail: -100 +(chain: id: 0 active: false index: 5) io-avail: 91(91), disk-write-tokens-avail: -110 # Add some io-tokens. -set-tokens io-tokens=400 elastic-disk-bw-tokens=0 tick-interval=250 +set-tokens io-tokens=400 disk-write-tokens=0 tick-interval=250 ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 400(400), elastic-disk-bw-tokens-avail: -100 +(chain: id: 0 active: false index: 5) io-avail: 400(400), disk-write-tokens-avail: -110 # Finally both tokens are positive and we grant until the elastic-disk-bw # tokens become negative. -set-tokens io-tokens=400 elastic-disk-bw-tokens=120 tick-interval=250 +set-tokens io-tokens=400 disk-write-tokens=120 tick-interval=250 ---- kv-elastic: granted in chain 0, and returning 200 GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 200(200), elastic-disk-bw-tokens-avail: -180 +(chain: id: 0 active: false index: 5) io-avail: 200(200), disk-write-tokens-avail: -190 # Note that TestGranterBasic hard-codes the models to be 0.5x+50, so # 0.5*40+50=70. So 70-10=60 additional tokens are needed based on the write # model. We also apply the ingest model so 0.5*0+50=50, which means a total of # 60+50 additional tokens are needed, hence the decrease by 110 of both -# io-avail and elastic-disk-bw-tokens-avail. +# io-avail and disk-write-tokens-avail. store-write-done work=kv-elastic orig-tokens=10 write-bytes=40 ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 90(90), elastic-disk-bw-tokens-avail: -290 +(chain: id: 0 active: false index: 5) io-avail: 90(90), disk-write-tokens-avail: -300 store-write-done work=kv orig-tokens=400 write-bytes=40 ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 370(370), elastic-disk-bw-tokens-avail: -290 +(chain: id: 0 active: false index: 5) io-avail: 370(370), disk-write-tokens-avail: -20 -# Both tokens become positive, since 280 tokens are returned, so one work is granted. +# Both tokens become positive, since 280 tokens are returned, so 2 work items +# are admitted until the tokens become negative. store-write-done work=kv-elastic orig-tokens=400 write-bytes=40 ---- +kv-elastic: granted in chain 0, and returning 200 +kv-elastic: granted in chain 0, and returning 200 GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 650(650), elastic-disk-bw-tokens-avail: -10 +(chain: id: 0 active: false index: 5) io-avail: 250(250), disk-write-tokens-avail: -140 -set-tokens io-tokens=400 elastic-io-tokens=50 elastic-disk-bw-tokens=120 tick-interval=250 +set-tokens io-tokens=400 elastic-io-tokens=50 disk-write-tokens=120 tick-interval=250 ---- -kv-elastic: granted in chain 0, and returning 200 GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 200(-150), elastic-disk-bw-tokens-avail: -90 +(chain: id: 0 active: false index: 5) io-avail: 400(50), disk-write-tokens-avail: -20 set-has-waiting-requests work=kv-elastic v=false ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 200(-150), elastic-disk-bw-tokens-avail: -90 +(chain: id: 0 active: false index: 5) io-avail: 400(50), disk-write-tokens-avail: -20 -set-tokens io-tokens=400 elastic-io-tokens=50 elastic-disk-bw-tokens=120 tick-interval=250 +set-tokens io-tokens=400 elastic-io-tokens=50 disk-write-tokens=120 tick-interval=250 ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 400(-100), elastic-disk-bw-tokens-avail: 30 +(chain: id: 0 active: false index: 5) io-avail: 400(50), disk-write-tokens-avail: 100 -set-tokens io-tokens=400 elastic-io-tokens=101 elastic-disk-bw-tokens=120 tick-interval=250 +set-tokens io-tokens=400 elastic-io-tokens=101 disk-write-tokens=120 tick-interval=250 ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 400(1), elastic-disk-bw-tokens-avail: 120 +(chain: id: 0 active: false index: 5) io-avail: 400(101), disk-write-tokens-avail: 120 try-get work=kv-elastic v=10 ---- kv-elastic: tryGet(10) returned true GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 390(-9), elastic-disk-bw-tokens-avail: 110 +(chain: id: 0 active: false index: 5) io-avail: 390(91), disk-write-tokens-avail: 110 ##################################################################### @@ -624,66 +626,66 @@ GrantCoordinator: init-store-grant-coordinator ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 153722867280912930(153722867280912930), elastic-disk-bw-tokens-avail: 153722867280912930 +(chain: id: 0 active: false index: 5) io-avail: 153722867280912930(153722867280912930), disk-write-tokens-avail: 153722867280912930 # Tokens set to 250 * 10 = 2500. -set-tokens io-tokens=10 elastic-disk-bw-tokens=10 tick-interval=1 +set-tokens io-tokens=10 disk-write-tokens=10 tick-interval=1 ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 2500(2500), elastic-disk-bw-tokens-avail: 2500 +(chain: id: 0 active: false index: 5) io-avail: 2500(2500), disk-write-tokens-avail: 2500 try-get work=kv-elastic v=2490 ---- kv-elastic: tryGet(2490) returned true GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 10(10), elastic-disk-bw-tokens-avail: 10 +(chain: id: 0 active: false index: 5) io-avail: 10(10), disk-write-tokens-avail: 10 # Initial tokens are effectively unlimited. try-get work=kv v=1 ---- kv-regular: tryGet(1) returned true GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 9(9), elastic-disk-bw-tokens-avail: 10 +(chain: id: 0 active: false index: 5) io-avail: 9(9), disk-write-tokens-avail: 9 # Set the io tokens to a smaller value. Note that since the IO tokens can # increase up to 6*250 and 10*250, we expect the tokens to increase to 15, and # 20 respectively. -set-tokens io-tokens=6 elastic-disk-bw-tokens=10 tick-interval=1 +set-tokens io-tokens=6 disk-write-tokens=10 tick-interval=1 ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 15(15), elastic-disk-bw-tokens-avail: 20 +(chain: id: 0 active: false index: 5) io-avail: 15(15), disk-write-tokens-avail: 19 -# Subtract 10 tokens for elastic work. Note that elastic-disk-bw-tokens-avail also decreases by 10. +# Subtract 10 tokens for elastic work. Note that disk-write-tokens-avail also decreases by 10. took-without-permission work=kv-elastic v=10 ---- kv-elastic: tookWithoutPermission(10) GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 5(5), elastic-disk-bw-tokens-avail: 10 +(chain: id: 0 active: false index: 5) io-avail: 5(5), disk-write-tokens-avail: 9 # Add 10 tokens. return-grant work=kv-elastic v=10 ---- kv-elastic: returnGrant(10) GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 15(15), elastic-disk-bw-tokens-avail: 20 +(chain: id: 0 active: false index: 5) io-avail: 15(15), disk-write-tokens-avail: 19 # If io-tokens is 10, we expect the tokens to accumulate upto 2500. So, we call # set-tokens 250 times, and ensure that the tokens are capped at 2500. -set-tokens-loop io-tokens=10 elastic-disk-bw-tokens=10 loop=250 +set-tokens-loop io-tokens=10 disk-write-tokens=10 loop=250 ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 2500(2500), elastic-disk-bw-tokens-avail: 2500 +(chain: id: 0 active: false index: 5) io-avail: 2500(2500), disk-write-tokens-avail: 2500 # Setup waiting requests that want 1300 tokens each. set-has-waiting-requests work=kv-elastic v=true ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 2500(2500), elastic-disk-bw-tokens-avail: 2500 +(chain: id: 0 active: false index: 5) io-avail: 2500(2500), disk-write-tokens-avail: 2500 set-return-value-from-granted work=kv-elastic v=1300 ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 2500(2500), elastic-disk-bw-tokens-avail: 2500 +(chain: id: 0 active: false index: 5) io-avail: 2500(2500), disk-write-tokens-avail: 2500 # Returning tokens triggers granting and 2 requests will be granted until the # tokens become <= 0. @@ -693,37 +695,37 @@ kv-regular: returnGrant(1) kv-elastic: granted in chain 0, and returning 1300 kv-elastic: granted in chain 0, and returning 1300 GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: -99(-99), elastic-disk-bw-tokens-avail: -100 +(chain: id: 0 active: false index: 5) io-avail: -99(-99), disk-write-tokens-avail: -99 # No tokens to give. try-get work=kv ---- kv-regular: tryGet(1) returned false GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: -99(-99), elastic-disk-bw-tokens-avail: -100 +(chain: id: 0 active: false index: 5) io-avail: -99(-99), disk-write-tokens-avail: -99 set-has-waiting-requests work=kv-elastic v=false ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: -99(-99), elastic-disk-bw-tokens-avail: -100 +(chain: id: 0 active: false index: 5) io-avail: -99(-99), disk-write-tokens-avail: -99 # Negative tokens available should be respected on a subsequent call to set-tokens. -set-tokens io-tokens=100 elastic-disk-bw-tokens=0 tick-interval=1 +set-tokens io-tokens=100 disk-write-tokens=0 tick-interval=1 ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 1(1), elastic-disk-bw-tokens-avail: -100 +(chain: id: 0 active: false index: 5) io-avail: 1(1), disk-write-tokens-avail: -99 # No elastic tokens to give. try-get work=kv-elastic ---- kv-elastic: tryGet(1) returned false GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 1(1), elastic-disk-bw-tokens-avail: -100 +(chain: id: 0 active: false index: 5) io-avail: 1(1), disk-write-tokens-avail: -99 # Switch to an unloaded system which ticks at a 250ms rate. With this interval, # we expect the available tokens to be at most 50, 110 respectively. We see the -# io-tokens clamp at 50, and the elastic-disk-bw-tokens increase to 10. -set-tokens io-tokens=50 elastic-disk-bw-tokens=110 tick-interval=250 +# io-tokens clamp at 50, and the disk-write-tokens increase to 10. +set-tokens io-tokens=50 disk-write-tokens=110 tick-interval=250 ---- GrantCoordinator: -(chain: id: 0 active: false index: 5) io-avail: 50(50), elastic-disk-bw-tokens-avail: 10 +(chain: id: 0 active: false index: 5) io-avail: 50(50), disk-write-tokens-avail: 11 diff --git a/pkg/util/admission/testdata/io_load_listener b/pkg/util/admission/testdata/io_load_listener index 108bcc07b1d6..6512c634ad7d 100644 --- a/pkg/util/admission/testdata/io_load_listener +++ b/pkg/util/admission/testdata/io_load_listener @@ -11,8 +11,9 @@ prep-admission-stats admitted=0 # Even though above the threshold, the first 60 ticks don't limit the tokens. set-state l0-bytes=10000 l0-added-write=1000 l0-files=21 l0-sublevels=21 ---- -compaction score 0.000 (21 ssts, 21 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:0 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.000 (21 ssts, 21 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false tick: 1, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false tick: 2, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false @@ -84,10 +85,11 @@ prep-admission-stats admitted=10000 write-bytes=40000 # removed), but smoothing it drops the tokens to 12,500. set-state l0-bytes=10000 l0-added-write=101000 l0-files=21 l0-sublevels=21 ---- -compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 98 KiB (write 98 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10000 (0 bypassed) with 39 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 2.25x+1 B (smoothed 2.00x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 5 B, compacted 98 KiB [≈49 KiB], flushed 1.4 MiB [≈0 B] (mult 1.00); admitting 12 KiB (rate 833 B/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:101000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:101000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:101000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:12500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:12500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:5} l0WriteLM:{multiplier:2 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:2.25 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:100000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 98 KiB (write 98 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10000 (0 bypassed) with 39 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 98 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 2.25x+1 B (smoothed 2.00x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 5 B, compacted 98 KiB [≈49 KiB], flushed 1.4 MiB [≈0 B] (mult 1.00); admitting 12 KiB (rate 833 B/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:101000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:101000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:12500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:12500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:5} l0WriteLM:{multiplier:2 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:2.25 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:100000 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 5 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.00x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.00x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=209(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=209(elastic 1) max-disk-bw-tokens=unlimited lastTick=false tick: 1, setAvailableTokens: io-tokens=209(elastic 0) elastic-disk-bw-tokens=unlimited max-byte-tokens=209(elastic 1) max-disk-bw-tokens=unlimited lastTick=false tick: 2, setAvailableTokens: io-tokens=209(elastic 0) elastic-disk-bw-tokens=unlimited max-byte-tokens=209(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -156,10 +158,11 @@ prep-admission-stats admitted=20000 write-bytes=80000 # Same delta as previous but smoothing bumps up the tokens to 25,000. set-state l0-bytes=10000 l0-added-write=201000 l0-files=21 l0-sublevels=21 ---- -compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 98 KiB (write 98 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10000 (0 bypassed) with 39 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 2.25x+1 B (smoothed 2.12x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 7 B, compacted 98 KiB [≈73 KiB], flushed 1.4 MiB [≈0 B] (mult 1.00); admitting 24 KiB (rate 1.6 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:201000 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:201000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:75000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:7} l0WriteLM:{multiplier:2.125 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:2.25 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:100000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 98 KiB (write 98 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10000 (0 bypassed) with 39 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 98 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 2.25x+1 B (smoothed 2.12x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 7 B, compacted 98 KiB [≈73 KiB], flushed 1.4 MiB [≈0 B] (mult 1.00); admitting 24 KiB (rate 1.6 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:201000 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:75000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:7} l0WriteLM:{multiplier:2.125 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:2.25 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:100000 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 7 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.12x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.12x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=417(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=417(elastic 1) max-disk-bw-tokens=unlimited lastTick=false tick: 1, setAvailableTokens: io-tokens=417(elastic 0) elastic-disk-bw-tokens=unlimited max-byte-tokens=417(elastic 1) max-disk-bw-tokens=unlimited lastTick=false tick: 2, setAvailableTokens: io-tokens=417(elastic 0) elastic-disk-bw-tokens=unlimited max-byte-tokens=417(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -224,10 +227,11 @@ tick: 59, setAvailableTokens: io-tokens=416(elastic 0) elastic-disk-bw-tokens=un # No delta. This used to trigger an overflow bug. set-state l0-bytes=10000 l0-added-write=201000 l0-files=21 l0-sublevels=21 print-only-first-tick=true ---- -compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 2.12x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 7 B, compacted 0 B [≈37 KiB], flushed 0 B [≈0 B] (mult 1.00); admitting 21 KiB (rate 1.4 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:201000 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:201000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:37500 smoothedCompactionByteTokens:21875 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:21875 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:7} l0WriteLM:{multiplier:2.125 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:0 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 2.12x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 7 B, compacted 0 B [≈37 KiB], flushed 0 B [≈0 B] (mult 1.00); admitting 21 KiB (rate 1.4 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:201000 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:37500 smoothedCompactionByteTokens:21875 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:21875 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:7} l0WriteLM:{multiplier:2.125 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 7 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.12x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.12x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=365(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=365(elastic 1) max-disk-bw-tokens=unlimited lastTick=false prep-admission-stats admitted=30000 write-bytes=120000 @@ -238,10 +242,11 @@ prep-admission-stats admitted=30000 write-bytes=120000 # don't limit the tokens. set-state l0-bytes=10000 l0-added-write=501000 l0-files=21 l0-sublevels=9 print-only-first-tick=true ---- -compaction score 0.450 (21 ssts, 9 sub-levels), L0 growth 293 KiB (write 293 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10000 (0 bypassed) with 39 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 3.00x+18 B (smoothed 2.56x+9 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 18 B, compacted 293 KiB [≈165 KiB], flushed 4.3 MiB [≈0 B] (mult 1.00); admitting 110 KiB (rate 7.3 KiB/s) (elastic 62 KiB rate 4.1 KiB/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:5000000000 IdleDuration:500000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:501000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:168750 smoothedCompactionByteTokens:112187.5 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:112187 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:63281 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:18} l0WriteLM:{multiplier:2.5625 constant:9} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:300000 intL0CompactedBytes:300000 intFlushTokens:4.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:300000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:3 constant:18} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:300000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.450 (21 ssts, 9 sub-levels), L0 growth 293 KiB (write 293 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10000 (0 bypassed) with 39 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 293 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 3.00x+18 B (smoothed 2.56x+9 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 18 B, compacted 293 KiB [≈165 KiB], flushed 4.3 MiB [≈0 B] (mult 1.00); admitting 110 KiB (rate 7.3 KiB/s) (elastic 62 KiB rate 4.1 KiB/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:5000000000 IdleDuration:500000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:168750 smoothedCompactionByteTokens:112187.5 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:112187 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:63281 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:18} l0WriteLM:{multiplier:2.5625 constant:9} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:300000 intL0CompactedBytes:300000 intFlushTokens:4.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:300000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:3 constant:18} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:300000 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 18 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.56x+9 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.56x+9 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=1870(elastic 1055) elastic-disk-bw-tokens=unlimited max-byte-tokens=1870(elastic 1055) max-disk-bw-tokens=unlimited lastTick=false prep-admission-stats admitted=40000 write-bytes=160000 @@ -250,10 +255,11 @@ prep-admission-stats admitted=40000 write-bytes=160000 set-state l0-bytes=10000 l0-added-write=501000 l0-files=21 l0-sublevels=6 print-only-first-tick=true ---- -compaction score 0.300 (21 ssts, 6 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10000 (0 bypassed) with 39 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 2.56x+4 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 18 B, compacted 0 B [≈82 KiB], flushed 0 B [≈0 B] (mult 1.00); admitting 129 KiB (rate 8.6 KiB/s) (elastic 62 KiB rate 4.1 KiB/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:6000000000 IdleDuration:600000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:501000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:84375 smoothedCompactionByteTokens:132031.25 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:132031 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:63281 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:18} l0WriteLM:{multiplier:2.5625 constant:4} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:0 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.300 (21 ssts, 6 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10000 (0 bypassed) with 39 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 2.56x+4 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 18 B, compacted 0 B [≈82 KiB], flushed 0 B [≈0 B] (mult 1.00); admitting 129 KiB (rate 8.6 KiB/s) (elastic 62 KiB rate 4.1 KiB/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:6000000000 IdleDuration:600000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:84375 smoothedCompactionByteTokens:132031.25 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:132031 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:63281 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:18} l0WriteLM:{multiplier:2.5625 constant:4} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 18 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.56x+4 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.56x+4 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=2201(elastic 1055) elastic-disk-bw-tokens=unlimited max-byte-tokens=2201(elastic 1055) max-disk-bw-tokens=unlimited lastTick=false prep-admission-stats admitted=50000 write-bytes=200000 @@ -262,10 +268,11 @@ prep-admission-stats admitted=50000 write-bytes=200000 set-state l0-bytes=10000 l0-added-write=501000 l0-files=21 l0-sublevels=3 print-only-first-tick=true ---- -compaction score 0.150 (21 ssts, 3 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10000 (0 bypassed) with 39 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 2.56x+2 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 18 B, compacted 0 B [≈41 KiB], flushed 0 B [≈0 B] (mult 1.00); admitting elastic 46 KiB (rate 3.1 KiB/s) due to L0 growth; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:7000000000 IdleDuration:700000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:501000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:42187 smoothedCompactionByteTokens:66015.625 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:47460 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:18} l0WriteLM:{multiplier:2.5625 constant:2} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:0 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.150 (21 ssts, 3 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10000 (0 bypassed) with 39 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 2.56x+2 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 18 B, compacted 0 B [≈41 KiB], flushed 0 B [≈0 B] (mult 1.00); admitting elastic 46 KiB (rate 3.1 KiB/s) due to L0 growth; write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:7000000000 IdleDuration:700000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:42187 smoothedCompactionByteTokens:66015.625 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:47460 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:18} l0WriteLM:{multiplier:2.5625 constant:2} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 18 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.56x+2 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.56x+2 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=unlimited(elastic 791) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic 791) max-disk-bw-tokens=unlimited lastTick=false @@ -279,8 +286,9 @@ prep-admission-stats admitted=0 set-state l0-bytes=1000 l0-added-write=1000 l0-added-ingested=0 l0-files=21 l0-sublevels=21 print-only-first-tick=true ---- -compaction score 0.000 (21 ssts, 21 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:0 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.000 (21 ssts, 21 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false # L0 will see an addition of 200,000 bytes. 150,000 bytes were mentioned by @@ -296,10 +304,11 @@ prep-admission-stats admitted=10 write-bytes=130000 ingested-bytes=20000 below-r # of 1.12 and 1.25 respectively. set-state l0-bytes=1000 l0-added-write=171000 l0-added-ingested=30000 l0-files=21 l0-sublevels=21 print-only-first-tick=true ---- -compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 195 KiB (write 166 KiB (ignored 0 B) ingest 29 KiB (ignored 0 B)): requests 10 (0 bypassed) with 127 KiB acc-write (0 B bypassed) + 20 KiB acc-ingest (0 B bypassed) + write-model 1.31x+1 B (smoothed 1.53x+1 B) + ingested-model 1.50x+1 B (smoothed 1.12x+1 B) + at-admission-tokens 1 B, compacted 195 KiB [≈98 KiB], flushed 2.4 MiB [≈0 B] (mult 1.00); admitting 24 KiB (rate 1.6 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:171000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:201000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.5288076923076923 constant:1} l0IngestLM:{multiplier:1.125 constant:1} ingestLM:{multiplier:1.2497500000000001 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:200000 intFlushTokens:2.55e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:170000 intL0IngestedBytes:30000 intLSMIngestedBytes:30000 intL0WriteAccountedBytes:130000 intIngestedAccountedBytes:20000 intL0WriteLinearModel:{multiplier:1.3076153846153846 constant:1} intL0IngestedLinearModel:{multiplier:1.4995 constant:1} intIngestedLinearModel:{multiplier:1.4995 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:200000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 195 KiB (write 166 KiB (ignored 0 B) ingest 29 KiB (ignored 0 B)): requests 10 (0 bypassed) with 127 KiB acc-write (0 B bypassed) + 20 KiB acc-ingest (0 B bypassed) + 195 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 1.31x+1 B (smoothed 1.53x+1 B) + ingested-model 1.50x+1 B (smoothed 1.12x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 195 KiB [≈98 KiB], flushed 2.4 MiB [≈0 B] (mult 1.00); admitting 24 KiB (rate 1.6 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:171000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.5288076923076923 constant:1} l0IngestLM:{multiplier:1.125 constant:1} ingestLM:{multiplier:1.2497500000000001 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:200000 intFlushTokens:2.55e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:170000 intL0IngestedBytes:30000 intLSMIngestedBytes:30000 intL0WriteAccountedBytes:130000 intIngestedAccountedBytes:20000 intL0WriteLinearModel:{multiplier:1.3076153846153846 constant:1} intL0IngestedLinearModel:{multiplier:1.4995 constant:1} intIngestedLinearModel:{multiplier:1.4995 constant:1} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:200000 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.53x+1 l0-ingest-lm: 1.12x+1 ingest-lm: 1.25x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.53x+1 l0-ingest-lm: 1.12x+1 ingest-lm: 1.25x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=417(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=417(elastic 1) max-disk-bw-tokens=unlimited lastTick=false # L0 will see an addition of 20,000 bytes, all of which are accounted for. @@ -311,10 +320,11 @@ prep-admission-stats admitted=20 write-bytes=150000 ingested-bytes=20000 set-state l0-bytes=1000 l0-added-write=191000 l0-added-ingested=30000 l0-files=21 l0-sublevels=21 print-only-first-tick=true ---- -compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 20 KiB (write 20 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10 (0 bypassed) with 20 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 1.00x+1 B (smoothed 1.26x+1 B) + ingested-model 0.00x+0 B (smoothed 1.12x+1 B) + at-admission-tokens 4.1 KiB, compacted 20 KiB [≈59 KiB], flushed 293 KiB [≈0 B] (mult 1.00); admitting 27 KiB (rate 1.8 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:221000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:191000 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:221000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:60000 smoothedCompactionByteTokens:27500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:27500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:4195} l0WriteLM:{multiplier:1.2641538461538462 constant:1} l0IngestLM:{multiplier:1.125 constant:1} ingestLM:{multiplier:1.2497500000000001 constant:1} aux:{intL0AddedBytes:20000 intL0CompactedBytes:20000 intFlushTokens:300000 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:20000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:20000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0.9995 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:20000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 20 KiB (write 20 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10 (0 bypassed) with 20 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 20 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 1.00x+1 B (smoothed 1.26x+1 B) + ingested-model 0.00x+0 B (smoothed 1.12x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 4.1 KiB, compacted 20 KiB [≈59 KiB], flushed 293 KiB [≈0 B] (mult 1.00); admitting 27 KiB (rate 1.8 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:221000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:191000 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:60000 smoothedCompactionByteTokens:27500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:27500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:4195} l0WriteLM:{multiplier:1.2641538461538462 constant:1} l0IngestLM:{multiplier:1.125 constant:1} ingestLM:{multiplier:1.2497500000000001 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:20000 intL0CompactedBytes:20000 intFlushTokens:300000 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:20000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:20000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0.9995 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:20000 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 4195 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.26x+1 l0-ingest-lm: 1.12x+1 ingest-lm: 1.25x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.26x+1 l0-ingest-lm: 1.12x+1 ingest-lm: 1.25x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=459(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=459(elastic 1) max-disk-bw-tokens=unlimited lastTick=false # L0 will see an addition of 20,000 bytes, but we think we have added 100,000 @@ -325,10 +335,11 @@ prep-admission-stats admitted=30 write-bytes=250000 ingested-bytes=20000 ingeste set-state l0-bytes=1000 l0-added-write=211000 l0-added-ingested=30000 l0-files=21 l0-sublevels=21 print-only-first-tick=true ---- -compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 20 KiB (write 20 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10 (0 bypassed) with 98 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.50x+1 B (smoothed 0.88x+1 B) + ingested-model 0.00x+0 B (smoothed 1.12x+1 B) + at-admission-tokens 3.0 KiB, compacted 20 KiB [≈39 KiB], flushed 293 KiB [≈0 B] (mult 1.00); admitting 23 KiB (rate 1.5 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:241000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:211000 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:241000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:40000 smoothedCompactionByteTokens:23750 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:23750 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:3097} l0WriteLM:{multiplier:0.8820769230769231 constant:1} l0IngestLM:{multiplier:1.125 constant:1} ingestLM:{multiplier:1.2497500000000001 constant:1} aux:{intL0AddedBytes:20000 intL0CompactedBytes:20000 intFlushTokens:300000 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:20000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:100000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0.5 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:20000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 20 KiB (write 20 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10 (0 bypassed) with 98 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 20 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.50x+1 B (smoothed 0.88x+1 B) + ingested-model 0.00x+0 B (smoothed 1.12x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 3.0 KiB, compacted 20 KiB [≈39 KiB], flushed 293 KiB [≈0 B] (mult 1.00); admitting 23 KiB (rate 1.5 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:241000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:211000 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:40000 smoothedCompactionByteTokens:23750 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:23750 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:3097} l0WriteLM:{multiplier:0.8820769230769231 constant:1} l0IngestLM:{multiplier:1.125 constant:1} ingestLM:{multiplier:1.2497500000000001 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:20000 intL0CompactedBytes:20000 intFlushTokens:300000 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:20000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:100000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0.5 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:20000 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 3097 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 0.88x+1 l0-ingest-lm: 1.12x+1 ingest-lm: 1.25x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 0.88x+1 l0-ingest-lm: 1.12x+1 ingest-lm: 1.25x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=396(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=396(elastic 1) max-disk-bw-tokens=unlimited lastTick=false # Test case with WAL failover. @@ -341,8 +352,9 @@ prep-admission-stats admitted=0 set-state l0-bytes=1000 l0-added-write=1000 l0-added-ingested=0 l0-files=21 l0-sublevels=21 print-only-first-tick=true ---- -compaction score 0.000 (21 ssts, 21 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:0 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.000 (21 ssts, 21 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false # L0 will see an addition of 200,000 bytes. All were mentioned in the admitted requests. @@ -352,10 +364,11 @@ prep-admission-stats admitted=10 write-bytes=200000 set-state l0-bytes=1000 l0-added-write=201000 l0-files=21 l0-sublevels=21 print-only-first-tick=true ---- -compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 195 KiB (write 195 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10 (0 bypassed) with 195 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 1.00x+1 B (smoothed 1.37x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 9.8 KiB, compacted 195 KiB [≈98 KiB], flushed 2.9 MiB [≈0 B] (mult 1.00); admitting 24 KiB (rate 1.6 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:201000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:201000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:10000} l0WriteLM:{multiplier:1.374975 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:200000 intFlushTokens:3e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:200000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:200000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0.99995 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:200000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 195 KiB (write 195 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10 (0 bypassed) with 195 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 195 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 1.00x+1 B (smoothed 1.37x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 9.8 KiB, compacted 195 KiB [≈98 KiB], flushed 2.9 MiB [≈0 B] (mult 1.00); admitting 24 KiB (rate 1.6 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:201000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:10000} l0WriteLM:{multiplier:1.374975 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:200000 intFlushTokens:3e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:200000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:200000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0.99995 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:200000 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 10000 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.37x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.37x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=417(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=417(elastic 1) max-disk-bw-tokens=unlimited lastTick=false prep-admission-stats admitted=10 write-bytes=200000 @@ -365,10 +378,11 @@ prep-admission-stats admitted=10 write-bytes=200000 # Even though compactions out of L0 did not happen, the smoothed compacted bytes stayed the same (98KiB) and so did the admission tokens (24KiB). set-state l0-bytes=201000 l0-added-write=401000 l0-files=41 l0-sublevels=41 print-only-first-tick=true wal-secondary-write-sec=1 ---- -compaction score 2.050[L0-overload] (41 ssts, 41 sub-levels), L0 growth 195 KiB (write 195 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.37x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 9.8 KiB, compacted 0 B [≈98 KiB], flushed 2.9 MiB [≈0 B] (mult 1.00); admitting (WAL failover) 24 KiB (rate 1.6 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:401000 curL0Bytes:201000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:401000 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:401000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:1000000000 smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:10000} l0WriteLM:{multiplier:1.374975 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:0 intFlushTokens:3e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:true prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:200000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:200000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 2.050[L0-overload] (41 ssts, 41 sub-levels), L0 growth 195 KiB (write 195 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 195 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.37x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 9.8 KiB, compacted 0 B [≈98 KiB], flushed 2.9 MiB [≈0 B] (mult 1.00); admitting (WAL failover) 24 KiB (rate 1.6 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:401000 curL0Bytes:201000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:401000 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:1000000000 smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:10000} l0WriteLM:{multiplier:1.374975 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:0 intFlushTokens:3e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:true prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:200000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:200000 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 10000 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.37x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.37x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=417(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=417(elastic 1) max-disk-bw-tokens=unlimited lastTick=false # Set admission.wal.failover.unlimited_tokens.enabled=true @@ -382,10 +396,11 @@ prep-admission-stats admitted=10 write-bytes=200000 # Even though compactions out of L0 did not happen, the smoothed compacted bytes stayed the same (98KiB). The admission tokens are unlimited due to the cluster setting. set-state l0-bytes=401000 l0-added-write=601000 l0-files=61 l0-sublevels=61 print-only-first-tick=true wal-secondary-write-sec=1 ---- -compaction score 3.050[L0-overload] (61 ssts, 61 sub-levels), L0 growth 195 KiB (write 195 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.37x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 9.8 KiB, compacted 0 B [≈98 KiB], flushed 2.9 MiB [≈0 B] (mult 1.00); admitting (WAL failover) all; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:601000 curL0Bytes:401000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:601000 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:601000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:2000000000 smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:10000} l0WriteLM:{multiplier:1.374975 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:0 intFlushTokens:3e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:true prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:200000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:200000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 3.050[L0-overload] (61 ssts, 61 sub-levels), L0 growth 195 KiB (write 195 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 195 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.37x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 9.8 KiB, compacted 0 B [≈98 KiB], flushed 2.9 MiB [≈0 B] (mult 1.00); admitting (WAL failover) all; write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:601000 curL0Bytes:401000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:601000 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:2000000000 smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:10000} l0WriteLM:{multiplier:1.374975 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:0 intFlushTokens:3e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:true prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:200000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:200000 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 10000 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.37x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.37x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false set-unlimited-wal-failover-tokens enabled=false @@ -404,10 +419,11 @@ prep-admission-stats admitted=10 write-bytes=200000 # secondary. set-state l0-bytes=601000 l0-added-write=801000 l0-files=61 l0-sublevels=5 print-only-first-tick=true wal-secondary-write-sec=1 flush-bytes=1000 flush-work-sec=8 flush-idle-sec=10 write-stall-count=4 ---- -compaction score 0.250 (61 ssts, 5 sub-levels), L0 growth 195 KiB (write 195 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.37x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 9.8 KiB, compacted 0 B [≈98 KiB], flushed 366 KiB [≈0 B] (mult 1.00); admitting (WAL failover) 24 KiB (rate 1.6 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 4 -{ioLoadListenerState:{cumL0AddedBytes:801000 curL0Bytes:601000 cumWriteStallCount:4 cumFlushWriteThroughput:{Bytes:801000 WorkDuration:12000000000 IdleDuration:410000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:801000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:3000000000 smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:10000} l0WriteLM:{multiplier:1.374975 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:0 intFlushTokens:375000 intFlushUtilization:0.4444444444444444 intWriteStalls:4 intWALFailover:true prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:200000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:200000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.250 (61 ssts, 5 sub-levels), L0 growth 195 KiB (write 195 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 195 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.37x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 9.8 KiB, compacted 0 B [≈98 KiB], flushed 366 KiB [≈0 B] (mult 1.00); admitting (WAL failover) 24 KiB (rate 1.6 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 4 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:801000 curL0Bytes:601000 cumWriteStallCount:4 cumFlushWriteThroughput:{Bytes:801000 WorkDuration:12000000000 IdleDuration:410000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:3000000000 smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:10000} l0WriteLM:{multiplier:1.374975 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:0 intFlushTokens:375000 intFlushUtilization:0.4444444444444444 intWriteStalls:4 intWALFailover:true prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:200000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:200000 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 10000 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.37x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.37x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=417(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=417(elastic 1) max-disk-bw-tokens=unlimited lastTick=false # Test case with flush tokens. @@ -420,27 +436,30 @@ prep-admission-stats admitted=0 set-state l0-bytes=10000 l0-added-write=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 (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:0 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.000 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false # 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-write=2000 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 1000 B (write 1000 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 1000 B [≈500 B], flushed 7.3 KiB [≈0 B] (mult 1.00); admitting all; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:2000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:2000 WorkDuration:3000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:2000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:500 smoothedCompactionByteTokens:500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:1000 intL0CompactedBytes:1000 intFlushTokens:7500 intFlushUtilization:0.0196078431372549 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:1000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:1000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 1000 B (write 1000 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 1000 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 1000 B [≈500 B], flushed 7.3 KiB [≈0 B] (mult 1.00); admitting all; write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:2000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:2000 WorkDuration:3000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:500 smoothedCompactionByteTokens:500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:1000 intL0CompactedBytes:1000 intFlushTokens:7500 intFlushUtilization:0.0196078431372549 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:1000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:1000 intAdjustedWriteBytes:0} doLogFlush:false} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false # Flush loop utilization is high enough, so we compute flush tokens for limiting admission. set-state l0-bytes=10000 l0-added-write=3000 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 1000 B (write 1000 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 1000 B [≈750 B], flushed 7.3 KiB [≈7.3 KiB] (mult 1.00); admitting 7.3 KiB (rate 500 B/s) (elastic 5.9 KiB rate 400 B/s) due to memtable flush (multiplier 1.000) (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:3000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:3000 WorkDuration:5000000000 IdleDuration:210000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:3000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:750 smoothedCompactionByteTokens:750 smoothedNumFlushTokens:7500 flushUtilTargetFraction:1 totalNumByteTokens:7500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:6000 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:1000 intL0CompactedBytes:1000 intFlushTokens:7500 intFlushUtilization:0.16666666666666666 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:1000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:1000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 1000 B (write 1000 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 1000 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 1000 B [≈750 B], flushed 7.3 KiB [≈7.3 KiB] (mult 1.00); admitting 7.3 KiB (rate 500 B/s) (elastic 5.9 KiB rate 400 B/s) due to memtable flush (multiplier 1.000) (used total: 0 B elastic 0 B); write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:3000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:3000 WorkDuration:5000000000 IdleDuration:210000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:750 smoothedCompactionByteTokens:750 smoothedNumFlushTokens:7500 flushUtilTargetFraction:1 totalNumByteTokens:7500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:6000 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:1000 intL0CompactedBytes:1000 intFlushTokens:7500 intFlushUtilization:0.16666666666666666 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:1000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:1000 intAdjustedWriteBytes:0} doLogFlush:false} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=125(elastic 100) elastic-disk-bw-tokens=unlimited max-byte-tokens=125(elastic 100) max-disk-bw-tokens=unlimited lastTick=false # Write stalls are happening, so decrease the flush utilization target @@ -448,10 +467,11 @@ setAvailableTokens: io-tokens=125(elastic 100) elastic-disk-bw-tokens=unlimited # now we flushed 10x the bytes, so the overall tokens increase. set-state l0-bytes=10000 l0-added-write=13000 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 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈5.2 KiB], flushed 73 KiB [≈40 KiB] (mult 0.97); admitting 39 KiB (rate 2.6 KiB/s) (elastic 31 KiB rate 2.1 KiB/s) due to memtable flush (multiplier 0.975) (used total: 0 B elastic 0 B); write stalls 1 -{ioLoadListenerState:{cumL0AddedBytes:13000 curL0Bytes:10000 cumWriteStallCount:1 cumFlushWriteThroughput:{Bytes:13000 WorkDuration:7000000000 IdleDuration:220000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:13000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:5375 smoothedCompactionByteTokens:5375 smoothedNumFlushTokens:41250 flushUtilTargetFraction:0.975 totalNumByteTokens:40218 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:32174 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:10000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈5.2 KiB], flushed 73 KiB [≈40 KiB] (mult 0.97); admitting 39 KiB (rate 2.6 KiB/s) (elastic 31 KiB rate 2.1 KiB/s) due to memtable flush (multiplier 0.975) (used total: 0 B elastic 0 B); write stalls 1 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:13000 curL0Bytes:10000 cumWriteStallCount:1 cumFlushWriteThroughput:{Bytes:13000 WorkDuration:7000000000 IdleDuration:220000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:5375 smoothedCompactionByteTokens:5375 smoothedNumFlushTokens:41250 flushUtilTargetFraction:0.975 totalNumByteTokens:40218 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:32174 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=671(elastic 537) elastic-disk-bw-tokens=unlimited max-byte-tokens=671(elastic 537) max-disk-bw-tokens=unlimited lastTick=false # Two write stalls happened, so decrease the flush utilization target fraction @@ -459,29 +479,32 @@ setAvailableTokens: io-tokens=671(elastic 537) elastic-disk-bw-tokens=unlimited # increasing, the overall flush tokens continue to increase. set-state l0-bytes=10000 l0-added-write=23000 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 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈7.5 KiB], flushed 73 KiB [≈57 KiB] (mult 0.92); admitting 52 KiB (rate 3.5 KiB/s) (elastic 42 KiB rate 2.8 KiB/s) due to memtable flush (multiplier 0.925) (used total: 0 B elastic 0 B); write stalls 2 -{ioLoadListenerState:{cumL0AddedBytes:23000 curL0Bytes:10000 cumWriteStallCount:3 cumFlushWriteThroughput:{Bytes:23000 WorkDuration:9000000000 IdleDuration:230000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:23000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:7687 smoothedCompactionByteTokens:7687.5 smoothedNumFlushTokens:58125 flushUtilTargetFraction:0.9249999999999999 totalNumByteTokens:53765 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:43012 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:2 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:10000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈7.5 KiB], flushed 73 KiB [≈57 KiB] (mult 0.92); admitting 52 KiB (rate 3.5 KiB/s) (elastic 42 KiB rate 2.8 KiB/s) due to memtable flush (multiplier 0.925) (used total: 0 B elastic 0 B); write stalls 2 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:23000 curL0Bytes:10000 cumWriteStallCount:3 cumFlushWriteThroughput:{Bytes:23000 WorkDuration:9000000000 IdleDuration:230000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:7687 smoothedCompactionByteTokens:7687.5 smoothedNumFlushTokens:58125 flushUtilTargetFraction:0.9249999999999999 totalNumByteTokens:53765 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:43012 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:2 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=897(elastic 717) elastic-disk-bw-tokens=unlimited max-byte-tokens=897(elastic 717) max-disk-bw-tokens=unlimited lastTick=false # Five more write stalls, so the the flush utilization target fraction is # decreased to 0.85. The smoothed peak flush rate continues to increase. set-state l0-bytes=10000 l0-added-write=33000 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 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈8.6 KiB], flushed 73 KiB [≈65 KiB] (mult 0.85); admitting 55 KiB (rate 3.7 KiB/s) (elastic 44 KiB rate 2.9 KiB/s) due to memtable flush (multiplier 0.850) (used total: 0 B elastic 0 B); write stalls 5 -{ioLoadListenerState:{cumL0AddedBytes:33000 curL0Bytes:10000 cumWriteStallCount:8 cumFlushWriteThroughput:{Bytes:33000 WorkDuration:11000000000 IdleDuration:240000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:33000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:8843 smoothedCompactionByteTokens:8843.75 smoothedNumFlushTokens:66562.5 flushUtilTargetFraction:0.8499999999999999 totalNumByteTokens:56578 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:45262 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:5 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:10000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈8.6 KiB], flushed 73 KiB [≈65 KiB] (mult 0.85); admitting 55 KiB (rate 3.7 KiB/s) (elastic 44 KiB rate 2.9 KiB/s) due to memtable flush (multiplier 0.850) (used total: 0 B elastic 0 B); write stalls 5 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:33000 curL0Bytes:10000 cumWriteStallCount:8 cumFlushWriteThroughput:{Bytes:33000 WorkDuration:11000000000 IdleDuration:240000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:8843 smoothedCompactionByteTokens:8843.75 smoothedNumFlushTokens:66562.5 flushUtilTargetFraction:0.8499999999999999 totalNumByteTokens:56578 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:45262 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:5 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=943(elastic 755) elastic-disk-bw-tokens=unlimited max-byte-tokens=943(elastic 755) max-disk-bw-tokens=unlimited lastTick=false # Another write stall, and the flush utilization target fraction drops to 0.825. set-state l0-bytes=10000 l0-added-write=43000 l0-files=1 l0-sublevels=2 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=9 print-only-first-tick=true ---- -compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.2 KiB], flushed 73 KiB [≈69 KiB] (mult 0.82); admitting 57 KiB (rate 3.8 KiB/s) (elastic 12 KiB rate 785 B/s) due to memtable flush (multiplier 0.825) (used total: 0 B elastic 0 B); write stalls 1 -{ioLoadListenerState:{cumL0AddedBytes:43000 curL0Bytes:10000 cumWriteStallCount:9 cumFlushWriteThroughput:{Bytes:43000 WorkDuration:13000000000 IdleDuration:250000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:43000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9421 smoothedCompactionByteTokens:9421.875 smoothedNumFlushTokens:70781.25 flushUtilTargetFraction:0.8249999999999998 totalNumByteTokens:58394 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:11776 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:10000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.2 KiB], flushed 73 KiB [≈69 KiB] (mult 0.82); admitting 57 KiB (rate 3.8 KiB/s) (elastic 12 KiB rate 785 B/s) due to memtable flush (multiplier 0.825) (used total: 0 B elastic 0 B); write stalls 1 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:43000 curL0Bytes:10000 cumWriteStallCount:9 cumFlushWriteThroughput:{Bytes:43000 WorkDuration:13000000000 IdleDuration:250000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9421 smoothedCompactionByteTokens:9421.875 smoothedNumFlushTokens:70781.25 flushUtilTargetFraction:0.8249999999999998 totalNumByteTokens:58394 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:11776 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=974(elastic 197) elastic-disk-bw-tokens=unlimited max-byte-tokens=974(elastic 197) max-disk-bw-tokens=unlimited lastTick=false # Set a lower bound of 1.3 on the flush utilization target fraction. @@ -492,20 +515,22 @@ set-min-flush-util percent=130 # to 1.3, which is also the lower bound. set-state l0-bytes=10000 l0-added-write=53000 l0-files=1 l0-sublevels=2 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=10 print-only-first-tick=true ---- -compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.5 KiB], flushed 73 KiB [≈71 KiB] (mult 1.30); admitting 92 KiB (rate 6.2 KiB/s) (elastic 12 KiB rate 809 B/s) due to memtable flush (multiplier 1.300) (used total: 0 B elastic 0 B); write stalls 1 -{ioLoadListenerState:{cumL0AddedBytes:53000 curL0Bytes:10000 cumWriteStallCount:10 cumFlushWriteThroughput:{Bytes:53000 WorkDuration:15000000000 IdleDuration:260000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:53000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9710 smoothedCompactionByteTokens:9710.9375 smoothedNumFlushTokens:72890.625 flushUtilTargetFraction:1.3 totalNumByteTokens:94757 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12137 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:10000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.5 KiB], flushed 73 KiB [≈71 KiB] (mult 1.30); admitting 92 KiB (rate 6.2 KiB/s) (elastic 12 KiB rate 809 B/s) due to memtable flush (multiplier 1.300) (used total: 0 B elastic 0 B); write stalls 1 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:53000 curL0Bytes:10000 cumWriteStallCount:10 cumFlushWriteThroughput:{Bytes:53000 WorkDuration:15000000000 IdleDuration:260000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9710 smoothedCompactionByteTokens:9710.9375 smoothedNumFlushTokens:72890.625 flushUtilTargetFraction:1.3 totalNumByteTokens:94757 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12137 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=1580(elastic 203) elastic-disk-bw-tokens=unlimited max-byte-tokens=1580(elastic 203) max-disk-bw-tokens=unlimited lastTick=false # 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-write=63000 l0-files=1 l0-sublevels=2 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=11 print-only-first-tick=true ---- -compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.6 KiB], flushed 73 KiB [≈72 KiB] (mult 1.30); admitting 94 KiB (rate 6.3 KiB/s) (elastic 12 KiB rate 821 B/s) due to memtable flush (multiplier 1.300) (used total: 0 B elastic 0 B); write stalls 1 -{ioLoadListenerState:{cumL0AddedBytes:63000 curL0Bytes:10000 cumWriteStallCount:11 cumFlushWriteThroughput:{Bytes:63000 WorkDuration:17000000000 IdleDuration:270000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:63000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9855 smoothedCompactionByteTokens:9855.46875 smoothedNumFlushTokens:73945.3125 flushUtilTargetFraction:1.3 totalNumByteTokens:96128 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12318 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:10000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.6 KiB], flushed 73 KiB [≈72 KiB] (mult 1.30); admitting 94 KiB (rate 6.3 KiB/s) (elastic 12 KiB rate 821 B/s) due to memtable flush (multiplier 1.300) (used total: 0 B elastic 0 B); write stalls 1 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:63000 curL0Bytes:10000 cumWriteStallCount:11 cumFlushWriteThroughput:{Bytes:63000 WorkDuration:17000000000 IdleDuration:270000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9855 smoothedCompactionByteTokens:9855.46875 smoothedNumFlushTokens:73945.3125 flushUtilTargetFraction:1.3 totalNumByteTokens:96128 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12318 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=1603(elastic 206) elastic-disk-bw-tokens=unlimited max-byte-tokens=1603(elastic 206) max-disk-bw-tokens=unlimited lastTick=false # Bump up the lower bound to 1.35, which is greater than the current flush @@ -517,115 +542,134 @@ set-min-flush-util percent=135 # increases to the new lower bound. set-state l0-bytes=10000 l0-added-write=73000 l0-files=1 l0-sublevels=2 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=12 print-only-first-tick=true ---- -compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.7 KiB], flushed 73 KiB [≈73 KiB] (mult 1.35); admitting 98 KiB (rate 6.5 KiB/s) (elastic 12 KiB rate 827 B/s) due to memtable flush (multiplier 1.350) (used total: 0 B elastic 0 B); write stalls 1 -{ioLoadListenerState:{cumL0AddedBytes:73000 curL0Bytes:10000 cumWriteStallCount:12 cumFlushWriteThroughput:{Bytes:73000 WorkDuration:19000000000 IdleDuration:280000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:73000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9927 smoothedCompactionByteTokens:9927.734375 smoothedNumFlushTokens:74472.65625 flushUtilTargetFraction:1.35 totalNumByteTokens:100538 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12408 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:10000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.7 KiB], flushed 73 KiB [≈73 KiB] (mult 1.35); admitting 98 KiB (rate 6.5 KiB/s) (elastic 12 KiB rate 827 B/s) due to memtable flush (multiplier 1.350) (used total: 0 B elastic 0 B); write stalls 1 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:73000 curL0Bytes:10000 cumWriteStallCount:12 cumFlushWriteThroughput:{Bytes:73000 WorkDuration:19000000000 IdleDuration:280000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9927 smoothedCompactionByteTokens:9927.734375 smoothedNumFlushTokens:74472.65625 flushUtilTargetFraction:1.35 totalNumByteTokens:100538 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12408 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=1676(elastic 207) elastic-disk-bw-tokens=unlimited max-byte-tokens=1676(elastic 207) max-disk-bw-tokens=unlimited lastTick=false # The flush utilization is too low, so there is no limit on flush tokens. set-state l0-bytes=10000 l0-added-write=83000 l0-files=1 l0-sublevels=2 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=100 write-stall-count=13 print-only-first-tick=true ---- -compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.7 KiB], flushed 73 KiB [≈73 KiB] (mult 1.35); admitting elastic 12 KiB (rate 830 B/s) due to L0 growth; write stalls 1 -{ioLoadListenerState:{cumL0AddedBytes:83000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:83000 WorkDuration:21000000000 IdleDuration:380000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:83000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9963 smoothedCompactionByteTokens:9963.8671875 smoothedNumFlushTokens:74472.65625 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12453 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.0196078431372549 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:10000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.7 KiB], flushed 73 KiB [≈73 KiB] (mult 1.35); admitting elastic 12 KiB (rate 830 B/s) due to L0 growth; write stalls 1 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:83000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:83000 WorkDuration:21000000000 IdleDuration:380000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9963 smoothedCompactionByteTokens:9963.8671875 smoothedNumFlushTokens:74472.65625 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12453 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.0196078431372549 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=unlimited(elastic 208) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic 208) max-disk-bw-tokens=unlimited lastTick=false # Flush utilization is high enough, so flush tokens are again limited. set-state l0-bytes=10000 l0-added-write=93000 l0-files=1 l0-sublevels=2 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=13 print-only-first-tick=true ---- -compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.7 KiB], flushed 73 KiB [≈73 KiB] (mult 1.35); admitting 98 KiB (rate 6.6 KiB/s) (elastic 12 KiB rate 831 B/s) due to memtable flush (multiplier 1.350) (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:93000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:93000 WorkDuration:23000000000 IdleDuration:390000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:93000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9981 smoothedCompactionByteTokens:9981.93359375 smoothedNumFlushTokens:74736.328125 flushUtilTargetFraction:1.35 totalNumByteTokens:100894 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12476 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:10000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.7 KiB], flushed 73 KiB [≈73 KiB] (mult 1.35); admitting 98 KiB (rate 6.6 KiB/s) (elastic 12 KiB rate 831 B/s) due to memtable flush (multiplier 1.350) (used total: 0 B elastic 0 B); write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:93000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:93000 WorkDuration:23000000000 IdleDuration:390000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9981 smoothedCompactionByteTokens:9981.93359375 smoothedNumFlushTokens:74736.328125 flushUtilTargetFraction:1.35 totalNumByteTokens:100894 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12476 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=1682(elastic 208) elastic-disk-bw-tokens=unlimited max-byte-tokens=1682(elastic 208) max-disk-bw-tokens=unlimited lastTick=false # No write stalls, and token utilization is high, which will have an effect # in the next pebbleMetricsTick. set-state l0-bytes=10000 l0-added-write=103000 l0-files=1 l0-sublevels=2 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.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.8 KiB], flushed 73 KiB [≈73 KiB] (mult 1.35); admitting 99 KiB (rate 6.6 KiB/s) (elastic 12 KiB rate 832 B/s) due to memtable flush (multiplier 1.350) (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:103000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:103000 WorkDuration:25000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:103000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9990 smoothedCompactionByteTokens:9990.966796875 smoothedNumFlushTokens:74868.1640625 flushUtilTargetFraction:1.35 totalNumByteTokens:101072 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12487 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:10000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.8 KiB], flushed 73 KiB [≈73 KiB] (mult 1.35); admitting 99 KiB (rate 6.6 KiB/s) (elastic 12 KiB rate 832 B/s) due to memtable flush (multiplier 1.350) (used total: 0 B elastic 0 B); write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:103000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:103000 WorkDuration:25000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9990 smoothedCompactionByteTokens:9990.966796875 smoothedNumFlushTokens:74868.1640625 flushUtilTargetFraction:1.35 totalNumByteTokens:101072 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12487 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=1685(elastic 209) elastic-disk-bw-tokens=unlimited max-byte-tokens=1685(elastic 209) max-disk-bw-tokens=unlimited lastTick=false # 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-write=113000 l0-files=1 l0-sublevels=2 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.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.8 KiB], flushed 73 KiB [≈73 KiB] (mult 1.38); admitting 101 KiB (rate 6.7 KiB/s) (elastic 12 KiB rate 832 B/s) due to memtable flush (multiplier 1.375) (used total: 197 KiB elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:113000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:113000 WorkDuration:27000000000 IdleDuration:410000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:113000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9995 smoothedCompactionByteTokens:9995.4833984375 smoothedNumFlushTokens:74934.08203125 flushUtilTargetFraction:1.375 totalNumByteTokens:103034 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12493 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 intWALFailover:false prevTokensUsed:202144 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:10000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.8 KiB], flushed 73 KiB [≈73 KiB] (mult 1.38); admitting 101 KiB (rate 6.7 KiB/s) (elastic 12 KiB rate 832 B/s) due to memtable flush (multiplier 1.375) (used total: 197 KiB elastic 0 B); write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:113000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:113000 WorkDuration:27000000000 IdleDuration:410000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9995 smoothedCompactionByteTokens:9995.4833984375 smoothedNumFlushTokens:74934.08203125 flushUtilTargetFraction:1.375 totalNumByteTokens:103034 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12493 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 intWALFailover:false prevTokensUsed:202144 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=1718(elastic 209) elastic-disk-bw-tokens=unlimited max-byte-tokens=1718(elastic 209) max-disk-bw-tokens=unlimited lastTick=false # 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-write=123000 l0-files=1 l0-sublevels=2 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.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.8 KiB], flushed 73 KiB [≈73 KiB] (mult 1.40); admitting 102 KiB (rate 6.8 KiB/s) (elastic 12 KiB rate 833 B/s) due to memtable flush (multiplier 1.400) (used total: 201 KiB elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:123000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:123000 WorkDuration:29000000000 IdleDuration:420000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:123000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9997 smoothedCompactionByteTokens:9997.74169921875 smoothedNumFlushTokens:74967.041015625 flushUtilTargetFraction:1.4 totalNumByteTokens:104953 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12496 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 intWALFailover:false prevTokensUsed:206068 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:10000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.8 KiB], flushed 73 KiB [≈73 KiB] (mult 1.40); admitting 102 KiB (rate 6.8 KiB/s) (elastic 12 KiB rate 833 B/s) due to memtable flush (multiplier 1.400) (used total: 201 KiB elastic 0 B); write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:123000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:123000 WorkDuration:29000000000 IdleDuration:420000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9997 smoothedCompactionByteTokens:9997.74169921875 smoothedNumFlushTokens:74967.041015625 flushUtilTargetFraction:1.4 totalNumByteTokens:104953 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12496 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 intWALFailover:false prevTokensUsed:206068 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=1750(elastic 209) elastic-disk-bw-tokens=unlimited max-byte-tokens=1750(elastic 209) max-disk-bw-tokens=unlimited lastTick=false # 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-write=133000 l0-files=1 l0-sublevels=2 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.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.8 KiB], flushed 73 KiB [≈73 KiB] (mult 1.38); admitting 101 KiB (rate 6.7 KiB/s) (elastic 12 KiB rate 833 B/s) due to memtable flush (multiplier 1.375) (used total: 205 KiB elastic 0 B); write stalls 1 -{ioLoadListenerState:{cumL0AddedBytes:133000 curL0Bytes:10000 cumWriteStallCount:14 cumFlushWriteThroughput:{Bytes:133000 WorkDuration:31000000000 IdleDuration:430000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:133000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9998 smoothedCompactionByteTokens:9998.870849609375 smoothedNumFlushTokens:74983.5205078125 flushUtilTargetFraction:1.375 totalNumByteTokens:103102 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12497 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:209906 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:10000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.8 KiB], flushed 73 KiB [≈73 KiB] (mult 1.38); admitting 101 KiB (rate 6.7 KiB/s) (elastic 12 KiB rate 833 B/s) due to memtable flush (multiplier 1.375) (used total: 205 KiB elastic 0 B); write stalls 1 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:133000 curL0Bytes:10000 cumWriteStallCount:14 cumFlushWriteThroughput:{Bytes:133000 WorkDuration:31000000000 IdleDuration:430000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9998 smoothedCompactionByteTokens:9998.870849609375 smoothedNumFlushTokens:74983.5205078125 flushUtilTargetFraction:1.375 totalNumByteTokens:103102 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12497 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:209906 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=1719(elastic 209) elastic-disk-bw-tokens=unlimited max-byte-tokens=1719(elastic 209) max-disk-bw-tokens=unlimited lastTick=false -# Test disk bandwidth tokens. +# Test disk bandwidth tokens. We also test the w-amp linear model here. init ---- -set-state l0-bytes=100 l0-added-write=0 bytes-read=0 bytes-written=0 provisioned-bandwidth=10 l0-files=1 l0-sublevels=1 print-only-first-tick=true +prep-admission-stats admitted=0 ---- -compaction score 0.000 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:0 curL0Bytes:100 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:0 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +{workCount:0 writeAccountedBytes:0 ingestedAccountedBytes:0 statsToIgnore:{ingestStats:{Bytes:0 ApproxIngestedIntoL0Bytes:0 MemtableOverlappingFiles:0} writeBytes:0} aboveRaftStats:{workCount:0 writeAccountedBytes:0 ingestedAccountedBytes:0} aux:{bypassedCount:0 writeBypassedAccountedBytes:0 ingestedBypassedAccountedBytes:0}} + +# Expect zeroes for most things. Smoothed write-amp will use the max value of model, smoothed with 0. +set-state l0-bytes=0 l0-added-write=0 bytes-read=0 bytes-written=0 provisioned-bandwidth=0 l0-files=1 l0-sublevels=1 print-only-first-tick=true +---- +compaction score 0.000 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:0 curL0Bytes:0 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false -set-state l0-bytes=100 l0-added-write=100000 bytes-read=1000000 bytes-written=2000000 provisioned-bandwidth=10 disk-bw-tokens-used=(100,100) l0-files=1 l0-sublevels=1 print-only-first-tick=true +# Admit some work. +prep-admission-stats admitted=10 write-bytes=10 ---- -compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 98 KiB (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 98 KiB [≈49 KiB], flushed 1.4 MiB [≈0 B] (mult 1.35); admitting all; elastic-disk-bw tokens 6.1 KiB (used 100 B, regular used 100 B): write model 1.75x+1 B ingest model 1.00x+1 B, disk bw read 65 KiB write 130 KiB provisioned 10 B; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:100000 curL0Bytes:100 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:100000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:1000000 bytesWritten:2000000 incomingLSMBytes:100000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:50000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskBWTokens:6250 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:66666 writeBandwidth:133333 provisionedBandwidth:10} intervalLSMInfo:{incomingBytes:100000 regularTokensUsed:100 elasticTokensUsed:100}}} ioThreshold:} -store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 -setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=105 max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=105 lastTick=false +{workCount:10 writeAccountedBytes:10 ingestedAccountedBytes:0 statsToIgnore:{ingestStats:{Bytes:0 ApproxIngestedIntoL0Bytes:0 MemtableOverlappingFiles:0} writeBytes:0} aboveRaftStats:{workCount:10 writeAccountedBytes:10 ingestedAccountedBytes:0} aux:{bypassedCount:0 writeBypassedAccountedBytes:0 ingestedBypassedAccountedBytes:0}} -# Considered moderate load because not enough history at low load. Elastic -# tokens don't increase since not fully utilized. -set-state l0-bytes=100 l0-added-write=200000 bytes-read=2000000 bytes-written=4000000 provisioned-bandwidth=4000000 disk-bw-tokens-used=(100,100) l0-files=1 l0-sublevels=1 print-only-first-tick=true +# Set provisioned bandwidth and some disk reads and writes. +# setAvailableTokens should show ((provisioned-bandwidth * adjustment-interval) - bytes-read) / num ticks. ((100*15) - 30) / 60 = 25. +set-state l0-bytes=10 l0-added-write=10 bytes-read=60 bytes-written=50 provisioned-bandwidth=100 disk-write-tokens-used=(100,100) l0-files=1 l0-sublevels=1 print-only-first-tick=true ---- -compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 98 KiB (write 98 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 98 KiB [≈73 KiB], flushed 1.4 MiB [≈0 B] (mult 1.35); admitting all; elastic-disk-bw tokens 6.1 KiB (used 100 B, regular used 100 B): write model 1.75x+1 B ingest model 1.00x+1 B, disk bw read 65 KiB write 130 KiB provisioned 3.8 MiB; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:200000 curL0Bytes:100 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:200000 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:2000000 bytesWritten:4000000 incomingLSMBytes:200000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:75000 smoothedCompactionByteTokens:75000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskBWTokens:6250 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:66666 writeBandwidth:133333 provisionedBandwidth:4000000} intervalLSMInfo:{incomingBytes:100000 regularTokensUsed:100 elasticTokensUsed:100}}} ioThreshold:} +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 10 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 150 B [≈0 B] (mult 1.35); admitting all; write stalls 0 +diskBandwidthLimiter (tokenUtilization +Inf, tokensUsed (elastic 100 B, regular 100 B) tokens (write 1.4 KiB (prev 0 B)), writeBW 3 B/s, readBW 4 B/s, provisioned 100 B/s) +{ioLoadListenerState:{cumL0AddedBytes:10 curL0Bytes:10 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:10 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:60 bytesWritten:50} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:1440 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10 intL0CompactedBytes:0 intFlushTokens:150 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedWriteBytes:0} doLogFlush:false} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 -setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=105 max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=105 lastTick=false +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 +setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=24 max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=24 lastTick=false -# Stay at moderate load since utilization increasing. -set-state l0-bytes=100 l0-added-write=300000 bytes-read=4000000 bytes-written=8000000 provisioned-bandwidth=4000000 disk-bw-tokens-used=(100,100) l0-files=1 l0-sublevels=2 print-only-first-tick=true +prep-admission-stats admitted=20 write-bytes=20 ---- -compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 98 KiB (write 98 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 98 KiB [≈85 KiB], flushed 1.4 MiB [≈0 B] (mult 1.35); admitting elastic 107 KiB (rate 7.1 KiB/s) due to L0 growth; elastic-disk-bw tokens 6.1 KiB (used 100 B, regular used 100 B): write model 1.75x+1 B ingest model 1.00x+1 B, disk bw read 130 KiB write 260 KiB provisioned 3.8 MiB; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:300000 curL0Bytes:100 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:300000 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:4000000 bytesWritten:8000000 incomingLSMBytes:300000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:87500 smoothedCompactionByteTokens:87500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:109375 elasticByteTokensAllocated:0 elasticDiskBWTokens:6250 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:133333 writeBandwidth:266666 provisionedBandwidth:4000000} intervalLSMInfo:{incomingBytes:100000 regularTokensUsed:100 elasticTokensUsed:100}}} ioThreshold:} -store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 -setAvailableTokens: io-tokens=unlimited(elastic 1823) elastic-disk-bw-tokens=105 max-byte-tokens=unlimited(elastic 1823) max-disk-bw-tokens=105 lastTick=false +{workCount:20 writeAccountedBytes:20 ingestedAccountedBytes:0 statsToIgnore:{ingestStats:{Bytes:0 ApproxIngestedIntoL0Bytes:0 MemtableOverlappingFiles:0} writeBytes:0} aboveRaftStats:{workCount:20 writeAccountedBytes:20 ingestedAccountedBytes:0} aux:{bypassedCount:0 writeBypassedAccountedBytes:0 ingestedBypassedAccountedBytes:0}} + +set-state l0-bytes=40 l0-added-write=40 bytes-read=120 bytes-written=400 provisioned-bandwidth=100 disk-write-tokens-used=(200,200) l0-files=1 l0-sublevels=1 print-only-first-tick=true +---- +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 30 B (write 30 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10 (0 bypassed) with 10 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 30 B adjusted-LSM-writes + 350 B adjusted-disk-writes + write-model 2.00x+1 B (smoothed 1.88x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 11.33x+1 B (smoothed 30.92x+1 B) + at-admission-tokens 2 B, compacted 0 B [≈0 B], flushed 435 B [≈0 B] (mult 1.35); admitting all; write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.28, tokensUsed (elastic 200 B, regular 200 B) tokens (write 1.4 KiB (prev 1.4 KiB)), writeBW 23 B/s, readBW 4 B/s, provisioned 100 B/s) +{ioLoadListenerState:{cumL0AddedBytes:40 curL0Bytes:40 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:40 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:120 bytesWritten:400} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:1440 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:2} l0WriteLM:{multiplier:1.875 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:30.916666666666668 constant:1} aux:{intL0AddedBytes:30 intL0CompactedBytes:0 intFlushTokens:435 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:30 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:10 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:2 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:11.333333333333334 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:30 intAdjustedWriteBytes:350} doLogFlush:false} ioThreshold:} +store-request-estimates: writeTokens: 2 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.88x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 30.92x+1 +setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=24 max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=24 lastTick=false + +prep-admission-stats admitted=60 write-bytes=70 +---- +{workCount:60 writeAccountedBytes:70 ingestedAccountedBytes:0 statsToIgnore:{ingestStats:{Bytes:0 ApproxIngestedIntoL0Bytes:0 MemtableOverlappingFiles:0} writeBytes:0} aboveRaftStats:{workCount:60 writeAccountedBytes:70 ingestedAccountedBytes:0} aux:{bypassedCount:0 writeBypassedAccountedBytes:0 ingestedBypassedAccountedBytes:0}} -# Drop to low load. -set-state l0-bytes=100 l0-added-write=400000 bytes-read=5000000 bytes-written=9000000 provisioned-bandwidth=5000000 disk-bw-tokens-used=(100,100) l0-files=1 l0-sublevels=2 print-only-first-tick=true +set-state l0-bytes=90 l0-added-write=80 bytes-read=180 bytes-written=1200 provisioned-bandwidth=100 disk-write-tokens-used=(400,500) l0-files=1 l0-sublevels=2 print-only-first-tick=true ---- -compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 98 KiB (write 98 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 98 KiB [≈92 KiB], flushed 1.4 MiB [≈0 B] (mult 1.35); admitting elastic 114 KiB (rate 7.6 KiB/s) due to L0 growth; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:400000 curL0Bytes:100 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:400000 WorkDuration:5000000000 IdleDuration:500000000000} diskBW:{bytesRead:5000000 bytesWritten:9000000 incomingLSMBytes:400000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:93750 smoothedCompactionByteTokens:93750 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:117187 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:66666 writeBandwidth:66666 provisionedBandwidth:5000000} intervalLSMInfo:{incomingBytes:100000 regularTokensUsed:100 elasticTokensUsed:100}}} ioThreshold:} +compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 40 B (write 40 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 40 (0 bypassed) with 50 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 40 B adjusted-LSM-writes + 800 B adjusted-disk-writes + write-model 0.50x+1 B (smoothed 1.19x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 19.00x+1 B (smoothed 24.96x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 600 B [≈0 B] (mult 1.35); admitting elastic 1 B (rate 0 B/s) due to L0 growth; write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.62, tokensUsed (elastic 500 B, regular 400 B) tokens (write 1.4 KiB (prev 1.4 KiB)), writeBW 53 B/s, readBW 4 B/s, provisioned 100 B/s) +{ioLoadListenerState:{cumL0AddedBytes:80 curL0Bytes:90 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:80 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:180 bytesWritten:1200} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:1440 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.1875 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:24.958333333333336 constant:1} aux:{intL0AddedBytes:40 intL0CompactedBytes:0 intFlushTokens:600 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:40 intL0WriteBytes:40 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:50 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0.5 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:19 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:40 intAdjustedWriteBytes:800} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 -setAvailableTokens: io-tokens=unlimited(elastic 1954) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic 1954) max-disk-bw-tokens=unlimited lastTick=false +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.19x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 24.96x+1 +setAvailableTokens: io-tokens=unlimited(elastic 1) elastic-disk-bw-tokens=24 max-byte-tokens=unlimited(elastic 1) max-disk-bw-tokens=24 lastTick=false # Test out max capacity calculations for a 1ms tick rate. init @@ -638,8 +682,9 @@ prep-admission-stats admitted=0 # Even though above the threshold, the first 60 ticks don't limit the tokens. set-state l0-bytes=10000 l0-added-write=1000 l0-files=21 l0-sublevels=21 print-only-first-tick=true loaded=true ---- -compaction score 0.000 (21 ssts, 21 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:0 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.000 (21 ssts, 21 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false prep-admission-stats admitted=10000 write-bytes=40000 @@ -652,10 +697,11 @@ prep-admission-stats admitted=10000 write-bytes=40000 # removed), but smoothing it drops the tokens to 12,500. set-state l0-bytes=10000 l0-added-write=101000 l0-files=21 l0-sublevels=21 print-only-first-tick=true loaded=true ---- -compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 98 KiB (write 98 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10000 (0 bypassed) with 39 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 2.25x+1 B (smoothed 2.00x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 5 B, compacted 98 KiB [≈49 KiB], flushed 1.4 MiB [≈0 B] (mult 1.35); admitting 12 KiB (rate 833 B/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:101000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:101000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:101000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:12500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:12500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:5} l0WriteLM:{multiplier:2 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:2.25 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:100000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 98 KiB (write 98 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10000 (0 bypassed) with 39 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 98 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 2.25x+1 B (smoothed 2.00x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 5 B, compacted 98 KiB [≈49 KiB], flushed 1.4 MiB [≈0 B] (mult 1.35); admitting 12 KiB (rate 833 B/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:101000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:101000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:12500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:12500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:5} l0WriteLM:{multiplier:2 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:2.25 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:100000 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 5 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.00x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.00x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=1(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=209(elastic 1) max-disk-bw-tokens=unlimited lastTick=false # Restrict disk bandwidth tokens, and then check the cap. @@ -664,17 +710,19 @@ init set-state l0-bytes=100 l0-added-write=0 bytes-read=0 bytes-written=0 provisioned-bandwidth=10 l0-files=1 l0-sublevels=1 print-only-first-tick=true loaded=true ---- -compaction score 0.000 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:0 curL0Bytes:100 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:0 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.000 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:0 curL0Bytes:100 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false set-state l0-bytes=100 l0-added-write=100000 bytes-read=1000000 bytes-written=2000000 provisioned-bandwidth=10 disk-bw-tokens-used=(100,100) l0-files=1 l0-sublevels=1 print-only-first-tick=true loaded=true ---- -compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 98 KiB (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 98 KiB [≈49 KiB], flushed 1.4 MiB [≈0 B] (mult 1.35); admitting all; elastic-disk-bw tokens 6.1 KiB (used 100 B, regular used 100 B): write model 1.75x+1 B ingest model 1.00x+1 B, disk bw read 65 KiB write 130 KiB provisioned 10 B; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:100000 curL0Bytes:100 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:100000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:1000000 bytesWritten:2000000 incomingLSMBytes:100000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:50000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskBWTokens:6250 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:66666 writeBandwidth:133333 provisionedBandwidth:10} intervalLSMInfo:{incomingBytes:100000 regularTokensUsed:100 elasticTokensUsed:100}}} ioThreshold:} +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 98 KiB (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 98 KiB [≈49 KiB], flushed 1.4 MiB [≈0 B] (mult 1.35); admitting all; write stalls 0 +diskBandwidthLimiter (tokenUtilization NaN, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 130 KiB/s, readBW 65 KiB/s, provisioned 10 B/s) +{ioLoadListenerState:{cumL0AddedBytes:100000 curL0Bytes:100 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:100000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:1000000 bytesWritten:2000000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:50000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:0 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedWriteBytes:0} doLogFlush:false} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 -setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=1 max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=105 lastTick=false +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 +setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=0 max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=0 lastTick=false # Test scoring and interpolation for compaction token smoothing. init @@ -686,8 +734,9 @@ prep-admission-stats admitted=0 set-state l0-bytes=10000 l0-added-write=1000 l0-files=5 l0-sublevels=5 print-only-first-tick=true loaded=true ---- -compaction score 0.000 (5 ssts, 5 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:0 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.000 (5 ssts, 5 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false # No restricting of tokens. @@ -697,10 +746,11 @@ tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk set-state l0-bytes=10000 l0-added-write=501000 l0-files=1 l0-sublevels=1 print-only-first-tick=true loaded=true ---- -compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 488 KiB (write 488 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 488 KiB [≈244 KiB], flushed 7.2 MiB [≈0 B] (mult 1.35); admitting all; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:501000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:250000 smoothedCompactionByteTokens:250000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:500000 intL0CompactedBytes:500000 intFlushTokens:7.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:500000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:500000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 488 KiB (write 488 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 488 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 488 KiB [≈244 KiB], flushed 7.2 MiB [≈0 B] (mult 1.35); admitting all; write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:250000 smoothedCompactionByteTokens:250000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:500000 intL0CompactedBytes:500000 intFlushTokens:7.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:500000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:500000 intAdjustedWriteBytes:0} doLogFlush:false} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false # Start restricting compaction tokens due to sublevels. We check if the value of @@ -713,10 +763,11 @@ setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-token # generating different output on an M1 macbook and linux with Intel Xeon. set-state l0-bytes=10000 l0-added-write=501002 l0-files=10 l0-sublevels=10 print-only-first-tick=true loaded=true ---- -compaction score 0.500 (10 ssts, 10 sub-levels), L0 growth 2 B (write 2 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 2 B [≈122 KiB], flushed 30 B [≈0 B] (mult 1.35); admitting 183 KiB (rate 12 KiB/s) (elastic 30 KiB rate 2.0 KiB/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:501002 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501002 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:501002} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:125001 smoothedCompactionByteTokens:187500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:187500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:31250 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:2 intL0CompactedBytes:2 intFlushTokens:30 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:2 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:2 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.500 (10 ssts, 10 sub-levels), L0 growth 2 B (write 2 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 2 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 2 B [≈122 KiB], flushed 30 B [≈0 B] (mult 1.35); admitting 183 KiB (rate 12 KiB/s) (elastic 30 KiB rate 2.0 KiB/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:501002 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501002 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:125001 smoothedCompactionByteTokens:187500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:187500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:31250 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:2 intL0CompactedBytes:2 intFlushTokens:30 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:2 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:2 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=13(elastic 3) elastic-disk-bw-tokens=unlimited max-byte-tokens=3125(elastic 521) max-disk-bw-tokens=unlimited lastTick=false # For a score of 0.75, we should use 3/4 * smoothedIntL0CompactedBytes. @@ -726,10 +777,11 @@ setAvailableTokens: io-tokens=13(elastic 3) elastic-disk-bw-tokens=unlimited max # 187500 is the previous smoothedCompactionByteTokens. set-state l0-bytes=10000 l0-added-write=501000 l0-files=10 l0-sublevels=15 print-only-first-tick=true loaded=true ---- -compaction score 0.750 (10 ssts, 15 sub-levels), L0 growth 0 B (write -2 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈61 KiB], flushed 0 B [≈0 B] (mult 1.35); admitting 114 KiB (rate 7.6 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:501000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:62500 smoothedCompactionByteTokens:117187.5 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:117187 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:-2 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:-2 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.750 (10 ssts, 15 sub-levels), L0 growth 0 B (write -2 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈61 KiB], flushed 0 B [≈0 B] (mult 1.35); admitting 114 KiB (rate 7.6 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:62500 smoothedCompactionByteTokens:117187.5 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:117187 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:-2 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=8(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=1954(elastic 1) max-disk-bw-tokens=unlimited lastTick=false # Full overload, score >= 1. @@ -740,10 +792,11 @@ setAvailableTokens: io-tokens=8(elastic 1) elastic-disk-bw-tokens=unlimited max- # at 20 sublevels. 117187.5 is the previous smoothedCompactionByteTokens. set-state l0-bytes=10000 l0-added-write=501000 l0-files=10 l0-sublevels=20 print-only-first-tick=true loaded=true ---- -compaction score 1.000 (10 ssts, 20 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈30 KiB], flushed 0 B [≈0 B] (mult 1.35); admitting 65 KiB (rate 4.3 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:5000000000 IdleDuration:500000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:501000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:31250 smoothedCompactionByteTokens:66406.25 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:66406 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:0 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 1.000 (10 ssts, 20 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈30 KiB], flushed 0 B [≈0 B] (mult 1.35); admitting 65 KiB (rate 4.3 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:5000000000 IdleDuration:500000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:31250 smoothedCompactionByteTokens:66406.25 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:66406 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=5(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=1107(elastic 1) max-disk-bw-tokens=unlimited lastTick=false # Low load, but not underload. @@ -752,10 +805,11 @@ setAvailableTokens: io-tokens=5(elastic 1) elastic-disk-bw-tokens=unlimited max- # 48828.125 = 0.5 * 66406.25 + 0.5 * 2 * 15625. Checks out. set-state l0-bytes=10000 l0-added-write=501000 l0-files=10 l0-sublevels=5 print-only-first-tick=true loaded=true ---- -compaction score 0.250 (10 ssts, 5 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈15 KiB], flushed 0 B [≈0 B] (mult 1.35); admitting 48 KiB (rate 3.2 KiB/s) (elastic 13 KiB rate 911 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:6000000000 IdleDuration:600000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:501000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:15625 smoothedCompactionByteTokens:48828.125 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:48828 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:13671 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:0 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.250 (10 ssts, 5 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈15 KiB], flushed 0 B [≈0 B] (mult 1.35); admitting 48 KiB (rate 3.2 KiB/s) (elastic 13 KiB rate 911 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:6000000000 IdleDuration:600000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:15625 smoothedCompactionByteTokens:48828.125 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:48828 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:13671 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=4(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=814(elastic 228) max-disk-bw-tokens=unlimited lastTick=false # Test minimum size per sub-level affecting the score. @@ -768,17 +822,19 @@ prep-admission-stats admitted=0 set-state l0-bytes=10000 l0-added-write=1000 l0-files=21 l0-sublevels=21 print-only-first-tick=true ---- -compaction score 0.000 (21 ssts, 21 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:0 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.000 (21 ssts, 21 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false # Score is 21/20. set-state l0-bytes=10000 l0-added-write=101000 l0-files=21 l0-sublevels=21 print-only-first-tick=true ---- -compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 98 KiB (write 98 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 98 KiB [≈49 KiB], flushed 1.4 MiB [≈0 B] (mult 1.35); admitting 12 KiB (rate 833 B/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:101000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:101000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:101000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:12500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:12500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:100000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 98 KiB (write 98 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 98 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 98 KiB [≈49 KiB], flushed 1.4 MiB [≈0 B] (mult 1.35); admitting 12 KiB (rate 833 B/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:101000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:101000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:12500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:12500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:100000 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=209(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=209(elastic 1) max-disk-bw-tokens=unlimited lastTick=false set-min-size-per-sub-level size=5000 @@ -787,19 +843,21 @@ set-min-size-per-sub-level size=5000 # Score is (21/3)/20 since min sub-levels is 21/3 and max is 10000/5000=2. set-state l0-bytes=10000 l0-added-write=201000 l0-files=21 l0-sublevels=21 print-only-first-tick=true ---- -compaction score 0.350 (21 ssts, 21 sub-levels), L0 growth 98 KiB (write 98 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 98 KiB [≈73 KiB], flushed 1.4 MiB [≈0 B] (mult 1.35); admitting 65 KiB (rate 4.3 KiB/s) (elastic 46 KiB rate 3.1 KiB/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:201000 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:201000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:75000 smoothedCompactionByteTokens:66250 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:66250 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:46875 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:100000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.350 (21 ssts, 21 sub-levels), L0 growth 98 KiB (write 98 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 98 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 98 KiB [≈73 KiB], flushed 1.4 MiB [≈0 B] (mult 1.35); admitting 65 KiB (rate 4.3 KiB/s) (elastic 46 KiB rate 3.1 KiB/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:201000 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:75000 smoothedCompactionByteTokens:66250 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:66250 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:46875 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:100000 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=1105(elastic 782) elastic-disk-bw-tokens=unlimited max-byte-tokens=1105(elastic 782) max-disk-bw-tokens=unlimited lastTick=false # Score is decided by max sub-levels, which is 50000/5000=10. So score is 10/20=0.5. set-state l0-bytes=50000 l0-added-write=260994 l0-files=21 l0-sublevels=21 print-only-first-tick=true ---- -compaction score 0.500 (21 ssts, 21 sub-levels), L0 growth 59 KiB (write 59 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 20 KiB [≈46 KiB], flushed 879 KiB [≈0 B] (mult 1.35); admitting 56 KiB (rate 3.7 KiB/s) (elastic 12 KiB rate 791 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:260994 curL0Bytes:50000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:260994 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:260994} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:47497 smoothedCompactionByteTokens:56873 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:56873 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:11874 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:59994 intL0CompactedBytes:19994 intFlushTokens:899910 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:59994 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:59994 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.500 (21 ssts, 21 sub-levels), L0 growth 59 KiB (write 59 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 59 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 20 KiB [≈46 KiB], flushed 879 KiB [≈0 B] (mult 1.35); admitting 56 KiB (rate 3.7 KiB/s) (elastic 12 KiB rate 791 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:260994 curL0Bytes:50000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:260994 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:47497 smoothedCompactionByteTokens:56873 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:56873 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:11874 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:59994 intL0CompactedBytes:19994 intFlushTokens:899910 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:59994 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:59994 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=948(elastic 198) elastic-disk-bw-tokens=unlimited max-byte-tokens=948(elastic 198) max-disk-bw-tokens=unlimited lastTick=false # Test compaction token lower bound. @@ -812,35 +870,39 @@ prep-admission-stats admitted=0 set-state l0-bytes=10000 l0-added-write=1000 l0-files=100 l0-sublevels=6 print-only-first-tick=true ---- -compaction score 0.000 (100 ssts, 6 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:0 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.000 (100 ssts, 6 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false # No compactions out of L0, but L0 token lower bound is set to 1MB/2 and then smoothed to give us "admitting 244 KiB (rate 16 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth(used token lower bound)" set-state l0-bytes=10000 l0-added-write=1000 l0-files=100 l0-sublevels=6 base-level=5 compacted-bytes=1000000 print-only-first-tick=true ---- -compaction score 0.300 (100 ssts, 6 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 1.35); admitting 244 KiB (rate 16 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth(used token lower bound) (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} cumCompactionStats:{writeBytes:1000000 numOutLevelsGauge:2} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:250000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:250000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:true perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:0 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.300 (100 ssts, 6 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 1.35); admitting 244 KiB (rate 16 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth(used token lower bound) (used total: 0 B elastic 0 B); write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:1000000 numOutLevelsGauge:2} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:250000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:250000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:true perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=4167(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=4167(elastic 1) max-disk-bw-tokens=unlimited lastTick=false # Repeat, but the exponential smoothing increases the bytes to admit. set-state l0-bytes=10000 l0-added-write=1000 l0-files=100 l0-sublevels=6 base-level=5 compacted-bytes=2000000 print-only-first-tick=true ---- -compaction score 0.300 (100 ssts, 6 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 1.35); admitting 366 KiB (rate 24 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth(used token lower bound) (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} cumCompactionStats:{writeBytes:2000000 numOutLevelsGauge:2} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:375000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:375000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:true perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:0 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.300 (100 ssts, 6 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 1.35); admitting 366 KiB (rate 24 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth(used token lower bound) (used total: 0 B elastic 0 B); write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:2000000 numOutLevelsGauge:2} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:375000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:375000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:true perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=6250(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=6250(elastic 1) max-disk-bw-tokens=unlimited lastTick=false # Repeat, but the exponential smoothing increases the bytes to admit. set-state l0-bytes=10000 l0-added-write=1000 l0-files=100 l0-sublevels=6 base-level=5 compacted-bytes=3000000 print-only-first-tick=true ---- -compaction score 0.300 (100 ssts, 6 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 1.35); admitting 427 KiB (rate 28 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth(used token lower bound) (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} cumCompactionStats:{writeBytes:3000000 numOutLevelsGauge:2} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:437500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:437500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:true perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:0 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.300 (100 ssts, 6 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 1.35); admitting 427 KiB (rate 28 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth(used token lower bound) (used total: 0 B elastic 0 B); write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:3000000 numOutLevelsGauge:2} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:437500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:437500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:true perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=7292(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=7292(elastic 1) max-disk-bw-tokens=unlimited lastTick=false # When we get to 10 sub-levels we stop using the lower bound, since at this @@ -849,8 +911,9 @@ setAvailableTokens: io-tokens=7292(elastic 1) elastic-disk-bw-tokens=unlimited m # incoming writes. set-state l0-bytes=10000 l0-added-write=1000 l0-files=100 l0-sublevels=10 base-level=5 compacted-bytes=3000000 print-only-first-tick=true ---- -compaction score 0.500 (100 ssts, 10 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 1.35); admitting 214 KiB (rate 14 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:5000000000 IdleDuration:500000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} cumCompactionStats:{writeBytes:3000000 numOutLevelsGauge:2} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:218750 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:218750 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:0 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +compaction score 0.500 (100 ssts, 10 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 1.35); admitting 214 KiB (rate 14 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:5000000000 IdleDuration:500000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:3000000 numOutLevelsGauge:2} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:218750 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:218750 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 -tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=3646(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=3646(elastic 1) max-disk-bw-tokens=unlimited lastTick=false diff --git a/pkg/util/admission/testdata/store_per_work_token_estimator b/pkg/util/admission/testdata/store_per_work_token_estimator index e2892ec51765..693e20e91c4e 100644 --- a/pkg/util/admission/testdata/store_per_work_token_estimator +++ b/pkg/util/admission/testdata/store_per_work_token_estimator @@ -4,32 +4,35 @@ init # First call initializes. update flushed=1000 ingested=0 admitted=10 write-accounted=500 ingested-accounted=0 ---- -interval state: {intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} +interval state: {intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedWriteBytes:0} at-admission-tokens: 1 L0-write-tokens: int: 0.00x+0 smoothed: 1.75x+1 per-work-accounted: 1 L0-ingest-tokens: int: 0.00x+0 smoothed: 0.75x+1 per-work-accounted: 1 ingest-tokens: int: 0.00x+0 smoothed: 1.00x+1 per-work-accounted: 1 +write-amp: int: 0.00x+0 smoothed: 50.50x+1 per-work-accounted: 1 # Writes account for ~1/2 of what is written, reflecting what can happen with # application to the state machine. No ingests. update flushed=1000 ingested=0 admitted=10 write-accounted=500 ingested-accounted=0 above-raft-count=8 above-raft-write=40 ---- -interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:500 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:1.98 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} +interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:500 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:1.98 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:1000 intAdjustedWriteBytes:0} at-admission-tokens: 5 L0-write-tokens: int: 1.98x+1 smoothed: 1.86x+1 per-work-accounted: 25 L0-ingest-tokens: int: 0.00x+0 smoothed: 0.75x+1 per-work-accounted: 1 ingest-tokens: int: 0.00x+0 smoothed: 1.00x+1 per-work-accounted: 1 +write-amp: int: 0.00x+0 smoothed: 50.50x+1 per-work-accounted: 1 # Same as previous, except some of these are bypassed. Will not affect the # model. Since there are no above-raft stats, at-admission-tokens is # unchanged. update flushed=1000 ingested=0 admitted=10 write-accounted=500 ingested-accounted=0 bypassed-count=4 bypassed-write=300 bypassed-ingested=0 ---- -interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:500 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:1.98 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:4 intL0WriteBypassedAccountedBytes:300 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} +interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:500 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:1.98 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:4 intL0WriteBypassedAccountedBytes:300 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:1000 intAdjustedWriteBytes:0} at-admission-tokens: 5 L0-write-tokens: int: 1.98x+1 smoothed: 1.92x+1 per-work-accounted: 37 L0-ingest-tokens: int: 0.00x+0 smoothed: 0.75x+1 per-work-accounted: 1 ingest-tokens: int: 0.00x+0 smoothed: 1.00x+1 per-work-accounted: 1 +write-amp: int: 0.00x+0 smoothed: 50.50x+1 per-work-accounted: 1 # Ingestion also happens. Bumps up the at-admission-tokens since at that time # we can't differentiate between writes and ingests. The constants in the @@ -43,11 +46,12 @@ ingest-tokens: int: 0.00x+0 smoothed: 1.00x+1 per-work-accounted: 1 # max multiplier of 1.5 is used and the rest handled in the additive term. update flushed=1000 ingested=1000 other-levels-ingested=9000 admitted=10 write-accounted=500 ingested-accounted=4000 bypassed-count=2 bypassed-write=0 bypassed-ingested=2000 above-raft-count=4 above-raft-write=100 ---- -interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:1000 intLSMIngestedBytes:10000 intL0WriteAccountedBytes:500 intIngestedAccountedBytes:4000 intL0WriteLinearModel:{multiplier:1.98 constant:1} intL0IngestedLinearModel:{multiplier:0.2475 constant:1} intIngestedLinearModel:{multiplier:1.5 constant:400} intBypassedWorkCount:2 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:2000 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} +interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:1000 intLSMIngestedBytes:10000 intL0WriteAccountedBytes:500 intIngestedAccountedBytes:4000 intL0WriteLinearModel:{multiplier:1.98 constant:1} intL0IngestedLinearModel:{multiplier:0.2475 constant:1} intIngestedLinearModel:{multiplier:1.5 constant:400} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:2 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:2000 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:11000 intAdjustedWriteBytes:0} at-admission-tokens: 19 L0-write-tokens: int: 1.98x+1 smoothed: 1.95x+1 per-work-accounted: 43 L0-ingest-tokens: int: 0.25x+1 smoothed: 0.50x+1 per-work-accounted: 200 ingest-tokens: int: 1.50x+400 smoothed: 1.25x+200 per-work-accounted: 200 +write-amp: int: 0.00x+0 smoothed: 50.50x+1 per-work-accounted: 1 # No ingestion into L0 observed by LSM, though ingested-accounted is non-zero # -- this updates the L0-ingest model since all these ingested bytes have gone @@ -55,58 +59,93 @@ ingest-tokens: int: 1.50x+400 smoothed: 1.25x+200 per-work-accounted: 200 # and the accounted bytes are 500, hence the multiplier close to 1.0. update flushed=1000 ingested=0 other-levels-ingested=500 admitted=10 write-accounted=450 ingested-accounted=500 above-raft-count=2 above-raft-write=225 above-write-ingested=250 ---- -interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:0 intLSMIngestedBytes:500 intL0WriteAccountedBytes:450 intIngestedAccountedBytes:500 intL0WriteLinearModel:{multiplier:2.2 constant:1} intL0IngestedLinearModel:{multiplier:0.001 constant:1} intIngestedLinearModel:{multiplier:0.98 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} +interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:0 intLSMIngestedBytes:500 intL0WriteAccountedBytes:450 intIngestedAccountedBytes:500 intL0WriteLinearModel:{multiplier:2.2 constant:1} intL0IngestedLinearModel:{multiplier:0.001 constant:1} intIngestedLinearModel:{multiplier:0.98 constant:1} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:1500 intAdjustedWriteBytes:0} at-admission-tokens: 120 L0-write-tokens: int: 2.20x+1 smoothed: 2.08x+1 per-work-accounted: 44 L0-ingest-tokens: int: 0.00x+1 smoothed: 0.25x+1 per-work-accounted: 125 ingest-tokens: int: 0.98x+1 smoothed: 1.11x+100 per-work-accounted: 125 +write-amp: int: 0.00x+0 smoothed: 50.50x+1 per-work-accounted: 1 # Large amount of ingestion. Does not bump up at-admission-tokens to a very # large value since the ingestions do below-raft admission. update flushed=1000 ingested=1000000 other-levels-ingested=2000000 admitted=10 write-accounted=450 ingested-accounted=2000000 above-raft-count=10 above-raft-write=450 ---- -interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:1000000 intLSMIngestedBytes:3000000 intL0WriteAccountedBytes:450 intIngestedAccountedBytes:2000000 intL0WriteLinearModel:{multiplier:2.2 constant:1} intL0IngestedLinearModel:{multiplier:0.499995 constant:1} intIngestedLinearModel:{multiplier:1.499995 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0} +interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:1000000 intLSMIngestedBytes:3000000 intL0WriteAccountedBytes:450 intIngestedAccountedBytes:2000000 intL0WriteLinearModel:{multiplier:2.2 constant:1} intL0IngestedLinearModel:{multiplier:0.499995 constant:1} intIngestedLinearModel:{multiplier:1.499995 constant:1} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:3001000 intAdjustedWriteBytes:0} at-admission-tokens: 124 L0-write-tokens: int: 2.20x+1 smoothed: 2.14x+1 per-work-accounted: 44 L0-ingest-tokens: int: 0.50x+1 smoothed: 0.37x+1 per-work-accounted: 100062 ingest-tokens: int: 1.50x+1 smoothed: 1.31x+50 per-work-accounted: 100062 +write-amp: int: 0.00x+0 smoothed: 50.50x+1 per-work-accounted: 1 # Large number of bytes ingested into L0, but only 2000 are not to be ignored. # So we can fit a reasonable model. The L0-ingest and ingest models are the # same since no bytes were ingested into levels other than L0. update flushed=0 ingested=1000000 admitted=10 write-accounted=0 ingested-accounted=2000 ignore-ingested-into-L0=998000 ---- -interval state: {intWorkCount:10 intL0WriteBytes:0 intL0IngestedBytes:1000000 intLSMIngestedBytes:1000000 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:2000 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0.995 constant:1} intIngestedLinearModel:{multiplier:0.995 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:998000} +interval state: {intWorkCount:10 intL0WriteBytes:0 intL0IngestedBytes:1000000 intLSMIngestedBytes:1000000 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:2000 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0.995 constant:1} intIngestedLinearModel:{multiplier:0.995 constant:1} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:998000 intAdjustedLSMWrites:2000 intAdjustedWriteBytes:0} at-admission-tokens: 124 L0-write-tokens: int: 0.00x+0 smoothed: 2.14x+1 per-work-accounted: 44 L0-ingest-tokens: int: 0.99x+1 smoothed: 0.68x+1 per-work-accounted: 50131 ingest-tokens: int: 0.99x+1 smoothed: 1.15x+25 per-work-accounted: 50131 +write-amp: int: 0.00x+0 smoothed: 50.50x+1 per-work-accounted: 1 # Repeat of the previous. update flushed=0 ingested=1000000 admitted=10 write-accounted=0 ingested-accounted=2000 ignore-ingested-into-L0=998000 ---- -interval state: {intWorkCount:10 intL0WriteBytes:0 intL0IngestedBytes:1000000 intLSMIngestedBytes:1000000 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:2000 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0.995 constant:1} intIngestedLinearModel:{multiplier:0.995 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:998000} +interval state: {intWorkCount:10 intL0WriteBytes:0 intL0IngestedBytes:1000000 intLSMIngestedBytes:1000000 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:2000 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0.995 constant:1} intIngestedLinearModel:{multiplier:0.995 constant:1} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:998000 intAdjustedLSMWrites:2000 intAdjustedWriteBytes:0} at-admission-tokens: 124 L0-write-tokens: int: 0.00x+0 smoothed: 2.14x+1 per-work-accounted: 44 L0-ingest-tokens: int: 0.99x+1 smoothed: 0.84x+1 per-work-accounted: 25165 ingest-tokens: int: 0.99x+1 smoothed: 1.07x+13 per-work-accounted: 25165 +write-amp: int: 0.00x+0 smoothed: 50.50x+1 per-work-accounted: 1 # Large number of bytes written into L0, but only 2000 are not to be ignored. # So we can fit a reasonable model. The above-raft writes are small, so # at-admission-tokens shrinks. update flushed=1000000 ingested=0 admitted=10 write-accounted=1000 ingested-accounted=0 ignored-written=998000 above-raft-count=9 above-raft-write=20 ---- -interval state: {intWorkCount:10 intL0WriteBytes:1000000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:1000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:1.99 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:998000 intL0IgnoredIngestedBytes:0} +interval state: {intWorkCount:10 intL0WriteBytes:1000000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:1000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:1.99 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:998000 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:2000 intAdjustedWriteBytes:0} at-admission-tokens: 64 L0-write-tokens: int: 1.99x+1 smoothed: 2.06x+1 per-work-accounted: 72 L0-ingest-tokens: int: 0.00x+0 smoothed: 0.84x+1 per-work-accounted: 25165 ingest-tokens: int: 0.00x+0 smoothed: 1.07x+6 per-work-accounted: 25165 +write-amp: int: 0.00x+0 smoothed: 50.50x+1 per-work-accounted: 1 # Large number of ignored bytes written into L0, but not yet flushed. update flushed=2000 ingested=0 admitted=10 write-accounted=1000 ingested-accounted=0 ignored-written=998000 ---- -interval state: {intWorkCount:10 intL0WriteBytes:2000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:1000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:998000 intL0IgnoredIngestedBytes:0} +interval state: {intWorkCount:10 intL0WriteBytes:2000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:1000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:998000 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedWriteBytes:0} at-admission-tokens: 64 L0-write-tokens: int: 0.00x+0 smoothed: 2.06x+1 per-work-accounted: 72 L0-ingest-tokens: int: 0.00x+0 smoothed: 0.84x+1 per-work-accounted: 25165 ingest-tokens: int: 0.00x+0 smoothed: 1.07x+3 per-work-accounted: 25165 +write-amp: int: 0.00x+0 smoothed: 50.50x+1 per-work-accounted: 1 + + +# Test write amp LM + +init +---- + +# First call initializes. +update flushed=1000 ingested=0 admitted=10 write-accounted=500 ingested-accounted=0 disk-writes=0 +---- +interval state: {intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedWriteBytes:0} +at-admission-tokens: 1 +L0-write-tokens: int: 0.00x+0 smoothed: 1.75x+1 per-work-accounted: 1 +L0-ingest-tokens: int: 0.00x+0 smoothed: 0.75x+1 per-work-accounted: 1 +ingest-tokens: int: 0.00x+0 smoothed: 1.00x+1 per-work-accounted: 1 +write-amp: int: 0.00x+0 smoothed: 50.50x+1 per-work-accounted: 1 + +# We expect the interval writes will be: +# intAdjustedLSMWrites:780 intAdjustedWriteBytes:4700. +# We expect the interval write-amp model multiplier, assuming 1 constant and 100 +# work count, to be: (4700-100)/780 = 5.8974 = 5.90. +update flushed=1000 ingested=80 admitted=100 write-accounted=500 ingested-accounted=10 disk-writes=5000 ignored-written=250 ignore-ingested-into-L0=50 +---- +interval state: {intWorkCount:100 intL0WriteBytes:1000 intL0IngestedBytes:80 intLSMIngestedBytes:80 intL0WriteAccountedBytes:500 intIngestedAccountedBytes:10 intL0WriteLinearModel:{multiplier:1.3 constant:1} intL0IngestedLinearModel:{multiplier:0.001 constant:1} intIngestedLinearModel:{multiplier:0.5 constant:1} intWriteAmpLinearModel:{multiplier:5.897435897435898 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:250 intL0IgnoredIngestedBytes:50 intAdjustedLSMWrites:780 intAdjustedWriteBytes:4700} +at-admission-tokens: 1 +L0-write-tokens: int: 1.30x+1 smoothed: 1.52x+1 per-work-accounted: 3 +L0-ingest-tokens: int: 0.00x+1 smoothed: 0.38x+1 per-work-accounted: 0 +ingest-tokens: int: 0.50x+1 smoothed: 0.75x+1 per-work-accounted: 0 +write-amp: int: 5.90x+1 smoothed: 28.20x+1 per-work-accounted: 4