Skip to content

Commit

Permalink
sstable: rename NewIterWithBlockPropertyFiltersAndContextEtc
Browse files Browse the repository at this point in the history
Rename to `NewPointIter`.
  • Loading branch information
RaduBerinde committed Jul 30, 2024
1 parent a6580b1 commit b6400d6
Show file tree
Hide file tree
Showing 10 changed files with 33 additions and 29 deletions.
2 changes: 1 addition & 1 deletion external_iterator.go
Original file line number Diff line number Diff line change
Expand Up @@ -156,7 +156,7 @@ func createExternalPointIter(ctx context.Context, it *Iterator) (topLevelIterato
// BlockPropertiesFilterer that includes obsoleteKeyBlockPropertyFilter.
transforms := sstable.IterTransforms{SyntheticSeqNum: sstable.SyntheticSeqNum(seqNum)}
seqNum--
pointIter, err = r.NewIterWithBlockPropertyFiltersAndContextEtc(
pointIter, err = r.NewPointIter(
ctx, transforms, it.opts.LowerBound, it.opts.UpperBound, nil, /* BlockPropertiesFilterer */
sstable.NeverUseFilterBlock,
&it.stats.InternalStats, it.opts.CategoryAndQoS, nil,
Expand Down
2 changes: 1 addition & 1 deletion level_iter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -173,7 +173,7 @@ func (lt *levelIterTest) newIters(
transforms := file.IterTransforms()
var set iterSet
if kinds.Point() {
iter, err := lt.readers[file.FileNum].NewIterWithBlockPropertyFiltersAndContextEtc(
iter, err := lt.readers[file.FileNum].NewPointIter(
ctx, transforms,
opts.LowerBound, opts.UpperBound, nil, sstable.AlwaysUseFilterBlock, iio.stats, sstable.CategoryAndQoS{},
nil, sstable.MakeTrivialReaderProvider(lt.readers[file.FileNum]))
Expand Down
2 changes: 1 addition & 1 deletion merging_iter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -175,7 +175,7 @@ func TestMergingIterCornerCases(t *testing.T) {
}
}
if kinds.Point() {
set.point, err = r.NewIterWithBlockPropertyFiltersAndContextEtc(
set.point, err = r.NewPointIter(
context.Background(),
sstable.NoTransforms,
opts.GetLowerBound(), opts.GetUpperBound(), nil, sstable.AlwaysUseFilterBlock, iio.stats,
Expand Down
4 changes: 2 additions & 2 deletions sstable/block_property_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -971,7 +971,7 @@ func TestBlockProperties(t *testing.T) {
} else if !ok {
return "filter excludes entire table"
}
iter, err := r.NewIterWithBlockPropertyFiltersAndContextEtc(
iter, err := r.NewPointIter(
context.Background(),
NoTransforms, lower, upper, filterer, NeverUseFilterBlock, &stats,
CategoryAndQoS{}, nil, MakeTrivialReaderProvider(r))
Expand Down Expand Up @@ -1054,7 +1054,7 @@ func TestBlockProperties_BoundLimited(t *testing.T) {
} else if !ok {
return "filter excludes entire table"
}
iter, err := r.NewIterWithBlockPropertyFiltersAndContextEtc(
iter, err := r.NewPointIter(
context.Background(),
NoTransforms, lower, upper, filterer, NeverUseFilterBlock, &stats,
CategoryAndQoS{}, nil, MakeTrivialReaderProvider(r))
Expand Down
4 changes: 2 additions & 2 deletions sstable/random_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -92,15 +92,15 @@ func runErrorInjectionTest(t *testing.T, seed int64) {
}, nil, nil)
}

// TOOD(jackson): NewIterWithBlockPropertyFiltersAndContextEtc returns an
// TOOD(jackson): NewPointIter returns an
// iterator over point keys only. Should we add variants of this test that run
// random operations on the range deletion and range key iterators?
var stats base.InternalIteratorStats
filterBlockSizeLimit := AlwaysUseFilterBlock
if rng.Intn(2) == 1 {
filterBlockSizeLimit = NeverUseFilterBlock
}
it, err := r.NewIterWithBlockPropertyFiltersAndContextEtc(
it, err := r.NewPointIter(
context.Background(),
NoTransforms,
nil /* lower TODO */, nil, /* upper TODO */
Expand Down
19 changes: 9 additions & 10 deletions sstable/reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -151,13 +151,12 @@ func (r *Reader) Close() error {
return nil
}

// NewIterWithBlockPropertyFiltersAndContextEtc returns an iterator for the
// point keys in the table.
// NewPointIter returns an iterator for the point keys in the table.
//
// If transform.HideObsoletePoints is set, the callee assumes that filterer
// already includes obsoleteKeyBlockPropertyFilter. The caller can satisfy this
// contract by first calling TryAddBlockPropertyFilterForHideObsoletePoints.
func (r *Reader) NewIterWithBlockPropertyFiltersAndContextEtc(
func (r *Reader) NewPointIter(
ctx context.Context,
transforms IterTransforms,
lower, upper []byte,
Expand All @@ -168,14 +167,14 @@ func (r *Reader) NewIterWithBlockPropertyFiltersAndContextEtc(
statsCollector *CategoryStatsCollector,
rp ReaderProvider,
) (Iterator, error) {
return r.newIterWithBlockPropertyFiltersAndContext(
return r.newPointIter(
ctx, transforms, lower, upper, filterer, filterBlockSizeLimit,
stats, categoryAndQoS, statsCollector, rp, nil)
}

// TryAddBlockPropertyFilterForHideObsoletePoints is expected to be called
// before the call to NewIterWithBlockPropertyFiltersAndContextEtc, to get the
// value of hideObsoletePoints and potentially add a block property filter.
// before the call to NewPointIter, to get the value of hideObsoletePoints and
// potentially add a block property filter.
func (r *Reader) TryAddBlockPropertyFilterForHideObsoletePoints(
snapshotForHideObsoletePoints base.SeqNum,
fileLargestSeqNum base.SeqNum,
Expand All @@ -189,7 +188,7 @@ func (r *Reader) TryAddBlockPropertyFilterForHideObsoletePoints(
return hideObsoletePoints, pointKeyFilters
}

func (r *Reader) newIterWithBlockPropertyFiltersAndContext(
func (r *Reader) newPointIter(
ctx context.Context,
transforms IterTransforms,
lower, upper []byte,
Expand Down Expand Up @@ -223,15 +222,15 @@ func (r *Reader) newIterWithBlockPropertyFiltersAndContext(
}

// NewIter returns an iterator for the point keys in the table. It is a
// simplified version of NewIterWithBlockPropertyFiltersAndContextEtc and
// should only be used for tests and tooling.
// simplified version of NewPointIter and should only be used for tests and
// tooling.
//
// NewIter must only be used when the Reader is guaranteed to outlive any
// LazyValues returned from the iter.
func (r *Reader) NewIter(transforms IterTransforms, lower, upper []byte) (Iterator, error) {
// TODO(radu): we should probably not use bloom filters in this case, as there
// likely isn't a cache set up.
return r.NewIterWithBlockPropertyFiltersAndContextEtc(
return r.NewPointIter(
context.TODO(), transforms, lower, upper, nil, AlwaysUseFilterBlock,
nil /* stats */, CategoryAndQoS{}, nil /* statsCollector */, MakeTrivialReaderProvider(r))
}
Expand Down
2 changes: 1 addition & 1 deletion sstable/reader_common.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ type CommonReader interface {
ctx context.Context, transforms FragmentIterTransforms,
) (keyspan.FragmentIterator, error)

NewIterWithBlockPropertyFiltersAndContextEtc(
NewPointIter(
ctx context.Context,
transforms IterTransforms,
lower, upper []byte,
Expand Down
8 changes: 4 additions & 4 deletions sstable/reader_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -448,7 +448,7 @@ func runVirtualReaderTest(t *testing.T, path string, blockSize, indexBlockSize i
}

transforms := IterTransforms{SyntheticSuffix: syntheticSuffix}
iter, err := v.NewIterWithBlockPropertyFiltersAndContextEtc(
iter, err := v.NewPointIter(
context.Background(), transforms, lower, upper, filterer, NeverUseFilterBlock,
&stats, CategoryAndQoS{}, nil, MakeTrivialReaderProvider(r))
if err != nil {
Expand Down Expand Up @@ -827,7 +827,7 @@ func runTestReader(t *testing.T, o WriterOptions, dir string, r *Reader, printVa
return "table does not intersect BlockPropertyFilter"
}
}
iter, err := r.NewIterWithBlockPropertyFiltersAndContextEtc(
iter, err := r.NewPointIter(
context.Background(),
transforms,
nil, /* lower */
Expand Down Expand Up @@ -1323,7 +1323,7 @@ func TestRandomizedPrefixSuffixRewriter(t *testing.T) {
}
eReader, err := newReader(f, opts)
require.NoError(t, err)
iter, err := eReader.newIterWithBlockPropertyFiltersAndContext(
iter, err := eReader.newPointIter(
context.Background(),
block.IterTransforms{SyntheticSuffix: syntheticSuffix, SyntheticPrefix: syntheticPrefix},
nil, nil, nil,
Expand Down Expand Up @@ -2448,7 +2448,7 @@ func BenchmarkIteratorScanObsolete(b *testing.B) {
}
}
transforms := IterTransforms{HideObsoletePoints: hideObsoletePoints}
iter, err := r.NewIterWithBlockPropertyFiltersAndContextEtc(
iter, err := r.NewPointIter(
context.Background(), transforms, nil, nil, filterer,
AlwaysUseFilterBlock, nil, CategoryAndQoS{}, nil,
MakeTrivialReaderProvider(r))
Expand Down
17 changes: 11 additions & 6 deletions sstable/reader_virtual.go
Original file line number Diff line number Diff line change
Expand Up @@ -101,11 +101,16 @@ func (v *VirtualReader) NewCompactionIter(
transforms, categoryAndQoS, statsCollector, rp, &v.vState, bufferPool)
}

// NewIterWithBlockPropertyFiltersAndContextEtc wraps
// Reader.NewIterWithBlockPropertyFiltersAndContext. We assume that the passed
// in [lower, upper) bounds will have at least some overlap with the virtual
// sstable bounds. No overlap is not currently supported in the iterator.
func (v *VirtualReader) NewIterWithBlockPropertyFiltersAndContextEtc(
// NewPointIter returns an iterator for the point keys in the table.
//
// If transform.HideObsoletePoints is set, the callee assumes that filterer
// already includes obsoleteKeyBlockPropertyFilter. The caller can satisfy this
// contract by first calling TryAddBlockPropertyFilterForHideObsoletePoints.
//
// We assume that the [lower, upper) bounds (if specified) will have at least
// some overlap with the virtual sstable bounds. No overlap is not currently
// supported in the iterator.
func (v *VirtualReader) NewPointIter(
ctx context.Context,
transforms IterTransforms,
lower, upper []byte,
Expand All @@ -116,7 +121,7 @@ func (v *VirtualReader) NewIterWithBlockPropertyFiltersAndContextEtc(
statsCollector *CategoryStatsCollector,
rp ReaderProvider,
) (Iterator, error) {
return v.reader.newIterWithBlockPropertyFiltersAndContext(
return v.reader.newPointIter(
ctx, transforms, lower, upper, filterer, filterBlockSizeLimit,
stats, categoryAndQoS, statsCollector, rp, &v.vState)
}
Expand Down
2 changes: 1 addition & 1 deletion table_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -596,7 +596,7 @@ func (c *tableCacheShard) newPointIter(
transforms, categoryAndQoS, dbOpts.sstStatsCollector, rp,
internalOpts.bufferPool)
} else {
iter, err = cr.NewIterWithBlockPropertyFiltersAndContextEtc(
iter, err = cr.NewPointIter(
ctx, transforms, opts.GetLowerBound(), opts.GetUpperBound(), filterer, filterBlockSizeLimit,
internalOpts.stats, categoryAndQoS, dbOpts.sstStatsCollector, rp)
}
Expand Down

0 comments on commit b6400d6

Please sign in to comment.