-
Notifications
You must be signed in to change notification settings - Fork 287
/
region_worker.go
867 lines (808 loc) · 27.7 KB
/
region_worker.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
// 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 kv
import (
"context"
"encoding/hex"
"reflect"
"runtime"
"sync"
"time"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/kvproto/pkg/cdcpb"
"github.com/pingcap/log"
"github.com/pingcap/tiflow/cdc/contextutil"
"github.com/pingcap/tiflow/cdc/model"
"github.com/pingcap/tiflow/pkg/config"
cerror "github.com/pingcap/tiflow/pkg/errors"
"github.com/pingcap/tiflow/pkg/regionspan"
"github.com/pingcap/tiflow/pkg/workerpool"
"github.com/prometheus/client_golang/prometheus"
"github.com/tikv/client-go/v2/oracle"
"go.uber.org/zap"
"golang.org/x/sync/errgroup"
)
var (
regionWorkerPool workerpool.WorkerPool
workerPoolOnce sync.Once
// The magic number here is keep the same with some magic numbers in some
// other components in TiCDC, including worker pool task chan size, mounter
// chan size etc.
// TODO: unified channel buffer mechanism
regionWorkerInputChanSize = 128
regionWorkerLowWatermark = int(float64(regionWorkerInputChanSize) * 0.2)
regionWorkerHighWatermark = int(float64(regionWorkerInputChanSize) * 0.7)
)
const (
minRegionStateBucket = 4
maxRegionStateBucket = 16
maxWorkerPoolSize = 64
maxResolvedLockPerLoop = 64
)
// regionStateManager provides the get/put way like a sync.Map, and it is divided
// into several buckets to reduce lock contention
type regionStateManager struct {
bucket int
states []*sync.Map
}
func newRegionStateManager(bucket int) *regionStateManager {
if bucket <= 0 {
bucket = runtime.NumCPU()
if bucket > maxRegionStateBucket {
bucket = maxRegionStateBucket
}
if bucket < minRegionStateBucket {
bucket = minRegionStateBucket
}
}
rsm := ®ionStateManager{
bucket: bucket,
states: make([]*sync.Map, bucket),
}
for i := range rsm.states {
rsm.states[i] = new(sync.Map)
}
return rsm
}
func (rsm *regionStateManager) getBucket(regionID uint64) int {
return int(regionID) % rsm.bucket
}
func (rsm *regionStateManager) getState(regionID uint64) (*regionFeedState, bool) {
bucket := rsm.getBucket(regionID)
if val, ok := rsm.states[bucket].Load(regionID); ok {
return val.(*regionFeedState), true
}
return nil, false
}
func (rsm *regionStateManager) setState(regionID uint64, state *regionFeedState) {
bucket := rsm.getBucket(regionID)
rsm.states[bucket].Store(regionID, state)
}
func (rsm *regionStateManager) delState(regionID uint64) {
bucket := rsm.getBucket(regionID)
rsm.states[bucket].Delete(regionID)
}
type regionWorkerMetrics struct {
// kv events related metrics
metricReceivedEventSize prometheus.Observer
metricDroppedEventSize prometheus.Observer
metricPullEventInitializedCounter prometheus.Counter
metricPullEventPrewriteCounter prometheus.Counter
metricPullEventCommitCounter prometheus.Counter
metricPullEventCommittedCounter prometheus.Counter
metricPullEventRollbackCounter prometheus.Counter
metricSendEventResolvedCounter prometheus.Counter
metricSendEventCommitCounter prometheus.Counter
metricSendEventCommittedCounter prometheus.Counter
// TODO: add region runtime related metrics
}
/*
`regionWorker` maintains N regions, it runs in background for each gRPC stream,
corresponding to one TiKV store. It receives `regionStatefulEvent` in a channel
from gRPC stream receiving goroutine, processes event as soon as possible and
sends `RegionFeedEvent` to output channel.
Besides the `regionWorker` maintains a background lock resolver, the lock resolver
maintains a resolved-ts based min heap to manager region resolved ts, so it doesn't
need to iterate each region every time when resolving lock.
Note: There exist two locks, one is lock for region states map, the other one is
lock for each region state(each region state has one lock).
`regionWorker` is single routine now, it will be extended to multiple goroutines
for event processing to increase throughput.
*/
type regionWorker struct {
parentCtx context.Context
session *eventFeedSession
inputCh chan *regionStatefulEvent
outputCh chan<- model.RegionFeedEvent
errorCh chan error
// event handlers in region worker
handles []workerpool.EventHandle
// how many workers in worker pool will be used for this region worker
concurrent int
statesManager *regionStateManager
rtsManager *regionTsManager
rtsUpdateCh chan *regionTsInfo
metrics *regionWorkerMetrics
storeAddr string
}
func newRegionWorker(s *eventFeedSession, addr string) *regionWorker {
worker := ®ionWorker{
session: s,
inputCh: make(chan *regionStatefulEvent, regionWorkerInputChanSize),
outputCh: s.eventCh,
errorCh: make(chan error, 1),
statesManager: newRegionStateManager(-1),
rtsManager: newRegionTsManager(),
rtsUpdateCh: make(chan *regionTsInfo, 1024),
storeAddr: addr,
concurrent: s.client.config.WorkerConcurrent,
}
return worker
}
func (w *regionWorker) initMetrics(ctx context.Context) {
changefeedID := contextutil.ChangefeedIDFromCtx(ctx)
metrics := ®ionWorkerMetrics{}
metrics.metricReceivedEventSize = eventSize.WithLabelValues("received")
metrics.metricDroppedEventSize = eventSize.WithLabelValues("dropped")
metrics.metricPullEventInitializedCounter = pullEventCounter.
WithLabelValues(cdcpb.Event_INITIALIZED.String(), changefeedID.Namespace, changefeedID.ID)
metrics.metricPullEventCommittedCounter = pullEventCounter.
WithLabelValues(cdcpb.Event_COMMITTED.String(), changefeedID.Namespace, changefeedID.ID)
metrics.metricPullEventCommitCounter = pullEventCounter.
WithLabelValues(cdcpb.Event_COMMIT.String(), changefeedID.Namespace, changefeedID.ID)
metrics.metricPullEventPrewriteCounter = pullEventCounter.
WithLabelValues(cdcpb.Event_PREWRITE.String(), changefeedID.Namespace, changefeedID.ID)
metrics.metricPullEventRollbackCounter = pullEventCounter.
WithLabelValues(cdcpb.Event_ROLLBACK.String(), changefeedID.Namespace, changefeedID.ID)
metrics.metricSendEventResolvedCounter = sendEventCounter.
WithLabelValues("native-resolved", changefeedID.Namespace, changefeedID.ID)
metrics.metricSendEventCommitCounter = sendEventCounter.
WithLabelValues("commit", changefeedID.Namespace, changefeedID.ID)
metrics.metricSendEventCommittedCounter = sendEventCounter.
WithLabelValues("committed", changefeedID.Namespace, changefeedID.ID)
w.metrics = metrics
}
func (w *regionWorker) getRegionState(regionID uint64) (*regionFeedState, bool) {
return w.statesManager.getState(regionID)
}
func (w *regionWorker) setRegionState(regionID uint64, state *regionFeedState) {
w.statesManager.setState(regionID, state)
}
func (w *regionWorker) delRegionState(regionID uint64) {
w.statesManager.delState(regionID)
}
// checkRegionStateEmpty returns true if there is no region state maintained.
// Note this function is not thread-safe
func (w *regionWorker) checkRegionStateEmpty() (empty bool) {
empty = true
for _, states := range w.statesManager.states {
states.Range(func(_, _ interface{}) bool {
empty = false
return false
})
if !empty {
return
}
}
return
}
// checkShouldExit checks whether the region worker should exit, if should exit
// return an error
func (w *regionWorker) checkShouldExit() error {
empty := w.checkRegionStateEmpty()
// If there is not region maintained by this region worker, exit it and
// cancel the gRPC stream.
if empty {
w.cancelStream(time.Duration(0))
return cerror.ErrRegionWorkerExit.GenWithStackByArgs()
}
return nil
}
func (w *regionWorker) handleSingleRegionError(err error, state *regionFeedState) error {
if state.lastResolvedTs > state.sri.ts {
state.sri.ts = state.lastResolvedTs
}
regionID := state.sri.verID.GetID()
log.Info("single region event feed disconnected",
zap.String("namesapce", w.session.client.changefeed.Namespace),
zap.String("changefeed", w.session.client.changefeed.ID),
zap.Uint64("regionID", regionID),
zap.Uint64("requestID", state.requestID),
zap.Stringer("span", state.sri.span),
zap.Uint64("checkpoint", state.sri.ts),
zap.String("error", err.Error()),
zap.Any("sri", state.sri))
// if state is already marked stopped, it must have been or would be processed by `onRegionFail`
if state.isStopped() {
return w.checkShouldExit()
}
// We need to ensure when the error is handled, `isStopped` must be set. So set it before sending the error.
state.markStopped()
w.delRegionState(regionID)
failpoint.Inject("kvClientSingleFeedProcessDelay", nil)
failpoint.Inject("kvClientErrUnreachable", func() {
if err == errUnreachable {
failpoint.Return(err)
}
})
// check and cancel gRPC stream before reconnecting region, in case of the
// scenario that region connects to the same TiKV store again and reuses
// resource in this region worker by accident.
retErr := w.checkShouldExit()
// `ErrPrewriteNotMatch` would cause duplicated request to the same region,
// so cancel the original gRPC stream before restarts a new stream.
if cerror.ErrPrewriteNotMatch.Equal(err) {
w.cancelStream(time.Second)
}
revokeToken := !state.initialized
// since the context used in region worker will be cancelled after region
// worker exits, we must use the parent context to prevent regionErrorInfo loss.
errInfo := newRegionErrorInfo(state.sri, err)
w.session.onRegionFail(w.parentCtx, errInfo, revokeToken)
return retErr
}
func (w *regionWorker) resolveLock(ctx context.Context) error {
// tikv resolved update interval is 1s, use half of the resolve lock interval
// as lock penalty.
resolveLockPenalty := 10
resolveLockInterval := 20 * time.Second
failpoint.Inject("kvClientResolveLockInterval", func(val failpoint.Value) {
resolveLockInterval = time.Duration(val.(int)) * time.Second
})
advanceCheckTicker := time.NewTicker(time.Second * 5)
defer advanceCheckTicker.Stop()
for {
select {
case <-ctx.Done():
return errors.Trace(ctx.Err())
case rtsUpdate := <-w.rtsUpdateCh:
w.rtsManager.Upsert(rtsUpdate)
case <-advanceCheckTicker.C:
currentTimeFromPD, err := w.session.client.pdClock.CurrentTime()
if err != nil {
log.Warn("failed to get current version from PD",
zap.Error(err),
zap.String("namesapce", w.session.client.changefeed.Namespace),
zap.String("changefeed", w.session.client.changefeed.ID))
continue
}
expired := make([]*regionTsInfo, 0)
for w.rtsManager.Len() > 0 {
item := w.rtsManager.Pop()
sinceLastResolvedTs := currentTimeFromPD.Sub(oracle.GetTimeFromTS(item.ts.resolvedTs))
// region does not reach resolve lock boundary, put it back
if sinceLastResolvedTs < resolveLockInterval {
w.rtsManager.Upsert(item)
break
}
expired = append(expired, item)
if len(expired) >= maxResolvedLockPerLoop {
break
}
}
if len(expired) == 0 {
continue
}
maxVersion := oracle.ComposeTS(oracle.GetPhysical(currentTimeFromPD.Add(-10*time.Second)), 0)
for _, rts := range expired {
state, ok := w.getRegionState(rts.regionID)
if !ok || state.isStopped() {
// state is already deleted or stopped, just continue,
// and don't need to push resolved ts back to heap.
continue
}
// recheck resolved ts from region state, which may be larger than that in resolved ts heap
lastResolvedTs := state.getLastResolvedTs()
sinceLastResolvedTs := currentTimeFromPD.Sub(oracle.GetTimeFromTS(lastResolvedTs))
if sinceLastResolvedTs >= resolveLockInterval {
sinceLastEvent := time.Since(rts.ts.eventTime)
if sinceLastResolvedTs > reconnectInterval && sinceLastEvent > reconnectInterval {
log.Warn("kv client reconnect triggered",
zap.String("namesapce", w.session.client.changefeed.Namespace),
zap.String("changefeed", w.session.client.changefeed.ID),
zap.Duration("duration", sinceLastResolvedTs), zap.Duration("sinceLastEvent", sinceLastResolvedTs))
return errReconnect
}
// Only resolve lock if the resolved-ts keeps unchanged for
// more than resolveLockPenalty times.
if rts.ts.penalty < resolveLockPenalty {
if lastResolvedTs > rts.ts.resolvedTs {
rts.ts.resolvedTs = lastResolvedTs
rts.ts.eventTime = time.Now()
rts.ts.penalty = 0
}
w.rtsManager.Upsert(rts)
continue
}
log.Warn("region not receiving resolved event from tikv or resolved ts is not pushing for too long time, try to resolve lock",
zap.String("namesapce", w.session.client.changefeed.Namespace),
zap.String("changefeed", w.session.client.changefeed.ID),
zap.String("addr", w.storeAddr),
zap.Uint64("regionID", rts.regionID),
zap.Stringer("span", state.getRegionSpan()),
zap.Duration("duration", sinceLastResolvedTs),
zap.Duration("lastEvent", sinceLastEvent),
zap.Uint64("resolvedTs", lastResolvedTs),
)
err = w.session.lockResolver.Resolve(ctx, rts.regionID, maxVersion)
if err != nil {
log.Warn("failed to resolve lock",
zap.Uint64("regionID", rts.regionID), zap.Error(err),
zap.String("namesapce", w.session.client.changefeed.Namespace),
zap.String("changefeed", w.session.client.changefeed.ID))
continue
}
rts.ts.penalty = 0
}
rts.ts.resolvedTs = lastResolvedTs
w.rtsManager.Upsert(rts)
}
}
}
}
func (w *regionWorker) processEvent(ctx context.Context, event *regionStatefulEvent) error {
if event.finishedCallbackCh != nil {
event.finishedCallbackCh <- struct{}{}
return nil
}
var err error
event.state.lock.Lock()
if event.changeEvent != nil {
w.metrics.metricReceivedEventSize.Observe(float64(event.changeEvent.Event.Size()))
switch x := event.changeEvent.Event.(type) {
case *cdcpb.Event_Entries_:
err = w.handleEventEntry(ctx, x, event.state)
if err != nil {
err = w.handleSingleRegionError(err, event.state)
}
case *cdcpb.Event_Admin_:
log.Info("receive admin event",
zap.Stringer("event", event.changeEvent),
zap.String("namesapce", w.session.client.changefeed.Namespace),
zap.String("changefeed", w.session.client.changefeed.ID))
case *cdcpb.Event_Error:
err = w.handleSingleRegionError(
cerror.WrapError(cerror.ErrEventFeedEventError, &eventError{err: x.Error}),
event.state,
)
case *cdcpb.Event_ResolvedTs:
if err = w.handleResolvedTs(ctx, x.ResolvedTs, event.state); err != nil {
err = w.handleSingleRegionError(err, event.state)
}
}
}
if event.resolvedTs != nil {
if err = w.handleResolvedTs(ctx, event.resolvedTs.Ts, event.state); err != nil {
err = w.handleSingleRegionError(err, event.state)
}
}
event.state.lock.Unlock()
return err
}
func (w *regionWorker) initPoolHandles(handleCount int) {
handles := make([]workerpool.EventHandle, 0, handleCount)
for i := 0; i < handleCount; i++ {
poolHandle := regionWorkerPool.RegisterEvent(func(ctx context.Context, eventI interface{}) error {
event := eventI.(*regionStatefulEvent)
return w.processEvent(ctx, event)
}).OnExit(func(err error) {
w.onHandleExit(err)
})
handles = append(handles, poolHandle)
}
w.handles = handles
}
func (w *regionWorker) onHandleExit(err error) {
select {
case w.errorCh <- err:
default:
}
}
func (w *regionWorker) eventHandler(ctx context.Context) error {
preprocess := func(event *regionStatefulEvent, ok bool) (
exitEventHandler bool,
skipEvent bool,
) {
// event == nil means the region worker should exit and re-establish
// all existing regions.
if !ok || event == nil {
log.Info("region worker closed by error",
zap.String("namesapce", w.session.client.changefeed.Namespace),
zap.String("changefeed", w.session.client.changefeed.ID))
exitEventHandler = true
return
}
if event.state.isStopped() {
skipEvent = true
}
return
}
pollEvent := func() (event *regionStatefulEvent, ok bool, err error) {
select {
case <-ctx.Done():
err = errors.Trace(ctx.Err())
case err = <-w.errorCh:
case event, ok = <-w.inputCh:
}
return
}
for {
event, ok, err := pollEvent()
if err != nil {
return err
}
exitEventHandler, skipEvent := preprocess(event, ok)
if exitEventHandler {
return cerror.ErrRegionWorkerExit.GenWithStackByArgs()
}
if skipEvent {
continue
}
// We measure whether the current worker is busy based on the input
// channel size. If the buffered event count is larger than the high
// watermark, we send events to worker pool to increase processing
// throughput. Otherwise we process event in local region worker to
// ensure low processing latency.
if len(w.inputCh) < regionWorkerHighWatermark {
err = w.processEvent(ctx, event)
if err != nil {
return err
}
} else {
err = w.handles[int(event.regionID)%w.concurrent].AddEvent(ctx, event)
if err != nil {
return err
}
// TODO: add events in batch
for len(w.inputCh) >= regionWorkerLowWatermark {
event, ok, err = pollEvent()
if err != nil {
return err
}
exitEventHandler, skipEvent := preprocess(event, ok)
if exitEventHandler {
return cerror.ErrRegionWorkerExit.GenWithStackByArgs()
}
if skipEvent {
continue
}
err = w.handles[int(event.regionID)%w.concurrent].AddEvent(ctx, event)
if err != nil {
return err
}
}
// Principle: events from the same region must be processed linearly.
//
// When buffered events exceed high watermark, we start to use worker
// pool to improve throughput, and we need a mechanism to quit worker
// pool when buffered events are less than low watermark, which means
// we should have a way to know whether events sent to the worker pool
// are all processed.
// Send a dummy event to each worker pool handler, after each of these
// events are processed, we can ensure all events sent to worker pool
// from this region worker are processed.
finishedCallbackCh := make(chan struct{}, len(w.handles))
for _, handle := range w.handles {
err = handle.AddEvent(ctx, ®ionStatefulEvent{finishedCallbackCh: finishedCallbackCh})
if err != nil {
return err
}
}
counter := len(w.handles)
checkEventsProcessed:
for {
select {
case <-ctx.Done():
return errors.Trace(ctx.Err())
case err = <-w.errorCh:
return err
case <-finishedCallbackCh:
counter--
if counter == 0 {
break checkEventsProcessed
}
}
}
}
}
}
func (w *regionWorker) collectWorkpoolError(ctx context.Context) error {
cases := make([]reflect.SelectCase, 0, len(w.handles)+1)
cases = append(cases, reflect.SelectCase{
Dir: reflect.SelectRecv,
Chan: reflect.ValueOf(ctx.Done()),
})
for _, handle := range w.handles {
cases = append(cases, reflect.SelectCase{
Dir: reflect.SelectRecv,
Chan: reflect.ValueOf(handle.ErrCh()),
})
}
idx, value, ok := reflect.Select(cases)
if idx == 0 {
return ctx.Err()
}
if !ok {
return nil
}
return value.Interface().(error)
}
func (w *regionWorker) checkErrorReconnect(err error) error {
if errors.Cause(err) == errReconnect {
w.cancelStream(time.Second)
// if stream is already deleted, just ignore errReconnect
return nil
}
return err
}
func (w *regionWorker) cancelStream(delay time.Duration) {
cancel, ok := w.session.getStreamCancel(w.storeAddr)
if ok {
// cancel the stream to trigger strem.Recv with context cancel error
// Note use context cancel is the only way to terminate a gRPC stream
cancel()
// Failover in stream.Recv has 0-100ms delay, the onRegionFail
// should be called after stream has been deleted. Add a delay here
// to avoid too frequent region rebuilt.
time.Sleep(delay)
} else {
log.Warn("gRPC stream cancel func not found",
zap.String("addr", w.storeAddr),
zap.String("namesapce", w.session.client.changefeed.Namespace),
zap.String("changefeed", w.session.client.changefeed.ID))
}
}
func (w *regionWorker) run(parentCtx context.Context) error {
defer func() {
for _, h := range w.handles {
h.Unregister()
}
}()
w.parentCtx = parentCtx
ctx, cancel := context.WithCancel(parentCtx)
wg, ctx := errgroup.WithContext(ctx)
w.initMetrics(ctx)
w.initPoolHandles(w.concurrent)
var retErr error
once := sync.Once{}
handleError := func(err error) error {
if err != nil {
once.Do(func() {
cancel()
retErr = err
})
}
return err
}
wg.Go(func() error {
return handleError(w.checkErrorReconnect(w.resolveLock(ctx)))
})
wg.Go(func() error {
return handleError(w.eventHandler(ctx))
})
_ = handleError(w.collectWorkpoolError(ctx))
_ = wg.Wait()
// ErrRegionWorkerExit means the region worker exits normally, but we don't
// need to terminate the other goroutines in errgroup
if cerror.ErrRegionWorkerExit.Equal(retErr) {
return nil
}
return retErr
}
func (w *regionWorker) handleEventEntry(
ctx context.Context,
x *cdcpb.Event_Entries_,
state *regionFeedState,
) error {
regionID := state.sri.verID.GetID()
for _, entry := range x.Entries.GetEntries() {
// if a region with kv range [a, z)
// and we only want the get [b, c) from this region,
// tikv will return all key events in the region although we specified [b, c) int the request.
// we can make tikv only return the events about the keys in the specified range.
comparableKey := regionspan.ToComparableKey(entry.GetKey())
// key for initialized event is nil
if !regionspan.KeyInSpan(comparableKey, state.sri.span) && entry.Type != cdcpb.Event_INITIALIZED {
w.metrics.metricDroppedEventSize.Observe(float64(entry.Size()))
continue
}
switch entry.Type {
case cdcpb.Event_INITIALIZED:
if time.Since(state.startFeedTime) > 20*time.Second {
log.Warn("The time cost of initializing is too much",
zap.String("namesapce", w.session.client.changefeed.Namespace),
zap.String("changefeed", w.session.client.changefeed.ID),
zap.Duration("duration", time.Since(state.startFeedTime)),
zap.Uint64("regionID", regionID))
}
w.metrics.metricPullEventInitializedCounter.Inc()
state.initialized = true
w.session.regionRouter.Release(state.sri.rpcCtx.Addr)
cachedEvents := state.matcher.matchCachedRow()
for _, cachedEvent := range cachedEvents {
revent, err := assembleRowEvent(regionID, cachedEvent)
if err != nil {
return errors.Trace(err)
}
select {
case w.outputCh <- revent:
w.metrics.metricSendEventCommitCounter.Inc()
case <-ctx.Done():
return errors.Trace(ctx.Err())
}
}
case cdcpb.Event_COMMITTED:
w.metrics.metricPullEventCommittedCounter.Inc()
revent, err := assembleRowEvent(regionID, entry)
if err != nil {
return errors.Trace(err)
}
if entry.CommitTs <= state.lastResolvedTs {
logPanic("The CommitTs must be greater than the resolvedTs",
zap.String("EventType", "COMMITTED"),
zap.Uint64("CommitTs", entry.CommitTs),
zap.Uint64("resolvedTs", state.lastResolvedTs),
zap.Uint64("regionID", regionID))
return errUnreachable
}
select {
case w.outputCh <- revent:
w.metrics.metricSendEventCommittedCounter.Inc()
case <-ctx.Done():
return errors.Trace(ctx.Err())
}
case cdcpb.Event_PREWRITE:
w.metrics.metricPullEventPrewriteCounter.Inc()
state.matcher.putPrewriteRow(entry)
case cdcpb.Event_COMMIT:
w.metrics.metricPullEventCommitCounter.Inc()
if entry.CommitTs <= state.lastResolvedTs {
logPanic("The CommitTs must be greater than the resolvedTs",
zap.String("EventType", "COMMIT"),
zap.Uint64("CommitTs", entry.CommitTs),
zap.Uint64("resolvedTs", state.lastResolvedTs),
zap.Uint64("regionID", regionID))
return errUnreachable
}
ok := state.matcher.matchRow(entry)
if !ok {
if !state.initialized {
state.matcher.cacheCommitRow(entry)
continue
}
return cerror.ErrPrewriteNotMatch.GenWithStackByArgs(
hex.EncodeToString(entry.GetKey()),
entry.GetStartTs(), entry.GetCommitTs(),
entry.GetType(), entry.GetOpType())
}
revent, err := assembleRowEvent(regionID, entry)
if err != nil {
return errors.Trace(err)
}
select {
case w.outputCh <- revent:
w.metrics.metricSendEventCommitCounter.Inc()
case <-ctx.Done():
return errors.Trace(ctx.Err())
}
case cdcpb.Event_ROLLBACK:
w.metrics.metricPullEventRollbackCounter.Inc()
state.matcher.rollbackRow(entry)
}
}
return nil
}
func (w *regionWorker) handleResolvedTs(
ctx context.Context,
resolvedTs uint64,
state *regionFeedState,
) error {
if !state.initialized {
return nil
}
regionID := state.sri.verID.GetID()
// Send resolved ts update in non-blocking way, since we can re-query real
// resolved ts from region state even if resolved ts update is discarded.
// NOTICE: We send any regionTsInfo to resolveLock thread to give us a chance to trigger resolveLock logic
// (1) if it is a fallback resolvedTs event, it will be discarded and accumulate penalty on the progress;
// (2) if it is a normal one, update rtsManager and check sinceLastResolvedTs
select {
case w.rtsUpdateCh <- ®ionTsInfo{regionID: regionID, ts: newResolvedTsItem(resolvedTs)}:
default:
}
if resolvedTs < state.lastResolvedTs {
log.Warn("The resolvedTs is fallen back in kvclient",
zap.String("namesapce", w.session.client.changefeed.Namespace),
zap.String("changefeed", w.session.client.changefeed.ID),
zap.String("EventType", "RESOLVED"),
zap.Uint64("resolvedTs", resolvedTs),
zap.Uint64("lastResolvedTs", state.lastResolvedTs),
zap.Uint64("regionID", regionID))
return nil
}
state.lastResolvedTs = resolvedTs
// emit a checkpointTs
revent := model.RegionFeedEvent{
RegionID: regionID,
Resolved: &model.ResolvedSpan{
Span: state.sri.span,
ResolvedTs: resolvedTs,
},
}
select {
case w.outputCh <- revent:
w.metrics.metricSendEventResolvedCounter.Inc()
case <-ctx.Done():
return errors.Trace(ctx.Err())
}
return nil
}
// evictAllRegions is used when gRPC stream meets error and re-establish, notify
// all existing regions to re-establish
func (w *regionWorker) evictAllRegions() {
for _, states := range w.statesManager.states {
states.Range(func(_, value interface{}) bool {
state := value.(*regionFeedState)
state.lock.Lock()
// if state is marked as stopped, it must have been or would be processed by `onRegionFail`
if state.isStopped() {
state.lock.Unlock()
return true
}
state.markStopped()
w.delRegionState(state.sri.verID.GetID())
if state.lastResolvedTs > state.sri.ts {
state.sri.ts = state.lastResolvedTs
}
revokeToken := !state.initialized
state.lock.Unlock()
// since the context used in region worker will be cancelled after
// region worker exits, we must use the parent context to prevent
// regionErrorInfo loss.
errInfo := newRegionErrorInfo(state.sri, cerror.ErrEventFeedAborted.FastGenByArgs())
w.session.onRegionFail(w.parentCtx, errInfo, revokeToken)
return true
})
}
}
func getWorkerPoolSize() (size int) {
cfg := config.GetGlobalServerConfig().KVClient
if cfg.WorkerPoolSize > 0 {
size = cfg.WorkerPoolSize
} else {
size = runtime.NumCPU() * 2
}
if size > maxWorkerPoolSize {
size = maxWorkerPoolSize
}
return
}
// InitWorkerPool initialize workerpool once, the workerpool must be initialized
// before any kv event is received.
func InitWorkerPool() {
workerPoolOnce.Do(func() {
size := getWorkerPoolSize()
regionWorkerPool = workerpool.NewDefaultWorkerPool(size)
})
}
// RunWorkerPool runs the worker pool used by the region worker in kv client v2
// It must be running before region worker starts to work
func RunWorkerPool(ctx context.Context) error {
InitWorkerPool()
errg, ctx := errgroup.WithContext(ctx)
errg.Go(func() error {
return errors.Trace(regionWorkerPool.Run(ctx))
})
return errg.Wait()
}