@@ -14,6 +14,7 @@ import (
14
14
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
15
15
"github.com/cockroachdb/cockroach/pkg/roachpb"
16
16
"github.com/cockroachdb/cockroach/pkg/settings"
17
+ "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
17
18
"github.com/cockroachdb/cockroach/pkg/util/hlc"
18
19
"github.com/cockroachdb/cockroach/pkg/util/span"
19
20
"github.com/cockroachdb/errors"
@@ -28,9 +29,12 @@ type AggregatorFrontier struct {
28
29
29
30
// NewAggregatorFrontier returns a new AggregatorFrontier.
30
31
func NewAggregatorFrontier (
31
- statementTime hlc.Timestamp , initialHighWater hlc.Timestamp , spans ... roachpb.Span ,
32
+ statementTime hlc.Timestamp ,
33
+ initialHighWater hlc.Timestamp ,
34
+ decoder TablePrefixDecoder ,
35
+ spans ... roachpb.Span ,
32
36
) (* AggregatorFrontier , error ) {
33
- rsf , err := newResolvedSpanFrontier (statementTime , initialHighWater , spans ... )
37
+ rsf , err := newResolvedSpanFrontier (statementTime , initialHighWater , decoder , spans ... )
34
38
if err != nil {
35
39
return nil , err
36
40
}
@@ -73,9 +77,12 @@ type CoordinatorFrontier struct {
73
77
74
78
// NewCoordinatorFrontier returns a new CoordinatorFrontier.
75
79
func NewCoordinatorFrontier (
76
- statementTime hlc.Timestamp , initialHighWater hlc.Timestamp , spans ... roachpb.Span ,
80
+ statementTime hlc.Timestamp ,
81
+ initialHighWater hlc.Timestamp ,
82
+ decoder TablePrefixDecoder ,
83
+ spans ... roachpb.Span ,
77
84
) (* CoordinatorFrontier , error ) {
78
- rsf , err := newResolvedSpanFrontier (statementTime , initialHighWater , spans ... )
85
+ rsf , err := newResolvedSpanFrontier (statementTime , initialHighWater , decoder , spans ... )
79
86
if err != nil {
80
87
return nil , err
81
88
}
@@ -184,15 +191,11 @@ func (f *CoordinatorFrontier) MakeCheckpoint(
184
191
return checkpoint .Make (f .Frontier (), f .Entries (), maxBytes , metrics )
185
192
}
186
193
187
- // spanFrontier is a type alias to make it possible to embed and forward calls
188
- // (e.g. Frontier()) to the underlying span.Frontier.
189
- type spanFrontier = span.Frontier
190
-
191
194
// resolvedSpanFrontier wraps a spanFrontier with additional bookkeeping fields
192
195
// used to track resolved spans for a changefeed and methods for computing
193
196
// lagging and checkpoint spans.
194
197
type resolvedSpanFrontier struct {
195
- spanFrontier
198
+ * span. MultiFrontier [descpb. ID ]
196
199
197
200
// statementTime is the statement time of the changefeed.
198
201
statementTime hlc.Timestamp
@@ -211,15 +214,17 @@ type resolvedSpanFrontier struct {
211
214
212
215
// newResolvedSpanFrontier returns a new resolvedSpanFrontier.
213
216
func newResolvedSpanFrontier (
214
- statementTime hlc.Timestamp , initialHighWater hlc.Timestamp , spans ... roachpb.Span ,
217
+ statementTime hlc.Timestamp ,
218
+ initialHighWater hlc.Timestamp ,
219
+ decoder TablePrefixDecoder ,
220
+ spans ... roachpb.Span ,
215
221
) (* resolvedSpanFrontier , error ) {
216
- sf , err := span .MakeFrontierAt ( initialHighWater , spans ... )
222
+ sf , err := span .NewMultiFrontierAt ( newTableIDPartitioner ( decoder ), initialHighWater , spans ... )
217
223
if err != nil {
218
224
return nil , err
219
225
}
220
- sf = span .MakeConcurrentFrontier (sf )
221
226
return & resolvedSpanFrontier {
222
- spanFrontier : sf ,
227
+ MultiFrontier : sf ,
223
228
statementTime : statementTime ,
224
229
initialHighWater : initialHighWater ,
225
230
}, nil
@@ -340,7 +345,7 @@ func (f *resolvedSpanFrontier) HasLaggingSpans(sv *settings.Values) bool {
340
345
// All returns an iterator over the resolved spans in the frontier.
341
346
func (f * resolvedSpanFrontier ) All () iter.Seq [jobspb.ResolvedSpan ] {
342
347
return func (yield func (jobspb.ResolvedSpan ) bool ) {
343
- for sp , ts := range f .spanFrontier . Entries () {
348
+ for sp , ts := range f .Entries () {
344
349
var boundaryType jobspb.ResolvedSpan_BoundaryType
345
350
if ok , bt := f .boundary .At (ts ); ok {
346
351
boundaryType = bt
@@ -411,3 +416,26 @@ func (b *resolvedSpanBoundary) Forward(newBoundary resolvedSpanBoundary) bool {
411
416
func (b * resolvedSpanBoundary ) SafeFormat (s redact.SafePrinter , _ rune ) {
412
417
s .Printf ("%v boundary (%v)" , b .typ , b .ts )
413
418
}
419
+
420
+ // A TablePrefixDecoder decodes table prefixes from keys.
421
+ // The production implementation is keys.SQLCodec.
422
+ type TablePrefixDecoder interface {
423
+ DecodeTablePrefix (key roachpb.Key ) ([]byte , uint32 , error )
424
+ }
425
+
426
+ func newTableIDPartitioner (decoder TablePrefixDecoder ) span.PartitionerFunc [descpb.ID ] {
427
+ return func (sp roachpb.Span ) (descpb.ID , error ) {
428
+ _ , startKeyTableID , err := decoder .DecodeTablePrefix (sp .Key )
429
+ if err != nil {
430
+ return 0 , err
431
+ }
432
+ _ , endKeyTableID , err := decoder .DecodeTablePrefix (sp .EndKey )
433
+ if err != nil {
434
+ return 0 , err
435
+ }
436
+ if startKeyTableID != endKeyTableID {
437
+ return 0 , errors .AssertionFailedf ("span encompassing multiple tables: %s" , sp )
438
+ }
439
+ return descpb .ID (startKeyTableID ), nil
440
+ }
441
+ }
0 commit comments