Skip to content

Commit e39a063

Browse files
craig[bot]pav-kv
andcommitted
Merge #147268
147268: kvserver: test snapshot preparation r=tbg a=pav-kv This PR introduces a basic datadriven test for `prepareSnapApply` path. TODO for follow-ups: 1. [ ] the excised span [can be wider](#147268 (comment)) when the subsumed ranges are wider than the ingested snapshot 2. [ ] seems like some range dels are written twice in [this](https://github.com/cockroachdb/cockroach/blob/c81390fdc04781b5cf2440fc9bc49d49b4265457/pkg/kv/kvserver/replica_raftstorage.go#L620-L624) path (probably benign, but can be deduped) Epic: CRDB-46488 Co-authored-by: Pavel Kalinnikov <[email protected]>
2 parents e2d1e23 + d8b4c5c commit e39a063

File tree

9 files changed

+206
-25
lines changed

9 files changed

+206
-25
lines changed

pkg/kv/kvserver/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -370,6 +370,7 @@ go_test(
370370
"scatter_test.go",
371371
"scheduler_test.go",
372372
"single_key_test.go",
373+
"snapshot_apply_prepare_test.go",
373374
"split_delay_helper_test.go",
374375
"split_queue_test.go",
375376
"split_trigger_helper_test.go",

pkg/kv/kvserver/print/BUILD.bazel

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,7 @@ go_test(
3030
srcs = ["debug_print_test.go"],
3131
embed = [":print"],
3232
deps = [
33+
"//pkg/kv/kvserver/concurrency/lock",
3334
"//pkg/kv/kvserver/kvserverpb",
3435
"//pkg/roachpb",
3536
"//pkg/storage",
@@ -38,6 +39,7 @@ go_test(
3839
"//pkg/util/hlc",
3940
"//pkg/util/leaktest",
4041
"//pkg/util/log",
42+
"//pkg/util/uuid",
4143
"@com_github_cockroachdb_pebble//:pebble",
4244
"@com_github_stretchr_testify//require",
4345
],

pkg/kv/kvserver/print/debug_print.go

Lines changed: 5 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -97,15 +97,12 @@ func SprintEngineKeyValue(k storage.EngineKey, v []byte) string {
9797
if key, err := k.ToMVCCKey(); err == nil {
9898
return SprintMVCCKeyValue(storage.MVCCKeyValue{Key: key, Value: v}, true /* printKey */)
9999
}
100+
} else if k.IsLockTableKey() {
101+
if key, err := k.ToLockTableKey(); err == nil {
102+
return fmt.Sprintf("%s: %s", key, SprintIntent(v))
103+
}
100104
}
101-
var sb strings.Builder
102-
fmt.Fprintf(&sb, "%s %x (%#x): ", k.Key, k.Version, k.Encode())
103-
if out, err := tryIntent(storage.MVCCKeyValue{Value: v}); err == nil {
104-
sb.WriteString(out)
105-
} else {
106-
fmt.Fprintf(&sb, "%x", v)
107-
}
108-
return sb.String()
105+
return fmt.Sprintf("%s %x (%#x): %x", k.Key, k.Version, k.Encode(), v)
109106
}
110107

111108
// SprintEngineRangeKeyValue is like PrintEngineRangeKeyValue, but returns a

pkg/kv/kvserver/print/debug_print_test.go

Lines changed: 12 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -10,6 +10,7 @@ import (
1010
"math"
1111
"testing"
1212

13+
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock"
1314
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb"
1415
"github.com/cockroachdb/cockroach/pkg/roachpb"
1516
"github.com/cockroachdb/cockroach/pkg/storage"
@@ -18,6 +19,7 @@ import (
1819
"github.com/cockroachdb/cockroach/pkg/util/hlc"
1920
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
2021
"github.com/cockroachdb/cockroach/pkg/util/log"
22+
"github.com/cockroachdb/cockroach/pkg/util/uuid"
2123
"github.com/cockroachdb/pebble"
2224
"github.com/stretchr/testify/require"
2325
)
@@ -38,10 +40,19 @@ func TestStringifyWriteBatch(t *testing.T) {
3840
Key: roachpb.Key("/db1"),
3941
Timestamp: hlc.Timestamp{WallTime: math.MaxInt64},
4042
}), []byte("test value"), nil /* WriteOptions */))
43+
44+
ltKey, _ := storage.LockTableKey{
45+
Key: roachpb.Key("/key"), Strength: lock.Intent, TxnUUID: uuid.UUID{},
46+
}.ToEngineKey(nil)
47+
require.NoError(t, batch.Set(ltKey.Encode(), []byte("intent"), nil))
48+
4149
wb.Data = batch.Repr()
4250
s, err = DecodeWriteBatch(wb.Data)
4351
require.NoError(t, err)
44-
require.Equal(t, "Put: 9223372036.854775807,0 \"/db1\" (0x2f646231007fffffffffffffff09): \"test value\"\n", s)
52+
require.Equal(t,
53+
`Put: 9223372036.854775807,0 "/db1" (0x2f646231007fffffffffffffff09): "test value"
54+
Put: "/key"/Intent/00000000-0000-0000-0000-000000000000: 696e74656e74
55+
`, s)
4556

4657
batch = pebble.Batch{}
4758
encodedKey, err := hex.DecodeString("017a6b12c089f704918df70bee8800010003623a9318c0384d07a6f22b858594df6012")
Lines changed: 116 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,116 @@
1+
// Copyright 2025 The Cockroach Authors.
2+
//
3+
// Use of this software is governed by the CockroachDB Software License
4+
// included in the /LICENSE file.
5+
6+
package kvserver
7+
8+
import (
9+
"context"
10+
"path/filepath"
11+
"testing"
12+
13+
"github.com/cockroachdb/cockroach/pkg/keys"
14+
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock"
15+
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb"
16+
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer"
17+
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader"
18+
"github.com/cockroachdb/cockroach/pkg/raft/raftpb"
19+
"github.com/cockroachdb/cockroach/pkg/roachpb"
20+
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
21+
"github.com/cockroachdb/cockroach/pkg/storage"
22+
"github.com/cockroachdb/cockroach/pkg/testutils/echotest"
23+
"github.com/cockroachdb/cockroach/pkg/testutils/storageutils"
24+
"github.com/cockroachdb/cockroach/pkg/util/hlc"
25+
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
26+
"github.com/cockroachdb/cockroach/pkg/util/log"
27+
"github.com/cockroachdb/cockroach/pkg/util/uuid"
28+
"github.com/cockroachdb/redact"
29+
"github.com/stretchr/testify/require"
30+
)
31+
32+
// TestPrepareSnapApply tests the snapshot application code, which prepares a
33+
// set of SSTs to be ingested together with the snapshot SSTs. It this test, the
34+
// snapshot [a,k) subsumes replicas [a,b) and [b,z). Note that the key span
35+
// covered by the subsumed replicas is wider than the snapshot's.
36+
func TestPrepareSnapApply(t *testing.T) {
37+
defer leaktest.AfterTest(t)()
38+
defer log.Scope(t).Close(t)
39+
40+
var sb redact.StringBuilder
41+
writeSST := func(_ context.Context, data []byte) error {
42+
// TODO(pav-kv): range deletions are printed separately from point keys, and
43+
// this out-of-order output looks confusing. Improve it.
44+
return storageutils.ReportSSTEntries(&sb, "sst", data)
45+
}
46+
47+
desc := func(id roachpb.RangeID, start, end string) *roachpb.RangeDescriptor {
48+
return &roachpb.RangeDescriptor{
49+
RangeID: id,
50+
StartKey: roachpb.RKey(start),
51+
EndKey: roachpb.RKey(end),
52+
}
53+
}
54+
55+
id := storage.FullReplicaID{RangeID: 123, ReplicaID: 4}
56+
eng := storage.NewDefaultInMemForTesting()
57+
defer eng.Close()
58+
59+
descA := desc(101, "a", "b")
60+
descB := desc(102, "b", "z")
61+
createRangeData(t, eng, *descA)
62+
createRangeData(t, eng, *descB)
63+
64+
sl := stateloader.Make(id.RangeID)
65+
ctx := context.Background()
66+
require.NoError(t, sl.SetRaftReplicaID(ctx, eng, id.ReplicaID))
67+
for _, rID := range []roachpb.RangeID{101, 102} {
68+
require.NoError(t, stateloader.Make(rID).SetRaftReplicaID(ctx, eng, id.ReplicaID))
69+
}
70+
71+
in := prepareSnapApplyInput{
72+
id: id,
73+
st: cluster.MakeTestingClusterSettings(),
74+
todoEng: eng,
75+
sl: sl,
76+
writeSST: writeSST,
77+
78+
truncState: kvserverpb.RaftTruncatedState{Index: 100, Term: 20},
79+
hardState: raftpb.HardState{Term: 20, Commit: 100},
80+
desc: desc(id.RangeID, "a", "k"),
81+
subsumedDescs: []*roachpb.RangeDescriptor{descA, descB},
82+
}
83+
84+
clearedUnreplicatedSpan, clearedSubsumedSpans, err := prepareSnapApply(ctx, in)
85+
require.NoError(t, err)
86+
87+
sb.Printf(">> unrepl: %v\n", clearedUnreplicatedSpan)
88+
for _, span := range rditer.MakeReplicatedKeySpans(in.desc) {
89+
sb.Printf(">> repl: %v\n", span)
90+
}
91+
for _, span := range clearedSubsumedSpans {
92+
sb.Printf(">> subsumed: %v\n", span)
93+
}
94+
sb.Printf(">> excise: %v\n", in.desc.KeySpan().AsRawSpanWithNoLocals())
95+
96+
echotest.Require(t, sb.String(), filepath.Join("testdata", t.Name()+".txt"))
97+
}
98+
99+
func createRangeData(t *testing.T, eng storage.Engine, desc roachpb.RangeDescriptor) {
100+
ts := hlc.Timestamp{WallTime: 1}
101+
for _, k := range []roachpb.Key{
102+
keys.RangeDescriptorKey(desc.StartKey), // system
103+
desc.StartKey.AsRawKey(), // user
104+
roachpb.Key(desc.EndKey).Prevish(2), // user
105+
} {
106+
// Put something under the system or user key.
107+
require.NoError(t, eng.PutMVCC(
108+
storage.MVCCKey{Key: k, Timestamp: ts}, storage.MVCCValue{},
109+
))
110+
// Put something under the corresponding lock key.
111+
ek, _ := storage.LockTableKey{
112+
Key: k, Strength: lock.Intent, TxnUUID: uuid.UUID{},
113+
}.ToEngineKey(nil)
114+
require.NoError(t, eng.PutEngineKey(ek, nil))
115+
}
116+
}
Lines changed: 32 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,32 @@
1+
echo
2+
----
3+
>> sst:
4+
set: /Local/RangeID/123/u/RaftHardState/0,0 -> term:20 vote:0 commit:100 lead:0 lead_epoch:0
5+
set: /Local/RangeID/123/u/RaftReplicaID/0,0 -> replica_id:4
6+
set: /Local/RangeID/123/u/RaftTruncatedState/0,0 -> index:100 term:20
7+
rangedel: /Local/RangeID/123/{u""-v""}
8+
>> sst:
9+
set: /Local/RangeID/101/u/RangeTombstone/0,0 -> next_replica_id:2147483647
10+
rangedel: /Local/RangeID/101/{u/RaftReplicaID-v""}
11+
>> sst:
12+
set: /Local/RangeID/102/u/RangeTombstone/0,0 -> next_replica_id:2147483647
13+
rangedel: /Local/RangeID/102/{u/RaftReplicaID-v""}
14+
>> sst:
15+
del: /Local/Lock"y\xff"/0300000000000000000000000000000000 -> /<empty>
16+
>> sst:
17+
del: "y\xff"/0.000000001,0 -> /<empty>
18+
>> unrepl: /Local/RangeID/123/{u""-v""}
19+
>> repl: /Local/RangeID/123/{r""-s""}
20+
>> repl: /Local/Range"{a"-k"}
21+
>> repl: /Local/Lock/Local/Range"{a"-k"}
22+
>> repl: /Local/Lock"{a"-k"}
23+
>> repl: {a-k}
24+
>> subsumed: /Local/RangeID/101/{r""-s""}
25+
>> subsumed: /Local/RangeID/101/{u""-v""}
26+
>> subsumed: /Local/RangeID/102/{r""-s""}
27+
>> subsumed: /Local/RangeID/102/{u""-v""}
28+
>> subsumed: /Local/Range"{k"-z"}
29+
>> subsumed: /Local/Lock/Local/Range"{k"-z"}
30+
>> subsumed: /Local/Lock"{k"-z"}
31+
>> subsumed: {k-z}
32+
>> excise: {a-k}

pkg/storage/engine_key.go

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -264,6 +264,11 @@ type LockTableKey struct {
264264
TxnUUID uuid.UUID
265265
}
266266

267+
// Format implements the fmt.Formatter interface.
268+
func (lk LockTableKey) Format(f fmt.State, _ rune) {
269+
fmt.Fprintf(f, "%s/%s/%s", lk.Key, lk.Strength, lk.TxnUUID)
270+
}
271+
267272
// replicatedLockStrengthToByte is a mapping between lock.Strength and the
268273
// strength byte persisted in a lock table key's encoding. See
269274
// LockTableKey.ToEngineKey().

pkg/testutils/storageutils/BUILD.bazel

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,7 @@ go_library(
1616
"//pkg/keys",
1717
"//pkg/kv/kvpb",
1818
"//pkg/kv/kvserver/kvserverbase",
19+
"//pkg/kv/kvserver/print",
1920
"//pkg/roachpb",
2021
"//pkg/settings/cluster",
2122
"//pkg/storage",
@@ -25,6 +26,7 @@ go_library(
2526
"//pkg/util/protoutil",
2627
"//pkg/util/syncutil",
2728
"//pkg/util/syncutil/singleflight",
29+
"@com_github_cockroachdb_errors//:errors",
2830
"@com_github_cockroachdb_pebble//:pebble",
2931
"@com_github_cockroachdb_pebble//sstable",
3032
"@com_github_cockroachdb_pebble//sstable/block",

pkg/testutils/storageutils/sst.go

Lines changed: 31 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -11,12 +11,14 @@ import (
1111
"testing"
1212

1313
"github.com/cockroachdb/cockroach/pkg/keys"
14+
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/print"
1415
"github.com/cockroachdb/cockroach/pkg/roachpb"
1516
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
1617
"github.com/cockroachdb/cockroach/pkg/storage"
1718
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
1819
"github.com/cockroachdb/cockroach/pkg/storage/mvccencoding"
1920
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
21+
"github.com/cockroachdb/errors"
2022
"github.com/cockroachdb/pebble"
2123
"github.com/cockroachdb/pebble/sstable"
2224
"github.com/cockroachdb/pebble/sstable/block"
@@ -150,9 +152,9 @@ func ReportSSTEntries(buf *redact.StringBuilder, name string, sst []byte) error
150152
if err := iter.Error(); err != nil {
151153
return err
152154
}
153-
key, err := storage.DecodeMVCCKey(kv.K.UserKey)
154-
if err != nil {
155-
return err
155+
key, ok := storage.DecodeEngineKey(kv.K.UserKey)
156+
if !ok {
157+
return errors.Errorf("invalid engine key: %x", kv.K.UserKey)
156158
}
157159
v, _, err := kv.Value(nil)
158160
if err != nil {
@@ -162,7 +164,20 @@ func ReportSSTEntries(buf *redact.StringBuilder, name string, sst []byte) error
162164
if err != nil {
163165
return err
164166
}
165-
buf.Printf("%s: %s -> %s\n", strings.ToLower(kv.Kind().String()), key, value)
167+
if !key.IsMVCCKey() {
168+
buf.Printf("%s: %s -> %s\n", strings.ToLower(kv.Kind().String()), key, value)
169+
continue
170+
}
171+
mk, err := key.ToMVCCKey()
172+
if err != nil {
173+
return err
174+
}
175+
if mk.IsValue() {
176+
buf.Printf("%s: %s -> %s\n", strings.ToLower(kv.Kind().String()), mk, value)
177+
} else {
178+
buf.Printf("%s: %s -> %s\n", strings.ToLower(kv.Kind().String()), mk,
179+
print.SprintMVCCKeyValue(storage.MVCCKeyValue{Key: mk, Value: v}, false /* printKey */))
180+
}
166181
}
167182

168183
// Dump rangedels.
@@ -172,13 +187,13 @@ func ReportSSTEntries(buf *redact.StringBuilder, name string, sst []byte) error
172187
defer rdIter.Close()
173188
s, err := rdIter.First()
174189
for ; s != nil; s, err = rdIter.Next() {
175-
start, err := storage.DecodeMVCCKey(s.Start)
176-
if err != nil {
177-
return err
190+
start, ok := storage.DecodeEngineKey(s.Start)
191+
if !ok {
192+
return errors.Errorf("invalid engine key: %x", s.Start)
178193
}
179-
end, err := storage.DecodeMVCCKey(s.End)
180-
if err != nil {
181-
return err
194+
end, ok := storage.DecodeEngineKey(s.End)
195+
if !ok {
196+
return errors.Errorf("invalid engine key: %x", s.End)
182197
}
183198
for _, k := range s.Keys {
184199
buf.Printf("%s: %s\n", strings.ToLower(k.Kind().String()),
@@ -197,13 +212,13 @@ func ReportSSTEntries(buf *redact.StringBuilder, name string, sst []byte) error
197212
defer rkIter.Close()
198213
s, err := rkIter.First()
199214
for ; s != nil; s, err = rkIter.Next() {
200-
start, err := storage.DecodeMVCCKey(s.Start)
201-
if err != nil {
202-
return err
215+
start, ok := storage.DecodeEngineKey(s.Start)
216+
if !ok {
217+
return errors.Errorf("invalid engine key: %x", s.Start)
203218
}
204-
end, err := storage.DecodeMVCCKey(s.End)
205-
if err != nil {
206-
return err
219+
end, ok := storage.DecodeEngineKey(s.End)
220+
if !ok {
221+
return errors.Errorf("invalid engine key: %x", s.End)
207222
}
208223
for _, k := range s.Keys {
209224
buf.Printf("%s: %s", strings.ToLower(k.Kind().String()),

0 commit comments

Comments
 (0)