diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index f11327a12b87..548d230a12df 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -4959,17 +4959,17 @@ func MVCCResolveWriteIntent( ctx context.Context, rw ReadWriter, ms *enginepb.MVCCStats, - intent roachpb.LockUpdate, + update roachpb.LockUpdate, opts MVCCResolveWriteIntentOptions, ) (ok bool, numBytes int64, resumeSpan *roachpb.Span, replLocksReleased bool, err error) { - if len(intent.Key) == 0 { + if len(update.Key) == 0 { return false, 0, nil, false, emptyKeyError() } - if len(intent.EndKey) > 0 { + if len(update.EndKey) > 0 { return false, 0, nil, false, errors.Errorf("can't resolve range intent as point intent") } if opts.TargetBytes < 0 { - return false, 0, &roachpb.Span{Key: intent.Key}, false, nil + return false, 0, &roachpb.Span{Key: update.Key}, false, nil } // Production code will use a buffered writer, which makes the numBytes @@ -4978,10 +4978,10 @@ func MVCCResolveWriteIntent( // the database. beforeBytes := rw.BufferedSize() - // Iterate over all locks held by intent.Txn on this key. + // Iterate over all locks held by update.Txn on this key. ltIter, err := NewLockTableIterator(ctx, rw, LockTableIteratorOptions{ Prefix: true, - MatchTxnID: intent.Txn.ID, + MatchTxnID: update.Txn.ID, ReadCategory: IntentResolutionReadCategory, }) if err != nil { @@ -4993,11 +4993,11 @@ func MVCCResolveWriteIntent( var ltSeekKey EngineKey ltSeekKey, buf.ltKeyBuf = LockTableKey{ - Key: intent.Key, + Key: update.Key, // lock.Intent is the first locking strength in the lock-table. As a // minor performance optimization, we seek to this version and iterate // instead of iterating from the beginning of the version prefix (i.e. - // keys.LockTableSingleKey(intent.Key)). This can seek past half of the + // keys.LockTableSingleKey(update.Key)). This can seek past half of the // LSM tombstones on this key in cases like those described in d1c91e0e // where intents are repeatedly written and removed on a specific key so // an intent is surrounded by a large number of tombstones during its @@ -5014,7 +5014,7 @@ func MVCCResolveWriteIntent( // a single seek and step in cases where only an intent is present. We // chose not to pessimize the common case to optimize the uncommon case. Strength: lock.Intent, - TxnUUID: intent.Txn.ID, + TxnUUID: update.Txn.ID, }.ToEngineKey(buf.ltKeyBuf) for valid, err := ltIter.SeekEngineKeyGE(ltSeekKey); ; valid, err = ltIter.NextEngineKey() { @@ -5027,9 +5027,9 @@ func MVCCResolveWriteIntent( if err != nil { return false, 0, nil, false, errors.Wrap(err, "decoding lock table key version") } - if txnID != intent.Txn.ID { + if txnID != update.Txn.ID { return false, 0, nil, false, errors.AssertionFailedf( - "unexpected txnID %v != %v while scanning lock table", txnID, intent.Txn.ID) + "unexpected txnID %v != %v while scanning lock table", txnID, update.Txn.ID) } if err := ltIter.ValueProto(&buf.meta); err != nil { return false, 0, nil, false, errors.Wrap(err, "unmarshaling lock table value") @@ -5050,9 +5050,9 @@ func MVCCResolveWriteIntent( } defer iter.Close() } - outcome, err = mvccResolveWriteIntent(ctx, rw, iter, ms, intent, &buf.meta, buf) + outcome, err = mvccResolveWriteIntent(ctx, rw, iter, ms, update, &buf.meta, buf) } else { - outcome, err = mvccReleaseLockInternal(ctx, rw, ms, intent, str, &buf.meta, buf) + outcome, err = mvccReleaseLockInternal(ctx, rw, ms, update, str, &buf.meta, buf) replLocksReleased = replLocksReleased || outcome != lockNoop } if err != nil { @@ -5173,58 +5173,57 @@ const ( // provided intent was resolved (a no-op, rewriting the intent, writing a // SingleDelete key, or writing a Delete key). // -// REQUIRES: intent and meta refer to the same intent on the same key. +// REQUIRES: update and meta refer to the same intent on the same key. // REQUIRES: iter surfaces range keys via IterKeyTypePointsAndRanges. func mvccResolveWriteIntent( ctx context.Context, writer Writer, iter MVCCIterator, ms *enginepb.MVCCStats, - // TODO(nvanbenschoten): rename this field to "update". - intent roachpb.LockUpdate, + update roachpb.LockUpdate, meta *enginepb.MVCCMetadata, buf *putBuffer, ) (outcome lockResolutionOutcome, err error) { - if meta.Txn == nil || meta.Txn.ID != intent.Txn.ID { - return lockNoop, errors.Errorf("txn does not match: %v != %v", meta.Txn, intent.Txn) + if meta.Txn == nil || meta.Txn.ID != update.Txn.ID { + return lockNoop, errors.Errorf("txn does not match: %v != %v", meta.Txn, update.Txn) } - metaKey := MakeMVCCMetadataKey(intent.Key) + metaKey := MakeMVCCMetadataKey(update.Key) origMetaKeySize := int64(metaKey.EncodedSize()) origMetaValSize := int64(meta.Size()) metaTimestamp := meta.Timestamp.ToTimestamp() canSingleDelHelper := singleDelOptimizationHelper{ _didNotUpdateMeta: meta.TxnDidNotUpdateMeta, - _hasIgnoredSeqs: len(intent.IgnoredSeqNums) > 0, + _hasIgnoredSeqs: len(update.IgnoredSeqNums) > 0, // NB: the value is only used if epochs match, so it doesn't // matter if we use the one from meta or incoming request here. - _epoch: intent.Txn.Epoch, + _epoch: update.Txn.Epoch, } - // A commit with a newer epoch than the intent effectively means that we + // An update with a newer epoch than the intent effectively means that we // wrote this intent before an earlier retry, but didn't write it again // after. We treat such intents as uncommitted. // - // A commit with a newer timestamp than the intent means that our timestamp + // An update with a newer timestamp than the intent means that our timestamp // was pushed during the course of an epoch. We treat such intents as // committed after moving their timestamp forward. This is possible if a // transaction writes an intent and then successfully refreshes its // timestamp to avoid a restart. // - // A commit with an older epoch than the intent should never happen because + // An update with an older epoch than the intent should never happen because // epoch increments require client action. This means that they can't be // caused by replays. // - // A commit with an older timestamp than the intent should not happen under + // An update with an older timestamp than the intent should not happen under // normal circumstances because a client should never bump its timestamp // after issuing an EndTxn request. Replays of intent writes that are pushed // forward due to WriteTooOld errors without client action combined with // replays of intent resolution make this configuration a possibility. We // treat such intents as uncommitted. - epochsMatch := meta.Txn.Epoch == intent.Txn.Epoch - timestampsValid := metaTimestamp.LessEq(intent.Txn.WriteTimestamp) - timestampChanged := metaTimestamp.Less(intent.Txn.WriteTimestamp) - commit := intent.Status == roachpb.COMMITTED && epochsMatch && timestampsValid + epochsMatch := meta.Txn.Epoch == update.Txn.Epoch + timestampsValid := metaTimestamp.LessEq(update.Txn.WriteTimestamp) + timestampChanged := metaTimestamp.Less(update.Txn.WriteTimestamp) + commit := update.Status == roachpb.COMMITTED && epochsMatch && timestampsValid // Note the small difference to commit epoch handling here: We allow // a push from a previous epoch to move a newer intent. That's not @@ -5251,9 +5250,9 @@ func mvccResolveWriteIntent( // used for resolving), but that costs latency. // TODO(tschottdorf): various epoch-related scenarios here deserve more // testing. - inProgress := !intent.Status.IsFinalized() && meta.Txn.Epoch >= intent.Txn.Epoch + inProgress := !update.Status.IsFinalized() && meta.Txn.Epoch >= update.Txn.Epoch pushed := inProgress && timestampChanged - latestKey := MVCCKey{Key: intent.Key, Timestamp: metaTimestamp} + latestKey := MVCCKey{Key: update.Key, Timestamp: metaTimestamp} // Handle partial txn rollbacks. If the current txn sequence // is part of a rolled back (ignored) seqnum range, we're going @@ -5265,7 +5264,7 @@ func mvccResolveWriteIntent( var rolledBackVal *MVCCValue buf.newMeta = *meta newMeta := &buf.newMeta - if len(intent.IgnoredSeqNums) > 0 { + if len(update.IgnoredSeqNums) > 0 { // NOTE: mvccMaybeRewriteIntentHistory mutates its meta argument. // TODO(nvanbenschoten): this is an awkward interface. We shouldn't // be mutating meta and we shouldn't be restoring the previous value @@ -5276,7 +5275,7 @@ func mvccResolveWriteIntent( // intact and corresponding to the stats in ms to ensure that later on (in // updateStatsOnResolve) the stats will be updated correctly based on the // old meta (meta) and the new meta (newMeta). - removeIntent, rolledBackVal, err = mvccMaybeRewriteIntentHistory(ctx, writer, intent.IgnoredSeqNums, newMeta, latestKey) + removeIntent, rolledBackVal, err = mvccMaybeRewriteIntentHistory(ctx, writer, update.IgnoredSeqNums, newMeta, latestKey) if err != nil { return lockNoop, err } @@ -5296,7 +5295,7 @@ func mvccResolveWriteIntent( // If we need to update the intent to roll back part of its intent // history, make sure that we don't regress its timestamp, even if the // caller provided an outdated timestamp. - intent.Txn.WriteTimestamp.Forward(metaTimestamp) + update.Txn.WriteTimestamp.Forward(metaTimestamp) } } @@ -5320,7 +5319,7 @@ func mvccResolveWriteIntent( if commit || pushed || rolledBackVal != nil { // The intent might be committing at a higher timestamp, or it might be // getting pushed. - newTimestamp := intent.Txn.WriteTimestamp + newTimestamp := update.Txn.WriteTimestamp // Assert that the intent timestamp never regresses. The logic above should // not allow this, regardless of the input to this function. @@ -5380,7 +5379,7 @@ func mvccResolveWriteIntent( // to the observed timestamp. newValue := oldValue newValue.LocalTimestamp = oldValue.GetLocalTimestamp(oldKey.Timestamp) - newValue.LocalTimestamp.Forward(intent.ClockWhilePending.Timestamp) + newValue.LocalTimestamp.Forward(update.ClockWhilePending.Timestamp) if !newValue.LocalTimestampNeeded(newKey.Timestamp) || !writer.ShouldWriteLocalTimestamps(ctx) { newValue.LocalTimestamp = hlc.ClockTimestamp{} } @@ -5459,7 +5458,7 @@ func mvccResolveWriteIntent( // Update stat counters related to resolving the intent. if ms != nil { - ms.Add(updateStatsOnResolve(intent.Key, prevIsValue, prevValSize, origMetaKeySize, origMetaValSize, + ms.Add(updateStatsOnResolve(update.Key, prevIsValue, prevValSize, origMetaKeySize, origMetaValSize, metaKeySize, metaValSize, meta, newMeta, commit)) } @@ -5469,9 +5468,9 @@ func mvccResolveWriteIntent( logicalOp = MVCCUpdateIntentOpType } writer.LogLogicalOp(logicalOp, MVCCLogicalOpDetails{ - Txn: intent.Txn, - Key: intent.Key, - Timestamp: intent.Txn.WriteTimestamp, + Txn: update.Txn, + Key: update.Key, + Timestamp: update.Txn.WriteTimestamp, }) // outcome is set up above. return outcome, nil @@ -5481,7 +5480,7 @@ func mvccResolveWriteIntent( // MVCCMetadata. // // Note that we have to support a somewhat unintuitive case - an ABORT with - // intent.Txn.Epoch < meta.Txn.Epoch: + // update.Txn.Epoch < meta.Txn.Epoch: // - writer1 writes key0 at epoch 0 // - writer2 with higher priority encounters intent at key0 (epoch 0) // - writer1 restarts, now at epoch one (txn record not updated) @@ -5499,8 +5498,8 @@ func mvccResolveWriteIntent( // Log the logical MVCC operation. writer.LogLogicalOp(MVCCAbortIntentOpType, MVCCLogicalOpDetails{ - Txn: intent.Txn, - Key: intent.Key, + Txn: update.Txn, + Key: update.Key, }) ok := false @@ -5575,7 +5574,7 @@ func mvccResolveWriteIntent( // Clear stat counters attributable to the intent we're aborting. if ms != nil { ms.Add(updateStatsOnClear( - intent.Key, origMetaKeySize, origMetaValSize, 0, 0, meta, nil, 0)) + update.Key, origMetaKeySize, origMetaValSize, 0, 0, meta, nil, 0)) } // outcome is set above before the clearLockMeta call. return outcome, nil @@ -5603,7 +5602,7 @@ func mvccResolveWriteIntent( // Update stat counters with older version. if ms != nil { - ms.Add(updateStatsOnClear(intent.Key, origMetaKeySize, origMetaValSize, metaKeySize, + ms.Add(updateStatsOnClear(update.Key, origMetaKeySize, origMetaValSize, metaKeySize, metaValSize, meta, &buf.newMeta, unsafeNextKey.Timestamp.WallTime)) } // outcome is set above before the clearLockMeta call. @@ -5684,7 +5683,7 @@ func MVCCResolveWriteIntentRange( ctx context.Context, rw ReadWriter, ms *enginepb.MVCCStats, - intent roachpb.LockUpdate, + update roachpb.LockUpdate, opts MVCCResolveWriteIntentRangeOptions, ) ( numKeys, numBytes int64, @@ -5696,7 +5695,7 @@ func MVCCResolveWriteIntentRange( keysExceeded := opts.MaxKeys < 0 bytesExceeded := opts.TargetBytes < 0 if keysExceeded || bytesExceeded { - resumeSpan := intent.Span // don't inline or `intent` would escape to heap + resumeSpan := update.Span // don't inline or `update` would escape to heap if keysExceeded { resumeReason = kvpb.RESUME_KEY_LIMIT } else if bytesExceeded { @@ -5705,12 +5704,12 @@ func MVCCResolveWriteIntentRange( return 0, 0, &resumeSpan, resumeReason, false, nil } - ltStart, _ := keys.LockTableSingleKey(intent.Key, nil) - ltEnd, _ := keys.LockTableSingleKey(intent.EndKey, nil) + ltStart, _ := keys.LockTableSingleKey(update.Key, nil) + ltEnd, _ := keys.LockTableSingleKey(update.EndKey, nil) ltIter, err := NewLockTableIterator(ctx, rw, LockTableIteratorOptions{ LowerBound: ltStart, UpperBound: ltEnd, - MatchTxnID: intent.Txn.ID, + MatchTxnID: update.Txn.ID, ReadCategory: IntentResolutionReadCategory, }) if err != nil { @@ -5720,8 +5719,8 @@ func MVCCResolveWriteIntentRange( var mvccIter MVCCIterator iterOpts := IterOptions{ KeyTypes: IterKeyTypePointsAndRanges, - LowerBound: intent.Key, - UpperBound: intent.EndKey, + LowerBound: update.Key, + UpperBound: update.EndKey, ReadCategory: IntentResolutionReadCategory, } if rw.ConsistentIterators() { @@ -5738,8 +5737,8 @@ func MVCCResolveWriteIntentRange( buf := newPutBuffer() defer buf.release() - intentEndKey := intent.EndKey - intent.EndKey = nil + intentEndKey := update.EndKey + update.EndKey = nil var lastResolvedKey roachpb.Key var lastResolvedKeyOk bool @@ -5789,20 +5788,20 @@ func MVCCResolveWriteIntentRange( lastResolvedKey = append(lastResolvedKey[:0], ltKey.Key...) lastResolvedKeyOk = false } - if ltKey.TxnUUID != intent.Txn.ID { + if ltKey.TxnUUID != update.Txn.ID { return 0, 0, nil, 0, false, errors.AssertionFailedf( - "unexpected txnID %v != %v while scanning lock table", ltKey.TxnUUID, intent.Txn.ID) + "unexpected txnID %v != %v while scanning lock table", ltKey.TxnUUID, update.Txn.ID) } - intent.Key = ltKey.Key + update.Key = ltKey.Key if err := ltIter.ValueProto(&buf.meta); err != nil { return 0, 0, nil, 0, false, errors.Wrap(err, "unmarshaling lock table value") } beforeBytes := rw.BufferedSize() var outcome lockResolutionOutcome if ltKey.Strength == lock.Intent { - outcome, err = mvccResolveWriteIntent(ctx, rw, mvccIter, ms, intent, &buf.meta, buf) + outcome, err = mvccResolveWriteIntent(ctx, rw, mvccIter, ms, update, &buf.meta, buf) } else { - outcome, err = mvccReleaseLockInternal(ctx, rw, ms, intent, ltKey.Strength, &buf.meta, buf) + outcome, err = mvccReleaseLockInternal(ctx, rw, ms, update, ltKey.Strength, &buf.meta, buf) replLocksReleased = replLocksReleased || outcome != lockNoop } if err != nil {