Skip to content

Commit

Permalink
Merge #82789
Browse files Browse the repository at this point in the history
82789: storage: clean up `pebbleIterator` construction/initialization r=nicktrav a=erikgrinaker

`pebbleIterator` construction and initialization has grown a bit bloated
and convoluted over time. This patch makes some minor changes to better
express the purpose of the various options:

* `init()` now simply initializes the `pebbleIterator` by resetting the
  struct and creating the iterator options, using and reconfiguring a
  raw Pebble iterator if given.

* `newPebbleIterator()` creates a new iterator from a given Pebble
  reader.

* `newPebbleIteratorByCloning()` creates a new iterator by cloning a
  given raw Pebble iterator and reconfiguring it.

* `initReuseOrCreate()` is a convenience method specialized for batch
  iterator reuse. It (re-)initializes an existing `pebbleIterator` by
  either:

  1. Cloning and reconfiguring a given raw Pebble iterator.
  2. Using and reconfiguring a given raw Pebble iterator.
  3. Creating a new raw Pebble iterator.

Additionally, the `clonableIter` interface has been removed, in favor of
passing `pebble.Iterator` directly.

Release note: None

Co-authored-by: Erik Grinaker <[email protected]>
  • Loading branch information
craig[bot] and erikgrinaker committed Jun 16, 2022
2 parents 8aea0c7 + f22f914 commit 2b082dd
Show file tree
Hide file tree
Showing 5 changed files with 83 additions and 82 deletions.
4 changes: 2 additions & 2 deletions pkg/storage/intent_interleaving_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -257,8 +257,8 @@ func newIntentInterleavingIterator(reader Reader, opts IterOptions) MVCCIterator
if reader.ConsistentIterators() {
iter = reader.NewMVCCIterator(MVCCKeyIterKind, opts)
} else {
iter = newPebbleIterator(
nil, intentIter.GetRawIter(), opts, StandardDurability, reader.SupportsRangeKeys())
iter = newPebbleIteratorByCloning(
intentIter.GetRawIter(), opts, StandardDurability, reader.SupportsRangeKeys())
}

