Skip to content

Commit

Permalink
db: remove experimental point tombstone weight option
Browse files Browse the repository at this point in the history
Remove the experimental point_tombstone_weight option. The introduction of this
option accidentally caused the loss of point tombstone compensation applied to
level scoring. It's been inconculsive whether setting this value >1.0 has ever
helped customers.
  • Loading branch information
jbowens committed May 24, 2023
1 parent a8d7345 commit 2ff7efb
Show file tree
Hide file tree
Showing 10 changed files with 201 additions and 61 deletions.
26 changes: 10 additions & 16 deletions compaction_picker.go
Original file line number Diff line number Diff line change
Expand Up @@ -676,11 +676,11 @@ type candidateLevelInfo struct {

// compensatedSize returns f's file size, inflated according to compaction
// priorities.
func compensatedSize(f *fileMetadata, pointTombstoneWeight float64) uint64 {
func compensatedSize(f *fileMetadata) uint64 {
sz := f.Size
// Add in the estimate of disk space that may be reclaimed by compacting
// the file's tombstones.
sz += uint64(float64(f.Stats.PointDeletionsBytesEstimate) * pointTombstoneWeight)
sz += uint64(f.Stats.PointDeletionsBytesEstimate)
sz += f.Stats.RangeDeletionsBytesEstimate
return sz
}
Expand All @@ -691,7 +691,6 @@ func compensatedSize(f *fileMetadata, pointTombstoneWeight float64) uint64 {
// asynchronously, so its values are marked as cacheable only if a file's
// stats have been loaded.
type compensatedSizeAnnotator struct {
pointTombstoneWeight float64
}

var _ manifest.Annotator = compensatedSizeAnnotator{}
Expand All @@ -709,7 +708,7 @@ func (a compensatedSizeAnnotator) Accumulate(
f *fileMetadata, dst interface{},
) (v interface{}, cacheOK bool) {
vptr := dst.(*uint64)
*vptr = *vptr + compensatedSize(f, a.pointTombstoneWeight)
*vptr = *vptr + compensatedSize(f)
return vptr, f.StatsValid()
}

Expand All @@ -724,10 +723,10 @@ func (a compensatedSizeAnnotator) Merge(src interface{}, dst interface{}) interf
// iterator. Note that this function is linear in the files available to the
// iterator. Use the compensatedSizeAnnotator if querying the total
// compensated size of a level.
func totalCompensatedSize(iter manifest.LevelIterator, pointTombstoneWeight float64) uint64 {
func totalCompensatedSize(iter manifest.LevelIterator) uint64 {
var sz uint64
for f := iter.First(); f != nil; f = iter.Next() {
sz += compensatedSize(f, pointTombstoneWeight)
sz += compensatedSize(f)
}
return sz
}
Expand Down Expand Up @@ -925,9 +924,7 @@ func (p *compactionPickerByScore) initLevelMaxBytes(inProgressCompactions []comp
}
}

func calculateSizeAdjust(
inProgressCompactions []compactionInfo, pointTombstoneWeight float64,
) [numLevels]int64 {
func calculateSizeAdjust(inProgressCompactions []compactionInfo) [numLevels]int64 {
// Compute a size adjustment for each level based on the in-progress
// compactions. We subtract the compensated size of start level inputs.
// Since compensated file sizes may be compensated because they reclaim
Expand All @@ -940,7 +937,7 @@ func calculateSizeAdjust(

for _, input := range c.inputs {
real := int64(input.files.SizeSum())
compensated := int64(totalCompensatedSize(input.files.Iter(), pointTombstoneWeight))
compensated := int64(totalCompensatedSize(input.files.Iter()))

if input.level != c.outputLevel {
sizeAdjust[input.level] -= compensated
Expand All @@ -967,10 +964,7 @@ func (p *compactionPickerByScore) calculateScores(
}
scores[0] = p.calculateL0Score(inProgressCompactions)

sizeAdjust := calculateSizeAdjust(
inProgressCompactions,
p.opts.Experimental.PointTombstoneWeight,
)
sizeAdjust := calculateSizeAdjust(inProgressCompactions)
for level := 1; level < numLevels; level++ {
levelSize := int64(levelCompensatedSize(p.vers.Levels[level])) + sizeAdjust[level]
scores[level].score = float64(levelSize) / float64(p.levelMaxBytes[level])
Expand Down Expand Up @@ -1127,7 +1121,7 @@ func (p *compactionPickerByScore) pickFile(
continue
}

compSz := compensatedSize(f, p.opts.Experimental.PointTombstoneWeight)
compSz := compensatedSize(f)
scaledRatio := overlappingBytes * 1024 / compSz
if scaledRatio < smallestRatio && !f.IsCompacting() {
smallestRatio = scaledRatio
Expand Down Expand Up @@ -1191,7 +1185,7 @@ func (p *compactionPickerByScore) pickAuto(env compactionEnv) (pc *pickedCompact
fmt.Fprintf(&buf, " %sL%d: %5.1f %5.1f %8s %8s",
marker, info.level, info.score, info.origScore,
humanize.Int64(int64(totalCompensatedSize(
p.vers.Levels[info.level].Iter(), p.opts.Experimental.PointTombstoneWeight,
p.vers.Levels[info.level].Iter(),
))),
humanize.Int64(p.levelMaxBytes[info.level]),
)
Expand Down
91 changes: 72 additions & 19 deletions compaction_picker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,10 @@ import (
"github.com/cockroachdb/datadriven"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/humanize"
"github.com/cockroachdb/pebble/internal/manifest"
"github.com/cockroachdb/pebble/internal/testkeys"
"github.com/cockroachdb/pebble/vfs"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -1256,44 +1259,26 @@ func TestCompactionPickerCompensatedSize(t *testing.T) {
size uint64
pointDelEstimateBytes uint64
rangeDelEstimateBytes uint64
pointTombstoneWeight float64
wantBytes uint64
}{
{
size: 100,
pointDelEstimateBytes: 0,
rangeDelEstimateBytes: 0,
pointTombstoneWeight: 1,
wantBytes: 100,
},
{
size: 100,
pointDelEstimateBytes: 10,
rangeDelEstimateBytes: 0,
pointTombstoneWeight: 1,
wantBytes: 100 + 10,
},
{
size: 100,
pointDelEstimateBytes: 10,
rangeDelEstimateBytes: 5,
pointTombstoneWeight: 1,
wantBytes: 100 + 10 + 5,
},
{
size: 100,
pointDelEstimateBytes: 10,
rangeDelEstimateBytes: 5,
pointTombstoneWeight: 2,
wantBytes: 100 + 20 + 5,
},
{
size: 100,
pointDelEstimateBytes: 10,
rangeDelEstimateBytes: 5,
pointTombstoneWeight: 0.5,
wantBytes: 100 + 5 + 5,
},
}

for _, tc := range testCases {
Expand All @@ -1302,12 +1287,80 @@ func TestCompactionPickerCompensatedSize(t *testing.T) {
f.InitPhysicalBacking()
f.Stats.PointDeletionsBytesEstimate = tc.pointDelEstimateBytes
f.Stats.RangeDeletionsBytesEstimate = tc.rangeDelEstimateBytes
gotBytes := compensatedSize(f, tc.pointTombstoneWeight)
gotBytes := compensatedSize(f)
require.Equal(t, tc.wantBytes, gotBytes)
})
}
}

func TestCompactionPickerScores(t *testing.T) {
fs := vfs.NewMem()
opts := &Options{
Comparer: testkeys.Comparer,
DisableAutomaticCompactions: true,
FormatMajorVersion: FormatNewest,
FS: fs,
}

d, err := Open("", opts)
require.NoError(t, err)
defer func() {
if d != nil {
require.NoError(t, closeAllSnapshots(d))
require.NoError(t, d.Close())
}
}()

var buf bytes.Buffer
datadriven.RunTest(t, "testdata/compaction_picker_scores", func(t *testing.T, td *datadriven.TestData) string {
switch td.Cmd {
case "disable-table-stats":
d.mu.Lock()
d.opts.private.disableTableStats = true
d.mu.Unlock()
return ""

case "enable-table-stats":
d.mu.Lock()
d.opts.private.disableTableStats = false
d.maybeCollectTableStatsLocked()
d.mu.Unlock()
return ""

case "define":
require.NoError(t, closeAllSnapshots(d))
require.NoError(t, d.Close())

d, err = runDBDefineCmd(td, opts)
if err != nil {
return err.Error()
}
d.mu.Lock()
s := d.mu.versions.currentVersion().String()
d.mu.Unlock()
return s

case "scores":
buf.Reset()
fmt.Fprintf(&buf, "L Size Score\n")
for l, lm := range d.Metrics().Levels {
if l < numLevels-1 {
fmt.Fprintf(&buf, "L%-3d\t%-7s%.1f\n", l, humanize.IEC.Int64(lm.Size), lm.Score)
} else {
fmt.Fprintf(&buf, "L%-3d\t%-7s-\n", l, humanize.IEC.Int64(lm.Size))
}
}
return buf.String()

case "wait-pending-table-stats":
return runTableStatsCmd(td, d)

default:
return fmt.Sprintf("unknown command: %s", td.Cmd)
}
})
}

func fileNums(files manifest.LevelSlice) string {
var ss []string
files.Each(func(f *fileMetadata) {
Expand Down
12 changes: 6 additions & 6 deletions data_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -774,6 +774,12 @@ func runDBDefineCmd(td *datadriven.TestData, opts *Options) (*DB, error) {
return nil, errors.New("Snapshots must be in ascending order")
}
}
case "lbase-max-bytes":
lbaseMaxBytes, err := strconv.ParseInt(arg.Vals[0], 10, 64)
if err != nil {
return nil, err
}
opts.LBaseMaxBytes = lbaseMaxBytes
case "level-max-bytes":
levelMaxBytes = map[int]int64{}
for i := range arg.Vals {
Expand Down Expand Up @@ -812,12 +818,6 @@ func runDBDefineCmd(td *datadriven.TestData, opts *Options) (*DB, error) {
for _, levelOpts := range opts.Levels {
levelOpts.BlockSize = size
}
case "point-tombstone-weight":
w, err := strconv.ParseFloat(arg.Vals[0], 64)
if err != nil {
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 {
Expand Down
1 change: 0 additions & 1 deletion metamorphic/options.go
Original file line number Diff line number Diff line change
Expand Up @@ -437,7 +437,6 @@ func randomOptions(
lopts.FilterPolicy = newTestingFilterPolicy(1 << rng.Intn(5))
}
opts.Levels = []pebble.LevelOptions{lopts}
opts.Experimental.PointTombstoneWeight = 1 + 10*rng.Float64() // 1 - 10

// Explicitly disable disk-backed FS's for the random configurations. The
// single standard test configuration that uses a disk-backed FS is
Expand Down
2 changes: 1 addition & 1 deletion open.go
Original file line number Diff line number Diff line change
Expand Up @@ -512,7 +512,7 @@ func Open(dirname string, opts *Options) (db *DB, _ error) {
}
d.mu.tableStats.cond.L = &d.mu.Mutex
d.mu.tableValidation.cond.L = &d.mu.Mutex
if !d.opts.ReadOnly && !d.opts.private.disableTableStats {
if !d.opts.ReadOnly {
d.maybeCollectTableStatsLocked()
}
d.calculateDiskAvailableBytes()
Expand Down
13 changes: 1 addition & 12 deletions options.go
Original file line number Diff line number Diff line change
Expand Up @@ -606,12 +606,6 @@ type Options struct {
// for CPUWorkPermissionGranter for more details.
CPUWorkPermissionGranter CPUWorkPermissionGranter

// PointTombstoneWeight is a float in the range [0, +inf) used to weight the
// point tombstone heuristics during compaction picking.
//
// The default value is 1, which results in no scaling of point tombstones.
PointTombstoneWeight float64

// EnableValueBlocks is used to decide whether to enable writing
// TableFormatPebblev3 sstables. This setting is only respected by a
// specific subset of format major versions: FormatSSTableValueBlocks,
Expand Down Expand Up @@ -1043,10 +1037,6 @@ func (o *Options) EnsureDefaults() *Options {
if o.Experimental.CPUWorkPermissionGranter == nil {
o.Experimental.CPUWorkPermissionGranter = defaultCPUWorkGranter{}
}
if o.Experimental.PointTombstoneWeight == 0 {
o.Experimental.PointTombstoneWeight = 1
}

if o.Experimental.MultiLevelCompactionHueristic == nil {
o.Experimental.MultiLevelCompactionHueristic = NoMultiLevel{}
}
Expand Down Expand Up @@ -1169,7 +1159,6 @@ func (o *Options) String() string {
fmt.Fprintf(&buf, " mem_table_stop_writes_threshold=%d\n", o.MemTableStopWritesThreshold)
fmt.Fprintf(&buf, " min_deletion_rate=%d\n", o.Experimental.MinDeletionRate)
fmt.Fprintf(&buf, " merger=%s\n", o.Merger.Name)
fmt.Fprintf(&buf, " point_tombstone_weight=%f\n", o.Experimental.PointTombstoneWeight)
fmt.Fprintf(&buf, " read_compaction_rate=%d\n", o.Experimental.ReadCompactionRate)
fmt.Fprintf(&buf, " read_sampling_multiplier=%d\n", o.Experimental.ReadSamplingMultiplier)
fmt.Fprintf(&buf, " strict_wal_tail=%t\n", o.private.strictWALTail)
Expand Down Expand Up @@ -1419,7 +1408,7 @@ func (o *Options) Parse(s string, hooks *ParseHooks) error {
// Do nothing; option existed in older versions of pebble, and
// may be meaningful again eventually.
case "point_tombstone_weight":
o.Experimental.PointTombstoneWeight, err = strconv.ParseFloat(value, 64)
// Do nothing; deprecated.
case "strict_wal_tail":
o.private.strictWALTail, err = strconv.ParseBool(value)
case "merger":
Expand Down
1 change: 0 additions & 1 deletion options_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -91,7 +91,6 @@ func TestOptionsString(t *testing.T) {
mem_table_stop_writes_threshold=2
min_deletion_rate=0
merger=pebble.concatenate
point_tombstone_weight=1.000000
read_compaction_rate=16000
read_sampling_multiplier=16
strict_wal_tail=true
Expand Down
5 changes: 2 additions & 3 deletions replay/testdata/replay
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,7 @@ tree
0 LOCK
98 MANIFEST-000001
122 MANIFEST-000008
1171 OPTIONS-000003
1137 OPTIONS-000003
0 marker.format-version.000007.008
0 marker.manifest.000002.MANIFEST-000008
simple/
Expand All @@ -24,7 +24,7 @@ tree
25 000004.log
795 000005.sst
98 MANIFEST-000001
1171 OPTIONS-000003
1137 OPTIONS-000003
0 marker.format-version.000001.008
0 marker.manifest.000001.MANIFEST-000001

Expand Down Expand Up @@ -57,7 +57,6 @@ cat build/OPTIONS-000003
mem_table_stop_writes_threshold=2
min_deletion_rate=0
merger=pebble.concatenate
point_tombstone_weight=1.000000
read_compaction_rate=16000
read_sampling_multiplier=16
strict_wal_tail=true
Expand Down
4 changes: 2 additions & 2 deletions replay/testdata/replay_paced
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@ tree
0 LOCK
122 MANIFEST-000008
205 MANIFEST-000011
1171 OPTIONS-000003
1137 OPTIONS-000003
0 marker.format-version.000007.008
0 marker.manifest.000003.MANIFEST-000011
high_read_amp/
Expand All @@ -27,7 +27,7 @@ tree
39 000009.log
769 000010.sst
157 MANIFEST-000011
1171 OPTIONS-000003
1137 OPTIONS-000003
0 marker.format-version.000001.008
0 marker.manifest.000001.MANIFEST-000011

Expand Down
Loading

0 comments on commit 2ff7efb

Please sign in to comment.