Skip to content

Commit

Permalink
storage: use must for MVCC iterator assertions
Browse files Browse the repository at this point in the history
Epic: none
Release note: None
  • Loading branch information
erikgrinaker committed Jul 28, 2023
1 parent 40a3971 commit 55d4b15
Show file tree
Hide file tree
Showing 2 changed files with 69 additions and 94 deletions.
1 change: 1 addition & 0 deletions pkg/storage/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -76,6 +76,7 @@ go_library(
"//pkg/util/log",
"//pkg/util/log/eventpb",
"//pkg/util/mon",
"//pkg/util/must",
"//pkg/util/protoutil",
"//pkg/util/syncutil",
"//pkg/util/sysutil",
Expand Down
162 changes: 68 additions & 94 deletions pkg/storage/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/iterutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/log/eventpb"
"github.com/cockroachdb/cockroach/pkg/util/must"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -1717,109 +1718,82 @@ func iterateOnReader(
// returning errors.AssertionFailedf for any violations. The iterator
// must be valid.
func assertSimpleMVCCIteratorInvariants(iter SimpleMVCCIterator) error {
key := iter.UnsafeKey()

// Keys can't be empty.
if len(key.Key) == 0 {
return errors.AssertionFailedf("valid iterator returned empty key")
}

// Can't be positioned in the lock table.
if bytes.HasPrefix(key.Key, keys.LocalRangeLockTablePrefix) {
return errors.AssertionFailedf("MVCC iterator positioned in lock table at %s", key)
}

// Any valid position must have either a point and/or range key.
hasPoint, hasRange := iter.HasPointAndRange()
if !hasPoint && !hasRange {
// NB: MVCCIncrementalIterator can return hasPoint=false,hasRange=false
// following a NextIgnoringTime() call. We explicitly allow this here.
if incrIter, ok := iter.(*MVCCIncrementalIterator); !ok || !incrIter.ignoringTime {
return errors.AssertionFailedf("valid iterator without point/range keys at %s", key)
}
}

// Range key assertions.
if hasRange {
// Must have bounds. The MVCCRangeKey.Validate() call below will make
// further bounds assertions.
bounds := iter.RangeBounds()
if len(bounds.Key) == 0 && len(bounds.EndKey) == 0 {
return errors.AssertionFailedf("hasRange=true but empty range bounds at %s", key)
}

// Iterator position must be within range key bounds.
if !bounds.ContainsKey(key.Key) {
return errors.AssertionFailedf("iterator position %s outside range bounds %s", key, bounds)
}

// Bounds must match range key stack.
rangeKeys := iter.RangeKeys()
if !rangeKeys.Bounds.Equal(bounds) {
return errors.AssertionFailedf("range bounds %s does not match range key %s",
bounds, rangeKeys.Bounds)
}

// Must have range keys.
if rangeKeys.IsEmpty() {
return errors.AssertionFailedf("hasRange=true but no range key versions at %s", key)
// nolint:errcheck
return must.Handle(context.TODO(), func(ctx context.Context) {
key := iter.UnsafeKey()

// Keys can't be empty.
must.Empty(ctx, key.Key, "empty key")

// Can't be positioned in the lock table.
must.NotPrefixBytes(ctx, key.Key, keys.LocalRangeLockTablePrefix, "MVCC iter in lock table")

// Any valid position must have either a point and/or range key.
hasPoint, hasRange := iter.HasPointAndRange()
if !hasPoint && !hasRange {
// NB: MVCCIncrementalIterator can return hasPoint=false,hasRange=false
// following a NextIgnoringTime() call. We explicitly allow this here.
if incrIter, ok := iter.(*MVCCIncrementalIterator); !ok || !incrIter.ignoringTime {
must.Fail(ctx, "iterator without point/range keys at %s", key)
}
}

for i, v := range rangeKeys.Versions {
// Range key must be valid.
rangeKey := rangeKeys.AsRangeKey(v)
if err := rangeKey.Validate(); err != nil {
return errors.NewAssertionErrorWithWrappedErrf(err, "invalid range key at %s", key)
}
// Range keys must be in descending timestamp order.
if i > 0 && !v.Timestamp.Less(rangeKeys.Versions[i-1].Timestamp) {
return errors.AssertionFailedf("range key %s not below version %s",
rangeKey, rangeKeys.Versions[i-1].Timestamp)
}
// Range keys must currently be tombstones.
if value, err := DecodeMVCCValue(v.Value); err != nil {
return errors.NewAssertionErrorWithWrappedErrf(err, "invalid range key value at %s",
rangeKey)
} else if !value.IsTombstone() {
return errors.AssertionFailedf("non-tombstone range key %s with value %x",
// Range key assertions.
if hasRange {
// Must have bounds. The MVCCRangeKey.Validate() call below will make
// further bounds assertions.
bounds := iter.RangeBounds()
must.True(ctx, len(bounds.Key) > 0 || len(bounds.EndKey) > 0,
"hasRange=true but empty range bounds at %s", key)

// Iterator position must be within range key bounds.
must.True(ctx, bounds.ContainsKey(key.Key),
"iterator position %s outside range key bounds %s", key, bounds)

// Bounds must match range key stack.
rangeKeys := iter.RangeKeys()
must.True(ctx, rangeKeys.Bounds.Equal(bounds),
"range key bounds %s do not match range key %s", bounds, rangeKeys.Bounds)

// Must have range keys.
must.NotEmpty(ctx, rangeKeys.Versions, "hasRange=true but no range key versions at %s", key)

for i, v := range rangeKeys.Versions {
// Range key must be valid.
rangeKey := rangeKeys.AsRangeKey(v)
must.NoError(ctx, rangeKey.Validate(), "invalid range key at %s", key)
// Range keys must be in descending timestamp order.
must.False(ctx, i > 0 && v.Timestamp.Less(rangeKeys.Versions[i-1].Timestamp),
"range key %s not below version %s", rangeKey, rangeKeys.Versions[i-1].Timestamp)
// Range keys must currently be tombstones.
value, err := DecodeMVCCValue(v.Value)
must.NoError(ctx, err, "invalid range key value at %s", rangeKey)
must.True(ctx, value.IsTombstone(), "non-tombstone range key %s with value %x",
rangeKey, value.Value.RawBytes)
}
}

}
if hasPoint {
value, err := iter.UnsafeValue()
if err != nil {
return err
}
valueLen := iter.ValueLen()
if len(value) != valueLen {
return errors.AssertionFailedf("length of UnsafeValue %d != ValueLen %d", len(value), valueLen)
}
if key.IsValue() {
valueLen2, isTombstone, err := iter.MVCCValueLenAndIsTombstone()
if err == nil {
if len(value) != valueLen2 {
return errors.AssertionFailedf("length of UnsafeValue %d != MVCCValueLenAndIsTombstone %d",
len(value), valueLen2)
}
if v, err := DecodeMVCCValue(value); err == nil {
if isTombstone != v.IsTombstone() {
return errors.AssertionFailedf("isTombstone from MVCCValueLenAndIsTombstone %t != MVCCValue.IsTombstone %t",
isTombstone, v.IsTombstone())
if hasPoint {
value, err := iter.UnsafeValue()
must.NoError(ctx, err, "failed to get value for %s", key)
must.Len(ctx, value, iter.ValueLen(), "UnsafeValue != ValueLen at %s", key)
if key.IsValue() {
valueLen, isTombstone, err := iter.MVCCValueLenAndIsTombstone()
if err == nil {
must.Len(ctx, value, valueLen, "UnsafeValue != MVCCValueLenAndIsTombstone at %s", key)
if v, err := DecodeMVCCValue(value); err == nil {
must.Equal(ctx, v.IsTombstone(), isTombstone,
"IsTombstone != MVCCValueLenAndIsTombstone at %s", key)
// Else err != nil. SimpleMVCCIterator is not responsile for data
// corruption since it is possible that the implementation of
// MVCCValueLenAndIsTombstone is fetching information from a
// different part of the store than where the value is stored.
}
// Else err != nil. SimpleMVCCIterator is not responsile for data
// corruption since it is possible that the implementation of
// MVCCValueLenAndIsTombstone is fetching information from a
// different part of the store than where the value is stored.
}
// Else err != nil. Ignore, since SimpleMVCCIterator is not to be held
// responsible for data corruption or tests writing non-MVCCValues.
}
// Else err != nil. Ignore, since SimpleMVCCIterator is not to be held
// responsible for data corruption or tests writing non-MVCCValues.
}
}

return nil
})
}

// assertMVCCIteratorInvariants asserts invariants in the MVCCIterator interface
Expand Down

0 comments on commit 55d4b15

Please sign in to comment.