Skip to content

Commit

Permalink
storage: break another dependency on intent interleaving in mvccPutIn…
Browse files Browse the repository at this point in the history
…ternal

This commit breaks another dependency on intent interleaving in
mvccPutInternal's call to replayTransactionalWrite. In this case,
we know the provisional value's timestamp, so we should use this
to read instead of the writer's read timestamp.

This was caught by the test:
```
TestReplicaTxnIdempotency/reissued_write_at_lower_timestamp
```

Release note: None
  • Loading branch information
nvanbenschoten committed Sep 12, 2023
1 parent 938b9a9 commit 6426a0c
Showing 1 changed file with 5 additions and 5 deletions.
10 changes: 5 additions & 5 deletions pkg/storage/mvcc.go
Original file line number Diff line number Diff line change
Expand Up @@ -1705,7 +1705,6 @@ func replayTransactionalWrite(
iter MVCCIterator,
meta *enginepb.MVCCMetadata,
key roachpb.Key,
timestamp hlc.Timestamp,
value roachpb.Value,
txn *roachpb.Transaction,
valueFn func(optionalValue) (roachpb.Value, error),
Expand All @@ -1717,7 +1716,10 @@ func replayTransactionalWrite(
// This is a special case. This is when the intent hasn't made it
// to the intent history yet. We must now assert the value written
// in the intent to the value we're trying to write.
writtenValue, _, err = mvccGet(ctx, iter, key, timestamp, MVCCGetOptions{
writtenValue, _, err = mvccGet(ctx, iter, key, meta.Timestamp.ToTimestamp(), MVCCGetOptions{
// NOTE: we pass Txn here to ensure that this read succeeds even if
// iter is interleaving intents. This is not needed if iter is a raw
// MVCCIterator.
Txn: txn,
Tombstones: true,
})
Expand Down Expand Up @@ -1979,7 +1981,7 @@ func mvccPutInternal(
// The transaction has executed at this sequence before. This is merely a
// replay of the transactional write. Assert that all is in order and return
// early.
return false, replayTransactionalWrite(ctx, iter, meta, key, readTimestamp, value, opts.Txn, valueFn, opts.ReplayWriteTimestampProtection)
return false, replayTransactionalWrite(ctx, iter, meta, key, value, opts.Txn, valueFn, opts.ReplayWriteTimestampProtection)
}

// We're overwriting the intent that was present at this key, before we do
Expand Down Expand Up @@ -2192,8 +2194,6 @@ func mvccPutInternal(
if valueFn != nil {
exVal, _, err := mvccGet(ctx, iter, key, readTimestamp, MVCCGetOptions{
Tombstones: true,
// Unlike above, we know that there are no intents on this
// key, so we don't need to perform an inconsistent read.
})
if err != nil {
return false, err
Expand Down

0 comments on commit 6426a0c

Please sign in to comment.