diff --git a/pkg/cmd/roachtest/tests/mvcc_gc.go b/pkg/cmd/roachtest/tests/mvcc_gc.go index 87a88212b353..2c83a961e481 100644 --- a/pkg/cmd/roachtest/tests/mvcc_gc.go +++ b/pkg/cmd/roachtest/tests/mvcc_gc.go @@ -280,7 +280,8 @@ func checkRangesConsistentAndHaveNoData(totals enginepb.MVCCStats, details range return errors.Errorf("table ranges contain garbage %s", totals.String()) } if totals.LiveBytes > 0 || totals.LiveCount > 0 || - totals.IntentBytes > 0 || totals.IntentCount > 0 || totals.LockCount > 0 { + totals.IntentBytes > 0 || totals.IntentCount > 0 || + totals.LockBytes > 0 || totals.LockCount > 0 { return errors.Errorf("table ranges contain live data %s", totals.String()) } if details.status != kvpb.CheckConsistencyResponse_RANGE_CONSISTENT.String() { diff --git a/pkg/kv/kvserver/kvserverpb/state.proto b/pkg/kv/kvserver/kvserverpb/state.proto index 22376374af08..9a18676be4a7 100644 --- a/pkg/kv/kvserver/kvserverpb/state.proto +++ b/pkg/kv/kvserver/kvserverpb/state.proto @@ -245,6 +245,7 @@ message RangeAppliedState { // state simply because we have introduced this field. uint64 raft_applied_index_term = 5 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/kv/kvpb.RaftTerm"]; } + // MVCCPersistentStats is convertible to MVCCStats, but uses signed variable // width encodings for most fields that make it efficient to store positive // values but inefficient to store negative values. This makes the encodings @@ -265,6 +266,7 @@ message MVCCPersistentStats { int64 val_count = 9; int64 intent_bytes = 10; int64 intent_count = 11; + int64 lock_bytes = 21; int64 lock_count = 16; int64 range_key_count = 17; int64 range_key_bytes = 18; diff --git a/pkg/kv/kvserver/rditer/stats.go b/pkg/kv/kvserver/rditer/stats.go index c32e66d02a70..b4533c998b56 100644 --- a/pkg/kv/kvserver/rditer/stats.go +++ b/pkg/kv/kvserver/rditer/stats.go @@ -21,7 +21,7 @@ import ( func ComputeStatsForRange( d *roachpb.RangeDescriptor, reader storage.Reader, nowNanos int64, ) (enginepb.MVCCStats, error) { - return ComputeStatsForRangeWithVisitors(d, reader, nowNanos, nil, nil) + return ComputeStatsForRangeWithVisitors(d, reader, nowNanos, storage.ComputeStatsVisitors{}) } // ComputeStatsForRangeWithVisitors is like ComputeStatsForRange but also @@ -30,13 +30,11 @@ func ComputeStatsForRangeWithVisitors( d *roachpb.RangeDescriptor, reader storage.Reader, nowNanos int64, - pointKeyVisitor func(storage.MVCCKey, []byte) error, - rangeKeyVisitor func(storage.MVCCRangeKeyValue) error, + visitors storage.ComputeStatsVisitors, ) (enginepb.MVCCStats, error) { var ms enginepb.MVCCStats - for _, keySpan := range makeReplicatedKeySpansExceptLockTable(d) { - msDelta, err := storage.ComputeStatsWithVisitors(reader, keySpan.Key, keySpan.EndKey, nowNanos, - pointKeyVisitor, rangeKeyVisitor) + for _, keySpan := range MakeReplicatedKeySpans(d) { + msDelta, err := storage.ComputeStatsWithVisitors(reader, keySpan.Key, keySpan.EndKey, nowNanos, visitors) if err != nil { return enginepb.MVCCStats{}, err } diff --git a/pkg/kv/kvserver/replica_consistency.go b/pkg/kv/kvserver/replica_consistency.go index f9c9e296d15e..063d32ab5a41 100644 --- a/pkg/kv/kvserver/replica_consistency.go +++ b/pkg/kv/kvserver/replica_consistency.go @@ -516,7 +516,9 @@ func CalcReplicaDigest( return limiter.WaitN(ctx, tokens) } - pointKeyVisitor := func(unsafeKey storage.MVCCKey, unsafeValue []byte) error { + var visitors storage.ComputeStatsVisitors + + visitors.PointKey = func(unsafeKey storage.MVCCKey, unsafeValue []byte) error { // Rate limit the scan through the range. if err := wait(int64(len(unsafeKey.Key) + len(unsafeValue))); err != nil { return err @@ -549,7 +551,7 @@ func CalcReplicaDigest( return err } - rangeKeyVisitor := func(rangeKV storage.MVCCRangeKeyValue) error { + visitors.RangeKey = func(rangeKV storage.MVCCRangeKeyValue) error { // Rate limit the scan through the range. err := wait( int64(len(rangeKV.RangeKey.StartKey) + len(rangeKV.RangeKey.EndKey) + len(rangeKV.Value))) @@ -591,12 +593,17 @@ func CalcReplicaDigest( return err } + visitors.LockTableKey = func(unsafeKey storage.LockTableKey, unsafeValue []byte) error { + // TODO(nvanbenschoten): rate limit scan through lock table and add to + // checksum to be included in consistency checks. + return nil + } + // In statsOnly mode, we hash only the RangeAppliedState. In regular mode, hash // all of the replicated key space. var result ReplicaDigest if !statsOnly { - ms, err := rditer.ComputeStatsForRangeWithVisitors(&desc, snap, 0, /* nowNanos */ - pointKeyVisitor, rangeKeyVisitor) + ms, err := rditer.ComputeStatsForRangeWithVisitors(&desc, snap, 0 /* nowNanos */, visitors) // Consume the remaining quota borrowed in the visitors. Do it even on // iteration error, but prioritize returning the latter if it occurs. if wErr := limiter.WaitN(ctx, batchSize); wErr != nil && err == nil { diff --git a/pkg/kv/kvserver/testdata/TestBelowRaftProtosDontChange/RangeAppliedState b/pkg/kv/kvserver/testdata/TestBelowRaftProtosDontChange/RangeAppliedState index 8d3e44f5616a..064632aeebce 100644 --- a/pkg/kv/kvserver/testdata/TestBelowRaftProtosDontChange/RangeAppliedState +++ b/pkg/kv/kvserver/testdata/TestBelowRaftProtosDontChange/RangeAppliedState @@ -1,3 +1,3 @@ echo ---- -14062697193383087404 +2796048770313977431 diff --git a/pkg/sql/logictest/testdata/logic_test/builtin_function b/pkg/sql/logictest/testdata/logic_test/builtin_function index 3ace53ee2252..a07d75d2876b 100644 --- a/pkg/sql/logictest/testdata/logic_test/builtin_function +++ b/pkg/sql/logictest/testdata/logic_test/builtin_function @@ -4005,7 +4005,8 @@ FROM crdb_internal.check_consistency(true, crdb_internal.tenant_span()[1], crdb_ ORDER BY range_id LIMIT 1 ---- -RANGE_CONSISTENT stats: {ContainsEstimates: LastUpdateNanos: LockAge: GCBytesAge: LiveBytes: LiveCount: KeyBytes: KeyCount: ValBytes: ValCount: IntentBytes: IntentCount: LockCount: RangeKeyCount: RangeKeyBytes: RangeValCount: RangeValBytes: SysBytes: SysCount: AbortSpanBytes:} +RANGE_CONSISTENT stats: {ContainsEstimates: LastUpdateNanos: LockAge: GCBytesAge: LiveBytes: LiveCount: KeyBytes: KeyCount: ValBytes: ValCount: IntentBytes: IntentCount: LockBytes: LockCount: RangeKeyCount: RangeKeyBytes: RangeValCount: RangeValBytes: SysBytes: SysCount: AbortSpanBytes:} + # Fill a table with consistency check results. This used to panic. diff --git a/pkg/storage/engine_key.go b/pkg/storage/engine_key.go index b0b05583cb25..7cfe689cd2f2 100644 --- a/pkg/storage/engine_key.go +++ b/pkg/storage/engine_key.go @@ -351,6 +351,11 @@ func (lk LockTableKey) ToEngineKey(buf []byte) (EngineKey, []byte) { return k, buf } +// EncodedSize returns the size of the LockTableKey when encoded. +func (lk LockTableKey) EncodedSize() int64 { + return int64(len(lk.Key)) + engineKeyVersionLockTableLen +} + // EngineRangeKeyValue is a raw value for a general range key as stored in the // engine. It consists of a version (suffix) and corresponding value. The range // key bounds are not included, but are surfaced via EngineRangeBounds(). diff --git a/pkg/storage/enginepb/mvcc.go b/pkg/storage/enginepb/mvcc.go index 0cc50120e19d..dc5a06f7a809 100644 --- a/pkg/storage/enginepb/mvcc.go +++ b/pkg/storage/enginepb/mvcc.go @@ -159,6 +159,7 @@ func (ms *MVCCStats) Add(oms MVCCStats) { ms.KeyCount += oms.KeyCount ms.ValCount += oms.ValCount ms.IntentCount += oms.IntentCount + ms.LockBytes += oms.LockBytes ms.LockCount += oms.LockCount ms.RangeKeyCount += oms.RangeKeyCount ms.RangeKeyBytes += oms.RangeKeyBytes @@ -190,6 +191,7 @@ func (ms *MVCCStats) Subtract(oms MVCCStats) { ms.KeyCount -= oms.KeyCount ms.ValCount -= oms.ValCount ms.IntentCount -= oms.IntentCount + ms.LockBytes -= oms.LockBytes ms.LockCount -= oms.LockCount ms.RangeKeyCount -= oms.RangeKeyCount ms.RangeKeyBytes -= oms.RangeKeyBytes diff --git a/pkg/storage/enginepb/mvcc.proto b/pkg/storage/enginepb/mvcc.proto index 19095a2f801d..2508d15ee25c 100644 --- a/pkg/storage/enginepb/mvcc.proto +++ b/pkg/storage/enginepb/mvcc.proto @@ -202,6 +202,19 @@ message MVCCStats { // It is equal to the number of meta keys in the system with // a non-empty Transaction proto. optional sfixed64 intent_count = 11 [(gogoproto.nullable) = false]; + // lock_bytes is the encoded size of replicated locks with shared or + // exclusive strengths, which are stored in the lock table keyspace. + // The field includes the size of the locks' keys and their values. + // + // For historical reasons, the field excludes the size of intent + // metadata key-values, even though they are also stored in the lock + // table keyspace. Intent metadata keys are tracked under key_bytes + // and their values are tracked under val_bytes. This is not to be + // confused with the provisional versioned values protected by the + // intents, which are tracked by the intent_bytes field (and also by + // key_bytes and val_bytes). Hence the vague "without their meta keys" + // comment above. + optional sfixed64 lock_bytes = 21 [(gogoproto.nullable) = false]; // lock_count is the number of replicated locks (shared, exclusive, or // intent strength) that are in the lock table. It is >= intent_count. optional sfixed64 lock_count = 16 [(gogoproto.nullable) = false]; diff --git a/pkg/storage/enginepb/mvcc3.proto b/pkg/storage/enginepb/mvcc3.proto index eb9e4da27924..cf5cb09f1332 100644 --- a/pkg/storage/enginepb/mvcc3.proto +++ b/pkg/storage/enginepb/mvcc3.proto @@ -233,6 +233,7 @@ message MVCCStatsDelta { sint64 val_count = 9; sint64 intent_bytes = 10; sint64 intent_count = 11; + sint64 lock_bytes = 21; sint64 lock_count = 16; sint64 range_key_count = 17; sint64 range_key_bytes = 18; diff --git a/pkg/storage/lock_table_iterator.go b/pkg/storage/lock_table_iterator.go index 8253c65403a4..e9129c7aee46 100644 --- a/pkg/storage/lock_table_iterator.go +++ b/pkg/storage/lock_table_iterator.go @@ -327,6 +327,16 @@ func (i *LockTableIterator) UnsafeRawEngineKey() []byte { return i.iter.UnsafeRawEngineKey() } +// UnsafeLockTableKey returns the current key as an unsafe LockTableKey. +// TODO(nvanbenschoten): use this more widely. +func (i *LockTableIterator) UnsafeLockTableKey() (LockTableKey, error) { + k, err := i.iter.UnsafeEngineKey() + if err != nil { + return LockTableKey{}, errors.Wrap(err, "retrieving lock table key") + } + return k.ToLockTableKey() +} + // LockTableKeyVersion returns the strength and txn ID from the version of the // current key. func (i *LockTableIterator) LockTableKeyVersion() (lock.Strength, uuid.UUID, error) { diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index f9ba855e4f74..e4576fe6c30f 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -559,6 +559,48 @@ func updateStatsOnResolve( return ms } +// updateStatsOnAcquireLock updates MVCCStats for acquiring a replicated shared +// or exclusive lock on a key. If orig is not nil, the lock acquisition is +// replacing an existing lock with a new lock that has the exact same txn ID and +// strength. +func updateStatsOnAcquireLock( + origKeySize, origValSize, keySize, valSize int64, orig, meta *enginepb.MVCCMetadata, +) enginepb.MVCCStats { + var ms enginepb.MVCCStats + + // Remove current lock counts. + if orig != nil { + // Move the (so far empty) stats to the timestamp at which the previous + // lock was acquired, which is where we wish to reclassify its initial + // contributions. + ms.AgeTo(orig.Timestamp.WallTime) + + // Subtract counts attributable to the lock we're replacing. + ms.LockBytes -= origKeySize + origValSize + ms.LockCount-- + } + + // Now add in the contributions from the new lock at the new acquisition + // timestamp. + ms.AgeTo(meta.Timestamp.WallTime) + ms.LockBytes += keySize + valSize + ms.LockCount++ + return ms +} + +// updateStatsOnReleaseLock updates MVCCStats for releasing a replicated shared +// or exclusive lock on a key. orig is the lock being released, and must not be +// nil. +func updateStatsOnReleaseLock( + origKeySize, origValSize int64, orig *enginepb.MVCCMetadata, +) enginepb.MVCCStats { + var ms enginepb.MVCCStats + ms.AgeTo(orig.Timestamp.WallTime) + ms.LockBytes -= origKeySize + origValSize + ms.LockCount-- + return ms +} + // updateStatsOnRangeKeyClear updates MVCCStats for clearing an entire // range key stack. func updateStatsOnRangeKeyClear(rangeKeys MVCCRangeKeyStack) enginepb.MVCCStats { @@ -1439,14 +1481,17 @@ func (b *putBuffer) release() { putBufferPool.Put(b) } -func (b *putBuffer) lockTableKey(key roachpb.Key, str lock.Strength, txnID uuid.UUID) EngineKey { - var lockTableKey EngineKey - lockTableKey, b.ltKeyBuf = LockTableKey{ +func (b *putBuffer) lockTableKey( + key roachpb.Key, str lock.Strength, txnID uuid.UUID, +) (ltEngKey EngineKey, keyBytes int64) { + ltKey := LockTableKey{ Key: key, Strength: str, TxnUUID: txnID, - }.ToEngineKey(b.ltKeyBuf) - return lockTableKey + } + ltEngKey, b.ltKeyBuf = ltKey.ToEngineKey(b.ltKeyBuf) + keyBytes = ltKey.EncodedSize() + return ltEngKey, keyBytes } func (b *putBuffer) marshalMeta(meta *enginepb.MVCCMetadata) (_ []byte, err error) { @@ -1483,7 +1528,11 @@ var trueValue = true // putLockMeta puts a lock at the given key with the provided strength and // value. func (b *putBuffer) putLockMeta( - writer Writer, key MVCCKey, str lock.Strength, meta *enginepb.MVCCMetadata, alreadyExists bool, + writer Writer, + key roachpb.Key, + str lock.Strength, + meta *enginepb.MVCCMetadata, + alreadyExists bool, ) (keyBytes, valBytes int64, err error) { if meta.Timestamp.ToTimestamp() != meta.Txn.WriteTimestamp { // The timestamps are supposed to be in sync. If they weren't, it wouldn't @@ -1491,7 +1540,7 @@ func (b *putBuffer) putLockMeta( return 0, 0, errors.AssertionFailedf( "meta.Timestamp != meta.Txn.WriteTimestamp: %s != %s", meta.Timestamp, meta.Txn.WriteTimestamp) } - lockTableKey := b.lockTableKey(key.Key, str, meta.Txn.ID) + lockTableKey, lockTableKeyBytes := b.lockTableKey(key, str, meta.Txn.ID) if alreadyExists { // Absence represents false. meta.TxnDidNotUpdateMeta = nil @@ -1505,7 +1554,17 @@ func (b *putBuffer) putLockMeta( if err = writer.PutEngineKey(lockTableKey, bytes); err != nil { return 0, 0, err } - return int64(key.EncodedSize()), int64(len(bytes)), nil + if str == lock.Intent { + // For historical reasons, intent metadata key-values use the encoded + // size of the unversioned MVCCKey that they are virtualized at (e.g. by + // the intentInterleavingIter) as their contribution to stats, instead + // of their real size in the lock table keyspace. + keyBytes = int64(MakeMVCCMetadataKey(key).EncodedSize()) + } else { + keyBytes = lockTableKeyBytes + } + valBytes = int64(len(bytes)) + return keyBytes, valBytes, nil } // clearLockMeta clears a lock at the given key and strength. @@ -1521,19 +1580,26 @@ func (b *putBuffer) putLockMeta( // doing single-clear. func (b *putBuffer) clearLockMeta( writer Writer, - key MVCCKey, + key roachpb.Key, str lock.Strength, txnDidNotUpdateMeta bool, txnUUID uuid.UUID, opts ClearOptions, ) (keyBytes, valBytes int64, err error) { - lockTableKey := b.lockTableKey(key.Key, str, txnUUID) + lockTableKey, lockTableKeyBytes := b.lockTableKey(key, str, txnUUID) if txnDidNotUpdateMeta { err = writer.SingleClearEngineKey(lockTableKey) } else { err = writer.ClearEngineKey(lockTableKey, opts) } - return int64(key.EncodedSize()), 0, err + if str == lock.Intent { + // See comment in putLockMeta. + keyBytes = int64(MakeMVCCMetadataKey(key).EncodedSize()) + } else { + keyBytes = lockTableKeyBytes + } + valBytes = 0 // cleared + return keyBytes, valBytes, err } // MVCCPut sets the value for a specified key. It will save the value @@ -2316,7 +2382,7 @@ func mvccPutInternal( alreadyExists := ok && meta.Txn != nil // Write the intent metadata key. metaKeySize, metaValSize, err = buf.putLockMeta( - writer, metaKey, lock.Intent, newMeta, alreadyExists) + writer, metaKey.Key, lock.Intent, newMeta, alreadyExists) if err != nil { return false, err } @@ -4958,10 +5024,10 @@ func mvccResolveWriteIntent( // to do anything to update the intent but to move the timestamp forward, // even if it can. metaKeySize, metaValSize, err = buf.putLockMeta( - writer, metaKey, lock.Intent, newMeta, true /* alreadyExists */) + writer, metaKey.Key, lock.Intent, newMeta, true /* alreadyExists */) } else { metaKeySize, metaValSize, err = buf.clearLockMeta( - writer, metaKey, lock.Intent, canSingleDelHelper.onCommitLock(), meta.Txn.ID, ClearOptions{ + writer, metaKey.Key, lock.Intent, canSingleDelHelper.onCommitLock(), meta.Txn.ID, ClearOptions{ ValueSizeKnown: true, ValueSize: uint32(origMetaValSize), }) @@ -5073,7 +5139,7 @@ func mvccResolveWriteIntent( if !ok { // If there is no other version, we should just clean up the key entirely. _, _, err := buf.clearLockMeta( - writer, metaKey, lock.Intent, canSingleDelHelper.onAbortLock(), meta.Txn.ID, ClearOptions{ + writer, metaKey.Key, lock.Intent, canSingleDelHelper.onAbortLock(), meta.Txn.ID, ClearOptions{ ValueSizeKnown: true, ValueSize: uint32(origMetaValSize), }) @@ -5095,7 +5161,7 @@ func mvccResolveWriteIntent( ValBytes: int64(nextValueLen), } metaKeySize, metaValSize, err := buf.clearLockMeta( - writer, metaKey, lock.Intent, canSingleDelHelper.onAbortLock(), meta.Txn.ID, ClearOptions{ + writer, metaKey.Key, lock.Intent, canSingleDelHelper.onAbortLock(), meta.Txn.ID, ClearOptions{ ValueSizeKnown: true, ValueSize: uint32(origMetaValSize), }) @@ -5445,17 +5511,24 @@ func MVCCAcquireLock( buf := newPutBuffer() defer buf.release() - metaKey := MakeMVCCMetadataKey(key) newMeta := &buf.newMeta newMeta.Txn = &txn.TxnMeta newMeta.Timestamp = txn.WriteTimestamp.ToLegacyTimestamp() - keyBytes, valBytes, err := buf.putLockMeta(rw, metaKey, str, newMeta, rolledBack) + keyBytes, valBytes, err := buf.putLockMeta(rw, key, str, newMeta, rolledBack) if err != nil { return err } - // TODO(nvanbenschoten): handle MVCCStats update after addressing #109645. - _, _, _ = ms, keyBytes, valBytes + // Update MVCC stats. + if ms != nil { + origMeta := ltScanner.foundOwn(str) + var origKeySize, origValSize int64 + if origMeta != nil { + origKeySize = keyBytes // same key + origValSize = int64(origMeta.Size()) + } + ms.Add(updateStatsOnAcquireLock(origKeySize, origValSize, keyBytes, valBytes, origMeta, newMeta)) + } return nil } @@ -5503,10 +5576,7 @@ func mvccReleaseLockInternal( txnDidNotUpdateMeta = canSingleDelHelper.onAbortLock() } - metaKey := MakeMVCCMetadataKey(update.Key) - origMetaKeySize := int64(metaKey.EncodedSize()) - origMetaValSize := int64(meta.Size()) - keyBytes, valBytes, err := buf.clearLockMeta(writer, metaKey, str, txnDidNotUpdateMeta, meta.Txn.ID, ClearOptions{ + keyBytes, _, err := buf.clearLockMeta(writer, update.Key, str, txnDidNotUpdateMeta, meta.Txn.ID, ClearOptions{ ValueSizeKnown: true, ValueSize: uint32(meta.Size()), }) @@ -5514,8 +5584,12 @@ func mvccReleaseLockInternal( return false, err } - // TODO(nvanbenschoten): handle MVCCStats update after addressing #109645. - _, _, _, _, _ = ms, origMetaKeySize, origMetaValSize, keyBytes, valBytes + // Update MVCC stats. + if ms != nil { + origKeySize := keyBytes // same key + origValSize := int64(meta.Size()) + ms.Add(updateStatsOnReleaseLock(origKeySize, origValSize, meta)) + } return true, nil @@ -6430,18 +6504,26 @@ func willOverflow(a, b int64) bool { // specifies the wall time in nanoseconds since the epoch and is used to compute // age-related stats quantities. func ComputeStats(r Reader, start, end roachpb.Key, nowNanos int64) (enginepb.MVCCStats, error) { - return ComputeStatsWithVisitors(r, start, end, nowNanos, nil, nil) + return ComputeStatsWithVisitors(r, start, end, nowNanos, ComputeStatsVisitors{}) +} + +// ComputeStatsVisitors holds a set of callbacks that are invoked on each key +// during stats computation. +type ComputeStatsVisitors struct { + PointKey func(MVCCKey, []byte) error + RangeKey func(MVCCRangeKeyValue) error + LockTableKey func(LockTableKey, []byte) error } -// ComputeStatsWithVisitors is like ComputeStats, but also takes a point and/or -// range key callback that is invoked for each key. +// ComputeStatsWithVisitors is like ComputeStats, but also takes callbacks that +// are invoked on each key. func ComputeStatsWithVisitors( - r Reader, - start, end roachpb.Key, - nowNanos int64, - pointKeyVisitor func(MVCCKey, []byte) error, - rangeKeyVisitor func(MVCCRangeKeyValue) error, + r Reader, start, end roachpb.Key, nowNanos int64, visitors ComputeStatsVisitors, ) (enginepb.MVCCStats, error) { + if isLockTableKey(start) { + return computeLockTableStatsWithVisitors(r, start, end, nowNanos, visitors.LockTableKey) + } + iter, err := r.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{ KeyTypes: IterKeyTypePointsAndRanges, LowerBound: start, @@ -6452,7 +6534,7 @@ func ComputeStatsWithVisitors( } defer iter.Close() iter.SeekGE(MVCCKey{Key: start}) - return computeStatsForIterWithVisitors(iter, nowNanos, pointKeyVisitor, rangeKeyVisitor) + return computeStatsForIterWithVisitors(iter, nowNanos, visitors.PointKey, visitors.RangeKey) } // ComputeStatsForIter is like ComputeStats, but scans across the given iterator @@ -6563,24 +6645,34 @@ func computeStatsForIterWithVisitors( } } - // Check for ignored keys. - if bytes.HasPrefix(unsafeKey.Key, keys.LocalRangeIDPrefix) { - // RangeID-local key. - _ /* rangeID */, infix, suffix, _ /* detail */, err := keys.DecodeRangeIDKey(unsafeKey.Key) - if err != nil { - return enginepb.MVCCStats{}, errors.Wrap(err, "unable to decode rangeID key") - } + isSys := isSysLocal(unsafeKey.Key) + if isSys { + // Check for ignored keys. + if bytes.HasPrefix(unsafeKey.Key, keys.LocalRangeIDPrefix) { + // RangeID-local key. + _ /* rangeID */, infix, suffix, _ /* detail */, err := keys.DecodeRangeIDKey(unsafeKey.Key) + if err != nil { + return enginepb.MVCCStats{}, errors.Wrap(err, "unable to decode rangeID key") + } - if infix.Equal(keys.LocalRangeIDReplicatedInfix) { - // Replicated RangeID-local key. - if suffix.Equal(keys.LocalRangeAppliedStateSuffix) { - // RangeAppliedState key. Ignore. - continue + if infix.Equal(keys.LocalRangeIDReplicatedInfix) { + // Replicated RangeID-local key. + if suffix.Equal(keys.LocalRangeAppliedStateSuffix) { + // RangeAppliedState key. Ignore. + continue + } } } + + // Check for lock table keys, which are not handled by this + // function. They are handled by computeLockTableStatsWithVisitors + // instead. + if bytes.HasPrefix(unsafeKey.Key, keys.LocalRangeLockTablePrefix) { + return enginepb.MVCCStats{}, errors.AssertionFailedf( + "lock table key encountered by ComputeStats: %s", unsafeKey.Key) + } } - isSys := isSysLocal(unsafeKey.Key) isValue := unsafeKey.IsValue() implicitMeta := isValue && !bytes.Equal(unsafeKey.Key, prevKey) prevKey = append(prevKey[:0], unsafeKey.Key...) @@ -6731,6 +6823,78 @@ func computeStatsForIterWithVisitors( return ms, nil } +// computeLockTableStatsWithVisitors performs stats computation for the lock +// table keys in the given span. It is split off from the main ComputeStats +// logic because lock table iteration requires an EngineIterator (which is +// wrapped in a LockTableIterator), while the main ComputeStats logic uses an +// (intent interleaving) MVCCIterator. +// +// Unlike computeStatsForIterWithVisitors, this function accepts a Reader and +// a start and end key. The start and end key must both be lock table keys. +func computeLockTableStatsWithVisitors( + r Reader, + start, end roachpb.Key, + nowNanos int64, + lockTableKeyVisitor func(LockTableKey, []byte) error, +) (enginepb.MVCCStats, error) { + iter, err := NewLockTableIterator(r, LockTableIteratorOptions{ + LowerBound: start, + UpperBound: end, + MatchMinStr: lock.Shared, // all locks + }) + if err != nil { + return enginepb.MVCCStats{}, err + } + defer iter.Close() + + var ms enginepb.MVCCStats + var meta enginepb.MVCCMetadata + var ok bool + for ok, err = iter.SeekEngineKeyGE(EngineKey{Key: start}); ok; ok, err = iter.NextEngineKey() { + key, err := iter.UnsafeLockTableKey() + if err != nil { + return enginepb.MVCCStats{}, err + } + if key.Strength == lock.Intent { + // The contributions of intents to the MVCCStats are handled by + // computeStatsForIterWithVisitors, which uses an intent + // interleaving iterator to interpret the mvcc keyspace. That + // function draws a distinction between provisional versioned values + // that are associated with intents and committed versioned values + // that are not. + // + // For simplicity, we ignore intents in this function. + continue + } + val, err := iter.UnsafeValue() + if err != nil { + return enginepb.MVCCStats{}, err + } + if err := protoutil.Unmarshal(val, &meta); err != nil { + return ms, errors.Wrap(err, "unable to decode MVCCMetadata") + } + + if lockTableKeyVisitor != nil { + if err := lockTableKeyVisitor(key, val); err != nil { + return enginepb.MVCCStats{}, err + } + } + + keyBytes := key.EncodedSize() + valBytes := int64(len(val)) + + ms.LockBytes += keyBytes + valBytes + ms.LockCount++ + ms.LockAge += nowNanos/1e9 - meta.Timestamp.WallTime/1e9 + } + if err != nil { + return enginepb.MVCCStats{}, err + } + + ms.LastUpdateNanos = nowNanos + return ms, nil +} + // MVCCIsSpanEmptyOptions configures the MVCCIsSpanEmpty function. type MVCCIsSpanEmptyOptions struct { // StartKey determines start of the checked span. diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index b98435d536db..841909194d19 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -173,6 +173,13 @@ func TestMVCCHistories(t *testing.T) { {Key: keys.MinKey, EndKey: roachpb.LocalMax}, {Key: keys.LocalMax, EndKey: roachpb.KeyMax}, } + // lockTableSpan returns the span of the lock table that corresponds to the + // given span. + lockTableSpan := func(s roachpb.Span) roachpb.Span { + k, _ := keys.LockTableSingleKey(s.Key, nil) + ek, _ := keys.LockTableSingleKey(s.EndKey, nil) + return roachpb.Span{Key: k, EndKey: ek} + } // Timestamp for MVCC stats calculations, in nanoseconds. const statsTS = 100e9 @@ -605,11 +612,15 @@ func TestMVCCHistories(t *testing.T) { } cmd := e.getCmd() - txnChange = txnChange || cmd.typ&typTxnUpdate != 0 - dataChange = dataChange || cmd.typ&typDataUpdate != 0 - locksChange = locksChange || cmd.typ&typLocksUpdate != 0 - - if trace || (stats && cmd.typ&typDataUpdate != 0) { + txnChangeForCmd := cmd.typ&typTxnUpdate != 0 + dataChangeForCmd := cmd.typ&typDataUpdate != 0 + locksChangeForCmd := cmd.typ&typLocksUpdate != 0 + txnChange = txnChange || txnChangeForCmd + dataChange = dataChange || dataChangeForCmd + locksChange = locksChange || locksChangeForCmd + statsForCmd := stats && (dataChangeForCmd || locksChangeForCmd) + + if trace || statsForCmd { // If tracing is also requested by the datadriven input, // we'll trace the statement in the actual results too. buf.Printf(">> %s", d.Cmd) @@ -627,6 +638,11 @@ func TestMVCCHistories(t *testing.T) { ms, err := storage.ComputeStats(e.engine, span.Key, span.EndKey, statsTS) require.NoError(t, err) msEngineBefore.Add(ms) + + lockSpan := lockTableSpan(span) + lockMs, err := storage.ComputeStats(e.engine, lockSpan.Key, lockSpan.EndKey, statsTS) + require.NoError(t, err) + msEngineBefore.Add(lockMs) } } msEvalBefore := *e.ms @@ -642,10 +658,10 @@ func TestMVCCHistories(t *testing.T) { // If tracing is enabled, we report the intermediate results // after each individual step in the script. // This may modify foundErr too. - reportResults(cmd.typ&typTxnUpdate != 0, cmd.typ&typDataUpdate != 0, cmd.typ&typLocksUpdate != 0) + reportResults(txnChangeForCmd, dataChangeForCmd, dataChangeForCmd) } - if stats && cmd.typ&typDataUpdate != 0 { + if statsForCmd { // If stats are enabled, emit evaluated stats returned by the // command, and compare them with the real computed stats diff. var msEngineDiff enginepb.MVCCStats @@ -653,6 +669,11 @@ func TestMVCCHistories(t *testing.T) { ms, err := storage.ComputeStats(e.engine, span.Key, span.EndKey, statsTS) require.NoError(t, err) msEngineDiff.Add(ms) + + lockSpan := lockTableSpan(span) + lockMs, err := storage.ComputeStats(e.engine, lockSpan.Key, lockSpan.EndKey, statsTS) + require.NoError(t, err) + msEngineDiff.Add(lockMs) } msEngineDiff.Subtract(msEngineBefore) @@ -695,12 +716,17 @@ func TestMVCCHistories(t *testing.T) { } // Calculate and output final stats if requested and the data changed. - if stats && dataChange { + if stats && (dataChange || locksChange) { var msFinal enginepb.MVCCStats for _, span := range spans { ms, err := storage.ComputeStats(e.engine, span.Key, span.EndKey, statsTS) require.NoError(t, err) msFinal.Add(ms) + + lockSpan := lockTableSpan(span) + lockMs, err := storage.ComputeStats(e.engine, lockSpan.Key, lockSpan.EndKey, statsTS) + require.NoError(t, err) + msFinal.Add(lockMs) } buf.Printf("stats: %s\n", formatStats(msFinal, false)) } @@ -2237,7 +2263,7 @@ func formatStats(ms enginepb.MVCCStats, delta bool) string { order := []string{"key_count", "key_bytes", "val_count", "val_bytes", "range_key_count", "range_key_bytes", "range_val_count", "range_val_bytes", "live_count", "live_bytes", "gc_bytes_age", - "intent_count", "intent_bytes", "lock_count", "lock_age"} + "intent_count", "intent_bytes", "lock_count", "lock_bytes", "lock_age"} sort.SliceStable(fields, func(i, j int) bool { for _, name := range order { if fields[i][1] == name { diff --git a/pkg/storage/mvcc_key.go b/pkg/storage/mvcc_key.go index fa78a8e173fd..77d72e2e015e 100644 --- a/pkg/storage/mvcc_key.go +++ b/pkg/storage/mvcc_key.go @@ -128,6 +128,8 @@ func (k MVCCKey) IsValue() bool { // // TODO(itsbilal): Reconcile this with Len(). Would require updating MVCC stats // tests to reflect the more accurate lengths provided by Len(). +// TODO(nvanbenschoten): Change the return value to an int64. That's what every +// caller wants. func (k MVCCKey) EncodedSize() int { n := len(k.Key) + 1 if k.IsValue() { diff --git a/pkg/storage/mvcc_stats_test.go b/pkg/storage/mvcc_stats_test.go index a88cdb29cce2..817b0766ac26 100644 --- a/pkg/storage/mvcc_stats_test.go +++ b/pkg/storage/mvcc_stats_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" + "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/testutils" @@ -59,12 +60,21 @@ func assertEqImpl( keyMin = keys.LocalMax keyMax = roachpb.KeyMax } + lockKeyMin, _ := keys.LockTableSingleKey(keyMin, nil) + lockKeyMax, _ := keys.LockTableSingleKey(keyMax, nil) for _, mvccStatsTest := range mvccStatsTests { compMS, err := mvccStatsTest.fn(rw, keyMin, keyMax, ms.LastUpdateNanos) if err != nil { t.Fatal(err) } + // NOTE: we use ComputeStats for the lock table stats because it is not + // supported by ComputeStatsForIter. + compLockMS, err := ComputeStats(rw, lockKeyMin, lockKeyMax, ms.LastUpdateNanos) + if err != nil { + t.Fatal(err) + } + compMS.Add(compLockMS) require.Equal(t, compMS, *ms, "%s: diff(ms, %s)", debug, mvccStatsTest.name) } } @@ -1758,8 +1768,18 @@ func TestMVCCStatsRandomized(t *testing.T) { endTime := hlc.MaxTimestamp clearRangeThreshold := int(s.rng.Int63n(5)) + // TODO(nvanbenschoten): this should be pushed into MVCCClearTimeRange, which + // does not currently handle replicated locks correctly. + locks, err := ScanLocks(ctx, s.batch, keySpan.Key, keySpan.EndKey, 1, 0) + if err == nil && len(locks) > 0 { + err = &kvpb.LockConflictError{Locks: locks} + } + if err != nil { + return false, err.Error() + } + desc := fmt.Sprintf("mvccClearTimeRange=%s, startTime=%s, endTime=%s", keySpan, startTime, endTime) - _, err := MVCCClearTimeRange(ctx, s.batch, s.MSDelta, keySpan.Key, keySpan.EndKey, + _, err = MVCCClearTimeRange(ctx, s.batch, s.MSDelta, keySpan.Key, keySpan.EndKey, startTime, endTime, nil /* leftPeekBound */, nil /* rightPeekBound */, clearRangeThreshold, 0, 0) if err != nil { desc += " " + err.Error() @@ -1768,6 +1788,16 @@ func TestMVCCStatsRandomized(t *testing.T) { return true, desc } + actions["AcquireLock"] = func(s *state) (bool, string) { + str := lock.Shared + if s.rng.Intn(2) != 0 { + str = lock.Exclusive + } + if err := MVCCAcquireLock(ctx, s.batch, s.Txn, str, s.key, s.MSDelta, 0); err != nil { + return false, err.Error() + } + return true, "" + } actions["EnsureTxn"] = func(s *state) (bool, string) { if s.Txn == nil { txn := roachpb.MakeTransaction("test", nil, 0, 0, s.TS, 0, 1, 0) diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go index af83661fd29b..5ed97075353e 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -125,6 +125,7 @@ func TestMVCCStatsAddSubForward(t *testing.T) { RangeKeyBytes: 1, RangeValCount: 1, RangeValBytes: 1, + LockBytes: 1, LockCount: 1, LockAge: 1, GCBytesAge: 1, diff --git a/pkg/storage/sst.go b/pkg/storage/sst.go index a24324ba223f..a42bdca40eee 100644 --- a/pkg/storage/sst.go +++ b/pkg/storage/sst.go @@ -1255,6 +1255,8 @@ func UpdateSSTTimestamps( // Calculate this delta by subtracting all the relevant stats at the // old timestamp, and then aging the stats to the new timestamp before // zeroing the stats again. + // TODO(nvanbenschoten): should this just be using MVCCStats.Add and + // MVCCStats.Subtract? statsDelta.AgeTo(from.WallTime) statsDelta.KeyBytes -= stats.KeyBytes statsDelta.ValBytes -= stats.ValBytes @@ -1263,6 +1265,7 @@ func UpdateSSTTimestamps( statsDelta.LiveBytes -= stats.LiveBytes statsDelta.IntentBytes -= stats.IntentBytes statsDelta.IntentCount -= stats.IntentCount + statsDelta.LockBytes -= stats.LockBytes statsDelta.LockCount -= stats.LockCount statsDelta.AgeTo(to.WallTime) statsDelta.KeyBytes += stats.KeyBytes @@ -1272,6 +1275,7 @@ func UpdateSSTTimestamps( statsDelta.LiveBytes += stats.LiveBytes statsDelta.IntentBytes += stats.IntentBytes statsDelta.IntentCount += stats.IntentCount + statsDelta.LockBytes += stats.LockBytes statsDelta.LockCount += stats.LockCount } diff --git a/pkg/storage/testdata/mvcc_histories/replicated_locks b/pkg/storage/testdata/mvcc_histories/replicated_locks index 246666a1949f..c978e2bd9856 100644 --- a/pkg/storage/testdata/mvcc_histories/replicated_locks +++ b/pkg/storage/testdata/mvcc_histories/replicated_locks @@ -1,21 +1,28 @@ -run ok +run stats ok put k=k1 v=v1 ts=5,0 put k=k2 v=v2 ts=5,0 put k=k3 v=v3 ts=5,0 ---- +>> put k=k1 v=v1 ts=5,0 +stats: key_count=+1 key_bytes=+15 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+22 +>> put k=k2 v=v2 ts=5,0 +stats: key_count=+1 key_bytes=+15 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+22 +>> put k=k3 v=v3 ts=5,0 +stats: key_count=+1 key_bytes=+15 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+22 >> at end: data: "k1"/5.000000000,0 -> /BYTES/v1 data: "k2"/5.000000000,0 -> /BYTES/v2 data: "k3"/5.000000000,0 -> /BYTES/v3 +stats: key_count=3 key_bytes=45 val_count=3 val_bytes=21 live_count=3 live_bytes=66 -run ok +run stats ok txn_begin t=A ts=10,0 txn_begin t=B ts=11,0 ---- >> at end: txn: "B" meta={id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=11.000000000,0 wto=false gul=0,0 -run ok +run stats ok with t=A check_for_acquire_lock k=k1 str=shared check_for_acquire_lock k=k2 str=shared @@ -24,32 +31,48 @@ with t=A acquire_lock k=k2 str=shared acquire_lock k=k3 str=exclusive ---- +>> acquire_lock k=k1 str=shared t=A +stats: lock_count=+1 lock_bytes=+67 lock_age=+90 +>> acquire_lock k=k2 str=shared t=A +stats: lock_count=+1 lock_bytes=+67 lock_age=+90 +>> acquire_lock k=k3 str=exclusive t=A +stats: lock_count=+1 lock_bytes=+67 lock_age=+90 >> at end: lock (Replicated): "k1"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true lock (Replicated): "k2"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true lock (Replicated): "k3"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true +stats: key_count=3 key_bytes=45 val_count=3 val_bytes=21 live_count=3 live_bytes=66 lock_count=3 lock_bytes=201 lock_age=270 # Reacquire with weaker, equal, and stronger strengths. All should succeed, but # only the stronger strength should actually write a new lock key. -run ok +run stats ok with t=A acquire_lock k=k2 str=shared acquire_lock k=k2 str=exclusive acquire_lock k=k3 str=shared acquire_lock k=k3 str=exclusive ---- +>> acquire_lock k=k2 str=shared t=A +stats: no change +>> acquire_lock k=k2 str=exclusive t=A +stats: lock_count=+1 lock_bytes=+67 lock_age=+90 +>> acquire_lock k=k3 str=shared t=A +stats: no change +>> acquire_lock k=k3 str=exclusive t=A +stats: no change >> at end: lock (Replicated): "k1"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true lock (Replicated): "k2"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true lock (Replicated): "k2"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true lock (Replicated): "k3"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true +stats: key_count=3 key_bytes=45 val_count=3 val_bytes=21 live_count=3 live_bytes=66 lock_count=4 lock_bytes=268 lock_age=360 # Reacquire with weaker, equal, and stronger strengths in new epoch. All should # succeed, but only the stronger strength acquisitions (in the new epoch) should # actually (re)write lock keys. -run ok +run stats ok with t=A txn_restart acquire_lock k=k1 str=shared @@ -58,17 +81,28 @@ with t=A acquire_lock k=k3 str=exclusive acquire_lock k=k3 str=shared ---- +>> acquire_lock k=k1 str=shared t=A +stats: no change +>> acquire_lock k=k2 str=shared t=A +stats: no change +>> acquire_lock k=k2 str=exclusive t=A +stats: no change +>> acquire_lock k=k3 str=exclusive t=A +stats: no change +>> acquire_lock k=k3 str=shared t=A +stats: no change >> at end: txn: "A" meta={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=0,0 lock (Replicated): "k1"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=0} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k2"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=0} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k2"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=0} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k3"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=0} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false +stats: key_count=3 key_bytes=45 val_count=3 val_bytes=21 live_count=3 live_bytes=66 lock_count=4 lock_bytes=268 lock_age=360 # Reacquisition of the same locks in the same epoch with later sequences should # be no-ops. -run ok +run stats ok with t=A txn_step acquire_lock k=k1 str=shared @@ -77,18 +111,29 @@ with t=A acquire_lock k=k3 str=exclusive acquire_lock k=k3 str=shared ---- +>> acquire_lock k=k1 str=shared t=A +stats: no change +>> acquire_lock k=k2 str=shared t=A +stats: no change +>> acquire_lock k=k2 str=exclusive t=A +stats: no change +>> acquire_lock k=k3 str=exclusive t=A +stats: no change +>> acquire_lock k=k3 str=shared t=A +stats: no change >> at end: txn: "A" meta={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=0,0 lock (Replicated): "k1"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=0} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k2"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=0} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k2"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=0} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k3"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=0} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false +stats: key_count=3 key_bytes=45 val_count=3 val_bytes=21 live_count=3 live_bytes=66 lock_count=4 lock_bytes=268 lock_age=360 # Reacquisition of the same locks in the same epoch with later sequences after # the earlier sequence has been rolled back should rewrite the locks with the # newer sequence. -run ok +run stats ok with t=A txn_ignore_seqs seqs=0-0 acquire_lock k=k1 str=shared @@ -97,26 +142,40 @@ with t=A acquire_lock k=k3 str=exclusive acquire_lock k=k3 str=shared ---- +>> acquire_lock k=k1 str=shared t=A +stats: lock_bytes=+2 +>> acquire_lock k=k2 str=shared t=A +stats: lock_bytes=+2 +>> acquire_lock k=k2 str=exclusive t=A +stats: lock_bytes=+2 +>> acquire_lock k=k3 str=exclusive t=A +stats: lock_bytes=+2 +>> acquire_lock k=k3 str=shared t=A +stats: no change >> at end: txn: "A" meta={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=0,0 isn=1 lock (Replicated): "k1"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k2"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k2"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k3"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false +stats: key_count=3 key_bytes=45 val_count=3 val_bytes=21 live_count=3 live_bytes=66 lock_count=4 lock_bytes=276 lock_age=360 # Txn B can only acquire a shared lock on k1. -run ok +run stats ok with t=B check_for_acquire_lock k=k1 str=shared acquire_lock k=k1 str=shared ---- +>> acquire_lock k=k1 str=shared t=B +stats: lock_count=+1 lock_bytes=+67 lock_age=+89 >> at end: lock (Replicated): "k1"/Shared -> txn={id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} ts=11.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true lock (Replicated): "k1"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k2"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k2"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k3"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false +stats: key_count=3 key_bytes=45 val_count=3 val_bytes=21 live_count=3 live_bytes=66 lock_count=5 lock_bytes=343 lock_age=449 run error check_for_acquire_lock t=B k=k1 str=exclusive @@ -218,29 +277,37 @@ error: (*kvpb.LockConflictError:) conflicting locks on "k1" # Intents are treated similarly to Exclusive locks. -run ok +run stats ok put t=A k=k4 v=v4 ---- +>> put t=A k=k4 v=v4 +stats: key_count=+1 key_bytes=+15 val_count=+1 val_bytes=+59 live_count=+1 live_bytes=+74 intent_count=+1 intent_bytes=+19 lock_count=+1 lock_age=+90 >> at end: data: "k1"/5.000000000,0 -> /BYTES/v1 data: "k2"/5.000000000,0 -> /BYTES/v2 data: "k3"/5.000000000,0 -> /BYTES/v3 meta: "k4"/0,0 -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k4"/10.000000000,0 -> /BYTES/v4 +stats: key_count=4 key_bytes=60 val_count=4 val_bytes=80 live_count=4 live_bytes=140 intent_count=1 intent_bytes=19 lock_count=6 lock_bytes=343 lock_age=539 -run ok +run stats ok with t=A check_for_acquire_lock k=k4 str=shared check_for_acquire_lock k=k4 str=exclusive acquire_lock k=k4 str=shared acquire_lock k=k4 str=exclusive ---- +>> acquire_lock k=k4 str=shared t=A +stats: no change +>> acquire_lock k=k4 str=exclusive t=A +stats: no change >> at end: lock (Replicated): "k1"/Shared -> txn={id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} ts=11.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true lock (Replicated): "k1"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k2"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k2"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k3"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false +stats: key_count=4 key_bytes=60 val_count=4 val_bytes=80 live_count=4 live_bytes=140 intent_count=1 intent_bytes=19 lock_count=6 lock_bytes=343 lock_age=539 run error check_for_acquire_lock t=B k=k4 str=shared @@ -261,11 +328,13 @@ error: (*kvpb.LockConflictError:) conflicting locks on "k4" # The intent history is considered when determining whether a reacquisition is # needed on the same key as a previous intent write. -run ok +run stats ok with t=A txn_step put k=k4 v=v4_prime ---- +>> put k=k4 v=v4_prime t=A +stats: val_bytes=+17 live_bytes=+17 intent_bytes=+6 >> at end: txn: "A" meta={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=2} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=0,0 isn=1 data: "k1"/5.000000000,0 -> /BYTES/v1 @@ -273,12 +342,15 @@ data: "k2"/5.000000000,0 -> /BYTES/v2 data: "k3"/5.000000000,0 -> /BYTES/v3 meta: "k4"/0,0 -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=2} ts=10.000000000,0 del=false klen=12 vlen=13 ih={{1 /BYTES/v4}} mergeTs= txnDidNotUpdateMeta=false data: "k4"/10.000000000,0 -> /BYTES/v4_prime +stats: key_count=4 key_bytes=60 val_count=4 val_bytes=97 live_count=4 live_bytes=157 intent_count=1 intent_bytes=25 lock_count=6 lock_bytes=343 lock_age=539 -run ok +run stats ok with t=A txn_step acquire_lock k=k4 str=shared ---- +>> acquire_lock k=k4 str=shared t=A +stats: no change >> at end: txn: "A" meta={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=3} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=0,0 isn=1 lock (Replicated): "k1"/Shared -> txn={id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} ts=11.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true @@ -286,6 +358,7 @@ lock (Replicated): "k1"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri lock (Replicated): "k2"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k2"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k3"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false +stats: key_count=4 key_bytes=60 val_count=4 val_bytes=97 live_count=4 live_bytes=157 intent_count=1 intent_bytes=25 lock_count=6 lock_bytes=343 lock_age=539 run with t=A @@ -318,7 +391,7 @@ lock (Replicated): "k4"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri # that we terminate scans at key "k4" to ignore the intent that we just wrote, # which is not ignored by non-locking scans. -run ok +run stats ok with k=k1 end=k4 scan t=A scan t=B @@ -472,11 +545,15 @@ error: (*kvpb.LockConflictError:) conflicting locks on "k3" # Writes to keys locked by only the writer should succeed, regardless of the # strength of the writer's lock. -run ok +run stats ok with t=A put k=k2 v=v2 put k=k3 v=v3 ---- +>> put k=k2 v=v2 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+59 live_bytes=+52 gc_bytes_age=+1710 intent_count=+1 intent_bytes=+19 lock_count=+1 lock_age=+90 +>> put k=k3 v=v3 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+59 live_bytes=+52 gc_bytes_age=+1710 intent_count=+1 intent_bytes=+19 lock_count=+1 lock_age=+90 >> at end: data: "k1"/5.000000000,0 -> /BYTES/v1 meta: "k2"/0,0 -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=3} ts=10.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -487,15 +564,20 @@ data: "k3"/10.000000000,0 -> /BYTES/v3 data: "k3"/5.000000000,0 -> /BYTES/v3 meta: "k4"/0,0 -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=2} ts=10.000000000,0 del=false klen=12 vlen=13 ih={{1 /BYTES/v4}} mergeTs= txnDidNotUpdateMeta=false data: "k4"/10.000000000,0 -> /BYTES/v4_prime +stats: key_count=4 key_bytes=84 val_count=6 val_bytes=215 live_count=4 live_bytes=261 gc_bytes_age=3420 intent_count=3 intent_bytes=63 lock_count=9 lock_bytes=414 lock_age=809 -run ok +run stats ok with t=A txn_step del k=k2 del k=k3 ---- +>> del k=k2 t=A del: "k2": found key true +stats: val_bytes=+4 live_count=-1 live_bytes=-74 gc_bytes_age=+7020 intent_bytes=-7 +>> del k=k3 t=A del: "k3": found key true +stats: val_bytes=+4 live_count=-1 live_bytes=-74 gc_bytes_age=+7020 intent_bytes=-7 >> at end: txn: "A" meta={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=4} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=0,0 isn=1 data: "k1"/5.000000000,0 -> /BYTES/v1 @@ -507,21 +589,24 @@ data: "k3"/10.000000000,0 -> / data: "k3"/5.000000000,0 -> /BYTES/v3 meta: "k4"/0,0 -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=2} ts=10.000000000,0 del=false klen=12 vlen=13 ih={{1 /BYTES/v4}} mergeTs= txnDidNotUpdateMeta=false data: "k4"/10.000000000,0 -> /BYTES/v4_prime +stats: key_count=4 key_bytes=84 val_count=6 val_bytes=223 live_count=2 live_bytes=113 gc_bytes_age=17460 intent_count=3 intent_bytes=49 lock_count=9 lock_bytes=414 lock_age=809 # Release locks. # Reset ignored sequence numbers for now. -run ok +run stats ok txn_ignore_seqs t=A seqs ---- >> at end: txn: "A" meta={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=4} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=0,0 # Pending resolution without epoch bump or savepoint rollback is a no-op. -run ok +run stats ok resolve_intent t=A k=k1 status=PENDING ---- +>> resolve_intent t=A k=k1 status=PENDING resolve_intent: "k1" -> resolved key = false +stats: no change >> at end: data: "k1"/5.000000000,0 -> /BYTES/v1 meta: "k2"/0,0 -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=4} ts=10.000000000,0 del=true klen=12 vlen=0 ih={{3 /BYTES/v2}} mergeTs= txnDidNotUpdateMeta=false @@ -538,12 +623,15 @@ lock (Replicated): "k2"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable lock (Replicated): "k2"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k3"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k4"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=3} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true +stats: key_count=4 key_bytes=84 val_count=6 val_bytes=223 live_count=2 live_bytes=113 gc_bytes_age=17460 intent_count=3 intent_bytes=49 lock_count=9 lock_bytes=414 lock_age=809 # Committed resolution releases the lock. -run ok +run stats ok resolve_intent t=A k=k1 status=COMMITTED ---- +>> resolve_intent t=A k=k1 status=COMMITTED resolve_intent: "k1" -> resolved key = true +stats: lock_count=-1 lock_bytes=-69 lock_age=-90 >> at end: data: "k1"/5.000000000,0 -> /BYTES/v1 meta: "k2"/0,0 -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=4} ts=10.000000000,0 del=true klen=12 vlen=0 ih={{3 /BYTES/v2}} mergeTs= txnDidNotUpdateMeta=false @@ -559,11 +647,14 @@ lock (Replicated): "k2"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable lock (Replicated): "k2"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k3"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k4"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=3} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true +stats: key_count=4 key_bytes=84 val_count=6 val_bytes=223 live_count=2 live_bytes=113 gc_bytes_age=17460 intent_count=3 intent_bytes=49 lock_count=8 lock_bytes=345 lock_age=719 # Re-acquire the lock. -run ok +run stats ok acquire_lock t=A k=k1 str=shared ---- +>> acquire_lock t=A k=k1 str=shared +stats: lock_count=+1 lock_bytes=+71 lock_age=+90 >> at end: lock (Replicated): "k1"/Shared -> txn={id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} ts=11.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true lock (Replicated): "k1"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=4} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true @@ -571,12 +662,15 @@ lock (Replicated): "k2"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable lock (Replicated): "k2"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k3"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k4"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=3} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true +stats: key_count=4 key_bytes=84 val_count=6 val_bytes=223 live_count=2 live_bytes=113 gc_bytes_age=17460 intent_count=3 intent_bytes=49 lock_count=9 lock_bytes=416 lock_age=809 # Aborted resolution releases the lock. -run ok +run stats ok resolve_intent t=A k=k1 status=ABORTED ---- +>> resolve_intent t=A k=k1 status=ABORTED resolve_intent: "k1" -> resolved key = true +stats: lock_count=-1 lock_bytes=-71 lock_age=-90 >> at end: data: "k1"/5.000000000,0 -> /BYTES/v1 meta: "k2"/0,0 -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=4} ts=10.000000000,0 del=true klen=12 vlen=0 ih={{3 /BYTES/v2}} mergeTs= txnDidNotUpdateMeta=false @@ -592,11 +686,14 @@ lock (Replicated): "k2"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable lock (Replicated): "k2"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k3"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k4"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=3} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true +stats: key_count=4 key_bytes=84 val_count=6 val_bytes=223 live_count=2 live_bytes=113 gc_bytes_age=17460 intent_count=3 intent_bytes=49 lock_count=8 lock_bytes=345 lock_age=719 # Re-acquire the lock. -run ok +run stats ok acquire_lock t=A k=k1 str=shared ---- +>> acquire_lock t=A k=k1 str=shared +stats: lock_count=+1 lock_bytes=+71 lock_age=+90 >> at end: lock (Replicated): "k1"/Shared -> txn={id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} ts=11.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true lock (Replicated): "k1"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=4} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true @@ -604,14 +701,17 @@ lock (Replicated): "k2"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable lock (Replicated): "k2"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k3"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k4"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=3} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true +stats: key_count=4 key_bytes=84 val_count=6 val_bytes=223 live_count=2 live_bytes=113 gc_bytes_age=17460 intent_count=3 intent_bytes=49 lock_count=9 lock_bytes=416 lock_age=809 # Pending resolution with newer epoch releases the lock. -run ok +run stats ok with t=A txn_restart resolve_intent k=k1 status=PENDING ---- +>> resolve_intent k=k1 status=PENDING t=A resolve_intent: "k1" -> resolved key = true +stats: lock_count=-1 lock_bytes=-71 lock_age=-90 >> at end: txn: "A" meta={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=2 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=0,0 data: "k1"/5.000000000,0 -> /BYTES/v1 @@ -628,11 +728,14 @@ lock (Replicated): "k2"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable lock (Replicated): "k2"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k3"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k4"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=3} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true +stats: key_count=4 key_bytes=84 val_count=6 val_bytes=223 live_count=2 live_bytes=113 gc_bytes_age=17460 intent_count=3 intent_bytes=49 lock_count=8 lock_bytes=345 lock_age=719 # Re-acquire the lock at the new epoch. -run ok +run stats ok acquire_lock t=A k=k1 str=shared ---- +>> acquire_lock t=A k=k1 str=shared +stats: lock_count=+1 lock_bytes=+69 lock_age=+90 >> at end: lock (Replicated): "k1"/Shared -> txn={id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} ts=11.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true lock (Replicated): "k1"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=2 ts=10.000000000,0 min=0,0 seq=0} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true @@ -640,15 +743,18 @@ lock (Replicated): "k2"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable lock (Replicated): "k2"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k3"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k4"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=3} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true +stats: key_count=4 key_bytes=84 val_count=6 val_bytes=223 live_count=2 live_bytes=113 gc_bytes_age=17460 intent_count=3 intent_bytes=49 lock_count=9 lock_bytes=414 lock_age=809 # Pending resolution with older epoch is a no-op, regardless of savepoint # rollback. -run ok +run stats ok with t=A txn_restart epoch=1 resolve_intent k=k1 status=PENDING ---- +>> resolve_intent k=k1 status=PENDING t=A resolve_intent: "k1" -> resolved key = false +stats: no change >> at end: txn: "A" meta={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=0,0 data: "k1"/5.000000000,0 -> /BYTES/v1 @@ -666,13 +772,16 @@ lock (Replicated): "k2"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable lock (Replicated): "k2"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k3"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k4"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=3} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true +stats: key_count=4 key_bytes=84 val_count=6 val_bytes=223 live_count=2 live_bytes=113 gc_bytes_age=17460 intent_count=3 intent_bytes=49 lock_count=9 lock_bytes=414 lock_age=809 -run ok +run stats ok with t=A txn_ignore_seqs seqs=0-0 resolve_intent k=k1 status=PENDING ---- +>> resolve_intent k=k1 status=PENDING t=A resolve_intent: "k1" -> resolved key = false +stats: no change >> at end: txn: "A" meta={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=0,0 isn=1 data: "k1"/5.000000000,0 -> /BYTES/v1 @@ -690,15 +799,18 @@ lock (Replicated): "k2"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable lock (Replicated): "k2"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k3"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k4"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=3} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true +stats: key_count=4 key_bytes=84 val_count=6 val_bytes=223 live_count=2 live_bytes=113 gc_bytes_age=17460 intent_count=3 intent_bytes=49 lock_count=9 lock_bytes=414 lock_age=809 # Pending resolution of the lock in the same epoch without a savepoint rollback # is a no-op. -run ok +run stats ok with t=A txn_restart epoch=2 resolve_intent k=k1 status=PENDING ---- +>> resolve_intent k=k1 status=PENDING t=A resolve_intent: "k1" -> resolved key = false +stats: no change >> at end: txn: "A" meta={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=2 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=0,0 data: "k1"/5.000000000,0 -> /BYTES/v1 @@ -716,15 +828,18 @@ lock (Replicated): "k2"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable lock (Replicated): "k2"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k3"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k4"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=3} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true +stats: key_count=4 key_bytes=84 val_count=6 val_bytes=223 live_count=2 live_bytes=113 gc_bytes_age=17460 intent_count=3 intent_bytes=49 lock_count=9 lock_bytes=414 lock_age=809 # Pending resolution of the lock in the same epoch with savepoint rollback # releases the lock. -run ok +run stats ok with t=A txn_ignore_seqs seqs=0-0 resolve_intent k=k1 status=PENDING ---- +>> resolve_intent k=k1 status=PENDING t=A resolve_intent: "k1" -> resolved key = true +stats: lock_count=-1 lock_bytes=-69 lock_age=-90 >> at end: txn: "A" meta={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=2 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=0,0 isn=1 data: "k1"/5.000000000,0 -> /BYTES/v1 @@ -741,25 +856,32 @@ lock (Replicated): "k2"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable lock (Replicated): "k2"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k3"/Exclusive -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=1} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=false lock (Replicated): "k4"/Shared -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=1 ts=10.000000000,0 min=0,0 seq=3} ts=10.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true +stats: key_count=4 key_bytes=84 val_count=6 val_bytes=223 live_count=2 live_bytes=113 gc_bytes_age=17460 intent_count=3 intent_bytes=49 lock_count=8 lock_bytes=345 lock_age=719 # Ranged resolution behaves like single-key resolution. # As always, it also resolves intents in the range. -run ok +run stats ok resolve_intent_range t=A k=k2 end=k5 status=COMMITTED ---- +>> resolve_intent_range t=A k=k2 end=k5 status=COMMITTED resolve_intent_range: "k2"-"k5" -> resolved 3 key(s) +stats: key_count=-1 key_bytes=-39 val_count=-3 val_bytes=-202 live_count=+1 live_bytes=-47 gc_bytes_age=-17460 intent_count=-3 intent_bytes=-49 lock_count=-7 lock_bytes=-278 lock_age=-630 >> at end: data: "k1"/5.000000000,0 -> /BYTES/v1 data: "k2"/5.000000000,0 -> /BYTES/v2 data: "k3"/5.000000000,0 -> /BYTES/v3 lock (Replicated): "k1"/Shared -> txn={id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} ts=11.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true +stats: key_count=3 key_bytes=45 val_count=3 val_bytes=21 live_count=3 live_bytes=66 lock_count=1 lock_bytes=67 lock_age=89 # Aborted resolution for other transaction releases the lock. -run ok +run stats ok resolve_intent t=B k=k1 status=ABORTED ---- +>> resolve_intent t=B k=k1 status=ABORTED resolve_intent: "k1" -> resolved key = true +stats: lock_count=-1 lock_bytes=-67 lock_age=-89 >> at end: data: "k1"/5.000000000,0 -> /BYTES/v1 data: "k2"/5.000000000,0 -> /BYTES/v2 data: "k3"/5.000000000,0 -> /BYTES/v3 +stats: key_count=3 key_bytes=45 val_count=3 val_bytes=21 live_count=3 live_bytes=66 diff --git a/pkg/storage/testdata/mvcc_histories/resolve_intent_pagination b/pkg/storage/testdata/mvcc_histories/resolve_intent_pagination index b95cd02f6684..f7b15d9fca20 100644 --- a/pkg/storage/testdata/mvcc_histories/resolve_intent_pagination +++ b/pkg/storage/testdata/mvcc_histories/resolve_intent_pagination @@ -27,12 +27,14 @@ meta: "f"/0,0 -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=0 data: "f"/1.000000000,0 -> /BYTES/f # Resolve none since targetBytes < 0. -run ok +run stats ok resolve_intent t=A k=c status=COMMITTED targetBytes=-1 batched ---- +>> resolve_intent t=A k=c status=COMMITTED targetBytes=-1 batched resolve_intent: "c" -> resolved key = false, 0 bytes resolve_intent: resume span ["c",/Min) resolve_intent: batch after write is empty +stats: no change >> at end: meta: "a"/0,0 -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=0} ts=1.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true data: "a"/1.000000000,0 -> /BYTES/a @@ -46,13 +48,16 @@ meta: "eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee"/0,0 -> txn={id=000000 data: "eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee"/1.000000000,0 -> /BYTES/e meta: "f"/0,0 -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=0} ts=1.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true data: "f"/1.000000000,0 -> /BYTES/f +stats: key_count=6 key_bytes=182 val_count=6 val_bytes=324 live_count=6 live_bytes=506 intent_count=6 intent_bytes=108 lock_count=6 lock_age=594 # Resolve intent "b". -run ok +run stats ok resolve_intent t=A k=b status=COMMITTED targetBytes=1 batched ---- +>> resolve_intent t=A k=b status=COMMITTED targetBytes=1 batched resolve_intent: "b" -> resolved key = true, 28 bytes resolve_intent: batch after write is non-empty +stats: val_bytes=-48 live_bytes=-48 intent_count=-1 intent_bytes=-18 lock_count=-1 lock_age=-99 >> at end: meta: "a"/0,0 -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=0} ts=1.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true data: "a"/1.000000000,0 -> /BYTES/a @@ -65,14 +70,17 @@ meta: "eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee"/0,0 -> txn={id=000000 data: "eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee"/1.000000000,0 -> /BYTES/e meta: "f"/0,0 -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=0} ts=1.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true data: "f"/1.000000000,0 -> /BYTES/f +stats: key_count=6 key_bytes=182 val_count=6 val_bytes=276 live_count=6 live_bytes=458 intent_count=5 intent_bytes=90 lock_count=5 lock_age=495 # Resolve none since maxKeys < 0. -run ok +run stats ok resolve_intent_range t=A k=a end=z status=COMMITTED maxKeys=-1 batched ---- +>> resolve_intent_range t=A k=a end=z status=COMMITTED maxKeys=-1 batched resolve_intent_range: "a"-"z" -> resolved 0 key(s), 0 bytes resolve_intent_range: resume span ["a","z") RESUME_KEY_LIMIT resolve_intent_range: batch after write is empty +stats: no change >> at end: meta: "a"/0,0 -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=0} ts=1.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true data: "a"/1.000000000,0 -> /BYTES/a @@ -85,14 +93,17 @@ meta: "eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee"/0,0 -> txn={id=000000 data: "eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee"/1.000000000,0 -> /BYTES/e meta: "f"/0,0 -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=0} ts=1.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true data: "f"/1.000000000,0 -> /BYTES/f +stats: key_count=6 key_bytes=182 val_count=6 val_bytes=276 live_count=6 live_bytes=458 intent_count=5 intent_bytes=90 lock_count=5 lock_age=495 # Resolve 2 intents "a" and "c". -run ok +run stats ok resolve_intent_range t=A k=a end=z status=COMMITTED maxKeys=2 batched ---- +>> resolve_intent_range t=A k=a end=z status=COMMITTED maxKeys=2 batched resolve_intent_range: "a"-"z" -> resolved 2 key(s), 56 bytes resolve_intent_range: resume span ["c\x00","z") RESUME_KEY_LIMIT resolve_intent_range: batch after write is non-empty +stats: val_bytes=-96 live_bytes=-96 intent_count=-2 intent_bytes=-36 lock_count=-2 lock_age=-198 >> at end: data: "a"/1.000000000,0 -> /BYTES/a data: "b"/1.000000000,0 -> /BYTES/b @@ -103,14 +114,17 @@ meta: "eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee"/0,0 -> txn={id=000000 data: "eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee"/1.000000000,0 -> /BYTES/e meta: "f"/0,0 -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=0} ts=1.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true data: "f"/1.000000000,0 -> /BYTES/f +stats: key_count=6 key_bytes=182 val_count=6 val_bytes=180 live_count=6 live_bytes=362 intent_count=3 intent_bytes=54 lock_count=3 lock_age=297 # Resolve none since targetBytes < 0. -run ok +run stats ok resolve_intent_range t=A k=a end=z status=COMMITTED targetBytes=-1 batched ---- +>> resolve_intent_range t=A k=a end=z status=COMMITTED targetBytes=-1 batched resolve_intent_range: "a"-"z" -> resolved 0 key(s), 0 bytes resolve_intent_range: resume span ["a","z") RESUME_BYTE_LIMIT resolve_intent_range: batch after write is empty +stats: no change >> at end: data: "a"/1.000000000,0 -> /BYTES/a data: "b"/1.000000000,0 -> /BYTES/b @@ -121,13 +135,16 @@ meta: "eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee"/0,0 -> txn={id=000000 data: "eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee"/1.000000000,0 -> /BYTES/e meta: "f"/0,0 -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=0} ts=1.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true data: "f"/1.000000000,0 -> /BYTES/f +stats: key_count=6 key_bytes=182 val_count=6 val_bytes=180 live_count=6 live_bytes=362 intent_count=3 intent_bytes=54 lock_count=3 lock_age=297 -run ok +run stats ok resolve_intent_range t=A k=a end=z status=COMMITTED targetBytes=99 batched ---- +>> resolve_intent_range t=A k=a end=z status=COMMITTED targetBytes=99 batched resolve_intent_range: "a"-"z" -> resolved 2 key(s), 154 bytes resolve_intent_range: resume span ["eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee\x00","z") RESUME_BYTE_LIMIT resolve_intent_range: batch after write is non-empty +stats: val_bytes=-96 live_bytes=-96 intent_count=-2 intent_bytes=-36 lock_count=-2 lock_age=-198 >> at end: data: "a"/1.000000000,0 -> /BYTES/a data: "b"/1.000000000,0 -> /BYTES/b @@ -136,6 +153,7 @@ data: "dddddddddddddddddddddddddddddddddddddddddddddddddd"/1.000000000,0 -> /BYT data: "eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee"/1.000000000,0 -> /BYTES/e meta: "f"/0,0 -> txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=0} ts=1.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true data: "f"/1.000000000,0 -> /BYTES/f +stats: key_count=6 key_bytes=182 val_count=6 val_bytes=84 live_count=6 live_bytes=266 intent_count=1 intent_bytes=18 lock_count=1 lock_age=99 # Test MaxKeys and TargetBytes for resolve intent range with replicated locks. @@ -168,12 +186,14 @@ lock (Replicated): "b"/Exclusive -> txn={id=00000002 key=/Min iso=Serializable p lock (Replicated): "b"/Shared -> txn={id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=0} ts=1.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true lock (Replicated): "c"/Shared -> txn={id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=0} ts=1.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true -run ok +run stats ok resolve_intent_range t=B k=a end=z status=COMMITTED maxKeys=1 batched ---- +>> resolve_intent_range t=B k=a end=z status=COMMITTED maxKeys=1 batched resolve_intent_range: "a"-"z" -> resolved 1 key(s), 56 bytes resolve_intent_range: resume span ["a\x00","z") RESUME_KEY_LIMIT resolve_intent_range: batch after write is non-empty +stats: val_bytes=-48 live_bytes=-48 intent_count=-1 intent_bytes=-18 lock_count=-2 lock_bytes=-66 lock_age=-198 >> at end: data: "a"/1.000000000,0 -> /BYTES/a meta: "b"/0,0 -> txn={id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=0} ts=1.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true @@ -181,23 +201,30 @@ data: "b"/1.000000000,0 -> /BYTES/b lock (Replicated): "b"/Exclusive -> txn={id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=0} ts=1.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true lock (Replicated): "b"/Shared -> txn={id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=0} ts=1.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true lock (Replicated): "c"/Shared -> txn={id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=0} ts=1.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true +stats: key_count=2 key_bytes=28 val_count=2 val_bytes=60 live_count=2 live_bytes=88 intent_count=1 intent_bytes=18 lock_count=4 lock_bytes=198 lock_age=396 -run ok +run stats ok resolve_intent_range t=B k=a end=z status=COMMITTED targetBytes=1 batched ---- +>> resolve_intent_range t=B k=a end=z status=COMMITTED targetBytes=1 batched resolve_intent_range: "a"-"z" -> resolved 1 key(s), 84 bytes resolve_intent_range: resume span ["b\x00","z") RESUME_BYTE_LIMIT resolve_intent_range: batch after write is non-empty +stats: val_bytes=-48 live_bytes=-48 intent_count=-1 intent_bytes=-18 lock_count=-3 lock_bytes=-132 lock_age=-297 >> at end: data: "a"/1.000000000,0 -> /BYTES/a data: "b"/1.000000000,0 -> /BYTES/b lock (Replicated): "c"/Shared -> txn={id=00000002 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=0} ts=1.000000000,0 del=false klen=0 vlen=0 mergeTs= txnDidNotUpdateMeta=true +stats: key_count=2 key_bytes=28 val_count=2 val_bytes=12 live_count=2 live_bytes=40 lock_count=1 lock_bytes=66 lock_age=99 -run ok +run stats ok resolve_intent_range t=B k=a end=z status=COMMITTED maxKeys=1 batched ---- +>> resolve_intent_range t=B k=a end=z status=COMMITTED maxKeys=1 batched resolve_intent_range: "a"-"z" -> resolved 1 key(s), 28 bytes resolve_intent_range: batch after write is non-empty +stats: lock_count=-1 lock_bytes=-66 lock_age=-99 >> at end: data: "a"/1.000000000,0 -> /BYTES/a data: "b"/1.000000000,0 -> /BYTES/b +stats: key_count=2 key_bytes=28 val_count=2 val_bytes=12 live_count=2 live_bytes=40