Skip to content

Commit 6956f7e

Browse files
committed
record: change WriteWALSyncOffsets to respect runtime format ratcheting
Changes WriteWALSyncOffsets a function so it always checks the current format version when creating a new WAL. The format major version can change (ratchet) at runtime, so WriteWALSyncOffsets should be a function so we can use the latest format version.
1 parent 4d38942 commit 6956f7e

File tree

13 files changed

+168
-71
lines changed

13 files changed

+168
-71
lines changed

commit_test.go

Lines changed: 6 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -253,8 +253,9 @@ func TestCommitPipelineWALClose(t *testing.T) {
253253
}
254254
p := newCommitPipeline(testEnv)
255255
wal = record.NewLogWriter(sf, 0 /* logNum */, record.LogWriterConfig{
256-
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
257-
QueueSemChan: p.logSyncQSem,
256+
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
257+
QueueSemChan: p.logSyncQSem,
258+
WriteWALSyncOffsets: func() bool { return false },
258259
})
259260

260261
// Launch N (commitConcurrency) goroutines which each create a batch and
@@ -394,8 +395,9 @@ func BenchmarkCommitPipeline(b *testing.B) {
394395
p := newCommitPipeline(nullCommitEnv)
395396
wal = record.NewLogWriter(io.Discard, 0, /* logNum */
396397
record.LogWriterConfig{
397-
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
398-
QueueSemChan: p.logSyncQSem,
398+
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
399+
QueueSemChan: p.logSyncQSem,
400+
WriteWALSyncOffsets: func() bool { return false },
399401
})
400402
const keySize = 8
401403
b.SetBytes(2 * keySize)

open.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -372,7 +372,7 @@ func Open(dirname string, opts *Options) (db *DB, err error) {
372372
QueueSemChan: d.commit.logSyncQSem,
373373
Logger: opts.Logger,
374374
EventListener: walEventListenerAdaptor{l: opts.EventListener},
375-
WriteWALSyncOffsets: FormatMajorVersion(d.mu.formatVers.vers.Load()) >= FormatWALSyncChunks,
375+
WriteWALSyncOffsets: func() bool { return d.FormatMajorVersion() >= FormatWALSyncChunks },
376376
}
377377
if opts.WALFailover != nil {
378378
walOpts.Secondary = opts.WALFailover.Secondary

open_test.go

Lines changed: 41 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1748,3 +1748,44 @@ func TestWALFailoverRandomized(t *testing.T) {
17481748
nextRandomOp()()
17491749
}
17501750
}
1751+
1752+
func TestWALCorruption(t *testing.T) {
1753+
fs := vfs.NewMem()
1754+
d, err := Open("", testingRandomized(t, &Options{
1755+
FS: fs,
1756+
FormatMajorVersion: FormatWALSyncChunks,
1757+
}))
1758+
require.NoError(t, err)
1759+
require.NoError(t, d.Flush())
1760+
1761+
fourKiBValue := bytes.Repeat([]byte{'a'}, 4096)
1762+
for i := 1; i <= 32; i++ {
1763+
require.NoError(t, d.Set([]byte(fmt.Sprintf("key-%d", i)), fourKiBValue, Sync))
1764+
}
1765+
require.NoError(t, d.Close())
1766+
1767+
// We should have two WALs.
1768+
logs, err := fs.List("")
1769+
require.NoError(t, err)
1770+
logs = slices.DeleteFunc(logs, func(s string) bool { return filepath.Ext(s) != ".log" })
1771+
sort.Slice(logs, func(i, j int) bool {
1772+
return logs[i] < logs[j]
1773+
})
1774+
lastLog := logs[len(logs)-1]
1775+
1776+
// Corrupt the WAL by zeroing four bytes, 100 bytes from the end
1777+
// of the file.
1778+
f, err := fs.OpenReadWrite(lastLog, vfs.WriteCategoryUnspecified)
1779+
require.NoError(t, err)
1780+
_, err = f.WriteAt([]byte{0, 0, 0, 0}, 100)
1781+
require.NoError(t, err)
1782+
require.NoError(t, f.Close())
1783+
t.Logf("zeroed four bytes in %s at offset %d\n", lastLog, 100)
1784+
1785+
// Re-opening the database should detect and report the corruption.
1786+
_, err = Open("", &Options{
1787+
FS: fs,
1788+
FormatMajorVersion: FormatWALSyncChunks,
1789+
})
1790+
require.True(t, errors.Is(err, ErrCorruption))
1791+
}

record/log_writer.go

Lines changed: 5 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -509,8 +509,10 @@ type LogWriterConfig struct {
509509
// callbacks are serialized since they are invoked from the flushLoop.
510510
ExternalSyncQueueCallback ExternalSyncQueueCallback
511511

512-
// WriteWALSyncOffsets represents whether to write the WAL sync chunk format.
513-
WriteWALSyncOffsets bool
512+
// WriteWALSyncOffsets determines whether to write WAL sync chunk offsets.
513+
// The format major version can change (ratchet) at runtime, so this must be
514+
// a function rather than a static bool to ensure we use the latest format version.
515+
WriteWALSyncOffsets func() bool
514516
}
515517

516518
// ExternalSyncQueueCallback is to be run when a PendingSync has been
@@ -552,7 +554,7 @@ func NewLogWriter(
552554
},
553555
}
554556

