From 45bc7dc6ebacd74ae6562a16d8027da4582cc293 Mon Sep 17 00:00:00 2001 From: Mira Radeva Date: Mon, 8 Jan 2024 16:13:19 -0500 Subject: [PATCH 1/3] storage: rename intent to update in MVCC intent resolution functions Rename the intent argument to the mvccResolveWriteIntent, MVCCResolveWriteIntent, and MVCCResolveWriteIntentRange functions to make it clear that it refers to the state passed in via ResolveIntent, and not the current value of the stored intent. Informs: #117553 Release note: None --- pkg/storage/mvcc.go | 115 ++++++++++++++++++++++---------------------- 1 file changed, 57 insertions(+), 58 deletions(-) diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 4cb1adc6e6eb..d47ac1e6e457 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -4876,17 +4876,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 @@ -4898,7 +4898,7 @@ func MVCCResolveWriteIntent( // Iterate over all locks held by intent.Txn on this key. ltIter, err := NewLockTableIterator(rw, LockTableIteratorOptions{ Prefix: true, - MatchTxnID: intent.Txn.ID, + MatchTxnID: update.Txn.ID, }) if err != nil { return false, 0, nil, false, err @@ -4909,11 +4909,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 @@ -4930,7 +4930,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() { @@ -4943,9 +4943,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") @@ -4965,9 +4965,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 { @@ -5088,58 +5088,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 @@ -5166,9 +5165,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 @@ -5180,7 +5179,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 @@ -5191,7 +5190,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 } @@ -5211,7 +5210,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) } } @@ -5235,7 +5234,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. @@ -5295,7 +5294,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{} } @@ -5374,7 +5373,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)) } @@ -5384,9 +5383,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 @@ -5396,7 +5395,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) @@ -5414,8 +5413,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 @@ -5490,7 +5489,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 @@ -5518,7 +5517,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. @@ -5599,7 +5598,7 @@ func MVCCResolveWriteIntentRange( ctx context.Context, rw ReadWriter, ms *enginepb.MVCCStats, - intent roachpb.LockUpdate, + update roachpb.LockUpdate, opts MVCCResolveWriteIntentRangeOptions, ) ( numKeys, numBytes int64, @@ -5611,7 +5610,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 { @@ -5620,12 +5619,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(rw, LockTableIteratorOptions{ LowerBound: ltStart, UpperBound: ltEnd, - MatchTxnID: intent.Txn.ID, + MatchTxnID: update.Txn.ID, }) if err != nil { return 0, 0, nil, 0, false, err @@ -5634,8 +5633,8 @@ func MVCCResolveWriteIntentRange( var mvccIter MVCCIterator iterOpts := IterOptions{ KeyTypes: IterKeyTypePointsAndRanges, - LowerBound: intent.Key, - UpperBound: intent.EndKey, + LowerBound: update.Key, + UpperBound: update.EndKey, } if rw.ConsistentIterators() { // Production code should always have consistent iterators. @@ -5651,8 +5650,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 @@ -5702,20 +5701,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 { From 0e44a0fcee2f6e6dda66c782191cb12a20b647a9 Mon Sep 17 00:00:00 2001 From: Mira Radeva Date: Tue, 9 Jan 2024 08:26:25 -0500 Subject: [PATCH 2/3] storage: repro a series of intent resolution bugs with ignored seq nums The logic in mvccResolveWriteIntent is structured in such a way that if an intent contains both ignored and non-ignored seq nums in its intent history, the intent may end up being updated instead of aborted or unmodified. For the following examples, assume the intent has a history ["a", "b"] where "a" is written first, and "b" is ignored. 1. The intent resolution has status aborted. Instead of aborting the intent, it is modified to have value "a" and an empty intent history. 2. The intent resolution has status pending, and the intent has a lower epoch than the resolution. The intent should be aborted because the new epoch may not write it again. Instead, it is updated with value "a" and an empty intent history. 3. Same as 2 but the intent resolution has status committed. 4. The intent resolution has status pending, the intent is not pushed and has a higher epoch than the resolution. The intent should not be updated because the intent history is updated only when the epochs match. Instead, it is updated with value "a" and an empty intent history. 5. Same as 4 but the intent is pushed. The intent should be updated to bump its timestamp in order to unblock the pusher. The intent history should not be updated for the same reason as in 3. Instead, the intent is updated with value "a" and an empty intent history. Additionally, in cases 1, 2, 3 and 4 above, the resulting intent is not committed but a MVCCCommitIntentOp is logged erroneously. This commit only reproduces the bugs. Informs: #117553 Release note: None --- .../testdata/mvcc_histories/ignored_seq_nums | 143 ++++++++++++++++++ .../mvcc_histories/ignored_seq_nums_abort | 31 ++++ .../mvcc_histories/ignored_seq_nums_commit | 48 ++++++ 3 files changed, 222 insertions(+) create mode 100644 pkg/storage/testdata/mvcc_histories/ignored_seq_nums_abort diff --git a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums index 40e27766c3f7..bc92ce8839f1 100644 --- a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums +++ b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums @@ -665,3 +665,146 @@ data: "m"/30.000000000,0 -> /BYTES/a data: "n"/45.000000000,0 -> {localTs=40.000000000,0}/BYTES/c meta: "o"/0,0 -> txn={id=00000006 key="o" iso=Serializable pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=10} ts=50.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=false data: "o"/50.000000000,0 -> /BYTES/a + +run ok +clear_range k=k end=p +---- +>> at end: + + +# Try to update a pending intent with a lower epoch than the resolution. +# The intent should be aborted because the new epoch may not write it again. + +# The test exposes a bug where the intent is updated instead of aborted. +# Moreover, it logs MVCCCommitIntentOp instead of MVCCUpdateIntentOp. + +run ok +with t=G k=p + txn_begin ts=50 + txn_step seq=10 + put v=a + txn_step seq=20 + put v=b + check_intent + get +---- +put: lock acquisition = {p id=00000007 key="p" iso=Serializable pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=10 Replicated Intent []} +put: lock acquisition = {p id=00000007 key="p" iso=Serializable pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=20 Replicated Intent []} +meta: "p" -> txn={id=00000007 key="p" iso=Serializable pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=20} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false +get: "p" -> /BYTES/b @50.000000000,0 +>> at end: +txn: "G" meta={id=00000007 key="p" iso=Serializable pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=20} lock=true stat=PENDING rts=50.000000000,0 wto=false gul=0,0 +meta: "p"/0,0 -> txn={id=00000007 key="p" iso=Serializable pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=20} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false +data: "p"/50.000000000,0 -> /BYTES/b + +run ok log-ops +with t=G k=p + txn_restart + txn_ignore_seqs seqs=(15-25) + resolve_intent status=PENDING + get +---- +resolve_intent: "p" -> resolved key = true +get: "p" -> +>> at end: +txn: "G" meta={id=00000007 key="p" iso=Serializable pri=0.00000000 epo=1 ts=50.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=50.000000000,0 wto=false gul=0,0 isn=1 +meta: "p"/0,0 -> txn={id=00000007 key="p" iso=Serializable pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=10} ts=50.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=false +data: "p"/50.000000000,0 -> /BYTES/a +logical op: *enginepb.MVCCCommitIntentOp + +run ok +clear_range k=p end=-p +---- +>> at end: + + +# Try to update a pending intent with a higher epoch than the resolution's epoch. +# The intent is not pushed, so it is not updated because the intent history +# should be updated only when the epochs match. + +# The test exposes a bug where the intent is actually updated, as evident by the +# changed intent history. +# Moreover, it logs MVCCCommitIntentOp instead of MVCCUpdateIntentOp. + +run ok +with t=H k=q + txn_begin ts=50 + txn_restart epoch=1 + txn_step seq=10 + put v=a + txn_step seq=20 + put v=b + check_intent + get +---- +put: lock acquisition = {q id=00000008 key="q" iso=Serializable pri=0.00000000 epo=1 ts=50.000000000,0 min=0,0 seq=10 Replicated Intent []} +put: lock acquisition = {q id=00000008 key="q" iso=Serializable pri=0.00000000 epo=1 ts=50.000000000,0 min=0,0 seq=20 Replicated Intent []} +meta: "q" -> txn={id=00000008 key="q" iso=Serializable pri=0.00000000 epo=1 ts=50.000000000,0 min=0,0 seq=20} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false +get: "q" -> /BYTES/b @50.000000000,0 +>> at end: +txn: "H" meta={id=00000008 key="q" iso=Serializable pri=0.00000000 epo=1 ts=50.000000000,0 min=0,0 seq=20} lock=true stat=PENDING rts=50.000000000,0 wto=false gul=0,0 +meta: "q"/0,0 -> txn={id=00000008 key="q" iso=Serializable pri=0.00000000 epo=1 ts=50.000000000,0 min=0,0 seq=20} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false +data: "q"/50.000000000,0 -> /BYTES/b + +run ok log-ops +with t=H k=q + txn_restart epoch=0 + txn_ignore_seqs seqs=(15-25) + resolve_intent status=PENDING + check_intent +---- +resolve_intent: "q" -> resolved key = true +meta: "q" -> txn={id=00000008 key="q" iso=Serializable pri=0.00000000 epo=1 ts=50.000000000,0 min=0,0 seq=10} ts=50.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=false +>> at end: +txn: "H" meta={id=00000008 key="q" iso=Serializable pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=50.000000000,0 wto=false gul=0,0 isn=1 +meta: "q"/0,0 -> txn={id=00000008 key="q" iso=Serializable pri=0.00000000 epo=1 ts=50.000000000,0 min=0,0 seq=10} ts=50.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=false +data: "q"/50.000000000,0 -> /BYTES/a +logical op: *enginepb.MVCCCommitIntentOp + +run ok +clear_range k=q end=-q +---- +>> at end: + + +# Update a pending intent with a higher epoch than the resolution's epoch. +# The intent is pushed, so its timestamp is updated to help the pusher make progress. +# The intent history is not updated because the epochs don't match. + +# The test exposes a bug where the intent history is actually updated. + +run ok +with t=I k=r + txn_begin ts=50 + txn_restart epoch=1 + txn_step seq=10 + put v=a + txn_step seq=20 + put v=b + check_intent + get +---- +put: lock acquisition = {r id=00000009 key="r" iso=Serializable pri=0.00000000 epo=1 ts=50.000000000,0 min=0,0 seq=10 Replicated Intent []} +put: lock acquisition = {r id=00000009 key="r" iso=Serializable pri=0.00000000 epo=1 ts=50.000000000,0 min=0,0 seq=20 Replicated Intent []} +meta: "r" -> txn={id=00000009 key="r" iso=Serializable pri=0.00000000 epo=1 ts=50.000000000,0 min=0,0 seq=20} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false +get: "r" -> /BYTES/b @50.000000000,0 +>> at end: +txn: "I" meta={id=00000009 key="r" iso=Serializable pri=0.00000000 epo=1 ts=50.000000000,0 min=0,0 seq=20} lock=true stat=PENDING rts=50.000000000,0 wto=false gul=0,0 +meta: "r"/0,0 -> txn={id=00000009 key="r" iso=Serializable pri=0.00000000 epo=1 ts=50.000000000,0 min=0,0 seq=20} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false +data: "r"/50.000000000,0 -> /BYTES/b + +run ok log-ops +with t=I k=r + txn_restart epoch=0 + txn_advance ts=60 + txn_ignore_seqs seqs=(15-25) + resolve_intent status=PENDING + check_intent +---- +resolve_intent: "r" -> resolved key = true +meta: "r" -> txn={id=00000009 key="r" iso=Serializable pri=0.00000000 epo=1 ts=60.000000000,0 min=0,0 seq=10} ts=60.000000000,0 del=false klen=12 vlen=20 mergeTs= txnDidNotUpdateMeta=false +>> at end: +txn: "I" meta={id=00000009 key="r" iso=Serializable pri=0.00000000 epo=0 ts=60.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=50.000000000,0 wto=false gul=0,0 isn=1 +meta: "r"/0,0 -> txn={id=00000009 key="r" iso=Serializable pri=0.00000000 epo=1 ts=60.000000000,0 min=0,0 seq=10} ts=60.000000000,0 del=false klen=12 vlen=20 mergeTs= txnDidNotUpdateMeta=false +data: "r"/60.000000000,0 -> {localTs=50.000000000,0}/BYTES/a +logical op: *enginepb.MVCCUpdateIntentOp diff --git a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_abort b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_abort new file mode 100644 index 000000000000..7467a3a51455 --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_abort @@ -0,0 +1,31 @@ +# Abort an intent whose history includes ignored seq nums. +# The test exposes a bug where even though the transaction is aborted, its +# intent is updated instead of aborted. + +# The test exposes a bug where even though the transaction is aborted, its +# intent is updated instead of aborted. +# Moreover, it logs MVCCCommitIntentOp instead of MVCCUpdateIntentOp. + +run ok log-ops +with t=A k=k + txn_begin ts=11 + txn_step seq=10 + put v=a + txn_step seq=20 + put v=b + txn_step seq=30 + txn_ignore_seqs seqs=(15-25) + resolve_intent status=ABORTED + get +---- +put: lock acquisition = {k id=00000001 key="k" iso=Serializable pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10 Replicated Intent []} +put: lock acquisition = {k id=00000001 key="k" iso=Serializable pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20 Replicated Intent []} +resolve_intent: "k" -> resolved key = true +get: "k" -> /BYTES/a @11.000000000,0 +>> at end: +txn: "A" meta={id=00000001 key="k" iso=Serializable pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=11.000000000,0 wto=false gul=0,0 isn=1 +meta: "k"/0,0 -> txn={id=00000001 key="k" iso=Serializable pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=false +data: "k"/11.000000000,0 -> /BYTES/a +logical op: *enginepb.MVCCWriteIntentOp +logical op: *enginepb.MVCCUpdateIntentOp +logical op: *enginepb.MVCCCommitIntentOp diff --git a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_commit b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_commit index 923ff3a6fa6e..860f115b052d 100644 --- a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_commit +++ b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_commit @@ -137,3 +137,51 @@ get k=k ---- scan: "k"-"l" -> get: "k" -> + + +# Commit an intent with a lower epoch than the resolution. +# The intent should be aborted because the new epoch may not write it again. + +# The test exposes a bug where the intent is updated instead of aborted. +# Moreover, it logs MVCCCommitIntentOp instead of MVCCUpdateIntentOp. + +run ok +with t=B k=b + txn_begin ts=12 + txn_step seq=10 + put v=a + txn_step seq=20 + put v=b + check_intent + get +---- +put: lock acquisition = {b id=00000003 key="b" iso=Serializable pri=0.00000000 epo=0 ts=12.000000000,0 min=0,0 seq=10 Replicated Intent []} +put: lock acquisition = {b id=00000003 key="b" iso=Serializable pri=0.00000000 epo=0 ts=12.000000000,0 min=0,0 seq=20 Replicated Intent []} +meta: "b" -> txn={id=00000003 key="b" iso=Serializable pri=0.00000000 epo=0 ts=12.000000000,0 min=0,0 seq=20} ts=12.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false +get: "b" -> /BYTES/b @12.000000000,0 +>> at end: +txn: "B" meta={id=00000003 key="b" iso=Serializable pri=0.00000000 epo=0 ts=12.000000000,0 min=0,0 seq=20} lock=true stat=PENDING rts=12.000000000,0 wto=false gul=0,0 +meta: "b"/0,0 -> txn={id=00000003 key="b" iso=Serializable pri=0.00000000 epo=0 ts=12.000000000,0 min=0,0 seq=20} ts=12.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false +data: "b"/12.000000000,0 -> /BYTES/b +data: "k"/11.000000000,0 -> /BYTES/c +data: "k/10"/11.000000000,0 -> /BYTES/10 +data: "k/30"/11.000000000,0 -> /BYTES/30 + + +run ok log-ops +with t=B k=b + txn_restart + txn_ignore_seqs seqs=(15-25) + resolve_intent status=COMMITTED + get +---- +resolve_intent: "b" -> resolved key = true +get: "b" -> +>> at end: +txn: "B" meta={id=00000003 key="b" iso=Serializable pri=0.00000000 epo=1 ts=12.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=12.000000000,0 wto=false gul=0,0 isn=1 +meta: "b"/0,0 -> txn={id=00000003 key="b" iso=Serializable pri=0.00000000 epo=0 ts=12.000000000,0 min=0,0 seq=10} ts=12.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=false +data: "b"/12.000000000,0 -> /BYTES/a +data: "k"/11.000000000,0 -> /BYTES/c +data: "k/10"/11.000000000,0 -> /BYTES/10 +data: "k/30"/11.000000000,0 -> /BYTES/30 +logical op: *enginepb.MVCCCommitIntentOp From bc25c51fc89c1c948df3b94838baf3c01ac3fb3b Mon Sep 17 00:00:00 2001 From: Mira Radeva Date: Tue, 9 Jan 2024 09:04:26 -0500 Subject: [PATCH 3/3] storage: fix a series of intent resolution bugs with ignored seq nums Previously, the logic in mvccResolveWriteIntent was structured in such a way that if an intent contained both ignored and non-ignored seq nums in its intent history, the intent may end up being updated instead of aborted or unmodified (see examples in 540efac). This commit fixes the bugs by ensuring that the intent history is modified only when an intent resolution update is not aborted, and the update and the actual intent have the same epoch. Fixes: #117553 Release note: None --- pkg/storage/mvcc.go | 15 ++++++---- .../testdata/mvcc_histories/ignored_seq_nums | 29 ++++++------------- .../mvcc_histories/ignored_seq_nums_abort | 11 ++----- .../mvcc_histories/ignored_seq_nums_commit | 7 +---- 4 files changed, 23 insertions(+), 39 deletions(-) diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index d47ac1e6e457..09fb0dfd7b1b 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -5179,7 +5179,13 @@ func mvccResolveWriteIntent( var rolledBackVal *MVCCValue buf.newMeta = *meta newMeta := &buf.newMeta - if len(update.IgnoredSeqNums) > 0 { + // Update the MVCC history only if: + // 1. There are IgnoredSeqNums present. + // 2. The update is not going to abort the intent; otherwise, the entire + // history will be removed anyway. + // 3. The epochs of the intent and the update match; otherwise the epochs may + // have different seq nums (and ignored seq nums). + if len(update.IgnoredSeqNums) > 0 && (commit || inProgress) && epochsMatch { // 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 @@ -5346,6 +5352,7 @@ func mvccResolveWriteIntent( // Update or remove the metadata key. var metaKeySize, metaValSize int64 + var logicalOp MVCCLogicalOpType if !commit { // Keep existing intent if we're updating it. We update the existing // metadata's timestamp instead of using the supplied intent meta to avoid @@ -5355,6 +5362,7 @@ func mvccResolveWriteIntent( outcome = lockOverwritten metaKeySize, metaValSize, err = buf.putLockMeta( writer, metaKey.Key, lock.Intent, newMeta, true /* alreadyExists */) + logicalOp = MVCCUpdateIntentOpType } else { outcome = lockClearedByDelete useSingleDelete := canSingleDelHelper.onCommitLock() @@ -5366,6 +5374,7 @@ func mvccResolveWriteIntent( ValueSizeKnown: true, ValueSize: uint32(origMetaValSize), }) + logicalOp = MVCCCommitIntentOpType } if err != nil { return lockNoop, err @@ -5378,10 +5387,6 @@ func mvccResolveWriteIntent( } // Log the logical MVCC operation. - logicalOp := MVCCCommitIntentOpType - if pushed { - logicalOp = MVCCUpdateIntentOpType - } writer.LogLogicalOp(logicalOp, MVCCLogicalOpDetails{ Txn: update.Txn, Key: update.Key, diff --git a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums index bc92ce8839f1..bc01389c4010 100644 --- a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums +++ b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums @@ -675,9 +675,6 @@ clear_range k=k end=p # Try to update a pending intent with a lower epoch than the resolution. # The intent should be aborted because the new epoch may not write it again. -# The test exposes a bug where the intent is updated instead of aborted. -# Moreover, it logs MVCCCommitIntentOp instead of MVCCUpdateIntentOp. - run ok with t=G k=p txn_begin ts=50 @@ -708,9 +705,8 @@ resolve_intent: "p" -> resolved key = true get: "p" -> >> at end: txn: "G" meta={id=00000007 key="p" iso=Serializable pri=0.00000000 epo=1 ts=50.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=50.000000000,0 wto=false gul=0,0 isn=1 -meta: "p"/0,0 -> txn={id=00000007 key="p" iso=Serializable pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=10} ts=50.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=false -data: "p"/50.000000000,0 -> /BYTES/a -logical op: *enginepb.MVCCCommitIntentOp + +logical op: *enginepb.MVCCAbortIntentOp run ok clear_range k=p end=-p @@ -722,10 +718,6 @@ clear_range k=p end=-p # The intent is not pushed, so it is not updated because the intent history # should be updated only when the epochs match. -# The test exposes a bug where the intent is actually updated, as evident by the -# changed intent history. -# Moreover, it logs MVCCCommitIntentOp instead of MVCCUpdateIntentOp. - run ok with t=H k=q txn_begin ts=50 @@ -753,13 +745,12 @@ with t=H k=q resolve_intent status=PENDING check_intent ---- -resolve_intent: "q" -> resolved key = true -meta: "q" -> txn={id=00000008 key="q" iso=Serializable pri=0.00000000 epo=1 ts=50.000000000,0 min=0,0 seq=10} ts=50.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=false +resolve_intent: "q" -> resolved key = false +meta: "q" -> txn={id=00000008 key="q" iso=Serializable pri=0.00000000 epo=1 ts=50.000000000,0 min=0,0 seq=20} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false >> at end: txn: "H" meta={id=00000008 key="q" iso=Serializable pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=50.000000000,0 wto=false gul=0,0 isn=1 -meta: "q"/0,0 -> txn={id=00000008 key="q" iso=Serializable pri=0.00000000 epo=1 ts=50.000000000,0 min=0,0 seq=10} ts=50.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=false -data: "q"/50.000000000,0 -> /BYTES/a -logical op: *enginepb.MVCCCommitIntentOp +meta: "q"/0,0 -> txn={id=00000008 key="q" iso=Serializable pri=0.00000000 epo=1 ts=50.000000000,0 min=0,0 seq=20} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false +data: "q"/50.000000000,0 -> /BYTES/b run ok clear_range k=q end=-q @@ -771,8 +762,6 @@ clear_range k=q end=-q # The intent is pushed, so its timestamp is updated to help the pusher make progress. # The intent history is not updated because the epochs don't match. -# The test exposes a bug where the intent history is actually updated. - run ok with t=I k=r txn_begin ts=50 @@ -802,9 +791,9 @@ with t=I k=r check_intent ---- resolve_intent: "r" -> resolved key = true -meta: "r" -> txn={id=00000009 key="r" iso=Serializable pri=0.00000000 epo=1 ts=60.000000000,0 min=0,0 seq=10} ts=60.000000000,0 del=false klen=12 vlen=20 mergeTs= txnDidNotUpdateMeta=false +meta: "r" -> txn={id=00000009 key="r" iso=Serializable pri=0.00000000 epo=1 ts=60.000000000,0 min=0,0 seq=20} ts=60.000000000,0 del=false klen=12 vlen=20 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false >> at end: txn: "I" meta={id=00000009 key="r" iso=Serializable pri=0.00000000 epo=0 ts=60.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=50.000000000,0 wto=false gul=0,0 isn=1 -meta: "r"/0,0 -> txn={id=00000009 key="r" iso=Serializable pri=0.00000000 epo=1 ts=60.000000000,0 min=0,0 seq=10} ts=60.000000000,0 del=false klen=12 vlen=20 mergeTs= txnDidNotUpdateMeta=false -data: "r"/60.000000000,0 -> {localTs=50.000000000,0}/BYTES/a +meta: "r"/0,0 -> txn={id=00000009 key="r" iso=Serializable pri=0.00000000 epo=1 ts=60.000000000,0 min=0,0 seq=20} ts=60.000000000,0 del=false klen=12 vlen=20 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false +data: "r"/60.000000000,0 -> {localTs=50.000000000,0}/BYTES/b logical op: *enginepb.MVCCUpdateIntentOp diff --git a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_abort b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_abort index 7467a3a51455..0fd2d6368926 100644 --- a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_abort +++ b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_abort @@ -2,10 +2,6 @@ # The test exposes a bug where even though the transaction is aborted, its # intent is updated instead of aborted. -# The test exposes a bug where even though the transaction is aborted, its -# intent is updated instead of aborted. -# Moreover, it logs MVCCCommitIntentOp instead of MVCCUpdateIntentOp. - run ok log-ops with t=A k=k txn_begin ts=11 @@ -21,11 +17,10 @@ with t=A k=k put: lock acquisition = {k id=00000001 key="k" iso=Serializable pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10 Replicated Intent []} put: lock acquisition = {k id=00000001 key="k" iso=Serializable pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20 Replicated Intent []} resolve_intent: "k" -> resolved key = true -get: "k" -> /BYTES/a @11.000000000,0 +get: "k" -> >> at end: txn: "A" meta={id=00000001 key="k" iso=Serializable pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=11.000000000,0 wto=false gul=0,0 isn=1 -meta: "k"/0,0 -> txn={id=00000001 key="k" iso=Serializable pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=false -data: "k"/11.000000000,0 -> /BYTES/a + logical op: *enginepb.MVCCWriteIntentOp logical op: *enginepb.MVCCUpdateIntentOp -logical op: *enginepb.MVCCCommitIntentOp +logical op: *enginepb.MVCCAbortIntentOp diff --git a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_commit b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_commit index 860f115b052d..d84ff0a7066d 100644 --- a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_commit +++ b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_commit @@ -142,9 +142,6 @@ get: "k" -> # Commit an intent with a lower epoch than the resolution. # The intent should be aborted because the new epoch may not write it again. -# The test exposes a bug where the intent is updated instead of aborted. -# Moreover, it logs MVCCCommitIntentOp instead of MVCCUpdateIntentOp. - run ok with t=B k=b txn_begin ts=12 @@ -179,9 +176,7 @@ resolve_intent: "b" -> resolved key = true get: "b" -> >> at end: txn: "B" meta={id=00000003 key="b" iso=Serializable pri=0.00000000 epo=1 ts=12.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=12.000000000,0 wto=false gul=0,0 isn=1 -meta: "b"/0,0 -> txn={id=00000003 key="b" iso=Serializable pri=0.00000000 epo=0 ts=12.000000000,0 min=0,0 seq=10} ts=12.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=false -data: "b"/12.000000000,0 -> /BYTES/a data: "k"/11.000000000,0 -> /BYTES/c data: "k/10"/11.000000000,0 -> /BYTES/10 data: "k/30"/11.000000000,0 -> /BYTES/30 -logical op: *enginepb.MVCCCommitIntentOp +logical op: *enginepb.MVCCAbortIntentOp