diff --git a/pkg/kv/kvserver/replica_consistency.go b/pkg/kv/kvserver/replica_consistency.go index 861696ab2edb..ce07f0b122dd 100644 --- a/pkg/kv/kvserver/replica_consistency.go +++ b/pkg/kv/kvserver/replica_consistency.go @@ -598,9 +598,8 @@ func (*Replica) sha512( var timestampBuf []byte hasher := sha512.New() - // TODO(erikgrinaker): add a range key visitor to hash range keys. pointKeyVisitor := func(unsafeKey storage.MVCCKey, unsafeValue []byte) error { - // Rate Limit the scan through the range + // Rate limit the scan through the range. if err := limiter.WaitN(ctx, int64(len(unsafeKey.Key)+len(unsafeValue))); err != nil { return err } @@ -643,6 +642,60 @@ func (*Replica) sha512( return err } + rangeKeyVisitor := func(rangeKV storage.MVCCRangeKeyValue) error { + // Rate limit the scan through the range. + err := limiter.WaitN(ctx, + int64(len(rangeKV.RangeKey.StartKey)+len(rangeKV.RangeKey.EndKey)+len(rangeKV.Value))) + if err != nil { + return err + } + + if snapshot != nil { + // Add (a copy of) the range key into the debug message. + rkv := roachpb.RaftSnapshotData_RangeKeyValue{ + Timestamp: rangeKV.RangeKey.Timestamp, + } + alloc, rkv.StartKey = alloc.Copy(rangeKV.RangeKey.StartKey, 0) + alloc, rkv.EndKey = alloc.Copy(rangeKV.RangeKey.EndKey, 0) + alloc, rkv.Value = alloc.Copy(rangeKV.Value, 0) + snapshot.RangeKV = append(snapshot.RangeKV, rkv) + } + + // Encode the length of the start key and end key. + binary.LittleEndian.PutUint64(intBuf[:], uint64(len(rangeKV.RangeKey.StartKey))) + if _, err := hasher.Write(intBuf[:]); err != nil { + return err + } + binary.LittleEndian.PutUint64(intBuf[:], uint64(len(rangeKV.RangeKey.EndKey))) + if _, err := hasher.Write(intBuf[:]); err != nil { + return err + } + binary.LittleEndian.PutUint64(intBuf[:], uint64(len(rangeKV.Value))) + if _, err := hasher.Write(intBuf[:]); err != nil { + return err + } + if _, err := hasher.Write(rangeKV.RangeKey.StartKey); err != nil { + return err + } + if _, err := hasher.Write(rangeKV.RangeKey.EndKey); err != nil { + return err + } + legacyTimestamp = rangeKV.RangeKey.Timestamp.ToLegacyTimestamp() + if size := legacyTimestamp.Size(); size > cap(timestampBuf) { + timestampBuf = make([]byte, size) + } else { + timestampBuf = timestampBuf[:size] + } + if _, err := protoutil.MarshalTo(&legacyTimestamp, timestampBuf); err != nil { + return err + } + if _, err := hasher.Write(timestampBuf); err != nil { + return err + } + _, err = hasher.Write(rangeKV.Value) + return err + } + var ms enginepb.MVCCStats // In statsOnly mode, we hash only the RangeAppliedState. In regular mode, hash // all of the replicated key space. @@ -660,7 +713,7 @@ func (*Replica) sha512( UpperBound: span.End, }) spanMS, err := storage.ComputeStatsForRangeWithVisitors( - iter, span.Start, span.End, 0 /* nowNanos */, pointKeyVisitor, nil /* rangeKeyVisitor */) + iter, span.Start, span.End, 0 /* nowNanos */, pointKeyVisitor, rangeKeyVisitor) iter.Close() if err != nil { return nil, err diff --git a/pkg/kv/kvserver/replica_consistency_diff.go b/pkg/kv/kvserver/replica_consistency_diff.go index 8d132fa31e37..92d9e8989934 100644 --- a/pkg/kv/kvserver/replica_consistency_diff.go +++ b/pkg/kv/kvserver/replica_consistency_diff.go @@ -12,6 +12,7 @@ package kvserver import ( "bytes" + fmt "fmt" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" @@ -26,6 +27,7 @@ type ReplicaSnapshotDiff struct { // holder. LeaseHolder bool Key roachpb.Key + EndKey roachpb.Key // only set for range keys (MVCCRangeKey) Timestamp hlc.Timestamp Value []byte } @@ -43,17 +45,33 @@ func (rsds ReplicaSnapshotDiffSlice) SafeFormat(buf redact.SafePrinter, _ rune) // Lease holder (RHS) has something follower (LHS) does not have. prefix = redact.SafeString("-") } - const format = `%s%s %s + if len(d.EndKey) > 0 { + const rangeKVFormat = `%s%s %s +%s ts:%s +%s value:%s +%s raw from/to/ts/value: %x %x %x %x +` + buf.Printf(rangeKVFormat, + prefix, d.Timestamp, roachpb.Span{Key: d.Key, EndKey: d.EndKey}, + prefix, d.Timestamp.GoTime(), + prefix, fmt.Sprintf("%q", d.Value), // just print the raw value for now + prefix, storage.EncodeMVCCKeyPrefix(d.Key), storage.EncodeMVCCKeyPrefix(d.EndKey), + storage.EncodeMVCCTimestampSuffix(d.Timestamp), d.Value) + + } else { + const kvFormat = `%s%s %s %s ts:%s %s value:%s %s raw mvcc_key/value: %x %x ` - mvccKey := storage.MVCCKey{Key: d.Key, Timestamp: d.Timestamp} - buf.Printf(format, - prefix, d.Timestamp, d.Key, - prefix, d.Timestamp.GoTime(), - prefix, SprintMVCCKeyValue(storage.MVCCKeyValue{Key: mvccKey, Value: d.Value}, false /* printKey */), - prefix, storage.EncodeMVCCKey(mvccKey), d.Value) + mvccKey := storage.MVCCKey{Key: d.Key, Timestamp: d.Timestamp} + buf.Printf(kvFormat, + prefix, d.Timestamp, d.Key, + prefix, d.Timestamp.GoTime(), + prefix, SprintMVCCKeyValue( + storage.MVCCKeyValue{Key: mvccKey, Value: d.Value}, false /* printKey */), + prefix, storage.EncodeMVCCKey(mvccKey), d.Value) + } } } @@ -66,15 +84,22 @@ func diffRange(l, r *roachpb.RaftSnapshotData) ReplicaSnapshotDiffSlice { if l == nil || r == nil { return nil } + var diff ReplicaSnapshotDiffSlice + diff = append(diff, diffKVs(l.KV, r.KV)...) + diff = append(diff, diffRangeKeys(l.RangeKV, r.RangeKV)...) + return diff +} + +func diffKVs(l, r []roachpb.RaftSnapshotData_KeyValue) ReplicaSnapshotDiffSlice { var diff []ReplicaSnapshotDiff i, j := 0, 0 for { var e, v roachpb.RaftSnapshotData_KeyValue - if i < len(l.KV) { - e = l.KV[i] + if i < len(l) { + e = l[i] } - if j < len(r.KV) { - v = r.KV[j] + if j < len(r) { + v = r[j] } addLeaseHolder := func() { @@ -88,41 +113,28 @@ func diffRange(l, r *roachpb.RaftSnapshotData) ReplicaSnapshotDiffSlice { // Compare keys. var comp int - // Check if it has finished traversing over all the lease holder keys. - if e.Key == nil { - if v.Key == nil { - // Done traversing over all the replica keys. Done! - break - } else { - comp = 1 - } + if e.Key == nil && v.Key == nil { + // Done! + break + } else if e.Key == nil { + // Finished traversing over all the lease holder keys. + comp = 1 + } else if v.Key == nil { + // Finished traversing over all the replica keys. + comp = -1 } else { - // Check if it has finished traversing over all the replica keys. - if v.Key == nil { - comp = -1 - } else { - // Both lease holder and replica keys exist. Compare them. - comp = bytes.Compare(e.Key, v.Key) - } + // Both lease holder and replica keys exist. Compare them. + comp = storage.MVCCKey{Key: e.Key, Timestamp: e.Timestamp}.Compare( + storage.MVCCKey{Key: v.Key, Timestamp: v.Timestamp}) } + switch comp { case -1: addLeaseHolder() - + case 1: + addReplica() case 0: - // Timestamp sorting is weird. Timestamp{} sorts first, the - // remainder sort in descending order. See storage/engine/doc.go. - if !e.Timestamp.EqOrdering(v.Timestamp) { - if e.Timestamp.IsEmpty() { - addLeaseHolder() - } else if v.Timestamp.IsEmpty() { - addReplica() - } else if v.Timestamp.Less(e.Timestamp) { - addLeaseHolder() - } else { - addReplica() - } - } else if !bytes.Equal(e.Value, v.Value) { + if !bytes.Equal(e.Value, v.Value) { addLeaseHolder() addReplica() } else { @@ -130,10 +142,66 @@ func diffRange(l, r *roachpb.RaftSnapshotData) ReplicaSnapshotDiffSlice { i++ j++ } + } + } + return diff +} +func diffRangeKeys(l, r []roachpb.RaftSnapshotData_RangeKeyValue) ReplicaSnapshotDiffSlice { + var diff []ReplicaSnapshotDiff + i, j := 0, 0 + for { + var e, v roachpb.RaftSnapshotData_RangeKeyValue + if i < len(l) { + e = l[i] + } + if j < len(r) { + v = r[j] + } + + addLeaseHolder := func() { + diff = append(diff, ReplicaSnapshotDiff{LeaseHolder: true, Key: e.StartKey, EndKey: e.EndKey, + Timestamp: e.Timestamp, Value: e.Value}) + i++ + } + addReplica := func() { + diff = append(diff, ReplicaSnapshotDiff{LeaseHolder: false, Key: v.StartKey, EndKey: v.EndKey, + Timestamp: v.Timestamp, Value: v.Value}) + j++ + } + + // Compare keys. + var comp int + if e.StartKey == nil && e.EndKey == nil && v.StartKey == nil && v.EndKey == nil { + // Done! + break + } else if e.StartKey == nil && e.EndKey == nil { + // Finished traversing over all the lease holder keys. + comp = 1 + } else if v.StartKey == nil && v.EndKey == nil { + // Finished traversing over all the replica keys. + comp = -1 + } else { + // Both lease holder and replica keys exist. Compare them. + eMVCC := storage.MVCCRangeKey{StartKey: e.StartKey, EndKey: e.EndKey, Timestamp: e.Timestamp} + vMVCC := storage.MVCCRangeKey{StartKey: v.StartKey, EndKey: v.EndKey, Timestamp: v.Timestamp} + comp = eMVCC.Compare(vMVCC) + } + + switch comp { + case -1: + addLeaseHolder() case 1: addReplica() - + case 0: + if !bytes.Equal(e.Value, v.Value) { + addLeaseHolder() + addReplica() + } else { + // No diff; skip. + i++ + j++ + } } } return diff diff --git a/pkg/kv/kvserver/replica_consistency_test.go b/pkg/kv/kvserver/replica_consistency_test.go index 66afdee534ff..9c5846463698 100644 --- a/pkg/kv/kvserver/replica_consistency_test.go +++ b/pkg/kv/kvserver/replica_consistency_test.go @@ -12,15 +12,22 @@ package kvserver import ( "context" + "fmt" + "strings" "testing" "time" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/echotest" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/cockroach/pkg/util/quotapool" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/stretchr/testify/require" @@ -118,3 +125,91 @@ func TestGetChecksumNotSuccessfulExitConditions(t *testing.T) { } require.Nil(t, rc.Checksum) } + +// TestReplicaChecksumSHA512 checks that a given dataset produces the expected +// checksum. +func TestReplicaChecksumSHA512(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + storage.DisableMetamorphicSimpleValueEncoding(t) + + ctx := context.Background() + sb := &strings.Builder{} + lim := quotapool.NewRateLimiter("rate", 1e9, 0) + eng := storage.NewDefaultInMemForTesting() + defer eng.Close() + + repl := &Replica{} // We don't actually need the replica at all, just the method. + desc := roachpb.RangeDescriptor{ + RangeID: 1, + StartKey: roachpb.RKey("a"), + EndKey: roachpb.RKey("z"), + } + + // Hash the empty state. + rh, err := repl.sha512(ctx, desc, eng, nil, roachpb.ChecksumMode_CHECK_FULL, lim) + require.NoError(t, err) + fmt.Fprintf(sb, "checksum0: %x\n", rh.SHA512[:]) + + // We incrementally add writes, and check the checksums after each write to + // make sure they differ such that each write affects the checksum. + kvs := []struct { + key string + endKey string + ts int64 + localTS int64 + value string + }{ + {"a", "", 1, 0, "a1"}, + {"a", "", 2, 0, ""}, + {"b", "", 3, 2, "b3"}, + {"i", "", 0, 0, "i0"}, + // Range keys can currently only be tombstones. + {"p", "q", 1, 0, ""}, + {"x", "z", 9, 8, ""}, + } + + for i, kv := range kvs { + key, endKey := roachpb.Key(kv.key), roachpb.Key(kv.endKey) + ts := hlc.Timestamp{WallTime: kv.ts} + localTS := hlc.ClockTimestamp{WallTime: kv.localTS} + var value roachpb.Value + if kv.value != "" { + value = roachpb.MakeValueFromString(kv.value) + } + + if len(endKey) > 0 { + require.NoError(t, storage.ExperimentalMVCCDeleteRangeUsingTombstone( + ctx, eng, nil, key, endKey, ts, localTS, nil, nil, 0)) + } else { + require.NoError(t, storage.MVCCPut(ctx, eng, nil, key, ts, localTS, value, nil)) + } + + rh, err = repl.sha512(ctx, desc, eng, nil, roachpb.ChecksumMode_CHECK_FULL, lim) + require.NoError(t, err) + fmt.Fprintf(sb, "checksum%d: %x\n", i+1, rh.SHA512[:]) + } + + // Run another check to obtain a snapshot and stats for the final state. + kvSnapshot := roachpb.RaftSnapshotData{} + rh, err = repl.sha512(ctx, desc, eng, &kvSnapshot, roachpb.ChecksumMode_CHECK_FULL, lim) + require.NoError(t, err) + + jsonpb := protoutil.JSONPb{Indent: " "} + json, err := jsonpb.Marshal(&rh.RecomputedMS) + require.NoError(t, err) + fmt.Fprintf(sb, "stats: %s\n", string(json)) + + fmt.Fprint(sb, "snapshot:\n") + for _, kv := range kvSnapshot.KV { + fmt.Fprintf(sb, " %s=%q\n", storage.MVCCKey{Key: kv.Key, Timestamp: kv.Timestamp}, kv.Value) + } + for _, rkv := range kvSnapshot.RangeKV { + fmt.Fprintf(sb, " %s=%q\n", + storage.MVCCRangeKey{StartKey: rkv.StartKey, EndKey: rkv.EndKey, Timestamp: rkv.Timestamp}, + rkv.Value) + } + + echotest.Require(t, sb.String(), testutils.TestDataPath(t, "replica_consistency_sha512")) +} diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index a6a0063110b4..16f996ec0ba3 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -53,6 +53,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/echotest" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -7520,9 +7521,7 @@ func TestDiffRange(t *testing.T) { // TODO(tschottdorf): this test should really pass the data through a // RocksDB engine to verify that the original snapshots sort correctly. - if diff := diffRange(nil, nil); diff != nil { - t.Fatalf("diff of nils = %v", diff) - } + require.Empty(t, diffRange(nil, nil)) timestamp := hlc.Timestamp{WallTime: 1729, Logical: 1} value := []byte("foo") @@ -7549,12 +7548,19 @@ func TestDiffRange(t *testing.T) { {Key: []byte("zeroleft"), Timestamp: hlc.Timestamp{WallTime: 1, Logical: 1}, Value: value}, {Key: []byte("zeroright"), Timestamp: hlc.Timestamp{WallTime: 1, Logical: 1}, Value: value}, }, + // Fragmented range keys. + RangeKV: []roachpb.RaftSnapshotData_RangeKeyValue{ + {StartKey: []byte("A"), EndKey: []byte("C"), Timestamp: timestamp}, + {StartKey: []byte("A"), EndKey: []byte("C"), Timestamp: timestamp.Add(0, -1)}, + {StartKey: []byte("C"), EndKey: []byte("F"), Timestamp: timestamp}, + {StartKey: []byte("P"), EndKey: []byte("R"), Timestamp: timestamp}, + {StartKey: []byte("S"), EndKey: []byte("T"), Timestamp: timestamp, Value: []byte{1}}, + {StartKey: []byte("X"), EndKey: []byte("Z"), Timestamp: timestamp}, + }, } // No diff works. - if diff := diffRange(leaderSnapshot, leaderSnapshot); diff != nil { - t.Fatalf("diff of equal snapshots = %v", diff) - } + require.Empty(t, diffRange(leaderSnapshot, leaderSnapshot)) replicaSnapshot := &roachpb.RaftSnapshotData{ KV: []roachpb.RaftSnapshotData_KeyValue{ @@ -7572,6 +7578,13 @@ func TestDiffRange(t *testing.T) { {Key: []byte("zeroright"), Timestamp: hlc.Timestamp{}, Value: value}, {Key: []byte("zeroright"), Timestamp: hlc.Timestamp{WallTime: 1, Logical: 1}, Value: value}, }, + RangeKV: []roachpb.RaftSnapshotData_RangeKeyValue{ + {StartKey: []byte("A"), EndKey: []byte("C"), Timestamp: timestamp}, + {StartKey: []byte("E"), EndKey: []byte("G"), Timestamp: timestamp}, + {StartKey: []byte("Q"), EndKey: []byte("R"), Timestamp: timestamp}, + {StartKey: []byte("S"), EndKey: []byte("T"), Timestamp: timestamp, Value: []byte{2}}, + {StartKey: []byte("X"), EndKey: []byte("Z"), Timestamp: timestamp.Add(0, 1)}, + }, } // The expected diff. @@ -7588,48 +7601,23 @@ func TestDiffRange(t *testing.T) { {LeaseHolder: false, Key: []byte("z"), Timestamp: timestamp, Value: value}, {LeaseHolder: true, Key: []byte("zeroleft"), Timestamp: hlc.Timestamp{}, Value: value}, {LeaseHolder: false, Key: []byte("zeroright"), Timestamp: hlc.Timestamp{}, Value: value}, - } - diff := diffRange(leaderSnapshot, replicaSnapshot) - - for i, e := range eDiff { - v := diff[i] - if e.LeaseHolder != v.LeaseHolder || !bytes.Equal(e.Key, v.Key) || e.Timestamp != v.Timestamp || !bytes.Equal(e.Value, v.Value) { - t.Fatalf("diff varies at row %d, want %v and got %v\n\ngot:\n%s\nexpected:\n%s", i, e, v, diff, eDiff) - } + {LeaseHolder: true, Key: []byte("A"), EndKey: []byte("C"), Timestamp: timestamp.Add(0, -1)}, + {LeaseHolder: true, Key: []byte("C"), EndKey: []byte("F"), Timestamp: timestamp}, + {LeaseHolder: false, Key: []byte("E"), EndKey: []byte("G"), Timestamp: timestamp}, + {LeaseHolder: true, Key: []byte("P"), EndKey: []byte("R"), Timestamp: timestamp}, + {LeaseHolder: false, Key: []byte("Q"), EndKey: []byte("R"), Timestamp: timestamp}, + {LeaseHolder: true, Key: []byte("S"), EndKey: []byte("T"), Timestamp: timestamp, Value: []byte{1}}, + {LeaseHolder: false, Key: []byte("S"), EndKey: []byte("T"), Timestamp: timestamp, Value: []byte{2}}, + {LeaseHolder: false, Key: []byte("X"), EndKey: []byte("Z"), Timestamp: timestamp.Add(0, 1)}, + {LeaseHolder: true, Key: []byte("X"), EndKey: []byte("Z"), Timestamp: timestamp}, } + diff := diffRange(leaderSnapshot, replicaSnapshot) + require.Equal(t, eDiff, diff) - // Document the stringifed output. This is what the consistency checker + // Assert the stringifed output. This is what the consistency checker // will actually print. - stringDiff := append(eDiff[:4], - ReplicaSnapshotDiff{Key: []byte("foo"), Value: value}, - ) - - const expDiff = `--- leaseholder -+++ follower --0.000001729,1 "a" -- ts:1970-01-01 00:00:00.000001729 +0000 UTC -- value:"foo" -- raw mvcc_key/value: 610000000000000006c1000000010d 666f6f -+0.000001729,1 "ab" -+ ts:1970-01-01 00:00:00.000001729 +0000 UTC -+ value:"foo" -+ raw mvcc_key/value: 61620000000000000006c1000000010d 666f6f --0.000001729,1 "abcd" -- ts:1970-01-01 00:00:00.000001729 +0000 UTC -- value:"foo" -- raw mvcc_key/value: 616263640000000000000006c1000000010d 666f6f -+0.000001729,1 "abcdef" -+ ts:1970-01-01 00:00:00.000001729 +0000 UTC -+ value:"foo" -+ raw mvcc_key/value: 6162636465660000000000000006c1000000010d 666f6f -+0,0 "foo" -+ ts:1970-01-01 00:00:00 +0000 UTC -+ value:"foo" -+ raw mvcc_key/value: 666f6f00 666f6f -` - - require.Equal(t, expDiff, stringDiff.String()) + echotest.Require(t, diff.String(), testutils.TestDataPath(t, "replica_consistency_diff")) } func TestSyncSnapshot(t *testing.T) { diff --git a/pkg/kv/kvserver/testdata/replica_consistency_diff b/pkg/kv/kvserver/testdata/replica_consistency_diff new file mode 100644 index 000000000000..fc447766d56f --- /dev/null +++ b/pkg/kv/kvserver/testdata/replica_consistency_diff @@ -0,0 +1,88 @@ +echo +---- +--- leaseholder ++++ follower +-0.000001729,1 "a" +- ts:1970-01-01 00:00:00.000001729 +0000 UTC +- value:"foo" +- raw mvcc_key/value: 610000000000000006c1000000010d 666f6f ++0.000001729,1 "ab" ++ ts:1970-01-01 00:00:00.000001729 +0000 UTC ++ value:"foo" ++ raw mvcc_key/value: 61620000000000000006c1000000010d 666f6f +-0.000001729,1 "abcd" +- ts:1970-01-01 00:00:00.000001729 +0000 UTC +- value:"foo" +- raw mvcc_key/value: 616263640000000000000006c1000000010d 666f6f ++0.000001729,1 "abcdef" ++ ts:1970-01-01 00:00:00.000001729 +0000 UTC ++ value:"foo" ++ raw mvcc_key/value: 6162636465660000000000000006c1000000010d 666f6f ++0.000001729,2 "abcdefg" ++ ts:1970-01-01 00:00:00.000001729 +0000 UTC ++ value:"foo" ++ raw mvcc_key/value: 616263646566670000000000000006c1000000020d 666f6f +-0.000001729,0 "abcdefg" +- ts:1970-01-01 00:00:00.000001729 +0000 UTC +- value:"foo" +- raw mvcc_key/value: 616263646566670000000000000006c109 666f6f +-0.000001729,1 "x" +- ts:1970-01-01 00:00:00.000001729 +0000 UTC +- value:"foo" +- raw mvcc_key/value: 780000000000000006c1000000010d 666f6f ++0.000001729,1 "x" ++ ts:1970-01-01 00:00:00.000001729 +0000 UTC ++ value:"bar" ++ raw mvcc_key/value: 780000000000000006c1000000010d 626172 +-0.000001729,1 "y" +- ts:1970-01-01 00:00:00.000001729 +0000 UTC +- value:"foo" +- raw mvcc_key/value: 790000000000000006c1000000010d 666f6f ++0.000001729,1 "z" ++ ts:1970-01-01 00:00:00.000001729 +0000 UTC ++ value:"foo" ++ raw mvcc_key/value: 7a0000000000000006c1000000010d 666f6f +-0,0 "zeroleft" +- ts:1970-01-01 00:00:00 +0000 UTC +- value:"foo" +- raw mvcc_key/value: 7a65726f6c65667400 666f6f ++0,0 "zeroright" ++ ts:1970-01-01 00:00:00 +0000 UTC ++ value:"foo" ++ raw mvcc_key/value: 7a65726f726967687400 666f6f +-0.000001729,0 {A-C} +- ts:1970-01-01 00:00:00.000001729 +0000 UTC +- value:"" +- raw from/to/ts/value: 4100 4300 00000000000006c109 +-0.000001729,1 {C-F} +- ts:1970-01-01 00:00:00.000001729 +0000 UTC +- value:"" +- raw from/to/ts/value: 4300 4600 00000000000006c1000000010d ++0.000001729,1 {E-G} ++ ts:1970-01-01 00:00:00.000001729 +0000 UTC ++ value:"" ++ raw from/to/ts/value: 4500 4700 00000000000006c1000000010d +-0.000001729,1 {P-R} +- ts:1970-01-01 00:00:00.000001729 +0000 UTC +- value:"" +- raw from/to/ts/value: 5000 5200 00000000000006c1000000010d ++0.000001729,1 {Q-R} ++ ts:1970-01-01 00:00:00.000001729 +0000 UTC ++ value:"" ++ raw from/to/ts/value: 5100 5200 00000000000006c1000000010d +-0.000001729,1 {S-T} +- ts:1970-01-01 00:00:00.000001729 +0000 UTC +- value:"\x01" +- raw from/to/ts/value: 5300 5400 00000000000006c1000000010d 01 ++0.000001729,1 {S-T} ++ ts:1970-01-01 00:00:00.000001729 +0000 UTC ++ value:"\x02" ++ raw from/to/ts/value: 5300 5400 00000000000006c1000000010d 02 ++0.000001729,2 {X-Z} ++ ts:1970-01-01 00:00:00.000001729 +0000 UTC ++ value:"" ++ raw from/to/ts/value: 5800 5a00 00000000000006c1000000020d +-0.000001729,1 {X-Z} +- ts:1970-01-01 00:00:00.000001729 +0000 UTC +- value:"" +- raw from/to/ts/value: 5800 5a00 00000000000006c1000000010d diff --git a/pkg/kv/kvserver/testdata/replica_consistency_sha512 b/pkg/kv/kvserver/testdata/replica_consistency_sha512 new file mode 100644 index 000000000000..c82d11f7634d --- /dev/null +++ b/pkg/kv/kvserver/testdata/replica_consistency_sha512 @@ -0,0 +1,28 @@ +echo +---- +checksum0: cf83e1357eefb8bdf1542850d66d8007d620e4050b5715dc83f4a921d36ce9ce47d0d13c5d85f2b0ff8318d2877eec2f63b931bd47417a81a538327af927da3e +checksum1: 67ba8f144d3c44d3ed82a5ec3758750642315af4de193c08a52a385e1027945cdd1c4c6a8f54112be2dd71d476924505fbe23fad8d40061df74c596dc583fd5f +checksum2: fc2a7a023637caa80bb923075f7d39451a328a3fe1357c08353e594810a0c22c56cad1b494662d02dcf3187e0ce44780e5b4b039ddcf5dee6ba974a542a4ffb3 +checksum3: 05186bceae59a178713407959a26110715a1e299e6a9f1b37fc3e0f8d5a0c66bedbff8378a5f7b2e3d31c929a3b985088c7b714019b9eeacc50b17924925320a +checksum4: 4f5cc8176d559bfab8e52b74851b103fd73b9e713ce12aa380a16fe177ca6e21db75e3e85a58341ab437a5a766a071a2fe6e1f03841d334da7be2295794eb813 +checksum5: 3c5d5856a626aa29913e9790033b9c23b6dc5e42bdf2e665f7b60f58bec495adc246bf4e5f5bf1acbfc78c713f2ec7820b4ba7202897bb9f824a0b7b9e9cc98d +checksum6: ebe7fd3f41a68c2608a8b10dcc9db3b39bdb6c097d3fd99411e89d75419bb58dd80faf9846aa5e47d8cabc9dcfc894c6ea58f7e035eaaa3ee55c31faed2c8000 +stats: { + "liveBytes": "53", + "liveCount": "2", + "keyBytes": "42", + "keyCount": "3", + "valBytes": "44", + "valCount": "4", + "rangeKeyCount": "2", + "rangeKeyBytes": "26", + "rangeValCount": "2", + "rangeValBytes": "9" +} +snapshot: + "a"/0.000000002,0="" + "a"/0.000000001,0="\x00\x00\x00\x00\x03a1" + "b"/0.000000003,0="\x00\x00\x00\x04e\n\x02\b\x02\x00\x00\x00\x00\x03b3" + "i"/0,0="\x12\x04\b\x00\x10\x00\x18\x00 \x00(\x002\a\x00\x00\x00\x00\x03i0" + {p-q}/0.000000001,0="" + {x-z}/0.000000009,0="\x00\x00\x00\x04e\n\x02\b\b" diff --git a/pkg/roachpb/internal_raft.proto b/pkg/roachpb/internal_raft.proto index 5837411d43ff..7460b4bcc089 100644 --- a/pkg/roachpb/internal_raft.proto +++ b/pkg/roachpb/internal_raft.proto @@ -34,8 +34,9 @@ message RangeTombstone { (gogoproto.customname) = "NextReplicaID", (gogoproto.casttype) = "ReplicaID"]; } -// RaftSnapshotData is the payload of a raftpb.Snapshot. It contains a raw copy of -// all of the range's data and metadata, including the raft log, abort span, etc. +// RaftSnapshotData is a historical vestige that used to carry snapshot data, +// but is now only used in CollectChecksumResponse to carry range KV data to +// generate diffs for checksum mismatches. message RaftSnapshotData { message KeyValue { optional bytes key = 1; @@ -44,6 +45,16 @@ message RaftSnapshotData { } repeated KeyValue KV = 2 [(gogoproto.nullable) = false, (gogoproto.customname) = "KV"]; + + message RangeKeyValue { + optional bytes start_key = 1; + optional bytes end_key = 2; + optional util.hlc.Timestamp timestamp = 3 [(gogoproto.nullable) = false]; + optional bytes value = 4; + } + repeated RangeKeyValue range_kv = 4 [(gogoproto.nullable) = false, + (gogoproto.customname) = "RangeKV"]; + // These are really raftpb.Entry, but we model them as raw bytes to avoid // roundtripping through memory. repeated bytes log_entries = 3;