-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
Copy pathchangefeed_processors.go
1718 lines (1516 loc) · 56.3 KB
/
changefeed_processors.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// Copyright 2018 The Cockroach Authors.
//
// Licensed as a CockroachDB Enterprise file under the Cockroach Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt
package changefeedccl
import (
"context"
"fmt"
"time"
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdcutils"
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase"
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeeddist"
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvevent"
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvfeed"
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/schemafeed"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/lease"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util/bufalloc"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/log/logcrash"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/span"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
)
type changeAggregator struct {
execinfra.ProcessorBase
flowCtx *execinfra.FlowCtx
spec execinfrapb.ChangeAggregatorSpec
memAcc mon.BoundAccount
// cancel shuts down the processor, both the `Next()` flow and the kvfeed.
cancel func()
// errCh contains the return values of the kvfeed.
errCh chan error
// kvFeedDoneCh is closed when the kvfeed exits.
kvFeedDoneCh chan struct{}
kvFeedMemMon *mon.BytesMonitor
// encoder is the Encoder to use for key and value serialization.
encoder Encoder
// sink is the Sink to write rows to. Resolved timestamps are never written
// by changeAggregator.
sink Sink
// changedRowBuf, if non-nil, contains changed rows to be emitted. Anything
// queued in `resolvedSpanBuf` is dependent on these having been emitted, so
// this one must be empty before moving on to that one.
changedRowBuf *encDatumRowBuffer
// resolvedSpanBuf contains resolved span updates to send to changeFrontier.
// If sink is a bufferSink, it must be emptied before these are sent.
resolvedSpanBuf encDatumRowBuffer
// eventProducer produces the next event from the kv feed.
eventProducer kvevent.Reader
// eventConsumer consumes the event.
eventConsumer kvEventConsumer
// lastFlush and flushFrequency keep track of the flush frequency.
lastFlush time.Time
flushFrequency time.Duration
// frontier keeps track of resolved timestamps for spans along with schema change
// boundary information.
frontier *schemaChangeFrontier
metrics *Metrics
sliMetrics *sliMetrics
knobs TestingKnobs
}
type timestampLowerBoundOracle interface {
inclusiveLowerBoundTS() hlc.Timestamp
}
type changeAggregatorLowerBoundOracle struct {
sf *span.Frontier
initialInclusiveLowerBound hlc.Timestamp
}
// inclusiveLowerBoundTs is used to generate a representative timestamp to name
// cloudStorageSink files. This timestamp is either the statement time (in case this
// changefeed job hasn't yet seen any resolved timestamps) or the successor timestamp to
// the local span frontier. This convention is chosen to preserve CDC's ordering
// guarantees. See comment on cloudStorageSink for more details.
func (o *changeAggregatorLowerBoundOracle) inclusiveLowerBoundTS() hlc.Timestamp {
if frontier := o.sf.Frontier(); !frontier.IsEmpty() {
// We call `Next()` here on the frontier because this allows us
// to name files using a timestamp that is an inclusive lower bound
// on the timestamps of the updates contained within the file.
// Files being created at the point this method is called are guaranteed
// to contain row updates with timestamps strictly greater than the local
// span frontier timestamp.
return frontier.Next()
}
// This should only be returned in the case where the changefeed job hasn't yet
// seen a resolved timestamp.
return o.initialInclusiveLowerBound
}
var _ execinfra.Processor = &changeAggregator{}
var _ execinfra.RowSource = &changeAggregator{}
func newChangeAggregatorProcessor(
flowCtx *execinfra.FlowCtx,
processorID int32,
spec execinfrapb.ChangeAggregatorSpec,
post *execinfrapb.PostProcessSpec,
output execinfra.RowReceiver,
) (execinfra.Processor, error) {
ctx := flowCtx.EvalCtx.Ctx()
memMonitor := execinfra.NewMonitor(ctx, flowCtx.EvalCtx.Mon, "changeagg-mem")
ca := &changeAggregator{
flowCtx: flowCtx,
spec: spec,
memAcc: memMonitor.MakeBoundAccount(),
}
if err := ca.Init(
ca,
post,
changefeeddist.ChangefeedResultTypes,
flowCtx,
processorID,
output,
memMonitor,
execinfra.ProcStateOpts{
TrailingMetaCallback: func() []execinfrapb.ProducerMetadata {
ca.close()
return nil
},
},
); err != nil {
return nil, err
}
var err error
if ca.encoder, err = getEncoder(ca.spec.Feed.Opts, ca.spec.Feed.Targets); err != nil {
return nil, err
}
// MinCheckpointFrequency controls how frequently the changeAggregator flushes the sink
// and checkpoints the local frontier to changeFrontier. It is used as a rough
// approximation of how latency-sensitive the changefeed user is. For a high latency
// user, such as cloud storage sink where flushes can take much longer, it is often set
// as the sink's flush frequency so as not to negate the sink's batch config.
//
// If a user does not specify a 'min_checkpoint_frequency' duration, we instead default
// to 30s, which is hopefully long enough to account for most possible sink latencies we
// could see without falling too behind.
//
// NB: As long as we periodically get new span-level resolved timestamps
// from the poller (which should always happen, even if the watched data is
// not changing), then this is sufficient and we don't have to do anything
// fancy with timers.
// // TODO(casper): add test for OptMinCheckpointFrequency.
if r, ok := ca.spec.Feed.Opts[changefeedbase.OptMinCheckpointFrequency]; ok && r != `` {
ca.flushFrequency, err = time.ParseDuration(r)
if err != nil {
return nil, err
}
} else {
ca.flushFrequency = changefeedbase.DefaultMinCheckpointFrequency
}
return ca, nil
}
// MustBeStreaming implements the execinfra.Processor interface.
func (ca *changeAggregator) MustBeStreaming() bool {
return true
}
// Start is part of the RowSource interface.
func (ca *changeAggregator) Start(ctx context.Context) {
ctx = ca.StartInternal(ctx, changeAggregatorProcName)
// Derive a separate context so that we can shutdown the poller. Note that
// we need to update both ctx (used throughout this function) and
// ProcessorBase.Ctx (used in all other methods) to the new context.
ctx, ca.cancel = context.WithCancel(ctx)
ca.Ctx = ctx
initialHighWater, needsInitialScan := getKVFeedInitialParameters(ca.spec)
frontierHighWater := initialHighWater
if needsInitialScan {
// The frontier highwater marks the latest timestamp we don't need to emit
// spans for, and therefore should be 0 if an initial scan is needed
frontierHighWater = hlc.Timestamp{}
}
spans, err := ca.setupSpansAndFrontier(frontierHighWater)
if err != nil {
ca.MoveToDraining(err)
ca.cancel()
return
}
timestampOracle := &changeAggregatorLowerBoundOracle{
sf: ca.frontier.SpanFrontier(),
initialInclusiveLowerBound: ca.spec.Feed.StatementTime,
}
if cfKnobs, ok := ca.flowCtx.TestingKnobs().Changefeed.(*TestingKnobs); ok {
ca.knobs = *cfKnobs
}
// TODO(yevgeniy): Introduce separate changefeed monitor that's a parent
// for all changefeeds to control memory allocated to all changefeeds.
pool := ca.flowCtx.Cfg.BackfillerMonitor
if ca.knobs.MemMonitor != nil {
pool = ca.knobs.MemMonitor
}
limit := changefeedbase.PerChangefeedMemLimit.Get(&ca.flowCtx.Cfg.Settings.SV)
kvFeedMemMon := mon.NewMonitorInheritWithLimit("kvFeed", limit, pool)
kvFeedMemMon.Start(ctx, pool, mon.BoundAccount{})
ca.kvFeedMemMon = kvFeedMemMon
// The job registry has a set of metrics used to monitor the various jobs it
// runs. They're all stored as the `metric.Struct` interface because of
// dependency cycles.
ca.metrics = ca.flowCtx.Cfg.JobRegistry.MetricsStruct().Changefeed.(*Metrics)
ca.sliMetrics, err = ca.metrics.getSLIMetrics(ca.spec.Feed.Opts[changefeedbase.OptMetricsScope])
if err != nil {
ca.MoveToDraining(err)
ca.cancel()
return
}
ca.sink, err = getSink(ctx, ca.flowCtx.Cfg, ca.spec.Feed, timestampOracle,
ca.spec.User(), ca.spec.JobID, ca.sliMetrics)
if err != nil {
err = changefeedbase.MarkRetryableError(err)
// Early abort in the case that there is an error creating the sink.
ca.MoveToDraining(err)
ca.cancel()
return
}
// This is the correct point to set up certain hooks depending on the sink
// type.
if b, ok := ca.sink.(*bufferSink); ok {
ca.changedRowBuf = &b.buf
}
ca.sink = &errorWrapperSink{wrapped: ca.sink}
ca.eventProducer, err = ca.startKVFeed(ctx, spans, initialHighWater, needsInitialScan, ca.sliMetrics)
if err != nil {
// Early abort in the case that there is an error creating the sink.
ca.MoveToDraining(err)
ca.cancel()
return
}
if ca.spec.Feed.Opts[changefeedbase.OptFormat] == string(changefeedbase.OptFormatNative) {
ca.eventConsumer = newNativeKVConsumer(ca.sink)
} else {
ca.eventConsumer = newKVEventToRowConsumer(
ctx, ca.flowCtx.Cfg, ca.frontier.SpanFrontier(), initialHighWater,
ca.sink, ca.encoder, ca.spec.Feed, ca.knobs)
}
}
func (ca *changeAggregator) startKVFeed(
ctx context.Context,
spans []roachpb.Span,
initialHighWater hlc.Timestamp,
needsInitialScan bool,
sm *sliMetrics,
) (kvevent.Reader, error) {
cfg := ca.flowCtx.Cfg
buf := kvevent.NewThrottlingBuffer(
kvevent.NewMemBuffer(ca.kvFeedMemMon.MakeBoundAccount(), &cfg.Settings.SV, &ca.metrics.KVFeedMetrics),
cdcutils.NodeLevelThrottler(&cfg.Settings.SV, &ca.metrics.ThrottleMetrics))
// KVFeed takes ownership of the kvevent.Writer portion of the buffer, while
// we return the kvevent.Reader part to the caller.
kvfeedCfg := ca.makeKVFeedCfg(ctx, spans, buf, initialHighWater, needsInitialScan, sm)
// Give errCh enough buffer both possible errors from supporting goroutines,
// but only the first one is ever used.
ca.errCh = make(chan error, 2)
ca.kvFeedDoneCh = make(chan struct{})
if err := ca.flowCtx.Stopper().RunAsyncTask(ctx, "changefeed-poller", func(ctx context.Context) {
defer close(ca.kvFeedDoneCh)
// Trying to call MoveToDraining here is racy (`MoveToDraining called in
// state stateTrailingMeta`), so return the error via a channel.
ca.errCh <- kvfeed.Run(ctx, kvfeedCfg)
ca.cancel()
}); err != nil {
// If err != nil then the RunAsyncTask closure never ran, which means we
// need to manually close ca.kvFeedDoneCh so `(*changeAggregator).close`
// doesn't hang.
close(ca.kvFeedDoneCh)
ca.errCh <- err
ca.cancel()
return nil, err
}
return buf, nil
}
func (ca *changeAggregator) makeKVFeedCfg(
ctx context.Context,
spans []roachpb.Span,
buf kvevent.Writer,
initialHighWater hlc.Timestamp,
needsInitialScan bool,
sm *sliMetrics,
) kvfeed.Config {
schemaChangeEvents := changefeedbase.SchemaChangeEventClass(
ca.spec.Feed.Opts[changefeedbase.OptSchemaChangeEvents])
schemaChangePolicy := changefeedbase.SchemaChangePolicy(
ca.spec.Feed.Opts[changefeedbase.OptSchemaChangePolicy])
_, withDiff := ca.spec.Feed.Opts[changefeedbase.OptDiff]
cfg := ca.flowCtx.Cfg
var sf schemafeed.SchemaFeed
if schemaChangePolicy == changefeedbase.OptSchemaChangePolicyIgnore {
sf = schemafeed.DoNothingSchemaFeed
} else {
sf = schemafeed.New(ctx, cfg, schemaChangeEvents, ca.spec.Feed.Targets,
initialHighWater, &ca.metrics.SchemaFeedMetrics)
}
return kvfeed.Config{
Writer: buf,
Settings: cfg.Settings,
DB: cfg.DB,
Codec: cfg.Codec,
Clock: cfg.DB.Clock(),
Gossip: cfg.Gossip,
Spans: spans,
BackfillCheckpoint: ca.spec.Checkpoint.Spans,
Targets: ca.spec.Feed.Targets,
Metrics: &ca.metrics.KVFeedMetrics,
OnBackfillCallback: ca.sliMetrics.getBackfillCallback(),
MM: ca.kvFeedMemMon,
InitialHighWater: initialHighWater,
WithDiff: withDiff,
NeedsInitialScan: needsInitialScan,
SchemaChangeEvents: schemaChangeEvents,
SchemaChangePolicy: schemaChangePolicy,
SchemaFeed: sf,
Knobs: ca.knobs.FeedKnobs,
}
}
// getKVFeedInitialParameters determines the starting timestamp for the kv and
// whether or not an initial scan is needed. The need for an initial scan is
// determined by whether the watched in the spec have a resolved timestamp. The
// higher layers mark each watch with the checkpointed resolved timestamp if no
// initial scan is needed.
//
// TODO(ajwerner): Utilize this partial checkpointing, especially in the face of
// of logical backfills of a single table while progress is made on others or
// get rid of it. See https://github.com/cockroachdb/cockroach/issues/43896.
func getKVFeedInitialParameters(
spec execinfrapb.ChangeAggregatorSpec,
) (initialHighWater hlc.Timestamp, needsInitialScan bool) {
for _, watch := range spec.Watches {
if initialHighWater.IsEmpty() || watch.InitialResolved.Less(initialHighWater) {
initialHighWater = watch.InitialResolved
}
}
// This will be true in the case where we have no cursor and we've never
// checkpointed a resolved timestamp or we have a cursor but we want an
// initial scan. The higher levels will coordinate that we only have empty
// watches when we need an initial scan.
if needsInitialScan = initialHighWater.IsEmpty(); needsInitialScan {
initialHighWater = spec.Feed.StatementTime
}
return initialHighWater, needsInitialScan
}
// setupSpans is called on start to extract the spans for this changefeed as a
// slice and creates a span frontier with the initial resolved timestampsc. This
// SpanFrontier only tracks the spans being watched on this node. There is a
// different SpanFrontier elsewhere for the entire changefeed. This object is
// used to filter out some previously emitted rows, and by the cloudStorageSink
// to name its output files in lexicographically monotonic fashion.
func (ca *changeAggregator) setupSpansAndFrontier(
initialHighWater hlc.Timestamp,
) (spans []roachpb.Span, err error) {
spans = make([]roachpb.Span, 0, len(ca.spec.Watches))
for _, watch := range ca.spec.Watches {
spans = append(spans, watch.Span)
}
ca.frontier, err = makeSchemaChangeFrontier(initialHighWater, spans...)
if err != nil {
return nil, err
}
// Checkpointed spans are spans that were above the highwater mark, and we
// must preserve that information in the frontier for future checkpointing.
// If we don't have a highwater yet (during initial scan) they must at least
// be from StatementTime, and given an initial highwater they must all by
// definition have been at or after initialHighWater.Next()
var checkpointedSpanTs hlc.Timestamp
if initialHighWater.IsEmpty() {
checkpointedSpanTs = ca.spec.Feed.StatementTime
} else {
checkpointedSpanTs = initialHighWater.Next()
}
for _, checkpointedSpan := range ca.spec.Checkpoint.Spans {
if _, err := ca.frontier.Forward(checkpointedSpan, checkpointedSpanTs); err != nil {
return nil, err
}
}
return spans, nil
}
// close has two purposes: to synchronize on the completion of the helper
// goroutines created by the Start method, and to clean up any resources used by
// the processor. Due to the fact that this method may be called even if the
// processor did not finish completion, there is an excessive amount of nil
// checking.
func (ca *changeAggregator) close() {
if ca.Closed {
return
}
ca.cancel()
// Wait for the poller to finish shutting down.
if ca.kvFeedDoneCh != nil {
<-ca.kvFeedDoneCh
}
if ca.sink != nil {
if err := ca.sink.Close(); err != nil {
log.Warningf(ca.Ctx, `error closing sink. goroutines may have leaked: %v`, err)
}
}
ca.memAcc.Close(ca.Ctx)
if ca.kvFeedMemMon != nil {
ca.kvFeedMemMon.Stop(ca.Ctx)
}
ca.MemMonitor.Stop(ca.Ctx)
ca.InternalClose()
}
// Next is part of the RowSource interface.
func (ca *changeAggregator) Next() (rowenc.EncDatumRow, *execinfrapb.ProducerMetadata) {
for ca.State == execinfra.StateRunning {
if !ca.changedRowBuf.IsEmpty() {
return ca.ProcessRowHelper(ca.changedRowBuf.Pop()), nil
} else if !ca.resolvedSpanBuf.IsEmpty() {
return ca.ProcessRowHelper(ca.resolvedSpanBuf.Pop()), nil
}
if err := ca.tick(); err != nil {
var e kvevent.ErrBufferClosed
if errors.As(err, &e) {
// ErrBufferClosed is a signal that
// our kvfeed has exited expectedly.
err = e.Unwrap()
if errors.Is(err, kvevent.ErrNormalRestartReason) {
err = nil
}
} else {
select {
// If the poller errored first, that's the
// interesting one, so overwrite `err`.
case err = <-ca.errCh:
default:
}
}
// Shut down the poller if it wasn't already.
ca.cancel()
ca.MoveToDraining(err)
break
}
}
return nil, ca.DrainHelper()
}
// tick is the workhorse behind Next(). It retrieves the next event from
// kvFeed, sends off this event to the event consumer, and flushes the sink
// if necessary.
func (ca *changeAggregator) tick() error {
event, err := ca.eventProducer.Get(ca.Ctx)
if err != nil {
return err
}
queuedNanos := timeutil.Since(event.BufferAddTimestamp()).Nanoseconds()
ca.metrics.QueueTimeNanos.Inc(queuedNanos)
switch event.Type() {
case kvevent.TypeKV:
// Keep track of SLI latency for non-backfill/rangefeed KV events.
if event.BackfillTimestamp().IsEmpty() {
ca.sliMetrics.AdmitLatency.RecordValue(timeutil.Since(event.Timestamp().GoTime()).Nanoseconds())
}
return ca.eventConsumer.ConsumeEvent(ca.Ctx, event)
case kvevent.TypeResolved:
a := event.DetachAlloc()
a.Release(ca.Ctx)
resolved := event.Resolved()
if ca.knobs.ShouldSkipResolved == nil || !ca.knobs.ShouldSkipResolved(resolved) {
return ca.noteResolvedSpan(resolved)
}
case kvevent.TypeFlush:
return ca.sink.Flush(ca.Ctx)
}
return nil
}
// noteResolvedSpan periodically flushes Frontier progress from the current
// changeAggregator node to the changeFrontier node to allow the changeFrontier
// to persist the overall changefeed's progress
func (ca *changeAggregator) noteResolvedSpan(resolved *jobspb.ResolvedSpan) error {
advanced, err := ca.frontier.ForwardResolvedSpan(*resolved)
if err != nil {
return err
}
forceFlush := resolved.BoundaryType != jobspb.ResolvedSpan_NONE
checkpointFrontier := advanced &&
(forceFlush || timeutil.Since(ca.lastFlush) > ca.flushFrequency)
// If backfilling we must also consider the Backfill Checkpointing frequency
checkpointBackfill := ca.spec.JobID != 0 && /* enterprise changefeed */
resolved.Timestamp.Equal(ca.frontier.BackfillTS()) &&
canCheckpointBackfill(&ca.flowCtx.Cfg.Settings.SV, ca.lastFlush)
if checkpointFrontier || checkpointBackfill {
defer func() {
ca.lastFlush = timeutil.Now()
}()
return ca.flushFrontier()
}
return nil
}
// flushFrontier flushes sink and emits resolved timestamp if needed.
func (ca *changeAggregator) flushFrontier() error {
// Make sure to flush the sink before forwarding resolved spans,
// otherwise, we could lose buffered messages and violate the
// at-least-once guarantee. This is also true for checkpointing the
// resolved spans in the job progress.
if err := ca.sink.Flush(ca.Ctx); err != nil {
return err
}
// Iterate frontier spans and build a list of spans to emit.
var batch jobspb.ResolvedSpans
ca.frontier.Entries(func(s roachpb.Span, ts hlc.Timestamp) span.OpResult {
boundaryType := jobspb.ResolvedSpan_NONE
if ca.frontier.boundaryTime.Equal(ts) {
boundaryType = ca.frontier.boundaryType
}
batch.ResolvedSpans = append(batch.ResolvedSpans, jobspb.ResolvedSpan{
Span: s,
Timestamp: ts,
BoundaryType: boundaryType,
})
return span.ContinueMatch
})
return ca.emitResolved(batch)
}
func (ca *changeAggregator) emitResolved(batch jobspb.ResolvedSpans) error {
for _, resolved := range batch.ResolvedSpans {
resolvedBytes, err := protoutil.Marshal(&resolved)
if err != nil {
return err
}
// Enqueue a row to be returned that indicates some span-level resolved
// timestamp has advanced. If any rows were queued in `sink`, they must
// be emitted first.
ca.resolvedSpanBuf.Push(rowenc.EncDatumRow{
rowenc.EncDatum{Datum: tree.NewDBytes(tree.DBytes(resolvedBytes))},
rowenc.EncDatum{Datum: tree.DNull}, // topic
rowenc.EncDatum{Datum: tree.DNull}, // key
rowenc.EncDatum{Datum: tree.DNull}, // value
})
ca.metrics.ResolvedMessages.Inc(1)
}
return nil
}
// ConsumerClosed is part of the RowSource interface.
func (ca *changeAggregator) ConsumerClosed() {
// The consumer is done, Next() will not be called again.
ca.close()
}
type kvEventConsumer interface {
// ConsumeEvent responsible for consuming kv event.
ConsumeEvent(ctx context.Context, event kvevent.Event) error
}
type kvEventToRowConsumer struct {
frontier *span.Frontier
encoder Encoder
scratch bufalloc.ByteAllocator
sink Sink
cursor hlc.Timestamp
knobs TestingKnobs
rfCache *rowFetcherCache
details jobspb.ChangefeedDetails
kvFetcher row.SpanKVFetcher
}
var _ kvEventConsumer = &kvEventToRowConsumer{}
func newKVEventToRowConsumer(
ctx context.Context,
cfg *execinfra.ServerConfig,
frontier *span.Frontier,
cursor hlc.Timestamp,
sink Sink,
encoder Encoder,
details jobspb.ChangefeedDetails,
knobs TestingKnobs,
) kvEventConsumer {
rfCache := newRowFetcherCache(
ctx,
cfg.Codec,
cfg.LeaseManager.(*lease.Manager),
cfg.CollectionFactory,
cfg.DB,
)
return &kvEventToRowConsumer{
frontier: frontier,
encoder: encoder,
sink: sink,
cursor: cursor,
rfCache: rfCache,
details: details,
knobs: knobs,
}
}
type tableDescriptorTopic struct {
catalog.TableDescriptor
}
var _ TopicDescriptor = &tableDescriptorTopic{}
// ConsumeEvent implements kvEventConsumer interface
func (c *kvEventToRowConsumer) ConsumeEvent(ctx context.Context, ev kvevent.Event) error {
if ev.Type() != kvevent.TypeKV {
return errors.AssertionFailedf("expected kv ev, got %v", ev.Type())
}
r, err := c.eventToRow(ctx, ev)
if err != nil {
return err
}
// Ensure that r updates are strictly newer than the least resolved timestamp
// being tracked by the local span frontier. The poller should not be forwarding
// r updates that have timestamps less than or equal to any resolved timestamp
// it's forwarded before.
// TODO(dan): This should be an assertion once we're confident this can never
// happen under any circumstance.
if r.updated.LessEq(c.frontier.Frontier()) && !r.updated.Equal(c.cursor) {
log.Errorf(ctx, "cdc ux violation: detected timestamp %s that is less than "+
"or equal to the local frontier %s.", r.updated, c.frontier.Frontier())
return nil
}
var keyCopy, valueCopy []byte
encodedKey, err := c.encoder.EncodeKey(ctx, r)
if err != nil {
return err
}
c.scratch, keyCopy = c.scratch.Copy(encodedKey, 0 /* extraCap */)
encodedValue, err := c.encoder.EncodeValue(ctx, r)
if err != nil {
return err
}
c.scratch, valueCopy = c.scratch.Copy(encodedValue, 0 /* extraCap */)
if c.knobs.BeforeEmitRow != nil {
if err := c.knobs.BeforeEmitRow(ctx); err != nil {
return err
}
}
if err := c.sink.EmitRow(
ctx, tableDescriptorTopic{r.tableDesc},
keyCopy, valueCopy, r.updated, r.mvccTimestamp, ev.DetachAlloc(),
); err != nil {
return err
}
if log.V(3) {
log.Infof(ctx, `r %s: %s -> %s`, r.tableDesc.GetName(), keyCopy, valueCopy)
}
return nil
}
func (c *kvEventToRowConsumer) eventToRow(
ctx context.Context, event kvevent.Event,
) (encodeRow, error) {
var r encodeRow
schemaTimestamp := event.KV().Value.Timestamp
prevSchemaTimestamp := schemaTimestamp
mvccTimestamp := event.MVCCTimestamp()
if backfillTs := event.BackfillTimestamp(); !backfillTs.IsEmpty() {
schemaTimestamp = backfillTs
prevSchemaTimestamp = schemaTimestamp.Prev()
}
desc, err := c.rfCache.TableDescForKey(ctx, event.KV().Key, schemaTimestamp)
if err != nil {
return r, err
}
r.tableDesc = desc
rf, err := c.rfCache.RowFetcherForTableDesc(desc)
if err != nil {
return r, err
}
// Get new value.
// TODO(dan): Handle tables with multiple column families.
// Reuse kvs to save allocations.
c.kvFetcher.KVs = c.kvFetcher.KVs[:0]
c.kvFetcher.KVs = append(c.kvFetcher.KVs, event.KV())
if err := rf.StartScanFrom(ctx, &c.kvFetcher, false /* traceKV */); err != nil {
return r, err
}
r.datums, err = rf.NextRow(ctx)
if err != nil {
return r, err
}
if r.datums == nil {
return r, errors.AssertionFailedf("unexpected empty datums")
}
r.datums = append(rowenc.EncDatumRow(nil), r.datums...)
r.deleted = rf.RowIsDeleted()
r.updated = schemaTimestamp
r.mvccTimestamp = mvccTimestamp
// Assert that we don't get a second row from the row.Fetcher. We
// fed it a single KV, so that would be surprising.
nextRow := encodeRow{
tableDesc: desc,
}
nextRow.datums, err = rf.NextRow(ctx)
if err != nil {
return r, err
}
if nextRow.datums != nil {
return r, errors.AssertionFailedf("unexpected non-empty datums")
}
// Get prev value, if necessary.
_, withDiff := c.details.Opts[changefeedbase.OptDiff]
if withDiff {
prevRF := rf
r.prevTableDesc = r.tableDesc
if prevSchemaTimestamp != schemaTimestamp {
// If the previous value is being interpreted under a different
// version of the schema, fetch the correct table descriptor and
// create a new row.Fetcher with it.
prevDesc, err := c.rfCache.TableDescForKey(ctx, event.KV().Key, prevSchemaTimestamp)
if err != nil {
return r, err
}
r.prevTableDesc = prevDesc
prevRF, err = c.rfCache.RowFetcherForTableDesc(prevDesc)
if err != nil {
return r, err
}
}
prevKV := roachpb.KeyValue{Key: event.KV().Key, Value: event.PrevValue()}
// TODO(dan): Handle tables with multiple column families.
// Reuse kvs to save allocations.
c.kvFetcher.KVs = c.kvFetcher.KVs[:0]
c.kvFetcher.KVs = append(c.kvFetcher.KVs, prevKV)
if err := prevRF.StartScanFrom(ctx, &c.kvFetcher, false /* traceKV */); err != nil {
return r, err
}
r.prevDatums, err = prevRF.NextRow(ctx)
if err != nil {
return r, err
}
if r.prevDatums == nil {
return r, errors.AssertionFailedf("unexpected empty datums")
}
r.prevDatums = append(rowenc.EncDatumRow(nil), r.prevDatums...)
r.prevDeleted = prevRF.RowIsDeleted()
// Assert that we don't get a second row from the row.Fetcher. We
// fed it a single KV, so that would be surprising.
nextRow := encodeRow{
prevTableDesc: r.prevTableDesc,
}
nextRow.prevDatums, err = prevRF.NextRow(ctx)
if err != nil {
return r, err
}
if nextRow.prevDatums != nil {
return r, errors.AssertionFailedf("unexpected non-empty datums")
}
}
return r, nil
}
type nativeKVConsumer struct {
sink Sink
}
var _ kvEventConsumer = &nativeKVConsumer{}
func newNativeKVConsumer(sink Sink) kvEventConsumer {
return &nativeKVConsumer{sink: sink}
}
type noTopic struct{}
var _ TopicDescriptor = &noTopic{}
func (n noTopic) GetName() string {
return ""
}
func (n noTopic) GetID() descpb.ID {
return 0
}
func (n noTopic) GetVersion() descpb.DescriptorVersion {
return 0
}
// ConsumeEvent implements kvEventConsumer interface.
func (c *nativeKVConsumer) ConsumeEvent(ctx context.Context, ev kvevent.Event) error {
if ev.Type() != kvevent.TypeKV {
return errors.AssertionFailedf("expected kv ev, got %v", ev.Type())
}
keyBytes := []byte(ev.KV().Key)
val := ev.KV().Value
valBytes, err := protoutil.Marshal(&val)
if err != nil {
return err
}
return c.sink.EmitRow(
ctx, &noTopic{}, keyBytes, valBytes, val.Timestamp, val.Timestamp, ev.DetachAlloc())
}
const (
emitAllResolved = 0
emitNoResolved = -1
)
type changeFrontier struct {
execinfra.ProcessorBase
flowCtx *execinfra.FlowCtx
spec execinfrapb.ChangeFrontierSpec
memAcc mon.BoundAccount
a tree.DatumAlloc
// input returns rows from one or more changeAggregator processors
input execinfra.RowSource
// frontier contains the current resolved timestamp high-water for the tracked
// span set.
frontier *schemaChangeFrontier
// encoder is the Encoder to use for resolved timestamp serialization.
encoder Encoder
// sink is the Sink to write resolved timestamps to. Rows are never written
// by changeFrontier.
sink Sink
// freqEmitResolved, if >= 0, is a lower bound on the duration between
// resolved timestamp emits.
freqEmitResolved time.Duration
// lastEmitResolved is the last time a resolved timestamp was emitted.
lastEmitResolved time.Time
// slowLogEveryN rate-limits the logging of slow spans
slowLogEveryN log.EveryN
// js, if non-nil, is called to checkpoint the changefeed's
// progress in the corresponding system job entry.
js *jobState
// highWaterAtStart is the greater of the job high-water and the timestamp the
// CHANGEFEED statement was run at. It's used in an assertion that we never
// regress the job high-water.
highWaterAtStart hlc.Timestamp
// passthroughBuf, in some but not all flows, contains changed row data to
// pass through unchanged to the gateway node.
passthroughBuf encDatumRowBuffer
// resolvedBuf, if non-nil, contains rows indicating a changefeed-level
// resolved timestamp to be returned. It depends on everything in
// `passthroughBuf` being sent, so that one needs to be emptied first.
resolvedBuf *encDatumRowBuffer
// metrics are monitoring counters shared between all changefeeds.
metrics *Metrics
// metricsID is used as the unique id of this changefeed in the
// metrics.MaxBehindNanos map.
metricsID int
}
const (
runStatusUpdateFrequency time.Duration = time.Minute
slowSpanMaxFrequency = 10 * time.Second
)
// jobState encapsulates changefeed job state.
type jobState struct {
job *jobs.Job
settings *cluster.Settings
metrics *Metrics
ts timeutil.TimeSource
// The last time we updated job run status.
lastRunStatusUpdate time.Time
// The last time we updated job progress.
lastProgressUpdate time.Time
// How long checkpoint (job progress update) expected to take.
checkpointDuration time.Duration
// Flag set if we skip some updates due to rapid progress update requests.
progressUpdatesSkipped bool
}
func newJobState(
j *jobs.Job, st *cluster.Settings, metrics *Metrics, ts timeutil.TimeSource,
) *jobState {
return &jobState{
job: j,
settings: st,
metrics: metrics,
ts: ts,
lastProgressUpdate: ts.Now(),
}
}
func canCheckpointBackfill(sv *settings.Values, lastCheckpoint time.Time) bool {
freq := changefeedbase.FrontierCheckpointFrequency.Get(sv)
if freq == 0 {
return false
}
return timeutil.Since(lastCheckpoint) > freq
}
func (j *jobState) canCheckpointBackfill() bool {
return canCheckpointBackfill(&j.settings.SV, j.lastProgressUpdate)
}
// canCheckpointHighWatermark returns true if we should update job high water mark (i.e. progress).
// Normally, whenever frontier changes, we update high water mark.
// However, if the rate of frontier changes is too high, we want to slow down
// the frequency of job progress updates. We do this by skipping some updates
// if the time to update the job progress is greater than the delta between
// previous and the current progress update time.
func (j *jobState) canCheckpointHighWatermark(frontierChanged bool) bool {
if !(frontierChanged || j.progressUpdatesSkipped) {
return false
}
minAdvance := changefeedbase.MinHighWaterMarkCheckpointAdvance.Get(&j.settings.SV)
if j.checkpointDuration > 0 &&
j.ts.Now().Before(j.lastProgressUpdate.Add(j.checkpointDuration+minAdvance)) {
// Updates are too rapid; skip some.
j.progressUpdatesSkipped = true
return false
}