From 7cab821f2c2e6c64edf978c77c2ee32eb9a608c6 Mon Sep 17 00:00:00 2001 From: sumeerbhola Date: Fri, 1 Dec 2023 17:56:09 -0500 Subject: [PATCH 1/2] admission: use compaction token lower bound when score is low When the overload score is such that the rate shaping is attempting to give out tokens greater than the compaction bandwidth out of L0, we can have an interaction with Pebble compaction picking which deprioritizes compactions out of L0. This can cause tokens to drop to very low values, including as low as 0 tokens. This is bad for regular traffic, which is typically user facing. This problem was introduced when we started shaping below 20 sub-levels -- previously by the time L0 reached 20 sub-levels there were enough compactions happening out of L0. By using a compaction token lower bound derived from the overall compaction bandwidth observed in the LSM, we ensure that there are sufficient admission tokens, even when the overload score is low. Fixes #115373 Epic: none Release note: None --- pkg/util/admission/io_load_listener.go | 110 +++++++++++--- pkg/util/admission/io_load_listener_test.go | 97 ++++++++++++- pkg/util/admission/testdata/io_load_listener | 145 +++++++++++++------ 3 files changed, 285 insertions(+), 67 deletions(-) diff --git a/pkg/util/admission/io_load_listener.go b/pkg/util/admission/io_load_listener.go index bb4fd75dac41..bfc2e667287c 100644 --- a/pkg/util/admission/io_load_listener.go +++ b/pkg/util/admission/io_load_listener.go @@ -208,6 +208,7 @@ type ioLoadListenerState struct { bytesWritten uint64 incomingLSMBytes uint64 } + cumCompactionStats cumStoreCompactionStats // Exponentially smoothed per interval values. @@ -243,6 +244,56 @@ type ioLoadListenerState struct { elasticDiskBWTokensAllocated int64 } +type cumStoreCompactionStats struct { + writeBytes uint64 + // Not cumulative. This is the number of levels from which bytes will be + // moved out to lower levels via compactions. + numOutLevelsGauge int +} + +func computeCumStoreCompactionStats(m *pebble.Metrics) cumStoreCompactionStats { + var compactedWriteBytes uint64 + baseLevel := -1 + for i := range m.Levels { + compactedWriteBytes += m.Levels[i].BytesCompacted + if i > 0 && m.Levels[i].Size > 0 && baseLevel < 0 { + baseLevel = i + } + } + if baseLevel < 0 { + baseLevel = len(m.Levels) - 1 + } + return cumStoreCompactionStats{ + writeBytes: compactedWriteBytes, + numOutLevelsGauge: len(m.Levels) - baseLevel, + } +} + +// computeL0CompactionTokensLowerBound is used to give some tokens to L0 even +// if compactions out of L0 are not happening because the compaction score is +// not high enough. This allows admission until compaction score is high +// enough, at which point we stop using l0CompactionTokensLowerBound. +// +// See https://github.com/cockroachdb/cockroach/issues/115373 for motivation. +func computeL0CompactionTokensLowerBound( + prev cumStoreCompactionStats, cur cumStoreCompactionStats, +) int64 { + // All levels are expected to have similar write amp, so we expect that each + // will be given equal compaction resources. This isn't actually true if the + // levels have very different scores, but it suffices for the purpose of + // this lower bound. Consider an actual incident where L0 was not + // being compacted because it only had 5 sub-levels, and + // numOutLevelsGauge=6, and the aggregate compaction bandwidth was ~140MB/s + // distributed over 7 concurrent compactions. The aggregated compacted in 15 + // seconds is 2100MB. So we return 2100MB/6 = 350MB of tokens. + intCompactedWriteBytes := int64(cur.writeBytes) - int64(prev.writeBytes) + if intCompactedWriteBytes < 0 { + // Defensive: ignore bogus stats. + intCompactedWriteBytes = 0 + } + return intCompactedWriteBytes / int64(cur.numOutLevelsGauge) +} + const unlimitedTokens = math.MaxInt64 // Token changes are made at a coarse time granularity of 15s since @@ -408,9 +459,11 @@ func (io *ioLoadListener) pebbleMetricsTick(ctx context.Context, metrics StoreMe io.perWorkTokenEstimator.updateEstimates(metrics.Levels[0], cumLSMIngestedBytes, sas) io.adjustTokensResult = adjustTokensResult{ ioLoadListenerState: ioLoadListenerState{ - cumL0AddedBytes: m.Levels[0].BytesFlushed + m.Levels[0].BytesIngested, - curL0Bytes: m.Levels[0].Size, - cumWriteStallCount: metrics.WriteStallCount, + cumL0AddedBytes: m.Levels[0].BytesFlushed + m.Levels[0].BytesIngested, + curL0Bytes: m.Levels[0].Size, + cumWriteStallCount: metrics.WriteStallCount, + cumFlushWriteThroughput: metrics.Flush.WriteThroughput, + cumCompactionStats: computeCumStoreCompactionStats(metrics.Metrics), // No initial limit, i.e, the first interval is unlimited. totalNumByteTokens: unlimitedTokens, totalNumElasticByteTokens: unlimitedTokens, @@ -429,7 +482,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.cumFlushWriteThroughput = metrics.Flush.WriteThroughput io.copyAuxEtcFromPerWorkEstimator() // Assume system starts off unloaded. @@ -554,9 +606,10 @@ func (io *ioLoadListener) adjustTokens(ctx context.Context, metrics StoreMetrics cumDiskBW := io.ioLoadListenerState.diskBW wt := metrics.Flush.WriteThroughput wt.Subtract(io.cumFlushWriteThroughput) + cumCompactionStats := computeCumStoreCompactionStats(metrics.Metrics) res := io.adjustTokensInner(ctx, io.ioLoadListenerState, - metrics.Levels[0], metrics.WriteStallCount, wt, + metrics.Levels[0], metrics.WriteStallCount, cumCompactionStats, wt, L0FileCountOverloadThreshold.Get(&io.settings.SV), L0SubLevelCountOverloadThreshold.Get(&io.settings.SV), L0MinimumSizePerSubLevel.Get(&io.settings.SV), @@ -630,9 +683,10 @@ type adjustTokensAuxComputations struct { intFlushUtilization float64 intWriteStalls int64 - prevTokensUsed int64 - prevTokensUsedByElasticWork int64 - tokenKind tokenKind + prevTokensUsed int64 + prevTokensUsedByElasticWork int64 + tokenKind tokenKind + usedCompactionTokensLowerBound bool perWorkTokensAux perWorkTokensAux doLogFlush bool @@ -650,6 +704,7 @@ func (io *ioLoadListener) adjustTokensInner( prev ioLoadListenerState, l0Metrics pebble.LevelMetrics, cumWriteStallCount int64, + cumCompactionStats cumStoreCompactionStats, flushWriteThroughput pebble.ThroughputMetric, threshNumFiles, threshNumSublevels int64, l0MinSizePerSubLevel int64, @@ -682,6 +737,9 @@ func (io *ioLoadListener) adjustTokensInner( intL0CompactedBytes = 0 } io.l0CompactedBytes.Inc(intL0CompactedBytes) + l0CompactionTokensLowerBound := computeL0CompactionTokensLowerBound( + io.cumCompactionStats, cumCompactionStats) + usedCompactionTokensLowerBound := false const alpha = 0.5 @@ -922,6 +980,10 @@ func (io *ioLoadListener) adjustTokensInner( // smoothedIntL0CompactedBytes at 1, and 2 * smoothedIntL0CompactedBytes // at 0.5. fTotalNumByteTokens = -score*(2*float64(smoothedIntL0CompactedBytes)) + 3*float64(smoothedIntL0CompactedBytes) + if fTotalNumByteTokens < float64(l0CompactionTokensLowerBound) { + fTotalNumByteTokens = float64(l0CompactionTokensLowerBound) + usedCompactionTokensLowerBound = true + } } else { // Medium load. Score in [1, 2). We use linear interpolation from // medium load to overload, to slowly give out fewer tokens as we @@ -950,6 +1012,12 @@ func (io *ioLoadListener) adjustTokensInner( // results show the sublevels hovering around 4, as expected. // // NB: at score >= 1.2 (12 sublevels), there are 0 elastic tokens. + // + // NB: we are not using l0CompactionTokensLowerBound at all for elastic + // tokens. This is because that lower bound is useful primarily when L0 is + // not seeing compactions because a compaction backlog has accumulated in + // other levels. For elastic work, we would rather clear that compaction + // backlog than admit some elastic work. totalNumElasticByteTokens = int64(float64(smoothedIntL0CompactedBytes) * (1.25 - 1.25*(score-0.2))) @@ -974,6 +1042,7 @@ func (io *ioLoadListener) adjustTokensInner( cumL0AddedBytes: cumL0AddedBytes, curL0Bytes: curL0Bytes, cumWriteStallCount: cumWriteStallCount, + cumCompactionStats: cumCompactionStats, smoothedIntL0CompactedBytes: smoothedIntL0CompactedBytes, smoothedCompactionByteTokens: smoothedCompactionByteTokens, smoothedNumFlushTokens: smoothedNumFlushTokens, @@ -986,15 +1055,16 @@ func (io *ioLoadListener) adjustTokensInner( elasticByteTokensAllocated: 0, }, aux: adjustTokensAuxComputations{ - intL0AddedBytes: intL0AddedBytes, - intL0CompactedBytes: intL0CompactedBytes, - intFlushTokens: intFlushTokens, - intFlushUtilization: intFlushUtilization, - intWriteStalls: intWriteStalls, - prevTokensUsed: prev.byteTokensUsed, - prevTokensUsedByElasticWork: prev.byteTokensUsedByElasticWork, - tokenKind: tokenKind, - doLogFlush: doLogFlush, + intL0AddedBytes: intL0AddedBytes, + intL0CompactedBytes: intL0CompactedBytes, + intFlushTokens: intFlushTokens, + intFlushUtilization: intFlushUtilization, + intWriteStalls: intWriteStalls, + prevTokensUsed: prev.byteTokensUsed, + prevTokensUsedByElasticWork: prev.byteTokensUsedByElasticWork, + tokenKind: tokenKind, + usedCompactionTokensLowerBound: usedCompactionTokensLowerBound, + doLogFlush: doLogFlush, }, ioThreshold: ioThreshold, } @@ -1063,7 +1133,11 @@ func (res adjustTokensResult) SafeFormat(p redact.SafePrinter, _ rune) { // NB: res.smoothedCompactionByteTokens is the same as // res.ioLoadListenerState.totalNumByteTokens (printed above) when // res.aux.tokenKind == compactionTokenKind. - p.Printf(" due to L0 growth") + lowerBoundBoolStr := "" + if res.aux.usedCompactionTokensLowerBound { + lowerBoundBoolStr = "(used token lower bound)" + } + p.Printf(" due to L0 growth%s", lowerBoundBoolStr) case flushTokenKind: p.Printf(" due to memtable flush (multiplier %.3f)", res.flushUtilTargetFraction) } diff --git a/pkg/util/admission/io_load_listener_test.go b/pkg/util/admission/io_load_listener_test.go index 97a9375a424c..21f2d00d9fd5 100644 --- a/pkg/util/admission/io_load_listener_test.go +++ b/pkg/util/admission/io_load_listener_test.go @@ -25,6 +25,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/echotest" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/datadriven" @@ -46,11 +48,11 @@ func TestIOLoadListener(t *testing.T) { ctx := context.Background() st := cluster.MakeTestingClusterSettings() - L0MinimumSizePerSubLevel.Override(ctx, &st.SV, 0) datadriven.RunTest(t, datapathutils.TestDataPath(t, "io_load_listener"), func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "init": + L0MinimumSizePerSubLevel.Override(ctx, &st.SV, 0) ioll = &ioLoadListener{ settings: st, kvRequester: req, @@ -140,6 +142,14 @@ func TestIOLoadListener(t *testing.T) { d.ScanArgs(t, "flush-work-sec", &flushWorkSec) d.ScanArgs(t, "flush-idle-sec", &flushIdleSec) } + if d.HasArg("base-level") { + var baseLevel int + d.ScanArgs(t, "base-level", &baseLevel) + metrics.Levels[baseLevel].Size = 1000 + var compactedBytes int + d.ScanArgs(t, "compacted-bytes", &compactedBytes) + metrics.Levels[baseLevel].BytesCompacted = uint64(compactedBytes) + } cumFlushIdle += time.Duration(flushIdleSec) * time.Second cumFlushWork += time.Duration(flushWorkSec) * time.Second @@ -295,8 +305,8 @@ func TestAdjustTokensInnerAndLogging(t *testing.T) { l0TokensProduced: metric.NewCounter(l0TokensProduced), } res := ioll.adjustTokensInner( - ctx, tt.prev, tt.l0Metrics, 12, pebble.ThroughputMetric{}, - 100, 10, 0, 0.50) + ctx, tt.prev, tt.l0Metrics, 12, cumStoreCompactionStats{numOutLevelsGauge: 1}, + pebble.ThroughputMetric{}, 100, 10, 0, 0.50) buf.Printf("%s\n", res) } echotest.Require(t, string(redact.Sprint(buf)), filepath.Join(datapathutils.TestDataPath(t, "format_adjust_tokens_stats.txt"))) @@ -546,3 +556,84 @@ func TestTokenAllocationTicker(t *testing.T) { ticker.adjustmentIntervalStartTime = timeutil.Now().Add(-17 * time.Second) require.Equal(t, 0, int(ticker.remainingTicks())) } + +func TestComputeCumStoreCompactionStats(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + for _, tc := range []struct { + name string + baseLevel int + writeBytes int64 + sizeBytes int64 + expected cumStoreCompactionStats + }{ + { + name: "base-l6-zero", + expected: cumStoreCompactionStats{ + numOutLevelsGauge: 1, + }, + }, + { + name: "base-l6", + baseLevel: 6, + writeBytes: 50, + sizeBytes: 500, + expected: cumStoreCompactionStats{ + writeBytes: 50, + numOutLevelsGauge: 1, + }, + }, + { + name: "base-l2", + baseLevel: 2, + writeBytes: 97, + sizeBytes: 397, + expected: cumStoreCompactionStats{ + writeBytes: 97, + numOutLevelsGauge: 5, + }, + }, + } { + t.Run(tc.name, func(t *testing.T) { + m := pebble.Metrics{} + var cumSizeBytes int64 + var cumWriteBytes uint64 + divisor := int64(len(m.Levels) - tc.baseLevel) + for i := tc.baseLevel; i < len(m.Levels); i++ { + m.Levels[i].Size = tc.sizeBytes / divisor + cumSizeBytes += m.Levels[i].Size + m.Levels[i].BytesCompacted = uint64(tc.writeBytes / divisor) + cumWriteBytes += m.Levels[i].BytesCompacted + } + if cumSizeBytes < tc.sizeBytes { + m.Levels[tc.baseLevel].Size += tc.sizeBytes - cumSizeBytes + } + if cumWriteBytes < uint64(tc.writeBytes) { + m.Levels[tc.baseLevel].BytesCompacted += uint64(tc.writeBytes) - cumWriteBytes + } + require.Equal(t, tc.expected, computeCumStoreCompactionStats(&m)) + }) + } +} + +func TestComputeL0CompactionTokensLowerBound(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + require.Equal(t, int64(1000), computeL0CompactionTokensLowerBound(cumStoreCompactionStats{ + writeBytes: 3000, + numOutLevelsGauge: 1, + }, cumStoreCompactionStats{ + writeBytes: 8000, + numOutLevelsGauge: 5, + })) + + require.Equal(t, int64(0), computeL0CompactionTokensLowerBound(cumStoreCompactionStats{ + writeBytes: 1000, + numOutLevelsGauge: 1, + }, cumStoreCompactionStats{ + writeBytes: 500, + numOutLevelsGauge: 2, + })) +} diff --git a/pkg/util/admission/testdata/io_load_listener b/pkg/util/admission/testdata/io_load_listener index 4ede2cc2b7cf..bc36e098bada 100644 --- a/pkg/util/admission/testdata/io_load_listener +++ b/pkg/util/admission/testdata/io_load_listener @@ -12,7 +12,7 @@ prep-admission-stats admitted=0 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]; admitting all; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} 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 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} 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 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:} 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 @@ -85,7 +85,7 @@ prep-admission-stats admitted=10000 write-bytes=40000 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 0 B [≈0 B]; 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:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:101000} smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:12500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:101000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:101000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:12500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 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:} 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 setAvailableTokens: io-tokens=209(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=209(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -157,7 +157,7 @@ prep-admission-stats admitted=20000 write-bytes=80000 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 0 B [≈0 B]; 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:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:201000} smoothedIntL0CompactedBytes:75000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:201000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:75000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 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:} 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 setAvailableTokens: io-tokens=417(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=417(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -225,7 +225,7 @@ tick: 59, setAvailableTokens: io-tokens=416(elastic 0) elastic-disk-bw-tokens=un 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]; 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:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:201000} smoothedIntL0CompactedBytes:37500 smoothedCompactionByteTokens:21875 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:201000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:37500 smoothedCompactionByteTokens:21875 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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 intWriteStalls:0 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:} 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 setAvailableTokens: io-tokens=365(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=365(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -239,7 +239,7 @@ prep-admission-stats admitted=30000 write-bytes=120000 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 0 B [≈0 B]; 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:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:501000} smoothedIntL0CompactedBytes:168750 smoothedCompactionByteTokens:112187.5 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:501000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:168750 smoothedCompactionByteTokens:112187.5 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 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:} 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 setAvailableTokens: io-tokens=1870(elastic 1055) elastic-disk-bw-tokens=unlimited max-byte-tokens=1870(elastic 1055) max-disk-bw-tokens=unlimited lastTick=false @@ -251,7 +251,7 @@ 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]; 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:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:501000} smoothedIntL0CompactedBytes:84375 smoothedCompactionByteTokens:132031.25 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:501000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:84375 smoothedCompactionByteTokens:132031.25 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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 intWriteStalls:0 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:} 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 setAvailableTokens: io-tokens=2201(elastic 1055) elastic-disk-bw-tokens=unlimited max-byte-tokens=2201(elastic 1055) max-disk-bw-tokens=unlimited lastTick=false @@ -263,7 +263,7 @@ 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]; 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:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:501000} smoothedIntL0CompactedBytes:42187 smoothedCompactionByteTokens:66015.625 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:501000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:42187 smoothedCompactionByteTokens:66015.625 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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 intWriteStalls:0 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:} 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 setAvailableTokens: io-tokens=unlimited(elastic 791) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic 791) max-disk-bw-tokens=unlimited lastTick=false @@ -280,7 +280,7 @@ 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]; admitting all; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} 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 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} 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 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:} 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 @@ -297,7 +297,7 @@ prep-admission-stats admitted=10 write-bytes=130000 ingested-bytes=20000 below-r 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 0 B [≈0 B]; 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:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:201000} smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:201000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 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:} 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 setAvailableTokens: io-tokens=417(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=417(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -312,7 +312,7 @@ 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 0 B [≈0 B]; 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:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:221000} smoothedIntL0CompactedBytes:60000 smoothedCompactionByteTokens:27500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:221000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:221000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:60000 smoothedCompactionByteTokens:27500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 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:} 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 setAvailableTokens: io-tokens=459(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=459(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -326,7 +326,7 @@ 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 0 B [≈0 B]; 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:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:241000} smoothedIntL0CompactedBytes:40000 smoothedCompactionByteTokens:23750 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:241000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:241000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:40000 smoothedCompactionByteTokens:23750 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 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:} 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 setAvailableTokens: io-tokens=396(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=396(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -342,7 +342,7 @@ 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]; admitting all; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} 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 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} 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 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:} 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 @@ -350,7 +350,7 @@ tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=1000 flush-work-sec=2 flush-idle-sec=100 print-only-first-tick=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 9.8 KiB (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 [≈4.9 KiB], flushed 7.3 KiB [≈0 B]; admitting all; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:2000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:5000 smoothedCompactionByteTokens:5000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:10000 intL0CompactedBytes:10000 intFlushTokens:7500 intFlushUtilization:0.0196078431372549 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:10000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:2000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:5000 smoothedCompactionByteTokens:5000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:10000 intL0CompactedBytes:10000 intFlushTokens:7500 intFlushUtilization:0.0196078431372549 intWriteStalls:0 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:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:10000 regularTokensUsed:0 elasticTokensUsed:0}}} 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=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false @@ -359,7 +359,7 @@ setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-token set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=1000 flush-work-sec=2 flush-idle-sec=10 print-only-first-tick=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (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 [≈2.4 KiB], flushed 7.3 KiB [≈7.3 KiB]; admitting 11 KiB (rate 750 B/s) (elastic 11 KiB rate 750 B/s) due to memtable flush (multiplier 1.500) (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:2000 WorkDuration:4000000000 IdleDuration:110000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:2500 smoothedCompactionByteTokens:2500 smoothedNumFlushTokens:7500 flushUtilTargetFraction:1.5 totalNumByteTokens:11250 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:11250 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:7500 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 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:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:2000 WorkDuration:4000000000 IdleDuration:110000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:2500 smoothedCompactionByteTokens:2500 smoothedNumFlushTokens:7500 flushUtilTargetFraction:1.5 totalNumByteTokens:11250 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:11250 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:7500 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 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:} 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=188(elastic 188) elastic-disk-bw-tokens=unlimited max-byte-tokens=188(elastic 188) max-disk-bw-tokens=unlimited lastTick=false @@ -370,7 +370,7 @@ setAvailableTokens: io-tokens=188(elastic 188) elastic-disk-bw-tokens=unlimited set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=1 print-only-first-tick=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (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 [≈1.2 KiB], flushed 73 KiB [≈40 KiB]; admitting 59 KiB (rate 4.0 KiB/s) (elastic 59 KiB rate 4.0 KiB/s) due to memtable flush (multiplier 1.475) (used total: 0 B elastic 0 B); write stalls 1 -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:1 cumFlushWriteThroughput:{Bytes:12000 WorkDuration:6000000000 IdleDuration:120000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:1250 smoothedCompactionByteTokens:1250 smoothedNumFlushTokens:41250 flushUtilTargetFraction:1.475 totalNumByteTokens:60843 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:60843 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:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 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:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:1 cumFlushWriteThroughput:{Bytes:12000 WorkDuration:6000000000 IdleDuration:120000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:1250 smoothedCompactionByteTokens:1250 smoothedNumFlushTokens:41250 flushUtilTargetFraction:1.475 totalNumByteTokens:60843 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:60843 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:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 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:} 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=1015(elastic 1015) elastic-disk-bw-tokens=unlimited max-byte-tokens=1015(elastic 1015) max-disk-bw-tokens=unlimited lastTick=false @@ -381,7 +381,7 @@ setAvailableTokens: io-tokens=1015(elastic 1015) elastic-disk-bw-tokens=unlimite set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=3 print-only-first-tick=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (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 [≈625 B], flushed 73 KiB [≈57 KiB]; admitting 81 KiB (rate 5.4 KiB/s) (elastic 81 KiB rate 5.4 KiB/s) due to memtable flush (multiplier 1.425) (used total: 0 B elastic 0 B); write stalls 2 -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:3 cumFlushWriteThroughput:{Bytes:22000 WorkDuration:8000000000 IdleDuration:130000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:625 smoothedCompactionByteTokens:625 smoothedNumFlushTokens:58125 flushUtilTargetFraction:1.4250000000000003 totalNumByteTokens:82828 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:82828 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:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:2 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 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:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:3 cumFlushWriteThroughput:{Bytes:22000 WorkDuration:8000000000 IdleDuration:130000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:625 smoothedCompactionByteTokens:625 smoothedNumFlushTokens:58125 flushUtilTargetFraction:1.4250000000000003 totalNumByteTokens:82828 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:82828 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:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:2 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 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:} 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=1381(elastic 1381) elastic-disk-bw-tokens=unlimited max-byte-tokens=1381(elastic 1381) max-disk-bw-tokens=unlimited lastTick=false @@ -391,7 +391,7 @@ setAvailableTokens: io-tokens=1381(elastic 1381) elastic-disk-bw-tokens=unlimite set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=8 print-only-first-tick=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (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 [≈312 B], flushed 73 KiB [≈65 KiB]; admitting 88 KiB (rate 5.8 KiB/s) (elastic 88 KiB rate 5.8 KiB/s) due to memtable flush (multiplier 1.350) (used total: 0 B elastic 0 B); write stalls 5 -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:8 cumFlushWriteThroughput:{Bytes:32000 WorkDuration:10000000000 IdleDuration:140000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:312 smoothedCompactionByteTokens:312.5 smoothedNumFlushTokens:66562.5 flushUtilTargetFraction:1.3500000000000005 totalNumByteTokens:89859 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:89859 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:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:5 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 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:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:8 cumFlushWriteThroughput:{Bytes:32000 WorkDuration:10000000000 IdleDuration:140000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:312 smoothedCompactionByteTokens:312.5 smoothedNumFlushTokens:66562.5 flushUtilTargetFraction:1.3500000000000005 totalNumByteTokens:89859 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:89859 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:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:5 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 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:} 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=1498(elastic 1498) elastic-disk-bw-tokens=unlimited max-byte-tokens=1498(elastic 1498) max-disk-bw-tokens=unlimited lastTick=false @@ -400,7 +400,7 @@ setAvailableTokens: io-tokens=1498(elastic 1498) elastic-disk-bw-tokens=unlimite set-state l0-bytes=10000 l0-added-write=11000 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 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 [≈156 B], flushed 73 KiB [≈69 KiB]; admitting 92 KiB (rate 6.1 KiB/s) (elastic 195 B rate 13 B/s) due to memtable flush (multiplier 1.325) (used total: 0 B elastic 0 B); write stalls 1 -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:9 cumFlushWriteThroughput:{Bytes:42000 WorkDuration:12000000000 IdleDuration:150000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:156 smoothedCompactionByteTokens:156.25 smoothedNumFlushTokens:70781.25 flushUtilTargetFraction:1.3250000000000006 totalNumByteTokens:93785 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:195 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:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 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:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:9 cumFlushWriteThroughput:{Bytes:42000 WorkDuration:12000000000 IdleDuration:150000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:156 smoothedCompactionByteTokens:156.25 smoothedNumFlushTokens:70781.25 flushUtilTargetFraction:1.3250000000000006 totalNumByteTokens:93785 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:195 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:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 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:} 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=1564(elastic 4) elastic-disk-bw-tokens=unlimited max-byte-tokens=1564(elastic 4) max-disk-bw-tokens=unlimited lastTick=false @@ -414,7 +414,7 @@ set-min-flush-util percent=130 set-state l0-bytes=10000 l0-added-write=11000 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 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 [≈78 B], flushed 73 KiB [≈71 KiB]; admitting 92 KiB (rate 6.2 KiB/s) (elastic 97 B rate 6 B/s) due to memtable flush (multiplier 1.300) (used total: 0 B elastic 0 B); write stalls 1 -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:10 cumFlushWriteThroughput:{Bytes:52000 WorkDuration:14000000000 IdleDuration:160000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:78 smoothedCompactionByteTokens:78.125 smoothedNumFlushTokens:72890.625 flushUtilTargetFraction:1.3000000000000007 totalNumByteTokens:94757 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:97 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:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 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:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:10 cumFlushWriteThroughput:{Bytes:52000 WorkDuration:14000000000 IdleDuration:160000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:78 smoothedCompactionByteTokens:78.125 smoothedNumFlushTokens:72890.625 flushUtilTargetFraction:1.3000000000000007 totalNumByteTokens:94757 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:97 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:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 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:} 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=1580(elastic 2) elastic-disk-bw-tokens=unlimited max-byte-tokens=1580(elastic 2) max-disk-bw-tokens=unlimited lastTick=false @@ -424,7 +424,7 @@ setAvailableTokens: io-tokens=1580(elastic 2) elastic-disk-bw-tokens=unlimited m set-state l0-bytes=10000 l0-added-write=11000 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 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 [≈39 B], flushed 73 KiB [≈72 KiB]; admitting 94 KiB (rate 6.3 KiB/s) (elastic 48 B rate 3 B/s) due to memtable flush (multiplier 1.300) (used total: 0 B elastic 0 B); write stalls 1 -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:11 cumFlushWriteThroughput:{Bytes:62000 WorkDuration:16000000000 IdleDuration:170000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:39 smoothedCompactionByteTokens:39.0625 smoothedNumFlushTokens:73945.3125 flushUtilTargetFraction:1.3000000000000007 totalNumByteTokens:96128 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:48 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:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 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:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:11 cumFlushWriteThroughput:{Bytes:62000 WorkDuration:16000000000 IdleDuration:170000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:39 smoothedCompactionByteTokens:39.0625 smoothedNumFlushTokens:73945.3125 flushUtilTargetFraction:1.3000000000000007 totalNumByteTokens:96128 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:48 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:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 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:} 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=1603(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=1603(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -439,7 +439,7 @@ set-min-flush-util percent=135 set-state l0-bytes=10000 l0-added-write=11000 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 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 [≈19 B], flushed 73 KiB [≈73 KiB]; admitting 98 KiB (rate 6.5 KiB/s) (elastic 23 B rate 1 B/s) due to memtable flush (multiplier 1.350) (used total: 0 B elastic 0 B); write stalls 1 -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:12 cumFlushWriteThroughput:{Bytes:72000 WorkDuration:18000000000 IdleDuration:180000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:19 smoothedCompactionByteTokens:19.53125 smoothedNumFlushTokens:74472.65625 flushUtilTargetFraction:1.35 totalNumByteTokens:100538 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:23 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:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 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:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:12 cumFlushWriteThroughput:{Bytes:72000 WorkDuration:18000000000 IdleDuration:180000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:19 smoothedCompactionByteTokens:19.53125 smoothedNumFlushTokens:74472.65625 flushUtilTargetFraction:1.35 totalNumByteTokens:100538 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:23 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:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 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:} 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=1676(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=1676(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -448,7 +448,7 @@ setAvailableTokens: io-tokens=1676(elastic 1) elastic-disk-bw-tokens=unlimited m set-state l0-bytes=10000 l0-added-write=11000 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 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 [≈9 B], flushed 73 KiB [≈73 KiB]; admitting elastic 11 B (rate 0 B/s) due to L0 growth; write stalls 1 -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:82000 WorkDuration:20000000000 IdleDuration:280000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:9 smoothedCompactionByteTokens:9.765625 smoothedNumFlushTokens:74472.65625 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:11 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:75000 intFlushUtilization:0.0196078431372549 intWriteStalls:1 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:82000 WorkDuration:20000000000 IdleDuration:280000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:9 smoothedCompactionByteTokens:9.765625 smoothedNumFlushTokens:74472.65625 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:11 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:75000 intFlushUtilization:0.0196078431372549 intWriteStalls:1 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:} 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 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -457,7 +457,7 @@ setAvailableTokens: io-tokens=unlimited(elastic 1) elastic-disk-bw-tokens=unlimi set-state l0-bytes=10000 l0-added-write=11000 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 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 [≈4 B], flushed 73 KiB [≈73 KiB]; admitting 98 KiB (rate 6.6 KiB/s) (elastic 5 B rate 0 B/s) due to memtable flush (multiplier 1.350) (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:92000 WorkDuration:22000000000 IdleDuration:290000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:4 smoothedCompactionByteTokens:4.8828125 smoothedNumFlushTokens:74736.328125 flushUtilTargetFraction:1.35 totalNumByteTokens:100894 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:5 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:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 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:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:92000 WorkDuration:22000000000 IdleDuration:290000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:4 smoothedCompactionByteTokens:4.8828125 smoothedNumFlushTokens:74736.328125 flushUtilTargetFraction:1.35 totalNumByteTokens:100894 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:5 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:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 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:} 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=1682(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=1682(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -467,7 +467,7 @@ setAvailableTokens: io-tokens=1682(elastic 1) elastic-disk-bw-tokens=unlimited m set-state l0-bytes=10000 l0-added-write=11000 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 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 [≈2 B], flushed 73 KiB [≈73 KiB]; admitting 99 KiB (rate 6.6 KiB/s) (elastic 2 B rate 0 B/s) due to memtable flush (multiplier 1.350) (used total: 0 B elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:102000 WorkDuration:24000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:2 smoothedCompactionByteTokens:2.44140625 smoothedNumFlushTokens:74868.1640625 flushUtilTargetFraction:1.35 totalNumByteTokens:101072 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:2 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:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 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:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:102000 WorkDuration:24000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:2 smoothedCompactionByteTokens:2.44140625 smoothedNumFlushTokens:74868.1640625 flushUtilTargetFraction:1.35 totalNumByteTokens:101072 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:2 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:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 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:} 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=1685(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=1685(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -477,7 +477,7 @@ setAvailableTokens: io-tokens=1685(elastic 1) elastic-disk-bw-tokens=unlimited m set-state l0-bytes=10000 l0-added-write=11000 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 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 [≈1 B], flushed 73 KiB [≈73 KiB]; admitting 101 KiB (rate 6.7 KiB/s) (elastic 1 B rate 0 B/s) due to memtable flush (multiplier 1.375) (used total: 197 KiB elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:112000 WorkDuration:26000000000 IdleDuration:310000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:1 smoothedCompactionByteTokens:1.220703125 smoothedNumFlushTokens:74934.08203125 flushUtilTargetFraction:1.375 totalNumByteTokens:103034 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:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:202144 prevTokensUsedByElasticWork:0 tokenKind:1 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:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:112000 WorkDuration:26000000000 IdleDuration:310000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:1 smoothedCompactionByteTokens:1.220703125 smoothedNumFlushTokens:74934.08203125 flushUtilTargetFraction:1.375 totalNumByteTokens:103034 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:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:202144 prevTokensUsedByElasticWork:0 tokenKind:1 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:} 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=1718(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=1718(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -487,7 +487,7 @@ setAvailableTokens: io-tokens=1718(elastic 1) elastic-disk-bw-tokens=unlimited m set-state l0-bytes=10000 l0-added-write=11000 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 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 73 KiB [≈73 KiB]; admitting 102 KiB (rate 6.8 KiB/s) (elastic 1 B rate 0 B/s) due to memtable flush (multiplier 1.400) (used total: 201 KiB elastic 0 B); write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:122000 WorkDuration:28000000000 IdleDuration:320000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0.6103515625 smoothedNumFlushTokens:74967.041015625 flushUtilTargetFraction:1.4 totalNumByteTokens:104953 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:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:206068 prevTokensUsedByElasticWork:0 tokenKind:1 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:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:122000 WorkDuration:28000000000 IdleDuration:320000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0.6103515625 smoothedNumFlushTokens:74967.041015625 flushUtilTargetFraction:1.4 totalNumByteTokens:104953 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:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:206068 prevTokensUsedByElasticWork:0 tokenKind:1 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:} 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=1750(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=1750(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -497,7 +497,7 @@ setAvailableTokens: io-tokens=1750(elastic 1) elastic-disk-bw-tokens=unlimited m set-state l0-bytes=10000 l0-added-write=11000 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 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 73 KiB [≈73 KiB]; admitting 101 KiB (rate 6.7 KiB/s) (elastic 1 B rate 0 B/s) due to memtable flush (multiplier 1.375) (used total: 205 KiB elastic 0 B); write stalls 1 -{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:14 cumFlushWriteThroughput:{Bytes:132000 WorkDuration:30000000000 IdleDuration:330000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0.30517578125 smoothedNumFlushTokens:74983.5205078125 flushUtilTargetFraction:1.375 totalNumByteTokens:103102 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:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:209906 prevTokensUsedByElasticWork:0 tokenKind:1 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:} +{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:14 cumFlushWriteThroughput:{Bytes:132000 WorkDuration:30000000000 IdleDuration:330000000000} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0.30517578125 smoothedNumFlushTokens:74983.5205078125 flushUtilTargetFraction:1.375 totalNumByteTokens:103102 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:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:209906 prevTokensUsedByElasticWork:0 tokenKind:1 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:} 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=1719(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=1719(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -509,13 +509,13 @@ 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 ---- 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]; admitting all; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:0 curL0Bytes:100 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes: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 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:0 curL0Bytes:100 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} 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 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:} 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 ---- 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 0 B [≈0 B]; 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:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:1000000 bytesWritten:2000000 incomingLSMBytes:100000} smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:50000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:100000 curL0Bytes:100 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:1000000 bytesWritten:2000000 incomingLSMBytes:100000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:50000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 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 @@ -525,7 +525,7 @@ setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-token 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 ---- 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 0 B [≈0 B]; 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:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:2000000 bytesWritten:4000000 incomingLSMBytes:200000} smoothedIntL0CompactedBytes:75000 smoothedCompactionByteTokens:75000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:200000 curL0Bytes:100 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:2000000 bytesWritten:4000000 incomingLSMBytes:200000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:75000 smoothedCompactionByteTokens:75000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 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:} 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 @@ -534,7 +534,7 @@ setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-token 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 ---- 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 0 B [≈0 B]; 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:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:4000000 bytesWritten:8000000 incomingLSMBytes:300000} smoothedIntL0CompactedBytes:87500 smoothedCompactionByteTokens:87500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:300000 curL0Bytes:100 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:4000000 bytesWritten:8000000 incomingLSMBytes:300000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:87500 smoothedCompactionByteTokens:87500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 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 @@ -543,7 +543,7 @@ setAvailableTokens: io-tokens=unlimited(elastic 1823) elastic-disk-bw-tokens=105 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 ---- 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 0 B [≈0 B]; 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:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:5000000 bytesWritten:9000000 incomingLSMBytes:400000} smoothedIntL0CompactedBytes:93750 smoothedCompactionByteTokens:93750 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:400000 curL0Bytes:100 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:5000000 bytesWritten:9000000 incomingLSMBytes:400000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:93750 smoothedCompactionByteTokens:93750 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 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:} 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 @@ -560,7 +560,7 @@ 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 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]; admitting all; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} 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 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} 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 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:} 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 @@ -574,7 +574,7 @@ prep-admission-stats admitted=10000 write-bytes=40000 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 0 B [≈0 B]; 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:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:101000} smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:12500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:101000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:101000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:12500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 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:} 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 setAvailableTokens: io-tokens=1(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=209(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -586,13 +586,13 @@ 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]; admitting all; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:0 curL0Bytes:100 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes: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 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:0 curL0Bytes:100 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} 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 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:} 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 0 B [≈0 B]; 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:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:1000000 bytesWritten:2000000 incomingLSMBytes:100000} smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:50000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:100000 curL0Bytes:100 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:1000000 bytesWritten:2000000 incomingLSMBytes:100000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:50000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 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=1 max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=105 lastTick=false @@ -608,7 +608,7 @@ 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]; admitting all; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} 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 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} 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 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:} 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. @@ -619,7 +619,7 @@ 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 0 B [≈0 B]; admitting all; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:501000} smoothedIntL0CompactedBytes:250000 smoothedCompactionByteTokens:250000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:501000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:250000 smoothedCompactionByteTokens:250000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 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:} 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=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false @@ -635,7 +635,7 @@ setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-token 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 0 B [≈0 B]; 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:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:501002} smoothedIntL0CompactedBytes:125001 smoothedCompactionByteTokens:187500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:501002 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:501002} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:125001 smoothedCompactionByteTokens:187500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 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:} 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=13(elastic 3) elastic-disk-bw-tokens=unlimited max-byte-tokens=3125(elastic 521) max-disk-bw-tokens=unlimited lastTick=false @@ -648,7 +648,7 @@ setAvailableTokens: io-tokens=13(elastic 3) elastic-disk-bw-tokens=unlimited max 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]; 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:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:501000} smoothedIntL0CompactedBytes:62500 smoothedCompactionByteTokens:117187.5 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:501000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:62500 smoothedCompactionByteTokens:117187.5 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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 intWriteStalls:0 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:} 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=8(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=1954(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -662,7 +662,7 @@ setAvailableTokens: io-tokens=8(elastic 1) elastic-disk-bw-tokens=unlimited max- 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]; 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:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:501000} smoothedIntL0CompactedBytes:31250 smoothedCompactionByteTokens:66406.25 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:501000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:31250 smoothedCompactionByteTokens:66406.25 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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 intWriteStalls:0 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:} 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=5(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=1107(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -674,7 +674,7 @@ setAvailableTokens: io-tokens=5(elastic 1) elastic-disk-bw-tokens=unlimited max- 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]; 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:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:501000} smoothedIntL0CompactedBytes:15625 smoothedCompactionByteTokens:48828.125 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:501000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:15625 smoothedCompactionByteTokens:48828.125 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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 intWriteStalls:0 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:} 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=4(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=814(elastic 228) max-disk-bw-tokens=unlimited lastTick=false @@ -690,14 +690,14 @@ 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]; admitting all; write stalls 0 -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} 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 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} 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 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:} 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 0 B [≈0 B]; 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:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:101000} smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:12500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:101000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:101000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:12500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 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:} 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=209(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=209(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -709,7 +709,7 @@ set-min-size-per-sub-level size=5000 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 0 B [≈0 B]; 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:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:201000} smoothedIntL0CompactedBytes:75000 smoothedCompactionByteTokens:66250 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:201000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:75000 smoothedCompactionByteTokens:66250 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 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:} 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=1105(elastic 782) elastic-disk-bw-tokens=unlimited max-byte-tokens=1105(elastic 782) max-disk-bw-tokens=unlimited lastTick=false @@ -718,7 +718,60 @@ setAvailableTokens: io-tokens=1105(elastic 782) elastic-disk-bw-tokens=unlimited 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 0 B [≈0 B]; 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:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:260994} smoothedIntL0CompactedBytes:47497 smoothedCompactionByteTokens:56873 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 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:} +{ioLoadListenerState:{cumL0AddedBytes:260994 curL0Bytes:50000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:260994} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} smoothedIntL0CompactedBytes:47497 smoothedCompactionByteTokens:56873 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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:0 intFlushUtilization:0 intWriteStalls:0 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:} 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=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. +init +---- + +prep-admission-stats admitted=0 +---- +{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}} + +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]; admitting all; write stalls 0 +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} 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 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:} +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]; 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:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} cumCompactionStats:{writeBytes:1000000 numOutLevelsGauge:2} smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:250000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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 intWriteStalls:0 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:} +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=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]; 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:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} cumCompactionStats:{writeBytes:2000000 numOutLevelsGauge:2} smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:375000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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 intWriteStalls:0 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:} +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=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]; 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:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} cumCompactionStats:{writeBytes:3000000 numOutLevelsGauge:2} smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:437500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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 intWriteStalls:0 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:} +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=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 +# point we are admitting what we can compact out, and if 10 sub-levels is not +# sufficient to prioritize L0 compactions then we ought to be throttling +# 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]; 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:0 WorkDuration:0 IdleDuration:0} diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} cumCompactionStats:{writeBytes:3000000 numOutLevelsGauge:2} smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:218750 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 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 intWriteStalls:0 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:} +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=3646(elastic 1) elastic-disk-bw-tokens=unlimited max-byte-tokens=3646(elastic 1) max-disk-bw-tokens=unlimited lastTick=false From 0eb8c0db3491966cbce455d74cba6e0d103d975a Mon Sep 17 00:00:00 2001 From: David Taylor Date: Sun, 3 Dec 2023 15:23:52 +0000 Subject: [PATCH 2/2] backup: use ingestion writer for backup data ssts Release note: none. Epic: none. --- pkg/ccl/backupccl/file_sst_sink.go | 3 ++- pkg/storage/sst_writer.go | 6 ++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/pkg/ccl/backupccl/file_sst_sink.go b/pkg/ccl/backupccl/file_sst_sink.go index b8653609356b..312ff458693d 100644 --- a/pkg/ccl/backupccl/file_sst_sink.go +++ b/pkg/ccl/backupccl/file_sst_sink.go @@ -175,7 +175,8 @@ func (s *fileSSTSink) open(ctx context.Context) error { } s.out = e } - s.sst = storage.MakeBackupSSTWriter(ctx, s.dest.Settings(), s.out) + // TODO(dt): make ExternalStorage.Writer return objstorage.Writable. + s.sst = storage.MakeIngestionSSTWriter(ctx, s.dest.Settings(), storage.NoopFinishAbortWritable(s.out)) return nil } diff --git a/pkg/storage/sst_writer.go b/pkg/storage/sst_writer.go index 0d9bb826b21a..3f4fffea7cb7 100644 --- a/pkg/storage/sst_writer.go +++ b/pkg/storage/sst_writer.go @@ -40,6 +40,12 @@ var _ Writer = &SSTWriter{} var _ ExportWriter = &SSTWriter{} var _ InternalWriter = &SSTWriter{} +// NoopFinishAbortWritable wraps an io.Writer to make a objstorage.Writable that +// will ignore Finish and Abort calls. +func NoopFinishAbortWritable(w io.Writer) objstorage.Writable { + return &noopFinishAbort{Writer: w} +} + // noopFinishAbort is used to wrap io.Writers for sstable.Writer. type noopFinishAbort struct { io.Writer