555-
if logWriterConfig.WriteWALSyncOffsets {
557+
if logWriterConfig.WriteWALSyncOffsets() {
556558
r.emitFragment = r.emitFragmentSyncOffsets
557559
} else {
558560
r.emitFragment = r.emitFragmentRecyclable

record/log_writer_test.go

Lines changed: 30 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -150,7 +150,8 @@ func TestSyncError(t *testing.T) {
150150

151151
injectedErr := errors.New("injected error")
152152
w := NewLogWriter(syncErrorFile{f, injectedErr}, 0, LogWriterConfig{
153-
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
153+
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
154+
WriteWALSyncOffsets: func() bool { return false },
154155
})
155156

156157
syncRecord := func() {
@@ -191,7 +192,10 @@ func (f *syncFile) Sync() error {
191192

192193
func TestSyncRecord(t *testing.T) {
193194
f := &syncFile{}
194-
w := NewLogWriter(f, 0, LogWriterConfig{WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{})})
195+
w := NewLogWriter(f, 0, LogWriterConfig{
196+
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
197+
WriteWALSyncOffsets: func() bool { return false },
198+
})
195199

196200
var syncErr error
197201
for i := 0; i < 100000; i++ {
@@ -217,8 +221,9 @@ func TestSyncRecordWithSignalChan(t *testing.T) {
217221
semChan <- struct{}{}
218222
}
219223
w := NewLogWriter(f, 0, LogWriterConfig{
220-
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
221-
QueueSemChan: semChan,
224+
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
225+
QueueSemChan: semChan,
226+
WriteWALSyncOffsets: func() bool { return false },
222227
})
223228
require.Equal(t, cap(semChan), len(semChan))
224229
var syncErr error
@@ -268,7 +273,8 @@ func TestMinSyncInterval(t *testing.T) {
268273
WALMinSyncInterval: func() time.Duration {
269274
return minSyncInterval
270275
},
271-
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
276+
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
277+
WriteWALSyncOffsets: func() bool { return false },
272278
})
273279

274280
var timer fakeTimer
@@ -340,7 +346,8 @@ func TestMinSyncIntervalClose(t *testing.T) {
340346
WALMinSyncInterval: func() time.Duration {
341347
return minSyncInterval
342348
},
343-
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
349+
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
350+
WriteWALSyncOffsets: func() bool { return false },
344351
})
345352

346353
var timer fakeTimer
@@ -390,7 +397,10 @@ func (f *syncFileWithWait) Sync() error {
390397
func TestMetricsWithoutSync(t *testing.T) {
391398
f := &syncFileWithWait{}
392399
f.writeWG.Add(1)
393-
w := NewLogWriter(f, 0, LogWriterConfig{WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{})})
400+
w := NewLogWriter(f, 0, LogWriterConfig{
401+
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
402+
WriteWALSyncOffsets: func() bool { return false },
403+
})
394404
offset, err := w.SyncRecord([]byte("hello"), nil, nil)
395405
require.NoError(t, err)
396406
const recordSize = 16
@@ -434,7 +444,8 @@ func TestMetricsWithSync(t *testing.T) {
434444
})
435445

436446
w := NewLogWriter(f, 0, LogWriterConfig{
437-
WALFsyncLatency: syncLatencyMicros,
447+
WALFsyncLatency: syncLatencyMicros,
448+
WriteWALSyncOffsets: func() bool { return false },
438449
},
439450
)
440451
var wg sync.WaitGroup
@@ -540,7 +551,8 @@ func TestQueueWALBlocks(t *testing.T) {
540551
return nil
541552
}))
542553
w := NewLogWriter(f, 0, LogWriterConfig{
543-
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
554+
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
555+
WriteWALSyncOffsets: func() bool { return false },
544556
})
545557
const numBlocks = 1024
546558
var b [blockSize]byte
@@ -657,6 +669,7 @@ func TestSyncRecordGeneralized(t *testing.T) {
657669
lastSync++
658670
cbChan <- struct{}{}
659671
},
672+
WriteWALSyncOffsets: func() bool { return false },
660673
})
661674
offset, err := w.SyncRecordGeneralized([]byte("hello"), &PendingSyncIndex{})
662675
require.NoError(t, err)
@@ -712,6 +725,7 @@ func TestSyncRecordGeneralizedWithCloseError(t *testing.T) {
712725
lastSync++
713726
cbChan <- struct{}{}
714727
},
728+
WriteWALSyncOffsets: func() bool { return false },
715729
})
716730
offset, err := w.SyncRecordGeneralized([]byte("hello"), &PendingSyncIndex{})
717731
require.NoError(t, err)
@@ -740,7 +754,10 @@ func TestSyncRecordGeneralizedWithCloseError(t *testing.T) {
740754

741755
func writeWALSyncRecords(t *testing.T, numRecords int, recordSizes []int) *syncFile {
742756
f := &syncFile{}
743-
w := NewLogWriter(f, 1, LogWriterConfig{WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}), WriteWALSyncOffsets: true})
757+
w := NewLogWriter(f, 1, LogWriterConfig{
758+
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
759+
WriteWALSyncOffsets: func() bool { return true },
760+
})
744761
var syncErr error
745762
for i := 0; i < numRecords; i++ {
746763
var syncWG sync.WaitGroup
@@ -847,7 +864,8 @@ func BenchmarkQueueWALBlocks(b *testing.B) {
847864
return nil
848865
}))
849866
w := NewLogWriter(f, 0, LogWriterConfig{
850-
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
867+
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
868+
WriteWALSyncOffsets: func() bool { return false },
851869
})
852870

