From c40959dbef5fdfe35496cd719457ac679b8da8ed Mon Sep 17 00:00:00 2001 From: Aaditya Sondhi <20070511+aadityasondhi@users.noreply.github.com> Date: Fri, 27 Sep 2024 07:54:31 -0400 Subject: [PATCH] admission, kvserver: snapshot integration for disk bandwidth MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This patch integrates raft snapshot ingestion with the disk write mechanism in admission control. The following internal machinery changes were made to make that possible: - `SnapshotQueue` was added as an implementation of the `requester` interface. Internally, it is a simple FIFO queue unlike the other work queue, since we can make the assumption that all snapshots are of the same priority and are processed as system tenant requests. - A new `kvStoreTokenChildGranter` was created to grant tokens to snapshot requests. - We now have a `StoreWorkType` that differentiates `regular`, `elastic`, and `snapshot` work for the store granters. This was necessary because snapshots do not incur the same write-amp as the other work types – they land into L6 of the LSM due to excises. We also only want these requests to be subject to pacing based on disk bandwidth. - We now prioritize store writes in the following order: `regular`, `snapshot`, `elastic`. - The `demuxHandle` of the `GrantCoordinator` now uses `StoreWorkType`. The integration point for the `SnapshotQueue` is in `Receive()` where we use a pacing mechanism to process incoming snapshots. Snapshots are subject to `snapshotBurstSize` amounts of disk writes before asking for further admission of the same size. The `multiSSTWriter` uses Pebble's SST size estimates to maintain a running count of disk writes incurred by the snapshot ingest. Once the SST is finalized we deduct/return further tokens. Closes #120708. Release note (ops change): Admission Control now has an integration for pacing snapshot ingest traffic based on disk bandwidth. `kvadmission.store.snapshot_ingest_bandwidth_control.enabled` is used to turn on this integration. Note that it requires provisioned bandwidth to be set for the store (or cluster through the cluster setting) for it to take effect. --- docs/generated/metrics/metrics.html | 1 + docs/generated/settings/settings.html | 1 + .../admission_control_snapshot_overload_io.go | 47 ++- pkg/kv/kvserver/kvadmission/kvadmission.go | 11 + pkg/kv/kvserver/replica_raftstorage.go | 4 + pkg/kv/kvserver/store_snapshot.go | 38 +++ pkg/storage/sst_writer.go | 7 + pkg/util/admission/BUILD.bazel | 3 + pkg/util/admission/admission.go | 2 +- pkg/util/admission/admissionpb/admissionpb.go | 31 ++ pkg/util/admission/disk_bandwidth.go | 28 +- pkg/util/admission/disk_bandwidth_test.go | 10 +- pkg/util/admission/grant_coordinator.go | 30 +- pkg/util/admission/granter.go | 162 ++++++---- pkg/util/admission/granter_test.go | 24 +- pkg/util/admission/io_load_listener_test.go | 8 +- pkg/util/admission/snapshot_queue.go | 305 ++++++++++++++++++ pkg/util/admission/snapshot_queue_test.go | 208 ++++++++++++ .../admission/testdata/disk_bandwidth_limiter | 14 +- pkg/util/admission/testdata/granter | 95 ++++++ pkg/util/admission/testdata/io_load_listener | 110 +++---- pkg/util/admission/testdata/snapshot_queue | 71 ++++ 22 files changed, 1043 insertions(+), 167 deletions(-) create mode 100644 pkg/util/admission/snapshot_queue.go create mode 100644 pkg/util/admission/snapshot_queue_test.go create mode 100644 pkg/util/admission/testdata/snapshot_queue diff --git a/docs/generated/metrics/metrics.html b/docs/generated/metrics/metrics.html index 3f0d347fa423..7c1233138483 100644 --- a/docs/generated/metrics/metrics.html +++ b/docs/generated/metrics/metrics.html @@ -131,6 +131,7 @@ STORAGEadmission.wait_durations.kv.high-priWait time durations for requests that waitedWait time DurationHISTOGRAMNANOSECONDSAVGNONE STORAGEadmission.wait_durations.kv.locking-normal-priWait time durations for requests that waitedWait time DurationHISTOGRAMNANOSECONDSAVGNONE STORAGEadmission.wait_durations.kv.normal-priWait time durations for requests that waitedWait time DurationHISTOGRAMNANOSECONDSAVGNONE +STORAGEadmission.wait_durations.snapshot_ingestWait time for snapshot ingest requests that waitedWait time DurationHISTOGRAMNANOSECONDSAVGNONE STORAGEadmission.wait_durations.sql-kv-responseWait time durations for requests that waitedWait time DurationHISTOGRAMNANOSECONDSAVGNONE STORAGEadmission.wait_durations.sql-kv-response.locking-normal-priWait time durations for requests that waitedWait time DurationHISTOGRAMNANOSECONDSAVGNONE STORAGEadmission.wait_durations.sql-kv-response.normal-priWait time durations for requests that waitedWait time DurationHISTOGRAMNANOSECONDSAVGNONE diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 2e395a64237a..3b0f4a710b04 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -128,6 +128,7 @@
kv.transaction.write_pipelining.enabled
(alias: kv.transaction.write_pipelining_enabled)
booleantrueif enabled, transactional writes are pipelined through Raft consensusServerless/Dedicated/Self-Hosted
kv.transaction.write_pipelining.max_batch_size
(alias: kv.transaction.write_pipelining_max_batch_size)
integer128if non-zero, defines that maximum size batch that will be pipelined through Raft consensusServerless/Dedicated/Self-Hosted
kvadmission.store.provisioned_bandwidth
byte size0 Bif set to a non-zero value, this is used as the provisioned bandwidth (in bytes/s), for each store. It can be overridden on a per-store basis using the --store flag. Note that setting the provisioned bandwidth to a positive value may enable disk bandwidth based admission control, since admission.disk_bandwidth_tokens.elastic.enabled defaults to trueDedicated/Self-Hosted +
kvadmission.store.snapshot_ingest_bandwidth_control.enabled
booleanfalseif set to true, snapshot ingests will be subject to disk write control in ACDedicated/Self-Hosted
obs.tablemetadata.automatic_updates.enabled
booleanfalseenables automatic updates of the table metadata cache system.table_metadataServerless/Dedicated/Self-Hosted
obs.tablemetadata.data_valid_duration
duration20m0sthe duration for which the data in system.table_metadata is considered validServerless/Dedicated/Self-Hosted
schedules.backup.gc_protection.enabled
booleantrueenable chaining of GC protection across backups run as part of a scheduleServerless/Dedicated/Self-Hosted diff --git a/pkg/cmd/roachtest/tests/admission_control_snapshot_overload_io.go b/pkg/cmd/roachtest/tests/admission_control_snapshot_overload_io.go index 87af5be32b62..cebff3f391b2 100644 --- a/pkg/cmd/roachtest/tests/admission_control_snapshot_overload_io.go +++ b/pkg/cmd/roachtest/tests/admission_control_snapshot_overload_io.go @@ -72,6 +72,8 @@ func registerSnapshotOverloadIO(r registry.Registry) { // sstables should ingest into L6. limitCompactionConcurrency: true, limitDiskBandwidth: false, + readPercent: 75, + workloadBlockBytes: 12288, })) // This tests the behaviour of snpashot ingestion in bandwidth constrained @@ -81,6 +83,8 @@ func registerSnapshotOverloadIO(r registry.Registry) { volumeSize: 1000, limitCompactionConcurrency: false, limitDiskBandwidth: true, + readPercent: 20, + workloadBlockBytes: 1024, })) } @@ -89,6 +93,8 @@ type admissionControlSnapshotOverloadIOOpts struct { volumeSize int limitCompactionConcurrency bool limitDiskBandwidth bool + readPercent int + workloadBlockBytes int } func runAdmissionControlSnapshotOverloadIO( @@ -143,18 +149,6 @@ func runAdmissionControlSnapshotOverloadIO( } } - if cfg.limitDiskBandwidth { - const bandwidthLimit = 128 - dataDir := "/mnt/data1" - if err := setBandwidthLimit(ctx, t, c, c.CRDBNodes(), "wbps", bandwidthLimit<<20 /* 128MiB */, false, dataDir); err != nil { - t.Fatal(err) - } - if _, err := db.ExecContext( - ctx, fmt.Sprintf("SET CLUSTER SETTING kvadmission.store.provisioned_bandwidth = '%dMiB'", bandwidthLimit)); err != nil { - t.Fatalf("failed to set kvadmission.store.provisioned_bandwidth: %v", err) - } - } - // Setup the prometheus instance and client. t.Status(fmt.Sprintf("setting up prometheus/grafana (<%s)", 2*time.Minute)) var statCollector clusterstats.StatCollector @@ -181,15 +175,38 @@ func runAdmissionControlSnapshotOverloadIO( "./cockroach workload init kv --drop --insert-count=40000000 "+ "--max-block-bytes=12288 --min-block-bytes=12288 {pgurl:1-3}") + // Now set disk bandwidth limits + if cfg.limitDiskBandwidth { + const bandwidthLimit = 128 + dataDir := "/mnt/data1" + if err := setBandwidthLimit(ctx, t, c, c.CRDBNodes(), "wbps", bandwidthLimit<<20 /* 128MiB */, false, dataDir); err != nil { + t.Fatal(err) + } + if _, err := db.ExecContext( + ctx, fmt.Sprintf("SET CLUSTER SETTING kvadmission.store.provisioned_bandwidth = '%dMiB'", bandwidthLimit)); err != nil { + t.Fatalf("failed to set kvadmission.store.provisioned_bandwidth: %v", err) + } + if _, err := db.ExecContext( + ctx, "SET CLUSTER SETTING kvadmission.store.snapshot_ingest_bandwidth_control.enabled = 'true'"); err != nil { + t.Fatalf("failed to set kvadmission.store.snapshot_ingest_bandwidth_control.enabled: %v", err) + } + } + t.Status(fmt.Sprintf("starting kv workload thread (<%s)", time.Minute)) m := c.NewMonitor(ctx, c.CRDBNodes()) m.Go(func(ctx context.Context) error { c.Run(ctx, option.WithNodes(c.WorkloadNode()), fmt.Sprintf("./cockroach workload run kv --tolerate-errors "+ - "--splits=1000 --histograms=%s/stats.json --read-percent=75 "+ - "--max-rate=600 --max-block-bytes=12288 --min-block-bytes=12288 "+ + "--splits=1000 --histograms=%s/stats.json --read-percent=%d "+ + "--max-rate=600 --max-block-bytes=%d --min-block-bytes=%d "+ "--concurrency=4000 --duration=%s {pgurl:1-2}", - t.PerfArtifactsDir(), (6*time.Hour).String())) + t.PerfArtifactsDir(), + cfg.readPercent, + cfg.workloadBlockBytes, + cfg.workloadBlockBytes, + (6*time.Hour).String(), + ), + ) return nil }) diff --git a/pkg/kv/kvserver/kvadmission/kvadmission.go b/pkg/kv/kvserver/kvadmission/kvadmission.go index 615601148b5a..1923164782ce 100644 --- a/pkg/kv/kvserver/kvadmission/kvadmission.go +++ b/pkg/kv/kvserver/kvadmission/kvadmission.go @@ -186,6 +186,9 @@ type Controller interface { _ context.Context, _ roachpb.TenantID, _ roachpb.StoreID, _ roachpb.RangeID, _ roachpb.ReplicaID, leaderTerm uint64, _ raftpb.Entry) replica_rac2.ACWorkQueue + // GetSnapshotQueue returns the SnapshotQueue which is used for ingesting raft + // snapshots. + GetSnapshotQueue(roachpb.StoreID) *admission.SnapshotQueue } // TenantWeightProvider can be periodically asked to provide the tenant @@ -707,6 +710,14 @@ func (n *controllerImpl) Admit(ctx context.Context, entry replica_rac2.EntryForA return true } +func (n *controllerImpl) GetSnapshotQueue(storeID roachpb.StoreID) *admission.SnapshotQueue { + sq := n.storeGrantCoords.TryGetSnapshotQueueForStore(storeID) + if sq == nil { + return nil + } + return sq.(*admission.SnapshotQueue) +} + // FollowerStoreWriteBytes captures stats about writes done to a store by a // replica that is not the leaseholder. These are used for admission control. type FollowerStoreWriteBytes struct { diff --git a/pkg/kv/kvserver/replica_raftstorage.go b/pkg/kv/kvserver/replica_raftstorage.go index ec439afe40f4..76012a1fc601 100644 --- a/pkg/kv/kvserver/replica_raftstorage.go +++ b/pkg/kv/kvserver/replica_raftstorage.go @@ -806,6 +806,10 @@ func (r *Replica) applySnapshot( writeBytes = uint64(inSnap.SSTSize) } } + // The "ignored" here is to ignore the writes to create the AC linear models + // for LSM writes. Since these writes typically correspond to actual writes + // onto the disk, we account for them separately in + // kvBatchSnapshotStrategy.Receive(). if r.store.cfg.KVAdmissionController != nil { r.store.cfg.KVAdmissionController.SnapshotIngestedOrWritten( r.store.StoreID(), ingestStats, writeBytes) diff --git a/pkg/kv/kvserver/store_snapshot.go b/pkg/kv/kvserver/store_snapshot.go index d1bd4156012c..8fdbda682280 100644 --- a/pkg/kv/kvserver/store_snapshot.go +++ b/pkg/kv/kvserver/store_snapshot.go @@ -31,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/admission" "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" @@ -159,6 +160,8 @@ type multiSSTWriter struct { dataSize int64 // The total size of the SSTs. sstSize int64 + // Incremental count of number of bytes written to disk. + writeBytes int64 // if skipClearForMVCCSpan is true, the MVCC span is not ClearEngineRange()d in // the same sstable. We rely on the caller to take care of clearing this span // through a different process (eg. IngestAndExcise on pebble). Note that @@ -340,6 +343,8 @@ func (msstw *multiSSTWriter) finalizeSST(ctx context.Context, nextKey *storage.E metaEndKey, nextKey) } } + // Account for any additional bytes written other than the KV data. + msstw.writeBytes += int64(msstw.currSST.Meta.Size) - msstw.currSST.DataSize msstw.dataSize += msstw.currSST.DataSize msstw.sstSize += int64(msstw.currSST.Meta.Size) msstw.currSpan++ @@ -424,9 +429,11 @@ func (msstw *multiSSTWriter) Put(ctx context.Context, key storage.EngineKey, val if err := msstw.rolloverSST(ctx, key, key); err != nil { return err } + prevWriteBytes := msstw.currSST.EstimatedSize() if err := msstw.currSST.PutEngineKey(key, value); err != nil { return errors.Wrap(err, "failed to put in sst") } + msstw.writeBytes += int64(msstw.currSST.EstimatedSize() - prevWriteBytes) return nil } @@ -440,6 +447,7 @@ func (msstw *multiSSTWriter) PutInternalPointKey( if err := msstw.rolloverSST(ctx, decodedKey, decodedKey); err != nil { return err } + prevWriteBytes := msstw.currSST.EstimatedSize() var err error switch kind { case pebble.InternalKeyKindSet, pebble.InternalKeyKindSetWithDelete: @@ -452,6 +460,7 @@ func (msstw *multiSSTWriter) PutInternalPointKey( if err != nil { return errors.Wrap(err, "failed to put in sst") } + msstw.writeBytes += int64(msstw.currSST.EstimatedSize() - prevWriteBytes) return nil } @@ -484,9 +493,11 @@ func (msstw *multiSSTWriter) PutInternalRangeDelete(ctx context.Context, start, if err := msstw.rolloverSST(ctx, decodedStart, decodedEnd); err != nil { return err } + prevWriteBytes := msstw.currSST.EstimatedSize() if err := msstw.currSST.ClearRawEncodedRange(start, end); err != nil { return errors.Wrap(err, "failed to put range delete in sst") } + msstw.writeBytes += int64(msstw.currSST.EstimatedSize() - prevWriteBytes) return nil } @@ -503,9 +514,11 @@ func (msstw *multiSSTWriter) PutInternalRangeKey( if err := msstw.rolloverSST(ctx, decodedStart, decodedEnd); err != nil { return err } + prevWriteBytes := msstw.currSST.EstimatedSize() if err := msstw.currSST.PutInternalRangeKey(start, end, key); err != nil { return errors.Wrap(err, "failed to put range key in sst") } + msstw.writeBytes += int64(msstw.currSST.EstimatedSize() - prevWriteBytes) return nil } @@ -519,12 +532,15 @@ func (msstw *multiSSTWriter) PutRangeKey( return err } if msstw.skipClearForMVCCSpan { + prevWriteBytes := msstw.currSST.EstimatedSize() // Skip the fragmenter. See the comment in skipClearForMVCCSpan. if err := msstw.currSST.PutEngineRangeKey(start, end, suffix, value); err != nil { return errors.Wrap(err, "failed to put range key in sst") } + msstw.writeBytes += int64(msstw.currSST.EstimatedSize() - prevWriteBytes) return nil } + startKey, endKey := storage.EngineKey{Key: start}.Encode(), storage.EngineKey{Key: end}.Encode() startTrailer := pebble.MakeInternalKeyTrailer(0, pebble.InternalKeyKindRangeKeySet) msstw.rangeKeyFrag.Add(rangekey.Span{ @@ -721,6 +737,14 @@ func (kvSS *kvBatchSnapshotStrategy) Receive( var sharedSSTs []pebble.SharedSSTMeta var externalSSTs []pebble.ExternalFile + var prevWriteBytes int64 + + snapshotQ := s.cfg.KVAdmissionController.GetSnapshotQueue(s.StoreID()) + // Using a nil pacer is effectively a noop if snapshot control is disabled. + var pacer *admission.SnapshotPacer = nil + if admission.DiskBandwidthForSnapshotIngest.Get(&s.cfg.Settings.SV) { + pacer = admission.NewSnapshotPacer(snapshotQ, s.StoreID()) + } for { timingTag.start("recv") @@ -755,6 +779,14 @@ func (kvSS *kvBatchSnapshotStrategy) Receive( for batchReader.Next() { // TODO(lyang24): maybe avoid decoding engine key twice. // msstw calls (i.e. PutInternalPointKey) can use the decoded engine key here as input. + + writeBytes := msstw.writeBytes - prevWriteBytes + // Calling nil pacer is a noop. + if err := pacer.Pace(ctx, writeBytes, false /* final */); err != nil { + return noSnap, errors.Wrapf(err, "snapshot admission pacer") + } + prevWriteBytes = msstw.writeBytes + ek, err := batchReader.EngineKey() if err != nil { return noSnap, err @@ -883,6 +915,12 @@ func (kvSS *kvBatchSnapshotStrategy) Receive( if err != nil { return noSnap, errors.Wrapf(err, "finishing sst for raft snapshot") } + // Defensive call to account for any discrepancies. The SST sizes should + // have been updated upon closing. + additionalWrites := sstSize - msstw.writeBytes + if err := pacer.Pace(ctx, additionalWrites, true /* final */); err != nil { + return noSnap, errors.Wrapf(err, "snapshot admission pacer") + } msstw.Close() timingTag.stop("sst") log.Eventf(ctx, "all data received from snapshot and all SSTs were finalized") diff --git a/pkg/storage/sst_writer.go b/pkg/storage/sst_writer.go index 3ba97567bbba..1eeb89daf493 100644 --- a/pkg/storage/sst_writer.go +++ b/pkg/storage/sst_writer.go @@ -560,6 +560,13 @@ func (fw *SSTWriter) BufferedSize() int { return 0 } +// EstimatedSize returns the underlying RawWriter's estimated size. Note that +// this size is an estimate as if the writer were to be closed at the time of +// calling. +func (fw *SSTWriter) EstimatedSize() uint64 { + return fw.fw.Raw().EstimatedSize() +} + // MemObject is an in-memory implementation of objstorage.Writable, intended // use with SSTWriter. type MemObject struct { diff --git a/pkg/util/admission/BUILD.bazel b/pkg/util/admission/BUILD.bazel index a64d1c77eeda..8182456031f3 100644 --- a/pkg/util/admission/BUILD.bazel +++ b/pkg/util/admission/BUILD.bazel @@ -15,6 +15,7 @@ go_library( "pacer.go", "scheduler_latency_listener.go", "sequencer.go", + "snapshot_queue.go", "sql_cpu_overload_indicator.go", "store_token_estimation.go", "testing_knobs.go", @@ -36,6 +37,7 @@ go_library( "//pkg/util/humanizeutil", "//pkg/util/log", "//pkg/util/metric", + "//pkg/util/queue", "//pkg/util/schedulerlatency", "//pkg/util/syncutil", "//pkg/util/timeutil", @@ -60,6 +62,7 @@ go_test( "replicated_write_admission_test.go", "scheduler_latency_listener_test.go", "sequencer_test.go", + "snapshot_queue_test.go", "store_token_estimation_test.go", "tokens_linear_model_test.go", "work_queue_test.go", diff --git a/pkg/util/admission/admission.go b/pkg/util/admission/admission.go index c88dce7f829d..b07c622ef40a 100644 --- a/pkg/util/admission/admission.go +++ b/pkg/util/admission/admission.go @@ -280,7 +280,7 @@ type granterWithIOTokens interface { ) (tokensUsed int64, tokensUsedByElasticWork int64) // getDiskTokensUsedAndReset returns the disk bandwidth tokens used since the // last such call. - getDiskTokensUsedAndReset() [admissionpb.NumWorkClasses]diskTokens + getDiskTokensUsedAndReset() [admissionpb.NumStoreWorkTypes]diskTokens // setLinearModels supplies the models to use when storeWriteDone or // storeReplicatedWorkAdmittedLocked is called, to adjust token consumption. // Note that these models are not used for token adjustment at admission diff --git a/pkg/util/admission/admissionpb/admissionpb.go b/pkg/util/admission/admissionpb/admissionpb.go index 86bb5ac3b18d..5c608add4730 100644 --- a/pkg/util/admission/admissionpb/admissionpb.go +++ b/pkg/util/admission/admissionpb/admissionpb.go @@ -189,6 +189,37 @@ const ( NumWorkClasses ) +// StoreWorkType represents the type of work, +type StoreWorkType int8 + +const ( + // RegularStoreWorkType is for type of store-specific work that corresponds to + // RegularWorkClass. + RegularStoreWorkType StoreWorkType = iota + // SnapshotIngestStoreWorkType is for snapshot work type. It is classified as + // ElasticWorkClass, but is prioritized higher than other work of that class. + SnapshotIngestStoreWorkType = 1 + // ElasticStoreWorkType is for store-specific work that corresponds to + // ElasticWorkClass, excluding SnapshotIngestStoreWorkType. + ElasticStoreWorkType = 2 + // NumStoreWorkTypes is the number of store work types. + NumStoreWorkTypes = 3 +) + +// WorkClassFromStoreWorkType translates StoreWorkType to a WorkClass +func WorkClassFromStoreWorkType(workType StoreWorkType) WorkClass { + var class WorkClass + switch workType { + case RegularStoreWorkType: + class = RegularWorkClass + case ElasticStoreWorkType: + class = ElasticWorkClass + case SnapshotIngestStoreWorkType: + class = ElasticWorkClass + } + return class +} + // WorkClassFromPri translates a WorkPriority to its given WorkClass. func WorkClassFromPri(pri WorkPriority) WorkClass { class := RegularWorkClass diff --git a/pkg/util/admission/disk_bandwidth.go b/pkg/util/admission/disk_bandwidth.go index 316d48ac16e9..66f1647bf88b 100644 --- a/pkg/util/admission/disk_bandwidth.go +++ b/pkg/util/admission/disk_bandwidth.go @@ -97,7 +97,7 @@ type intervalDiskLoadInfo struct { type diskBandwidthLimiterState struct { tokens diskTokens prevTokens diskTokens - usedTokens [admissionpb.NumWorkClasses]diskTokens + usedTokens [admissionpb.NumStoreWorkTypes]diskTokens diskBWUtil float64 diskLoad intervalDiskLoadInfo } @@ -125,7 +125,7 @@ type diskTokens struct { // computeElasticTokens is called every adjustmentInterval. func (d *diskBandwidthLimiter) computeElasticTokens( - id intervalDiskLoadInfo, usedTokens [admissionpb.NumWorkClasses]diskTokens, + id intervalDiskLoadInfo, usedTokens [admissionpb.NumStoreWorkTypes]diskTokens, ) diskTokens { // We are using disk read bytes over the previous adjustment interval as a // proxy for future reads. It is a somewhat bad proxy, but for now we are ok @@ -143,7 +143,7 @@ func (d *diskBandwidthLimiter) computeElasticTokens( // elastic writes completely due to out-sized reads from above. diskWriteTokens = int64(math.Max(0, float64(diskWriteTokens))) - totalUsedTokens := sumDiskTokens(usedTokens[admissionpb.ElasticWorkClass], usedTokens[admissionpb.RegularWorkClass]) + totalUsedTokens := sumDiskTokens(usedTokens) tokens := diskTokens{ readByteTokens: 0, writeByteTokens: diskWriteTokens, @@ -164,10 +164,12 @@ func (d *diskBandwidthLimiter) computeElasticTokens( func (d *diskBandwidthLimiter) SafeFormat(p redact.SafePrinter, _ rune) { ib := humanizeutil.IBytes p.Printf("diskBandwidthLimiter (tokenUtilization %.2f, tokensUsed (elastic %s, "+ - "regular %s) tokens (write %s (prev %s)), writeBW %s/s, readBW %s/s, provisioned %s/s)", + "snapshot %s, regular %s) tokens (write %s (prev %s)), writeBW %s/s, readBW %s/s, "+ + "provisioned %s/s)", d.state.diskBWUtil, - ib(d.state.usedTokens[admissionpb.ElasticWorkClass].writeByteTokens), - ib(d.state.usedTokens[admissionpb.RegularWorkClass].writeByteTokens), + ib(d.state.usedTokens[admissionpb.ElasticStoreWorkType].writeByteTokens), + ib(d.state.usedTokens[admissionpb.SnapshotIngestStoreWorkType].writeByteTokens), + ib(d.state.usedTokens[admissionpb.RegularStoreWorkType].writeByteTokens), ib(d.state.tokens.writeByteTokens), ib(d.state.prevTokens.writeByteTokens), ib(d.state.diskLoad.intWriteBytes/adjustmentInterval), @@ -180,11 +182,13 @@ func (d *diskBandwidthLimiter) String() string { return redact.StringWithoutMarkers(d) } -func sumDiskTokens(l diskTokens, r diskTokens) diskTokens { - return diskTokens{ - readByteTokens: l.readByteTokens + r.readByteTokens, - writeByteTokens: l.writeByteTokens + r.writeByteTokens, - readIOPSTokens: l.readIOPSTokens + r.readIOPSTokens, - writeIOPSTokens: l.writeIOPSTokens + r.writeIOPSTokens, +func sumDiskTokens(tokens [admissionpb.NumStoreWorkTypes]diskTokens) diskTokens { + var sumTokens diskTokens + for i := 0; i < admissionpb.NumStoreWorkTypes; i++ { + sumTokens.readByteTokens += tokens[i].readByteTokens + sumTokens.writeByteTokens += tokens[i].writeByteTokens + sumTokens.readIOPSTokens += tokens[i].readIOPSTokens + sumTokens.writeIOPSTokens += tokens[i].writeIOPSTokens } + return sumTokens } diff --git a/pkg/util/admission/disk_bandwidth_test.go b/pkg/util/admission/disk_bandwidth_test.go index eb8c884279d6..e23233176267 100644 --- a/pkg/util/admission/disk_bandwidth_test.go +++ b/pkg/util/admission/disk_bandwidth_test.go @@ -49,12 +49,14 @@ func TestDiskBandwidthLimiter(t *testing.T) { intProvisionedDiskBytes: int64(intProvisionedBytes), elasticBandwidthMaxUtil: 0.9, } - var regularTokensUsed, elasticTokensUsed int64 + var regularTokensUsed, snapshotTokensUsed, elasticTokensUsed int64 d.ScanArgs(t, "regular-tokens-used", ®ularTokensUsed) + d.ScanArgs(t, "snapshot-tokens-used", &snapshotTokensUsed) d.ScanArgs(t, "elastic-tokens-used", &elasticTokensUsed) - usedTokens := [admissionpb.NumWorkClasses]diskTokens{ - {writeByteTokens: regularTokensUsed}, // regular - {writeByteTokens: elasticTokensUsed}, // elastic + usedTokens := [admissionpb.NumStoreWorkTypes]diskTokens{ + {writeByteTokens: regularTokensUsed}, // regular + {writeByteTokens: snapshotTokensUsed}, // snapshot + {writeByteTokens: elasticTokensUsed}, // elastic } dbl.computeElasticTokens(diskLoad, usedTokens) diff --git a/pkg/util/admission/grant_coordinator.go b/pkg/util/admission/grant_coordinator.go index 0c5d23f0e112..44a1e8a09afa 100644 --- a/pkg/util/admission/grant_coordinator.go +++ b/pkg/util/admission/grant_coordinator.go @@ -60,7 +60,8 @@ type StoreGrantCoordinators struct { l0TokensProduced *metric.Counter // These metrics are shared by WorkQueues across stores. - workQueueMetrics [admissionpb.NumWorkClasses]*WorkQueueMetrics + workQueueMetrics [admissionpb.NumWorkClasses]*WorkQueueMetrics + snapshotQueueMetrics *SnapshotMetrics gcMap syncutil.Map[roachpb.StoreID, GrantCoordinator] // numStores is used to track the number of stores which have been added @@ -181,21 +182,25 @@ func (sgc *StoreGrantCoordinators) initGrantCoordinator(storeID roachpb.StoreID) // This is IO work, so override the usesTokens value. opts.usesTokens = true // TODO(sumeer): add per-store WorkQueue state for debug.zip and db console. - granters := [admissionpb.NumWorkClasses]granterWithStoreReplicatedWorkAdmitted{ + storeGranters := [admissionpb.NumWorkClasses]granterWithStoreReplicatedWorkAdmitted{ &kvStoreTokenChildGranter{ - workClass: admissionpb.RegularWorkClass, - parent: kvg, + workType: admissionpb.RegularStoreWorkType, + parent: kvg, }, &kvStoreTokenChildGranter{ - workClass: admissionpb.ElasticWorkClass, - parent: kvg, + workType: admissionpb.ElasticStoreWorkType, + parent: kvg, }, } + snapshotGranter := &kvStoreTokenChildGranter{ + workType: admissionpb.SnapshotIngestStoreWorkType, + parent: kvg, + } storeReq := sgc.makeStoreRequesterFunc( sgc.ambientCtx, storeID, - granters, + storeGranters, sgc.settings, sgc.workQueueMetrics, opts, @@ -208,6 +213,7 @@ func (sgc *StoreGrantCoordinators) initGrantCoordinator(storeID roachpb.StoreID) requesters := storeReq.getRequesters() kvg.regularRequester = requesters[admissionpb.RegularWorkClass] kvg.elasticRequester = requesters[admissionpb.ElasticWorkClass] + kvg.snapshotRequester = makeSnapshotQueue(snapshotGranter, sgc.snapshotQueueMetrics) coord.granters[KVWork] = kvg coord.ioLoadListener = &ioLoadListener{ storeID: storeID, @@ -231,6 +237,13 @@ func (sgc *StoreGrantCoordinators) TryGetQueueForStore(storeID roachpb.StoreID) return nil } +func (sgc *StoreGrantCoordinators) TryGetSnapshotQueueForStore(storeID roachpb.StoreID) requester { + if granter, ok := sgc.gcMap.Load(storeID); ok { + return granter.granters[KVWork].(*kvStoreTokenGranter).snapshotRequester + } + return nil +} + func (sgc *StoreGrantCoordinators) close() { // closeCh can be nil in tests that never called SetPebbleMetricsProvider. if sgc.closeCh != nil { @@ -463,6 +476,8 @@ func makeStoresGrantCoordinators( storeWorkQueueMetrics := [admissionpb.NumWorkClasses]*WorkQueueMetrics{ regularStoreWorkQueueMetrics, elasticStoreWorkQueueMetrics, } + snapshotQueueMetrics := makeSnapshotQueueMetrics(registry) + makeStoreRequester := makeStoreWorkQueue if opts.makeStoreRequesterFunc != nil { makeStoreRequester = opts.makeStoreRequesterFunc @@ -479,6 +494,7 @@ func makeStoresGrantCoordinators( l0CompactedBytes: metrics.L0CompactedBytes, l0TokensProduced: metrics.L0TokensProduced, workQueueMetrics: storeWorkQueueMetrics, + snapshotQueueMetrics: snapshotQueueMetrics, onLogEntryAdmitted: onLogEntryAdmitted, knobs: knobs, } diff --git a/pkg/util/admission/granter.go b/pkg/util/admission/granter.go index 6a483779275c..7d0ea166e381 100644 --- a/pkg/util/admission/granter.go +++ b/pkg/util/admission/granter.go @@ -289,9 +289,10 @@ func (tg *tokenGranter) tryGrantLocked(grantChainID grantChainID) grantResult { // tokens, which are based on disk bandwidth as a constrained resource, and // apply to all the elastic incoming bytes into the LSM. type kvStoreTokenGranter struct { - coord *GrantCoordinator - regularRequester requester - elasticRequester requester + coord *GrantCoordinator + regularRequester requester + elasticRequester requester + snapshotRequester requester coordMu struct { // holds fields protected by coord.mu.Lock // There is no rate limiting in granting these tokens. That is, they are @@ -306,7 +307,7 @@ type kvStoreTokenGranter struct { // TODO(aaditya): add support for read/IOPS tokens. // Disk bandwidth tokens. diskTokensAvailable diskTokens - diskTokensUsed [admissionpb.NumWorkClasses]diskTokens + diskTokensUsed [admissionpb.NumStoreWorkTypes]diskTokens } ioTokensExhaustedDurationMetric [admissionpb.NumWorkClasses]*metric.Counter @@ -329,8 +330,8 @@ var _ granterWithIOTokens = &kvStoreTokenGranter{} // kvStoreTokenChildGranter handles a particular workClass. Its methods // pass-through to the parent after adding the workClass as a parameter. type kvStoreTokenChildGranter struct { - workClass admissionpb.WorkClass - parent *kvStoreTokenGranter + workType admissionpb.StoreWorkType + parent *kvStoreTokenGranter } var _ granterWithStoreReplicatedWorkAdmitted = &kvStoreTokenChildGranter{} @@ -343,17 +344,17 @@ func (cg *kvStoreTokenChildGranter) grantKind() grantKind { // tryGet implements granter. func (cg *kvStoreTokenChildGranter) tryGet(count int64) bool { - return cg.parent.tryGet(cg.workClass, count) + return cg.parent.tryGet(cg.workType, count) } // returnGrant implements granter. func (cg *kvStoreTokenChildGranter) returnGrant(count int64) { - cg.parent.returnGrant(cg.workClass, count) + cg.parent.returnGrant(cg.workType, count) } // tookWithoutPermission implements granter. func (cg *kvStoreTokenChildGranter) tookWithoutPermission(count int64) { - cg.parent.tookWithoutPermission(cg.workClass, count) + cg.parent.tookWithoutPermission(cg.workType, count) } // continueGrantChain implements granter. @@ -373,23 +374,23 @@ func (cg *kvStoreTokenChildGranter) storeWriteDone( // granter was previously exhausted but is no longer so, we're allowed to // admit other waiting requests. return cg.parent.storeReplicatedWorkAdmittedLocked( - cg.workClass, originalTokens, storeReplicatedWorkAdmittedInfo(doneInfo), true /* canGrantAnother */) + cg.workType, originalTokens, storeReplicatedWorkAdmittedInfo(doneInfo), true /* canGrantAnother */) } // storeReplicatedWorkAdmitted implements granterWithStoreReplicatedWorkAdmitted. func (cg *kvStoreTokenChildGranter) storeReplicatedWorkAdmittedLocked( originalTokens int64, admittedInfo storeReplicatedWorkAdmittedInfo, ) (additionalTokens int64) { - return cg.parent.storeReplicatedWorkAdmittedLocked(cg.workClass, originalTokens, admittedInfo, false /* canGrantAnother */) + return cg.parent.storeReplicatedWorkAdmittedLocked(cg.workType, originalTokens, admittedInfo, false /* canGrantAnother */) } -func (sg *kvStoreTokenGranter) tryGet(workClass admissionpb.WorkClass, count int64) bool { - return sg.coord.tryGet(KVWork, count, int8(workClass)) +func (sg *kvStoreTokenGranter) tryGet(workType admissionpb.StoreWorkType, count int64) bool { + return sg.coord.tryGet(KVWork, count, int8(workType)) } // tryGetLocked implements granterWithLockedCalls. func (sg *kvStoreTokenGranter) tryGetLocked(count int64, demuxHandle int8) grantResult { - wc := admissionpb.WorkClass(demuxHandle) + wt := admissionpb.StoreWorkType(demuxHandle) // NB: ideally if regularRequester.hasWaitingRequests() returns true and // wc==elasticWorkClass we should reject this request, since it means that // more important regular work is waiting. However, we rely on the @@ -416,66 +417,96 @@ func (sg *kvStoreTokenGranter) tryGetLocked(count int64, demuxHandle int8) grant // needed. We are generally okay with this since the model changes // infrequently (every 15s), and the disk bandwidth limiter is designed to // generally under admit and only pace elastic work. - adjustedDiskWriteTokens := sg.writeAmpLM.applyLinearModel(count) - switch wc { - case admissionpb.RegularWorkClass: + diskWriteTokens := count + if wt != admissionpb.SnapshotIngestStoreWorkType { + // Snapshot ingests do not incur the write amplification described above, so + // we skip applying the model for those writes. + diskWriteTokens = sg.writeAmpLM.applyLinearModel(count) + } + switch wt { + case admissionpb.RegularStoreWorkType: if sg.coordMu.availableIOTokens[admissionpb.RegularWorkClass] > 0 { - sg.subtractTokensLocked(count, count, false) - sg.coordMu.diskTokensAvailable.writeByteTokens -= adjustedDiskWriteTokens - sg.coordMu.diskTokensUsed[wc].writeByteTokens += adjustedDiskWriteTokens + sg.subtractIOTokensLocked(count, count, false) + sg.coordMu.diskTokensAvailable.writeByteTokens -= diskWriteTokens + sg.coordMu.diskTokensUsed[wt].writeByteTokens += diskWriteTokens return grantSuccess } - case admissionpb.ElasticWorkClass: + case admissionpb.ElasticStoreWorkType: if sg.coordMu.diskTokensAvailable.writeByteTokens > 0 && sg.coordMu.availableIOTokens[admissionpb.RegularWorkClass] > 0 && sg.coordMu.availableIOTokens[admissionpb.ElasticWorkClass] > 0 { - sg.subtractTokensLocked(count, count, false) + sg.subtractIOTokensLocked(count, count, false) sg.coordMu.elasticIOTokensUsedByElastic += count - sg.coordMu.diskTokensAvailable.writeByteTokens -= adjustedDiskWriteTokens - sg.coordMu.diskTokensUsed[wc].writeByteTokens += adjustedDiskWriteTokens + sg.coordMu.diskTokensAvailable.writeByteTokens -= diskWriteTokens + sg.coordMu.diskTokensUsed[wt].writeByteTokens += diskWriteTokens + return grantSuccess + } + case admissionpb.SnapshotIngestStoreWorkType: + // Snapshot ingests do not go into L0, so we only subject them to + // writeByteTokens. + if sg.coordMu.diskTokensAvailable.writeByteTokens > 0 { + sg.coordMu.diskTokensAvailable.writeByteTokens -= diskWriteTokens + sg.coordMu.diskTokensUsed[wt].writeByteTokens += diskWriteTokens return grantSuccess } } return grantFailLocal } -func (sg *kvStoreTokenGranter) returnGrant(workClass admissionpb.WorkClass, count int64) { - sg.coord.returnGrant(KVWork, count, int8(workClass)) +func (sg *kvStoreTokenGranter) returnGrant(workType admissionpb.StoreWorkType, count int64) { + sg.coord.returnGrant(KVWork, count, int8(workType)) } // returnGrantLocked implements granterWithLockedCalls. func (sg *kvStoreTokenGranter) returnGrantLocked(count int64, demuxHandle int8) { - wc := admissionpb.WorkClass(demuxHandle) - // Return count tokens to the "IO tokens". - sg.subtractTokensLocked(-count, -count, false) - if wc == admissionpb.ElasticWorkClass { - sg.coordMu.elasticIOTokensUsedByElastic -= count + wt := admissionpb.StoreWorkType(demuxHandle) + // Return store tokens. + sg.subtractTokensForStoreWorkTypeLocked(wt, -count) +} + +// subtractTokensForStoreWorkTypeLocked is a helper function that subtracts +// tokens from all store tokens for a given admissionpb.StoreWorkType. Count can +// be negative to add tokens. +func (sg *kvStoreTokenGranter) subtractTokensForStoreWorkTypeLocked( + wt admissionpb.StoreWorkType, count int64, +) { + if wt != admissionpb.SnapshotIngestStoreWorkType { + // Adjust count tokens for "IO tokens". + sg.subtractIOTokensLocked(count, count, false) + } + if wt == admissionpb.ElasticStoreWorkType { + sg.coordMu.elasticIOTokensUsedByElastic += count + } + // Adjust tokens for disk bandwidth bucket. + switch wt { + case admissionpb.RegularStoreWorkType, admissionpb.ElasticStoreWorkType: + diskTokenCount := sg.writeAmpLM.applyLinearModel(count) + sg.coordMu.diskTokensAvailable.writeByteTokens -= diskTokenCount + sg.coordMu.diskTokensUsed[wt].writeByteTokens += diskTokenCount + case admissionpb.SnapshotIngestStoreWorkType: + // Do not apply the writeAmpLM since these writes do not incur additional + // write-amp. + sg.coordMu.diskTokensAvailable.writeByteTokens -= count + sg.coordMu.diskTokensUsed[wt].writeByteTokens += count } - // Return tokens to disk bandwidth bucket. - diskTokenCount := sg.writeAmpLM.applyLinearModel(count) - sg.coordMu.diskTokensAvailable.writeByteTokens += diskTokenCount - sg.coordMu.diskTokensUsed[wc].writeByteTokens -= diskTokenCount } -func (sg *kvStoreTokenGranter) tookWithoutPermission(workClass admissionpb.WorkClass, count int64) { - sg.coord.tookWithoutPermission(KVWork, count, int8(workClass)) +func (sg *kvStoreTokenGranter) tookWithoutPermission( + workType admissionpb.StoreWorkType, count int64, +) { + sg.coord.tookWithoutPermission(KVWork, count, int8(workType)) } // tookWithoutPermissionLocked implements granterWithLockedCalls. func (sg *kvStoreTokenGranter) tookWithoutPermissionLocked(count int64, demuxHandle int8) { - wc := admissionpb.WorkClass(demuxHandle) - sg.subtractTokensLocked(count, count, false) - if wc == admissionpb.ElasticWorkClass { - sg.coordMu.elasticIOTokensUsedByElastic += count - } - diskTokenCount := sg.writeAmpLM.applyLinearModel(count) - sg.coordMu.diskTokensAvailable.writeByteTokens -= diskTokenCount - sg.coordMu.diskTokensUsed[wc].writeByteTokens += diskTokenCount + wt := admissionpb.StoreWorkType(demuxHandle) + // Deduct store tokens. + sg.subtractTokensForStoreWorkTypeLocked(wt, count) } -// subtractTokensLocked is a helper function that subtracts count tokens (count +// subtractIOTokensLocked is a helper function that subtracts count tokens (count // can be negative, in which case this is really an addition). -func (sg *kvStoreTokenGranter) subtractTokensLocked( +func (sg *kvStoreTokenGranter) subtractIOTokensLocked( count int64, elasticCount int64, settingAvailableTokens bool, ) { sg.subtractTokensLockedForWorkClass(admissionpb.RegularWorkClass, count, settingAvailableTokens) @@ -514,31 +545,39 @@ func (sg *kvStoreTokenGranter) subtractTokensLockedForWorkClass( // requesterHasWaitingRequests implements granterWithLockedCalls. func (sg *kvStoreTokenGranter) requesterHasWaitingRequests() bool { - return sg.regularRequester.hasWaitingRequests() || sg.elasticRequester.hasWaitingRequests() + return sg.regularRequester.hasWaitingRequests() || + sg.elasticRequester.hasWaitingRequests() || + sg.snapshotRequester.hasWaitingRequests() } // tryGrantLocked implements granterWithLockedCalls. func (sg *kvStoreTokenGranter) tryGrantLocked(grantChainID grantChainID) grantResult { - // First try granting to regular requester. - for wc := range sg.coordMu.diskTokensUsed { + // NB: We grant work in the following priority order: regular, snapshot + // ingest, elastic work. Snapshot ingests are a special type of elastic work. + // They queue separately in the SnapshotQueue and get priority over other + // elastic work since they are used for node re-balancing and up-replication, + // which are typically higher priority than other background writes. + for wt := 0; wt < admissionpb.NumStoreWorkTypes; wt++ { req := sg.regularRequester - if admissionpb.WorkClass(wc) == admissionpb.ElasticWorkClass { + if admissionpb.StoreWorkType(wt) == admissionpb.ElasticStoreWorkType { req = sg.elasticRequester + } else if admissionpb.StoreWorkType(wt) == admissionpb.SnapshotIngestStoreWorkType { + req = sg.snapshotRequester } if req.hasWaitingRequests() { - res := sg.tryGetLocked(1, int8(wc)) + res := sg.tryGetLocked(1, int8(wt)) if res == grantSuccess { tookTokenCount := req.granted(grantChainID) if tookTokenCount == 0 { // Did not accept grant. - sg.returnGrantLocked(1, int8(wc)) + sg.returnGrantLocked(1, int8(wt)) // Continue with the loop since this requester does not have waiting // requests. If the loop terminates we will correctly return // grantFailLocal. } else { // May have taken more. if tookTokenCount > 1 { - sg.tookWithoutPermissionLocked(tookTokenCount-1, int8(wc)) + sg.tookWithoutPermissionLocked(tookTokenCount-1, int8(wt)) } return grantSuccess } @@ -573,7 +612,7 @@ func (sg *kvStoreTokenGranter) setAvailableTokens( // tookWithoutPermission or because tryGet will satisfy requests until // availableIOTokens become <= 0. We want to remember this previous // over-allocation. - sg.subtractTokensLocked(-ioTokens, -elasticIOTokens, true) + sg.subtractIOTokensLocked(-ioTokens, -elasticIOTokens, true) if sg.coordMu.availableIOTokens[admissionpb.RegularWorkClass] > ioTokenCapacity { sg.coordMu.availableIOTokens[admissionpb.RegularWorkClass] = ioTokenCapacity } @@ -610,11 +649,11 @@ func (sg *kvStoreTokenGranter) setAvailableTokens( // getDiskTokensUsedAndResetLocked implements granterWithIOTokens. func (sg *kvStoreTokenGranter) getDiskTokensUsedAndReset() ( - usedTokens [admissionpb.NumWorkClasses]diskTokens, + usedTokens [admissionpb.NumStoreWorkTypes]diskTokens, ) { sg.coord.mu.Lock() defer sg.coord.mu.Unlock() - for i := 0; i < int(admissionpb.NumWorkClasses); i++ { + for i := 0; i < admissionpb.NumStoreWorkTypes; i++ { usedTokens[i] = sg.coordMu.diskTokensUsed[i] sg.coordMu.diskTokensUsed[i] = diskTokens{} } @@ -637,12 +676,13 @@ func (sg *kvStoreTokenGranter) setLinearModels( } func (sg *kvStoreTokenGranter) storeReplicatedWorkAdmittedLocked( - wc admissionpb.WorkClass, + wt admissionpb.StoreWorkType, originalTokens int64, admittedInfo storeReplicatedWorkAdmittedInfo, canGrantAnother bool, ) (additionalTokens int64) { // Reminder: coord.mu protects the state in the kvStoreTokenGranter. + wc := admissionpb.WorkClassFromStoreWorkType(wt) exhaustedFunc := func() bool { return sg.coordMu.availableIOTokens[admissionpb.RegularWorkClass] <= 0 || (wc == admissionpb.ElasticWorkClass && (sg.coordMu.diskTokensAvailable.writeByteTokens <= 0 || @@ -653,8 +693,8 @@ func (sg *kvStoreTokenGranter) storeReplicatedWorkAdmittedLocked( actualL0IngestTokens := sg.l0IngestLM.applyLinearModel(admittedInfo.IngestedBytes) actualL0Tokens := actualL0WriteTokens + actualL0IngestTokens additionalL0TokensNeeded := actualL0Tokens - originalTokens - sg.subtractTokensLocked(additionalL0TokensNeeded, additionalL0TokensNeeded, false) - if wc == admissionpb.ElasticWorkClass { + sg.subtractIOTokensLocked(additionalL0TokensNeeded, additionalL0TokensNeeded, false) + if wt == admissionpb.ElasticStoreWorkType { sg.coordMu.elasticIOTokensUsedByElastic += additionalL0TokensNeeded } @@ -665,7 +705,7 @@ func (sg *kvStoreTokenGranter) storeReplicatedWorkAdmittedLocked( originalDiskTokens := sg.writeAmpLM.applyLinearModel(originalTokens) additionalDiskWriteTokens := actualDiskWriteTokens - originalDiskTokens sg.coordMu.diskTokensAvailable.writeByteTokens -= additionalDiskWriteTokens - sg.coordMu.diskTokensUsed[wc].writeByteTokens += additionalDiskWriteTokens + sg.coordMu.diskTokensUsed[wt].writeByteTokens += additionalDiskWriteTokens if canGrantAnother && (additionalL0TokensNeeded < 0) { isExhausted := exhaustedFunc() diff --git a/pkg/util/admission/granter_test.go b/pkg/util/admission/granter_test.go index 778891f151bc..32eb043f057b 100644 --- a/pkg/util/admission/granter_test.go +++ b/pkg/util/admission/granter_test.go @@ -51,7 +51,9 @@ func TestGranterBasic(t *testing.T) { var ambientCtx log.AmbientContext // requesters[numWorkKinds] is used for kv elastic work, when working with a // store grant coordinator. - var requesters [numWorkKinds + 1]*testRequester + // requesters[numWorkKinds + 1] is used for snapshot ingest, when working with a + // store grant coordinator. + var requesters [numWorkKinds + 2]*testRequester var coord *GrantCoordinator clearRequesterAndCoord := func() { coord = nil @@ -107,6 +109,7 @@ func TestGranterBasic(t *testing.T) { metrics := makeGrantCoordinatorMetrics() regularWorkQueueMetrics := makeWorkQueueMetrics("regular", registry) elasticWorkQUeueMetrics := makeWorkQueueMetrics("elastic", registry) + snapshotQueueMetrics := makeSnapshotQueueMetrics(registry) workQueueMetrics := [admissionpb.NumWorkClasses]*WorkQueueMetrics{ regularWorkQueueMetrics, elasticWorkQUeueMetrics, } @@ -148,6 +151,7 @@ func TestGranterBasic(t *testing.T) { l0CompactedBytes: metrics.L0CompactedBytes, l0TokensProduced: metrics.L0TokensProduced, workQueueMetrics: workQueueMetrics, + snapshotQueueMetrics: snapshotQueueMetrics, disableTickerForTesting: true, knobs: &TestingKnobs{}, } @@ -158,6 +162,22 @@ func TestGranterBasic(t *testing.T) { coord, ok = storeCoordinators.gcMap.Load(1) require.True(t, ok) kvStoreGranter := coord.granters[KVWork].(*kvStoreTokenGranter) + // Defensive check: `SetPebbleMetricsProvider` should initialize the SnapshotQueue. + require.NotNil(t, kvStoreGranter.snapshotRequester) + snapshotGranter := kvStoreGranter.snapshotRequester.(*SnapshotQueue).snapshotGranter + require.NotNil(t, snapshotGranter) + snapshotReq := &testRequester{ + workKind: KVWork, + granter: snapshotGranter, + additionalID: "-snapshot", + usesTokens: true, + buf: &buf, + returnValueFromGranted: 0, + } + kvStoreGranter.snapshotRequester = snapshotReq + snapshotQueue := storeCoordinators.TryGetSnapshotQueueForStore(1) + require.NotNil(t, snapshotQueue) + requesters[numWorkKinds+1] = snapshotReq // Use the same model for the IO linear models. tlm := tokensLinearModel{multiplier: 0.5, constant: 50} // Use w-amp of 1 for the purpose of this test. @@ -482,6 +502,8 @@ func scanWorkKind(t *testing.T, d *datadriven.TestData) int8 { return int8(SQLStatementRootStartWork) case "kv-elastic": return int8(numWorkKinds) + case "kv-snapshot": + return int8(numWorkKinds + 1) } panic("unknown WorkKind") } diff --git a/pkg/util/admission/io_load_listener_test.go b/pkg/util/admission/io_load_listener_test.go index 0df001f10be8..66655a3b8983 100644 --- a/pkg/util/admission/io_load_listener_test.go +++ b/pkg/util/admission/io_load_listener_test.go @@ -419,7 +419,7 @@ func (r *testRequesterForIOLL) setStoreRequestEstimates(estimates storeRequestEs type testGranterWithIOTokens struct { buf strings.Builder allTokensUsed bool - diskBandwidthTokensUsed [admissionpb.NumWorkClasses]diskTokens + diskBandwidthTokensUsed [admissionpb.NumStoreWorkTypes]diskTokens } var _ granterWithIOTokens = &testGranterWithIOTokens{} @@ -450,7 +450,7 @@ func (g *testGranterWithIOTokens) setAvailableTokens( } func (g *testGranterWithIOTokens) getDiskTokensUsedAndReset() ( - usedTokens [admissionpb.NumWorkClasses]diskTokens, + usedTokens [admissionpb.NumStoreWorkTypes]diskTokens, ) { return g.diskBandwidthTokensUsed } @@ -503,9 +503,9 @@ func (g *testGranterNonNegativeTokens) setAvailableTokens( } func (g *testGranterNonNegativeTokens) getDiskTokensUsedAndReset() ( - usedTokens [admissionpb.NumWorkClasses]diskTokens, + usedTokens [admissionpb.NumStoreWorkTypes]diskTokens, ) { - return [admissionpb.NumWorkClasses]diskTokens{} + return [admissionpb.NumStoreWorkTypes]diskTokens{} } func (g *testGranterNonNegativeTokens) setLinearModels( diff --git a/pkg/util/admission/snapshot_queue.go b/pkg/util/admission/snapshot_queue.go new file mode 100644 index 000000000000..8df982a7897e --- /dev/null +++ b/pkg/util/admission/snapshot_queue.go @@ -0,0 +1,305 @@ +// Copyright 2024 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package admission + +import ( + "context" + "sync" + "time" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/metric" + "github.com/cockroachdb/cockroach/pkg/util/queue" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/errors" +) + +type snapshotWorkItem struct { + admitCh chan bool + count int64 + enqueueingTime time.Time + mu struct { + // These fields are updated after creation. The mutex in SnapshotQueue must + // be held to read and write to these fields. + inQueue bool + cancelled bool + } +} + +// SnapshotBurstSize represents the maximum number of bytes a snapshot ingest +// request can write before asking for admission. +// +// TODO(aaditya): Maybe make it a cluster setting. +const SnapshotBurstSize = 1 << 20 // 1MB + +var snapshotWorkItemPool = sync.Pool{ + New: func() interface{} { + return &snapshotWorkItem{} + }, +} + +// DiskBandwidthForSnapshotIngest determines whether range snapshot ingests will +// be subject to disk write control tokens in Admission Control. +var DiskBandwidthForSnapshotIngest = settings.RegisterBoolSetting( + settings.SystemOnly, "kvadmission.store.snapshot_ingest_bandwidth_control.enabled", + "if set to true, snapshot ingests will be subject to disk write control in AC", + // TODO(aaditya): Enable by default once enough experimentation is done. + false, + settings.WithPublic) + +var snapshotWaitDur = metric.Metadata{ + Name: "admission.wait_durations.snapshot_ingest", + Help: "Wait time for snapshot ingest requests that waited", + Measurement: "Wait time Duration", + Unit: metric.Unit_NANOSECONDS, +} + +type SnapshotMetrics struct { + WaitDurations metric.IHistogram +} + +func makeSnapshotQueueMetrics(registry *metric.Registry) *SnapshotMetrics { + m := &SnapshotMetrics{ + WaitDurations: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: snapshotWaitDur, + Duration: base.DefaultHistogramWindowInterval(), + BucketConfig: metric.IOLatencyBuckets, + }), + } + registry.AddMetricStruct(m) + return m +} + +// snapshotRequester is a wrapper used for test purposes. +type snapshotRequester interface { + requester + Admit(ctx context.Context, count int64) error +} + +// SnapshotQueue implements the requester interface. It is used to request +// admission for KV range snapshot requests. Internally, it uses queue.Queue to +// maintain FIFO-ordering. +type SnapshotQueue struct { + snapshotGranter granter + mu struct { + syncutil.Mutex + q *queue.Queue[*snapshotWorkItem] + } + metrics *SnapshotMetrics + ts timeutil.TimeSource +} + +func makeSnapshotQueue(snapshotGranter granter, metrics *SnapshotMetrics) *SnapshotQueue { + sq := &SnapshotQueue{ + snapshotGranter: snapshotGranter, + metrics: metrics, + } + // We ignore the error here since we are not applying any options. + q, _ := queue.NewQueue[*snapshotWorkItem]() + sq.mu.q = q + sq.ts = timeutil.DefaultTimeSource{} + return sq +} + +var _ requester = &SnapshotQueue{} +var _ snapshotRequester = &SnapshotQueue{} + +func (s *SnapshotQueue) hasWaitingRequests() bool { + s.mu.Lock() + defer s.mu.Unlock() + return !s.mu.q.Empty() +} + +func (s *SnapshotQueue) granted(_ grantChainID) int64 { + s.mu.Lock() + defer s.mu.Unlock() + var item *snapshotWorkItem + // Loop until we find a valid item, or queue becomes empty. + for { + item = s.popLocked() + if item == nil { + return 0 + } + if item.mu.cancelled { + // Item was cancelled, we can ignore it. + releaseSnapshotWorkItem(item) + continue + } + break + } + item.admitCh <- true + return item.count +} + +func (s *SnapshotQueue) close() { + s.mu.Lock() + defer s.mu.Unlock() + for !s.mu.q.Empty() { + s.mu.q.Dequeue() + } +} + +// Admit is called whenever a snapshot ingest request needs to update the number +// of byte tokens it is using. Note that it accepts negative values, in which +// case it will return the tokens back to the granter. +func (s *SnapshotQueue) Admit(ctx context.Context, count int64) error { + if count == 0 { + return nil + } + if count < 0 { + s.snapshotGranter.returnGrant(count) + return nil + } + if s.snapshotGranter.tryGet(count) { + return nil + } + // We were unable to get tokens for admission, so we queue. + shouldRelease := true + item := newSnapshotWorkItem(count) + defer func() { + if shouldRelease { + s.mu.Lock() + defer s.mu.Unlock() + releaseSnapshotWorkItem(item) + } + }() + + func() { + s.mu.Lock() + defer s.mu.Unlock() + s.addLocked(item) + }() + + // Start waiting for admission. + select { + case <-ctx.Done(): + waitDur := timeutil.Since(item.enqueueingTime).Nanoseconds() + s.mu.Lock() + defer s.mu.Unlock() + if !item.mu.inQueue { + s.snapshotGranter.returnGrant(item.count) + } + // TODO(aaditya): Ideally, we also remove the item from the actual queue. + // Right now, if we cancel the work, it remains in the queue. A call to + // hasWaitingRequests() will return true even if all items in the queue are + // cancelled but this is a very rare occurrence. A call to granted() will + // work around this and skip any work item that is cancelled. This is + // non-ideal behavior, but still provides accurate token accounting. + shouldRelease = false + item.mu.cancelled = true + deadline, _ := ctx.Deadline() + s.metrics.WaitDurations.RecordValue(waitDur) + return errors.Wrapf(ctx.Err(), + "context canceled while waiting in queue: deadline: %v, start: %v, dur: %v", + deadline, item.enqueueingTime, waitDur) + case <-item.admitCh: + waitDur := timeutil.Since(item.enqueueingTime).Nanoseconds() + s.metrics.WaitDurations.RecordValue(waitDur) + return nil + } +} + +func (s *SnapshotQueue) addLocked(item *snapshotWorkItem) { + item.enqueueingTime = timeutil.Now() + s.mu.q.Enqueue(item) + item.mu.inQueue = true +} + +func (s *SnapshotQueue) popLocked() *snapshotWorkItem { + item, ok := s.mu.q.Dequeue() + if !ok { + return nil + } + item.mu.inQueue = false + return item +} + +func (s *SnapshotQueue) empty() bool { + s.mu.Lock() + defer s.mu.Unlock() + return s.mu.q.Empty() +} + +func releaseSnapshotWorkItem(sw *snapshotWorkItem) { + ch := sw.admitCh + select { + case <-ch: + panic("channel must be empty and not closed") + default: + } + *sw = snapshotWorkItem{ + admitCh: ch, + } + waitingWorkPool.Put(sw) +} + +func newSnapshotWorkItem(count int64) *snapshotWorkItem { + item := snapshotWorkItemPool.Get().(*snapshotWorkItem) + ch := item.admitCh + if ch == nil { + ch = make(chan bool, 1) + } + *item = snapshotWorkItem{ + admitCh: ch, + enqueueingTime: timeutil.Now(), + count: count, + } + item.mu.cancelled = false + item.mu.inQueue = false + return item +} + +type SnapshotPacer struct { + snapshotQ snapshotRequester + storeID roachpb.StoreID + intWriteBytes int64 +} + +func NewSnapshotPacer(q snapshotRequester, s roachpb.StoreID) *SnapshotPacer { + return &SnapshotPacer{ + snapshotQ: q, + storeID: s, + intWriteBytes: 0, + } +} + +func (p *SnapshotPacer) Pace(ctx context.Context, writeBytes int64, final bool) error { + // Return early if nil pacer or nil snapshotQ. + if p == nil { + return nil + } + if p.snapshotQ == nil { + log.Errorf(ctx, "unable to find snapshot queue for store: %s", p.storeID) + return nil + } + + p.intWriteBytes += writeBytes + if p.intWriteBytes <= SnapshotBurstSize && !final { + return nil + } + if err := p.snapshotQ.Admit(ctx, p.intWriteBytes); err != nil { + return errors.Wrapf(err, "snapshot admission queue") + } + p.intWriteBytes = 0 + return nil +} + +func (p *SnapshotPacer) Close() { + if p == nil { + return + } + p.snapshotQ = nil +} diff --git a/pkg/util/admission/snapshot_queue_test.go b/pkg/util/admission/snapshot_queue_test.go new file mode 100644 index 000000000000..b2454a05894a --- /dev/null +++ b/pkg/util/admission/snapshot_queue_test.go @@ -0,0 +1,208 @@ +// Copyright 2024 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package admission + +import ( + "context" + "fmt" + "strconv" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" + "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" + "github.com/stretchr/testify/require" +) + +// TestSnapshotQueue is a datadriven with test data in testdata/snapshot_queue. +func TestSnapshotQueue(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + var q *SnapshotQueue + closeFn := func() { + if q != nil { + q.close() + } + } + defer closeFn() + + var tg *testGranter + var buf builderWithMu + var wrkMap workMap + initialTime := timeutil.FromUnixMicros(int64(0)) + registry := metric.NewRegistry() + metrics := makeSnapshotQueueMetrics(registry) + + datadriven.RunTest(t, datapathutils.TestDataPath(t, "snapshot_queue"), + func(t *testing.T, d *datadriven.TestData) string { + switch d.Cmd { + case "init": + closeFn() + tg = &testGranter{gk: token, buf: &buf} + q = makeSnapshotQueue(tg, metrics) + q.ts = timeutil.NewManualTime(initialTime) + tg.r = q + wrkMap.resetMap() + return "" + + case "admit": + var id int + var createTime int + var count int + d.ScanArgs(t, "id", &id) + if _, ok := wrkMap.get(id); ok { + panic(fmt.Sprintf("id %d is already used", id)) + } + d.ScanArgs(t, "count", &count) + d.ScanArgs(t, "create-time-millis", &createTime) + q.ts.(*timeutil.ManualTime).AdvanceTo(timeutil.FromUnixNanos(int64(createTime) * time.Millisecond.Nanoseconds())) + ctx, cancel := context.WithCancel(context.Background()) + wrkMap.set(id, &testWork{cancel: cancel}) + go func(ctx context.Context, id int, count int) { + err := q.Admit(ctx, int64(count)) + if err != nil { + buf.printf("id %d: admit failed", id) + wrkMap.delete(id) + } else { + buf.printf("id %d: admit succeeded", id) + wrkMap.setAdmitted(id, StoreWorkHandle{}) + } + }(ctx, id, count) + // Need deterministic output, and this is racing with the goroutine + // which is trying to get admitted. Retry to let it get scheduled. + maybeRetryWithWait(t, d.Expected, d.Rewrite, buf.String) + return buf.stringAndReset() + + case "set-try-get-return-value": + var v bool + d.ScanArgs(t, "v", &v) + tg.returnValueFromTryGet = v + return "" + + case "granted": + rv := tg.r.granted(noGrantChain) + if rv > 0 { + // Need deterministic output, and this is racing with the goroutine that was + // admitted. Retry a few times. + maybeRetryWithWait(t, d.Expected, d.Rewrite, buf.String) + } + tg.buf.printf("granted: returned %d", rv) + return buf.stringAndReset() + + case "cancel-work": + var id int + d.ScanArgs(t, "id", &id) + work, ok := wrkMap.get(id) + if !ok { + return fmt.Sprintf("unknown id: %d", id) + } + if work.admitted { + return fmt.Sprintf("work already admitted id: %d", id) + } + work.cancel() + // Need deterministic output, and this is racing with the goroutine + // whose work is canceled. Retry to let it get scheduled. + maybeRetryWithWait(t, d.Expected, d.Rewrite, buf.String) + return buf.stringAndReset() + + case "empty": + // Need deterministic output, and this is racing with the goroutine + // whose work is canceled. Retry to let it get scheduled. + maybeRetryWithWait(t, d.Expected, d.Rewrite, func() string { + return strconv.FormatBool(q.empty()) + }) + return strconv.FormatBool(q.empty()) + + default: + return fmt.Sprintf("unknown command: %s", d.Cmd) + } + }) +} + +func TestSnapshotPacer(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + + var pacer *SnapshotPacer = nil + // Should not panic on nil pacer. + require.NoError(t, pacer.Pace(ctx, 1, false)) + + q := &testingSnapshotQueue{} + pacer = NewSnapshotPacer(q, 1) + defer pacer.Close() + + // Should not ask for admission since write bytes = burst size. + writeBytes := int64(SnapshotBurstSize) + require.NoError(t, pacer.Pace(ctx, writeBytes, false)) + require.False(t, q.admitted) + require.Equal(t, writeBytes, pacer.intWriteBytes) + require.Equal(t, int64(0), q.admitCount) + + // Do another write, should go over threshold and seek admission. + require.NoError(t, pacer.Pace(ctx, 1, false)) + require.True(t, q.admitted) + require.Equal(t, int64(0), pacer.intWriteBytes) + require.Equal(t, writeBytes+1, q.admitCount) + + // Not enough bytes since last admission. Should not ask for admission. + q.admitted = false + q.admitCount = 0 + require.NoError(t, pacer.Pace(ctx, 5, false)) + require.False(t, q.admitted) + require.Equal(t, int64(5), pacer.intWriteBytes) + require.Equal(t, int64(0), q.admitCount) + + // We now go above the threshold again. Should ask for admission. + require.NoError(t, pacer.Pace(ctx, writeBytes, false)) + require.True(t, q.admitted) + require.Equal(t, writeBytes+5, q.admitCount) + require.Equal(t, int64(0), pacer.intWriteBytes) + + // If final call to pacer, we should admit regardless of size. + q.admitted = false + q.admitCount = 0 + require.NoError(t, pacer.Pace(ctx, -1, true)) + require.True(t, q.admitted) + require.Equal(t, int64(-1), q.admitCount) +} + +// testingSnapshotQueue is used to test SnapshotPacer. +type testingSnapshotQueue struct { + admitted bool + admitCount int64 +} + +var _ snapshotRequester = &testingSnapshotQueue{} + +func (ts *testingSnapshotQueue) hasWaitingRequests() bool { + return false +} + +func (ts *testingSnapshotQueue) granted(grantChainID grantChainID) int64 { + return 0 +} + +func (ts *testingSnapshotQueue) close() { + // Do nothing. +} + +func (ts *testingSnapshotQueue) Admit(ctx context.Context, count int64) error { + ts.admitted = true + ts.admitCount = count + return nil +} diff --git a/pkg/util/admission/testdata/disk_bandwidth_limiter b/pkg/util/admission/testdata/disk_bandwidth_limiter index 971111974055..92c8bec1bcc9 100644 --- a/pkg/util/admission/testdata/disk_bandwidth_limiter +++ b/pkg/util/admission/testdata/disk_bandwidth_limiter @@ -1,18 +1,18 @@ init ---- -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) # Since token utilization is from the previous estimation loop, we expect that to be +Inf in the first iteration. -compute int-read-bytes=50 int-write-bytes=100 int-provisioned-bytes=1000 regular-tokens-used=60 elastic-tokens-used=20 +compute int-read-bytes=50 int-write-bytes=100 int-provisioned-bytes=1000 regular-tokens-used=60 snapshot-tokens-used=5 elastic-tokens-used=20 ---- -diskBandwidthLimiter (tokenUtilization +Inf, tokensUsed (elastic 20 B, regular 60 B) tokens (write 850 B (prev 0 B)), writeBW 6 B/s, readBW 3 B/s, provisioned 66 B/s) +diskBandwidthLimiter (tokenUtilization +Inf, tokensUsed (elastic 20 B, snapshot 5 B, regular 60 B) tokens (write 850 B (prev 0 B)), writeBW 6 B/s, readBW 3 B/s, provisioned 66 B/s) # Utilization is now not Inf. For token calculation, we adjust using the max of smoothed read bytes and the current value. Here we use 80. -compute int-read-bytes=80 int-write-bytes=150 int-provisioned-bytes=800 regular-tokens-used=100 elastic-tokens-used=30 +compute int-read-bytes=80 int-write-bytes=150 int-provisioned-bytes=800 regular-tokens-used=100 snapshot-tokens-used=5 elastic-tokens-used=30 ---- -diskBandwidthLimiter (tokenUtilization 0.15, tokensUsed (elastic 30 B, regular 100 B) tokens (write 640 B (prev 850 B)), writeBW 10 B/s, readBW 5 B/s, provisioned 53 B/s) +diskBandwidthLimiter (tokenUtilization 0.16, tokensUsed (elastic 30 B, snapshot 5 B, regular 100 B) tokens (write 640 B (prev 850 B)), writeBW 10 B/s, readBW 5 B/s, provisioned 53 B/s) # Now we use 65 for read bandwidth adjustment. -compute int-read-bytes=50 int-write-bytes=100 int-provisioned-bytes=800 regular-tokens-used=10 elastic-tokens-used=30 +compute int-read-bytes=50 int-write-bytes=100 int-provisioned-bytes=800 regular-tokens-used=10 snapshot-tokens-used=5 elastic-tokens-used=30 ---- -diskBandwidthLimiter (tokenUtilization 0.06, tokensUsed (elastic 30 B, regular 10 B) tokens (write 655 B (prev 640 B)), writeBW 6 B/s, readBW 3 B/s, provisioned 53 B/s) +diskBandwidthLimiter (tokenUtilization 0.07, tokensUsed (elastic 30 B, snapshot 5 B, regular 10 B) tokens (write 655 B (prev 640 B)), writeBW 6 B/s, readBW 3 B/s, provisioned 53 B/s) diff --git a/pkg/util/admission/testdata/granter b/pkg/util/admission/testdata/granter index 97a2ad92f357..d859aedaf0c6 100644 --- a/pkg/util/admission/testdata/granter +++ b/pkg/util/admission/testdata/granter @@ -729,3 +729,98 @@ set-tokens io-tokens=50 disk-write-tokens=110 tick-interval=250 ---- GrantCoordinator: (chain: id: 0 active: false index: 5) io-avail: 50(50), disk-write-tokens-avail: 11 + + +##################################################################### +# Test store grant coordinator with snapshot ingests +init-store-grant-coordinator +---- +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: 153722867280912930(153722867280912930), disk-write-tokens-avail: 153722867280912930 + +set-tokens io-tokens=50 disk-write-tokens=110 tick-interval=250 +---- +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: 50(50), disk-write-tokens-avail: 110 + +# Try get disk write tokens for snapshots. +try-get work=kv-snapshot v=50 +---- +kv-snapshot: tryGet(50) returned true +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: 50(50), disk-write-tokens-avail: 60 + +# Set waiting requests for all store requesters. Then test priority ordering of grants. +set-has-waiting-requests work=kv v=true +---- +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: 50(50), disk-write-tokens-avail: 60 + +set-return-value-from-granted work=kv v=10 +---- +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: 50(50), disk-write-tokens-avail: 60 + +set-has-waiting-requests work=kv-elastic v=true +---- +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: 50(50), disk-write-tokens-avail: 60 + +set-return-value-from-granted work=kv-elastic v=5 +---- +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: 50(50), disk-write-tokens-avail: 60 + +set-has-waiting-requests work=kv-snapshot v=true +---- +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: 50(50), disk-write-tokens-avail: 60 + +set-return-value-from-granted work=kv-snapshot v=20 +---- +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: 50(50), disk-write-tokens-avail: 60 + +set-tokens io-tokens=50 disk-write-tokens=10 tick-interval=250 +---- +kv-regular: granted in chain 0, and returning 10 +kv-regular: granted in chain 0, and returning 10 +kv-regular: granted in chain 0, and returning 10 +kv-regular: granted in chain 0, and returning 10 +kv-regular: granted in chain 0, and returning 10 +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: 0(0), disk-write-tokens-avail: -40 + +set-has-waiting-requests work=kv v=false +---- +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: 0(0), disk-write-tokens-avail: -40 + +set-tokens io-tokens=50 disk-write-tokens=50 tick-interval=250 +---- +kv-snapshot: granted in chain 0, and returning 20 +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: 50(50), disk-write-tokens-avail: -10 + +set-has-waiting-requests work=kv-snapshot v=false +---- +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: 50(50), disk-write-tokens-avail: -10 + +set-tokens io-tokens=10 disk-write-tokens=15 tick-interval=250 +---- +kv-elastic: granted in chain 0, and returning 5 +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: 5(5), disk-write-tokens-avail: 0 + +set-has-waiting-requests work=kv-elastic v=false +---- +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: 5(5), disk-write-tokens-avail: 0 + +# Return grant for snapshots, should only return disk tokens. +return-grant work=kv-snapshot v=20 +---- +kv-snapshot: returnGrant(20) +GrantCoordinator: +(chain: id: 0 active: false index: 5) io-avail: 5(5), disk-write-tokens-avail: 20 diff --git a/pkg/util/admission/testdata/io_load_listener b/pkg/util/admission/testdata/io_load_listener index f45e0667d1ae..4ef47e2f7c2a 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) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false tick: 1, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false @@ -86,7 +86,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) + 98 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 2.25x+1 B (smoothed 2.00x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 5 B, compacted 98 KiB [≈49 KiB], flushed 1.4 MiB [≈0 B] (mult 1.00); admitting 12 KiB (rate 833 B/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:101000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:101000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:12500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:12500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:5} l0WriteLM:{multiplier:2 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:2.25 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:100000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 5 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.00x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -159,7 +159,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) + 98 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 2.25x+1 B (smoothed 2.12x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 7 B, compacted 98 KiB [≈73 KiB], flushed 1.4 MiB [≈0 B] (mult 1.00); admitting 24 KiB (rate 1.6 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:201000 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:75000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:7} l0WriteLM:{multiplier:2.125 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:2.25 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:100000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 7 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.12x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -228,7 +228,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) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 2.12x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 7 B, compacted 0 B [≈37 KiB], flushed 0 B [≈0 B] (mult 1.00); admitting 21 KiB (rate 1.4 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:201000 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:37500 smoothedCompactionByteTokens:21875 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:21875 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:7} l0WriteLM:{multiplier:2.125 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 7 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.12x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -243,7 +243,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) + 293 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 3.00x+18 B (smoothed 2.56x+9 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 18 B, compacted 293 KiB [≈165 KiB], flushed 4.3 MiB [≈0 B] (mult 1.00); admitting 110 KiB (rate 7.3 KiB/s) (elastic 62 KiB rate 4.1 KiB/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:5000000000 IdleDuration:500000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:168750 smoothedCompactionByteTokens:112187.5 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:112187 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:63281 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:18} l0WriteLM:{multiplier:2.5625 constant:9} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:300000 intL0CompactedBytes:300000 intFlushTokens:4.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:300000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:3 constant:18} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:300000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 18 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.56x+9 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -256,7 +256,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) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 2.56x+4 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 18 B, compacted 0 B [≈82 KiB], flushed 0 B [≈0 B] (mult 1.00); admitting 129 KiB (rate 8.6 KiB/s) (elastic 62 KiB rate 4.1 KiB/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:6000000000 IdleDuration:600000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:84375 smoothedCompactionByteTokens:132031.25 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:132031 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:63281 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:18} l0WriteLM:{multiplier:2.5625 constant:4} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 18 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.56x+4 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -269,7 +269,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) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 2.56x+2 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 18 B, compacted 0 B [≈41 KiB], flushed 0 B [≈0 B] (mult 1.00); admitting elastic 46 KiB (rate 3.1 KiB/s) due to L0 growth; write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:7000000000 IdleDuration:700000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:42187 smoothedCompactionByteTokens:66015.625 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:47460 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:18} l0WriteLM:{multiplier:2.5625 constant:2} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 18 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.56x+2 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -287,7 +287,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) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false @@ -305,7 +305,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) + 195 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 1.31x+1 B (smoothed 1.53x+1 B) + ingested-model 1.50x+1 B (smoothed 1.12x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 195 KiB [≈98 KiB], flushed 2.4 MiB [≈0 B] (mult 1.00); admitting 24 KiB (rate 1.6 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:171000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.5288076923076923 constant:1} l0IngestLM:{multiplier:1.125 constant:1} ingestLM:{multiplier:1.2497500000000001 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:200000 intFlushTokens:2.55e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:170000 intL0IngestedBytes:30000 intLSMIngestedBytes:30000 intL0WriteAccountedBytes:130000 intIngestedAccountedBytes:20000 intL0WriteLinearModel:{multiplier:1.3076153846153846 constant:1} intL0IngestedLinearModel:{multiplier:1.4995 constant:1} intIngestedLinearModel:{multiplier:1.4995 constant:1} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:200000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.53x+1 l0-ingest-lm: 1.12x+1 ingest-lm: 1.25x+1 write-amp-lm: 50.50x+1 @@ -321,7 +321,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) + 20 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 1.00x+1 B (smoothed 1.26x+1 B) + ingested-model 0.00x+0 B (smoothed 1.12x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 4.1 KiB, compacted 20 KiB [≈59 KiB], flushed 293 KiB [≈0 B] (mult 1.00); admitting 27 KiB (rate 1.8 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:221000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:191000 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:60000 smoothedCompactionByteTokens:27500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:27500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:4195} l0WriteLM:{multiplier:1.2641538461538462 constant:1} l0IngestLM:{multiplier:1.125 constant:1} ingestLM:{multiplier:1.2497500000000001 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:20000 intL0CompactedBytes:20000 intFlushTokens:300000 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:20000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:20000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0.9995 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:20000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 4195 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.26x+1 l0-ingest-lm: 1.12x+1 ingest-lm: 1.25x+1 write-amp-lm: 50.50x+1 @@ -336,7 +336,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) + 20 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.50x+1 B (smoothed 0.88x+1 B) + ingested-model 0.00x+0 B (smoothed 1.12x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 3.0 KiB, compacted 20 KiB [≈39 KiB], flushed 293 KiB [≈0 B] (mult 1.00); admitting 23 KiB (rate 1.5 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:241000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:211000 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:40000 smoothedCompactionByteTokens:23750 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:23750 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:3097} l0WriteLM:{multiplier:0.8820769230769231 constant:1} l0IngestLM:{multiplier:1.125 constant:1} ingestLM:{multiplier:1.2497500000000001 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:20000 intL0CompactedBytes:20000 intFlushTokens:300000 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:20000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:100000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0.5 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:20000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 3097 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 0.88x+1 l0-ingest-lm: 1.12x+1 ingest-lm: 1.25x+1 write-amp-lm: 50.50x+1 @@ -353,7 +353,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) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false @@ -365,7 +365,7 @@ prep-admission-stats admitted=10 write-bytes=200000 set-state l0-bytes=1000 l0-added-write=201000 l0-files=21 l0-sublevels=21 print-only-first-tick=true ---- compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 195 KiB (write 195 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10 (0 bypassed) with 195 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 195 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 1.00x+1 B (smoothed 1.37x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 9.8 KiB, compacted 195 KiB [≈98 KiB], flushed 2.9 MiB [≈0 B] (mult 1.00); admitting 24 KiB (rate 1.6 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:201000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:10000} l0WriteLM:{multiplier:1.374975 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:200000 intFlushTokens:3e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:200000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:200000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0.99995 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:200000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 10000 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.37x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -379,7 +379,7 @@ prep-admission-stats admitted=10 write-bytes=200000 set-state l0-bytes=201000 l0-added-write=401000 l0-files=41 l0-sublevels=41 print-only-first-tick=true wal-secondary-write-sec=1 ---- compaction score 2.050[L0-overload] (41 ssts, 41 sub-levels), L0 growth 195 KiB (write 195 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 195 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.37x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 9.8 KiB, compacted 0 B [≈98 KiB], flushed 2.9 MiB [≈0 B] (mult 1.00); admitting (WAL failover) 24 KiB (rate 1.6 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:401000 curL0Bytes:201000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:401000 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:1000000000 smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:10000} l0WriteLM:{multiplier:1.374975 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:0 intFlushTokens:3e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:true prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:200000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:200000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 10000 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.37x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -397,7 +397,7 @@ prep-admission-stats admitted=10 write-bytes=200000 set-state l0-bytes=401000 l0-added-write=601000 l0-files=61 l0-sublevels=61 print-only-first-tick=true wal-secondary-write-sec=1 ---- compaction score 3.050[L0-overload] (61 ssts, 61 sub-levels), L0 growth 195 KiB (write 195 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 195 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.37x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 9.8 KiB, compacted 0 B [≈98 KiB], flushed 2.9 MiB [≈0 B] (mult 1.00); admitting (WAL failover) all; write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:601000 curL0Bytes:401000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:601000 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:2000000000 smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:10000} l0WriteLM:{multiplier:1.374975 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:0 intFlushTokens:3e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:true prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:200000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:200000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 10000 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.37x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -420,7 +420,7 @@ prep-admission-stats admitted=10 write-bytes=200000 set-state l0-bytes=601000 l0-added-write=801000 l0-files=61 l0-sublevels=5 print-only-first-tick=true wal-secondary-write-sec=1 flush-bytes=1000 flush-work-sec=8 flush-idle-sec=10 write-stall-count=4 ---- compaction score 0.250 (61 ssts, 5 sub-levels), L0 growth 195 KiB (write 195 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 195 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.37x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 9.8 KiB, compacted 0 B [≈98 KiB], flushed 366 KiB [≈0 B] (mult 1.00); admitting (WAL failover) 24 KiB (rate 1.6 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 4 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:801000 curL0Bytes:601000 cumWriteStallCount:4 cumFlushWriteThroughput:{Bytes:801000 WorkDuration:12000000000 IdleDuration:410000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:3000000000 smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:10000} l0WriteLM:{multiplier:1.374975 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:0 intFlushTokens:375000 intFlushUtilization:0.4444444444444444 intWriteStalls:4 intWALFailover:true prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:200000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:200000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 10000 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.37x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -437,7 +437,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) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false @@ -446,7 +446,7 @@ tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk set-state l0-bytes=10000 l0-added-write=2000 l0-files=1 l0-sublevels=1 flush-bytes=1000 flush-work-sec=2 flush-idle-sec=100 print-only-first-tick=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 1000 B (write 1000 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 1000 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 1000 B [≈500 B], flushed 7.3 KiB [≈0 B] (mult 1.00); admitting all; write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:2000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:2000 WorkDuration:3000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:500 smoothedCompactionByteTokens:500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:1000 intL0CompactedBytes:1000 intFlushTokens:7500 intFlushUtilization:0.0196078431372549 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:1000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:1000 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -456,7 +456,7 @@ setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-token set-state l0-bytes=10000 l0-added-write=3000 l0-files=1 l0-sublevels=1 flush-bytes=1000 flush-work-sec=2 flush-idle-sec=10 print-only-first-tick=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 1000 B (write 1000 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 1000 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 1000 B [≈750 B], flushed 7.3 KiB [≈7.3 KiB] (mult 1.00); admitting 7.3 KiB (rate 500 B/s) (elastic 5.9 KiB rate 400 B/s) due to memtable flush (multiplier 1.000) (used total: 0 B elastic 0 B); write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:3000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:3000 WorkDuration:5000000000 IdleDuration:210000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:750 smoothedCompactionByteTokens:750 smoothedNumFlushTokens:7500 flushUtilTargetFraction:1 totalNumByteTokens:7500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:6000 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:1000 intL0CompactedBytes:1000 intFlushTokens:7500 intFlushUtilization:0.16666666666666666 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:1000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:1000 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -468,7 +468,7 @@ setAvailableTokens: io-tokens=125(elastic 100) elastic-disk-bw-tokens=unlimited set-state l0-bytes=10000 l0-added-write=13000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=1 print-only-first-tick=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈5.2 KiB], flushed 73 KiB [≈40 KiB] (mult 0.97); admitting 39 KiB (rate 2.6 KiB/s) (elastic 31 KiB rate 2.1 KiB/s) due to memtable flush (multiplier 0.975) (used total: 0 B elastic 0 B); write stalls 1 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:13000 curL0Bytes:10000 cumWriteStallCount:1 cumFlushWriteThroughput:{Bytes:13000 WorkDuration:7000000000 IdleDuration:220000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:5375 smoothedCompactionByteTokens:5375 smoothedNumFlushTokens:41250 flushUtilTargetFraction:0.975 totalNumByteTokens:40218 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:32174 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -480,7 +480,7 @@ setAvailableTokens: io-tokens=671(elastic 537) elastic-disk-bw-tokens=unlimited set-state l0-bytes=10000 l0-added-write=23000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=3 print-only-first-tick=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈7.5 KiB], flushed 73 KiB [≈57 KiB] (mult 0.92); admitting 52 KiB (rate 3.5 KiB/s) (elastic 42 KiB rate 2.8 KiB/s) due to memtable flush (multiplier 0.925) (used total: 0 B elastic 0 B); write stalls 2 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:23000 curL0Bytes:10000 cumWriteStallCount:3 cumFlushWriteThroughput:{Bytes:23000 WorkDuration:9000000000 IdleDuration:230000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:7687 smoothedCompactionByteTokens:7687.5 smoothedNumFlushTokens:58125 flushUtilTargetFraction:0.9249999999999999 totalNumByteTokens:53765 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:43012 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:2 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -491,7 +491,7 @@ setAvailableTokens: io-tokens=897(elastic 717) elastic-disk-bw-tokens=unlimited set-state l0-bytes=10000 l0-added-write=33000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=8 print-only-first-tick=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈8.6 KiB], flushed 73 KiB [≈65 KiB] (mult 0.85); admitting 55 KiB (rate 3.7 KiB/s) (elastic 44 KiB rate 2.9 KiB/s) due to memtable flush (multiplier 0.850) (used total: 0 B elastic 0 B); write stalls 5 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:33000 curL0Bytes:10000 cumWriteStallCount:8 cumFlushWriteThroughput:{Bytes:33000 WorkDuration:11000000000 IdleDuration:240000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:8843 smoothedCompactionByteTokens:8843.75 smoothedNumFlushTokens:66562.5 flushUtilTargetFraction:0.8499999999999999 totalNumByteTokens:56578 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:45262 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:5 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -501,7 +501,7 @@ setAvailableTokens: io-tokens=943(elastic 755) elastic-disk-bw-tokens=unlimited set-state l0-bytes=10000 l0-added-write=43000 l0-files=1 l0-sublevels=2 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=9 print-only-first-tick=true ---- compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.2 KiB], flushed 73 KiB [≈69 KiB] (mult 0.82); admitting 57 KiB (rate 3.8 KiB/s) (elastic 12 KiB rate 785 B/s) due to memtable flush (multiplier 0.825) (used total: 0 B elastic 0 B); write stalls 1 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:43000 curL0Bytes:10000 cumWriteStallCount:9 cumFlushWriteThroughput:{Bytes:43000 WorkDuration:13000000000 IdleDuration:250000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9421 smoothedCompactionByteTokens:9421.875 smoothedNumFlushTokens:70781.25 flushUtilTargetFraction:0.8249999999999998 totalNumByteTokens:58394 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:11776 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -516,7 +516,7 @@ set-min-flush-util percent=130 set-state l0-bytes=10000 l0-added-write=53000 l0-files=1 l0-sublevels=2 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=10 print-only-first-tick=true ---- compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.5 KiB], flushed 73 KiB [≈71 KiB] (mult 1.30); admitting 92 KiB (rate 6.2 KiB/s) (elastic 12 KiB rate 809 B/s) due to memtable flush (multiplier 1.300) (used total: 0 B elastic 0 B); write stalls 1 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:53000 curL0Bytes:10000 cumWriteStallCount:10 cumFlushWriteThroughput:{Bytes:53000 WorkDuration:15000000000 IdleDuration:260000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9710 smoothedCompactionByteTokens:9710.9375 smoothedNumFlushTokens:72890.625 flushUtilTargetFraction:1.3 totalNumByteTokens:94757 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12137 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -527,7 +527,7 @@ setAvailableTokens: io-tokens=1580(elastic 203) elastic-disk-bw-tokens=unlimited set-state l0-bytes=10000 l0-added-write=63000 l0-files=1 l0-sublevels=2 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=11 print-only-first-tick=true ---- compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.6 KiB], flushed 73 KiB [≈72 KiB] (mult 1.30); admitting 94 KiB (rate 6.3 KiB/s) (elastic 12 KiB rate 821 B/s) due to memtable flush (multiplier 1.300) (used total: 0 B elastic 0 B); write stalls 1 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:63000 curL0Bytes:10000 cumWriteStallCount:11 cumFlushWriteThroughput:{Bytes:63000 WorkDuration:17000000000 IdleDuration:270000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9855 smoothedCompactionByteTokens:9855.46875 smoothedNumFlushTokens:73945.3125 flushUtilTargetFraction:1.3 totalNumByteTokens:96128 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12318 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -543,7 +543,7 @@ set-min-flush-util percent=135 set-state l0-bytes=10000 l0-added-write=73000 l0-files=1 l0-sublevels=2 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=12 print-only-first-tick=true ---- compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.7 KiB], flushed 73 KiB [≈73 KiB] (mult 1.35); admitting 98 KiB (rate 6.5 KiB/s) (elastic 12 KiB rate 827 B/s) due to memtable flush (multiplier 1.350) (used total: 0 B elastic 0 B); write stalls 1 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:73000 curL0Bytes:10000 cumWriteStallCount:12 cumFlushWriteThroughput:{Bytes:73000 WorkDuration:19000000000 IdleDuration:280000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9927 smoothedCompactionByteTokens:9927.734375 smoothedNumFlushTokens:74472.65625 flushUtilTargetFraction:1.35 totalNumByteTokens:100538 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12408 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -553,7 +553,7 @@ setAvailableTokens: io-tokens=1676(elastic 207) elastic-disk-bw-tokens=unlimited set-state l0-bytes=10000 l0-added-write=83000 l0-files=1 l0-sublevels=2 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=100 write-stall-count=13 print-only-first-tick=true ---- compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.7 KiB], flushed 73 KiB [≈73 KiB] (mult 1.35); admitting elastic 12 KiB (rate 830 B/s) due to L0 growth; write stalls 1 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:83000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:83000 WorkDuration:21000000000 IdleDuration:380000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9963 smoothedCompactionByteTokens:9963.8671875 smoothedNumFlushTokens:74472.65625 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12453 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.0196078431372549 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -563,7 +563,7 @@ setAvailableTokens: io-tokens=unlimited(elastic 208) elastic-disk-bw-tokens=unli set-state l0-bytes=10000 l0-added-write=93000 l0-files=1 l0-sublevels=2 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=13 print-only-first-tick=true ---- compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.7 KiB], flushed 73 KiB [≈73 KiB] (mult 1.35); admitting 98 KiB (rate 6.6 KiB/s) (elastic 12 KiB rate 831 B/s) due to memtable flush (multiplier 1.350) (used total: 0 B elastic 0 B); write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:93000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:93000 WorkDuration:23000000000 IdleDuration:390000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9981 smoothedCompactionByteTokens:9981.93359375 smoothedNumFlushTokens:74736.328125 flushUtilTargetFraction:1.35 totalNumByteTokens:100894 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12476 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -574,7 +574,7 @@ setAvailableTokens: io-tokens=1682(elastic 208) elastic-disk-bw-tokens=unlimited set-state l0-bytes=10000 l0-added-write=103000 l0-files=1 l0-sublevels=2 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=13 all-tokens-used=true print-only-first-tick=true ---- compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.8 KiB], flushed 73 KiB [≈73 KiB] (mult 1.35); admitting 99 KiB (rate 6.6 KiB/s) (elastic 12 KiB rate 832 B/s) due to memtable flush (multiplier 1.350) (used total: 0 B elastic 0 B); write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:103000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:103000 WorkDuration:25000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9990 smoothedCompactionByteTokens:9990.966796875 smoothedNumFlushTokens:74868.1640625 flushUtilTargetFraction:1.35 totalNumByteTokens:101072 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12487 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -585,7 +585,7 @@ setAvailableTokens: io-tokens=1685(elastic 209) elastic-disk-bw-tokens=unlimited set-state l0-bytes=10000 l0-added-write=113000 l0-files=1 l0-sublevels=2 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=13 all-tokens-used=true print-only-first-tick=true ---- compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.8 KiB], flushed 73 KiB [≈73 KiB] (mult 1.38); admitting 101 KiB (rate 6.7 KiB/s) (elastic 12 KiB rate 832 B/s) due to memtable flush (multiplier 1.375) (used total: 197 KiB elastic 0 B); write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:113000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:113000 WorkDuration:27000000000 IdleDuration:410000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9995 smoothedCompactionByteTokens:9995.4833984375 smoothedNumFlushTokens:74934.08203125 flushUtilTargetFraction:1.375 totalNumByteTokens:103034 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12493 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 intWALFailover:false prevTokensUsed:202144 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -596,7 +596,7 @@ setAvailableTokens: io-tokens=1718(elastic 209) elastic-disk-bw-tokens=unlimited set-state l0-bytes=10000 l0-added-write=123000 l0-files=1 l0-sublevels=2 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=13 all-tokens-used=true print-only-first-tick=true ---- compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.8 KiB], flushed 73 KiB [≈73 KiB] (mult 1.40); admitting 102 KiB (rate 6.8 KiB/s) (elastic 12 KiB rate 833 B/s) due to memtable flush (multiplier 1.400) (used total: 201 KiB elastic 0 B); write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:123000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:123000 WorkDuration:29000000000 IdleDuration:420000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9997 smoothedCompactionByteTokens:9997.74169921875 smoothedNumFlushTokens:74967.041015625 flushUtilTargetFraction:1.4 totalNumByteTokens:104953 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12496 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 intWALFailover:false prevTokensUsed:206068 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -607,7 +607,7 @@ setAvailableTokens: io-tokens=1750(elastic 209) elastic-disk-bw-tokens=unlimited set-state l0-bytes=10000 l0-added-write=133000 l0-files=1 l0-sublevels=2 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=14 all-tokens-used=true print-only-first-tick=true ---- compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.8 KiB], flushed 73 KiB [≈73 KiB] (mult 1.38); admitting 101 KiB (rate 6.7 KiB/s) (elastic 12 KiB rate 833 B/s) due to memtable flush (multiplier 1.375) (used total: 205 KiB elastic 0 B); write stalls 1 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:133000 curL0Bytes:10000 cumWriteStallCount:14 cumFlushWriteThroughput:{Bytes:133000 WorkDuration:31000000000 IdleDuration:430000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9998 smoothedCompactionByteTokens:9998.870849609375 smoothedNumFlushTokens:74983.5205078125 flushUtilTargetFraction:1.375 totalNumByteTokens:103102 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12497 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:209906 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -625,7 +625,7 @@ prep-admission-stats admitted=0 set-state l0-bytes=0 l0-added-write=0 bytes-read=0 bytes-written=0 provisioned-bandwidth=0 l0-files=1 l0-sublevels=1 print-only-first-tick=true ---- compaction score 0.000 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:0 curL0Bytes:0 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false @@ -639,7 +639,7 @@ prep-admission-stats admitted=10 write-bytes=10 set-state l0-bytes=10 l0-added-write=10 bytes-read=60 bytes-written=50 provisioned-bandwidth=100 disk-write-tokens-used=(100,100) l0-files=1 l0-sublevels=1 print-only-first-tick=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 10 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 150 B [≈0 B] (mult 1.35); admitting all; write stalls 0 -diskBandwidthLimiter (tokenUtilization +Inf, tokensUsed (elastic 100 B, regular 100 B) tokens (write 1.4 KiB (prev 0 B)), writeBW 3 B/s, readBW 4 B/s, provisioned 100 B/s) +diskBandwidthLimiter (tokenUtilization +Inf, tokensUsed (elastic 0 B, snapshot 100 B, regular 100 B) tokens (write 1.4 KiB (prev 0 B)), writeBW 3 B/s, readBW 4 B/s, provisioned 100 B/s) {ioLoadListenerState:{cumL0AddedBytes:10 curL0Bytes:10 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:10 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:60 bytesWritten:50} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:1440 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10 intL0CompactedBytes:0 intFlushTokens:150 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -652,7 +652,7 @@ prep-admission-stats admitted=20 write-bytes=20 set-state l0-bytes=40 l0-added-write=40 bytes-read=120 bytes-written=400 provisioned-bandwidth=100 disk-write-tokens-used=(200,200) l0-files=1 l0-sublevels=1 print-only-first-tick=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 30 B (write 30 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10 (0 bypassed) with 10 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 30 B adjusted-LSM-writes + 350 B adjusted-disk-writes + write-model 2.00x+1 B (smoothed 1.88x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 11.33x+1 B (smoothed 30.92x+1 B) + at-admission-tokens 2 B, compacted 0 B [≈0 B], flushed 435 B [≈0 B] (mult 1.35); admitting all; write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.28, tokensUsed (elastic 200 B, regular 200 B) tokens (write 1.4 KiB (prev 1.4 KiB)), writeBW 23 B/s, readBW 4 B/s, provisioned 100 B/s) +diskBandwidthLimiter (tokenUtilization 0.28, tokensUsed (elastic 0 B, snapshot 200 B, regular 200 B) tokens (write 1.4 KiB (prev 1.4 KiB)), writeBW 23 B/s, readBW 4 B/s, provisioned 100 B/s) {ioLoadListenerState:{cumL0AddedBytes:40 curL0Bytes:40 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:40 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:120 bytesWritten:400} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:1440 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:2} l0WriteLM:{multiplier:1.875 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:30.916666666666668 constant:1} aux:{intL0AddedBytes:30 intL0CompactedBytes:0 intFlushTokens:435 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:30 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:10 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:2 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:11.333333333333334 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:30 intAdjustedDiskWriteBytes:350} doLogFlush:false} ioThreshold:} store-request-estimates: writeTokens: 2 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.88x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 30.92x+1 @@ -665,7 +665,7 @@ prep-admission-stats admitted=60 write-bytes=70 set-state l0-bytes=90 l0-added-write=80 bytes-read=180 bytes-written=1200 provisioned-bandwidth=100 disk-write-tokens-used=(400,500) l0-files=1 l0-sublevels=2 print-only-first-tick=true ---- compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 40 B (write 40 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 40 (0 bypassed) with 50 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 40 B adjusted-LSM-writes + 800 B adjusted-disk-writes + write-model 0.50x+1 B (smoothed 1.19x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 19.00x+1 B (smoothed 24.96x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 600 B [≈0 B] (mult 1.35); admitting elastic 1 B (rate 0 B/s) due to L0 growth; write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.62, tokensUsed (elastic 500 B, regular 400 B) tokens (write 1.4 KiB (prev 1.4 KiB)), writeBW 53 B/s, readBW 4 B/s, provisioned 100 B/s) +diskBandwidthLimiter (tokenUtilization 0.62, tokensUsed (elastic 0 B, snapshot 500 B, regular 400 B) tokens (write 1.4 KiB (prev 1.4 KiB)), writeBW 53 B/s, readBW 4 B/s, provisioned 100 B/s) {ioLoadListenerState:{cumL0AddedBytes:80 curL0Bytes:90 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:80 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:180 bytesWritten:1200} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:1440 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.1875 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:24.958333333333336 constant:1} aux:{intL0AddedBytes:40 intL0CompactedBytes:0 intFlushTokens:600 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:40 intL0WriteBytes:40 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:50 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0.5 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:19 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:40 intAdjustedDiskWriteBytes:800} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.19x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 24.96x+1 @@ -683,7 +683,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) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false @@ -698,7 +698,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) + 98 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 2.25x+1 B (smoothed 2.00x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 5 B, compacted 98 KiB [≈49 KiB], flushed 1.4 MiB [≈0 B] (mult 1.35); admitting 12 KiB (rate 833 B/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:101000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:101000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:12500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:12500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:5} l0WriteLM:{multiplier:2 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:2.25 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:100000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 5 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.00x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -711,14 +711,14 @@ 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) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:0 curL0Bytes:100 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false set-state l0-bytes=100 l0-added-write=100000 bytes-read=1000000 bytes-written=2000000 provisioned-bandwidth=10 disk-bw-tokens-used=(100,100) l0-files=1 l0-sublevels=1 print-only-first-tick=true loaded=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 98 KiB (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 98 KiB [≈49 KiB], flushed 1.4 MiB [≈0 B] (mult 1.35); admitting all; write stalls 0 -diskBandwidthLimiter (tokenUtilization NaN, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 130 KiB/s, readBW 65 KiB/s, provisioned 10 B/s) +diskBandwidthLimiter (tokenUtilization NaN, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 130 KiB/s, readBW 65 KiB/s, provisioned 10 B/s) {ioLoadListenerState:{cumL0AddedBytes:100000 curL0Bytes:100 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:100000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:1000000 bytesWritten:2000000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:50000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:0 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -735,7 +735,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) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false @@ -747,7 +747,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) + 488 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 488 KiB [≈244 KiB], flushed 7.2 MiB [≈0 B] (mult 1.35); admitting all; write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:250000 smoothedCompactionByteTokens:250000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:500000 intL0CompactedBytes:500000 intFlushTokens:7.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:500000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:500000 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -764,7 +764,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) + 2 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 2 B [≈122 KiB], flushed 30 B [≈0 B] (mult 1.35); admitting 183 KiB (rate 12 KiB/s) (elastic 30 KiB rate 2.0 KiB/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:501002 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501002 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:125001 smoothedCompactionByteTokens:187500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:187500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:31250 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:2 intL0CompactedBytes:2 intFlushTokens:30 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:2 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:2 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -778,7 +778,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) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈61 KiB], flushed 0 B [≈0 B] (mult 1.35); admitting 114 KiB (rate 7.6 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:62500 smoothedCompactionByteTokens:117187.5 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:117187 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:-2 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -793,7 +793,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) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈30 KiB], flushed 0 B [≈0 B] (mult 1.35); admitting 65 KiB (rate 4.3 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:5000000000 IdleDuration:500000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:31250 smoothedCompactionByteTokens:66406.25 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:66406 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -806,7 +806,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) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈15 KiB], flushed 0 B [≈0 B] (mult 1.35); admitting 48 KiB (rate 3.2 KiB/s) (elastic 13 KiB rate 911 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:6000000000 IdleDuration:600000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:15625 smoothedCompactionByteTokens:48828.125 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:48828 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:13671 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -823,7 +823,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 ---- compaction score 0.000 (21 ssts, 21 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false @@ -831,7 +831,7 @@ tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk 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) + 98 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 98 KiB [≈49 KiB], flushed 1.4 MiB [≈0 B] (mult 1.35); admitting 12 KiB (rate 833 B/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:101000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:101000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:12500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:12500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:100000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -844,7 +844,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) + 98 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 98 KiB [≈73 KiB], flushed 1.4 MiB [≈0 B] (mult 1.35); admitting 65 KiB (rate 4.3 KiB/s) (elastic 46 KiB rate 3.1 KiB/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:201000 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:75000 smoothedCompactionByteTokens:66250 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:66250 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:46875 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:100000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -854,7 +854,7 @@ 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) + 59 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 20 KiB [≈46 KiB], flushed 879 KiB [≈0 B] (mult 1.35); admitting 56 KiB (rate 3.7 KiB/s) (elastic 12 KiB rate 791 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:260994 curL0Bytes:50000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:260994 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:47497 smoothedCompactionByteTokens:56873 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:56873 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:11874 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:59994 intL0CompactedBytes:19994 intFlushTokens:899910 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:59994 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:59994 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -871,7 +871,7 @@ prep-admission-stats admitted=0 set-state l0-bytes=10000 l0-added-write=1000 l0-files=100 l0-sublevels=6 print-only-first-tick=true ---- compaction score 0.000 (100 ssts, 6 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false @@ -879,7 +879,7 @@ tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk 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) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 1.35); admitting 244 KiB (rate 16 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth(used token lower bound) (used total: 0 B elastic 0 B); write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:1000000 numOutLevelsGauge:2} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:250000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:250000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:true perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -889,7 +889,7 @@ setAvailableTokens: io-tokens=4167(elastic 1) elastic-disk-bw-tokens=unlimited m 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) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 1.35); admitting 366 KiB (rate 24 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth(used token lower bound) (used total: 0 B elastic 0 B); write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:2000000 numOutLevelsGauge:2} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:375000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:375000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:true perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -899,7 +899,7 @@ setAvailableTokens: io-tokens=6250(elastic 1) elastic-disk-bw-tokens=unlimited m 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) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 1.35); admitting 427 KiB (rate 28 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth(used token lower bound) (used total: 0 B elastic 0 B); write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:3000000 numOutLevelsGauge:2} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:437500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:437500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:true perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 @@ -912,7 +912,7 @@ setAvailableTokens: io-tokens=7292(elastic 1) elastic-disk-bw-tokens=unlimited m 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) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 1.35); admitting 214 KiB (rate 14 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 -diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) {ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:5000000000 IdleDuration:500000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:3000000 numOutLevelsGauge:2} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:218750 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:218750 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 elasticDiskWriteTokens:9223372036854775807 elasticDiskWriteTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 diff --git a/pkg/util/admission/testdata/snapshot_queue b/pkg/util/admission/testdata/snapshot_queue new file mode 100644 index 000000000000..a5e7ef9bb3e3 --- /dev/null +++ b/pkg/util/admission/testdata/snapshot_queue @@ -0,0 +1,71 @@ +# Base case, simple happy path, admit 1 request. +init +---- + +set-try-get-return-value v=true +---- + +admit id=1 count=1 create-time-millis=1 +---- +tryGet: returning true +id 1: admit succeeded + +empty +---- +true + +# Now we queue 2 requests. +set-try-get-return-value v=false +---- + +admit id=2 count=1 create-time-millis=2 +---- +tryGet: returning false + +admit id=3 count=5 create-time-millis=3 +---- +tryGet: returning false + +granted +---- +id 2: admit succeeded +granted: returned 1 + +empty +---- +false + +granted +---- +id 3: admit succeeded +granted: returned 5 + +empty +---- +true + +# Test context cancellation behavior. +set-try-get-return-value v=false +---- + +admit id=4 count=10 create-time-millis=5 +---- +tryGet: returning false + +cancel-work id=4 +---- +id 4: admit failed + +admit id=5 count=15 create-time-millis=8 +---- +tryGet: returning false + +# We skip the cancelled work in the queue. +granted +---- +id 5: admit succeeded +granted: returned 15 + +empty +---- +true