Skip to content

Commit

Permalink
kv/storage: introduce and track local timestamps for MVCC versions
Browse files Browse the repository at this point in the history
Fixes cockroachdb#36431.
Fixes cockroachdb#49360.
Replaces cockroachdb#72121.

This commit fixes the potential for a stale read as detailed in cockroachdb#36431 using the
"remember when intents were written" approach described in
cockroachdb#36431 (comment) and
later expanded on in
cockroachdb#72121 (comment).

This bug requires a combination of skewed clocks, multi-key transactions split
across ranges whose leaseholders are stored on different nodes, a transaction
read refresh, and the use of observed timestamps to avoid an uncertainty
restart. With the combination of these four factors, it was possible to
construct an ordering of events that violated real-time ordering and allowed a
transaction to observe a stale read. Upon the discovery of the bug, we
[introduced](cockroachdb/jepsen#19) the `multi-register`
test to the Jepsen test suite, and have since observed the test fail when
combined with the `strobe-skews` nemesis due to this bug in cockroachdb#49360 (and a few
issues linked to that one). This commit stabilizes that test.

\### Explanation

The combination of all of the factors listed above can lead to the stale read
because it breaks one of the invariants that the observed timestamp
infrastructure[^1] relied upon for correctness. Specifically, observed
timestamps relied on the guarantee that a leaseholder's clock must always be
equal to or greater than the version timestamp of all writes that it has served.
However, this guarantee did not always hold. It does hold for non-transactional
writes. It also holds for transactions that perform all of their intent writes
at the same timestamp and then commit at this timestamp. However, it does not
hold for transactions which move their commit timestamp forward over their
lifetime before committing, writing intents at different timestamps along the
way and "pulling them up" to the commit timestamp after committing.

In violating the invariant, this third case reveals an ambiguity in what it
means for a leaseholder to "serve a write at a timestamp". The meaning of this
phrase is straightforward for non-transactional writes. However, for an intent
write whose original timestamp is provisional and whose eventual commit
timestamp is stored indirectly in its transaction record at its time of commit,
the meaning is less clear. This reconciliation to move the intent write's
timestamp up to its transaction's commit timestamp is asynchronous from the
transaction commit (and after it has been externally acknowledged). So even if a
leaseholder has only served writes with provisional timestamps up to timestamp
100 (placing a lower bound on its clock of 100), it can be in possession of
intents that, when resolved, will carry a timestamp of 200. To uphold the
real-time ordering property, this value must be observed by any transaction that
begins after the value's transaction committed and was acknowledged. So for
observed timestamps to be correct as currently written, we would need a
guarantee that this value's leaseholder would never return an observed timestamp
< 200 at any point after the transaction commits. But with the transaction
commit possibly occurring on another node and with communication to resolve the
intent occurring asynchronously, this seems like an impossible guarantee to
make.

This would appear to undermine observed timestamps to the point where they
cannot be used. However, we can claw back correctness without sacrificing
performance by recognizing that only a small fraction[^2] of transactions commit
at a different timestamps than the one they used while writing intents. We can
also recognize that if we were to compare observed timestamps against the
timestamp that a committed value was originally written (its provisional value
if it was once an intent) instead of the timestamp that it had been moved to on
commit, then the invariant would hold.

This commit exploits this second observation by adding a second timestamp to
each MVCC key called the "local timestamp". The MVCC key's existing version
timestamp dictates the key's visibility to readers and is tied to the writer's
commit timestamp. The local clock timestamp records the value of the local HLC
clock on the leaseholder when the key was originally written. It is used to make
claims about the relative real time ordering of the key's writer and readers
when comparing a reader's uncertainty interval (and observed timestamps) to the
key. Ignoring edge cases, readers with an observed timestamp from the key's
leaseholder that is greater than the local clock timestamp stored in the key
cannot make claims about real time ordering and must consider it possible that
the key's write occurred before the read began. However, readers with an
observed timestamp from the key's leaseholder that is less than the clock
timestamp can claim that the reader captured that observed timestamp before the
key was written and therefore can consider the key's write to have been
concurrent with the read. In doing so, the reader can avoid an uncertainty
restart.

For more, see the updates made in this commit to pkg/kv/kvserver/observedts/doc.go.

