Skip to content

Commit

Permalink
sstable: move PrefixReplacement to sstable
Browse files Browse the repository at this point in the history
We change `MakeVirtualReader` to not use the file metadata types,
resolving the last dependency on `manifest`; and we move the
`PrefixReplacement` struct to `sstable`. Also, we can now use
`sstable.SyntheticSuffix` in `FileMetadata`.
  • Loading branch information
RaduBerinde committed Feb 16, 2024
1 parent ee1fc33 commit 9077044
Show file tree
Hide file tree
Showing 10 changed files with 134 additions and 146 deletions.
2 changes: 1 addition & 1 deletion data_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1182,7 +1182,7 @@ func runSSTablePropertiesCmd(t *testing.T, td *datadriven.TestData, d *DB) strin
var v sstable.VirtualReader
props := r.Properties.String()
if m != nil && m.Virtual {
v = sstable.MakeVirtualReader(r, m.VirtualMeta(), false /* isShared */)
v = sstable.MakeVirtualReader(r, m.VirtualMeta().VirtualReaderParams(false /* isShared */))
props = v.Properties.String()
}
if len(td.Input) == 0 {
Expand Down
2 changes: 1 addition & 1 deletion ingest.go
Original file line number Diff line number Diff line change
Expand Up @@ -226,7 +226,7 @@ func ingestLoad1External(
meta.FileBacking.Size = e.Size

if len(e.SyntheticPrefix) != 0 {
meta.PrefixReplacement = &manifest.PrefixReplacement{
meta.PrefixReplacement = &sstable.PrefixReplacement{
ContentPrefix: e.ContentPrefix,
SyntheticPrefix: e.SyntheticPrefix,
}
Expand Down
46 changes: 19 additions & 27 deletions internal/manifest/version.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/invariants"
"github.com/cockroachdb/pebble/sstable"
)

// Compare exports the base.Compare type.
Expand Down Expand Up @@ -272,40 +273,16 @@ type FileMetadata struct {

// PrefixReplacement is used for virtual files where the backing file has a
// different prefix on its keys than the span in which it is being exposed.
PrefixReplacement *PrefixReplacement
SyntheticSuffix []byte
PrefixReplacement *sstable.PrefixReplacement

SyntheticSuffix sstable.SyntheticSuffix
}

// InternalKeyBounds returns the set of overall table bounds.
func (m *FileMetadata) InternalKeyBounds() (InternalKey, InternalKey) {
return m.Smallest, m.Largest
}

// PrefixReplacement represents a read-time replacement of a key prefix.
type PrefixReplacement struct {
ContentPrefix, SyntheticPrefix []byte
}

// ReplaceArg replaces the new prefix in the argument with the original prefix.
func (p *PrefixReplacement) ReplaceArg(src []byte) []byte {
return p.replace(src, p.SyntheticPrefix, p.ContentPrefix)
}

// ReplaceResult replaces the original prefix in the result with the new prefix.
func (p *PrefixReplacement) ReplaceResult(key []byte) []byte {
return p.replace(key, p.ContentPrefix, p.SyntheticPrefix)
}

func (p *PrefixReplacement) replace(key, from, to []byte) []byte {
if !bytes.HasPrefix(key, from) {
panic(fmt.Sprintf("unexpected prefix in replace: %s", key))
}
result := make([]byte, 0, len(to)+(len(key)-len(from)))
result = append(result, to...)
result = append(result, key[len(from):]...)
return result
}

// PhysicalFileMeta is used by functions which want a guarantee that their input
// belongs to a physical sst and not a virtual sst.
//
Expand Down Expand Up @@ -351,6 +328,21 @@ type VirtualFileMeta struct {
*FileMetadata
}

// VirtualReaderParams fills in the parameters necessary to create a virtual
// sstable reader.
func (m VirtualFileMeta) VirtualReaderParams(isShared bool) sstable.VirtualReaderParams {
return sstable.VirtualReaderParams{
Lower: m.Smallest,
Upper: m.Largest,
FileNum: m.FileNum,
IsShared: isShared,
Size: m.Size,
BackingSize: m.FileBacking.Size,
PrefixReplacement: m.PrefixReplacement,
SyntheticSuffix: m.SyntheticSuffix,
}
}

// PhysicalMeta should be the only source of creating the PhysicalFileMeta
// wrapper type.
func (m *FileMetadata) PhysicalMeta() PhysicalFileMeta {
Expand Down
5 changes: 3 additions & 2 deletions internal/manifest/version_edit.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/invariants"
"github.com/cockroachdb/pebble/sstable"
)

// TODO(peter): describe the MANIFEST file format, independently of the C++
Expand Down Expand Up @@ -340,7 +341,7 @@ func (v *VersionEdit) Decode(r io.Reader) error {
virtual bool
backingFileNum uint64
}{}
var virtualPrefix *PrefixReplacement
var virtualPrefix *sstable.PrefixReplacement
var syntheticSuffix []byte
if tag == tagNewFile4 || tag == tagNewFile5 {
for {
Expand Down Expand Up @@ -391,7 +392,7 @@ func (v *VersionEdit) Decode(r io.Reader) error {
if err != nil {
return err
}
virtualPrefix = &PrefixReplacement{
virtualPrefix = &sstable.PrefixReplacement{
ContentPrefix: content,
SyntheticPrefix: synthetic,
}
Expand Down
3 changes: 2 additions & 1 deletion internal/manifest/version_edit_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/record"
"github.com/cockroachdb/pebble/sstable"
"github.com/kr/pretty"
"github.com/stretchr/testify/require"
)
Expand Down Expand Up @@ -50,7 +51,7 @@ func TestVERoundTripAndAccumulate(t *testing.T) {
CreationTime: 809060,
SmallestSeqNum: 9,
LargestSeqNum: 11,
PrefixReplacement: &PrefixReplacement{
PrefixReplacement: &sstable.PrefixReplacement{
ContentPrefix: []byte("before"),
SyntheticPrefix: []byte("after"),
},
Expand Down
30 changes: 30 additions & 0 deletions sstable/reader_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@
package sstable

import (
"bytes"
"fmt"
"os"
"sync"
Expand Down Expand Up @@ -34,6 +35,35 @@ type Iterator interface {
SetCloseHook(fn func(i Iterator) error)
}

// PrefixReplacement represents a read-time replacement of a key prefix.
type PrefixReplacement struct {
// ContentPrefix is the existing prefix that each key is expected to have.
ContentPrefix []byte
// SyntheticPrefix replaces the ContentPrefix in all keys. If ContentPrefix is
// empty, we are just prepending the synthetic prefix.
SyntheticPrefix []byte
}

// ReplaceArg replaces the new prefix in the argument with the original prefix.
func (p *PrefixReplacement) ReplaceArg(src []byte) []byte {
return p.replace(src, p.SyntheticPrefix, p.ContentPrefix)
}

// ReplaceResult replaces the original prefix in the result with the new prefix.
func (p *PrefixReplacement) ReplaceResult(key []byte) []byte {
return p.replace(key, p.ContentPrefix, p.SyntheticPrefix)
}

func (p *PrefixReplacement) replace(key, from, to []byte) []byte {
if !bytes.HasPrefix(key, from) {
panic(fmt.Sprintf("unexpected prefix in replace: %s", key))
}
result := make([]byte, 0, len(to)+(len(key)-len(from)))
result = append(result, to...)
result = append(result, key[len(from):]...)
return result
}

// Iterator positioning optimizations and singleLevelIterator and
// twoLevelIterator:
//
Expand Down
94 changes: 23 additions & 71 deletions sstable/reader_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@ import (
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/cache"
"github.com/cockroachdb/pebble/internal/humanize"
"github.com/cockroachdb/pebble/internal/manifest"
"github.com/cockroachdb/pebble/internal/testkeys"
"github.com/cockroachdb/pebble/objstorage"
"github.com/cockroachdb/pebble/objstorage/objstorageprovider"
Expand Down Expand Up @@ -187,12 +186,11 @@ func TestVirtualReader(t *testing.T) {

// Set during the latest build command.
var r *Reader
var meta manifest.PhysicalFileMeta
var wMeta *WriterMetadata
var bp BufferPool

// Set during the latest virtualize command.
var vMeta1 manifest.VirtualFileMeta
var v VirtualReader
var v *VirtualReader

defer func() {
if r != nil {
Expand All @@ -201,32 +199,6 @@ func TestVirtualReader(t *testing.T) {
}
}()

createPhysicalMeta := func(w *WriterMetadata, r *Reader) (manifest.PhysicalFileMeta, error) {
meta := &manifest.FileMetadata{}
meta.FileNum = nextFileNum()
meta.CreationTime = time.Now().Unix()
meta.Size = w.Size
meta.SmallestSeqNum = w.SmallestSeqNum
meta.LargestSeqNum = w.LargestSeqNum

if w.HasPointKeys {
meta.ExtendPointKeyBounds(r.Compare, w.SmallestPoint, w.LargestPoint)
}
if w.HasRangeDelKeys {
meta.ExtendPointKeyBounds(r.Compare, w.SmallestRangeDel, w.LargestRangeDel)
}
if w.HasRangeKeys {
meta.ExtendRangeKeyBounds(r.Compare, w.SmallestRangeKey, w.LargestRangeKey)
}
meta.InitPhysicalBacking()

if err := meta.Validate(r.Compare, r.opts.Comparer.FormatKey); err != nil {
return manifest.PhysicalFileMeta{}, err
}

return meta.PhysicalMeta(), nil
}

formatWMeta := func(m *WriterMetadata) string {
var b bytes.Buffer
if m.HasPointKeys {
Expand Down Expand Up @@ -281,11 +253,8 @@ func TestVirtualReader(t *testing.T) {
bp.Release()
_ = r.Close()
r = nil
meta.FileMetadata = nil
vMeta1.FileMetadata = nil
v = VirtualReader{}
v = nil
}
var wMeta *WriterMetadata
var err error
writerOpts := &WriterOptions{
TableFormat: TableFormatMax,
Expand All @@ -307,13 +276,6 @@ func TestVirtualReader(t *testing.T) {
return err.Error()
}
bp.Init(5)

// Create a fake filemetada using the writer meta.
meta, err = createPhysicalMeta(wMeta, r)
if err != nil {
return err.Error()
}
r.fileNum = meta.FileBacking.DiskFileNum
return formatWMeta(wMeta)

case "virtualize":
Expand All @@ -323,49 +285,39 @@ func TestVirtualReader(t *testing.T) {
// this command the bounds must be valid keys. In general, and for
// this command, range key/range del spans must also not span across
// virtual sstable bounds.
if meta.FileMetadata == nil {
if wMeta == nil {
return "build must be called at least once before virtualize"
}
if vMeta1.FileMetadata != nil {
vMeta1.FileMetadata = nil
v = VirtualReader{}
}
v = nil
var params VirtualReaderParams

// Parse the virtualization bounds.
bounds := strings.Split(td.CmdArgs[0].String(), "-")
params.Lower = base.ParseInternalKey(bounds[0])
params.Upper = base.ParseInternalKey(bounds[1])

var syntheticSuffix []byte
if td.HasArg("suffix") {
var synthSuffixStr string
td.ScanArgs(t, "suffix", &synthSuffixStr)
syntheticSuffix = []byte(synthSuffixStr)
params.SyntheticSuffix = []byte(synthSuffixStr)
}

vMeta := &manifest.FileMetadata{
FileBacking: meta.FileBacking,
SmallestSeqNum: meta.SmallestSeqNum,
LargestSeqNum: meta.LargestSeqNum,
Virtual: true,
SyntheticSuffix: syntheticSuffix,
}
// Parse the virtualization bounds.
bounds := strings.Split(td.CmdArgs[0].String(), "-")
vMeta.Smallest = base.ParseInternalKey(bounds[0])
vMeta.Largest = base.ParseInternalKey(bounds[1])
vMeta.FileNum = nextFileNum()
params.FileNum = nextFileNum()
params.BackingSize = wMeta.Size
var err error
vMeta.Size, err = r.EstimateDiskUsage(vMeta.Smallest.UserKey, vMeta.Largest.UserKey)
params.Size, err = r.EstimateDiskUsage(params.Lower.UserKey, params.Upper.UserKey)
if err != nil {
return err.Error()
}
vMeta.ValidateVirtual(meta.FileMetadata)

vMeta1 = vMeta.VirtualMeta()
v = MakeVirtualReader(r, vMeta1, false /* isSharedIngested */)
return formatVirtualReader(&v)
vr := MakeVirtualReader(r, params)
v = &vr
return formatVirtualReader(v)

case "citer":
// Creates a compaction iterator from the virtual reader, and then
// just scans the keyspace. Which is all a compaction iterator is
// used for. This tests the First and Next calls.
if vMeta1.FileMetadata == nil {
if v == nil {
return "virtualize must be called before creating compaction iters"
}

Expand All @@ -387,7 +339,7 @@ func TestVirtualReader(t *testing.T) {
return buf.String()

case "constrain":
if vMeta1.FileMetadata == nil {
if v == nil {
return "virtualize must be called before constrain"
}
splits := strings.Split(td.CmdArgs[0].String(), ",")
Expand All @@ -407,7 +359,7 @@ func TestVirtualReader(t *testing.T) {
return buf.String()

case "scan-range-del":
if vMeta1.FileMetadata == nil {
if v == nil {
return "virtualize must be called before scan-range-del"
}
iter, err := v.NewRawRangeDelIter()
Expand All @@ -430,7 +382,7 @@ func TestVirtualReader(t *testing.T) {
return buf.String()

case "scan-range-key":
if vMeta1.FileMetadata == nil {
if v == nil {
return "virtualize must be called before scan-range-key"
}
iter, err := v.NewRawRangeKeyIter()
Expand All @@ -453,7 +405,7 @@ func TestVirtualReader(t *testing.T) {
return buf.String()

case "iter":
if vMeta1.FileMetadata == nil {
if v == nil {
return "virtualize must be called before iter"
}
var lower, upper []byte
Expand Down
Loading

0 comments on commit 9077044

Please sign in to comment.