diff --git a/pkg/storage/BUILD.bazel b/pkg/storage/BUILD.bazel index d407e8e784e0..6c895b332650 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -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", diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index be393d0e1355..586e1664f611 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -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" @@ -1713,185 +1714,142 @@ func iterateOnReader( } // assertSimpleMVCCIteratorInvariants asserts invariants in the -// SimpleMVCCIterator interface that should hold for all implementations, -// returning errors.AssertionFailedf for any violations. The iterator -// must be valid. +// SimpleMVCCIterator interface that should hold for all implementations. 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) + // nolint:errcheck + return must.Handle(context.Background(), func(ctx context.Context) { + key := iter.UnsafeKey() + + // Keys can't be empty. + must.NotEmpty(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) + } } - // Must have range keys. - if rangeKeys.IsEmpty() { - return errors.AssertionFailedf("hasRange=true but no range key versions at %s", key) - } + // 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 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 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. + if i > 0 { + must.True(ctx, v.Timestamp.Less(rangeKeys.Versions[i-1].Timestamp), + "range key %s not below version %s", rangeKey, rangeKeys.Versions[i-1].Timestamp) + } - 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 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 -// that should hold for all implementations, returning errors.AssertionFailedf -// for any violations. It calls through to assertSimpleMVCCIteratorInvariants(). -// The iterator must be valid. +// that should hold for all implementations. It calls through to +// assertSimpleMVCCIteratorInvariants(). The iterator must be valid. func assertMVCCIteratorInvariants(iter MVCCIterator) error { // Assert SimpleMVCCIterator invariants. if err := assertSimpleMVCCIteratorInvariants(iter); err != nil { return err } - key := iter.UnsafeKey().Clone() - - // UnsafeRawMVCCKey must match Key. - if r, err := DecodeMVCCKey(iter.UnsafeRawMVCCKey()); err != nil { - return errors.NewAssertionErrorWithWrappedErrf( - err, "failed to decode UnsafeRawMVCCKey at %s", - key, - ) - } else if !r.Equal(key) { - return errors.AssertionFailedf("UnsafeRawMVCCKey %s does not match Key %s", r, key) - } - - // UnsafeRawKey must either be an MVCC key matching Key, or a lock table key - // that refers to it. - if engineKey, ok := DecodeEngineKey(iter.UnsafeRawKey()); !ok { - return errors.AssertionFailedf("failed to decode UnsafeRawKey as engine key at %s", key) - } else if engineKey.IsMVCCKey() { - if k, err := engineKey.ToMVCCKey(); err != nil { - return errors.NewAssertionErrorWithWrappedErrf(err, "invalid UnsafeRawKey at %s", key) - } else if !k.Equal(key) { - return errors.AssertionFailedf("UnsafeRawKey %s does not match Key %s", k, key) - } - } else if engineKey.IsLockTableKey() { - if k, err := engineKey.ToLockTableKey(); err != nil { - return errors.NewAssertionErrorWithWrappedErrf(err, "invalid UnsafeRawKey at %s", key) - } else if !k.Key.Equal(key.Key) { - return errors.AssertionFailedf("UnsafeRawKey lock table key %s does not match Key %s", k, key) - } else if !key.Timestamp.IsEmpty() { - return errors.AssertionFailedf( - "UnsafeRawKey lock table key %s for Key %s with non-zero timestamp", k, key, - ) + // nolint:errcheck + return must.Handle(context.Background(), func(ctx context.Context) { + key := iter.UnsafeKey().Clone() + + // UnsafeRawMVCCKey must match Key. + raw, err := DecodeMVCCKey(iter.UnsafeRawMVCCKey()) + must.NoError(ctx, err, "failed to decode raw MVCC key at %s", key) + must.True(ctx, raw.Equal(key), "UnsafeRawMVCCKey %s != Key %s", raw, key) + + // UnsafeRawKey must either be an MVCC key matching Key, or a lock table key + // that refers to it. + engineKey, ok := DecodeEngineKey(iter.UnsafeRawKey()) + must.True(ctx, ok, "failed to decode engine key at %s", key) + if engineKey.IsMVCCKey() { + mvccKey, err := engineKey.ToMVCCKey() + must.NoError(ctx, err, "MVCC key conversion failed at %s", key) + must.True(ctx, mvccKey.Equal(key), "UnsafeRawKey %s != Key %s", mvccKey, key) + } else if engineKey.IsLockTableKey() { + ltKey, err := engineKey.ToLockTableKey() + must.NoError(ctx, err, "failed to decode lock table key at %s", key) + must.EqualBytes(ctx, ltKey.Key, key.Key, "UnsafeRawKey != Key", ltKey.Key, key.Key) + must.Zero(ctx, key.Timestamp, "lock table key with timestamp at %s", key) + } else { + must.Fail(ctx, "unknown engine key type for %s", engineKey) } - } else { - return errors.AssertionFailedf("unknown type for engine key %s", engineKey) - } - // Value must equal UnsafeValue. - u, err := iter.UnsafeValue() - if err != nil { - return err - } - v, err := iter.Value() - if err != nil { - return err - } - if !bytes.Equal(v, u) { - return errors.AssertionFailedf("Value %x does not match UnsafeValue %x at %s", v, u, key) - } - - // For prefix iterators, any range keys must be point-sized. We've already - // asserted that the range key covers the iterator position. - if iter.IsPrefix() { - if _, hasRange := iter.HasPointAndRange(); hasRange { - if bounds := iter.RangeBounds(); !bounds.EndKey.Equal(bounds.Key.Next()) { - return errors.AssertionFailedf("prefix iterator with wide range key %s", bounds) + // Value must equal UnsafeValue. + u, err := iter.UnsafeValue() + must.NoError(ctx, err, "failed to fetch value") + v, err := iter.Value() + must.NoError(ctx, err, "failed to fetch value") + must.EqualBytes(ctx, v, u, "Value != UnsafeValue at %s", key) + + // For prefix iterators, any range keys must be point-sized. We've already + // asserted that the range key covers the iterator position. + if iter.IsPrefix() { + if _, hasRange := iter.HasPointAndRange(); hasRange { + b := iter.RangeBounds() + must.EqualBytes(ctx, b.EndKey, b.Key.Next(), "prefix iterator with wide range key") } } - } - - return nil + }) } // ScanConflictingIntentsForDroppingLatchesEarly scans intents using only the diff --git a/pkg/storage/intent_interleaving_iter.go b/pkg/storage/intent_interleaving_iter.go index fe0d45cb51a7..8b05f460d852 100644 --- a/pkg/storage/intent_interleaving_iter.go +++ b/pkg/storage/intent_interleaving_iter.go @@ -12,6 +12,7 @@ package storage import ( "bytes" + "context" "fmt" "math/rand" "sync" @@ -21,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" + "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" @@ -696,8 +698,8 @@ func (i *intentInterleavingIter) tryDecodeLockKey( } func (i *intentInterleavingIter) Valid() (bool, error) { - if util.RaceEnabled && i.valid { - if err := i.assertInvariants(); err != nil { + if i.valid { + if err := must.Expensive(i.assertInvariants); err != nil { return false, err } } @@ -1326,133 +1328,104 @@ func (i *intentInterleavingIter) IsPrefix() bool { return i.prefix } -// assertInvariants asserts internal iterator invariants, returning an -// AssertionFailedf for any violations. It must be called on a valid iterator -// after a complete state transition. +// assertInvariants asserts internal iterator invariants. It must be called on +// a valid iterator after a complete state transition. func (i *intentInterleavingIter) assertInvariants() error { // Assert general MVCCIterator invariants. if err := assertMVCCIteratorInvariants(i); err != nil { return err } - // The underlying iterator must not have errored. - iterValid, err := i.iter.Valid() - if err != nil { - return errors.NewAssertionErrorWithWrappedErrf(err, "valid iter but i.iter errored") - } - intentValid := i.intentKey != nil + // nolint:errcheck + return must.Handle(context.Background(), func(ctx context.Context) { + // The underlying iterator must not have errored. + iterValid, err := i.iter.Valid() + must.NoError(ctx, err, "valid iter but i.iter errored") + intentValid := i.intentKey != nil - // At least one of the iterators must be valid. The iterator's validity state - // should match i.iterValid. - if !iterValid && !intentValid { - return errors.AssertionFailedf("i.valid=%t but both iterators are invalid", i.valid) - } - if iterValid != i.iterValid { - return errors.AssertionFailedf("i.iterValid=%t but i.iter.Valid=%t", i.iterValid, iterValid) - } + // At least one of the iterators must be valid. The iterator's validity state + // should match i.iterValid. + must.True(ctx, iterValid || intentValid, "i.valid=%t but both iterators are invalid", i.valid) + must.Equal(ctx, iterValid, i.iterValid, "i.iter.Valid() != i.iterValid") - // i.dir must be either 1 or -1. - if i.dir != 1 && i.dir != -1 { - return errors.AssertionFailedf("i.dir=%v is not valid", i.dir) - } + // i.dir must be either 1 or -1. + must.Contains(ctx, []int{1, -1}, i.dir, "invalid i.dir") - // For valid iterators, the stored key must match the iterator key. - if iterValid { - if key := i.iter.UnsafeKey(); !i.iterKey.Equal(key) { - return errors.AssertionFailedf("i.iterKey=%q does not match i.iter.UnsafeKey=%q", - i.iterKey, key) - } - } - if intentValid { - intentKey := i.intentKey.Clone() - if engineKey, err := i.intentIter.UnsafeEngineKey(); err != nil { - return errors.NewAssertionErrorWithWrappedErrf(err, "valid i.intentIter errored") - } else if !engineKey.IsLockTableKey() { - return errors.AssertionFailedf("i.intentIter on non-locktable key %s", engineKey) - } else if key, err := keys.DecodeLockTableSingleKey(engineKey.Key); err != nil { - return errors.NewAssertionErrorWithWrappedErrf(err, "failed to decode lock table key %s", - engineKey) - } else if !intentKey.Equal(key) { - return errors.AssertionFailedf("i.intentKey %q != i.intentIter.UnsafeEngineKey() %q", - intentKey, key) - } - // If i.intentKey is set (i.e. intentValid is true), then intentIterState - // must be valid. The inverse is not always true. - if i.intentIterState != pebble.IterValid { - return errors.AssertionFailedf("i.intentKey=%q, but i.intentIterState=%v is not IterValid", - i.intentKey, i.intentIterState) - } - // If i.intentKey is set, then i.intentKeyAsNoTimestampMVCCKey must either - // be nil or equal to it with a \x00 byte appended. - if i.intentKeyAsNoTimestampMVCCKey != nil && - !bytes.Equal(i.intentKeyAsNoTimestampMVCCKey, append(i.intentKey.Clone(), 0)) { - return errors.AssertionFailedf( - "i.intentKeyAsNoTimestampMVCCKey=%q differs from i.intentKey=%q", - i.intentKeyAsNoTimestampMVCCKey, i.intentKey) + // For valid iterators, the stored key must match the iterator key. + if iterValid { + key := i.iter.UnsafeKey() + must.EqualBytes(ctx, key.Key, i.iterKey.Key, "i.iter.UnsafeKey != i.iterKey") + must.Equal(ctx, key.Timestamp, i.iterKey.Timestamp, "i.iter.UnsafeKey != i.iterKey") } - } - // Check intentCmp depending on the iterator validity. We already know that - // one of the iterators must be valid. - if iterValid && intentValid { - if cmp := i.intentKey.Compare(i.iterKey.Key); i.intentCmp != cmp { - return errors.AssertionFailedf("i.intentCmp=%v does not match %v for intentKey=%q iterKey=%q", - i.intentCmp, cmp, i.intentKey, i.iterKey) - } - } else if iterValid { - if i.intentCmp != i.dir { - return errors.AssertionFailedf("i.intentCmp=%v != i.dir=%v for invalid i.intentIter", - i.intentCmp, i.dir) + if intentValid { + intentKey := i.intentKey.Clone() + engineKey, err := i.intentIter.UnsafeEngineKey() + must.NoError(ctx, err, "i.intentIter UnsafeEngineKey failed") + must.True(ctx, engineKey.IsLockTableKey(), "i.intentIter outside locktable at %s", engineKey) + ltKey, err := keys.DecodeLockTableSingleKey(engineKey.Key) + must.NoError(ctx, err, "failed to decode lock table key %s", engineKey) + must.EqualBytes(ctx, ltKey, intentKey, "intentIter.UnsafeEngineKey != i.intentKey") + + // If i.intentKey is set (i.e. intentValid is true), then intentIterState + // must be valid. The inverse is not always true. + must.Equal(ctx, i.intentIterState, pebble.IterValid, "invalid iter state at %s", intentKey) + + // If i.intentKey is set, then i.intentKeyAsNoTimestampMVCCKey must either + // be nil or equal to it with a \x00 byte appended. + if i.intentKeyAsNoTimestampMVCCKey != nil { + must.EqualBytes(ctx, i.intentKeyAsNoTimestampMVCCKey, append(intentKey, 0), + "i.intentKeyAsNoTimestampMVCCKey != i.intentKey") + } } - } else if intentValid { - if i.intentCmp != -i.dir { - return errors.AssertionFailedf("i.intentCmp=%v == i.dir=%v for invalid i.iter", - i.intentCmp, i.dir) + + // Check intentCmp depending on the iterator validity. We already know that + // one of the iterators must be valid. + if iterValid && intentValid { + must.Equal(ctx, i.intentKey.Compare(i.iterKey.Key), i.intentCmp, + "i.intentKey.Compare != i.intentCmp at %s", i.intentKey) + } else if iterValid { + must.Equal(ctx, i.intentCmp, i.dir, + "i.intentCmp != i.dir for invalid i.intentIter at %s", i.iterKey) + } else if intentValid { + must.Equal(ctx, i.intentCmp, -i.dir, + "i.intentCmp != -i.dir for invalid i.iter at %s", i.intentKey) } - } - // When on an intent in the forward direction, we must be on a provisional - // value and any range key must cover it. - if i.dir > 0 && i.isCurAtIntentIterForward() { - if !iterValid { - return errors.AssertionFailedf( - "missing provisional value for i.intentKey=%q: i.iter exhausted", i.intentKey) - } else if i.intentCmp != 0 { - return errors.AssertionFailedf( - "missing provisional value for i.intentKey=%q: i.intentCmp=%v is not 0", - i.intentKey, i.intentCmp) - } else if hasPoint, hasRange := i.iter.HasPointAndRange(); !hasPoint { - return errors.AssertionFailedf( - "missing provisional value for i.intentKey=%q: i.iter on bare range key", - i.intentKey) - } else if hasRange { - if bounds := i.iter.RangeBounds(); !bounds.ContainsKey(i.intentKey) { - return errors.AssertionFailedf("i.intentKey=%q not covered by i.iter range key %q", - bounds, i.intentKey) + // When on an intent in the forward direction, we must be on a provisional + // value and any range key must cover it. + if i.dir > 0 && i.isCurAtIntentIterForward() { + must.True(ctx, iterValid, + "missing provisional value for i.intentKey %s; i.iter exhausted", i.intentKey) + must.Zero(ctx, i.intentCmp, + "missing provisional value for i.intentKey %s; i.intentCmp != 0", i.intentKey) + + hasPoint, hasRange := i.iter.HasPointAndRange() + must.True(ctx, hasPoint, + "missing provisional value for i.intentKey %s: i.iter on bare range key", i.intentKey) + if hasRange { + bounds := i.iter.RangeBounds() + must.True(ctx, bounds.ContainsKey(i.intentKey), + "i.intentKey %s not covered by i.iter range key %s", i.intentKey, bounds) } } - } - // Check i.iterBareRangeAtIntent, which is only valid for i.intentCmp == 0. - if i.intentCmp == 0 { - if i.dir > 0 && i.iterBareRangeAtIntent { - return errors.AssertionFailedf("i.dir=%v can't have i.iterBareRangeAtIntent=%v", - i.dir, i.iterBareRangeAtIntent) - } - if i.dir < 0 && i.iterBareRangeAtIntent { - if hasPoint, hasRange := i.iter.HasPointAndRange(); hasPoint || !hasRange { - return errors.AssertionFailedf("i.iterBareRangeAtIntent=%v but hasPoint=%t hasRange=%t", + // Check i.iterBareRangeAtIntent, which is only valid for i.intentCmp == 0. + if i.intentCmp == 0 { + must.False(ctx, i.dir > 0 && i.iterBareRangeAtIntent, + "i.dir=%v can't have i.iterBareRangeAtIntent=%v", i.dir, i.iterBareRangeAtIntent) + + if i.dir < 0 && i.iterBareRangeAtIntent { + hasPoint, hasRange := i.iter.HasPointAndRange() + must.True(ctx, hasRange && !hasPoint, + "i.iterBareRangeAtIntent=%v but hasPoint=%t hasRange=%t", i.iterBareRangeAtIntent, hasPoint, hasRange) - } - // We've already asserted key equality for i.intentCmp == 0. - if !i.iterKey.Timestamp.IsEmpty() { - return errors.AssertionFailedf("i.iterBareRangeAtIntent=%v but i.iterKey has timestamp %s", - i.iterBareRangeAtIntent, i.iterKey.Timestamp) + // We've already asserted key equality for i.intentCmp == 0. + must.Zero(ctx, i.iterKey.Timestamp, + "i.iterBareRangeAtIntent=%v but i.iterKey has timestamp", i.iterBareRangeAtIntent) } } - } - - return nil + }) } // unsafeMVCCIterator is used in RaceEnabled test builds to randomly inject diff --git a/pkg/storage/intent_interleaving_iter_test.go b/pkg/storage/intent_interleaving_iter_test.go index 6bbcbde24be2..b63883471f74 100644 --- a/pkg/storage/intent_interleaving_iter_test.go +++ b/pkg/storage/intent_interleaving_iter_test.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/must" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/uint128" "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -211,6 +212,10 @@ func checkAndOutputIter(iter MVCCIterator, b *strings.Builder) { func TestIntentInterleavingIter(t *testing.T) { defer leaktest.AfterTest(t)() + // Disable fatal assertions, since we want to capture assertion failures in + // error output. + must.DisableFatalAssertions(t) + var eng Engine defer func() { if eng != nil { diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index fff579b168ad..c99d3561219a 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -40,6 +40,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/must" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/uint128" "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -165,6 +166,10 @@ func TestMVCCHistories(t *testing.T) { // duplicate each test file for the two cases. storage.DisableMetamorphicSimpleValueEncoding(t) + // Disable fatal assertions, since we want to capture assertion failures as + // error output. + must.DisableFatalAssertions(t) + ctx := context.Background() // intentInterleavingIter doesn't allow iterating from the local to the global diff --git a/pkg/storage/mvcc_incremental_iterator.go b/pkg/storage/mvcc_incremental_iterator.go index 42e1dc0c62a1..d9d56e461b49 100644 --- a/pkg/storage/mvcc_incremental_iterator.go +++ b/pkg/storage/mvcc_incremental_iterator.go @@ -11,6 +11,7 @@ package storage import ( + "context" "fmt" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" @@ -18,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/must" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/errors" ) @@ -590,8 +592,8 @@ func (i *MVCCIncrementalIterator) advance(seeked bool) { // Valid implements SimpleMVCCIterator. func (i *MVCCIncrementalIterator) Valid() (bool, error) { - if util.RaceEnabled && i.valid { - if err := i.assertInvariants(); err != nil { + if i.valid { + if err := must.Expensive(i.assertInvariants); err != nil { return false, err } } @@ -780,139 +782,103 @@ func (i *MVCCIncrementalIterator) assertInvariants() error { return err } - // The underlying iterator must be valid when the MVCCIncrementalIterator is. - if ok, err := i.iter.Valid(); err != nil || !ok { - errMsg := err.Error() - return errors.AssertionFailedf("i.iter is invalid with err=%s", errMsg) - } + // nolint:errcheck + return must.Handle(context.Background(), func(ctx context.Context) { + // The underlying iterator must be valid when the MVCCIncrementalIterator is. + ok, err := i.iter.Valid() + must.NoError(ctx, err, "i.iter errored") + must.True(ctx, ok, "i.iter is invalid") - iterKey := i.iter.UnsafeKey() + iterKey := i.iter.UnsafeKey().Clone() - // endTime must be set, and be at or after startTime. - if i.endTime.IsEmpty() { - return errors.AssertionFailedf("i.endTime not set") - } - if i.endTime.Less(i.startTime) { - return errors.AssertionFailedf("i.endTime %s before i.startTime %s", i.endTime, i.startTime) - } + // endTime must be set, and be at or after startTime. + must.NotZero(ctx, i.endTime, "i.endTime not set") + must.True(ctx, i.startTime.LessEq(i.endTime), "i.endTime before i.startTime") - // If startTime is empty, the TBI should be disabled in non-metamorphic builds. - if !util.IsMetamorphicBuild() && i.startTime.IsEmpty() && i.timeBoundIter != nil { - return errors.AssertionFailedf("TBI enabled without i.startTime") - } + // If startTime is empty, the TBI should be disabled in non-metamorphic builds. + must.False(ctx, !util.IsMetamorphicBuild() && i.startTime.IsEmpty() && i.timeBoundIter != nil, + "TBI enabled without i.startTime") - // If the TBI is enabled, its position should be <= iter unless iter is on an intent. - if i.timeBoundIter != nil && iterKey.Timestamp.IsSet() { - if ok, _ := i.timeBoundIter.Valid(); ok { - if tbiKey := i.timeBoundIter.UnsafeKey(); tbiKey.Compare(iterKey) > 0 { - return errors.AssertionFailedf("TBI at %q ahead of i.iter at %q", tbiKey, iterKey) + // If the TBI is enabled, its position should be <= iter unless iter is on an intent. + if i.timeBoundIter != nil && iterKey.Timestamp.IsSet() { + if ok, _ := i.timeBoundIter.Valid(); ok { + tbiKey := i.timeBoundIter.UnsafeKey() + must.True(ctx, tbiKey.Compare(iterKey) <= 0, "TBI %q ahead of i.iter %q", tbiKey, iterKey) } } - } - // i.meta should match the underlying iterator's key. - if hasPoint, _ := i.iter.HasPointAndRange(); hasPoint { + // i.meta should match the underlying iterator's key. metaTS := i.meta.Timestamp.ToTimestamp() - if iterKey.Timestamp.IsSet() && !metaTS.EqOrdering(iterKey.Timestamp) { - return errors.AssertionFailedf("i.meta.Timestamp %s differs from i.iter.UnsafeKey %s", - metaTS, iterKey) - } - if metaTS.IsEmpty() && i.meta.Txn == nil { - return errors.AssertionFailedf("empty i.meta for point key %s", iterKey) - } - } else { - if i.meta.Timestamp.ToTimestamp().IsSet() || i.meta.Txn != nil { - return errors.AssertionFailedf("i.iter hasPoint=false but non-empty i.meta %+v", i.meta) + if hasPoint, _ := i.iter.HasPointAndRange(); hasPoint { + must.True(ctx, iterKey.Timestamp.IsEmpty() || metaTS.EqOrdering(iterKey.Timestamp), + "i.meta.Timestamp %s differs from i.iter.UnsafeKey %s", metaTS, iterKey) + must.True(ctx, metaTS.IsSet() || i.meta.Txn != nil, "empty i.meta for point key %s", iterKey) + } else { + must.True(ctx, metaTS.IsEmpty() && i.meta.Txn == nil, + "i.iter hasPoint=false but non-empty i.meta %+v", i.meta) } - } - // Unlike most SimpleMVCCIterators, it's possible to return - // hasPoint=false,hasRange=false following a NextIgnoringTime() call. - hasPoint, hasRange := i.HasPointAndRange() - if !hasPoint && !hasRange { - if !i.ignoringTime { - return errors.AssertionFailedf( - "hasPoint=false,hasRange=false invalid when i.ignoringTime=false") - } - if i.RangeKeysIgnoringTime().IsEmpty() { - return errors.AssertionFailedf( + // Unlike most SimpleMVCCIterators, it's possible to return + // hasPoint=false,hasRange=false following a NextIgnoringTime() call. + hasPoint, hasRange := i.HasPointAndRange() + if !hasPoint && !hasRange { + must.True(ctx, i.ignoringTime, "hasPoint=false,hasRange=false when i.ignoringTime=false") + must.False(ctx, i.RangeKeysIgnoringTime().IsEmpty(), "hasPoint=false,hasRange=false and RangeKeysIgnoringTime() returned nothing") } - } - // Point keys and range keys must be within the time bounds, unless - // we're ignoring time bounds. - assertInRange := func(ts hlc.Timestamp, format string, args ...interface{}) error { - if i.startTime.IsSet() && ts.LessEq(i.startTime) || i.endTime.Less(ts) { - return errors.AssertionFailedf("%s not in range (%s-%s]", - fmt.Sprintf(format, args...), i.startTime, i.endTime) + // Point keys and range keys must be within the time bounds, unless + // we're ignoring time bounds. + mustInRange := func(ts hlc.Timestamp, arg fmt.Stringer) { + must.True(ctx, i.startTime.Less(ts) && ts.LessEq(i.endTime), + "%s not in range (%s-%s]", arg, i.startTime, i.endTime) } - return nil - } - key := i.UnsafeKey() - - if hasPoint && !i.ignoringTime { - if key.Timestamp.IsEmpty() { - intent := key.Clone() - intent.Timestamp = i.meta.Timestamp.ToTimestamp() - if err := assertInRange(intent.Timestamp, "intent %s", intent); err != nil { - return err - } - } else { - if err := assertInRange(key.Timestamp, "point key %s", key); err != nil { - return err + key := i.UnsafeKey() + + if hasPoint && !i.ignoringTime { + if key.Timestamp.IsEmpty() { + intent := key.Clone() + intent.Timestamp = i.meta.Timestamp.ToTimestamp() + mustInRange(intent.Timestamp, intent) + } else { + mustInRange(key.Timestamp, key) } } - } - if hasRange { - rangeKeys := i.RangeKeys() - for _, v := range rangeKeys.Versions { - if err := assertInRange(v.Timestamp, "range key %s", rangeKeys.AsRangeKey(v)); err != nil { - return err + if hasRange { + rangeKeys := i.RangeKeys() + for _, v := range rangeKeys.Versions { + mustInRange(v.Timestamp, rangeKeys.AsRangeKey(v)) } } - } - // Check that intents are processed according to intentPolicy. - if hasPoint && key.Timestamp.IsEmpty() && i.intentPolicy != MVCCIncrementalIterIntentPolicyEmit { - return errors.AssertionFailedf("emitted intent %s not allowed by i.intentPolicy %v", - key, i.intentPolicy) - } - if len(i.intents) > 0 && i.intentPolicy != MVCCIncrementalIterIntentPolicyAggregate { - return errors.AssertionFailedf("i.intents set but not allowed by i.intentPolicy %v", - i.intentPolicy) - } - for _, intent := range i.intents { - intentKey := MVCCKey{Key: intent.Key, Timestamp: intent.Txn.WriteTimestamp} - if err := assertInRange(intentKey.Timestamp, "gathered intent %s", intentKey); err != nil { - return err + // Check that intents are processed according to intentPolicy. + must.False(ctx, hasPoint && key.Timestamp.IsEmpty() && i.intentPolicy != MVCCIncrementalIterIntentPolicyEmit, + "emitted intent %s not allowed by i.intentPolicy %v", key, i.intentPolicy) + must.False(ctx, len(i.intents) > 0 && i.intentPolicy != MVCCIncrementalIterIntentPolicyAggregate, + "i.intents set but not allowed by i.intentPolicy %v", i.intentPolicy) + for _, intent := range i.intents { + intentKey := MVCCKey{Key: intent.Key, Timestamp: intent.Txn.WriteTimestamp} + mustInRange(intentKey.Timestamp, intentKey) } - } - // RangeKeys() must be a subset of RangeKeysIgnoringTime(). - if hasRange { - rangeKeys := i.RangeKeys() - rangeKeysIgnoringTime := i.RangeKeysIgnoringTime() - if !rangeKeys.Bounds.Equal(rangeKeysIgnoringTime.Bounds) { - return errors.AssertionFailedf("RangeKeys=%s does not match RangeKeysIgnoringTime=%s", + if hasRange { + // RangeKeys() must be a subset of RangeKeysIgnoringTime(). + rangeKeys := i.RangeKeys() + rangeKeysIgnoringTime := i.RangeKeysIgnoringTime() + must.True(ctx, rangeKeys.Bounds.Equal(rangeKeysIgnoringTime.Bounds), + "RangeKeys=%s does not match RangeKeysIgnoringTime=%s", rangeKeys.Bounds, rangeKeysIgnoringTime.Bounds) - } - trimmedVersions := rangeKeysIgnoringTime.Versions - trimmedVersions.Trim(rangeKeys.Oldest(), rangeKeys.Newest()) - if !rangeKeys.Versions.Equal(trimmedVersions) { - return errors.AssertionFailedf("RangeKeys=%s not subset of RangeKeysIgnoringTime=%s", - rangeKeys, rangeKeysIgnoringTime) - } - } else { - // RangeKeysIgnoringTime must cover the current iterator position. - if rangeKeys := i.RangeKeysIgnoringTime(); !rangeKeys.IsEmpty() { - if !rangeKeys.Bounds.ContainsKey(key.Key) { - return errors.AssertionFailedf("RangeKeysIgnoringTime %s does not cover position %s", - rangeKeys.Bounds, key) - } - } - } + trimmedVersions := rangeKeysIgnoringTime.Versions + trimmedVersions.Trim(rangeKeys.Oldest(), rangeKeys.Newest()) + must.True(ctx, rangeKeys.Versions.Equal(trimmedVersions), + "RangeKeys=%s not subset of RangeKeysIgnoringTime=%s", rangeKeys, rangeKeysIgnoringTime) - return nil + } else { + // RangeKeysIgnoringTime must cover the current iterator position. + rangeKeys := i.RangeKeysIgnoringTime() + must.True(ctx, rangeKeys.IsEmpty() || rangeKeys.Bounds.ContainsKey(key.Key), + "RangeKeysIgnoringTime %s does not cover position %s", rangeKeys.Bounds, key) + } + }) } diff --git a/pkg/storage/pebble_iterator.go b/pkg/storage/pebble_iterator.go index b77b72a663d1..7bab86341cee 100644 --- a/pkg/storage/pebble_iterator.go +++ b/pkg/storage/pebble_iterator.go @@ -12,13 +12,14 @@ package storage import ( "bytes" + "context" "math" "sync" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/pebbleiter" - "github.com/cockroachdb/cockroach/pkg/util" + "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" @@ -419,10 +420,8 @@ func (p *pebbleIterator) Valid() (bool, error) { return false, nil } - if util.RaceEnabled { - if err := p.assertMVCCInvariants(); err != nil { - return false, err - } + if err := must.Expensive(p.assertMVCCInvariants); err != nil { + return false, err } return true, nil } @@ -972,38 +971,27 @@ func (p *pebbleIterator) destroy() { } } -// assertMVCCInvariants asserts internal MVCC iterator invariants, returning an -// AssertionFailedf on any failures. It must be called on a valid iterator after -// a complete state transition. +// assertMVCCInvariants asserts internal MVCC iterator invariants. It must be +// called on a valid iterator after a complete state transition. func (p *pebbleIterator) assertMVCCInvariants() error { // Assert general MVCCIterator API invariants. if err := assertMVCCIteratorInvariants(p); err != nil { return err } - // The underlying iterator must be valid, with !mvccDone. - if !p.iter.Valid() { - errMsg := p.iter.Error().Error() - return errors.AssertionFailedf("underlying iter is invalid, with err=%s", errMsg) - } - if p.mvccDone { - return errors.AssertionFailedf("valid iter with mvccDone set") - } + // nolint:errcheck + return must.Handle(context.Background(), func(ctx context.Context) { + // The underlying iterator must be valid, with !mvccDone. + must.True(ctx, p.iter.Valid(), "underlying iter is invalid, err=%s", p.iter.Error()) + must.False(ctx, p.mvccDone, "valid iter with mvccDone=true") - // The position must match the underlying iter. - if key, iterKey := p.UnsafeKey(), p.iter.Key(); !bytes.Equal(EncodeMVCCKey(key), iterKey) { - return errors.AssertionFailedf("UnsafeKey %s does not match iterator key %x", key, iterKey) - } + // The position must match the underlying iter. + must.EqualBytes(ctx, EncodeMVCCKey(p.UnsafeKey()), p.iter.Key(), "UnsafeKey != iter key") - // The iterator must be marked as in use. - if !p.inuse { - return errors.AssertionFailedf("valid iter with inuse=false") - } + // The iterator must be marked as in use. + must.True(ctx, p.inuse, "valid iter with inuse=false") - // Prefix must be exposed. - if p.prefix != p.IsPrefix() { - return errors.AssertionFailedf("IsPrefix() does not match prefix=%v", p.prefix) - } - - return nil + // Prefix must be exposed. + must.Equal(ctx, p.IsPrefix(), p.prefix, "IsPrefix != prefix") + }) } diff --git a/pkg/storage/read_as_of_iterator.go b/pkg/storage/read_as_of_iterator.go index f37a2f2e878f..33e74561af48 100644 --- a/pkg/storage/read_as_of_iterator.go +++ b/pkg/storage/read_as_of_iterator.go @@ -11,10 +11,11 @@ package storage import ( + "context" + "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/errors" + "github.com/cockroachdb/cockroach/pkg/util/must" ) // ReadAsOfIterator wraps a SimpleMVCCIterator and only surfaces the latest @@ -71,8 +72,8 @@ func (f *ReadAsOfIterator) SeekGE(originalKey MVCCKey) { // Valid implements the simpleMVCCIterator. func (f *ReadAsOfIterator) Valid() (bool, error) { - if util.RaceEnabled && f.valid { - if err := f.assertInvariants(); err != nil { + if f.valid { + if err := must.Expensive(f.assertInvariants); err != nil { return false, err } } @@ -217,33 +218,25 @@ func (f *ReadAsOfIterator) assertInvariants() error { return err } - // asOf must be set. - if f.asOf.IsEmpty() { - return errors.AssertionFailedf("f.asOf is empty") - } - - // The underlying iterator must be valid. - if ok, err := f.iter.Valid(); !ok || err != nil { - errMsg := err.Error() - return errors.AssertionFailedf("invalid underlying iter with err=%s", errMsg) - } - - // Keys can't be intents or inline values, and must have timestamps at or - // below the readAsOf timestamp. - key := f.UnsafeKey() - if key.Timestamp.IsEmpty() { - return errors.AssertionFailedf("emitted key %s has no timestamp", key) - } - if f.asOf.Less(key.Timestamp) { - return errors.AssertionFailedf("emitted key %s above asOf timestamp %s", key, f.asOf) - } - - // Tombstones should not be emitted. - if _, isTombstone, err := f.MVCCValueLenAndIsTombstone(); err != nil { - return errors.NewAssertionErrorWithWrappedErrf(err, "invalid value") - } else if isTombstone { - return errors.AssertionFailedf("emitted tombstone for key %s", key) - } - - return nil + // nolint:errcheck + return must.Handle(context.Background(), func(ctx context.Context) { + // asOf must be set. + must.NotZero(ctx, f.asOf, "f.asOf is empty") + + // The underlying iterator must be valid. + ok, err := f.iter.Valid() + must.NoError(ctx, err, "underlying iter errored") + must.True(ctx, ok, "underlying iter is invalid") + + // Keys can't be intents or inline values, and must have timestamps at or + // below the readAsOf timestamp. + key := f.UnsafeKey() + must.NotZero(ctx, key.Timestamp, "emitted key %s has no timestamp", key) + must.True(ctx, key.Timestamp.LessEq(f.asOf), "emitted key %s above asOf %s", key, f.asOf) + + // Tombstones should not be emitted. + _, isTombstone, err := f.MVCCValueLenAndIsTombstone() + must.NoError(ctx, err, "invalid value") + must.False(ctx, isTombstone, "emitted tombstone for key %s", key) + }) } diff --git a/pkg/storage/testdata/intent_interleaving_iter/error_race b/pkg/storage/testdata/intent_interleaving_iter/error_race index 25f0f6322126..b1fd481eeaf4 100644 --- a/pkg/storage/testdata/intent_interleaving_iter/error_race +++ b/pkg/storage/testdata/intent_interleaving_iter/error_race @@ -22,13 +22,13 @@ prev seek-lt k=e prev ---- -seek-ge "a"/0,0: output: err: missing provisional value for i.intentKey="b": i.iter exhausted +seek-ge "a"/0,0: output: err: missing provisional value for i.intentKey "b"; i.iter exhausted next: output: err: intentIter at intent, but iter not at provisional value seek-lt "e"/0,0: output: meta k=d ts=40.000000000,0 txn=4 next: output: err: intent has no provisional value -seek-ge "d"/0,0: output: err: missing provisional value for i.intentKey="d": i.iter exhausted +seek-ge "d"/0,0: output: err: missing provisional value for i.intentKey "d"; i.iter exhausted next-key: output: err: intentIter at intent, but iter not at provisional value -seek-ge "d"/0,0: output: err: missing provisional value for i.intentKey="d": i.iter exhausted +seek-ge "d"/0,0: output: err: missing provisional value for i.intentKey "d"; i.iter exhausted prev: output: err: iter not at provisional value, cmp: -1 seek-lt "e"/0,0: output: meta k=d ts=40.000000000,0 txn=4 prev: output: err: reverse iteration discovered intent without provisional value @@ -62,8 +62,8 @@ prev seek-ge "a"/0,0: output: meta k=a ts=10.000000000,0 txn=1 next: output: value k=a ts=10.000000000,0 v=a10 next: output: meta k=b ts=20.000000000,0 txn=4 -next: output: err: i.intentCmp=1 does not match 0 for intentKey="b" iterKey="b"/20.000000000,0 -next: output: err: missing provisional value for i.intentKey="b": i.intentCmp=-1 is not 0 +next: output: err: i.intentKey.Compare != i.intentCmp at "b": 0 != 1 +next: output: err: missing provisional value for i.intentKey "b"; i.intentCmp != 0: expected zero value, got -1 next: output: err: intentIter at intent, but iter not at provisional value seek-lt "d"/0,0: output: value k=c ts=30.000000000,0 v=c30 prev: output: meta k=c ts=30.000000000,0 txn=4