853871
b.StartTimer()
@@ -881,6 +899,7 @@ func BenchmarkWriteWALBlocksAllocs(b *testing.B) {
881899
w := NewLogWriter(f, 0, LogWriterConfig{
882900
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
883901
ExternalSyncQueueCallback: func(doneSync PendingSyncIndex, err error) {},
902+
WriteWALSyncOffsets: func() bool { return false },
884903
})
885904

886905
var psi PendingSyncIndex

record/record_test.go

Lines changed: 33 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -96,7 +96,9 @@ func testGenerator(t *testing.T, reset func(), gen func() (string, bool)) {
9696
t.Run("LogWriter", func(t *testing.T) {
9797
testGeneratorWriter(t, reset, gen, func(w io.Writer) recordWriter {
9898
return NewLogWriter(w, 0 /* logNum */, LogWriterConfig{
99-
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{})})
99+
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
100+
WriteWALSyncOffsets: func() bool { return false },
101+
})
100102
})
101103
})
102104
}
@@ -640,7 +642,9 @@ func TestNoLastRecordOffset(t *testing.T) {
640642
func TestInvalidLogNum(t *testing.T) {
641643
var buf bytes.Buffer
642644
w := NewLogWriter(&buf, 1, LogWriterConfig{
643-
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{})})
645+
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
646+
WriteWALSyncOffsets: func() bool { return false },
647+
})
644648
for i := 0; i < 10; i++ {
645649
s := fmt.Sprintf("%04d\n", i)
646650
_, err := w.WriteRecord([]byte(s))
@@ -735,7 +739,9 @@ func TestRecycleLog(t *testing.T) {
735739
}
736740

737741
w := NewLogWriter(limitedBuf, base.DiskFileNum(i), LogWriterConfig{
738-
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{})})
742+
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
743+
WriteWALSyncOffsets: func() bool { return false },
744+
})
739745
sizes := make([]int, 10+rnd.IntN(100))
740746
for j := range sizes {
741747
data := randBlock()
@@ -779,7 +785,9 @@ func TestRecycleLog(t *testing.T) {
779785
func TestTruncatedLog(t *testing.T) {
780786
backing := make([]byte, 2*blockSize)
781787
w := NewLogWriter(bytes.NewBuffer(backing[:0]), base.DiskFileNum(1), LogWriterConfig{
782-
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{})})
788+
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
789+
WriteWALSyncOffsets: func() bool { return false },
790+
})
783791
// Write a record that spans 2 blocks.
784792
_, err := w.WriteRecord(bytes.Repeat([]byte("s"), blockSize+100))
785793
require.NoError(t, err)
@@ -795,15 +803,19 @@ func TestTruncatedLog(t *testing.T) {
795803
func TestRecycleLogWithPartialBlock(t *testing.T) {
796804
backing := make([]byte, 27)
797805
w := NewLogWriter(bytes.NewBuffer(backing[:0]), base.DiskFileNum(1), LogWriterConfig{
798-
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{})})
806+
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
807+
WriteWALSyncOffsets: func() bool { return false },
808+
})
799809
// Will write a chunk with 11 byte header + 5 byte payload.
800810
_, err := w.WriteRecord([]byte("aaaaa"))
801811
require.NoError(t, err)
802812
// Close will write a 11-byte EOF chunk.
803813
require.NoError(t, w.Close())
804814