*iiIter = intentInterleavingIter{
Expand Down
4 changes: 2 additions & 2 deletions pkg/storage/mvcc.go
Original file line number Diff line number Diff line change
Expand Up @@ -3613,8 +3613,8 @@ func MVCCResolveWriteIntentRange(
mvccIter = rw.NewMVCCIterator(MVCCKeyIterKind, iterOpts)
} else {
// For correctness, we need mvccIter to be consistent with engineIter.
mvccIter = newPebbleIterator(
nil, engineIter.GetRawIter(), iterOpts, StandardDurability, rw.SupportsRangeKeys())
mvccIter = newPebbleIteratorByCloning(
engineIter.GetRawIter(), iterOpts, StandardDurability, rw.SupportsRangeKeys())
}
iterAndBuf := GetBufUsingIter(mvccIter)
defer func() {
Expand Down
41 changes: 17 additions & 24 deletions pkg/storage/pebble.go
Original file line number Diff line number Diff line change
Expand Up @@ -1002,10 +1002,7 @@ func (p *Pebble) NewMVCCIterator(iterKind MVCCIterKind, opts IterOptions) MVCCIt
return iter
}

iter := newPebbleIterator(p.db, nil, opts, StandardDurability, p.SupportsRangeKeys())
if iter == nil {
panic("couldn't create a new iterator")
}
iter := newPebbleIterator(p.db, opts, StandardDurability, p.SupportsRangeKeys())
if util.RaceEnabled {
return wrapInUnsafeIter(iter)
}
Expand All @@ -1017,11 +1014,7 @@ func (p *Pebble) NewEngineIterator(opts IterOptions) EngineIterator {
if opts.KeyTypes != IterKeyTypePointsOnly {
panic("EngineIterator does not support range keys")
}
iter := newPebbleIterator(p.db, nil, opts, StandardDurability, p.SupportsRangeKeys())
if iter == nil {
panic("couldn't create a new iterator")
}
return iter
return newPebbleIterator(p.db, opts, StandardDurability, p.SupportsRangeKeys())
}

// ConsistentIterators implements the Engine interface.
Expand Down Expand Up @@ -1787,8 +1780,8 @@ type pebbleReadOnly struct {
prefixEngineIter pebbleIterator
normalEngineIter pebbleIterator

iter cloneableIter
iterUnused bool
iter *pebble.Iterator
iterUsed bool // avoids cloning after PinEngineStateForIterators()
durability DurabilityRequirement
closed bool
}
Expand Down Expand Up @@ -1832,7 +1825,7 @@ func (p *pebbleReadOnly) Close() {
panic("closing an already-closed pebbleReadOnly")
}
p.closed = true
if p.iterUnused {
if p.iter != nil && !p.iterUsed {
err := p.iter.Close()
if err != nil {
panic(err)
Expand Down Expand Up @@ -1929,18 +1922,19 @@ func (p *pebbleReadOnly) NewMVCCIterator(iterKind MVCCIterKind, opts IterOptions
iter = &p.prefixIter
}
if iter.inuse {
return newPebbleIterator(p.parent.db, p.iter, opts, p.durability, p.SupportsRangeKeys())
return newPebbleIteratorByCloning(p.iter, opts, p.durability, p.SupportsRangeKeys())
}

if iter.iter != nil {
iter.setOptions(opts, p.durability)
} else {
iter.init(p.parent.db, p.iter, p.iterUnused, opts, p.durability, p.SupportsRangeKeys())
iter.initReuseOrCreate(
p.parent.db, p.iter, p.iterUsed, opts, p.durability, p.SupportsRangeKeys())
if p.iter == nil {
// For future cloning.
p.iter = iter.iter
}
p.iterUnused = false
p.iterUsed = true
iter.reusable = true
}

Expand All @@ -1966,18 +1960,19 @@ func (p *pebbleReadOnly) NewEngineIterator(opts IterOptions) EngineIterator {
iter = &p.prefixEngineIter
}
if iter.inuse {
return newPebbleIterator(p.parent.db, p.iter, opts, p.durability, p.SupportsRangeKeys())
return newPebbleIteratorByCloning(p.iter, opts, p.durability, p.SupportsRangeKeys())
}

if iter.iter != nil {
iter.setOptions(opts, p.durability)
} else {
iter.init(p.parent.db, p.iter, p.iterUnused, opts, p.durability, p.SupportsRangeKeys())
iter.initReuseOrCreate(
p.parent.db, p.iter, p.iterUsed, opts, p.durability, p.SupportsRangeKeys())
if p.iter == nil {
// For future cloning.
p.iter = iter.iter
}
p.iterUnused = false
p.iterUsed = true
iter.reusable = true
}

Expand All @@ -2003,10 +1998,8 @@ func (p *pebbleReadOnly) PinEngineStateForIterators() error {
o = &pebble.IterOptions{OnlyReadGuaranteedDurable: true}
}
p.iter = p.parent.db.NewIter(o)
// Since the iterator is being created just to pin the state of the engine
// for future iterators, we'll avoid cloning it the next time we want an
// iterator and instead just re-use what we created here.
p.iterUnused = true
// NB: p.iterUsed == false avoids cloning this in NewMVCCIterator(), since
// we've just created it.
}
return nil
}
Expand Down Expand Up @@ -2188,7 +2181,7 @@ func (p *pebbleSnapshot) NewMVCCIterator(iterKind MVCCIterKind, opts IterOptions
}

iter := MVCCIterator(newPebbleIterator(
p.snapshot, nil, opts, StandardDurability, p.SupportsRangeKeys()))
p.snapshot, opts, StandardDurability, p.SupportsRangeKeys()))
if util.RaceEnabled {
iter = wrapInUnsafeIter(iter)
}
Expand All @@ -2200,7 +2193,7 @@ func (p pebbleSnapshot) NewEngineIterator(opts IterOptions) EngineIterator {
if opts.KeyTypes != IterKeyTypePointsOnly {
panic("EngineIterator does not support range keys")
}
return newPebbleIterator(p.snapshot, nil, opts, StandardDurability, p.SupportsRangeKeys())
return newPebbleIterator(p.snapshot, opts, StandardDurability, p.SupportsRangeKeys())
}

