diff --git a/pkg/cli/debug.go b/pkg/cli/debug.go index 3f0d8f65699e..f7e80d7d7eed 100644 --- a/pkg/cli/debug.go +++ b/pkg/cli/debug.go @@ -1127,7 +1127,7 @@ func removeDeadReplicas( return nil, err } intent.Status = roachpb.ABORTED - if err := engine.MVCCResolveWriteIntent(ctx, batch, &ms, intent); err != nil { + if _, err := engine.MVCCResolveWriteIntent(ctx, batch, &ms, intent); err != nil { return nil, err } // With the intent resolved, we can try again. diff --git a/pkg/storage/batcheval/cmd_end_transaction.go b/pkg/storage/batcheval/cmd_end_transaction.go index f208e3a98c99..363a072b87d1 100644 --- a/pkg/storage/batcheval/cmd_end_transaction.go +++ b/pkg/storage/batcheval/cmd_end_transaction.go @@ -473,8 +473,11 @@ func resolveLocalIntents( return nil } resolveMS := ms - resolveAllowance-- - return engine.MVCCResolveWriteIntentUsingIter(ctx, batch, iterAndBuf, resolveMS, intent) + ok, err := engine.MVCCResolveWriteIntentUsingIter(ctx, batch, iterAndBuf, resolveMS, intent) + if ok { + resolveAllowance-- + } + return err } // For intent ranges, cut into parts inside and outside our key // range. Resolve locally inside, delegate the rest. In particular, @@ -505,7 +508,8 @@ func resolveLocalIntents( } } - if WriteAbortSpanOnResolve(txn.Status) { + removedAny := resolveAllowance != intentResolutionBatchSize + if WriteAbortSpanOnResolve(txn.Status, args.Poison, removedAny) { if err := SetAbortSpan(ctx, evalCtx, batch, ms, txn.TxnMeta, args.Poison); err != nil { return nil, err } diff --git a/pkg/storage/batcheval/cmd_refresh_range_test.go b/pkg/storage/batcheval/cmd_refresh_range_test.go index 34f56d6a0dd6..33294ca6bc32 100644 --- a/pkg/storage/batcheval/cmd_refresh_range_test.go +++ b/pkg/storage/batcheval/cmd_refresh_range_test.go @@ -83,7 +83,7 @@ func TestRefreshRangeTimeBoundIterator(t *testing.T) { // (committed). The sstable also has a second write at a different (older) // timestamp, because if it were empty other than the deletion tombstone, it // would not have any timestamp bounds and would be selected for every read. - if err := engine.MVCCResolveWriteIntent(ctx, db, nil, roachpb.Intent{ + if _, err := engine.MVCCResolveWriteIntent(ctx, db, nil, roachpb.Intent{ Span: roachpb.Span{Key: k}, Txn: txn.TxnMeta, Status: roachpb.COMMITTED, diff --git a/pkg/storage/batcheval/cmd_resolve_intent.go b/pkg/storage/batcheval/cmd_resolve_intent.go index b88dbd998db6..65efe5581fb6 100644 --- a/pkg/storage/batcheval/cmd_resolve_intent.go +++ b/pkg/storage/batcheval/cmd_resolve_intent.go @@ -39,7 +39,7 @@ func declareKeysResolveIntentCombined( status = t.Status txnID = t.IntentTxn.ID } - if WriteAbortSpanOnResolve(status) { + if status == roachpb.ABORTED { spans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.AbortSpanKey(header.RangeID, txnID)}) } } @@ -52,7 +52,7 @@ func declareKeysResolveIntent( func resolveToMetricType(status roachpb.TransactionStatus, poison bool) *result.Metrics { var typ result.Metrics - if WriteAbortSpanOnResolve(status) { + if status == roachpb.ABORTED { if poison { typ.ResolvePoison = 1 } else { @@ -82,14 +82,15 @@ func ResolveIntent( Txn: args.IntentTxn, Status: args.Status, } - if err := engine.MVCCResolveWriteIntent(ctx, batch, ms, intent); err != nil { + ok, err := engine.MVCCResolveWriteIntent(ctx, batch, ms, intent) + if err != nil { return result.Result{}, err } var res result.Result res.Local.Metrics = resolveToMetricType(args.Status, args.Poison) - if WriteAbortSpanOnResolve(args.Status) { + if WriteAbortSpanOnResolve(args.Status, args.Poison, ok) { if err := SetAbortSpan(ctx, cArgs.EvalCtx, batch, ms, args.IntentTxn, args.Poison); err != nil { return result.Result{}, err } diff --git a/pkg/storage/batcheval/cmd_resolve_intent_range.go b/pkg/storage/batcheval/cmd_resolve_intent_range.go index fce07a5c5ab3..39306ddf5f5d 100644 --- a/pkg/storage/batcheval/cmd_resolve_intent_range.go +++ b/pkg/storage/batcheval/cmd_resolve_intent_range.go @@ -67,7 +67,7 @@ func ResolveIntentRange( var res result.Result res.Local.Metrics = resolveToMetricType(args.Status, args.Poison) - if WriteAbortSpanOnResolve(args.Status) { + if WriteAbortSpanOnResolve(args.Status, args.Poison, numKeys > 0) { if err := SetAbortSpan(ctx, cArgs.EvalCtx, batch, ms, args.IntentTxn, args.Poison); err != nil { return result.Result{}, err } diff --git a/pkg/storage/batcheval/transaction.go b/pkg/storage/batcheval/transaction.go index 0f7a681f93ae..48f923c59747 100644 --- a/pkg/storage/batcheval/transaction.go +++ b/pkg/storage/batcheval/transaction.go @@ -54,9 +54,25 @@ func VerifyTransaction( } // WriteAbortSpanOnResolve returns true if the abort span must be written when -// the transaction with the given status is resolved. -func WriteAbortSpanOnResolve(status roachpb.TransactionStatus) bool { - return status == roachpb.ABORTED +// the transaction with the given status is resolved. It avoids instructing the +// caller to write to the abort span if the caller didn't actually remove any +// intents but intends to poison. +func WriteAbortSpanOnResolve(status roachpb.TransactionStatus, poison, removedIntents bool) bool { + if status != roachpb.ABORTED { + // Only update the AbortSpan for aborted transactions, + return false + } + if !poison { + // We can remove any entries from the AbortSpan. + return true + } + // We only need to add AbortSpan entries for transactions that we have + // invalidated by removing intents. This avoids leaking AbortSpan entries if + // a request raced with txn record GC and mistakenly interpreted a committed + // txn as aborted only to return to the intent it wanted to push and find it + // already resolved. We're only required to write an entry if we do + // something that could confuse/invalidate a zombie transaction. + return removedIntents } // SetAbortSpan clears any AbortSpan entry if poison is false. diff --git a/pkg/storage/batcheval/transaction_test.go b/pkg/storage/batcheval/transaction_test.go index f46ecc6d7a21..5bfa10ee47df 100644 --- a/pkg/storage/batcheval/transaction_test.go +++ b/pkg/storage/batcheval/transaction_test.go @@ -12,6 +12,7 @@ package batcheval import ( "context" + "math" "testing" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -126,6 +127,7 @@ func TestSetAbortSpan(t *testing.T) { args := CommandArgs{ EvalCtx: rec, Args: &req, + MaxKeys: math.MaxInt64, } var resp roachpb.ResolveIntentRangeResponse @@ -143,7 +145,7 @@ func TestSetAbortSpan(t *testing.T) { // EndTransactionRequest // /////////////////////////////////////////////////////////////////////// { - name: "end txn, rollback, no poison, abort span missing", + name: "end txn, rollback, no poison, intent missing, abort span missing", run: func(b engine.ReadWriter, rec EvalContext) error { return endTxn(b, rec, false /* commit */, false /* poison */) }, @@ -151,7 +153,7 @@ func TestSetAbortSpan(t *testing.T) { exp: nil, }, { - name: "end txn, rollback, no poison, abort span present", + name: "end txn, rollback, no poison, intent missing, abort span present", before: addAbortSpanEntry, run: func(b engine.ReadWriter, rec EvalContext) error { return endTxn(b, rec, false /* commit */, false /* poison */) @@ -160,7 +162,83 @@ func TestSetAbortSpan(t *testing.T) { exp: nil, }, { - name: "end txn, rollback, poison, abort span missing", + name: "end txn, rollback, poison, intent missing, abort span missing", + run: func(b engine.ReadWriter, rec EvalContext) error { + return endTxn(b, rec, false /* commit */, true /* poison */) + }, + // Poisoning, but no intents found, should not add an abort span entry. + exp: nil, + }, + { + name: "end txn, rollback, poison, intent missing, abort span present", + before: addAbortSpanEntry, + run: func(b engine.ReadWriter, rec EvalContext) error { + return endTxn(b, rec, false /* commit */, true /* poison */) + }, + // Poisoning, but no intents found, don't touch abort span. + exp: &txnPrevAbortSpanEntry, + }, + { + name: "end txn, commit, no poison, intent missing, abort span missing", + run: func(b engine.ReadWriter, rec EvalContext) error { + return endTxn(b, rec, true /* commit */, false /* poison */) + }, + // Not poisoning, should not add an abort span entry. + exp: nil, + }, + { + // NOTE: this request doesn't make sense, but we handle it. An abort + // span shouldn't be present if the transaction is still committable. + name: "end txn, commit, no poison, intent missing, abort span present", + before: addAbortSpanEntry, + run: func(b engine.ReadWriter, rec EvalContext) error { + return endTxn(b, rec, true /* commit */, false /* poison */) + }, + // Not aborted, don't touch abort span. + exp: &txnPrevAbortSpanEntry, + }, + { + // NOTE: this request doesn't make sense, but we handle it. An EndTxn + // should never pass Commit = true and Poison = true. + name: "end txn, commit, poison, intent missing, abort span missing", + run: func(b engine.ReadWriter, rec EvalContext) error { + return endTxn(b, rec, true /* commit */, true /* poison */) + }, + // Poisoning but not aborted, should not add an abort span entry. + exp: nil, + }, + { + // NOTE: this request doesn't make sense, but we handle it. An EndTxn + // should never pass Commit = true and Poison = true. + name: "end txn, commit, poison, intent missing, abort span present", + before: addAbortSpanEntry, + run: func(b engine.ReadWriter, rec EvalContext) error { + return endTxn(b, rec, true /* commit */, true /* poison */) + }, + // Not aborted, don't touch abort span. + exp: &txnPrevAbortSpanEntry, + }, + { + name: "end txn, rollback, no poison, intent present, abort span missing", + before: addIntent, + run: func(b engine.ReadWriter, rec EvalContext) error { + return endTxn(b, rec, false /* commit */, false /* poison */) + }, + // Not poisoning, should not add an abort span entry. + exp: nil, + }, + { + name: "end txn, rollback, no poison, intent present, abort span present", + before: compose(addIntent, addAbortSpanEntry), + run: func(b engine.ReadWriter, rec EvalContext) error { + return endTxn(b, rec, false /* commit */, false /* poison */) + }, + // Not poisoning, should clean up abort span entry. + exp: nil, + }, + { + name: "end txn, rollback, poison, intent present, abort span missing", + before: addIntent, run: func(b engine.ReadWriter, rec EvalContext) error { return endTxn(b, rec, false /* commit */, true /* poison */) }, @@ -168,8 +246,8 @@ func TestSetAbortSpan(t *testing.T) { exp: &txnAbortSpanEntry, }, { - name: "end txn, rollback, poison, abort span present", - before: addAbortSpanEntry, + name: "end txn, rollback, poison, intent present, abort span present", + before: compose(addIntent, addAbortSpanEntry), run: func(b engine.ReadWriter, rec EvalContext) error { return endTxn(b, rec, false /* commit */, true /* poison */) }, @@ -177,7 +255,8 @@ func TestSetAbortSpan(t *testing.T) { exp: &txnAbortSpanEntry, }, { - name: "end txn, commit, no poison, abort span missing", + name: "end txn, commit, no poison, intent present, abort span missing", + before: addIntent, run: func(b engine.ReadWriter, rec EvalContext) error { return endTxn(b, rec, true /* commit */, false /* poison */) }, @@ -187,8 +266,8 @@ func TestSetAbortSpan(t *testing.T) { { // NOTE: this request doesn't make sense, but we handle it. An abort // span shouldn't be present if the transaction is still committable. - name: "end txn, commit, no poison, abort span present", - before: addAbortSpanEntry, + name: "end txn, commit, no poison, intent present, abort span present", + before: compose(addIntent, addAbortSpanEntry), run: func(b engine.ReadWriter, rec EvalContext) error { return endTxn(b, rec, true /* commit */, false /* poison */) }, @@ -198,7 +277,8 @@ func TestSetAbortSpan(t *testing.T) { { // NOTE: this request doesn't make sense, but we handle it. An EndTxn // should never pass Commit = true and Poison = true. - name: "end txn, commit, poison, abort span missing", + name: "end txn, commit, poison, intent present, abort span missing", + before: addIntent, run: func(b engine.ReadWriter, rec EvalContext) error { return endTxn(b, rec, true /* commit */, true /* poison */) }, @@ -208,8 +288,8 @@ func TestSetAbortSpan(t *testing.T) { { // NOTE: this request doesn't make sense, but we handle it. An EndTxn // should never pass Commit = true and Poison = true. - name: "end txn, commit, poison, abort span present", - before: addAbortSpanEntry, + name: "end txn, commit, poison, intent present, abort span present", + before: compose(addIntent, addAbortSpanEntry), run: func(b engine.ReadWriter, rec EvalContext) error { return endTxn(b, rec, true /* commit */, true /* poison */) }, @@ -333,8 +413,8 @@ func TestSetAbortSpan(t *testing.T) { run: func(b engine.ReadWriter, rec EvalContext) error { return resolveIntent(b, rec, roachpb.ABORTED, true /* poison */) }, - // Poisoning, should add an abort span entry. - exp: &txnAbortSpanEntry, + // Poisoning, but no intents found, should not add an abort span entry. + exp: nil, }, { name: "resolve intent, txn aborted, poison, intent missing, abort span present", @@ -342,8 +422,8 @@ func TestSetAbortSpan(t *testing.T) { run: func(b engine.ReadWriter, rec EvalContext) error { return resolveIntent(b, rec, roachpb.ABORTED, true /* poison */) }, - // Poisoning, should update abort span entry. - exp: &txnAbortSpanEntry, + // Poisoning, but no intents found, don't touch abort span. + exp: &txnPrevAbortSpanEntry, }, { name: "resolve intent, txn aborted, poison, intent present, abort span missing", @@ -560,8 +640,8 @@ func TestSetAbortSpan(t *testing.T) { run: func(b engine.ReadWriter, rec EvalContext) error { return resolveIntentRange(b, rec, roachpb.ABORTED, true /* poison */) }, - // Poisoning, should add an abort span entry. - exp: &txnAbortSpanEntry, + // Poisoning, but no intents found, should not add an abort span entry. + exp: nil, }, { name: "resolve intent range, txn aborted, poison, intent missing, abort span present", @@ -569,8 +649,8 @@ func TestSetAbortSpan(t *testing.T) { run: func(b engine.ReadWriter, rec EvalContext) error { return resolveIntentRange(b, rec, roachpb.ABORTED, true /* poison */) }, - // Poisoning, should update abort span entry. - exp: &txnAbortSpanEntry, + // Poisoning, but no intents found, don't touch abort span. + exp: &txnPrevAbortSpanEntry, }, { name: "resolve intent range, txn aborted, poison, intent present, abort span missing", diff --git a/pkg/storage/client_split_test.go b/pkg/storage/client_split_test.go index 72b988746db2..c7de17156849 100644 --- a/pkg/storage/client_split_test.go +++ b/pkg/storage/client_split_test.go @@ -126,6 +126,17 @@ func TestStoreSplitAbortSpan(t *testing.T) { populateAbortSpan := func(key roachpb.Key, ts hlc.Timestamp) *roachpb.ResolveIntentRequest { pushee := txn(key, ts) + + // First write an intent on the key... + incArgs := incrementArgs(key, 1) + _, pErr := client.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{Txn: pushee}, incArgs) + if pErr != nil { + t.Fatalf("while sending +%v: %s", incArgs, pErr) + } + + // Then resolve the intent and poison. Without the intent write, the + // intent resolution would be a no-op and wouldn't leave an AbortSpan + // entry. expAll = append(expAll, roachpb.AbortSpanEntry{ Key: key, Timestamp: ts, @@ -185,7 +196,7 @@ func TestStoreSplitAbortSpan(t *testing.T) { } for _, arg := range args { - _, pErr := client.SendWrapped(context.Background(), store.TestSender(), arg) + _, pErr := client.SendWrapped(ctx, store.TestSender(), arg) if pErr != nil { t.Fatalf("while sending +%v: %s", arg, pErr) } diff --git a/pkg/storage/engine/bench_test.go b/pkg/storage/engine/bench_test.go index 8aa037104827..10d27da70e8e 100644 --- a/pkg/storage/engine/bench_test.go +++ b/pkg/storage/engine/bench_test.go @@ -137,7 +137,7 @@ func setupMVCCData( key := keys[idx] txnMeta := txn.TxnMeta txnMeta.WriteTimestamp = hlc.Timestamp{WallTime: int64(counts[idx]) * 5} - if err := MVCCResolveWriteIntent(ctx, batch, nil /* ms */, roachpb.Intent{ + if _, err := MVCCResolveWriteIntent(ctx, batch, nil /* ms */, roachpb.Intent{ Span: roachpb.Span{Key: key}, Status: roachpb.COMMITTED, Txn: txnMeta, diff --git a/pkg/storage/engine/mvcc.go b/pkg/storage/engine/mvcc.go index 161aeaae463c..ff0451899495 100644 --- a/pkg/storage/engine/mvcc.go +++ b/pkg/storage/engine/mvcc.go @@ -2506,7 +2506,8 @@ func MVCCIterate( // MVCCResolveWriteIntent either commits or aborts (rolls back) an // extant write intent for a given txn according to commit parameter. -// ResolveWriteIntent will skip write intents of other txns. +// ResolveWriteIntent will skip write intents of other txns. It returns +// whether or not an intent was found to resolve. // // Transaction epochs deserve a bit of explanation. The epoch for a // transaction is incremented on transaction retries. A transaction @@ -2530,12 +2531,12 @@ func MVCCIterate( // when they're not at the timestamp the Txn mandates them to be. func MVCCResolveWriteIntent( ctx context.Context, engine ReadWriter, ms *enginepb.MVCCStats, intent roachpb.Intent, -) error { +) (bool, error) { iterAndBuf := GetBufUsingIter(engine.NewIterator(IterOptions{Prefix: true})) - err := MVCCResolveWriteIntentUsingIter(ctx, engine, iterAndBuf, ms, intent) + ok, err := MVCCResolveWriteIntentUsingIter(ctx, engine, iterAndBuf, ms, intent) // Using defer would be more convenient, but it is measurably slower. iterAndBuf.Cleanup() - return err + return ok, err } // MVCCResolveWriteIntentUsingIter is a variant of MVCCResolveWriteIntent that @@ -2546,17 +2547,16 @@ func MVCCResolveWriteIntentUsingIter( iterAndBuf IterAndBuf, ms *enginepb.MVCCStats, intent roachpb.Intent, -) error { +) (bool, error) { if len(intent.Key) == 0 { - return emptyKeyError() + return false, emptyKeyError() } if len(intent.EndKey) > 0 { - return errors.Errorf("can't resolve range intent as point intent") + return false, errors.Errorf("can't resolve range intent as point intent") } - _, err := mvccResolveWriteIntent( + return mvccResolveWriteIntent( ctx, engine, iterAndBuf.iter, ms, intent, iterAndBuf.buf, false, /* forRange */ ) - return err } // unsafeNextVersion positions the iterator at the successor to latestKey. If this value diff --git a/pkg/storage/engine/mvcc_incremental_iterator_test.go b/pkg/storage/engine/mvcc_incremental_iterator_test.go index 6dd3aed7e05e..7e7c9ea6b4ec 100644 --- a/pkg/storage/engine/mvcc_incremental_iterator_test.go +++ b/pkg/storage/engine/mvcc_incremental_iterator_test.go @@ -207,11 +207,11 @@ func TestMVCCIncrementalIterator(t *testing.T) { t.Run("intents", assertEqualKVs(e, fn, keyMin, keyMax, ts4.Next(), tsMax, kvs())) intent1 := roachpb.Intent{Span: roachpb.Span{Key: testKey1}, Txn: txn1.TxnMeta, Status: roachpb.COMMITTED} - if err := MVCCResolveWriteIntent(ctx, e, nil, intent1); err != nil { + if _, err := MVCCResolveWriteIntent(ctx, e, nil, intent1); err != nil { t.Fatal(err) } intent2 := roachpb.Intent{Span: roachpb.Span{Key: testKey2}, Txn: txn2.TxnMeta, Status: roachpb.ABORTED} - if err := MVCCResolveWriteIntent(ctx, e, nil, intent2); err != nil { + if _, err := MVCCResolveWriteIntent(ctx, e, nil, intent2); err != nil { t.Fatal(err) } t.Run("intents", assertEqualKVs(e, fn, keyMin, keyMax, tsMin, tsMax, kvs(kv1_4_4, kv2_2_2))) @@ -295,11 +295,11 @@ func TestMVCCIncrementalIterator(t *testing.T) { t.Run("intents", assertEqualKVs(e, fn, keyMin, keyMax, ts4.Next(), tsMax, kvs())) intent1 := roachpb.Intent{Span: roachpb.Span{Key: testKey1}, Txn: txn1.TxnMeta, Status: roachpb.COMMITTED} - if err := MVCCResolveWriteIntent(ctx, e, nil, intent1); err != nil { + if _, err := MVCCResolveWriteIntent(ctx, e, nil, intent1); err != nil { t.Fatal(err) } intent2 := roachpb.Intent{Span: roachpb.Span{Key: testKey2}, Txn: txn2.TxnMeta, Status: roachpb.ABORTED} - if err := MVCCResolveWriteIntent(ctx, e, nil, intent2); err != nil { + if _, err := MVCCResolveWriteIntent(ctx, e, nil, intent2); err != nil { t.Fatal(err) } t.Run("intents", assertEqualKVs(e, fn, keyMin, keyMax, tsMin, tsMax, kvs(kv1_4_4, kv1_3Deleted, kv1_2_2, kv1_1_1, kv2_2_2))) diff --git a/pkg/storage/engine/mvcc_logical_ops_test.go b/pkg/storage/engine/mvcc_logical_ops_test.go index e9514faa0e8b..c953f012aae8 100644 --- a/pkg/storage/engine/mvcc_logical_ops_test.go +++ b/pkg/storage/engine/mvcc_logical_ops_test.go @@ -97,7 +97,7 @@ func TestMVCCOpLogWriter(t *testing.T) { } txn2Pushed := *txn2 txn2Pushed.WriteTimestamp = hlc.Timestamp{Logical: 6} - if err := MVCCResolveWriteIntent(ctx, ol, nil, roachpb.Intent{ + if _, err := MVCCResolveWriteIntent(ctx, ol, nil, roachpb.Intent{ Span: roachpb.Span{Key: testKey3}, Txn: txn2Pushed.TxnMeta, Status: txn2Pushed.Status, @@ -106,7 +106,7 @@ func TestMVCCOpLogWriter(t *testing.T) { } txn2Abort := txn2Pushed txn2Abort.Status = roachpb.ABORTED - if err := MVCCResolveWriteIntent(ctx, ol, nil, roachpb.Intent{ + if _, err := MVCCResolveWriteIntent(ctx, ol, nil, roachpb.Intent{ Span: roachpb.Span{Key: testKey3}, Txn: txn2Abort.TxnMeta, Status: txn2Abort.Status, diff --git a/pkg/storage/engine/mvcc_stats_test.go b/pkg/storage/engine/mvcc_stats_test.go index 9b44f9802b2d..b017b02d6172 100644 --- a/pkg/storage/engine/mvcc_stats_test.go +++ b/pkg/storage/engine/mvcc_stats_test.go @@ -112,7 +112,7 @@ func TestMVCCStatsDeleteCommitMovesTimestamp(t *testing.T) { ts4 := hlc.Timestamp{WallTime: 4 * 1e9} txn.Status = roachpb.COMMITTED txn.WriteTimestamp.Forward(ts4) - if err := MVCCResolveWriteIntent(ctx, engine, aggMS, roachpb.Intent{ + if _, err := MVCCResolveWriteIntent(ctx, engine, aggMS, roachpb.Intent{ Span: roachpb.Span{Key: key}, Status: txn.Status, Txn: txn.TxnMeta, }); err != nil { t.Fatal(err) @@ -192,7 +192,7 @@ func TestMVCCStatsPutCommitMovesTimestamp(t *testing.T) { ts4 := hlc.Timestamp{WallTime: 4 * 1e9} txn.Status = roachpb.COMMITTED txn.WriteTimestamp.Forward(ts4) - if err := MVCCResolveWriteIntent(ctx, engine, aggMS, roachpb.Intent{ + if _, err := MVCCResolveWriteIntent(ctx, engine, aggMS, roachpb.Intent{ Span: roachpb.Span{Key: key}, Status: txn.Status, Txn: txn.TxnMeta, }); err != nil { t.Fatal(err) @@ -270,7 +270,7 @@ func TestMVCCStatsPutPushMovesTimestamp(t *testing.T) { // push as it would happen for a SNAPSHOT txn) ts4 := hlc.Timestamp{WallTime: 4 * 1e9} txn.WriteTimestamp.Forward(ts4) - if err := MVCCResolveWriteIntent(ctx, engine, aggMS, roachpb.Intent{ + if _, err := MVCCResolveWriteIntent(ctx, engine, aggMS, roachpb.Intent{ Span: roachpb.Span{Key: key}, Status: txn.Status, Txn: txn.TxnMeta, }); err != nil { t.Fatal(err) @@ -605,7 +605,7 @@ func TestMVCCStatsDelDelCommitMovesTimestamp(t *testing.T) { txnCommit := txn.Clone() txnCommit.Status = roachpb.COMMITTED txnCommit.WriteTimestamp.Forward(ts3) - if err := MVCCResolveWriteIntent(ctx, engine, &aggMS, roachpb.Intent{ + if _, err := MVCCResolveWriteIntent(ctx, engine, &aggMS, roachpb.Intent{ Span: roachpb.Span{Key: key}, Status: txnCommit.Status, Txn: txnCommit.TxnMeta, }); err != nil { t.Fatal(err) @@ -634,7 +634,7 @@ func TestMVCCStatsDelDelCommitMovesTimestamp(t *testing.T) { txnAbort := txn.Clone() txnAbort.Status = roachpb.ABORTED txnAbort.WriteTimestamp.Forward(ts3) - if err := MVCCResolveWriteIntent(ctx, engine, &aggMS, roachpb.Intent{ + if _, err := MVCCResolveWriteIntent(ctx, engine, &aggMS, roachpb.Intent{ Span: roachpb.Span{Key: key}, Status: txnAbort.Status, Txn: txnAbort.TxnMeta, }); err != nil { t.Fatal(err) @@ -764,7 +764,7 @@ func TestMVCCStatsPutDelPutMovesTimestamp(t *testing.T) { txnAbort := txn.Clone() txnAbort.Status = roachpb.ABORTED // doesn't change m2ValSize, fortunately - if err := MVCCResolveWriteIntent(ctx, engine, &aggMS, roachpb.Intent{ + if _, err := MVCCResolveWriteIntent(ctx, engine, &aggMS, roachpb.Intent{ Span: roachpb.Span{Key: key}, Status: txnAbort.Status, Txn: txnAbort.TxnMeta, }); err != nil { t.Fatal(err) @@ -1235,7 +1235,7 @@ func TestMVCCStatsTxnSysPutAbort(t *testing.T) { // Now abort the intent. txn.Status = roachpb.ABORTED - if err := MVCCResolveWriteIntent(ctx, engine, aggMS, roachpb.Intent{ + if _, err := MVCCResolveWriteIntent(ctx, engine, aggMS, roachpb.Intent{ Span: roachpb.Span{Key: key}, Status: txn.Status, Txn: txn.TxnMeta, }); err != nil { t.Fatal(err) @@ -1481,7 +1481,7 @@ func TestMVCCStatsRandomized(t *testing.T) { desc := fmt.Sprintf("ranged=%t", ranged) if s.Txn != nil { if !ranged { - if err := MVCCResolveWriteIntent(ctx, s.eng, s.MS, s.intent(status)); err != nil { + if _, err := MVCCResolveWriteIntent(ctx, s.eng, s.MS, s.intent(status)); err != nil { return desc + ": " + err.Error() } } else { diff --git a/pkg/storage/engine/mvcc_test.go b/pkg/storage/engine/mvcc_test.go index 4cb1efb05eb4..d2c4efbfe11a 100644 --- a/pkg/storage/engine/mvcc_test.go +++ b/pkg/storage/engine/mvcc_test.go @@ -356,7 +356,7 @@ func TestMVCCEmptyKey(t *testing.T) { if _, _, _, err := MVCCScan(ctx, engine, testKey1, key, math.MaxInt64, ts, MVCCScanOptions{}); err == nil { t.Error("expected empty key error") } - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{}); err == nil { + if _, err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{}); err == nil { t.Error("expected empty key error") } }) @@ -3222,7 +3222,7 @@ func TestMVCCConditionalPutWithTxn(t *testing.T) { txnCommit := txn txnCommit.Status = roachpb.COMMITTED txnCommit.WriteTimestamp = clock.Now().Add(1, 0) - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ + if _, err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ Span: roachpb.Span{Key: testKey1}, Status: txnCommit.Status, Txn: txnCommit.TxnMeta, @@ -3373,7 +3373,7 @@ func TestMVCCInitPutWithTxn(t *testing.T) { txnCommit := txn txnCommit.Status = roachpb.COMMITTED txnCommit.WriteTimestamp = clock.Now().Add(1, 0) - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ + if _, err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ Span: roachpb.Span{Key: testKey1}, Status: txnCommit.Status, Txn: txnCommit.TxnMeta, @@ -3777,7 +3777,7 @@ func TestMVCCResolveTxn(t *testing.T) { } // Resolve will write with txn1's timestamp which is 0,1. - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ + if _, err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ Span: roachpb.Span{Key: testKey1}, Txn: txn1Commit.TxnMeta, Status: txn1Commit.Status, @@ -3822,7 +3822,7 @@ func TestMVCCResolveNewerIntent(t *testing.T) { } // Resolve will succeed but should remove the intent. - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ + if _, err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ Span: roachpb.Span{Key: testKey1}, Txn: txn1Commit.TxnMeta, Status: txn1Commit.Status, @@ -3871,7 +3871,7 @@ func TestMVCCResolveIntentTxnTimestampMismatch(t *testing.T) { // A bug (see #7654) caused intents to just stay where they were instead // of being moved forward in the situation set up above. - if err := MVCCResolveWriteIntent(ctx, engine, nil, intent); err != nil { + if _, err := MVCCResolveWriteIntent(ctx, engine, nil, intent); err != nil { t.Fatal(err) } @@ -4088,7 +4088,7 @@ func TestMVCCAbortTxn(t *testing.T) { txn1AbortWithTS := txn1Abort.Clone() txn1AbortWithTS.WriteTimestamp = hlc.Timestamp{Logical: 1} - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ + if _, err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ Span: roachpb.Span{Key: testKey1}, Txn: txn1AbortWithTS.TxnMeta, Status: txn1AbortWithTS.Status, @@ -4135,7 +4135,7 @@ func TestMVCCAbortTxnWithPreviousVersion(t *testing.T) { txn1AbortWithTS := txn1Abort.Clone() txn1AbortWithTS.WriteTimestamp = hlc.Timestamp{WallTime: 2} - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ + if _, err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ Span: roachpb.Span{Key: testKey1}, Status: txn1AbortWithTS.Status, Txn: txn1AbortWithTS.TxnMeta, @@ -4205,7 +4205,7 @@ func TestMVCCWriteWithDiffTimestampsAndEpochs(t *testing.T) { txne2Commit := txne2 txne2Commit.Status = roachpb.COMMITTED txne2Commit.WriteTimestamp = hlc.Timestamp{WallTime: 1} - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ + if _, err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ Span: roachpb.Span{Key: testKey1}, Status: txne2Commit.Status, Txn: txne2Commit.TxnMeta, @@ -4608,7 +4608,7 @@ func TestMVCCGetWithPushedTimestamp(t *testing.T) { } // Resolve the intent, pushing its timestamp forward. txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ + if _, err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ Span: roachpb.Span{Key: testKey1}, Status: txn.Status, Txn: txn.TxnMeta, @@ -4702,7 +4702,7 @@ func TestMVCCResolveWithUpdatedTimestamp(t *testing.T) { // Resolve with a higher commit timestamp -- this should rewrite the // intent when making it permanent. txn := makeTxn(*txn1Commit, hlc.Timestamp{WallTime: 1}) - if err = MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ + if _, err = MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ Span: roachpb.Span{Key: testKey1}, Status: txn.Status, Txn: txn.TxnMeta, @@ -4756,7 +4756,7 @@ func TestMVCCResolveWithPushedTimestamp(t *testing.T) { // Resolve with a higher commit timestamp, but with still-pending transaction. // This represents a straightforward push (i.e. from a read/write conflict). txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) - if err = MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ + if _, err = MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ Span: roachpb.Span{Key: testKey1}, Status: txn.Status, Txn: txn.TxnMeta, @@ -4797,7 +4797,7 @@ func TestMVCCResolveTxnNoOps(t *testing.T) { defer engine.Close() // Resolve a non existent key; noop. - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ + if _, err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ Span: roachpb.Span{Key: testKey1}, Status: txn1Commit.Status, Txn: txn1Commit.TxnMeta, @@ -4809,7 +4809,7 @@ func TestMVCCResolveTxnNoOps(t *testing.T) { if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ + if _, err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ Span: roachpb.Span{Key: testKey1}, Status: txn2Commit.Status, Txn: txn2Commit.TxnMeta, @@ -4824,7 +4824,7 @@ func TestMVCCResolveTxnNoOps(t *testing.T) { txn1CommitWithTS := txn2Commit.Clone() txn1CommitWithTS.WriteTimestamp = hlc.Timestamp{WallTime: 1} - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ + if _, err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ Span: roachpb.Span{Key: testKey2}, Status: txn1CommitWithTS.Status, Txn: txn1CommitWithTS.TxnMeta, @@ -5702,7 +5702,7 @@ func TestResolveIntentWithLowerEpoch(t *testing.T) { t.Fatal(err) } // Resolve the intent with a low epoch. - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ + if _, err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ Span: roachpb.Span{Key: testKey1}, Status: txn1.Status, Txn: txn1.TxnMeta, @@ -5887,7 +5887,7 @@ func TestMVCCIntentHistory(t *testing.T) { t.Fatal(err) } // Resolve the intent so we can use another transaction on this key. - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ + if _, err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ Span: roachpb.Span{Key: key}, Status: roachpb.COMMITTED, Txn: txn.TxnMeta, @@ -6099,7 +6099,7 @@ func TestMVCCIntentHistory(t *testing.T) { } // Resolve the intent. - if err = MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ + if _, err = MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ Span: roachpb.Span{Key: key}, Status: roachpb.COMMITTED, Txn: txn.TxnMeta, diff --git a/pkg/storage/replica_test.go b/pkg/storage/replica_test.go index 17944bb765bc..1a0e8f80d504 100644 --- a/pkg/storage/replica_test.go +++ b/pkg/storage/replica_test.go @@ -9025,11 +9025,21 @@ func TestNoopRequestsNotProposed(t *testing.T) { expProposal: false, }, { - name: "resolve aborted intent req", - req: resolveAbortedIntentReq, + name: "resolve aborted intent req, with intent", + setup: func(ctx context.Context, repl *Replica) *roachpb.Error { + return sendReq(ctx, repl, putReq, txn) + }, + req: resolveAbortedIntentReq, // Not a no-op - the request needs to poison the abort span. expProposal: true, }, + { + name: "resolve aborted intent req, without intent", + req: resolveAbortedIntentReq, + // No-op - the intent is missing, so there's nothing to resolve. + // This also means that the abort span isn't written. + expProposal: false, + }, { name: "redundant resolve aborted intent req", setup: func(ctx context.Context, repl *Replica) *roachpb.Error {