805815
w = NewLogWriter(bytes.NewBuffer(backing[:0]), base.DiskFileNum(2), LogWriterConfig{
806-
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{})})
816+
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
817+
WriteWALSyncOffsets: func() bool { return false },
818+
})
807819
// Will write a chunk with 11 byte header + 1 byte payload.
808820
_, err = w.WriteRecord([]byte("a"))
809821
require.NoError(t, err)
@@ -826,15 +838,19 @@ func TestRecycleLogNumberOverflow(t *testing.T) {
826838

827839
backing := make([]byte, 27)
828840
w := NewLogWriter(bytes.NewBuffer(backing[:0]), base.DiskFileNum(math.MaxUint32), LogWriterConfig{
829-
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{})})
841+
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
842+
WriteWALSyncOffsets: func() bool { return false },
843+
})
830844
// Will write a chunk with 11 byte header + 5 byte payload.
831845
_, err := w.WriteRecord([]byte("aaaaa"))
832846
require.NoError(t, err)
833847
// Close will write a 11-byte EOF chunk.
834848
require.NoError(t, w.Close())
835849

836850
w = NewLogWriter(bytes.NewBuffer(backing[:0]), base.DiskFileNum(math.MaxUint32+1), LogWriterConfig{
837-
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{})})
851+
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
852+
WriteWALSyncOffsets: func() bool { return false },
853+
})
838854
// Will write a chunk with 11 byte header + 1 byte payload.
839855
_, err = w.WriteRecord([]byte("a"))
840856
require.NoError(t, err)
@@ -856,7 +872,9 @@ func TestRecycleLogWithPartialRecord(t *testing.T) {
856872
// Write a record that is larger than the log block size.
857873
backing1 := make([]byte, 2*blockSize)
858874
w := NewLogWriter(bytes.NewBuffer(backing1[:0]), base.DiskFileNum(1), LogWriterConfig{
859-
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{})})
875+
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
876+
WriteWALSyncOffsets: func() bool { return false },
877+
})
860878
_, err := w.WriteRecord(bytes.Repeat([]byte("a"), recordSize))
861879
require.NoError(t, err)
862880
require.NoError(t, w.Close())
@@ -865,7 +883,9 @@ func TestRecycleLogWithPartialRecord(t *testing.T) {
865883
// the block size.
866884
backing2 := make([]byte, 2*blockSize)
867885
w = NewLogWriter(bytes.NewBuffer(backing2[:0]), base.DiskFileNum(2), LogWriterConfig{
868-
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{})})
886+
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
887+
WriteWALSyncOffsets: func() bool { return false },
888+
})
869889
_, err = w.WriteRecord(bytes.Repeat([]byte("b"), recordSize))
870890
require.NoError(t, err)
871891
require.NoError(t, w.Close())
@@ -1083,7 +1103,9 @@ func BenchmarkRecordWrite(b *testing.B) {
10831103
for _, size := range []int{8, 16, 32, 64, 256, 1028, 4096, 65_536} {
10841104
b.Run(fmt.Sprintf("size=%d", size), func(b *testing.B) {
10851105
w := NewLogWriter(io.Discard, 0 /* logNum */, LogWriterConfig{
1086-
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{})})
1106+
WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
1107+
WriteWALSyncOffsets: func() bool { return false },
1108+
})
10871109
defer w.Close()
10881110
buf := make([]byte, size)
10891111

testdata/event_listener

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -223,16 +223,16 @@ metrics
223223
| | | | ingested | moved | written | | amp
224224
level | tables size val-bl vtables | score | in | tables size | tables size | tables size | read | r w
225225
------+-----------------------------+-------+-------+--------------+--------------+--------------+-------+---------
226-
0 | 2 1.5KB 0B 0 | 0.40 | 81B | 1 746B | 0 0B | 3 2.2KB | 0B | 2 27.5
226+
0 | 2 1.5KB 0B 0 | 0.40 | 97B | 1 746B | 0 0B | 3 2.2KB | 0B | 2 23.0
227227
1 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0
228228
2 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0
229229
3 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0
230230
4 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0
231231
5 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0
232232
6 | 1 746B 0B 0 | - | 1.5KB | 0 0B | 0 0B | 1 746B | 1.5KB | 1 0.5
233-
total | 3 2.2KB 0B 0 | - | 827B | 1 746B | 0 0B | 4 3.7KB | 1.5KB | 3 4.6
233+
total | 3 2.2KB 0B 0 | - | 843B | 1 746B | 0 0B | 4 3.7KB | 1.5KB | 3 4.5
234234
-------------------------------------------------------------------------------------------------------------------
235-
WAL: 1 files (0B) in: 48B written: 81B (69% overhead)
235+
WAL: 1 files (0B) in: 48B written: 97B (102% overhead)
236236
Flushes: 3
237237
Compactions: 1 estimated debt: 2.2KB in progress: 0 (0B)
238238
default: 1 delete: 0 elision: 0 move: 0 read: 0 tombstone-density: 0 rewrite: 0 copy: 0 multi-level: 0
@@ -324,7 +324,7 @@ metrics
324324
| | | | ingested | moved | written | | amp
325325
level | tables size val-bl vtables | score | in | tables size | tables size | tables size | read | r w
326326
------+-----------------------------+-------+-------+--------------+--------------+--------------+-------+---------
327-
0 | 4 2.9KB 0B 0 | 0.80 | 108B | 2 1.5KB | 0 0B | 4 2.9KB | 0B | 4 27.5
327+
0 | 4 2.9KB 0B 0 | 0.80 | 132B | 2 1.5KB | 0 0B | 4 2.9KB | 0B | 4 22.5
328328
1 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0
329329
2 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0
330330
3 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0
@@ -333,7 +333,7 @@ level | tables size val-bl vtables | score | in | tables size | tables siz
333333
6 | 2 1.5KB 0B 0 | - | 1.5KB | 1 746B | 0 0B | 1 746B | 1.5KB | 1 0.5
334334
total | 6 4.4KB 0B 0 | - | 2.3KB | 3 2.2KB | 0 0B | 5 5.9KB | 1.5KB | 5 2.6
335335
-------------------------------------------------------------------------------------------------------------------
336-
WAL: 1 files (0B) in: 82B written: 108B (32% overhead)
336+
WAL: 1 files (0B) in: 82B written: 132B (61% overhead)
337337
Flushes: 6
338338
Compactions: 1 estimated debt: 4.4KB in progress: 0 (0B)
339339
default: 1 delete: 0 elision: 0 move: 0 read: 0 tombstone-density: 0 rewrite: 0 copy: 0 multi-level: 0

0 commit comments

Comments
 (0)