// ConsistentIterators implements the Reader interface.
Expand Down
26 changes: 13 additions & 13 deletions pkg/storage/pebble_batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,9 +48,9 @@ type pebbleBatch struct {
prefixEngineIter pebbleIterator
normalEngineIter pebbleIterator

iter cloneableIter
iter *pebble.Iterator
iterUsed bool // avoids cloning after PinEngineStateForIterators()
writeOnly bool
iterUnused bool
containsRangeKeys bool
closed bool

Expand Down Expand Up @@ -114,7 +114,7 @@ func (p *pebbleBatch) Close() {
}
p.closed = true

if p.iterUnused {
if p.iter != nil && !p.iterUsed {
if err := p.iter.Close(); err != nil {
panic(err)
}
Expand Down Expand Up @@ -221,18 +221,19 @@ func (p *pebbleBatch) NewMVCCIterator(iterKind MVCCIterKind, opts IterOptions) M
handle = p.db
}
if iter.inuse {
return newPebbleIterator(handle, p.iter, opts, StandardDurability, p.SupportsRangeKeys())
return newPebbleIteratorByCloning(p.iter, opts, StandardDurability, p.SupportsRangeKeys())
}

if iter.iter != nil {
iter.setOptions(opts, StandardDurability)
} else {
iter.init(handle, p.iter, p.iterUnused, opts, StandardDurability, p.SupportsRangeKeys())
iter.initReuseOrCreate(
handle, p.iter, p.iterUsed, opts, StandardDurability, p.SupportsRangeKeys())
if p.iter == nil {
// For future cloning.
p.iter = iter.iter
}
p.iterUnused = false
p.iterUsed = true
}

iter.inuse = true
Expand Down Expand Up @@ -261,18 +262,19 @@ func (p *pebbleBatch) NewEngineIterator(opts IterOptions) EngineIterator {
handle = p.db
}
if iter.inuse {
return newPebbleIterator(handle, p.iter, opts, StandardDurability, p.SupportsRangeKeys())
return newPebbleIteratorByCloning(p.iter, opts, StandardDurability, p.SupportsRangeKeys())
}

if iter.iter != nil {
iter.setOptions(opts, StandardDurability)
} else {
iter.init(handle, p.iter, p.iterUnused, opts, StandardDurability, p.SupportsRangeKeys())
iter.initReuseOrCreate(
handle, p.iter, p.iterUsed, opts, StandardDurability, p.SupportsRangeKeys())
if p.iter == nil {
// For future cloning.
p.iter = iter.iter
}
p.iterUnused = false
p.iterUsed = true
}

iter.inuse = true
Expand All @@ -297,10 +299,8 @@ func (p *pebbleBatch) PinEngineStateForIterators() error {
} else {
p.iter = p.db.NewIter(nil)
}
// Since the iterator is being created just to pin the state of the engine
// for future iterators, we'll avoid cloning it the next time we want an
// iterator and instead just re-use what we created here.
p.iterUnused = true
// NB: p.iterUsed == false avoids cloning this in NewMVCCIterator(). We've
// just created it, so cloning it would just be overhead.
}
return nil
}
Expand Down
90 changes: 49 additions & 41 deletions pkg/storage/pebble_iterator.go
Original file line number Diff line number Diff line change
Expand Up @@ -72,67 +72,75 @@ var pebbleIterPool = sync.Pool{
},
}

