Skip to content

Commit

Permalink
kv: don't rewrite txn record on PushTxn(TIMESTAMP)
Browse files Browse the repository at this point in the history
Fixes cockroachdb#94728.

With the previous commit, transactions will check the timestamp cache
before committing to determine whether they have had their commit
timestamp pushed. This commit exploits this to avoid ever rewriting a
transaction's record on a timestamp push. Instead, the timestamp cache
is used, regardless of whether the record already existed or not. Doing
so avoids consensus.
  • Loading branch information
nvanbenschoten committed Jan 26, 2023
1 parent bc97890 commit 9128ec7
Show file tree
Hide file tree
Showing 2 changed files with 34 additions and 27 deletions.
50 changes: 29 additions & 21 deletions pkg/kv/kvserver/batcheval/cmd_push_txn.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import (
"context"
"time"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset"
Expand Down Expand Up @@ -298,40 +299,47 @@ func PushTxn(
case roachpb.PUSH_ABORT:
// If aborting the transaction, set the new status.
reply.PusheeTxn.Status = roachpb.ABORTED
// If the transaction record was already present, forward the timestamp
// to accommodate AbortSpan GC. See method comment for details.
// Forward the timestamp to accommodate AbortSpan GC. See method comment for
// details.
reply.PusheeTxn.WriteTimestamp.Forward(reply.PusheeTxn.LastActive())
// If the transaction record was already present, persist the updates to it.
// If not, then we don't want to create it. This could allow for finalized
// transactions to be revived. Instead, we obey the invariant that only the
// transaction's own coordinator can issue requests that create its
// transaction record. To ensure that a timestamp push or an abort is
// respected for transactions without transaction records, we rely on markers
// in the timestamp cache.
if ok {
reply.PusheeTxn.WriteTimestamp.Forward(reply.PusheeTxn.LastActive())
txnRecord := reply.PusheeTxn.AsRecord()
if err := storage.MVCCPutProto(ctx, readWriter, cArgs.Stats, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord); err != nil {
return result.Result{}, err
}
}
case roachpb.PUSH_TIMESTAMP:
if existTxn.Status != roachpb.PENDING {
return result.Result{}, errors.AssertionFailedf(
"PUSH_TIMESTAMP succeeded against non-PENDING txn: %v", existTxn)
}
// Otherwise, update timestamp to be one greater than the request's
// timestamp. This new timestamp will be use to update the read timestamp
// cache. If the transaction record was not already present then we rely on
// the timestamp cache to prevent the record from ever being written with a
// timestamp beneath this timestamp.
// timestamp. This new timestamp will be used to update the read timestamp
// cache. We rely on the timestamp cache to prevent the record from ever
// being committed with a timestamp beneath this timestamp.
reply.PusheeTxn.WriteTimestamp.Forward(args.PushTo)
// If the transaction record was already present, continue to update the
// transaction record until all nodes are running v23.1. v22.2 nodes won't
// know to check the timestamp cache again on commit to learn about any
// successful timestamp pushes.
// TODO(nvanbenschoten): remove this logic in v23.2.
if ok && !cArgs.EvalCtx.ClusterSettings().Version.IsActive(ctx, clusterversion.V23_1) {
txnRecord := reply.PusheeTxn.AsRecord()
if err := storage.MVCCPutProto(ctx, readWriter, cArgs.Stats, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord); err != nil {
return result.Result{}, err
}
}
default:
return result.Result{}, errors.AssertionFailedf("unexpected push type: %v", pushType)
}

// If the transaction record was already present, persist the updates to it.
// If not, then we don't want to create it. This could allow for finalized
// transactions to be revived. Instead, we obey the invariant that only the
// transaction's own coordinator can issue requests that create its
// transaction record. To ensure that a timestamp push or an abort is
// respected for transactions without transaction records, we rely on markers
// in the timestamp cache.
if ok {
txnRecord := reply.PusheeTxn.AsRecord()
if err := storage.MVCCPutProto(ctx, readWriter, cArgs.Stats, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord); err != nil {
return result.Result{}, err
}
}

result := result.Result{}
result.Local.UpdatedTxns = []*roachpb.Transaction{&reply.PusheeTxn}
return result, nil
Expand Down
11 changes: 5 additions & 6 deletions pkg/kv/kvserver/replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -11991,12 +11991,11 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) {
pt.PushTo = now
return sendWrappedWithErr(roachpb.Header{}, &pt)
},
expTxn: func(txn *roachpb.Transaction, pushTs hlc.Timestamp) roachpb.TransactionRecord {
record := txn.AsRecord()
record.WriteTimestamp.Forward(pushTs)
record.Priority = pusher.Priority - 1
return record
},
// The transaction record **is not** updated in this case. Instead, the
// push is communicated through the timestamp cache. When the pushee goes
// to commit, it will consult the timestamp cache and find that it must
// commit above the push timestamp.
expTxn: txnWithoutChanges,
},
{
name: "push transaction (abort) after heartbeat transaction",
Expand Down

0 comments on commit 9128ec7

Please sign in to comment.