To avoid the bulk of the performance hit from adding this new timestamp to the
MVCC key encoding, the commit optimizes the clock timestamp away in the common
case where it leads the version timestamp. Only in the rare cases where the
local timestamp trails the version timestamp (e.g. future-time writes, async
intent resolution with a new commit timestamp) does the local timestamp need to
be explicitly represented in the key encoding. This is possible because it is
safe for the local clock timestamp to be rounded down, as this will simply lead
to additional uncertainty restarts. However, it is not safe for the local clock
timestamp to be rounded up, as this could lead to stale reads.

\### Future improvements

As noted in cockroachdb#72121 (comment),
this commit paves a path towards the complete removal of synthetic timestamps,
which were originally introduced in support of non-blocking transactions and
GLOBAL tables.

The synthetic bit's first role of providing dynamic typing for `ClockTimestamps`
is no longer necessary now that we never need to "push" transaction-domain
timestamps into HLC clocks. Instead, the invariant that underpins observed
timestamps is enforced by "pulling" local timestamps from the leaseholder's HLC
clock.

The synthetic bit's second role of disabling observed timestamps is replaced by
the generalization provided by "local timestamps". Local timestamps precisely
track when an MVCC version was written in the leaseholder's clock timestamp
domain. This establishes a total ordering across clock observations (local
timestamp assignment for writers and observed timestamps for readers) and
establish a partial ordering between writer and reader transactions. As a
result, the use of observed timestamps during uncertainty checking becomes a
comparison between two `ClockTimestamps`, the version's local timestamp and the
reader's observed timestamp.

\### Correctness testing

I was not able to stress `jepsen/multi-register/strobe-skews` hard enough to
cause it to fail, even on master. We've only seen the test fail a handful of
times over the past few years, so this isn't much of a surprise. Still, this
prevents us from saying anything concrete about an reduced failure rate.

However, the commit does add a new test called
`TestTxnReadWithinUncertaintyIntervalAfterIntentResolution` which controls
manual clocks directly and was able to deterministically reproduce the stale
read before this fix in a few different ways. After this fix, the test passes.

\### Performance analysis

This correctness fix will lead to an increased rate of transaction retries under
some workloads.

TODO(nvanbenschoten):
- observe TPC-C performance
-- top-line performance
-- uncertainty retry rate
-- commit-wait rate (should be zero)
- compare YCSB performance

----

Release note (bug fix): fixed a rare race condition that could allow for a
transaction to serve a stale read and violate real-time ordering under moderate
clock skew.

