From 25a8e9bb8d9586e5090979f24dec11712e9f4b3c Mon Sep 17 00:00:00 2001 From: Jackson Owens Date: Tue, 11 Apr 2023 15:06:47 -0400 Subject: [PATCH] db: add DeleteSized and corresponding compaction heuristics Add a new write operation and corresponding internal key kind for delete operations that know the size of the value they're deleting. SSTable writers aggregate these estimates into table properties that are then used to improve compaction-picking heuristics. This is particularly useful in scenarios with heterogeneous value sizes that are easily misestimated by existing heuristics that rely on averages. When a DELSIZED tombstone deletes a key, it's mutated to reflect that the deletion has already occurred. If the size carried within the tombstone's value was accurate, the tombstone's value is removed. If the size was not accurate, the tombstone is mutated into an ordinary DEL tombstone which will be subject to typical average-based heuristics. Close #2340. --- batch.go | 98 +- batch_test.go | 66 +- checkpoint_test.go | 4 +- compaction_iter.go | 129 +- compaction_iter_test.go | 31 +- compaction_test.go | 19 +- data_test.go | 16 + db.go | 56 +- db_test.go | 2 +- event_listener_test.go | 2 +- flushable_test.go | 2 +- format_major_version.go | 28 +- format_major_version_test.go | 9 +- ingest_test.go | 12 +- internal.go | 1 + internal/base/internal.go | 15 +- internal/base/internal_test.go | 2 +- internal/rangekey/rangekey_test.go | 4 + iterator.go | 8 +- iterator_test.go | 6 +- level_checker.go | 2 +- mem_table_test.go | 2 +- metamorphic/generator.go | 10 +- metamorphic/ops.go | 30 +- metamorphic/options.go | 9 + metamorphic/test.go | 4 + open_test.go | 4 +- options.go | 6 +- options_test.go | 2 +- range_del_test.go | 4 +- replay/replay.go | 10 + scan_internal.go | 2 +- sstable/format.go | 2 +- sstable/internal.go | 1 + sstable/properties.go | 19 +- sstable/testdata/writer_value_blocks | 2 +- sstable/writer.go | 25 +- table_cache_test.go | 4 +- table_stats.go | 179 +- table_stats_test.go | 3 + testdata/checkpoint | 110 +- testdata/compaction_delete_only_hints | 6 +- testdata/compaction_iter_delete_sized | 1477 +++++++++++++++++ testdata/compaction_tombstones | 32 +- testdata/event_listener | 63 +- testdata/flushable_ingest | 14 +- testdata/ingest | 2 +- testdata/ingest_load | 7 + testdata/iter_histories/range_keys_simple | 2 +- ...manual_compaction_file_boundaries_delsized | 520 ++++++ testdata/marked_for_compaction | 2 +- testdata/metrics | 10 +- testdata/table_stats | 58 +- tool/db.go | 101 +- tool/find.go | 1 + tool/sstable.go | 3 + tool/testdata/db_properties | 106 +- tool/testdata/sstable_properties | 296 ++-- tool/wal.go | 3 + 59 files changed, 3116 insertions(+), 527 deletions(-) create mode 100644 testdata/compaction_iter_delete_sized create mode 100644 testdata/manual_compaction_file_boundaries_delsized diff --git a/batch.go b/batch.go index ff84fe3ecd..0b76c42856 100644 --- a/batch.go +++ b/batch.go @@ -265,6 +265,13 @@ type Batch struct { // then it will only contain key kinds of IngestSST. ingestedSSTBatch bool + // minimumFormatMajorVersion indicates the format major version required in + // order to commit this batch. If an operation requires a particular format + // major version, it ratchets the batch's minimumFormatMajorVersion. When + // the batch is committed, this is validated against the database's current + // format major version. + minimumFormatMajorVersion FormatMajorVersion + // Synchronous Apply uses the commit WaitGroup for both publishing the // seqnum and waiting for the WAL fsync (if needed). Asynchronous // ApplyNoSyncWait, which implies WriteOptions.Sync is true, uses the commit @@ -413,6 +420,7 @@ func (b *Batch) refreshMemTableSize() { b.countRangeDels = 0 b.countRangeKeys = 0 + b.minimumFormatMajorVersion = 0 for r := b.Reader(); ; { kind, key, value, ok := r.Next() if !ok { @@ -423,12 +431,22 @@ func (b *Batch) refreshMemTableSize() { b.countRangeDels++ case InternalKeyKindRangeKeySet, InternalKeyKindRangeKeyUnset, InternalKeyKindRangeKeyDelete: b.countRangeKeys++ + case InternalKeyKindDeleteSized: + if b.minimumFormatMajorVersion < ExperimentalFormatDeleteSized { + b.minimumFormatMajorVersion = ExperimentalFormatDeleteSized + } case InternalKeyKindIngestSST: + if b.minimumFormatMajorVersion < FormatFlushableIngest { + b.minimumFormatMajorVersion = FormatFlushableIngest + } // This key kind doesn't contribute to the memtable size. continue } b.memTableSize += memTableEntrySize(len(key), len(value)) } + if b.countRangeKeys > 0 && b.minimumFormatMajorVersion < FormatRangeKeys { + b.minimumFormatMajorVersion = FormatRangeKeys + } } // Apply the operations contained in the batch to the receiver batch. @@ -680,6 +698,72 @@ func (b *Batch) DeleteDeferred(keyLen int) *DeferredBatchOp { return &b.deferredOp } +// DeleteSized behaves identically to Delete, but takes an additional +// argument indicating the size of the value being deleted. DeleteSized +// should be preferred when the caller has the expectation that there exists +// a single internal KV pair for the key (eg, the key has not been +// overwritten recently), and the caller knows the size of its value. +// +// DeleteSized will record the value size within the tombstone and use it to +// inform compaction-picking heuristics which strive to reduce space +// amplification in the LSM. This "calling your shot" mechanic allows the +// storage engine to more accurately estimate and reduce space amplification. +// +// It is safe to modify the contents of the arguments after DeleteSized +// returns. +func (b *Batch) DeleteSized(key []byte, deletedValueSize uint32, _ *WriteOptions) error { + deferredOp := b.DeleteSizedDeferred(len(key), deletedValueSize) + copy(b.deferredOp.Key, key) + // TODO(peter): Manually inline DeferredBatchOp.Finish(). Check if in a + // later Go release this is unnecessary. + if b.index != nil { + if err := b.index.Add(deferredOp.offset); err != nil { + return err + } + } + return nil +} + +// DeleteSizedDeferred is similar to DeleteSized in that it adds a sized delete +// operation to the batch, except it only takes in key length instead of a +// complete key slice, letting the caller encode into the DeferredBatchOp.Key +// slice and then call Finish() on the returned object. +func (b *Batch) DeleteSizedDeferred(keyLen int, deletedValueSize uint32) *DeferredBatchOp { + if b.minimumFormatMajorVersion < ExperimentalFormatDeleteSized { + b.minimumFormatMajorVersion = ExperimentalFormatDeleteSized + } + + // Encode the sum of the key length and the value in the value. + v := uint64(deletedValueSize) + uint64(keyLen) + + // Encode `v` as a varint. + var buf [binary.MaxVarintLen64]byte + n := 0 + { + x := v + for x >= 0x80 { + buf[n] = byte(x) | 0x80 + x >>= 7 + n++ + } + buf[n] = byte(x) + n++ + } + + // NB: In batch entries and sstable entries, values are stored as + // varstrings. Here, the value is itself a simple varint. This results in an + // unnecessary double layer of encoding: + // varint(n) varint(deletedValueSize) + // The first varint will always be 1-byte, since a varint-encoded uint64 + // will never exceed 128 bytes. This unnecessary extra byte and wrapping is + // preserved to avoid special casing across the database, and in particular + // in sstable block decoding which is performance sensitive. + b.prepareDeferredKeyValueRecord(keyLen, n, InternalKeyKindDeleteSized) + b.deferredOp.index = b.index + copy(b.deferredOp.Value, buf[:n]) + return &b.deferredOp +} + // SingleDelete adds an action to the batch that single deletes the entry for key. // See Writer.SingleDelete for more details on the semantics of SingleDelete. // @@ -782,6 +866,9 @@ func (b *Batch) rangeKeySetDeferred(startLen, internalValueLen int) *DeferredBat func (b *Batch) incrementRangeKeysCount() { b.countRangeKeys++ + if b.minimumFormatMajorVersion < FormatRangeKeys { + b.minimumFormatMajorVersion = FormatRangeKeys + } if b.index != nil { b.rangeKeys = nil b.rangeKeysSeqNum = 0 @@ -895,6 +982,7 @@ func (b *Batch) ingestSST(fileNum base.FileNum) { // is not reset because for the InternalKeyKindIngestSST the count is the // number of sstable paths which have been added to the batch. b.memTableSize = origMemTableSize + b.minimumFormatMajorVersion = FormatFlushableIngest } // Empty returns true if the batch is empty, and false otherwise. @@ -1199,6 +1287,7 @@ func (b *Batch) Reset() { b.commitStats = BatchCommitStats{} b.commitErr = nil b.applied.Store(false) + b.minimumFormatMajorVersion = 0 if b.data != nil { if cap(b.data) > batchMaxRetainedSize { // If the capacity of the buffer is larger than our maximum @@ -1365,7 +1454,8 @@ func (r *BatchReader) Next() (kind InternalKeyKind, ukey []byte, value []byte, o } switch kind { case InternalKeyKindSet, InternalKeyKindMerge, InternalKeyKindRangeDelete, - InternalKeyKindRangeKeySet, InternalKeyKindRangeKeyUnset, InternalKeyKindRangeKeyDelete: + InternalKeyKindRangeKeySet, InternalKeyKindRangeKeyUnset, InternalKeyKindRangeKeyDelete, + InternalKeyKindDeleteSized: *r, value, ok = batchDecodeStr(*r) if !ok { return 0, nil, nil, false @@ -1500,7 +1590,8 @@ func (i *batchIter) value() []byte { switch InternalKeyKind(data[offset]) { case InternalKeyKindSet, InternalKeyKindMerge, InternalKeyKindRangeDelete, - InternalKeyKindRangeKeySet, InternalKeyKindRangeKeyUnset, InternalKeyKindRangeKeyDelete: + InternalKeyKindRangeKeySet, InternalKeyKindRangeKeyUnset, InternalKeyKindRangeKeyDelete, + InternalKeyKindDeleteSized: _, value, ok := batchDecodeStr(data[keyEnd:]) if !ok { return nil @@ -1944,7 +2035,8 @@ func (i *flushableBatchIter) value() base.LazyValue { var ok bool switch kind { case InternalKeyKindSet, InternalKeyKindMerge, InternalKeyKindRangeDelete, - InternalKeyKindRangeKeySet, InternalKeyKindRangeKeyUnset, InternalKeyKindRangeKeyDelete: + InternalKeyKindRangeKeySet, InternalKeyKindRangeKeyUnset, InternalKeyKindRangeKeyDelete, + InternalKeyKindDeleteSized: keyEnd := i.offsets[i.index].keyEnd _, value, ok = batchDecodeStr(i.data[keyEnd:]) if !ok { diff --git a/batch_test.go b/batch_test.go index 104677b510..720f178ac8 100644 --- a/batch_test.go +++ b/batch_test.go @@ -32,6 +32,7 @@ func TestBatch(t *testing.T) { type testCase struct { kind InternalKeyKind key, value string + valueInt uint32 } verifyTestCases := func(b *Batch, testCases []testCase) { @@ -68,31 +69,33 @@ func TestBatch(t *testing.T) { // deferred variants. This is a consequence of these keys' more complex // value encodings. testCases := []testCase{ - {InternalKeyKindIngestSST, encodeFileNum(1), ""}, - {InternalKeyKindSet, "roses", "red"}, - {InternalKeyKindSet, "violets", "blue"}, - {InternalKeyKindDelete, "roses", ""}, - {InternalKeyKindSingleDelete, "roses", ""}, - {InternalKeyKindSet, "", ""}, - {InternalKeyKindSet, "", "non-empty"}, - {InternalKeyKindDelete, "", ""}, - {InternalKeyKindSingleDelete, "", ""}, - {InternalKeyKindSet, "grass", "green"}, - {InternalKeyKindSet, "grass", "greener"}, - {InternalKeyKindSet, "eleventy", strings.Repeat("!!11!", 100)}, - {InternalKeyKindDelete, "nosuchkey", ""}, - {InternalKeyKindSingleDelete, "nosuchkey", ""}, - {InternalKeyKindSet, "binarydata", "\x00"}, - {InternalKeyKindSet, "binarydata", "\xff"}, - {InternalKeyKindMerge, "merge", "mergedata"}, - {InternalKeyKindMerge, "merge", ""}, - {InternalKeyKindMerge, "", ""}, - {InternalKeyKindRangeDelete, "a", "b"}, - {InternalKeyKindRangeDelete, "", ""}, - {InternalKeyKindLogData, "logdata", ""}, - {InternalKeyKindLogData, "", ""}, - {InternalKeyKindRangeKeyDelete, "grass", "green"}, - {InternalKeyKindRangeKeyDelete, "", ""}, + {InternalKeyKindIngestSST, encodeFileNum(1), "", 0}, + {InternalKeyKindSet, "roses", "red", 0}, + {InternalKeyKindSet, "violets", "blue", 0}, + {InternalKeyKindDelete, "roses", "", 0}, + {InternalKeyKindSingleDelete, "roses", "", 0}, + {InternalKeyKindSet, "", "", 0}, + {InternalKeyKindSet, "", "non-empty", 0}, + {InternalKeyKindDelete, "", "", 0}, + {InternalKeyKindSingleDelete, "", "", 0}, + {InternalKeyKindSet, "grass", "green", 0}, + {InternalKeyKindSet, "grass", "greener", 0}, + {InternalKeyKindSet, "eleventy", strings.Repeat("!!11!", 100), 0}, + {InternalKeyKindDelete, "nosuchkey", "", 0}, + {InternalKeyKindDeleteSized, "eleventy", string(binary.AppendUvarint([]byte(nil), 508)), 500}, + {InternalKeyKindSingleDelete, "nosuchkey", "", 0}, + {InternalKeyKindSet, "binarydata", "\x00", 0}, + {InternalKeyKindSet, "binarydata", "\xff", 0}, + {InternalKeyKindMerge, "merge", "mergedata", 0}, + {InternalKeyKindMerge, "merge", "", 0}, + {InternalKeyKindMerge, "", "", 0}, + {InternalKeyKindRangeDelete, "a", "b", 0}, + {InternalKeyKindRangeDelete, "", "", 0}, + {InternalKeyKindLogData, "logdata", "", 0}, + {InternalKeyKindLogData, "", "", 0}, + {InternalKeyKindRangeKeyDelete, "grass", "green", 0}, + {InternalKeyKindRangeKeyDelete, "", "", 0}, + {InternalKeyKindDeleteSized, "nosuchkey", string(binary.AppendUvarint([]byte(nil), 11)), 2}, } var b Batch for _, tc := range testCases { @@ -103,6 +106,8 @@ func TestBatch(t *testing.T) { _ = b.Merge([]byte(tc.key), []byte(tc.value), nil) case InternalKeyKindDelete: _ = b.Delete([]byte(tc.key), nil) + case InternalKeyKindDeleteSized: + _ = b.DeleteSized([]byte(tc.key), tc.valueInt, nil) case InternalKeyKindSingleDelete: _ = b.SingleDelete([]byte(tc.key), nil) case InternalKeyKindRangeDelete: @@ -139,6 +144,10 @@ func TestBatch(t *testing.T) { copy(d.Key, key) copy(d.Value, value) d.Finish() + case InternalKeyKindDeleteSized: + d := b.DeleteSizedDeferred(len(tc.key), tc.valueInt) + copy(d.Key, key) + d.Finish() case InternalKeyKindSingleDelete: d := b.SingleDeleteDeferred(len(key)) copy(d.Key, key) @@ -334,6 +343,7 @@ func TestBatchReset(t *testing.T) { require.Equal(t, batchHeaderLen, len(b.data)) require.Equal(t, uint64(0), b.SeqNum()) require.Equal(t, uint64(0), b.memTableSize) + require.Equal(t, FormatMajorVersion(0x00), b.minimumFormatMajorVersion) require.Equal(t, b.deferredOp, DeferredBatchOp{}) _ = b.Repr() @@ -441,7 +451,7 @@ func TestIndexedBatchMutation(t *testing.T) { opts := &Options{ Comparer: testkeys.Comparer, FS: vfs.NewMem(), - FormatMajorVersion: FormatNewest, + FormatMajorVersion: internalFormatNewest, } d, err := Open("", opts) require.NoError(t, err) @@ -540,7 +550,7 @@ func TestIndexedBatchMutation(t *testing.T) { func TestIndexedBatch_GlobalVisibility(t *testing.T) { opts := &Options{ FS: vfs.NewMem(), - FormatMajorVersion: FormatNewest, + FormatMajorVersion: internalFormatNewest, Comparer: testkeys.Comparer, } d, err := Open("", opts) @@ -1452,7 +1462,7 @@ func TestBatchSpanCaching(t *testing.T) { opts := &Options{ Comparer: testkeys.Comparer, FS: vfs.NewMem(), - FormatMajorVersion: FormatNewest, + FormatMajorVersion: internalFormatNewest, } d, err := Open("", opts) require.NoError(t, err) diff --git a/checkpoint_test.go b/checkpoint_test.go index b09f455640..f28fbeadf7 100644 --- a/checkpoint_test.go +++ b/checkpoint_test.go @@ -31,7 +31,7 @@ func TestCheckpoint(t *testing.T) { var memLog base.InMemLogger opts := &Options{ FS: vfs.WithLogging(mem, memLog.Infof), - FormatMajorVersion: FormatNewest, + FormatMajorVersion: internalFormatNewest, L0CompactionThreshold: 10, } @@ -296,7 +296,7 @@ func TestCheckpointManyFiles(t *testing.T) { const checkpointPath = "checkpoint" opts := &Options{ FS: vfs.NewMem(), - FormatMajorVersion: FormatNewest, + FormatMajorVersion: internalFormatNewest, DisableAutomaticCompactions: true, } // Disable compression to speed up the test. diff --git a/compaction_iter.go b/compaction_iter.go index e09e5b0cea..b013036b6b 100644 --- a/compaction_iter.go +++ b/compaction_iter.go @@ -6,6 +6,7 @@ package pebble import ( "bytes" + "encoding/binary" "fmt" "io" "sort" @@ -386,7 +387,7 @@ func (i *compactionIter) Next() (*InternalKey, []byte) { } switch i.iterKey.Kind() { - case InternalKeyKindDelete, InternalKeyKindSingleDelete: + case InternalKeyKindDelete, InternalKeyKindSingleDelete, InternalKeyKindDeleteSized: if i.elideTombstone(i.iterKey.UserKey) { if i.curSnapshotIdx == 0 { // If we're at the last snapshot stripe and the tombstone @@ -410,6 +411,12 @@ func (i *compactionIter) Next() (*InternalKey, []byte) { i.skip = true return &i.key, i.value + case InternalKeyKindDeleteSized: + // We may skip subsequent keys because of this tombstone. Scan + // ahead to see just how much data this tombstone drops and if + // the tombstone's value should be updated accordingly. + return i.deleteSizedNext() + case InternalKeyKindSingleDelete: if i.singleDeleteNext() { return &i.key, i.value @@ -686,11 +693,12 @@ func (i *compactionIter) setNext() { i.skip = true return case sameStripeSkippable: - // We're still in the same stripe. If this is a DEL/SINGLEDEL, we - // stop looking and emit a SETWITHDEL. Subsequent keys are - // eligible for skipping. + // We're still in the same stripe. If this is a + // DEL/SINGLEDEL/DELSIZED, we stop looking and emit a SETWITHDEL. + // Subsequent keys are eligible for skipping. if i.iterKey.Kind() == InternalKeyKindDelete || - i.iterKey.Kind() == InternalKeyKindSingleDelete { + i.iterKey.Kind() == InternalKeyKindSingleDelete || + i.iterKey.Kind() == InternalKeyKindDeleteSized { i.key.SetKind(InternalKeyKindSetWithDelete) i.skip = true return @@ -715,7 +723,7 @@ func (i *compactionIter) mergeNext(valueMerger ValueMerger) stripeChangeType { } key := i.iterKey switch key.Kind() { - case InternalKeyKindDelete, InternalKeyKindSingleDelete: + case InternalKeyKindDelete, InternalKeyKindSingleDelete, InternalKeyKindDeleteSized: // We've hit a deletion tombstone. Return everything up to this point and // then skip entries until the next snapshot stripe. We change the kind // of the result key to a Set so that it shadows keys in lower @@ -802,9 +810,9 @@ func (i *compactionIter) singleDeleteNext() bool { key := i.iterKey switch key.Kind() { - case InternalKeyKindDelete, InternalKeyKindMerge, InternalKeyKindSetWithDelete: - // We've hit a Delete, Merge or SetWithDelete, transform the - // SingleDelete into a full Delete. + case InternalKeyKindDelete, InternalKeyKindMerge, InternalKeyKindSetWithDelete, InternalKeyKindDeleteSized: + // We've hit a Delete, DeleteSized, Merge, SetWithDelete, transform + // the SingleDelete into a full Delete. i.key.SetKind(InternalKeyKindDelete) i.skip = true return true @@ -825,6 +833,109 @@ func (i *compactionIter) singleDeleteNext() bool { } } +// deleteSizedNext processes a DELSIZED point tombstone. Unlike ordinary DELs, +// these tombstones carry a value that's a varint indicating the size of the +// entry (len(key)+len(value)) that the tombstone is expected to delete. +// +// When a deleteSizedNext is encountered, we skip ahead to see which keys, if +// any, are elided as a result of the tombstone. +func (i *compactionIter) deleteSizedNext() (*base.InternalKey, []byte) { + i.saveKey() + i.valid = true + i.skip = true + + // The DELSIZED tombstone may have no value at all. This happens when the + // tombstone has already deleted the key that the user originally predicted. + // In this case, we still peek forward in case there's another DELSIZED key + // with a lower sequence number, in which case we'll adopt its value. + if len(i.iterValue) == 0 { + i.value = i.valueBuf[:0] + } else { + i.valueBuf = append(i.valueBuf[:0], i.iterValue...) + i.value = i.valueBuf + } + + // Loop through all the keys within this stripe that are skippable, + // totalling the size of their keys and values. + i.pos = iterPosNext + var elidedSize uint64 + for i.nextInStripe() == sameStripeSkippable { + elidedSize += uint64(len(i.iterKey.UserKey)) + uint64(len(i.iterValue)) + + if i.iterKey.Kind() == InternalKeyKindDeleteSized { + // We encountered a DELSIZED that's deleted by the original + // DELSIZED. These tombstones could've been intended to delete two + // distinct values, eg: + // + // a.DELSIZED.9 a.SET.7 a.DELSIZED.5 a.SET.4 + // + // If a.DELSIZED.9 has already deleted a.SET.7, its size has already + // been zeroed out. In this case, we want to adopt the value of the + // DELSIZED with the lower sequence number, in case the a.SET.4 key + // has not yet been elided. + i.valueBuf = append(i.valueBuf[:0], i.iterValue...) + i.value = i.valueBuf + // Reset the elided total. + elidedSize = 0 + } + } + // Reset skip if we landed outside the original stripe. Otherwise, we landed + // in the same stripe on a non-skippable key. In that case we should preserve + // `i.skip == true` such that later keys in the stripe will continue to be + // skipped. + if i.iterStripeChange == newStripeNewKey || i.iterStripeChange == newStripeSameKey { + i.skip = false + } + + if elidedSize == 0 { + // If we didn't find any keys that we could elide due to this tombstone + // (or at least none since the last DELSIZED encountered), we can return + // the key and our existing saved value verbatim. + return &i.key, i.value + } + + // Some key(s) were elided as a result of this tombstone. Decode the + // tombstone's value to see if it matches the amount of data actually + // elided. + var v uint64 + if len(i.value) > 0 { + var n int + v, n = binary.Uvarint(i.value) + if n != len(i.value) { + i.err = base.CorruptionErrorf("DELSIZED holds invalid value: %x", errors.Safe(i.value)) + i.valid = false + return nil, nil + } + } + + // If the user's prediction of the value size was correct, then from now on + // it's predicted to delete nothing. Return an empty value. + // + // NB: It's important that we still always return the tombstone, because + // DELSIZED's values are a best effort optimization, not a correctness + // mechanic. There is no guarantee that this key was only set once before + // the DELSIZED was written. + + // If the tombstone carried a larger size than what was actually elided, + // it's unclear what to do. The user-provided size was wrong, so it's + // unlikely to be accurate or meaningful. We could: + // + // 1. return the DELSIZED with the original user-provided size unmodified + // 2. return the DELZIZED with a zeroed size to reflect that a key was + // elided, even if it wasn't the anticipated size. + // 3. subtract the elided size from the estimate and re-encode. + // 4. convert the DELSIZED into a value-less DEL, so that ordinary DEL + // heuristics apply. + // + // We opt for (4) under the rationale that we can't rely on the + // user-provided size for accuracy, so ordinary DEL heuristics are safer. + i.value = i.valueBuf[:0] + if elidedSize != v { + i.key.SetKind(InternalKeyKindDelete) + } + return &i.key, i.value +} + func (i *compactionIter) saveKey() { i.keyBuf = append(i.keyBuf[:0], i.iterKey.UserKey...) i.key.UserKey = i.keyBuf diff --git a/compaction_iter_test.go b/compaction_iter_test.go index 52b05d291e..38f7cfc66a 100644 --- a/compaction_iter_test.go +++ b/compaction_iter_test.go @@ -6,6 +6,7 @@ package pebble import ( "bytes" + "encoding/binary" "fmt" "io" "sort" @@ -18,6 +19,7 @@ import ( "github.com/cockroachdb/pebble/internal/keyspan" "github.com/cockroachdb/pebble/internal/rangekey" "github.com/cockroachdb/pebble/internal/testkeys" + "github.com/stretchr/testify/require" ) func TestSnapshotIndex(t *testing.T) { @@ -90,7 +92,10 @@ func TestCompactionIter(t *testing.T) { if formatVersion < FormatSetWithDelete { return "testdata/compaction_iter" } - return "testdata/compaction_iter_set_with_del" + if formatVersion < ExperimentalFormatDeleteSized { + return "testdata/compaction_iter_set_with_del" + } + return "testdata/compaction_iter_delete_sized" } newIter := func(formatVersion FormatMajorVersion) *compactionIter { @@ -150,7 +155,16 @@ func TestCompactionIter(t *testing.T) { for _, key := range strings.Split(d.Input, "\n") { j := strings.Index(key, ":") keys = append(keys, base.ParseInternalKey(key[:j])) - vals = append(vals, []byte(key[j+1:])) + + if strings.HasPrefix(key[j+1:], "varint(") { + valueStr := strings.TrimSuffix(strings.TrimPrefix(key[j+1:], "varint("), ")") + v, err := strconv.ParseUint(valueStr, 10, 64) + require.NoError(t, err) + encodedValue := binary.AppendUvarint([]byte(nil), v) + vals = append(vals, encodedValue) + } else { + vals = append(vals, []byte(key[j+1:])) + } } return "" @@ -243,7 +257,16 @@ func TestCompactionIter(t *testing.T) { snapshotPinned = " (pinned)" } } - fmt.Fprintf(&b, "%s:%s%s\n", iter.Key(), iter.Value(), snapshotPinned) + v := string(iter.Value()) + if iter.Key().Kind() == base.InternalKeyKindDeleteSized && len(iter.Value()) > 0 { + vn, n := binary.Uvarint(iter.Value()) + if n != len(iter.Value()) { + v = fmt.Sprintf("err: %0x value not a uvarint", iter.Value()) + } else { + v = fmt.Sprintf("varint(%d)", vn) + } + } + fmt.Fprintf(&b, "%s:%s%s\n", iter.Key(), v, snapshotPinned) if iter.Key().Kind() == InternalKeyKindRangeDelete { iter.rangeDelFrag.Add(keyspan.Span{ Start: append([]byte{}, iter.Key().UserKey...), @@ -276,7 +299,7 @@ func TestCompactionIter(t *testing.T) { FormatMostCompatible, FormatSetWithDelete - 1, FormatSetWithDelete, - FormatNewest, + internalFormatNewest, } for _, formatVersion := range formatVersions { t.Run(fmt.Sprintf("version-%s", formatVersion), func(t *testing.T) { diff --git a/compaction_test.go b/compaction_test.go index 3129399b71..b375d8294d 100644 --- a/compaction_test.go +++ b/compaction_test.go @@ -1581,7 +1581,7 @@ func TestManualCompaction(t *testing.T) { { testData: "testdata/manual_compaction_set_with_del", minVersion: FormatSetWithDelete, - maxVersion: FormatNewest, + maxVersion: internalFormatNewest, }, { testData: "testdata/singledel_manual_compaction", @@ -1591,18 +1591,23 @@ func TestManualCompaction(t *testing.T) { { testData: "testdata/singledel_manual_compaction_set_with_del", minVersion: FormatSetWithDelete, - maxVersion: FormatNewest, + maxVersion: internalFormatNewest, }, { testData: "testdata/manual_compaction_range_keys", minVersion: FormatRangeKeys, - maxVersion: FormatNewest, + maxVersion: internalFormatNewest, verbose: true, }, { testData: "testdata/manual_compaction_file_boundaries", minVersion: FormatMostCompatible, - maxVersion: FormatNewest, + maxVersion: ExperimentalFormatDeleteSized - 1, + }, + { + testData: "testdata/manual_compaction_file_boundaries_delsized", + minVersion: ExperimentalFormatDeleteSized, + maxVersion: internalFormatNewest, }, } @@ -1948,7 +1953,7 @@ func TestCompactionDeleteOnlyHints(t *testing.T) { compactInfo = &info }, }, - FormatMajorVersion: FormatNewest, + FormatMajorVersion: internalFormatNewest, }).WithFSDefaults() // Collection of table stats can trigger compactions. As we want full @@ -2215,7 +2220,7 @@ func TestCompactionTombstones(t *testing.T) { compactInfo = &info }, }, - FormatMajorVersion: FormatNewest, + FormatMajorVersion: internalFormatNewest, }).WithFSDefaults() var err error d, err = runDBDefineCmd(td, opts) @@ -3695,7 +3700,7 @@ func TestMarkedForCompaction(t *testing.T) { FS: mem, DebugCheck: DebugCheckLevels, DisableAutomaticCompactions: true, - FormatMajorVersion: FormatNewest, + FormatMajorVersion: internalFormatNewest, EventListener: &EventListener{ CompactionEnd: func(info CompactionInfo) { // Fix the job ID and durations for determinism. diff --git a/data_test.go b/data_test.go index ce1d06f6a4..164da879e1 100644 --- a/data_test.go +++ b/data_test.go @@ -527,6 +527,16 @@ func runBatchDefineCmd(d *datadriven.TestData, b *Batch) error { return errors.Errorf("%s expects 1 argument", parts[0]) } err = b.Delete([]byte(parts[1]), nil) + case "del-sized": + if len(parts) != 3 { + return errors.Errorf("%s expects 1 argument", parts[0]) + } + var valSize uint64 + valSize, err = strconv.ParseUint(parts[2], 10, 32) + if err != nil { + return err + } + err = b.DeleteSized([]byte(parts[1]), uint32(valSize), nil) case "singledel": if len(parts) != 2 { return errors.Errorf("%s expects 1 argument", parts[0]) @@ -808,6 +818,12 @@ func runDBDefineCmd(td *datadriven.TestData, opts *Options) (*DB, error) { return nil, errors.Errorf("%s: could not parse %q as float: %s", td.Cmd, arg.Vals[0], err) } opts.Experimental.PointTombstoneWeight = w + case "format-major-version": + fmv, err := strconv.Atoi(arg.Vals[0]) + if err != nil { + return nil, err + } + opts.FormatMajorVersion = FormatMajorVersion(fmv) } } d, err := Open("", opts) diff --git a/db.go b/db.go index 0cc2f3c695..b4173ffe01 100644 --- a/db.go +++ b/db.go @@ -96,6 +96,22 @@ type Writer interface { // It is safe to modify the contents of the arguments after Delete returns. Delete(key []byte, o *WriteOptions) error + // DeleteSized behaves identically to Delete, but takes an additional + // argument indicating the size of the value being deleted. DeleteSized + // should be preferred when the caller has the expectation that there exists + // a single internal KV pair for the key (eg, the key has not been + // overwritten recently), and the caller knows the size of its value. + // + // DeleteSized will record the value size within the tombstone and use it to + // inform compaction-picking heuristics which strive to reduce space + // amplification in the LSM. This "calling your shot" mechanic allows the + // storage engine to more accurately estimate and reduce space + // amplification. + // + // It is safe to modify the contents of the arguments after DeleteSized + // returns. + DeleteSized(key []byte, valueSize uint32, _ *WriteOptions) error + // SingleDelete is similar to Delete in that it deletes the value for the given key. Like Delete, // it is a blind operation that will succeed even if the given key does not exist. // @@ -620,6 +636,30 @@ func (d *DB) Delete(key []byte, opts *WriteOptions) error { return nil } +// DeleteSized behaves identically to Delete, but takes an additional +// argument indicating the size of the value being deleted. DeleteSized +// should be preferred when the caller has the expectation that there exists +// a single internal KV pair for the key (eg, the key has not been +// overwritten recently), and the caller knows the size of its value. +// +// DeleteSized will record the value size within the tombstone and use it to +// inform compaction-picking heuristics which strive to reduce space +// amplification in the LSM. This "calling your shot" mechanic allows the +// storage engine to more accurately estimate and reduce space amplification. +// +// It is safe to modify the contents of the arguments after DeleteSized +// returns. +func (d *DB) DeleteSized(key []byte, valueSize uint32, opts *WriteOptions) error { + b := newBatch(d) + _ = b.DeleteSized(key, valueSize, opts) + if err := d.Apply(b, opts); err != nil { + return err + } + // Only release the batch on success. + b.release() + return nil +} + // SingleDelete adds an action to the batch that single deletes the entry for key. // See Writer.SingleDelete for more details on the semantics of SingleDelete. // @@ -786,17 +826,19 @@ func (d *DB) applyInternal(batch *Batch, opts *WriteOptions, noSyncWait bool) er return errors.New("pebble: WAL disabled") } - if batch.countRangeKeys > 0 { - if d.split == nil { - return errNoSplit - } - if d.FormatMajorVersion() < FormatRangeKeys { + if batch.minimumFormatMajorVersion != FormatMostCompatible { + if fmv := d.FormatMajorVersion(); fmv < batch.minimumFormatMajorVersion { panic(fmt.Sprintf( - "pebble: range keys require at least format major version %d (current: %d)", - FormatRangeKeys, d.FormatMajorVersion(), + "pebble: batch requires at least format major version %d (current: %d)", + batch.minimumFormatMajorVersion, fmv, )) } + } + if batch.countRangeKeys > 0 { + if d.split == nil { + return errNoSplit + } // TODO(jackson): Assert that all range key operands are suffixless. } diff --git a/db_test.go b/db_test.go index 1bea7dc7d9..2a140c3966 100644 --- a/db_test.go +++ b/db_test.go @@ -1095,7 +1095,7 @@ func TestDBClosed(t *testing.T) { require.True(t, errors.Is(catch(func() { _ = d.Ingest(nil) }), ErrClosed)) require.True(t, errors.Is(catch(func() { _ = d.LogData(nil, nil) }), ErrClosed)) require.True(t, errors.Is(catch(func() { _ = d.Merge(nil, nil, nil) }), ErrClosed)) - require.True(t, errors.Is(catch(func() { _ = d.RatchetFormatMajorVersion(FormatNewest) }), ErrClosed)) + require.True(t, errors.Is(catch(func() { _ = d.RatchetFormatMajorVersion(internalFormatNewest) }), ErrClosed)) require.True(t, errors.Is(catch(func() { _ = d.Set(nil, nil, nil) }), ErrClosed)) require.True(t, errors.Is(catch(func() { _ = d.NewSnapshot() }), ErrClosed)) diff --git a/event_listener_test.go b/event_listener_test.go index 20ba8aa8f4..ad31c2ff81 100644 --- a/event_listener_test.go +++ b/event_listener_test.go @@ -37,7 +37,7 @@ func TestEventListener(t *testing.T) { lel := MakeLoggingEventListener(&memLog) opts := &Options{ FS: vfs.WithLogging(mem, memLog.Infof), - FormatMajorVersion: FormatNewest, + FormatMajorVersion: internalFormatNewest, EventListener: &lel, MaxManifestFileSize: 1, L0CompactionThreshold: 10, diff --git a/flushable_test.go b/flushable_test.go index 713a2229ea..7c5273da13 100644 --- a/flushable_test.go +++ b/flushable_test.go @@ -33,7 +33,7 @@ func TestIngestedSSTFlushableAPI(t *testing.T) { L0CompactionThreshold: 100, L0StopWritesThreshold: 100, DebugCheck: DebugCheckLevels, - FormatMajorVersion: FormatNewest, + FormatMajorVersion: internalFormatNewest, } // Disable automatic compactions because otherwise we'll race with // delete-only compactions triggered by ingesting range tombstones. diff --git a/format_major_version.go b/format_major_version.go index 6d96297075..b2a561fb8b 100644 --- a/format_major_version.go +++ b/format_major_version.go @@ -151,8 +151,20 @@ const ( // compactions for files marked for compaction are complete. FormatPrePebblev1MarkedCompacted + // ExperimentalFormatDeleteSized is a format major version that adds support for + // deletion tombstones that encode the size of the value they're expected to + // delete. This format major version is required before the associated key + // kind may be committed through batch applications or ingests. + ExperimentalFormatDeleteSized + + // internalFormatNewest holds the newest format major version, including + // experimental ones excluded from the exported FormatNewest constant until + // they've stabilized. Used in tests. + internalFormatNewest FormatMajorVersion = iota - 1 + // FormatNewest always contains the most recent format major version. - FormatNewest FormatMajorVersion = iota - 1 + // TODO(jackson): restore FormatNewest to internalFormatNewest. + FormatNewest FormatMajorVersion = FormatPrePebblev1MarkedCompacted ) // MaxTableFormat returns the maximum sstable.TableFormat that can be used at @@ -170,6 +182,8 @@ func (v FormatMajorVersion) MaxTableFormat() sstable.TableFormat { return sstable.TableFormatPebblev2 case FormatSSTableValueBlocks, FormatFlushableIngest, FormatPrePebblev1MarkedCompacted: return sstable.TableFormatPebblev3 + case ExperimentalFormatDeleteSized: + return sstable.TableFormatPebblev4 default: panic(fmt.Sprintf("pebble: unsupported format major version: %s", v)) } @@ -186,7 +200,8 @@ func (v FormatMajorVersion) MinTableFormat() sstable.TableFormat { return sstable.TableFormatLevelDB case FormatMinTableFormatPebblev1, FormatPrePebblev1Marked, FormatUnusedPrePebblev1MarkedCompacted, FormatSSTableValueBlocks, - FormatFlushableIngest, FormatPrePebblev1MarkedCompacted: + FormatFlushableIngest, FormatPrePebblev1MarkedCompacted, + ExperimentalFormatDeleteSized: return sstable.TableFormatPebblev1 default: panic(fmt.Sprintf("pebble: unsupported format major version: %s", v)) @@ -319,6 +334,9 @@ var formatMajorVersionMigrations = map[FormatMajorVersion]func(*DB) error{ } return d.finalizeFormatVersUpgrade(FormatPrePebblev1MarkedCompacted) }, + ExperimentalFormatDeleteSized: func(d *DB) error { + return d.finalizeFormatVersUpgrade(ExperimentalFormatDeleteSized) + }, } const formatVersionMarkerName = `format-version` @@ -341,7 +359,7 @@ func lookupFormatMajorVersion( if vers == FormatDefault { return 0, nil, errors.Newf("pebble: default format major version should not persisted", vers) } - if vers > FormatNewest { + if vers > internalFormatNewest { return 0, nil, errors.Newf("pebble: database %q written in format major version %d", dirname, vers) } return vers, m, nil @@ -377,8 +395,8 @@ func (d *DB) ratchetFormatMajorVersionLocked(formatVers FormatMajorVersion) erro if d.opts.ReadOnly { return ErrReadOnly } - if formatVers > FormatNewest { - // Guard against accidentally forgetting to update FormatNewest. + if formatVers > internalFormatNewest { + // Guard against accidentally forgetting to update internalFormatNewest. return errors.Errorf("pebble: unknown format version %d", formatVers) } if d.mu.formatVers.vers > formatVers { diff --git a/format_major_version_test.go b/format_major_version_test.go index 0dc95f6792..ca136351bf 100644 --- a/format_major_version_test.go +++ b/format_major_version_test.go @@ -62,6 +62,8 @@ func TestRatchetFormat(t *testing.T) { require.Equal(t, FormatFlushableIngest, d.FormatMajorVersion()) require.NoError(t, d.RatchetFormatMajorVersion(FormatPrePebblev1MarkedCompacted)) require.Equal(t, FormatPrePebblev1MarkedCompacted, d.FormatMajorVersion()) + require.NoError(t, d.RatchetFormatMajorVersion(ExperimentalFormatDeleteSized)) + require.Equal(t, ExperimentalFormatDeleteSized, d.FormatMajorVersion()) require.NoError(t, d.Close()) @@ -69,7 +71,7 @@ func TestRatchetFormat(t *testing.T) { // database should Open using the persisted FormatNewest. d, err = Open("", (&Options{FS: fs}).WithFSDefaults()) require.NoError(t, err) - require.Equal(t, FormatNewest, d.FormatMajorVersion()) + require.Equal(t, internalFormatNewest, d.FormatMajorVersion()) require.NoError(t, d.Close()) // Move the marker to a version that does not exist. @@ -225,17 +227,18 @@ func TestFormatMajorVersions_TableFormat(t *testing.T) { FormatSSTableValueBlocks: {sstable.TableFormatPebblev1, sstable.TableFormatPebblev3}, FormatFlushableIngest: {sstable.TableFormatPebblev1, sstable.TableFormatPebblev3}, FormatPrePebblev1MarkedCompacted: {sstable.TableFormatPebblev1, sstable.TableFormatPebblev3}, + ExperimentalFormatDeleteSized: {sstable.TableFormatPebblev1, sstable.TableFormatPebblev4}, } // Valid versions. - for fmv := FormatMostCompatible; fmv <= FormatNewest; fmv++ { + for fmv := FormatMostCompatible; fmv <= internalFormatNewest; fmv++ { got := [2]sstable.TableFormat{fmv.MinTableFormat(), fmv.MaxTableFormat()} require.Equalf(t, m[fmv], got, "got %s; want %s", got, m[fmv]) require.True(t, got[0] <= got[1] /* min <= max */) } // Invalid versions. - fmv := FormatNewest + 1 + fmv := internalFormatNewest + 1 require.Panics(t, func() { _ = fmv.MaxTableFormat() }) require.Panics(t, func() { _ = fmv.MinTableFormat() }) } diff --git a/ingest_test.go b/ingest_test.go index 3ecb8d9093..22a8ab9b42 100644 --- a/ingest_test.go +++ b/ingest_test.go @@ -146,7 +146,7 @@ func TestIngestLoadRand(t *testing.T) { mem := vfs.NewMem() rng := rand.New(rand.NewSource(uint64(time.Now().UnixNano()))) cmp := DefaultComparer.Compare - version := FormatNewest + version := internalFormatNewest randBytes := func(size int) []byte { data := make([]byte, size) @@ -232,7 +232,7 @@ func TestIngestLoadInvalid(t *testing.T) { Comparer: DefaultComparer, FS: mem, }).WithFSDefaults() - if _, _, err := ingestLoad(opts, FormatNewest, []string{"invalid"}, 0, []base.DiskFileNum{base.FileNum(1).DiskFileNum()}); err == nil { + if _, _, err := ingestLoad(opts, internalFormatNewest, []string{"invalid"}, 0, []base.DiskFileNum{base.FileNum(1).DiskFileNum()}); err == nil { t.Fatalf("expected error, but found success") } } @@ -444,7 +444,7 @@ func TestOverlappingIngestedSSTs(t *testing.T) { L0CompactionThreshold: 100, L0StopWritesThreshold: 100, DebugCheck: DebugCheckLevels, - FormatMajorVersion: FormatNewest, + FormatMajorVersion: internalFormatNewest, }).WithFSDefaults() // Disable automatic compactions because otherwise we'll race with // delete-only compactions triggered by ingesting range tombstones. @@ -754,7 +754,7 @@ func TestIngestTargetLevel(t *testing.T) { var err error opts := Options{ - FormatMajorVersion: FormatNewest, + FormatMajorVersion: internalFormatNewest, } opts.WithFSDefaults() if d, err = runDBDefineCmd(td, &opts); err != nil { @@ -826,7 +826,7 @@ func TestIngest(t *testing.T) { EventListener: &EventListener{FlushEnd: func(info FlushInfo) { flushed = true }}, - FormatMajorVersion: FormatNewest, + FormatMajorVersion: internalFormatNewest, } // Disable automatic compactions because otherwise we'll race with // delete-only compactions triggered by ingesting range tombstones. @@ -1220,7 +1220,7 @@ func TestIngestFlushQueuedMemTable(t *testing.T) { // Test with a format major version prior to FormatFlushableIngest and one // after. Both should result in the same statistic calculations. - for _, fmv := range []FormatMajorVersion{FormatFlushableIngest - 1, FormatNewest} { + for _, fmv := range []FormatMajorVersion{FormatFlushableIngest - 1, internalFormatNewest} { func(fmv FormatMajorVersion) { mem := vfs.NewMem() d, err := Open("", &Options{ diff --git a/internal.go b/internal.go index b505207fac..61a4284dae 100644 --- a/internal.go +++ b/internal.go @@ -23,6 +23,7 @@ const ( InternalKeyKindRangeKeyUnset = base.InternalKeyKindRangeKeyUnset InternalKeyKindRangeKeyDelete = base.InternalKeyKindRangeKeyDelete InternalKeyKindIngestSST = base.InternalKeyKindIngestSST + InternalKeyKindDeleteSized = base.InternalKeyKindDeleteSized InternalKeyKindInvalid = base.InternalKeyKindInvalid InternalKeySeqNumBatch = base.InternalKeySeqNumBatch InternalKeySeqNumMax = base.InternalKeySeqNumMax diff --git a/internal/base/internal.go b/internal/base/internal.go index 5be485dcfa..6f954bb851 100644 --- a/internal/base/internal.go +++ b/internal/base/internal.go @@ -66,8 +66,15 @@ const ( // batch, or in an sstable. InternalKeyKindIngestSST InternalKeyKind = 22 - // This maximum value isn't part of the file format. It's unlikely, - // but future extensions may increase this value. + // InternalKeyKindDeleteSized keys behave identically to + // InternalKeyKindDelete keys, except that they hold an associated uint64 + // value indicating the (len(key)+len(value)) of the shadowed entry the + // tombstone is expected to delete. This value is used to inform compaction + // heuristics, but is not required to be accurate for correctness. + InternalKeyKindDeleteSized = 23 + + // This maximum value isn't part of the file format. Future extensions may + // increase this value. // // When constructing an internal key to pass to DB.Seek{GE,LE}, // internalKeyComparer sorts decreasing by kind (after sorting increasing by @@ -75,7 +82,7 @@ const ( // which sorts 'less than or equal to' any other valid internalKeyKind, when // searching for any kind of internal key formed by a certain user key and // seqNum. - InternalKeyKindMax InternalKeyKind = 22 + InternalKeyKindMax InternalKeyKind = 23 // InternalKeyZeroSeqnumMaxTrailer is the largest trailer with a // zero sequence number. @@ -118,6 +125,7 @@ var internalKeyKindNames = []string{ InternalKeyKindRangeKeyUnset: "RANGEKEYUNSET", InternalKeyKindRangeKeyDelete: "RANGEKEYDEL", InternalKeyKindIngestSST: "INGESTSST", + InternalKeyKindDeleteSized: "DELSIZED", InternalKeyKindInvalid: "INVALID", } @@ -204,6 +212,7 @@ var kindsMap = map[string]InternalKeyKind{ "RANGEKEYUNSET": InternalKeyKindRangeKeyUnset, "RANGEKEYDEL": InternalKeyKindRangeKeyDelete, "INGESTSST": InternalKeyKindIngestSST, + "DELSIZED": InternalKeyKindDeleteSized, } // ParseInternalKey parses the string representation of an internal key. The diff --git a/internal/base/internal_test.go b/internal/base/internal_test.go index a561dbf9fc..39466cd97d 100644 --- a/internal/base/internal_test.go +++ b/internal/base/internal_test.go @@ -41,7 +41,7 @@ func TestInvalidInternalKey(t *testing.T) { "\x01\x02\x03\x04\x05\x06\x07", "foo", "foo\x08\x07\x06\x05\x04\x03\x02", - "foo\x17\x07\x06\x05\x04\x03\x02\x01", + "foo\x18\x07\x06\x05\x04\x03\x02\x01", } for _, tc := range testCases { k := DecodeInternalKey([]byte(tc)) diff --git a/internal/rangekey/rangekey_test.go b/internal/rangekey/rangekey_test.go index 2092805f3f..deec970db7 100644 --- a/internal/rangekey/rangekey_test.go +++ b/internal/rangekey/rangekey_test.go @@ -222,6 +222,10 @@ func TestIsRangeKey(t *testing.T) { kind: base.InternalKeyKindDelete, want: false, }, + { + kind: base.InternalKeyKindDeleteSized, + want: false, + }, { kind: base.InternalKeyKindSet, want: false, diff --git a/iterator.go b/iterator.go index 3348274122..996817c78b 100644 --- a/iterator.go +++ b/iterator.go @@ -568,7 +568,7 @@ func (i *Iterator) findNextEntry(limit []byte) { i.iterValidityState = IterValid return - case InternalKeyKindDelete, InternalKeyKindSingleDelete: + case InternalKeyKindDelete, InternalKeyKindSingleDelete, InternalKeyKindDeleteSized: i.nextUserKey() continue @@ -631,7 +631,7 @@ func (i *Iterator) nextPointCurrentUserKey() bool { i.err = base.CorruptionErrorf("pebble: unexpected range key set mid-user key") return false - case InternalKeyKindDelete, InternalKeyKindSingleDelete: + case InternalKeyKindDelete, InternalKeyKindSingleDelete, InternalKeyKindDeleteSized: return false case InternalKeyKindSet, InternalKeyKindSetWithDelete: @@ -929,7 +929,7 @@ func (i *Iterator) findPrevEntry(limit []byte) { // return the key even if the MERGE point key is deleted. rangeKeyBoundary = true - case InternalKeyKindDelete, InternalKeyKindSingleDelete: + case InternalKeyKindDelete, InternalKeyKindSingleDelete, InternalKeyKindDeleteSized: i.value = LazyValue{} i.iterValidityState = IterExhausted valueMerger = nil @@ -1092,7 +1092,7 @@ func (i *Iterator) mergeNext(key InternalKey, valueMerger ValueMerger) { return } switch key.Kind() { - case InternalKeyKindDelete, InternalKeyKindSingleDelete: + case InternalKeyKindDelete, InternalKeyKindSingleDelete, InternalKeyKindDeleteSized: // We've hit a deletion tombstone. Return everything up to this // point. return diff --git a/iterator_test.go b/iterator_test.go index 744d641be4..a4393eaab5 100644 --- a/iterator_test.go +++ b/iterator_test.go @@ -886,7 +886,7 @@ func TestIteratorStats(t *testing.T) { mem = vfs.NewMem() require.NoError(t, mem.MkdirAll("ext", 0755)) - opts := &Options{Comparer: testkeys.Comparer, FS: mem, FormatMajorVersion: FormatNewest} + opts := &Options{Comparer: testkeys.Comparer, FS: mem, FormatMajorVersion: internalFormatNewest} // Automatic compactions may make some testcases non-deterministic. opts.DisableAutomaticCompactions = true var err error @@ -1263,7 +1263,7 @@ func TestIteratorBlockIntervalFilter(t *testing.T) { } opts := &Options{ FS: mem, - FormatMajorVersion: FormatNewest, + FormatMajorVersion: internalFormatNewest, BlockPropertyCollectors: bpCollectors, } lo := LevelOptions{BlockSize: 1, IndexBlockSize: 1} @@ -1379,7 +1379,7 @@ func TestIteratorRandomizedBlockIntervalFilter(t *testing.T) { mem := vfs.NewMem() opts := &Options{ FS: mem, - FormatMajorVersion: FormatNewest, + FormatMajorVersion: internalFormatNewest, BlockPropertyCollectors: []func() BlockPropertyCollector{ func() BlockPropertyCollector { return sstable.NewBlockIntervalCollector( diff --git a/level_checker.go b/level_checker.go index 4bbd9a40e6..96321380fa 100644 --- a/level_checker.go +++ b/level_checker.go @@ -165,7 +165,7 @@ func (m *simpleMergingIter) step() bool { if m.valueMerger != nil { // Ongoing series of MERGE records. switch item.key.Kind() { - case InternalKeyKindSingleDelete, InternalKeyKindDelete: + case InternalKeyKindSingleDelete, InternalKeyKindDelete, InternalKeyKindDeleteSized: var closer io.Closer _, closer, m.err = m.valueMerger.Finish(true /* includesBase */) if m.err == nil && closer != nil { diff --git a/mem_table_test.go b/mem_table_test.go index 700257073a..bfe2b6e16d 100644 --- a/mem_table_test.go +++ b/mem_table_test.go @@ -36,7 +36,7 @@ func (m *memTable) get(key []byte) (value []byte, err error) { return nil, ErrNotFound } switch ikey.Kind() { - case InternalKeyKindDelete, InternalKeyKindSingleDelete: + case InternalKeyKindDelete, InternalKeyKindSingleDelete, InternalKeyKindDeleteSized: return nil, ErrNotFound default: return val.InPlaceValue(), nil diff --git a/metamorphic/generator.go b/metamorphic/generator.go index 95ecca5a21..35bab0c446 100644 --- a/metamorphic/generator.go +++ b/metamorphic/generator.go @@ -505,7 +505,15 @@ func (g *generator) dbRatchetFormatMajorVersion() { // metamorphic tests support and the newest. At runtime, the generated // version may be behind the database's format major version, in which case // RatchetFormatMajorVersion should deterministically error. - n := int(pebble.FormatNewest - minimumFormatMajorVersion) + + // TODO(jackson): When FormatDeleteSized is stabilized, return this to just + // using `FormatNewest`. + newestTODO := pebble.FormatNewest + if newestTODO < pebble.ExperimentalFormatDeleteSized { + newestTODO = pebble.ExperimentalFormatDeleteSized + } + + n := int(newestTODO - minimumFormatMajorVersion) vers := pebble.FormatMajorVersion(g.rng.Intn(n + 1)) g.add(&dbRatchetFormatMajorVersionOp{vers: vers}) } diff --git a/metamorphic/ops.go b/metamorphic/ops.go index 1cf31ca5d2..77cff3de2a 100644 --- a/metamorphic/ops.go +++ b/metamorphic/ops.go @@ -7,6 +7,7 @@ package metamorphic import ( "bytes" "crypto/rand" + "encoding/binary" "fmt" "io" "path/filepath" @@ -190,11 +191,26 @@ type deleteOp struct { func (o *deleteOp) run(t *test, h historyRecorder) { w := t.getWriter(o.writerID) - err := w.Delete(o.key, t.writeOpts) + var err error + if t.testOpts.deleteSized && t.isFMV(pebble.ExperimentalFormatDeleteSized) { + // Call DeleteSized with a deterministic size derived from the index. + // The size does not need to be accurate for correctness. + err = w.DeleteSized(o.key, hashSize(t.idx), t.writeOpts) + } else { + err = w.Delete(o.key, t.writeOpts) + } h.Recordf("%s // %v", o, err) } -func (o *deleteOp) String() string { return fmt.Sprintf("%s.Delete(%q)", o.writerID, o.key) } +func hashSize(index int) uint32 { + const maxSize = 16 << 10 /* 16 KB */ + // Fibonacci hash https://probablydance.com/2018/06/16/fibonacci-hashing-the-optimization-that-the-world-forgot-or-a-better-alternative-to-integer-modulo/ + return uint32((11400714819323198485 * uint64(index)) % maxSize) +} + +func (o *deleteOp) String() string { + return fmt.Sprintf("%s.Delete(%q)", o.writerID, o.key) +} func (o *deleteOp) receiver() objID { return o.writerID } func (o *deleteOp) syncObjs() objIDSlice { return nil } @@ -591,6 +607,16 @@ func (o *ingestOp) collapseBatch( switch key.Kind() { case pebble.InternalKeyKindDelete: err = collapsed.Delete(key.UserKey, nil) + case pebble.InternalKeyKindDeleteSized: + v, _ := binary.Uvarint(value.InPlaceValue()) + // Batch.DeleteSized takes just the length of the value being + // deleted and adds the key's length to derive the overall entry + // size of the value being deleted. This has already been done + // to the key we're reading from the batch, so we must subtract + // the key length from the encoded value before calling + // collapsed.DeleteSized, which will again add the key length + // before encoding. + err = collapsed.DeleteSized(key.UserKey, uint32(v-uint64(len(key.UserKey))), nil) case pebble.InternalKeyKindSingleDelete: err = collapsed.SingleDelete(key.UserKey, nil) case pebble.InternalKeyKindSet: diff --git a/metamorphic/options.go b/metamorphic/options.go index b38d3faf86..fc0f400e42 100644 --- a/metamorphic/options.go +++ b/metamorphic/options.go @@ -53,6 +53,9 @@ func parseOptions( case "TestOptions.ingest_using_apply": opts.ingestUsingApply = true return true + case "TestOptions.delete_sized": + opts.deleteSized = true + return true case "TestOptions.replace_single_delete": opts.replaceSingleDelete = true return true @@ -110,6 +113,9 @@ func optionsToString(opts *TestOptions) string { if opts.ingestUsingApply { fmt.Fprint(&buf, " ingest_using_apply=true\n") } + if opts.deleteSized { + fmt.Fprint(&buf, " delete_sized=true\n") + } if opts.replaceSingleDelete { fmt.Fprint(&buf, " replace_single_delete=true\n") } @@ -180,6 +186,8 @@ type TestOptions struct { threads int // Use Batch.Apply rather than DB.Ingest. ingestUsingApply bool + // Use Batch.DeleteSized rather than Batch.Delete. + deleteSized bool // Replace a SINGLEDEL with a DELETE. replaceSingleDelete bool // The path on the local filesystem where the initial state of the database @@ -441,6 +449,7 @@ func randomOptions( opts.DisableWAL = false } testOpts.ingestUsingApply = rng.Intn(2) != 0 + testOpts.deleteSized = rng.Intn(2) != 0 testOpts.replaceSingleDelete = rng.Intn(2) != 0 testOpts.disableBlockPropertyCollector = rng.Intn(2) != 0 if !testOpts.disableBlockPropertyCollector { diff --git a/metamorphic/test.go b/metamorphic/test.go index 43c1975842..ecd232252b 100644 --- a/metamorphic/test.go +++ b/metamorphic/test.go @@ -174,6 +174,10 @@ func (t *test) init(h *history, dir string, testOpts *TestOptions) error { return nil } +func (t *test) isFMV(fmv pebble.FormatMajorVersion) bool { + return t.db.FormatMajorVersion() >= fmv +} + func (t *test) restartDB() error { if !t.testOpts.strictFS { return nil diff --git a/open_test.go b/open_test.go index cb0a38a752..d2433de6c9 100644 --- a/open_test.go +++ b/open_test.go @@ -191,13 +191,13 @@ func TestNewDBFilenames(t *testing.T) { "MANIFEST-000001", "OPTIONS-000003", }, - FormatNewest: { + internalFormatNewest: { "000002.log", "CURRENT", "LOCK", "MANIFEST-000001", "OPTIONS-000003", - "marker.format-version.000013.014", + "marker.format-version.000014.015", "marker.manifest.000001.MANIFEST-000001", }, } diff --git a/options.go b/options.go index 2bdc9d505c..58d09aabbd 100644 --- a/options.go +++ b/options.go @@ -1374,7 +1374,7 @@ func (o *Options) Parse(s string, hooks *ParseHooks) error { // version is valid right here. var v uint64 v, err = strconv.ParseUint(value, 10, 64) - if vers := FormatMajorVersion(v); vers > FormatNewest || vers == FormatDefault { + if vers := FormatMajorVersion(v); vers > internalFormatNewest || vers == FormatDefault { err = errors.Newf("unknown format major version %d", o.FormatMajorVersion) } if err == nil { @@ -1594,9 +1594,9 @@ func (o *Options) Validate() error { fmt.Fprintf(&buf, "MemTableStopWritesThreshold (%d) must be >= 2\n", o.MemTableStopWritesThreshold) } - if o.FormatMajorVersion > FormatNewest { + if o.FormatMajorVersion > internalFormatNewest { fmt.Fprintf(&buf, "FormatMajorVersion (%d) must be <= %d\n", - o.FormatMajorVersion, FormatNewest) + o.FormatMajorVersion, internalFormatNewest) } if o.TableCache != nil && o.Cache != o.TableCache.cache { fmt.Fprintf(&buf, "underlying cache in the TableCache and the Cache dont match\n") diff --git a/options_test.go b/options_test.go index fc04812b8e..2f0d20f1f3 100644 --- a/options_test.go +++ b/options_test.go @@ -26,7 +26,7 @@ func (o *Options) testingRandomized() *Options { if o.FormatMajorVersion == FormatDefault { // Pick a random format major version from the range // [MostCompatible, FormatNewest]. - o.FormatMajorVersion = FormatMajorVersion(rand.Intn(int(FormatNewest)) + 1) + o.FormatMajorVersion = FormatMajorVersion(rand.Intn(int(internalFormatNewest)) + 1) } return o } diff --git a/range_del_test.go b/range_del_test.go index c63a4c4a1c..8e0c10176a 100644 --- a/range_del_test.go +++ b/range_del_test.go @@ -96,7 +96,7 @@ func TestFlushDelay(t *testing.T) { Comparer: testkeys.Comparer, FlushDelayDeleteRange: 10 * time.Millisecond, FlushDelayRangeKey: 10 * time.Millisecond, - FormatMajorVersion: FormatNewest, + FormatMajorVersion: internalFormatNewest, } d, err := Open("", opts) require.NoError(t, err) @@ -195,7 +195,7 @@ func TestFlushDelayStress(t *testing.T) { Comparer: testkeys.Comparer, FlushDelayDeleteRange: time.Duration(rng.Intn(10)+1) * time.Millisecond, FlushDelayRangeKey: time.Duration(rng.Intn(10)+1) * time.Millisecond, - FormatMajorVersion: FormatNewest, + FormatMajorVersion: internalFormatNewest, MemTableSize: 8192, } diff --git a/replay/replay.go b/replay/replay.go index bed52fa0c1..48ac8f6164 100644 --- a/replay/replay.go +++ b/replay/replay.go @@ -12,6 +12,7 @@ package replay import ( "context" + "encoding/binary" "fmt" "io" "os" @@ -1036,6 +1037,15 @@ func loadFlushedSSTableKeys( switch bufs.keys[i].Kind() { case base.InternalKeyKindDelete: err = b.Delete(bufs.keys[i].UserKey, nil) + case base.InternalKeyKindDeleteSized: + v, _ := binary.Uvarint(bufs.keys[i].value) + // Batch.DeleteSized takes just the length of the value being + // deleted and adds the key's length to derive the overall entry + // size of the value being deleted. This has already been done to + // the key we're reading from the sstable, so we must subtract the + // key length from the encoded value before calling b.DeleteSized, + // which will again add the key length before encoding. + err = b.DeleteSized(bufs.keys[i].UserKey, uint32(v-uint64(len(bufs.keys[i].UserKey))), nil) case base.InternalKeyKindSet, base.InternalKeyKindSetWithDelete: err = b.Set(bufs.keys[i].UserKey, bufs.keys[i].value, nil) case base.InternalKeyKindMerge: diff --git a/scan_internal.go b/scan_internal.go index 4984b0f6e3..76d20d29ed 100644 --- a/scan_internal.go +++ b/scan_internal.go @@ -220,7 +220,7 @@ func (p *pointCollapsingIterator) findNextEntry() (*base.InternalKey, base.LazyV continue } switch p.savedKey.Kind() { - case InternalKeyKindSet, InternalKeyKindDelete, InternalKeyKindSetWithDelete: + case InternalKeyKindSet, InternalKeyKindDelete, InternalKeyKindSetWithDelete, InternalKeyKindDeleteSized: p.saveKey() // Note that we return SETs directly, even if they would otherwise get // compacted into a Del to turn into a SetWithDelete. This is a fast diff --git a/sstable/format.go b/sstable/format.go index 91909835c2..e8c620028f 100644 --- a/sstable/format.go +++ b/sstable/format.go @@ -24,7 +24,7 @@ const ( TableFormatPebblev1 // Block properties. TableFormatPebblev2 // Range keys. TableFormatPebblev3 // Value blocks. - TableFormatPebblev4 // Reserved. + TableFormatPebblev4 // DELSIZED tombstones. TableFormatMax = TableFormatPebblev4 ) diff --git a/sstable/internal.go b/sstable/internal.go index 9bab1819bd..039da91074 100644 --- a/sstable/internal.go +++ b/sstable/internal.go @@ -23,6 +23,7 @@ const ( InternalKeyKindLogData = base.InternalKeyKindLogData InternalKeyKindSingleDelete = base.InternalKeyKindSingleDelete InternalKeyKindRangeDelete = base.InternalKeyKindRangeDelete + InternalKeyKindDeleteSized = base.InternalKeyKindDeleteSized InternalKeyKindMax = base.InternalKeyKindMax InternalKeyKindInvalid = base.InternalKeyKindInvalid InternalKeySeqNumBatch = base.InternalKeySeqNumBatch diff --git a/sstable/properties.go b/sstable/properties.go index ca417cefb1..c85aa1db60 100644 --- a/sstable/properties.go +++ b/sstable/properties.go @@ -108,6 +108,9 @@ type Properties struct { // The number of deletion entries in this table, including both point and // range deletions. NumDeletions uint64 `prop:"rocksdb.deleted.keys"` + // The number of point deletion entries ("tombstones") in this table that + // carry a size hint indicating the size of the value the tombstone deletes. + NumSizedDeletions uint64 `prop:"pebble.num.deletions.sized"` // The number of entries in this table. NumEntries uint64 `prop:"rocksdb.num.entries"` // The number of merge operands in the table. @@ -139,6 +142,10 @@ type Properties struct { // Total raw key size of point deletion tombstones. This value is comparable // to RawKeySize. RawPointTombstoneKeySize uint64 `prop:"pebble.raw.point-tombstone.key.size"` + // Sum of the raw value sizes carried by point deletion tombstones + // containing size estimates. See the DeleteSized key kind. This value is + // comparable to Raw{Key,Value}Size. + RawPointTombstoneValueSize uint64 `prop:"pebble.raw.point-tombstone.value.size"` // Total raw rangekey key size. RawRangeKeyKeySize uint64 `prop:"pebble.raw.range-key.key.size"` // Total raw rangekey value size. @@ -348,15 +355,21 @@ func (p *Properties) save(tblFormat TableFormat, w *rawBlockWriter) { p.saveUvarint(m, unsafe.Offsetof(p.NumDataBlocks), p.NumDataBlocks) p.saveUvarint(m, unsafe.Offsetof(p.NumEntries), p.NumEntries) p.saveUvarint(m, unsafe.Offsetof(p.NumDeletions), p.NumDeletions) + if p.NumSizedDeletions > 0 { + p.saveUvarint(m, unsafe.Offsetof(p.NumSizedDeletions), p.NumSizedDeletions) + } p.saveUvarint(m, unsafe.Offsetof(p.NumMergeOperands), p.NumMergeOperands) p.saveUvarint(m, unsafe.Offsetof(p.NumRangeDeletions), p.NumRangeDeletions) - // NB: We only write out the RawTombstoneKeySize for Pebble formats. This - // isn't strictly necessary because unrecognized properties are interpreted - // as user-defined properties, however writing them prevents byte-for-byte + // NB: We only write out some properties for Pebble formats. This isn't + // strictly necessary because unrecognized properties are interpreted as + // user-defined properties, however writing them prevents byte-for-byte // equivalence with RocksDB files that some of our testing requires. if p.RawPointTombstoneKeySize > 0 && tblFormat >= TableFormatPebblev1 { p.saveUvarint(m, unsafe.Offsetof(p.RawPointTombstoneKeySize), p.RawPointTombstoneKeySize) } + if p.RawPointTombstoneValueSize > 0 { + p.saveUvarint(m, unsafe.Offsetof(p.RawPointTombstoneValueSize), p.RawPointTombstoneValueSize) + } if p.NumRangeKeys() > 0 { p.saveUvarint(m, unsafe.Offsetof(p.NumRangeKeyDels), p.NumRangeKeyDels) p.saveUvarint(m, unsafe.Offsetof(p.NumRangeKeySets), p.NumRangeKeySets) diff --git a/sstable/testdata/writer_value_blocks b/sstable/testdata/writer_value_blocks index 745cc97ad8..869b849afb 100644 --- a/sstable/testdata/writer_value_blocks +++ b/sstable/testdata/writer_value_blocks @@ -327,7 +327,7 @@ layout 775 rocksdb.raw.key.size (16) 791 rocksdb.raw.value.size (14) 805 [restart 98] - 813 [trailer compression=none checksum=0x89a91fa9] + 813 [trailer compression=none checksum=0xfb9d6722] 818 meta-index (32) 818 rocksdb.properties block:98/715 [restart] 842 [restart 818] diff --git a/sstable/writer.go b/sstable/writer.go index d8186881c1..83ebad4ca0 100644 --- a/sstable/writer.go +++ b/sstable/writer.go @@ -932,7 +932,22 @@ func (w *Writer) addPoint(key InternalKey, value []byte) error { switch key.Kind() { case InternalKeyKindDelete, InternalKeyKindSingleDelete: w.props.NumDeletions++ - w.props.RawPointTombstoneKeySize += uint64(key.Size()) + w.props.RawPointTombstoneKeySize += uint64(len(key.UserKey)) + case InternalKeyKindDeleteSized: + var size uint64 + if len(value) > 0 { + var n int + size, n = binary.Uvarint(value) + if n <= 0 { + w.err = errors.Newf("%s key's value (%x) does not parse as uvarint", + errors.Safe(key.Kind().String()), value) + return w.err + } + } + w.props.NumDeletions++ + w.props.NumSizedDeletions++ + w.props.RawPointTombstoneKeySize += uint64(len(key.UserKey)) + w.props.RawPointTombstoneValueSize += size case InternalKeyKindMerge: w.props.NumMergeOperands++ } @@ -1729,12 +1744,20 @@ func (w *Writer) assertFormatCompatibility() error { ) } + // PebbleDBv3: value blocks. if (w.props.NumValueBlocks > 0 || w.props.NumValuesInValueBlocks > 0 || w.props.ValueBlocksSize > 0) && w.tableFormat < TableFormatPebblev3 { return errors.Newf( "table format version %s is less than the minimum required version %s for value blocks", w.tableFormat, TableFormatPebblev3) } + + // PebbleDBv4: DELSIZED tombstones. + if w.props.NumSizedDeletions > 0 && w.tableFormat < TableFormatPebblev4 { + return errors.Newf( + "table format version %s is less than the minimum required version %s for sized deletion tombstones", + w.tableFormat, TableFormatPebblev4) + } return nil } diff --git a/table_cache_test.go b/table_cache_test.go index 5c34a96893..32f6413656 100644 --- a/table_cache_test.go +++ b/table_cache_test.go @@ -252,7 +252,7 @@ func TestVirtualReadsWiring(t *testing.T) { d, err = Open("", &Options{ FS: vfs.NewMem(), - FormatMajorVersion: FormatNewest, + FormatMajorVersion: internalFormatNewest, Comparer: testkeys.Comparer, // Compactions which conflict with virtual sstable creation can be // picked by Pebble. We disable that. @@ -1056,7 +1056,7 @@ func TestTableCacheClockPro(t *testing.T) { func BenchmarkNewItersAlloc(b *testing.B) { opts := &Options{ FS: vfs.NewMem(), - FormatMajorVersion: FormatNewest, + FormatMajorVersion: internalFormatNewest, } d, err := Open("", opts) require.NoError(b, err) diff --git a/table_stats.go b/table_stats.go index dadcabfad3..25f28e3510 100644 --- a/table_stats.go +++ b/table_stats.go @@ -304,12 +304,12 @@ func (d *DB) loadTablePointKeyStats( // We could write the ranges of 'clusters' of point tombstones to // a sstable property and call averageValueSizeBeneath for each of // these narrower ranges to improve the estimate. - avgValSize, err := d.averagePhysicalValueSizeBeneath(v, level, meta) + avgValLogicalSize, compressionRatio, err := d.estimateSizesBeneath(v, level, meta) if err != nil { return err } stats.PointDeletionsBytesEstimate = - pointDeletionsBytesEstimate(meta.Size, &r.Properties, avgValSize) + pointDeletionsBytesEstimate(meta.Size, &r.Properties, avgValLogicalSize, compressionRatio) return nil } @@ -420,51 +420,60 @@ func (d *DB) loadTableRangeDelStats( return compactionHints, err } -func (d *DB) averagePhysicalValueSizeBeneath( +func (d *DB) estimateSizesBeneath( v *version, level int, meta physicalMeta, -) (avgValueSize uint64, err error) { +) (avgValueLogicalSize, compressionRatio float64, err error) { // Find all files in lower levels that overlap with meta, // summing their value sizes and entry counts. + file := meta.FileMetadata var fileSum, keySum, valSum, entryCount uint64 + + addPhysicalTableStats := func(r *sstable.Reader) (err error) { + fileSum += file.Size + entryCount += r.Properties.NumEntries + keySum += r.Properties.RawKeySize + valSum += r.Properties.RawValueSize + return nil + } + addVirtualTableStats := func(v sstable.VirtualReader) (err error) { + fileSum += file.Size + entryCount += file.Stats.NumEntries + keySum += v.Properties.RawKeySize + valSum += v.Properties.RawValueSize + return nil + } + + // Include the file itself. This is important because in some instances, the + // computed compression ratio is applied to the tombstones contained within + // `meta` itself. If there are no files beneath `meta` in the LSM, we would + // calculate a compression ratio of 0 which is not accurate for the file's + // own tombstones. + if err = d.tableCache.withReader(meta, addPhysicalTableStats); err != nil { + return 0, 0, err + } + for l := level + 1; l < numLevels; l++ { overlaps := v.Overlaps(l, d.cmp, meta.Smallest.UserKey, meta.Largest.UserKey, meta.Largest.IsExclusiveSentinel()) iter := overlaps.Iter() - for file := iter.First(); file != nil; file = iter.Next() { + for file = iter.First(); file != nil; file = iter.Next() { var err error if file.Virtual { - err = d.tableCache.withVirtualReader( - file.VirtualMeta(), - func(v sstable.VirtualReader) (err error) { - fileSum += file.Size - entryCount += file.Stats.NumEntries - keySum += v.Properties.RawKeySize - valSum += v.Properties.RawValueSize - return nil - }) + err = d.tableCache.withVirtualReader(file.VirtualMeta(), addVirtualTableStats) } else { - err = d.tableCache.withReader( - file.PhysicalMeta(), - func(r *sstable.Reader) (err error) { - fileSum += file.Size - entryCount += r.Properties.NumEntries - keySum += r.Properties.RawKeySize - valSum += r.Properties.RawValueSize - return nil - }) + err = d.tableCache.withReader(file.PhysicalMeta(), addPhysicalTableStats) } - if err != nil { - return 0, err + return 0, 0, err } } } if entryCount == 0 { - return 0, nil + return 0, 0, nil } - // RawKeySize and RawValueSize are uncompressed totals. Scale the value sum - // according to the data size to account for compression, index blocks and - // metadata overhead. Eg: + // RawKeySize and RawValueSize are uncompressed totals. We'll need to scale + // the value sum according to the data size to account for compression, + // index blocks and metadata overhead. Eg: // // Compression rate × Average uncompressed value size // @@ -473,10 +482,14 @@ func (d *DB) averagePhysicalValueSizeBeneath( // FileSize RawValueSize // ----------------------- × ------------ // RawKeySize+RawValueSize NumEntries + // + // We return the average logical value size plus the compression ratio, + // leaving the scaling to the caller. This allows the caller to perform + // additional compression ratio scaling if necessary. uncompressedSum := float64(keySum + valSum) - compressionRatio := float64(fileSum) / uncompressedSum - avgCompressedValueSize := (float64(valSum) / float64(entryCount)) * compressionRatio - return uint64(avgCompressedValueSize), nil + compressionRatio = float64(fileSum) / uncompressedSum + avgValueLogicalSize = (float64(valSum) / float64(entryCount)) + return avgValueLogicalSize, compressionRatio, nil } func (d *DB) estimateReclaimedSizeBeneath( @@ -593,12 +606,12 @@ func maybeSetStatsFromProperties(meta physicalMeta, props *sstable.Properties) b return false } - // If a table is more than 10% point deletions, don't calculate the - // PointDeletionsBytesEstimate statistic using our limited knowledge. The - // table stats collector can populate the stats and calculate an average - // of value size of all the tables beneath the table in the LSM, which - // will be more accurate. - if props.NumDeletions > props.NumEntries/10 { + // If a table is more than 10% point deletions without user-provided size + // estimates, don't calculate the PointDeletionsBytesEstimate statistic + // using our limited knowledge. The table stats collector can populate the + // stats and calculate an average of value size of all the tables beneath + // the table in the LSM, which will be more accurate. + if unsizedDels := (props.NumDeletions - props.NumSizedDeletions); unsizedDels > props.NumEntries/10 { return false } @@ -608,8 +621,8 @@ func maybeSetStatsFromProperties(meta physicalMeta, props *sstable.Properties) b // doesn't require any additional IO and since the number of point // deletions in the file is low, the error introduced by this crude // estimate is expected to be small. - avgValSize := estimateValuePhysicalSize(meta.Size, props) - pointEstimate = pointDeletionsBytesEstimate(meta.Size, props, avgValSize) + avgValSize, compressionRatio := estimatePhysicalSizes(meta.Size, props) + pointEstimate = pointDeletionsBytesEstimate(meta.Size, props, avgValSize, compressionRatio) } meta.Stats.NumEntries = props.NumEntries @@ -623,7 +636,7 @@ func maybeSetStatsFromProperties(meta physicalMeta, props *sstable.Properties) b } func pointDeletionsBytesEstimate( - fileSize uint64, props *sstable.Properties, avgValPhysicalSize uint64, + fileSize uint64, props *sstable.Properties, avgValLogicalSize, compressionRatio float64, ) (estimate uint64) { if props.NumEntries == 0 { return 0 @@ -639,43 +652,79 @@ func pointDeletionsBytesEstimate( // If there are covered key(s), we also get to drop key and value bytes for // each covered key. // - // We estimate assuming that each point tombstone on average covers 1 key. + // Some point tombstones (DELSIZEDs) carry a user-provided estimate of the + // uncompressed size of entries that will be elided by fully compacting the + // tombstone. For these tombstones, there's no guesswork—we use the + // RawPointTombstoneValueSizeHint property which is the sum of all these + // tombstones' encoded values. + // + // For un-sized point tombstones (DELs), we estimate assuming that each + // point tombstone on average covers 1 key and using average vaue sizes. // This is almost certainly an overestimate, but that's probably okay // because point tombstones can slow range iterations even when they don't - // cover a key. It may be beneficial in the future to more accurately - // estimate which tombstones cover keys and which do not. - - // Calculate the contribution of the key sizes: 1x for the point tombstone - // itself and 1x for the key that's expected to exist lower in the LSM. - var keysLogicalSize uint64 + // cover a key. + // + // TODO(jackson): This logic doesn't directly incorporate fixed per-key + // overhead (8-byte trailer, plus at least 1 byte encoding the length of the + // key and 1 byte encoding the length of the value). This overhead is + // indirectly incorporated through the compression ratios, but that results + // in the overhead being smeared per key-byte and value-byte, rather than + // per-entry. This per-key fixed overhead can be nontrivial, especially for + // dense swaths of point tombstones. Give some thought as to whether we + // should directly include fixed per-key overhead in the calculations. + + // Below, we calculate the tombstone contributions and the shadowed keys' + // contributions separately. + var tombstonesLogicalSize float64 + var shadowedLogicalSize float64 + + // 1. Calculate the contribution of the tombstone keys themselves. if props.RawPointTombstoneKeySize > 0 { - // This table has a RawPointTombstoneKeySize property, so we know the - // exact size of the logical, uncompressed bytes of the point tombstone - // keys. - keysLogicalSize = 2 * props.RawPointTombstoneKeySize + tombstonesLogicalSize += float64(props.RawPointTombstoneKeySize) } else { // This sstable predates the existence of the RawPointTombstoneKeySize // property. We can use the average key size within the file itself and // the count of point deletions to estimate the size. - keysLogicalSize = 2 * numPointDels * props.RawKeySize / props.NumEntries + tombstonesLogicalSize += float64(numPointDels * props.RawKeySize / props.NumEntries) } - // Scale the logical key size by the logical:physical ratio of the file to + + // 2. Calculate the contribution of the keys shadowed by tombstones. + // + // 2a. First account for keys shadowed by DELSIZED tombstones. THE DELSIZED + // tombstones encode the size of both the key and value of the shadowed KV + // entries. These sizes are aggregated into a sstable property. + shadowedLogicalSize += float64(props.RawPointTombstoneValueSize) + + // 2b. Calculate the contribution of the KV entries shadowed by ordinary DEL + // keys. + numUnsizedDels := numPointDels - props.NumSizedDeletions + { + // The shadowed keys have the same exact user keys as the tombstones + // themselves, so we can use the `tombstonesLogicalSize` we computed + // earlier as an estimate. There's a complication that + // `tombstonesLogicalSize` may include DELSIZED keys we already + // accounted for. + shadowedLogicalSize += float64(tombstonesLogicalSize) / float64(numPointDels) * float64(numUnsizedDels) + + // Calculate the contribution of the deleted values. The caller has + // already computed an average logical size (possibly computed across + // many sstables). + shadowedLogicalSize += float64(numUnsizedDels) * avgValLogicalSize + } + + // Scale both tombstone and shadowed totals by logical:physical ratios to // account for compression, metadata overhead, etc. // // Physical FileSize // ----------- = ----------------------- // Logical RawKeySize+RawValueSize // - estimate += (keysLogicalSize * fileSize) / (props.RawKeySize + props.RawValueSize) - - // Calculate the contribution of the deleted values. The caller has already - // computed an average physical size (possibly comptued across many - // sstables), so there's no need to scale it. - estimate += numPointDels * avgValPhysicalSize - return estimate + return uint64((tombstonesLogicalSize + shadowedLogicalSize) * compressionRatio) } -func estimateValuePhysicalSize(fileSize uint64, props *sstable.Properties) (avgValSize uint64) { +func estimatePhysicalSizes( + fileSize uint64, props *sstable.Properties, +) (avgValLogicalSize, compressionRatio float64) { // RawKeySize and RawValueSize are uncompressed totals. Scale according to // the data size to account for compression, index blocks and metadata // overhead. Eg: @@ -689,9 +738,9 @@ func estimateValuePhysicalSize(fileSize uint64, props *sstable.Properties) (avgV // RawKeySize+RawValueSize NumEntries // uncompressedSum := props.RawKeySize + props.RawValueSize - compressionRatio := float64(fileSize) / float64(uncompressedSum) - avgCompressedValueSize := (float64(props.RawValueSize) / float64(props.NumEntries)) * compressionRatio - return uint64(avgCompressedValueSize) + compressionRatio = float64(fileSize) / float64(uncompressedSum) + avgValLogicalSize = (float64(props.RawValueSize) / float64(props.NumEntries)) + return avgValLogicalSize, compressionRatio } // newCombinedDeletionKeyspanIter returns a keyspan.FragmentIterator that diff --git a/table_stats_test.go b/table_stats_test.go index 550eefa03d..6f9ca8f59d 100644 --- a/table_stats_test.go +++ b/table_stats_test.go @@ -137,6 +137,9 @@ func TestTableStats(t *testing.T) { d.mu.Unlock() return s + case "properties": + return runSSTablePropertiesCmd(t, td, d) + default: return fmt.Sprintf("unknown command: %s", td.Cmd) } diff --git a/testdata/checkpoint b/testdata/checkpoint index 131e6ebacc..986d23b111 100644 --- a/testdata/checkpoint +++ b/testdata/checkpoint @@ -77,6 +77,10 @@ create: db/marker.format-version.000013.014 close: db/marker.format-version.000013.014 remove: db/marker.format-version.000012.013 sync: db +create: db/marker.format-version.000014.015 +close: db/marker.format-version.000014.015 +remove: db/marker.format-version.000013.014 +sync: db create: db/temporary.000003.dbtmp sync: db/temporary.000003.dbtmp close: db/temporary.000003.dbtmp @@ -139,9 +143,9 @@ close: open-dir: checkpoints/checkpoint1 link: db/OPTIONS-000003 -> checkpoints/checkpoint1/OPTIONS-000003 open-dir: checkpoints/checkpoint1 -create: checkpoints/checkpoint1/marker.format-version.000001.014 -sync-data: checkpoints/checkpoint1/marker.format-version.000001.014 -close: checkpoints/checkpoint1/marker.format-version.000001.014 +create: checkpoints/checkpoint1/marker.format-version.000001.015 +sync-data: checkpoints/checkpoint1/marker.format-version.000001.015 +close: checkpoints/checkpoint1/marker.format-version.000001.015 sync: checkpoints/checkpoint1 close: checkpoints/checkpoint1 link: db/000005.sst -> checkpoints/checkpoint1/000005.sst @@ -179,9 +183,9 @@ close: checkpoints open-dir: checkpoints/checkpoint2 link: db/OPTIONS-000003 -> checkpoints/checkpoint2/OPTIONS-000003 open-dir: checkpoints/checkpoint2 -create: checkpoints/checkpoint2/marker.format-version.000001.014 -sync-data: checkpoints/checkpoint2/marker.format-version.000001.014 -close: checkpoints/checkpoint2/marker.format-version.000001.014 +create: checkpoints/checkpoint2/marker.format-version.000001.015 +sync-data: checkpoints/checkpoint2/marker.format-version.000001.015 +close: checkpoints/checkpoint2/marker.format-version.000001.015 sync: checkpoints/checkpoint2 close: checkpoints/checkpoint2 link: db/000007.sst -> checkpoints/checkpoint2/000007.sst @@ -214,9 +218,9 @@ close: checkpoints open-dir: checkpoints/checkpoint3 link: db/OPTIONS-000003 -> checkpoints/checkpoint3/OPTIONS-000003 open-dir: checkpoints/checkpoint3 -create: checkpoints/checkpoint3/marker.format-version.000001.014 -sync-data: checkpoints/checkpoint3/marker.format-version.000001.014 -close: checkpoints/checkpoint3/marker.format-version.000001.014 +create: checkpoints/checkpoint3/marker.format-version.000001.015 +sync-data: checkpoints/checkpoint3/marker.format-version.000001.015 +close: checkpoints/checkpoint3/marker.format-version.000001.015 sync: checkpoints/checkpoint3 close: checkpoints/checkpoint3 link: db/000005.sst -> checkpoints/checkpoint3/000005.sst @@ -252,34 +256,34 @@ close: db/000009.sst sync: db sync: db/MANIFEST-000001 open: db/000005.sst -read-at(744, 53): db/000005.sst -read-at(707, 37): db/000005.sst -read-at(79, 628): db/000005.sst -read-at(52, 27): db/000005.sst +read-at(739, 53): db/000005.sst +read-at(702, 37): db/000005.sst +read-at(74, 628): db/000005.sst +read-at(47, 27): db/000005.sst open: db/000005.sst close: db/000005.sst open: db/000009.sst -read-at(732, 53): db/000009.sst -read-at(695, 37): db/000009.sst -read-at(67, 628): db/000009.sst -read-at(40, 27): db/000009.sst +read-at(734, 53): db/000009.sst +read-at(697, 37): db/000009.sst +read-at(69, 628): db/000009.sst +read-at(42, 27): db/000009.sst open: db/000009.sst close: db/000009.sst open: db/000007.sst -read-at(744, 53): db/000007.sst -read-at(707, 37): db/000007.sst -read-at(79, 628): db/000007.sst -read-at(52, 27): db/000007.sst +read-at(739, 53): db/000007.sst +read-at(702, 37): db/000007.sst +read-at(74, 628): db/000007.sst +read-at(47, 27): db/000007.sst open: db/000007.sst close: db/000007.sst open: db/000005.sst -read-at(0, 52): db/000005.sst +read-at(0, 47): db/000005.sst open: db/000007.sst -read-at(0, 52): db/000007.sst +read-at(0, 47): db/000007.sst create: db/000010.sst close: db/000005.sst open: db/000009.sst -read-at(0, 40): db/000009.sst +read-at(0, 42): db/000009.sst close: db/000007.sst close: db/000009.sst sync-data: db/000010.sst @@ -307,7 +311,7 @@ CURRENT LOCK MANIFEST-000001 OPTIONS-000003 -marker.format-version.000013.014 +marker.format-version.000014.015 marker.manifest.000001.MANIFEST-000001 list checkpoints/checkpoint1 @@ -317,7 +321,7 @@ list checkpoints/checkpoint1 000007.sst MANIFEST-000001 OPTIONS-000003 -marker.format-version.000001.014 +marker.format-version.000001.015 marker.manifest.000001.MANIFEST-000001 open checkpoints/checkpoint1 readonly @@ -337,17 +341,17 @@ close: checkpoints/checkpoint1/000006.log scan checkpoints/checkpoint1 ---- open: checkpoints/checkpoint1/000007.sst -read-at(744, 53): checkpoints/checkpoint1/000007.sst -read-at(707, 37): checkpoints/checkpoint1/000007.sst -read-at(79, 628): checkpoints/checkpoint1/000007.sst -read-at(52, 27): checkpoints/checkpoint1/000007.sst -read-at(0, 52): checkpoints/checkpoint1/000007.sst +read-at(739, 53): checkpoints/checkpoint1/000007.sst +read-at(702, 37): checkpoints/checkpoint1/000007.sst +read-at(74, 628): checkpoints/checkpoint1/000007.sst +read-at(47, 27): checkpoints/checkpoint1/000007.sst +read-at(0, 47): checkpoints/checkpoint1/000007.sst open: checkpoints/checkpoint1/000005.sst -read-at(744, 53): checkpoints/checkpoint1/000005.sst -read-at(707, 37): checkpoints/checkpoint1/000005.sst -read-at(79, 628): checkpoints/checkpoint1/000005.sst -read-at(52, 27): checkpoints/checkpoint1/000005.sst -read-at(0, 52): checkpoints/checkpoint1/000005.sst +read-at(739, 53): checkpoints/checkpoint1/000005.sst +read-at(702, 37): checkpoints/checkpoint1/000005.sst +read-at(74, 628): checkpoints/checkpoint1/000005.sst +read-at(47, 27): checkpoints/checkpoint1/000005.sst +read-at(0, 47): checkpoints/checkpoint1/000005.sst a 1 b 5 c 3 @@ -382,7 +386,7 @@ list checkpoints/checkpoint2 000007.sst MANIFEST-000001 OPTIONS-000003 -marker.format-version.000001.014 +marker.format-version.000001.015 marker.manifest.000001.MANIFEST-000001 open checkpoints/checkpoint2 readonly @@ -402,11 +406,11 @@ close: checkpoints/checkpoint2/000006.log scan checkpoints/checkpoint2 ---- open: checkpoints/checkpoint2/000007.sst -read-at(744, 53): checkpoints/checkpoint2/000007.sst -read-at(707, 37): checkpoints/checkpoint2/000007.sst -read-at(79, 628): checkpoints/checkpoint2/000007.sst -read-at(52, 27): checkpoints/checkpoint2/000007.sst -read-at(0, 52): checkpoints/checkpoint2/000007.sst +read-at(739, 53): checkpoints/checkpoint2/000007.sst +read-at(702, 37): checkpoints/checkpoint2/000007.sst +read-at(74, 628): checkpoints/checkpoint2/000007.sst +read-at(47, 27): checkpoints/checkpoint2/000007.sst +read-at(0, 47): checkpoints/checkpoint2/000007.sst b 5 d 7 e 8 @@ -422,7 +426,7 @@ list checkpoints/checkpoint3 000007.sst MANIFEST-000001 OPTIONS-000003 -marker.format-version.000001.014 +marker.format-version.000001.015 marker.manifest.000001.MANIFEST-000001 open checkpoints/checkpoint3 readonly @@ -442,17 +446,17 @@ close: checkpoints/checkpoint3/000006.log scan checkpoints/checkpoint3 ---- open: checkpoints/checkpoint3/000007.sst -read-at(744, 53): checkpoints/checkpoint3/000007.sst -read-at(707, 37): checkpoints/checkpoint3/000007.sst -read-at(79, 628): checkpoints/checkpoint3/000007.sst -read-at(52, 27): checkpoints/checkpoint3/000007.sst -read-at(0, 52): checkpoints/checkpoint3/000007.sst +read-at(739, 53): checkpoints/checkpoint3/000007.sst +read-at(702, 37): checkpoints/checkpoint3/000007.sst +read-at(74, 628): checkpoints/checkpoint3/000007.sst +read-at(47, 27): checkpoints/checkpoint3/000007.sst +read-at(0, 47): checkpoints/checkpoint3/000007.sst open: checkpoints/checkpoint3/000005.sst -read-at(744, 53): checkpoints/checkpoint3/000005.sst -read-at(707, 37): checkpoints/checkpoint3/000005.sst -read-at(79, 628): checkpoints/checkpoint3/000005.sst -read-at(52, 27): checkpoints/checkpoint3/000005.sst -read-at(0, 52): checkpoints/checkpoint3/000005.sst +read-at(739, 53): checkpoints/checkpoint3/000005.sst +read-at(702, 37): checkpoints/checkpoint3/000005.sst +read-at(74, 628): checkpoints/checkpoint3/000005.sst +read-at(47, 27): checkpoints/checkpoint3/000005.sst +read-at(0, 47): checkpoints/checkpoint3/000005.sst a 1 b 5 c 3 diff --git a/testdata/compaction_delete_only_hints b/testdata/compaction_delete_only_hints index 05e07241e0..6bee9d15e6 100644 --- a/testdata/compaction_delete_only_hints +++ b/testdata/compaction_delete_only_hints @@ -88,7 +88,7 @@ maybe-compact Deletion hints: (none) Compactions: - [JOB 100] compacted(delete-only) L2 [000005] (784 B) + L3 [000006] (784 B) -> L6 [] (0 B), in 1.0s (2.0s total), output rate 0 B/s + [JOB 100] compacted(delete-only) L2 [000005] (786 B) + L3 [000006] (786 B) -> L6 [] (0 B), in 1.0s (2.0s total), output rate 0 B/s # Verify that compaction correctly handles the presence of multiple # overlapping hints which might delete a file multiple times. All of the @@ -127,7 +127,7 @@ maybe-compact Deletion hints: (none) Compactions: - [JOB 100] compacted(delete-only) L2 [000006] (784 B) + L3 [000007] (784 B) -> L6 [] (0 B), in 1.0s (2.0s total), output rate 0 B/s + [JOB 100] compacted(delete-only) L2 [000006] (786 B) + L3 [000007] (786 B) -> L6 [] (0 B), in 1.0s (2.0s total), output rate 0 B/s # Test a range tombstone that is already compacted into L6. @@ -206,7 +206,7 @@ maybe-compact Deletion hints: (none) Compactions: - [JOB 100] compacted(delete-only) L2 [000005] (784 B) + L3 [000006] (784 B) -> L6 [] (0 B), in 1.0s (2.0s total), output rate 0 B/s + [JOB 100] compacted(delete-only) L2 [000005] (786 B) + L3 [000006] (786 B) -> L6 [] (0 B), in 1.0s (2.0s total), output rate 0 B/s # A deletion hint present on an sstable in a higher level should NOT result in a # deletion-only compaction incorrectly removing an sstable in L6 following an diff --git a/testdata/compaction_iter_delete_sized b/testdata/compaction_iter_delete_sized new file mode 100644 index 0000000000..c1663fcbfe --- /dev/null +++ b/testdata/compaction_iter_delete_sized @@ -0,0 +1,1477 @@ +define +a.SET.1:b +---- + +iter +first +next +---- +a#1,1:b +. + +define +a.SET.2:c +a.SET.1:b +---- + +iter +first +next +---- +a#2,1:c +. + +iter snapshots=0 +first +next +---- +a#2,1:c +. + +iter snapshots=1 +first +next +---- +a#2,1:c +. + +iter snapshots=2 +first +next +next +---- +a#2,1:c +a#1,1:b +. + +define +a.DEL.2: +a.SET.1:b +---- + +iter +first +next +---- +a#2,0: +. + +iter elide-tombstones=true +first +---- +. + +iter elide-tombstones=true snapshots=2 +first +next +next +---- +a#2,0: +a#1,1:b +. + +iter elide-tombstones=true snapshots=1 +first +next +---- +a#2,0: +. + +define +a.DEL.2: +a.SET.1:b +b.SET.3:c +---- + +iter +first +next +next +---- +a#2,0: +b#3,1:c +. + +iter snapshots=1 +first +next +next +---- +a#2,0: +b#3,1:c +. + +iter snapshots=2 +first +next +next +next +---- +a#2,0: +a#1,1:b +b#3,1:c +. + +define +a.SET.1:a +b.SET.2:b +c.SET.3:c +---- + +iter +first +next +next +next +---- +a#1,1:a +b#2,1:b +c#3,1:c +. + +define +a.MERGE.3:d +a.MERGE.2:c +a.SET.1:b +b.MERGE.2:b +b.MERGE.1:a +---- + +iter +first +next +next +---- +a#3,1:bcd[base] +b#2,2:ab +. + +iter snapshots=3 +first +next +next +next +---- +a#3,2:d +a#2,1:bc[base] +b#2,2:ab +. + +define +a.SET.9:b +a.DEL.8: +a.SET.7:d +a.DEL.6: +a.SET.5:f +---- + +iter +first +next +---- +a#9,18:b +. + +iter snapshots=6 +first +next +next +---- +a#9,18:b +a#5,1:f +. + +iter snapshots=7 +first +next +next +---- +a#9,18:b +a#6,0: +. + +iter snapshots=8 +first +next +next +---- +a#9,18:b +a#7,18:d +. + +iter snapshots=9 +first +next +next +---- +a#9,1:b +a#8,0: +. + +iter snapshots=10 +first +next +---- +a#9,18:b +. + +iter snapshots=(5,6,7,8,9) +first +next +next +next +next +next +---- +a#9,1:b +a#8,0: +a#7,1:d +a#6,0: +a#5,1:f +. + +define +a.INVALID.2:b +a.SET.1:c +---- + +iter +first +---- +err=invalid internal key kind: 255 + +define +a.SET.2:b +a.INVALID.1:c +---- + +iter +first +next +---- +a#2,18:b +err=invalid internal key kind: 255 + +define +a.MERGE.2:b +a.INVALID.1:c +---- + +iter +first +next +---- +a#2,2:b +err=invalid internal key kind: 255 + +define +a.INVALID.2:c +a.RANGEDEL.1:d +---- + +iter +first +tombstones +---- +err=invalid internal key kind: 255 +. + +define +a.MERGE.2:b +a.MERGE.1:c +a.MERGE.0:d +---- + +iter snapshots=(1,2) +first +next +next +next +---- +a#2,2:b +a#1,2:c +a#0,2:d +. + +define +a.SET.2:b +a.RANGEDEL.1:c +b.RANGEDEL.4:d +b.SET.2:e +c.SET.3:f +---- + +iter +first +next +next +next +tombstones +---- +a#2,18:b +a#1,15:c +b#4,15:d +. +a-b#1 +b-c#4 +c-d#4 +. + +iter snapshots=2 +first +next +next +next +tombstones +---- +a#2,1:b +a#1,15:c +b#4,15:d +. +a-b#1 +b-c#4 +b-c#1 +c-d#4 +. + +iter snapshots=3 +first +next +next +next +next +tombstones +---- +a#2,18:b +a#1,15:c +b#4,15:d +b#2,1:e +. +a-b#1 +b-c#4 +b-c#1 +c-d#4 +. + +iter snapshots=4 +first +next +next +next +next +next +tombstones +---- +a#2,18:b +a#1,15:c +b#4,15:d +b#2,1:e +c#3,1:f +. +a-b#1 +b-c#4 +b-c#1 +c-d#4 +. + +define +a.RANGEDEL.3:e +b.SET.4:b +c.SET.3:c +d.SET.2:d +e.SET.1:e +---- + +iter +first +next +next +next +next +tombstones +---- +a#3,15:e +b#4,1:b +c#3,1:c +e#1,1:e +. +a-e#3 +. + +define +a.RANGEDEL.3:e +b.MERGE.4:b +c.MERGE.3:c +d.MERGE.2:d +e.MERGE.1:e +---- + +iter +first +next +next +next +next +tombstones +---- +a#3,15:e +b#4,2:b +c#3,2:c +e#1,2:e +. +a-e#3 +. + +define +a.RANGEDEL.3:c +b.MERGE.5:e +b.MERGE.4:d +b.MERGE.2:c +b.MERGE.1:b +d.MERGE.5:c +d.MERGE.4:b +d.RANGEDEL.3:f +d.MERGE.2:e +d.MERGE.1:d +---- + +iter +first +next +next +next +next +tombstones +---- +a#3,15:c +b#5,1:de[base] +d#5,2:bc +d#3,15:f +. +a-c#3 +d-f#3 +. + +define +a.RANGEDEL.3:d +b.RANGEDEL.2:e +c.RANGEDEL.1:f +---- + +iter +first +next +next +next +tombstones +---- +a#3,15:d +b#2,15:e +c#1,15:f +. +a-b#3 +b-c#3 +c-d#3 +d-e#2 +e-f#1 +. + +iter snapshots=2 +first +next +next +next +tombstones +---- +a#3,15:d +b#2,15:e +c#1,15:f +. +a-b#3 +b-c#3 +c-d#3 +c-d#1 +d-e#2 +d-e#1 +e-f#1 +. + +iter snapshots=3 +first +next +next +next +tombstones +---- +a#3,15:d +b#2,15:e +c#1,15:f +. +a-b#3 +b-c#3 +b-c#2 +c-d#3 +c-d#2 +d-e#2 +e-f#1 +. + +iter snapshots=(2,3) +first +next +next +next +tombstones +---- +a#3,15:d +b#2,15:e +c#1,15:f +. +a-b#3 +b-c#3 +b-c#2 +c-d#3 +c-d#2 +c-d#1 +d-e#2 +d-e#1 +e-f#1 +. + +define +a.RANGEDEL.10:k +f.SET.9:f +f.SET.8:f +---- + +iter snapshots=(9,10) +first +next +tombstones f +next +tombstones +---- +a#10,15:k +f#9,1:f +a-f#10 +. +f#8,1:f +f-k#10 +. + +define +f.RANGEDEL.10:k +f.SET.9:f +f.SET.8:f +---- + +iter snapshots=(9,10) +first +next +tombstones f +next +tombstones +---- +f#10,15:k +f#9,1:f +. +f#8,1:f +f-k#10 +. + +define +a.SET.1:a +b.RANGEDEL.2:d +c.RANGEDEL.3:e +d.SET.4:d +---- + +iter +first +next +next +next +tombstones c +tombstones +---- +a#1,1:a +b#2,15:d +c#3,15:e +d#4,1:d +b-c#2 +. +c-d#3 +d-e#3 +. + +iter snapshots=3 +first +next +next +next +tombstones c +tombstones +---- +a#1,1:a +b#2,15:d +c#3,15:e +d#4,1:d +b-c#2 +. +c-d#3 +c-d#2 +d-e#3 +. + +define +a.SET.1:a +b.RANGEDEL.2:d +c.SET.4:d +---- + +iter +first +next +next +tombstones c +tombstones +---- +a#1,1:a +b#2,15:d +c#4,1:d +b-c#2 +. +c-d#2 +. + +define +a.RANGEDEL.2:d +a.SET.2:a +b.SET.2:b +c.SET.2:c +---- + +iter +first +next +next +next +next +---- +a#2,15:d +a#2,1:a +b#2,1:b +c#2,1:c +. + +define +a.SINGLEDEL.1: +---- + +iter +first +next +---- +a#1,7: +. + +iter elide-tombstones=true +first +---- +. + +define +a.SINGLEDEL.2: +a.SINGLEDEL.1: +---- + +iter +first +next +---- +a#2,7: +. + +define +a.SINGLEDEL.3: +a.SINGLEDEL.2: +a.SET.1:a +---- + +iter +first +---- +. + +define +a.SET.3:a +b.SINGLEDEL.2: +b.DEL.1: +---- + +iter +first +next +next +---- +a#3,1:a +b#2,0: +. + +define +a.SINGLEDEL.2: +a.DEL.1: +---- + +iter +first +next +---- +a#2,0: +. + +iter elide-tombstones=true +first +---- +. + +define +a.SINGLEDEL.2: +a.MERGE.1: +---- + +iter +first +next +---- +a#2,0: +. + +iter elide-tombstones=true +first +---- +. + +define +a.SINGLEDEL.2: +a.SET.1:b +---- + +iter +first +---- +. + +# SET that meets a SINGLEDEL is transformed into a SETWITHDEL. + +define +a.SET.2:b +a.SINGLEDEL.1: +---- + +iter +first +next +---- +a#2,18:b +. + +define +a.MERGE.6:b +a.SINGLEDEL.5: +a.SET.4:a +---- + +iter +first +next +---- +a#6,1:b[base] +. + +# Non-deterministic use of SINGLEDEL where there are two older SETs that have +# not been deleted or single deleted. It is permitted to shadow both. +define +a.MERGE.6:b +a.SINGLEDEL.5: +a.SET.4:a +a.SET.3:a +---- + +iter +first +next +---- +a#6,1:b[base] +. + +define +a.SINGLEDEL.2: +a.SET.1:b +b.SET.3:c +---- + +iter +first +next +---- +b#3,1:c +. + +define +a.SINGLEDEL.3: +a.SET.2:b +a.SET.1:a +---- + +iter +first +next +---- +a#1,1:a +. + +define +a.SINGLEDEL.3: +a.MERGE.2:b +a.MERGE.1:a +---- + +iter +first +next +---- +a#3,0: +. + +define +a.SINGLEDEL.4: +a.SET.3:val +a.SINGLEDEL.2: +a.SET.1:val +---- + +iter +first +---- +. + +iter snapshots=2 +first +next +next +---- +a#2,7: +a#1,1:val +. + +define +a.SINGLEDEL.4: +a.SET.3:val +a.DEL.2: +a.SET.1:val +---- + +iter +first +next +---- +a#2,0: +. + +iter snapshots=2 +first +next +next +---- +a#2,0: +a#1,1:val +. + +iter snapshots=3 +first +next +---- +a#2,0: +. + +iter snapshots=(2,3) +first +next +next +---- +a#2,0: +a#1,1:val +. + +define +a.SINGLEDEL.4: +a.SET.3:c +a.MERGE.2:b +a.SET.1:a +---- + +iter +first +next +---- +a#2,1:ab[base] +. + +iter snapshots=2 +first +next +next +---- +a#2,2:b +a#1,1:a +. + +iter snapshots=3 +first +next +---- +a#2,1:ab[base] +. + +iter snapshots=(2,3,4) +first +next +next +next +next +---- +a#4,7: +a#3,1:c +a#2,2:b +a#1,1:a +. + +define +a.SINGLEDEL.3: +a.RANGEDEL.2:c +a.SET.1:val +---- + +iter +first +next +next +tombstones +---- +a#3,7: +a#2,15:c +. +a-c#2 +. + +define +a.RANGEDEL.3:d +a.DEL.2: +a.SET.1:a +d.DEL.2: +---- + +iter +first +next +next +tombstones +---- +a#3,15:d +d#2,0: +. +a-d#3 +. + +iter snapshots=3 +first +next +next +next +---- +a#3,15:d +a#2,0: +d#2,0: +. + +iter snapshots=2 +first +next +next +next +---- +a#3,15:d +a#1,1:a +d#2,0: +. + +iter snapshots=1 +first +next +next +---- +a#3,15:d +d#2,0: +. + +define +a.MERGE.2:a +b.RANGEDEL.1:c +---- + +iter +first +tombstones a +next +next +tombstones +---- +a#2,2:a +. +b#1,15:c +. +b-c#1 +. + +define +a.MERGE.2:v2 +a.RANGEDEL.1:b +a.MERGE.1:v1 +---- + +iter allow-zero-seqnum=true +first +next +next +next +tombstones +---- +a#2,2:v2 +a#1,15:b +a#0,2:v1 +. +a-b#1 +. + +# Verify that we transform merge+del -> set. + +define +a.MERGE.5:5 +a.DEL.3: +a.MERGE.1:1 +---- + +iter +first +next +---- +a#5,1:5[base] +. + +iter allow-zero-seqnum=true +first +next +---- +a#0,1:5[base] +. + +iter elide-tombstones=true +first +next +---- +a#5,1:5[base] +. + +iter snapshots=2 +first +next +next +---- +a#5,1:5[base] +a#1,2:1 +. + +iter snapshots=2 elide-tombstones=true +first +next +next +---- +a#5,1:5[base] +a#1,2:1 +. + +# Verify that we transform merge+rangedel -> set. This isn't strictly +# necessary, but provides consistency with the behavior for merge+del. + +define +a.RANGEDEL.3:c +b.MERGE.5:5 +b.SET.2:2 +b.MERGE.1:1 +---- + +iter +first +next +next +---- +a#3,15:c +b#5,1:5[base] +. + +iter allow-zero-seqnum=true +first +next +next +---- +a#3,15:c +b#0,1:5[base] +. + +iter snapshots=2 +first +next +next +---- +a#3,15:c +b#5,1:5[base] +b#1,2:1 + +define +a.RANGEDEL.3:c +b.MERGE.5:5 +b.MERGE.2:2 +b.MERGE.1:1 +---- + +iter +first +next +next +---- +a#3,15:c +b#5,1:5[base] +. + +iter snapshots=2 +first +next +next +---- +a#3,15:c +b#5,1:5[base] +b#1,2:1 + +# SET that meets a DEL is transformed into a SETWITHDEL. + +define +a.SET.2:b +a.DEL.1: +---- + +iter +first +next +---- +a#2,18:b +. + +iter snapshots=2 +first +next +next +---- +a#2,1:b +a#1,0: +. + +define +a.SET.3:c +a.DEL.2: +a.SET.1:b +---- + +iter +first +next +---- +a#3,18:c +. + +iter snapshots=2 +first +next +next +---- +a#3,18:c +a#1,1:b +. + +define +a.SET.3:c +a.SET.2:b +a.DEL.1: +---- + +iter +first +next +---- +a#3,18:c +. + +iter snapshots=3 +first +next +next +---- +a#3,1:c +a#2,18:b +. + +iter snapshots=2 +first +next +next +---- +a#3,1:c +a#1,0: +. + +define +a.DEL.3: +a.SET.2:b +a.DEL.1: +---- + +iter +first +next +---- +a#3,0: +. + +iter snapshots=3 +first +next +next +---- +a#3,0: +a#2,18:b +. + +iter snapshots=2 +first +next +next +---- +a#3,0: +a#1,0: +. + +# SETWITHDEL-eligible entries at or under a RANGEDEL at the same user key should +# be skipped. +define +a.SET.3:c +a.RANGEDEL.2:z +a.SET.2:b +a.DEL.1: +---- + +iter allow-zero-seqnum=true +first +next +next +---- +a#0,18:c +a#2,15:z +. + +iter allow-zero-seqnum=true snapshots=3 +first +next +next +next +---- +a#3,1:c +a#2,15:z +a#0,18:b +. + +iter allow-zero-seqnum=true snapshots=2 +first +next +next +next +---- +a#3,18:c +a#2,15:z +a#1,0: +. + +define +a.SET.4:c +a.RANGEDEL.3:z +a.SET.2:b +a.DEL.1: +---- + +iter +first +next +next +---- +a#4,18:c +a#3,15:z +. + +# Invalid keys are emitted under SETWITHDEL. + +define +a.SET.2:b +a.INVALID.1: +---- + +iter +first +next +---- +a#2,18:b +err=invalid internal key kind: 255 + +define +a.SET.3:c +a.INVALID.2: +a.SET.1:b +---- + +iter +first +next +---- +a#3,18:c +err=invalid internal key kind: 255 + +# SINGLEDEL that meets a SETWITHDEL is transformed into a DEL. + +define +a.SINGLEDEL.3: +a.SETWITHDEL.2:d +b.SET.1:c +---- + +iter +first +next +next +---- +a#3,0: +b#1,1:c +. + +iter snapshots=2 +first +next +next +---- +a#3,0: +b#1,1:c +. + +iter snapshots=3 +first +next +next +next +---- +a#3,7: +a#2,18:d +b#1,1:c +. + +define +a.SETWITHDEL.3:3 +a.SET.2:d +b.SET.1:c +---- + +iter +first +next +next +---- +a#3,18:3 +b#1,1:c +. + +iter snapshots=3 +first +next +next +next +---- +a#3,18:3 +a#2,1:d +b#1,1:c +. + +# Test a DELSIZED whose encoded value matches the size of a deleted key. The +# DELSIZED's value should be removed, reflecting that the tombstone already +# dropped the key that it was expected to drop. + +define +a.SET.9:foo +b.DELSIZED.8:varint(11) +b.SET.5:helloworld +c.SET.2:bar +---- + +iter +first +next +next +---- +a#9,1:foo +b#8,23: +c#2,1:bar + + +# Test two DELSIZEDs meeting. The lower-sequenced number value should carry +# forward, at the higher sequence number. + +define +a.SET.9:foo +b.DELSIZED.9:varint(20) +b.DELSIZED.8:varint(10) +c.SET.2:bar +---- + +iter +first +next +next +---- +a#9,1:foo +b#9,23:varint(10) +c#2,1:bar + +# Test a DELSIZED whose encoded value is larger than the size of the deleted +# key. The DELSIZED should be replaced by an ordinary DEL with the same sequence +# number. + +define +a.SET.2:foo +b.DELSIZED.8:varint(25) +b.SET.3:hello +c.SET.9:bar +---- + +iter +first +next +next +---- +a#2,1:foo +b#8,0: +c#9,1:bar diff --git a/testdata/compaction_tombstones b/testdata/compaction_tombstones index cedf8b38c5..c197cff21d 100644 --- a/testdata/compaction_tombstones +++ b/testdata/compaction_tombstones @@ -80,12 +80,12 @@ wait-pending-table-stats num-entries: 2 num-deletions: 1 num-range-key-sets: 0 -point-deletions-bytes-estimate: 778 +point-deletions-bytes-estimate: 108 range-deletions-bytes-estimate: 0 maybe-compact ---- -[JOB 100] compacted(elision-only) L6 [000004] (822 B) + L6 [] (0 B) -> L6 [000005] (771 B), in 1.0s (2.0s total), output rate 771 B/s +[JOB 100] compacted(elision-only) L6 [000004] (823 B) + L6 [] (0 B) -> L6 [000005] (772 B), in 1.0s (2.0s total), output rate 772 B/s version ---- @@ -119,8 +119,8 @@ wait-pending-table-stats num-entries: 6 num-deletions: 2 num-range-key-sets: 0 -point-deletions-bytes-estimate: 308 -range-deletions-bytes-estimate: 76 +point-deletions-bytes-estimate: 48 +range-deletions-bytes-estimate: 66 maybe-compact ---- @@ -134,7 +134,7 @@ close-snapshot close-snapshot 103 ---- -[JOB 100] compacted(elision-only) L6 [000004] (1012 B) + L6 [] (0 B) -> L6 [] (0 B), in 1.0s (2.0s total), output rate 0 B/s +[JOB 100] compacted(elision-only) L6 [000004] (1001 B) + L6 [] (0 B) -> L6 [] (0 B), in 1.0s (2.0s total), output rate 0 B/s # Test a table that contains both deletions and non-deletions, but whose # non-deletions well outnumber its deletions. The table should not be @@ -152,7 +152,7 @@ wait-pending-table-stats num-entries: 11 num-deletions: 1 num-range-key-sets: 0 -point-deletions-bytes-estimate: 170 +point-deletions-bytes-estimate: 26 range-deletions-bytes-estimate: 0 close-snapshot @@ -197,7 +197,7 @@ num-entries: 5 num-deletions: 1 num-range-key-sets: 0 point-deletions-bytes-estimate: 0 -range-deletions-bytes-estimate: 16488 +range-deletions-bytes-estimate: 16492 # Because we set max bytes low, maybe-compact will trigger an automatic # compaction in preference over an elision-only compaction. @@ -233,7 +233,7 @@ wait-pending-table-stats num-entries: 3 num-deletions: 3 num-range-key-sets: 0 -point-deletions-bytes-estimate: 14782 +point-deletions-bytes-estimate: 6966 range-deletions-bytes-estimate: 0 # By plain file size, 000005 should be picked because it is larger and @@ -319,11 +319,11 @@ num-entries: 3 num-deletions: 1 num-range-key-sets: 0 point-deletions-bytes-estimate: 0 -range-deletions-bytes-estimate: 39 +range-deletions-bytes-estimate: 41 maybe-compact ---- -[JOB 100] compacted(elision-only) L6 [000004] (1001 B) + L6 [] (0 B) -> L6 [000005] (778 B), in 1.0s (2.0s total), output rate 778 B/s +[JOB 100] compacted(elision-only) L6 [000004] (1003 B) + L6 [] (0 B) -> L6 [000005] (778 B), in 1.0s (2.0s total), output rate 778 B/s # Close the DB, asserting that the reference counts balance. close @@ -359,7 +359,7 @@ wait-pending-table-stats num-entries: 2 num-deletions: 1 num-range-key-sets: 0 -point-deletions-bytes-estimate: 5193 +point-deletions-bytes-estimate: 2786 range-deletions-bytes-estimate: 0 wait-pending-table-stats @@ -369,11 +369,11 @@ num-entries: 2 num-deletions: 1 num-range-key-sets: 0 point-deletions-bytes-estimate: 0 -range-deletions-bytes-estimate: 8244 +range-deletions-bytes-estimate: 8246 maybe-compact ---- -[JOB 100] compacted(default) L5 [000005] (849 B) + L6 [000007] (13 K) -> L6 [000008] (4.8 K), in 1.0s (2.0s total), output rate 4.8 K/s +[JOB 100] compacted(default) L5 [000005] (850 B) + L6 [000007] (13 K) -> L6 [000008] (4.8 K), in 1.0s (2.0s total), output rate 4.8 K/s # The same LSM as above. However, this time, with point tombstone weighting at # 2x, the table with the point tombstone (000004) will be selected as the @@ -402,7 +402,7 @@ wait-pending-table-stats num-entries: 2 num-deletions: 1 num-range-key-sets: 0 -point-deletions-bytes-estimate: 5193 +point-deletions-bytes-estimate: 2786 range-deletions-bytes-estimate: 0 wait-pending-table-stats @@ -412,8 +412,8 @@ num-entries: 2 num-deletions: 1 num-range-key-sets: 0 point-deletions-bytes-estimate: 0 -range-deletions-bytes-estimate: 8244 +range-deletions-bytes-estimate: 8246 maybe-compact ---- -[JOB 100] compacted(default) L5 [000004] (821 B) + L6 [000006] (13 K) -> L6 [000008] (4.8 K), in 1.0s (2.0s total), output rate 4.8 K/s +[JOB 100] compacted(default) L5 [000005] (850 B) + L6 [000007] (13 K) -> L6 [000008] (4.8 K), in 1.0s (2.0s total), output rate 4.8 K/s diff --git a/testdata/event_listener b/testdata/event_listener index afdc789b40..4bfb91792d 100644 --- a/testdata/event_listener +++ b/testdata/event_listener @@ -94,6 +94,11 @@ close: db/marker.format-version.000013.014 remove: db/marker.format-version.000012.013 sync: db upgraded to format version: 014 +create: db/marker.format-version.000014.015 +close: db/marker.format-version.000014.015 +remove: db/marker.format-version.000013.014 +sync: db +upgraded to format version: 015 create: db/temporary.000003.dbtmp sync: db/temporary.000003.dbtmp close: db/temporary.000003.dbtmp @@ -122,7 +127,7 @@ close: db/marker.manifest.000002.MANIFEST-000006 remove: db/marker.manifest.000001.MANIFEST-000001 sync: db [JOB 5] MANIFEST created 000006 -[JOB 5] flushed 1 memtable to L0 [000005] (770 B), in 1.0s (2.0s total), output rate 770 B/s +[JOB 5] flushed 1 memtable to L0 [000005] (771 B), in 1.0s (2.0s total), output rate 771 B/s compact ---- @@ -146,28 +151,28 @@ close: db/marker.manifest.000003.MANIFEST-000009 remove: db/marker.manifest.000002.MANIFEST-000006 sync: db [JOB 7] MANIFEST created 000009 -[JOB 7] flushed 1 memtable to L0 [000008] (770 B), in 1.0s (2.0s total), output rate 770 B/s +[JOB 7] flushed 1 memtable to L0 [000008] (771 B), in 1.0s (2.0s total), output rate 771 B/s remove: db/MANIFEST-000001 [JOB 7] MANIFEST deleted 000001 [JOB 8] compacting(default) L0 [000005 000008] (1.5 K) + L6 [] (0 B) open: db/000005.sst -read-at(717, 53): db/000005.sst -read-at(680, 37): db/000005.sst -read-at(52, 628): db/000005.sst -read-at(25, 27): db/000005.sst +read-at(718, 53): db/000005.sst +read-at(681, 37): db/000005.sst +read-at(53, 628): db/000005.sst +read-at(26, 27): db/000005.sst open: db/000005.sst close: db/000005.sst open: db/000008.sst -read-at(717, 53): db/000008.sst -read-at(680, 37): db/000008.sst -read-at(52, 628): db/000008.sst -read-at(25, 27): db/000008.sst +read-at(718, 53): db/000008.sst +read-at(681, 37): db/000008.sst +read-at(53, 628): db/000008.sst +read-at(26, 27): db/000008.sst open: db/000008.sst close: db/000008.sst open: db/000005.sst -read-at(0, 25): db/000005.sst +read-at(0, 26): db/000005.sst open: db/000008.sst -read-at(0, 25): db/000008.sst +read-at(0, 26): db/000008.sst close: db/000008.sst close: db/000005.sst create: db/000010.sst @@ -183,7 +188,7 @@ close: db/marker.manifest.000004.MANIFEST-000011 remove: db/marker.manifest.000003.MANIFEST-000009 sync: db [JOB 8] MANIFEST created 000011 -[JOB 8] compacted(default) L0 [000005 000008] (1.5 K) + L6 [] (0 B) -> L6 [000010] (770 B), in 1.0s (3.0s total), output rate 770 B/s +[JOB 8] compacted(default) L0 [000005 000008] (1.5 K) + L6 [] (0 B) -> L6 [000010] (771 B), in 1.0s (3.0s total), output rate 771 B/s close: db/000005.sst close: db/000008.sst remove: db/000005.sst @@ -218,7 +223,7 @@ close: db/marker.manifest.000005.MANIFEST-000014 remove: db/marker.manifest.000004.MANIFEST-000011 sync: db [JOB 10] MANIFEST created 000014 -[JOB 10] flushed 1 memtable to L0 [000013] (770 B), in 1.0s (2.0s total), output rate 770 B/s +[JOB 10] flushed 1 memtable to L0 [000013] (771 B), in 1.0s (2.0s total), output rate 771 B/s enable-file-deletions ---- @@ -238,11 +243,11 @@ link: ext/0 -> db/000015.sst [JOB 12] ingesting: sstable created 000015 sync: db open: db/000013.sst -read-at(717, 53): db/000013.sst -read-at(680, 37): db/000013.sst -read-at(52, 628): db/000013.sst -read-at(25, 27): db/000013.sst -read-at(0, 25): db/000013.sst +read-at(718, 53): db/000013.sst +read-at(681, 37): db/000013.sst +read-at(53, 628): db/000013.sst +read-at(26, 27): db/000013.sst +read-at(0, 26): db/000013.sst create: db/MANIFEST-000016 close: db/MANIFEST-000014 sync: db/MANIFEST-000016 @@ -258,13 +263,13 @@ metrics ---- __level_____count____size___score______in__ingest(sz_cnt)____move(sz_cnt)___write(sz_cnt)____read___r-amp___w-amp WAL 1 27 B - 48 B - - - - 108 B - - - 2.2 - 0 2 1.6 K 0.40 81 B 826 B 1 0 B 0 2.3 K 3 0 B 2 28.5 + 0 2 1.6 K 0.40 81 B 826 B 1 0 B 0 2.3 K 3 0 B 2 28.6 1 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 2 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 3 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 4 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 5 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 6 1 770 B - 1.5 K 0 B 0 0 B 0 770 B 1 1.5 K 1 0.5 + 6 1 771 B - 1.5 K 0 B 0 0 B 0 771 B 1 1.5 K 1 0.5 total 3 2.3 K - 934 B 826 B 1 0 B 0 3.9 K 4 1.5 K 3 4.3 flush 3 0 B 0 0 (ingest = tables-ingested, move = ingested-as-flushable) compact 1 2.3 K 0 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) @@ -273,7 +278,7 @@ compact 1 2.3 K 0 B 0 (size == esti zmemtbl 0 0 B ztbl 0 0 B bcache 8 1.4 K 11.1% (score == hit-rate) - tcache 1 728 B 40.0% (score == hit-rate) + tcache 1 744 B 40.0% (score == hit-rate) snaps 0 - 0 (score == earliest seq num) titers 0 filter - - 0.0% (score == utility) @@ -330,7 +335,7 @@ sync-data: db/000022.sst close: db/000022.sst sync: db sync: db/MANIFEST-000016 -[JOB 17] flushed 1 memtable to L0 [000022] (770 B), in 1.0s (2.0s total), output rate 770 B/s +[JOB 17] flushed 1 memtable to L0 [000022] (771 B), in 1.0s (2.0s total), output rate 771 B/s remove: db/MANIFEST-000011 [JOB 17] MANIFEST deleted 000011 [JOB 18] flushing 2 ingested tables @@ -353,13 +358,13 @@ metrics ---- __level_____count____size___score______in__ingest(sz_cnt)____move(sz_cnt)___write(sz_cnt)____read___r-amp___w-amp WAL 1 29 B - 82 B - - - - 110 B - - - 1.3 - 0 4 3.1 K 0.80 81 B 1.6 K 2 0 B 0 3.0 K 4 0 B 4 38.0 + 0 4 3.1 K 0.80 81 B 1.6 K 2 0 B 0 3.0 K 4 0 B 4 38.1 1 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 2 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 3 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 4 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 5 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 6 2 1.6 K - 1.5 K 826 B 1 0 B 0 770 B 1 1.5 K 1 0.5 + 6 2 1.6 K - 1.5 K 826 B 1 0 B 0 771 B 1 1.5 K 1 0.5 total 6 4.7 K - 2.5 K 2.4 K 3 0 B 0 6.3 K 5 1.5 K 5 2.5 flush 6 1.6 K 2 1 (ingest = tables-ingested, move = ingested-as-flushable) compact 1 4.7 K 0 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) @@ -368,7 +373,7 @@ compact 1 4.7 K 0 B 0 (size == esti zmemtbl 0 0 B ztbl 0 0 B bcache 16 2.9 K 14.3% (score == hit-rate) - tcache 1 728 B 50.0% (score == hit-rate) + tcache 1 744 B 50.0% (score == hit-rate) snaps 0 - 0 (score == earliest seq num) titers 0 filter - - 0.0% (score == utility) @@ -393,9 +398,9 @@ close: open-dir: checkpoint link: db/OPTIONS-000003 -> checkpoint/OPTIONS-000003 open-dir: checkpoint -create: checkpoint/marker.format-version.000001.014 -sync-data: checkpoint/marker.format-version.000001.014 -close: checkpoint/marker.format-version.000001.014 +create: checkpoint/marker.format-version.000001.015 +sync-data: checkpoint/marker.format-version.000001.015 +close: checkpoint/marker.format-version.000001.015 sync: checkpoint close: checkpoint link: db/000013.sst -> checkpoint/000013.sst diff --git a/testdata/flushable_ingest b/testdata/flushable_ingest index 6efc0ef175..dffb8f77a1 100644 --- a/testdata/flushable_ingest +++ b/testdata/flushable_ingest @@ -61,7 +61,7 @@ LOCK MANIFEST-000001 OPTIONS-000003 ext -marker.format-version.000013.014 +marker.format-version.000014.015 marker.manifest.000001.MANIFEST-000001 # Test basic WAL replay @@ -83,7 +83,7 @@ LOCK MANIFEST-000001 OPTIONS-000003 ext -marker.format-version.000013.014 +marker.format-version.000014.015 marker.manifest.000001.MANIFEST-000001 open @@ -392,7 +392,7 @@ LOCK MANIFEST-000001 OPTIONS-000003 ext -marker.format-version.000013.014 +marker.format-version.000014.015 marker.manifest.000001.MANIFEST-000001 close @@ -413,7 +413,7 @@ LOCK MANIFEST-000001 OPTIONS-000003 ext -marker.format-version.000013.014 +marker.format-version.000014.015 marker.manifest.000001.MANIFEST-000001 open @@ -446,7 +446,7 @@ MANIFEST-000001 MANIFEST-000012 OPTIONS-000013 ext -marker.format-version.000013.014 +marker.format-version.000014.015 marker.manifest.000002.MANIFEST-000012 # Make sure that the new mutable memtable can accept writes. @@ -590,7 +590,7 @@ LOCK MANIFEST-000001 OPTIONS-000003 ext -marker.format-version.000013.014 +marker.format-version.000014.015 marker.manifest.000001.MANIFEST-000001 close @@ -610,7 +610,7 @@ MANIFEST-000001 OPTIONS-000003 ext ext1 -marker.format-version.000013.014 +marker.format-version.000014.015 marker.manifest.000001.MANIFEST-000001 ignoreSyncs false diff --git a/testdata/ingest b/testdata/ingest index b25433b81a..a811e12a1e 100644 --- a/testdata/ingest +++ b/testdata/ingest @@ -48,7 +48,7 @@ compact 0 0 B 0 B 0 (size == esti zmemtbl 0 0 B ztbl 0 0 B bcache 8 1.5 K 42.9% (score == hit-rate) - tcache 1 728 B 50.0% (score == hit-rate) + tcache 1 744 B 50.0% (score == hit-rate) snaps 0 - 0 (score == earliest seq num) titers 0 filter - - 0.0% (score == utility) diff --git a/testdata/ingest_load b/testdata/ingest_load index 76d5dcaaaa..3e5d690c72 100644 --- a/testdata/ingest_load +++ b/testdata/ingest_load @@ -143,3 +143,10 @@ rangekey: a-z:{(#0,RANGEKEYSET,@1,foo)} 1: a#0,21-z#72057594037927935,15 points: a#0,15-z#72057594037927935,15 ranges: a#0,21-z#72057594037927935,21 + +# Loading tables at an unsupported table format results in an error. + +load writer-version=15 db-version=14 +a.SET.0: +---- +pebble: table format (Pebble,v4) is not within range supported at DB format major version 14, ((Pebble,v1),(Pebble,v3)) diff --git a/testdata/iter_histories/range_keys_simple b/testdata/iter_histories/range_keys_simple index 3a05df06f2..6748821e70 100644 --- a/testdata/iter_histories/range_keys_simple +++ b/testdata/iter_histories/range_keys_simple @@ -422,7 +422,7 @@ reset format-major-version=6 batch commit range-key-set a z @5 boop ---- -pebble: range keys require at least format major version 8 (current: 6) +pebble: batch requires at least format major version 8 (current: 6) # Constructing iterator over range keys on a DB that doesn't support them # results in an error. diff --git a/testdata/manual_compaction_file_boundaries_delsized b/testdata/manual_compaction_file_boundaries_delsized new file mode 100644 index 0000000000..b83e57e265 --- /dev/null +++ b/testdata/manual_compaction_file_boundaries_delsized @@ -0,0 +1,520 @@ +# Test the file-size grandparent boundary alignment heuristic. This test sets up +# L3 with a file at each of 'a', 'b', ..., 'z'. It also creates a single file in +# L2 spanning a-z. Then, it commits, flushes and compacts into L1 keys 'a@1', +# 'aa@1', 'ab@1', ..., 'zz@1'. Finally, it tests compacting L1 into L2. +# +# With L3 as the grandparent level, the alignment heuristic should attempt to +# align the output files with grandparent's boundaries. Each output file should +# have a key range formed by the prefix of a single letter. + +define target-file-sizes=(5000, 5000, 5000, 5000) +L2 + a.SET.101: + z.SET.102: +L3 + a.SET.001: +L3 + b.SET.002: +L3 + c.SET.003: +L3 + d.SET.004: +L3 + e.SET.005: +L3 + f.SET.006: +L3 + g.SET.007: +L3 + h.SET.008: +L3 + i.SET.009: +L3 + j.SET.010: +L3 + k.SET.011: +L3 + l.SET.012: +L3 + m.SET.013: +L3 + n.SET.014: +L3 + o.SET.015: +L3 + p.SET.016: +L3 + q.SET.017: +L3 + r.SET.018: +L3 + s.SET.019: +L3 + t.SET.020: +L3 + u.SET.021: +L3 + v.SET.022: +L3 + w.SET.023: +L3 + x.SET.024: +L3 + y.SET.025: +L3 + z.SET.026: +---- +2: + 000004:[a#101,SET-z#102,SET] +3: + 000005:[a#1,SET-a#1,SET] + 000006:[b#2,SET-b#2,SET] + 000007:[c#3,SET-c#3,SET] + 000008:[d#4,SET-d#4,SET] + 000009:[e#5,SET-e#5,SET] + 000010:[f#6,SET-f#6,SET] + 000011:[g#7,SET-g#7,SET] + 000012:[h#8,SET-h#8,SET] + 000013:[i#9,SET-i#9,SET] + 000014:[j#10,SET-j#10,SET] + 000015:[k#11,SET-k#11,SET] + 000016:[l#12,SET-l#12,SET] + 000017:[m#13,SET-m#13,SET] + 000018:[n#14,SET-n#14,SET] + 000019:[o#15,SET-o#15,SET] + 000020:[p#16,SET-p#16,SET] + 000021:[q#17,SET-q#17,SET] + 000022:[r#18,SET-r#18,SET] + 000023:[s#19,SET-s#19,SET] + 000024:[t#20,SET-t#20,SET] + 000025:[u#21,SET-u#21,SET] + 000026:[v#22,SET-v#22,SET] + 000027:[w#23,SET-w#23,SET] + 000028:[x#24,SET-x#24,SET] + 000029:[y#25,SET-y#25,SET] + 000030:[z#26,SET-z#26,SET] + +populate keylen=2 vallen=200 timestamps=(1) +---- +wrote 702 keys + +flush +---- +0.0: + 000033:[a@1#103,SET-av@1#125,SET] + 000034:[aw@1#126,SET-br@1#148,SET] + 000035:[bs@1#149,SET-cn@1#171,SET] + 000036:[co@1#172,SET-dj@1#194,SET] + 000037:[dk@1#195,SET-ef@1#217,SET] + 000038:[eg@1#218,SET-fb@1#240,SET] + 000039:[fc@1#241,SET-fy@1#263,SET] + 000040:[fz@1#264,SET-gu@1#286,SET] + 000041:[gv@1#287,SET-hq@1#309,SET] + 000042:[hr@1#310,SET-im@1#332,SET] + 000043:[in@1#333,SET-ji@1#355,SET] + 000044:[jj@1#356,SET-ke@1#378,SET] + 000045:[kf@1#379,SET-la@1#401,SET] + 000046:[lb@1#402,SET-lx@1#424,SET] + 000047:[ly@1#425,SET-mt@1#447,SET] + 000048:[mu@1#448,SET-np@1#470,SET] + 000049:[nq@1#471,SET-ol@1#493,SET] + 000050:[om@1#494,SET-ph@1#516,SET] + 000051:[pi@1#517,SET-qd@1#539,SET] + 000052:[qe@1#540,SET-r@1#562,SET] + 000053:[ra@1#563,SET-rw@1#585,SET] + 000054:[rx@1#586,SET-ss@1#608,SET] + 000055:[st@1#609,SET-to@1#631,SET] + 000056:[tp@1#632,SET-uk@1#654,SET] + 000057:[ul@1#655,SET-vg@1#677,SET] + 000058:[vh@1#678,SET-wc@1#700,SET] + 000059:[wd@1#701,SET-wz@1#723,SET] + 000060:[x@1#724,SET-xv@1#746,SET] + 000061:[xw@1#747,SET-yr@1#769,SET] + 000062:[ys@1#770,SET-zn@1#792,SET] + 000063:[zo@1#793,SET-zz@1#804,SET] +2: + 000004:[a#101,SET-z#102,SET] +3: + 000005:[a#1,SET-a#1,SET] + 000006:[b#2,SET-b#2,SET] + 000007:[c#3,SET-c#3,SET] + 000008:[d#4,SET-d#4,SET] + 000009:[e#5,SET-e#5,SET] + 000010:[f#6,SET-f#6,SET] + 000011:[g#7,SET-g#7,SET] + 000012:[h#8,SET-h#8,SET] + 000013:[i#9,SET-i#9,SET] + 000014:[j#10,SET-j#10,SET] + 000015:[k#11,SET-k#11,SET] + 000016:[l#12,SET-l#12,SET] + 000017:[m#13,SET-m#13,SET] + 000018:[n#14,SET-n#14,SET] + 000019:[o#15,SET-o#15,SET] + 000020:[p#16,SET-p#16,SET] + 000021:[q#17,SET-q#17,SET] + 000022:[r#18,SET-r#18,SET] + 000023:[s#19,SET-s#19,SET] + 000024:[t#20,SET-t#20,SET] + 000025:[u#21,SET-u#21,SET] + 000026:[v#22,SET-v#22,SET] + 000027:[w#23,SET-w#23,SET] + 000028:[x#24,SET-x#24,SET] + 000029:[y#25,SET-y#25,SET] + 000030:[z#26,SET-z#26,SET] + +compact a-zz L0 +---- +1: + 000064:[a@1#103,SET-av@1#125,SET] + 000065:[aw@1#126,SET-br@1#148,SET] + 000066:[bs@1#149,SET-cn@1#171,SET] + 000067:[co@1#172,SET-dj@1#194,SET] + 000068:[dk@1#195,SET-ef@1#217,SET] + 000069:[eg@1#218,SET-fb@1#240,SET] + 000070:[fc@1#241,SET-fy@1#263,SET] + 000071:[fz@1#264,SET-gu@1#286,SET] + 000072:[gv@1#287,SET-hq@1#309,SET] + 000073:[hr@1#310,SET-im@1#332,SET] + 000074:[in@1#333,SET-ji@1#355,SET] + 000075:[jj@1#356,SET-ke@1#378,SET] + 000076:[kf@1#379,SET-la@1#401,SET] + 000077:[lb@1#402,SET-lx@1#424,SET] + 000078:[ly@1#425,SET-mt@1#447,SET] + 000079:[mu@1#448,SET-np@1#470,SET] + 000080:[nq@1#471,SET-ol@1#493,SET] + 000081:[om@1#494,SET-ph@1#516,SET] + 000082:[pi@1#517,SET-qd@1#539,SET] + 000083:[qe@1#540,SET-r@1#562,SET] + 000084:[ra@1#563,SET-rw@1#585,SET] + 000085:[rx@1#586,SET-ss@1#608,SET] + 000086:[st@1#609,SET-to@1#631,SET] + 000087:[tp@1#632,SET-uk@1#654,SET] + 000088:[ul@1#655,SET-vg@1#677,SET] + 000089:[vh@1#678,SET-wc@1#700,SET] + 000090:[wd@1#701,SET-wz@1#723,SET] + 000091:[x@1#724,SET-xv@1#746,SET] + 000092:[xw@1#747,SET-yr@1#769,SET] + 000093:[ys@1#770,SET-zn@1#792,SET] + 000094:[zo@1#793,SET-zz@1#804,SET] +2: + 000004:[a#101,SET-z#102,SET] +3: + 000005:[a#1,SET-a#1,SET] + 000006:[b#2,SET-b#2,SET] + 000007:[c#3,SET-c#3,SET] + 000008:[d#4,SET-d#4,SET] + 000009:[e#5,SET-e#5,SET] + 000010:[f#6,SET-f#6,SET] + 000011:[g#7,SET-g#7,SET] + 000012:[h#8,SET-h#8,SET] + 000013:[i#9,SET-i#9,SET] + 000014:[j#10,SET-j#10,SET] + 000015:[k#11,SET-k#11,SET] + 000016:[l#12,SET-l#12,SET] + 000017:[m#13,SET-m#13,SET] + 000018:[n#14,SET-n#14,SET] + 000019:[o#15,SET-o#15,SET] + 000020:[p#16,SET-p#16,SET] + 000021:[q#17,SET-q#17,SET] + 000022:[r#18,SET-r#18,SET] + 000023:[s#19,SET-s#19,SET] + 000024:[t#20,SET-t#20,SET] + 000025:[u#21,SET-u#21,SET] + 000026:[v#22,SET-v#22,SET] + 000027:[w#23,SET-w#23,SET] + 000028:[x#24,SET-x#24,SET] + 000029:[y#25,SET-y#25,SET] + 000030:[z#26,SET-z#26,SET] + +# Perform the actual test. Compacting L1 into L2 should use L3's boundaries to +# inform compaction output splitting. +# +compact a-zz L1 +---- +2: + 000095:[a#101,SET-az@1#129,SET] + 000096:[b@1#130,SET-bz@1#156,SET] + 000097:[c@1#157,SET-cz@1#183,SET] + 000098:[d@1#184,SET-dz@1#210,SET] + 000099:[e@1#211,SET-ez@1#237,SET] + 000100:[f@1#238,SET-fz@1#264,SET] + 000101:[g@1#265,SET-gz@1#291,SET] + 000102:[h@1#292,SET-hz@1#318,SET] + 000103:[i@1#319,SET-iz@1#345,SET] + 000104:[j@1#346,SET-jz@1#372,SET] + 000105:[k@1#373,SET-kz@1#399,SET] + 000106:[l@1#400,SET-lz@1#426,SET] + 000107:[m@1#427,SET-mz@1#453,SET] + 000108:[n@1#454,SET-nz@1#480,SET] + 000109:[o@1#481,SET-oz@1#507,SET] + 000110:[p@1#508,SET-pz@1#534,SET] + 000111:[q@1#535,SET-qz@1#561,SET] + 000112:[r@1#562,SET-rz@1#588,SET] + 000113:[s@1#589,SET-sz@1#615,SET] + 000114:[t@1#616,SET-tz@1#642,SET] + 000115:[u@1#643,SET-uz@1#669,SET] + 000116:[v@1#670,SET-vz@1#696,SET] + 000117:[w@1#697,SET-wz@1#723,SET] + 000118:[x@1#724,SET-xz@1#750,SET] + 000119:[y@1#751,SET-yz@1#777,SET] + 000120:[z#102,SET-zr@1#796,SET] + 000121:[zs@1#797,SET-zz@1#804,SET] +3: + 000005:[a#1,SET-a#1,SET] + 000006:[b#2,SET-b#2,SET] + 000007:[c#3,SET-c#3,SET] + 000008:[d#4,SET-d#4,SET] + 000009:[e#5,SET-e#5,SET] + 000010:[f#6,SET-f#6,SET] + 000011:[g#7,SET-g#7,SET] + 000012:[h#8,SET-h#8,SET] + 000013:[i#9,SET-i#9,SET] + 000014:[j#10,SET-j#10,SET] + 000015:[k#11,SET-k#11,SET] + 000016:[l#12,SET-l#12,SET] + 000017:[m#13,SET-m#13,SET] + 000018:[n#14,SET-n#14,SET] + 000019:[o#15,SET-o#15,SET] + 000020:[p#16,SET-p#16,SET] + 000021:[q#17,SET-q#17,SET] + 000022:[r#18,SET-r#18,SET] + 000023:[s#19,SET-s#19,SET] + 000024:[t#20,SET-t#20,SET] + 000025:[u#21,SET-u#21,SET] + 000026:[v#22,SET-v#22,SET] + 000027:[w#23,SET-w#23,SET] + 000028:[x#24,SET-x#24,SET] + 000029:[y#25,SET-y#25,SET] + 000030:[z#26,SET-z#26,SET] + +file-sizes +---- +L2: + 000095:[a#101,1-az@1#129,1]: 7637 bytes (7.5 K) + 000096:[b@1#130,1-bz@1#156,1]: 6629 bytes (6.5 K) + 000097:[c@1#157,1-cz@1#183,1]: 6629 bytes (6.5 K) + 000098:[d@1#184,1-dz@1#210,1]: 6629 bytes (6.5 K) + 000099:[e@1#211,1-ez@1#237,1]: 6629 bytes (6.5 K) + 000100:[f@1#238,1-fz@1#264,1]: 6629 bytes (6.5 K) + 000101:[g@1#265,1-gz@1#291,1]: 6629 bytes (6.5 K) + 000102:[h@1#292,1-hz@1#318,1]: 6629 bytes (6.5 K) + 000103:[i@1#319,1-iz@1#345,1]: 6629 bytes (6.5 K) + 000104:[j@1#346,1-jz@1#372,1]: 6629 bytes (6.5 K) + 000105:[k@1#373,1-kz@1#399,1]: 6629 bytes (6.5 K) + 000106:[l@1#400,1-lz@1#426,1]: 6629 bytes (6.5 K) + 000107:[m@1#427,1-mz@1#453,1]: 6629 bytes (6.5 K) + 000108:[n@1#454,1-nz@1#480,1]: 6629 bytes (6.5 K) + 000109:[o@1#481,1-oz@1#507,1]: 6629 bytes (6.5 K) + 000110:[p@1#508,1-pz@1#534,1]: 6629 bytes (6.5 K) + 000111:[q@1#535,1-qz@1#561,1]: 6628 bytes (6.5 K) + 000112:[r@1#562,1-rz@1#588,1]: 6629 bytes (6.5 K) + 000113:[s@1#589,1-sz@1#615,1]: 6629 bytes (6.5 K) + 000114:[t@1#616,1-tz@1#642,1]: 6629 bytes (6.5 K) + 000115:[u@1#643,1-uz@1#669,1]: 6629 bytes (6.5 K) + 000116:[v@1#670,1-vz@1#696,1]: 6629 bytes (6.5 K) + 000117:[w@1#697,1-wz@1#723,1]: 6629 bytes (6.5 K) + 000118:[x@1#724,1-xz@1#750,1]: 6629 bytes (6.5 K) + 000119:[y@1#751,1-yz@1#777,1]: 6629 bytes (6.5 K) + 000120:[z#102,1-zr@1#796,1]: 5909 bytes (5.8 K) + 000121:[zs@1#797,1-zz@1#804,1]: 2491 bytes (2.4 K) +L3: + 000005:[a#1,1-a#1,1]: 10776 bytes (10 K) + 000006:[b#2,1-b#2,1]: 10776 bytes (10 K) + 000007:[c#3,1-c#3,1]: 10776 bytes (10 K) + 000008:[d#4,1-d#4,1]: 10776 bytes (10 K) + 000009:[e#5,1-e#5,1]: 10776 bytes (10 K) + 000010:[f#6,1-f#6,1]: 10776 bytes (10 K) + 000011:[g#7,1-g#7,1]: 10776 bytes (10 K) + 000012:[h#8,1-h#8,1]: 10776 bytes (10 K) + 000013:[i#9,1-i#9,1]: 10776 bytes (10 K) + 000014:[j#10,1-j#10,1]: 10776 bytes (10 K) + 000015:[k#11,1-k#11,1]: 10776 bytes (10 K) + 000016:[l#12,1-l#12,1]: 10776 bytes (10 K) + 000017:[m#13,1-m#13,1]: 10776 bytes (10 K) + 000018:[n#14,1-n#14,1]: 10776 bytes (10 K) + 000019:[o#15,1-o#15,1]: 10776 bytes (10 K) + 000020:[p#16,1-p#16,1]: 10776 bytes (10 K) + 000021:[q#17,1-q#17,1]: 10776 bytes (10 K) + 000022:[r#18,1-r#18,1]: 10776 bytes (10 K) + 000023:[s#19,1-s#19,1]: 10776 bytes (10 K) + 000024:[t#20,1-t#20,1]: 10776 bytes (10 K) + 000025:[u#21,1-u#21,1]: 10776 bytes (10 K) + 000026:[v#22,1-v#22,1]: 10776 bytes (10 K) + 000027:[w#23,1-w#23,1]: 10776 bytes (10 K) + 000028:[x#24,1-x#24,1]: 10776 bytes (10 K) + 000029:[y#25,1-y#25,1]: 10776 bytes (10 K) + 000030:[z#26,1-z#26,1]: 10776 bytes (10 K) + +# Test a scenario where there exists a grandparent file (in L3), but the L1->L2 +# compaction doesn't reach it until late in the compaction. The output file +# should be split at 2x the target file size (~10K), despite not being aligned +# with a grandparent. +# +# Additionally, when the compaction does reach the grandparent's start bound, +# the compaction should NOT split the output if the current output is less than +# 0.5x the target file size (~2.5K). +# +# Lastly, once past the final grandparent, the compaction should optimize for +# cutting as close to file size as possible, resulting in an output file ~5K. + +define target-file-sizes=(5000, 5000, 5000, 5000) +L1 + a.SET.201: + b.SET.202: + c.SET.203: + d.SET.204: + e.SET.205: + f.SET.206: + g.SET.207: + h.SET.208: + i.SET.209: + j.SET.210: + k.SET.211: + l.SET.212: + m.SET.213: + n.SET.214: + o.SET.215: +L2 + a.SET.101: + z.SET.102: +L3 + m.SET.001: +---- +1: + 000004:[a#201,SET-o#215,SET] +2: + 000005:[a#101,SET-z#102,SET] +3: + 000006:[m#1,SET-m#1,SET] + +compact a-zz L1 +---- +2: + 000007:[a#201,SET-j#210,SET] + 000008:[k#211,SET-o#215,SET] + 000009:[z#102,SET-z#102,SET] +3: + 000006:[m#1,SET-m#1,SET] + +file-sizes +---- +L2: + 000007:[a#201,1-j#210,1]: 10958 bytes (11 K) + 000008:[k#211,1-o#215,1]: 5865 bytes (5.7 K) + 000009:[z#102,1-z#102,1]: 781 bytes (781 B) +L3: + 000006:[m#1,1-m#1,1]: 10776 bytes (10 K) + +# Test the file-size splitter's adaptive tolerance for early-splitting at a +# grandparent boundary. The L1->L2 compaction has many opportunities to split at +# a grandparent boundary at file sizes ≥ 2.5K. Because it's seen more than 8 +# grandparent boundaries, waits until file size is ≥ 90% of the target file size +# (eg, ~4.5K). + +define target-file-sizes=(5000, 5000, 5000, 5000) +L1 + a.SET.201: + b.SET.202: + c.SET.203: + d.SET.204: + e.SET.205: + f.SET.206: + g.SET.207: + h.SET.208: + i.SET.209: + j.SET.210: + k.SET.211: + l.SET.212: + m.SET.213: + n.SET.214: + o.SET.215: +L2 + a.SET.101: + z.SET.102: +L3 + a.SET.001: +L3 + ab.SET.002: +L3 + ac.SET.003: +L3 + ad.SET.004: +L3 + ad.SET.005: +L3 + ad.SET.006: +L3 + ad.SET.007: +L3 + ad.SET.008: +L3 + c.SET.009: +L3 + d.SET.010: +L3 + e.SET.011: +L3 + f.SET.012: +L3 + m.SET.013: +---- +1: + 000004:[a#201,SET-o#215,SET] +2: + 000005:[a#101,SET-z#102,SET] +3: + 000006:[a#1,SET-a#1,SET] + 000007:[ab#2,SET-ab#2,SET] + 000008:[ac#3,SET-ac#3,SET] + 000013:[ad#8,SET-ad#8,SET] + 000012:[ad#7,SET-ad#7,SET] + 000011:[ad#6,SET-ad#6,SET] + 000010:[ad#5,SET-ad#5,SET] + 000009:[ad#4,SET-ad#4,SET] + 000014:[c#9,SET-c#9,SET] + 000015:[d#10,SET-d#10,SET] + 000016:[e#11,SET-e#11,SET] + 000017:[f#12,SET-f#12,SET] + 000018:[m#13,SET-m#13,SET] + +compact a-zz L1 +---- +2: + 000019:[a#201,SET-e#205,SET] + 000020:[f#206,SET-l#212,SET] + 000021:[m#213,SET-z#102,SET] +3: + 000006:[a#1,SET-a#1,SET] + 000007:[ab#2,SET-ab#2,SET] + 000008:[ac#3,SET-ac#3,SET] + 000013:[ad#8,SET-ad#8,SET] + 000012:[ad#7,SET-ad#7,SET] + 000011:[ad#6,SET-ad#6,SET] + 000010:[ad#5,SET-ad#5,SET] + 000009:[ad#4,SET-ad#4,SET] + 000014:[c#9,SET-c#9,SET] + 000015:[d#10,SET-d#10,SET] + 000016:[e#11,SET-e#11,SET] + 000017:[f#12,SET-f#12,SET] + 000018:[m#13,SET-m#13,SET] + +file-sizes +---- +L2: + 000019:[a#201,1-e#205,1]: 5865 bytes (5.7 K) + 000020:[f#206,1-l#212,1]: 7893 bytes (7.7 K) + 000021:[m#213,1-z#102,1]: 3827 bytes (3.7 K) +L3: + 000006:[a#1,1-a#1,1]: 1776 bytes (1.7 K) + 000007:[ab#2,1-ab#2,1]: 1777 bytes (1.7 K) + 000008:[ac#3,1-ac#3,1]: 1777 bytes (1.7 K) + 000013:[ad#8,1-ad#8,1]: 1777 bytes (1.7 K) + 000012:[ad#7,1-ad#7,1]: 1777 bytes (1.7 K) + 000011:[ad#6,1-ad#6,1]: 1777 bytes (1.7 K) + 000010:[ad#5,1-ad#5,1]: 1777 bytes (1.7 K) + 000009:[ad#4,1-ad#4,1]: 1777 bytes (1.7 K) + 000014:[c#9,1-c#9,1]: 1776 bytes (1.7 K) + 000015:[d#10,1-d#10,1]: 1776 bytes (1.7 K) + 000016:[e#11,1-e#11,1]: 1776 bytes (1.7 K) + 000017:[f#12,1-f#12,1]: 1776 bytes (1.7 K) + 000018:[m#13,1-m#13,1]: 1776 bytes (1.7 K) diff --git a/testdata/marked_for_compaction b/testdata/marked_for_compaction index 5464a24aa8..6d17017e96 100644 --- a/testdata/marked_for_compaction +++ b/testdata/marked_for_compaction @@ -21,7 +21,7 @@ marked L0.000004 maybe-compact ---- [JOB 100] compacted(rewrite) L1 [000005] (779 B) + L1 [] (0 B) -> L1 [000006] (779 B), in 1.0s (2.0s total), output rate 779 B/s -[JOB 100] compacted(rewrite) L0 [000004] (773 B) + L0 [] (0 B) -> L0 [000007] (773 B), in 1.0s (2.0s total), output rate 773 B/s +[JOB 100] compacted(rewrite) L0 [000004] (774 B) + L0 [] (0 B) -> L0 [000007] (774 B), in 1.0s (2.0s total), output rate 774 B/s 0.0: 000007:[c#11,SET-c#11,SET] points:[c#11,SET-c#11,SET] 1: diff --git a/testdata/metrics b/testdata/metrics index 464be0f7c0..81981b62e6 100644 --- a/testdata/metrics +++ b/testdata/metrics @@ -34,7 +34,7 @@ compact 0 0 B 0 B 0 (size == esti zmemtbl 1 256 K ztbl 0 0 B bcache 4 697 B 0.0% (score == hit-rate) - tcache 1 728 B 0.0% (score == hit-rate) + tcache 1 744 B 0.0% (score == hit-rate) snaps 0 - 0 (score == earliest seq num) titers 1 filter - - 0.0% (score == utility) @@ -82,7 +82,7 @@ compact 1 0 B 0 B 0 (size == esti zmemtbl 2 512 K ztbl 2 1.5 K bcache 8 1.4 K 42.9% (score == hit-rate) - tcache 2 1.4 K 66.7% (score == hit-rate) + tcache 2 1.5 K 66.7% (score == hit-rate) snaps 0 - 0 (score == earliest seq num) titers 2 filter - - 0.0% (score == utility) @@ -115,7 +115,7 @@ compact 1 0 B 0 B 0 (size == esti zmemtbl 1 256 K ztbl 2 1.5 K bcache 8 1.4 K 42.9% (score == hit-rate) - tcache 2 1.4 K 66.7% (score == hit-rate) + tcache 2 1.5 K 66.7% (score == hit-rate) snaps 0 - 0 (score == earliest seq num) titers 2 filter - - 0.0% (score == utility) @@ -145,7 +145,7 @@ compact 1 0 B 0 B 0 (size == esti zmemtbl 1 256 K ztbl 1 770 B bcache 4 697 B 42.9% (score == hit-rate) - tcache 1 728 B 66.7% (score == hit-rate) + tcache 1 744 B 66.7% (score == hit-rate) snaps 0 - 0 (score == earliest seq num) titers 1 filter - - 0.0% (score == utility) @@ -369,7 +369,7 @@ compact 2 5.7 K 0 B 0 (size == esti zmemtbl 0 0 B ztbl 0 0 B bcache 16 2.9 K 34.4% (score == hit-rate) - tcache 3 2.1 K 57.9% (score == hit-rate) + tcache 3 2.2 K 57.9% (score == hit-rate) snaps 0 - 0 (score == earliest seq num) titers 0 filter - - 0.0% (score == utility) diff --git a/testdata/table_stats b/testdata/table_stats index c28bb3972e..d370ee88f2 100644 --- a/testdata/table_stats +++ b/testdata/table_stats @@ -15,7 +15,7 @@ wait-pending-table-stats num-entries: 3 num-deletions: 1 num-range-key-sets: 0 -point-deletions-bytes-estimate: 517 +point-deletions-bytes-estimate: 76 range-deletions-bytes-estimate: 0 compact a-c @@ -488,3 +488,59 @@ num-deletions: 1 num-range-key-sets: 0 point-deletions-bytes-estimate: 0 range-deletions-bytes-estimate: 26 + +# Test point tombstone compensation that uses DELSIZED keys. + +define format-major-version=15 +L6 + bar.SET.0: + bax.SET.0: + foo.SET.0: + moo.SET.0: +---- +6: + 000004:[bar#0,SET-moo#0,SET] + +batch +set a apple +set b banana +set c coconut +del-sized foo 100000 +del moo +---- + +flush +---- +0.0: + 000006:[a#10,SET-moo#14,DEL] +6: + 000004:[bar#0,SET-moo#0,SET] + +# The foo DELSIZED tombstone should cause the +# `pebble.raw.point-tombstone.value.size` property to be 100000 + len(foo) = +# 100003. + +properties file=000006 +num.deletions +deleted.keys +raw.point-tombstone +---- +num.deletions: + pebble.num.deletions.sized: 1 +deleted.keys: + rocksdb.deleted.keys: 2 +raw.point-tombstone: + pebble.raw.point-tombstone.key.size: 6 + pebble.raw.point-tombstone.value.size: 100003 + +# And the size hint should then appear in the point-deletions-bytes-estimate, +# scaled according to the computed 'compression ratio'. + +wait-pending-table-stats +000006 +---- +num-entries: 5 +num-deletions: 2 +num-range-key-sets: 0 +point-deletions-bytes-estimate: 112974 +range-deletions-bytes-estimate: 0 diff --git a/tool/db.go b/tool/db.go index 849b319916..4047097b36 100644 --- a/tool/db.go +++ b/tool/db.go @@ -582,6 +582,10 @@ func (d *dbT) runProperties(cmd *cobra.Command, args []string) { propArgs(all, func(p *props) interface{} { return humanize.Uint64(p.SnapshotPinnedKeySize) })...) fmt.Fprintf(tw, " pinned-value\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\n", propArgs(all, func(p *props) interface{} { return humanize.Uint64(p.SnapshotPinnedValueSize) })...) + fmt.Fprintf(tw, " point-del-key-size\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\n", + propArgs(all, func(p *props) interface{} { return humanize.Uint64(p.RawPointTombstoneKeySize) })...) + fmt.Fprintf(tw, " point-del-value-size\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\n", + propArgs(all, func(p *props) interface{} { return humanize.Uint64(p.RawPointTombstoneValueSize) })...) fmt.Fprintln(tw, "records\t\t\t\t\t\t\t\t") fmt.Fprintf(tw, " set\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\n", @@ -589,7 +593,9 @@ func (d *dbT) runProperties(cmd *cobra.Command, args []string) { return humanize.SI.Uint64(p.NumEntries - p.NumDeletions - p.NumMergeOperands) })...) fmt.Fprintf(tw, " delete\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\n", - propArgs(all, func(p *props) interface{} { return humanize.SI.Uint64(p.NumDeletions) })...) + propArgs(all, func(p *props) interface{} { return humanize.SI.Uint64(p.NumDeletions - p.NumRangeDeletions) })...) + fmt.Fprintf(tw, " delete-sized\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\n", + propArgs(all, func(p *props) interface{} { return humanize.SI.Uint64(p.NumSizedDeletions) })...) fmt.Fprintf(tw, " range-delete\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\n", propArgs(all, func(p *props) interface{} { return humanize.SI.Uint64(p.NumRangeDeletions) })...) fmt.Fprintf(tw, " range-key-sets\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\n", @@ -645,27 +651,30 @@ func propArgs(props []props, getProp func(*props) interface{}) []interface{} { } type props struct { - Count uint64 - SmallestSeqNum uint64 - LargestSeqNum uint64 - DataSize uint64 - FilterSize uint64 - IndexSize uint64 - NumDataBlocks uint64 - NumIndexBlocks uint64 - NumDeletions uint64 - NumEntries uint64 - NumMergeOperands uint64 - NumRangeDeletions uint64 - NumRangeKeySets uint64 - NumRangeKeyUnSets uint64 - NumRangeKeyDeletes uint64 - RawKeySize uint64 - RawValueSize uint64 - SnapshotPinnedKeys uint64 - SnapshotPinnedKeySize uint64 - SnapshotPinnedValueSize uint64 - TopLevelIndexSize uint64 + Count uint64 + SmallestSeqNum uint64 + LargestSeqNum uint64 + DataSize uint64 + FilterSize uint64 + IndexSize uint64 + NumDataBlocks uint64 + NumIndexBlocks uint64 + NumDeletions uint64 + NumSizedDeletions uint64 + NumEntries uint64 + NumMergeOperands uint64 + NumRangeDeletions uint64 + NumRangeKeySets uint64 + NumRangeKeyUnSets uint64 + NumRangeKeyDeletes uint64 + RawKeySize uint64 + RawPointTombstoneKeySize uint64 + RawPointTombstoneValueSize uint64 + RawValueSize uint64 + SnapshotPinnedKeys uint64 + SnapshotPinnedKeySize uint64 + SnapshotPinnedValueSize uint64 + TopLevelIndexSize uint64 } func (p *props) update(o props) { @@ -682,6 +691,7 @@ func (p *props) update(o props) { p.NumDataBlocks += o.NumDataBlocks p.NumIndexBlocks += o.NumIndexBlocks p.NumDeletions += o.NumDeletions + p.NumSizedDeletions += o.NumSizedDeletions p.NumEntries += o.NumEntries p.NumMergeOperands += o.NumMergeOperands p.NumRangeDeletions += o.NumRangeDeletions @@ -689,6 +699,8 @@ func (p *props) update(o props) { p.NumRangeKeyUnSets += o.NumRangeKeyUnSets p.NumRangeKeyDeletes += o.NumRangeKeyDeletes p.RawKeySize += o.RawKeySize + p.RawPointTombstoneKeySize += o.RawPointTombstoneKeySize + p.RawPointTombstoneValueSize += o.RawPointTombstoneValueSize p.RawValueSize += o.RawValueSize p.SnapshotPinnedKeySize += o.SnapshotPinnedKeySize p.SnapshotPinnedValueSize += o.SnapshotPinnedValueSize @@ -710,27 +722,30 @@ func (d *dbT) addProps( return err } p.update(props{ - Count: 1, - SmallestSeqNum: m.SmallestSeqNum, - LargestSeqNum: m.LargestSeqNum, - DataSize: r.Properties.DataSize, - FilterSize: r.Properties.FilterSize, - IndexSize: r.Properties.IndexSize, - NumDataBlocks: r.Properties.NumDataBlocks, - NumIndexBlocks: 1 + r.Properties.IndexPartitions, - NumDeletions: r.Properties.NumDeletions, - NumEntries: r.Properties.NumEntries, - NumMergeOperands: r.Properties.NumMergeOperands, - NumRangeDeletions: r.Properties.NumRangeDeletions, - NumRangeKeySets: r.Properties.NumRangeKeySets, - NumRangeKeyUnSets: r.Properties.NumRangeKeyUnsets, - NumRangeKeyDeletes: r.Properties.NumRangeKeyDels, - RawKeySize: r.Properties.RawKeySize, - RawValueSize: r.Properties.RawValueSize, - SnapshotPinnedKeySize: r.Properties.SnapshotPinnedKeySize, - SnapshotPinnedValueSize: r.Properties.SnapshotPinnedValueSize, - SnapshotPinnedKeys: r.Properties.SnapshotPinnedKeys, - TopLevelIndexSize: r.Properties.TopLevelIndexSize, + Count: 1, + SmallestSeqNum: m.SmallestSeqNum, + LargestSeqNum: m.LargestSeqNum, + DataSize: r.Properties.DataSize, + FilterSize: r.Properties.FilterSize, + IndexSize: r.Properties.IndexSize, + NumDataBlocks: r.Properties.NumDataBlocks, + NumIndexBlocks: 1 + r.Properties.IndexPartitions, + NumDeletions: r.Properties.NumDeletions, + NumSizedDeletions: r.Properties.NumSizedDeletions, + NumEntries: r.Properties.NumEntries, + NumMergeOperands: r.Properties.NumMergeOperands, + NumRangeDeletions: r.Properties.NumRangeDeletions, + NumRangeKeySets: r.Properties.NumRangeKeySets, + NumRangeKeyUnSets: r.Properties.NumRangeKeyUnsets, + NumRangeKeyDeletes: r.Properties.NumRangeKeyDels, + RawKeySize: r.Properties.RawKeySize, + RawPointTombstoneKeySize: r.Properties.RawPointTombstoneKeySize, + RawPointTombstoneValueSize: r.Properties.RawPointTombstoneValueSize, + RawValueSize: r.Properties.RawValueSize, + SnapshotPinnedKeySize: r.Properties.SnapshotPinnedKeySize, + SnapshotPinnedValueSize: r.Properties.SnapshotPinnedValueSize, + SnapshotPinnedKeys: r.Properties.SnapshotPinnedKeys, + TopLevelIndexSize: r.Properties.TopLevelIndexSize, }) return r.Close() } diff --git a/tool/find.go b/tool/find.go index 0980cd9e72..a3e1d83b9d 100644 --- a/tool/find.go +++ b/tool/find.go @@ -353,6 +353,7 @@ func (f *findT) searchLogs(stdout io.Writer, searchKey []byte, refs []findRef) [ ikey := base.MakeInternalKey(ukey, seqNum, kind) switch kind { case base.InternalKeyKindDelete, + base.InternalKeyKindDeleteSized, base.InternalKeyKindSet, base.InternalKeyKindMerge, base.InternalKeyKindSingleDelete, diff --git a/tool/sstable.go b/tool/sstable.go index a7efd2b40a..e4f9c02263 100644 --- a/tool/sstable.go +++ b/tool/sstable.go @@ -320,10 +320,13 @@ func (s *sstableT) runProperties(cmd *cobra.Command, args []string) { fmt.Fprintf(tw, " raw-value\t%s\n", humanize.Uint64(r.Properties.RawValueSize)) fmt.Fprintf(tw, " pinned-key\t%d\n", r.Properties.SnapshotPinnedKeySize) fmt.Fprintf(tw, " pinned-val\t%d\n", r.Properties.SnapshotPinnedValueSize) + fmt.Fprintf(tw, " point-del-key-size\t%d\n", r.Properties.RawPointTombstoneKeySize) + fmt.Fprintf(tw, " point-del-value-size\t%d\n", r.Properties.RawPointTombstoneValueSize) fmt.Fprintf(tw, "records\t%d\n", r.Properties.NumEntries) fmt.Fprintf(tw, " set\t%d\n", r.Properties.NumEntries- (r.Properties.NumDeletions+r.Properties.NumMergeOperands)) fmt.Fprintf(tw, " delete\t%d\n", r.Properties.NumPointDeletions()) + fmt.Fprintf(tw, " delete-sized\t%d\n", r.Properties.NumSizedDeletions) fmt.Fprintf(tw, " range-delete\t%d\n", r.Properties.NumRangeDeletions) fmt.Fprintf(tw, " range-key-set\t%d\n", r.Properties.NumRangeKeySets) fmt.Fprintf(tw, " range-key-unset\t%d\n", r.Properties.NumRangeKeyUnsets) diff --git a/tool/testdata/db_properties b/tool/testdata/db_properties index 63c753cac5..cc5625fb30 100644 --- a/tool/testdata/db_properties +++ b/tool/testdata/db_properties @@ -10,57 +10,63 @@ open non-existent/: file does not exist db properties ../testdata/db-stage-4 ---- - L0 L1 L2 L3 L4 L5 L6 TOTAL -count 1 0 0 0 0 0 0 1 -seq num - smallest 3 0 0 0 0 0 0 3 - largest 5 0 0 0 0 0 0 5 -size - data 62 B 0 B 0 B 0 B 0 B 0 B 0 B 62 B - blocks 1 0 0 0 0 0 0 1 - index 27 B 0 B 0 B 0 B 0 B 0 B 0 B 27 B - blocks 1 0 0 0 0 0 0 1 - top-level 0 B 0 B 0 B 0 B 0 B 0 B 0 B 0 B - filter 0 B 0 B 0 B 0 B 0 B 0 B 0 B 0 B - raw-key 33 B 0 B 0 B 0 B 0 B 0 B 0 B 33 B - raw-value 9 B 0 B 0 B 0 B 0 B 0 B 0 B 9 B - pinned-key 0 B 0 B 0 B 0 B 0 B 0 B 0 B 0 B - pinned-value 0 B 0 B 0 B 0 B 0 B 0 B 0 B 0 B -records - set 2 0 0 0 0 0 0 2 - delete 1 0 0 0 0 0 0 1 - range-delete 0 0 0 0 0 0 0 0 - range-key-sets 0 0 0 0 0 0 0 0 - range-key-unsets 0 0 0 0 0 0 0 0 - range-key-deletes 0 0 0 0 0 0 0 0 - merge 0 0 0 0 0 0 0 0 - pinned 0 0 0 0 0 0 0 0 + L0 L1 L2 L3 L4 L5 L6 TOTAL +count 1 0 0 0 0 0 0 1 +seq num + smallest 3 0 0 0 0 0 0 3 + largest 5 0 0 0 0 0 0 5 +size + data 62 B 0 B 0 B 0 B 0 B 0 B 0 B 62 B + blocks 1 0 0 0 0 0 0 1 + index 27 B 0 B 0 B 0 B 0 B 0 B 0 B 27 B + blocks 1 0 0 0 0 0 0 1 + top-level 0 B 0 B 0 B 0 B 0 B 0 B 0 B 0 B + filter 0 B 0 B 0 B 0 B 0 B 0 B 0 B 0 B + raw-key 33 B 0 B 0 B 0 B 0 B 0 B 0 B 33 B + raw-value 9 B 0 B 0 B 0 B 0 B 0 B 0 B 9 B + pinned-key 0 B 0 B 0 B 0 B 0 B 0 B 0 B 0 B + pinned-value 0 B 0 B 0 B 0 B 0 B 0 B 0 B 0 B + point-del-key-size 0 B 0 B 0 B 0 B 0 B 0 B 0 B 0 B + point-del-value-size 0 B 0 B 0 B 0 B 0 B 0 B 0 B 0 B +records + set 2 0 0 0 0 0 0 2 + delete 1 0 0 0 0 0 0 1 + delete-sized 0 0 0 0 0 0 0 0 + range-delete 0 0 0 0 0 0 0 0 + range-key-sets 0 0 0 0 0 0 0 0 + range-key-unsets 0 0 0 0 0 0 0 0 + range-key-deletes 0 0 0 0 0 0 0 0 + merge 0 0 0 0 0 0 0 0 + pinned 0 0 0 0 0 0 0 0 db properties ./testdata/mixed ---- - L0 L1 L2 L3 L4 L5 L6 TOTAL -count 1 0 0 0 0 0 0 1 -seq num - smallest 1 0 0 0 0 0 0 1 - largest 29 0 0 0 0 0 0 29 -size - data 236 B 0 B 0 B 0 B 0 B 0 B 0 B 236 B - blocks 1 0 0 0 0 0 0 1 - index 29 B 0 B 0 B 0 B 0 B 0 B 0 B 29 B - blocks 1 0 0 0 0 0 0 1 - top-level 0 B 0 B 0 B 0 B 0 B 0 B 0 B 0 B - filter 0 B 0 B 0 B 0 B 0 B 0 B 0 B 0 B - raw-key 286 B 0 B 0 B 0 B 0 B 0 B 0 B 286 B - raw-value 0 B 0 B 0 B 0 B 0 B 0 B 0 B 0 B - pinned-key 0 B 0 B 0 B 0 B 0 B 0 B 0 B 0 B - pinned-value 0 B 0 B 0 B 0 B 0 B 0 B 0 B 0 B -records - set 26 0 0 0 0 0 0 26 - delete 0 0 0 0 0 0 0 0 - range-delete 0 0 0 0 0 0 0 0 - range-key-sets 1 0 0 0 0 0 0 1 - range-key-unsets 1 0 0 0 0 0 0 1 - range-key-deletes 1 0 0 0 0 0 0 1 - merge 0 0 0 0 0 0 0 0 - pinned 0 0 0 0 0 0 0 0 + L0 L1 L2 L3 L4 L5 L6 TOTAL +count 1 0 0 0 0 0 0 1 +seq num + smallest 1 0 0 0 0 0 0 1 + largest 29 0 0 0 0 0 0 29 +size + data 236 B 0 B 0 B 0 B 0 B 0 B 0 B 236 B + blocks 1 0 0 0 0 0 0 1 + index 29 B 0 B 0 B 0 B 0 B 0 B 0 B 29 B + blocks 1 0 0 0 0 0 0 1 + top-level 0 B 0 B 0 B 0 B 0 B 0 B 0 B 0 B + filter 0 B 0 B 0 B 0 B 0 B 0 B 0 B 0 B + raw-key 286 B 0 B 0 B 0 B 0 B 0 B 0 B 286 B + raw-value 0 B 0 B 0 B 0 B 0 B 0 B 0 B 0 B + pinned-key 0 B 0 B 0 B 0 B 0 B 0 B 0 B 0 B + pinned-value 0 B 0 B 0 B 0 B 0 B 0 B 0 B 0 B + point-del-key-size 0 B 0 B 0 B 0 B 0 B 0 B 0 B 0 B + point-del-value-size 0 B 0 B 0 B 0 B 0 B 0 B 0 B 0 B +records + set 26 0 0 0 0 0 0 26 + delete 0 0 0 0 0 0 0 0 + delete-sized 0 0 0 0 0 0 0 0 + range-delete 0 0 0 0 0 0 0 0 + range-key-sets 1 0 0 0 0 0 0 1 + range-key-unsets 1 0 0 0 0 0 0 1 + range-key-deletes 1 0 0 0 0 0 0 1 + merge 0 0 0 0 0 0 0 0 + pinned 0 0 0 0 0 0 0 0 diff --git a/tool/testdata/sstable_properties b/tool/testdata/sstable_properties index a5b1841a35..2ba0fe541f 100644 --- a/tool/testdata/sstable_properties +++ b/tool/testdata/sstable_properties @@ -6,123 +6,132 @@ sstable properties ../sstable/testdata/h.sst ---- h.sst -version 0 -size - file 15 K - data 14 K - blocks 14 - index 325 B - blocks 1 - top-level 0 B - filter 0 B - raw-key 23 K - raw-value 1.9 K - pinned-key 0 - pinned-val 0 -records 1727 - set 1710 - delete 0 - range-delete 17 - range-key-set 0 - range-key-unset 0 - range-key-delete 0 - merge 0 - global-seq-num 0 - pinned 0 -index - key internal key - value raw encoded -comparer leveldb.BytewiseComparator -merger - -filter - - prefix false - whole-key false -compression Snappy - options window_bits=-14; level=32767; strategy=0; max_dict_bytes=0; zstd_max_train_bytes=0; enabled=0; -user properties - collectors [KeyCountPropertyCollector] - test.key-count 1727 +version 0 +size + file 15 K + data 14 K + blocks 14 + index 325 B + blocks 1 + top-level 0 B + filter 0 B + raw-key 23 K + raw-value 1.9 K + pinned-key 0 + pinned-val 0 + point-del-key-size 0 + point-del-value-size 0 +records 1727 + set 1710 + delete 0 + delete-sized 0 + range-delete 17 + range-key-set 0 + range-key-unset 0 + range-key-delete 0 + merge 0 + global-seq-num 0 + pinned 0 +index + key internal key + value raw encoded +comparer leveldb.BytewiseComparator +merger - +filter - + prefix false + whole-key false +compression Snappy + options window_bits=-14; level=32767; strategy=0; max_dict_bytes=0; zstd_max_train_bytes=0; enabled=0; +user properties + collectors [KeyCountPropertyCollector] + test.key-count 1727 sstable properties ../sstable/testdata/h.ldb ---- h.ldb -version 0 -size - file 15 K - data 14 K - blocks 14 - index 325 B - blocks 1 - top-level 0 B - filter 0 B - raw-key 23 K - raw-value 1.9 K - pinned-key 0 - pinned-val 0 -records 1727 - set 1710 - delete 0 - range-delete 17 - range-key-set 0 - range-key-unset 0 - range-key-delete 0 - merge 0 - global-seq-num 0 - pinned 0 -index - key internal key - value raw encoded -comparer leveldb.BytewiseComparator -merger - -filter - - prefix false - whole-key false -compression Snappy - options window_bits=-14; level=32767; strategy=0; max_dict_bytes=0; zstd_max_train_bytes=0; enabled=0; -user properties - collectors [] +version 0 +size + file 15 K + data 14 K + blocks 14 + index 325 B + blocks 1 + top-level 0 B + filter 0 B + raw-key 23 K + raw-value 1.9 K + pinned-key 0 + pinned-val 0 + point-del-key-size 0 + point-del-value-size 0 +records 1727 + set 1710 + delete 0 + delete-sized 0 + range-delete 17 + range-key-set 0 + range-key-unset 0 + range-key-delete 0 + merge 0 + global-seq-num 0 + pinned 0 +index + key internal key + value raw encoded +comparer leveldb.BytewiseComparator +merger - +filter - + prefix false + whole-key false +compression Snappy + options window_bits=-14; level=32767; strategy=0; max_dict_bytes=0; zstd_max_train_bytes=0; enabled=0; +user properties + collectors [] sstable properties ../sstable/testdata/h.no-compression.two_level_index.sst ---- h.no-compression.two_level_index.sst -version 0 -size - file 28 K - data 26 K - blocks 14 - index 408 B - blocks 4 - top-level 70 B - filter 0 B - raw-key 23 K - raw-value 1.9 K - pinned-key 0 - pinned-val 0 -records 1727 - set 1710 - delete 0 - range-delete 17 - range-key-set 0 - range-key-unset 0 - range-key-delete 0 - merge 0 - global-seq-num 0 - pinned 0 -index - key internal key - value raw encoded -comparer leveldb.BytewiseComparator -merger - -filter - - prefix false - whole-key false -compression NoCompression - options window_bits=-14; level=32767; strategy=0; max_dict_bytes=0; zstd_max_train_bytes=0; enabled=0; -user properties - collectors [KeyCountPropertyCollector] - test.key-count 1727 +version 0 +size + file 28 K + data 26 K + blocks 14 + index 408 B + blocks 4 + top-level 70 B + filter 0 B + raw-key 23 K + raw-value 1.9 K + pinned-key 0 + pinned-val 0 + point-del-key-size 0 + point-del-value-size 0 +records 1727 + set 1710 + delete 0 + delete-sized 0 + range-delete 17 + range-key-set 0 + range-key-unset 0 + range-key-delete 0 + merge 0 + global-seq-num 0 + pinned 0 +index + key internal key + value raw encoded +comparer leveldb.BytewiseComparator +merger - +filter - + prefix false + whole-key false +compression NoCompression + options window_bits=-14; level=32767; strategy=0; max_dict_bytes=0; zstd_max_train_bytes=0; enabled=0; +user properties + collectors [KeyCountPropertyCollector] + test.key-count 1727 sstable properties -v @@ -206,38 +215,41 @@ sstable properties testdata/mixed/000005.sst ---- 000005.sst -version 0 -size - file 1.1 K - data 236 B - blocks 1 - index 29 B - blocks 1 - top-level 0 B - filter 0 B - raw-key 286 B - raw-value 0 B - pinned-key 0 - pinned-val 0 -records 26 - set 26 - delete 0 - range-delete 0 - range-key-set 1 - range-key-unset 1 - range-key-delete 1 - merge 0 - global-seq-num 0 - pinned 0 -index - key internal key - value raw encoded -comparer pebble.internal.testkeys -merger pebble.concatenate -filter - - prefix false - whole-key false -compression Snappy - options window_bits=-14; level=32767; strategy=0; max_dict_bytes=0; zstd_max_train_bytes=0; enabled=0; -user properties - collectors [] +version 0 +size + file 1.1 K + data 236 B + blocks 1 + index 29 B + blocks 1 + top-level 0 B + filter 0 B + raw-key 286 B + raw-value 0 B + pinned-key 0 + pinned-val 0 + point-del-key-size 0 + point-del-value-size 0 +records 26 + set 26 + delete 0 + delete-sized 0 + range-delete 0 + range-key-set 1 + range-key-unset 1 + range-key-delete 1 + merge 0 + global-seq-num 0 + pinned 0 +index + key internal key + value raw encoded +comparer pebble.internal.testkeys +merger pebble.concatenate +filter - + prefix false + whole-key false +compression Snappy + options window_bits=-14; level=32767; strategy=0; max_dict_bytes=0; zstd_max_train_bytes=0; enabled=0; +user properties + collectors [] diff --git a/tool/wal.go b/tool/wal.go index 4d460217b8..66fcedb8d1 100644 --- a/tool/wal.go +++ b/tool/wal.go @@ -156,6 +156,9 @@ func (w *walT) runDump(cmd *cobra.Command, args []string) { } else { fmt.Fprintf(stdout, "%s", s.Pretty(w.fmtKey.fn)) } + case base.InternalKeyKindDeleteSized: + v, _ := binary.Uvarint(value) + fmt.Fprintf(stdout, "%s,%d", w.fmtKey.fn(ukey), v) } fmt.Fprintf(stdout, ")\n") }