diff --git a/pkg/keys/keys.go b/pkg/keys/keys.go index a4e1cb8963d7..a9b4b22483a3 100644 --- a/pkg/keys/keys.go +++ b/pkg/keys/keys.go @@ -393,22 +393,28 @@ func QueueLastProcessedKey(key roachpb.RKey, queue string) roachpb.Key { } // LockTableSingleKey creates a key under which all single-key locks for the -// given key can be found. Note that there can be multiple locks for the given -// key, but those are distinguished using the "version" which is not in scope -// of the keys package. +// given key can be found. buf is used as scratch-space to avoid allocations +// -- its contents will be overwritten and not appended to. +// Note that there can be multiple locks for the given key, but those are +// distinguished using the "version" which is not in scope of the keys +// package. // For a scan [start, end) the corresponding lock table scan is // [LTSK(start), LTSK(end)). -func LockTableSingleKey(key roachpb.Key) roachpb.Key { +func LockTableSingleKey(key roachpb.Key, buf []byte) (roachpb.Key, []byte) { + // The +3 accounts for the bytesMarker and terminator. + keyLen := len(LocalRangeLockTablePrefix) + len(LockTableSingleKeyInfix) + len(key) + 3 + if cap(buf) < keyLen { + buf = make([]byte, 0, keyLen) + } else { + buf = buf[:0] + } // Don't unwrap any local prefix on key using Addr(key). This allow for // doubly-local lock table keys. For example, local range descriptor keys can // be locked during split and merge transactions. - // The +3 account for the bytesMarker and terminator. - buf := make(roachpb.Key, 0, - len(LocalRangeLockTablePrefix)+len(LockTableSingleKeyInfix)+len(key)+3) buf = append(buf, LocalRangeLockTablePrefix...) buf = append(buf, LockTableSingleKeyInfix...) buf = encoding.EncodeBytesAscending(buf, key) - return buf + return buf, buf } // DecodeLockTableSingleKey decodes the single-key lock table key to return the key @@ -448,6 +454,12 @@ func IsLocal(k roachpb.Key) bool { return bytes.HasPrefix(k, localPrefix) } +// IsLocalStoreKey performs a cheap check that returns true iff the parameter +// is a local store key. +func IsLocalStoreKey(k roachpb.Key) bool { + return bytes.HasPrefix(k, localStorePrefix) +} + // Addr returns the address for the key, used to lookup the range containing the // key. In the normal case, this is simply the key's value. However, for local // keys, such as transaction records, the address is the inner encoded key, with diff --git a/pkg/keys/keys_test.go b/pkg/keys/keys_test.go index 493d8d73389d..1bafe4d86690 100644 --- a/pkg/keys/keys_test.go +++ b/pkg/keys/keys_test.go @@ -718,7 +718,7 @@ func TestLockTableKeyEncodeDecode(t *testing.T) { } for _, test := range testCases { t.Run("", func(t *testing.T) { - ltKey := LockTableSingleKey(test.key) + ltKey, _ := LockTableSingleKey(test.key, nil) require.True(t, bytes.HasPrefix(ltKey, expectedPrefix)) k, err := DecodeLockTableSingleKey(ltKey) require.NoError(t, err) diff --git a/pkg/storage/BUILD.bazel b/pkg/storage/BUILD.bazel index 9c71ff3c81ba..4bf312428768 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -14,6 +14,7 @@ go_library( "error.go", "file_util.go", "in_mem.go", + "intent_interleaving_iter.go", "multi_iterator.go", "mvcc.go", "mvcc_incremental_iterator.go", @@ -81,6 +82,7 @@ go_library( "//pkg/settings/cluster", "//pkg/storage/enginepb", "//pkg/storage/fs", + "//pkg/util", "//pkg/util/bufalloc", "//pkg/util/encoding", "//pkg/util/envutil", @@ -116,6 +118,7 @@ go_test( "disk_map_test.go", "engine_key_test.go", "engine_test.go", + "intent_interleaving_iter_test.go", "main_test.go", "multi_iterator_test.go", "mvcc_history_test.go", @@ -147,6 +150,7 @@ go_test( "//pkg/testutils", "//pkg/testutils/skip", "//pkg/testutils/zerofields", + "//pkg/util", "//pkg/util/caller", "//pkg/util/encoding", "//pkg/util/fileutil", diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index 3f4a62d23a68..c0e1218dc09c 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -60,7 +60,8 @@ type SimpleMVCCIterator interface { // NextKey advances the iterator to the next MVCC key. This operation is // distinct from Next which advances to the next version of the current key // or the next key if the iterator is currently located at the last version - // for a key. + // for a key. NextKey must not be used to switch iteration direction from + // reverse iteration to forward iteration. NextKey() // UnsafeKey returns the same value as Key, but the memory is invalidated on // the next call to {Next,NextKey,Prev,SeekGE,SeekLT,Close}. @@ -125,7 +126,8 @@ type MVCCIterator interface { // and the encoded SST data specified, within the provided key range. Returns // stats on skipped KVs, or an error if a collision is found. CheckForKeyCollisions(sstData []byte, start, end roachpb.Key) (enginepb.MVCCStats, error) - // SetUpperBound installs a new upper bound for this iterator. + // SetUpperBound installs a new upper bound for this iterator. The caller can modify + // the parameter after this function returns. SetUpperBound(roachpb.Key) // Stats returns statistics about the iterator. Stats() IteratorStats @@ -294,7 +296,8 @@ type Reader interface { NewMVCCIterator(iterKind MVCCIterKind, opts IterOptions) MVCCIterator // NewEngineIterator returns a new instance of an EngineIterator over this // engine. The caller must invoke EngineIterator.Close() when finished - // with the iterator to free resources. + // with the iterator to free resources. The caller can change IterOptions + // after this function returns. NewEngineIterator(opts IterOptions) EngineIterator } diff --git a/pkg/storage/engine_key.go b/pkg/storage/engine_key.go index 33570314b6f0..a25f897e74ac 100644 --- a/pkg/storage/engine_key.go +++ b/pkg/storage/engine_key.go @@ -251,8 +251,9 @@ func (lk LockTableKey) ToEngineKey() EngineKey { if lk.Strength != lock.Exclusive { panic("unsupported lock strength") } + ltKey, _ := keys.LockTableSingleKey(lk.Key, nil) k := EngineKey{ - Key: keys.LockTableSingleKey(lk.Key), + Key: ltKey, Version: make([]byte, engineKeyVersionLockTableLen), } k.Version[0] = byte(lk.Strength) diff --git a/pkg/storage/intent_interleaving_iter.go b/pkg/storage/intent_interleaving_iter.go new file mode 100644 index 000000000000..29a40850c931 --- /dev/null +++ b/pkg/storage/intent_interleaving_iter.go @@ -0,0 +1,638 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package storage + +import ( + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/errors" +) + +// intentInterleavingIter makes separated intents appear as interleaved. It +// relies on the following assumptions: +// - There can also be intents that are physically interleaved. +// However, for a particular roachpb.Key there will be at most one intent, +// either interleaved or separated. +// - An intent will have a corresponding provisional value. +// - The only single key locks in the lock table key space are intents. +// +// Semantically, the functionality is equivalent to merging two MVCCIterators: +// - A MVCCIterator on the MVCC key space. +// - A MVCCIterator constructed by wrapping an EngineIterator on the lock table +// key space where the EngineKey is transformed into the corresponding +// intent key and appears as MVCCKey{Key: intentKey}. +// The implementation below is specialized to reduce unnecessary comparisons +// and iteration, by utilizing the aforementioned assumptions. The intentIter +// iterates over the lock table key space and iter over the MVCC key space. +// They are kept synchronized in the following way (for forward iteration): +// - At the same MVCCKey.Key: the intentIter is at the intent and iter at the +// provisional value. +// - At different MVCCKey.Keys: the intentIter is ahead of iter, at the first +// key after iter's MVCCKey.Key that has a separated intent. +// Note that in both cases the iterators are apart by the minimal possible +// distance. This minimal distance rule applies for reverse iteration too, and +// can be used to construct similar invariants. +// The one exception to the minimal distance rule is a sub-case of prefix +// iteration, when we know that no separated intents need to be seen, and so +// don't bother positioning intentIter. +type intentInterleavingIter struct { + prefix bool + // iter is for iterating over MVCC keys and interleaved intents. + iter MVCCIterator + // intentIter is for iterating over separated intents, so that + // intentInterleavingIter can make them look as if they were interleaved. + intentIter EngineIterator + // The decoded key from the lock table. This is an unsafe key + // in that it is only valid when intentIter has not been + // repositioned. It is nil if the intentIter is considered to be + // exhausted. Note that the intentIter may still be positioned + // at a valid position in the case of prefix iteration, but the + // state of the intentKey overrides that state. + intentKey roachpb.Key + // - cmp output of (intentKey, current iter key) when both are non-nil. + // This does not take timestamps into consideration. So if intentIter + // is at an intent, and iter is at the corresponding provisional value, + // cmp will be 0. See the longer struct-level comment for more on the + // relative positioning of intentIter and iter. + // - intentKey==nil, current iter key!=nil, cmp=dir + // (i.e., the nil key is akin to infinity in the forward direction + // and -infinity in the reverse direction, since that iterator is + // exhausted). + // - intentKey!=nil, current iter key==nil, cmp=-dir. + // - If both are nil. cmp is undefined and valid=false. + intentCmp int + // The current direction. +1 for forward, -1 for reverse. + dir int + valid bool + err error + + hasUpperBound bool + + intentKeyBuf []byte +} + +var _ MVCCIterator = &intentInterleavingIter{} + +func newIntentInterleavingIterator(reader Reader, opts IterOptions) MVCCIterator { + if !opts.MinTimestampHint.IsEmpty() || !opts.MaxTimestampHint.IsEmpty() { + panic("intentInterleavingIter must not be used with timestamp hints") + } + intentOpts := opts + var intentKeyBuf []byte + if opts.LowerBound != nil { + intentOpts.LowerBound, intentKeyBuf = keys.LockTableSingleKey(opts.LowerBound, nil) + } + if opts.UpperBound != nil { + intentOpts.UpperBound, _ = keys.LockTableSingleKey(opts.UpperBound, nil) + } + // Note that we can reuse intentKeyBuf after NewEngineIterator returns. + intentIter := reader.NewEngineIterator(intentOpts) + + // We assume that callers iterating forward will set an upper bound, + // and callers iterating in reverse will set a lower bound, which + // will prevent them from accidentally iterating into the lock-table + // key space. The MVCCIterator implementations require one of the bounds + // or prefix iteration. We remember whether the upper bound has been + // set, so if not set, we can set the upper bound when SeekGE is called + // for prefix iteration. + iter := reader.NewMVCCIterator(MVCCKeyIterKind, opts) + return &intentInterleavingIter{ + prefix: opts.Prefix, + iter: iter, + intentIter: intentIter, + hasUpperBound: opts.UpperBound != nil, + intentKeyBuf: intentKeyBuf, + } +} + +func (i *intentInterleavingIter) SeekGE(key MVCCKey) { + i.dir = +1 + i.valid = true + i.err = nil + + if i.prefix { + // Caller will use a mix of SeekGE and Next. If key is before the lock table key + // space, make sure there is an upper bound, if not explicitly set at creation time + // or using SetUpperBound. We do not set hasUpperBound to true since this is + // an implicit (not set by the user) upper-bound, and we want to change it on + // a subsequent call to SeekGE. + if !i.hasUpperBound && keys.IsLocal(key.Key) && !keys.IsLocalStoreKey(key.Key) { + i.iter.SetUpperBound(keys.LocalRangeLockTablePrefix) + } + } + var intentSeekKey roachpb.Key + if key.Timestamp.IsEmpty() { + // Common case. + intentSeekKey, i.intentKeyBuf = keys.LockTableSingleKey(key.Key, i.intentKeyBuf) + } else if !i.prefix { + // Seeking to a specific version, so go past the intent. + intentSeekKey, i.intentKeyBuf = keys.LockTableSingleKey(key.Key.Next(), i.intentKeyBuf) + } else { + // Else seeking to a particular version and using prefix iteration, + // so don't expect to ever see the intent. NB: intentSeekKey is nil. + i.intentKey = nil + } + + if intentSeekKey != nil { + valid, err := i.intentIter.SeekEngineKeyGE(EngineKey{Key: intentSeekKey}) + if err != nil { + i.err = err + i.valid = false + return + } + if err := i.tryDecodeLockKey(valid); err != nil { + return + } + } + i.iter.SeekGE(key) + i.computePos() +} + +func (i *intentInterleavingIter) computePos() { + valid, err := i.iter.Valid() + if err != nil || (!valid && i.intentKey == nil) { + i.err = err + i.valid = false + return + } + // INVARIANT: err == nil && (valid || i.intentKey != nil) + if !valid { + i.intentCmp = -i.dir + } else if i.intentKey == nil { + i.intentCmp = i.dir + } else { + i.intentCmp = i.intentKey.Compare(i.iter.UnsafeKey().Key) + } +} + +func (i *intentInterleavingIter) tryDecodeLockKey(valid bool) error { + if !valid { + // NB: this does not set i.valid = false, since this method does not care + // about the state of i.iter, which may be valid. It is the caller's + // responsibility to additionally use the state of i.iter to appropriately + // set i.valid. + i.intentKey = nil + return nil + } + engineKey, err := i.intentIter.UnsafeEngineKey() + if err != nil { + i.err = err + i.valid = false + return err + } + if i.intentKey, err = keys.DecodeLockTableSingleKey(engineKey.Key); err != nil { + i.err = err + i.valid = false + return err + } + return nil +} + +func (i *intentInterleavingIter) Valid() (bool, error) { + return i.valid, i.err +} + +func (i *intentInterleavingIter) Next() { + if i.err != nil { + return + } + if i.dir < 0 { + // Switching from reverse to forward iteration. + isCurAtIntent := i.isCurAtIntentIter() + i.dir = +1 + if !i.valid { + // Both iterators are exhausted, since intentKey is synchronized with + // intentIter for non-prefix iteration, so step both forward. + i.valid = true + valid, err := i.intentIter.NextEngineKey() + if err != nil { + i.err = err + i.valid = false + return + } + if err := i.tryDecodeLockKey(valid); err != nil { + return + } + i.iter.Next() + i.computePos() + return + } + // At least one of the iterators is not exhausted. + if isCurAtIntent { + // iter precedes the intentIter, so must be at the lowest version of the + // preceding key or exhausted. So step it forward. It will now point to + // a key that is the same as the intent key since an intent always has a + // corresponding provisional value, and provisional values must have a + // higher timestamp than any committed value on a key. Note that the + // code below does not specifically care if a bug (external to this + // code) violates the invariant that the iter is pointing to the + // provisional value, but it does care that iter is pointing to some + // version of that key. + i.iter.Next() + i.intentCmp = 0 + if valid, err := i.iter.Valid(); err != nil || !valid { + if err == nil { + err = errors.Errorf("intent has no provisional value") + } + i.err = err + i.valid = false + return + } + if util.RaceEnabled { + cmp := i.intentKey.Compare(i.iter.UnsafeKey().Key) + if cmp != 0 { + i.err = errors.Errorf("intent has no provisional value, cmp: %d", cmp) + i.valid = false + return + } + } + } else { + // The intentIter precedes the iter. It could be for the same key, iff + // this key has an intent, or an earlier key. Either way, stepping + // forward will take it to an intent for a later key. Note that iter + // could also be positioned at an intent. We are assuming that there + // isn't a bug (external to this code) that has caused two intents to be + // present for the same key. + valid, err := i.intentIter.NextEngineKey() + if err != nil { + i.err = err + i.valid = false + return + } + i.intentCmp = +1 + if err := i.tryDecodeLockKey(valid); err != nil { + return + } + if util.RaceEnabled && valid { + cmp := i.intentKey.Compare(i.iter.UnsafeKey().Key) + if cmp <= 0 { + i.err = errors.Errorf("intentIter incorrectly positioned, cmp: %d", cmp) + i.valid = false + return + } + } + } + } + if !i.valid { + return + } + if i.intentCmp <= 0 { + // The iterator is positioned at an intent in intentIter. iter must be + // positioned at the provisional value. Note that the code below does not + // specifically care if a bug (external to this code) violates the + // invariant that the iter is pointing to the provisional value, but it + // does care that iter is pointing to some version of that key. + if i.intentCmp != 0 { + i.err = errors.Errorf("intentIter at intent, but iter not at provisional value") + i.valid = false + return + } + valid, err := i.intentIter.NextEngineKey() + if err != nil { + i.err = err + i.valid = false + return + } + if err := i.tryDecodeLockKey(valid); err != nil { + return + } + if valid, err := i.iter.Valid(); err != nil || !valid { + if err == nil { + err = errors.Errorf("iter expected to be at provisional value, but is exhausted") + } + i.err = err + i.valid = false + return + } + i.intentCmp = +1 + if util.RaceEnabled && i.intentKey != nil { + cmp := i.intentKey.Compare(i.iter.UnsafeKey().Key) + if cmp <= 0 { + i.err = errors.Errorf("intentIter incorrectly positioned, cmp: %d", cmp) + i.valid = false + return + } + } + } else { + // Common case: + // The iterator is positioned at iter. It could be a value or an intent, + // though usually it will be a value. + i.iter.Next() + i.computePos() + } +} + +func (i *intentInterleavingIter) NextKey() { + // NextKey is not called to switch directions, i.e., we must already + // be in the forward direction. + if i.dir < 0 { + i.err = errors.Errorf("NextKey cannot be used to switch iteration direction") + i.valid = false + return + } + if !i.valid { + return + } + if i.intentCmp <= 0 { + // The iterator is positioned at an intent in intentIter. iter must be + // positioned at the provisional value. + if i.intentCmp != 0 { + i.err = errors.Errorf("intentIter at intent, but iter not at provisional value") + i.valid = false + return + } + valid, err := i.intentIter.NextEngineKey() + if err != nil { + i.err = err + i.valid = false + return + } + if err := i.tryDecodeLockKey(valid); err != nil { + return + } + // Step the iter to NextKey(), i.e., past all the versions of this key. + i.iter.NextKey() + i.computePos() + } else { + // The iterator is positioned at iter. It could be a value or an intent, + // though usually it will be a value. + // Step the iter to NextKey(), i.e., past all the versions of this key. + i.iter.NextKey() + i.computePos() + } +} + +func (i *intentInterleavingIter) isCurAtIntentIter() bool { + // When both iter and intentIter are exhausted, the return value is + // immaterial since this function won't be called. We examine the remaining + // cases below. + // + // During forward iteration (dir > 0), we have the following cases: + // - iter is exhausted: intentCmp < 0. This will never happen and callers + // check. Returns true. + // - intentIter is exhausted: intentCmp > 0. Returns false. + // - Neither is exhausted: + // - intentCmp < 0. This will never happen and callers check. Returns true. + // - intentCmp = 0. Returns true. + // - intentCmp > 0. Returns false. + // + // During reverse iteration (dir < 0), we have the following cases: + // - iter is exhausted: intentCmp > 0. Returns true. + // - intentIter is exhausted: intentCmp < 0. Returns false. + // - Neither is exhausted: + // - intentCmp <= 0. Returns false. + // - intentCmp > 0. Returns true. + return (i.dir > 0 && i.intentCmp <= 0) || (i.dir < 0 && i.intentCmp > 0) +} + +func (i *intentInterleavingIter) UnsafeKey() MVCCKey { + // If there is a separated intent there cannot also be an interleaved intent + // for the same key. + if i.isCurAtIntentIter() { + return MVCCKey{Key: i.intentKey} + } + return i.iter.UnsafeKey() +} + +func (i *intentInterleavingIter) UnsafeValue() []byte { + if i.isCurAtIntentIter() { + return i.intentIter.UnsafeValue() + } + return i.iter.UnsafeValue() +} + +func (i *intentInterleavingIter) Key() MVCCKey { + key := i.UnsafeKey() + keyCopy := make([]byte, len(key.Key)) + copy(keyCopy, key.Key) + key.Key = keyCopy + return key +} + +func (i *intentInterleavingIter) Value() []byte { + if i.isCurAtIntentIter() { + return i.intentIter.Value() + } + return i.iter.Value() +} + +func (i *intentInterleavingIter) Close() { + i.iter.Close() + i.intentIter.Close() +} + +func (i *intentInterleavingIter) SeekLT(key MVCCKey) { + i.dir = -1 + i.valid = true + i.err = nil + + var intentSeekKey roachpb.Key + if key.Timestamp.IsEmpty() { + // Common case. + intentSeekKey, i.intentKeyBuf = keys.LockTableSingleKey(key.Key, i.intentKeyBuf) + } else { + // Seeking to a specific version, so need to see the intent. + if i.prefix { + i.err = errors.Errorf("prefix iteration is not permitted with SeekLT") + i.valid = false + return + } + // Since we need to see the intent for key.Key, and we don't have SeekLE, call + // Next() on the key before doing SeekLT. + intentSeekKey, i.intentKeyBuf = keys.LockTableSingleKey(key.Key.Next(), i.intentKeyBuf) + } + valid, err := i.intentIter.SeekEngineKeyLT(EngineKey{Key: intentSeekKey}) + if err != nil { + i.err = err + i.valid = false + return + } + if err := i.tryDecodeLockKey(valid); err != nil { + return + } + i.iter.SeekLT(key) + i.computePos() +} + +func (i *intentInterleavingIter) Prev() { + if i.err != nil { + return + } + if i.dir > 0 { + // Switching from forward to reverse iteration. + isCurAtIntent := i.isCurAtIntentIter() + i.dir = -1 + if !i.valid { + // Both iterators are exhausted, so step both backward. + i.valid = true + valid, err := i.intentIter.PrevEngineKey() + if err != nil { + i.err = err + i.valid = false + return + } + if err := i.tryDecodeLockKey(valid); err != nil { + return + } + i.iter.Prev() + i.computePos() + return + } + // At least one of the iterators is not exhausted. + if isCurAtIntent { + // iter is after the intentIter, so must be at the provisional value. + // Step it backward. It will now point to a key that is before the + // intent key. Note that the code below does not specifically care if a + // bug (external to this code) violates the invariant that the + // provisional value is the highest timestamp key, but it does care that + // there is a timestamped value for this key (which it checks below). + // The internal invariant of this iterator implementation will ensure + // that iter is pointing to the highest timestamped key. + if i.intentCmp != 0 { + i.err = errors.Errorf("iter not at provisional value, cmp: %d", i.intentCmp) + i.valid = false + return + } + i.iter.Prev() + i.intentCmp = +1 + valid, err := i.iter.Valid() + if err != nil { + i.err = err + i.valid = false + return + } + if util.RaceEnabled && valid { + cmp := i.intentKey.Compare(i.iter.UnsafeKey().Key) + if cmp <= 0 { + i.err = errors.Errorf("intentIter should be after iter, cmp: %d", cmp) + i.valid = false + return + } + } + } else { + // The intentIter is after the iter. We don't know whether the iter key + // has an intent. Note that the iter could itself be positioned at an + // intent. + valid, err := i.intentIter.PrevEngineKey() + if err != nil { + i.err = err + i.valid = false + return + } + if err := i.tryDecodeLockKey(valid); err != nil { + return + } + i.computePos() + } + } + if !i.valid { + return + } + if i.intentCmp > 0 { + // The iterator is positioned at an intent in intentIter, and iter is + // exhausted or positioned at a versioned value of a preceding key. + // Stepping intentIter backward will ensure that intentKey is <= the key + // of iter (when neither is exhausted). + intentIterValid, err := i.intentIter.PrevEngineKey() + if err != nil { + i.err = err + i.valid = false + return + } + if err := i.tryDecodeLockKey(intentIterValid); err != nil { + return + } + if iterValid, err := i.iter.Valid(); err != nil || !iterValid { + // It !iterValid, the intentIter can no longer be valid either. + if !iterValid && err == nil && intentIterValid { + i.err = errors.Errorf("reverse iteration discovered intent without provisional value") + } else { + i.err = err + } + i.valid = false + return + } + // iterValid == true. So positioned at iter. + i.intentCmp = -1 + if i.intentKey != nil { + i.intentCmp = i.intentKey.Compare(i.iter.UnsafeKey().Key) + if i.intentCmp > 0 { + i.err = errors.Errorf("intentIter should not be after iter") + i.valid = false + return + } + } + } else { + // Common case: + // The iterator is positioned at iter. It could be a value or an intent, + // though usually it will be a value. + i.iter.Prev() + i.computePos() + } +} + +func (i *intentInterleavingIter) UnsafeRawKey() []byte { + if i.isCurAtIntentIter() { + // TODO(sumeer): this is inefficient, but the users of UnsafeRawKey are + // incorrect, so this method will go away. + key, err := i.intentIter.UnsafeEngineKey() + if err != nil { + // Should be able to parse it again. + panic(err) + } + return key.Encode() + } + return i.iter.UnsafeRawKey() +} + +func (i *intentInterleavingIter) ValueProto(msg protoutil.Message) error { + value := i.UnsafeValue() + return protoutil.Unmarshal(value, msg) +} + +func (i *intentInterleavingIter) ComputeStats( + start, end roachpb.Key, nowNanos int64, +) (enginepb.MVCCStats, error) { + return ComputeStatsForRange(i, start, end, nowNanos) +} + +func (i *intentInterleavingIter) FindSplitKey( + start, end, minSplitKey roachpb.Key, targetSize int64, +) (MVCCKey, error) { + return findSplitKeyUsingIterator(i, start, end, minSplitKey, targetSize) +} + +func (i *intentInterleavingIter) CheckForKeyCollisions( + sstData []byte, start, end roachpb.Key, +) (enginepb.MVCCStats, error) { + return checkForKeyCollisionsGo(i, sstData, start, end) +} + +func (i *intentInterleavingIter) SetUpperBound(key roachpb.Key) { + i.iter.SetUpperBound(key) + var intentUpperBound roachpb.Key + intentUpperBound, i.intentKeyBuf = keys.LockTableSingleKey(key, i.intentKeyBuf) + // Note that we can reuse intentKeyBuf after SetUpperBound returns. + i.intentIter.SetUpperBound(intentUpperBound) + i.hasUpperBound = key != nil +} + +func (i *intentInterleavingIter) Stats() IteratorStats { + // Only used in tests. + return i.iter.Stats() +} + +func (i *intentInterleavingIter) SupportsPrev() bool { + return true +} diff --git a/pkg/storage/intent_interleaving_iter_test.go b/pkg/storage/intent_interleaving_iter_test.go new file mode 100644 index 000000000000..b0a063b36b1a --- /dev/null +++ b/pkg/storage/intent_interleaving_iter_test.go @@ -0,0 +1,295 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package storage + +import ( + "bytes" + "fmt" + "strings" + "testing" + + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "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/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/cockroach/pkg/util/uint128" + "github.com/cockroachdb/cockroach/pkg/util/uuid" + "github.com/cockroachdb/datadriven" +) + +// TODO(sumeer): +// - randomized test with random intents that are interleaved and +// compare with no interleaved intents without using intentInterleavingIter. +// - microbenchmarks to compare with non-interleaved intents. + +func scanRoachKey(t *testing.T, td *datadriven.TestData, field string) roachpb.Key { + var k string + td.ScanArgs(t, field, &k) + rk := roachpb.Key(k) + if strings.HasPrefix(k, "L") { + return append(keys.LocalRangePrefix, rk[1:]...) + } + return rk +} + +func makePrintableKey(k MVCCKey) MVCCKey { + if bytes.HasPrefix(k.Key, keys.LocalRangePrefix) { + k.Key = append([]byte("L"), k.Key[len(keys.LocalRangePrefix):]...) + } + return k +} + +func scanSeekKey(t *testing.T, td *datadriven.TestData) MVCCKey { + key := MVCCKey{Key: scanRoachKey(t, td, "k")} + if td.HasArg("ts") { + var ts int + td.ScanArgs(t, "ts", &ts) + key.Timestamp.WallTime = int64(ts) + } + return key +} + +func checkAndOutputIter(iter MVCCIterator, b *strings.Builder) { + valid, err := iter.Valid() + if err != nil { + fmt.Fprintf(b, "output: err: %s\n", err) + return + } + if !valid { + fmt.Fprintf(b, "output: .\n") + return + } + k1 := makePrintableKey(iter.UnsafeKey()) + k2 := makePrintableKey(iter.Key()) + if !k1.Equal(k2) { + fmt.Fprintf(b, "output: key: %s != %s\n", k1, k2) + return + } + v1 := iter.UnsafeValue() + v2 := iter.Value() + if !bytes.Equal(v1, v2) { + fmt.Fprintf(b, "output: value: %x != %x\n", v1, v2) + return + } + if k1.Timestamp.IsEmpty() { + var meta enginepb.MVCCMetadata + if err := protoutil.Unmarshal(v1, &meta); err != nil { + fmt.Fprintf(b, "output: meta parsing: %s\n", err) + return + } + if meta.Timestamp.ToTimestamp().IsEmpty() { + fmt.Fprintf(b, "output: meta k=%s\n", string(k1.Key)) + } else { + uuid := meta.Txn.ID.ToUint128() + var hiStr string + if uuid.Hi != 0 { + hiStr = fmt.Sprintf("%d,", uuid.Hi) + } + fmt.Fprintf(b, "output: meta k=%s ts=%d txn=%s%d\n", + string(k1.Key), meta.Timestamp.WallTime, hiStr, uuid.Lo) + } + return + } + fmt.Fprintf(b, "output: value k=%s ts=%d v=%s\n", + string(k1.Key), k1.Timestamp.WallTime, string(v1)) +} + +// TestIntentInterleavingIter is a datadriven test consisting of two commands: +// - define: defines key-value pairs in the lock table and MVCC key spaces. +// Intents can be in both key spaces, and inline meta and MVCC values in +// the latter. +// meta k= ts= txn= defines an intent +// meta k= defines an inline meta +// value k= ts= v= defines an MVCC value +// It is acceptable to define intents without provisional values to test +// out error checking code paths. +// - iter: for iterating, is defined as +// iter [lower=] [upper=] [prefix=] +// followed by newline separated sequence of operations: +// next, prev, seek-lt, seek-ge, set-upper, next-key +// +// A key starting with L is interpreted as a local-range key. +func TestIntentInterleavingIter(t *testing.T) { + defer leaktest.AfterTest(t)() + + var eng Engine + defer func() { + if eng != nil { + eng.Close() + } + }() + + datadriven.Walk(t, "testdata/intent_interleaving_iter", func(t *testing.T, path string) { + if (util.RaceEnabled && strings.HasSuffix(path, "race_off")) || + (!util.RaceEnabled && strings.HasSuffix(path, "race")) { + return + } + datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { + switch d.Cmd { + case "define": + if eng != nil { + eng.Close() + } + eng = createTestPebbleEngine() + batch := eng.NewBatch() + var locksSection bool + var mvccSection bool + // pos is the original : prefix computed by + // datadriven. It points to the top "define" command itself. + // We editing d.Pos in-place below by extending `pos` upon + // each new line. + pos := d.Pos + for i, line := range strings.Split(d.Input, "\n") { + switch line { + case "locks": + locksSection = true + mvccSection = false + continue + case "mvcc": + locksSection = false + mvccSection = true + continue + } + // Compute a line prefix, to clarify error message. We + // prefix a newline character because some text editor do + // not know how to jump to the location of an error if + // there are multiple file:line prefixes on the same line. + d.Pos = fmt.Sprintf("\n%s: (+%d)", pos, i+1) + if !locksSection && !mvccSection { + t.Fatalf("%s: not a locks or mvcc section", d.Pos) + } + var err error + if d.Cmd, d.CmdArgs, err = datadriven.ParseLine(line); err != nil { + t.Fatalf("%s: %s", d.Pos, err) + } + switch d.Cmd { + case "meta": + key := scanRoachKey(t, d, "k") + // We don't bother populating most fields in the proto. + var meta enginepb.MVCCMetadata + var txnUUID uuid.UUID + if locksSection || d.HasArg("ts") { + var ts, txn int + d.ScanArgs(t, "ts", &ts) + meta.Timestamp.WallTime = int64(ts) + d.ScanArgs(t, "txn", &txn) + txnUUID = uuid.FromUint128(uint128.FromInts(0, uint64(txn))) + meta.Txn = &enginepb.TxnMeta{ID: txnUUID} + } + val, err := protoutil.Marshal(&meta) + if err != nil { + return err.Error() + } + if mvccSection { + // This is an abuse of PutUnversioned, but we know the + // implementation and can be sure that it will not change the key. + if err := batch.PutUnversioned(key, val); err != nil { + return err.Error() + } + } else { + ltKey := LockTableKey{Key: key, Strength: lock.Exclusive, TxnUUID: txnUUID[:]} + if err := batch.PutEngineKey(ltKey.ToEngineKey(), val); err != nil { + return err.Error() + } + } + case "value": + if locksSection { + t.Fatalf("%s: value in locks section", d.Pos) + } + key := scanRoachKey(t, d, "k") + var ts int + d.ScanArgs(t, "ts", &ts) + var value string + d.ScanArgs(t, "v", &value) + mvccKey := MVCCKey{Key: key, Timestamp: hlc.Timestamp{WallTime: int64(ts)}} + if err := batch.PutMVCC(mvccKey, []byte(value)); err != nil { + return err.Error() + } + } + } + d.Pos = pos + if err := batch.Commit(true); err != nil { + return err.Error() + } + return "" + + case "iter": + var opts IterOptions + if d.HasArg("lower") { + opts.LowerBound = scanRoachKey(t, d, "lower") + } + if d.HasArg("upper") { + opts.UpperBound = scanRoachKey(t, d, "upper") + } + if d.HasArg("prefix") { + d.ScanArgs(t, "prefix", &opts.Prefix) + } + iter := newIntentInterleavingIterator(eng, opts) + var b strings.Builder + defer iter.Close() + // pos is the original : prefix computed by + // datadriven. It points to the top "define" command itself. + // We editing d.Pos in-place below by extending `pos` upon + // each new line. + pos := d.Pos + for i, line := range strings.Split(d.Input, "\n") { + // Compute a line prefix, to clarify error message. We + // prefix a newline character because some text editor do + // not know how to jump to the location of an error if + // there are multiple file:line prefixes on the same line. + d.Pos = fmt.Sprintf("\n%s: (+%d)", pos, i+1) + var err error + if d.Cmd, d.CmdArgs, err = datadriven.ParseLine(line); err != nil { + t.Fatalf("%s: %s", d.Pos, err) + } + switch d.Cmd { + case "seek-ge": + key := scanSeekKey(t, d) + iter.SeekGE(key) + fmt.Fprintf(&b, "seek-ge %s: ", makePrintableKey(key)) + checkAndOutputIter(iter, &b) + case "next": + iter.Next() + fmt.Fprintf(&b, "next: ") + checkAndOutputIter(iter, &b) + case "seek-lt": + key := scanSeekKey(t, d) + iter.SeekLT(key) + fmt.Fprintf(&b, "seek-lt %s: ", makePrintableKey(key)) + checkAndOutputIter(iter, &b) + case "prev": + iter.Prev() + fmt.Fprintf(&b, "prev: ") + checkAndOutputIter(iter, &b) + case "next-key": + iter.NextKey() + fmt.Fprintf(&b, "next-key: ") + checkAndOutputIter(iter, &b) + case "set-upper": + k := scanRoachKey(t, d, "k") + iter.SetUpperBound(k) + fmt.Fprintf(&b, "set-upper %s\n", string(makePrintableKey(MVCCKey{Key: k}).Key)) + default: + fmt.Fprintf(&b, "unknown command: %s\n", d.Cmd) + } + } + d.Pos = pos + return b.String() + default: + return fmt.Sprintf("unknown command: %s", d.Cmd) + } + }) + }) +} diff --git a/pkg/storage/pebble_iterator.go b/pkg/storage/pebble_iterator.go index b143607e6bbc..38eafa1b43c8 100644 --- a/pkg/storage/pebble_iterator.go +++ b/pkg/storage/pebble_iterator.go @@ -398,6 +398,12 @@ func IsValidSplitKey(key roachpb.Key) bool { // FindSplitKey implements the MVCCIterator interface. func (p *pebbleIterator) FindSplitKey( start, end, minSplitKey roachpb.Key, targetSize int64, +) (MVCCKey, error) { + return findSplitKeyUsingIterator(p, start, end, minSplitKey, targetSize) +} + +func findSplitKeyUsingIterator( + iter MVCCIterator, start, end, minSplitKey roachpb.Key, targetSize int64, ) (MVCCKey, error) { const timestampLen = 12 @@ -424,12 +430,16 @@ func (p *pebbleIterator) FindSplitKey( // terminate iteration because the iterator's upper bound has already been // set to end. mvccMinSplitKey := MakeMVCCMetadataKey(minSplitKey) - p.SeekGE(MakeMVCCMetadataKey(start)) - for ; p.iter.Valid(); p.iter.Next() { - mvccKey, err := DecodeMVCCKey(p.iter.Key()) + iter.SeekGE(MakeMVCCMetadataKey(start)) + for ; ; iter.Next() { + valid, err := iter.Valid() if err != nil { return MVCCKey{}, err } + if !valid { + break + } + mvccKey := iter.UnsafeKey() diff := targetSize - sizeSoFar if diff < 0 { @@ -470,7 +480,7 @@ func (p *pebbleIterator) FindSplitKey( bestSplitKey.Key = append(bestSplitKey.Key[:0], prevKey.Key...) } - sizeSoFar += int64(len(p.iter.Value())) + sizeSoFar += int64(len(iter.Value())) if mvccKey.IsValue() && bytes.Equal(prevKey.Key, mvccKey.Key) { // We only advanced timestamps, but not new mvcc keys. sizeSoFar += timestampLen diff --git a/pkg/storage/testdata/intent_interleaving_iter/basic b/pkg/storage/testdata/intent_interleaving_iter/basic new file mode 100644 index 000000000000..9b4ef8a55dde --- /dev/null +++ b/pkg/storage/testdata/intent_interleaving_iter/basic @@ -0,0 +1,428 @@ +# Both separated and interleaved intents, and one inline meta. +define +locks +meta k=a ts=20 txn=1 +mvcc +value k=a ts=20 v=a20 +value k=a ts=10 v=a10 +meta k=b ts=30 txn=2 +value k=b ts=30 v=b30 +meta k=c +value k=d ts=25 v=d25 +---- + +# Exercise basic forward and reverse iteration. +iter lower=a upper=f +seek-ge k=a +next +next +next +next +next +next +next +prev +prev +prev +prev +prev +prev +prev +prev +set-upper k=c +seek-ge k=b +next +next +prev +prev +prev +seek-lt k=b +next +prev +prev +prev +prev +next +---- +seek-ge "a"/0,0: output: meta k=a ts=20 txn=1 +next: output: value k=a ts=20 v=a20 +next: output: value k=a ts=10 v=a10 +next: output: meta k=b ts=30 txn=2 +next: output: value k=b ts=30 v=b30 +next: output: meta k=c +next: output: value k=d ts=25 v=d25 +next: output: . +prev: output: value k=d ts=25 v=d25 +prev: output: meta k=c +prev: output: value k=b ts=30 v=b30 +prev: output: meta k=b ts=30 txn=2 +prev: output: value k=a ts=10 v=a10 +prev: output: value k=a ts=20 v=a20 +prev: output: meta k=a ts=20 txn=1 +prev: output: . +set-upper c +seek-ge "b"/0,0: output: meta k=b ts=30 txn=2 +next: output: value k=b ts=30 v=b30 +next: output: . +prev: output: value k=b ts=30 v=b30 +prev: output: meta k=b ts=30 txn=2 +prev: output: value k=a ts=10 v=a10 +seek-lt "b"/0,0: output: value k=a ts=10 v=a10 +next: output: meta k=b ts=30 txn=2 +prev: output: value k=a ts=10 v=a10 +prev: output: value k=a ts=20 v=a20 +prev: output: meta k=a ts=20 txn=1 +prev: output: . +next: output: meta k=a ts=20 txn=1 + + +# More forward and reverse iteration. +iter upper=b +seek-ge k=a +next +next +next +prev +prev +prev +---- +seek-ge "a"/0,0: output: meta k=a ts=20 txn=1 +next: output: value k=a ts=20 v=a20 +next: output: value k=a ts=10 v=a10 +next: output: . +prev: output: value k=a ts=10 v=a10 +prev: output: value k=a ts=20 v=a20 +prev: output: meta k=a ts=20 txn=1 + +# Prefix iteration. +iter prefix=true +seek-ge k=b +next +next +seek-ge k=a +next +next +next +prev +seek-ge k=f +seek-ge k=c +next +---- +seek-ge "b"/0,0: output: meta k=b ts=30 txn=2 +next: output: value k=b ts=30 v=b30 +next: output: . +seek-ge "a"/0,0: output: meta k=a ts=20 txn=1 +next: output: value k=a ts=20 v=a20 +next: output: value k=a ts=10 v=a10 +next: output: . +prev: output: err: pebble: unsupported reverse prefix iteration +seek-ge "f"/0,0: output: . +seek-ge "c"/0,0: output: meta k=c +next: output: . + +# Prefix iteration and NextKey. What we will see after the prefix is +# exhausted is undefined. +iter prefix=true +seek-ge k=d +next-key +seek-ge k=a +next-key +seek-ge k=a +next +next-key +---- +seek-ge "d"/0,0: output: value k=d ts=25 v=d25 +next-key: output: . +seek-ge "a"/0,0: output: meta k=a ts=20 txn=1 +next-key: output: meta k=b ts=30 txn=2 +seek-ge "a"/0,0: output: meta k=a ts=20 txn=1 +next: output: value k=a ts=20 v=a20 +next-key: output: meta k=b ts=30 txn=2 + +# Seek to particular timestamp. +iter lower=a upper=f +seek-ge k=a ts=10 +next +next +seek-ge k=a ts=25 +next +next +seek-lt k=a ts=1 +prev +prev +prev +next +next +prev +next +seek-lt k=a ts=15 +prev +prev +next +seek-lt k=a ts=25 +prev +next +seek-ge k=a ts=5 +next +next +prev +seek-lt k=b ts=40 +prev +prev +prev +next +---- +seek-ge "a"/0.000000010,0: output: value k=a ts=10 v=a10 +next: output: meta k=b ts=30 txn=2 +next: output: value k=b ts=30 v=b30 +seek-ge "a"/0.000000025,0: output: value k=a ts=20 v=a20 +next: output: value k=a ts=10 v=a10 +next: output: meta k=b ts=30 txn=2 +seek-lt "a"/0.000000001,0: output: value k=a ts=10 v=a10 +prev: output: value k=a ts=20 v=a20 +prev: output: meta k=a ts=20 txn=1 +prev: output: . +next: output: meta k=a ts=20 txn=1 +next: output: value k=a ts=20 v=a20 +prev: output: meta k=a ts=20 txn=1 +next: output: value k=a ts=20 v=a20 +seek-lt "a"/0.000000015,0: output: value k=a ts=20 v=a20 +prev: output: meta k=a ts=20 txn=1 +prev: output: . +next: output: meta k=a ts=20 txn=1 +seek-lt "a"/0.000000025,0: output: meta k=a ts=20 txn=1 +prev: output: . +next: output: meta k=a ts=20 txn=1 +seek-ge "a"/0.000000005,0: output: meta k=b ts=30 txn=2 +next: output: value k=b ts=30 v=b30 +next: output: meta k=c +prev: output: value k=b ts=30 v=b30 +seek-lt "b"/0.000000040,0: output: meta k=b ts=30 txn=2 +prev: output: value k=a ts=10 v=a10 +prev: output: value k=a ts=20 v=a20 +prev: output: meta k=a ts=20 txn=1 +next: output: value k=a ts=20 v=a20 + +# Seek to particular timestamp and prefix iteration. What we will +# see after the prefix is exhausted is undefined. +iter prefix=true +seek-ge k=a ts=25 +next +next +seek-ge k=a ts=15 +next +seek-ge k=a ts=5 +seek-lt k=a ts=1 +---- +seek-ge "a"/0.000000025,0: output: value k=a ts=20 v=a20 +next: output: value k=a ts=10 v=a10 +next: output: . +seek-ge "a"/0.000000015,0: output: value k=a ts=10 v=a10 +next: output: . +seek-ge "a"/0.000000005,0: output: . +seek-lt "a"/0.000000001,0: output: err: prefix iteration is not permitted with SeekLT + +# Exercise NextKey +iter lower=a upper=f +seek-ge k=a +next-key +prev +next-key +next +seek-ge k=b +next-key +prev +prev +prev +next +next-key +next-key +next-key +---- +seek-ge "a"/0,0: output: meta k=a ts=20 txn=1 +next-key: output: meta k=b ts=30 txn=2 +prev: output: value k=a ts=10 v=a10 +next-key: output: err: NextKey cannot be used to switch iteration direction +next: output: err: NextKey cannot be used to switch iteration direction +seek-ge "b"/0,0: output: meta k=b ts=30 txn=2 +next-key: output: meta k=c +prev: output: value k=b ts=30 v=b30 +prev: output: meta k=b ts=30 txn=2 +prev: output: value k=a ts=10 v=a10 +next: output: meta k=b ts=30 txn=2 +next-key: output: meta k=c +next-key: output: value k=d ts=25 v=d25 +next-key: output: . + + +# Multiple separated intents and multiple interleaved intents. +define +locks +meta k=b ts=20 txn=2 +meta k=d ts=40 txn=4 +meta k=e ts=50 txn=5 +mvcc +meta k=a ts=10 txn=1 +value k=a ts=10 v=a10 +value k=b ts=20 v=b20 +meta k=c ts=30 txn=3 +value k=c ts=30 v=c30 +value k=d ts=40 v=d40 +value k=e ts=50 v=e50 +---- + +# Exercise basic forward and reverse iteration. +iter lower=a upper=f +seek-ge k=a +next +next +prev +prev +prev +next +next +next +next +next +next +next +next +next +next-key +prev +prev +prev +prev +prev +prev +prev +prev +prev +prev +prev +---- +seek-ge "a"/0,0: output: meta k=a ts=10 txn=1 +next: output: value k=a ts=10 v=a10 +next: output: meta k=b ts=20 txn=2 +prev: output: value k=a ts=10 v=a10 +prev: output: meta k=a ts=10 txn=1 +prev: output: . +next: output: meta k=a ts=10 txn=1 +next: output: value k=a ts=10 v=a10 +next: output: meta k=b ts=20 txn=2 +next: output: value k=b ts=20 v=b20 +next: output: meta k=c ts=30 txn=3 +next: output: value k=c ts=30 v=c30 +next: output: meta k=d ts=40 txn=4 +next: output: value k=d ts=40 v=d40 +next: output: meta k=e ts=50 txn=5 +next-key: output: . +prev: output: value k=e ts=50 v=e50 +prev: output: meta k=e ts=50 txn=5 +prev: output: value k=d ts=40 v=d40 +prev: output: meta k=d ts=40 txn=4 +prev: output: value k=c ts=30 v=c30 +prev: output: meta k=c ts=30 txn=3 +prev: output: value k=b ts=20 v=b20 +prev: output: meta k=b ts=20 txn=2 +prev: output: value k=a ts=10 v=a10 +prev: output: meta k=a ts=10 txn=1 +prev: output: . + +# Error case: Multiple separated intents with no provisional values +define +locks +meta k=b ts=20 txn=2 +meta k=d ts=40 txn=4 +---- + +iter lower=a upper=f +seek-ge k=a +next +seek-lt k=e +next +seek-ge k=d +next-key +seek-ge k=d +prev +seek-lt k=e +prev +---- +seek-ge "a"/0,0: output: meta k=b ts=20 txn=2 +next: output: err: intentIter at intent, but iter not at provisional value +seek-lt "e"/0,0: output: meta k=d ts=40 txn=4 +next: output: err: intent has no provisional value +seek-ge "d"/0,0: output: meta k=d ts=40 txn=4 +next-key: output: err: intentIter at intent, but iter not at provisional value +seek-ge "d"/0,0: output: meta k=d ts=40 txn=4 +prev: output: err: iter not at provisional value, cmp: -1 +seek-lt "e"/0,0: output: meta k=d ts=40 txn=4 +prev: output: err: reverse iteration discovered intent without provisional value + +# Local range keys. This exercises local keys having separated locks. +define +locks +meta k=Lb ts=20 txn=2 +meta k=Lc ts=30 txn=4 +mvcc +meta k=La ts=10 txn=1 +value k=La ts=10 v=a10 +value k=Lb ts=20 v=b20 +value k=Lc ts=30 v=c30 +---- + +iter lower=La upper=Ld +seek-ge k=La +next +next +next +next +next +next +prev +prev +prev +prev +prev +prev +prev +---- +seek-ge "La"/0,0: output: meta k=La ts=10 txn=1 +next: output: value k=La ts=10 v=a10 +next: output: meta k=Lb ts=20 txn=2 +next: output: value k=Lb ts=20 v=b20 +next: output: meta k=Lc ts=30 txn=4 +next: output: value k=Lc ts=30 v=c30 +next: output: . +prev: output: value k=Lc ts=30 v=c30 +prev: output: meta k=Lc ts=30 txn=4 +prev: output: value k=Lb ts=20 v=b20 +prev: output: meta k=Lb ts=20 txn=2 +prev: output: value k=La ts=10 v=a10 +prev: output: meta k=La ts=10 txn=1 +prev: output: . + +iter prefix=true +seek-ge k=Lb +next +next +seek-ge k=Lc +next +next +seek-ge k=La ts=25 +next +seek-ge k=Lb ts=25 +next +seek-ge k=Lc ts=25 +---- +seek-ge "Lb"/0,0: output: meta k=Lb ts=20 txn=2 +next: output: value k=Lb ts=20 v=b20 +next: output: . +seek-ge "Lc"/0,0: output: meta k=Lc ts=30 txn=4 +next: output: value k=Lc ts=30 v=c30 +next: output: . +seek-ge "La"/0.000000025,0: output: value k=La ts=10 v=a10 +next: output: . +seek-ge "Lb"/0.000000025,0: output: value k=Lb ts=20 v=b20 +next: output: . +seek-ge "Lc"/0.000000025,0: output: . diff --git a/pkg/storage/testdata/intent_interleaving_iter/error_race b/pkg/storage/testdata/intent_interleaving_iter/error_race new file mode 100644 index 000000000000..d0e6309e05e3 --- /dev/null +++ b/pkg/storage/testdata/intent_interleaving_iter/error_race @@ -0,0 +1,42 @@ +# The test output for error cases can sometimes be different for +# different RaceEnabled settings, because of extra error checking +# when RaceEnabled=true. This file contains the output when +# RaceEnabled=true + +# Error case: Multiple intents for a key +define +locks +meta k=a ts=10 txn=1 +meta k=b ts=20 txn=2 +meta k=b ts=20 txn=4 +meta k=c ts=30 txn=4 +mvcc +value k=a ts=10 v=a10 +value k=b ts=20 v=b20 +value k=c ts=30 v=c30 +---- + +iter lower=a upper=d +seek-ge k=a +next +next +next +next +next +seek-lt k=d +prev +prev +prev +prev +---- +seek-ge "a"/0,0: output: meta k=a ts=10 txn=1 +next: output: value k=a ts=10 v=a10 +next: output: meta k=b ts=20 txn=4 +next: output: err: intentIter incorrectly positioned, cmp: 0 +next: output: err: intentIter incorrectly positioned, cmp: 0 +next: output: err: intentIter incorrectly positioned, cmp: 0 +seek-lt "d"/0,0: output: value k=c ts=30 v=c30 +prev: output: meta k=c ts=30 txn=4 +prev: output: value k=b ts=20 v=b20 +prev: output: meta k=b ts=20 txn=2 +prev: output: err: intentIter should not be after iter diff --git a/pkg/storage/testdata/intent_interleaving_iter/error_race_off b/pkg/storage/testdata/intent_interleaving_iter/error_race_off new file mode 100644 index 000000000000..6c3cd72e06fd --- /dev/null +++ b/pkg/storage/testdata/intent_interleaving_iter/error_race_off @@ -0,0 +1,46 @@ +# The test output for error cases can sometimes be different for +# different RaceEnabled settings, because of extra error checking +# when RaceEnabled=true. This file contains the output when +# RaceEnabled=false + +# Error case: Multiple intents for a key +define +locks +meta k=a ts=10 txn=1 +meta k=b ts=20 txn=2 +meta k=b ts=20 txn=4 +meta k=c ts=30 txn=4 +mvcc +value k=a ts=10 v=a10 +value k=b ts=20 v=b20 +value k=c ts=30 v=c30 +---- + +# We don't catch the error immediately on forward iteration, since unnecessary +# key comparisons are elided (except for RaceEnabled=true). Which is why the +# intent for k=b is returned twice. But continued forward iteration eventually +# catches it. +iter lower=a upper=d +seek-ge k=a +next +next +next +next +next +seek-lt k=d +prev +prev +prev +prev +---- +seek-ge "a"/0,0: output: meta k=a ts=10 txn=1 +next: output: value k=a ts=10 v=a10 +next: output: meta k=b ts=20 txn=4 +next: output: value k=b ts=20 v=b20 +next: output: meta k=b ts=20 txn=2 +next: output: err: intentIter at intent, but iter not at provisional value +seek-lt "d"/0,0: output: value k=c ts=30 v=c30 +prev: output: meta k=c ts=30 txn=4 +prev: output: value k=b ts=20 v=b20 +prev: output: meta k=b ts=20 txn=2 +prev: output: err: intentIter should not be after iter