Skip to content

Commit 034e856

Browse files
committed
sql/bulkingest: switch BulkMergeSpec keys from string to roachpb.Key
The fields StartKey and EndKey in execinfrapb.BulkMergeSpec were previously typed as strings. However, they are logically roachpb.Key, and using the correct type simplifies downstream usage. This change is a preparatory refactor to ease the integration of upcoming ingest processor changes. Informs: #156659 Release note: none
1 parent 222df72 commit 034e856

File tree

5 files changed

+64
-64
lines changed

5 files changed

+64
-64
lines changed

pkg/sql/bulkingest/split_picker.go

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -43,7 +43,7 @@ func pickSplits(
4343

4444
// Validate SSTs are ordered and non-overlapping.
4545
for i := 1; i < len(ssts); i++ {
46-
prev, curr := roachpb.Key(ssts[i-1].StartKey), roachpb.Key(ssts[i].StartKey)
46+
prev, curr := ssts[i-1].StartKey, ssts[i].StartKey
4747
if !less(prev, curr) {
4848
return nil, errors.Newf("SSTs not in order: %s >= %s", prev, curr)
4949
}
@@ -58,12 +58,12 @@ func pickSplits(
5858
for _, span := range spans {
5959
spanSSTStartIdx := sstIdx
6060
for ; sstIdx < len(ssts); sstIdx++ {
61-
sstStart := roachpb.Key(ssts[sstIdx].StartKey)
61+
sstStart := ssts[sstIdx].StartKey
6262
if !less(sstStart, span.EndKey) {
6363
break
6464
}
6565

66-
sstEnd := roachpb.Key(ssts[sstIdx].EndKey)
66+
sstEnd := ssts[sstIdx].EndKey
6767
if !less(sstEnd, span.EndKey) && !sstEnd.Equal(span.EndKey) {
6868
return nil, errors.Newf("SST ending at %s extends beyond containing span ending at %s",
6969
sstEnd, span.EndKey)
@@ -112,7 +112,7 @@ func pickSplitsForSpan(
112112
spanStart := span.Key
113113

114114
for i := 1; i < len(ssts); i++ {
115-
splitPoint := roachpb.Key(ssts[i].StartKey)
115+
splitPoint := ssts[i].StartKey
116116

117117
// Validate that the split point is already at a safe split point.
118118
safeSplitPoint, err := keys.EnsureSafeSplitKey(splitPoint)
@@ -156,7 +156,7 @@ func overlaps(a, b roachpb.Span) bool {
156156
// spanFromSST returns the span that matches the SST's start and end keys.
157157
func spanFromSST(sst execinfrapb.BulkMergeSpec_SST) roachpb.Span {
158158
return roachpb.Span{
159-
Key: roachpb.Key(sst.StartKey),
160-
EndKey: roachpb.Key(sst.EndKey),
159+
Key: sst.StartKey,
160+
EndKey: sst.EndKey,
161161
}
162162
}

pkg/sql/bulkingest/split_picker_test.go

Lines changed: 45 additions & 45 deletions
Original file line numberDiff line numberDiff line change
@@ -37,7 +37,7 @@ func TestPickSplits(t *testing.T) {
3737
{
3838
name: "empty spans",
3939
spans: nil,
40-
ssts: []execinfrapb.BulkMergeSpec_SST{{StartKey: "a", EndKey: "b"}},
40+
ssts: []execinfrapb.BulkMergeSpec_SST{{StartKey: roachpb.Key("a"), EndKey: roachpb.Key("b")}},
4141
expectedError: "no spans provided",
4242
},
4343
{
@@ -46,7 +46,7 @@ func TestPickSplits(t *testing.T) {
4646
{Key: roachpb.Key("c"), EndKey: roachpb.Key("d")},
4747
{Key: roachpb.Key("a"), EndKey: roachpb.Key("b")},
4848
},
49-
ssts: []execinfrapb.BulkMergeSpec_SST{{StartKey: "a", EndKey: "b"}},
49+
ssts: []execinfrapb.BulkMergeSpec_SST{{StartKey: roachpb.Key("a"), EndKey: roachpb.Key("b")}},
5050
expectedError: "spans not ordered: \"c\" >= \"a\"",
5151
},
5252
{
@@ -55,7 +55,7 @@ func TestPickSplits(t *testing.T) {
5555
{Key: roachpb.Key("a"), EndKey: roachpb.Key("c")},
5656
{Key: roachpb.Key("b"), EndKey: roachpb.Key("d")},
5757
},
58-
ssts: []execinfrapb.BulkMergeSpec_SST{{StartKey: "a", EndKey: "b"}},
58+
ssts: []execinfrapb.BulkMergeSpec_SST{{StartKey: roachpb.Key("a"), EndKey: roachpb.Key("b")}},
5959
expectedError: "spans are overlapping: \"c\" overlaps with \"b\"",
6060
},
6161
{
@@ -64,8 +64,8 @@ func TestPickSplits(t *testing.T) {
6464
{Key: roachpb.Key("a"), EndKey: roachpb.Key("z")},
6565
},
6666
ssts: []execinfrapb.BulkMergeSpec_SST{
67-
{StartKey: "c", EndKey: "d"},
68-
{StartKey: "a", EndKey: "b"},
67+
{StartKey: roachpb.Key("c"), EndKey: roachpb.Key("d")},
68+
{StartKey: roachpb.Key("a"), EndKey: roachpb.Key("b")},
6969
},
7070
expectedError: "SSTs not in order",
7171
},
@@ -75,8 +75,8 @@ func TestPickSplits(t *testing.T) {
7575
{Key: roachpb.Key("a"), EndKey: roachpb.Key("z")},
7676
},
7777
ssts: []execinfrapb.BulkMergeSpec_SST{
78-
{StartKey: "a", EndKey: "c"},
79-
{StartKey: "b", EndKey: "d"},
78+
{StartKey: roachpb.Key("a"), EndKey: roachpb.Key("c")},
79+
{StartKey: roachpb.Key("b"), EndKey: roachpb.Key("d")},
8080
},
8181
expectedError: "overlapping SSTs",
8282
},
@@ -86,7 +86,7 @@ func TestPickSplits(t *testing.T) {
8686
{Key: roachpb.Key("a"), EndKey: roachpb.Key("c")},
8787
},
8888
ssts: []execinfrapb.BulkMergeSpec_SST{
89-
{StartKey: "a", EndKey: "d"},
89+
{StartKey: roachpb.Key("a"), EndKey: roachpb.Key("d")},
9090
},
9191
expectedError: "SST ending at \"d\" extends beyond containing span ending at \"c\"",
9292
},
@@ -96,7 +96,7 @@ func TestPickSplits(t *testing.T) {
9696
{Key: roachpb.Key("b"), EndKey: roachpb.Key("d")},
9797
},
9898
ssts: []execinfrapb.BulkMergeSpec_SST{
99-
{StartKey: "a", EndKey: "c"},
99+
{StartKey: roachpb.Key("a"), EndKey: roachpb.Key("c")},
100100
},
101101
expectedError: "SST starting at \"a\" begins before containing span starting at \"b\"",
102102
},
@@ -116,7 +116,7 @@ func TestPickSplits(t *testing.T) {
116116
{Key: roachpb.Key("a"), EndKey: roachpb.Key("z")},
117117
},
118118
ssts: []execinfrapb.BulkMergeSpec_SST{
119-
{StartKey: "c", EndKey: "d"},
119+
{StartKey: roachpb.Key("c"), EndKey: roachpb.Key("d")},
120120
},
121121
expected: []roachpb.Span{
122122
{Key: roachpb.Key("a"), EndKey: roachpb.Key("z")},
@@ -128,9 +128,9 @@ func TestPickSplits(t *testing.T) {
128128
{Key: roachpb.Key("a"), EndKey: roachpb.Key("z")},
129129
},
130130
ssts: []execinfrapb.BulkMergeSpec_SST{
131-
{StartKey: "c", EndKey: "d"},
132-
{StartKey: "f", EndKey: "g"},
133-
{StartKey: "i", EndKey: "j"},
131+
{StartKey: roachpb.Key("c"), EndKey: roachpb.Key("d")},
132+
{StartKey: roachpb.Key("f"), EndKey: roachpb.Key("g")},
133+
{StartKey: roachpb.Key("i"), EndKey: roachpb.Key("j")},
134134
},
135135
expected: []roachpb.Span{
136136
{Key: roachpb.Key("a"), EndKey: roachpb.Key("f")},
@@ -145,10 +145,10 @@ func TestPickSplits(t *testing.T) {
145145
{Key: roachpb.Key("m"), EndKey: roachpb.Key("z")},
146146
},
147147
ssts: []execinfrapb.BulkMergeSpec_SST{
148-
{StartKey: "c", EndKey: "d"},
149-
{StartKey: "f", EndKey: "g"},
150-
{StartKey: "o", EndKey: "p"},
151-
{StartKey: "r", EndKey: "s"},
148+
{StartKey: roachpb.Key("c"), EndKey: roachpb.Key("d")},
149+
{StartKey: roachpb.Key("f"), EndKey: roachpb.Key("g")},
150+
{StartKey: roachpb.Key("o"), EndKey: roachpb.Key("p")},
151+
{StartKey: roachpb.Key("r"), EndKey: roachpb.Key("s")},
152152
},
153153
expected: []roachpb.Span{
154154
{Key: roachpb.Key("a"), EndKey: roachpb.Key("f")},
@@ -164,8 +164,8 @@ func TestPickSplits(t *testing.T) {
164164
{Key: roachpb.Key("e"), EndKey: roachpb.Key("z")},
165165
},
166166
ssts: []execinfrapb.BulkMergeSpec_SST{
167-
{StartKey: "f", EndKey: "g"},
168-
{StartKey: "i", EndKey: "j"},
167+
{StartKey: roachpb.Key("f"), EndKey: roachpb.Key("g")},
168+
{StartKey: roachpb.Key("i"), EndKey: roachpb.Key("j")},
169169
},
170170
expected: []roachpb.Span{
171171
{Key: roachpb.Key("a"), EndKey: roachpb.Key("e")},
@@ -180,8 +180,8 @@ func TestPickSplits(t *testing.T) {
180180
{Key: roachpb.Key("k"), EndKey: roachpb.Key("z")},
181181
},
182182
ssts: []execinfrapb.BulkMergeSpec_SST{
183-
{StartKey: "c", EndKey: "d"},
184-
{StartKey: "f", EndKey: "g"},
183+
{StartKey: roachpb.Key("c"), EndKey: roachpb.Key("d")},
184+
{StartKey: roachpb.Key("f"), EndKey: roachpb.Key("g")},
185185
},
186186
expected: []roachpb.Span{
187187
{Key: roachpb.Key("a"), EndKey: roachpb.Key("f")},
@@ -196,9 +196,9 @@ func TestPickSplits(t *testing.T) {
196196
{Key: roachpb.Key("k"), EndKey: roachpb.Key("z")},
197197
},
198198
ssts: []execinfrapb.BulkMergeSpec_SST{
199-
{StartKey: "a", EndKey: "b"},
200-
{StartKey: "j", EndKey: "k"},
201-
{StartKey: "k", EndKey: "l"},
199+
{StartKey: roachpb.Key("a"), EndKey: roachpb.Key("b")},
200+
{StartKey: roachpb.Key("j"), EndKey: roachpb.Key("k")},
201+
{StartKey: roachpb.Key("k"), EndKey: roachpb.Key("l")},
202202
},
203203
expected: []roachpb.Span{
204204
{Key: roachpb.Key("a"), EndKey: roachpb.Key("j")},
@@ -213,11 +213,11 @@ func TestPickSplits(t *testing.T) {
213213
{Key: roachpb.Key("c"), EndKey: roachpb.Key("f")},
214214
},
215215
ssts: []execinfrapb.BulkMergeSpec_SST{
216-
{StartKey: "a", EndKey: "b"},
217-
{StartKey: "d", EndKey: "e"},
218-
{StartKey: "x", EndKey: "z"}, // This SST is completely outside all spans
216+
{StartKey: roachpb.Key("a"), EndKey: roachpb.Key("b")},
217+
{StartKey: roachpb.Key("d"), EndKey: roachpb.Key("e")},
218+
{StartKey: roachpb.Key("x"), EndKey: roachpb.Key("z")}, // This SST is completely outside all spans
219219
},
220-
expectedError: "SST starting at x not contained in any span",
220+
expectedError: "SST starting at \"x\" not contained in any span",
221221
},
222222
{
223223
name: "sst in gap between non-contiguous spans",
@@ -226,8 +226,8 @@ func TestPickSplits(t *testing.T) {
226226
{Key: roachpb.Key("e"), EndKey: roachpb.Key("g")}, // Gap from [c, e)
227227
},
228228
ssts: []execinfrapb.BulkMergeSpec_SST{
229-
{StartKey: "a", EndKey: "b"},
230-
{StartKey: "c", EndKey: "d"}, // This SST is in the gap [c, e)
229+
{StartKey: roachpb.Key("a"), EndKey: roachpb.Key("b")},
230+
{StartKey: roachpb.Key("c"), EndKey: roachpb.Key("d")}, // This SST is in the gap [c, e)
231231
},
232232
expectedError: "SST starting at \"c\" begins before containing span starting at \"e\"",
233233
},
@@ -269,7 +269,7 @@ func TestPickSplitsForSpan(t *testing.T) {
269269
name: "single sst",
270270
span: roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("z")},
271271
ssts: []execinfrapb.BulkMergeSpec_SST{
272-
{StartKey: "c", EndKey: "d"},
272+
{StartKey: roachpb.Key("c"), EndKey: roachpb.Key("d")},
273273
},
274274
expected: []roachpb.Span{
275275
{Key: roachpb.Key("a"), EndKey: roachpb.Key("z")},
@@ -279,8 +279,8 @@ func TestPickSplitsForSpan(t *testing.T) {
279279
name: "two ssts",
280280
span: roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("z")},
281281
ssts: []execinfrapb.BulkMergeSpec_SST{
282-
{StartKey: "c", EndKey: "d"},
283-
{StartKey: "f", EndKey: "g"},
282+
{StartKey: roachpb.Key("c"), EndKey: roachpb.Key("d")},
283+
{StartKey: roachpb.Key("f"), EndKey: roachpb.Key("g")},
284284
},
285285
expected: []roachpb.Span{
286286
{Key: roachpb.Key("a"), EndKey: roachpb.Key("f")},
@@ -291,9 +291,9 @@ func TestPickSplitsForSpan(t *testing.T) {
291291
name: "three ssts",
292292
span: roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("z")},
293293
ssts: []execinfrapb.BulkMergeSpec_SST{
294-
{StartKey: "c", EndKey: "d"},
295-
{StartKey: "f", EndKey: "g"},
296-
{StartKey: "i", EndKey: "j"},
294+
{StartKey: roachpb.Key("c"), EndKey: roachpb.Key("d")},
295+
{StartKey: roachpb.Key("f"), EndKey: roachpb.Key("g")},
296+
{StartKey: roachpb.Key("i"), EndKey: roachpb.Key("j")},
297297
},
298298
expected: []roachpb.Span{
299299
{Key: roachpb.Key("a"), EndKey: roachpb.Key("f")},
@@ -305,8 +305,8 @@ func TestPickSplitsForSpan(t *testing.T) {
305305
name: "sst at span boundary",
306306
span: roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("z")},
307307
ssts: []execinfrapb.BulkMergeSpec_SST{
308-
{StartKey: "a", EndKey: "b"},
309-
{StartKey: "y", EndKey: "z"},
308+
{StartKey: roachpb.Key("a"), EndKey: roachpb.Key("b")},
309+
{StartKey: roachpb.Key("y"), EndKey: roachpb.Key("z")},
310310
},
311311
expected: []roachpb.Span{
312312
{Key: roachpb.Key("a"), EndKey: roachpb.Key("y")},
@@ -317,9 +317,9 @@ func TestPickSplitsForSpan(t *testing.T) {
317317
name: "adjacent ssts",
318318
span: roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("z")},
319319
ssts: []execinfrapb.BulkMergeSpec_SST{
320-
{StartKey: "c", EndKey: "d"},
321-
{StartKey: "d", EndKey: "e"},
322-
{StartKey: "e", EndKey: "f"},
320+
{StartKey: roachpb.Key("c"), EndKey: roachpb.Key("d")},
321+
{StartKey: roachpb.Key("d"), EndKey: roachpb.Key("e")},
322+
{StartKey: roachpb.Key("e"), EndKey: roachpb.Key("f")},
323323
},
324324
expected: []roachpb.Span{
325325
{Key: roachpb.Key("a"), EndKey: roachpb.Key("d")},
@@ -369,8 +369,8 @@ func TestPickSplitsForSpanErrors(t *testing.T) {
369369
sst2End := codec.IndexPrefix(50, 2)
370370

371371
return []execinfrapb.BulkMergeSpec_SST{
372-
{StartKey: string(sst1Start), EndKey: string(sst1End)},
373-
{StartKey: string(sst2Start), EndKey: string(sst2End)},
372+
{StartKey: sst1Start, EndKey: sst1End},
373+
{StartKey: sst2Start, EndKey: sst2End},
374374
}
375375
}(),
376376
expectedError: "SST 1 start key .* is not at a safe split point.*SST writer should have ensured safe boundaries",
@@ -395,8 +395,8 @@ func TestPickSplitsForSpanErrors(t *testing.T) {
395395
sst2End := codec.IndexPrefix(50, 2)
396396

397397
return []execinfrapb.BulkMergeSpec_SST{
398-
{StartKey: string(sst1Start), EndKey: string(sst1End)},
399-
{StartKey: string(sst2Start), EndKey: string(sst2End)},
398+
{StartKey: sst1Start, EndKey: sst1End},
399+
{StartKey: sst2Start, EndKey: sst2End},
400400
}
401401
}(),
402402
expectedError: "SST 1 has unsafe start key",

pkg/sql/bulksst/combine_file_info.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -43,8 +43,8 @@ func CombineFileInfo(
4343
for _, file := range files {
4444
for _, sst := range file.SST {
4545
result = append(result, execinfrapb.BulkMergeSpec_SST{
46-
StartKey: string(sst.StartKey),
47-
EndKey: string(sst.EndKey),
46+
StartKey: sst.StartKey,
47+
EndKey: sst.EndKey,
4848
URI: sst.URI,
4949
})
5050
}

pkg/sql/bulksst/combine_file_info_test.go

Lines changed: 9 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -53,7 +53,7 @@ func TestCombineFileInfo(t *testing.T) {
5353
},
5454
schemaSpans: []roachpb.Span{span("a", "z")},
5555
expectedSSTs: []execinfrapb.BulkMergeSpec_SST{
56-
{URI: "file1.sst", StartKey: "b", EndKey: "d"},
56+
{URI: "file1.sst", StartKey: roachpb.Key("b"), EndKey: roachpb.Key("d")},
5757
},
5858
expectedMergeSpan: []roachpb.Span{span("a", "z")},
5959
},
@@ -73,8 +73,8 @@ func TestCombineFileInfo(t *testing.T) {
7373
},
7474
schemaSpans: []roachpb.Span{span("a", "z")},
7575
expectedSSTs: []execinfrapb.BulkMergeSpec_SST{
76-
{URI: "file1.sst", StartKey: "a", EndKey: "e"},
77-
{URI: "file2.sst", StartKey: "e", EndKey: "z"},
76+
{URI: "file1.sst", StartKey: roachpb.Key("a"), EndKey: roachpb.Key("e")},
77+
{URI: "file2.sst", StartKey: roachpb.Key("e"), EndKey: roachpb.Key("z")},
7878
},
7979
expectedMergeSpan: []roachpb.Span{
8080
span("a", "c"),
@@ -102,8 +102,8 @@ func TestCombineFileInfo(t *testing.T) {
102102
span("m", "z"),
103103
},
104104
expectedSSTs: []execinfrapb.BulkMergeSpec_SST{
105-
{URI: "file1.sst", StartKey: "a", EndKey: "m"},
106-
{URI: "file2.sst", StartKey: "m", EndKey: "z"},
105+
{URI: "file1.sst", StartKey: roachpb.Key("a"), EndKey: roachpb.Key("m")},
106+
{URI: "file2.sst", StartKey: roachpb.Key("m"), EndKey: roachpb.Key("z")},
107107
},
108108
expectedMergeSpan: []roachpb.Span{
109109
span("a", "c"),
@@ -133,8 +133,8 @@ func TestCombineFileInfo(t *testing.T) {
133133
span("m", "z"),
134134
},
135135
expectedSSTs: []execinfrapb.BulkMergeSpec_SST{
136-
{URI: "file1.sst", StartKey: "a", EndKey: "m"},
137-
{URI: "file2.sst", StartKey: "m", EndKey: "z"},
136+
{URI: "file1.sst", StartKey: roachpb.Key("a"), EndKey: roachpb.Key("m")},
137+
{URI: "file2.sst", StartKey: roachpb.Key("m"), EndKey: roachpb.Key("z")},
138138
},
139139
expectedMergeSpan: []roachpb.Span{
140140
span("a", "c"),
@@ -159,8 +159,8 @@ func TestCombineFileInfo(t *testing.T) {
159159
},
160160
schemaSpans: []roachpb.Span{span("a", "z")},
161161
expectedSSTs: []execinfrapb.BulkMergeSpec_SST{
162-
{URI: "file1a.sst", StartKey: "a", EndKey: "e"},
163-
{URI: "file1b.sst", StartKey: "e", EndKey: "m"},
162+
{URI: "file1a.sst", StartKey: roachpb.Key("a"), EndKey: roachpb.Key("e")},
163+
{URI: "file1b.sst", StartKey: roachpb.Key("e"), EndKey: roachpb.Key("m")},
164164
},
165165
expectedMergeSpan: []roachpb.Span{
166166
span("a", "c"),

pkg/sql/execinfrapb/processors_bulk_io.proto

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -589,9 +589,9 @@ message BulkMergeSpec {
589589
message SST {
590590
optional string uri = 1 [(gogoproto.nullable) = false, (gogoproto.customname) = "URI"];
591591
// start_key is the first key in the SST.
592-
optional string start_key = 2 [(gogoproto.nullable) = false];
592+
optional bytes start_key = 2 [(gogoproto.nullable) = false, (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key"];
593593
// end_key is the last key in the SST.
594-
optional string end_key = 3 [(gogoproto.nullable) = false];
594+
optional bytes end_key = 3 [(gogoproto.nullable) = false, (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key"];
595595
}
596596

597597
// ssts is the list of input SSTs to merge.

0 commit comments

Comments
 (0)