From d5b3fe7c91e1da9745b337e66ad26c776fde0aee Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Wed, 2 Mar 2022 22:45:33 -0500 Subject: [PATCH] kv: plumb local timestamp into MVCC Unused plumbing in preparation for a later commit. --- pkg/ccl/storageccl/engineccl/bench_test.go | 2 +- .../storageccl/engineccl/encrypted_fs_test.go | 1 + pkg/cli/debug.go | 7 +- .../kvcoord/dist_sender_server_test.go | 4 +- pkg/kv/kvserver/abortspan/abortspan.go | 6 +- pkg/kv/kvserver/batch_spanset_test.go | 1 + .../batcheval/cmd_add_sstable_test.go | 2 +- .../batcheval/cmd_clear_range_test.go | 2 +- .../kvserver/batcheval/cmd_conditional_put.go | 6 +- pkg/kv/kvserver/batcheval/cmd_delete.go | 2 +- pkg/kv/kvserver/batcheval/cmd_delete_range.go | 4 +- .../kvserver/batcheval/cmd_end_transaction.go | 8 +- .../batcheval/cmd_end_transaction_test.go | 8 +- pkg/kv/kvserver/batcheval/cmd_export_test.go | 4 +- .../kvserver/batcheval/cmd_heartbeat_txn.go | 2 +- pkg/kv/kvserver/batcheval/cmd_increment.go | 3 +- pkg/kv/kvserver/batcheval/cmd_init_put.go | 6 +- pkg/kv/kvserver/batcheval/cmd_push_txn.go | 2 +- pkg/kv/kvserver/batcheval/cmd_put.go | 4 +- .../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 | 10 +- pkg/kv/kvserver/batcheval/cmd_refresh_test.go | 4 +- .../batcheval/cmd_resolve_intent_test.go | 4 +- .../batcheval/cmd_revert_range_test.go | 10 +- pkg/kv/kvserver/batcheval/cmd_scan_test.go | 4 +- .../batcheval/cmd_truncate_log_test.go | 2 +- pkg/kv/kvserver/batcheval/intent_test.go | 4 +- pkg/kv/kvserver/batcheval/transaction_test.go | 2 +- pkg/kv/kvserver/client_merge_test.go | 4 +- pkg/kv/kvserver/consistency_queue_test.go | 2 +- pkg/kv/kvserver/gc/data_distribution_test.go | 2 +- pkg/kv/kvserver/gc/gc_test.go | 4 +- pkg/kv/kvserver/loqrecovery/apply.go | 5 +- .../kvserver/loqrecovery/recovery_env_test.go | 9 +- pkg/kv/kvserver/mvcc_gc_queue_test.go | 8 +- .../rangefeed/catchup_scan_bench_test.go | 2 +- .../kvserver/rangefeed/catchup_scan_test.go | 4 +- 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 | 3 +- pkg/kv/kvserver/replica_destroy.go | 2 +- pkg/kv/kvserver/replica_evaluate_test.go | 2 +- pkg/kv/kvserver/replica_raft.go | 10 +- pkg/kv/kvserver/replica_raftstorage.go | 6 +- pkg/kv/kvserver/replica_test.go | 16 +- pkg/kv/kvserver/stateloader/stateloader.go | 22 +- pkg/kv/kvserver/store.go | 13 +- pkg/kv/kvserver/store_init.go | 14 +- pkg/kv/kvserver/store_snapshot.go | 2 +- pkg/kv/kvserver/store_test.go | 3 +- pkg/kv/kvserver/stores.go | 2 +- pkg/kv/kvserver/txn_wait_queue_test.go | 2 +- pkg/server/node_test.go | 2 +- pkg/server/node_tombstone_storage.go | 2 +- pkg/server/server_test.go | 5 +- pkg/server/settings_cache.go | 2 +- pkg/storage/bench_pebble_test.go | 4 +- pkg/storage/bench_test.go | 29 +- pkg/storage/engine_test.go | 8 +- pkg/storage/metamorphic/operations.go | 12 +- pkg/storage/mvcc.go | 109 +++-- pkg/storage/mvcc_history_test.go | 48 +- pkg/storage/mvcc_incremental_iterator_test.go | 55 +-- pkg/storage/mvcc_logical_ops_test.go | 16 +- pkg/storage/mvcc_stats_test.go | 58 +-- pkg/storage/mvcc_test.go | 449 +++++++++--------- pkg/storage/pebble_mvcc_scanner_test.go | 2 +- pkg/storage/pebble_test.go | 2 +- pkg/storage/sst_test.go | 2 +- 73 files changed, 563 insertions(+), 513 deletions(-) diff --git a/pkg/ccl/storageccl/engineccl/bench_test.go b/pkg/ccl/storageccl/engineccl/bench_test.go index af76e0f51033..8d50740c1083 100644 --- a/pkg/ccl/storageccl/engineccl/bench_test.go +++ b/pkg/ccl/storageccl/engineccl/bench_test.go @@ -100,7 +100,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, value, nil); err != nil { + if err := storage.MVCCPut(ctx, batch, nil, key, timestamp, hlc.ClockTimestamp{}, value, nil); err != nil { return nil, err } } diff --git a/pkg/ccl/storageccl/engineccl/encrypted_fs_test.go b/pkg/ccl/storageccl/engineccl/encrypted_fs_test.go index cbd36a07ec47..e23922101241 100644 --- a/pkg/ccl/storageccl/engineccl/encrypted_fs_test.go +++ b/pkg/ccl/storageccl/engineccl/encrypted_fs_test.go @@ -399,6 +399,7 @@ func TestPebbleEncryption2(t *testing.T) { nil, /* ms */ roachpb.Key(key), hlc.Timestamp{}, + hlc.ClockTimestamp{}, roachpb.MakeValueFromBytes([]byte(val)), nil, /* txn */ ) diff --git a/pkg/cli/debug.go b/pkg/cli/debug.go index e7d27cb0aec9..03b4b6d187a9 100644 --- a/pkg/cli/debug.go +++ b/pkg/cli/debug.go @@ -1280,7 +1280,7 @@ func removeDeadReplicas( if err != nil { return nil, errors.Wrap(err, "loading MVCCStats") } - err = storage.MVCCPutProto(ctx, batch, &ms, key, clock.Now(), nil /* txn */, &desc) + err = storage.MVCCPutProto(ctx, batch, &ms, key, clock.Now(), hlc.ClockTimestamp{}, nil, &desc) if wiErr := (*roachpb.WriteIntentError)(nil); errors.As(err, &wiErr) { if len(wiErr.Intents) != 1 { return nil, errors.Errorf("expected 1 intent, found %d: %s", len(wiErr.Intents), wiErr) @@ -1328,7 +1328,7 @@ func removeDeadReplicas( // A crude form of the intent resolution process: abort the // transaction by deleting its record. txnKey := keys.TransactionKey(intent.Txn.Key, intent.Txn.ID) - if err := storage.MVCCDelete(ctx, batch, &ms, txnKey, hlc.Timestamp{}, nil); err != nil { + if err := storage.MVCCDelete(ctx, batch, &ms, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil); err != nil { return nil, err } update := roachpb.LockUpdate{ @@ -1340,8 +1340,7 @@ func removeDeadReplicas( return nil, err } // With the intent resolved, we can try again. - if err := storage.MVCCPutProto(ctx, batch, &ms, key, clock.Now(), - nil /* txn */, &desc); err != nil { + if err := storage.MVCCPutProto(ctx, batch, &ms, key, clock.Now(), hlc.ClockTimestamp{}, nil, &desc); err != nil { return nil, err } } else if err != nil { diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go b/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go index 9a8b8aa3dd82..903b30ad4bcf 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go @@ -75,9 +75,7 @@ func TestRangeLookupWithOpenTransaction(t *testing.T) { key := testutils.MakeKey(keys.Meta1Prefix, roachpb.KeyMax) now := s.Clock().Now() txn := roachpb.MakeTransaction("txn", roachpb.Key("foobar"), 0, now, 0, int32(s.SQLInstanceID())) - if err := storage.MVCCPutProto( - context.Background(), s.Engines()[0], - nil, key, now, &txn, &roachpb.RangeDescriptor{}); err != nil { + if err := storage.MVCCPutProto(context.Background(), s.Engines()[0], nil, key, now, hlc.ClockTimestamp{}, &txn, &roachpb.RangeDescriptor{}); err != nil { t.Fatal(err) } diff --git a/pkg/kv/kvserver/abortspan/abortspan.go b/pkg/kv/kvserver/abortspan/abortspan.go index 8ecc24c9ef4f..5c5bb5f4153a 100644 --- a/pkg/kv/kvserver/abortspan/abortspan.go +++ b/pkg/kv/kvserver/abortspan/abortspan.go @@ -123,7 +123,7 @@ func (sc *AbortSpan) Del( ctx context.Context, reader storage.ReadWriter, ms *enginepb.MVCCStats, txnID uuid.UUID, ) error { key := keys.AbortSpanKey(sc.rangeID, txnID) - return storage.MVCCDelete(ctx, reader, ms, key, hlc.Timestamp{}, nil /* txn */) + return storage.MVCCDelete(ctx, reader, ms, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil /* txn */) } // Put writes an entry for the specified transaction ID. @@ -136,7 +136,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{}, nil /* txn */, entry) + return storage.MVCCPutProto(ctx, readWriter, ms, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil /* txn */, entry) } // CopyTo copies the abort span entries to the abort span for the range @@ -181,7 +181,7 @@ func (sc *AbortSpan) CopyTo( } return storage.MVCCPutProto(ctx, w, ms, keys.AbortSpanKey(newRangeID, txnID), - hlc.Timestamp{}, nil, &entry, + hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &entry, ) }); 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 d7d9a4e6ea3c..67d0718e7509 100644 --- a/pkg/kv/kvserver/batch_spanset_test.go +++ b/pkg/kv/kvserver/batch_spanset_test.go @@ -581,6 +581,7 @@ func TestSpanSetMVCCResolveWriteIntentRange(t *testing.T) { nil, // ms roachpb.Key("b"), hlc.Timestamp{WallTime: 10}, // irrelevant + hlc.ClockTimestamp{}, // irrelevant value, nil, // txn ); err != nil { diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go index 352c7b1807bd..7635e89a539e 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go @@ -683,7 +683,7 @@ func TestEvalAddSSTable(t *testing.T) { if kv.WallTimestamp == intentTS { txn = &intentTxn } - require.NoError(t, storage.MVCCPut(ctx, b, nil, kv.Key(), kv.Timestamp(), kv.Value(), txn)) + require.NoError(t, storage.MVCCPut(ctx, b, nil, kv.Key(), kv.Timestamp(), hlc.ClockTimestamp{}, kv.Value(), txn)) } require.NoError(t, b.Commit(false)) stats := engineStats(t, engine, 0) diff --git a/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go b/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go index c31a23aa22ed..84ed9a861541 100644 --- a/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go @@ -105,7 +105,7 @@ func TestCmdClearRangeBytesThreshold(t *testing.T) { var stats enginepb.MVCCStats for i := 0; i < test.keyCount; i++ { key := roachpb.Key(fmt.Sprintf("%04d", i)) - if err := storage.MVCCPut(ctx, eng, &stats, key, hlc.Timestamp{WallTime: int64(i % 2)}, value, nil); err != nil { + if err := storage.MVCCPut(ctx, eng, &stats, key, hlc.Timestamp{WallTime: int64(i % 2)}, hlc.ClockTimestamp{}, value, nil); err != nil { t.Fatal(err) } } diff --git a/pkg/kv/kvserver/batcheval/cmd_conditional_put.go b/pkg/kv/kvserver/batcheval/cmd_conditional_put.go index 17416eb641fc..4ce3ce46e2df 100644 --- a/pkg/kv/kvserver/batcheval/cmd_conditional_put.go +++ b/pkg/kv/kvserver/batcheval/cmd_conditional_put.go @@ -67,9 +67,11 @@ func ConditionalPut( handleMissing := storage.CPutMissingBehavior(args.AllowIfDoesNotExist) var err error if args.Blind { - err = storage.MVCCBlindConditionalPut(ctx, readWriter, cArgs.Stats, args.Key, ts, args.Value, expVal, handleMissing, h.Txn) + err = storage.MVCCBlindConditionalPut( + ctx, readWriter, cArgs.Stats, args.Key, ts, cArgs.Now, args.Value, expVal, handleMissing, h.Txn) } else { - err = storage.MVCCConditionalPut(ctx, readWriter, cArgs.Stats, args.Key, ts, args.Value, expVal, handleMissing, h.Txn) + err = storage.MVCCConditionalPut( + ctx, readWriter, cArgs.Stats, args.Key, ts, cArgs.Now, args.Value, expVal, handleMissing, h.Txn) } // NB: even if MVCC returns an error, it may still have written an intent // into the batch. This allows callers to consume errors like WriteTooOld diff --git a/pkg/kv/kvserver/batcheval/cmd_delete.go b/pkg/kv/kvserver/batcheval/cmd_delete.go index 38a40b19d18c..ee293240fb25 100644 --- a/pkg/kv/kvserver/batcheval/cmd_delete.go +++ b/pkg/kv/kvserver/batcheval/cmd_delete.go @@ -29,7 +29,7 @@ func Delete( args := cArgs.Args.(*roachpb.DeleteRequest) h := cArgs.Header - err := storage.MVCCDelete(ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, h.Txn) + err := storage.MVCCDelete(ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, cArgs.Now, h.Txn) // NB: even if MVCC returns an error, it may still have written an intent // into the batch. This allows callers to consume errors like WriteTooOld // without re-evaluating the batch. This behavior isn't particularly diff --git a/pkg/kv/kvserver/batcheval/cmd_delete_range.go b/pkg/kv/kvserver/batcheval/cmd_delete_range.go index 60554d27ad4b..58a11e462816 100644 --- a/pkg/kv/kvserver/batcheval/cmd_delete_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_delete_range.go @@ -58,8 +58,8 @@ 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, h.Txn, returnKeys, - ) + ctx, readWriter, cArgs.Stats, args.Key, args.EndKey, + h.MaxSpanRequestKeys, timestamp, cArgs.Now, h.Txn, returnKeys) if err == nil && args.ReturnKeys { reply.Keys = deleted } diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go index be14b556213e..734c8b3065d3 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go @@ -600,7 +600,7 @@ func updateStagingTxn( txn.LockSpans = args.LockSpans txn.InFlightWrites = args.InFlightWrites txnRecord := txn.AsRecord() - return storage.MVCCPutProto(ctx, readWriter, ms, key, hlc.Timestamp{}, nil /* txn */, &txnRecord) + return storage.MVCCPutProto(ctx, readWriter, ms, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord) } // updateFinalizedTxn persists the COMMITTED or ABORTED transaction record with @@ -627,12 +627,12 @@ func updateFinalizedTxn( // BatchRequest writes. return nil } - return storage.MVCCDelete(ctx, readWriter, ms, key, hlc.Timestamp{}, nil /* txn */) + return storage.MVCCDelete(ctx, readWriter, ms, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil) } txn.LockSpans = externalLocks txn.InFlightWrites = nil txnRecord := txn.AsRecord() - return storage.MVCCPutProto(ctx, readWriter, ms, key, hlc.Timestamp{}, nil /* txn */, &txnRecord) + return storage.MVCCPutProto(ctx, readWriter, ms, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord) } // RunCommitTrigger runs the commit trigger from an end transaction request. @@ -1011,7 +1011,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{}, nil, &replicaGCTS); err != nil { + if err := storage.MVCCPutProto(ctx, batch, nil, keys.RangeLastReplicaGCTimestampKey(split.RightDesc.RangeID), hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &replicaGCTS); 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 19cb34232cda..3449ae7ffdf2 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go @@ -908,7 +908,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{}, nil, c.existingTxn); err != nil { + if err := storage.MVCCPutProto(ctx, batch, nil, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, c.existingTxn); err != nil { t.Fatal(err) } } @@ -1012,13 +1012,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, v, &txn); err != nil { + if err := storage.MVCCPut(ctx, batch, nil, k, ts, hlc.ClockTimestamp{}, v, &txn); err != nil { t.Fatal(err) } // Write another value. v.SetString("b") txn.Sequence = 2 - if err := storage.MVCCPut(ctx, batch, nil, k, ts, v, &txn); err != nil { + if err := storage.MVCCPut(ctx, batch, nil, k, ts, hlc.ClockTimestamp{}, v, &txn); err != nil { t.Fatal(err) } @@ -1031,7 +1031,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{}, nil, &txnRec); err != nil { + if err := storage.MVCCPutProto(ctx, batch, nil, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRec); err != nil { t.Fatal(err) } } diff --git a/pkg/kv/kvserver/batcheval/cmd_export_test.go b/pkg/kv/kvserver/batcheval/cmd_export_test.go index d77126d00a35..837ff893d3bd 100644 --- a/pkg/kv/kvserver/batcheval/cmd_export_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_export_test.go @@ -737,7 +737,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, value, nil); err != nil { + if err := storage.MVCCPut(ctx, batch, nil, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { t.Fatal(err) } @@ -748,7 +748,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, value, nil); err != nil { + if err := storage.MVCCPut(ctx, batch, nil, key, ts, hlc.ClockTimestamp{}, value, nil); 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 ba4817001a8a..52032f7fdfa6 100644 --- a/pkg/kv/kvserver/batcheval/cmd_heartbeat_txn.go +++ b/pkg/kv/kvserver/batcheval/cmd_heartbeat_txn.go @@ -79,7 +79,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{}, nil, &txnRecord); err != nil { + if err := storage.MVCCPutProto(ctx, readWriter, cArgs.Stats, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord); 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 7e2ed30eeb7b..aac9e412ef09 100644 --- a/pkg/kv/kvserver/batcheval/cmd_increment.go +++ b/pkg/kv/kvserver/batcheval/cmd_increment.go @@ -32,7 +32,8 @@ func Increment( h := cArgs.Header reply := resp.(*roachpb.IncrementResponse) - newVal, err := storage.MVCCIncrement(ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, h.Txn, args.Increment) + newVal, err := storage.MVCCIncrement( + ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, cArgs.Now, h.Txn, args.Increment) reply.NewValue = newVal // NB: even if MVCC returns an error, it may still have written an intent // into the batch. This allows callers to consume errors like WriteTooOld diff --git a/pkg/kv/kvserver/batcheval/cmd_init_put.go b/pkg/kv/kvserver/batcheval/cmd_init_put.go index 745f4e1d5c3f..1ec82e748183 100644 --- a/pkg/kv/kvserver/batcheval/cmd_init_put.go +++ b/pkg/kv/kvserver/batcheval/cmd_init_put.go @@ -34,9 +34,11 @@ func InitPut( var err error if args.Blind { - err = storage.MVCCBlindInitPut(ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, args.Value, args.FailOnTombstones, h.Txn) + err = storage.MVCCBlindInitPut( + ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, cArgs.Now, args.Value, args.FailOnTombstones, h.Txn) } else { - err = storage.MVCCInitPut(ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, args.Value, args.FailOnTombstones, h.Txn) + err = storage.MVCCInitPut( + ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, cArgs.Now, args.Value, args.FailOnTombstones, h.Txn) } // NB: even if MVCC returns an error, it may still have written an intent // into the batch. This allows callers to consume errors like WriteTooOld diff --git a/pkg/kv/kvserver/batcheval/cmd_push_txn.go b/pkg/kv/kvserver/batcheval/cmd_push_txn.go index ab7c60ddf61d..8134aa90d086 100644 --- a/pkg/kv/kvserver/batcheval/cmd_push_txn.go +++ b/pkg/kv/kvserver/batcheval/cmd_push_txn.go @@ -325,7 +325,7 @@ func PushTxn( // in the timestamp cache. if ok { txnRecord := reply.PusheeTxn.AsRecord() - if err := storage.MVCCPutProto(ctx, readWriter, cArgs.Stats, key, hlc.Timestamp{}, nil, &txnRecord); err != nil { + if err := storage.MVCCPutProto(ctx, readWriter, cArgs.Stats, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord); 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 65f260f961c1..283164a2d5ea 100644 --- a/pkg/kv/kvserver/batcheval/cmd_put.go +++ b/pkg/kv/kvserver/batcheval/cmd_put.go @@ -54,9 +54,9 @@ func Put( } var err error if args.Blind { - err = storage.MVCCBlindPut(ctx, readWriter, ms, args.Key, ts, args.Value, h.Txn) + err = storage.MVCCBlindPut(ctx, readWriter, ms, args.Key, ts, cArgs.Now, args.Value, h.Txn) } else { - err = storage.MVCCPut(ctx, readWriter, ms, args.Key, ts, args.Value, h.Txn) + err = storage.MVCCPut(ctx, readWriter, ms, args.Key, ts, cArgs.Now, args.Value, h.Txn) } // NB: even if MVCC returns an error, it may still have written an intent // into the batch. This allows callers to consume errors like WriteTooOld 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 04a56500cf89..763e0326b9e0 100644 --- a/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go @@ -42,14 +42,14 @@ func TestQueryResolvedTimestamp(t *testing.T) { return hlc.Timestamp{WallTime: ts} } writeValue := func(k string, ts int64) { - require.NoError(t, storage.MVCCDelete(ctx, db, nil, roachpb.Key(k), makeTS(ts), nil)) + require.NoError(t, storage.MVCCDelete(ctx, db, nil, roachpb.Key(k), makeTS(ts), hlc.ClockTimestamp{}, nil)) } writeIntent := func(k string, ts int64) { txn := roachpb.MakeTransaction("test", roachpb.Key(k), 0, makeTS(ts), 0, 1) - require.NoError(t, storage.MVCCDelete(ctx, db, nil, roachpb.Key(k), makeTS(ts), &txn)) + require.NoError(t, storage.MVCCDelete(ctx, db, nil, roachpb.Key(k), makeTS(ts), hlc.ClockTimestamp{}, &txn)) } writeInline := func(k string) { - require.NoError(t, storage.MVCCDelete(ctx, db, nil, roachpb.Key(k), hlc.Timestamp{}, nil)) + require.NoError(t, storage.MVCCDelete(ctx, db, nil, roachpb.Key(k), hlc.Timestamp{}, hlc.ClockTimestamp{}, nil)) } // Setup: (with separated intents the actual key layout in the store is not what is listed below.) diff --git a/pkg/kv/kvserver/batcheval/cmd_recover_txn.go b/pkg/kv/kvserver/batcheval/cmd_recover_txn.go index dd7111d47918..15b980715090 100644 --- a/pkg/kv/kvserver/batcheval/cmd_recover_txn.go +++ b/pkg/kv/kvserver/batcheval/cmd_recover_txn.go @@ -216,7 +216,7 @@ func RecoverTxn( reply.RecoveredTxn.Status = roachpb.ABORTED } txnRecord := reply.RecoveredTxn.AsRecord() - if err := storage.MVCCPutProto(ctx, readWriter, cArgs.Stats, key, hlc.Timestamp{}, nil, &txnRecord); err != nil { + if err := storage.MVCCPutProto(ctx, readWriter, cArgs.Stats, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord); 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 97964b5b0729..e955b83a83fb 100644 --- a/pkg/kv/kvserver/batcheval/cmd_recover_txn_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_recover_txn_test.go @@ -48,7 +48,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{}, nil, &txnRecord); err != nil { + if err := storage.MVCCPutProto(ctx, db, nil, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord); err != nil { t.Fatal(err) } @@ -224,7 +224,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{}, nil, &txnRecord); err != nil { + if err := storage.MVCCPutProto(ctx, db, nil, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord); 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 43c0ca0f3984..c40cebf6a81c 100644 --- a/pkg/kv/kvserver/batcheval/cmd_refresh_range_bench_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_refresh_range_bench_test.go @@ -258,7 +258,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, value, nil); err != nil { + if err := storage.MVCCPut(ctx, batch, nil /* ms */, key, ts, hlc.ClockTimestamp{}, value, nil); 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 f65d544f5317..cd5f1523b6e8 100644 --- a/pkg/kv/kvserver/batcheval/cmd_refresh_range_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_refresh_range_test.go @@ -70,10 +70,10 @@ func TestRefreshRangeTimeBoundIterator(t *testing.T) { }, ReadTimestamp: ts1, } - if err := storage.MVCCPut(ctx, db, nil, k, txn.ReadTimestamp, v, txn); err != nil { + if err := storage.MVCCPut(ctx, db, nil, k, txn.ReadTimestamp, hlc.ClockTimestamp{}, v, txn); err != nil { t.Fatal(err) } - if err := storage.MVCCPut(ctx, db, nil, roachpb.Key("unused1"), ts4, v, nil); err != nil { + if err := storage.MVCCPut(ctx, db, nil, roachpb.Key("unused1"), ts4, hlc.ClockTimestamp{}, v, nil); err != nil { t.Fatal(err) } if err := db.Flush(); err != nil { @@ -92,7 +92,7 @@ func TestRefreshRangeTimeBoundIterator(t *testing.T) { if _, err := storage.MVCCResolveWriteIntent(ctx, db, nil, intent); err != nil { t.Fatal(err) } - if err := storage.MVCCPut(ctx, db, nil, roachpb.Key("unused2"), ts1, v, nil); err != nil { + if err := storage.MVCCPut(ctx, db, nil, roachpb.Key("unused2"), ts1, hlc.ClockTimestamp{}, v, nil); err != nil { t.Fatal(err) } if err := db.Flush(); err != nil { @@ -181,7 +181,7 @@ func TestRefreshRangeError(t *testing.T) { }, ReadTimestamp: ts2, } - if err := storage.MVCCPut(ctx, db, nil, k, txn.ReadTimestamp, v, txn); err != nil { + if err := storage.MVCCPut(ctx, db, nil, k, txn.ReadTimestamp, hlc.ClockTimestamp{}, v, txn); err != nil { t.Fatal(err) } @@ -246,7 +246,7 @@ func TestRefreshRangeTimestampBounds(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, v, nil)) + require.NoError(t, storage.MVCCPut(ctx, db, nil, k, ts2, hlc.ClockTimestamp{}, v, nil)) for _, tc := range []struct { from, to hlc.Timestamp diff --git a/pkg/kv/kvserver/batcheval/cmd_refresh_test.go b/pkg/kv/kvserver/batcheval/cmd_refresh_test.go index 30a7bb8566b7..56ee5ee25394 100644 --- a/pkg/kv/kvserver/batcheval/cmd_refresh_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_refresh_test.go @@ -59,7 +59,7 @@ func TestRefreshError(t *testing.T) { }, ReadTimestamp: ts2, } - if err := storage.MVCCPut(ctx, db, nil, k, txn.ReadTimestamp, v, txn); err != nil { + if err := storage.MVCCPut(ctx, db, nil, k, txn.ReadTimestamp, hlc.ClockTimestamp{}, v, txn); err != nil { t.Fatal(err) } @@ -120,7 +120,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, v, nil)) + require.NoError(t, storage.MVCCPut(ctx, db, nil, k, ts2, hlc.ClockTimestamp{}, v, nil)) 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 ffc9023e4a2f..94d4e5ff6a98 100644 --- a/pkg/kv/kvserver/batcheval/cmd_resolve_intent_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_resolve_intent_test.go @@ -167,13 +167,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, v, &txn); err != nil { + if err := storage.MVCCPut(ctx, batch, nil, k, ts, hlc.ClockTimestamp{}, v, &txn); err != nil { t.Fatal(err) } // Write another value. v.SetString("b") txn.Sequence = 1 - if err := storage.MVCCPut(ctx, batch, nil, k, ts, v, &txn); err != nil { + if err := storage.MVCCPut(ctx, batch, nil, k, ts, hlc.ClockTimestamp{}, v, &txn); err != nil { t.Fatal(err) } if err := batch.Commit(true); err != nil { diff --git a/pkg/kv/kvserver/batcheval/cmd_revert_range_test.go b/pkg/kv/kvserver/batcheval/cmd_revert_range_test.go index 8ba29e967fbb..5a2915488ddb 100644 --- a/pkg/kv/kvserver/batcheval/cmd_revert_range_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_revert_range_test.go @@ -95,7 +95,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), value, nil); err != nil { + if err := storage.MVCCPut(ctx, eng, &stats, key, baseTime.Add(int64(i%10), 0), hlc.ClockTimestamp{}, value, nil); err != nil { t.Fatal(err) } } @@ -109,7 +109,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), value, nil); err != nil { + if err := storage.MVCCPut(ctx, eng, &stats, key, tsA.Add(int64(i%5), 1), hlc.ClockTimestamp{}, value, nil); err != nil { t.Fatal(err) } } @@ -122,7 +122,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)), value, nil); err != nil { + if err := storage.MVCCPut(ctx, eng, &stats, key, tsB.Add(1, int32(i%5)), hlc.ClockTimestamp{}, value, nil); err != nil { t.Fatal(err) } } @@ -195,7 +195,7 @@ func TestCmdRevertRange(t *testing.T) { txn := roachpb.MakeTransaction("test", nil, roachpb.NormalUserPriority, tsC, 1, 1) if err := storage.MVCCPut( - ctx, eng, &stats, []byte("0012"), tsC, roachpb.MakeValueFromBytes([]byte("i")), &txn, + ctx, eng, &stats, []byte("0012"), tsC, hlc.ClockTimestamp{}, roachpb.MakeValueFromBytes([]byte("i")), &txn, ); err != nil { t.Fatal(err) } @@ -206,7 +206,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)), value, nil); err != nil { + if err := storage.MVCCPut(ctx, eng, &stats, key, tsC.Add(10, int32(i%5)), hlc.ClockTimestamp{}, value, nil); 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 0d02c88315ba..640b372d3bfb 100644 --- a/pkg/kv/kvserver/batcheval/cmd_scan_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_scan_test.go @@ -98,7 +98,7 @@ func testScanReverseScanInner( // Write to k1 and k2. for _, k := range []roachpb.Key{k1, k2} { - err := storage.MVCCPut(ctx, eng, nil, k, ts, roachpb.MakeValueFromString("value-"+string(k)), nil) + err := storage.MVCCPut(ctx, eng, nil, k, ts, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("value-"+string(k)), nil) require.NoError(t, err) } @@ -199,7 +199,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, roachpb.MakeValueFromString("value"), nil) + err := storage.MVCCPut(ctx, eng, nil, key, ts, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("value"), nil) 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 8bc4e3dcc15b..481d9af8c716 100644 --- a/pkg/kv/kvserver/batcheval/cmd_truncate_log_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_truncate_log_test.go @@ -31,7 +31,7 @@ func putTruncatedState( key := keys.RaftTruncatedStateKey(rangeID) if err := storage.MVCCPutProto( context.Background(), eng, nil, key, - hlc.Timestamp{}, nil /* txn */, &truncState, + hlc.Timestamp{}, hlc.ClockTimestamp{}, nil /* txn */, &truncState, ); err != nil { t.Fatal(err) } diff --git a/pkg/kv/kvserver/batcheval/intent_test.go b/pkg/kv/kvserver/batcheval/intent_test.go index f287e11da403..16d997395b7a 100644 --- a/pkg/kv/kvserver/batcheval/intent_test.go +++ b/pkg/kv/kvserver/batcheval/intent_test.go @@ -129,9 +129,9 @@ func TestCollectIntentsUsesSameIterator(t *testing.T) { txn := roachpb.MakeTransaction("test", key, roachpb.NormalUserPriority, ts, 0, 1) var err error if delete { - err = storage.MVCCDelete(ctx, db, nil, key, ts, &txn) + err = storage.MVCCDelete(ctx, db, nil, key, ts, hlc.ClockTimestamp{}, &txn) } else { - err = storage.MVCCPut(ctx, db, nil, key, ts, val, &txn) + err = storage.MVCCPut(ctx, db, nil, key, ts, hlc.ClockTimestamp{}, val, &txn) } require.NoError(t, err) diff --git a/pkg/kv/kvserver/batcheval/transaction_test.go b/pkg/kv/kvserver/batcheval/transaction_test.go index b7481b86738d..2b63eab6f401 100644 --- a/pkg/kv/kvserver/batcheval/transaction_test.go +++ b/pkg/kv/kvserver/batcheval/transaction_test.go @@ -84,7 +84,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, val, &txn) + return storage.MVCCPut(ctx, b, ms, intentKey, txn.ReadTimestamp, hlc.ClockTimestamp{}, val, &txn) } 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 d4a6b0d0e10a..36c1e2bd60c1 100644 --- a/pkg/kv/kvserver/client_merge_test.go +++ b/pkg/kv/kvserver/client_merge_test.go @@ -3842,7 +3842,9 @@ func TestStoreRangeMergeRaftSnapshot(t *testing.T) { } tombstoneKey := keys.RangeTombstoneKey(rangeID) tombstoneValue := &roachpb.RangeTombstone{NextReplicaID: math.MaxInt32} - if err := storage.MVCCBlindPutProto(context.Background(), &sst, nil, tombstoneKey, hlc.Timestamp{}, tombstoneValue, nil); err != nil { + if err := storage.MVCCBlindPutProto( + context.Background(), &sst, nil, tombstoneKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, tombstoneValue, nil, + ); err != nil { return err } err := sst.Finish() diff --git a/pkg/kv/kvserver/consistency_queue_test.go b/pkg/kv/kvserver/consistency_queue_test.go index eb370481f5b8..951f071e7aee 100644 --- a/pkg/kv/kvserver/consistency_queue_test.go +++ b/pkg/kv/kvserver/consistency_queue_test.go @@ -416,7 +416,7 @@ func TestCheckConsistencyInconsistent(t *testing.T) { val.SetInt(42) diffTimestamp = ts.Clock().Now() if err := storage.MVCCPut( - context.Background(), store1.Engine(), nil, diffKey, diffTimestamp, val, nil, + context.Background(), store1.Engine(), nil, diffKey, diffTimestamp, hlc.ClockTimestamp{}, val, nil, ); err != nil { t.Fatal(err) } diff --git a/pkg/kv/kvserver/gc/data_distribution_test.go b/pkg/kv/kvserver/gc/data_distribution_test.go index 097761727e06..f032b7eb120d 100644 --- a/pkg/kv/kvserver/gc/data_distribution_test.go +++ b/pkg/kv/kvserver/gc/data_distribution_test.go @@ -64,7 +64,7 @@ func (ds dataDistribution) setupTest( txn.WriteTimestamp = ts } err := storage.MVCCPut(ctx, eng, &ms, kv.Key.Key, ts, - roachpb.Value{RawBytes: kv.Value}, txn) + hlc.ClockTimestamp{}, roachpb.Value{RawBytes: kv.Value}, txn) require.NoError(t, err) } if !kv.Key.Timestamp.Less(maxTs) { diff --git a/pkg/kv/kvserver/gc/gc_test.go b/pkg/kv/kvserver/gc/gc_test.go index 4e216b0c8d1e..4cb02651962d 100644 --- a/pkg/kv/kvserver/gc/gc_test.go +++ b/pkg/kv/kvserver/gc/gc_test.go @@ -120,7 +120,7 @@ func TestIntentAgeThresholdSetting(t *testing.T) { WallTime: intentTs.Nanoseconds(), } txn := roachpb.MakeTransaction("txn", key, roachpb.NormalUserPriority, intentHlc, 1000, 0) - require.NoError(t, storage.MVCCPut(ctx, eng, nil, key, intentHlc, value, &txn)) + require.NoError(t, storage.MVCCPut(ctx, eng, nil, key, intentHlc, hlc.ClockTimestamp{}, value, &txn)) require.NoError(t, eng.Flush()) // Prepare test fixtures for GC run. @@ -172,7 +172,7 @@ func TestIntentCleanupBatching(t *testing.T) { txn := roachpb.MakeTransaction("txn", key, roachpb.NormalUserPriority, intentHlc, 1000, 0) for _, suffix := range objectKeys { key := []byte{prefix, suffix} - require.NoError(t, storage.MVCCPut(ctx, eng, nil, key, intentHlc, value, &txn)) + require.NoError(t, storage.MVCCPut(ctx, eng, nil, key, intentHlc, hlc.ClockTimestamp{}, value, &txn)) } require.NoError(t, eng.Flush()) } diff --git a/pkg/kv/kvserver/loqrecovery/apply.go b/pkg/kv/kvserver/loqrecovery/apply.go index b5eb2310ebe5..b76450249dc0 100644 --- a/pkg/kv/kvserver/loqrecovery/apply.go +++ b/pkg/kv/kvserver/loqrecovery/apply.go @@ -257,7 +257,7 @@ func applyReplicaUpdate( // A crude form of the intent resolution process: abort the // transaction by deleting its record. txnKey := keys.TransactionKey(intent.Txn.Key, intent.Txn.ID) - if err := storage.MVCCDelete(ctx, readWriter, &ms, txnKey, hlc.Timestamp{}, nil); err != nil { + if err := storage.MVCCDelete(ctx, readWriter, &ms, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil); err != nil { return PrepareReplicaReport{}, err } update := roachpb.LockUpdate{ @@ -285,7 +285,8 @@ func applyReplicaUpdate( if err := storage.MVCCPutProto( ctx, readWriter, &ms, key, clock.Now(), - nil /* txn */, &newDesc); err != nil { + hlc.ClockTimestamp{}, nil /* txn */, &newDesc, + ); err != nil { return PrepareReplicaReport{}, err } report.Descriptor = newDesc diff --git a/pkg/kv/kvserver/loqrecovery/recovery_env_test.go b/pkg/kv/kvserver/loqrecovery/recovery_env_test.go index 06f42716a8fc..5a7b8a8492fe 100644 --- a/pkg/kv/kvserver/loqrecovery/recovery_env_test.go +++ b/pkg/kv/kvserver/loqrecovery/recovery_env_test.go @@ -206,8 +206,9 @@ func (e *quorumRecoveryEnv) handleReplicationData(t *testing.T, d datadriven.Tes buildReplicaDescriptorFromTestData(t, replica) eng := e.getOrCreateStore(ctx, t, replica.StoreID, replica.NodeID) - if err = storage.MVCCPutProto(ctx, eng, nil, key, clock.Now(), nil, /* txn */ - &desc); err != nil { + if err = storage.MVCCPutProto( + ctx, eng, nil, key, clock.Now(), hlc.ClockTimestamp{}, nil /* txn */, &desc, + ); err != nil { t.Fatalf("failed to write range descriptor into store: %v", err) } @@ -426,8 +427,8 @@ func (e *quorumRecoveryEnv) getOrCreateStore( StoreID: storeID, } if err = storage.MVCCPutProto( - context.Background(), eng, nil, keys.StoreIdentKey(), hlc.Timestamp{}, nil, - &sIdent); err != nil { + context.Background(), eng, nil, keys.StoreIdentKey(), hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &sIdent, + ); err != nil { t.Fatalf("failed to populate test store ident: %v", err) } wrapped.engine = eng diff --git a/pkg/kv/kvserver/mvcc_gc_queue_test.go b/pkg/kv/kvserver/mvcc_gc_queue_test.go index 9ae102908a26..b44d3c031a52 100644 --- a/pkg/kv/kvserver/mvcc_gc_queue_test.go +++ b/pkg/kv/kvserver/mvcc_gc_queue_test.go @@ -282,7 +282,7 @@ 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, value, txn); err != nil { + if err := storage.MVCCPut(ctx, eng, &eachMS, key, ts, hlc.ClockTimestamp{}, value, txn); err != nil { t.Fatal(err) } for i := 1; i < numOps; i++ { @@ -311,7 +311,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, value, nil /* txn */); err != nil { + if err := storage.MVCCPut(ctx, eng, ms, key, now, hlc.ClockTimestamp{}, value, nil); err != nil { t.Fatal(err) } if len(firstSl) < cacheFirstLen { @@ -855,7 +855,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{}, nil, txn); err != nil { + if err := storage.MVCCPutProto(ctx, tc.engine, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, txn); err != nil { t.Fatal(err) } } @@ -1057,7 +1057,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{}, nil, &ts); err != nil { + if err := storage.MVCCPutProto(ctx, tc.engine, nil, lpv.key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &ts); 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 7d13f32a92d5..428a6ef0dd16 100644 --- a/pkg/kv/kvserver/rangefeed/catchup_scan_bench_test.go +++ b/pkg/kv/kvserver/rangefeed/catchup_scan_bench_test.go @@ -262,7 +262,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, value, nil); err != nil { + if err := storage.MVCCPut(ctx, batch, nil /* ms */, key, ts, hlc.ClockTimestamp{}, value, nil); 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 62bc973d3c0b..184b509aa6ee 100644 --- a/pkg/kv/kvserver/rangefeed/catchup_scan_test.go +++ b/pkg/kv/kvserver/rangefeed/catchup_scan_test.go @@ -83,12 +83,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, v, nil); err != nil { + if err := storage.MVCCPut(ctx, eng, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { t.Fatal(err) } } // Put with an intent. - if err := storage.MVCCPut(ctx, eng, nil, kv1_4_4.Key.Key, txn.ReadTimestamp, val, &txn); err != nil { + if err := storage.MVCCPut(ctx, eng, nil, kv1_4_4.Key.Key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val, &txn); err != nil { t.Fatal(err) } testutils.RunTrueAndFalse(t, "useTBI", func(t *testing.T, useTBI bool) { diff --git a/pkg/kv/kvserver/rangefeed/task_test.go b/pkg/kv/kvserver/rangefeed/task_test.go index 87b695a7a228..9c87c4f93e40 100644 --- a/pkg/kv/kvserver/rangefeed/task_test.go +++ b/pkg/kv/kvserver/rangefeed/task_test.go @@ -266,7 +266,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, roachpb.Value{RawBytes: kv.Value}, op.txn) + err := storage.MVCCPut(ctx, engine, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, roachpb.Value{RawBytes: kv.Value}, 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 8ae5efa1001d..eaedadbafaaa 100644 --- a/pkg/kv/kvserver/rditer/replica_data_iter_test.go +++ b/pkg/kv/kvserver/rditer/replica_data_iter_test.go @@ -104,7 +104,7 @@ func createRangeData( allKeys := []storage.MVCCKey{} for _, keyTS := range keyTSs { - if err := storage.MVCCPut(context.Background(), eng, nil, keyTS.key, keyTS.ts, roachpb.MakeValueFromString("value"), nil); err != nil { + if err := storage.MVCCPut(context.Background(), eng, nil, keyTS.key, keyTS.ts, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("value"), nil); err != nil { t.Fatal(err) } allKeys = append(allKeys, storage.MVCCKey{Key: keyTS.key, Timestamp: keyTS.ts}) diff --git a/pkg/kv/kvserver/readsummary/persist.go b/pkg/kv/kvserver/readsummary/persist.go index 9c03878591ac..bde7509f6e33 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{}, nil, sum) + return storage.MVCCPutProto(ctx, readWriter, ms, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, sum) } diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 49edaace9871..94224e199ce5 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -1174,7 +1174,8 @@ 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.Engine(), nil, key, hlc.Timestamp{}, nil, ×tamp) + return storage.MVCCPutProto( + ctx, r.store.Engine(), nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, ×tamp) } // getQueueLastProcessed returns the last processed timestamp for the diff --git a/pkg/kv/kvserver/replica_destroy.go b/pkg/kv/kvserver/replica_destroy.go index a6a8ed9fd58f..ba9518b08d3f 100644 --- a/pkg/kv/kvserver/replica_destroy.go +++ b/pkg/kv/kvserver/replica_destroy.go @@ -249,5 +249,5 @@ func writeTombstoneKey( } // "Blind" because ms == nil and timestamp.IsEmpty(). return storage.MVCCBlindPutProto(ctx, writer, nil, tombstoneKey, - hlc.Timestamp{}, tombstone, nil) + hlc.Timestamp{}, hlc.ClockTimestamp{}, tombstone, nil) } diff --git a/pkg/kv/kvserver/replica_evaluate_test.go b/pkg/kv/kvserver/replica_evaluate_test.go index fb1ce01672f9..dd94e385702f 100644 --- a/pkg/kv/kvserver/replica_evaluate_test.go +++ b/pkg/kv/kvserver/replica_evaluate_test.go @@ -712,7 +712,7 @@ 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, + context.Background(), eng, nil /* ms */, roachpb.Key(k), ts, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("value-"+k), txn)) } } diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index 8754d90acdca..2a171cc88508 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -2116,8 +2116,14 @@ func handleTruncatedStateBelowRaftPreApply( // The suggested truncated state moves us forward; apply it and tell // the caller as much. if err := storage.MVCCPutProto( - ctx, readWriter, nil /* ms */, prefixBuf.RaftTruncatedStateKey(), - hlc.Timestamp{}, nil /* txn */, suggestedTruncatedState, + ctx, + readWriter, + nil, /* ms */ + prefixBuf.RaftTruncatedStateKey(), + hlc.Timestamp{}, + hlc.ClockTimestamp{}, + nil, /* txn */ + suggestedTruncatedState, ); err != nil { return false, errors.Wrap(err, "unable to write RaftTruncatedState") } diff --git a/pkg/kv/kvserver/replica_raftstorage.go b/pkg/kv/kvserver/replica_raftstorage.go index c27522be587c..5da1475c1448 100644 --- a/pkg/kv/kvserver/replica_raftstorage.go +++ b/pkg/kv/kvserver/replica_raftstorage.go @@ -661,7 +661,7 @@ func (r *Replica) append( value.InitChecksum(key) var err error if ent.Index > prevLastIndex { - err = storage.MVCCBlindPut(ctx, writer, &diff, key, hlc.Timestamp{}, value, nil /* txn */) + err = storage.MVCCBlindPut(ctx, writer, &diff, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, value, nil /* txn */) } else { // We type assert `writer` to also be an engine.ReadWriter only in // the case where we're replacing existing entries. @@ -669,7 +669,7 @@ func (r *Replica) append( if !ok { panic("expected writer to be a engine.ReadWriter when overwriting log entries") } - err = storage.MVCCPut(ctx, eng, &diff, key, hlc.Timestamp{}, value, nil /* txn */) + err = storage.MVCCPut(ctx, eng, &diff, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, value, nil /* txn */) } if err != nil { return 0, 0, 0, err @@ -690,7 +690,7 @@ func (r *Replica) append( // 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, eng, &diff, r.raftMu.stateLoader.RaftLogKey(i), - hlc.Timestamp{}, nil /* txn */) + hlc.Timestamp{}, hlc.ClockTimestamp{}, nil) if err != nil { return 0, 0, 0, err } diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index a30aac136395..90acce579380 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -1191,7 +1191,7 @@ func TestReplicaGossipConfigsOnLease(t *testing.T) { key := keys.SystemSQLCodec.TablePrefix(keys.MaxSystemConfigDescID) var val roachpb.Value val.SetInt(42) - if err := storage.MVCCPut(context.Background(), tc.engine, nil, key, hlc.Timestamp{}, val, nil); err != nil { + if err := storage.MVCCPut(context.Background(), tc.engine, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, val, nil); err != nil { t.Fatal(err) } @@ -1989,7 +1989,7 @@ func TestOptimizePuts(t *testing.T) { for i, c := range testCases { if c.exKey != nil { if err := storage.MVCCPut(context.Background(), tc.engine, nil, c.exKey, - hlc.Timestamp{}, roachpb.MakeValueFromString("foo"), nil); err != nil { + hlc.Timestamp{}, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("foo"), nil); err != nil { t.Fatal(err) } } @@ -3370,7 +3370,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{}, roachpb.MakeValueFromString("never read in this test"), nil) + err := storage.MVCCPut(ctx, tc.engine, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("never read in this test"), nil) if err != nil { t.Fatal(err) } @@ -4452,7 +4452,7 @@ func TestEndTxnWithErrors(t *testing.T) { existTxnRecord := existTxn.AsRecord() txnKey := keys.TransactionKey(test.key, txn.ID) if err := storage.MVCCPutProto( - ctx, tc.repl.store.Engine(), nil, txnKey, hlc.Timestamp{}, nil, &existTxnRecord, + ctx, tc.repl.store.Engine(), nil, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &existTxnRecord, ); err != nil { t.Fatal(err) } @@ -4495,7 +4495,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.Engine(), nil, txnKey, hlc.Timestamp{}, nil, &existTxnRec) + err := storage.MVCCPutProto(ctx, tc.repl.store.Engine(), nil, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &existTxnRec) require.NoError(t, err) // End the transaction, verify expected error, shouldn't deadlock. @@ -7040,7 +7040,7 @@ func TestReplicaLoadSystemConfigSpanIntent(t *testing.T) { v := roachpb.MakeValueFromString("foo") testutils.SucceedsSoon(t, func() error { if err := storage.MVCCPut(ctx, repl.store.Engine(), &enginepb.MVCCStats{}, - keys.SystemConfigSpan.Key, repl.store.Clock().Now(), v, nil); err != nil { + keys.SystemConfigSpan.Key, repl.store.Clock().Now(), hlc.ClockTimestamp{}, v, nil); err != nil { return err } @@ -11080,7 +11080,7 @@ func TestReplicaPushed1PC(t *testing.T) { // Write a value outside the transaction. tc.manualClock.Increment(10) ts2 := tc.Clock().Now() - if err := storage.MVCCPut(ctx, tc.engine, nil, k, ts2, roachpb.MakeValueFromString("one"), nil); err != nil { + if err := storage.MVCCPut(ctx, tc.engine, nil, k, ts2, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("one"), nil); err != nil { t.Fatalf("writing interfering value: %+v", err) } @@ -13482,7 +13482,7 @@ func setMockPutWithEstimates(containsEstimatesDelta int64) (undo func()) { ms := cArgs.Stats ms.ContainsEstimates += containsEstimatesDelta ts := cArgs.Header.Timestamp - return result.Result{}, storage.MVCCBlindPut(ctx, readWriter, ms, args.Key, ts, args.Value, cArgs.Header.Txn) + return result.Result{}, storage.MVCCBlindPut(ctx, readWriter, ms, args.Key, ts, hlc.ClockTimestamp{}, args.Value, cArgs.Header.Txn) } batcheval.UnregisterCommand(roachpb.Put) diff --git a/pkg/kv/kvserver/stateloader/stateloader.go b/pkg/kv/kvserver/stateloader/stateloader.go index 9769a25a4d90..ecabcd9fe8bd 100644 --- a/pkg/kv/kvserver/stateloader/stateloader.go +++ b/pkg/kv/kvserver/stateloader/stateloader.go @@ -158,7 +158,7 @@ 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{}, nil, &lease) + hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &lease) } // LoadRangeAppliedState loads the Range applied state. @@ -213,7 +213,8 @@ func (rsl StateLoader) SetRangeAppliedState( // The RangeAppliedStateKey is not included in stats. This is also reflected // in C.MVCCComputeStats and ComputeStatsForRange. ms := (*enginepb.MVCCStats)(nil) - return storage.MVCCPutProto(ctx, readWriter, ms, rsl.RangeAppliedStateKey(), hlc.Timestamp{}, nil, &as) + return storage.MVCCPutProto(ctx, readWriter, ms, rsl.RangeAppliedStateKey(), + hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &as) } // SetMVCCStats overwrites the MVCC stats. This needs to perform a read on the @@ -264,8 +265,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{}, nil, threshold) + return storage.MVCCPutProto(ctx, readWriter, ms, rsl.RangeGCThresholdKey(), + hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, threshold) } // LoadVersion loads the replica version. @@ -285,8 +286,8 @@ func (rsl StateLoader) SetVersion( ms *enginepb.MVCCStats, version *roachpb.Version, ) error { - return storage.MVCCPutProto(ctx, readWriter, ms, - rsl.RangeVersionKey(), hlc.Timestamp{}, nil, version) + return storage.MVCCPutProto(ctx, readWriter, ms, rsl.RangeVersionKey(), + hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, version) } // The rest is not technically part of ReplicaState. @@ -347,7 +348,8 @@ func (rsl StateLoader) SetRaftTruncatedState( writer, nil, /* ms */ rsl.RaftTruncatedStateKey(), - hlc.Timestamp{}, /* timestamp */ + hlc.Timestamp{}, /* timestamp */ + hlc.ClockTimestamp{}, /* localTimestamp */ truncState, nil, /* txn */ ) @@ -377,7 +379,8 @@ func (rsl StateLoader) SetHardState( writer, nil, /* ms */ rsl.RaftHardStateKey(), - hlc.Timestamp{}, /* timestamp */ + hlc.Timestamp{}, /* timestamp */ + hlc.ClockTimestamp{}, /* localTimestamp */ &hs, nil, /* txn */ ) @@ -451,7 +454,8 @@ func (rsl StateLoader) SetRaftReplicaID( writer, nil, /* ms */ rsl.RaftReplicaIDKey(), - hlc.Timestamp{}, /* timestamp */ + hlc.Timestamp{}, /* timestamp */ + hlc.ClockTimestamp{}, /* localTimestamp */ &rid, nil, /* txn */ ) diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index caab991c5a05..2bc90dd9786d 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -2662,6 +2662,7 @@ func (s *Store) WriteLastUpTimestamp(ctx context.Context, time hlc.Timestamp) er nil, keys.StoreLastUpKey(), hlc.Timestamp{}, + hlc.ClockTimestamp{}, nil, &time, ) @@ -2697,6 +2698,7 @@ func (s *Store) WriteHLCUpperBound(ctx context.Context, time int64) error { nil, keys.StoreHLCUpperBoundKey(), hlc.Timestamp{}, + hlc.ClockTimestamp{}, nil, &ts, ); err != nil { @@ -3597,7 +3599,16 @@ func (s *storeForTruncatorImpl) getEngine() storage.Engine { func WriteClusterVersion( ctx context.Context, eng storage.Engine, cv clusterversion.ClusterVersion, ) error { - err := storage.MVCCPutProto(ctx, eng, nil, keys.StoreClusterVersionKey(), hlc.Timestamp{}, nil, &cv) + err := storage.MVCCPutProto( + ctx, + eng, + nil, + keys.StoreClusterVersionKey(), + hlc.Timestamp{}, + hlc.ClockTimestamp{}, + nil, + &cv, + ) if err != nil { return err } diff --git a/pkg/kv/kvserver/store_init.go b/pkg/kv/kvserver/store_init.go index 321c8fc21ac1..4cbd9f971021 100644 --- a/pkg/kv/kvserver/store_init.go +++ b/pkg/kv/kvserver/store_init.go @@ -58,6 +58,7 @@ func InitEngine(ctx context.Context, eng storage.Engine, ident roachpb.StoreIden nil, keys.StoreIdentKey(), hlc.Timestamp{}, + hlc.ClockTimestamp{}, nil, &ident, ); err != nil { @@ -203,7 +204,7 @@ func WriteInitialClusterData( // Range descriptor. if err := storage.MVCCPutProto( ctx, batch, nil /* ms */, keys.RangeDescriptorKey(desc.StartKey), - now, nil /* txn */, desc, + now, hlc.ClockTimestamp{}, nil /* txn */, desc, ); err != nil { return err } @@ -211,14 +212,15 @@ func WriteInitialClusterData( // Replica GC timestamp. if err := storage.MVCCPutProto( ctx, batch, nil /* ms */, keys.RangeLastReplicaGCTimestampKey(desc.RangeID), - hlc.Timestamp{}, nil /* txn */, &now, + hlc.Timestamp{}, hlc.ClockTimestamp{}, nil /* txn */, &now, ); err != nil { return err } // Range addressing for meta2. meta2Key := keys.RangeMetaKey(endKey) - if err := storage.MVCCPutProto(ctx, batch, firstRangeMS, meta2Key.AsRawKey(), - now, nil /* txn */, desc, + if err := storage.MVCCPutProto( + ctx, batch, firstRangeMS, meta2Key.AsRawKey(), + now, hlc.ClockTimestamp{}, nil /* txn */, desc, ); err != nil { return err } @@ -228,7 +230,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, nil /* txn */, desc, + ctx, batch, nil /* ms */, meta1Key.AsRawKey(), now, hlc.ClockTimestamp{}, nil /* txn */, desc, ); err != nil { return err } @@ -239,7 +241,7 @@ func WriteInitialClusterData( // Initialize the checksums. kv.Value.InitChecksum(kv.Key) if err := storage.MVCCPut( - ctx, batch, nil /* ms */, kv.Key, now, kv.Value, nil, /* txn */ + ctx, batch, nil /* ms */, kv.Key, now, hlc.ClockTimestamp{}, kv.Value, nil, /* txn */ ); err != nil { return err } diff --git a/pkg/kv/kvserver/store_snapshot.go b/pkg/kv/kvserver/store_snapshot.go index 16206dcc9650..c383e9a98e23 100644 --- a/pkg/kv/kvserver/store_snapshot.go +++ b/pkg/kv/kvserver/store_snapshot.go @@ -1020,7 +1020,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, nil /* txn */, &desc, + ctx, eng, &ms, keys.RangeDescriptorKey(desc.StartKey), now, hlc.ClockTimestamp{}, nil /* txn */, &desc, ); err != nil { return err } diff --git a/pkg/kv/kvserver/store_test.go b/pkg/kv/kvserver/store_test.go index aa345f03be90..2cb131e961f6 100644 --- a/pkg/kv/kvserver/store_test.go +++ b/pkg/kv/kvserver/store_test.go @@ -337,6 +337,7 @@ func TestIterateIDPrefixKeys(t *testing.T) { nil, /* ms */ key, hlc.Timestamp{}, + hlc.ClockTimestamp{}, roachpb.MakeValueFromString("fake value for "+key.String()), nil, /* txn */ ); err != nil { @@ -370,7 +371,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{}, nil /* txn */, &tombstone, + ctx, eng, nil /* ms */, keys.RangeTombstoneKey(rangeID), hlc.Timestamp{}, hlc.ClockTimestamp{}, nil /* txn */, &tombstone, ); err != nil { t.Fatal(err) } diff --git a/pkg/kv/kvserver/stores.go b/pkg/kv/kvserver/stores.go index 9d4bf5f5c9e4..5a9414a36f79 100644 --- a/pkg/kv/kvserver/stores.go +++ b/pkg/kv/kvserver/stores.go @@ -282,7 +282,7 @@ func (ls *Stores) updateBootstrapInfoLocked(bi *gossip.BootstrapInfo) error { var err error ls.storeMap.Range(func(k int64, v unsafe.Pointer) bool { s := (*Store)(v) - err = storage.MVCCPutProto(ctx, s.engine, nil, keys.StoreGossipKey(), hlc.Timestamp{}, nil, bi) + err = storage.MVCCPutProto(ctx, s.engine, nil, keys.StoreGossipKey(), hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, bi) 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 c24e343626c2..4526725fb97e 100644 --- a/pkg/kv/kvserver/txn_wait_queue_test.go +++ b/pkg/kv/kvserver/txn_wait_queue_test.go @@ -39,7 +39,7 @@ import ( func writeTxnRecord(ctx context.Context, tc *testContext, txn *roachpb.Transaction) error { key := keys.TransactionKey(txn.Key, txn.ID) - return storage.MVCCPutProto(ctx, tc.store.Engine(), nil, key, hlc.Timestamp{}, nil, txn) + return storage.MVCCPutProto(ctx, tc.store.Engine(), nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, txn) } // createTxnForPushQueue creates a txn struct and writes a "fake" diff --git a/pkg/server/node_test.go b/pkg/server/node_test.go index 56e3d1d243ca..85d7c19b14b4 100644 --- a/pkg/server/node_test.go +++ b/pkg/server/node_test.go @@ -265,7 +265,7 @@ func TestCorruptedClusterID(t *testing.T) { StoreID: 1, } if err := storage.MVCCPutProto( - ctx, e, nil /* ms */, keys.StoreIdentKey(), hlc.Timestamp{}, nil /* txn */, &sIdent, + ctx, e, nil /* ms */, keys.StoreIdentKey(), hlc.Timestamp{}, hlc.ClockTimestamp{}, nil /* txn */, &sIdent, ); err != nil { t.Fatal(err) } diff --git a/pkg/server/node_tombstone_storage.go b/pkg/server/node_tombstone_storage.go index caa517d6a472..e009366f31a6 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 /* MVCCStats */, k, hlc.Timestamp{}, v, nil, /* txn */ + ctx, eng, nil /* ms */, k, hlc.Timestamp{}, hlc.ClockTimestamp{}, v, nil, /* txn */ ); err != nil { return err } diff --git a/pkg/server/server_test.go b/pkg/server/server_test.go index bfb3f875d3fa..520d26de582f 100644 --- a/pkg/server/server_test.go +++ b/pkg/server/server_test.go @@ -569,7 +569,8 @@ func TestClusterIDMismatch(t *testing.T) { StoreID: roachpb.StoreID(i + 1), } if err := storage.MVCCPutProto( - context.Background(), e, nil, keys.StoreIdentKey(), hlc.Timestamp{}, nil, &sIdent); err != nil { + context.Background(), e, nil, keys.StoreIdentKey(), hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &sIdent, + ); err != nil { t.Fatal(err) } @@ -1160,7 +1161,7 @@ func TestAssertEnginesEmpty(t *testing.T) { require.NoError(t, assertEnginesEmpty([]storage.Engine{eng})) require.NoError(t, storage.MVCCPutProto(ctx, eng, nil, keys.StoreClusterVersionKey(), - hlc.Timestamp{}, nil, &roachpb.Version{Major: 21, Minor: 1, Internal: 122})) + hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &roachpb.Version{Major: 21, Minor: 1, Internal: 122})) 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 7a0196572838..d4320c572cf6 100644 --- a/pkg/server/settings_cache.go +++ b/pkg/server/settings_cache.go @@ -100,7 +100,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{}, kv.Value, nil, + ctx, batch, nil, keys.StoreCachedSettingsKey(kv.Key), hlc.Timestamp{}, hlc.ClockTimestamp{}, kv.Value, nil, ); err != nil { return err } diff --git a/pkg/storage/bench_pebble_test.go b/pkg/storage/bench_pebble_test.go index 7a540849a7c2..34f294f02cbc 100644 --- a/pkg/storage/bench_pebble_test.go +++ b/pkg/storage/bench_pebble_test.go @@ -265,10 +265,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{}, value, nil /* txn */); err != nil { + if err := MVCCPut(ctx, db, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, value, nil); err != nil { b.Fatal(err) } - if err := MVCCDelete(ctx, db, nil, key, hlc.Timestamp{}, nil /* txn */); err != nil { + if err := MVCCDelete(ctx, db, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil); err != nil { b.Fatal(err) } } diff --git a/pkg/storage/bench_test.go b/pkg/storage/bench_test.go index f9b46600afd3..7c848a84dd8b 100644 --- a/pkg/storage/bench_test.go +++ b/pkg/storage/bench_test.go @@ -244,7 +244,7 @@ func setupKeysWithIntent( putTxn = &otherTxn } key := makeKey(nil, j) - require.NoError(b, MVCCPut(context.Background(), batch, nil, key, ts, value, putTxn)) + require.NoError(b, MVCCPut(context.Background(), batch, nil, key, ts, hlc.ClockTimestamp{}, value, putTxn)) } require.NoError(b, batch.Commit(true)) batch.Close() @@ -657,7 +657,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, value, nil); err != nil { + if err := MVCCPut(ctx, batch, nil, key, timestamp, hlc.ClockTimestamp{}, value, nil); err != nil { return nil, err } } @@ -755,7 +755,7 @@ func setupMVCCData( txn.ReadTimestamp = ts txn.WriteTimestamp = ts } - if err := MVCCPut(ctx, batch, nil /* ms */, key, ts, value, txn); err != nil { + if err := MVCCPut(ctx, batch, nil, key, ts, hlc.ClockTimestamp{}, value, txn); err != nil { b.Fatal(err) } } @@ -957,7 +957,7 @@ func runMVCCPut(ctx context.Context, b *testing.B, emk engineMaker, valueSize in 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, value, nil); err != nil { + if err := MVCCPut(ctx, eng, nil, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { b.Fatalf("failed put: %+v", err) } } @@ -979,7 +979,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, value, nil); err != nil { + if err := MVCCBlindPut(ctx, eng, nil, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { b.Fatalf("failed put: %+v", err) } } @@ -1003,7 +1003,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, value, nil); err != nil { + if err := MVCCPut(ctx, eng, nil, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { b.Fatalf("failed put: %+v", err) } } @@ -1015,7 +1015,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, value, expected, CPutFailIfMissing, nil); err != nil { + if err := MVCCConditionalPut(ctx, eng, nil, key, ts, hlc.ClockTimestamp{}, value, expected, CPutFailIfMissing, nil); err != nil { b.Fatalf("failed put: %+v", err) } } @@ -1037,7 +1037,9 @@ func runMVCCBlindConditionalPut(ctx context.Context, b *testing.B, emk engineMak 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 := MVCCBlindConditionalPut(ctx, eng, nil, key, ts, value, nil, CPutFailIfMissing, nil); err != nil { + if err := MVCCBlindConditionalPut( + ctx, eng, nil, key, ts, hlc.ClockTimestamp{}, value, nil, CPutFailIfMissing, nil, + ); err != nil { b.Fatalf("failed put: %+v", err) } } @@ -1059,7 +1061,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, value, false, nil); err != nil { + if err := MVCCInitPut(ctx, eng, nil, key, ts, hlc.ClockTimestamp{}, value, false, nil); err != nil { b.Fatalf("failed put: %+v", err) } } @@ -1081,7 +1083,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, value, false, nil); err != nil { + if err := MVCCBlindInitPut(ctx, eng, nil, key, ts, hlc.ClockTimestamp{}, value, false, nil); err != nil { b.Fatalf("failed put: %+v", err) } } @@ -1111,7 +1113,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, value, nil); err != nil { + if err := MVCCPut(ctx, batch, nil, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { b.Fatalf("failed put: %+v", err) } } @@ -1260,6 +1262,7 @@ func runMVCCDeleteRange(ctx context.Context, b *testing.B, emk engineMaker, valu roachpb.KeyMax, math.MaxInt64, hlc.MaxTimestamp, + hlc.ClockTimestamp{}, nil, false, ); err != nil { @@ -1416,7 +1419,7 @@ func runMVCCGarbageCollect( }) } for j := 0; j < opts.numVersions; j++ { - if err := MVCCPut(ctx, batch, nil /* ms */, key, ts.Add(0, int32(j)), val, nil); err != nil { + if err := MVCCPut(ctx, batch, nil, key, ts.Add(0, int32(j)), hlc.ClockTimestamp{}, val, nil); err != nil { b.Fatal(err) } } @@ -1472,7 +1475,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, value, nil); err != nil { + if err := MVCCBlindPut(ctx, batch, nil, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { b.Fatal(err) } } diff --git a/pkg/storage/engine_test.go b/pkg/storage/engine_test.go index b917225ec2af..cc8653983721 100644 --- a/pkg/storage/engine_test.go +++ b/pkg/storage/engine_test.go @@ -178,15 +178,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{}, - roachpb.MakeValueFromString("x"), nil); err != nil { + if err := MVCCPut(context.Background(), batch, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("x"), nil); 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{}, nil); err != nil { + if err := MVCCDelete(context.Background(), batch, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } @@ -1647,7 +1645,7 @@ func TestScanIntents(t *testing.T) { defer eng.Close() for _, key := range keys { - err := MVCCPut(ctx, eng, nil, key, txn1.ReadTimestamp, roachpb.Value{RawBytes: key}, txn1) + err := MVCCPut(ctx, eng, nil, key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, roachpb.Value{RawBytes: key}, txn1) require.NoError(t, err) } diff --git a/pkg/storage/metamorphic/operations.go b/pkg/storage/metamorphic/operations.go index 9e52e9b5aee4..6773db820d67 100644 --- a/pkg/storage/metamorphic/operations.go +++ b/pkg/storage/metamorphic/operations.go @@ -206,7 +206,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.WriteTimestamp, m.value, txn) + err := storage.MVCCPut(ctx, writer, nil, m.key, txn.WriteTimestamp, hlc.ClockTimestamp{}, m.value, txn) if err != nil { return fmt.Sprintf("error: %s", err) } @@ -229,7 +229,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.WriteTimestamp, m.value, m.expVal, true, txn) + err := storage.MVCCConditionalPut(ctx, writer, nil, m.key, + txn.WriteTimestamp, hlc.ClockTimestamp{}, m.value, m.expVal, true, txn) if err != nil { return fmt.Sprintf("error: %s", err) } @@ -251,7 +252,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.WriteTimestamp, m.value, false, txn) + err := storage.MVCCInitPut(ctx, writer, nil, m.key, txn.WriteTimestamp, hlc.ClockTimestamp{}, m.value, false, txn) if err != nil { return fmt.Sprintf("error: %s", err) } @@ -278,7 +279,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, txn, true) + keys, _, _, err := storage.MVCCDeleteRange(ctx, writer, nil, m.key, m.endKey, + 0, txn.WriteTimestamp, hlc.ClockTimestamp{}, txn, true) if err != nil { return fmt.Sprintf("error: %s", err) } @@ -331,7 +333,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.WriteTimestamp, txn) + err := storage.MVCCDelete(ctx, writer, nil, m.key, txn.WriteTimestamp, hlc.ClockTimestamp{}, txn) if err != nil { return fmt.Sprintf("error: %s", err) } diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 55f9fe3074a0..c7c5ac51fbf6 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -636,6 +636,7 @@ func MVCCPutProto( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, txn *roachpb.Transaction, msg protoutil.Message, ) error { @@ -644,7 +645,7 @@ func MVCCPutProto( return err } value.InitChecksum(key) - return MVCCPut(ctx, rw, ms, key, timestamp, value, txn) + return MVCCPut(ctx, rw, ms, key, timestamp, localTimestamp, value, txn) } // MVCCBlindPutProto sets the given key to the protobuf-serialized byte string @@ -656,6 +657,7 @@ func MVCCBlindPutProto( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, msg protoutil.Message, txn *roachpb.Transaction, ) error { @@ -664,7 +666,7 @@ func MVCCBlindPutProto( return err } value.InitChecksum(key) - return MVCCBlindPut(ctx, writer, ms, key, timestamp, value, txn) + return MVCCBlindPut(ctx, writer, ms, key, timestamp, localTimestamp, value, txn) } // MVCCGetOptions bundles options for the MVCCGet family of functions. @@ -987,6 +989,7 @@ func MVCCPut( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, value roachpb.Value, txn *roachpb.Transaction, ) error { @@ -998,7 +1001,7 @@ func MVCCPut( iter = rw.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{Prefix: true}) defer iter.Close() } - return mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, value, txn, nil /* valueFn */) + return mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, localTimestamp, value, txn, nil) } // MVCCBlindPut is a fast-path of MVCCPut. See the MVCCPut comments for details @@ -1017,10 +1020,11 @@ func MVCCBlindPut( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, value roachpb.Value, txn *roachpb.Transaction, ) error { - return mvccPutUsingIter(ctx, writer, nil, ms, key, timestamp, value, txn, nil /* valueFn */) + return mvccPutUsingIter(ctx, writer, nil, ms, key, timestamp, localTimestamp, value, txn, nil) } // MVCCDelete marks the key deleted so that it will not be returned in @@ -1035,12 +1039,13 @@ func MVCCDelete( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, txn *roachpb.Transaction, ) error { iter := newMVCCIterator(rw, timestamp.IsEmpty(), IterOptions{Prefix: true}) defer iter.Close() - return mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, noValue, txn, nil /* valueFn */) + return mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, localTimestamp, noValue, txn, nil) } var noValue = roachpb.Value{} @@ -1056,6 +1061,7 @@ func mvccPutUsingIter( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, value roachpb.Value, txn *roachpb.Transaction, valueFn func(optionalValue) ([]byte, error), @@ -1070,8 +1076,7 @@ func mvccPutUsingIter( buf := newPutBuffer() - err := mvccPutInternal(ctx, writer, iter, ms, key, timestamp, rawBytes, - txn, buf, valueFn) + err := mvccPutInternal(ctx, writer, iter, ms, key, timestamp, localTimestamp, rawBytes, txn, buf, valueFn) // Using defer would be more convenient, but it is measurably slower. buf.release() @@ -1262,6 +1267,15 @@ func replayTransactionalWrite( // read timestamp. (One could imagine instead requiring that the timestamp // 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 +// 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 +// timestamp is empty (hlc.ClockTimestamp{}), the value will not be +// assigned an explicit local timestamp. The effect of this is that +// readers treat the local timestamp as being equal to the version +// timestamp. func mvccPutInternal( ctx context.Context, writer Writer, @@ -1269,6 +1283,7 @@ func mvccPutInternal( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, value []byte, txn *roachpb.Transaction, buf *putBuffer, @@ -1656,6 +1671,7 @@ func MVCCIncrement( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, txn *roachpb.Transaction, inc int64, ) (int64, error) { @@ -1664,7 +1680,7 @@ func MVCCIncrement( var int64Val int64 var newInt64Val int64 - err := mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, noValue, txn, func(value optionalValue) ([]byte, error) { + valueFn := func(value optionalValue) ([]byte, error) { if value.IsPresent() { var err error if int64Val, err = value.GetInt(); err != nil { @@ -1688,7 +1704,8 @@ func MVCCIncrement( newValue.SetInt(newInt64Val) newValue.InitChecksum(key) return newValue.RawBytes, nil - }) + } + err := mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, localTimestamp, noValue, txn, valueFn) return newInt64Val, err } @@ -1726,6 +1743,7 @@ func MVCCConditionalPut( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, value roachpb.Value, expVal []byte, allowIfDoesNotExist CPutMissingBehavior, @@ -1734,7 +1752,8 @@ func MVCCConditionalPut( iter := newMVCCIterator(rw, timestamp.IsEmpty(), IterOptions{Prefix: true}) defer iter.Close() - return mvccConditionalPutUsingIter(ctx, rw, iter, ms, key, timestamp, value, expVal, allowIfDoesNotExist, txn) + return mvccConditionalPutUsingIter( + ctx, rw, iter, ms, key, timestamp, localTimestamp, value, expVal, allowIfDoesNotExist, txn) } // MVCCBlindConditionalPut is a fast-path of MVCCConditionalPut. See the @@ -1752,12 +1771,14 @@ func MVCCBlindConditionalPut( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, value roachpb.Value, expVal []byte, allowIfDoesNotExist CPutMissingBehavior, txn *roachpb.Transaction, ) error { - return mvccConditionalPutUsingIter(ctx, writer, nil, ms, key, timestamp, value, expVal, allowIfDoesNotExist, txn) + return mvccConditionalPutUsingIter( + ctx, writer, nil, ms, key, timestamp, localTimestamp, value, expVal, allowIfDoesNotExist, txn) } func mvccConditionalPutUsingIter( @@ -1767,27 +1788,27 @@ func mvccConditionalPutUsingIter( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, value roachpb.Value, expBytes []byte, allowNoExisting CPutMissingBehavior, txn *roachpb.Transaction, ) error { - return mvccPutUsingIter( - ctx, writer, iter, ms, key, timestamp, noValue, txn, - func(existVal optionalValue) ([]byte, error) { - if expValPresent, existValPresent := len(expBytes) != 0, existVal.IsPresent(); expValPresent && existValPresent { - if !bytes.Equal(expBytes, existVal.TagAndDataBytes()) { - return nil, &roachpb.ConditionFailedError{ - ActualValue: existVal.ToPointer(), - } - } - } else if expValPresent != existValPresent && (existValPresent || !bool(allowNoExisting)) { + valueFn := func(existVal optionalValue) ([]byte, error) { + if expValPresent, existValPresent := len(expBytes) != 0, existVal.IsPresent(); expValPresent && existValPresent { + if !bytes.Equal(expBytes, existVal.TagAndDataBytes()) { return nil, &roachpb.ConditionFailedError{ ActualValue: existVal.ToPointer(), } } - return value.RawBytes, nil - }) + } else if expValPresent != existValPresent && (existValPresent || !bool(allowNoExisting)) { + return nil, &roachpb.ConditionFailedError{ + ActualValue: existVal.ToPointer(), + } + } + return value.RawBytes, nil + } + return mvccPutUsingIter(ctx, writer, iter, ms, key, timestamp, localTimestamp, noValue, txn, valueFn) } // MVCCInitPut sets the value for a specified key if the key doesn't exist. It @@ -1805,13 +1826,14 @@ func MVCCInitPut( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, value roachpb.Value, failOnTombstones bool, txn *roachpb.Transaction, ) error { iter := newMVCCIterator(rw, timestamp.IsEmpty(), IterOptions{Prefix: true}) defer iter.Close() - return mvccInitPutUsingIter(ctx, rw, iter, ms, key, timestamp, value, failOnTombstones, txn) + return mvccInitPutUsingIter(ctx, rw, iter, ms, key, timestamp, localTimestamp, value, failOnTombstones, txn) } // MVCCBlindInitPut is a fast-path of MVCCInitPut. See the MVCCInitPut @@ -1828,11 +1850,13 @@ func MVCCBlindInitPut( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, value roachpb.Value, failOnTombstones bool, txn *roachpb.Transaction, ) error { - return mvccInitPutUsingIter(ctx, rw, nil, ms, key, timestamp, value, failOnTombstones, txn) + return mvccInitPutUsingIter( + ctx, rw, nil, ms, key, timestamp, localTimestamp, value, failOnTombstones, txn) } func mvccInitPutUsingIter( @@ -1842,27 +1866,27 @@ func mvccInitPutUsingIter( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, value roachpb.Value, failOnTombstones bool, txn *roachpb.Transaction, ) error { - return mvccPutUsingIter( - ctx, rw, iter, ms, key, timestamp, noValue, txn, - func(existVal optionalValue) ([]byte, error) { - if failOnTombstones && existVal.IsTombstone() { - // We found a tombstone and failOnTombstones is true: fail. - return nil, &roachpb.ConditionFailedError{ - ActualValue: existVal.ToPointer(), - } + valueFn := func(existVal optionalValue) ([]byte, error) { + if failOnTombstones && existVal.IsTombstone() { + // We found a tombstone and failOnTombstones is true: fail. + return nil, &roachpb.ConditionFailedError{ + ActualValue: existVal.ToPointer(), } - if existVal.IsPresent() && !existVal.EqualTagAndData(value) { - // The existing value does not match the supplied value. - return nil, &roachpb.ConditionFailedError{ - ActualValue: existVal.ToPointer(), - } + } + if existVal.IsPresent() && !existVal.EqualTagAndData(value) { + // The existing value does not match the supplied value. + return nil, &roachpb.ConditionFailedError{ + ActualValue: existVal.ToPointer(), } - return value.RawBytes, nil - }) + } + return value.RawBytes, nil + } + return mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, localTimestamp, noValue, txn, valueFn) } // mvccKeyFormatter is an fmt.Formatter for MVCC Keys. @@ -2163,6 +2187,7 @@ func MVCCDeleteRange( key, endKey roachpb.Key, max int64, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, txn *roachpb.Transaction, returnKeys bool, ) ([]roachpb.Key, *roachpb.Span, int64, error) { @@ -2189,7 +2214,9 @@ func MVCCDeleteRange( var keys []roachpb.Key for i, kv := range res.KVs { - if err := mvccPutInternal(ctx, rw, iter, ms, kv.Key, timestamp, nil, txn, buf, nil); err != nil { + if err := mvccPutInternal( + ctx, rw, iter, ms, kv.Key, timestamp, localTimestamp, nil, txn, buf, nil, + ); err != nil { return nil, nil, 0, err } if returnKeys { diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index cd3f571bf4e0..702040da32b2 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -55,13 +55,13 @@ import ( // resolve_intent t= k= [status=] // check_intent k= [none] // -// cput [t=] [ts=[,]] [resolve [status=]] k= v= [raw] [cond=] -// del [t=] [ts=[,]] [resolve [status=]] k= -// del_range [t=] [ts=[,]] [resolve [status=]] k= [end=] [max=] [returnKeys] -// get [t=] [ts=[,]] [resolve [status=]] k= [inconsistent] [tombstones] [failOnMoreRecent] [localUncertaintyLimit=[,]] [globalUncertaintyLimit=[,]] -// increment [t=] [ts=[,]] [resolve [status=]] k= [inc=] -// put [t=] [ts=[,]] [resolve [status=]] k= v= [raw] -// scan [t=] [ts=[,]] [resolve [status=]] k= [end=] [inconsistent] [tombstones] [reverse] [failOnMoreRecent] [localUncertaintyLimit=[,]] [globalUncertaintyLimit=[,]] [max=] [targetbytes=] [avoidExcess] [allowEmpty] +// cput [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] k= v= [raw] [cond=] +// del [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] k= +// del_range [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] k= [end=] [max=] [returnKeys] +// increment [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] k= [inc=] +// put [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] k= v= [raw] +// get [t=] [ts=[,]] [resolve [status=]] k= [inconsistent] [tombstones] [failOnMoreRecent] [localUncertaintyLimit=[,]] [globalUncertaintyLimit=[,]] +// scan [t=] [ts=[,]] [resolve [status=]] k= [end=] [inconsistent] [tombstones] [reverse] [failOnMoreRecent] [localUncertaintyLimit=[,]] [globalUncertaintyLimit=[,]] [max=] [targetbytes=] [avoidExcess] [allowEmpty] // // merge [ts=[,]] k= v= [raw] // @@ -423,7 +423,7 @@ func cmdTxnBegin(e *evalCtx) error { var txnName string e.scanArg("t", &txnName) ts := e.getTs(nil) - globalUncertaintyLimit := e.getTsWithName(nil, "globalUncertaintyLimit") + globalUncertaintyLimit := e.getTsWithName("globalUncertaintyLimit") key := roachpb.KeyMin if e.hasArg("k") { key = e.getKey() @@ -587,6 +587,7 @@ func cmdClearRange(e *evalCtx) error { func cmdCPut(e *evalCtx) error { txn := e.getTxn(optional) ts := e.getTs(txn) + localTs := hlc.ClockTimestamp(e.getTsWithName("localTs")) key := e.getKey() val := e.getVal() @@ -603,7 +604,7 @@ func cmdCPut(e *evalCtx) error { resolve, resolveStatus := e.getResolve() return e.withWriter("cput", func(rw ReadWriter) error { - if err := MVCCConditionalPut(e.ctx, rw, nil, key, ts, val, expVal, behavior, txn); err != nil { + if err := MVCCConditionalPut(e.ctx, rw, nil, key, ts, localTs, val, expVal, behavior, txn); err != nil { return err } if resolve { @@ -617,9 +618,10 @@ func cmdDelete(e *evalCtx) error { txn := e.getTxn(optional) key := e.getKey() ts := e.getTs(txn) + localTs := hlc.ClockTimestamp(e.getTsWithName("localTs")) resolve, resolveStatus := e.getResolve() return e.withWriter("del", func(rw ReadWriter) error { - if err := MVCCDelete(e.ctx, rw, nil, key, ts, txn); err != nil { + if err := MVCCDelete(e.ctx, rw, nil, key, ts, localTs, txn); err != nil { return err } if resolve { @@ -633,6 +635,7 @@ func cmdDeleteRange(e *evalCtx) error { txn := e.getTxn(optional) key, endKey := e.getKeyRange() ts := e.getTs(txn) + localTs := hlc.ClockTimestamp(e.getTsWithName("localTs")) returnKeys := e.hasArg("returnKeys") max := 0 if e.hasArg("max") { @@ -641,7 +644,8 @@ func cmdDeleteRange(e *evalCtx) error { resolve, resolveStatus := e.getResolve() return e.withWriter("del_range", func(rw ReadWriter) error { - deleted, resumeSpan, num, err := MVCCDeleteRange(e.ctx, rw, nil, key, endKey, int64(max), ts, txn, returnKeys) + deleted, resumeSpan, num, err := MVCCDeleteRange( + e.ctx, rw, nil, key, endKey, int64(max), ts, localTs, txn, returnKeys) if err != nil { return err } @@ -676,8 +680,8 @@ func cmdGet(e *evalCtx) error { opts.FailOnMoreRecent = true } opts.Uncertainty = uncertainty.Interval{ - GlobalLimit: e.getTsWithName(nil, "globalUncertaintyLimit"), - LocalLimit: hlc.ClockTimestamp(e.getTsWithName(nil, "localUncertaintyLimit")), + GlobalLimit: e.getTsWithName("globalUncertaintyLimit"), + LocalLimit: hlc.ClockTimestamp(e.getTsWithName("localUncertaintyLimit")), } if opts.Txn != nil { if !opts.Uncertainty.GlobalLimit.IsEmpty() { @@ -703,6 +707,7 @@ func cmdGet(e *evalCtx) error { func cmdIncrement(e *evalCtx) error { txn := e.getTxn(optional) ts := e.getTs(txn) + localTs := hlc.ClockTimestamp(e.getTsWithName("localTs")) key := e.getKey() inc := int64(1) @@ -715,7 +720,7 @@ func cmdIncrement(e *evalCtx) error { resolve, resolveStatus := e.getResolve() return e.withWriter("increment", func(rw ReadWriter) error { - curVal, err := MVCCIncrement(e.ctx, rw, nil, key, ts, txn, inc) + curVal, err := MVCCIncrement(e.ctx, rw, nil, key, ts, localTs, txn, inc) if err != nil { return err } @@ -746,6 +751,7 @@ func cmdMerge(e *evalCtx) error { func cmdPut(e *evalCtx) error { txn := e.getTxn(optional) ts := e.getTs(txn) + localTs := hlc.ClockTimestamp(e.getTsWithName("localTs")) key := e.getKey() val := e.getVal() @@ -753,7 +759,7 @@ func cmdPut(e *evalCtx) error { resolve, resolveStatus := e.getResolve() return e.withWriter("put", func(rw ReadWriter) error { - if err := MVCCPut(e.ctx, rw, nil, key, ts, val, txn); err != nil { + if err := MVCCPut(e.ctx, rw, nil, key, ts, localTs, val, txn); err != nil { return err } if resolve { @@ -782,8 +788,8 @@ func cmdScan(e *evalCtx) error { opts.FailOnMoreRecent = true } opts.Uncertainty = uncertainty.Interval{ - GlobalLimit: e.getTsWithName(nil, "globalUncertaintyLimit"), - LocalLimit: hlc.ClockTimestamp(e.getTsWithName(nil, "localUncertaintyLimit")), + GlobalLimit: e.getTsWithName("globalUncertaintyLimit"), + LocalLimit: hlc.ClockTimestamp(e.getTsWithName("localUncertaintyLimit")), } if opts.Txn != nil { if !opts.Uncertainty.GlobalLimit.IsEmpty() { @@ -899,10 +905,14 @@ func (e *evalCtx) getResolve() (bool, roachpb.TransactionStatus) { } func (e *evalCtx) getTs(txn *roachpb.Transaction) hlc.Timestamp { - return e.getTsWithName(txn, "ts") + return e.getTsWithTxnAndName(txn, "ts") } -func (e *evalCtx) getTsWithName(txn *roachpb.Transaction, name string) hlc.Timestamp { +func (e *evalCtx) getTsWithName(name string) hlc.Timestamp { + return e.getTsWithTxnAndName(nil, name) +} + +func (e *evalCtx) getTsWithTxnAndName(txn *roachpb.Transaction, name string) hlc.Timestamp { var ts hlc.Timestamp if txn != nil { ts = txn.ReadTimestamp diff --git a/pkg/storage/mvcc_incremental_iterator_test.go b/pkg/storage/mvcc_incremental_iterator_test.go index e5f971b07c2a..28a2bf88aec6 100644 --- a/pkg/storage/mvcc_incremental_iterator_test.go +++ b/pkg/storage/mvcc_incremental_iterator_test.go @@ -538,7 +538,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, v, nil); err != nil { + if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { t.Fatal(err) } } @@ -576,7 +576,7 @@ func TestMVCCIncrementalIteratorNextIgnoringTime(t *testing.T) { }) // Exercise deletion. - if err := MVCCDelete(ctx, e, nil, testKey1, ts3, nil); err != nil { + if err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } // Returns the kv_1_1_1 even though it is outside (startTime, endTime]. @@ -597,7 +597,7 @@ func TestMVCCIncrementalIteratorNextIgnoringTime(t *testing.T) { ReadTimestamp: ts4, } txn1Val := roachpb.Value{RawBytes: testValue4} - if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, txn1Val, &txn1); err != nil { + if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, txn1Val, &txn1); err != nil { t.Fatal(err) } @@ -675,7 +675,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, v, nil); err != nil { + if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { t.Fatal(err) } } @@ -710,7 +710,7 @@ func TestMVCCIncrementalIteratorNextKeyIgnoringTime(t *testing.T) { }) // Exercise deletion. - if err := MVCCDelete(ctx, e, nil, testKey1, ts3, nil); err != nil { + if err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } // Returns the kv_1_1_1 even though it is outside (startTime, endTime]. @@ -731,7 +731,7 @@ func TestMVCCIncrementalIteratorNextKeyIgnoringTime(t *testing.T) { ReadTimestamp: ts4, } txn1Val := roachpb.Value{RawBytes: testValue4} - if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, txn1Val, &txn1); err != nil { + if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, txn1Val, &txn1); err != nil { t.Fatal(err) } @@ -797,7 +797,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, v, nil); err != nil { + if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { t.Fatal(err) } } @@ -912,11 +912,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, v, nil); err != nil { + if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { t.Fatal(err) } } - if err := MVCCPut(ctx, e, nil, txn.TxnMeta.Key, txn.ReadTimestamp, val, &txn); err != nil { + if err := MVCCPut(ctx, e, nil, txn.TxnMeta.Key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val, &txn); err != nil { t.Fatal(err) } t.Run(engineImpl.name, func(t *testing.T) { @@ -1096,7 +1096,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, v, nil); err != nil { + if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { t.Fatal(err) } } @@ -1114,18 +1114,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, nil); err != nil { + if err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } t.Run("del", assertEqualKVs(e, localMax, keyMax, ts1, tsMax, latest, kvs(kv1Deleted3, kv2_2_2))) // Exercise intent handling. txn1, txn1Val, intentErr1 := makeKVTxn(testKey1, testValue4, ts4) - if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, txn1Val, &txn1); err != nil { + if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, txn1Val, &txn1); err != nil { t.Fatal(err) } txn2, txn2Val, intentErr2 := makeKVTxn(testKey2, testValue4, ts4) - if err := MVCCPut(ctx, e, nil, txn2.TxnMeta.Key, txn2.ReadTimestamp, txn2Val, &txn2); err != nil { + if err := MVCCPut(ctx, e, nil, txn2.TxnMeta.Key, txn2.ReadTimestamp, hlc.ClockTimestamp{}, txn2Val, &txn2); err != nil { t.Fatal(err) } t.Run("intents-1", @@ -1164,7 +1164,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, v, nil); err != nil { + if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { t.Fatal(err) } } @@ -1182,18 +1182,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, nil); err != nil { + if err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); 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, txn1Val, intentErr1 := makeKVTxn(testKey1, testValue4, ts4) - if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, txn1Val, &txn1); err != nil { + if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, txn1Val, &txn1); err != nil { t.Fatal(err) } txn2, txn2Val, intentErr2 := makeKVTxn(testKey2, testValue4, ts4) - if err := MVCCPut(ctx, e, nil, txn2.TxnMeta.Key, txn2.ReadTimestamp, txn2Val, &txn2); err != nil { + if err := MVCCPut(ctx, e, nil, txn2.TxnMeta.Key, txn2.ReadTimestamp, hlc.ClockTimestamp{}, txn2Val, &txn2); err != nil { t.Fatal(err) } // Single intent tests are verifying behavior when intent collection is not enabled. @@ -1280,7 +1280,7 @@ func TestMVCCIncrementalIteratorIntentRewrittenConcurrently(t *testing.T) { }, ReadTimestamp: ts1, } - if err := MVCCPut(ctx, e, nil, kA, ts1, vA1, txn); err != nil { + if err := MVCCPut(ctx, e, nil, kA, ts1, hlc.ClockTimestamp{}, vA1, txn); err != nil { t.Fatal(err) } @@ -1298,7 +1298,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, vA2, txn); err != nil { + if err := MVCCPut(ctx, b, nil, kA, ts1, hlc.ClockTimestamp{}, vA2, txn); err != nil { return err } return b.Commit(false) @@ -1394,17 +1394,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, vA1, txnA1)) - require.NoError(t, MVCCPut(ctx, db, nil, kB, txnB1.ReadTimestamp, vB1, txnB1)) - require.NoError(t, MVCCPut(ctx, db, nil, kC, txnC1.ReadTimestamp, vC1, txnC1)) + 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, db.Flush()) require.NoError(t, db.Compact()) _, err := MVCCResolveWriteIntent(ctx, db, nil, intent(txnA1)) require.NoError(t, err) _, err = MVCCResolveWriteIntent(ctx, db, nil, intent(txnB1)) require.NoError(t, err) - require.NoError(t, MVCCPut(ctx, db, nil, kA, ts2, vA2, nil)) - require.NoError(t, MVCCPut(ctx, db, nil, kA, txnA3.WriteTimestamp, vA3, txnA3)) + 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, db.Flush()) // The kA ts1 intent has been resolved. There's now a new intent on kA, but @@ -1453,9 +1453,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}, v, txn, - ); err != nil { + if err := MVCCPut(ctx, db1, nil, roachpb.Key(key), hlc.Timestamp{WallTime: ts}, hlc.ClockTimestamp{}, v, txn); err != nil { t.Fatal(err) } } @@ -1760,8 +1758,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 /* ms */, key, ts, value, nil); err != nil { + if err := MVCCPut(context.Background(), batch, nil, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { b.Fatal(err) } } diff --git a/pkg/storage/mvcc_logical_ops_test.go b/pkg/storage/mvcc_logical_ops_test.go index bd690ba791be..bf14945e23d5 100644 --- a/pkg/storage/mvcc_logical_ops_test.go +++ b/pkg/storage/mvcc_logical_ops_test.go @@ -39,36 +39,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}, value1, nil); err != nil { + if err := MVCCPut(ctx, ol, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } txn1ts := makeTxn(*txn1, hlc.Timestamp{Logical: 2}) - if err := MVCCPut(ctx, ol, nil, testKey1, txn1ts.ReadTimestamp, value2, txn1ts); err != nil { + if err := MVCCPut(ctx, ol, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, 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}, value1, nil); err != nil { + if err := MVCCPut(ctx, ol, nil, localKey, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, ol, nil, localKey, txn1ts.ReadTimestamp, value2, txn1ts); err != nil { + if err := MVCCPut(ctx, ol, nil, localKey, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, 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, value2, txn1ts); err != nil { + if err := MVCCPut(ctx, ol, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, ol, nil, localKey, txn1ts.ReadTimestamp, value2, txn1ts); err != nil { + if err := MVCCPut(ctx, ol, nil, localKey, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, 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, value3, txn1LargerTS); err != nil { + if err := MVCCPut(ctx, ol, nil, testKey2, txn1LargerTS.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn1LargerTS); err != nil { t.Fatal(err) } @@ -92,7 +92,7 @@ func TestMVCCOpLogWriter(t *testing.T) { // Write another intent, push it, then abort it. txn2ts := makeTxn(*txn2, hlc.Timestamp{Logical: 5}) - if err := MVCCPut(ctx, ol, nil, testKey3, txn2ts.ReadTimestamp, value4, txn2ts); err != nil { + if err := MVCCPut(ctx, ol, nil, testKey3, txn2ts.ReadTimestamp, hlc.ClockTimestamp{}, value4, txn2ts); err != nil { t.Fatal(err) } txn2Pushed := *txn2 diff --git a/pkg/storage/mvcc_stats_test.go b/pkg/storage/mvcc_stats_test.go index 5cc09743c86d..855d5e9fe01d 100644 --- a/pkg/storage/mvcc_stats_test.go +++ b/pkg/storage/mvcc_stats_test.go @@ -97,7 +97,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, value, nil); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, value, nil); err != nil { t.Fatal(err) } @@ -122,7 +122,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, txn); err != nil { + if err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { t.Fatal(err) } @@ -180,7 +180,7 @@ func TestMVCCStatsPutCommitMovesTimestamp(t *testing.T) { } // Write an intent at t=1s. value := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, ts1, value, txn); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, value, txn); err != nil { t.Fatal(err) } @@ -262,7 +262,7 @@ func TestMVCCStatsPutPushMovesTimestamp(t *testing.T) { } // Write an intent. value := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, value, txn); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, value, txn); err != nil { t.Fatal(err) } @@ -352,7 +352,7 @@ func TestMVCCStatsDeleteMovesTimestamp(t *testing.T) { // Write an intent. value := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, value, txn); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, value, txn); err != nil { t.Fatal(err) } @@ -417,7 +417,7 @@ func TestMVCCStatsDeleteMovesTimestamp(t *testing.T) { }).Size()) require.EqualValues(t, m2ValSize, 64) - if err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, txn); err != nil { + if err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { t.Fatal(err) } @@ -470,7 +470,7 @@ func TestMVCCStatsPutMovesDeletionTimestamp(t *testing.T) { } // Write a deletion tombstone intent. - if err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, txn); err != nil { + if err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { t.Fatal(err) } @@ -528,7 +528,7 @@ func TestMVCCStatsPutMovesDeletionTimestamp(t *testing.T) { }).Size()) require.EqualValues(t, m2ValSize, 54) - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, value, txn); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, value, txn); err != nil { t.Fatal(err) } @@ -580,7 +580,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, nil /* txn */); err != nil { + if err := MVCCDelete(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } @@ -604,7 +604,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, txn); err != nil { + if err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { t.Fatal(err) } @@ -724,7 +724,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, value, nil /* txn */); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, value, nil); err != nil { t.Fatal(err) } @@ -754,7 +754,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, txn); err != nil { + if err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { t.Fatal(err) } @@ -835,7 +835,7 @@ func TestMVCCStatsPutDelPutMovesTimestamp(t *testing.T) { require.EqualValues(t, vVal2Size, 14) txn.WriteTimestamp.Forward(ts3) - if err := MVCCPut(ctx, engine, &aggMS, key, txn.ReadTimestamp, val2, txn); err != nil { + if err := MVCCPut(ctx, engine, &aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val2, txn); err != nil { t.Fatal(err) } @@ -894,10 +894,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, nil); err != nil { + if err := MVCCDelete(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } - if err := MVCCDelete(ctx, engine, aggMS, key, ts2, nil); err != nil { + if err := MVCCDelete(ctx, engine, aggMS, key, ts2, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } @@ -975,7 +975,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, value, txn); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, value, txn); err != nil { t.Fatal(err) } @@ -1020,7 +1020,7 @@ func TestMVCCStatsPutIntentTimestampNotPutTimestamp(t *testing.T) { {Sequence: 0, Value: value.RawBytes}, }, }).Size()) - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, value, txn); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, value, txn); err != nil { t.Fatal(err) } @@ -1068,7 +1068,7 @@ func TestMVCCStatsPutWaitDeleteGC(t *testing.T) { // Write a value at ts1. val1 := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, ts1, val1, nil /* txn */); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, val1, nil); err != nil { t.Fatal(err) } @@ -1094,7 +1094,7 @@ func TestMVCCStatsPutWaitDeleteGC(t *testing.T) { // Delete the value at ts5. - if err := MVCCDelete(ctx, engine, aggMS, key, ts2, nil /* txn */); err != nil { + if err := MVCCDelete(ctx, engine, aggMS, key, ts2, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } @@ -1161,7 +1161,7 @@ func TestMVCCStatsTxnSysPutPut(t *testing.T) { // Write an intent at ts1. val1 := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, val1, txn); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val1, txn); err != nil { t.Fatal(err) } @@ -1210,7 +1210,7 @@ func TestMVCCStatsTxnSysPutPut(t *testing.T) { }).Size()) require.EqualValues(t, mVal2Size, 64) - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, val2, txn); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val2, txn); err != nil { t.Fatal(err) } @@ -1251,7 +1251,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, val1, txn); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val1, txn); err != nil { t.Fatal(err) } @@ -1321,7 +1321,7 @@ func TestMVCCStatsSysPutPut(t *testing.T) { // Write a value at ts1. val1 := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, ts1, val1, nil /* txn */); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, val1, nil); err != nil { t.Fatal(err) } @@ -1347,7 +1347,7 @@ func TestMVCCStatsSysPutPut(t *testing.T) { // Put another value at ts2. - if err := MVCCPut(ctx, engine, aggMS, key, ts2, val2, nil /* txn */); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, ts2, hlc.ClockTimestamp{}, val2, nil); err != nil { t.Fatal(err) } @@ -1495,7 +1495,7 @@ func TestMVCCStatsRandomized(t *testing.T) { actions := make(map[string]func(*state) string) actions["Put"] = func(s *state) string { - if err := MVCCPut(ctx, s.eng, s.MS, s.key, s.TS, s.rngVal(), s.Txn); err != nil { + if err := MVCCPut(ctx, s.eng, s.MS, s.key, s.TS, hlc.ClockTimestamp{}, s.rngVal(), s.Txn); err != nil { return err.Error() } return "" @@ -1503,13 +1503,13 @@ func TestMVCCStatsRandomized(t *testing.T) { actions["InitPut"] = func(s *state) string { failOnTombstones := (s.rng.Intn(2) == 0) desc := fmt.Sprintf("failOnTombstones=%t", failOnTombstones) - if err := MVCCInitPut(ctx, s.eng, s.MS, s.key, s.TS, s.rngVal(), failOnTombstones, s.Txn); err != nil { + if err := MVCCInitPut(ctx, s.eng, s.MS, s.key, s.TS, hlc.ClockTimestamp{}, s.rngVal(), failOnTombstones, s.Txn); err != nil { return desc + ": " + err.Error() } return desc } actions["Del"] = func(s *state) string { - if err := MVCCDelete(ctx, s.eng, s.MS, s.key, s.TS, s.Txn); err != nil { + if err := MVCCDelete(ctx, s.eng, s.MS, s.key, s.TS, hlc.ClockTimestamp{}, s.Txn); err != nil { return err.Error() } return "" @@ -1525,7 +1525,9 @@ func TestMVCCStatsRandomized(t *testing.T) { } else { keyMin = keys.LocalMax } - if _, _, _, err := MVCCDeleteRange(ctx, s.eng, s.MS, keyMin, keyMax, max, s.TS, s.Txn, returnKeys); err != nil { + if _, _, _, err := MVCCDeleteRange( + ctx, s.eng, s.MS, keyMin, keyMax, max, s.TS, hlc.ClockTimestamp{}, s.Txn, returnKeys, + ); err != nil { return desc + ": " + err.Error() } return desc diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go index 539d2a0e6f33..6060927203a2 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -293,10 +293,10 @@ func TestMVCCGetNoMoreOldVersion(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } @@ -322,7 +322,7 @@ func TestMVCCGetAndDelete(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{}) @@ -333,7 +333,7 @@ func TestMVCCGetAndDelete(t *testing.T) { t.Fatal("the value should not be empty") } - err = MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, nil) + err = MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, nil) if err != nil { t.Fatal(err) } @@ -382,15 +382,11 @@ func TestMVCCWriteWithOlderTimestampAfterDeletionOfNonexistentKey(t *testing.T) engine := engineImpl.create() defer engine.Close() - if err := MVCCDelete( - context.Background(), engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, nil, - ); err != nil { + if err := MVCCDelete(context.Background(), engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } - if err := MVCCPut( - context.Background(), engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil, - ); !testutils.IsError( + if err := MVCCPut(context.Background(), engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); !testutils.IsError( err, "write for key \"/db1\" at timestamp 0.000000001,0 too old; wrote at 0.000000003,1", ) { t.Fatal(err) @@ -437,7 +433,7 @@ func TestMVCCInlineWithTxn(t *testing.T) { defer engine.Close() // Put an inline value. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } @@ -459,7 +455,7 @@ func TestMVCCInlineWithTxn(t *testing.T) { } // Verify inline put with txn is an error. - err = MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{}, value2, txn2) + err = MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{}, hlc.ClockTimestamp{}, value2, txn2) if !testutils.IsError(err, "writes not allowed within transactions") { t.Errorf("unexpected error: %+v", err) } @@ -477,7 +473,7 @@ func TestMVCCDeleteMissingKey(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, nil); err != nil { + if err := MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } // Verify nothing is written to the engine. @@ -501,7 +497,7 @@ func TestMVCCGetAndDeleteInTxn(t *testing.T) { txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) txn.Sequence++ - if err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value1, txn); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn); err != nil { t.Fatal(err) } @@ -515,7 +511,7 @@ func TestMVCCGetAndDeleteInTxn(t *testing.T) { txn.Sequence++ txn.WriteTimestamp = hlc.Timestamp{WallTime: 3} - if err := MVCCDelete(ctx, engine, nil, testKey1, txn.ReadTimestamp, txn); err != nil { + if err := MVCCDelete(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { t.Fatal(err) } @@ -559,7 +555,7 @@ func TestMVCCGetWriteIntentError(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, value1, txn1); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { t.Fatal(err) } @@ -616,7 +612,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, v, txnMap[i]); err != nil { + if err := MVCCPut(ctx, engine, nil, kv.Key, kv.Value.Timestamp, hlc.ClockTimestamp{}, v, txnMap[i]); err != nil { t.Fatal(err) } } @@ -723,11 +719,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}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 2}) - if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, value2, txn1ts); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { t.Fatal(err) } @@ -757,7 +753,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, value1, txn2); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, txn2.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn2); err != nil { t.Fatal(err) } val, intent, err := MVCCGet(ctx, engine, testKey2, hlc.Timestamp{WallTime: 2}, @@ -797,11 +793,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}, v1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, v1, nil); err != nil { t.Fatal(err) } txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 2}) - if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, v2, txn1ts); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, v2, txn1ts); err != nil { t.Fatal(err) } @@ -843,7 +839,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, v1, txn2); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, txn2.ReadTimestamp, hlc.ClockTimestamp{}, v1, txn2); err != nil { t.Fatal(err) } val := roachpb.Value{} @@ -862,10 +858,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}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, txn1ts.ReadTimestamp, v2, txn1ts); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, v2, txn1ts); err != nil { t.Fatal(err) } val := roachpb.Value{} @@ -903,7 +899,7 @@ func TestMVCCInvalidateIterator(t *testing.T) { ts2 := hlc.Timestamp{WallTime: 2} key := roachpb.Key("a") - if err := MVCCPut(ctx, engine, nil, key, ts1, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, key, ts1, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } @@ -1005,7 +1001,7 @@ func TestMVCCPutAfterBatchIterCreate(t *testing.T) { iter.Next() // key2/5 // Lay down an intent on key3, which will go at key3/0 and sort before key3/5. - err = MVCCDelete(context.Background(), batch, nil, testKey3, txn.WriteTimestamp, txn) + err = MVCCDelete(context.Background(), batch, nil, testKey3, txn.WriteTimestamp, hlc.ClockTimestamp{}, txn) if err != nil { t.Fatal(err) } @@ -1022,28 +1018,28 @@ func TestMVCCPutAfterBatchIterCreate(t *testing.T) { } func mvccScanTest(ctx context.Context, t *testing.T, engine Engine) { - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, value4, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 3}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value3, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 4}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 4}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 5}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 5}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } @@ -1143,19 +1139,19 @@ func TestMVCCScanMaxNum(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { t.Fatal(err) } @@ -1235,19 +1231,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}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, roachpb.Key("/a"), hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, roachpb.Key("/a"), hlc.Timestamp{WallTime: 2}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, roachpb.Key("/a"), hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, roachpb.Key("/aa"), hlc.Timestamp{WallTime: 2}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, roachpb.Key("/aa"), hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, roachpb.Key("/aa"), hlc.Timestamp{WallTime: 3}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, roachpb.Key("/aa"), hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value3, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, roachpb.Key("/b"), hlc.Timestamp{WallTime: 1}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, roachpb.Key("/b"), hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { t.Fatal(err) } @@ -1277,17 +1273,17 @@ func TestMVCCScanInTxn(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) - if err := MVCCPut(ctx, engine, nil, testKey3, txn.ReadTimestamp, value3, txn); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, txn.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { t.Fatal(err) } @@ -1339,24 +1335,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, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, ts1, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } txn1ts2 := makeTxn(*txn1, ts2) - if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts2.ReadTimestamp, value2, txn1ts2); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts2.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts2); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, ts3, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, ts3, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, ts4, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, ts4, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } txn2ts5 := makeTxn(*txn2, ts5) - if err := MVCCPut(ctx, engine, nil, testKey3, txn2ts5.ReadTimestamp, value3, txn2ts5); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, txn2ts5.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn2ts5); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, ts6, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, ts6, hlc.ClockTimestamp{}, value4, nil); err != nil { t.Fatal(err) } @@ -1417,29 +1413,28 @@ func TestMVCCDeleteRange(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey5, hlc.Timestamp{WallTime: 1}, value5, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey5, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value5, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, value6, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value6, nil); 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}, nil, false, - ) + deleted, resumeSpan, num, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, + 2, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, false) if err != nil { t.Fatal(err) } @@ -1494,8 +1489,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}, nil, false) + deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, + -1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, false) if err != nil { t.Fatal(err) } @@ -1522,8 +1517,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}, nil, false) + deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, testKey4, keyMax, + 0, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, false) if err != nil { t.Fatal(err) } @@ -1547,8 +1542,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}, nil, false) + deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, localMax, testKey2, + 0, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, false) if err != nil { t.Fatal(err) } @@ -1583,28 +1578,28 @@ func TestMVCCDeleteRangeReturnKeys(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey5, hlc.Timestamp{WallTime: 1}, value5, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey5, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value5, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, value6, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value6, nil); 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}, nil, true) + deleted, resumeSpan, num, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, + 2, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, true) if err != nil { t.Fatal(err) } @@ -1638,8 +1633,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}, nil, true) + deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, + -1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, true) if err != nil { t.Fatal(err) } @@ -1666,8 +1661,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}, nil, true) + deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, testKey4, keyMax, + math.MaxInt64, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, true) if err != nil { t.Fatal(err) } @@ -1697,8 +1692,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}, nil, true) + deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, localMax, testKey2, + math.MaxInt64, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, true) if err != nil { t.Fatal(err) } @@ -1734,31 +1729,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}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } txn.Sequence++ - if err := MVCCPut(ctx, engine, nil, testKey2, txn.ReadTimestamp, value2, txn); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, txn.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn); err != nil { t.Fatal(err) } txn.Sequence++ - if err := MVCCPut(ctx, engine, nil, testKey3, txn.ReadTimestamp, value3, txn); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, txn.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { t.Fatal(err) } - if _, _, _, err := MVCCDeleteRange( - ctx, engine, nil, testKey2, testKey4, math.MaxInt64, hlc.Timestamp{WallTime: 1}, nil, false, - ); err == nil { + if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey4, + math.MaxInt64, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, nil, 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, txn, false, - ); err != nil { + if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey4, + math.MaxInt64, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn, false); err != nil { t.Fatal(err) } }) @@ -1778,21 +1771,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}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, txn1ts.ReadTimestamp, value2, txn1ts); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, txn2ts.ReadTimestamp, value3, txn2ts); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, txn2ts.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn2ts); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { t.Fatal(err) } - if _, _, _, err := MVCCDeleteRange( - ctx, engine, nil, testKey2, testKey4, math.MaxInt64, txn1ts.ReadTimestamp, txn1ts, false, + if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey4, + math.MaxInt64, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, txn1ts, false, ); err == nil { t.Fatal("expected error on uncommitted write intent") } @@ -1812,31 +1805,23 @@ func TestMVCCUncommittedDeleteRangeVisible(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut( - ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil, - ); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut( - ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, value2, nil, - ); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut( - ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, value3, nil, - ); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { t.Fatal(err) } - if err := MVCCDelete( - ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 2, Logical: 1}, nil, - ); err != nil { + if err := MVCCDelete(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 2, Logical: 1}, hlc.ClockTimestamp{}, nil); 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, txn, false, + if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey1, testKey4, + math.MaxInt64, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn, false, ); err != nil { t.Fatal(err) } @@ -1862,15 +1847,15 @@ func TestMVCCDeleteRangeOldTimestamp(t *testing.T) { t.Run(engineImpl.name, func(t *testing.T) { engine := engineImpl.create() defer engine.Close() - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil) + err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil) if err != nil { t.Fatal(err) } - err = MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 3}, value2, nil) + err = MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value2, nil) if err != nil { t.Fatal(err) } - err = MVCCDelete(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 5}, nil) + err = MVCCDelete(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 5}, hlc.ClockTimestamp{}, nil) if err != nil { t.Fatal(err) } @@ -1878,9 +1863,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}, nil, true, - ) + keys, resume, keyCount, err := MVCCDeleteRange(ctx, b, nil, testKey1, testKey4, + math.MaxInt64, hlc.Timestamp{WallTime: 4}, hlc.ClockTimestamp{}, nil, true) require.Nil(t, keys) require.Nil(t, resume) require.Equal(t, int64(0), keyCount) @@ -1890,9 +1874,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}, nil, true, - ) + keys, resume, keyCount, err = MVCCDeleteRange(ctx, b, nil, testKey1, testKey4, + math.MaxInt64, hlc.Timestamp{WallTime: 5}, hlc.ClockTimestamp{}, nil, true) require.Nil(t, keys) require.Nil(t, resume) require.Equal(t, int64(0), keyCount) @@ -1903,9 +1886,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}, nil, true, - ) + keys, resume, keyCount, err = MVCCDeleteRange(ctx, b, nil, testKey1, testKey4, + math.MaxInt64, hlc.Timestamp{WallTime: 6}, hlc.ClockTimestamp{}, nil, true) require.Equal(t, []roachpb.Key{testKey1}, keys) require.Nil(t, resume) require.Equal(t, int64(1), keyCount) @@ -1935,20 +1917,19 @@ func TestMVCCDeleteRangeInline(t *testing.T) { {testKey4, value4}, {testKey5, value5}, } { - if err := MVCCPut(ctx, engine, nil, kv.key, hlc.Timestamp{Logical: 0}, kv.value, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, kv.key, hlc.Timestamp{Logical: 0}, hlc.ClockTimestamp{}, kv.value, nil); 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}, value6, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value6, nil); 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}, nil, true, - ) + deleted, resumeSpan, num, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, + 2, hlc.Timestamp{Logical: 0}, hlc.ClockTimestamp{}, nil, true) if err != nil { t.Fatal(err) } @@ -1964,23 +1945,23 @@ 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}, nil, true, + if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey1, testKey6, + 1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, 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. const writeTooOldErrString = "WriteTooOldError" - if _, _, _, err := MVCCDeleteRange( - ctx, engine, nil, testKey6, keyMax, 1, hlc.Timestamp{Logical: 0}, nil, true, + if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey6, keyMax, + 1, hlc.Timestamp{Logical: 0}, hlc.ClockTimestamp{}, nil, true, ); !testutils.IsError(err, writeTooOldErrString) { t.Fatalf("got error %v, expected error with text '%s'", err, writeTooOldErrString) } // Attempt to delete inline keys in a transaction; should fail. - if _, _, _, err := MVCCDeleteRange( - ctx, engine, nil, testKey2, testKey6, 2, hlc.Timestamp{Logical: 0}, txn1, true, + if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, + 2, hlc.Timestamp{Logical: 0}, hlc.ClockTimestamp{}, txn1, true, ); !testutils.IsError(err, "writes not allowed within transactions") { t.Errorf("unexpected error: %+v", err) } @@ -2076,12 +2057,12 @@ func TestMVCCClearTimeRange(t *testing.T) { // when read. setupKVs := func(t *testing.T) Engine { engine := engineImpl.create() - require.NoError(t, MVCCPut(ctx, engine, nil, testKey2, ts1, value1, nil)) - require.NoError(t, MVCCPut(ctx, engine, nil, testKey2, ts2, value2, nil)) - require.NoError(t, MVCCPut(ctx, engine, nil, testKey5, ts2, value2, nil)) - require.NoError(t, MVCCPut(ctx, engine, nil, testKey1, ts3, value3, nil)) - require.NoError(t, MVCCPut(ctx, engine, nil, testKey5, ts4, value4, nil)) - require.NoError(t, MVCCPut(ctx, engine, nil, testKey2, ts4, value4, nil)) + require.NoError(t, MVCCPut(ctx, engine, nil, testKey2, ts1, hlc.ClockTimestamp{}, value1, nil)) + require.NoError(t, MVCCPut(ctx, engine, nil, testKey2, ts2, hlc.ClockTimestamp{}, value2, nil)) + require.NoError(t, MVCCPut(ctx, engine, nil, testKey5, ts2, hlc.ClockTimestamp{}, value2, nil)) + require.NoError(t, MVCCPut(ctx, engine, nil, testKey1, ts3, hlc.ClockTimestamp{}, value3, nil)) + require.NoError(t, MVCCPut(ctx, engine, nil, testKey5, ts4, hlc.ClockTimestamp{}, value4, nil)) + require.NoError(t, MVCCPut(ctx, engine, nil, testKey2, ts4, hlc.ClockTimestamp{}, value4, nil)) return engine } @@ -2241,7 +2222,7 @@ func TestMVCCClearTimeRange(t *testing.T) { txn := roachpb.MakeTransaction("test", nil, roachpb.NormalUserPriority, ts3, 1, 1) setupKVsWithIntent := func(t *testing.T) Engine { e := setupKVs(t) - require.NoError(t, MVCCPut(ctx, e, &enginepb.MVCCStats{}, testKey3, ts3, value3, &txn)) + require.NoError(t, MVCCPut(ctx, e, &enginepb.MVCCStats{}, testKey3, ts3, hlc.ClockTimestamp{}, value3, &txn)) return e } t.Run("clear everything hitting intent fails", func(t *testing.T) { @@ -2357,24 +2338,24 @@ func TestMVCCClearTimeRangeOnRandomData(t *testing.T) { key := roachpb.Key(fmt.Sprintf("%05d", k)) if rand.Float64() > 0.8 { - require.NoError(t, MVCCDelete(ctx, e, &ms, key, hlc.Timestamp{WallTime: ts}, nil)) + require.NoError(t, MVCCDelete(ctx, e, &ms, key, hlc.Timestamp{WallTime: ts}, hlc.ClockTimestamp{}, nil)) } else { v := roachpb.MakeValueFromString(fmt.Sprintf("v-%d", i)) - require.NoError(t, MVCCPut(ctx, e, &ms, key, hlc.Timestamp{WallTime: ts}, v, nil)) + require.NoError(t, MVCCPut(ctx, e, &ms, key, hlc.Timestamp{WallTime: ts}, hlc.ClockTimestamp{}, v, nil)) } } 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)}, v, nil)) + require.NoError(t, MVCCPut(ctx, e, &ms, key, hlc.Timestamp{WallTime: int64(swathTime)}, hlc.ClockTimestamp{}, v, nil)) } // 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)}, v, nil)) + require.NoError(t, MVCCPut(ctx, e, &ms, key, hlc.Timestamp{WallTime: int64(randTimeRange + 1)}, hlc.ClockTimestamp{}, v, nil)) } ms.AgeTo(2000) @@ -2441,25 +2422,25 @@ func TestMVCCInitPut(t *testing.T) { engine := engineImpl.create() defer engine.Close() - err := MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, value1, false, nil) + err := MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, false, nil) if err != nil { t.Fatal(err) } // A repeat of the command will still succeed - err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 2}, value1, false, nil) + err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 2}, hlc.ClockTimestamp{}, value1, false, nil) if err != nil { t.Fatal(err) } // Delete. - err = MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 3}, nil) + err = MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 3}, hlc.ClockTimestamp{}, nil) 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}, value1, true, nil) + err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 4}, hlc.ClockTimestamp{}, value1, true, nil) if e := (*roachpb.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) @@ -2471,13 +2452,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}, value1, false, nil) + err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 5}, hlc.ClockTimestamp{}, value1, false, nil) 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}, value2, false, nil) + err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 6}, hlc.ClockTimestamp{}, value2, false, nil) if e := (*roachpb.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", @@ -2537,14 +2518,14 @@ func TestMVCCInitPutWithTxn(t *testing.T) { txn := *txn1 txn.Sequence++ - err := MVCCInitPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value1, false, &txn) + err := MVCCInitPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value1, false, &txn) if err != nil { t.Fatal(err) } // A repeat of the command will still succeed. txn.Sequence++ - err = MVCCInitPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value1, false, &txn) + err = MVCCInitPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value1, false, &txn) if err != nil { t.Fatal(err) } @@ -2553,7 +2534,7 @@ func TestMVCCInitPutWithTxn(t *testing.T) { // will still succeed. txn.Sequence++ txn.Epoch = 2 - err = MVCCInitPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value2, false, &txn) + err = MVCCInitPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value2, false, &txn) if err != nil { t.Fatal(err) } @@ -2568,7 +2549,7 @@ func TestMVCCInitPutWithTxn(t *testing.T) { } // Write value4 with an old timestamp without txn...should get an error. - err = MVCCInitPut(ctx, engine, nil, testKey1, clock.Now(), value4, false, nil) + err = MVCCInitPut(ctx, engine, nil, testKey1, clock.Now(), hlc.ClockTimestamp{}, value4, false, nil) if e := (*roachpb.ConditionFailedError)(nil); errors.As(err, &e) { if !bytes.Equal(e.ActualValue.RawBytes, value2.RawBytes) { t.Fatalf("the value %s in get result does not match the value %s in request", @@ -2593,28 +2574,28 @@ func TestMVCCReverseScan(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 3}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value4, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey5, hlc.Timestamp{WallTime: 3}, value5, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey5, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value5, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 3}, value6, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value6, nil); err != nil { t.Fatal(err) } @@ -2729,10 +2710,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}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 3}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value3, nil); err != nil { t.Fatal(err) } @@ -2769,12 +2750,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)}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: int64(i)}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } } txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 11}) - if err := MVCCPut(ctx, engine, nil, testKey2, txn1ts.ReadTimestamp, value2, txn1ts); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { t.Fatal(err) } @@ -2821,7 +2802,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)}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: int64(i)}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } } @@ -2862,7 +2843,7 @@ func TestMVCCResolveTxn(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, value1, txn1); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { t.Fatal(err) } @@ -2912,12 +2893,12 @@ func TestMVCCResolveNewerIntent(t *testing.T) { defer engine.Close() // Write first value. - if err := MVCCPut(ctx, engine, nil, testKey1, txn1Commit.WriteTimestamp, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1Commit.WriteTimestamp, hlc.ClockTimestamp{}, value1, nil); 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, value2, txn1) + err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1) if !errors.HasType(err, (*roachpb.WriteTooOldError)(nil)) { t.Fatalf("expected write too old error; got %s", err) } @@ -2954,7 +2935,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, value1, txn); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, tsEarly, hlc.ClockTimestamp{}, value1, txn); err != nil { t.Fatal(err) } @@ -3005,17 +2986,17 @@ func TestMVCCConditionalPutOldTimestamp(t *testing.T) { t.Run(engineImpl.name, func(t *testing.T) { engine := engineImpl.create() defer engine.Close() - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil) + err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil) if err != nil { t.Fatal(err) } - err = MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, value2, nil) + err = MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value2, nil) if err != nil { t.Fatal(err) } // Check nothing is written if the value doesn't match. - err = MVCCConditionalPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, value3, value1.TagAndDataBytes(), CPutFailIfMissing, nil) + err = MVCCConditionalPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, value3, value1.TagAndDataBytes(), CPutFailIfMissing, nil) if err == nil { t.Errorf("unexpected success on conditional put") } @@ -3025,7 +3006,7 @@ func TestMVCCConditionalPutOldTimestamp(t *testing.T) { // But if value does match the most recently written version, we'll get // a write too old error but still write updated value. - err = MVCCConditionalPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, value3, value2.TagAndDataBytes(), CPutFailIfMissing, nil) + err = MVCCConditionalPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, value3, value2.TagAndDataBytes(), CPutFailIfMissing, nil) if err == nil { t.Errorf("unexpected success on conditional put") } @@ -3058,7 +3039,7 @@ func TestMVCCMultiplePutOldTimestamp(t *testing.T) { engine := engineImpl.create() defer engine.Close() - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, value1, nil) + err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value1, nil) if err != nil { t.Fatal(err) } @@ -3067,7 +3048,7 @@ func TestMVCCMultiplePutOldTimestamp(t *testing.T) { // intent is written at the advanced timestamp. txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) txn.Sequence++ - err = MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value2, txn) + err = MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn) if !errors.HasType(err, (*roachpb.WriteTooOldError)(nil)) { t.Errorf("expected WriteTooOldError on Put; got %v", err) } @@ -3085,7 +3066,7 @@ func TestMVCCMultiplePutOldTimestamp(t *testing.T) { // Put again and verify no WriteTooOldError, but timestamp should continue // to be set to (3,1). txn.Sequence++ - err = MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value3, txn) + err = MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn) if err != nil { t.Error(err) } @@ -3115,7 +3096,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, value1, nil) + err := MVCCPut(ctx, engine, nil, testKey1, timestamp, hlc.ClockTimestamp{}, value1, nil) require.EqualError(t, err, expectedErrorString) }) @@ -3138,7 +3119,7 @@ func TestMVCCPutOldOrigTimestampNewCommitTimestamp(t *testing.T) { engine := engineImpl.create() defer engine.Close() - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, value1, nil) + err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value1, nil) if err != nil { t.Fatal(err) } @@ -3149,7 +3130,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, value2, txn) + err = MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn) // Verify that the Put returned a WriteTooOld with the ActualTime set to the // transactions provisional commit timestamp. @@ -3182,7 +3163,7 @@ func TestMVCCAbortTxn(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, value1, txn1); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { t.Fatal(err) } @@ -3221,14 +3202,14 @@ func TestMVCCAbortTxnWithPreviousVersion(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 2}) - if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, value3, txn1ts); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn1ts); err != nil { t.Fatal(err) } @@ -3274,7 +3255,7 @@ func TestMVCCWriteWithDiffTimestampsAndEpochs(t *testing.T) { // Start with epoch 1. txn := *txn1 txn.Sequence++ - if err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value1, &txn); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value1, &txn); err != nil { t.Fatal(err) } // Now write with greater timestamp and epoch 2. @@ -3282,22 +3263,22 @@ func TestMVCCWriteWithDiffTimestampsAndEpochs(t *testing.T) { txne2.Sequence++ txne2.Epoch = 2 txne2.WriteTimestamp = hlc.Timestamp{WallTime: 1} - if err := MVCCPut(ctx, engine, nil, testKey1, txne2.ReadTimestamp, value2, &txne2); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txne2.ReadTimestamp, hlc.ClockTimestamp{}, value2, &txne2); err != nil { t.Fatal(err) } // Try a write with an earlier timestamp; this is just ignored. txne2.Sequence++ txne2.WriteTimestamp = hlc.Timestamp{WallTime: 1} - if err := MVCCPut(ctx, engine, nil, testKey1, txne2.ReadTimestamp, value1, &txne2); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txne2.ReadTimestamp, hlc.ClockTimestamp{}, value1, &txne2); err != nil { t.Fatal(err) } // Try a write with an earlier epoch; again ignored. - if err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value1, &txn); err == nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value1, &txn); err == nil { t.Fatal("unexpected success of a write with an earlier epoch") } // Try a write with different value using both later timestamp and epoch. txne2.Sequence++ - if err := MVCCPut(ctx, engine, nil, testKey1, txne2.ReadTimestamp, value3, &txne2); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txne2.ReadTimestamp, hlc.ClockTimestamp{}, value3, &txne2); err != nil { t.Fatal(err) } // Resolve the intent. @@ -3312,7 +3293,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}, value4, nil) + err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value4, nil) if wtoErr := (*roachpb.WriteTooOldError)(nil); !errors.As(err, &wtoErr) { t.Fatal("unexpected success") } else if wtoErr.ActualTimestamp != expTS { @@ -3325,7 +3306,7 @@ func TestMVCCWriteWithDiffTimestampsAndEpochs(t *testing.T) { err, value.Timestamp, expTS, value4.RawBytes, value.RawBytes) } // Now write an intent with exactly the same timestamp--ties also get WriteTooOldError. - err = MVCCPut(ctx, engine, nil, testKey1, txn2.ReadTimestamp, value5, txn2) + err = MVCCPut(ctx, engine, nil, testKey1, txn2.ReadTimestamp, hlc.ClockTimestamp{}, value5, txn2) intentTS := expTS.Next() if wtoErr := (*roachpb.WriteTooOldError)(nil); !errors.As(err, &wtoErr) { t.Fatal("unexpected success") @@ -3373,12 +3354,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}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, nil); 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, value2, txn1ts); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { t.Fatal(err) } // Try reading using different txns & epochs. @@ -3434,11 +3415,11 @@ func TestMVCCGetWithDiffEpochsAndTimestamps(t *testing.T) { defer engine.Close() // Write initial value without a txn at timestamp 1. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } // Write another value without a txn at timestamp 3. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } // Now write using txn1, epoch 1. @@ -3446,7 +3427,7 @@ func TestMVCCGetWithDiffEpochsAndTimestamps(t *testing.T) { // Bump epoch 1's write timestamp to timestamp 4. txn1ts.WriteTimestamp = hlc.Timestamp{WallTime: 4} // Expected to hit WriteTooOld error but to still lay down intent. - err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, value3, txn1ts) + err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn1ts) if wtoErr := (*roachpb.WriteTooOldError)(nil); !errors.As(err, &wtoErr) { t.Fatalf("unexpectedly not WriteTooOld: %+v", err) } else if expTS, actTS := txn1ts.WriteTimestamp, wtoErr.ActualTimestamp; expTS != actTS { @@ -3503,7 +3484,7 @@ func TestMVCCGetWithOldEpoch(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1e2.ReadTimestamp, value2, txn1e2); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1e2.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1e2); err != nil { t.Fatal(err) } _, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{ @@ -3548,7 +3529,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, value1, &txn); err != nil { + if err := MVCCPut(ctx, engine, nil, key, txn.WriteTimestamp, hlc.ClockTimestamp{}, value1, &txn); err != nil { t.Fatal(err) } } @@ -3556,17 +3537,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, &txn, true, - ) + origDeleted, _, origNum, err := MVCCDeleteRange(ctx, engine, nil, + prefix, prefix.PrefixEnd(), math.MaxInt64, txn.WriteTimestamp, hlc.ClockTimestamp{}, &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, &txn, true, - ) + deleted, _, num, err := MVCCDeleteRange(ctx, engine, nil, + prefix, prefix.PrefixEnd(), math.MaxInt64, txn.WriteTimestamp, hlc.ClockTimestamp{}, &txn, true) if tc.expErr != "" && err != nil { if !testutils.IsError(err, tc.expErr) { t.Fatalf("unexpected error: %+v", err) @@ -3609,7 +3588,7 @@ func TestMVCCGetWithPushedTimestamp(t *testing.T) { defer engine.Close() // Start with epoch 1. - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, value1, txn1); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { t.Fatal(err) } // Resolve the intent, pushing its timestamp forward. @@ -3639,10 +3618,10 @@ func TestMVCCResolveWithDiffEpochs(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, value1, txn1); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, txn1e2.ReadTimestamp, value2, txn1e2); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, txn1e2.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1e2); err != nil { t.Fatal(err) } num, _, err := MVCCResolveWriteIntentRange(ctx, engine, nil, @@ -3685,7 +3664,7 @@ func TestMVCCResolveWithUpdatedTimestamp(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, value1, txn1); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { t.Fatal(err) } @@ -3738,7 +3717,7 @@ func TestMVCCResolveWithPushedTimestamp(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, value1, txn1); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { t.Fatal(err) } value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{ @@ -3800,7 +3779,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}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } if _, err := MVCCResolveWriteIntent(ctx, engine, nil, @@ -3809,7 +3788,7 @@ func TestMVCCResolveTxnNoOps(t *testing.T) { } // Write intent and resolve with different txn. - if err := MVCCPut(ctx, engine, nil, testKey2, txn1.ReadTimestamp, value2, txn1); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1); err != nil { t.Fatal(err) } @@ -3833,16 +3812,16 @@ func TestMVCCResolveTxnRange(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, value1, txn1); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{Logical: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, txn2.ReadTimestamp, value3, txn2); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, txn2.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn2); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, txn1.ReadTimestamp, value4, txn1); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value4, txn1); err != nil { t.Fatal(err) } @@ -3923,14 +3902,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, value1, txn1); err != nil { + if err := MVCCPut(ctx, engine, nil, key0, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { t.Fatal(err) } txn2ts := makeTxn(*txn2, hlc.Timestamp{Logical: 2}) - if err := MVCCPut(ctx, engine, nil, key1, txn2ts.ReadTimestamp, value2, txn2ts); err != nil { + if err := MVCCPut(ctx, engine, nil, key1, txn2ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn2ts); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, key2, hlc.Timestamp{Logical: 3}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, key2, hlc.Timestamp{Logical: 3}, hlc.ClockTimestamp{}, value3, nil); err != nil { t.Fatal(err) } } @@ -4049,7 +4028,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, p.values[i], txn)) + require.NoError(t, MVCCPut(ctx, eng, nil, p.key, txn.ReadTimestamp, hlc.ClockTimestamp{}, p.values[i], txn)) } } } @@ -4447,7 +4426,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}, val, nil); err != nil { + if err := MVCCPut(ctx, engine, ms, []byte(k), hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, val, nil); err != nil { t.Fatal(err) } } @@ -4752,7 +4731,7 @@ func TestFindValidSplitKeys(t *testing.T) { // between MVCC versions, so this shouldn't have any effect. for j := 1; j <= 3; j++ { ts := hlc.Timestamp{Logical: int32(j)} - if err := MVCCPut(ctx, engine, ms, []byte(k), ts, val, nil); err != nil { + if err := MVCCPut(ctx, engine, ms, []byte(k), ts, hlc.ClockTimestamp{}, val, nil); err != nil { t.Fatal(err) } } @@ -4855,7 +4834,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}, val, nil); err != nil { + if err := MVCCPut(ctx, engine, ms, key, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, val, nil); err != nil { t.Fatal(err) } } @@ -4916,14 +4895,14 @@ 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, nil); err != nil { + if err := MVCCDelete(ctx, engine, ms, test.key, val.Timestamp, hlc.ClockTimestamp{}, nil); 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, valCpy, nil); err != nil { + if err := MVCCPut(ctx, engine, ms, test.key, val.Timestamp, hlc.ClockTimestamp{}, valCpy, nil); err != nil { t.Fatal(err) } } @@ -5022,7 +5001,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, valCpy, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, test.key, val.Timestamp, hlc.ClockTimestamp{}, valCpy, nil); err != nil { t.Fatal(err) } } @@ -5056,7 +5035,7 @@ func TestMVCCGarbageCollectIntent(t *testing.T) { key := roachpb.Key("a") { val1 := roachpb.MakeValueFromBytes(bytes) - if err := MVCCPut(ctx, engine, nil, key, ts1, val1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, key, ts1, hlc.ClockTimestamp{}, val1, nil); err != nil { t.Fatal(err) } } @@ -5064,7 +5043,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, txn); err != nil { + if err := MVCCDelete(ctx, engine, nil, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { t.Fatal(err) } keys := []roachpb.GCRequest_GCKey{ @@ -5168,9 +5147,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, val, nil, - ); err != nil { + if err := MVCCPut(ctx, engine, ms, roachpb.Key(key.key), ts, hlc.ClockTimestamp{}, val, nil); err != nil { t.Fatal(err) } } @@ -5314,7 +5291,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, value1, txn1e2); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1e2.ReadTimestamp, hlc.ClockTimestamp{}, value1, 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 0a61de339f31..597eb9a21c7a 100644 --- a/pkg/storage/pebble_mvcc_scanner_test.go +++ b/pkg/storage/pebble_mvcc_scanner_test.go @@ -211,7 +211,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, val, &txn1)) + require.NoError(t, MVCCPut(context.Background(), batch, nil, key, ts1, hlc.ClockTimestamp{}, val, &txn1)) } require.NoError(t, batch.Commit(true)) }() diff --git a/pkg/storage/pebble_test.go b/pkg/storage/pebble_test.go index 44409107043d..c1e95b8a93d7 100644 --- a/pkg/storage/pebble_test.go +++ b/pkg/storage/pebble_test.go @@ -664,7 +664,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, val.value, val.txn); err != nil { + if err := MVCCPut(ctx, batch, nil, val.key, val.timestamp, hlc.ClockTimestamp{}, val.value, val.txn); err != nil { return err } } diff --git a/pkg/storage/sst_test.go b/pkg/storage/sst_test.go index d05f0f250742..fc467713ce3f 100644 --- a/pkg/storage/sst_test.go +++ b/pkg/storage/sst_test.go @@ -74,7 +74,7 @@ func TestCheckSSTConflictsMaxIntents(t *testing.T) { require.NoError(t, batch.PutMVCC(MVCCKey{Key: roachpb.Key(key), Timestamp: txn1TS}, []byte("value"))) } for _, key := range intents { - require.NoError(t, MVCCPut(ctx, batch, nil, roachpb.Key(key), txn1TS, roachpb.MakeValueFromString("intent"), txn1)) + require.NoError(t, MVCCPut(ctx, batch, nil, roachpb.Key(key), txn1TS, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("intent"), txn1)) } require.NoError(t, batch.Commit(true)) batch.Close()