type cloneableIter interface {
Clone() (*pebble.Iterator, error)
Close() error
// newPebbleIterator creates a new Pebble iterator for the given Pebble reader.
func newPebbleIterator(
handle pebble.Reader, opts IterOptions, durability DurabilityRequirement, supportsRangeKeys bool,
) *pebbleIterator {
p := pebbleIterPool.Get().(*pebbleIterator)
p.reusable = false // defensive
p.init(nil, opts, durability, supportsRangeKeys)
p.iter = handle.NewIter(&p.options)
return p
}

// Instantiates a new Pebble iterator, or gets one from the pool.
func newPebbleIterator(
handle pebble.Reader,
iterToClone cloneableIter,
opts IterOptions,
durability DurabilityRequirement,
supportsRangeKeys bool,
// newPebbleIteratorByCloning creates a new Pebble iterator by cloning the given
// iterator and reconfiguring it.
func newPebbleIteratorByCloning(
iter *pebble.Iterator, opts IterOptions, durability DurabilityRequirement, supportsRangeKeys bool,
) *pebbleIterator {
iter := pebbleIterPool.Get().(*pebbleIterator)
iter.reusable = false // defensive
iter.init(handle, iterToClone, false /* iterUnused */, opts, durability, supportsRangeKeys)
return iter
var err error
if iter, err = iter.Clone(); err != nil {
panic(err)
}
p := pebbleIterPool.Get().(*pebbleIterator)
p.reusable = false // defensive
p.init(iter, opts, durability, supportsRangeKeys)
return p
}

// init resets this pebbleIterator for use with the specified arguments. The
// current instance could either be a cached pebbleIterator (e.g. in
// pebbleBatch), or a newly-instantiated one through newPebbleIterator. The
// underlying *pebble.Iterator is created using iterToClone, if non-nil, else it
// is created using handle.
// init resets this pebbleIterator for use with the specified arguments,
// reconfiguring the given iter. It is valid to pass a nil iter and then create
// p.iter using p.options, to avoid redundant reconfiguration via SetOptions().
func (p *pebbleIterator) init(
handle pebble.Reader,
iterToClone cloneableIter,
iterUnused bool,
opts IterOptions,
durability DurabilityRequirement,
supportsRangeKeys bool, // TODO(erikgrinaker): remove after 22.2.
iter *pebble.Iterator, opts IterOptions, durability DurabilityRequirement, supportsRangeKeys bool,
) {
*p = pebbleIterator{
iter: iter,
inuse: true,
keyBuf: p.keyBuf,
lowerBoundBuf: p.lowerBoundBuf,
upperBoundBuf: p.upperBoundBuf,
rangeKeyMaskingBuf: p.rangeKeyMaskingBuf,
reusable: p.reusable,
supportsRangeKeys: supportsRangeKeys,
}
p.setOptions(opts, durability)
}

if iterToClone != nil {
if iterUnused {
// NB: If the iterator was never used (at the time of writing, this means
// that the iterator was created by `PinEngineStateForIterators()`), we
// don't need to clone it.
p.iter = iterToClone.(*pebble.Iterator)
} else {
var err error
if p.iter, err = iterToClone.Clone(); err != nil {
panic(err)
}
// initReuseOrCreate is a convenience method that (re-)initializes an existing
// pebbleIterator in one out of three ways:
//
// 1. iter != nil && !clone: use and reconfigure the given raw Pebble iterator.
// 2. iter != nil && clone: clone and reconfigure the given raw Pebble iterator.
// 3. iter == nil: create a new iterator from handle.
func (p *pebbleIterator) initReuseOrCreate(
handle pebble.Reader,
iter *pebble.Iterator,
clone bool,
opts IterOptions,
durability DurabilityRequirement,
supportsRangeKeys bool, // TODO(erikgrinaker): remove after 22.2
) {
if clone && iter != nil {
var err error
if iter, err = iter.Clone(); err != nil {
panic(err)
}
}

p.setOptions(opts, durability)

if p.iter == nil {
p.init(iter, opts, durability, supportsRangeKeys)
if iter == nil {
p.iter = handle.NewIter(&p.options)
}
p.inuse = true
}

// setOptions updates the options for a pebbleIterator. If p.iter is non-nil, it
Expand Down

0 comments on commit 2b082dd

Please sign in to comment.