forked from pingcap/tiflow
-
Notifications
You must be signed in to change notification settings - Fork 0
/
capture.go
614 lines (555 loc) · 18.9 KB
/
capture.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
// 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 capture
import (
"context"
"fmt"
"io"
"sync"
"time"
"github.com/google/uuid"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/log"
"github.com/pingcap/tiflow/cdc/model"
"github.com/pingcap/tiflow/cdc/owner"
"github.com/pingcap/tiflow/cdc/processor"
"github.com/pingcap/tiflow/cdc/processor/pipeline/system"
ssystem "github.com/pingcap/tiflow/cdc/sorter/leveldb/system"
"github.com/pingcap/tiflow/pkg/config"
cdcContext "github.com/pingcap/tiflow/pkg/context"
cerror "github.com/pingcap/tiflow/pkg/errors"
"github.com/pingcap/tiflow/pkg/etcd"
"github.com/pingcap/tiflow/pkg/orchestrator"
"github.com/pingcap/tiflow/pkg/p2p"
"github.com/pingcap/tiflow/pkg/upstream"
"github.com/pingcap/tiflow/pkg/util"
"github.com/pingcap/tiflow/pkg/version"
"go.etcd.io/etcd/client/v3/concurrency"
"go.etcd.io/etcd/server/v3/mvcc"
"go.uber.org/zap"
"golang.org/x/time/rate"
)
const cleanMetaDuration = 10 * time.Second
// Capture represents a Capture server, it monitors the changefeed information in etcd and schedules Task on it.
type Capture struct {
// captureMu is used to protect the capture info and processorManager.
captureMu sync.Mutex
info *model.CaptureInfo
processorManager *processor.Manager
pdEndpoints []string
UpstreamManager *upstream.Manager
ownerMu sync.Mutex
owner owner.Owner
// session keeps alive between the capture and etcd
session *concurrency.Session
election *concurrency.Election
EtcdClient *etcd.CDCEtcdClient
sorterSystem *ssystem.System
tableActorSystem *system.System
// MessageServer is the receiver of the messages from the other nodes.
// It should be recreated each time the capture is restarted.
MessageServer *p2p.MessageServer
// MessageRouter manages the clients to send messages to all peers.
MessageRouter p2p.MessageRouter
// grpcService is a wrapper that can hold a MessageServer.
// The instance should last for the whole life of the server,
// regardless of server restarting.
// This design is to solve the problem that grpc-go cannot gracefully
// unregister a service.
grpcService *p2p.ServerWrapper
cancel context.CancelFunc
newProcessorManager func(upstreamManager *upstream.Manager) *processor.Manager
newOwner func(upstreamManager *upstream.Manager) owner.Owner
}
// NewCapture returns a new Capture instance
func NewCapture(pdEnpoints []string, etcdClient *etcd.CDCEtcdClient, grpcService *p2p.ServerWrapper) *Capture {
return &Capture{
EtcdClient: etcdClient,
grpcService: grpcService,
cancel: func() {},
pdEndpoints: pdEnpoints,
newProcessorManager: processor.NewManager,
newOwner: owner.NewOwner,
}
}
// NewCapture4Test returns a new Capture instance for test.
func NewCapture4Test(o owner.Owner) *Capture {
res := &Capture{
info: &model.CaptureInfo{ID: "capture-for-test", AdvertiseAddr: "127.0.0.1", Version: "test"},
}
res.owner = o
return res
}
func (c *Capture) reset(ctx context.Context) error {
conf := config.GetGlobalServerConfig()
sess, err := concurrency.NewSession(c.EtcdClient.Client.Unwrap(),
concurrency.WithTTL(conf.CaptureSessionTTL))
if err != nil {
return errors.Trace(err)
}
c.captureMu.Lock()
defer c.captureMu.Unlock()
c.info = &model.CaptureInfo{
ID: uuid.New().String(),
AdvertiseAddr: conf.AdvertiseAddr,
Version: version.ReleaseVersion,
}
if c.UpstreamManager != nil {
c.UpstreamManager.Close()
}
c.UpstreamManager = upstream.NewManager(ctx)
err = c.UpstreamManager.Add(upstream.DefaultUpstreamID, c.pdEndpoints, conf.Security)
if err != nil {
return errors.Trace(err)
}
c.processorManager = c.newProcessorManager(c.UpstreamManager)
if c.session != nil {
// It can't be handled even after it fails, so we ignore it.
_ = c.session.Close()
}
c.session = sess
c.election = concurrency.NewElection(sess, etcd.CaptureOwnerKey)
if c.tableActorSystem != nil {
c.tableActorSystem.Stop()
}
if conf.Debug.EnableTableActor {
c.tableActorSystem = system.NewSystem()
err = c.tableActorSystem.Start(ctx)
if err != nil {
return errors.Annotate(
cerror.WrapError(cerror.ErrNewCaptureFailed, err),
"create table actor system")
}
}
if conf.Debug.EnableDBSorter {
if c.sorterSystem != nil {
err := c.sorterSystem.Stop()
if err != nil {
log.Warn("stop sorter system failed", zap.Error(err))
}
}
// Sorter dir has been set and checked when server starts.
// See https://github.com/pingcap/tiflow/blob/9dad09/cdc/server.go#L275
sortDir := config.GetGlobalServerConfig().Sorter.SortDir
memPercentage := float64(config.GetGlobalServerConfig().Sorter.MaxMemoryPercentage) / 100
c.sorterSystem = ssystem.NewSystem(sortDir, memPercentage, conf.Debug.DB)
err = c.sorterSystem.Start(ctx)
if err != nil {
return errors.Annotate(
cerror.WrapError(cerror.ErrNewCaptureFailed, err),
"create sorter system")
}
}
c.grpcService.Reset(nil)
if c.MessageRouter != nil {
c.MessageRouter.Close()
c.MessageRouter.Wait()
c.MessageRouter = nil
}
messageServerConfig := conf.Debug.Messages.ToMessageServerConfig()
c.MessageServer = p2p.NewMessageServer(c.info.ID, messageServerConfig)
c.grpcService.Reset(c.MessageServer)
messageClientConfig := conf.Debug.Messages.ToMessageClientConfig()
// Puts the advertise-addr of the local node to the client config.
// This is for metrics purpose only, so that the receiver knows which
// node the connections are from.
advertiseAddr := conf.AdvertiseAddr
messageClientConfig.AdvertisedAddr = advertiseAddr
c.MessageRouter = p2p.NewMessageRouter(c.info.ID, conf.Security, messageClientConfig)
log.Info("init capture",
zap.String("captureID", c.info.ID),
zap.String("captureAddr", c.info.AdvertiseAddr))
return nil
}
// Run runs the capture
func (c *Capture) Run(ctx context.Context) error {
defer log.Info("the capture routine has exited")
// Limit the frequency of reset capture to avoid frequent recreating of resources
rl := rate.NewLimiter(0.05, 2)
for {
select {
case <-ctx.Done():
return nil
default:
}
ctx, cancel := context.WithCancel(ctx)
c.cancel = cancel
err := rl.Wait(ctx)
if err != nil {
if errors.Cause(err) == context.Canceled {
return nil
}
return errors.Trace(err)
}
err = c.run(ctx)
// if capture suicided, reset the capture and run again.
// if the canceled error throw, there are two possible scenarios:
// 1. the internal context canceled, it means some error happened in
// the internal, and the routine is exited, we should restart
// the capture.
// 2. the parent context canceled, it means that the caller of
// the capture hope the capture to exit, and this loop will return
// in the above `select` block.
// if there are some **internal** context deadline exceeded (IO/network
// timeout), reset the capture and run again.
//
// TODO: make sure the internal cancel should return the real error
// instead of context.Canceled.
if cerror.ErrCaptureSuicide.Equal(err) ||
context.Canceled == errors.Cause(err) ||
context.DeadlineExceeded == errors.Cause(err) {
log.Info("capture recovered", zap.String("captureID", c.info.ID))
continue
}
return errors.Trace(err)
}
}
func (c *Capture) run(stdCtx context.Context) error {
err := c.reset(stdCtx)
if err != nil {
log.Error("reset capture failed", zap.Error(err))
return errors.Trace(err)
}
ctx := cdcContext.NewContext(stdCtx, &cdcContext.GlobalVars{
CaptureInfo: c.info,
EtcdClient: c.EtcdClient,
TableActorSystem: c.tableActorSystem,
SorterSystem: c.sorterSystem,
MessageServer: c.MessageServer,
MessageRouter: c.MessageRouter,
})
err = c.register(ctx)
if err != nil {
return errors.Trace(err)
}
defer func() {
timeoutCtx, cancel := context.WithTimeout(context.Background(), cleanMetaDuration)
if err := ctx.GlobalVars().EtcdClient.DeleteCaptureInfo(timeoutCtx, c.info.ID); err != nil {
log.Warn("failed to delete capture info when capture exited", zap.Error(err))
}
cancel()
}()
wg := new(sync.WaitGroup)
var ownerErr, processorErr, messageServerErr error
wg.Add(1)
go func() {
defer wg.Done()
defer c.AsyncClose()
// when the campaignOwner returns an error, it means that the owner throws an unrecoverable serious errors
// (recoverable errors are intercepted in the owner tick)
// so we should also stop the owner and let capture restart or exit
ownerErr = c.campaignOwner(ctx)
log.Info("the owner routine has exited", zap.Error(ownerErr))
}()
wg.Add(1)
go func() {
defer wg.Done()
defer c.AsyncClose()
conf := config.GetGlobalServerConfig()
processorFlushInterval := time.Duration(conf.ProcessorFlushInterval)
globalState := orchestrator.NewGlobalState()
globalState.SetOnCaptureAdded(func(captureID model.CaptureID, addr string) {
c.MessageRouter.AddPeer(captureID, addr)
})
globalState.SetOnCaptureRemoved(func(captureID model.CaptureID) {
c.MessageRouter.RemovePeer(captureID)
})
// when the etcd worker of processor returns an error, it means that the processor throws an unrecoverable serious errors
// (recoverable errors are intercepted in the processor tick)
// so we should also stop the processor and let capture restart or exit
processorErr = c.runEtcdWorker(ctx, c.processorManager, globalState, processorFlushInterval, util.RoleProcessor.String())
log.Info("the processor routine has exited", zap.Error(processorErr))
}()
wg.Add(1)
go func() {
defer wg.Done()
defer c.AsyncClose()
defer c.grpcService.Reset(nil)
messageServerErr = c.MessageServer.Run(ctx)
}()
wg.Wait()
if ownerErr != nil {
return errors.Annotate(ownerErr, "owner exited with error")
}
if processorErr != nil {
return errors.Annotate(processorErr, "processor exited with error")
}
if messageServerErr != nil {
return errors.Annotate(messageServerErr, "message server exited with error")
}
return nil
}
// Info gets the capture info
func (c *Capture) Info() (model.CaptureInfo, error) {
c.captureMu.Lock()
defer c.captureMu.Unlock()
// when c.reset has not been called yet, c.info is nil.
if c.info != nil {
return *c.info, nil
}
return model.CaptureInfo{}, cerror.ErrCaptureNotInitialized.GenWithStackByArgs()
}
func (c *Capture) campaignOwner(ctx cdcContext.Context) error {
// In most failure cases, we don't return error directly, just run another
// campaign loop. We treat campaign loop as a special background routine.
conf := config.GetGlobalServerConfig()
ownerFlushInterval := time.Duration(conf.OwnerFlushInterval)
failpoint.Inject("ownerFlushIntervalInject", func(val failpoint.Value) {
ownerFlushInterval = time.Millisecond * time.Duration(val.(int))
})
// Limit the frequency of elections to avoid putting too much pressure on the etcd server
rl := rate.NewLimiter(0.05, 2)
for {
select {
case <-ctx.Done():
return nil
default:
}
err := rl.Wait(ctx)
if err != nil {
if errors.Cause(err) == context.Canceled {
return nil
}
return errors.Trace(err)
}
// Campaign to be an owner, it blocks until it becomes the owner
if err := c.campaign(ctx); err != nil {
switch errors.Cause(err) {
case context.Canceled:
return nil
case mvcc.ErrCompacted:
// the revision we requested is compacted, just retry
continue
}
log.Warn("campaign owner failed", zap.Error(err))
// if campaign owner failed, restart capture
return cerror.ErrCaptureSuicide.GenWithStackByArgs()
}
ownerRev, err := c.EtcdClient.GetOwnerRevision(ctx, c.info.ID)
if err != nil {
if errors.Cause(err) == context.Canceled {
return nil
}
return errors.Trace(err)
}
// We do a copy of the globalVars here to avoid
// accidental modifications and potential race conditions.
globalVars := *ctx.GlobalVars()
newGlobalVars := &globalVars
newGlobalVars.OwnerRevision = ownerRev
ownerCtx := cdcContext.NewContext(ctx, newGlobalVars)
log.Info("campaign owner successfully",
zap.String("captureID", c.info.ID),
zap.Int64("ownerRev", ownerRev))
owner := c.newOwner(c.UpstreamManager)
c.setOwner(owner)
globalState := orchestrator.NewGlobalState()
globalState.SetOnCaptureAdded(func(captureID model.CaptureID, addr string) {
c.MessageRouter.AddPeer(captureID, addr)
})
globalState.SetOnCaptureRemoved(func(captureID model.CaptureID) {
c.MessageRouter.RemovePeer(captureID)
})
err = c.runEtcdWorker(ownerCtx, owner, orchestrator.NewGlobalState(), ownerFlushInterval, util.RoleOwner.String())
c.setOwner(nil)
log.Info("run owner exited", zap.Error(err))
// if owner exits, resign the owner key
if resignErr := c.resign(ctx); resignErr != nil {
// if resigning owner failed, return error to let capture exits
return errors.Annotatef(resignErr, "resign owner failed, capture: %s", c.info.ID)
}
if err != nil {
// for errors, return error and let capture exits or restart
return errors.Trace(err)
}
// if owner exits normally, continue the campaign loop and try to election owner again
}
}
func (c *Capture) runEtcdWorker(
ctx cdcContext.Context,
reactor orchestrator.Reactor,
reactorState orchestrator.ReactorState,
timerInterval time.Duration,
role string,
) error {
etcdWorker, err := orchestrator.NewEtcdWorker(ctx.GlobalVars().EtcdClient.Client, etcd.EtcdKeyBase, reactor, reactorState)
if err != nil {
return errors.Trace(err)
}
if err := etcdWorker.Run(ctx, c.session, timerInterval, role); err != nil {
// We check ttl of lease instead of check `session.Done`, because
// `session.Done` is only notified when etcd client establish a
// new keepalive request, there could be a time window as long as
// 1/3 of session ttl that `session.Done` can't be triggered even
// the lease is already revoked.
switch {
case cerror.ErrEtcdSessionDone.Equal(err),
cerror.ErrLeaseExpired.Equal(err):
log.Warn("session is disconnected", zap.Error(err))
return cerror.ErrCaptureSuicide.GenWithStackByArgs()
}
lease, inErr := ctx.GlobalVars().EtcdClient.Client.TimeToLive(ctx, c.session.Lease())
if inErr != nil {
return cerror.WrapError(cerror.ErrPDEtcdAPIError, inErr)
}
if lease.TTL == int64(-1) {
log.Warn("session is disconnected", zap.Error(err))
return cerror.ErrCaptureSuicide.GenWithStackByArgs()
}
return errors.Trace(err)
}
return nil
}
func (c *Capture) setOwner(owner owner.Owner) {
c.ownerMu.Lock()
defer c.ownerMu.Unlock()
c.owner = owner
}
// GetOwner returns owner if it is the owner.
func (c *Capture) GetOwner() (owner.Owner, error) {
c.ownerMu.Lock()
defer c.ownerMu.Unlock()
if c.owner == nil {
return nil, cerror.ErrNotOwner.GenWithStackByArgs()
}
return c.owner, nil
}
// campaign to be an owner.
func (c *Capture) campaign(ctx cdcContext.Context) error {
failpoint.Inject("capture-campaign-compacted-error", func() {
failpoint.Return(errors.Trace(mvcc.ErrCompacted))
})
return cerror.WrapError(cerror.ErrCaptureCampaignOwner, c.election.Campaign(ctx, c.info.ID))
}
// resign lets an owner start a new election.
func (c *Capture) resign(ctx cdcContext.Context) error {
failpoint.Inject("capture-resign-failed", func() {
failpoint.Return(errors.New("capture resign failed"))
})
return cerror.WrapError(cerror.ErrCaptureResignOwner, c.election.Resign(ctx))
}
// register registers the capture information in etcd
func (c *Capture) register(ctx cdcContext.Context) error {
err := ctx.GlobalVars().EtcdClient.PutCaptureInfo(ctx, c.info, c.session.Lease())
if err != nil {
return cerror.WrapError(cerror.ErrCaptureRegister, err)
}
return nil
}
// AsyncClose closes the capture by deregister it from etcd
// Note: this function should be reentrant
func (c *Capture) AsyncClose() {
defer c.cancel()
// Safety: Here we mainly want to stop the owner
// and ignore it if the owner does not exist or is not set.
o, _ := c.GetOwner()
if o != nil {
o.AsyncStop()
log.Info("owner closed")
}
c.captureMu.Lock()
defer c.captureMu.Unlock()
if c.processorManager != nil {
c.processorManager.AsyncClose()
}
log.Info("processor manager closed")
if c.tableActorSystem != nil {
c.tableActorSystem.Stop()
c.tableActorSystem = nil
}
log.Info("table actor system closed")
if c.sorterSystem != nil {
err := c.sorterSystem.Stop()
if err != nil {
log.Warn("stop sorter system failed", zap.Error(err))
}
c.sorterSystem = nil
}
log.Info("sorter actor system closed")
c.grpcService.Reset(nil)
if c.MessageRouter != nil {
c.MessageRouter.Close()
c.MessageRouter.Wait()
c.MessageRouter = nil
}
log.Info("message router closed")
}
// WriteDebugInfo writes the debug info into writer.
func (c *Capture) WriteDebugInfo(ctx context.Context, w io.Writer) {
wait := func(done <-chan error) {
var err error
select {
case <-ctx.Done():
err = ctx.Err()
case err = <-done:
}
if err != nil {
log.Warn("write debug info failed", zap.Error(err))
}
}
// Safety: Because we are mainly outputting information about the owner here,
// if the owner does not exist or is not set, the information will not be output.
o, _ := c.GetOwner()
if o != nil {
doneOwner := make(chan error, 1)
fmt.Fprintf(w, "\n\n*** owner info ***:\n\n")
o.WriteDebugInfo(w, doneOwner)
// wait the debug info printed
wait(doneOwner)
}
doneM := make(chan error, 1)
c.captureMu.Lock()
if c.processorManager != nil {
fmt.Fprintf(w, "\n\n*** processors info ***:\n\n")
c.processorManager.WriteDebugInfo(ctx, w, doneM)
}
// NOTICE: we must release the lock before wait the debug info process down.
// Otherwise, the capture initialization and request response will compete
// for captureMu resulting in a deadlock.
c.captureMu.Unlock()
// wait the debug info printed
wait(doneM)
}
// IsOwner returns whether the capture is an owner
func (c *Capture) IsOwner() bool {
c.ownerMu.Lock()
defer c.ownerMu.Unlock()
return c.owner != nil
}
// GetOwnerCaptureInfo return the owner capture info of current TiCDC cluster
func (c *Capture) GetOwnerCaptureInfo(ctx context.Context) (*model.CaptureInfo, error) {
_, captureInfos, err := c.EtcdClient.GetCaptures(ctx)
if err != nil {
return nil, err
}
ownerID, err := c.EtcdClient.GetOwnerID(ctx, etcd.CaptureOwnerKey)
if err != nil {
return nil, err
}
for _, captureInfo := range captureInfos {
if captureInfo.ID == ownerID {
return captureInfo, nil
}
}
return nil, cerror.ErrOwnerNotFound.FastGenByArgs()
}
// StatusProvider returns owner's StatusProvider.
func (c *Capture) StatusProvider() owner.StatusProvider {
c.ownerMu.Lock()
defer c.ownerMu.Unlock()
if c.owner == nil {
return nil
}
return owner.NewStatusProvider(c.owner)
}