Skip to content

Commit

Permalink
kv: plumb local timestamp into MVCC
Browse files Browse the repository at this point in the history
Unused plumbing in preparation for a later commit.
  • Loading branch information
nvanbenschoten committed Apr 15, 2022
1 parent 9c403d8 commit c86f30b
Show file tree
Hide file tree
Showing 72 changed files with 561 additions and 511 deletions.
2 changes: 1 addition & 1 deletion pkg/ccl/storageccl/engineccl/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
}
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/storageccl/engineccl/encrypted_fs_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -399,6 +399,7 @@ func TestPebbleEncryption2(t *testing.T) {
nil, /* ms */
roachpb.Key(key),
hlc.Timestamp{},
hlc.ClockTimestamp{},
roachpb.MakeValueFromBytes([]byte(val)),
nil, /* txn */
)
Expand Down
7 changes: 3 additions & 4 deletions pkg/cli/debug.go
Original file line number Diff line number Diff line change
Expand Up @@ -1303,7 +1303,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)
Expand Down Expand Up @@ -1351,7 +1351,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{
Expand All @@ -1363,8 +1363,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 {
Expand Down
4 changes: 1 addition & 3 deletions pkg/kv/kvclient/kvcoord/dist_sender_server_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}

Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/abortspan/abortspan.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand All @@ -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
Expand Down Expand Up @@ -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")
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/batch_spanset_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/cmd_clear_range_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
}
Expand Down
6 changes: 4 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_conditional_put.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/cmd_delete.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_delete_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down
8 changes: 4 additions & 4 deletions pkg/kv/kvserver/batcheval/cmd_end_transaction.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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.
Expand Down Expand Up @@ -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")
}

Expand Down
8 changes: 4 additions & 4 deletions pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
}
Expand Down Expand Up @@ -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)
}

Expand All @@ -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)
}
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_export_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}

Expand All @@ -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)
}
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/cmd_heartbeat_txn.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
}
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_increment.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
6 changes: 4 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_init_put.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/cmd_push_txn.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_put.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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.)
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/cmd_recover_txn.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}

Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_recover_txn_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}

Expand Down Expand Up @@ -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)
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/cmd_refresh_range_bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
}
Expand Down
10 changes: 5 additions & 5 deletions pkg/kv/kvserver/batcheval/cmd_refresh_range_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand All @@ -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 {
Expand Down Expand Up @@ -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)
}

Expand Down Expand Up @@ -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
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_refresh_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}

Expand Down Expand Up @@ -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
Expand Down
Loading

0 comments on commit c86f30b

Please sign in to comment.