[^1]: see [pkg/kv/kvserver/observedts/doc.go](https://github.com/cockroachdb/cockroach/blob/master/pkg/kv/kvserver/observedts/doc.go)
for an explanation of the role of observed timestamps in the transaction model.
This commit updates that documentation to include this fix.

[^2]: see analysis in cockroachdb#36431 (comment).
  • Loading branch information
nvanbenschoten committed Apr 15, 2022
1 parent c86f30b commit b40e44b
Show file tree
Hide file tree
Showing 51 changed files with 5,124 additions and 721 deletions.
5 changes: 3 additions & 2 deletions pkg/ccl/backupccl/restore_data_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -475,9 +475,10 @@ func (rd *restoreDataProcessor) processRestoreSpanEntry(
continue
}

keyScratch = append(keyScratch[:0], iter.UnsafeKey().Key...)
key := iter.UnsafeKey()
keyScratch = append(keyScratch[:0], key.Key...)
key.Key = keyScratch
valueScratch = append(valueScratch[:0], iter.UnsafeValue()...)
key := storage.MVCCKey{Key: keyScratch, Timestamp: iter.UnsafeKey().Timestamp}
value := roachpb.Value{RawBytes: valueScratch}
iter.NextKey()

Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/backupccl/restore_data_processor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -390,6 +390,7 @@ func runTestIngest(t *testing.T, init func(*cluster.Settings)) {
for i := range kvs {
if i < len(expectedKVs) {
expectedKVs[i].Key.Timestamp = kvs[i].Key.Timestamp
expectedKVs[i].Key.LocalTimestamp = hlc.ClockTimestamp{}
}
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/kvfeed/scanner.go
Original file line number Diff line number Diff line change
Expand Up @@ -236,7 +236,7 @@ func slurpScanResponse(
for len(br) > 0 {
var kv roachpb.KeyValue
var err error
kv.Key, kv.Value.Timestamp, kv.Value.RawBytes, br, err = enginepb.ScanDecodeKeyValue(br)
kv.Key, kv.Value.Timestamp, _, kv.Value.RawBytes, br, err = enginepb.ScanDecodeKeyValue(br)
if err != nil {
return errors.Wrapf(err, `decoding changes for %s`, span)
}
Expand Down
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/tests/acceptance.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ func registerAcceptance(r registry.Registry) {
},
{
name: "version-upgrade",
skip: "WIP: unskip when version checks are added to local_timestamp writes",
fn: func(ctx context.Context, t test.Test, c cluster.Cluster) {
runVersionUpgrade(ctx, t, c)
},
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/bulk/sst_batcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -201,6 +201,7 @@ func (b *SSTBatcher) AddMVCCKey(ctx context.Context, key storage.MVCCKey, value
b.batchTS = b.db.Clock().Now()
}
key.Timestamp = b.batchTS
key.LocalTimestamp = hlc.ClockTimestamp{}
}

// Update the range currently represented in this batch, as necessary.
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1466,7 +1466,8 @@ func TestAddSSTableSSTTimestampToRequestTimestampRespectsClosedTS(t *testing.T)
kvs, err := storage.Scan(store.Engine(), roachpb.Key("key"), roachpb.Key("key").Next(), 0)
require.NoError(t, err)
require.Len(t, kvs, 1)
require.Equal(t, storage.MVCCKey{Key: roachpb.Key("key"), Timestamp: writeTS}, kvs[0].Key)
expKey := storage.MVCCKey{Key: roachpb.Key("key"), Timestamp: writeTS}.Normalize()
require.Equal(t, expKey, kvs[0].Key)
v, err := roachpb.Value{RawBytes: kvs[0].Value}.GetBytes()
require.NoError(t, err)
require.Equal(t, "sst", string(v))
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/below_raft_protos_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -64,14 +64,15 @@ var belowRaftGoldenProtos = map[reflect.Type]fixture{
m := enginepb.NewPopulatedMVCCMetadata(r, false)
m.Txn = nil // never populated below Raft
m.Timestamp.Synthetic = nil // never populated below Raft
m.LocalTimestamp = nil // never populated below Raft
if m.MergeTimestamp != nil {
m.MergeTimestamp.Synthetic = nil // never populated below Raft
}
m.TxnDidNotUpdateMeta = nil // never populated below Raft
return m
},
emptySum: 7551962144604783939,
populatedSum: 6170112718709472849,
populatedSum: 12812489297533931627,
},
reflect.TypeOf(&enginepb.RangeAppliedState{}): {
populatedConstructor: func(r *rand.Rand) protoutil.Message {
Expand Down
247 changes: 247 additions & 0 deletions pkg/kv/kvserver/client_replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -492,6 +492,253 @@ func TestTxnReadWithinUncertaintyInterval(t *testing.T) {
})
}

// TestTxnReadWithinUncertaintyIntervalAfterIntentResolution tests cases where a
// reader transaction observes a committed value that was committed before the
// reader began, but that was resolved after the reader began. The test ensures
// that even if the reader has collected an observed timestamp from the node
// that holds the intent, and even if this observed timestamp is less than the
// timestamp that the intent is eventually committed at, the reader still
// considers the value to be in its uncertainty interval. Not doing so could
// allow for stale read, which would be a violation of linearizability.
//
// This is a regression test for #36431. Before this issue was addressed,
// it was possible for the following series of events to lead to a stale
// read:
// - txn W is coordinated by node B. It lays down an intent on node A (key k) at
// ts 95.
// - txn W gets pushed to ts 105 (taken from B's clock). It refreshes
// successfully and commits at 105. Node A's clock is at, say, 100; this is
// within clock offset bounds.
// - after all this, txn R starts on node A. It gets assigned ts 100. The txn
// has no uncertainty for node A.
// - txn W's async intent resolution comes around and resolves the intent on
// node A, moving the value fwd from ts 95 to 105.
// - txn R reads key k and doesn't see anything. There's a value at 105, but the
// txn have no uncertainty due to an observed timestamp. This is a stale read.
//
// The test's rangedResolution parameter dictates whether the intent is
// asynchronously resolved using point or ranged intent resolution.
//
// The test's movedWhilePending parameter dictates whether the intent is moved
// to a higher timestamp first by a PENDING intent resolution and then COMMITTED
// at that same timestamp, or whether it is moved to a higher timestamp at the
// same time as it is COMMITTED.
//
// The test's alreadyResolved parameter dictates whether the intent is
// already resolved by the time the reader observes it, or whether the
// reader must resolve the intent itself.
//
func TestTxnReadWithinUncertaintyIntervalAfterIntentResolution(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

testutils.RunTrueAndFalse(t, "rangedResolution", func(t *testing.T, rangedResolution bool) {
testutils.RunTrueAndFalse(t, "movedWhilePending", func(t *testing.T, movedWhilePending bool) {
testutils.RunTrueAndFalse(t, "alreadyResolved", func(t *testing.T, alreadyResolved bool) {
testTxnReadWithinUncertaintyIntervalAfterIntentResolution(
t, rangedResolution, movedWhilePending, alreadyResolved,
)
})
})
})
}

func testTxnReadWithinUncertaintyIntervalAfterIntentResolution(
t *testing.T, rangedResolution, movedWhilePending, alreadyResolved bool,
) {
const numNodes = 2
var manuals []*hlc.HybridManualClock
var clocks []*hlc.Clock
for i := 0; i < numNodes; i++ {
manuals = append(manuals, hlc.NewHybridManualClock())
}
serverArgs := make(map[int]base.TestServerArgs)
for i := 0; i < numNodes; i++ {
serverArgs[i] = base.TestServerArgs{
Knobs: base.TestingKnobs{
Server: &server.TestingKnobs{
ClockSource: manuals[i].UnixNano,
},
Store: &kvserver.StoreTestingKnobs{
IntentResolverKnobs: kvserverbase.IntentResolverTestingKnobs{
// Disable async intent resolution, so that the test can carefully
// control when intent resolution occurs.
DisableAsyncIntentResolution: true,
},
},
},
}
}
ctx := context.Background()
tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ServerArgsPerNode: serverArgs,
})
defer tc.Stopper().Stop(ctx)

// Split off two scratch ranges.
keyA, keyB := roachpb.Key("a"), roachpb.Key("b")
tc.SplitRangeOrFatal(t, keyA)
_, keyBDesc := tc.SplitRangeOrFatal(t, keyB)
// Place key A's sole replica on node 1 and key B's sole replica on node 2.
tc.AddVotersOrFatal(t, keyB, tc.Target(1))
tc.TransferRangeLeaseOrFatal(t, keyBDesc, tc.Target(1))
tc.RemoveVotersOrFatal(t, keyB, tc.Target(0))

// Pause the servers' clocks going forward.
var maxNanos int64
for i, m := range manuals {
m.Pause()
if cur := m.UnixNano(); cur > maxNanos {
maxNanos = cur
}
clocks = append(clocks, tc.Servers[i].Clock())
}
// After doing so, perfectly synchronize them.
for _, m := range manuals {
m.Increment(maxNanos - m.UnixNano())
}

// Create a new writer transaction.
maxOffset := clocks[0].MaxOffset().Nanoseconds()
require.NotZero(t, maxOffset)
writerTxn := roachpb.MakeTransaction("test_writer", keyA, 1, clocks[0].Now(), maxOffset, int32(tc.Servers[0].NodeID()))

// Write to key A and key B in the writer transaction.
for _, key := range []roachpb.Key{keyA, keyB} {
put := putArgs(key, []byte("val"))
resp, pErr := kv.SendWrappedWith(ctx, tc.Servers[0].DistSender(), roachpb.Header{Txn: &writerTxn}, put)
require.Nil(t, pErr)
writerTxn.Update(resp.Header().Txn)
}

// Move the clock on just the first server and bump the transaction commit
// timestamp to this value. The clock on the second server will trail behind.
manuals[0].Increment(100)
require.True(t, writerTxn.WriteTimestamp.Forward(clocks[0].Now()))

// Refresh the writer transaction's timestamp.
writerTxn.ReadTimestamp.Forward(writerTxn.WriteTimestamp)

// Commit the writer transaction. Key A will be synchronously resolved because
// it is on the same range as the transaction record. However, key B will be
// handed to the IntentResolver for asynchronous resolution. Because we
// disabled async resolution, it will not be resolved yet.
et, etH := endTxnArgs(&writerTxn, true /* commit */)
et.LockSpans = []roachpb.Span{
{Key: keyA}, {Key: keyB},
}
if rangedResolution {
for i := range et.LockSpans {
et.LockSpans[i].EndKey = et.LockSpans[i].Key.Next()
}
}
etResp, pErr := kv.SendWrappedWith(ctx, tc.Servers[0].DistSender(), etH, et)
require.Nil(t, pErr)
writerTxn.Update(etResp.Header().Txn)

// Create a new reader transaction. The reader uses the second server as a
// gateway, so its initial read timestamp actually trails the commit timestamp
// of the writer transaction due to clock skew between the two servers. This
// is the classic case where the reader's uncertainty interval is needed to
// avoid stale reads. Remember that the reader transaction began after the
// writer transaction committed and received an ack, so it must observe the
// writer's writes if it is to respect real-time ordering.
//
// NB: we use writerTxn.MinTimestamp instead of clocks[1].Now() so that a
// stray clock update doesn't influence the reader's read timestamp.
readerTxn := roachpb.MakeTransaction("test_reader", keyA, 1, writerTxn.MinTimestamp, maxOffset, int32(tc.Servers[1].NodeID()))
require.True(t, readerTxn.ReadTimestamp.Less(writerTxn.WriteTimestamp))
require.False(t, readerTxn.GlobalUncertaintyLimit.Less(writerTxn.WriteTimestamp))

// Collect an observed timestamp from each of the nodes. We read the key
// following (Key.Next) each of the written keys to avoid conflicting with
// read values. We read keyB first to avoid advancing the clock on node 2
// before we collect an observed timestamp from it.
//
// NOTE: this wasn't even a necessary step to hit #36431, because new
// transactions are always an observed timestamp from their own gateway node.
for i, key := range []roachpb.Key{keyB, keyA} {
get := getArgs(key.Next())
resp, pErr := kv.SendWrappedWith(ctx, tc.Servers[1].DistSender(), roachpb.Header{Txn: &readerTxn}, get)
require.Nil(t, pErr)
require.Nil(t, resp.(*roachpb.GetResponse).Value)
readerTxn.Update(resp.Header().Txn)
require.Len(t, readerTxn.ObservedTimestamps, i+1)
}

// Resolve the intent on key B zero, one, or two times.
{
resolveIntentArgs := func(status roachpb.TransactionStatus) roachpb.Request {
if rangedResolution {
return &roachpb.ResolveIntentRangeRequest{
RequestHeader: roachpb.RequestHeader{Key: keyB, EndKey: keyB.Next()},
IntentTxn: writerTxn.TxnMeta,
Status: status,
}
} else {
return &roachpb.ResolveIntentRequest{
RequestHeader: roachpb.RequestHeader{Key: keyB},
IntentTxn: writerTxn.TxnMeta,
Status: status,
}
}
}

if movedWhilePending {
// First change the intent's timestamp without committing it. This
// exercises the case where the intent's timestamp is moved forward by a
// PENDING intent resolution request and kept the same when the intent is
// eventually COMMITTED. This PENDING intent resolution may still be
// evaluated after the transaction commit has been acknowledged in
// real-time, so it still needs to lead to the committed value retaining
// its original local timestamp.
//
// For instance, consider the following timeline:
//
// 1. txn W writes intent on key A @ time 10
// 2. txn W writes intent on key B @ time 10
// 3. high priority reader @ 15 reads key B
// 4. high priority reader pushes txn W to time 15
// 5. txn W commits @ 15 and resolves key A synchronously
// 6. txn R begins and collects observed timestamp from key B's node @
// time 11
// 7. high priority reader moves intent on key B to time 15
// 8. async intent resolution commits intent on key B, still @ time 15
// 9. txn R reads key B with read ts 11, observed ts 11, and uncertainty
// interval [11, 21]. If step 7 updated the intent's local timestamp
// to the current time when changing its version timestamp, txn R
// could use its observed timestamp to avoid an uncertainty error,
// leading to a stale read.
//
resolve := resolveIntentArgs(roachpb.PENDING)
_, pErr = kv.SendWrapped(ctx, tc.Servers[0].DistSender(), resolve)
require.Nil(t, pErr)
}

if alreadyResolved {
// Resolve the committed value on key B to COMMITTED.
resolve := resolveIntentArgs(roachpb.COMMITTED)
_, pErr = kv.SendWrapped(ctx, tc.Servers[0].DistSender(), resolve)
require.Nil(t, pErr)
}
}

// Read key A and B in the reader transaction. Both should produce
// ReadWithinUncertaintyIntervalErrors.
for _, key := range []roachpb.Key{keyA, keyB} {
get := getArgs(key)
_, pErr := kv.SendWrappedWith(ctx, tc.Servers[0].DistSender(), roachpb.Header{Txn: &readerTxn}, get)
require.NotNil(t, pErr)
var rwuiErr *roachpb.ReadWithinUncertaintyIntervalError
require.True(t, errors.As(pErr.GetDetail(), &rwuiErr))
require.Equal(t, readerTxn.ReadTimestamp, rwuiErr.ReadTimestamp)
require.Equal(t, readerTxn.GlobalUncertaintyLimit, rwuiErr.GlobalUncertaintyLimit)
require.Equal(t, readerTxn.ObservedTimestamps, rwuiErr.ObservedTimestamps)
require.Equal(t, writerTxn.WriteTimestamp, rwuiErr.ExistingTimestamp)
}
}

// TestTxnReadWithinUncertaintyIntervalAfterLeaseTransfer tests a case where a
// transaction observes a committed value in its uncertainty interval that was
// written under a previous leaseholder. In the test, the transaction does
Expand Down
9 changes: 9 additions & 0 deletions pkg/kv/kvserver/client_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -102,6 +102,15 @@ func heartbeatArgs(
}, roachpb.Header{Txn: txn}
}

func endTxnArgs(txn *roachpb.Transaction, commit bool) (*roachpb.EndTxnRequest, roachpb.Header) {
return &roachpb.EndTxnRequest{
RequestHeader: roachpb.RequestHeader{
Key: txn.Key, // not allowed when going through TxnCoordSender, but we're not
},
Commit: commit,
}, roachpb.Header{Txn: txn}
}

func pushTxnArgs(
pusher, pushee *roachpb.Transaction, pushType roachpb.PushTxnType,
) *roachpb.PushTxnRequest {
Expand Down
2 changes: 2 additions & 0 deletions pkg/kv/kvserver/consistency_queue_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -279,13 +279,15 @@ func TestCheckConsistencyInconsistent(t *testing.T) {

// mock this out for a consistent string below.
diff[0].Timestamp = hlc.Timestamp{Logical: 987, WallTime: 123}
diff[0].LocalTimestamp = hlc.ClockTimestamp{Logical: 678, WallTime: 234}

act := diff.String()

exp := `--- leaseholder
+++ follower
+0.000000123,987 "e"
+ ts:1970-01-01 00:00:00.000000123 +0000 UTC
+ localTs:0.000000234,678
+ value:"\x00\x00\x00\x00\x01T"
+ raw mvcc_key/value: 6500000000000000007b000003db0d 000000000154
`
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/gc/gc_iterator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ func TestGCIterator(t *testing.T) {
Key: storage.MVCCKey{
Key: k,
Timestamp: hlc.Timestamp{WallTime: ts * time.Nanosecond.Nanoseconds()},
},
}.Normalize(),
Value: val,
},
txn: txn,
Expand Down
6 changes: 4 additions & 2 deletions pkg/kv/kvserver/replica_consistency.go
Original file line number Diff line number Diff line change
Expand Up @@ -607,7 +607,8 @@ func (*Replica) sha512(
if snapshot != nil {
// Add (a copy of) the kv pair into the debug message.
kv := roachpb.RaftSnapshotData_KeyValue{
Timestamp: unsafeKey.Timestamp,
Timestamp: unsafeKey.Timestamp,
LocalTimestamp: unsafeKey.LocalTimestamp,
}
alloc, kv.Key = alloc.Copy(unsafeKey.Key, 0)
alloc, kv.Value = alloc.Copy(unsafeValue, 0)
Expand Down Expand Up @@ -683,7 +684,8 @@ func (*Replica) sha512(
if snapshot != nil {
// Add LeaseAppliedState to the diff.
kv := roachpb.RaftSnapshotData_KeyValue{
Timestamp: hlc.Timestamp{},
Timestamp: hlc.Timestamp{},
LocalTimestamp: hlc.ClockTimestamp{},
}
kv.Key = keys.RangeAppliedStateKey(desc.RangeID)
var v roachpb.Value
Expand Down
Loading

0 comments on commit b40e44b

Please sign in to comment.