Skip to content

Commit 580eb1a

Browse files
committed
db,valsep: move valsep tests to appropriate pkg
1 parent 4cb1291 commit 580eb1a

File tree

7 files changed

+128
-126
lines changed

7 files changed

+128
-126
lines changed

blob_rewrite_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -67,7 +67,7 @@ func TestBlobRewrite(t *testing.T) {
6767
tw = sstable.NewRawWriter(w, sstable.WriterOptions{
6868
TableFormat: sstable.TableFormatMax,
6969
})
70-
tw = &loggingRawWriter{w: &buf, RawWriter: tw}
70+
tw = &sstable.LoggingRawWriter{LogWriter: &buf, RawWriter: tw}
7171
}
7272

7373
datadriven.RunTest(t, "testdata/blob_rewrite",

compaction.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -235,7 +235,7 @@ type tableCompaction struct {
235235
// resulting version has been installed (if successful), but the compaction
236236
// goroutine is still cleaning up (eg, deleting obsolete files).
237237
versionEditApplied bool
238-
// getValueSeparation constructs a compact.ValueSeparation for use in a
238+
// getValueSeparation constructs a valsep.ValueSeparation for use in a
239239
// compaction. It implements heuristics around choosing whether a compaction
240240
// should:
241241
//

compaction_value_separation.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,7 @@ var neverSeparateValues getValueSeparation = func(JobID, *tableCompaction, Value
2424
}
2525

2626
// determineCompactionValueSeparation determines whether a compaction should
27-
// separate values into blob files. It returns a compact.ValueSeparation
27+
// separate values into blob files. It returns a valsep.ValueSeparation
2828
// implementation that should be used for the compaction.
2929
//
3030
// It assumes that the compaction will write tables at d.TableFormat() or above.

data_test.go

Lines changed: 92 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -905,11 +905,12 @@ func runDBDefineCmd(td *datadriven.TestData, opts *Options) (*DB, error) {
905905
}
906906
d.mu.versions.dynamicBaseLevel = false
907907

908-
// We use a custom compact.ValueSeparation implementation that wraps
909-
// preserveBlobReferences. It parses values containing human-readable blob
910-
// references (as understood by blobtest.Values) and preserves the
911-
// references. It also accumulates all the handles so that after writing
912-
// all the tables, we can construct all the referenced blob files and add
908+
// We use a custom valsep.ValueSeparation implementation that wraps
909+
// valsep.ValueSeparator in preserveAllHotBlobReferences mode. It
910+
// parses values containing human-readable blob references (as
911+
// understood by blobtest.Values) and preserves the references. It
912+
// also accumulates all the handles so that after writing all the
913+
// tables, we can construct all the referenced blob files and add
913914
// them to the final version edit.
914915
valueSeparator := &defineDBValueSeparator{
915916
metas: make(map[base.BlobFileID]*manifest.PhysicalBlobFile),
@@ -1922,3 +1923,89 @@ func streamFilterBetweenGrep(start, end string) stream.Filter {
19221923
return nil
19231924
})
19241925
}
1926+
1927+
// defineDBValueSeparator is a valsep.ValueSeparation implementation used by
1928+
// datadriven tests when defining a database state. It is a wrapper around
1929+
// preserveBlobReferences that also parses string representations of blob
1930+
// references from values.
1931+
type defineDBValueSeparator struct {
1932+
bv blobtest.Values
1933+
metas map[base.BlobFileID]*manifest.PhysicalBlobFile
1934+
pbr valsep.ValueSeparation
1935+
kv base.InternalKV
1936+
}
1937+
1938+
// Assert that *defineDBValueSeparator implements the valsep.ValueSeparation interface.
1939+
var _ valsep.ValueSeparation = (*defineDBValueSeparator)(nil)
1940+
1941+
// SetNextOutputConfig implements the valsep.ValueSeparation interface.
1942+
func (vs *defineDBValueSeparator) SetNextOutputConfig(config valsep.ValueSeparationOutputConfig) {}
1943+
1944+
// Kind implements the ValueSeparation interface.
1945+
func (vs *defineDBValueSeparator) Kind() sstable.ValueSeparationKind {
1946+
return vs.pbr.Kind()
1947+
}
1948+
1949+
// MinimumSize implements the ValueSeparation interface.
1950+
func (vs *defineDBValueSeparator) MinimumSize() int { return vs.pbr.MinimumSize() }
1951+
1952+
// EstimatedFileSize returns an estimate of the disk space consumed by the current
1953+
// blob file if it were closed now.
1954+
func (vs *defineDBValueSeparator) EstimatedFileSize() uint64 {
1955+
return vs.pbr.EstimatedFileSize()
1956+
}
1957+
1958+
// EstimatedReferenceSize returns an estimate of the disk space consumed by the
1959+
// current output sstable's blob references so far.
1960+
func (vs *defineDBValueSeparator) EstimatedReferenceSize() uint64 {
1961+
return vs.pbr.EstimatedReferenceSize()
1962+
}
1963+
1964+
// Add adds the provided key-value pair to the sstable, possibly separating the
1965+
// value into a blob file.
1966+
func (vs *defineDBValueSeparator) Add(
1967+
tw sstable.RawWriter, kv *base.InternalKV, forceObsolete bool, _ bool,
1968+
) error {
1969+
// In datadriven tests, all defined values are in-place initially. See
1970+
// runDBDefineCmdReuseFS.
1971+
v := kv.V.InPlaceValue()
1972+
// If the value doesn't begin with "blob", don't separate it.
1973+
if !bytes.HasPrefix(v, []byte("blob")) {
1974+
return tw.Add(kv.K, v, forceObsolete)
1975+
}
1976+
1977+
// This looks like a blob reference. Parse it.
1978+
iv, err := vs.bv.ParseInternalValue(string(v))
1979+
if err != nil {
1980+
return err
1981+
}
1982+
lv := iv.LazyValue()
1983+
// If we haven't seen this blob file before, fabricate a metadata for it.
1984+
fileID := lv.Fetcher.BlobFileID
1985+
meta, ok := vs.metas[fileID]
1986+
if !ok {
1987+
meta = &manifest.PhysicalBlobFile{
1988+
FileNum: base.DiskFileNum(fileID),
1989+
CreationTime: uint64(time.Now().Unix()),
1990+
}
1991+
vs.metas[fileID] = meta
1992+
}
1993+
meta.Size += uint64(lv.Fetcher.Attribute.ValueLen)
1994+
meta.ValueSize += uint64(lv.Fetcher.Attribute.ValueLen)
1995+
1996+
// Return a KV that uses the original key but our constructed blob reference.
1997+
vs.kv.K = kv.K
1998+
vs.kv.V = iv
1999+
return vs.pbr.Add(tw, &vs.kv, forceObsolete, false /* isLikelyMVCCGarbage */)
2000+
}
2001+
2002+
// FinishOutput implements valsep.ValueSeparation.
2003+
func (d *defineDBValueSeparator) FinishOutput() (valsep.ValueSeparationMetadata, error) {
2004+
m, err := d.pbr.FinishOutput()
2005+
if err != nil {
2006+
return valsep.ValueSeparationMetadata{}, err
2007+
}
2008+
// TODO(jackson): Support setting a specific depth from the datadriven test.
2009+
m.BlobReferenceDepth = manifest.BlobReferenceDepth(len(m.BlobReferences))
2010+
return m, nil
2011+
}

sstable/writer.go

Lines changed: 22 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -6,6 +6,8 @@ package sstable
66

77
import (
88
"context"
9+
"fmt"
10+
"io"
911

1012
"github.com/cockroachdb/errors"
1113
"github.com/cockroachdb/pebble/internal/base"
@@ -470,3 +472,23 @@ func (m *WriterMetadata) updateSeqNum(seqNum base.SeqNum) {
470472
m.LargestSeqNum = seqNum
471473
}
472474
}
475+
476+
// LoggingRawWriter wraps a sstable.RawWriter and logs calls to Add and
477+
// AddWithBlobHandle to provide observability into the separation of values into
478+
// blob files. This is intended for testing.
479+
type LoggingRawWriter struct {
480+
LogWriter io.Writer
481+
RawWriter
482+
}
483+
484+
func (w *LoggingRawWriter) Add(key base.InternalKey, value []byte, forceObsolete bool) error {
485+
fmt.Fprintf(w.LogWriter, "RawWriter.Add(%q, %q, %t)\n", key, value, forceObsolete)
486+
return w.RawWriter.Add(key, value, forceObsolete)
487+
}
488+
489+
func (w *LoggingRawWriter) AddWithBlobHandle(
490+
key base.InternalKey, h blob.InlineHandle, attr base.ShortAttribute, forceObsolete bool,
491+
) error {
492+
fmt.Fprintf(w.LogWriter, "RawWriter.AddWithBlobHandle(%q, %q, %x, %t)\n", key, h, attr, forceObsolete)
493+
return w.RawWriter.AddWithBlobHandle(key, h, attr, forceObsolete)
494+
}
File renamed without changes.

compaction_value_separation_test.go renamed to valsep/value_separation_test.go

Lines changed: 11 additions & 118 deletions
Original file line numberDiff line numberDiff line change
@@ -2,16 +2,14 @@
22
// of this source code is governed by a BSD-style license that can be found in
33
// the LICENSE file.
44

5-
package pebble
5+
package valsep
66

77
import (
88
"bytes"
99
"context"
1010
"fmt"
11-
"io"
1211
"strings"
1312
"testing"
14-
"time"
1513

1614
"github.com/cockroachdb/crlib/crstrings"
1715
"github.com/cockroachdb/datadriven"
@@ -24,15 +22,14 @@ import (
2422
"github.com/cockroachdb/pebble/objstorage/objstorageprovider"
2523
"github.com/cockroachdb/pebble/sstable"
2624
"github.com/cockroachdb/pebble/sstable/blob"
27-
"github.com/cockroachdb/pebble/valsep"
2825
"github.com/cockroachdb/pebble/vfs"
2926
"github.com/stretchr/testify/require"
3027
)
3128

3229
func TestValueSeparationPolicy(t *testing.T) {
3330
var (
3431
bv blobtest.Values
35-
vs valsep.ValueSeparation
32+
vs ValueSeparation
3633
tw sstable.RawWriter
3734
fn base.DiskFileNum
3835
buf bytes.Buffer
@@ -62,7 +59,7 @@ func TestValueSeparationPolicy(t *testing.T) {
6259
tw = sstable.NewRawWriter(w, sstable.WriterOptions{
6360
TableFormat: sstable.TableFormatPebblev6,
6461
})
65-
tw = &loggingRawWriter{w: &buf, RawWriter: tw}
62+
tw = &sstable.LoggingRawWriter{LogWriter: &buf, RawWriter: tw}
6663
}
6764

6865
datadriven.RunTest(t, "testdata/value_separation_policy",
@@ -76,7 +73,7 @@ func TestValueSeparationPolicy(t *testing.T) {
7673
bv = blobtest.Values{}
7774
switch x := d.CmdArgs[0].String(); x {
7875
case "never-separate-values":
79-
vs = valsep.NeverSeparateValues{}
76+
vs = NeverSeparateValues{}
8077
case "preserve-blob-references":
8178
lines := crstrings.Lines(d.Input)
8279
inputBlobPhysicalFiles := make(map[base.BlobFileID]*manifest.PhysicalBlobFile, len(lines))
@@ -86,7 +83,7 @@ func TestValueSeparationPolicy(t *testing.T) {
8683
fn = max(fn, bfm.Physical.FileNum)
8784
inputBlobPhysicalFiles[bfm.FileID] = bfm.Physical
8885
}
89-
pbr := valsep.NewPreserveAllHotBlobReferences(
86+
pbr := NewPreserveAllHotBlobReferences(
9087
inputBlobPhysicalFiles, /* blob file set */
9188
manifest.BlobReferenceDepth(0),
9289
sstable.ValueSeparationDefault,
@@ -105,15 +102,15 @@ func TestValueSeparationPolicy(t *testing.T) {
105102
t.Fatalf("unknown short attribute extractor: %s", arg.String())
106103
}
107104
}
108-
newSep := valsep.NewWriteNewBlobFiles(
105+
newSep := NewWriteNewBlobFiles(
109106
testkeys.Comparer,
110107
func() (objstorage.Writable, objstorage.ObjectMetadata, error) {
111108
fn++
112109
return objStore.Create(ctx, base.FileTypeBlob, fn, objstorage.CreateOptions{})
113110
},
114111
blob.FileWriterOptions{},
115112
minimumSize,
116-
valsep.WriteNewBlobFilesOptions{
113+
WriteNewBlobFilesOptions{
117114
ShortAttrExtractor: shortAttrExtractor,
118115
InvalidValueCallback: func(userKey []byte, value []byte, err error) {
119116
fmt.Fprintf(&buf, "# invalid value for key %q, value: %q: %s\n", userKey, value, err)
@@ -183,115 +180,11 @@ func TestValueSeparationPolicy(t *testing.T) {
183180
})
184181
}
185182

186-
func errShortAttrExtractor(key []byte, keyPrefixLen int, value []byte) (ShortAttribute, error) {
183+
func errShortAttrExtractor(
184+
key []byte, keyPrefixLen int, value []byte,
185+
) (base.ShortAttribute, error) {
187186
return 0, errors.New("short attribute extractor error")
188187
}
189188

190189
// Assert that errShortAttrExtractor implements the ShortAttributeExtractor
191-
var _ ShortAttributeExtractor = errShortAttrExtractor
192-
193-
// loggingRawWriter wraps a sstable.RawWriter and logs calls to Add and
194-
// AddWithBlobHandle to provide observability into the separation of values into
195-
// blob files.
196-
type loggingRawWriter struct {
197-
w io.Writer
198-
sstable.RawWriter
199-
}
200-
201-
func (w *loggingRawWriter) Add(key InternalKey, value []byte, forceObsolete bool) error {
202-
fmt.Fprintf(w.w, "RawWriter.Add(%q, %q, %t)\n", key, value, forceObsolete)
203-
return w.RawWriter.Add(key, value, forceObsolete)
204-
}
205-
206-
func (w *loggingRawWriter) AddWithBlobHandle(
207-
key InternalKey, h blob.InlineHandle, attr base.ShortAttribute, forceObsolete bool,
208-
) error {
209-
fmt.Fprintf(w.w, "RawWriter.AddWithBlobHandle(%q, %q, %x, %t)\n", key, h, attr, forceObsolete)
210-
return w.RawWriter.AddWithBlobHandle(key, h, attr, forceObsolete)
211-
}
212-
213-
// defineDBValueSeparator is a compact.ValueSeparation implementation used by
214-
// datadriven tests when defining a database state. It is a wrapper around
215-
// preserveBlobReferences that also parses string representations of blob
216-
// references from values.
217-
type defineDBValueSeparator struct {
218-
bv blobtest.Values
219-
metas map[base.BlobFileID]*manifest.PhysicalBlobFile
220-
pbr valsep.ValueSeparation
221-
kv base.InternalKV
222-
}
223-
224-
// Assert that *defineDBValueSeparator implements the compact.ValueSeparation interface.
225-
var _ valsep.ValueSeparation = (*defineDBValueSeparator)(nil)
226-
227-
// SetNextOutputConfig implements the compact.ValueSeparation interface.
228-
func (vs *defineDBValueSeparator) SetNextOutputConfig(config valsep.ValueSeparationOutputConfig) {}
229-
230-
// Kind implements the ValueSeparation interface.
231-
func (vs *defineDBValueSeparator) Kind() sstable.ValueSeparationKind {
232-
return vs.pbr.Kind()
233-
}
234-
235-
// MinimumSize implements the ValueSeparation interface.
236-
func (vs *defineDBValueSeparator) MinimumSize() int { return vs.pbr.MinimumSize() }
237-
238-
// EstimatedFileSize returns an estimate of the disk space consumed by the current
239-
// blob file if it were closed now.
240-
func (vs *defineDBValueSeparator) EstimatedFileSize() uint64 {
241-
return vs.pbr.EstimatedFileSize()
242-
}
243-
244-
// EstimatedReferenceSize returns an estimate of the disk space consumed by the
245-
// current output sstable's blob references so far.
246-
func (vs *defineDBValueSeparator) EstimatedReferenceSize() uint64 {
247-
return vs.pbr.EstimatedReferenceSize()
248-
}
249-
250-
// Add adds the provided key-value pair to the sstable, possibly separating the
251-
// value into a blob file.
252-
func (vs *defineDBValueSeparator) Add(
253-
tw sstable.RawWriter, kv *base.InternalKV, forceObsolete bool, _ bool,
254-
) error {
255-
// In datadriven tests, all defined values are in-place initially. See
256-
// runDBDefineCmdReuseFS.
257-
v := kv.V.InPlaceValue()
258-
// If the value doesn't begin with "blob", don't separate it.
259-
if !bytes.HasPrefix(v, []byte("blob")) {
260-
return tw.Add(kv.K, v, forceObsolete)
261-
}
262-
263-
// This looks like a blob reference. Parse it.
264-
iv, err := vs.bv.ParseInternalValue(string(v))
265-
if err != nil {
266-
return err
267-
}
268-
lv := iv.LazyValue()
269-
// If we haven't seen this blob file before, fabricate a metadata for it.
270-
fileID := lv.Fetcher.BlobFileID
271-
meta, ok := vs.metas[fileID]
272-
if !ok {
273-
meta = &manifest.PhysicalBlobFile{
274-
FileNum: base.DiskFileNum(fileID),
275-
CreationTime: uint64(time.Now().Unix()),
276-
}
277-
vs.metas[fileID] = meta
278-
}
279-
meta.Size += uint64(lv.Fetcher.Attribute.ValueLen)
280-
meta.ValueSize += uint64(lv.Fetcher.Attribute.ValueLen)
281-
282-
// Return a KV that uses the original key but our constructed blob reference.
283-
vs.kv.K = kv.K
284-
vs.kv.V = iv
285-
return vs.pbr.Add(tw, &vs.kv, forceObsolete, false /* isLikelyMVCCGarbage */)
286-
}
287-
288-
// FinishOutput implements compact.ValueSeparation.
289-
func (d *defineDBValueSeparator) FinishOutput() (valsep.ValueSeparationMetadata, error) {
290-
m, err := d.pbr.FinishOutput()
291-
if err != nil {
292-
return valsep.ValueSeparationMetadata{}, err
293-
}
294-
// TODO(jackson): Support setting a specific depth from the datadriven test.
295-
m.BlobReferenceDepth = manifest.BlobReferenceDepth(len(m.BlobReferences))
296-
return m, nil
297-
}
190+
var _ base.ShortAttributeExtractor = errShortAttrExtractor

0 commit comments

Comments
 (0)