diff --git a/pkg/storage/intent_interleaving_iter_test.go b/pkg/storage/intent_interleaving_iter_test.go index 25e48072d2bb..2d35534ec925 100644 --- a/pkg/storage/intent_interleaving_iter_test.go +++ b/pkg/storage/intent_interleaving_iter_test.go @@ -212,6 +212,9 @@ func checkAndOutputIter(iter MVCCIterator, b *strings.Builder) { func TestIntentInterleavingIter(t *testing.T) { defer leaktest.AfterTest(t)() + // Disable the metamorphic value for deterministic iteration stats. + DisableMetamorphicLockTableItersBeforeSeek(t) + var eng Engine defer func() { if eng != nil { diff --git a/pkg/storage/lock_table_iterator.go b/pkg/storage/lock_table_iterator.go index e9129c7aee46..eafb42db40df 100644 --- a/pkg/storage/lock_table_iterator.go +++ b/pkg/storage/lock_table_iterator.go @@ -18,6 +18,7 @@ import ( "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/protoutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" @@ -64,6 +65,10 @@ type LockTableIterator struct { // If set, return locks held by any transaction with this strength or // stronger. matchMinStr lock.Strength + // Used to avoid iterating over all shared locks on a key when not necessary, + // given the filtering criteria. See the comment about "skip past locks" above + // for details about why this is important. + itersBeforeSeek lockTableItersBeforeSeekHelper } var _ EngineIterator = &LockTableIterator{} @@ -127,9 +132,10 @@ func NewLockTableIterator( } ltIter := lockTableIteratorPool.Get().(*LockTableIterator) *ltIter = LockTableIterator{ - iter: iter, - matchTxnID: opts.MatchTxnID, - matchMinStr: opts.MatchMinStr, + iter: iter, + matchTxnID: opts.MatchTxnID, + matchMinStr: opts.MatchMinStr, + itersBeforeSeek: ltIter.itersBeforeSeek, } return ltIter, nil } @@ -248,6 +254,7 @@ func (i *LockTableIterator) PrevEngineKeyWithLimit( func (i *LockTableIterator) advanceToMatchingLock( dir int, limit roachpb.Key, ) (state pebble.IterValidityState, err error) { + defer i.itersBeforeSeek.reset() for { engineKey, err := i.iter.UnsafeEngineKey() if err != nil { @@ -261,14 +268,75 @@ func (i *LockTableIterator) advanceToMatchingLock( return pebble.IterValid, nil } - // TODO(nvanbenschoten): implement a maxItersBeforeSeek-like algorithm - // to skip over ignored locks and bound the work performed by the - // iterator for ignored locks. + // We found a non-matching lock. Determine whether to step or seek past it. + // We only ever seek if we found a shared lock, because no other locking + // strength allows for multiple locks to be held by different transactions + // on the same key. + var seek bool + if str == lock.Shared { + seek = i.itersBeforeSeek.shouldSeek(engineKey.Key) + } - if dir < 0 { - state, err = i.iter.PrevEngineKeyWithLimit(limit) + // Advance to the next key, either by stepping or seeking. + if seek { + ltKey, ltKeyErr := engineKey.ToLockTableKey() + if ltKeyErr != nil { + return 0, ltKeyErr + } + seekKeyBuf := &i.itersBeforeSeek.seekKeyBuf + var seekKey EngineKey + if dir < 0 { + // If iterating backwards and searching for locks held by a specific + // transaction, determine whether we have yet to reach key/shared/txnID + // or have already passed it. If we have not yet passed it, seek to the + // specific version, remembering to offset the txn ID by 1 to account + // for the exclusive reverse seek. Otherwise, seek past the maximum + // (first) txn ID to the previous locking strength (exclusive). + // NOTE: Recall that txnIDs in the lock table key version are ordered in + // reverse lexicographical order. + if i.matchTxnID != uuid.Nil && bytes.Compare(txnID.GetBytes(), i.matchTxnID.GetBytes()) < 0 { + // The subtraction cannot underflow because matchTxnID cannot be the + // zero UUID if we are in this branch, with the iterator positioned + // after the matchTxnID. Assert for good measure. + if i.matchTxnID == uuid.Nil { + panic("matchTxnID is unexpectedly the zero UUID") + } + ltKey.TxnUUID = uuid.FromUint128(i.matchTxnID.ToUint128().Sub(1)) + seekKey, *seekKeyBuf = ltKey.ToEngineKey(*seekKeyBuf) + } else { + ltKey.TxnUUID = uuid.Max + seekKey, *seekKeyBuf = ltKey.ToEngineKey(*seekKeyBuf) + } + state, err = i.iter.SeekEngineKeyLTWithLimit(seekKey, limit) + } else { + // If iterating forwards and searching for locks held by a specific + // transaction, determine whether we have yet to reach /key/shared/txnID + // or have already passed it. If we have not yet passed it, seek to the + // specific version. Otherwise, seek to the next key prefix. + // NOTE: Recall that txnIDs in the lock table key version are ordered in + // reverse lexicographical order. + // NOTE: Recall that shared locks are ordered last for a given key. + if i.matchTxnID != uuid.Nil && bytes.Compare(txnID.GetBytes(), i.matchTxnID.GetBytes()) > 0 { + ltKey.TxnUUID = i.matchTxnID + seekKey, *seekKeyBuf = ltKey.ToEngineKey(*seekKeyBuf) + } else { + // TODO(nvanbenschoten): for now, we call SeekEngineKeyGEWithLimit + // with the prefix of the next lock table key. If EngineIterator + // exposed an interface that called NextPrefix(), we could use that + // instead. This will require adding a NextPrefixWithLimit() method + // to pebble. + var seekKeyPrefix roachpb.Key + seekKeyPrefix, *seekKeyBuf = keys.LockTableSingleNextKey(ltKey.Key, *seekKeyBuf) + seekKey = EngineKey{Key: seekKeyPrefix} + } + state, err = i.iter.SeekEngineKeyGEWithLimit(seekKey, limit) + } } else { - state, err = i.iter.NextEngineKeyWithLimit(limit) + if dir < 0 { + state, err = i.iter.PrevEngineKeyWithLimit(limit) + } else { + state, err = i.iter.NextEngineKeyWithLimit(limit) + } } if state != pebble.IterValid || err != nil { return state, err @@ -288,7 +356,9 @@ func (i *LockTableIterator) matchingLock(str lock.Strength, txnID uuid.UUID) boo // Close implements the EngineIterator interface. func (i *LockTableIterator) Close() { i.iter.Close() - *i = LockTableIterator{} + *i = LockTableIterator{ + itersBeforeSeek: i.itersBeforeSeek, + } lockTableIteratorPool.Put(i) } @@ -408,3 +478,90 @@ func checkLockTableKeyOrNil(key roachpb.Key) error { } return checkLockTableKey(key) } + +// defaultLockTableItersBeforeSeek is the default value for the +// lockTableItersBeforeSeek metamorphic value. +const defaultLockTableItersBeforeSeek = 5 + +// lockTableItersBeforeSeek is the number of iterations to perform across the +// shared locks on a single user key before seeking past them. This is used to +// avoid iterating over all shared locks on a key when not necessary, given the +// filtering criteria. +var lockTableItersBeforeSeek = util.ConstantWithMetamorphicTestRange( + "lock-table-iters-before-seek", + defaultLockTableItersBeforeSeek, /* defaultValue */ + 0, /* min */ + 3, /* max */ +) + +// DisableMetamorphicLockTableItersBeforeSeek disables the metamorphic value for +// the duration of a test, resetting it at the end. +func DisableMetamorphicLockTableItersBeforeSeek(t interface { + Helper() + Cleanup(func()) +}) { + t.Helper() + prev := lockTableItersBeforeSeek + lockTableItersBeforeSeek = defaultLockTableItersBeforeSeek + t.Cleanup(func() { + lockTableItersBeforeSeek = prev + }) +} + +// lockTableItersBeforeSeekHelper is a helper struct that keeps track of the +// number of iterations performed across the shared locks on a single user key +// while searching for matching locks in the lock table. It is used to determine +// when to seek past the shared locks to avoid O(ignored_locks) work. +// +// This is similar to the dynamic itersBeforeSeek algorithm that is used by +// pebbleMVCCScanner when scanning over mvcc versions for a key. However, we +// don't adaptively adjust the number of itersBeforeSeek as we go. Instead, we +// reset the iteration counter to lockTableItersBeforeSeek (default: 5) on each +// new key prefix. Doing something more sophisticated introduces complexity and +// it's not clear that this is worth it. +// +// The zero value is ready to use. +type lockTableItersBeforeSeekHelper struct { + curItersBeforeSeek int + curKeyPrefix roachpb.Key + + // Buffers that avoids allocations. + keyPrefixBuf []byte + seekKeyBuf []byte +} + +func (h *lockTableItersBeforeSeekHelper) reset() { + // Clearing the curKeyPrefix ensures that the next call to shouldSeek() will + // save the new key prefix and reset curItersBeforeSeek. This is why the zero + // value of the struct is ready to use. + h.curKeyPrefix = nil +} + +func (h *lockTableItersBeforeSeekHelper) shouldSeek(keyPrefix roachpb.Key) bool { + if h.alwaysSeek() { + return true + } + if !h.curKeyPrefix.Equal(keyPrefix) { + // New key prefix (or curKeyPrefix was nil). Save it and reset the iteration + // count. + h.saveKeyPrefix(keyPrefix) + h.curItersBeforeSeek = lockTableItersBeforeSeek + } else { + // Same key prefix as before. Check if we should seek. + if h.curItersBeforeSeek == 0 { + return true + } + } + h.curItersBeforeSeek-- + return false +} + +func (h *lockTableItersBeforeSeekHelper) alwaysSeek() bool { + // Only returns true in tests when the metamorphic value is set to 0. + return lockTableItersBeforeSeek == 0 +} + +func (h *lockTableItersBeforeSeekHelper) saveKeyPrefix(keyPrefix roachpb.Key) { + h.keyPrefixBuf = append(h.keyPrefixBuf[:0], keyPrefix...) + h.curKeyPrefix = h.keyPrefixBuf +} diff --git a/pkg/storage/lock_table_iterator_test.go b/pkg/storage/lock_table_iterator_test.go index 35f96d2fddae..3b2d01d3dc46 100644 --- a/pkg/storage/lock_table_iterator_test.go +++ b/pkg/storage/lock_table_iterator_test.go @@ -186,6 +186,9 @@ func TestLockTableIterator(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + // Disable the metamorphic value for deterministic iteration stats. + DisableMetamorphicLockTableItersBeforeSeek(t) + var eng Engine defer func() { if eng != nil { @@ -639,3 +642,42 @@ func TestLockTableIteratorEquivalence(t *testing.T) { require.NoError(t, quick.CheckEqual(lockTableIter, preFilterIter, nil)) } + +func TestLockTableItersBeforeSeekHelper(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + // Disable the metamorphic value. + DisableMetamorphicLockTableItersBeforeSeek(t) + + // Check that the value is 5. + require.Equal(t, 5, lockTableItersBeforeSeek) + + var h lockTableItersBeforeSeekHelper + keyA, keyB := roachpb.Key("a"), roachpb.Key("b") + + // Seek to keyA. Should start stepping. + require.False(t, h.shouldSeek(keyA)) + // Step. Same key. Should step again. + require.False(t, h.shouldSeek(keyA)) + // Step. Same key. Should step again. + require.False(t, h.shouldSeek(keyA)) + // Step. Same key. Should step again. + require.False(t, h.shouldSeek(keyA)) + // Step. Same key. Should step again. + require.False(t, h.shouldSeek(keyA)) + // Step. Same key. Should start seeking. + require.True(t, h.shouldSeek(keyA)) + // Seek. Same key. Should keep seeking if not new key prefix. + require.True(t, h.shouldSeek(keyA)) + // Seek. New key. Should start stepping again. + require.False(t, h.shouldSeek(keyB)) + + // Test that the key is copied and not referenced. + for i := 0; i < lockTableItersBeforeSeek; i++ { + keyUnstable := roachpb.Key("unstable") + require.False(t, h.shouldSeek(keyUnstable)) + keyUnstable[0] = 'a' + } + require.True(t, h.shouldSeek(roachpb.Key("unstable"))) +} diff --git a/pkg/storage/pebble_mvcc_scanner.go b/pkg/storage/pebble_mvcc_scanner.go index 3524c35698ce..feb5f3250dc8 100644 --- a/pkg/storage/pebble_mvcc_scanner.go +++ b/pkg/storage/pebble_mvcc_scanner.go @@ -461,7 +461,7 @@ type pebbleMVCCScanner struct { // Stores any error returned. If non-nil, iteration short circuits. err error // Number of iterations to try before we do a Seek/SeekReverse. Stays within - // [0, maxItersBeforeSeek] and defaults to maxItersBeforeSeek/2 . + // [0, maxItersBeforeSeek] and defaults to maxItersBeforeSeek/2. itersBeforeSeek int // machine is the state machine for how the iterator should be advanced in // order to handle scans and reverse scans. @@ -711,7 +711,7 @@ func (p *pebbleMVCCScanner) afterScan() (*roachpb.Span, kvpb.ResumeReason, int64 return nil, 0, 0, nil } -// Increments itersBeforeSeek while ensuring it stays <= maxItersBeforeSeek +// Increments itersBeforeSeek while ensuring it stays <= maxItersBeforeSeek. func (p *pebbleMVCCScanner) incrementItersBeforeSeek() { p.itersBeforeSeek++ if p.itersBeforeSeek > maxItersBeforeSeek { diff --git a/pkg/storage/testdata/intent_interleaving_iter/basic b/pkg/storage/testdata/intent_interleaving_iter/basic index ae55433f7dce..62d74bf10db4 100644 --- a/pkg/storage/testdata/intent_interleaving_iter/basic +++ b/pkg/storage/testdata/intent_interleaving_iter/basic @@ -72,7 +72,7 @@ next: output: value k=b ts=30.000000000,0 v=b30 next: output: meta k=c next: output: value k=d ts=25.000000000,0 v=d25 next: output: . -stats: (interface (dir, seek, step): (fwd, 2, 19), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 2, 15), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 3, 18), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 3, 14), (rev, 0, 0)) prev: output: value k=d ts=25.000000000,0 v=d25 prev: output: meta k=c prev: output: value k=b ts=30.000000000,0 v=b30 @@ -81,16 +81,16 @@ prev: output: value k=a ts=10.000000000,0 v=a10 prev: output: value k=a ts=20.000000000,0 v=a20 prev: output: meta k=a ts=20.000000000,0 txn=1 prev: output: . -stats: (interface (dir, seek, step): (fwd, 2, 19), (rev, 0, 19)), (internal (dir, seek, step): (fwd, 2, 15), (rev, 2, 15)) +stats: (interface (dir, seek, step): (fwd, 3, 18), (rev, 1, 18)), (internal (dir, seek, step): (fwd, 3, 14), (rev, 3, 15)) seek-ge "b"/0,0: output: meta k=b ts=30.000000000,0 txn=2 -stats: (interface (dir, seek, step): (fwd, 4, 19), (rev, 0, 19)), (internal (dir, seek, step): (fwd, 4, 15), (rev, 2, 15)) +stats: (interface (dir, seek, step): (fwd, 5, 18), (rev, 1, 18)), (internal (dir, seek, step): (fwd, 5, 14), (rev, 3, 15)) next: output: value k=b ts=30.000000000,0 v=b30 next: output: meta k=c prev: output: value k=b ts=30.000000000,0 v=b30 prev: output: meta k=b ts=30.000000000,0 txn=2 prev: output: value k=a ts=10.000000000,0 v=a10 seek-lt "b"/0,0: output: value k=a ts=10.000000000,0 v=a10 -stats: (interface (dir, seek, step): (fwd, 4, 23), (rev, 2, 27)), (internal (dir, seek, step): (fwd, 4, 18), (rev, 4, 26)) +stats: (interface (dir, seek, step): (fwd, 5, 22), (rev, 3, 26)), (internal (dir, seek, step): (fwd, 5, 17), (rev, 5, 26)) next: output: meta k=b ts=30.000000000,0 txn=2 prev: output: value k=a ts=10.000000000,0 v=a10 prev: output: value k=a ts=20.000000000,0 v=a20 diff --git a/pkg/storage/testdata/lock_table_iterator/iters_before_seek b/pkg/storage/testdata/lock_table_iterator/iters_before_seek index a6d8fb80ed42..584d618ea1c1 100644 --- a/pkg/storage/testdata/lock_table_iterator/iters_before_seek +++ b/pkg/storage/testdata/lock_table_iterator/iters_before_seek @@ -5,8 +5,6 @@ # of positions in the shared lock key ordering. Then do so for different minimum # lock strength levels. -# TODO(nvanbenschoten): implement the lockTableItersBeforeSeek optimization. - define lock k=a str=intent txn=1 lock k=a str=exclusive txn=1 @@ -69,7 +67,7 @@ next: output: k=b str=shared txn=1 next: output: k=c str=intent txn=1 next: output: k=c str=exclusive txn=1 next: output: . (exhausted) -stats: (interface (dir, seek, step): (fwd, 1, 36), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 36), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 3, 18), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 3, 18), (rev, 0, 0)) # txn1, reverse iteration iter lower=a match-txn-id=1 @@ -93,7 +91,7 @@ prev: output: k=a str=shared txn=1 prev: output: k=a str=exclusive txn=1 prev: output: k=a str=intent txn=1 prev: output: . (exhausted) -stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 1, 36)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 1, 36)) +stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 3, 18)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 3, 20)) # txn1, prefix iteration iter prefix=true match-txn-id=1 @@ -107,7 +105,7 @@ seek-ge k=b: output: k=b str=intent txn=1 next: output: k=b str=exclusive txn=1 next: output: k=b str=shared txn=1 next: output: . (exhausted) -stats: (interface (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 2, 8), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 2, 8), (rev, 0, 0)) # txn1, forward iteration with limit iter upper=z match-txn-id=1 @@ -121,7 +119,7 @@ seek-ge-with-limit k=a limit=b: output: k=a str=intent txn=1 next-with-limit limit=b: output: k=a str=exclusive txn=1 next-with-limit limit=b: output: k=a str=shared txn=1 next-with-limit limit=b: output: . (at limit) -stats: (interface (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 2, 8), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 2, 8), (rev, 0, 0)) # txn1, reverse iteration with limit iter lower=a match-txn-id=1 @@ -139,7 +137,7 @@ prev-with-limit limit=b: output: k=b str=shared txn=1 prev-with-limit limit=b: output: k=b str=exclusive txn=1 prev-with-limit limit=b: output: k=b str=intent txn=1 prev-with-limit limit=b: output: . (at limit) -stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 1, 19)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 1, 19)) +stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 2, 10)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 2, 11)) # txn3, forward iteration @@ -152,7 +150,7 @@ stats seek-ge k=a: output: k=a str=shared txn=3 next: output: k=b str=shared txn=3 next: output: . (exhausted) -stats: (interface (dir, seek, step): (fwd, 1, 36), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 36), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 3, 22), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 3, 22), (rev, 0, 0)) # txn3, reverse iteration iter lower=a match-txn-id=3 @@ -164,7 +162,7 @@ stats seek-lt k=d: output: k=b str=shared txn=3 prev: output: k=a str=shared txn=3 prev: output: . (exhausted) -stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 1, 36)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 1, 36)) +stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 3, 22)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 3, 24)) # txn3, prefix iteration iter prefix=true match-txn-id=3 @@ -174,7 +172,7 @@ stats ---- seek-ge k=b: output: k=b str=shared txn=3 next: output: . (exhausted) -stats: (interface (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 2, 10), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 2, 10), (rev, 0, 0)) # txn3, forward iteration with limit iter upper=z match-txn-id=3 @@ -184,7 +182,7 @@ stats ---- seek-ge-with-limit k=a limit=b: output: k=a str=shared txn=3 next-with-limit limit=b: output: . (at limit) -stats: (interface (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 2, 10), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 2, 10), (rev, 0, 0)) # txn3, reverse iteration with limit iter lower=a match-txn-id=3 @@ -194,7 +192,7 @@ stats ---- seek-lt-with-limit k=d limit=b: output: k=b str=shared txn=3 prev-with-limit limit=b: output: . (at limit) -stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 1, 19)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 1, 19)) +stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 2, 12)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 2, 13)) # txn8, forward iteration @@ -207,7 +205,7 @@ stats seek-ge k=a: output: k=a str=shared txn=8 next: output: k=b str=shared txn=8 next: output: . (exhausted) -stats: (interface (dir, seek, step): (fwd, 1, 36), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 36), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 5, 28), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 5, 28), (rev, 0, 0)) # txn8, reverse iteration iter lower=a match-txn-id=8 @@ -219,7 +217,7 @@ stats seek-lt k=d: output: k=b str=shared txn=8 prev: output: k=a str=shared txn=8 prev: output: . (exhausted) -stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 1, 36)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 1, 36)) +stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 5, 28)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 5, 32)) # txn8, prefix iteration iter prefix=true match-txn-id=8 @@ -229,7 +227,7 @@ stats ---- seek-ge k=b: output: k=b str=shared txn=8 next: output: . (exhausted) -stats: (interface (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 3, 13), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 3, 13), (rev, 0, 0)) # txn8, forward iteration with limit iter upper=z match-txn-id=8 @@ -239,7 +237,7 @@ stats ---- seek-ge-with-limit k=a limit=b: output: k=a str=shared txn=8 next-with-limit limit=b: output: . (at limit) -stats: (interface (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 3, 13), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 3, 13), (rev, 0, 0)) # txn8, reverse iteration with limit iter lower=a match-txn-id=8 @@ -249,7 +247,7 @@ stats ---- seek-lt-with-limit k=d limit=b: output: k=b str=shared txn=8 prev-with-limit limit=b: output: . (at limit) -stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 1, 19)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 1, 19)) +stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 3, 15)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 3, 17)) # txn13, forward iteration @@ -262,7 +260,7 @@ stats seek-ge k=a: output: k=a str=shared txn=13 next: output: k=b str=shared txn=13 next: output: . (exhausted) -stats: (interface (dir, seek, step): (fwd, 1, 36), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 36), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 3, 22), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 3, 22), (rev, 0, 0)) # txn13, reverse iteration iter lower=a match-txn-id=13 @@ -274,7 +272,7 @@ stats seek-lt k=d: output: k=b str=shared txn=13 prev: output: k=a str=shared txn=13 prev: output: . (exhausted) -stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 1, 36)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 1, 36)) +stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 3, 22)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 3, 24)) # txn13, prefix iteration iter prefix=true match-txn-id=13 @@ -284,7 +282,7 @@ stats ---- seek-ge k=b: output: k=b str=shared txn=13 next: output: . (exhausted) -stats: (interface (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 2, 10), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 2, 10), (rev, 0, 0)) # txn13, forward iteration with limit iter upper=z match-txn-id=13 @@ -294,7 +292,7 @@ stats ---- seek-ge-with-limit k=a limit=b: output: k=a str=shared txn=13 next-with-limit limit=b: output: . (at limit) -stats: (interface (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 2, 10), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 2, 10), (rev, 0, 0)) # txn13, reverse iteration with limit iter lower=a match-txn-id=13 @@ -304,7 +302,7 @@ stats ---- seek-lt-with-limit k=d limit=b: output: k=b str=shared txn=13 prev-with-limit limit=b: output: . (at limit) -stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 1, 19)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 1, 19)) +stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 2, 12)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 2, 13)) # txn15, forward iteration @@ -317,7 +315,7 @@ stats seek-ge k=a: output: k=a str=shared txn=15 next: output: k=b str=shared txn=15 next: output: . (exhausted) -stats: (interface (dir, seek, step): (fwd, 1, 36), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 36), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 3, 18), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 3, 18), (rev, 0, 0)) # txn15, reverse iteration iter lower=a match-txn-id=15 @@ -329,7 +327,7 @@ stats seek-lt k=d: output: k=b str=shared txn=15 prev: output: k=a str=shared txn=15 prev: output: . (exhausted) -stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 1, 36)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 1, 36)) +stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 3, 18)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 3, 20)) # txn15, prefix iteration iter prefix=true match-txn-id=15 @@ -339,7 +337,7 @@ stats ---- seek-ge k=b: output: k=b str=shared txn=15 next: output: . (exhausted) -stats: (interface (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 2, 8), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 2, 8), (rev, 0, 0)) # txn15, forward iteration with limit iter upper=z match-txn-id=15 @@ -349,7 +347,7 @@ stats ---- seek-ge-with-limit k=a limit=b: output: k=a str=shared txn=15 next-with-limit limit=b: output: . (at limit) -stats: (interface (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 2, 8), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 2, 8), (rev, 0, 0)) # txn15, reverse iteration with limit iter lower=a match-txn-id=15 @@ -359,7 +357,7 @@ stats ---- seek-lt-with-limit k=d limit=b: output: k=b str=shared txn=15 prev-with-limit limit=b: output: . (at limit) -stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 1, 19)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 1, 19)) +stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 2, 10)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 2, 11)) # shared min strength, forward iteration @@ -671,7 +669,7 @@ next: output: k=b str=exclusive txn=1 next: output: k=c str=intent txn=1 next: output: k=c str=exclusive txn=1 next: output: . (exhausted) -stats: (interface (dir, seek, step): (fwd, 1, 36), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 36), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 3, 16), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 3, 16), (rev, 0, 0)) # exclusive min strength, reverse iteration iter lower=a match-min-str=exclusive @@ -691,7 +689,7 @@ prev: output: k=b str=intent txn=1 prev: output: k=a str=exclusive txn=1 prev: output: k=a str=intent txn=1 prev: output: . (exhausted) -stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 1, 36)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 1, 36)) +stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 3, 16)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 3, 18)) # exclusive min strength, prefix iteration iter prefix=true match-min-str=exclusive @@ -703,7 +701,7 @@ stats seek-ge k=b: output: k=b str=intent txn=1 next: output: k=b str=exclusive txn=1 next: output: . (exhausted) -stats: (interface (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 2, 7), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 2, 7), (rev, 0, 0)) # exclusive min strength, forward iteration with limit iter upper=z match-min-str=exclusive @@ -715,7 +713,7 @@ stats seek-ge-with-limit k=a limit=b: output: k=a str=intent txn=1 next-with-limit limit=b: output: k=a str=exclusive txn=1 next-with-limit limit=b: output: . (at limit) -stats: (interface (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 2, 7), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 2, 7), (rev, 0, 0)) # exclusive min strength, reverse iteration with limit iter lower=a match-min-str=exclusive @@ -731,7 +729,7 @@ prev-with-limit limit=b: output: k=c str=intent txn=1 prev-with-limit limit=b: output: k=b str=exclusive txn=1 prev-with-limit limit=b: output: k=b str=intent txn=1 prev-with-limit limit=b: output: . (at limit) -stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 1, 19)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 1, 19)) +stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 2, 9)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 2, 10)) # intent min strength, forward iteration @@ -746,7 +744,7 @@ seek-ge k=a: output: k=a str=intent txn=1 next: output: k=b str=intent txn=1 next: output: k=c str=intent txn=1 next: output: . (exhausted) -stats: (interface (dir, seek, step): (fwd, 1, 36), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 36), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 3, 16), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 3, 16), (rev, 0, 0)) # intent min strength, reverse iteration iter lower=a match-min-str=intent @@ -760,7 +758,7 @@ seek-lt k=d: output: k=c str=intent txn=1 prev: output: k=b str=intent txn=1 prev: output: k=a str=intent txn=1 prev: output: . (exhausted) -stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 1, 36)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 1, 36)) +stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 3, 16)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 3, 18)) # intent min strength, prefix iteration iter prefix=true match-min-str=intent @@ -770,7 +768,7 @@ stats ---- seek-ge k=b: output: k=b str=intent txn=1 next: output: . (exhausted) -stats: (interface (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 2, 7), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 2, 7), (rev, 0, 0)) # intent min strength, forward iteration with limit iter upper=z match-min-str=intent @@ -780,7 +778,7 @@ stats ---- seek-ge-with-limit k=a limit=b: output: k=a str=intent txn=1 next-with-limit limit=b: output: . (at limit) -stats: (interface (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 17), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 2, 7), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 2, 7), (rev, 0, 0)) # intent min strength, reverse iteration with limit iter lower=a match-min-str=intent @@ -792,4 +790,4 @@ stats seek-lt-with-limit k=d limit=b: output: k=c str=intent txn=1 prev-with-limit limit=b: output: k=b str=intent txn=1 prev-with-limit limit=b: output: . (at limit) -stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 1, 19)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 1, 19)) +stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 2, 9)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 2, 10)) diff --git a/pkg/util/uuid/uuid.go b/pkg/util/uuid/uuid.go index fd0951511ac4..0a8d18fa375e 100644 --- a/pkg/util/uuid/uuid.go +++ b/pkg/util/uuid/uuid.go @@ -24,6 +24,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/cockroach/pkg/util/uint128" "github.com/cockroachdb/errors" ) @@ -97,10 +98,13 @@ func TimestampFromV1(u UUID) (Timestamp, error) { // String parse helpers. var urnPrefix = []byte("urn:uuid:") -// Nil is the nil UUID, as specified in RFC-4122, that has all 128 bits set to +// Nil is the nil UUID, as specified in RFC-4122, which has all 128 bits set to // zero. var Nil = UUID{} +// Max is the maximum possible UUID, which has all 128 bits set to 1. +var Max = FromUint128(uint128.FromInts(math.MaxUint64, math.MaxUint64)) + // Predefined namespace UUIDs. var ( NamespaceDNS = Must(FromString("6ba7b810-9dad-11d1-80b4-00c04fd430c8"))