Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
19 changes: 4 additions & 15 deletions bloom/bloom.go
Original file line number Diff line number Diff line change
Expand Up @@ -7,7 +7,6 @@ package bloom // import "github.com/cockroachdb/pebble/bloom"

import (
"encoding/binary"
"fmt"
"sync"

"github.com/cockroachdb/pebble/internal/base"
Expand Down Expand Up @@ -228,21 +227,11 @@ func (p FilterPolicy) Name() string {
}

// MayContain implements the pebble.FilterPolicy interface.
func (p FilterPolicy) MayContain(ftype base.FilterType, f, key []byte) bool {
switch ftype {
case base.TableFilter:
return tableFilter(f).MayContain(key)
default:
panic(fmt.Sprintf("unknown filter type: %v", ftype))
}
func (p FilterPolicy) MayContain(f, key []byte) bool {
return tableFilter(f).MayContain(key)
}

// NewWriter implements the pebble.FilterPolicy interface.
func (p FilterPolicy) NewWriter(ftype base.FilterType) base.FilterWriter {
switch ftype {
case base.TableFilter:
return newTableFilterWriter(int(p))
default:
panic(fmt.Sprintf("unknown filter type: %v", ftype))
}
func (p FilterPolicy) NewWriter() base.FilterWriter {
return newTableFilterWriter(int(p))
}
5 changes: 2 additions & 3 deletions bloom/bloom_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,6 @@ import (
"strings"
"testing"

"github.com/cockroachdb/pebble/internal/base"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -37,7 +36,7 @@ func (f tableFilter) String() string {
}

func newTableFilter(bitsPerKey int, keys ...[]byte) tableFilter {
w := FilterPolicy(bitsPerKey).NewWriter(base.TableFilter)
w := FilterPolicy(bitsPerKey).NewWriter()
for _, key := range keys {
w.AddKey(key)
}
Expand Down Expand Up @@ -210,7 +209,7 @@ func BenchmarkBloomFilter(b *testing.B) {
b.ResetTimer()
policy := FilterPolicy(10)
for i := 0; i < b.N; i++ {
w := policy.NewWriter(base.TableFilter)
w := policy.NewWriter()
for _, key := range keys {
w.AddKey(key)
}
Expand Down
1 change: 0 additions & 1 deletion cmd/pebble/db.go
Original file line number Diff line number Diff line change
Expand Up @@ -100,7 +100,6 @@ func newPebbleDB(dir string) DB {
l.BlockSize = 32 << 10 // 32 KB
l.IndexBlockSize = 256 << 10 // 256 KB
l.FilterPolicy = bloom.FilterPolicy(10)
l.FilterType = pebble.TableFilter
}
opts.Levels[6].FilterPolicy = pebble.NoFilterPolicy
opts.FlushSplitBytes = opts.TargetFileSizes[0]
Expand Down
1 change: 0 additions & 1 deletion external_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -208,7 +208,6 @@ func buildSeparatedValuesDB(
o.Levels[0].BlockSize = 32 << 10 // 32 KB
o.Levels[0].IndexBlockSize = 256 << 10 // 256 KB
o.Levels[0].FilterPolicy = bloom.FilterPolicy(10)
o.Levels[0].FilterType = pebble.TableFilter
db, err := pebble.Open("", o)
require.NoError(tb, err)

Expand Down
26 changes: 5 additions & 21 deletions internal/base/options.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,22 +12,6 @@ const (
SizeClassAwareBlockSizeThreshold = 60
)

// FilterType is the level at which to apply a filter: block or table.
type FilterType int

// The available filter types.
const (
TableFilter FilterType = iota
)

func (t FilterType) String() string {
switch t {
case TableFilter:
return "table"
}
return "unknown"
}

// FilterWriter provides an interface for creating filter blocks. See
// FilterPolicy for more details about filters.
type FilterWriter interface {
Expand Down Expand Up @@ -59,20 +43,20 @@ type FilterPolicy interface {
// MayContain returns whether the encoded filter may contain given key.
// False positives are possible, where it returns true for keys not in the
// original set.
MayContain(ftype FilterType, filter, key []byte) bool
MayContain(filter, key []byte) bool

// NewWriter creates a new FilterWriter.
NewWriter(ftype FilterType) FilterWriter
NewWriter() FilterWriter
}

// NoFilterPolicy implements the "none" filter policy.
var NoFilterPolicy FilterPolicy = noFilter{}

type noFilter struct{}

func (noFilter) Name() string { return "none" }
func (noFilter) MayContain(ftype FilterType, filter, key []byte) bool { return true }
func (noFilter) NewWriter(ftype FilterType) FilterWriter { panic("not implemented") }
func (noFilter) Name() string { return "none" }
func (noFilter) MayContain(filter, key []byte) bool { return true }
func (noFilter) NewWriter() FilterWriter { panic("not implemented") }

// BlockPropertyFilter is used in an Iterator to filter sstables and blocks
// within the sstable. It should not maintain any per-sstable state, and must
Expand Down
1 change: 0 additions & 1 deletion merging_iter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -575,7 +575,6 @@ func buildLevelsForMergingIterSeqSeek(
}
if writeBloomFilters {
writerOptions.FilterPolicy = filterPolicy
writerOptions.FilterType = base.TableFilter
}
if forceTwoLevelIndex {
if i == 0 && j == 0 {
Expand Down
16 changes: 1 addition & 15 deletions options.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,14 +49,6 @@ const (
type SpanPolicy = base.SpanPolicy
type ValueStoragePolicyAdjustment = base.ValueStoragePolicyAdjustment

// FilterType exports the base.FilterType type.
type FilterType = base.FilterType

// Exported TableFilter constants.
const (
TableFilter = base.TableFilter
)

// FilterWriter exports the base.FilterWriter type.
type FilterWriter = base.FilterWriter

Expand Down Expand Up @@ -463,10 +455,6 @@ type LevelOptions struct {
// the previous level for all other levels.
FilterPolicy FilterPolicy

// FilterType is a legacy field. The default and only possible value is
// TableFilter.
FilterType FilterType

// IndexBlockSize is the target uncompressed size in bytes of each index
// block. When the index block size is larger than this target, two-level
// indexes are automatically enabled. Setting this option to a large value
Expand Down Expand Up @@ -1832,7 +1820,7 @@ func (o *Options) String() string {
fmt.Fprintf(&buf, " block_size_threshold=%d\n", l.BlockSizeThreshold)
fmt.Fprintf(&buf, " compression=%s\n", l.Compression().Name)
fmt.Fprintf(&buf, " filter_policy=%s\n", l.FilterPolicy.Name())
fmt.Fprintf(&buf, " filter_type=%s\n", l.FilterType)
fmt.Fprintf(&buf, " filter_type=table\n")
fmt.Fprintf(&buf, " index_block_size=%d\n", l.IndexBlockSize)
fmt.Fprintf(&buf, " target_file_size=%d\n", o.TargetFileSizes[i])
}
Expand Down Expand Up @@ -2350,7 +2338,6 @@ func (o *Options) Parse(s string, hooks *ParseHooks) error {
case "filter_type":
switch value {
case "table":
l.FilterType = TableFilter
default:
// Tolerate unknown options, but log them.
if o.Logger != nil {
Expand Down Expand Up @@ -2629,7 +2616,6 @@ func (o *Options) MakeWriterOptions(level int, format sstable.TableFormat) sstab
writerOpts.BlockSizeThreshold = levelOpts.BlockSizeThreshold
writerOpts.Compression = levelOpts.Compression()
writerOpts.FilterPolicy = levelOpts.FilterPolicy
writerOpts.FilterType = levelOpts.FilterType
writerOpts.IndexBlockSize = levelOpts.IndexBlockSize
return writerOpts
}
Expand Down
8 changes: 1 addition & 7 deletions sstable/colblk_writer.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,6 @@ import (
"bytes"
"context"
"encoding/binary"
"fmt"
"math"
"slices"
"sync"
Expand Down Expand Up @@ -151,12 +150,7 @@ func newColumnarWriter(
w.valueBlock = valblk.NewWriter(flushGovernor, &w.layout.physBlockMaker, func(compressedSize int) {})
}
if o.FilterPolicy != base.NoFilterPolicy {
switch o.FilterType {
case TableFilter:
w.filterBlock = newTableFilterWriter(o.FilterPolicy)
default:
panic(fmt.Sprintf("unknown filter type: %v", o.FilterType))
}
w.filterBlock = newTableFilterWriter(o.FilterPolicy)
}

numBlockPropertyCollectors := len(o.BlockPropertyCollectors)
Expand Down
4 changes: 2 additions & 2 deletions sstable/filter.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ func newTableFilterReader(policy FilterPolicy, metrics *FilterMetricsTracker) *t
}

func (f *tableFilterReader) mayContain(data, key []byte) bool {
mayContain := f.policy.MayContain(TableFilter, data, key)
mayContain := f.policy.MayContain(data, key)
if f.metrics != nil {
if mayContain {
f.metrics.misses.Add(1)
Expand All @@ -77,7 +77,7 @@ type tableFilterWriter struct {
func newTableFilterWriter(policy FilterPolicy) *tableFilterWriter {
return &tableFilterWriter{
policy: policy,
writer: policy.NewWriter(TableFilter),
writer: policy.NewWriter(),
}
}

Expand Down
17 changes: 0 additions & 17 deletions sstable/options.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,14 +41,6 @@ var ignoredInternalProperties = map[string]struct{}{
"rocksdb.compression_options": {},
}

// FilterType exports the base.FilterType type.
type FilterType = base.FilterType

// Exported TableFilter constants.
const (
TableFilter = base.TableFilter
)

// FilterWriter exports the base.FilterWriter type.
type FilterWriter = base.FilterWriter

Expand Down Expand Up @@ -199,15 +191,6 @@ type WriterOptions struct {
// The default value is NoFilterPolicy.
FilterPolicy FilterPolicy

// FilterType defines whether an existing filter policy is applied at a
// block-level or table-level. Block-level filters use less memory to create,
// but are slower to access as a check for the key in the index must first be
// performed to locate the filter block. A table-level filter will require
// memory proportional to the number of keys in an sstable to create, but
// avoids the index lookup when determining if a key is present. Table-level
// filters should be preferred except under constrained memory situations.
FilterType FilterType

// IndexBlockSize is the target uncompressed size in bytes of each index
// block. When the index block size is larger than this target, two-level
// indexes are automatically enabled. Setting this option to a large value
Expand Down
2 changes: 1 addition & 1 deletion sstable/reader_iter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -733,7 +733,7 @@ func (c *controllableFilterPolicy) Name() string {
return c.name
}

func (c *controllableFilterPolicy) MayContain(ftype FilterType, filter, key []byte) bool {
func (c *controllableFilterPolicy) MayContain(filter, key []byte) bool {
return c.mayContainResult
}

Expand Down
4 changes: 2 additions & 2 deletions sstable/reader_lazy_loading_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -113,7 +113,7 @@ type testFilterPolicyImpl struct{}

func (testFilterPolicyImpl) Name() string { return "test.filter" }

func (testFilterPolicyImpl) MayContain(ftype FilterType, filter, key []byte) bool {
func (testFilterPolicyImpl) MayContain(filter, key []byte) bool {
// For the test, return false for keys starting with "nonexistent" to simulate bloom filter miss
prefix := "nonexistent"
if len(key) >= len(prefix) && string(key[:len(prefix)]) == prefix {
Expand All @@ -122,7 +122,7 @@ func (testFilterPolicyImpl) MayContain(ftype FilterType, filter, key []byte) boo
return true
}

func (testFilterPolicyImpl) NewWriter(ftype FilterType) FilterWriter {
func (testFilterPolicyImpl) NewWriter() FilterWriter {
return &testFilterWriter{}
}

Expand Down
3 changes: 0 additions & 3 deletions sstable/reader_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -389,17 +389,14 @@ func TestReader(t *testing.T) {
"bloom10bit": {
// The standard policy.
FilterPolicy: bloom.FilterPolicy(10),
FilterType: base.TableFilter,
},
"bloom1bit": {
// A policy with many false positives.
FilterPolicy: bloom.FilterPolicy(1),
FilterType: base.TableFilter,
},
"bloom100bit": {
// A policy unlikely to have false positives.
FilterPolicy: bloom.FilterPolicy(100),
FilterType: base.TableFilter,
},
}

Expand Down
7 changes: 1 addition & 6 deletions sstable/rowblk_writer.go
Original file line number Diff line number Diff line change
Expand Up @@ -1739,12 +1739,7 @@ func newRowWriter(writable objstorage.Writable, o WriterOptions) *RawRowWriter {
}

if o.FilterPolicy != base.NoFilterPolicy {
switch o.FilterType {
case TableFilter:
w.filter = newTableFilterWriter(o.FilterPolicy)
default:
panic(fmt.Sprintf("unknown filter type: %v", o.FilterType))
}
w.filter = newTableFilterWriter(o.FilterPolicy)
}

w.props.ComparerName = o.Comparer.Name
Expand Down
8 changes: 4 additions & 4 deletions sstable/table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -385,14 +385,14 @@ type countingFilterPolicy struct {
trueNegatives int
}

func (c *countingFilterPolicy) MayContain(ftype FilterType, filter, key []byte) bool {
func (c *countingFilterPolicy) MayContain(filter, key []byte) bool {
got := true
if c.degenerate {
// When degenerate is true, we override the embedded FilterPolicy's
// MayContain method to always return true. Doing so is a valid, if
// inefficient, implementation of the FilterPolicy interface.
} else {
got = c.FilterPolicy.MayContain(ftype, filter, key)
got = c.FilterPolicy.MayContain(filter, key)
}
wordCount := hamletWordCount()
_, want := wordCount[string(key)]
Expand Down Expand Up @@ -422,7 +422,7 @@ func TestWriterRoundTrip(t *testing.T) {
t.Run(fmt.Sprintf("bloom=%s", name), func(t *testing.T) {
fs := vfs.NewMem()
err := buildHamletTestSST(
fs, "test.sst", block.DefaultCompression, fp, TableFilter,
fs, "test.sst", block.DefaultCompression, fp,
nil /* comparer */, blockSize, indexBlockSize,
)
require.NoError(t, err)
Expand Down Expand Up @@ -525,7 +525,7 @@ func TestMetaIndexEntriesSorted(t *testing.T) {
defer leaktest.AfterTest(t)()
fs := vfs.NewMem()
err := buildHamletTestSST(fs, "test.sst", block.DefaultCompression, nil, /* filter policy */
TableFilter, nil, 4096, 4096)
nil, 4096, 4096)
require.NoError(t, err)
f, err := fs.Open("test.sst")
require.NoError(t, err)
Expand Down
4 changes: 1 addition & 3 deletions sstable/test_fixtures.go
Original file line number Diff line number Diff line change
Expand Up @@ -122,7 +122,6 @@ func buildHamletTestSST(
filename string,
compression *block.CompressionProfile,
fp FilterPolicy,
ftype FilterType,
comparer *Comparer,
blockSize int,
indexBlockSize int,
Expand All @@ -141,7 +140,6 @@ func buildHamletTestSST(
Comparer: comparer,
Compression: compression,
FilterPolicy: fp,
FilterType: ftype,
IndexBlockSize: indexBlockSize,
MergerName: "nullptr",
TableFormat: fixtureFormat,
Expand Down Expand Up @@ -257,7 +255,7 @@ func (tf TestFixtureInfo) Build(fs vfs.FS, filename string) error {
}

return buildHamletTestSST(
fs, filename, tf.Compression, fp, base.TableFilter,
fs, filename, tf.Compression, fp,
comparer,
fixtureBlockSize,
tf.IndexBlockSize,
Expand Down