Skip to content

Commit

Permalink
compact: some Iter cleanup
Browse files Browse the repository at this point in the history
Remove the `Valid`, `Key`, `Value` methods which were only used during
tests (and the `valid` field which is now unused).
  • Loading branch information
RaduBerinde committed May 6, 2024
1 parent c58ac88 commit f03e7ef
Show file tree
Hide file tree
Showing 2 changed files with 15 additions and 51 deletions.
43 changes: 3 additions & 40 deletions internal/compact/iterator.go
Original file line number Diff line number Diff line change
Expand Up @@ -186,9 +186,7 @@ type Iter struct {
// Temporary buffer used for storing the previous value, which may be an
// unsafe, i.iter-owned slice that could be altered when the iterator is
// advanced.
valueBuf []byte
// Is the current entry valid?
valid bool
valueBuf []byte
iterKV *base.InternalKV
iterValue []byte
iterStripeChange stripeChangeType
Expand Down Expand Up @@ -405,7 +403,6 @@ func (i *Iter) Next() (*base.InternalKey, []byte) {
}

i.pos = iterPosCurForward
i.valid = false

for i.iterKV != nil {
// If we entered a new snapshot stripe with the same key, any key we
Expand Down Expand Up @@ -475,7 +472,6 @@ func (i *Iter) Next() (*base.InternalKey, []byte) {
// statistics when they apply their own snapshot striping logic.
i.snapshotPinned = false
i.value = i.iterValue
i.valid = true
return &i.key, i.value
}

Expand Down Expand Up @@ -539,7 +535,6 @@ func (i *Iter) Next() (*base.InternalKey, []byte) {
case base.InternalKeyKindDelete:
i.saveKey()
i.value = i.iterValue
i.valid = true
i.skip = true
return &i.key, i.value

Expand Down Expand Up @@ -600,7 +595,6 @@ func (i *Iter) Next() (*base.InternalKey, []byte) {
}
if i.err == nil {
if needDelete {
i.valid = false
if i.closeValueCloser() != nil {
return nil, nil
}
Expand All @@ -611,7 +605,6 @@ func (i *Iter) Next() (*base.InternalKey, []byte) {
return &i.key, i.value
}
if i.err != nil {
i.valid = false
// TODO(sumeer): why is MarkCorruptionError only being called for
// MERGE?
i.err = base.MarkCorruptionError(i.err)
Expand All @@ -620,7 +613,6 @@ func (i *Iter) Next() (*base.InternalKey, []byte) {

default:
i.err = base.CorruptionErrorf("invalid internal key kind: %d", errors.Safe(i.iterKV.Kind()))
i.valid = false
return nil, nil
}
}
Expand All @@ -643,9 +635,6 @@ func (i *Iter) closeValueCloser() error {

i.err = i.valueCloser.Close()
i.valueCloser = nil
if i.err != nil {
i.valid = false
}
return i.err
}

Expand Down Expand Up @@ -763,7 +752,6 @@ func (i *Iter) nextInStripeHelper() stripeChangeType {
kind := i.iterKV.Kind()
i.iterKV = nil
i.err = base.CorruptionErrorf("invalid internal key kind: %d", errors.Safe(kind))
i.valid = false
return newStripeNewKey
}
if i.curSnapshotIdx == origSnapshotIdx {
Expand All @@ -781,7 +769,6 @@ func (i *Iter) setNext() {
// Save the current key.
i.saveKey()
i.value = i.iterValue
i.valid = true
i.maybeZeroSeqnum(i.curSnapshotIdx)

// If this key is already a SETWITHDEL we can early return and skip the remaining
Expand Down Expand Up @@ -819,7 +806,6 @@ func (i *Iter) setNext() {
// Do nothing
default:
i.err = base.CorruptionErrorf("invalid internal key kind: %d", errors.Safe(i.iterKV.Kind()))
i.valid = false
}
default:
panic("pebble: unexpected stripeChangeType: " + strconv.Itoa(int(i.iterStripeChange)))
Expand All @@ -830,7 +816,6 @@ func (i *Iter) setNext() {
func (i *Iter) mergeNext(valueMerger base.ValueMerger) {
// Save the current key.
i.saveKey()
i.valid = true

// Loop looking for older values in the current snapshot stripe and merge
// them.
Expand Down Expand Up @@ -880,7 +865,6 @@ func (i *Iter) mergeNext(valueMerger base.ValueMerger) {
// MERGE + (SET*) -> SET.
i.err = valueMerger.MergeOlder(i.iterValue)
if i.err != nil {
i.valid = false
return
}
i.key.SetKind(base.InternalKeyKindSet)
Expand All @@ -892,13 +876,11 @@ func (i *Iter) mergeNext(valueMerger base.ValueMerger) {
// continue looping.
i.err = valueMerger.MergeOlder(i.iterValue)
if i.err != nil {
i.valid = false
return
}

default:
i.err = base.CorruptionErrorf("invalid internal key kind: %d", errors.Safe(i.iterKV.Kind()))
i.valid = false
return
}
}
Expand All @@ -918,7 +900,6 @@ func (i *Iter) singleDeleteNext() bool {
// Save the current key.
i.saveKey()
i.value = i.iterValue
i.valid = true

// Loop until finds a key to be passed to the next level.
for {
Expand Down Expand Up @@ -983,7 +964,6 @@ func (i *Iter) singleDeleteNext() bool {
default:
panic("unreachable")
}
i.valid = false
return false

case base.InternalKeyKindSingleDelete:
Expand All @@ -995,7 +975,6 @@ func (i *Iter) singleDeleteNext() bool {

default:
i.err = base.CorruptionErrorf("invalid internal key kind: %d", errors.Safe(i.iterKV.Kind()))
i.valid = false
return false
}
}
Expand Down Expand Up @@ -1109,7 +1088,6 @@ func (i *Iter) skipDueToSingleDeleteElision() {
// any, are elided as a result of the tombstone.
func (i *Iter) deleteSizedNext() (*base.InternalKey, []byte) {
i.saveKey()
i.valid = true
i.skip = true

// The DELSIZED tombstone may have no value at all. This happens when the
Expand Down Expand Up @@ -1213,7 +1191,6 @@ func (i *Iter) deleteSizedNext() (*base.InternalKey, []byte) {
expectedSize, n := binary.Uvarint(i.value)
if n != len(i.value) {
i.err = base.CorruptionErrorf("DELSIZED holds invalid value: %x", errors.Safe(i.value))
i.valid = false
return nil, nil
}
elidedSize := uint64(len(i.iterKV.K.UserKey)) + uint64(len(i.iterValue))
Expand Down Expand Up @@ -1252,7 +1229,6 @@ func (i *Iter) deleteSizedNext() (*base.InternalKey, []byte) {

default:
i.err = base.CorruptionErrorf("invalid internal key kind: %d", errors.Safe(i.iterKV.Kind()))
i.valid = false
return nil, nil
}
}
Expand All @@ -1278,22 +1254,9 @@ func (i *Iter) saveKey() {
i.frontiers.Advance(i.key.UserKey)
}

// Key returns the current key.
func (i *Iter) Key() base.InternalKey {
return i.key
}

// Value returns the current value.
func (i *Iter) Value() []byte {
return i.value
}

// Valid returns whether the iterator is positioned at a valid key/value pair.
func (i *Iter) Valid() bool {
return i.valid
}

// Error returns any error encountered.
//
// Note that Close will return the error as well.
func (i *Iter) Error() error {
return i.err
}
Expand Down
23 changes: 12 additions & 11 deletions internal/compact/iterator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -195,15 +195,17 @@ func TestCompactionIter(t *testing.T) {
if len(parts) == 0 {
continue
}
var key *base.InternalKey
var value []byte
switch parts[0] {
case "first":
iter.First()
key, value = iter.First()
case "next":
iter.Next()
key, value = iter.Next()
default:
d.Fatalf(t, "unknown iter command: %s", parts[0])
}
if iter.Valid() {
if key != nil {
snapshotPinned := ""
if printSnapshotPinned {
snapshotPinned = " (not pinned)"
Expand All @@ -218,18 +220,17 @@ func TestCompactionIter(t *testing.T) {
forceObsolete = " (force obsolete)"
}
}
v := string(iter.Value())
kind := iter.Key().Kind()
if kind == base.InternalKeyKindDeleteSized && len(iter.Value()) > 0 {
vn, n := binary.Uvarint(iter.Value())
if n != len(iter.Value()) {
v = fmt.Sprintf("err: %0x value not a uvarint", iter.Value())
v := string(value)
if key.Kind() == base.InternalKeyKindDeleteSized && len(value) > 0 {
vn, n := binary.Uvarint(value)
if n != len(value) {
v = fmt.Sprintf("err: %0x value not a uvarint", value)
} else {
v = fmt.Sprintf("varint(%d)", vn)
}
}
fmt.Fprintf(&b, "%s:%s%s%s", iter.Key(), v, snapshotPinned, forceObsolete)
if kind == base.InternalKeyKindRangeDelete || rangekey.IsRangeKey(kind) {
fmt.Fprintf(&b, "%s:%s%s%s", key, v, snapshotPinned, forceObsolete)
if key.Kind() == base.InternalKeyKindRangeDelete || rangekey.IsRangeKey(key.Kind()) {
fmt.Fprintf(&b, "; Span() = %s", iter.Span())
}
fmt.Fprintln(&b)
Expand Down

0 comments on commit f03e7ef

Please sign in to comment.