Skip to content

Commit

Permalink
sstable: drop support for v1 range deletion blocks
Browse files Browse the repository at this point in the history
The v1 range deletion blocks predate Pebble. Our format major versions that
required compaction of all pre-Pebble sstables ensure that no v1 range deletion
blocks should exist in the wild without an accompanying v2 range deletion
block. This required regeneration of the sstable fixtures.
  • Loading branch information
jbowens committed Jun 27, 2024
1 parent 870777c commit 89f4160
Show file tree
Hide file tree
Showing 21 changed files with 94 additions and 297 deletions.
8 changes: 0 additions & 8 deletions sstable/data_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -197,14 +197,6 @@ func runBuildRawCmd(
}

w := NewWriter(f0, *opts)
for i := range td.CmdArgs {
arg := &td.CmdArgs[i]
if arg.Key == "range-del-v1" {
w.rangeDelV1Format = true
break
}
}

for _, data := range strings.Split(td.Input, "\n") {
if strings.HasPrefix(data, "rangekey:") {
data = strings.TrimPrefix(data, "rangekey:")
Expand Down
96 changes: 28 additions & 68 deletions sstable/reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -176,25 +176,24 @@ func init() {

// Reader is a table reader.
type Reader struct {
readable objstorage.Readable
cacheID uint64
fileNum base.DiskFileNum
err error
indexBH block.Handle
filterBH block.Handle
rangeDelBH block.Handle
rangeKeyBH block.Handle
rangeDelTransform blockTransform
valueBIH valueBlocksIndexHandle
propertiesBH block.Handle
metaIndexBH block.Handle
footerBH block.Handle
opts ReaderOptions
Compare Compare
Equal Equal
FormatKey base.FormatKey
Split Split
tableFilter *tableFilterReader
readable objstorage.Readable
cacheID uint64
fileNum base.DiskFileNum
err error
indexBH block.Handle
filterBH block.Handle
rangeDelBH block.Handle
rangeKeyBH block.Handle
valueBIH valueBlocksIndexHandle
propertiesBH block.Handle
metaIndexBH block.Handle
footerBH block.Handle
opts ReaderOptions
Compare Compare
Equal Equal
FormatKey base.FormatKey
Split Split
tableFilter *tableFilterReader
// Keep types that are not multiples of 8 bytes at the end and with
// decreasing size.
Properties Properties
Expand Down Expand Up @@ -452,7 +451,7 @@ func (r *Reader) readRangeDel(
stats *base.InternalIteratorStats, iterStats *iterStatsAccumulator,
) (block.BufferHandle, error) {
ctx := objiotracing.WithBlockType(context.Background(), objiotracing.MetadataBlock)
return r.readBlock(ctx, r.rangeDelBH, r.rangeDelTransform, nil /* readHandle */, stats, iterStats, nil /* buffer pool */)
return r.readBlock(ctx, r.rangeDelBH, nil /* transform */, nil /* readHandle */, stats, iterStats, nil /* buffer pool */)
}

func (r *Reader) readRangeKey(
Expand Down Expand Up @@ -611,49 +610,6 @@ func (r *Reader) readBlock(
return h, nil
}

func (r *Reader) transformRangeDelV1(b []byte) ([]byte, error) {
// Convert v1 (RocksDB format) range-del blocks to v2 blocks on the fly. The
// v1 format range-del blocks have unfragmented and unsorted range
// tombstones. We need properly fragmented and sorted range tombstones in
// order to serve from them directly.
iter := &rowblk.Iter{}
if err := iter.Init(r.Compare, r.Split, b, NoTransforms); err != nil {
return nil, err
}
var tombstones []keyspan.Span
for kv := iter.First(); kv != nil; kv = iter.Next() {
t := keyspan.Span{
Start: kv.K.UserKey,
End: kv.InPlaceValue(),
Keys: []keyspan.Key{{Trailer: kv.K.Trailer}},
}
tombstones = append(tombstones, t)
}
keyspan.Sort(r.Compare, tombstones)

// Fragment the tombstones, outputting them directly to a block writer.
rangeDelBlock := rowblk.Writer{
RestartInterval: 1,
}
frag := keyspan.Fragmenter{
Cmp: r.Compare,
Format: r.FormatKey,
Emit: func(s keyspan.Span) {
for _, k := range s.Keys {
startIK := InternalKey{UserKey: s.Start, Trailer: k.Trailer}
rangeDelBlock.Add(startIK, s.End)
}
},
}
for i := range tombstones {
frag.Add(tombstones[i])
}
frag.Finish()

// Return the contents of the constructed v2 format range-del block.
return rangeDelBlock.Finish(), nil
}

func (r *Reader) readMetaindex(metaindexBH block.Handle, readHandle objstorage.ReadHandle) error {
// We use a BufferPool when reading metaindex blocks in order to avoid
// populating the block cache with these blocks. In heavy-write workloads,
Expand Down Expand Up @@ -728,11 +684,15 @@ func (r *Reader) readMetaindex(metaindexBH block.Handle, readHandle objstorage.R

if bh, ok := meta[metaRangeDelV2Name]; ok {
r.rangeDelBH = bh
} else if bh, ok := meta[metaRangeDelName]; ok {
r.rangeDelBH = bh
if !r.rawTombstones {
r.rangeDelTransform = r.transformRangeDelV1
}
} else if _, ok := meta[metaRangeDelV1Name]; ok {
// This version of Pebble requires a format major version at least as
// high as FormatFlushableIngest (see pebble.FormatMinSupported). In
// this format major verison, we have a guarantee that we've compacted
// away all RocksDB sstables. It should not be possible to encounter an
// sstable with a v1 range deletion block but not a v2 range deletion
// block.
err := errors.Newf("pebble/table: unexpected range-del block type: %s", metaRangeDelV1Name)
return errors.Mark(err, base.ErrCorruption)
}

if bh, ok := meta[metaRangeKeyName]; ok {
Expand Down
2 changes: 1 addition & 1 deletion sstable/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -212,7 +212,7 @@ const (
metaRangeKeyName = "pebble.range_key"
metaValueIndexName = "pebble.value_index"
metaPropertiesName = "rocksdb.properties"
metaRangeDelName = "rocksdb.range_del"
metaRangeDelV1Name = "rocksdb.range_del"
metaRangeDelV2Name = "rocksdb.range_del2"

// Index Types.
Expand Down
3 changes: 1 addition & 2 deletions sstable/test_fixtures.go
Original file line number Diff line number Diff line change
Expand Up @@ -147,8 +147,7 @@ func buildHamletTestSST(
}

w := NewWriter(objstorageprovider.NewFileWritable(f0), writerOpts)
// Use rangeDelV1Format for testing byte equality with RocksDB.
w.rangeDelV1Format = true
// NB: We don't have byte equality with RocksDB any longer.
var rangeDelLength int
var rangeDelCounter int
var rangeDelStart InternalKey
Expand Down
Binary file modified sstable/testdata/h.no-compression.sst
Binary file not shown.
Binary file modified sstable/testdata/h.no-compression.two_level_index.sst
Binary file not shown.
Binary file modified sstable/testdata/h.sst
Binary file not shown.
Binary file not shown.
Binary file modified sstable/testdata/h.table-bloom.no-compression.sst
Binary file not shown.
Binary file modified sstable/testdata/h.table-bloom.sst
Binary file not shown.
Binary file modified sstable/testdata/h.zstd-compression.sst
Binary file not shown.
59 changes: 0 additions & 59 deletions sstable/testdata/writer
Original file line number Diff line number Diff line change
Expand Up @@ -238,65 +238,6 @@ rangekey: a-b:{(#1,RANGEKEYSET,@t10,foo) (#1,RANGEKEYUNSET,@t10) (#1,RANGEKEYDEL
rangekey: [a#1,RANGEKEYSET-b#inf,RANGEKEYDEL]
seqnums: [1-1]

# The range-del-v1 format supports unfragmented and unsorted range
# tombstones.

build-raw range-del-v1
a.RANGEDEL.1:c
a.RANGEDEL.2:c
----
rangedel: [a#2,RANGEDEL-c#inf,RANGEDEL]
seqnums: [1-2]

scan-range-del
----
a-c:{(#2,RANGEDEL) (#1,RANGEDEL)}

build-raw range-del-v1
a.RANGEDEL.1:c
b.RANGEDEL.2:d
----
rangedel: [a#1,RANGEDEL-d#inf,RANGEDEL]
seqnums: [1-2]

scan-range-del
----
a-b:{(#1,RANGEDEL)}
b-c:{(#2,RANGEDEL) (#1,RANGEDEL)}
c-d:{(#2,RANGEDEL)}

build-raw range-del-v1
a.RANGEDEL.2:c
a.RANGEDEL.1:d
----
rangedel: [a#2,RANGEDEL-d#inf,RANGEDEL]
seqnums: [1-2]

scan-range-del
----
a-c:{(#2,RANGEDEL) (#1,RANGEDEL)}
c-d:{(#1,RANGEDEL)}

# This matches an early test case, except we're passing overlapping
# range tombstones to the sstable writer and requiring them to be
# fragmented at read time.

build-raw range-del-v1
j.RANGEDEL.1:z
f.RANGEDEL.2:s
a.RANGEDEL.3:m
----
rangedel: [a#3,RANGEDEL-z#inf,RANGEDEL]
seqnums: [1-3]

scan-range-del
----
a-f:{(#3,RANGEDEL)}
f-j:{(#3,RANGEDEL) (#2,RANGEDEL)}
j-m:{(#3,RANGEDEL) (#2,RANGEDEL) (#1,RANGEDEL)}
m-s:{(#2,RANGEDEL) (#1,RANGEDEL)}
s-z:{(#1,RANGEDEL)}

# Setting a very small index-block-size results in a two-level index.

build block-size=1 index-block-size=1
Expand Down
59 changes: 0 additions & 59 deletions sstable/testdata/writer_v3
Original file line number Diff line number Diff line change
Expand Up @@ -211,65 +211,6 @@ rangekey: a-b:{(#1,RANGEKEYSET,@t10,foo) (#1,RANGEKEYUNSET,@t10) (#1,RANGEKEYDEL
rangekey: [a#1,RANGEKEYSET-b#inf,RANGEKEYDEL]
seqnums: [1-1]

# The range-del-v1 format supports unfragmented and unsorted range
# tombstones.

build-raw range-del-v1
a.RANGEDEL.1:c
a.RANGEDEL.2:c
----
rangedel: [a#2,RANGEDEL-c#inf,RANGEDEL]
seqnums: [1-2]

scan-range-del
----
a-c:{(#2,RANGEDEL) (#1,RANGEDEL)}

build-raw range-del-v1
a.RANGEDEL.1:c
b.RANGEDEL.2:d
----
rangedel: [a#1,RANGEDEL-d#inf,RANGEDEL]
seqnums: [1-2]

scan-range-del
----
a-b:{(#1,RANGEDEL)}
b-c:{(#2,RANGEDEL) (#1,RANGEDEL)}
c-d:{(#2,RANGEDEL)}

build-raw range-del-v1
a.RANGEDEL.2:c
a.RANGEDEL.1:d
----
rangedel: [a#2,RANGEDEL-d#inf,RANGEDEL]
seqnums: [1-2]

scan-range-del
----
a-c:{(#2,RANGEDEL) (#1,RANGEDEL)}
c-d:{(#1,RANGEDEL)}

# This matches an early test case, except we're passing overlapping
# range tombstones to the sstable writer and requiring them to be
# fragmented at read time.

build-raw range-del-v1
j.RANGEDEL.1:z
f.RANGEDEL.2:s
a.RANGEDEL.3:m
----
rangedel: [a#3,RANGEDEL-z#inf,RANGEDEL]
seqnums: [1-3]

scan-range-del
----
a-f:{(#3,RANGEDEL)}
f-j:{(#3,RANGEDEL) (#2,RANGEDEL)}
j-m:{(#3,RANGEDEL) (#2,RANGEDEL) (#1,RANGEDEL)}
m-s:{(#2,RANGEDEL) (#1,RANGEDEL)}
s-z:{(#1,RANGEDEL)}

# Setting a very small index-block-size results in a two-level index.

build block-size=1 index-block-size=1
Expand Down
78 changes: 21 additions & 57 deletions sstable/writer.go
Original file line number Diff line number Diff line change
Expand Up @@ -170,11 +170,7 @@ type Writer struct {
// re-read many times from the disk. The top level index, which has a much
// smaller memory footprint, can be used to prevent the entire index block from
// being loaded into the block cache.
twoLevelIndex bool
// Internal flag to allow creation of range-del-v1 format blocks. Only used
// for testing. Note that v2 format blocks are backwards compatible with v1
// format blocks.
rangeDelV1Format bool
twoLevelIndex bool
indexBlock *indexBlockBuf
rangeDelBlock rowblk.Writer
rangeKeyBlock rowblk.Writer
Expand Down Expand Up @@ -1049,7 +1045,7 @@ func (w *Writer) prettyTombstone(k InternalKey, value []byte) fmt.Formatter {
}

func (w *Writer) addTombstone(key InternalKey, value []byte) error {
if !w.disableKeyOrderChecks && !w.rangeDelV1Format && w.rangeDelBlock.EntryCount() > 0 {
if !w.disableKeyOrderChecks && w.rangeDelBlock.EntryCount() > 0 {
// Check that tombstones are being added in fragmented order. If the two
// tombstones overlap, their start and end keys must be identical.
prevKey := w.rangeDelBlock.CurKey()
Expand Down Expand Up @@ -1089,34 +1085,12 @@ func (w *Writer) addTombstone(key InternalKey, value []byte) error {

w.meta.updateSeqNum(key.SeqNum())

switch {
case w.rangeDelV1Format:
// Range tombstones are not fragmented in the v1 (i.e. RocksDB) range
// deletion block format, so we need to track the largest range tombstone
// end key as every range tombstone is added.
//
// Note that writing the v1 format is only supported for tests.
if w.props.NumRangeDeletions == 0 {
w.meta.SetSmallestRangeDelKey(key.Clone())
w.meta.SetLargestRangeDelKey(base.MakeRangeDeleteSentinelKey(value).Clone())
} else {
if base.InternalCompare(w.compare, w.meta.SmallestRangeDel, key) > 0 {
w.meta.SetSmallestRangeDelKey(key.Clone())
}
end := base.MakeRangeDeleteSentinelKey(value)
if base.InternalCompare(w.compare, w.meta.LargestRangeDel, end) < 0 {
w.meta.SetLargestRangeDelKey(end.Clone())
}
}

default:
// Range tombstones are fragmented in the v2 range deletion block format,
// so the start key of the first range tombstone added will be the smallest
// range tombstone key. The largest range tombstone key will be determined
// in Writer.Close() as the end key of the last range tombstone added.
if w.props.NumRangeDeletions == 0 {
w.meta.SetSmallestRangeDelKey(key.Clone())
}
// Range tombstones are fragmented in the v2 range deletion block format,
// so the start key of the first range tombstone added will be the smallest
// range tombstone key. The largest range tombstone key will be determined
// in Writer.Close() as the end key of the last range tombstone added.
if w.props.NumRangeDeletions == 0 {
w.meta.SetSmallestRangeDelKey(key.Clone())
}

w.props.NumEntries++
Expand Down Expand Up @@ -2050,33 +2024,23 @@ func (w *Writer) Close() (err error) {
// metaindex block must be sorted by key.
var rangeDelBH block.Handle
if w.props.NumRangeDeletions > 0 {
if !w.rangeDelV1Format {
// Because the range tombstones are fragmented in the v2 format, the end
// key of the last added range tombstone will be the largest range
// tombstone key. Note that we need to make this into a range deletion
// sentinel because sstable boundaries are inclusive while the end key of
// a range deletion tombstone is exclusive. A Clone() is necessary as
// rangeDelBlock.curValue is the same slice that will get passed
// into w.writer, and some implementations of vfs.File mutate the
// slice passed into Write(). Also, w.meta will often outlive the
// blockWriter, and so cloning curValue allows the rangeDelBlock's
// internal buffer to get gc'd.
k := base.MakeRangeDeleteSentinelKey(w.rangeDelBlock.CurValue()).Clone()
w.meta.SetLargestRangeDelKey(k)
}
// Because the range tombstones are fragmented, the end key of the last
// added range tombstone will be the largest range tombstone key. Note
// that we need to make this into a range deletion sentinel because
// sstable boundaries are inclusive while the end key of a range
// deletion tombstone is exclusive. A Clone() is necessary as
// rangeDelBlock.curValue is the same slice that will get passed into
// w.writer, and some implementations of vfs.File mutate the slice
// passed into Write(). Also, w.meta will often outlive the blockWriter,
// and so cloning curValue allows the rangeDelBlock's internal buffer to
// get gc'd.
k := base.MakeRangeDeleteSentinelKey(w.rangeDelBlock.CurValue()).Clone()
w.meta.SetLargestRangeDelKey(k)
rangeDelBH, err = w.writeBlock(w.rangeDelBlock.Finish(), NoCompression, &w.blockBuf)
if err != nil {
return err
}
// The v2 range-del block encoding is backwards compatible with the v1
// encoding. We add meta-index entries for both the old name and the new
// name so that old code can continue to find the range-del block and new
// code knows that the range tombstones in the block are fragmented and
// sorted.
metaindex.add(metaRangeDelName, rangeDelBH)
if !w.rangeDelV1Format {
metaindex.add(metaRangeDelV2Name, rangeDelBH)
}
metaindex.add(metaRangeDelV2Name, rangeDelBH)
}

// Write the range-key block, flushing any remaining spans from the
Expand Down
Loading

0 comments on commit 89f4160

Please sign in to comment.