From a583955e277b58357ddcff55eaf59cec1e858797 Mon Sep 17 00:00:00 2001 From: Alex Sarkesian Date: Wed, 26 Jul 2023 02:29:52 -0400 Subject: [PATCH] storage: refactor mvcc write parameters This change introduces `MVCCWriteOptions`, a structure for bundling parameters for `MVCCPut`, `MVCCDelete`, and their many variants, and refactors usages of these functions across the code base in order to move the existing function arguments into this structure. In addition to allowing the code to eliminate specifying default values in many callers, this enables the ability to pass new flags to write operations such as the replay protection needed to address #103817. Part of: #103817 Release note: None --- pkg/ccl/storageccl/engineccl/bench_test.go | 2 +- .../storageccl/engineccl/encrypted_fs_test.go | 4 +- .../kvcoord/dist_sender_server_test.go | 2 +- pkg/kv/kvserver/abortspan/abortspan.go | 8 +- pkg/kv/kvserver/batch_spanset_test.go | 4 +- .../batcheval/cmd_add_sstable_test.go | 2 +- .../batcheval/cmd_clear_range_test.go | 4 +- .../kvserver/batcheval/cmd_conditional_put.go | 10 +- pkg/kv/kvserver/batcheval/cmd_delete.go | 8 +- pkg/kv/kvserver/batcheval/cmd_delete_range.go | 6 +- .../batcheval/cmd_delete_range_test.go | 10 +- .../kvserver/batcheval/cmd_end_transaction.go | 9 +- .../batcheval/cmd_end_transaction_test.go | 10 +- pkg/kv/kvserver/batcheval/cmd_export_test.go | 4 +- .../kvserver/batcheval/cmd_heartbeat_txn.go | 2 +- pkg/kv/kvserver/batcheval/cmd_increment.go | 8 +- pkg/kv/kvserver/batcheval/cmd_init_put.go | 10 +- pkg/kv/kvserver/batcheval/cmd_push_txn.go | 4 +- pkg/kv/kvserver/batcheval/cmd_put.go | 11 +- .../batcheval/cmd_query_intent_test.go | 2 +- .../cmd_query_resolved_timestamp_test.go | 6 +- pkg/kv/kvserver/batcheval/cmd_recover_txn.go | 2 +- .../batcheval/cmd_recover_txn_test.go | 4 +- .../batcheval/cmd_refresh_range_bench_test.go | 2 +- .../batcheval/cmd_refresh_range_test.go | 12 +- pkg/kv/kvserver/batcheval/cmd_refresh_test.go | 4 +- .../batcheval/cmd_resolve_intent_test.go | 6 +- .../batcheval/cmd_revert_range_test.go | 10 +- pkg/kv/kvserver/batcheval/cmd_scan_test.go | 4 +- .../batcheval/cmd_truncate_log_test.go | 4 +- pkg/kv/kvserver/batcheval/intent_test.go | 4 +- pkg/kv/kvserver/batcheval/transaction_test.go | 2 +- pkg/kv/kvserver/client_merge_test.go | 2 +- pkg/kv/kvserver/consistency_queue_test.go | 4 +- pkg/kv/kvserver/gc/data_distribution_test.go | 4 +- pkg/kv/kvserver/gc/gc_test.go | 4 +- pkg/kv/kvserver/kvstorage/cluster_version.go | 4 +- pkg/kv/kvserver/kvstorage/destroy.go | 4 +- pkg/kv/kvserver/kvstorage/init.go | 4 +- pkg/kv/kvserver/logstore/logstore.go | 11 +- pkg/kv/kvserver/logstore/stateloader.go | 24 +- pkg/kv/kvserver/loqrecovery/apply.go | 6 +- pkg/kv/kvserver/loqrecovery/apply_test.go | 2 +- pkg/kv/kvserver/loqrecovery/record.go | 16 +- .../kvserver/loqrecovery/recovery_env_test.go | 4 +- pkg/kv/kvserver/mvcc_gc_queue_test.go | 20 +- .../rangefeed/catchup_scan_bench_test.go | 2 +- .../kvserver/rangefeed/catchup_scan_test.go | 18 +- pkg/kv/kvserver/rangefeed/task_test.go | 2 +- .../kvserver/rditer/replica_data_iter_test.go | 2 +- pkg/kv/kvserver/readsummary/persist.go | 2 +- pkg/kv/kvserver/replica.go | 2 +- pkg/kv/kvserver/replica_consistency_test.go | 2 +- pkg/kv/kvserver/replica_evaluate_test.go | 4 +- pkg/kv/kvserver/replica_raft.go | 4 +- pkg/kv/kvserver/replica_test.go | 18 +- pkg/kv/kvserver/stateloader/stateloader.go | 20 +- pkg/kv/kvserver/store.go | 8 +- pkg/kv/kvserver/store_init.go | 16 +- pkg/kv/kvserver/store_snapshot.go | 2 +- pkg/kv/kvserver/store_test.go | 6 +- pkg/kv/kvserver/stores.go | 2 +- pkg/kv/kvserver/txn_wait_queue_test.go | 6 +- pkg/server/node_test.go | 6 +- pkg/server/node_tombstone_storage.go | 2 +- pkg/server/server_test.go | 7 +- pkg/server/settings_cache.go | 2 +- pkg/storage/bench_data_test.go | 2 +- pkg/storage/bench_pebble_test.go | 4 +- pkg/storage/bench_test.go | 33 +- pkg/storage/engine_test.go | 39 +- pkg/storage/metamorphic/operations.go | 14 +- pkg/storage/mvcc.go | 157 +++---- pkg/storage/mvcc_history_test.go | 42 +- pkg/storage/mvcc_incremental_iterator_test.go | 52 +-- pkg/storage/mvcc_logical_ops_test.go | 18 +- pkg/storage/mvcc_stats_test.go | 72 +-- pkg/storage/mvcc_test.go | 429 +++++++++--------- pkg/storage/pebble_mvcc_scanner_test.go | 2 +- pkg/storage/pebble_test.go | 2 +- pkg/storage/sst_test.go | 2 +- 81 files changed, 666 insertions(+), 624 deletions(-) diff --git a/pkg/ccl/storageccl/engineccl/bench_test.go b/pkg/ccl/storageccl/engineccl/bench_test.go index 0b8fa255f042..6c3f8d5ab110 100644 --- a/pkg/ccl/storageccl/engineccl/bench_test.go +++ b/pkg/ccl/storageccl/engineccl/bench_test.go @@ -93,7 +93,7 @@ func loadTestData( timestamp := hlc.Timestamp{WallTime: minWallTime + rand.Int63n(int64(batchTimeSpan))} value := roachpb.MakeValueFromBytes(randutil.RandBytes(rng, valueBytes)) value.InitChecksum(key) - if err := storage.MVCCPut(ctx, batch, nil, key, timestamp, hlc.ClockTimestamp{}, value, nil); err != nil { + if err := storage.MVCCPut(ctx, batch, key, timestamp, value, storage.MVCCWriteOptions{}); err != nil { tb.Fatal(err) } } diff --git a/pkg/ccl/storageccl/engineccl/encrypted_fs_test.go b/pkg/ccl/storageccl/engineccl/encrypted_fs_test.go index 1d3abff8238a..8f9055778aff 100644 --- a/pkg/ccl/storageccl/engineccl/encrypted_fs_test.go +++ b/pkg/ccl/storageccl/engineccl/encrypted_fs_test.go @@ -397,12 +397,10 @@ func TestPebbleEncryption2(t *testing.T) { err = storage.MVCCPut( context.Background(), db, - nil, /* ms */ roachpb.Key(key), hlc.Timestamp{}, - hlc.ClockTimestamp{}, roachpb.MakeValueFromBytes([]byte(val)), - nil, /* txn */ + storage.MVCCWriteOptions{}, ) require.NoError(t, err) require.NoError(t, db.Flush()) diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go b/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go index bdc23647bd64..813eb8356a51 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go @@ -80,7 +80,7 @@ func TestRangeLookupWithOpenTransaction(t *testing.T) { key := testutils.MakeKey(keys.Meta1Prefix, roachpb.KeyMax) now := s.Clock().Now() txn := roachpb.MakeTransaction("txn", roachpb.Key("foobar"), isolation.Serializable, 0, now, 0, int32(s.SQLInstanceID())) - if err := storage.MVCCPutProto(context.Background(), s.Engines()[0], nil, key, now, hlc.ClockTimestamp{}, &txn, &roachpb.RangeDescriptor{}); err != nil { + if err := storage.MVCCPutProto(context.Background(), s.Engines()[0], key, now, &roachpb.RangeDescriptor{}, storage.MVCCWriteOptions{Txn: &txn}); err != nil { t.Fatal(err) } diff --git a/pkg/kv/kvserver/abortspan/abortspan.go b/pkg/kv/kvserver/abortspan/abortspan.go index 01942e11faa5..afb50200c651 100644 --- a/pkg/kv/kvserver/abortspan/abortspan.go +++ b/pkg/kv/kvserver/abortspan/abortspan.go @@ -120,7 +120,7 @@ func (sc *AbortSpan) Del( ctx context.Context, reader storage.ReadWriter, ms *enginepb.MVCCStats, txnID uuid.UUID, ) error { key := keys.AbortSpanKey(sc.rangeID, txnID) - _, err := storage.MVCCDelete(ctx, reader, ms, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil /* txn */) + _, err := storage.MVCCDelete(ctx, reader, key, hlc.Timestamp{}, storage.MVCCWriteOptions{Stats: ms}) return err } @@ -134,7 +134,7 @@ func (sc *AbortSpan) Put( ) error { log.VEventf(ctx, 2, "writing abort span entry for %s", txnID.Short()) key := keys.AbortSpanKey(sc.rangeID, txnID) - return storage.MVCCPutProto(ctx, readWriter, ms, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil /* txn */, entry) + return storage.MVCCPutProto(ctx, readWriter, key, hlc.Timestamp{}, entry, storage.MVCCWriteOptions{Stats: ms}) } // CopyTo copies the abort span entries to the abort span for the range @@ -178,9 +178,9 @@ func (sc *AbortSpan) CopyTo( if err != nil { return err } - return storage.MVCCPutProto(ctx, w, ms, + return storage.MVCCPutProto(ctx, w, keys.AbortSpanKey(newRangeID, txnID), - hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &entry, + hlc.Timestamp{}, &entry, storage.MVCCWriteOptions{Stats: ms}, ) }); err != nil { return errors.Wrap(err, "AbortSpan.CopyTo") diff --git a/pkg/kv/kvserver/batch_spanset_test.go b/pkg/kv/kvserver/batch_spanset_test.go index 593c3ae41331..91c77e645760 100644 --- a/pkg/kv/kvserver/batch_spanset_test.go +++ b/pkg/kv/kvserver/batch_spanset_test.go @@ -545,12 +545,10 @@ func TestSpanSetMVCCResolveWriteIntentRange(t *testing.T) { if err := storage.MVCCPut( ctx, eng, - nil, // ms roachpb.Key("b"), hlc.Timestamp{WallTime: 10}, // irrelevant - hlc.ClockTimestamp{}, // irrelevant value, - nil, // txn + storage.MVCCWriteOptions{}, // irrelevant ); err != nil { t.Fatal(err) } diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go index 0cde5e1dba14..84cc5b24fdf9 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go @@ -1142,7 +1142,7 @@ func TestEvalAddSSTable(t *testing.T) { kv.Key.Timestamp.WallTime *= 1e9 v, err := storage.DecodeMVCCValue(kv.Value) require.NoError(t, err) - require.NoError(t, storage.MVCCPut(ctx, b, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v.Value, txn)) + require.NoError(t, storage.MVCCPut(ctx, b, kv.Key.Key, kv.Key.Timestamp, v.Value, storage.MVCCWriteOptions{Txn: txn})) case storage.MVCCRangeKeyValue: v, err := storage.DecodeMVCCValue(kv.Value) require.NoError(t, err) diff --git a/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go b/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go index 91260dea23b4..0f8f2f640d6d 100644 --- a/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go @@ -152,8 +152,8 @@ func TestCmdClearRange(t *testing.T) { // Write some random point keys within the cleared span, above the range tombstones. for i := 0; i < tc.keyCount; i++ { key := roachpb.Key(fmt.Sprintf("%04d", i)) - require.NoError(t, storage.MVCCPut(ctx, eng, nil, key, - hlc.Timestamp{WallTime: int64(4+i%2) * 1e9}, hlc.ClockTimestamp{}, value, nil)) + require.NoError(t, storage.MVCCPut(ctx, eng, key, + hlc.Timestamp{WallTime: int64(4+i%2) * 1e9}, value, storage.MVCCWriteOptions{})) } // Calculate the range stats. diff --git a/pkg/kv/kvserver/batcheval/cmd_conditional_put.go b/pkg/kv/kvserver/batcheval/cmd_conditional_put.go index 330357c7eaa0..07b0898e4121 100644 --- a/pkg/kv/kvserver/batcheval/cmd_conditional_put.go +++ b/pkg/kv/kvserver/batcheval/cmd_conditional_put.go @@ -51,6 +51,12 @@ func ConditionalPut( args := cArgs.Args.(*kvpb.ConditionalPutRequest) h := cArgs.Header + opts := storage.MVCCWriteOptions{ + Txn: h.Txn, + LocalTimestamp: cArgs.Now, + Stats: cArgs.Stats, + } + var ts hlc.Timestamp if !args.Inline { ts = h.Timestamp @@ -60,10 +66,10 @@ func ConditionalPut( var err error if args.Blind { err = storage.MVCCBlindConditionalPut( - ctx, readWriter, cArgs.Stats, args.Key, ts, cArgs.Now, args.Value, args.ExpBytes, handleMissing, h.Txn) + ctx, readWriter, args.Key, ts, args.Value, args.ExpBytes, handleMissing, opts) } else { err = storage.MVCCConditionalPut( - ctx, readWriter, cArgs.Stats, args.Key, ts, cArgs.Now, args.Value, args.ExpBytes, handleMissing, h.Txn) + ctx, readWriter, args.Key, ts, args.Value, args.ExpBytes, handleMissing, opts) } if err != nil { return result.Result{}, err diff --git a/pkg/kv/kvserver/batcheval/cmd_delete.go b/pkg/kv/kvserver/batcheval/cmd_delete.go index ba41d7cd4065..e5321313c27b 100644 --- a/pkg/kv/kvserver/batcheval/cmd_delete.go +++ b/pkg/kv/kvserver/batcheval/cmd_delete.go @@ -31,9 +31,15 @@ func Delete( h := cArgs.Header reply := resp.(*kvpb.DeleteResponse) + opts := storage.MVCCWriteOptions{ + Txn: h.Txn, + LocalTimestamp: cArgs.Now, + Stats: cArgs.Stats, + } + var err error reply.FoundKey, err = storage.MVCCDelete( - ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, cArgs.Now, h.Txn, + ctx, readWriter, args.Key, h.Timestamp, opts, ) if err != nil { return result.Result{}, err diff --git a/pkg/kv/kvserver/batcheval/cmd_delete_range.go b/pkg/kv/kvserver/batcheval/cmd_delete_range.go index 6d39578efab6..7820b0d811e8 100644 --- a/pkg/kv/kvserver/batcheval/cmd_delete_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_delete_range.go @@ -228,8 +228,10 @@ func DeleteRange( // can update the Result's AcquiredLocks field. returnKeys := args.ReturnKeys || h.Txn != nil deleted, resumeSpan, num, err := storage.MVCCDeleteRange( - ctx, readWriter, cArgs.Stats, args.Key, args.EndKey, - h.MaxSpanRequestKeys, timestamp, cArgs.Now, h.Txn, returnKeys) + ctx, readWriter, args.Key, args.EndKey, + h.MaxSpanRequestKeys, timestamp, + storage.MVCCWriteOptions{Txn: h.Txn, LocalTimestamp: cArgs.Now, Stats: cArgs.Stats}, + returnKeys) if err != nil { return result.Result{}, err } diff --git a/pkg/kv/kvserver/batcheval/cmd_delete_range_test.go b/pkg/kv/kvserver/batcheval/cmd_delete_range_test.go index 43060106f022..42620de94a6c 100644 --- a/pkg/kv/kvserver/batcheval/cmd_delete_range_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_delete_range_test.go @@ -60,12 +60,12 @@ func TestDeleteRangeTombstone(t *testing.T) { var localTS hlc.ClockTimestamp txn := roachpb.MakeTransaction("test", nil /* baseKey */, isolation.Serializable, roachpb.NormalUserPriority, hlc.Timestamp{WallTime: 5e9}, 0, 0) - require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("b"), hlc.Timestamp{WallTime: 2e9}, localTS, roachpb.MakeValueFromString("b2"), nil)) - require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("c"), hlc.Timestamp{WallTime: 4e9}, localTS, roachpb.MakeValueFromString("c4"), nil)) - require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("d"), hlc.Timestamp{WallTime: 2e9}, localTS, roachpb.MakeValueFromString("d2"), nil)) - _, err := storage.MVCCDelete(ctx, rw, nil, roachpb.Key("d"), hlc.Timestamp{WallTime: 3e9}, localTS, nil) + require.NoError(t, storage.MVCCPut(ctx, rw, roachpb.Key("b"), hlc.Timestamp{WallTime: 2e9}, roachpb.MakeValueFromString("b2"), storage.MVCCWriteOptions{})) + require.NoError(t, storage.MVCCPut(ctx, rw, roachpb.Key("c"), hlc.Timestamp{WallTime: 4e9}, roachpb.MakeValueFromString("c4"), storage.MVCCWriteOptions{})) + require.NoError(t, storage.MVCCPut(ctx, rw, roachpb.Key("d"), hlc.Timestamp{WallTime: 2e9}, roachpb.MakeValueFromString("d2"), storage.MVCCWriteOptions{})) + _, err := storage.MVCCDelete(ctx, rw, roachpb.Key("d"), hlc.Timestamp{WallTime: 3e9}, storage.MVCCWriteOptions{}) require.NoError(t, err) - require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("i"), hlc.Timestamp{WallTime: 5e9}, localTS, roachpb.MakeValueFromString("i5"), &txn)) + require.NoError(t, storage.MVCCPut(ctx, rw, roachpb.Key("i"), hlc.Timestamp{WallTime: 5e9}, roachpb.MakeValueFromString("i5"), storage.MVCCWriteOptions{Txn: &txn})) require.NoError(t, storage.MVCCDeleteRangeUsingTombstone(ctx, rw, nil, roachpb.Key("f"), roachpb.Key("h"), hlc.Timestamp{WallTime: 3e9}, localTS, nil, nil, false, 0, nil)) require.NoError(t, storage.MVCCDeleteRangeUsingTombstone(ctx, rw, nil, roachpb.Key("Z"), roachpb.Key("a"), hlc.Timestamp{WallTime: 100e9}, localTS, nil, nil, false, 0, nil)) require.NoError(t, storage.MVCCDeleteRangeUsingTombstone(ctx, rw, nil, roachpb.Key("z"), roachpb.Key("|"), hlc.Timestamp{WallTime: 100e9}, localTS, nil, nil, false, 0, nil)) diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go index cceb6a8ec461..e8b0b1deecaa 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go @@ -685,7 +685,7 @@ func updateStagingTxn( txn.LockSpans = args.LockSpans txn.InFlightWrites = args.InFlightWrites txnRecord := txn.AsRecord() - return storage.MVCCPutProto(ctx, readWriter, ms, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord) + return storage.MVCCPutProto(ctx, readWriter, key, hlc.Timestamp{}, &txnRecord, storage.MVCCWriteOptions{Stats: ms}) } // updateFinalizedTxn persists the COMMITTED or ABORTED transaction record with @@ -703,6 +703,7 @@ func updateFinalizedTxn( recordAlreadyExisted bool, externalLocks []roachpb.Span, ) error { + opts := storage.MVCCWriteOptions{Stats: ms} if !evalCtx.EvalKnobs().DisableTxnAutoGC && len(externalLocks) == 0 { if log.V(2) { log.Infof(ctx, "auto-gc'ed %s (%d locks)", txn.Short(), len(args.LockSpans)) @@ -713,13 +714,13 @@ func updateFinalizedTxn( // BatchRequest writes. return nil } - _, err := storage.MVCCDelete(ctx, readWriter, ms, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil) + _, err := storage.MVCCDelete(ctx, readWriter, key, hlc.Timestamp{}, opts) return err } txn.LockSpans = externalLocks txn.InFlightWrites = nil txnRecord := txn.AsRecord() - return storage.MVCCPutProto(ctx, readWriter, ms, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord) + return storage.MVCCPutProto(ctx, readWriter, key, hlc.Timestamp{}, &txnRecord, opts) } // RunCommitTrigger runs the commit trigger from an end transaction request. @@ -1092,7 +1093,7 @@ func splitTriggerHelper( if err != nil { return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to fetch last replica GC timestamp") } - if err := storage.MVCCPutProto(ctx, batch, nil, keys.RangeLastReplicaGCTimestampKey(split.RightDesc.RangeID), hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &replicaGCTS); err != nil { + if err := storage.MVCCPutProto(ctx, batch, keys.RangeLastReplicaGCTimestampKey(split.RightDesc.RangeID), hlc.Timestamp{}, &replicaGCTS, storage.MVCCWriteOptions{}); err != nil { return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to copy last replica GC timestamp") } diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go index d77670954cea..f43ee2fe4dc3 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go @@ -1098,7 +1098,7 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { // Write the existing transaction record, if necessary. txnKey := keys.TransactionKey(txn.Key, txn.ID) if c.existingTxn != nil { - if err := storage.MVCCPutProto(ctx, batch, nil, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, c.existingTxn); err != nil { + if err := storage.MVCCPutProto(ctx, batch, txnKey, hlc.Timestamp{}, c.existingTxn, storage.MVCCWriteOptions{}); err != nil { t.Fatal(err) } } @@ -1200,13 +1200,13 @@ func TestPartialRollbackOnEndTransaction(t *testing.T) { // Write a first value at key. v.SetString("a") txn.Sequence = 1 - if err := storage.MVCCPut(ctx, batch, nil, k, ts, hlc.ClockTimestamp{}, v, &txn); err != nil { + if err := storage.MVCCPut(ctx, batch, k, ts, v, storage.MVCCWriteOptions{Txn: &txn}); err != nil { t.Fatal(err) } // Write another value. v.SetString("b") txn.Sequence = 2 - if err := storage.MVCCPut(ctx, batch, nil, k, ts, hlc.ClockTimestamp{}, v, &txn); err != nil { + if err := storage.MVCCPut(ctx, batch, k, ts, v, storage.MVCCWriteOptions{Txn: &txn}); err != nil { t.Fatal(err) } @@ -1219,7 +1219,7 @@ func TestPartialRollbackOnEndTransaction(t *testing.T) { txnKey := keys.TransactionKey(txn.Key, txn.ID) if storeTxnBeforeEndTxn { txnRec := txn.AsRecord() - if err := storage.MVCCPutProto(ctx, batch, nil, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRec); err != nil { + if err := storage.MVCCPutProto(ctx, batch, txnKey, hlc.Timestamp{}, &txnRec, storage.MVCCWriteOptions{}); err != nil { t.Fatal(err) } } @@ -1648,7 +1648,7 @@ func TestResolveLocalLocks(t *testing.T) { txn.Status = roachpb.COMMITTED for i := 0; i < numKeys; i++ { - err := storage.MVCCPut(ctx, batch, nil, intToKey(i), ts, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("a"), &txn) + err := storage.MVCCPut(ctx, batch, intToKey(i), ts, roachpb.MakeValueFromString("a"), storage.MVCCWriteOptions{Txn: &txn}) require.NoError(t, err) } resolvedLocks, externalLocks, err := resolveLocalLocksWithPagination( diff --git a/pkg/kv/kvserver/batcheval/cmd_export_test.go b/pkg/kv/kvserver/batcheval/cmd_export_test.go index 87754c46a133..65fb7a7e54b7 100644 --- a/pkg/kv/kvserver/batcheval/cmd_export_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_export_test.go @@ -808,7 +808,7 @@ func TestRandomKeyAndTimestampExport(t *testing.T) { valueSize := randutil.RandIntInRange(rnd, averageValueSize-100, averageValueSize+100) value := roachpb.MakeValueFromBytes(randutil.RandBytes(rnd, valueSize)) value.InitChecksum(key) - if err := storage.MVCCPut(ctx, batch, nil, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { + if err := storage.MVCCPut(ctx, batch, key, ts, value, storage.MVCCWriteOptions{}); err != nil { t.Fatal(err) } @@ -819,7 +819,7 @@ func TestRandomKeyAndTimestampExport(t *testing.T) { ts = hlc.Timestamp{WallTime: int64(curWallTime), Logical: int32(curLogical)} value = roachpb.MakeValueFromBytes(randutil.RandBytes(rnd, 200)) value.InitChecksum(key) - if err := storage.MVCCPut(ctx, batch, nil, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { + if err := storage.MVCCPut(ctx, batch, key, ts, value, storage.MVCCWriteOptions{}); err != nil { t.Fatal(err) } } diff --git a/pkg/kv/kvserver/batcheval/cmd_heartbeat_txn.go b/pkg/kv/kvserver/batcheval/cmd_heartbeat_txn.go index a0cb38db7f01..24d7fc2e1c37 100644 --- a/pkg/kv/kvserver/batcheval/cmd_heartbeat_txn.go +++ b/pkg/kv/kvserver/batcheval/cmd_heartbeat_txn.go @@ -93,7 +93,7 @@ func HeartbeatTxn( // is up for debate. txn.LastHeartbeat.Forward(args.Now) txnRecord := txn.AsRecord() - if err := storage.MVCCPutProto(ctx, readWriter, cArgs.Stats, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord); err != nil { + if err := storage.MVCCPutProto(ctx, readWriter, key, hlc.Timestamp{}, &txnRecord, storage.MVCCWriteOptions{Stats: cArgs.Stats}); err != nil { return result.Result{}, err } } diff --git a/pkg/kv/kvserver/batcheval/cmd_increment.go b/pkg/kv/kvserver/batcheval/cmd_increment.go index 027096f58952..be576e3d1c48 100644 --- a/pkg/kv/kvserver/batcheval/cmd_increment.go +++ b/pkg/kv/kvserver/batcheval/cmd_increment.go @@ -32,9 +32,15 @@ func Increment( h := cArgs.Header reply := resp.(*kvpb.IncrementResponse) + opts := storage.MVCCWriteOptions{ + Txn: h.Txn, + LocalTimestamp: cArgs.Now, + Stats: cArgs.Stats, + } + var err error reply.NewValue, err = storage.MVCCIncrement( - ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, cArgs.Now, h.Txn, args.Increment) + ctx, readWriter, args.Key, h.Timestamp, opts, args.Increment) if err != nil { return result.Result{}, err } diff --git a/pkg/kv/kvserver/batcheval/cmd_init_put.go b/pkg/kv/kvserver/batcheval/cmd_init_put.go index f9a8aad3c1f7..0bb1a5a21c0a 100644 --- a/pkg/kv/kvserver/batcheval/cmd_init_put.go +++ b/pkg/kv/kvserver/batcheval/cmd_init_put.go @@ -32,6 +32,12 @@ func InitPut( args := cArgs.Args.(*kvpb.InitPutRequest) h := cArgs.Header + opts := storage.MVCCWriteOptions{ + Txn: h.Txn, + LocalTimestamp: cArgs.Now, + Stats: cArgs.Stats, + } + if args.FailOnTombstones && cArgs.EvalCtx.EvalKnobs().DisableInitPutFailOnTombstones { args.FailOnTombstones = false } @@ -39,10 +45,10 @@ func InitPut( var err error if args.Blind { err = storage.MVCCBlindInitPut( - ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, cArgs.Now, args.Value, args.FailOnTombstones, h.Txn) + ctx, readWriter, args.Key, h.Timestamp, args.Value, args.FailOnTombstones, opts) } else { err = storage.MVCCInitPut( - ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, cArgs.Now, args.Value, args.FailOnTombstones, h.Txn) + ctx, readWriter, args.Key, h.Timestamp, args.Value, args.FailOnTombstones, opts) } if err != nil { return result.Result{}, err diff --git a/pkg/kv/kvserver/batcheval/cmd_push_txn.go b/pkg/kv/kvserver/batcheval/cmd_push_txn.go index f5238c1c3d7c..ba68798a1435 100644 --- a/pkg/kv/kvserver/batcheval/cmd_push_txn.go +++ b/pkg/kv/kvserver/batcheval/cmd_push_txn.go @@ -315,7 +315,7 @@ func PushTxn( // 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 { + if err := storage.MVCCPutProto(ctx, readWriter, key, hlc.Timestamp{}, &txnRecord, storage.MVCCWriteOptions{Stats: cArgs.Stats}); err != nil { return result.Result{}, err } } @@ -336,7 +336,7 @@ func PushTxn( // 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 { + if err := storage.MVCCPutProto(ctx, readWriter, key, hlc.Timestamp{}, &txnRecord, storage.MVCCWriteOptions{Stats: cArgs.Stats}); err != nil { return result.Result{}, err } } diff --git a/pkg/kv/kvserver/batcheval/cmd_put.go b/pkg/kv/kvserver/batcheval/cmd_put.go index 3a887e0332d3..54282352200a 100644 --- a/pkg/kv/kvserver/batcheval/cmd_put.go +++ b/pkg/kv/kvserver/batcheval/cmd_put.go @@ -48,7 +48,12 @@ func Put( ) (result.Result, error) { args := cArgs.Args.(*kvpb.PutRequest) h := cArgs.Header - ms := cArgs.Stats + + opts := storage.MVCCWriteOptions{ + Txn: h.Txn, + LocalTimestamp: cArgs.Now, + Stats: cArgs.Stats, + } var ts hlc.Timestamp if !args.Inline { @@ -56,9 +61,9 @@ func Put( } var err error if args.Blind { - err = storage.MVCCBlindPut(ctx, readWriter, ms, args.Key, ts, cArgs.Now, args.Value, h.Txn) + err = storage.MVCCBlindPut(ctx, readWriter, args.Key, ts, args.Value, opts) } else { - err = storage.MVCCPut(ctx, readWriter, ms, args.Key, ts, cArgs.Now, args.Value, h.Txn) + err = storage.MVCCPut(ctx, readWriter, args.Key, ts, args.Value, opts) } if err != nil { return result.Result{}, err diff --git a/pkg/kv/kvserver/batcheval/cmd_query_intent_test.go b/pkg/kv/kvserver/batcheval/cmd_query_intent_test.go index 4161f060de6a..ab903789b66a 100644 --- a/pkg/kv/kvserver/batcheval/cmd_query_intent_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_query_intent_test.go @@ -40,7 +40,7 @@ func TestQueryIntent(t *testing.T) { writeIntent := func(k roachpb.Key, ts int64) roachpb.Transaction { txn := roachpb.MakeTransaction("test", k, 0, 0, makeTS(ts), 0, 1) - _, err := storage.MVCCDelete(ctx, db, nil, k, makeTS(ts), hlc.ClockTimestamp{}, &txn) + _, err := storage.MVCCDelete(ctx, db, k, makeTS(ts), storage.MVCCWriteOptions{Txn: &txn}) require.NoError(t, err) return txn } diff --git a/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go b/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go index 9b0d01bf8ec2..5ae795d04aae 100644 --- a/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go @@ -44,16 +44,16 @@ func TestQueryResolvedTimestamp(t *testing.T) { return hlc.Timestamp{WallTime: ts} } writeValue := func(k string, ts int64) { - _, err := storage.MVCCDelete(ctx, db, nil, roachpb.Key(k), makeTS(ts), hlc.ClockTimestamp{}, nil) + _, err := storage.MVCCDelete(ctx, db, roachpb.Key(k), makeTS(ts), storage.MVCCWriteOptions{}) require.NoError(t, err) } writeIntent := func(k string, ts int64) { txn := roachpb.MakeTransaction("test", roachpb.Key(k), 0, 0, makeTS(ts), 0, 1) - _, err := storage.MVCCDelete(ctx, db, nil, roachpb.Key(k), makeTS(ts), hlc.ClockTimestamp{}, &txn) + _, err := storage.MVCCDelete(ctx, db, roachpb.Key(k), makeTS(ts), storage.MVCCWriteOptions{Txn: &txn}) require.NoError(t, err) } writeInline := func(k string) { - _, err := storage.MVCCDelete(ctx, db, nil, roachpb.Key(k), hlc.Timestamp{}, hlc.ClockTimestamp{}, nil) + _, err := storage.MVCCDelete(ctx, db, roachpb.Key(k), hlc.Timestamp{}, storage.MVCCWriteOptions{}) require.NoError(t, err) } diff --git a/pkg/kv/kvserver/batcheval/cmd_recover_txn.go b/pkg/kv/kvserver/batcheval/cmd_recover_txn.go index 57021077f471..45c021a00fe7 100644 --- a/pkg/kv/kvserver/batcheval/cmd_recover_txn.go +++ b/pkg/kv/kvserver/batcheval/cmd_recover_txn.go @@ -219,7 +219,7 @@ func RecoverTxn( reply.RecoveredTxn.Status = roachpb.ABORTED } txnRecord := reply.RecoveredTxn.AsRecord() - if err := storage.MVCCPutProto(ctx, readWriter, cArgs.Stats, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord); err != nil { + if err := storage.MVCCPutProto(ctx, readWriter, key, hlc.Timestamp{}, &txnRecord, storage.MVCCWriteOptions{Stats: cArgs.Stats}); err != nil { return result.Result{}, err } diff --git a/pkg/kv/kvserver/batcheval/cmd_recover_txn_test.go b/pkg/kv/kvserver/batcheval/cmd_recover_txn_test.go index 381b265bc1d8..4a7363829afd 100644 --- a/pkg/kv/kvserver/batcheval/cmd_recover_txn_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_recover_txn_test.go @@ -49,7 +49,7 @@ func TestRecoverTxn(t *testing.T) { // Write the transaction record. txnKey := keys.TransactionKey(txn.Key, txn.ID) txnRecord := txn.AsRecord() - if err := storage.MVCCPutProto(ctx, db, nil, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord); err != nil { + if err := storage.MVCCPutProto(ctx, db, txnKey, hlc.Timestamp{}, &txnRecord, storage.MVCCWriteOptions{}); err != nil { t.Fatal(err) } @@ -225,7 +225,7 @@ func TestRecoverTxnRecordChanged(t *testing.T) { // request is evaluated. txnKey := keys.TransactionKey(txn.Key, txn.ID) txnRecord := c.changedTxn.AsRecord() - if err := storage.MVCCPutProto(ctx, db, nil, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord); err != nil { + if err := storage.MVCCPutProto(ctx, db, txnKey, hlc.Timestamp{}, &txnRecord, storage.MVCCWriteOptions{}); err != nil { t.Fatal(err) } diff --git a/pkg/kv/kvserver/batcheval/cmd_refresh_range_bench_test.go b/pkg/kv/kvserver/batcheval/cmd_refresh_range_bench_test.go index 61859eef49ed..2b8eafc80f4e 100644 --- a/pkg/kv/kvserver/batcheval/cmd_refresh_range_bench_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_refresh_range_bench_test.go @@ -252,7 +252,7 @@ func setupData( value := roachpb.MakeValueFromBytes(randutil.RandBytes(rng, opts.valueBytes)) value.InitChecksum(key) ts := hlc.Timestamp{WallTime: int64((pos + 1) * 5)} - if err := storage.MVCCPut(ctx, batch, nil /* ms */, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { + if err := storage.MVCCPut(ctx, batch, key, ts, value, storage.MVCCWriteOptions{}); err != nil { b.Fatal(err) } } diff --git a/pkg/kv/kvserver/batcheval/cmd_refresh_range_test.go b/pkg/kv/kvserver/batcheval/cmd_refresh_range_test.go index ff59fe5f9dd7..494bfff5346c 100644 --- a/pkg/kv/kvserver/batcheval/cmd_refresh_range_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_refresh_range_test.go @@ -39,9 +39,9 @@ func TestRefreshRange(t *testing.T) { // Write an MVCC point key at b@3, MVCC point tombstone at b@5, and MVCC range // tombstone at [d-f)@7. require.NoError(t, storage.MVCCPut( - ctx, eng, nil, roachpb.Key("b"), hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("value"), nil)) + ctx, eng, roachpb.Key("b"), hlc.Timestamp{WallTime: 3}, roachpb.MakeValueFromString("value"), storage.MVCCWriteOptions{})) require.NoError(t, storage.MVCCPut( - ctx, eng, nil, roachpb.Key("c"), hlc.Timestamp{WallTime: 5}, hlc.ClockTimestamp{}, roachpb.Value{}, nil)) + ctx, eng, roachpb.Key("c"), hlc.Timestamp{WallTime: 5}, roachpb.Value{}, storage.MVCCWriteOptions{})) require.NoError(t, storage.MVCCDeleteRangeUsingTombstone( ctx, eng, nil, roachpb.Key("d"), roachpb.Key("f"), hlc.Timestamp{WallTime: 7}, hlc.ClockTimestamp{}, nil, nil, false, 0, nil)) @@ -155,10 +155,10 @@ func TestRefreshRangeTimeBoundIterator(t *testing.T) { }, ReadTimestamp: ts1, } - if err := storage.MVCCPut(ctx, db, nil, k, txn.ReadTimestamp, hlc.ClockTimestamp{}, v, txn); err != nil { + if err := storage.MVCCPut(ctx, db, k, txn.ReadTimestamp, v, storage.MVCCWriteOptions{Txn: txn}); err != nil { t.Fatal(err) } - if err := storage.MVCCPut(ctx, db, nil, roachpb.Key("unused1"), ts4, hlc.ClockTimestamp{}, v, nil); err != nil { + if err := storage.MVCCPut(ctx, db, roachpb.Key("unused1"), ts4, v, storage.MVCCWriteOptions{}); err != nil { t.Fatal(err) } if err := db.Flush(); err != nil { @@ -177,7 +177,7 @@ func TestRefreshRangeTimeBoundIterator(t *testing.T) { if _, _, _, err := storage.MVCCResolveWriteIntent(ctx, db, nil, intent, storage.MVCCResolveWriteIntentOptions{}); err != nil { t.Fatal(err) } - if err := storage.MVCCPut(ctx, db, nil, roachpb.Key("unused2"), ts1, hlc.ClockTimestamp{}, v, nil); err != nil { + if err := storage.MVCCPut(ctx, db, roachpb.Key("unused2"), ts1, v, storage.MVCCWriteOptions{}); err != nil { t.Fatal(err) } if err := db.Flush(); err != nil { @@ -266,7 +266,7 @@ func TestRefreshRangeError(t *testing.T) { }, ReadTimestamp: ts2, } - if err := storage.MVCCPut(ctx, db, nil, k, txn.ReadTimestamp, hlc.ClockTimestamp{}, v, txn); err != nil { + if err := storage.MVCCPut(ctx, db, k, txn.ReadTimestamp, v, storage.MVCCWriteOptions{Txn: txn}); err != nil { t.Fatal(err) } diff --git a/pkg/kv/kvserver/batcheval/cmd_refresh_test.go b/pkg/kv/kvserver/batcheval/cmd_refresh_test.go index 808f43bf2856..b94c73931608 100644 --- a/pkg/kv/kvserver/batcheval/cmd_refresh_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_refresh_test.go @@ -60,7 +60,7 @@ func TestRefreshError(t *testing.T) { }, ReadTimestamp: ts2, } - if err := storage.MVCCPut(ctx, db, nil, k, txn.ReadTimestamp, hlc.ClockTimestamp{}, v, txn); err != nil { + if err := storage.MVCCPut(ctx, db, k, txn.ReadTimestamp, v, storage.MVCCWriteOptions{Txn: txn}); err != nil { t.Fatal(err) } @@ -121,7 +121,7 @@ func TestRefreshTimestampBounds(t *testing.T) { ts3 := hlc.Timestamp{WallTime: 3} // Write to a key at time ts2. - require.NoError(t, storage.MVCCPut(ctx, db, nil, k, ts2, hlc.ClockTimestamp{}, v, nil)) + require.NoError(t, storage.MVCCPut(ctx, db, k, ts2, v, storage.MVCCWriteOptions{})) for _, tc := range []struct { from, to hlc.Timestamp diff --git a/pkg/kv/kvserver/batcheval/cmd_resolve_intent_test.go b/pkg/kv/kvserver/batcheval/cmd_resolve_intent_test.go index 9b127b775882..fdd53a2c65a3 100644 --- a/pkg/kv/kvserver/batcheval/cmd_resolve_intent_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_resolve_intent_test.go @@ -169,13 +169,13 @@ func TestResolveIntentAfterPartialRollback(t *testing.T) { // Write a first value at key. v.SetString("a") txn.Sequence = 0 - if err := storage.MVCCPut(ctx, batch, nil, k, ts, hlc.ClockTimestamp{}, v, &txn); err != nil { + if err := storage.MVCCPut(ctx, batch, k, ts, v, storage.MVCCWriteOptions{Txn: &txn}); err != nil { t.Fatal(err) } // Write another value. v.SetString("b") txn.Sequence = 1 - if err := storage.MVCCPut(ctx, batch, nil, k, ts, hlc.ClockTimestamp{}, v, &txn); err != nil { + if err := storage.MVCCPut(ctx, batch, k, ts, v, storage.MVCCWriteOptions{Txn: &txn}); err != nil { t.Fatal(err) } if err := batch.Commit(true); err != nil { @@ -305,7 +305,7 @@ func TestResolveIntentWithTargetBytes(t *testing.T) { st := cluster.MakeTestingClusterSettings() for i, testKey := range testKeys { - err := storage.MVCCPut(ctx, batch, nil, testKey, ts, hlc.ClockTimestamp{}, values[i], &txn) + err := storage.MVCCPut(ctx, batch, testKey, ts, values[i], storage.MVCCWriteOptions{Txn: &txn}) require.NoError(t, err) } initialBytes := batch.Len() diff --git a/pkg/kv/kvserver/batcheval/cmd_revert_range_test.go b/pkg/kv/kvserver/batcheval/cmd_revert_range_test.go index d89b5971c470..523b08f271cc 100644 --- a/pkg/kv/kvserver/batcheval/cmd_revert_range_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_revert_range_test.go @@ -69,7 +69,7 @@ func TestCmdRevertRange(t *testing.T) { key := roachpb.Key(fmt.Sprintf("%04d", i)) var value roachpb.Value value.SetString(fmt.Sprintf("%d", i)) - if err := storage.MVCCPut(ctx, eng, &stats, key, baseTime.Add(int64(i%10), 0), hlc.ClockTimestamp{}, value, nil); err != nil { + if err := storage.MVCCPut(ctx, eng, key, baseTime.Add(int64(i%10), 0), value, storage.MVCCWriteOptions{Stats: &stats}); err != nil { t.Fatal(err) } } @@ -83,7 +83,7 @@ func TestCmdRevertRange(t *testing.T) { key := roachpb.Key(fmt.Sprintf("%04d", i)) var value roachpb.Value value.SetString(fmt.Sprintf("%d-rev-a", i)) - if err := storage.MVCCPut(ctx, eng, &stats, key, tsA.Add(int64(i%5), 1), hlc.ClockTimestamp{}, value, nil); err != nil { + if err := storage.MVCCPut(ctx, eng, key, tsA.Add(int64(i%5), 1), value, storage.MVCCWriteOptions{Stats: &stats}); err != nil { t.Fatal(err) } } @@ -96,7 +96,7 @@ func TestCmdRevertRange(t *testing.T) { key := roachpb.Key(fmt.Sprintf("%04d", i)) var value roachpb.Value value.SetString(fmt.Sprintf("%d-rev-b", i)) - if err := storage.MVCCPut(ctx, eng, &stats, key, tsB.Add(1, int32(i%5)), hlc.ClockTimestamp{}, value, nil); err != nil { + if err := storage.MVCCPut(ctx, eng, key, tsB.Add(1, int32(i%5)), value, storage.MVCCWriteOptions{Stats: &stats}); err != nil { t.Fatal(err) } } @@ -169,7 +169,7 @@ func TestCmdRevertRange(t *testing.T) { txn := roachpb.MakeTransaction("test", nil, isolation.Serializable, roachpb.NormalUserPriority, tsC, 1, 1) if err := storage.MVCCPut( - ctx, eng, &stats, []byte("0012"), tsC, hlc.ClockTimestamp{}, roachpb.MakeValueFromBytes([]byte("i")), &txn, + ctx, eng, []byte("0012"), tsC, roachpb.MakeValueFromBytes([]byte("i")), storage.MVCCWriteOptions{Txn: &txn, Stats: &stats}, ); err != nil { t.Fatal(err) } @@ -180,7 +180,7 @@ func TestCmdRevertRange(t *testing.T) { key := roachpb.Key(fmt.Sprintf("%04d", i)) var value roachpb.Value value.SetString(fmt.Sprintf("%d-rev-b", i)) - if err := storage.MVCCPut(ctx, eng, &stats, key, tsC.Add(10, int32(i%5)), hlc.ClockTimestamp{}, value, nil); err != nil { + if err := storage.MVCCPut(ctx, eng, key, tsC.Add(10, int32(i%5)), value, storage.MVCCWriteOptions{Stats: &stats}); err != nil { t.Fatalf("writing key %s: %+v", key, err) } } diff --git a/pkg/kv/kvserver/batcheval/cmd_scan_test.go b/pkg/kv/kvserver/batcheval/cmd_scan_test.go index eb3c594fe8bd..4cb31e773f42 100644 --- a/pkg/kv/kvserver/batcheval/cmd_scan_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_scan_test.go @@ -88,7 +88,7 @@ func testScanReverseScanInner( // Write to k1 and k2. for _, k := range []roachpb.Key{k1, k2} { - err := storage.MVCCPut(ctx, eng, nil, k, ts, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("value-"+string(k)), nil) + err := storage.MVCCPut(ctx, eng, k, ts, roachpb.MakeValueFromString("value-"+string(k)), storage.MVCCWriteOptions{}) require.NoError(t, err) } @@ -185,7 +185,7 @@ func TestScanReverseScanWholeRows(t *testing.T) { for r := 0; r < 2; r++ { for cf := uint32(0); cf < 3; cf++ { key := makeRowKey(t, r, cf) - err := storage.MVCCPut(ctx, eng, nil, key, ts, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("value"), nil) + err := storage.MVCCPut(ctx, eng, key, ts, roachpb.MakeValueFromString("value"), storage.MVCCWriteOptions{}) require.NoError(t, err) rowKeys = append(rowKeys, key) } diff --git a/pkg/kv/kvserver/batcheval/cmd_truncate_log_test.go b/pkg/kv/kvserver/batcheval/cmd_truncate_log_test.go index 3d452cb84636..14428ef2cbe9 100644 --- a/pkg/kv/kvserver/batcheval/cmd_truncate_log_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_truncate_log_test.go @@ -35,8 +35,8 @@ func putTruncatedState( ) { key := keys.RaftTruncatedStateKey(rangeID) if err := storage.MVCCPutProto( - context.Background(), eng, nil, key, - hlc.Timestamp{}, hlc.ClockTimestamp{}, nil /* txn */, &truncState, + context.Background(), eng, key, + hlc.Timestamp{}, &truncState, storage.MVCCWriteOptions{}, ); err != nil { t.Fatal(err) } diff --git a/pkg/kv/kvserver/batcheval/intent_test.go b/pkg/kv/kvserver/batcheval/intent_test.go index e658a1bde434..f50ed9dfb896 100644 --- a/pkg/kv/kvserver/batcheval/intent_test.go +++ b/pkg/kv/kvserver/batcheval/intent_test.go @@ -131,9 +131,9 @@ func TestCollectIntentsUsesSameIterator(t *testing.T) { txn := roachpb.MakeTransaction("test", key, isolation.Serializable, roachpb.NormalUserPriority, ts, 0, 1) var err error if delete { - _, err = storage.MVCCDelete(ctx, db, nil, key, ts, hlc.ClockTimestamp{}, &txn) + _, err = storage.MVCCDelete(ctx, db, key, ts, storage.MVCCWriteOptions{Txn: &txn}) } else { - err = storage.MVCCPut(ctx, db, nil, key, ts, hlc.ClockTimestamp{}, val, &txn) + err = storage.MVCCPut(ctx, db, key, ts, val, storage.MVCCWriteOptions{Txn: &txn}) } require.NoError(t, err) diff --git a/pkg/kv/kvserver/batcheval/transaction_test.go b/pkg/kv/kvserver/batcheval/transaction_test.go index 2bfd0ea1e682..f22c511bb58d 100644 --- a/pkg/kv/kvserver/batcheval/transaction_test.go +++ b/pkg/kv/kvserver/batcheval/transaction_test.go @@ -85,7 +85,7 @@ func TestUpdateAbortSpan(t *testing.T) { type evalFn func(storage.ReadWriter, EvalContext, *enginepb.MVCCStats) error addIntent := func(b storage.ReadWriter, _ EvalContext, ms *enginepb.MVCCStats) error { val := roachpb.MakeValueFromString("val") - return storage.MVCCPut(ctx, b, ms, intentKey, txn.ReadTimestamp, hlc.ClockTimestamp{}, val, &txn) + return storage.MVCCPut(ctx, b, intentKey, txn.ReadTimestamp, val, storage.MVCCWriteOptions{Txn: &txn, Stats: ms}) } addPrevAbortSpanEntry := func(b storage.ReadWriter, rec EvalContext, ms *enginepb.MVCCStats) error { return UpdateAbortSpan(ctx, rec, b, ms, prevTxn.TxnMeta, true /* poison */) diff --git a/pkg/kv/kvserver/client_merge_test.go b/pkg/kv/kvserver/client_merge_test.go index df7152e7d2e0..8291a1fc9665 100644 --- a/pkg/kv/kvserver/client_merge_test.go +++ b/pkg/kv/kvserver/client_merge_test.go @@ -3893,7 +3893,7 @@ func TestStoreRangeMergeRaftSnapshot(t *testing.T) { tombstoneKey := keys.RangeTombstoneKey(rangeID) tombstoneValue := &kvserverpb.RangeTombstone{NextReplicaID: math.MaxInt32} if err := storage.MVCCBlindPutProto( - context.Background(), &sst, nil, tombstoneKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, tombstoneValue, nil, + context.Background(), &sst, tombstoneKey, hlc.Timestamp{}, tombstoneValue, storage.MVCCWriteOptions{}, ); err != nil { return err } diff --git a/pkg/kv/kvserver/consistency_queue_test.go b/pkg/kv/kvserver/consistency_queue_test.go index bd53ed544dc1..39219daed2e9 100644 --- a/pkg/kv/kvserver/consistency_queue_test.go +++ b/pkg/kv/kvserver/consistency_queue_test.go @@ -343,8 +343,8 @@ func TestCheckConsistencyInconsistent(t *testing.T) { var val roachpb.Value val.SetInt(42) // Put an inconsistent key "e" to s2, and have s1 and s3 still agree. - require.NoError(t, storage.MVCCPut(context.Background(), s2.TODOEngine(), nil, - roachpb.Key("e"), tc.Server(0).Clock().Now(), hlc.ClockTimestamp{}, val, nil)) + require.NoError(t, storage.MVCCPut(context.Background(), s2.TODOEngine(), + roachpb.Key("e"), tc.Server(0).Clock().Now(), val, storage.MVCCWriteOptions{})) // Run consistency check again, this time it should find something. resp = runConsistencyCheck() diff --git a/pkg/kv/kvserver/gc/data_distribution_test.go b/pkg/kv/kvserver/gc/data_distribution_test.go index 516e6a4f4d85..569237bb20e4 100644 --- a/pkg/kv/kvserver/gc/data_distribution_test.go +++ b/pkg/kv/kvserver/gc/data_distribution_test.go @@ -79,8 +79,8 @@ func (ds dataDistribution) setupTest( if txn.WriteTimestamp.IsEmpty() { txn.WriteTimestamp = ts } - err := storage.MVCCPut(ctx, eng, &ms, kv.Key.Key, ts, - hlc.ClockTimestamp{}, roachpb.Value{RawBytes: kv.Value}, txn) + err := storage.MVCCPut(ctx, eng, kv.Key.Key, ts, + roachpb.Value{RawBytes: kv.Value}, storage.MVCCWriteOptions{Txn: txn, Stats: &ms}) require.NoError(t, err, "failed to insert value for key %s, value length=%d", kv.Key.String(), len(kv.Value)) } diff --git a/pkg/kv/kvserver/gc/gc_test.go b/pkg/kv/kvserver/gc/gc_test.go index 04b9c123930e..97facbd32e7a 100644 --- a/pkg/kv/kvserver/gc/gc_test.go +++ b/pkg/kv/kvserver/gc/gc_test.go @@ -136,7 +136,7 @@ func TestIntentAgeThresholdSetting(t *testing.T) { WallTime: intentTs.Nanoseconds(), } txn := roachpb.MakeTransaction("txn", key, isolation.Serializable, roachpb.NormalUserPriority, intentHlc, 1000, 0) - require.NoError(t, storage.MVCCPut(ctx, eng, nil, key, intentHlc, hlc.ClockTimestamp{}, value, &txn)) + require.NoError(t, storage.MVCCPut(ctx, eng, key, intentHlc, value, storage.MVCCWriteOptions{Txn: &txn})) require.NoError(t, eng.Flush()) // Prepare test fixtures for GC run. @@ -197,7 +197,7 @@ func TestIntentCleanupBatching(t *testing.T) { txn := roachpb.MakeTransaction("txn", key, isolation.Serializable, roachpb.NormalUserPriority, intentHlc, 1000, 0) for _, suffix := range objectKeys { key := []byte{prefix, suffix} - require.NoError(t, storage.MVCCPut(ctx, eng, nil, key, intentHlc, hlc.ClockTimestamp{}, value, &txn)) + require.NoError(t, storage.MVCCPut(ctx, eng, key, intentHlc, value, storage.MVCCWriteOptions{Txn: &txn})) } require.NoError(t, eng.Flush()) } diff --git a/pkg/kv/kvserver/kvstorage/cluster_version.go b/pkg/kv/kvserver/kvstorage/cluster_version.go index 36d37964c9a9..c5a02b43546d 100644 --- a/pkg/kv/kvserver/kvstorage/cluster_version.go +++ b/pkg/kv/kvserver/kvstorage/cluster_version.go @@ -35,12 +35,10 @@ func WriteClusterVersion( if err := storage.MVCCPutProto( ctx, eng, - nil, keys.DeprecatedStoreClusterVersionKey(), hlc.Timestamp{}, - hlc.ClockTimestamp{}, - nil, &cv, + storage.MVCCWriteOptions{}, ); err != nil { return err } diff --git a/pkg/kv/kvserver/kvstorage/destroy.go b/pkg/kv/kvserver/kvstorage/destroy.go index 8a24cae11067..89ef23160b30 100644 --- a/pkg/kv/kvserver/kvstorage/destroy.go +++ b/pkg/kv/kvserver/kvstorage/destroy.go @@ -148,6 +148,6 @@ func DestroyReplica( tombstone := kvserverpb.RangeTombstone{NextReplicaID: nextReplicaID} // "Blind" because ms == nil and timestamp.IsEmpty(). - return storage.MVCCBlindPutProto(ctx, writer, nil, tombstoneKey, - hlc.Timestamp{}, hlc.ClockTimestamp{}, &tombstone, nil) + return storage.MVCCBlindPutProto(ctx, writer, tombstoneKey, + hlc.Timestamp{}, &tombstone, storage.MVCCWriteOptions{}) } diff --git a/pkg/kv/kvserver/kvstorage/init.go b/pkg/kv/kvserver/kvstorage/init.go index fd65085c6ddc..68ecdb81d16e 100644 --- a/pkg/kv/kvserver/kvstorage/init.go +++ b/pkg/kv/kvserver/kvstorage/init.go @@ -60,12 +60,10 @@ func InitEngine(ctx context.Context, eng storage.Engine, ident roachpb.StoreIden if err := storage.MVCCPutProto( ctx, batch, - nil, keys.StoreIdentKey(), hlc.Timestamp{}, - hlc.ClockTimestamp{}, - nil, &ident, + storage.MVCCWriteOptions{}, ); err != nil { batch.Close() return err diff --git a/pkg/kv/kvserver/logstore/logstore.go b/pkg/kv/kvserver/logstore/logstore.go index 769e2fabae68..40dc17ee17fd 100644 --- a/pkg/kv/kvserver/logstore/logstore.go +++ b/pkg/kv/kvserver/logstore/logstore.go @@ -380,6 +380,9 @@ func logAppend( return prev, nil } var diff enginepb.MVCCStats + opts := storage.MVCCWriteOptions{ + Stats: &diff, + } value := valPool.Get().(*roachpb.Value) value.RawBytes = value.RawBytes[:0] defer valPool.Put(value) @@ -393,9 +396,9 @@ func logAppend( value.InitChecksum(key) var err error if kvpb.RaftIndex(ent.Index) > prev.LastIndex { - err = storage.MVCCBlindPut(ctx, rw, &diff, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, *value, nil /* txn */) + err = storage.MVCCBlindPut(ctx, rw, key, hlc.Timestamp{}, *value, opts) } else { - err = storage.MVCCPut(ctx, rw, &diff, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, *value, nil /* txn */) + err = storage.MVCCPut(ctx, rw, key, hlc.Timestamp{}, *value, opts) } if err != nil { return RaftState{}, err @@ -408,8 +411,8 @@ func logAppend( for i := newLastIndex + 1; i <= prev.LastIndex; i++ { // Note that the caller is in charge of deleting any sideloaded payloads // (which they must only do *after* the batch has committed). - _, err := storage.MVCCDelete(ctx, rw, &diff, keys.RaftLogKeyFromPrefix(raftLogPrefix, i), - hlc.Timestamp{}, hlc.ClockTimestamp{}, nil) + _, err := storage.MVCCDelete(ctx, rw, keys.RaftLogKeyFromPrefix(raftLogPrefix, i), + hlc.Timestamp{}, opts) if err != nil { return RaftState{}, err } diff --git a/pkg/kv/kvserver/logstore/stateloader.go b/pkg/kv/kvserver/logstore/stateloader.go index ef379618a3d3..a3c384b58905 100644 --- a/pkg/kv/kvserver/logstore/stateloader.go +++ b/pkg/kv/kvserver/logstore/stateloader.go @@ -107,16 +107,14 @@ func (sl StateLoader) SetRaftTruncatedState( if (*truncState == kvserverpb.RaftTruncatedState{}) { return errors.New("cannot persist empty RaftTruncatedState") } - // "Blind" because ms == nil and timestamp.IsEmpty(). + // "Blind" because opts.Stats == nil and timestamp.IsEmpty(). return storage.MVCCBlindPutProto( ctx, writer, - nil, /* ms */ sl.RaftTruncatedStateKey(), - hlc.Timestamp{}, /* timestamp */ - hlc.ClockTimestamp{}, /* localTimestamp */ + hlc.Timestamp{}, /* timestamp */ truncState, - nil, /* txn */ + storage.MVCCWriteOptions{}, /* txn */ ) } @@ -138,16 +136,14 @@ func (sl StateLoader) LoadHardState( func (sl StateLoader) SetHardState( ctx context.Context, writer storage.Writer, hs raftpb.HardState, ) error { - // "Blind" because ms == nil and timestamp.IsEmpty(). + // "Blind" because opts.Stats == nil and timestamp.IsEmpty(). return storage.MVCCBlindPutProto( ctx, writer, - nil, /* ms */ sl.RaftHardStateKey(), - hlc.Timestamp{}, /* timestamp */ - hlc.ClockTimestamp{}, /* localTimestamp */ + hlc.Timestamp{}, /* timestamp */ &hs, - nil, /* txn */ + storage.MVCCWriteOptions{}, /* opts */ ) } @@ -191,16 +187,14 @@ func (sl StateLoader) SetRaftReplicaID( ctx context.Context, writer storage.Writer, replicaID roachpb.ReplicaID, ) error { rid := kvserverpb.RaftReplicaID{ReplicaID: replicaID} - // "Blind" because ms == nil and timestamp.IsEmpty(). + // "Blind" because opts.Stats == nil and timestamp.IsEmpty(). return storage.MVCCBlindPutProto( ctx, writer, - nil, /* ms */ sl.RaftReplicaIDKey(), - hlc.Timestamp{}, /* timestamp */ - hlc.ClockTimestamp{}, /* localTimestamp */ + hlc.Timestamp{}, /* timestamp */ &rid, - nil, /* txn */ + storage.MVCCWriteOptions{}, /* opts */ ) } diff --git a/pkg/kv/kvserver/loqrecovery/apply.go b/pkg/kv/kvserver/loqrecovery/apply.go index 615ea5eb8c16..37db9ef2f05d 100644 --- a/pkg/kv/kvserver/loqrecovery/apply.go +++ b/pkg/kv/kvserver/loqrecovery/apply.go @@ -266,7 +266,7 @@ func applyReplicaUpdate( // A crude form of the intent resolution process: abort the // transaction by deleting its record. txnKey := keys.TransactionKey(res.Intent.Txn.Key, res.Intent.Txn.ID) - if _, err := storage.MVCCDelete(ctx, readWriter, &ms, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := storage.MVCCDelete(ctx, readWriter, txnKey, hlc.Timestamp{}, storage.MVCCWriteOptions{Stats: &ms}); err != nil { return PrepareReplicaReport{}, err } update := roachpb.LockUpdate{ @@ -293,8 +293,8 @@ func applyReplicaUpdate( newDesc.NextReplicaID = update.NextReplicaID if err := storage.MVCCPutProto( - ctx, readWriter, &ms, key, clock.Now(), - hlc.ClockTimestamp{}, nil /* txn */, &newDesc, + ctx, readWriter, key, clock.Now(), + &newDesc, storage.MVCCWriteOptions{Stats: &ms}, ); err != nil { return PrepareReplicaReport{}, err } diff --git a/pkg/kv/kvserver/loqrecovery/apply_test.go b/pkg/kv/kvserver/loqrecovery/apply_test.go index 0cc83adfcdce..787e137e0dd5 100644 --- a/pkg/kv/kvserver/loqrecovery/apply_test.go +++ b/pkg/kv/kvserver/loqrecovery/apply_test.go @@ -92,7 +92,7 @@ func createEngineOrFatal(ctx context.Context, t *testing.T, uuid uuid.UUID, id i StoreID: roachpb.StoreID(id), } if err = storage.MVCCPutProto( - context.Background(), eng, nil, keys.StoreIdentKey(), hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &sIdent, + context.Background(), eng, keys.StoreIdentKey(), hlc.Timestamp{}, &sIdent, storage.MVCCWriteOptions{}, ); err != nil { t.Fatalf("failed to populate test store ident: %v", err) } diff --git a/pkg/kv/kvserver/loqrecovery/record.go b/pkg/kv/kvserver/loqrecovery/record.go index 0b980aa8b95a..76199429b13a 100644 --- a/pkg/kv/kvserver/loqrecovery/record.go +++ b/pkg/kv/kvserver/loqrecovery/record.go @@ -174,17 +174,17 @@ func writeNodeRecoveryResults( actions loqrecoverypb.DeferredRecoveryActions, ) error { var fullErr error - err := storage.MVCCPutProto(ctx, writer, nil, keys.StoreLossOfQuorumRecoveryStatusKey(), - hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &result) + err := storage.MVCCPutProto(ctx, writer, keys.StoreLossOfQuorumRecoveryStatusKey(), + hlc.Timestamp{}, &result, storage.MVCCWriteOptions{}) fullErr = errors.Wrap(err, "failed to write loss of quorum recovery plan application status") if !actions.Empty() { - err = storage.MVCCPutProto(ctx, writer, nil, keys.StoreLossOfQuorumRecoveryCleanupActionsKey(), - hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &actions) + err = storage.MVCCPutProto(ctx, writer, keys.StoreLossOfQuorumRecoveryCleanupActionsKey(), + hlc.Timestamp{}, &actions, storage.MVCCWriteOptions{}) fullErr = errors.CombineErrors(fullErr, errors.Wrap(err, "failed to write loss of quorum recovery cleanup action")) } else { - _, err = storage.MVCCDelete(ctx, writer, nil, keys.StoreLossOfQuorumRecoveryCleanupActionsKey(), - hlc.Timestamp{}, hlc.ClockTimestamp{}, nil) + _, err = storage.MVCCDelete(ctx, writer, keys.StoreLossOfQuorumRecoveryCleanupActionsKey(), + hlc.Timestamp{}, storage.MVCCWriteOptions{}) fullErr = errors.CombineErrors(fullErr, errors.Wrap(err, "failed to clean loss of quorum recovery cleanup action")) } @@ -222,7 +222,7 @@ func ReadCleanupActionsInfo( // RemoveCleanupActionsInfo removes cleanup actions info if it is present in the // reader. func RemoveCleanupActionsInfo(ctx context.Context, writer storage.ReadWriter) error { - _, err := storage.MVCCDelete(ctx, writer, nil, keys.StoreLossOfQuorumRecoveryCleanupActionsKey(), - hlc.Timestamp{}, hlc.ClockTimestamp{}, nil) + _, err := storage.MVCCDelete(ctx, writer, keys.StoreLossOfQuorumRecoveryCleanupActionsKey(), + hlc.Timestamp{}, storage.MVCCWriteOptions{}) return err } diff --git a/pkg/kv/kvserver/loqrecovery/recovery_env_test.go b/pkg/kv/kvserver/loqrecovery/recovery_env_test.go index 6d4604d44b83..1c86a4f0dc33 100644 --- a/pkg/kv/kvserver/loqrecovery/recovery_env_test.go +++ b/pkg/kv/kvserver/loqrecovery/recovery_env_test.go @@ -328,7 +328,7 @@ func (e *quorumRecoveryEnv) handleReplicationData(t *testing.T, d datadriven.Tes eng := e.getOrCreateStore(ctx, t, replica.StoreID, replica.NodeID) if err = storage.MVCCPutProto( - ctx, eng, nil, key, clock.Now(), hlc.ClockTimestamp{}, nil /* txn */, &desc, + ctx, eng, key, clock.Now(), &desc, storage.MVCCWriteOptions{}, ); err != nil { t.Fatalf("failed to write range descriptor into store: %v", err) } @@ -622,7 +622,7 @@ func (e *quorumRecoveryEnv) getOrCreateStore( StoreID: storeID, } if err = storage.MVCCPutProto( - context.Background(), eng, nil, keys.StoreIdentKey(), hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &sIdent, + context.Background(), eng, keys.StoreIdentKey(), hlc.Timestamp{}, &sIdent, storage.MVCCWriteOptions{}, ); err != nil { t.Fatalf("failed to populate test store ident: %v", err) } diff --git a/pkg/kv/kvserver/mvcc_gc_queue_test.go b/pkg/kv/kvserver/mvcc_gc_queue_test.go index 1516185c4f6a..bbcb07eaefe5 100644 --- a/pkg/kv/kvserver/mvcc_gc_queue_test.go +++ b/pkg/kv/kvserver/mvcc_gc_queue_test.go @@ -300,7 +300,11 @@ func (cws *cachedWriteSimulator) multiKey( ts := hlc.Timestamp{}.Add(ms.LastUpdateNanos, 0) key, value := []byte("multikey"), cws.value(size) var eachMS enginepb.MVCCStats - if err := storage.MVCCPut(ctx, eng, &eachMS, key, ts, hlc.ClockTimestamp{}, value, txn); err != nil { + opts := storage.MVCCWriteOptions{ + Txn: txn, + Stats: &eachMS, + } + if err := storage.MVCCPut(ctx, eng, key, ts, value, opts); err != nil { t.Fatal(err) } for i := 1; i < numOps; i++ { @@ -329,7 +333,7 @@ func (cws *cachedWriteSimulator) singleKeySteady( for i := 0; i < qps; i++ { now := initialNow.Add(elapsed.Nanoseconds(), int32(i)) - if err := storage.MVCCPut(ctx, eng, ms, key, now, hlc.ClockTimestamp{}, value, nil); err != nil { + if err := storage.MVCCPut(ctx, eng, key, now, value, storage.MVCCWriteOptions{Stats: ms}); err != nil { t.Fatal(err) } if len(firstSl) < cacheFirstLen { @@ -524,9 +528,9 @@ func TestFullRangeDeleteHeuristic(t *testing.T) { if rng.Float32() > 0.5 { value.SetBytes(make([]byte, 20)) } - require.NoError(t, storage.MVCCPut(ctx, rw, &ms, key, - hlc.Timestamp{WallTime: time.Millisecond.Nanoseconds() * int64(i)}, hlc.ClockTimestamp{}, - value, nil)) + require.NoError(t, storage.MVCCPut(ctx, rw, key, + hlc.Timestamp{WallTime: time.Millisecond.Nanoseconds() * int64(i)}, + value, storage.MVCCWriteOptions{Stats: &ms})) } return ms, hlc.Timestamp{WallTime: time.Millisecond.Nanoseconds() * int64(valCount)} } @@ -537,7 +541,7 @@ func TestFullRangeDeleteHeuristic(t *testing.T) { } deleteWithPoints := func(rw storage.ReadWriter, delTime hlc.Timestamp, ms *enginepb.MVCCStats) { for _, key := range keys { - require.NoError(t, storage.MVCCPut(ctx, rw, ms, key, delTime, hlc.ClockTimestamp{}, roachpb.Value{}, nil)) + require.NoError(t, storage.MVCCPut(ctx, rw, key, delTime, roachpb.Value{}, storage.MVCCWriteOptions{Stats: ms})) } } @@ -1127,7 +1131,7 @@ func TestMVCCGCQueueTransactionTable(t *testing.T) { txns[strKey] = *txn for _, addrKey := range []roachpb.Key{baseKey, outsideKey} { key := keys.TransactionKey(addrKey, txn.ID) - if err := storage.MVCCPutProto(ctx, tc.engine, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, txn); err != nil { + if err := storage.MVCCPutProto(ctx, tc.engine, key, hlc.Timestamp{}, txn, storage.MVCCWriteOptions{}); err != nil { t.Fatal(err) } } @@ -1331,7 +1335,7 @@ func TestMVCCGCQueueLastProcessedTimestamps(t *testing.T) { ts := tc.Clock().Now() for _, lpv := range lastProcessedVals { - if err := storage.MVCCPutProto(ctx, tc.engine, nil, lpv.key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &ts); err != nil { + if err := storage.MVCCPutProto(ctx, tc.engine, lpv.key, hlc.Timestamp{}, &ts, storage.MVCCWriteOptions{}); err != nil { t.Fatal(err) } } diff --git a/pkg/kv/kvserver/rangefeed/catchup_scan_bench_test.go b/pkg/kv/kvserver/rangefeed/catchup_scan_bench_test.go index c1a76ce35221..ec1b7abbf37e 100644 --- a/pkg/kv/kvserver/rangefeed/catchup_scan_bench_test.go +++ b/pkg/kv/kvserver/rangefeed/catchup_scan_bench_test.go @@ -307,7 +307,7 @@ func setupData( value := roachpb.MakeValueFromBytes(randutil.RandBytes(rng, opts.valueBytes)) value.InitChecksum(key) ts := hlc.Timestamp{WallTime: int64((pos + 1) * 5)} - if err := storage.MVCCPut(ctx, batch, nil /* ms */, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { + if err := storage.MVCCPut(ctx, batch, key, ts, value, storage.MVCCWriteOptions{}); err != nil { b.Fatal(err) } } diff --git a/pkg/kv/kvserver/rangefeed/catchup_scan_test.go b/pkg/kv/kvserver/rangefeed/catchup_scan_test.go index 0048f7cdff14..624767be2769 100644 --- a/pkg/kv/kvserver/rangefeed/catchup_scan_test.go +++ b/pkg/kv/kvserver/rangefeed/catchup_scan_test.go @@ -103,12 +103,12 @@ func TestCatchupScan(t *testing.T) { // Put with no intent. for _, kv := range []storage.MVCCKeyValue{kv1_1_1, kv1_2_2, kv1_3_3, kv2_1_1, kv2_2_2, kv2_5_3} { v := roachpb.Value{RawBytes: kv.Value} - if err := storage.MVCCPut(ctx, eng, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { + if err := storage.MVCCPut(ctx, eng, kv.Key.Key, kv.Key.Timestamp, v, storage.MVCCWriteOptions{}); err != nil { t.Fatal(err) } } // Put with an intent. - if err := storage.MVCCPut(ctx, eng, nil, kv1_4_4.Key.Key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val, &txn); err != nil { + if err := storage.MVCCPut(ctx, eng, kv1_4_4.Key.Key, txn.ReadTimestamp, val, storage.MVCCWriteOptions{Txn: &txn}); err != nil { t.Fatal(err) } testutils.RunTrueAndFalse(t, "withDiff", func(t *testing.T, withDiff bool) { @@ -152,7 +152,7 @@ func TestCatchupScanInlineError(t *testing.T) { defer eng.Close() // Write an inline value. - require.NoError(t, storage.MVCCPut(ctx, eng, nil, roachpb.Key("inline"), hlc.Timestamp{}, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("foo"), nil)) + require.NoError(t, storage.MVCCPut(ctx, eng, roachpb.Key("inline"), hlc.Timestamp{}, roachpb.MakeValueFromString("foo"), storage.MVCCWriteOptions{})) // Run a catchup scan across the span and watch it error. span := roachpb.Span{Key: keys.LocalMax, EndKey: keys.MaxKey} @@ -184,15 +184,15 @@ func TestCatchupScanSeesOldIntent(t *testing.T) { tsIntent := tsCutoff.Add(-10, 0) // the intent is below the lower bound tsVersionInWindow := tsCutoff.Add(10, 0) // an unrelated version is above the lower bound - require.NoError(t, storage.MVCCPut(ctx, eng, nil, roachpb.Key("b"), - tsVersionInWindow, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("foo"), nil)) + require.NoError(t, storage.MVCCPut(ctx, eng, roachpb.Key("b"), + tsVersionInWindow, roachpb.MakeValueFromString("foo"), storage.MVCCWriteOptions{})) txn := roachpb.MakeTransaction("foo", roachpb.Key("d"), isolation.Serializable, roachpb.NormalUserPriority, tsIntent, 100, 0) - require.NoError(t, storage.MVCCPut(ctx, eng, nil, roachpb.Key("d"), - tsIntent, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("intent"), &txn)) + require.NoError(t, storage.MVCCPut(ctx, eng, roachpb.Key("d"), + tsIntent, roachpb.MakeValueFromString("intent"), storage.MVCCWriteOptions{Txn: &txn})) - require.NoError(t, storage.MVCCPut(ctx, eng, nil, roachpb.Key("e"), - tsVersionInWindow, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("bar"), nil)) + require.NoError(t, storage.MVCCPut(ctx, eng, roachpb.Key("e"), + tsVersionInWindow, roachpb.MakeValueFromString("bar"), storage.MVCCWriteOptions{})) // Run a catchup scan across the span and watch it succeed. span := roachpb.Span{Key: keys.LocalMax, EndKey: keys.MaxKey} diff --git a/pkg/kv/kvserver/rangefeed/task_test.go b/pkg/kv/kvserver/rangefeed/task_test.go index ecc948979c4f..32e7b3adec1c 100644 --- a/pkg/kv/kvserver/rangefeed/task_test.go +++ b/pkg/kv/kvserver/rangefeed/task_test.go @@ -299,7 +299,7 @@ func TestInitResolvedTSScan(t *testing.T) { } for _, op := range testData { kv := op.kv - err := storage.MVCCPut(ctx, engine, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, roachpb.Value{RawBytes: kv.Value}, op.txn) + err := storage.MVCCPut(ctx, engine, kv.Key.Key, kv.Key.Timestamp, roachpb.Value{RawBytes: kv.Value}, storage.MVCCWriteOptions{Txn: op.txn}) require.NoError(t, err) } return engine diff --git a/pkg/kv/kvserver/rditer/replica_data_iter_test.go b/pkg/kv/kvserver/rditer/replica_data_iter_test.go index f163fcea5c09..d533b8ea564e 100644 --- a/pkg/kv/kvserver/rditer/replica_data_iter_test.go +++ b/pkg/kv/kvserver/rditer/replica_data_iter_test.go @@ -126,7 +126,7 @@ func createRangeData( }) for _, pk := range ps { - require.NoError(t, storage.MVCCPut(ctx, eng, nil, pk.Key, pk.Timestamp, localTS, value, nil)) + require.NoError(t, storage.MVCCPut(ctx, eng, pk.Key, pk.Timestamp, value, storage.MVCCWriteOptions{LocalTimestamp: localTS})) } for _, rk := range rs { require.NoError(t, eng.PutMVCCRangeKey(rk, storage.MVCCValue{})) diff --git a/pkg/kv/kvserver/readsummary/persist.go b/pkg/kv/kvserver/readsummary/persist.go index bde7509f6e33..e1f9dcd115c5 100644 --- a/pkg/kv/kvserver/readsummary/persist.go +++ b/pkg/kv/kvserver/readsummary/persist.go @@ -44,5 +44,5 @@ func Set( sum *rspb.ReadSummary, ) error { key := keys.RangePriorReadSummaryKey(rangeID) - return storage.MVCCPutProto(ctx, readWriter, ms, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, sum) + return storage.MVCCPutProto(ctx, readWriter, key, hlc.Timestamp{}, sum, storage.MVCCWriteOptions{Stats: ms}) } diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index d4b48bf890ee..ceb7e5033fb5 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -1436,7 +1436,7 @@ func (r *Replica) GetLastReplicaGCTimestamp(ctx context.Context) (hlc.Timestamp, func (r *Replica) setLastReplicaGCTimestamp(ctx context.Context, timestamp hlc.Timestamp) error { key := keys.RangeLastReplicaGCTimestampKey(r.RangeID) return storage.MVCCPutProto( - ctx, r.store.TODOEngine(), nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, ×tamp) + ctx, r.store.TODOEngine(), key, hlc.Timestamp{}, ×tamp, storage.MVCCWriteOptions{}) } // getQueueLastProcessed returns the last processed timestamp for the diff --git a/pkg/kv/kvserver/replica_consistency_test.go b/pkg/kv/kvserver/replica_consistency_test.go index d5be211ce6f7..7857c1a30d4c 100644 --- a/pkg/kv/kvserver/replica_consistency_test.go +++ b/pkg/kv/kvserver/replica_consistency_test.go @@ -306,7 +306,7 @@ func TestReplicaChecksumSHA512(t *testing.T) { require.NoError(t, storage.MVCCDeleteRangeUsingTombstone( ctx, eng, nil, key, endKey, ts, localTS, nil, nil, false, 0, nil)) } else { - require.NoError(t, storage.MVCCPut(ctx, eng, nil, key, ts, localTS, value, nil)) + require.NoError(t, storage.MVCCPut(ctx, eng, key, ts, value, storage.MVCCWriteOptions{LocalTimestamp: localTS})) } rd, err = CalcReplicaDigest(ctx, desc, eng, kvpb.ChecksumMode_CHECK_FULL, unlim) diff --git a/pkg/kv/kvserver/replica_evaluate_test.go b/pkg/kv/kvserver/replica_evaluate_test.go index d75de6663b09..432ea9d4157e 100644 --- a/pkg/kv/kvserver/replica_evaluate_test.go +++ b/pkg/kv/kvserver/replica_evaluate_test.go @@ -734,8 +734,8 @@ func writeABCDEFIntents(t *testing.T, d *data, txn *roachpb.Transaction) { func writeABCDEFWith(t *testing.T, eng storage.Engine, ts hlc.Timestamp, txn *roachpb.Transaction) { for _, k := range []string{"a", "b", "c", "d", "e", "f"} { require.NoError(t, storage.MVCCPut( - context.Background(), eng, nil /* ms */, roachpb.Key(k), ts, hlc.ClockTimestamp{}, - roachpb.MakeValueFromString("value-"+k), txn)) + context.Background(), eng, roachpb.Key(k), ts, + roachpb.MakeValueFromString("value-"+k), storage.MVCCWriteOptions{Txn: txn})) } } diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index 82d43d97d8dc..5c6b2ac688cd 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -2586,12 +2586,10 @@ func handleTruncatedStateBelowRaftPreApply( if err := storage.MVCCPutProto( ctx, readWriter, - nil, /* ms */ prefixBuf.RaftTruncatedStateKey(), hlc.Timestamp{}, - hlc.ClockTimestamp{}, - nil, /* txn */ suggestedTruncatedState, + storage.MVCCWriteOptions{}, ); err != nil { return false, errors.Wrap(err, "unable to write RaftTruncatedState") } diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 2a9a7aa6d653..b70bfea4729e 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -1786,8 +1786,8 @@ func TestOptimizePuts(t *testing.T) { require.NoError(t, storage.MVCCDeleteRangeUsingTombstone(ctx, tc.engine, nil, c.exKey, c.exEndKey, hlc.MinTimestamp, hlc.ClockTimestamp{}, nil, nil, false, 0, nil)) } else if c.exKey != nil { - require.NoError(t, storage.MVCCPut(ctx, tc.engine, nil, c.exKey, - hlc.Timestamp{}, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("foo"), nil)) + require.NoError(t, storage.MVCCPut(ctx, tc.engine, c.exKey, + hlc.Timestamp{}, roachpb.MakeValueFromString("foo"), storage.MVCCWriteOptions{})) } batch := kvpb.BatchRequest{} for _, r := range c.reqs { @@ -3492,7 +3492,7 @@ func TestReplicaAbortSpanReadError(t *testing.T) { // Overwrite Abort span entry with garbage for the last op. key := keys.AbortSpanKey(tc.repl.RangeID, txn.ID) - err := storage.MVCCPut(ctx, tc.engine, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("never read in this test"), nil) + err := storage.MVCCPut(ctx, tc.engine, key, hlc.Timestamp{}, roachpb.MakeValueFromString("never read in this test"), storage.MVCCWriteOptions{}) if err != nil { t.Fatal(err) } @@ -4571,7 +4571,7 @@ func TestEndTxnWithErrors(t *testing.T) { existTxnRecord := existTxn.AsRecord() txnKey := keys.TransactionKey(test.key, txn.ID) if err := storage.MVCCPutProto( - ctx, tc.repl.store.TODOEngine(), nil, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &existTxnRecord, + ctx, tc.repl.store.TODOEngine(), txnKey, hlc.Timestamp{}, &existTxnRecord, storage.MVCCWriteOptions{}, ); err != nil { t.Fatal(err) } @@ -4614,7 +4614,7 @@ func TestEndTxnWithErrorAndSyncIntentResolution(t *testing.T) { existTxn.Status = roachpb.ABORTED existTxnRec := existTxn.AsRecord() txnKey := keys.TransactionKey(txn.Key, txn.ID) - err := storage.MVCCPutProto(ctx, tc.repl.store.TODOEngine(), nil, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &existTxnRec) + err := storage.MVCCPutProto(ctx, tc.repl.store.TODOEngine(), txnKey, hlc.Timestamp{}, &existTxnRec, storage.MVCCWriteOptions{}) require.NoError(t, err) // End the transaction, verify expected error, shouldn't deadlock. @@ -10985,7 +10985,7 @@ func TestReplicaPushed1PC(t *testing.T) { // Write a value outside the transaction. tc.manualClock.Advance(10) ts2 := tc.Clock().Now() - if err := storage.MVCCPut(ctx, tc.engine, nil, k, ts2, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("one"), nil); err != nil { + if err := storage.MVCCPut(ctx, tc.engine, k, ts2, roachpb.MakeValueFromString("one"), storage.MVCCWriteOptions{}); err != nil { t.Fatalf("writing interfering value: %+v", err) } @@ -13491,10 +13491,10 @@ func setMockPutWithEstimates(containsEstimatesDelta int64) (undo func()) { ctx context.Context, readWriter storage.ReadWriter, cArgs batcheval.CommandArgs, _ kvpb.Response, ) (result.Result, error) { args := cArgs.Args.(*kvpb.PutRequest) - ms := cArgs.Stats - ms.ContainsEstimates += containsEstimatesDelta + opts := storage.MVCCWriteOptions{Txn: cArgs.Header.Txn, Stats: cArgs.Stats} + opts.Stats.ContainsEstimates += containsEstimatesDelta ts := cArgs.Header.Timestamp - return result.Result{}, storage.MVCCBlindPut(ctx, readWriter, ms, args.Key, ts, hlc.ClockTimestamp{}, args.Value, cArgs.Header.Txn) + return result.Result{}, storage.MVCCBlindPut(ctx, readWriter, args.Key, ts, args.Value, opts) } batcheval.UnregisterCommand(kvpb.Put) diff --git a/pkg/kv/kvserver/stateloader/stateloader.go b/pkg/kv/kvserver/stateloader/stateloader.go index ac6856cb27d6..5f0befffb15b 100644 --- a/pkg/kv/kvserver/stateloader/stateloader.go +++ b/pkg/kv/kvserver/stateloader/stateloader.go @@ -166,8 +166,8 @@ func (rsl StateLoader) LoadLease( func (rsl StateLoader) SetLease( ctx context.Context, readWriter storage.ReadWriter, ms *enginepb.MVCCStats, lease roachpb.Lease, ) error { - return storage.MVCCPutProto(ctx, readWriter, ms, rsl.RangeLeaseKey(), - hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &lease) + return storage.MVCCPutProto(ctx, readWriter, rsl.RangeLeaseKey(), + hlc.Timestamp{}, &lease, storage.MVCCWriteOptions{Stats: ms}) } // SetLeaseBlind persists a lease using a blind write, updating the MVCC stats @@ -253,8 +253,8 @@ func (rsl StateLoader) SetRangeAppliedState( // The RangeAppliedStateKey is not included in stats. This is also reflected // in ComputeStats. ms := (*enginepb.MVCCStats)(nil) - return storage.MVCCPutProto(ctx, readWriter, ms, rsl.RangeAppliedStateKey(), - hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, as) + return storage.MVCCPutProto(ctx, readWriter, rsl.RangeAppliedStateKey(), + hlc.Timestamp{}, as, storage.MVCCWriteOptions{Stats: ms}) } // SetMVCCStats overwrites the MVCC stats. This needs to perform a read on the @@ -307,8 +307,8 @@ func (rsl StateLoader) SetGCThreshold( if threshold == nil { return errors.New("cannot persist nil GCThreshold") } - return storage.MVCCPutProto(ctx, readWriter, ms, rsl.RangeGCThresholdKey(), - hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, threshold) + return storage.MVCCPutProto(ctx, readWriter, rsl.RangeGCThresholdKey(), + hlc.Timestamp{}, threshold, storage.MVCCWriteOptions{Stats: ms}) } // LoadGCHint loads GC hint. @@ -331,8 +331,8 @@ func (rsl StateLoader) SetGCHint( if hint == nil { return false, errors.New("cannot persist nil GCHint") } - if err := storage.MVCCPutProto(ctx, readWriter, ms, rsl.RangeGCHintKey(), - hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, hint); err != nil { + if err := storage.MVCCPutProto(ctx, readWriter, rsl.RangeGCHintKey(), + hlc.Timestamp{}, hint, storage.MVCCWriteOptions{Stats: ms}); err != nil { return false, err } return true, nil @@ -355,8 +355,8 @@ func (rsl StateLoader) SetVersion( ms *enginepb.MVCCStats, version *roachpb.Version, ) error { - return storage.MVCCPutProto(ctx, readWriter, ms, rsl.RangeVersionKey(), - hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, version) + return storage.MVCCPutProto(ctx, readWriter, rsl.RangeVersionKey(), + hlc.Timestamp{}, version, storage.MVCCWriteOptions{Stats: ms}) } // UninitializedReplicaState returns the ReplicaState of an uninitialized diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 15ba1e15d058..faa42cafc4ac 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -2546,12 +2546,10 @@ func (s *Store) WriteLastUpTimestamp(ctx context.Context, time hlc.Timestamp) er return storage.MVCCPutProto( ctx, s.TODOEngine(), // TODO(sep-raft-log): probably state engine - nil, keys.StoreLastUpKey(), hlc.Timestamp{}, - hlc.ClockTimestamp{}, - nil, &time, + storage.MVCCWriteOptions{}, ) } @@ -2582,12 +2580,10 @@ func (s *Store) WriteHLCUpperBound(ctx context.Context, time int64) error { if err := storage.MVCCPutProto( ctx, batch, - nil, keys.StoreHLCUpperBoundKey(), hlc.Timestamp{}, - hlc.ClockTimestamp{}, - nil, &ts, + storage.MVCCWriteOptions{}, ); err != nil { return err } diff --git a/pkg/kv/kvserver/store_init.go b/pkg/kv/kvserver/store_init.go index bed46a0ba29b..e73f6f579e6d 100644 --- a/pkg/kv/kvserver/store_init.go +++ b/pkg/kv/kvserver/store_init.go @@ -168,24 +168,24 @@ func WriteInitialClusterData( // Range descriptor. if err := storage.MVCCPutProto( - ctx, batch, nil /* ms */, keys.RangeDescriptorKey(desc.StartKey), - now, hlc.ClockTimestamp{}, nil /* txn */, desc, + ctx, batch, keys.RangeDescriptorKey(desc.StartKey), + now, desc, storage.MVCCWriteOptions{}, ); err != nil { return err } // Replica GC timestamp. if err := storage.MVCCPutProto( - ctx, batch, nil /* ms */, keys.RangeLastReplicaGCTimestampKey(desc.RangeID), - hlc.Timestamp{}, hlc.ClockTimestamp{}, nil /* txn */, &now, + ctx, batch, keys.RangeLastReplicaGCTimestampKey(desc.RangeID), + hlc.Timestamp{}, &now, storage.MVCCWriteOptions{}, ); err != nil { return err } // Range addressing for meta2. meta2Key := keys.RangeMetaKey(endKey) if err := storage.MVCCPutProto( - ctx, batch, firstRangeMS, meta2Key.AsRawKey(), - now, hlc.ClockTimestamp{}, nil /* txn */, desc, + ctx, batch, meta2Key.AsRawKey(), + now, desc, storage.MVCCWriteOptions{Stats: firstRangeMS}, ); err != nil { return err } @@ -195,7 +195,7 @@ func WriteInitialClusterData( // Range addressing for meta1. meta1Key := keys.RangeMetaKey(keys.RangeMetaKey(roachpb.RKeyMax)) if err := storage.MVCCPutProto( - ctx, batch, nil /* ms */, meta1Key.AsRawKey(), now, hlc.ClockTimestamp{}, nil /* txn */, desc, + ctx, batch, meta1Key.AsRawKey(), now, desc, storage.MVCCWriteOptions{}, ); err != nil { return err } @@ -206,7 +206,7 @@ func WriteInitialClusterData( // Initialize the checksums. kv.Value.InitChecksum(kv.Key) if err := storage.MVCCPut( - ctx, batch, nil /* ms */, kv.Key, now, hlc.ClockTimestamp{}, kv.Value, nil, /* txn */ + ctx, batch, kv.Key, now, kv.Value, storage.MVCCWriteOptions{}, ); err != nil { return err } diff --git a/pkg/kv/kvserver/store_snapshot.go b/pkg/kv/kvserver/store_snapshot.go index 3c52af6b4b51..0dc36e5a34ce 100644 --- a/pkg/kv/kvserver/store_snapshot.go +++ b/pkg/kv/kvserver/store_snapshot.go @@ -1381,7 +1381,7 @@ func SendEmptySnapshot( var ms enginepb.MVCCStats // Seed an empty range into the new engine. if err := storage.MVCCPutProto( - ctx, eng, &ms, keys.RangeDescriptorKey(desc.StartKey), now, hlc.ClockTimestamp{}, nil /* txn */, &desc, + ctx, eng, keys.RangeDescriptorKey(desc.StartKey), now, &desc, storage.MVCCWriteOptions{Stats: &ms}, ); err != nil { return err } diff --git a/pkg/kv/kvserver/store_test.go b/pkg/kv/kvserver/store_test.go index 0810d6c385ee..ef582ce57fb8 100644 --- a/pkg/kv/kvserver/store_test.go +++ b/pkg/kv/kvserver/store_test.go @@ -381,12 +381,10 @@ func TestIterateIDPrefixKeys(t *testing.T) { if err := storage.MVCCPut( ctx, eng, - nil, /* ms */ key, hlc.Timestamp{}, - hlc.ClockTimestamp{}, roachpb.MakeValueFromString("fake value for "+key.String()), - nil, /* txn */ + storage.MVCCWriteOptions{}, ); err != nil { t.Fatal(err) } @@ -418,7 +416,7 @@ func TestIterateIDPrefixKeys(t *testing.T) { t.Logf("writing tombstone at rangeID=%d", rangeID) if err := storage.MVCCPutProto( - ctx, eng, nil /* ms */, keys.RangeTombstoneKey(rangeID), hlc.Timestamp{}, hlc.ClockTimestamp{}, nil /* txn */, &tombstone, + ctx, eng, keys.RangeTombstoneKey(rangeID), hlc.Timestamp{}, &tombstone, storage.MVCCWriteOptions{}, ); err != nil { t.Fatal(err) } diff --git a/pkg/kv/kvserver/stores.go b/pkg/kv/kvserver/stores.go index 064e792264b2..5109ea28bab4 100644 --- a/pkg/kv/kvserver/stores.go +++ b/pkg/kv/kvserver/stores.go @@ -296,7 +296,7 @@ func (ls *Stores) updateBootstrapInfoLocked(bi *gossip.BootstrapInfo) error { ls.storeMap.Range(func(k int64, v unsafe.Pointer) bool { s := (*Store)(v) // TODO(sep-raft-log): see ReadBootstrapInfo. - err = storage.MVCCPutProto(ctx, s.TODOEngine(), nil, keys.StoreGossipKey(), hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, bi) + err = storage.MVCCPutProto(ctx, s.TODOEngine(), keys.StoreGossipKey(), hlc.Timestamp{}, bi, storage.MVCCWriteOptions{}) return err == nil }) return err diff --git a/pkg/kv/kvserver/txn_wait_queue_test.go b/pkg/kv/kvserver/txn_wait_queue_test.go index 32e00bdb5000..b803ac02a440 100644 --- a/pkg/kv/kvserver/txn_wait_queue_test.go +++ b/pkg/kv/kvserver/txn_wait_queue_test.go @@ -39,9 +39,9 @@ import ( "github.com/stretchr/testify/require" ) -func writeTxnRecord(ctx context.Context, tc *testContext, txn *roachpb.Transaction) error { - key := keys.TransactionKey(txn.Key, txn.ID) - return storage.MVCCPutProto(ctx, tc.store.TODOEngine(), nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, txn) +func writeTxnRecord(ctx context.Context, tc *testContext, txnRecord *roachpb.Transaction) error { + key := keys.TransactionKey(txnRecord.Key, txnRecord.ID) + return storage.MVCCPutProto(ctx, tc.store.TODOEngine(), key, hlc.Timestamp{}, txnRecord, storage.MVCCWriteOptions{}) } // createTxnForPushQueue creates a txn struct and writes a "fake" diff --git a/pkg/server/node_test.go b/pkg/server/node_test.go index 4ea8481bb159..85c7c6ff0557 100644 --- a/pkg/server/node_test.go +++ b/pkg/server/node_test.go @@ -272,7 +272,7 @@ func TestCorruptedClusterID(t *testing.T) { StoreID: 1, } if err := storage.MVCCPutProto( - ctx, e, nil /* ms */, keys.StoreIdentKey(), hlc.Timestamp{}, hlc.ClockTimestamp{}, nil /* txn */, &sIdent, + ctx, e, keys.StoreIdentKey(), hlc.Timestamp{}, &sIdent, storage.MVCCWriteOptions{}, ); err != nil { t.Fatal(err) } @@ -938,8 +938,8 @@ func TestDiskStatsMap(t *testing.T) { engineIDs := []roachpb.StoreID{10, 5} for i := range engines { ident := roachpb.StoreIdent{StoreID: engineIDs[i]} - require.NoError(t, storage.MVCCBlindPutProto(ctx, engines[i], nil, keys.StoreIdentKey(), - hlc.Timestamp{}, hlc.ClockTimestamp{}, &ident, nil)) + require.NoError(t, storage.MVCCBlindPutProto(ctx, engines[i], keys.StoreIdentKey(), + hlc.Timestamp{}, &ident, storage.MVCCWriteOptions{})) } var dsm diskStatsMap clusterProvisionedBW := int64(150) diff --git a/pkg/server/node_tombstone_storage.go b/pkg/server/node_tombstone_storage.go index 312a97c44a40..9d7a576ef247 100644 --- a/pkg/server/node_tombstone_storage.go +++ b/pkg/server/node_tombstone_storage.go @@ -136,7 +136,7 @@ func (s *nodeTombstoneStorage) SetDecommissioned( } if err := storage.MVCCPut( - ctx, eng, nil /* ms */, k, hlc.Timestamp{}, hlc.ClockTimestamp{}, v, nil, /* txn */ + ctx, eng, k, hlc.Timestamp{}, v, storage.MVCCWriteOptions{}, ); err != nil { return err } diff --git a/pkg/server/server_test.go b/pkg/server/server_test.go index 4114dc68205c..dac3c9a2dbae 100644 --- a/pkg/server/server_test.go +++ b/pkg/server/server_test.go @@ -484,9 +484,8 @@ func TestClusterIDMismatch(t *testing.T) { StoreID: roachpb.StoreID(i + 1), } if err := storage.MVCCPutProto( - context.Background(), e, nil, keys.StoreIdentKey(), hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &sIdent, + context.Background(), e, keys.StoreIdentKey(), hlc.Timestamp{}, &sIdent, storage.MVCCWriteOptions{}, ); err != nil { - t.Fatal(err) } engines[i] = e @@ -1088,8 +1087,8 @@ func TestAssertEnginesEmpty(t *testing.T) { require.NoError(t, assertEnginesEmpty([]storage.Engine{eng})) - require.NoError(t, storage.MVCCPutProto(ctx, eng, nil, keys.DeprecatedStoreClusterVersionKey(), - hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &roachpb.Version{Major: 21, Minor: 1, Internal: 122})) + require.NoError(t, storage.MVCCPutProto(ctx, eng, keys.DeprecatedStoreClusterVersionKey(), + hlc.Timestamp{}, &roachpb.Version{Major: 21, Minor: 1, Internal: 122}, storage.MVCCWriteOptions{})) require.NoError(t, assertEnginesEmpty([]storage.Engine{eng})) batch := eng.NewBatch() diff --git a/pkg/server/settings_cache.go b/pkg/server/settings_cache.go index 0067a1a06bd8..2e73ada82704 100644 --- a/pkg/server/settings_cache.go +++ b/pkg/server/settings_cache.go @@ -102,7 +102,7 @@ func storeCachedSettingsKVs(ctx context.Context, eng storage.Engine, kvs []roach for _, kv := range kvs { kv.Value.Timestamp = hlc.Timestamp{} // nb: Timestamp is not part of checksum if err := storage.MVCCPut( - ctx, batch, nil, keys.StoreCachedSettingsKey(kv.Key), hlc.Timestamp{}, hlc.ClockTimestamp{}, kv.Value, nil, + ctx, batch, keys.StoreCachedSettingsKey(kv.Key), hlc.Timestamp{}, kv.Value, storage.MVCCWriteOptions{}, ); err != nil { return err } diff --git a/pkg/storage/bench_data_test.go b/pkg/storage/bench_data_test.go index 5f8f21949e90..61522d5cda1e 100644 --- a/pkg/storage/bench_data_test.go +++ b/pkg/storage/bench_data_test.go @@ -326,7 +326,7 @@ func (d mvccBenchData) Build(ctx context.Context, b *testing.B, eng Engine) erro txn.ReadTimestamp = ts txn.WriteTimestamp = ts } - require.NoError(b, MVCCPut(ctx, batch, nil, key, ts, hlc.ClockTimestamp{}, value, txn)) + require.NoError(b, MVCCPut(ctx, batch, key, ts, value, MVCCWriteOptions{Txn: txn})) } resolveLastIntent := func(batch Batch, idx int) { diff --git a/pkg/storage/bench_pebble_test.go b/pkg/storage/bench_pebble_test.go index d0f7fbbdca4c..c5efbd19d972 100644 --- a/pkg/storage/bench_pebble_test.go +++ b/pkg/storage/bench_pebble_test.go @@ -361,10 +361,10 @@ func BenchmarkMVCCPutDelete_Pebble(b *testing.B) { key := encoding.EncodeVarintAscending(nil, blockID) key = encoding.EncodeVarintAscending(key, blockNum) - if err := MVCCPut(ctx, db, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, value, nil); err != nil { + if err := MVCCPut(ctx, db, key, hlc.Timestamp{}, value, MVCCWriteOptions{}); err != nil { b.Fatal(err) } - if _, err := MVCCDelete(ctx, db, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := MVCCDelete(ctx, db, key, hlc.Timestamp{}, MVCCWriteOptions{}); err != nil { b.Fatal(err) } } diff --git a/pkg/storage/bench_test.go b/pkg/storage/bench_test.go index 03104108d3e9..050567efd66c 100644 --- a/pkg/storage/bench_test.go +++ b/pkg/storage/bench_test.go @@ -318,7 +318,7 @@ func setupKeysWithIntent( putTxn = &otherTxn } key := makeKey(nil, j) - require.NoError(b, MVCCPut(context.Background(), batch, nil, key, ts, hlc.ClockTimestamp{}, value, putTxn)) + require.NoError(b, MVCCPut(context.Background(), batch, key, ts, value, MVCCWriteOptions{Txn: putTxn})) } require.NoError(b, batch.Commit(true)) batch.Close() @@ -714,7 +714,7 @@ func loadTestData(dir string, numKeys, numBatches, batchTimeSpan, valueBytes int timestamp := hlc.Timestamp{WallTime: minWallTime + rand.Int63n(int64(batchTimeSpan))} value := roachpb.MakeValueFromBytes(randutil.RandBytes(rng, valueBytes)) value.InitChecksum(key) - if err := MVCCPut(ctx, batch, nil, key, timestamp, hlc.ClockTimestamp{}, value, nil); err != nil { + if err := MVCCPut(ctx, batch, key, timestamp, value, MVCCWriteOptions{}); err != nil { return nil, err } } @@ -889,7 +889,7 @@ func runMVCCPut( for j := 0; j < versions; j++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(i))) ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} - if err := MVCCPut(ctx, rw, nil, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { + if err := MVCCPut(ctx, rw, key, ts, value, MVCCWriteOptions{}); err != nil { b.Fatalf("failed put: %+v", err) } } @@ -912,7 +912,7 @@ func runMVCCBlindPut(ctx context.Context, b *testing.B, emk engineMaker, valueSi for i := 0; i < b.N; i++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(i))) ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} - if err := MVCCBlindPut(ctx, eng, nil, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { + if err := MVCCBlindPut(ctx, eng, key, ts, value, MVCCWriteOptions{}); err != nil { b.Fatalf("failed put: %+v", err) } } @@ -936,7 +936,7 @@ func runMVCCConditionalPut( for i := 0; i < b.N; i++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(i))) ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} - if err := MVCCPut(ctx, eng, nil, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { + if err := MVCCPut(ctx, eng, key, ts, value, MVCCWriteOptions{}); err != nil { b.Fatalf("failed put: %+v", err) } } @@ -948,7 +948,7 @@ func runMVCCConditionalPut( for i := 0; i < b.N; i++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(i))) ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} - if err := MVCCConditionalPut(ctx, eng, nil, key, ts, hlc.ClockTimestamp{}, value, expected, CPutFailIfMissing, nil); err != nil { + if err := MVCCConditionalPut(ctx, eng, key, ts, value, expected, CPutFailIfMissing, MVCCWriteOptions{}); err != nil { b.Fatalf("failed put: %+v", err) } } @@ -971,7 +971,7 @@ func runMVCCBlindConditionalPut(ctx context.Context, b *testing.B, emk engineMak key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(i))) ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} if err := MVCCBlindConditionalPut( - ctx, eng, nil, key, ts, hlc.ClockTimestamp{}, value, nil, CPutFailIfMissing, nil, + ctx, eng, key, ts, value, nil, CPutFailIfMissing, MVCCWriteOptions{}, ); err != nil { b.Fatalf("failed put: %+v", err) } @@ -994,7 +994,7 @@ func runMVCCInitPut(ctx context.Context, b *testing.B, emk engineMaker, valueSiz for i := 0; i < b.N; i++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(i))) ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} - if err := MVCCInitPut(ctx, eng, nil, key, ts, hlc.ClockTimestamp{}, value, false, nil); err != nil { + if err := MVCCInitPut(ctx, eng, key, ts, value, false, MVCCWriteOptions{}); err != nil { b.Fatalf("failed put: %+v", err) } } @@ -1016,7 +1016,7 @@ func runMVCCBlindInitPut(ctx context.Context, b *testing.B, emk engineMaker, val for i := 0; i < b.N; i++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(i))) ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} - if err := MVCCBlindInitPut(ctx, eng, nil, key, ts, hlc.ClockTimestamp{}, value, false, nil); err != nil { + if err := MVCCBlindInitPut(ctx, eng, key, ts, value, false, MVCCWriteOptions{}); err != nil { b.Fatalf("failed put: %+v", err) } } @@ -1046,7 +1046,7 @@ func runMVCCBatchPut(ctx context.Context, b *testing.B, emk engineMaker, valueSi for j := i; j < end; j++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(j))) ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} - if err := MVCCPut(ctx, batch, nil, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { + if err := MVCCPut(ctx, batch, key, ts, value, MVCCWriteOptions{}); err != nil { b.Fatalf("failed put: %+v", err) } } @@ -1179,13 +1179,13 @@ func runMVCCDeleteRange(ctx context.Context, b *testing.B, valueBytes int) { if _, _, _, err := MVCCDeleteRange( ctx, eng, - &enginepb.MVCCStats{}, keys.LocalMax, roachpb.KeyMax, math.MaxInt64, hlc.MaxTimestamp, - hlc.ClockTimestamp{}, - nil, + MVCCWriteOptions{ + Stats: &enginepb.MVCCStats{}, + }, false, ); err != nil { b.Fatal(err) @@ -1432,8 +1432,7 @@ func runMVCCGarbageCollect( break } for _, key := range pointKeys { - if err := MVCCPut(ctx, batch, nil, key, pts, hlc.ClockTimestamp{}, val, - nil); err != nil { + if err := MVCCPut(ctx, batch, key, pts, val, MVCCWriteOptions{}); err != nil { b.Fatal(err) } } @@ -1493,7 +1492,7 @@ func runBatchApplyBatchRepr( for i := 0; i < batchSize; i++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(order[i]))) ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} - if err := MVCCBlindPut(ctx, batch, nil, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { + if err := MVCCBlindPut(ctx, batch, key, ts, value, MVCCWriteOptions{}); err != nil { b.Fatal(err) } } @@ -1902,7 +1901,7 @@ func BenchmarkMVCCScannerWithIntentsAndVersions(b *testing.B) { // Put the keys for this iteration. for j := 0; j < numKeys; j++ { key := makeKey(nil, j) - require.NoError(b, MVCCPut(ctx, batch, nil, key, ts, hlc.ClockTimestamp{}, value, &txn)) + require.NoError(b, MVCCPut(ctx, batch, key, ts, value, MVCCWriteOptions{Txn: &txn})) } numPrevKeys = numKeys // Read the keys from the Batch and write them to a sstable to ingest. diff --git a/pkg/storage/engine_test.go b/pkg/storage/engine_test.go index 2d38125772d8..3dd429e1f73b 100644 --- a/pkg/storage/engine_test.go +++ b/pkg/storage/engine_test.go @@ -177,13 +177,13 @@ func TestEngineBatchStaleCachedIterator(t *testing.T) { // Put a value so that the deletion below finds a value to seek // to. - if err := MVCCPut(context.Background(), batch, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("x"), nil); err != nil { + if err := MVCCPut(context.Background(), batch, key, hlc.Timestamp{}, roachpb.MakeValueFromString("x"), MVCCWriteOptions{}); err != nil { t.Fatal(err) } // Seek the iterator to `key` and clear the value (but without // telling the iterator about that). - if _, err := MVCCDelete(context.Background(), batch, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := MVCCDelete(context.Background(), batch, key, hlc.Timestamp{}, MVCCWriteOptions{}); err != nil { t.Fatal(err) } @@ -1473,7 +1473,7 @@ func TestGetIntent(t *testing.T) { for _, keyName := range []string{"a", "aa"} { key := roachpb.Key(keyName) - err := MVCCPut(ctx, reader, nil, key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, roachpb.Value{RawBytes: key}, txn1) + err := MVCCPut(ctx, reader, key, txn1.ReadTimestamp, roachpb.Value{RawBytes: key}, MVCCWriteOptions{Txn: txn1}) require.NoError(t, err) } @@ -1482,7 +1482,7 @@ func TestGetIntent(t *testing.T) { txn2 := &roachpb.Transaction{TxnMeta: enginepb.TxnMeta{Key: roachpb.Key("a"), ID: txn2ID, Epoch: 2, WriteTimestamp: txn2TS, MinTimestamp: txn2TS, CoordinatorNodeID: 2}, ReadTimestamp: txn2TS} key := roachpb.Key("b") - err = MVCCPut(ctx, reader, nil, key, txn2.ReadTimestamp, hlc.ClockTimestamp{}, roachpb.Value{RawBytes: key}, txn2) + err = MVCCPut(ctx, reader, key, txn2.ReadTimestamp, roachpb.Value{RawBytes: key}, MVCCWriteOptions{Txn: txn2}) require.NoError(t, err) tests := []struct { @@ -1557,7 +1557,7 @@ func TestScanIntents(t *testing.T) { defer eng.Close() for _, key := range keys { - err := MVCCPut(ctx, eng, nil, key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, roachpb.Value{RawBytes: key}, txn1) + err := MVCCPut(ctx, eng, key, txn1.ReadTimestamp, roachpb.Value{RawBytes: key}, MVCCWriteOptions{Txn: txn1}) require.NoError(t, err) } @@ -1604,22 +1604,21 @@ func TestEngineClearRange(t *testing.T) { // // However, certain clearers cannot clear intents, range keys, or point keys. writeInitialData := func(t *testing.T, rw ReadWriter) { - var localTS hlc.ClockTimestamp txn := roachpb.MakeTransaction("test", nil, isolation.Serializable, roachpb.NormalUserPriority, wallTS(6), 1, 1) - require.NoError(t, MVCCPut(ctx, rw, nil, roachpb.Key("c"), wallTS(1), localTS, stringValue("c1").Value, nil)) + require.NoError(t, MVCCPut(ctx, rw, roachpb.Key("c"), wallTS(1), stringValue("c1").Value, MVCCWriteOptions{})) require.NoError(t, rw.PutMVCCRangeKey(rangeKey("d", "h", 1), MVCCValue{})) require.NoError(t, rw.PutMVCCRangeKey(rangeKey("a", "f", 2), MVCCValue{})) - require.NoError(t, MVCCPut(ctx, rw, nil, roachpb.Key("g"), wallTS(2), localTS, stringValue("g2").Value, nil)) - require.NoError(t, MVCCPut(ctx, rw, nil, roachpb.Key("e"), wallTS(3), localTS, stringValue("e3").Value, nil)) + require.NoError(t, MVCCPut(ctx, rw, roachpb.Key("g"), wallTS(2), stringValue("g2").Value, MVCCWriteOptions{})) + require.NoError(t, MVCCPut(ctx, rw, roachpb.Key("e"), wallTS(3), stringValue("e3").Value, MVCCWriteOptions{})) require.NoError(t, rw.PutMVCCRangeKey(rangeKey("a", "f", 4), MVCCValue{})) require.NoError(t, rw.PutMVCCRangeKey(rangeKey("g", "h", 4), MVCCValue{})) - require.NoError(t, MVCCPut(ctx, rw, nil, roachpb.Key("a"), wallTS(5), localTS, stringValue("a2").Value, nil)) - require.NoError(t, MVCCPut(ctx, rw, nil, roachpb.Key("b"), wallTS(5), localTS, stringValue("b2").Value, nil)) - require.NoError(t, MVCCPut(ctx, rw, nil, roachpb.Key("c"), wallTS(5), localTS, stringValue("c2").Value, nil)) - require.NoError(t, MVCCPut(ctx, rw, nil, roachpb.Key("a"), wallTS(6), localTS, stringValue("a6").Value, &txn)) - require.NoError(t, MVCCPut(ctx, rw, nil, roachpb.Key("b"), wallTS(6), localTS, stringValue("b6").Value, &txn)) - require.NoError(t, MVCCPut(ctx, rw, nil, roachpb.Key("e"), wallTS(6), localTS, stringValue("e6").Value, &txn)) - require.NoError(t, MVCCPut(ctx, rw, nil, roachpb.Key("g"), wallTS(6), localTS, stringValue("g6").Value, &txn)) + require.NoError(t, MVCCPut(ctx, rw, roachpb.Key("a"), wallTS(5), stringValue("a2").Value, MVCCWriteOptions{})) + require.NoError(t, MVCCPut(ctx, rw, roachpb.Key("b"), wallTS(5), stringValue("b2").Value, MVCCWriteOptions{})) + require.NoError(t, MVCCPut(ctx, rw, roachpb.Key("c"), wallTS(5), stringValue("c2").Value, MVCCWriteOptions{})) + require.NoError(t, MVCCPut(ctx, rw, roachpb.Key("a"), wallTS(6), stringValue("a6").Value, MVCCWriteOptions{Txn: &txn})) + require.NoError(t, MVCCPut(ctx, rw, roachpb.Key("b"), wallTS(6), stringValue("b6").Value, MVCCWriteOptions{Txn: &txn})) + require.NoError(t, MVCCPut(ctx, rw, roachpb.Key("e"), wallTS(6), stringValue("e6").Value, MVCCWriteOptions{Txn: &txn})) + require.NoError(t, MVCCPut(ctx, rw, roachpb.Key("g"), wallTS(6), stringValue("g6").Value, MVCCWriteOptions{Txn: &txn})) } start, end := roachpb.Key("b"), roachpb.Key("g") @@ -2082,7 +2081,7 @@ func TestScanConflictingIntentsForDroppingLatchesEarly(t *testing.T) { setup: func(t *testing.T, rw ReadWriter, txn *roachpb.Transaction) { conflictingTxn := newTxn(belowTxnTS) // test txn should see this intent err := MVCCPut( - ctx, rw, nil, keyA, conflictingTxn.WriteTimestamp, hlc.ClockTimestamp{}, val, conflictingTxn, + ctx, rw, keyA, conflictingTxn.WriteTimestamp, val, MVCCWriteOptions{Txn: conflictingTxn}, ) require.NoError(t, err) }, @@ -2096,7 +2095,7 @@ func TestScanConflictingIntentsForDroppingLatchesEarly(t *testing.T) { setup: func(t *testing.T, rw ReadWriter, txn *roachpb.Transaction) { conflictingTxn := newTxn(aboveTxnTS) // test txn shouldn't see this intent err := MVCCPut( - ctx, rw, nil, keyA, conflictingTxn.WriteTimestamp, hlc.ClockTimestamp{}, val, conflictingTxn, + ctx, rw, keyA, conflictingTxn.WriteTimestamp, val, MVCCWriteOptions{Txn: conflictingTxn}, ) require.NoError(t, err) }, @@ -2108,7 +2107,7 @@ func TestScanConflictingIntentsForDroppingLatchesEarly(t *testing.T) { { name: "bounds do not include (latest) own write", setup: func(t *testing.T, rw ReadWriter, txn *roachpb.Transaction) { - err := MVCCPut(ctx, rw, nil, keyA, txn.WriteTimestamp, hlc.ClockTimestamp{}, val, txn) + err := MVCCPut(ctx, rw, keyA, txn.WriteTimestamp, val, MVCCWriteOptions{Txn: txn}) require.NoError(t, err) }, start: keyB, @@ -2339,7 +2338,7 @@ func TestScanConflictingIntentsForDroppingLatchesEarlyReadYourOwnWrites(t *testi txn.Sequence = tc.intentSequenceNumber txn.ReadTimestamp = tc.intentTS txn.WriteTimestamp = tc.intentTS - err := MVCCPut(ctx, eng, nil, keyA, txn.WriteTimestamp, hlc.ClockTimestamp{}, val, txn) + err := MVCCPut(ctx, eng, keyA, txn.WriteTimestamp, val, MVCCWriteOptions{Txn: txn}) require.NoError(t, err) // Set up the read. diff --git a/pkg/storage/metamorphic/operations.go b/pkg/storage/metamorphic/operations.go index 46ef730e0ce6..9b6e83f1291b 100644 --- a/pkg/storage/metamorphic/operations.go +++ b/pkg/storage/metamorphic/operations.go @@ -207,7 +207,7 @@ func (m mvccPutOp) run(ctx context.Context) string { txn.Sequence++ writer := m.m.getReadWriter(m.writer) - err := storage.MVCCPut(ctx, writer, nil, m.key, txn.ReadTimestamp, hlc.ClockTimestamp{}, m.value, txn) + err := storage.MVCCPut(ctx, writer, m.key, txn.ReadTimestamp, m.value, storage.MVCCWriteOptions{Txn: txn}) if err != nil { if writeTooOldErr := (*kvpb.WriteTooOldError)(nil); errors.As(err, &writeTooOldErr) { txn.WriteTimestamp.Forward(writeTooOldErr.ActualTimestamp) @@ -236,8 +236,8 @@ func (m mvccCPutOp) run(ctx context.Context) string { writer := m.m.getReadWriter(m.writer) txn.Sequence++ - err := storage.MVCCConditionalPut(ctx, writer, nil, m.key, - txn.ReadTimestamp, hlc.ClockTimestamp{}, m.value, m.expVal, true, txn) + err := storage.MVCCConditionalPut(ctx, writer, m.key, + txn.ReadTimestamp, m.value, m.expVal, true, storage.MVCCWriteOptions{Txn: txn}) if err != nil { if writeTooOldErr := (*kvpb.WriteTooOldError)(nil); errors.As(err, &writeTooOldErr) { txn.WriteTimestamp.Forward(writeTooOldErr.ActualTimestamp) @@ -265,7 +265,7 @@ func (m mvccInitPutOp) run(ctx context.Context) string { writer := m.m.getReadWriter(m.writer) txn.Sequence++ - err := storage.MVCCInitPut(ctx, writer, nil, m.key, txn.ReadTimestamp, hlc.ClockTimestamp{}, m.value, false, txn) + err := storage.MVCCInitPut(ctx, writer, m.key, txn.ReadTimestamp, m.value, false, storage.MVCCWriteOptions{Txn: txn}) if err != nil { if writeTooOldErr := (*kvpb.WriteTooOldError)(nil); errors.As(err, &writeTooOldErr) { txn.WriteTimestamp.Forward(writeTooOldErr.ActualTimestamp) @@ -298,8 +298,8 @@ func (m mvccDeleteRangeOp) run(ctx context.Context) string { txn.Sequence++ - keys, _, _, err := storage.MVCCDeleteRange(ctx, writer, nil, m.key, m.endKey, - 0, txn.WriteTimestamp, hlc.ClockTimestamp{}, txn, true) + keys, _, _, err := storage.MVCCDeleteRange(ctx, writer, m.key, m.endKey, + 0, txn.WriteTimestamp, storage.MVCCWriteOptions{Txn: txn}, true) if err != nil { return fmt.Sprintf("error: %s", err) } @@ -377,7 +377,7 @@ func (m mvccDeleteOp) run(ctx context.Context) string { writer := m.m.getReadWriter(m.writer) txn.Sequence++ - _, err := storage.MVCCDelete(ctx, writer, nil, m.key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn) + _, err := storage.MVCCDelete(ctx, writer, m.key, txn.ReadTimestamp, storage.MVCCWriteOptions{Txn: txn}) if err != nil { if writeTooOldErr := (*kvpb.WriteTooOldError)(nil); errors.As(err, &writeTooOldErr) { txn.WriteTimestamp.Forward(writeTooOldErr.ActualTimestamp) diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index ea92fb503f0b..bc5387c31fe6 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -832,19 +832,17 @@ func MVCCGetProto( func MVCCPutProto( ctx context.Context, rw ReadWriter, - ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, - localTimestamp hlc.ClockTimestamp, - txn *roachpb.Transaction, msg protoutil.Message, + opts MVCCWriteOptions, ) error { value := roachpb.Value{} if err := value.SetProto(msg); err != nil { return err } value.InitChecksum(key) - return MVCCPut(ctx, rw, ms, key, timestamp, localTimestamp, value, txn) + return MVCCPut(ctx, rw, key, timestamp, value, opts) } // MVCCBlindPutProto sets the given key to the protobuf-serialized byte string @@ -853,19 +851,17 @@ func MVCCPutProto( func MVCCBlindPutProto( ctx context.Context, writer Writer, - ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, - localTimestamp hlc.ClockTimestamp, msg protoutil.Message, - txn *roachpb.Transaction, + opts MVCCWriteOptions, ) error { value := roachpb.Value{} if err := value.SetProto(msg); err != nil { return err } value.InitChecksum(key) - return MVCCBlindPut(ctx, writer, ms, key, timestamp, localTimestamp, value, txn) + return MVCCBlindPut(ctx, writer, key, timestamp, value, opts) } // MVCCBlindPutInlineWithPrev updates an inline value using a blind put when the @@ -912,7 +908,7 @@ func MVCCBlindPutInlineWithPrev( } // TODO(jackson): Thread origMetaValSize through so that a resulting // ClearUnversioned sets ClearOptions.ValueSize[Known]. - return MVCCBlindPut(ctx, rw, ms, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, value, nil) + return MVCCBlindPut(ctx, rw, key, hlc.Timestamp{}, value, MVCCWriteOptions{Stats: ms}) } // LockTableView is a transaction-bound view into an in-memory collections of @@ -1497,17 +1493,15 @@ func (b *putBuffer) putIntentMeta( func MVCCPut( ctx context.Context, rw ReadWriter, - ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, - localTimestamp hlc.ClockTimestamp, value roachpb.Value, - txn *roachpb.Transaction, + opts MVCCWriteOptions, ) error { // If we're not tracking stats for the key and we're writing a non-versioned // key we can utilize a blind put to avoid reading any existing value. var iter MVCCIterator - blind := ms == nil && timestamp.IsEmpty() + blind := opts.Stats == nil && timestamp.IsEmpty() if !blind { iter = newMVCCIterator( rw, timestamp, false /* rangeKeyMasking */, false /* noInterleavedIntents */, IterOptions{ @@ -1517,7 +1511,7 @@ func MVCCPut( ) defer iter.Close() } - return mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, localTimestamp, value, txn, nil) + return mvccPutUsingIter(ctx, rw, iter, key, timestamp, value, nil, opts) } // MVCCBlindPut is a fast-path of MVCCPut. See the MVCCPut comments for details @@ -1533,14 +1527,12 @@ func MVCCPut( func MVCCBlindPut( ctx context.Context, writer Writer, - ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, - localTimestamp hlc.ClockTimestamp, value roachpb.Value, - txn *roachpb.Transaction, + opts MVCCWriteOptions, ) error { - return mvccPutUsingIter(ctx, writer, nil, ms, key, timestamp, localTimestamp, value, txn, nil) + return mvccPutUsingIter(ctx, writer, nil, key, timestamp, value, nil, opts) } // MVCCDelete marks the key deleted so that it will not be returned in @@ -1555,11 +1547,9 @@ func MVCCBlindPut( func MVCCDelete( ctx context.Context, rw ReadWriter, - ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, - localTimestamp hlc.ClockTimestamp, - txn *roachpb.Transaction, + opts MVCCWriteOptions, ) (foundKey bool, err error) { iter := newMVCCIterator( rw, timestamp, false /* rangeKeyMasking */, false /* noInterleavedIntents */, IterOptions{ @@ -1574,7 +1564,7 @@ func MVCCDelete( // TODO(yuzefovich): can we avoid the put if the key does not exist? return mvccPutInternal( - ctx, rw, iter, ms, key, timestamp, localTimestamp, noValue, txn, buf, nil) + ctx, rw, iter, key, timestamp, noValue, buf, nil, opts) } var noValue = roachpb.Value{} @@ -1587,13 +1577,11 @@ func mvccPutUsingIter( ctx context.Context, writer Writer, iter MVCCIterator, - ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, - localTimestamp hlc.ClockTimestamp, value roachpb.Value, - txn *roachpb.Transaction, valueFn func(optionalValue) (roachpb.Value, error), + opts MVCCWriteOptions, ) error { buf := newPutBuffer() defer buf.release() @@ -1601,7 +1589,7 @@ func mvccPutUsingIter( // Most callers don't care about the returned exReplaced value. The ones that // do can call mvccPutInternal directly. _, err := mvccPutInternal( - ctx, writer, iter, ms, key, timestamp, localTimestamp, value, txn, buf, valueFn) + ctx, writer, iter, key, timestamp, value, buf, valueFn, opts) return err } @@ -1761,6 +1749,7 @@ func replayTransactionalWrite( return errors.AssertionFailedf("transaction %s with sequence %d has a different value %+v after recomputing from what was written: %+v", txn.ID, txn.Sequence, value.RawBytes, writtenValue.RawBytes) } + return nil } @@ -1802,7 +1791,7 @@ func replayTransactionalWrite( // parameter be set to hlc.Timestamp{} when writing transactionally, but // hlc.Timestamp{} is already used as a sentinel for inline puts.) // -// The local timestamp parameter dictates the local clock timestamp +// The opts.LocalTimestamp parameter dictates the local clock timestamp // assigned to the key-value. It should be taken from the local HLC // clock on the leaseholder that is performing the write and must be // below the leaseholder's lease expiration. If the supplied local @@ -1814,14 +1803,12 @@ func mvccPutInternal( ctx context.Context, writer Writer, iter MVCCIterator, - ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, - localTimestamp hlc.ClockTimestamp, value roachpb.Value, - txn *roachpb.Transaction, buf *putBuffer, valueFn func(optionalValue) (roachpb.Value, error), + opts MVCCWriteOptions, ) (bool, error) { if len(key) == 0 { return false, emptyKeyError() @@ -1850,7 +1837,7 @@ func mvccPutInternal( // aren't allowed within transactions. MVCC range tombstones cannot exist // across them either. if putIsInline { - if txn != nil { + if opts.Txn != nil { return false, errors.Errorf("%q: inline writes not allowed within transactions", metaKey) } var metaKeySize, metaValSize int64 @@ -1868,8 +1855,8 @@ func mvccPutInternal( buf.meta = enginepb.MVCCMetadata{RawBytes: value.RawBytes} metaKeySize, metaValSize, err = buf.putInlineMeta(writer, metaKey, &buf.meta) } - if ms != nil { - updateStatsForInline(ms, key, origMetaKeySize, origMetaValSize, metaKeySize, metaValSize) + if opts.Stats != nil { + updateStatsForInline(opts.Stats, key, origMetaKeySize, origMetaValSize, metaKeySize, metaValSize) } return ok && !buf.meta.Deleted, err } @@ -1881,14 +1868,14 @@ func mvccPutInternal( // definition for rationale. readTimestamp := timestamp writeTimestamp := timestamp - if txn != nil { - readTimestamp = txn.ReadTimestamp + if opts.Txn != nil { + readTimestamp = opts.Txn.ReadTimestamp if readTimestamp != timestamp { return false, errors.AssertionFailedf( "mvccPutInternal: txn's read timestamp %s does not match timestamp %s", readTimestamp, timestamp) } - writeTimestamp = txn.WriteTimestamp + writeTimestamp = opts.Txn.WriteTimestamp } timestamp = hlc.Timestamp{} // prevent accidental use below @@ -1896,7 +1883,7 @@ func mvccPutInternal( // Determine what the logical operation is. Are we writing an intent // or a value directly? logicalOp := MVCCWriteValueOpType - if txn != nil { + if opts.Txn != nil { logicalOp = MVCCWriteIntentOpType } @@ -1918,20 +1905,20 @@ func mvccPutInternal( // handling, since they cannot be transactional. if meta.Txn != nil { // There is an uncommitted write intent. - if txn == nil || meta.Txn.ID != txn.ID { + if opts.Txn == nil || meta.Txn.ID != opts.Txn.ID { // The current Put operation does not come from the same // transaction. return false, &kvpb.WriteIntentError{Intents: []roachpb.Intent{ roachpb.MakeIntent(meta.Txn, key), }} - } else if txn.Epoch < meta.Txn.Epoch { + } else if opts.Txn.Epoch < meta.Txn.Epoch { return false, errors.Errorf("put with epoch %d came after put with epoch %d in txn %s", - txn.Epoch, meta.Txn.Epoch, txn.ID) - } else if txn.Epoch == meta.Txn.Epoch && txn.Sequence <= meta.Txn.Sequence { + opts.Txn.Epoch, meta.Txn.Epoch, opts.Txn.ID) + } else if opts.Txn.Epoch == meta.Txn.Epoch && opts.Txn.Sequence <= meta.Txn.Sequence { // 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, txn, valueFn) + return false, replayTransactionalWrite(ctx, iter, meta, key, readTimestamp, value, opts.Txn, valueFn) } // We're overwriting the intent that was present at this key, before we do @@ -1961,8 +1948,8 @@ func mvccPutInternal( // Set when the current provisional value is not ignored due to a txn // restart or a savepoint rollback. Represents an encoded MVCCValue. var curProvValRaw []byte - if txn.Epoch == meta.Txn.Epoch /* last write inside txn */ { - if !enginepb.TxnSeqIsIgnored(meta.Txn.Sequence, txn.IgnoredSeqNums) { + if opts.Txn.Epoch == meta.Txn.Epoch /* last write inside txn */ { + if !enginepb.TxnSeqIsIgnored(meta.Txn.Sequence, opts.Txn.IgnoredSeqNums) { // Seqnum of last write is not ignored. Retrieve the value. iter.SeekGE(oldVersionKey) var hasPoint bool @@ -1988,7 +1975,7 @@ func mvccPutInternal( exVal = makeOptionalValue(curIntentVal.Value) } else { // Seqnum of last write was ignored. Try retrieving the value from the history. - prevIntent, prevIntentOk := meta.GetPrevIntentSeq(txn.Sequence, txn.IgnoredSeqNums) + prevIntent, prevIntentOk := meta.GetPrevIntentSeq(opts.Txn.Sequence, opts.Txn.IgnoredSeqNums) if prevIntentOk { prevIntentVal, err := DecodeMVCCValue(prevIntent.Value) if err != nil { @@ -2110,7 +2097,7 @@ func mvccPutInternal( // transaction are ignored, and there are no past committed values // at this key. In that case, we can also blow up the intent // history. - if txn.Epoch == meta.Txn.Epoch && exVal.exists { + if opts.Txn.Epoch == meta.Txn.Epoch && exVal.exists { // Only add the current provisional value to the intent // history if the current sequence number is not ignored. There's no // reason to add past committed values or a value already in the intent @@ -2161,7 +2148,7 @@ func mvccPutInternal( versionValue := MVCCValue{} versionValue.Value = value - versionValue.LocalTimestamp = localTimestamp + versionValue.LocalTimestamp = opts.LocalTimestamp if buildutil.CrdbTestBuild { if seq, seqOK := kvnemesisutil.FromContext(ctx); seqOK { @@ -2181,8 +2168,8 @@ func mvccPutInternal( newMeta := &buf.newMeta { var txnMeta *enginepb.TxnMeta - if txn != nil { - txnMeta = &txn.TxnMeta + if opts.Txn != nil { + txnMeta = &opts.Txn.TxnMeta // If we bumped the WriteTimestamp, we update both the TxnMeta and the // MVCCMetadata.Timestamp. if txnMeta.WriteTimestamp != versionKey.Timestamp { @@ -2232,7 +2219,7 @@ func mvccPutInternal( } // Update MVCC stats. - if ms != nil { + if opts.Stats != nil { // Adjust the stats metadata for MVCC range tombstones. The MVCC stats // update only cares about changes to real point keys, but the above logic // needs to care about MVCC range tombstones for conflict purposes. @@ -2248,7 +2235,7 @@ func mvccPutInternal( if meta != nil { meta.Timestamp = origRealKeyChanged.ToLegacyTimestamp() } - ms.Add(updateStatsOnPut(key, prevIsValue, prevValSize, origMetaKeySize, origMetaValSize, + opts.Stats.Add(updateStatsOnPut(key, prevIsValue, prevValSize, origMetaKeySize, origMetaValSize, metaKeySize, metaValSize, meta, newMeta)) } @@ -2279,11 +2266,9 @@ func mvccPutInternal( func MVCCIncrement( ctx context.Context, rw ReadWriter, - ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, - localTimestamp hlc.ClockTimestamp, - txn *roachpb.Transaction, + opts MVCCWriteOptions, inc int64, ) (int64, error) { iter := newMVCCIterator( @@ -2321,7 +2306,8 @@ func MVCCIncrement( newValue.InitChecksum(key) return newValue, nil } - err := mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, localTimestamp, noValue, txn, valueFn) + + err := mvccPutUsingIter(ctx, rw, iter, key, timestamp, noValue, valueFn, opts) return newInt64Val, err } @@ -2356,14 +2342,12 @@ const ( func MVCCConditionalPut( ctx context.Context, rw ReadWriter, - ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, - localTimestamp hlc.ClockTimestamp, value roachpb.Value, expVal []byte, allowIfDoesNotExist CPutMissingBehavior, - txn *roachpb.Transaction, + opts MVCCWriteOptions, ) error { iter := newMVCCIterator( rw, timestamp, false /* rangeKeyMasking */, false /* noInterleavedIntents */, IterOptions{ @@ -2374,7 +2358,7 @@ func MVCCConditionalPut( defer iter.Close() return mvccConditionalPutUsingIter( - ctx, rw, iter, ms, key, timestamp, localTimestamp, value, expVal, allowIfDoesNotExist, txn) + ctx, rw, iter, key, timestamp, value, expVal, allowIfDoesNotExist, opts) } // MVCCBlindConditionalPut is a fast-path of MVCCConditionalPut. See the @@ -2389,31 +2373,27 @@ func MVCCConditionalPut( func MVCCBlindConditionalPut( ctx context.Context, writer Writer, - ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, - localTimestamp hlc.ClockTimestamp, value roachpb.Value, expVal []byte, allowIfDoesNotExist CPutMissingBehavior, - txn *roachpb.Transaction, + opts MVCCWriteOptions, ) error { return mvccConditionalPutUsingIter( - ctx, writer, nil, ms, key, timestamp, localTimestamp, value, expVal, allowIfDoesNotExist, txn) + ctx, writer, nil, key, timestamp, value, expVal, allowIfDoesNotExist, opts) } func mvccConditionalPutUsingIter( ctx context.Context, writer Writer, iter MVCCIterator, - ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, - localTimestamp hlc.ClockTimestamp, value roachpb.Value, expBytes []byte, allowNoExisting CPutMissingBehavior, - txn *roachpb.Transaction, + opts MVCCWriteOptions, ) error { valueFn := func(existVal optionalValue) (roachpb.Value, error) { if expValPresent, existValPresent := len(expBytes) != 0, existVal.IsPresent(); expValPresent && existValPresent { @@ -2429,7 +2409,7 @@ func mvccConditionalPutUsingIter( } return value, nil } - return mvccPutUsingIter(ctx, writer, iter, ms, key, timestamp, localTimestamp, noValue, txn, valueFn) + return mvccPutUsingIter(ctx, writer, iter, key, timestamp, noValue, valueFn, opts) } // MVCCInitPut sets the value for a specified key if the key doesn't exist. It @@ -2444,13 +2424,11 @@ func mvccConditionalPutUsingIter( func MVCCInitPut( ctx context.Context, rw ReadWriter, - ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, - localTimestamp hlc.ClockTimestamp, value roachpb.Value, failOnTombstones bool, - txn *roachpb.Transaction, + opts MVCCWriteOptions, ) error { iter := newMVCCIterator( rw, timestamp, false /* rangeKeyMasking */, false /* noInterleavedIntents */, IterOptions{ @@ -2459,7 +2437,7 @@ func MVCCInitPut( }, ) defer iter.Close() - return mvccInitPutUsingIter(ctx, rw, iter, ms, key, timestamp, localTimestamp, value, failOnTombstones, txn) + return mvccInitPutUsingIter(ctx, rw, iter, key, timestamp, value, failOnTombstones, opts) } // MVCCBlindInitPut is a fast-path of MVCCInitPut. See the MVCCInitPut @@ -2473,29 +2451,25 @@ func MVCCInitPut( func MVCCBlindInitPut( ctx context.Context, rw ReadWriter, - ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, - localTimestamp hlc.ClockTimestamp, value roachpb.Value, failOnTombstones bool, - txn *roachpb.Transaction, + opts MVCCWriteOptions, ) error { return mvccInitPutUsingIter( - ctx, rw, nil, ms, key, timestamp, localTimestamp, value, failOnTombstones, txn) + ctx, rw, nil, key, timestamp, value, failOnTombstones, opts) } func mvccInitPutUsingIter( ctx context.Context, rw ReadWriter, iter MVCCIterator, - ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, - localTimestamp hlc.ClockTimestamp, value roachpb.Value, failOnTombstones bool, - txn *roachpb.Transaction, + opts MVCCWriteOptions, ) error { valueFn := func(existVal optionalValue) (roachpb.Value, error) { if failOnTombstones && existVal.IsTombstone() { @@ -2512,7 +2486,7 @@ func mvccInitPutUsingIter( } return value, nil } - return mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, localTimestamp, noValue, txn, valueFn) + return mvccPutUsingIter(ctx, rw, iter, key, timestamp, noValue, valueFn, opts) } // mvccKeyFormatter is an fmt.Formatter for MVCC Keys. @@ -3038,12 +3012,10 @@ func MVCCClearTimeRange( func MVCCDeleteRange( ctx context.Context, rw ReadWriter, - ms *enginepb.MVCCStats, key, endKey roachpb.Key, max int64, timestamp hlc.Timestamp, - localTimestamp hlc.ClockTimestamp, - txn *roachpb.Transaction, + opts MVCCWriteOptions, returnKeys bool, ) ([]roachpb.Key, *roachpb.Span, int64, error) { // Scan to find the keys to delete. @@ -3075,8 +3047,8 @@ func MVCCDeleteRange( // need to perform the initial scan at the previous sequence number so that // we don't see the result from equal or later sequences. var scanTxn *roachpb.Transaction - if txn != nil { - prevSeqTxn := txn.Clone() + if opts.Txn != nil { + prevSeqTxn := opts.Txn.Clone() prevSeqTxn.Sequence-- scanTxn = prevSeqTxn } @@ -3100,7 +3072,7 @@ func MVCCDeleteRange( var keys []roachpb.Key for i, kv := range res.KVs { if _, err := mvccPutInternal( - ctx, rw, iter, ms, kv.Key, timestamp, localTimestamp, noValue, txn, buf, nil, + ctx, rw, iter, kv.Key, timestamp, noValue, buf, nil, opts, ); err != nil { return nil, nil, 0, err } @@ -3281,8 +3253,8 @@ func MVCCPredicateDeleteRange( } else { // Use Point tombstones for i := int64(0); i < runSize; i++ { - if _, err := mvccPutInternal(ctx, rw, pointTombstoneIter, ms, buf[i], endTime, localTimestamp, noValue, - nil, pointTombstoneBuf, nil); err != nil { + if _, err := mvccPutInternal(ctx, rw, pointTombstoneIter, buf[i], endTime, noValue, + pointTombstoneBuf, nil, MVCCWriteOptions{LocalTimestamp: localTimestamp, Stats: ms}); err != nil { return err } } @@ -3432,6 +3404,9 @@ func MVCCPredicateDeleteRange( // When deleting an entire Raft range, passing the current MVCCStats as // msCovered and setting left/rightPeekBound to start/endKey will make the // deletion significantly faster. +// +// TODO(sarkesian): Consider accepting MVCCWriteOptions for this function +// and its relevant callers. func MVCCDeleteRangeUsingTombstone( ctx context.Context, rw ReadWriter, @@ -3919,6 +3894,14 @@ func buildScanIntents(data []byte) ([]roachpb.Intent, error) { return intents, nil } +// MVCCWriteOptions bundles options for the MVCCPut and MVCCDelete families of functions. +type MVCCWriteOptions struct { + // See the comment on mvccPutInternal for details on these parameters. + Txn *roachpb.Transaction + LocalTimestamp hlc.ClockTimestamp + Stats *enginepb.MVCCStats +} + // MVCCScanOptions bundles options for the MVCCScan family of functions. type MVCCScanOptions struct { // See the documentation for MVCCScan for information on these parameters. diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index d7c1a9f13b4d..fff579b168ad 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -1101,7 +1101,12 @@ func cmdCPut(e *evalCtx) error { resolve, resolveStatus := e.getResolve() return e.withWriter("cput", func(rw storage.ReadWriter) error { - if err := storage.MVCCConditionalPut(e.ctx, rw, e.ms, key, ts, localTs, val, expVal, behavior, txn); err != nil { + opts := storage.MVCCWriteOptions{ + Txn: txn, + LocalTimestamp: localTs, + Stats: e.ms, + } + if err := storage.MVCCConditionalPut(e.ctx, rw, key, ts, val, expVal, behavior, opts); err != nil { return err } if resolve { @@ -1122,7 +1127,12 @@ func cmdInitPut(e *evalCtx) error { resolve, resolveStatus := e.getResolve() return e.withWriter("initput", func(rw storage.ReadWriter) error { - if err := storage.MVCCInitPut(e.ctx, rw, e.ms, key, ts, localTs, val, failOnTombstones, txn); err != nil { + opts := storage.MVCCWriteOptions{ + Txn: txn, + LocalTimestamp: localTs, + Stats: e.ms, + } + if err := storage.MVCCInitPut(e.ctx, rw, key, ts, val, failOnTombstones, opts); err != nil { return err } if resolve { @@ -1139,7 +1149,12 @@ func cmdDelete(e *evalCtx) error { localTs := hlc.ClockTimestamp(e.getTsWithName("localTs")) resolve, resolveStatus := e.getResolve() return e.withWriter("del", func(rw storage.ReadWriter) error { - foundKey, err := storage.MVCCDelete(e.ctx, rw, e.ms, key, ts, localTs, txn) + opts := storage.MVCCWriteOptions{ + Txn: txn, + LocalTimestamp: localTs, + Stats: e.ms, + } + foundKey, err := storage.MVCCDelete(e.ctx, rw, key, ts, opts) if err == nil || errors.HasType(err, &kvpb.WriteTooOldError{}) { // We want to output foundKey even if a WriteTooOldError is returned, // since the error may be swallowed/deferred during evaluation. @@ -1168,8 +1183,13 @@ func cmdDeleteRange(e *evalCtx) error { resolve, resolveStatus := e.getResolve() return e.withWriter("del_range", func(rw storage.ReadWriter) error { + opts := storage.MVCCWriteOptions{ + Txn: txn, + LocalTimestamp: localTs, + Stats: e.ms, + } deleted, resumeSpan, num, err := storage.MVCCDeleteRange( - e.ctx, rw, e.ms, key, endKey, int64(max), ts, localTs, txn, returnKeys) + e.ctx, rw, key, endKey, int64(max), ts, opts, returnKeys) if err != nil { return err } @@ -1322,7 +1342,12 @@ func cmdIncrement(e *evalCtx) error { resolve, resolveStatus := e.getResolve() return e.withWriter("increment", func(rw storage.ReadWriter) error { - curVal, err := storage.MVCCIncrement(e.ctx, rw, e.ms, key, ts, localTs, txn, inc) + opts := storage.MVCCWriteOptions{ + Txn: txn, + LocalTimestamp: localTs, + Stats: e.ms, + } + curVal, err := storage.MVCCIncrement(e.ctx, rw, key, ts, opts, inc) if err != nil { return err } @@ -1358,7 +1383,12 @@ func cmdPut(e *evalCtx) error { resolve, resolveStatus := e.getResolve() return e.withWriter("put", func(rw storage.ReadWriter) error { - if err := storage.MVCCPut(e.ctx, rw, e.ms, key, ts, localTs, val, txn); err != nil { + opts := storage.MVCCWriteOptions{ + Txn: txn, + LocalTimestamp: localTs, + Stats: e.ms, + } + if err := storage.MVCCPut(e.ctx, rw, key, ts, val, opts); err != nil { return err } if resolve { diff --git a/pkg/storage/mvcc_incremental_iterator_test.go b/pkg/storage/mvcc_incremental_iterator_test.go index 9dd59ed6b1fc..7e272f838518 100644 --- a/pkg/storage/mvcc_incremental_iterator_test.go +++ b/pkg/storage/mvcc_incremental_iterator_test.go @@ -466,7 +466,7 @@ func TestMVCCIncrementalIteratorNextIgnoringTime(t *testing.T) { for _, kv := range kvs(kv1_1_1, kv1_2_2, kv2_2_2) { v := roachpb.Value{RawBytes: kv.Value} - if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { + if err := MVCCPut(ctx, e, kv.Key.Key, kv.Key.Timestamp, v, MVCCWriteOptions{}); err != nil { t.Fatal(err) } } @@ -505,7 +505,7 @@ func TestMVCCIncrementalIteratorNextIgnoringTime(t *testing.T) { }) // Exercise deletion. - if _, err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := MVCCDelete(ctx, e, testKey1, ts3, MVCCWriteOptions{}); err != nil { t.Fatal(err) } // Returns the kv_1_1_1 even though it is outside (startTime, endTime]. @@ -525,7 +525,7 @@ func TestMVCCIncrementalIteratorNextIgnoringTime(t *testing.T) { }, ReadTimestamp: ts4, } - if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn1); err != nil { + if err := MVCCPut(ctx, e, txn1.TxnMeta.Key, txn1.ReadTimestamp, testValue4, MVCCWriteOptions{Txn: &txn1}); err != nil { t.Fatal(err) } @@ -602,7 +602,7 @@ func TestMVCCIncrementalIteratorNextKeyIgnoringTime(t *testing.T) { for _, kv := range kvs(kv1_1_1, kv1_2_2, kv2_2_2) { v := roachpb.Value{RawBytes: kv.Value} - if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { + if err := MVCCPut(ctx, e, kv.Key.Key, kv.Key.Timestamp, v, MVCCWriteOptions{}); err != nil { t.Fatal(err) } } @@ -638,7 +638,7 @@ func TestMVCCIncrementalIteratorNextKeyIgnoringTime(t *testing.T) { }) // Exercise deletion. - if _, err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := MVCCDelete(ctx, e, testKey1, ts3, MVCCWriteOptions{}); err != nil { t.Fatal(err) } // Returns the kv_1_1_1 even though it is outside (startTime, endTime]. @@ -658,7 +658,7 @@ func TestMVCCIncrementalIteratorNextKeyIgnoringTime(t *testing.T) { }, ReadTimestamp: ts4, } - if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn1); err != nil { + if err := MVCCPut(ctx, e, txn1.TxnMeta.Key, txn1.ReadTimestamp, testValue4, MVCCWriteOptions{Txn: &txn1}); err != nil { t.Fatal(err) } @@ -723,7 +723,7 @@ func TestMVCCIncrementalIteratorInlinePolicy(t *testing.T) { defer e.Close() for _, kv := range []MVCCKeyValue{inline1_1_1, kv2_1_1, kv2_2_2, inline3_2_1} { v := roachpb.Value{RawBytes: kv.Value} - if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { + if err := MVCCPut(ctx, e, kv.Key.Key, kv.Key.Timestamp, v, MVCCWriteOptions{}); err != nil { t.Fatal(err) } } @@ -793,11 +793,11 @@ func TestMVCCIncrementalIteratorIntentPolicy(t *testing.T) { defer e.Close() for _, kv := range []MVCCKeyValue{kv1_1_1, kv1_2_2, kv1_3_3, kv2_1_1} { v := roachpb.Value{RawBytes: kv.Value} - if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { + if err := MVCCPut(ctx, e, kv.Key.Key, kv.Key.Timestamp, v, MVCCWriteOptions{}); err != nil { t.Fatal(err) } } - if err := MVCCPut(ctx, e, nil, txn.TxnMeta.Key, txn.ReadTimestamp, hlc.ClockTimestamp{}, testValue2, &txn); err != nil { + if err := MVCCPut(ctx, e, txn.TxnMeta.Key, txn.ReadTimestamp, testValue2, MVCCWriteOptions{Txn: &txn}); err != nil { t.Fatal(err) } t.Run("PolicyError returns error if an intent is in the time range", func(t *testing.T) { @@ -958,7 +958,7 @@ func TestMVCCIncrementalIterator(t *testing.T) { for _, kv := range kvs(kv1_1_1, kv1_2_2, kv2_2_2) { v := roachpb.Value{RawBytes: kv.Value} - if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { + if err := MVCCPut(ctx, e, kv.Key.Key, kv.Key.Timestamp, v, MVCCWriteOptions{}); err != nil { t.Fatal(err) } } @@ -976,18 +976,18 @@ func TestMVCCIncrementalIterator(t *testing.T) { t.Run("kv [1-2)", assertEqualKVs(e, testKey1, testKey2, tsMin, tsMax, latest, kvs(kv1_2_2))) // Exercise deletion. - if _, err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := MVCCDelete(ctx, e, testKey1, ts3, MVCCWriteOptions{}); err != nil { t.Fatal(err) } t.Run("del", assertEqualKVs(e, localMax, keyMax, ts1, tsMax, latest, kvs(kv1Deleted3, kv2_2_2))) // Exercise intent handling. txn1, intentErr1 := makeKVTxn(testKey1, ts4) - if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn1); err != nil { + if err := MVCCPut(ctx, e, txn1.TxnMeta.Key, txn1.ReadTimestamp, testValue4, MVCCWriteOptions{Txn: &txn1}); err != nil { t.Fatal(err) } txn2, intentErr2 := makeKVTxn(testKey2, ts4) - if err := MVCCPut(ctx, e, nil, txn2.TxnMeta.Key, txn2.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn2); err != nil { + if err := MVCCPut(ctx, e, txn2.TxnMeta.Key, txn2.ReadTimestamp, testValue4, MVCCWriteOptions{Txn: &txn2}); err != nil { t.Fatal(err) } t.Run("intents-1", @@ -1024,7 +1024,7 @@ func TestMVCCIncrementalIterator(t *testing.T) { for _, kv := range kvs(kv1_1_1, kv1_2_2, kv2_2_2) { v := roachpb.Value{RawBytes: kv.Value} - if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { + if err := MVCCPut(ctx, e, kv.Key.Key, kv.Key.Timestamp, v, MVCCWriteOptions{}); err != nil { t.Fatal(err) } } @@ -1042,18 +1042,18 @@ func TestMVCCIncrementalIterator(t *testing.T) { t.Run("kv [1-2)", assertEqualKVs(e, testKey1, testKey2, tsMin, tsMax, all, kvs(kv1_2_2, kv1_1_1))) // Exercise deletion. - if _, err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := MVCCDelete(ctx, e, testKey1, ts3, MVCCWriteOptions{}); err != nil { t.Fatal(err) } t.Run("del", assertEqualKVs(e, localMax, keyMax, ts1, tsMax, all, kvs(kv1Deleted3, kv1_2_2, kv2_2_2))) // Exercise intent handling. txn1, intentErr1 := makeKVTxn(testKey1, ts4) - if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn1); err != nil { + if err := MVCCPut(ctx, e, txn1.TxnMeta.Key, txn1.ReadTimestamp, testValue4, MVCCWriteOptions{Txn: &txn1}); err != nil { t.Fatal(err) } txn2, intentErr2 := makeKVTxn(testKey2, ts4) - if err := MVCCPut(ctx, e, nil, txn2.TxnMeta.Key, txn2.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn2); err != nil { + if err := MVCCPut(ctx, e, txn2.TxnMeta.Key, txn2.ReadTimestamp, testValue4, MVCCWriteOptions{Txn: &txn2}); err != nil { t.Fatal(err) } // Single intent tests are verifying behavior when intent collection is not enabled. @@ -1142,7 +1142,7 @@ func TestMVCCIncrementalIteratorIntentRewrittenConcurrently(t *testing.T) { }, ReadTimestamp: ts1, } - if err := MVCCPut(ctx, e, nil, kA, ts1, hlc.ClockTimestamp{}, vA1, txn); err != nil { + if err := MVCCPut(ctx, e, kA, ts1, vA1, MVCCWriteOptions{Txn: txn}); err != nil { t.Fatal(err) } @@ -1160,7 +1160,7 @@ func TestMVCCIncrementalIteratorIntentRewrittenConcurrently(t *testing.T) { // in intentInterleavingIter to be violated. b := e.NewBatch() defer b.Close() - if err := MVCCPut(ctx, b, nil, kA, ts1, hlc.ClockTimestamp{}, vA2, txn); err != nil { + if err := MVCCPut(ctx, b, kA, ts1, vA2, MVCCWriteOptions{Txn: txn}); err != nil { return err } return b.Commit(false) @@ -1255,17 +1255,17 @@ func TestMVCCIncrementalIteratorIntentDeletion(t *testing.T) { // kA:3 -> vA3 // kA:2 -> vA2 // kB -> (intent deletion) - require.NoError(t, MVCCPut(ctx, db, nil, kA, txnA1.ReadTimestamp, hlc.ClockTimestamp{}, vA1, txnA1)) - require.NoError(t, MVCCPut(ctx, db, nil, kB, txnB1.ReadTimestamp, hlc.ClockTimestamp{}, vB1, txnB1)) - require.NoError(t, MVCCPut(ctx, db, nil, kC, txnC1.ReadTimestamp, hlc.ClockTimestamp{}, vC1, txnC1)) + require.NoError(t, MVCCPut(ctx, db, kA, txnA1.ReadTimestamp, vA1, MVCCWriteOptions{Txn: txnA1})) + require.NoError(t, MVCCPut(ctx, db, kB, txnB1.ReadTimestamp, vB1, MVCCWriteOptions{Txn: txnB1})) + require.NoError(t, MVCCPut(ctx, db, kC, txnC1.ReadTimestamp, vC1, MVCCWriteOptions{Txn: txnC1})) require.NoError(t, db.Flush()) require.NoError(t, db.Compact()) _, _, _, err := MVCCResolveWriteIntent(ctx, db, nil, intent(txnA1), MVCCResolveWriteIntentOptions{}) require.NoError(t, err) _, _, _, err = MVCCResolveWriteIntent(ctx, db, nil, intent(txnB1), MVCCResolveWriteIntentOptions{}) require.NoError(t, err) - require.NoError(t, MVCCPut(ctx, db, nil, kA, ts2, hlc.ClockTimestamp{}, vA2, nil)) - require.NoError(t, MVCCPut(ctx, db, nil, kA, txnA3.WriteTimestamp, hlc.ClockTimestamp{}, vA3, txnA3)) + require.NoError(t, MVCCPut(ctx, db, kA, ts2, vA2, MVCCWriteOptions{})) + require.NoError(t, MVCCPut(ctx, db, kA, txnA3.WriteTimestamp, vA3, MVCCWriteOptions{Txn: txnA3})) require.NoError(t, db.Flush()) // The kA ts1 intent has been resolved. There's now a new intent on kA, but @@ -1314,7 +1314,7 @@ func TestMVCCIncrementalIteratorIntentStraddlesSStables(t *testing.T) { put := func(key, value string, ts int64, txn *roachpb.Transaction) { v := roachpb.MakeValueFromString(value) - if err := MVCCPut(ctx, db1, nil, roachpb.Key(key), hlc.Timestamp{WallTime: ts}, hlc.ClockTimestamp{}, v, txn); err != nil { + if err := MVCCPut(ctx, db1, roachpb.Key(key), hlc.Timestamp{WallTime: ts}, v, MVCCWriteOptions{Txn: txn}); err != nil { t.Fatal(err) } } @@ -1600,7 +1600,7 @@ func BenchmarkMVCCIncrementalIteratorForOldData(b *testing.B) { value := roachpb.MakeValueFromBytes(randutil.RandBytes(rng, valueSize)) value.InitChecksum(key) ts := hlc.Timestamp{WallTime: baseTimestamp + 100*int64(i%keyAgeInterval)} - if err := MVCCPut(context.Background(), batch, nil, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { + if err := MVCCPut(context.Background(), batch, key, ts, value, MVCCWriteOptions{}); err != nil { b.Fatal(err) } } diff --git a/pkg/storage/mvcc_logical_ops_test.go b/pkg/storage/mvcc_logical_ops_test.go index d111f2658ba1..9c1de35f3805 100644 --- a/pkg/storage/mvcc_logical_ops_test.go +++ b/pkg/storage/mvcc_logical_ops_test.go @@ -37,36 +37,36 @@ func TestMVCCOpLogWriter(t *testing.T) { defer ol.Close() // Write a value and an intent. - if err := MVCCPut(ctx, ol, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, ol, testKey1, hlc.Timestamp{Logical: 1}, value1, MVCCWriteOptions{}); err != nil { t.Fatal(err) } txn1ts := makeTxn(*txn1, hlc.Timestamp{Logical: 2}) - if err := MVCCPut(ctx, ol, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { + if err := MVCCPut(ctx, ol, testKey1, txn1ts.ReadTimestamp, value2, MVCCWriteOptions{Txn: txn1ts}); err != nil { t.Fatal(err) } // Write a value and an intent on local keys. localKey := keys.MakeRangeIDPrefix(1) - if err := MVCCPut(ctx, ol, nil, localKey, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, ol, localKey, hlc.Timestamp{Logical: 1}, value1, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, ol, nil, localKey, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { + if err := MVCCPut(ctx, ol, localKey, txn1ts.ReadTimestamp, value2, MVCCWriteOptions{Txn: txn1ts}); err != nil { t.Fatal(err) } // Update the intents and write another. txn1ts.Sequence++ txn1ts.WriteTimestamp = hlc.Timestamp{Logical: 3} - if err := MVCCPut(ctx, ol, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { + if err := MVCCPut(ctx, ol, testKey1, txn1ts.ReadTimestamp, value2, MVCCWriteOptions{Txn: txn1ts}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, ol, nil, localKey, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { + if err := MVCCPut(ctx, ol, localKey, txn1ts.ReadTimestamp, value2, MVCCWriteOptions{Txn: txn1ts}); err != nil { t.Fatal(err) } // Set the txn timestamp to a larger value than the intent. txn1LargerTS := makeTxn(*txn1, hlc.Timestamp{Logical: 4}) txn1LargerTS.WriteTimestamp = hlc.Timestamp{Logical: 4} - if err := MVCCPut(ctx, ol, nil, testKey2, txn1LargerTS.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn1LargerTS); err != nil { + if err := MVCCPut(ctx, ol, testKey2, txn1LargerTS.ReadTimestamp, value3, MVCCWriteOptions{Txn: txn1LargerTS}); err != nil { t.Fatal(err) } @@ -91,7 +91,7 @@ func TestMVCCOpLogWriter(t *testing.T) { // Write another intent, push it, then abort it. txn2 := makeTxn(*txn2, hlc.Timestamp{Logical: 5}) txn2.IsoLevel = isolation.ReadCommitted - if err := MVCCPut(ctx, ol, nil, testKey3, txn2.ReadTimestamp, hlc.ClockTimestamp{}, value4, txn2); err != nil { + if err := MVCCPut(ctx, ol, testKey3, txn2.ReadTimestamp, value4, MVCCWriteOptions{Txn: txn2}); err != nil { t.Fatal(err) } txn2Pushed := *txn2 @@ -112,7 +112,7 @@ func TestMVCCOpLogWriter(t *testing.T) { } // Write an inline value. This should be ignored by the log. - if err := MVCCPut(ctx, ol, nil, testKey6, hlc.Timestamp{}, hlc.ClockTimestamp{}, value6, nil); err != nil { + if err := MVCCPut(ctx, ol, testKey6, hlc.Timestamp{}, value6, MVCCWriteOptions{}); err != nil { t.Fatal(err) } diff --git a/pkg/storage/mvcc_stats_test.go b/pkg/storage/mvcc_stats_test.go index bbe78ef13e40..498f7c215571 100644 --- a/pkg/storage/mvcc_stats_test.go +++ b/pkg/storage/mvcc_stats_test.go @@ -98,7 +98,7 @@ func TestMVCCStatsDeleteCommitMovesTimestamp(t *testing.T) { ts1 := hlc.Timestamp{WallTime: 1e9} // Put a value. value := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, value, nil); err != nil { + if err := MVCCPut(ctx, engine, key, ts1, value, MVCCWriteOptions{Stats: aggMS}); err != nil { t.Fatal(err) } @@ -126,7 +126,7 @@ func TestMVCCStatsDeleteCommitMovesTimestamp(t *testing.T) { TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts3}, ReadTimestamp: ts3, } - if _, err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { + if _, err := MVCCDelete(ctx, engine, key, txn.ReadTimestamp, MVCCWriteOptions{Txn: txn, Stats: aggMS}); err != nil { t.Fatal(err) } @@ -188,7 +188,7 @@ func TestMVCCStatsPutCommitMovesTimestamp(t *testing.T) { } // Write an intent at t=1s. value := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, value, txn); err != nil { + if err := MVCCPut(ctx, engine, key, ts1, value, MVCCWriteOptions{Txn: txn, Stats: aggMS}); err != nil { t.Fatal(err) } @@ -277,7 +277,7 @@ func TestMVCCStatsPutPushMovesTimestamp(t *testing.T) { } // Write an intent. value := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, value, txn); err != nil { + if err := MVCCPut(ctx, engine, key, txn.ReadTimestamp, value, MVCCWriteOptions{Txn: txn, Stats: aggMS}); err != nil { t.Fatal(err) } @@ -374,7 +374,7 @@ func TestMVCCStatsDeleteMovesTimestamp(t *testing.T) { // Write an intent. value := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, value, txn); err != nil { + if err := MVCCPut(ctx, engine, key, txn.ReadTimestamp, value, MVCCWriteOptions{Txn: txn, Stats: aggMS}); err != nil { t.Fatal(err) } @@ -449,7 +449,7 @@ func TestMVCCStatsDeleteMovesTimestamp(t *testing.T) { } require.EqualValues(t, expM2ValSize, m2ValSize) - if _, err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { + if _, err := MVCCDelete(ctx, engine, key, txn.ReadTimestamp, MVCCWriteOptions{Txn: txn, Stats: aggMS}); err != nil { t.Fatal(err) } @@ -504,7 +504,7 @@ func TestMVCCStatsPutMovesDeletionTimestamp(t *testing.T) { } // Write a deletion tombstone intent. - if _, err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { + if _, err := MVCCDelete(ctx, engine, key, txn.ReadTimestamp, MVCCWriteOptions{Txn: txn, Stats: aggMS}); err != nil { t.Fatal(err) } @@ -577,7 +577,7 @@ func TestMVCCStatsPutMovesDeletionTimestamp(t *testing.T) { require.EqualValues(t, vVal2Size, 17) } - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, value, txn); err != nil { + if err := MVCCPut(ctx, engine, key, txn.ReadTimestamp, value, MVCCWriteOptions{Txn: txn, Stats: aggMS}); err != nil { t.Fatal(err) } @@ -625,7 +625,7 @@ func TestMVCCStatsDelDelCommitMovesTimestamp(t *testing.T) { ts3 := hlc.Timestamp{WallTime: 3e9} // Write a non-transactional tombstone at t=1s. - if _, err := MVCCDelete(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := MVCCDelete(ctx, engine, key, ts1, MVCCWriteOptions{Stats: aggMS}); err != nil { t.Fatal(err) } @@ -654,7 +654,7 @@ func TestMVCCStatsDelDelCommitMovesTimestamp(t *testing.T) { TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts2}, ReadTimestamp: ts2, } - if _, err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { + if _, err := MVCCDelete(ctx, engine, key, txn.ReadTimestamp, MVCCWriteOptions{Txn: txn, Stats: aggMS}); err != nil { t.Fatal(err) } @@ -779,7 +779,7 @@ func TestMVCCStatsPutDelPutMovesTimestamp(t *testing.T) { // Write a non-transactional value at t=1s. value := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, value, nil); err != nil { + if err := MVCCPut(ctx, engine, key, ts1, value, MVCCWriteOptions{Stats: aggMS}); err != nil { t.Fatal(err) } @@ -812,7 +812,7 @@ func TestMVCCStatsPutDelPutMovesTimestamp(t *testing.T) { TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts2}, ReadTimestamp: ts2, } - if _, err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { + if _, err := MVCCDelete(ctx, engine, key, txn.ReadTimestamp, MVCCWriteOptions{Txn: txn, Stats: aggMS}); err != nil { t.Fatal(err) } @@ -903,7 +903,7 @@ func TestMVCCStatsPutDelPutMovesTimestamp(t *testing.T) { } txn.WriteTimestamp.Forward(ts3) - if err := MVCCPut(ctx, engine, &aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val2, txn); err != nil { + if err := MVCCPut(ctx, engine, key, txn.ReadTimestamp, val2, MVCCWriteOptions{Txn: txn, Stats: &aggMS}); err != nil { t.Fatal(err) } @@ -963,10 +963,10 @@ func TestMVCCStatsDelDelGC(t *testing.T) { ts2 := hlc.Timestamp{WallTime: 2e9} // Write tombstones at ts1 and ts2. - if _, err := MVCCDelete(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := MVCCDelete(ctx, engine, key, ts1, MVCCWriteOptions{Stats: aggMS}); err != nil { t.Fatal(err) } - if _, err := MVCCDelete(ctx, engine, aggMS, key, ts2, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := MVCCDelete(ctx, engine, key, ts2, MVCCWriteOptions{Stats: aggMS}); err != nil { t.Fatal(err) } @@ -1046,7 +1046,7 @@ func TestMVCCStatsPutIntentTimestampNotPutTimestamp(t *testing.T) { } // Write an intent at 2s+1. value := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, value, txn); err != nil { + if err := MVCCPut(ctx, engine, key, txn.ReadTimestamp, value, MVCCWriteOptions{Txn: txn, Stats: aggMS}); err != nil { t.Fatal(err) } @@ -1096,7 +1096,7 @@ func TestMVCCStatsPutIntentTimestampNotPutTimestamp(t *testing.T) { {Sequence: 0, Value: encValue}, }, }).Size()) - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, value, txn); err != nil { + if err := MVCCPut(ctx, engine, key, txn.ReadTimestamp, value, MVCCWriteOptions{Txn: txn, Stats: aggMS}); err != nil { t.Fatal(err) } @@ -1141,7 +1141,7 @@ func TestMVCCStatsPutWaitDeleteGC(t *testing.T) { // Write a value at ts1. val1 := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, val1, nil); err != nil { + if err := MVCCPut(ctx, engine, key, ts1, val1, MVCCWriteOptions{Stats: aggMS}); err != nil { t.Fatal(err) } @@ -1171,7 +1171,7 @@ func TestMVCCStatsPutWaitDeleteGC(t *testing.T) { // Delete the value at ts5. - if _, err := MVCCDelete(ctx, engine, aggMS, key, ts2, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := MVCCDelete(ctx, engine, key, ts2, MVCCWriteOptions{Stats: aggMS}); err != nil { t.Fatal(err) } @@ -1240,7 +1240,7 @@ func TestMVCCStatsTxnSysPutPut(t *testing.T) { // Write an intent at ts1. val1 := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val1, txn); err != nil { + if err := MVCCPut(ctx, engine, key, txn.ReadTimestamp, val1, MVCCWriteOptions{Txn: txn, Stats: aggMS}); err != nil { t.Fatal(err) } @@ -1303,7 +1303,7 @@ func TestMVCCStatsTxnSysPutPut(t *testing.T) { } require.EqualValues(t, expMVal2Size, mVal2Size) - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val2, txn); err != nil { + if err := MVCCPut(ctx, engine, key, txn.ReadTimestamp, val2, MVCCWriteOptions{Txn: txn, Stats: aggMS}); err != nil { t.Fatal(err) } @@ -1340,7 +1340,7 @@ func TestMVCCStatsTxnSysPutAbort(t *testing.T) { // Write a system intent at ts1. val1 := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val1, txn); err != nil { + if err := MVCCPut(ctx, engine, key, txn.ReadTimestamp, val1, MVCCWriteOptions{Txn: txn, Stats: aggMS}); err != nil { t.Fatal(err) } @@ -1415,7 +1415,7 @@ func TestMVCCStatsSysPutPut(t *testing.T) { // Write a value at ts1. val1 := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, val1, nil); err != nil { + if err := MVCCPut(ctx, engine, key, ts1, val1, MVCCWriteOptions{Stats: aggMS}); err != nil { t.Fatal(err) } @@ -1449,7 +1449,7 @@ func TestMVCCStatsSysPutPut(t *testing.T) { // Put another value at ts2. - if err := MVCCPut(ctx, engine, aggMS, key, ts2, hlc.ClockTimestamp{}, val2, nil); err != nil { + if err := MVCCPut(ctx, engine, key, ts2, val2, MVCCWriteOptions{Stats: aggMS}); err != nil { t.Fatal(err) } @@ -1642,26 +1642,42 @@ func TestMVCCStatsRandomized(t *testing.T) { } actions["Put"] = func(s *state) (bool, string) { - if err := MVCCPut(ctx, s.batch, s.MSDelta, s.key, s.TS, hlc.ClockTimestamp{}, s.rngVal(), s.Txn); err != nil { + opts := MVCCWriteOptions{ + Txn: s.Txn, + Stats: s.MSDelta, + } + if err := MVCCPut(ctx, s.batch, s.key, s.TS, s.rngVal(), opts); err != nil { return false, err.Error() } return true, "" } actions["InitPut"] = func(s *state) (bool, string) { + opts := MVCCWriteOptions{ + Txn: s.Txn, + Stats: s.MSDelta, + } failOnTombstones := s.rng.Intn(2) == 0 desc := fmt.Sprintf("failOnTombstones=%t", failOnTombstones) - if err := MVCCInitPut(ctx, s.batch, s.MSDelta, s.key, s.TS, hlc.ClockTimestamp{}, s.rngVal(), failOnTombstones, s.Txn); err != nil { + if err := MVCCInitPut(ctx, s.batch, s.key, s.TS, s.rngVal(), failOnTombstones, opts); err != nil { return false, desc + ": " + err.Error() } return true, desc } actions["Del"] = func(s *state) (bool, string) { - if _, err := MVCCDelete(ctx, s.batch, s.MSDelta, s.key, s.TS, hlc.ClockTimestamp{}, s.Txn); err != nil { + opts := MVCCWriteOptions{ + Txn: s.Txn, + Stats: s.MSDelta, + } + if _, err := MVCCDelete(ctx, s.batch, s.key, s.TS, opts); err != nil { return false, err.Error() } return true, "" } actions["DelRange"] = func(s *state) (bool, string) { + opts := MVCCWriteOptions{ + Txn: s.Txn, + Stats: s.MSDelta, + } keySpan := currentKeySpan(s) mvccRangeDel := !s.isLocalKey && s.Txn == nil && s.rng.Intn(2) == 0 @@ -1699,7 +1715,7 @@ func TestMVCCStatsRandomized(t *testing.T) { if !mvccRangeDel { desc = fmt.Sprintf("mvccDeleteRange=%s, returnKeys=%t, max=%d", keySpan, returnKeys, max) _, _, _, err = MVCCDeleteRange( - ctx, s.batch, s.MSDelta, keySpan.Key, keySpan.EndKey, max, s.TS, hlc.ClockTimestamp{}, s.Txn, returnKeys, + ctx, s.batch, keySpan.Key, keySpan.EndKey, max, s.TS, opts, returnKeys, ) } else if predicates == (kvpb.DeleteRangePredicates{}) { desc = fmt.Sprintf("mvccDeleteRangeUsingTombstone=%s", diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go index 7c09c4d5493e..5bc6a09bbbef 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -271,10 +271,10 @@ func TestMVCCGetNoMoreOldVersion(t *testing.T) { // b // // If we search for a, the scan should not return "b". - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey1, hlc.Timestamp{WallTime: 3}, value1, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey2, hlc.Timestamp{WallTime: 1}, value2, MVCCWriteOptions{}); err != nil { t.Fatal(err) } valueRes, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{}) @@ -294,7 +294,7 @@ func TestMVCCGetWithValueHeader(t *testing.T) { engine := NewDefaultInMemForTesting() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1, Logical: 1}, hlc.ClockTimestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1, Logical: 1}, value1, MVCCWriteOptions{LocalTimestamp: hlc.ClockTimestamp{WallTime: 1}}); err != nil { t.Fatal(err) } valueRes, vh, err := MVCCGetWithValueHeader(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{}) @@ -306,7 +306,7 @@ func TestMVCCGetWithValueHeader(t *testing.T) { } require.Equal(t, hlc.ClockTimestamp{WallTime: 1}, vh.LocalTimestamp) - _, err = MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{WallTime: 2, Logical: 1}, nil) + _, err = MVCCDelete(ctx, engine, testKey1, hlc.Timestamp{WallTime: 3}, MVCCWriteOptions{LocalTimestamp: hlc.ClockTimestamp{WallTime: 2, Logical: 1}}) if err != nil { t.Fatal(err) } @@ -355,10 +355,10 @@ func TestMVCCWriteWithOlderTimestampAfterDeletionOfNonexistentKey(t *testing.T) engine := NewDefaultInMemForTesting() defer engine.Close() - _, err := MVCCDelete(context.Background(), engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, nil) + _, err := MVCCDelete(context.Background(), engine, testKey1, hlc.Timestamp{WallTime: 3}, MVCCWriteOptions{}) require.NoError(t, err) - err = MVCCPut(context.Background(), engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil) + err = MVCCPut(context.Background(), engine, testKey1, hlc.Timestamp{WallTime: 1}, value1, MVCCWriteOptions{}) require.ErrorAs(t, err, new(*kvpb.WriteTooOldError)) require.Regexp(t, err, "WriteTooOldError: write for key \"/db1\" at timestamp 0.000000001,0 too old; must write at or above 0.000000003,1") @@ -377,7 +377,7 @@ func TestMVCCInlineWithTxn(t *testing.T) { defer engine.Close() // Put an inline value. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{}, hlc.ClockTimestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey1, hlc.Timestamp{}, value1, MVCCWriteOptions{}); err != nil { t.Fatal(err) } @@ -399,7 +399,7 @@ func TestMVCCInlineWithTxn(t *testing.T) { } // Verify inline put with txn is an error. - err = MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{}, hlc.ClockTimestamp{}, value2, txn2) + err = MVCCPut(ctx, engine, testKey2, hlc.Timestamp{}, value2, MVCCWriteOptions{Txn: txn2}) if !testutils.IsError(err, "writes not allowed within transactions") { t.Errorf("unexpected error: %+v", err) } @@ -415,7 +415,7 @@ func TestMVCCDeleteMissingKey(t *testing.T) { engine := NewDefaultInMemForTesting() defer engine.Close() - if _, err := MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := MVCCDelete(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCWriteOptions{}); err != nil { t.Fatal(err) } require.Empty(t, mvccGetRaw(t, engine, mvccKey(testKey1))) @@ -432,7 +432,7 @@ func TestMVCCGetAndDeleteInTxn(t *testing.T) { txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) txn.Sequence++ - if err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn); err != nil { + if err := MVCCPut(ctx, engine, testKey1, txn.ReadTimestamp, value1, MVCCWriteOptions{Txn: txn}); err != nil { t.Fatal(err) } @@ -446,7 +446,7 @@ func TestMVCCGetAndDeleteInTxn(t *testing.T) { txn.Sequence++ txn.WriteTimestamp = hlc.Timestamp{WallTime: 3} - if _, err := MVCCDelete(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { + if _, err := MVCCDelete(ctx, engine, testKey1, txn.ReadTimestamp, MVCCWriteOptions{Txn: txn}); err != nil { t.Fatal(err) } @@ -485,7 +485,7 @@ func TestMVCCGetWriteIntentError(t *testing.T) { engine := NewDefaultInMemForTesting() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { + if err := MVCCPut(ctx, engine, testKey1, txn1.ReadTimestamp, value1, MVCCWriteOptions{Txn: txn1}); err != nil { t.Fatal(err) } @@ -538,7 +538,7 @@ func TestMVCCScanWriteIntentError(t *testing.T) { for i, kv := range fixtureKVs { v := *protoutil.Clone(&kv.Value).(*roachpb.Value) v.Timestamp = hlc.Timestamp{} - if err := MVCCPut(ctx, engine, nil, kv.Key, kv.Value.Timestamp, hlc.ClockTimestamp{}, v, txnMap[i]); err != nil { + if err := MVCCPut(ctx, engine, kv.Key, kv.Value.Timestamp, v, MVCCWriteOptions{Txn: txnMap[i]}); err != nil { t.Fatal(err) } } @@ -641,11 +641,11 @@ func TestMVCCGetInconsistent(t *testing.T) { defer engine.Close() // Put two values to key 1, the latest with a txn. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, value1, MVCCWriteOptions{}); err != nil { t.Fatal(err) } txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 2}) - if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { + if err := MVCCPut(ctx, engine, testKey1, txn1ts.ReadTimestamp, value2, MVCCWriteOptions{Txn: txn1ts}); err != nil { t.Fatal(err) } @@ -675,7 +675,7 @@ func TestMVCCGetInconsistent(t *testing.T) { } // Write a single intent for key 2 and verify get returns empty. - if err := MVCCPut(ctx, engine, nil, testKey2, txn2.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn2); err != nil { + if err := MVCCPut(ctx, engine, testKey2, txn2.ReadTimestamp, value1, MVCCWriteOptions{Txn: txn2}); err != nil { t.Fatal(err) } res, err := MVCCGet(ctx, engine, testKey2, hlc.Timestamp{WallTime: 2}, @@ -711,11 +711,11 @@ func TestMVCCGetProtoInconsistent(t *testing.T) { v2 := roachpb.MakeValueFromBytes(bytes2) // Put two values to key 1, the latest with a txn. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, v1, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, v1, MVCCWriteOptions{}); err != nil { t.Fatal(err) } txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 2}) - if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, v2, txn1ts); err != nil { + if err := MVCCPut(ctx, engine, testKey1, txn1ts.ReadTimestamp, v2, MVCCWriteOptions{Txn: txn1ts}); err != nil { t.Fatal(err) } @@ -757,7 +757,7 @@ func TestMVCCGetProtoInconsistent(t *testing.T) { { // Write a single intent for key 2 and verify get returns empty. - if err := MVCCPut(ctx, engine, nil, testKey2, txn2.ReadTimestamp, hlc.ClockTimestamp{}, v1, txn2); err != nil { + if err := MVCCPut(ctx, engine, testKey2, txn2.ReadTimestamp, v1, MVCCWriteOptions{Txn: txn2}); err != nil { t.Fatal(err) } val := roachpb.Value{} @@ -776,10 +776,10 @@ func TestMVCCGetProtoInconsistent(t *testing.T) { // Write a malformed value (not an encoded MVCCKeyValue) and a // write intent to key 3; the parse error is returned instead of the // write intent. - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey3, hlc.Timestamp{WallTime: 1}, value3, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, v2, txn1ts); err != nil { + if err := MVCCPut(ctx, engine, testKey3, txn1ts.ReadTimestamp, v2, MVCCWriteOptions{Txn: txn1ts}); err != nil { t.Fatal(err) } val := roachpb.Value{} @@ -813,7 +813,7 @@ func TestMVCCInvalidateIterator(t *testing.T) { ts2 := hlc.Timestamp{WallTime: 2} key := roachpb.Key("a") - if err := MVCCPut(ctx, engine, nil, key, ts1, hlc.ClockTimestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, key, ts1, value1, MVCCWriteOptions{}); err != nil { t.Fatal(err) } @@ -868,28 +868,28 @@ func TestMVCCInvalidateIterator(t *testing.T) { } func mvccScanTest(ctx context.Context, t *testing.T, engine Engine) { - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, value1, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, value4, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey2, hlc.Timestamp{WallTime: 1}, value2, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey2, hlc.Timestamp{WallTime: 3}, value3, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey3, hlc.Timestamp{WallTime: 1}, value3, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 4}, hlc.ClockTimestamp{}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey3, hlc.Timestamp{WallTime: 4}, value2, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey4, hlc.Timestamp{WallTime: 1}, value4, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 5}, hlc.ClockTimestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey4, hlc.Timestamp{WallTime: 5}, value1, MVCCWriteOptions{}); err != nil { t.Fatal(err) } @@ -982,19 +982,19 @@ func TestMVCCScanMaxNum(t *testing.T) { engine := NewDefaultInMemForTesting() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, value1, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey2, hlc.Timestamp{WallTime: 1}, value2, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey3, hlc.Timestamp{WallTime: 1}, value3, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey4, hlc.Timestamp{WallTime: 1}, value4, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey6, hlc.Timestamp{WallTime: 1}, value4, MVCCWriteOptions{}); err != nil { t.Fatal(err) } @@ -1070,19 +1070,19 @@ func TestMVCCScanWithKeyPrefix(t *testing.T) { // b // In this case, if we scan from "a"-"b", we wish to skip // a and a and find "aa'. - if err := MVCCPut(ctx, engine, nil, roachpb.Key("/a"), hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, roachpb.Key("/a"), hlc.Timestamp{WallTime: 1}, value1, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, roachpb.Key("/a"), hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, roachpb.Key("/a"), hlc.Timestamp{WallTime: 2}, value2, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, roachpb.Key("/aa"), hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, roachpb.Key("/aa"), hlc.Timestamp{WallTime: 2}, value2, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, roachpb.Key("/aa"), hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, roachpb.Key("/aa"), hlc.Timestamp{WallTime: 3}, value3, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, roachpb.Key("/b"), hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, roachpb.Key("/b"), hlc.Timestamp{WallTime: 1}, value3, MVCCWriteOptions{}); err != nil { t.Fatal(err) } @@ -1108,17 +1108,17 @@ func TestMVCCScanInTxn(t *testing.T) { engine := NewDefaultInMemForTesting() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, value1, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey2, hlc.Timestamp{WallTime: 1}, value2, MVCCWriteOptions{}); err != nil { t.Fatal(err) } txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) - if err := MVCCPut(ctx, engine, nil, testKey3, txn.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn); err != nil { + if err := MVCCPut(ctx, engine, testKey3, txn.ReadTimestamp, value3, MVCCWriteOptions{Txn: txn}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey4, hlc.Timestamp{WallTime: 1}, value4, MVCCWriteOptions{}); err != nil { t.Fatal(err) } @@ -1166,24 +1166,24 @@ func TestMVCCScanInconsistent(t *testing.T) { ts4 := hlc.Timestamp{WallTime: 4} ts5 := hlc.Timestamp{WallTime: 5} ts6 := hlc.Timestamp{WallTime: 6} - if err := MVCCPut(ctx, engine, nil, testKey1, ts1, hlc.ClockTimestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey1, ts1, value1, MVCCWriteOptions{}); err != nil { t.Fatal(err) } txn1ts2 := makeTxn(*txn1, ts2) - if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts2.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts2); err != nil { + if err := MVCCPut(ctx, engine, testKey1, txn1ts2.ReadTimestamp, value2, MVCCWriteOptions{Txn: txn1ts2}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, ts3, hlc.ClockTimestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey2, ts3, value1, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, ts4, hlc.ClockTimestamp{}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey2, ts4, value2, MVCCWriteOptions{}); err != nil { t.Fatal(err) } txn2ts5 := makeTxn(*txn2, ts5) - if err := MVCCPut(ctx, engine, nil, testKey3, txn2ts5.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn2ts5); err != nil { + if err := MVCCPut(ctx, engine, testKey3, txn2ts5.ReadTimestamp, value3, MVCCWriteOptions{Txn: txn2ts5}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, ts6, hlc.ClockTimestamp{}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey4, ts6, value4, MVCCWriteOptions{}); err != nil { t.Fatal(err) } @@ -1240,28 +1240,28 @@ func TestMVCCDeleteRange(t *testing.T) { engine := NewDefaultInMemForTesting() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, value1, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey2, hlc.Timestamp{WallTime: 1}, value2, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey3, hlc.Timestamp{WallTime: 1}, value3, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey4, hlc.Timestamp{WallTime: 1}, value4, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey5, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value5, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey5, hlc.Timestamp{WallTime: 1}, value5, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value6, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey6, hlc.Timestamp{WallTime: 1}, value6, MVCCWriteOptions{}); err != nil { t.Fatal(err) } // Attempt to delete two keys. - deleted, resumeSpan, num, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, - 2, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, false) + deleted, resumeSpan, num, err := MVCCDeleteRange(ctx, engine, testKey2, testKey6, + 2, hlc.Timestamp{WallTime: 2}, MVCCWriteOptions{}, false) if err != nil { t.Fatal(err) } @@ -1316,8 +1316,8 @@ func TestMVCCDeleteRange(t *testing.T) { } // Attempt to delete no keys. - deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, - -1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, false) + deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, testKey2, testKey6, + -1, hlc.Timestamp{WallTime: 2}, MVCCWriteOptions{}, false) if err != nil { t.Fatal(err) } @@ -1344,8 +1344,8 @@ func TestMVCCDeleteRange(t *testing.T) { t.Fatal("the value should not be empty") } - deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, testKey4, keyMax, - 0, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, false) + deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, testKey4, keyMax, + 0, hlc.Timestamp{WallTime: 2}, MVCCWriteOptions{}, false) if err != nil { t.Fatal(err) } @@ -1369,8 +1369,8 @@ func TestMVCCDeleteRange(t *testing.T) { t.Fatalf("the value should not be empty: %+v", res.KVs) } - deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, localMax, testKey2, - 0, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, false) + deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, localMax, testKey2, + 0, hlc.Timestamp{WallTime: 2}, MVCCWriteOptions{}, false) if err != nil { t.Fatal(err) } @@ -1401,28 +1401,28 @@ func TestMVCCDeleteRangeReturnKeys(t *testing.T) { engine := NewDefaultInMemForTesting() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, value1, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey2, hlc.Timestamp{WallTime: 1}, value2, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey3, hlc.Timestamp{WallTime: 1}, value3, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey4, hlc.Timestamp{WallTime: 1}, value4, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey5, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value5, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey5, hlc.Timestamp{WallTime: 1}, value5, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value6, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey6, hlc.Timestamp{WallTime: 1}, value6, MVCCWriteOptions{}); err != nil { t.Fatal(err) } // Attempt to delete two keys. - deleted, resumeSpan, num, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, - 2, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, true) + deleted, resumeSpan, num, err := MVCCDeleteRange(ctx, engine, testKey2, testKey6, + 2, hlc.Timestamp{WallTime: 2}, MVCCWriteOptions{}, true) if err != nil { t.Fatal(err) } @@ -1456,8 +1456,8 @@ func TestMVCCDeleteRangeReturnKeys(t *testing.T) { } // Attempt to delete no keys. - deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, - -1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, true) + deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, testKey2, testKey6, + -1, hlc.Timestamp{WallTime: 2}, MVCCWriteOptions{}, true) if err != nil { t.Fatal(err) } @@ -1484,8 +1484,8 @@ func TestMVCCDeleteRangeReturnKeys(t *testing.T) { t.Fatal("the value should not be empty") } - deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, testKey4, keyMax, - math.MaxInt64, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, true) + deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, testKey4, keyMax, + math.MaxInt64, hlc.Timestamp{WallTime: 2}, MVCCWriteOptions{}, true) if err != nil { t.Fatal(err) } @@ -1515,8 +1515,8 @@ func TestMVCCDeleteRangeReturnKeys(t *testing.T) { t.Fatal("the value should not be empty") } - deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, localMax, testKey2, - math.MaxInt64, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, true) + deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, localMax, testKey2, + math.MaxInt64, hlc.Timestamp{WallTime: 2}, MVCCWriteOptions{}, true) if err != nil { t.Fatal(err) } @@ -1548,29 +1548,29 @@ func TestMVCCDeleteRangeFailed(t *testing.T) { defer engine.Close() txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, value1, MVCCWriteOptions{}); err != nil { t.Fatal(err) } txn.Sequence++ - if err := MVCCPut(ctx, engine, nil, testKey2, txn.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn); err != nil { + if err := MVCCPut(ctx, engine, testKey2, txn.ReadTimestamp, value2, MVCCWriteOptions{Txn: txn}); err != nil { t.Fatal(err) } txn.Sequence++ - if err := MVCCPut(ctx, engine, nil, testKey3, txn.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn); err != nil { + if err := MVCCPut(ctx, engine, testKey3, txn.ReadTimestamp, value3, MVCCWriteOptions{Txn: txn}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey4, hlc.Timestamp{WallTime: 1}, value4, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey4, - math.MaxInt64, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, nil, false); err == nil { + if _, _, _, err := MVCCDeleteRange(ctx, engine, testKey2, testKey4, + math.MaxInt64, hlc.Timestamp{WallTime: 1}, MVCCWriteOptions{}, false); err == nil { t.Fatal("expected error on uncommitted write intent") } txn.Sequence++ - if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey4, - math.MaxInt64, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn, false); err != nil { + if _, _, _, err := MVCCDeleteRange(ctx, engine, testKey2, testKey4, + math.MaxInt64, txn.ReadTimestamp, MVCCWriteOptions{Txn: txn}, false); err != nil { t.Fatal(err) } } @@ -1586,21 +1586,21 @@ func TestMVCCDeleteRangeConcurrentTxn(t *testing.T) { txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) txn2ts := makeTxn(*txn2, hlc.Timestamp{WallTime: 2}) - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, value1, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { + if err := MVCCPut(ctx, engine, testKey2, txn1ts.ReadTimestamp, value2, MVCCWriteOptions{Txn: txn1ts}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, txn2ts.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn2ts); err != nil { + if err := MVCCPut(ctx, engine, testKey3, txn2ts.ReadTimestamp, value3, MVCCWriteOptions{Txn: txn2ts}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey4, hlc.Timestamp{WallTime: 1}, value4, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey4, - math.MaxInt64, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, txn1ts, false, + if _, _, _, err := MVCCDeleteRange(ctx, engine, testKey2, testKey4, + math.MaxInt64, txn1ts.ReadTimestamp, MVCCWriteOptions{Txn: txn1ts}, false, ); err == nil { t.Fatal("expected error on uncommitted write intent") } @@ -1616,23 +1616,23 @@ func TestMVCCUncommittedDeleteRangeVisible(t *testing.T) { engine := NewDefaultInMemForTesting() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, value1, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey2, hlc.Timestamp{WallTime: 1}, value2, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey3, hlc.Timestamp{WallTime: 1}, value3, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if _, err := MVCCDelete(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 2, Logical: 1}, hlc.ClockTimestamp{}, nil); err != nil { + if _, err := MVCCDelete(ctx, engine, testKey2, hlc.Timestamp{WallTime: 2, Logical: 1}, MVCCWriteOptions{}); err != nil { t.Fatal(err) } txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 2, Logical: 2}) - if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey1, testKey4, - math.MaxInt64, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn, false, + if _, _, _, err := MVCCDeleteRange(ctx, engine, testKey1, testKey4, + math.MaxInt64, txn.ReadTimestamp, MVCCWriteOptions{Txn: txn}, false, ); err != nil { t.Fatal(err) } @@ -1654,15 +1654,15 @@ func TestMVCCDeleteRangeOldTimestamp(t *testing.T) { ctx := context.Background() engine := NewDefaultInMemForTesting() defer engine.Close() - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil) + err := MVCCPut(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, value1, MVCCWriteOptions{}) if err != nil { t.Fatal(err) } - err = MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value2, nil) + err = MVCCPut(ctx, engine, testKey2, hlc.Timestamp{WallTime: 3}, value2, MVCCWriteOptions{}) if err != nil { t.Fatal(err) } - _, err = MVCCDelete(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 5}, hlc.ClockTimestamp{}, nil) + _, err = MVCCDelete(ctx, engine, testKey2, hlc.Timestamp{WallTime: 5}, MVCCWriteOptions{}) if err != nil { t.Fatal(err) } @@ -1670,8 +1670,8 @@ func TestMVCCDeleteRangeOldTimestamp(t *testing.T) { // Delete at a time before the tombstone. Should return a WriteTooOld error. b := engine.NewBatch() defer b.Close() - keys, resume, keyCount, err := MVCCDeleteRange(ctx, b, nil, testKey1, testKey4, - math.MaxInt64, hlc.Timestamp{WallTime: 4}, hlc.ClockTimestamp{}, nil, true) + keys, resume, keyCount, err := MVCCDeleteRange(ctx, b, testKey1, testKey4, + math.MaxInt64, hlc.Timestamp{WallTime: 4}, MVCCWriteOptions{}, true) require.Nil(t, keys) require.Nil(t, resume) require.Equal(t, int64(0), keyCount) @@ -1681,8 +1681,8 @@ func TestMVCCDeleteRangeOldTimestamp(t *testing.T) { // Delete at the same time as the tombstone. Should return a WriteTooOld error. b = engine.NewBatch() defer b.Close() - keys, resume, keyCount, err = MVCCDeleteRange(ctx, b, nil, testKey1, testKey4, - math.MaxInt64, hlc.Timestamp{WallTime: 5}, hlc.ClockTimestamp{}, nil, true) + keys, resume, keyCount, err = MVCCDeleteRange(ctx, b, testKey1, testKey4, + math.MaxInt64, hlc.Timestamp{WallTime: 5}, MVCCWriteOptions{}, true) require.Nil(t, keys) require.Nil(t, resume) require.Equal(t, int64(0), keyCount) @@ -1693,8 +1693,8 @@ func TestMVCCDeleteRangeOldTimestamp(t *testing.T) { // include the tombstone in the returned keys. b = engine.NewBatch() defer b.Close() - keys, resume, keyCount, err = MVCCDeleteRange(ctx, b, nil, testKey1, testKey4, - math.MaxInt64, hlc.Timestamp{WallTime: 6}, hlc.ClockTimestamp{}, nil, true) + keys, resume, keyCount, err = MVCCDeleteRange(ctx, b, testKey1, testKey4, + math.MaxInt64, hlc.Timestamp{WallTime: 6}, MVCCWriteOptions{}, true) require.Equal(t, []roachpb.Key{testKey1}, keys) require.Nil(t, resume) require.Equal(t, int64(1), keyCount) @@ -1720,19 +1720,19 @@ func TestMVCCDeleteRangeInline(t *testing.T) { {testKey4, value4}, {testKey5, value5}, } { - if err := MVCCPut(ctx, engine, nil, kv.key, hlc.Timestamp{Logical: 0}, hlc.ClockTimestamp{}, kv.value, nil); err != nil { + if err := MVCCPut(ctx, engine, kv.key, hlc.Timestamp{Logical: 0}, kv.value, MVCCWriteOptions{}); err != nil { t.Fatalf("%d: %+v", i, err) } } // Create one non-inline value (non-zero timestamp). - if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value6, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey6, hlc.Timestamp{WallTime: 1}, value6, MVCCWriteOptions{}); err != nil { t.Fatal(err) } // Attempt to delete two inline keys, should succeed. - deleted, resumeSpan, num, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, - 2, hlc.Timestamp{Logical: 0}, hlc.ClockTimestamp{}, nil, true) + deleted, resumeSpan, num, err := MVCCDeleteRange(ctx, engine, testKey2, testKey6, + 2, hlc.Timestamp{Logical: 0}, MVCCWriteOptions{}, true) if err != nil { t.Fatal(err) } @@ -1748,22 +1748,22 @@ func TestMVCCDeleteRangeInline(t *testing.T) { // Attempt to delete inline keys at a timestamp; should fail. const inlineMismatchErrString = "put is inline" - if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey1, testKey6, - 1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, true, + if _, _, _, err := MVCCDeleteRange(ctx, engine, testKey1, testKey6, + 1, hlc.Timestamp{WallTime: 2}, MVCCWriteOptions{}, true, ); !testutils.IsError(err, inlineMismatchErrString) { t.Fatalf("got error %v, expected error with text '%s'", err, inlineMismatchErrString) } // Attempt to delete non-inline key at zero timestamp; should fail. - if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey6, keyMax, - 1, hlc.Timestamp{Logical: 0}, hlc.ClockTimestamp{}, nil, true, + if _, _, _, err := MVCCDeleteRange(ctx, engine, testKey6, keyMax, + 1, hlc.Timestamp{Logical: 0}, MVCCWriteOptions{}, true, ); !testutils.IsError(err, inlineMismatchErrString) { t.Fatalf("got error %v, expected error with text '%s'", err, inlineMismatchErrString) } // Attempt to delete inline keys in a transaction; should fail. - if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, - 2, hlc.Timestamp{Logical: 0}, hlc.ClockTimestamp{}, txn1, true, + if _, _, _, err := MVCCDeleteRange(ctx, engine, testKey2, testKey6, + 2, hlc.Timestamp{Logical: 0}, MVCCWriteOptions{Txn: txn1}, true, ); !testutils.IsError(err, "writes not allowed within transactions") { t.Errorf("unexpected error: %+v", err) } @@ -1852,12 +1852,12 @@ func TestMVCCClearTimeRange(t *testing.T) { // keys eng := NewDefaultInMemForTesting() defer eng.Close() - require.NoError(t, MVCCPut(ctx, eng, nil, testKey2, ts1, hlc.ClockTimestamp{}, value1, nil)) - require.NoError(t, MVCCPut(ctx, eng, nil, testKey2, ts2, hlc.ClockTimestamp{}, value2, nil)) - require.NoError(t, MVCCPut(ctx, eng, nil, testKey5, ts2, hlc.ClockTimestamp{}, value2, nil)) - require.NoError(t, MVCCPut(ctx, eng, nil, testKey1, ts3, hlc.ClockTimestamp{}, value3, nil)) - require.NoError(t, MVCCPut(ctx, eng, nil, testKey5, ts4, hlc.ClockTimestamp{}, value4, nil)) - require.NoError(t, MVCCPut(ctx, eng, nil, testKey2, ts4, hlc.ClockTimestamp{}, value4, nil)) + require.NoError(t, MVCCPut(ctx, eng, testKey2, ts1, value1, MVCCWriteOptions{})) + require.NoError(t, MVCCPut(ctx, eng, testKey2, ts2, value2, MVCCWriteOptions{})) + require.NoError(t, MVCCPut(ctx, eng, testKey5, ts2, value2, MVCCWriteOptions{})) + require.NoError(t, MVCCPut(ctx, eng, testKey1, ts3, value3, MVCCWriteOptions{})) + require.NoError(t, MVCCPut(ctx, eng, testKey5, ts4, value4, MVCCWriteOptions{})) + require.NoError(t, MVCCPut(ctx, eng, testKey2, ts4, value4, MVCCWriteOptions{})) assertKVs := func(t *testing.T, reader Reader, at hlc.Timestamp, expected []roachpb.KeyValue) { t.Helper() @@ -2000,7 +2000,7 @@ func TestMVCCClearTimeRange(t *testing.T) { // Add an intent at k3@ts3. txn := roachpb.MakeTransaction("test", nil, isolation.Serializable, roachpb.NormalUserPriority, ts3, 1, 1) addIntent := func(t *testing.T, rw ReadWriter) { - require.NoError(t, MVCCPut(ctx, rw, nil, testKey3, ts3, hlc.ClockTimestamp{}, value3, &txn)) + require.NoError(t, MVCCPut(ctx, rw, testKey3, ts3, value3, MVCCWriteOptions{Txn: &txn})) } t.Run("clear everything hitting intent fails", func(t *testing.T) { b := eng.NewBatch() @@ -2095,25 +2095,25 @@ func TestMVCCClearTimeRangeOnRandomData(t *testing.T) { key := roachpb.Key(fmt.Sprintf("%05d", k)) if rand.Float64() > 0.8 { - _, err := MVCCDelete(ctx, e, &ms, key, hlc.Timestamp{WallTime: ts}, hlc.ClockTimestamp{}, nil) + _, err := MVCCDelete(ctx, e, key, hlc.Timestamp{WallTime: ts}, MVCCWriteOptions{Stats: &ms}) require.NoError(t, err) } else { v := roachpb.MakeValueFromString(fmt.Sprintf("v-%d", i)) - require.NoError(t, MVCCPut(ctx, e, &ms, key, hlc.Timestamp{WallTime: ts}, hlc.ClockTimestamp{}, v, nil)) + require.NoError(t, MVCCPut(ctx, e, key, hlc.Timestamp{WallTime: ts}, v, MVCCWriteOptions{Stats: &ms})) } } swathTime := rand.Intn(randTimeRange-100) + 100 for i := swathStart; i < swathEnd; i++ { key := roachpb.Key(fmt.Sprintf("%05d", i)) v := roachpb.MakeValueFromString(fmt.Sprintf("v-%d", i)) - require.NoError(t, MVCCPut(ctx, e, &ms, key, hlc.Timestamp{WallTime: int64(swathTime)}, hlc.ClockTimestamp{}, v, nil)) + require.NoError(t, MVCCPut(ctx, e, key, hlc.Timestamp{WallTime: int64(swathTime)}, v, MVCCWriteOptions{Stats: &ms})) } // Add another swath of keys above to exercise an after-iteration range flush. for i := keyRange; i < keyRange+200; i++ { key := roachpb.Key(fmt.Sprintf("%05d", i)) v := roachpb.MakeValueFromString(fmt.Sprintf("v-%d", i)) - require.NoError(t, MVCCPut(ctx, e, &ms, key, hlc.Timestamp{WallTime: int64(randTimeRange + 1)}, hlc.ClockTimestamp{}, v, nil)) + require.NoError(t, MVCCPut(ctx, e, key, hlc.Timestamp{WallTime: int64(randTimeRange + 1)}, v, MVCCWriteOptions{Stats: &ms})) } ms.AgeTo(2000) @@ -2178,25 +2178,25 @@ func TestMVCCInitPut(t *testing.T) { engine := NewDefaultInMemForTesting() defer engine.Close() - err := MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, false, nil) + err := MVCCInitPut(ctx, engine, testKey1, hlc.Timestamp{Logical: 1}, value1, false, MVCCWriteOptions{}) if err != nil { t.Fatal(err) } // A repeat of the command will still succeed - err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 2}, hlc.ClockTimestamp{}, value1, false, nil) + err = MVCCInitPut(ctx, engine, testKey1, hlc.Timestamp{Logical: 2}, value1, false, MVCCWriteOptions{}) if err != nil { t.Fatal(err) } // Delete. - _, err = MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 3}, hlc.ClockTimestamp{}, nil) + _, err = MVCCDelete(ctx, engine, testKey1, hlc.Timestamp{Logical: 3}, MVCCWriteOptions{}) if err != nil { t.Fatal(err) } // Reinserting the value fails if we fail on tombstones. - err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 4}, hlc.ClockTimestamp{}, value1, true, nil) + err = MVCCInitPut(ctx, engine, testKey1, hlc.Timestamp{Logical: 4}, value1, true, MVCCWriteOptions{}) if e := (*kvpb.ConditionFailedError)(nil); errors.As(err, &e) { if !bytes.Equal(e.ActualValue.RawBytes, nil) { t.Fatalf("the value %s in get result is not a tombstone", e.ActualValue.RawBytes) @@ -2208,13 +2208,13 @@ func TestMVCCInitPut(t *testing.T) { } // But doesn't if we *don't* fail on tombstones. - err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 5}, hlc.ClockTimestamp{}, value1, false, nil) + err = MVCCInitPut(ctx, engine, testKey1, hlc.Timestamp{Logical: 5}, value1, false, MVCCWriteOptions{}) if err != nil { t.Fatal(err) } // A repeat of the command with a different value will fail. - err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 6}, hlc.ClockTimestamp{}, value2, false, nil) + err = MVCCInitPut(ctx, engine, testKey1, hlc.Timestamp{Logical: 6}, value2, false, MVCCWriteOptions{}) if e := (*kvpb.ConditionFailedError)(nil); errors.As(err, &e) { if !bytes.Equal(e.ActualValue.RawBytes, value1.RawBytes) { t.Fatalf("the value %s in get result does not match the value %s in request", @@ -2270,19 +2270,19 @@ func TestMVCCInitPutWithTxn(t *testing.T) { txn := *txn1 txn.Sequence++ - err := MVCCInitPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value1, false, &txn) + err := MVCCInitPut(ctx, engine, testKey1, txn.ReadTimestamp, value1, false, MVCCWriteOptions{Txn: &txn}) require.NoError(t, err) // A repeat of the command will still succeed. txn.Sequence++ - err = MVCCInitPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value1, false, &txn) + err = MVCCInitPut(ctx, engine, testKey1, txn.ReadTimestamp, value1, false, MVCCWriteOptions{Txn: &txn}) require.NoError(t, err) // A repeat of the command with a different value at a different epoch // will still succeed. txn.Sequence++ txn.Epoch = 2 - err = MVCCInitPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value2, false, &txn) + err = MVCCInitPut(ctx, engine, testKey1, txn.ReadTimestamp, value2, false, MVCCWriteOptions{Txn: &txn}) require.NoError(t, err) // Commit value3. @@ -2295,7 +2295,7 @@ func TestMVCCInitPutWithTxn(t *testing.T) { require.NoError(t, err) // Write value4 with an old timestamp without txn...should get an error. - err = MVCCInitPut(ctx, engine, nil, testKey1, clock.Now(), hlc.ClockTimestamp{}, value4, false, nil) + err = MVCCInitPut(ctx, engine, testKey1, clock.Now(), value4, false, MVCCWriteOptions{}) require.ErrorAs(t, err, new(*kvpb.WriteTooOldError)) } @@ -2309,28 +2309,28 @@ func TestMVCCReverseScan(t *testing.T) { engine := NewDefaultInMemForTesting() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, value1, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, value2, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey2, hlc.Timestamp{WallTime: 1}, value3, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey2, hlc.Timestamp{WallTime: 3}, value4, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey3, hlc.Timestamp{WallTime: 1}, value1, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey4, hlc.Timestamp{WallTime: 1}, value2, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey5, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value5, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey5, hlc.Timestamp{WallTime: 3}, value5, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value6, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey6, hlc.Timestamp{WallTime: 3}, value6, MVCCWriteOptions{}); err != nil { t.Fatal(err) } @@ -2441,10 +2441,10 @@ func TestMVCCReverseScanFirstKeyInFuture(t *testing.T) { // Before fixing #17825, the MVCC version scan on key3 would fall out of the // scan bounds and if it never found another valid key before reaching // KeyMax, would stop the ReverseScan from continuing. - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey2, hlc.Timestamp{WallTime: 1}, value2, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey3, hlc.Timestamp{WallTime: 3}, value3, MVCCWriteOptions{}); err != nil { t.Fatal(err) } @@ -2477,12 +2477,12 @@ func TestMVCCReverseScanSeeksOverRepeatedKeys(t *testing.T) { // written. Repeat the key enough times to make sure the `SeekForPrev()` // optimization will be used. for i := 1; i <= 10; i++ { - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: int64(i)}, hlc.ClockTimestamp{}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey2, hlc.Timestamp{WallTime: int64(i)}, value2, MVCCWriteOptions{}); err != nil { t.Fatal(err) } } txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 11}) - if err := MVCCPut(ctx, engine, nil, testKey2, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { + if err := MVCCPut(ctx, engine, testKey2, txn1ts.ReadTimestamp, value2, MVCCWriteOptions{Txn: txn1ts}); err != nil { t.Fatal(err) } @@ -2527,7 +2527,7 @@ func TestMVCCReverseScanStopAtSmallestKey(t *testing.T) { defer engine.Close() for i := 1; i <= numPuts; i++ { - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: int64(i)}, hlc.ClockTimestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey1, hlc.Timestamp{WallTime: int64(i)}, value1, MVCCWriteOptions{}); err != nil { t.Fatal(err) } } @@ -2564,7 +2564,7 @@ func TestMVCCResolveTxn(t *testing.T) { engine := NewDefaultInMemForTesting() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { + if err := MVCCPut(ctx, engine, testKey1, txn1.ReadTimestamp, value1, MVCCWriteOptions{Txn: txn1}); err != nil { t.Fatal(err) } @@ -2611,12 +2611,12 @@ func TestMVCCResolveNewerIntent(t *testing.T) { defer engine.Close() // Write first value. - if err := MVCCPut(ctx, engine, nil, testKey1, txn1Commit.WriteTimestamp, hlc.ClockTimestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey1, txn1Commit.WriteTimestamp, value1, MVCCWriteOptions{}); err != nil { t.Fatal(err) } // Now, put down an intent which should return a write too old error // (but will still write the intent at tx1Commit.Timestamp+1. - err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1) + err := MVCCPut(ctx, engine, testKey1, txn1.ReadTimestamp, value2, MVCCWriteOptions{Txn: txn1}) if !errors.HasType(err, (*kvpb.WriteTooOldError)(nil)) { t.Fatalf("expected write too old error; got %s", err) } @@ -2767,7 +2767,7 @@ func TestMVCCResolveIntentTxnTimestampMismatch(t *testing.T) { txn.TxnMeta.WriteTimestamp.Forward(tsEarly.Add(10, 0)) // Write an intent which has txn.WriteTimestamp > meta.timestamp. - if err := MVCCPut(ctx, engine, nil, testKey1, tsEarly, hlc.ClockTimestamp{}, value1, txn); err != nil { + if err := MVCCPut(ctx, engine, testKey1, tsEarly, value1, MVCCWriteOptions{Txn: txn}); err != nil { t.Fatal(err) } @@ -2812,9 +2812,9 @@ func TestMVCCConditionalPutOldTimestamp(t *testing.T) { ctx := context.Background() engine := NewDefaultInMemForTesting() defer engine.Close() - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil) + err := MVCCPut(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, value1, MVCCWriteOptions{}) require.NoError(t, err) - err = MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value2, nil) + err = MVCCPut(ctx, engine, testKey1, hlc.Timestamp{WallTime: 3}, value2, MVCCWriteOptions{}) require.NoError(t, err) // Check that a write too old error is thrown, regardless of whether the value @@ -2825,7 +2825,7 @@ func TestMVCCConditionalPutOldTimestamp(t *testing.T) { // Condition matches. value2, } { - err = MVCCConditionalPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, value3, expVal.TagAndDataBytes(), CPutFailIfMissing, nil) + err = MVCCConditionalPut(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, value3, expVal.TagAndDataBytes(), CPutFailIfMissing, MVCCWriteOptions{}) require.ErrorAs(t, err, new(*kvpb.WriteTooOldError)) // Either way, no new value is written. @@ -2848,14 +2848,14 @@ func TestMVCCMultiplePutOldTimestamp(t *testing.T) { engine := NewDefaultInMemForTesting() defer engine.Close() - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value1, nil) + err := MVCCPut(ctx, engine, testKey1, hlc.Timestamp{WallTime: 3}, value1, MVCCWriteOptions{}) require.NoError(t, err) // Verify the first txn Put returns a write too old error and does not // write a new value. txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) txn.Sequence++ - err = MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn) + err = MVCCPut(ctx, engine, testKey1, txn.ReadTimestamp, value2, MVCCWriteOptions{Txn: txn}) var wtoErr *kvpb.WriteTooOldError require.ErrorAs(t, err, &wtoErr) expTS := hlc.Timestamp{WallTime: 3, Logical: 1} @@ -2869,7 +2869,7 @@ func TestMVCCMultiplePutOldTimestamp(t *testing.T) { // timestamp and verify no WriteTooOldError. txn.BumpReadTimestamp(wtoErr.ActualTimestamp) txn.Sequence++ - err = MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn) + err = MVCCPut(ctx, engine, testKey1, txn.ReadTimestamp, value3, MVCCWriteOptions{Txn: txn}) require.NoError(t, err) // Verify new value was actually written at (3, 1). valueRes, err = MVCCGet(ctx, engine, testKey1, hlc.MaxTimestamp, MVCCGetOptions{Txn: txn}) @@ -2888,7 +2888,7 @@ func TestMVCCPutNegativeTimestampError(t *testing.T) { timestamp := hlc.Timestamp{WallTime: -1} expectedErrorString := fmt.Sprintf("cannot write to %q at timestamp %s", testKey1, timestamp) - err := MVCCPut(ctx, engine, nil, testKey1, timestamp, hlc.ClockTimestamp{}, value1, nil) + err := MVCCPut(ctx, engine, testKey1, timestamp, value1, MVCCWriteOptions{}) require.EqualError(t, err, expectedErrorString) } @@ -2904,7 +2904,7 @@ func TestMVCCPutOldOrigTimestampNewCommitTimestamp(t *testing.T) { engine := NewDefaultInMemForTesting() defer engine.Close() - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value1, nil) + err := MVCCPut(ctx, engine, testKey1, hlc.Timestamp{WallTime: 3}, value1, MVCCWriteOptions{}) require.NoError(t, err) // Perform a transactional Put with a transaction whose read timestamp is @@ -2913,7 +2913,7 @@ func TestMVCCPutOldOrigTimestampNewCommitTimestamp(t *testing.T) { txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) txn.WriteTimestamp = hlc.Timestamp{WallTime: 5} txn.Sequence++ - err = MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn) + err = MVCCPut(ctx, engine, testKey1, txn.ReadTimestamp, value2, MVCCWriteOptions{Txn: txn}) // Verify that the Put returned a WriteTooOld with the ActualTime set to the // transactions provisional commit timestamp. @@ -2936,7 +2936,7 @@ func TestMVCCAbortTxn(t *testing.T) { engine := NewDefaultInMemForTesting() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { + if err := MVCCPut(ctx, engine, testKey1, txn1.ReadTimestamp, value1, MVCCWriteOptions{Txn: txn1}); err != nil { t.Fatal(err) } @@ -2968,14 +2968,14 @@ func TestMVCCAbortTxnWithPreviousVersion(t *testing.T) { engine := NewDefaultInMemForTesting() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey1, hlc.Timestamp{Logical: 1}, value1, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, value2, MVCCWriteOptions{}); err != nil { t.Fatal(err) } txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 2}) - if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn1ts); err != nil { + if err := MVCCPut(ctx, engine, testKey1, txn1ts.ReadTimestamp, value3, MVCCWriteOptions{Txn: txn1ts}); err != nil { t.Fatal(err) } @@ -3020,27 +3020,27 @@ func TestMVCCWriteWithDiffTimestampsAndEpochs(t *testing.T) { // Start with epoch 1. txn := *txn1 txn.Sequence++ - err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value1, &txn) + err := MVCCPut(ctx, engine, testKey1, txn.ReadTimestamp, value1, MVCCWriteOptions{Txn: &txn}) require.NoError(t, err) // Now write with greater timestamp and epoch 2. txne2 := txn txne2.Sequence++ txne2.Epoch = 2 txne2.WriteTimestamp = hlc.Timestamp{WallTime: 1} - err = MVCCPut(ctx, engine, nil, testKey1, txne2.ReadTimestamp, hlc.ClockTimestamp{}, value2, &txne2) + err = MVCCPut(ctx, engine, testKey1, txne2.ReadTimestamp, value2, MVCCWriteOptions{Txn: &txne2}) require.NoError(t, err) // Try a write with an earlier timestamp; this is just ignored. txne2.Sequence++ txne2.WriteTimestamp = hlc.Timestamp{WallTime: 1} - err = MVCCPut(ctx, engine, nil, testKey1, txne2.ReadTimestamp, hlc.ClockTimestamp{}, value1, &txne2) + err = MVCCPut(ctx, engine, testKey1, txne2.ReadTimestamp, value1, MVCCWriteOptions{Txn: &txne2}) require.NoError(t, err) // Try a write with an earlier epoch; ignored with error. - err = MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value1, &txn) + err = MVCCPut(ctx, engine, testKey1, txn.ReadTimestamp, value1, MVCCWriteOptions{Txn: &txn}) require.Error(t, err) require.Regexp(t, "put with epoch 1 came after put with epoch 2 in txn", err) // Try a write with different value using both later timestamp and epoch. txne2.Sequence++ - err = MVCCPut(ctx, engine, nil, testKey1, txne2.ReadTimestamp, hlc.ClockTimestamp{}, value3, &txne2) + err = MVCCPut(ctx, engine, testKey1, txne2.ReadTimestamp, value3, MVCCWriteOptions{Txn: &txne2}) require.NoError(t, err) // Resolve the intent. txne2Commit := txne2 @@ -3054,7 +3054,7 @@ func TestMVCCWriteWithDiffTimestampsAndEpochs(t *testing.T) { expTS := txne2Commit.WriteTimestamp.Next() // Now try writing an earlier value without a txn--should get WriteTooOldError. - err = MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value4, nil) + err = MVCCPut(ctx, engine, testKey1, hlc.Timestamp{Logical: 1}, value4, MVCCWriteOptions{}) var wtoErr *kvpb.WriteTooOldError require.ErrorAs(t, err, &wtoErr) require.Equal(t, expTS, wtoErr.ActualTimestamp) @@ -3087,12 +3087,12 @@ func TestMVCCGetWithDiffEpochs(t *testing.T) { defer engine.Close() // Write initial value without a txn. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey1, hlc.Timestamp{Logical: 1}, value1, MVCCWriteOptions{}); err != nil { t.Fatal(err) } // Now write using txn1, epoch 1. txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) - if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { + if err := MVCCPut(ctx, engine, testKey1, txn1ts.ReadTimestamp, value2, MVCCWriteOptions{Txn: txn1ts}); err != nil { t.Fatal(err) } // Try reading using different txns & epochs. @@ -3144,14 +3144,14 @@ func TestMVCCGetWithDiffEpochsAndTimestamps(t *testing.T) { defer engine.Close() // Write initial value without a txn at timestamp 1. - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil) + err := MVCCPut(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, value1, MVCCWriteOptions{}) require.NoError(t, err) // Write another value without a txn at timestamp 3. - err = MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value2, nil) + err = MVCCPut(ctx, engine, testKey1, hlc.Timestamp{WallTime: 3}, value2, MVCCWriteOptions{}) require.NoError(t, err) // Now write using txn1, epoch 1. txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 4}) - err = MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn1ts) + err = MVCCPut(ctx, engine, testKey1, txn1ts.ReadTimestamp, value3, MVCCWriteOptions{Txn: txn1ts}) require.NoError(t, err) // Try reading using different epochs & timestamps. testCases := []struct { @@ -3200,7 +3200,7 @@ func TestMVCCGetWithOldEpoch(t *testing.T) { engine := NewDefaultInMemForTesting() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1e2.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1e2); err != nil { + if err := MVCCPut(ctx, engine, testKey1, txn1e2.ReadTimestamp, value2, MVCCWriteOptions{Txn: txn1e2}); err != nil { t.Fatal(err) } _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{ @@ -3241,7 +3241,7 @@ func TestMVCCDeleteRangeWithSequence(t *testing.T) { for i := enginepb.TxnSeq(0); i < 3; i++ { key := append(prefix, []byte(strconv.Itoa(int(i)))...) txn.Sequence = 2 + i - if err := MVCCPut(ctx, engine, nil, key, txn.WriteTimestamp, hlc.ClockTimestamp{}, value1, &txn); err != nil { + if err := MVCCPut(ctx, engine, key, txn.WriteTimestamp, value1, MVCCWriteOptions{Txn: &txn}); err != nil { t.Fatal(err) } } @@ -3249,15 +3249,15 @@ func TestMVCCDeleteRangeWithSequence(t *testing.T) { // Perform the initial DeleteRange. const origSeq = 6 txn.Sequence = origSeq - origDeleted, _, origNum, err := MVCCDeleteRange(ctx, engine, nil, - prefix, prefix.PrefixEnd(), math.MaxInt64, txn.WriteTimestamp, hlc.ClockTimestamp{}, &txn, true) + origDeleted, _, origNum, err := MVCCDeleteRange(ctx, engine, prefix, prefix.PrefixEnd(), + math.MaxInt64, txn.WriteTimestamp, MVCCWriteOptions{Txn: &txn}, true) if err != nil { t.Fatal(err) } txn.Sequence = tc.sequence - deleted, _, num, err := MVCCDeleteRange(ctx, engine, nil, - prefix, prefix.PrefixEnd(), math.MaxInt64, txn.WriteTimestamp, hlc.ClockTimestamp{}, &txn, true) + deleted, _, num, err := MVCCDeleteRange(ctx, engine, prefix, prefix.PrefixEnd(), + math.MaxInt64, txn.WriteTimestamp, MVCCWriteOptions{Txn: &txn}, true) if tc.expErr != "" && err != nil { if !testutils.IsError(err, tc.expErr) { t.Fatalf("unexpected error: %+v", err) @@ -3296,7 +3296,7 @@ func TestMVCCGetWithPushedTimestamp(t *testing.T) { defer engine.Close() // Start with epoch 1. - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { + if err := MVCCPut(ctx, engine, testKey1, txn1.ReadTimestamp, value1, MVCCWriteOptions{Txn: txn1}); err != nil { t.Fatal(err) } // Resolve the intent, pushing its timestamp forward. @@ -3323,10 +3323,10 @@ func TestMVCCResolveWithDiffEpochs(t *testing.T) { engine := NewDefaultInMemForTesting() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { + if err := MVCCPut(ctx, engine, testKey1, txn1.ReadTimestamp, value1, MVCCWriteOptions{Txn: txn1}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, txn1e2.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1e2); err != nil { + if err := MVCCPut(ctx, engine, testKey2, txn1e2.ReadTimestamp, value2, MVCCWriteOptions{Txn: txn1e2}); err != nil { t.Fatal(err) } numKeys, _, _, _, err := MVCCResolveWriteIntentRange(ctx, engine, nil, @@ -3365,7 +3365,7 @@ func TestMVCCResolveWithUpdatedTimestamp(t *testing.T) { engine := NewDefaultInMemForTesting() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { + if err := MVCCPut(ctx, engine, testKey1, txn1.ReadTimestamp, value1, MVCCWriteOptions{Txn: txn1}); err != nil { t.Fatal(err) } @@ -3415,7 +3415,7 @@ func TestMVCCResolveWithPushedTimestamp(t *testing.T) { engine := NewDefaultInMemForTesting() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { + if err := MVCCPut(ctx, engine, testKey1, txn1.ReadTimestamp, value1, MVCCWriteOptions{Txn: txn1}); err != nil { t.Fatal(err) } valueRes, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{ @@ -3475,7 +3475,7 @@ func TestMVCCResolveTxnNoOps(t *testing.T) { } // Add key and resolve despite there being no intent. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey1, hlc.Timestamp{Logical: 1}, value1, MVCCWriteOptions{}); err != nil { t.Fatal(err) } if _, _, _, err := MVCCResolveWriteIntent(ctx, engine, nil, @@ -3485,7 +3485,7 @@ func TestMVCCResolveTxnNoOps(t *testing.T) { } // Write intent and resolve with different txn. - if err := MVCCPut(ctx, engine, nil, testKey2, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1); err != nil { + if err := MVCCPut(ctx, engine, testKey2, txn1.ReadTimestamp, value2, MVCCWriteOptions{Txn: txn1}); err != nil { t.Fatal(err) } @@ -3506,16 +3506,16 @@ func TestMVCCResolveTxnRange(t *testing.T) { engine := NewDefaultInMemForTesting() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { + if err := MVCCPut(ctx, engine, testKey1, txn1.ReadTimestamp, value1, MVCCWriteOptions{Txn: txn1}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, testKey2, hlc.Timestamp{Logical: 1}, value2, MVCCWriteOptions{}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, txn2.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn2); err != nil { + if err := MVCCPut(ctx, engine, testKey3, txn2.ReadTimestamp, value3, MVCCWriteOptions{Txn: txn2}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value4, txn1); err != nil { + if err := MVCCPut(ctx, engine, testKey4, txn1.ReadTimestamp, value4, MVCCWriteOptions{Txn: txn1}); err != nil { t.Fatal(err) } @@ -3592,14 +3592,14 @@ func TestMVCCResolveTxnRangeResume(t *testing.T) { key0 := roachpb.Key(fmt.Sprintf("%02d%d", i+0, i+0)) key1 := roachpb.Key(fmt.Sprintf("%02d%d", i+1, i+1)) key2 := roachpb.Key(fmt.Sprintf("%02d%d", i+2, i+2)) - if err := MVCCPut(ctx, engine, nil, key0, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { + if err := MVCCPut(ctx, engine, key0, txn1.ReadTimestamp, value1, MVCCWriteOptions{Txn: txn1}); err != nil { t.Fatal(err) } txn2ts := makeTxn(*txn2, hlc.Timestamp{Logical: 2}) - if err := MVCCPut(ctx, engine, nil, key1, txn2ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn2ts); err != nil { + if err := MVCCPut(ctx, engine, key1, txn2ts.ReadTimestamp, value2, MVCCWriteOptions{Txn: txn2ts}); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, key2, hlc.Timestamp{Logical: 3}, hlc.ClockTimestamp{}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, key2, hlc.Timestamp{Logical: 3}, value3, MVCCWriteOptions{}); err != nil { t.Fatal(err) } } @@ -3713,7 +3713,7 @@ func writeToEngine( log.Infof(ctx, "Put: %s, seq: %d, writets: %s", p.key.String(), txn.Sequence, txn.WriteTimestamp.String()) } - require.NoError(t, MVCCPut(ctx, eng, nil, p.key, txn.ReadTimestamp, hlc.ClockTimestamp{}, p.values[i], txn)) + require.NoError(t, MVCCPut(ctx, eng, p.key, txn.ReadTimestamp, p.values[i], MVCCWriteOptions{Txn: txn})) } } } @@ -4112,7 +4112,7 @@ func TestFindSplitKey(t *testing.T) { v := strings.Repeat("X", 10-len(k)) val := roachpb.MakeValueFromString(v) // Write the key and value through MVCC - if err := MVCCPut(ctx, engine, ms, []byte(k), hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, val, nil); err != nil { + if err := MVCCPut(ctx, engine, []byte(k), hlc.Timestamp{Logical: 1}, val, MVCCWriteOptions{Stats: ms}); err != nil { t.Fatal(err) } } @@ -4477,7 +4477,7 @@ func TestFindBalancedSplitKeys(t *testing.T) { expKey = key } val := roachpb.MakeValueFromString(strings.Repeat("X", test.valSizes[j])) - if err := MVCCPut(ctx, engine, ms, key, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, val, nil); err != nil { + if err := MVCCPut(ctx, engine, key, hlc.Timestamp{Logical: 1}, val, MVCCWriteOptions{Stats: ms}); err != nil { t.Fatal(err) } } @@ -4524,7 +4524,7 @@ func testPopulateKeysWithVersions( ts := hlc.Timestamp{Logical: int32(j)} require.NoError( t, - MVCCPut(ctx, engine, ms, []byte(k), ts, hlc.ClockTimestamp{}, val, nil), + MVCCPut(ctx, engine, []byte(k), ts, val, MVCCWriteOptions{Stats: ms}), ) } } @@ -4758,16 +4758,15 @@ func TestMVCCGarbageCollect(t *testing.T) { } for _, val := range test.vals[i : i+1] { if i == len(test.vals)-1 && test.isDeleted { - if _, err := MVCCDelete(ctx, engine, ms, test.key, val.Timestamp, hlc.ClockTimestamp{}, - nil); err != nil { + if _, err := MVCCDelete(ctx, engine, test.key, val.Timestamp, MVCCWriteOptions{Stats: ms}); err != nil { t.Fatal(err) } continue } valCpy := *protoutil.Clone(&val).(*roachpb.Value) valCpy.Timestamp = hlc.Timestamp{} - if err := MVCCPut(ctx, engine, ms, test.key, val.Timestamp, hlc.ClockTimestamp{}, - valCpy, nil); err != nil { + if err := MVCCPut(ctx, engine, test.key, val.Timestamp, + valCpy, MVCCWriteOptions{Stats: ms}); err != nil { t.Fatal(err) } } @@ -4907,7 +4906,7 @@ func TestMVCCGarbageCollectNonDeleted(t *testing.T) { for _, val := range test.vals { valCpy := *protoutil.Clone(&val).(*roachpb.Value) valCpy.Timestamp = hlc.Timestamp{} - if err := MVCCPut(ctx, engine, nil, test.key, val.Timestamp, hlc.ClockTimestamp{}, valCpy, nil); err != nil { + if err := MVCCPut(ctx, engine, test.key, val.Timestamp, valCpy, MVCCWriteOptions{}); err != nil { t.Fatal(err) } } @@ -4941,7 +4940,7 @@ func TestMVCCGarbageCollectIntent(t *testing.T) { key := roachpb.Key("a") { val1 := roachpb.MakeValueFromBytes(bytes) - if err := MVCCPut(ctx, engine, nil, key, ts1, hlc.ClockTimestamp{}, val1, nil); err != nil { + if err := MVCCPut(ctx, engine, key, ts1, val1, MVCCWriteOptions{}); err != nil { t.Fatal(err) } } @@ -4949,7 +4948,7 @@ func TestMVCCGarbageCollectIntent(t *testing.T) { TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts2}, ReadTimestamp: ts2, } - if _, err := MVCCDelete(ctx, engine, nil, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { + if _, err := MVCCDelete(ctx, engine, key, txn.ReadTimestamp, MVCCWriteOptions{Txn: txn}); err != nil { t.Fatal(err) } keys := []kvpb.GCRequest_GCKey{ @@ -5043,7 +5042,7 @@ func TestMVCCGarbageCollectUsesSeekLTAppropriately(t *testing.T) { for _, seconds := range key.timestamps { val := roachpb.MakeValueFromBytes(bytes) ts := toHLC(seconds) - if err := MVCCPut(ctx, engine, ms, roachpb.Key(key.key), ts, hlc.ClockTimestamp{}, val, nil); err != nil { + if err := MVCCPut(ctx, engine, roachpb.Key(key.key), ts, val, MVCCWriteOptions{Stats: ms}); err != nil { t.Fatal(err) } } @@ -5186,11 +5185,11 @@ func (d rangeTestData) populateEngine( for _, v := range d { if v.rangeTombstone.Timestamp.IsEmpty() { if v.point.Value != nil { - require.NoError(t, MVCCPut(ctx, engine, ms, v.point.Key.Key, v.point.Key.Timestamp, - hlc.ClockTimestamp{}, roachpb.MakeValueFromBytes(v.point.Value), v.txn), + require.NoError(t, MVCCPut(ctx, engine, v.point.Key.Key, v.point.Key.Timestamp, + roachpb.MakeValueFromBytes(v.point.Value), MVCCWriteOptions{Txn: v.txn, Stats: ms}), "failed to insert test value into engine (%s)", v.point.Key.String()) } else { - _, err := MVCCDelete(ctx, engine, ms, v.point.Key.Key, v.point.Key.Timestamp, hlc.ClockTimestamp{}, v.txn) + _, err := MVCCDelete(ctx, engine, v.point.Key.Key, v.point.Key.Timestamp, MVCCWriteOptions{Txn: v.txn, Stats: ms}) require.NoError(t, err, "failed to insert tombstone value into engine (%s)", v.point.Key.String()) } ts = v.point.Key.Timestamp @@ -6095,7 +6094,7 @@ func TestResolveIntentWithLowerEpoch(t *testing.T) { defer engine.Close() // Lay down an intent with a high epoch. - if err := MVCCPut(ctx, engine, nil, testKey1, txn1e2.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1e2); err != nil { + if err := MVCCPut(ctx, engine, testKey1, txn1e2.ReadTimestamp, value1, MVCCWriteOptions{Txn: txn1e2}); err != nil { t.Fatal(err) } // Resolve the intent with a low epoch. diff --git a/pkg/storage/pebble_mvcc_scanner_test.go b/pkg/storage/pebble_mvcc_scanner_test.go index e47433608390..7de6ddc30b56 100644 --- a/pkg/storage/pebble_mvcc_scanner_test.go +++ b/pkg/storage/pebble_mvcc_scanner_test.go @@ -215,7 +215,7 @@ func TestMVCCScanWithMemoryAccounting(t *testing.T) { defer batch.Close() for i := 0; i < 10; i++ { key := makeKey(nil, i) - require.NoError(t, MVCCPut(context.Background(), batch, nil, key, ts1, hlc.ClockTimestamp{}, val, &txn1)) + require.NoError(t, MVCCPut(context.Background(), batch, key, ts1, val, MVCCWriteOptions{Txn: &txn1})) } require.NoError(t, batch.Commit(true)) }() diff --git a/pkg/storage/pebble_test.go b/pkg/storage/pebble_test.go index 753a3b5f9238..9329425e8c11 100644 --- a/pkg/storage/pebble_test.go +++ b/pkg/storage/pebble_test.go @@ -470,7 +470,7 @@ func fillInData(ctx context.Context, engine Engine, data []testValue) error { batch := engine.NewBatch() defer batch.Close() for _, val := range data { - if err := MVCCPut(ctx, batch, nil, val.key, val.timestamp, hlc.ClockTimestamp{}, val.value, val.txn); err != nil { + if err := MVCCPut(ctx, batch, val.key, val.timestamp, val.value, MVCCWriteOptions{Txn: val.txn}); err != nil { return err } } diff --git a/pkg/storage/sst_test.go b/pkg/storage/sst_test.go index ce4b0846cd9b..8a937a384f51 100644 --- a/pkg/storage/sst_test.go +++ b/pkg/storage/sst_test.go @@ -79,7 +79,7 @@ func TestCheckSSTConflictsMaxIntents(t *testing.T) { require.NoError(t, batch.PutMVCC(mvccKey, mvccValue)) } for _, key := range intents { - require.NoError(t, MVCCPut(ctx, batch, nil, roachpb.Key(key), txn1TS, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("intent"), txn1)) + require.NoError(t, MVCCPut(ctx, batch, roachpb.Key(key), txn1TS, roachpb.MakeValueFromString("intent"), MVCCWriteOptions{Txn: txn1})) } require.NoError(t, batch.Commit(true)) batch.Close()