-
Notifications
You must be signed in to change notification settings - Fork 274
/
changefeed.go
executable file
·1061 lines (942 loc) · 33.3 KB
/
changefeed.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 2021 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.
package owner
import (
"context"
"fmt"
"math"
"strings"
"sync"
"time"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/log"
"github.com/pingcap/tidb/pkg/errno"
"github.com/pingcap/tiflow/cdc/async"
"github.com/pingcap/tiflow/cdc/entry"
"github.com/pingcap/tiflow/cdc/model"
"github.com/pingcap/tiflow/cdc/puller"
"github.com/pingcap/tiflow/cdc/redo"
"github.com/pingcap/tiflow/cdc/scheduler"
"github.com/pingcap/tiflow/cdc/scheduler/schedulepb"
"github.com/pingcap/tiflow/cdc/vars"
"github.com/pingcap/tiflow/pkg/config"
cerror "github.com/pingcap/tiflow/pkg/errors"
pfilter "github.com/pingcap/tiflow/pkg/filter"
"github.com/pingcap/tiflow/pkg/pdutil"
redoCfg "github.com/pingcap/tiflow/pkg/redo"
"github.com/pingcap/tiflow/pkg/sink/observer"
"github.com/pingcap/tiflow/pkg/txnutil/gc"
"github.com/pingcap/tiflow/pkg/upstream"
"github.com/pingcap/tiflow/pkg/util"
"github.com/prometheus/client_golang/prometheus"
"github.com/tikv/client-go/v2/oracle"
"go.uber.org/atomic"
"go.uber.org/zap"
)
// Changefeed is the tick logic of changefeed.
type Changefeed interface {
// Tick is called periodically to drive the changefeed's internal logic.
// The main logic of changefeed is in this function, including the calculation of many kinds of ts,
// maintain table components, error handling, etc.
//
// It can be called in etcd ticks, so it should never be blocked.
// Tick Returns: checkpointTs, minTableBarrierTs
Tick(context.Context, *model.ChangeFeedInfo,
*model.ChangeFeedStatus,
map[model.CaptureID]*model.CaptureInfo) (model.Ts, model.Ts)
// Close closes the changefeed.
Close(ctx context.Context)
// GetScheduler returns the scheduler of this changefeed.
GetScheduler() scheduler.Scheduler
}
var _ Changefeed = (*changefeed)(nil)
// newScheduler creates a new scheduler from context.
// This function is factored out to facilitate unit testing.
func newScheduler(
ctx context.Context,
changeFeedID model.ChangeFeedID,
up *upstream.Upstream, epoch uint64,
cfg *config.SchedulerConfig,
redoMetaManager redo.MetaManager,
globalVars *vars.GlobalVars,
) (scheduler.Scheduler, error) {
messageServer := globalVars.MessageServer
messageRouter := globalVars.MessageRouter
ownerRev := globalVars.OwnerRevision
captureID := globalVars.CaptureInfo.ID
ret, err := scheduler.NewScheduler(
ctx, captureID, changeFeedID, messageServer, messageRouter, ownerRev, epoch, up, cfg, redoMetaManager)
return ret, errors.Trace(err)
}
type changefeed struct {
id model.ChangeFeedID
upstream *upstream.Upstream
cfg *config.SchedulerConfig
scheduler scheduler.Scheduler
globalVars *vars.GlobalVars
// barriers will be created when a changefeed is initialized
// and will be destroyed when a changefeed is closed.
barriers *barriers
feedStateManager FeedStateManager
resolvedTs model.Ts
// lastSyncedTs is the lastest resolvedTs that has been synced to downstream.
// pullerResolvedTs is the minimum resolvedTs of all pullers.
// we don't need to initialize lastSyncedTs and pullerResolvedTs specially
// because it will be updated in tick.
lastSyncedTs model.Ts
pullerResolvedTs model.Ts
// ddl related fields
ddlManager *ddlManager
redoDDLMgr redo.DDLManager
redoMetaMgr redo.MetaManager
schema entry.SchemaStorage
ddlSink DDLSink
ddlPuller puller.DDLPuller
// The changefeed will start a backend goroutine in the function `initialize`
// for DDLPuller and redo manager. `wg` is used to manage this backend goroutine.
wg sync.WaitGroup
// state related fields
initialized *atomic.Bool
initializer *async.Initializer
// isRemoved is true if the changefeed is removed,
// which means it will be removed from memory forever
isRemoved bool
// isReleased is true if the changefeed's resources were released,
// but it will still be kept in the memory, and it will be check
// in every tick. Such as the changefeed that is stopped or encountered an error.
isReleased bool
errCh chan error
warningCh chan error
// cancel the running goroutine start by `DDLPuller`
cancel context.CancelFunc
metricsChangefeedCheckpointTsGauge prometheus.Gauge
metricsChangefeedCheckpointTsLagGauge prometheus.Gauge
metricsChangefeedCheckpointLagDuration prometheus.Observer
metricsChangefeedResolvedTsGauge prometheus.Gauge
metricsChangefeedResolvedTsLagGauge prometheus.Gauge
metricsChangefeedResolvedTsLagDuration prometheus.Observer
metricsCurrentPDTsGauge prometheus.Gauge
metricsChangefeedBarrierTsGauge prometheus.Gauge
metricsChangefeedTickDuration prometheus.Observer
metricsChangefeedCreateTimeGuage prometheus.Gauge
metricsChangefeedRestartTimeGauge prometheus.Gauge
downstreamObserver observer.Observer
observerLastTick *atomic.Time
newDDLPuller func(ctx context.Context,
up *upstream.Upstream,
startTs uint64,
changefeed model.ChangeFeedID,
schemaStorage entry.SchemaStorage,
filter pfilter.Filter,
) puller.DDLPuller
newSink func(
changefeedID model.ChangeFeedID, info *model.ChangeFeedInfo,
reportError func(err error), reportWarning func(err error),
) DDLSink
newScheduler func(
ctx context.Context, changefeedID model.ChangeFeedID,
up *upstream.Upstream, epoch uint64, cfg *config.SchedulerConfig,
redoMetaManager redo.MetaManager, globalVars *vars.GlobalVars,
) (scheduler.Scheduler, error)
newDownstreamObserver func(
ctx context.Context,
changefeedID model.ChangeFeedID,
sinkURIStr string, replCfg *config.ReplicaConfig,
opts ...observer.NewObserverOption,
) (observer.Observer, error)
lastDDLTs uint64 // Timestamp of the last executed DDL. Only used for tests.
// The latest changefeed info and status from meta storage. they are updated in every Tick.
latestInfo *model.ChangeFeedInfo
latestStatus *model.ChangeFeedStatus
}
func (c *changefeed) GetScheduler() scheduler.Scheduler {
return c.scheduler
}
// NewChangefeed creates a new changefeed.
func NewChangefeed(
id model.ChangeFeedID,
cfInfo *model.ChangeFeedInfo,
cfStatus *model.ChangeFeedStatus,
feedStateManager FeedStateManager,
up *upstream.Upstream,
cfg *config.SchedulerConfig,
globalVars *vars.GlobalVars,
) *changefeed {
c := &changefeed{
id: id,
latestInfo: cfInfo,
latestStatus: cfStatus,
// The scheduler will be created lazily.
scheduler: nil,
barriers: newBarriers(),
feedStateManager: feedStateManager,
upstream: up,
errCh: make(chan error, defaultErrChSize),
warningCh: make(chan error, defaultErrChSize),
cancel: func() {},
newDDLPuller: puller.NewDDLPuller,
newSink: newDDLSink,
newDownstreamObserver: observer.NewObserver,
initialized: atomic.NewBool(false),
globalVars: globalVars,
}
c.newScheduler = newScheduler
c.cfg = cfg
c.initializer = async.NewInitializer()
return c
}
func newChangefeed4Test(
id model.ChangeFeedID,
cfInfo *model.ChangeFeedInfo,
cfStatus *model.ChangeFeedStatus,
cfstateManager FeedStateManager, up *upstream.Upstream,
newDDLPuller func(ctx context.Context,
up *upstream.Upstream,
startTs uint64,
changefeed model.ChangeFeedID,
schemaStorage entry.SchemaStorage,
filter pfilter.Filter,
) puller.DDLPuller,
newSink func(
changefeedID model.ChangeFeedID, info *model.ChangeFeedInfo,
reportError func(err error), reportWarning func(err error),
) DDLSink,
newScheduler func(ctx context.Context, id model.ChangeFeedID,
up *upstream.Upstream, epoch uint64, cfg *config.SchedulerConfig, redoMetaManager redo.MetaManager,
globalVars *vars.GlobalVars,
) (scheduler.Scheduler, error),
newDownstreamObserver func(
ctx context.Context,
changefeedID model.ChangeFeedID,
sinkURIStr string, replCfg *config.ReplicaConfig,
opts ...observer.NewObserverOption,
) (observer.Observer, error),
globalVars *vars.GlobalVars,
) *changefeed {
cfg := config.NewDefaultSchedulerConfig()
c := NewChangefeed(id, cfInfo, cfStatus, cfstateManager, up, cfg, globalVars)
c.newDDLPuller = newDDLPuller
c.newSink = newSink
c.newScheduler = newScheduler
c.newDownstreamObserver = newDownstreamObserver
return c
}
func (c *changefeed) Tick(ctx context.Context,
cfInfo *model.ChangeFeedInfo,
cfStatus *model.ChangeFeedStatus,
captures map[model.CaptureID]*model.CaptureInfo,
) (model.Ts, model.Ts) {
startTime := time.Now()
c.latestInfo = cfInfo
c.latestStatus = cfStatus
// Handle all internal warnings.
noMoreWarnings := false
for !noMoreWarnings {
select {
case err := <-c.warningCh:
c.handleWarning(err)
default:
noMoreWarnings = true
}
}
if skip, err := c.checkUpstream(); skip {
if err != nil {
c.handleErr(ctx, err)
}
return 0, 0
}
checkpointTs, minTableBarrierTs, err := c.tick(ctx, captures, cfInfo, cfStatus)
// The tick duration is recorded only if changefeed has completed initialization
if c.initialized.Load() {
costTime := time.Since(startTime)
if costTime > changefeedLogsWarnDuration {
log.Warn("changefeed tick took too long",
zap.String("namespace", c.id.Namespace),
zap.String("changefeed", c.id.ID),
zap.Duration("duration", costTime))
}
c.metricsChangefeedTickDuration.Observe(costTime.Seconds())
}
if err != nil {
log.Error("changefeed tick failed", zap.Error(err))
c.handleErr(ctx, err)
}
return checkpointTs, minTableBarrierTs
}
func (c *changefeed) Throw(ctx context.Context) func(error) {
return func(err error) {
select {
case <-ctx.Done():
case c.errCh <- errors.Trace(err):
}
}
}
func (c *changefeed) handleErr(ctx context.Context, err error) {
log.Error("an error occurred in Owner",
zap.String("namespace", c.id.Namespace),
zap.String("changefeed", c.id.ID), zap.Error(err))
var code string
if rfcCode, ok := cerror.RFCCode(err); ok {
code = string(rfcCode)
} else {
code = string(cerror.ErrOwnerUnknown.RFCCode())
}
c.feedStateManager.HandleError(&model.RunningError{
Time: time.Now(),
Addr: config.GetGlobalServerConfig().AdvertiseAddr,
Code: code,
Message: err.Error(),
})
c.releaseResources(ctx)
}
func (c *changefeed) handleWarning(err error) {
log.Warn("an warning occurred in Owner",
zap.String("namespace", c.id.Namespace),
zap.String("changefeed", c.id.ID), zap.Error(err))
var code string
if rfcCode, ok := cerror.RFCCode(err); ok {
code = string(rfcCode)
} else {
code = string(cerror.ErrOwnerUnknown.RFCCode())
}
c.feedStateManager.HandleWarning(&model.RunningError{
Time: time.Now(),
Addr: config.GetGlobalServerConfig().AdvertiseAddr,
Code: code,
Message: err.Error(),
})
}
func (c *changefeed) checkStaleCheckpointTs(
ctx context.Context, checkpointTs uint64,
cfInfo *model.ChangeFeedInfo,
) error {
if cfInfo.NeedBlockGC() {
failpoint.Inject("InjectChangefeedFastFailError", func() error {
return cerror.ErrStartTsBeforeGC.FastGen("InjectChangefeedFastFailError")
})
if err := c.upstream.GCManager.CheckStaleCheckpointTs(ctx, c.id, checkpointTs); err != nil {
return errors.Trace(err)
}
}
return nil
}
// tick is the main logic of changefeed.
// tick returns the checkpointTs and minTableBarrierTs.
func (c *changefeed) tick(ctx context.Context,
captures map[model.CaptureID]*model.CaptureInfo,
cfInfo *model.ChangeFeedInfo,
cfStatus *model.ChangeFeedStatus,
) (model.Ts, model.Ts, error) {
adminJobPending := c.feedStateManager.Tick(c.resolvedTs, cfStatus, cfInfo)
preCheckpointTs := cfInfo.GetCheckpointTs(cfStatus)
// checkStaleCheckpointTs must be called before `feedStateManager.ShouldRunning()`
// to ensure all changefeeds, no matter whether they are running or not, will be checked.
if err := c.checkStaleCheckpointTs(ctx, preCheckpointTs, cfInfo); err != nil {
return 0, 0, errors.Trace(err)
}
if !c.feedStateManager.ShouldRunning() {
c.isRemoved = c.feedStateManager.ShouldRemoved()
c.releaseResources(ctx)
return 0, 0, nil
}
if adminJobPending {
return 0, 0, nil
}
if !c.initialized.Load() {
initialized, err := c.initializer.TryInitialize(ctx,
func(ctx context.Context) error {
return c.initialize(ctx, cfInfo, cfStatus)
},
c.globalVars.ChangefeedThreadPool)
if err != nil {
return 0, 0, errors.Trace(err)
}
if !initialized {
return 0, 0, nil
}
}
select {
case err := <-c.errCh:
return 0, 0, errors.Trace(err)
default:
}
if c.redoMetaMgr.Enabled() {
if !c.redoMetaMgr.Running() {
return 0, 0, nil
}
}
allPhysicalTables, barrier, err := c.ddlManager.tick(ctx, preCheckpointTs)
if err != nil {
return 0, 0, errors.Trace(err)
}
err = c.handleBarrier(ctx, cfInfo, cfStatus, barrier)
if err != nil {
return 0, 0, errors.Trace(err)
}
log.Debug("owner handles barrier",
zap.String("namespace", c.id.Namespace),
zap.String("changefeed", c.id.ID),
zap.Uint64("preCheckpointTs", preCheckpointTs),
zap.Uint64("preResolvedTs", c.resolvedTs),
zap.Uint64("globalBarrierTs", barrier.GlobalBarrierTs),
zap.Uint64("minTableBarrierTs", barrier.MinTableBarrierTs),
zap.Any("tableBarrier", barrier.TableBarriers))
if barrier.GlobalBarrierTs < preCheckpointTs {
// This condition implies that the DDL resolved-ts has not yet reached checkpointTs,
// which implies that it would be premature to schedule tables or to update status.
// So we return here.
return 0, 0, nil
}
watermark, err := c.scheduler.Tick(
ctx, preCheckpointTs, allPhysicalTables, captures,
barrier)
if err != nil {
return 0, 0, errors.Trace(err)
}
if watermark.LastSyncedTs != scheduler.CheckpointCannotProceed {
if c.lastSyncedTs < watermark.LastSyncedTs {
c.lastSyncedTs = watermark.LastSyncedTs
} else if c.lastSyncedTs > watermark.LastSyncedTs {
log.Warn("LastSyncedTs should not be greater than newLastSyncedTs",
zap.Uint64("c.LastSyncedTs", c.lastSyncedTs),
zap.Uint64("newLastSyncedTs", watermark.LastSyncedTs))
}
}
if watermark.PullerResolvedTs != scheduler.CheckpointCannotProceed && watermark.PullerResolvedTs != math.MaxUint64 {
if watermark.PullerResolvedTs > c.pullerResolvedTs {
c.pullerResolvedTs = watermark.PullerResolvedTs
} else if watermark.PullerResolvedTs < c.pullerResolvedTs {
log.Warn("the newPullerResolvedTs should not be smaller than c.pullerResolvedTs",
zap.Uint64("c.pullerResolvedTs", c.pullerResolvedTs),
zap.Uint64("newPullerResolvedTs", watermark.PullerResolvedTs))
}
}
pdTime := c.upstream.PDClock.CurrentTime()
currentTs := oracle.GetPhysical(pdTime)
// CheckpointCannotProceed implies that not all tables are being replicated normally,
// so in that case there is no need to advance the global watermarks.
if watermark.CheckpointTs == scheduler.CheckpointCannotProceed {
if cfStatus != nil {
// We should keep the metrics updated even if the scheduler cannot
// advance the watermarks for now.
c.updateMetrics(currentTs, cfStatus.CheckpointTs, c.resolvedTs)
}
return 0, 0, nil
}
log.Debug("owner prepares to update status",
zap.Uint64("prevResolvedTs", c.resolvedTs),
zap.Uint64("newResolvedTs", watermark.ResolvedTs),
zap.Uint64("newCheckpointTs", watermark.CheckpointTs),
zap.String("namespace", c.id.Namespace),
zap.String("changefeed", c.id.ID))
// resolvedTs should never regress.
if watermark.ResolvedTs > c.resolvedTs {
c.resolvedTs = watermark.ResolvedTs
}
// MinTableBarrierTs should never regress
if barrier.MinTableBarrierTs < cfStatus.MinTableBarrierTs {
barrier.MinTableBarrierTs = cfStatus.MinTableBarrierTs
}
failpoint.Inject("ChangefeedOwnerDontUpdateCheckpoint", func() {
if c.lastDDLTs != 0 && cfStatus.CheckpointTs >= c.lastDDLTs {
log.Info("owner won't update checkpoint because of failpoint",
zap.String("namespace", c.id.Namespace),
zap.String("changefeed", c.id.ID),
zap.Uint64("keepCheckpoint", cfStatus.CheckpointTs),
zap.Uint64("skipCheckpoint", watermark.CheckpointTs))
watermark.CheckpointTs = cfStatus.CheckpointTs
}
})
failpoint.Inject("ChangefeedOwnerNotUpdateCheckpoint", func() {
watermark.CheckpointTs = cfStatus.CheckpointTs
})
c.updateMetrics(currentTs, watermark.CheckpointTs, c.resolvedTs)
c.tickDownstreamObserver(ctx)
return watermark.CheckpointTs, barrier.MinTableBarrierTs, nil
}
func (c *changefeed) initialize(ctx context.Context,
cfInfo *model.ChangeFeedInfo,
cfStatus *model.ChangeFeedStatus,
) (err error) {
if c.initialized.Load() || cfStatus == nil {
// If `c.latestStatus` is nil it means the changefeed struct is just created, it needs to
// 1. use startTs as checkpointTs and resolvedTs, if it's a new created changefeed; or
// 2. load checkpointTs and resolvedTs from etcd, if it's an existing changefeed.
// And then it can continue to initialize.
return nil
}
c.isReleased = false
// clean the errCh
// When the changefeed is resumed after being stopped, the changefeed instance will be reused,
// So we should make sure that the errCh is empty when the changefeed is restarting
LOOP1:
for {
select {
case <-c.errCh:
default:
break LOOP1
}
}
LOOP2:
for {
select {
case <-c.warningCh:
default:
break LOOP2
}
}
checkpointTs := cfStatus.CheckpointTs
if c.resolvedTs == 0 {
c.resolvedTs = checkpointTs
}
minTableBarrierTs := cfStatus.MinTableBarrierTs
failpoint.Inject("NewChangefeedNoRetryError", func() {
failpoint.Return(cerror.ErrStartTsBeforeGC.GenWithStackByArgs(checkpointTs-300, checkpointTs))
})
failpoint.Inject("NewChangefeedRetryError", func() {
failpoint.Return(errors.New("failpoint injected retriable error"))
})
if cfInfo.Config.CheckGCSafePoint {
// Check TiDB GC safepoint does not exceed the checkpoint.
//
// We update TTL to 10 minutes,
// 1. to delete the service GC safepoint effectively,
// 2. in case owner update TiCDC service GC safepoint fails.
//
// Also, it unblocks TiDB GC, because the service GC safepoint is set to
// 1 hour TTL during creating changefeed.
//
// See more gc doc.
ensureTTL := int64(10 * 60)
err = gc.EnsureChangefeedStartTsSafety(
ctx, c.upstream.PDClient,
c.globalVars.EtcdClient.GetEnsureGCServiceID(gc.EnsureGCServiceInitializing),
c.id, ensureTTL, checkpointTs)
if err != nil {
return errors.Trace(err)
}
// clean service GC safepoint '-creating-' and '-resuming-' if there are any.
err = gc.UndoEnsureChangefeedStartTsSafety(
ctx, c.upstream.PDClient,
c.globalVars.EtcdClient.GetEnsureGCServiceID(gc.EnsureGCServiceCreating),
c.id,
)
if err != nil {
return errors.Trace(err)
}
err = gc.UndoEnsureChangefeedStartTsSafety(
ctx, c.upstream.PDClient,
c.globalVars.EtcdClient.GetEnsureGCServiceID(gc.EnsureGCServiceResuming),
c.id,
)
if err != nil {
return errors.Trace(err)
}
}
var ddlStartTs model.Ts
// This means there was a ddl job when the changefeed was paused.
// We don't know whether the ddl job is finished or not, so we need to
// start the ddl puller from the `checkpointTs-1` to execute the ddl job
// again.
// FIXME: TiCDC can't handle some ddl jobs correctly in this situation.
// For example, if the ddl job is `add index`, TiCDC will execute the ddl
// job again and cause the index to be added twice. We need to fix this
// problem in the future. See:https://github.com/pingcap/tiflow/issues/2543
if checkpointTs == minTableBarrierTs {
ddlStartTs = checkpointTs - 1
} else {
ddlStartTs = checkpointTs
}
c.barriers = newBarriers()
if util.GetOrZero(cfInfo.Config.EnableSyncPoint) {
// firstSyncPointStartTs = k * syncPointInterval,
// which >= startTs, and choose the minimal k
syncPointInterval := util.GetOrZero(cfInfo.Config.SyncPointInterval)
k := oracle.GetTimeFromTS(c.resolvedTs).Sub(time.Unix(0, 0)) / syncPointInterval
if oracle.GetTimeFromTS(c.resolvedTs).Sub(time.Unix(0, 0))%syncPointInterval != 0 || oracle.ExtractLogical(c.resolvedTs) != 0 {
k += 1
}
firstSyncPointTs := oracle.GoTimeToTS(time.Unix(0, 0).Add(k * syncPointInterval))
c.barriers.Update(syncPointBarrier, firstSyncPointTs)
}
c.barriers.Update(finishBarrier, cfInfo.GetTargetTs())
filter, err := pfilter.NewFilter(cfInfo.Config, "")
if err != nil {
return errors.Trace(err)
}
c.schema, err = entry.NewSchemaStorage(
c.upstream.KVStorage, ddlStartTs,
cfInfo.Config.ForceReplicate, c.id, util.RoleOwner, filter)
if err != nil {
return errors.Trace(err)
}
cancelCtx, cancel := context.WithCancel(ctx)
c.cancel = cancel
sourceID, err := pdutil.GetSourceID(ctx, c.upstream.PDClient)
if err != nil {
return errors.Trace(err)
}
cfInfo.Config.Sink.TiDBSourceID = sourceID
log.Info("get sourceID from PD", zap.Uint64("sourceID", sourceID), zap.Stringer("changefeedID", c.id))
c.ddlSink = c.newSink(c.id, cfInfo, c.Throw(ctx), func(err error) {
select {
case <-ctx.Done():
case c.warningCh <- err:
}
})
c.ddlSink.run(cancelCtx)
c.ddlPuller = c.newDDLPuller(cancelCtx, c.upstream, ddlStartTs, c.id, c.schema, filter)
c.wg.Add(1)
go func() {
defer c.wg.Done()
c.Throw(ctx)(c.ddlPuller.Run(cancelCtx))
}()
c.downstreamObserver, err = c.newDownstreamObserver(ctx, c.id, cfInfo.SinkURI, cfInfo.Config)
if err != nil {
return err
}
c.observerLastTick = atomic.NewTime(time.Time{})
c.redoDDLMgr = redo.NewDDLManager(c.id, cfInfo.Config.Consistent, ddlStartTs)
if c.redoDDLMgr.Enabled() {
c.wg.Add(1)
go func() {
defer c.wg.Done()
c.Throw(ctx)(c.redoDDLMgr.Run(cancelCtx))
}()
}
c.redoMetaMgr = redo.NewMetaManager(c.id, cfInfo.Config.Consistent, checkpointTs)
if c.redoMetaMgr.Enabled() {
c.wg.Add(1)
go func() {
defer c.wg.Done()
c.Throw(ctx)(c.redoMetaMgr.Run(cancelCtx))
}()
log.Info("owner creates redo manager",
zap.String("namespace", c.id.Namespace),
zap.String("changefeed", c.id.ID))
}
c.ddlManager = newDDLManager(
c.id,
ddlStartTs,
cfStatus.CheckpointTs,
c.ddlSink,
filter,
c.ddlPuller,
c.schema,
c.redoDDLMgr,
c.redoMetaMgr,
util.GetOrZero(cfInfo.Config.BDRMode),
cfInfo.Config.Sink.ShouldSendAllBootstrapAtStart(),
)
// create scheduler
cfg := *c.cfg
cfg.ChangefeedSettings = cfInfo.Config.Scheduler
epoch := cfInfo.Epoch
c.scheduler, err = c.newScheduler(ctx, c.id, c.upstream, epoch, &cfg, c.redoMetaMgr, c.globalVars)
if err != nil {
return errors.Trace(err)
}
c.initMetrics()
c.initialized.Store(true)
c.metricsChangefeedCreateTimeGuage.Set(float64(oracle.GetPhysical(cfInfo.CreateTime)))
c.metricsChangefeedRestartTimeGauge.Set(float64(oracle.GetPhysical(time.Now())))
log.Info("changefeed initialized",
zap.String("namespace", c.id.Namespace),
zap.String("changefeed", c.id.ID),
zap.Uint64("changefeedEpoch", epoch),
zap.Uint64("checkpointTs", checkpointTs),
zap.Uint64("resolvedTs", c.resolvedTs),
zap.String("info", cfInfo.String()))
return nil
}
func (c *changefeed) initMetrics() {
c.metricsChangefeedCheckpointTsGauge = changefeedCheckpointTsGauge.
WithLabelValues(c.id.Namespace, c.id.ID)
c.metricsChangefeedCheckpointTsLagGauge = changefeedCheckpointTsLagGauge.
WithLabelValues(c.id.Namespace, c.id.ID)
c.metricsChangefeedCheckpointLagDuration = changefeedCheckpointLagDuration.
WithLabelValues(c.id.Namespace, c.id.ID)
c.metricsChangefeedResolvedTsGauge = changefeedResolvedTsGauge.
WithLabelValues(c.id.Namespace, c.id.ID)
c.metricsChangefeedResolvedTsLagGauge = changefeedResolvedTsLagGauge.
WithLabelValues(c.id.Namespace, c.id.ID)
c.metricsChangefeedResolvedTsLagDuration = changefeedResolvedTsLagDuration.
WithLabelValues(c.id.Namespace, c.id.ID)
c.metricsCurrentPDTsGauge = currentPDTsGauge.WithLabelValues(c.id.Namespace, c.id.ID)
c.metricsChangefeedBarrierTsGauge = changefeedBarrierTsGauge.
WithLabelValues(c.id.Namespace, c.id.ID)
c.metricsChangefeedTickDuration = changefeedTickDuration.
WithLabelValues(c.id.Namespace, c.id.ID)
c.metricsChangefeedCreateTimeGuage = changefeedStartTimeGauge.
WithLabelValues(c.id.Namespace, c.id.ID, "create")
c.metricsChangefeedRestartTimeGauge = changefeedStartTimeGauge.
WithLabelValues(c.id.Namespace, c.id.ID, "restart")
}
// releaseResources is idempotent.
func (c *changefeed) releaseResources(ctx context.Context) {
c.initializer.Terminate()
c.cleanupMetrics()
if c.isReleased {
return
}
// Must clean redo manager before calling cancel, otherwise
// the manager can be closed internally.
c.cleanupRedoManager(ctx, c.latestInfo)
c.cleanupChangefeedServiceGCSafePoints(ctx)
if c.cancel != nil {
c.cancel()
}
c.cancel = func() {}
if c.ddlPuller != nil {
c.ddlPuller.Close()
}
c.wg.Wait()
if c.ddlSink != nil {
canceledCtx, cancel := context.WithCancel(context.Background())
cancel()
// TODO(dongmen): remove ctx from func ddlSink.close(), it is useless.
// We don't need to wait ddlSink Close, pass a canceled context is ok
if err := c.ddlSink.close(canceledCtx); err != nil {
log.Warn("owner close ddlSink failed",
zap.String("namespace", c.id.Namespace),
zap.String("changefeed", c.id.ID),
zap.Error(err))
}
}
if c.scheduler != nil {
c.scheduler.Close(ctx)
c.scheduler = nil
}
if c.downstreamObserver != nil {
_ = c.downstreamObserver.Close()
}
c.schema = nil
c.barriers = nil
c.resolvedTs = 0
c.initialized.Store(false)
c.isReleased = true
// when closing a changefeed, we must clean the warningCh.
// otherwise, the old warning errors will be handled when the reused changefeed instance is ticked again
OUT:
for {
select {
case err := <-c.warningCh:
log.Warn("drain owner warnings",
zap.String("namespace", c.id.Namespace),
zap.String("changefeed", c.id.ID),
zap.Error(err))
default:
break OUT
}
}
log.Info("changefeed closed",
zap.String("namespace", c.id.Namespace),
zap.String("changefeed", c.id.ID),
zap.Bool("isRemoved", c.isRemoved))
}
func (c *changefeed) cleanupMetrics() {
changefeedCheckpointTsGauge.DeleteLabelValues(c.id.Namespace, c.id.ID)
changefeedCheckpointTsLagGauge.DeleteLabelValues(c.id.Namespace, c.id.ID)
changefeedCheckpointLagDuration.DeleteLabelValues(c.id.Namespace, c.id.ID)
c.metricsChangefeedCheckpointTsGauge = nil
c.metricsChangefeedCheckpointTsLagGauge = nil
c.metricsChangefeedCheckpointLagDuration = nil
changefeedResolvedTsGauge.DeleteLabelValues(c.id.Namespace, c.id.ID)
changefeedResolvedTsLagGauge.DeleteLabelValues(c.id.Namespace, c.id.ID)
changefeedResolvedTsLagDuration.DeleteLabelValues(c.id.Namespace, c.id.ID)
currentPDTsGauge.DeleteLabelValues(c.id.Namespace, c.id.ID)
c.metricsChangefeedResolvedTsGauge = nil
c.metricsChangefeedResolvedTsLagGauge = nil
c.metricsChangefeedResolvedTsLagDuration = nil
c.metricsCurrentPDTsGauge = nil
changefeedTickDuration.DeleteLabelValues(c.id.Namespace, c.id.ID)
c.metricsChangefeedTickDuration = nil
changefeedBarrierTsGauge.DeleteLabelValues(c.id.Namespace, c.id.ID)
c.metricsChangefeedBarrierTsGauge = nil
if c.isRemoved {
changefeedStatusGauge.DeleteLabelValues(c.id.Namespace, c.id.ID)
changefeedCheckpointTsGauge.DeleteLabelValues(c.id.Namespace, c.id.ID, "create")
changefeedCheckpointTsLagGauge.DeleteLabelValues(c.id.Namespace, c.id.ID, "restart")
}
}
// cleanup redo logs if changefeed is removed and redo log is enabled
func (c *changefeed) cleanupRedoManager(ctx context.Context, cfInfo *model.ChangeFeedInfo) {
if c.isRemoved {
if cfInfo == nil || cfInfo.Config == nil ||
cfInfo.Config.Consistent == nil {
log.Warn("changefeed is removed, but state is not complete", zap.Any("info", cfInfo))
return
}
if !redoCfg.IsConsistentEnabled(cfInfo.Config.Consistent.Level) {
return
}
// when removing a paused changefeed, the redo manager is nil, create a new one
if c.redoMetaMgr == nil {
c.redoMetaMgr = redo.NewMetaManager(c.id, cfInfo.Config.Consistent, 0)
}
err := c.redoMetaMgr.Cleanup(ctx)
if err != nil {
log.Error("cleanup redo logs failed", zap.String("changefeed", c.id.ID), zap.Error(err))
}
}
}
func (c *changefeed) cleanupChangefeedServiceGCSafePoints(ctx context.Context) {
if !c.isRemoved {
return
}
serviceIDs := []string{
c.globalVars.EtcdClient.GetEnsureGCServiceID(gc.EnsureGCServiceCreating),
c.globalVars.EtcdClient.GetEnsureGCServiceID(gc.EnsureGCServiceResuming),
c.globalVars.EtcdClient.GetEnsureGCServiceID(gc.EnsureGCServiceInitializing),
}
for _, serviceID := range serviceIDs {
err := gc.UndoEnsureChangefeedStartTsSafety(
ctx,
c.upstream.PDClient,
serviceID,
c.id)
if err != nil {
log.Error("failed to remove gc safepoint",
zap.String("namespace", c.id.Namespace),
zap.String("changefeed", c.id.ID),
zap.String("serviceID", serviceID))
}
}
}
// handleBarrier calculates the barrierTs of the changefeed.
// barrierTs is used to control the data that can be flush to downstream.
func (c *changefeed) handleBarrier(ctx context.Context,
cfInfo *model.ChangeFeedInfo,
cfStatus *model.ChangeFeedStatus,
barrier *schedulepb.BarrierWithMinTs,
) error {
barrierTp, barrierTs := c.barriers.Min()
c.metricsChangefeedBarrierTsGauge.Set(float64(oracle.ExtractPhysical(barrierTs)))
// It means:
// 1. All data before the barrierTs was sent to downstream.
// 2. No more data after barrierTs was sent to downstream.
checkpointReachBarrier := barrierTs == cfStatus.CheckpointTs
if checkpointReachBarrier {
switch barrierTp {
case syncPointBarrier:
nextSyncPointTs := oracle.GoTimeToTS(
oracle.GetTimeFromTS(barrierTs).
Add(util.GetOrZero(cfInfo.Config.SyncPointInterval)),
)
if err := c.ddlSink.emitSyncPoint(ctx, barrierTs); err != nil {
return errors.Trace(err)
}
c.barriers.Update(syncPointBarrier, nextSyncPointTs)
case finishBarrier:
c.feedStateManager.MarkFinished()
default:
log.Error("Unknown barrier type", zap.Int("barrierType", int(barrierTp)))
return cerror.ErrUnexpected.FastGenByArgs("Unknown barrier type")
}
}
// If there are other barriers less than ddl barrier,
// we should wait for them.
// Note: There may be some tableBarrierTs larger than otherBarrierTs,
// but we can ignore them because they will be handled in the processor.
if barrier.GlobalBarrierTs > barrierTs {
log.Debug("There are other barriers less than ddl barrier, wait for them",
zap.Uint64("otherBarrierTs", barrierTs),
zap.Uint64("globalBarrierTs", barrier.GlobalBarrierTs))
barrier.GlobalBarrierTs = barrierTs
}
if barrier.MinTableBarrierTs > barrierTs {
log.Debug("There are other barriers less than min table barrier, wait for them",
zap.Uint64("otherBarrierTs", barrierTs),
zap.Uint64("minTableBarrierTs", barrier.GlobalBarrierTs))
barrier.MinTableBarrierTs = barrierTs
}
return nil
}
func (c *changefeed) updateMetrics(currentTs int64, checkpointTs, resolvedTs model.Ts) {
phyCkpTs := oracle.ExtractPhysical(checkpointTs)
c.metricsChangefeedCheckpointTsGauge.Set(float64(phyCkpTs))
checkpointLag := float64(currentTs-phyCkpTs) / 1e3
c.metricsChangefeedCheckpointTsLagGauge.Set(checkpointLag)
c.metricsChangefeedCheckpointLagDuration.Observe(checkpointLag)
phyRTs := oracle.ExtractPhysical(resolvedTs)
c.metricsChangefeedResolvedTsGauge.Set(float64(phyRTs))
resolvedLag := float64(currentTs-phyRTs) / 1e3
c.metricsChangefeedResolvedTsLagGauge.Set(resolvedLag)
c.metricsChangefeedResolvedTsLagDuration.Observe(resolvedLag)
c.metricsCurrentPDTsGauge.Set(float64(currentTs))
}
func (c *changefeed) Close(ctx context.Context) {
startTime := time.Now()
c.releaseResources(ctx)
costTime := time.Since(startTime)
if costTime > changefeedLogsWarnDuration {
log.Warn("changefeed close took too long",