Skip to content

Commit

Permalink
admission: add l0 control metrics + settings
Browse files Browse the repository at this point in the history
Part of cockroachdb#82743. We add cluster settings to control:
  - smoothing alpha for byte token computations;
  - reduction factor for L0 compaction tokens, based on observed
    compactions;

We've found these to be useful in internal experiments, and also when
looking to paper over L0 compaction variability effects up in AC.

While here, print out observed smoothed compaction bytes in
io_load_listener logging and introduce metrics for
  - l0 compacted bytes;
  - generated l0 tokens;
  - l0 tokens returned.

Release note: None
  • Loading branch information
irfansharif committed Aug 29, 2023
1 parent be7105f commit 4c2ec63
Show file tree
Hide file tree
Showing 7 changed files with 144 additions and 54 deletions.
27 changes: 21 additions & 6 deletions pkg/util/admission/grant_coordinator.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,9 @@ type StoreGrantCoordinators struct {
kvElasticIOTokensAvailable *metric.Gauge
kvIOTokensTookWithoutPermission *metric.Counter
kvIOTotalTokensTaken *metric.Counter
kvIOTotalTokensReturned *metric.Counter
l0CompactedBytes *metric.Counter
l0TokensProduced *metric.Counter

// These metrics are shared by WorkQueues across stores.
workQueueMetrics *WorkQueueMetrics
Expand Down Expand Up @@ -166,12 +169,13 @@ func (sgc *StoreGrantCoordinators) initGrantCoordinator(storeID roachpb.StoreID)
// Setting tokens to unlimited is defensive. We expect that
// pebbleMetricsTick and allocateIOTokensTick will get called during
// initialization, which will also set these to unlimited.
startingIOTokens: unlimitedTokens / unloadedDuration.ticksInAdjustmentInterval(),
ioTokensExhaustedDurationMetric: sgc.kvIOTokensExhaustedDuration,
availableTokensMetrics: sgc.kvIOTokensAvailable,
availableElasticTokensMetric: sgc.kvElasticIOTokensAvailable,
tookWithoutPermissionMetric: sgc.kvIOTokensTookWithoutPermission,
totalTokensTaken: sgc.kvIOTotalTokensTaken,
startingIOTokens: unlimitedTokens / unloadedDuration.ticksInAdjustmentInterval(),
ioTokensExhaustedDurationMetric: sgc.kvIOTokensExhaustedDuration,
availableTokensMetric: sgc.kvIOTokensAvailable,
availableElasticTokensMetric: sgc.kvElasticIOTokensAvailable,
tokensTakenWithoutPermissionMetric: sgc.kvIOTokensTookWithoutPermission,
tokensTakenMetric: sgc.kvIOTotalTokensTaken,
tokensReturnedMetric: sgc.kvIOTotalTokensReturned,
}
kvg.coordMu.availableIOTokens = unlimitedTokens / unloadedDuration.ticksInAdjustmentInterval()
kvg.coordMu.availableElasticIOTokens = kvg.coordMu.availableIOTokens
Expand Down Expand Up @@ -215,6 +219,8 @@ func (sgc *StoreGrantCoordinators) initGrantCoordinator(storeID roachpb.StoreID)
perWorkTokenEstimator: makeStorePerWorkTokenEstimator(),
diskBandwidthLimiter: makeDiskBandwidthLimiter(),
kvGranter: kvg,
l0CompactedBytes: sgc.l0CompactedBytes,
l0TokensProduced: sgc.l0TokensProduced,
}
return coord
}
Expand Down Expand Up @@ -468,8 +474,11 @@ func makeStoresGrantCoordinators(
kvIOTokensExhaustedDuration: metrics.KVIOTokensExhaustedDuration,
kvIOTokensTookWithoutPermission: metrics.KVIOTokensTookWithoutPermission,
kvIOTotalTokensTaken: metrics.KVIOTotalTokensTaken,
kvIOTotalTokensReturned: metrics.KVIOTotalTokensReturned,
kvIOTokensAvailable: metrics.KVIOTokensAvailable,
kvElasticIOTokensAvailable: metrics.KVElasticIOTokensAvailable,
l0CompactedBytes: metrics.L0CompactedBytes,
l0TokensProduced: metrics.L0TokensProduced,
workQueueMetrics: storeWorkQueueMetrics,
onLogEntryAdmitted: onLogEntryAdmitted,
knobs: knobs,
Expand Down Expand Up @@ -1015,8 +1024,11 @@ type GrantCoordinatorMetrics struct {
KVIOTokensExhaustedDuration *metric.Counter
KVIOTokensTookWithoutPermission *metric.Counter
KVIOTotalTokensTaken *metric.Counter
KVIOTotalTokensReturned *metric.Counter
KVIOTokensAvailable *metric.Gauge
KVElasticIOTokensAvailable *metric.Gauge
L0CompactedBytes *metric.Counter
L0TokensProduced *metric.Counter
SQLLeafStartUsedSlots *metric.Gauge
SQLRootStartUsedSlots *metric.Gauge
}
Expand All @@ -1038,8 +1050,11 @@ func makeGrantCoordinatorMetrics() GrantCoordinatorMetrics {
SQLRootStartUsedSlots: metric.NewGauge(addName(workKindString(SQLStatementRootStartWork), usedSlots)),
KVIOTokensTookWithoutPermission: metric.NewCounter(kvIONumIOTokensTookWithoutPermission),
KVIOTotalTokensTaken: metric.NewCounter(kvIOTotalTokensTaken),
KVIOTotalTokensReturned: metric.NewCounter(kvIOTotalTokensReturned),
KVIOTokensAvailable: metric.NewGauge(kvIOTokensAvailable),
KVElasticIOTokensAvailable: metric.NewGauge(kvElasticIOTokensAvailable),
L0CompactedBytes: metric.NewCounter(l0CompactedBytes),
L0TokensProduced: metric.NewCounter(l0TokensProduced),
}
return m
}
Expand Down
69 changes: 51 additions & 18 deletions pkg/util/admission/granter.go
Original file line number Diff line number Diff line change
Expand Up @@ -313,13 +313,15 @@ type kvStoreTokenGranter struct {
// startingIOTokens is the number of tokens set by
// setAvailableTokens. It is used to compute the tokens used, by
// computing startingIOTokens-availableIOTokens.
startingIOTokens int64
ioTokensExhaustedDurationMetric *metric.Counter
availableTokensMetrics *metric.Gauge
availableElasticTokensMetric *metric.Gauge
tookWithoutPermissionMetric *metric.Counter
totalTokensTaken *metric.Counter
exhaustedStart time.Time
startingIOTokens int64
ioTokensExhaustedDurationMetric *metric.Counter
availableTokensMetric *metric.Gauge
availableElasticTokensMetric *metric.Gauge
tokensReturnedMetric *metric.Counter
tokensTakenMetric *metric.Counter
tokensTakenWithoutPermissionMetric *metric.Counter

exhaustedStart time.Time

// Estimation models.
l0WriteLM, l0IngestLM, ingestLM tokensLinearModel
Expand Down Expand Up @@ -374,8 +376,10 @@ func (cg *kvStoreTokenChildGranter) storeWriteDone(
// it. The one difference is that post token adjustments, if we observe the
// granter was previously exhausted but is no longer so, we're allowed to
// admit other waiting requests.
return cg.parent.storeReplicatedWorkAdmittedLocked(
additionalTokensTaken := cg.parent.storeReplicatedWorkAdmittedLocked(
cg.workClass, originalTokens, storeReplicatedWorkAdmittedInfo(doneInfo), true /* canGrantAnother */)
cg.parent.tokensTakenWithoutPermissionMetric.Inc(additionalTokensTaken)
return additionalTokensTaken
}

// storeReplicatedWorkAdmitted implements granterWithStoreReplicatedWorkAdmitted.
Expand Down Expand Up @@ -404,7 +408,6 @@ func (sg *kvStoreTokenGranter) tryGetLocked(count int64, demuxHandle int8) grant
if sg.coordMu.availableIOTokens > 0 {
sg.subtractTokensLocked(count, count, false)
sg.coordMu.diskBWTokensUsed[wc] += count
sg.totalTokensTaken.Inc(count)
return grantSuccess
}
case admissionpb.ElasticWorkClass:
Expand All @@ -414,7 +417,6 @@ func (sg *kvStoreTokenGranter) tryGetLocked(count int64, demuxHandle int8) grant
sg.subtractTokensLocked(count, count, false)
sg.coordMu.elasticIOTokensUsedByElastic += count
sg.coordMu.diskBWTokensUsed[wc] += count
sg.totalTokensTaken.Inc(count)
return grantSuccess
}
}
Expand Down Expand Up @@ -446,8 +448,7 @@ func (sg *kvStoreTokenGranter) tookWithoutPermission(workClass admissionpb.WorkC
func (sg *kvStoreTokenGranter) tookWithoutPermissionLocked(count int64, demuxHandle int8) {
wc := admissionpb.WorkClass(demuxHandle)
sg.subtractTokensLocked(count, count, false)
sg.tookWithoutPermissionMetric.Inc(count)
sg.totalTokensTaken.Inc(count)
sg.tokensTakenWithoutPermissionMetric.Inc(count)
if wc == admissionpb.ElasticWorkClass {
sg.coordMu.elasticDiskBWTokensAvailable -= count
sg.coordMu.elasticIOTokensUsedByElastic += count
Expand All @@ -462,6 +463,19 @@ func (sg *kvStoreTokenGranter) subtractTokensLocked(
) {
avail := sg.coordMu.availableIOTokens
sg.coordMu.availableIOTokens -= count
sg.coordMu.availableElasticIOTokens -= elasticCount
// Only update when not unlimited. Keep it whatever it was last otherwise.
if sg.coordMu.availableIOTokens != unlimitedTokens {
sg.availableTokensMetric.Update(sg.coordMu.availableIOTokens)
}
if sg.coordMu.availableElasticIOTokens != unlimitedTokens {
sg.availableElasticTokensMetric.Update(sg.coordMu.availableElasticIOTokens)
}
if count > 0 {
sg.tokensTakenMetric.Inc(count)
} else {
sg.tokensReturnedMetric.Inc(count)
}
if count > 0 && avail > 0 && sg.coordMu.availableIOTokens <= 0 {
// Transition from > 0 to <= 0.
sg.exhaustedStart = timeutil.Now()
Expand All @@ -477,9 +491,6 @@ func (sg *kvStoreTokenGranter) subtractTokensLocked(
sg.exhaustedStart = now
}
}
sg.availableTokensMetrics.Update(sg.coordMu.availableIOTokens)
sg.coordMu.availableElasticIOTokens -= elasticCount
sg.availableElasticTokensMetric.Update(sg.coordMu.availableElasticIOTokens)
}

// requesterHasWaitingRequests implements granterWithLockedCalls.
Expand Down Expand Up @@ -570,10 +581,14 @@ func (sg *kvStoreTokenGranter) setAvailableTokens(
sg.coordMu.availableElasticIOTokens =
min(sg.coordMu.availableElasticIOTokens, sg.coordMu.availableIOTokens)
}

// Only update when not unlimited. Keep it whatever it was last otherwise.
if sg.coordMu.availableIOTokens != unlimitedTokens {
sg.availableTokensMetric.Update(sg.coordMu.availableIOTokens)
}
if sg.coordMu.availableElasticIOTokens != unlimitedTokens {
sg.availableElasticTokensMetric.Update(sg.coordMu.availableElasticIOTokens)
}
sg.startingIOTokens = sg.coordMu.availableIOTokens
sg.availableTokensMetrics.Update(sg.coordMu.availableIOTokens)
sg.availableElasticTokensMetric.Update(sg.coordMu.availableElasticIOTokens)

sg.coordMu.elasticDiskBWTokensAvailable += elasticDiskBandwidthTokens
if sg.coordMu.elasticDiskBWTokensAvailable > elasticDiskBandwidthTokensCapacity {
Expand Down Expand Up @@ -748,6 +763,12 @@ var (
Measurement: "Tokens",
Unit: metric.Unit_COUNT,
}
kvIOTotalTokensReturned = metric.Metadata{
Name: "admission.granter.io_tokens_returned.kv",
Help: "Total number of tokens returned",
Measurement: "Tokens",
Unit: metric.Unit_COUNT,
}
kvIOTokensAvailable = metric.Metadata{
Name: "admission.granter.io_tokens_available.kv",
Help: "Number of tokens available",
Expand All @@ -760,6 +781,18 @@ var (
Measurement: "Tokens",
Unit: metric.Unit_COUNT,
}
l0CompactedBytes = metric.Metadata{
Name: "admission.l0_compacted_bytes.kv",
Help: "Total bytes compacted out of L0 (used to generate IO tokens)",
Measurement: "Tokens",
Unit: metric.Unit_COUNT,
}
l0TokensProduced = metric.Metadata{
Name: "admission.l0_tokens_produced.kv",
Help: "Total bytes produced for L0 writes",
Measurement: "Tokens",
Unit: metric.Unit_COUNT,
}
)

// TODO(irfansharif): we are lacking metrics for IO tokens and load, including
Expand Down
3 changes: 3 additions & 0 deletions pkg/util/admission/granter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -142,6 +142,9 @@ func TestGranterBasic(t *testing.T) {
kvElasticIOTokensAvailable: metrics.KVElasticIOTokensAvailable,
kvIOTokensTookWithoutPermission: metrics.KVIOTokensTookWithoutPermission,
kvIOTotalTokensTaken: metrics.KVIOTotalTokensTaken,
kvIOTotalTokensReturned: metrics.KVIOTotalTokensReturned,
l0CompactedBytes: metrics.L0CompactedBytes,
l0TokensProduced: metrics.L0TokensProduced,
workQueueMetrics: workQueueMetrics,
disableTickerForTesting: true,
knobs: &TestingKnobs{},
Expand Down
39 changes: 33 additions & 6 deletions pkg/util/admission/io_load_listener.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb"
"github.com/cockroachdb/cockroach/pkg/util/humanizeutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metric"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/logtags"
Expand Down Expand Up @@ -108,6 +109,23 @@ var L0MinimumSizePerSubLevel = settings.RegisterIntSetting(
"when non-zero, this indicates the minimum size that is needed to count towards one sub-level",
5<<20, settings.NonNegativeInt)

// L0CompactionAlpha is the exponential smoothing term used when measuring L0
// compactions, which in turn is used to generate IO tokens.
var L0CompactionAlpha = settings.RegisterFloatSetting(
settings.TenantWritable,
"admission.l0_compacted_alpha",
"exponential smoothing term used when measuring L0 compactions to generate IO tokens",
0.5, settings.PositiveFloat)

// L0ReductionFactor is the exponential smoothing term used when measuring L0
// compactions, which in turn is used to generate IO tokens.
var L0ReductionFactor = settings.RegisterFloatSetting(
settings.TenantWritable,
"admission.l0_reduction_factor",
"once overloaded, factor by which we reduce L0 compaction tokens based on observed compactions",
2.0,
settings.FloatWithMinimum(1.0))

// Experimental observations:
// - Sub-level count of ~40 caused a node heartbeat latency p90, p99 of 2.5s,
// 4s. With a setting that limits sub-level count to 10, before the system
Expand Down Expand Up @@ -188,6 +206,9 @@ type ioLoadListener struct {
adjustTokensResult
perWorkTokenEstimator storePerWorkTokenEstimator
diskBandwidthLimiter diskBandwidthLimiter

l0CompactedBytes *metric.Counter
l0TokensProduced *metric.Counter
}

type ioLoadListenerState struct {
Expand Down Expand Up @@ -641,7 +662,7 @@ type adjustTokensAuxComputations struct {

// adjustTokensInner is used for computing tokens based on compaction and
// flush bottlenecks.
func (*ioLoadListener) adjustTokensInner(
func (io *ioLoadListener) adjustTokensInner(
ctx context.Context,
prev ioLoadListenerState,
l0Metrics pebble.LevelMetrics,
Expand Down Expand Up @@ -677,9 +698,11 @@ func (*ioLoadListener) adjustTokensInner(
// bytes (gauge).
intL0CompactedBytes = 0
}
const alpha = 0.5
io.l0CompactedBytes.Inc(intL0CompactedBytes)

// Compaction scheduling can be uneven in prioritizing L0 for compactions,
// so smooth out what is being removed by compactions.
alpha := L0CompactionAlpha.Get(&io.settings.SV)
smoothedIntL0CompactedBytes := int64(alpha*float64(intL0CompactedBytes) + (1-alpha)*float64(prev.smoothedIntL0CompactedBytes))

// Flush tokens:
Expand Down Expand Up @@ -868,6 +891,7 @@ func (*ioLoadListener) adjustTokensInner(
// threshold.
var totalNumByteTokens int64
var smoothedCompactionByteTokens float64
l0ReductionFactor := L0ReductionFactor.Get(&io.settings.SV)

score, _ := ioThreshold.Score()
// Multiplying score by 2 for ease of calculation.
Expand Down Expand Up @@ -909,7 +933,7 @@ func (*ioLoadListener) adjustTokensInner(
// Don't admit more byte work than we can remove via compactions.
// totalNumByteTokens tracks our goal for admission. Scale down
// since we want to get under the thresholds over time.
fTotalNumByteTokens = float64(smoothedIntL0CompactedBytes / 2.0)
fTotalNumByteTokens = float64(smoothedIntL0CompactedBytes) / l0ReductionFactor
} else if score >= 0.5 && score < 1 {
// Low load. Score in [0.5, 1). Tokens should be
// smoothedIntL0CompactedBytes at 1, and 2 * smoothedIntL0CompactedBytes
Expand All @@ -919,8 +943,8 @@ func (*ioLoadListener) adjustTokensInner(
// Medium load. Score in [1, 2). We use linear interpolation from
// medium load to overload, to slowly give out fewer tokens as we
// move towards overload.
halfSmoothedBytes := float64(smoothedIntL0CompactedBytes / 2.0)
fTotalNumByteTokens = -score*halfSmoothedBytes + 3*halfSmoothedBytes
reducedSmoothedBytes := float64(smoothedIntL0CompactedBytes) / l0ReductionFactor
fTotalNumByteTokens = -score*reducedSmoothedBytes + 3*reducedSmoothedBytes
}
smoothedCompactionByteTokens = alpha*fTotalNumByteTokens + (1-alpha)*prev.smoothedCompactionByteTokens
if float64(math.MaxInt64) < smoothedCompactionByteTokens {
Expand Down Expand Up @@ -958,6 +982,9 @@ func (*ioLoadListener) adjustTokensInner(
if totalNumElasticByteTokens > totalNumByteTokens {
totalNumElasticByteTokens = totalNumByteTokens
}

io.l0TokensProduced.Inc(totalNumByteTokens)

// Install the latest cumulative stats.
return adjustTokensResult{
ioLoadListenerState: ioLoadListenerState{
Expand Down Expand Up @@ -1047,7 +1074,7 @@ func (res adjustTokensResult) SafeFormat(p redact.SafePrinter, _ rune) {
ib(m/adjustmentInterval))
switch res.aux.tokenKind {
case compactionTokenKind:
p.Printf(" due to L0 growth")
p.Printf(" due to L0 growth [≈%s]", ib(int64(res.smoothedCompactionByteTokens)))
case flushTokenKind:
p.Printf(" due to memtable flush (multiplier %.3f)", res.flushUtilTargetFraction)
}
Expand Down
18 changes: 15 additions & 3 deletions pkg/util/admission/io_load_listener_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/echotest"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb"
"github.com/cockroachdb/cockroach/pkg/util/metric"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/datadriven"
"github.com/cockroachdb/pebble"
Expand Down Expand Up @@ -55,6 +56,8 @@ func TestIOLoadListener(t *testing.T) {
kvRequester: req,
perWorkTokenEstimator: makeStorePerWorkTokenEstimator(),
diskBandwidthLimiter: makeDiskBandwidthLimiter(),
l0CompactedBytes: metric.NewCounter(l0CompactedBytes),
l0TokensProduced: metric.NewCounter(l0TokensProduced),
}
// The mutex is needed by ioLoadListener but is not useful in this
// test -- the channels provide synchronization and prevent this
Expand Down Expand Up @@ -214,8 +217,10 @@ func TestIOLoadListenerOverflow(t *testing.T) {
ctx := context.Background()
st := cluster.MakeTestingClusterSettings()
ioll := ioLoadListener{
settings: st,
kvRequester: req,
settings: st,
kvRequester: req,
l0CompactedBytes: metric.NewCounter(l0CompactedBytes),
l0TokensProduced: metric.NewCounter(l0TokensProduced),
}
ioll.kvGranter = kvGranter
// Bug 1: overflow when totalNumByteTokens is too large.
Expand Down Expand Up @@ -275,7 +280,12 @@ func TestAdjustTokensInnerAndLogging(t *testing.T) {
var buf redact.StringBuilder
for _, tt := range tests {
buf.Printf("%s:\n", tt.name)
res := (*ioLoadListener)(nil).adjustTokensInner(
ioll := &ioLoadListener{
settings: cluster.MakeTestingClusterSettings(),
l0CompactedBytes: metric.NewCounter(l0CompactedBytes),
l0TokensProduced: metric.NewCounter(l0TokensProduced),
}
res := ioll.adjustTokensInner(
ctx, tt.prev, tt.l0Metrics, 12, pebble.ThroughputMetric{},
100, 10, 0, 0.50)
buf.Printf("%s\n", res)
Expand Down Expand Up @@ -316,6 +326,8 @@ func TestBadIOLoadListenerStats(t *testing.T) {
kvRequester: req,
perWorkTokenEstimator: makeStorePerWorkTokenEstimator(),
diskBandwidthLimiter: makeDiskBandwidthLimiter(),
l0CompactedBytes: metric.NewCounter(l0CompactedBytes),
l0TokensProduced: metric.NewCounter(l0TokensProduced),
}
ioll.kvGranter = kvGranter
for i := 0; i < 100; i++ {
Expand Down
2 changes: 1 addition & 1 deletion pkg/util/admission/testdata/format_adjust_tokens_stats.txt
Original file line number Diff line number Diff line change
Expand Up @@ -3,4 +3,4 @@ echo
zero:
compaction score 0.000 (0 ssts, 0 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 0.00x+0 B) + ingested-model 0.00x+0 B (smoothed 0.00x+0 B) + at-admission-tokens 0 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B]; admitting all; elastic-disk-bw tokens 0 B (used 0 B, regular used 0 B): write model 0.00x+0 B ingest model 0.00x+0 B, disk bw read 0 B write 0 B provisioned 0 B; write stalls 12
real-numbers:
compaction score 2.700[L0-overload] (195 ssts, 27 sub-levels), L0 growth 577 MiB (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 0.00x+0 B) + ingested-model 0.00x+0 B (smoothed 0.00x+0 B) + at-admission-tokens 0 B, compacted 77 MiB [≈62 MiB], flushed 0 B [≈0 B]; admitting 116 MiB (rate 7.7 MiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); elastic-disk-bw tokens 0 B (used 0 B, regular used 0 B): write model 0.00x+0 B ingest model 0.00x+0 B, disk bw read 0 B write 0 B provisioned 0 B; write stalls 2
compaction score 2.700[L0-overload] (195 ssts, 27 sub-levels), L0 growth 577 MiB (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 0.00x+0 B) + ingested-model 0.00x+0 B (smoothed 0.00x+0 B) + at-admission-tokens 0 B, compacted 77 MiB [≈62 MiB], flushed 0 B [≈0 B]; admitting 116 MiB (rate 7.7 MiB/s) (elastic 1 B rate 0 B/s) due to L0 growth [≈116 MiB] (used total: 0 B elastic 0 B); elastic-disk-bw tokens 0 B (used 0 B, regular used 0 B): write model 0.00x+0 B ingest model 0.00x+0 B, disk bw read 0 B write 0 B provisioned 0 B; write stalls 2
Loading

0 comments on commit 4c2ec63

Please sign in to comment.