/
shared_client.go
951 lines (842 loc) · 31.5 KB
/
shared_client.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
// Copyright 2023 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/binary"
"fmt"
"sync"
"sync/atomic"
"time"
"blainsmith.com/go/seahash"
"github.com/pingcap/errors"
"github.com/pingcap/kvproto/pkg/cdcpb"
"github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/pingcap/kvproto/pkg/metapb"
"github.com/pingcap/log"
"github.com/pingcap/tiflow/cdc/kv/regionlock"
"github.com/pingcap/tiflow/cdc/kv/sharedconn"
"github.com/pingcap/tiflow/cdc/model"
"github.com/pingcap/tiflow/cdc/processor/tablepb"
"github.com/pingcap/tiflow/pkg/chann"
"github.com/pingcap/tiflow/pkg/config"
cerror "github.com/pingcap/tiflow/pkg/errors"
"github.com/pingcap/tiflow/pkg/pdutil"
"github.com/pingcap/tiflow/pkg/spanz"
"github.com/pingcap/tiflow/pkg/txnutil"
"github.com/pingcap/tiflow/pkg/util"
"github.com/pingcap/tiflow/pkg/version"
"github.com/prometheus/client_golang/prometheus"
kvclientv2 "github.com/tikv/client-go/v2/kv"
"github.com/tikv/client-go/v2/oracle"
"github.com/tikv/client-go/v2/tikv"
pd "github.com/tikv/pd/client"
"go.uber.org/zap"
"golang.org/x/sync/errgroup"
)
const (
dialTimeout = 10 * time.Second
// Maximum total sleep time(in ms), 20 seconds.
tikvRequestMaxBackoff = 20000
// TiCDC may open numerous gRPC streams,
// with 65535 bytes window size, 10K streams takes about 27GB memory.
//
// 65535 bytes, the initial window size in http2 spec.
grpcInitialWindowSize = (1 << 16) - 1
// 8 MB The value for initial window size on a connection
grpcInitialConnWindowSize = 1 << 23
// 256 MB The maximum message size the client can receive
grpcMaxCallRecvMsgSize = 1 << 28
// TiCDC always interacts with region leader, every time something goes wrong,
// failed region will be reloaded via `BatchLoadRegionsWithKeyRange` API. So we
// don't need to force reload region anymore.
regionScheduleReload = false
scanRegionsConcurrency = 1024
loadRegionRetryInterval time.Duration = 100 * time.Millisecond
resolveLockMinInterval time.Duration = 10 * time.Second
invalidSubscriptionID SubscriptionID = SubscriptionID(0)
)
var (
// To generate an ID for a new subscription. And the subscription ID will also be used as
// `RequestId` in region requests of the table.
subscriptionIDGen atomic.Uint64
// To generate a streamID in `newStream`.
streamIDGen atomic.Uint64
)
var (
// unreachable error, only used in unit test
errUnreachable = errors.New("kv client unreachable error")
logPanic = log.Panic
)
var (
metricFeedNotLeaderCounter = eventFeedErrorCounter.WithLabelValues("NotLeader")
metricFeedEpochNotMatchCounter = eventFeedErrorCounter.WithLabelValues("EpochNotMatch")
metricFeedRegionNotFoundCounter = eventFeedErrorCounter.WithLabelValues("RegionNotFound")
metricFeedDuplicateRequestCounter = eventFeedErrorCounter.WithLabelValues("DuplicateRequest")
metricFeedUnknownErrorCounter = eventFeedErrorCounter.WithLabelValues("Unknown")
metricFeedRPCCtxUnavailable = eventFeedErrorCounter.WithLabelValues("RPCCtxUnavailable")
metricStoreSendRequestErr = eventFeedErrorCounter.WithLabelValues("SendRequestToStore")
metricKvIsBusyCounter = eventFeedErrorCounter.WithLabelValues("KvIsBusy")
)
type eventError struct {
err *cdcpb.Error
}
// Error implement error interface.
func (e *eventError) Error() string {
return e.err.String()
}
type rpcCtxUnavailableErr struct {
verID tikv.RegionVerID
}
func (e *rpcCtxUnavailableErr) Error() string {
return fmt.Sprintf("cannot get rpcCtx for region %v. ver:%v, confver:%v",
e.verID.GetID(), e.verID.GetVer(), e.verID.GetConfVer())
}
type sendRequestToStoreErr struct{}
func (e *sendRequestToStoreErr) Error() string { return "send request to store error" }
// SubscriptionID comes from `SharedClient.AllocSubscriptionID`.
type SubscriptionID uint64
// MultiplexingEvent wrap a region event with
// SubscriptionID to indicate which subscription it belongs to.
type MultiplexingEvent struct {
model.RegionFeedEvent
SubscriptionID SubscriptionID
Start time.Time
}
// newMultiplexingEvent creates a new MultiplexingEvent.
func newMultiplexingEvent(e model.RegionFeedEvent, table *subscribedTable) MultiplexingEvent {
return MultiplexingEvent{
RegionFeedEvent: e,
SubscriptionID: table.subscriptionID,
Start: time.Now(),
}
}
// SharedClient is shared by many tables to pull events from TiKV.
// All exported Methods are thread-safe.
type SharedClient struct {
changefeed model.ChangeFeedID
config *config.ServerConfig
metrics sharedClientMetrics
clusterID uint64
filterLoop bool
pd pd.Client
grpcPool *sharedconn.ConnAndClientPool
regionCache *tikv.RegionCache
pdClock pdutil.Clock
lockResolver txnutil.LockResolver
totalSpans struct {
sync.RWMutex
v map[SubscriptionID]*subscribedTable
}
workers []*sharedRegionWorker
// Note: stores is only motified in handleRegion goroutine,
// so it is not protected by a lock.
stores map[string]*requestedStore
// rangeTaskCh is used to receive range tasks.
// The tasks will be handled in `handleRangeTask` goroutine.
rangeTaskCh *chann.DrainableChann[rangeTask]
// regionCh is used to receive region tasks have been locked in rangeLock.
// The region will be handled in `handleRegions` goroutine.
regionCh *chann.DrainableChann[regionInfo]
// resolveLockTaskCh is used to receive resolve lock tasks.
// The tasks will be handled in `handleResolveLockTasks` goroutine.
resolveLockTaskCh *chann.DrainableChann[resolveLockTask]
errCh *chann.DrainableChann[regionErrorInfo]
logRegionDetails func(msg string, fields ...zap.Field)
}
type resolveLockTask struct {
regionID uint64
targetTs uint64
state *regionlock.LockedRangeState
create time.Time
}
// rangeTask represents a task to subscribe a range span of a table.
// It can be a part of a table or a whole table, it also can be a part of a region.
type rangeTask struct {
span tablepb.Span
subscribedTable *subscribedTable
}
// requestedStore represents a store that has been connected.
// A store may have multiple streams.
type requestedStore struct {
storeID uint64
storeAddr string
// Use to select a stream to send request.
nextStream atomic.Uint32
streams []*requestedStream
}
func (rs *requestedStore) getStream() *requestedStream {
index := rs.nextStream.Add(1) % uint32(len(rs.streams))
return rs.streams[index]
}
// subscribedTable represents a table to subscribe.
// It contains the span of the table, the startTs of the table, and the output event channel.
type subscribedTable struct {
subscriptionID SubscriptionID
startTs model.Ts
// The whole span of the table.
span tablepb.Span
// The range lock of the table,
// it is used to prevent duplicate requests to the same region range,
// and it also used to calculate this table's resolvedTs.
rangeLock *regionlock.RangeLock
// The output event channel of the table.
eventCh chan<- MultiplexingEvent
// To handle table removing.
stopped atomic.Bool
// To handle stale lock resolvings.
tryResolveLock func(regionID uint64, state *regionlock.LockedRangeState)
staleLocksTargetTs atomic.Uint64
lastAdvanceTime atomic.Int64
}
// NewSharedClient creates a client.
func NewSharedClient(
changefeed model.ChangeFeedID,
cfg *config.ServerConfig,
filterLoop bool,
pd pd.Client,
grpcPool *sharedconn.ConnAndClientPool,
regionCache *tikv.RegionCache,
pdClock pdutil.Clock,
lockResolver txnutil.LockResolver,
) *SharedClient {
s := &SharedClient{
changefeed: changefeed,
config: cfg,
clusterID: 0,
filterLoop: filterLoop,
pd: pd,
grpcPool: grpcPool,
regionCache: regionCache,
pdClock: pdClock,
lockResolver: lockResolver,
rangeTaskCh: chann.NewAutoDrainChann[rangeTask](),
regionCh: chann.NewAutoDrainChann[regionInfo](),
resolveLockTaskCh: chann.NewAutoDrainChann[resolveLockTask](),
errCh: chann.NewAutoDrainChann[regionErrorInfo](),
stores: make(map[string]*requestedStore),
}
s.totalSpans.v = make(map[SubscriptionID]*subscribedTable)
if cfg.Debug.Puller.LogRegionDetails {
s.logRegionDetails = log.Info
} else {
s.logRegionDetails = log.Debug
}
s.initMetrics()
return s
}
// AllocSubscriptionID gets an ID can be used in `Subscribe`.
func (s *SharedClient) AllocSubscriptionID() SubscriptionID {
return SubscriptionID(subscriptionIDGen.Add(1))
}
// Subscribe the given table span.
// NOTE: `span.TableID` must be set correctly.
// It new a subscribedTable and store it in `s.totalSpans`,
// and send a rangeTask to `s.rangeTaskCh`.
// The rangeTask will be handled in `handleRangeTasks` goroutine.
func (s *SharedClient) Subscribe(subID SubscriptionID, span tablepb.Span, startTs uint64, eventCh chan<- MultiplexingEvent) {
if span.TableID == 0 {
log.Panic("event feed subscribe with zero tablepb.Span.TableID",
zap.String("namespace", s.changefeed.Namespace),
zap.String("changefeed", s.changefeed.ID))
}
rt := s.newSubscribedTable(subID, span, startTs, eventCh)
s.totalSpans.Lock()
s.totalSpans.v[subID] = rt
s.totalSpans.Unlock()
s.rangeTaskCh.In() <- rangeTask{span: span, subscribedTable: rt}
log.Info("event feed subscribes table success",
zap.String("namespace", s.changefeed.Namespace),
zap.String("changefeed", s.changefeed.ID),
zap.Any("subscriptionID", rt.subscriptionID),
zap.String("span", rt.span.String()))
}
// Unsubscribe the given table span. All covered regions will be deregistered asynchronously.
// NOTE: `span.TableID` must be set correctly.
func (s *SharedClient) Unsubscribe(subID SubscriptionID) {
// NOTE: `subID` is cleared from `s.totalSpans` in `onTableDrained`.
s.totalSpans.Lock()
rt := s.totalSpans.v[subID]
s.totalSpans.Unlock()
if rt != nil {
s.setTableStopped(rt)
}
log.Info("event feed unsubscribes table",
zap.String("namespace", s.changefeed.Namespace),
zap.String("changefeed", s.changefeed.ID),
zap.Any("subscriptionID", rt.subscriptionID),
zap.Bool("exists", rt != nil))
}
// ResolveLock is a function. If outsider subscribers find a span resolved timestamp is
// advanced slowly or stopped, they can try to resolve locks in the given span.
func (s *SharedClient) ResolveLock(subID SubscriptionID, targetTs uint64) {
s.totalSpans.Lock()
rt := s.totalSpans.v[subID]
s.totalSpans.Unlock()
if rt != nil {
rt.resolveStaleLocks(s, targetTs)
}
}
// RegionCount returns subscribed region count for the span.
func (s *SharedClient) RegionCount(subID SubscriptionID) uint64 {
s.totalSpans.RLock()
defer s.totalSpans.RUnlock()
if rt := s.totalSpans.v[subID]; rt != nil {
return uint64(rt.rangeLock.Len())
}
return 0
}
// Run the client.
func (s *SharedClient) Run(ctx context.Context) error {
s.clusterID = s.pd.GetClusterID(ctx)
g, ctx := errgroup.WithContext(ctx)
s.workers = make([]*sharedRegionWorker, 0, s.config.KVClient.WorkerConcurrent)
for i := uint(0); i < s.config.KVClient.WorkerConcurrent; i++ {
worker := newSharedRegionWorker(s)
g.Go(func() error { return worker.run(ctx) })
s.workers = append(s.workers, worker)
}
g.Go(func() error { return s.handleRangeTasks(ctx) })
g.Go(func() error { return s.handleRegions(ctx, g) })
g.Go(func() error { return s.handleErrors(ctx) })
g.Go(func() error { return s.handleResolveLockTasks(ctx) })
g.Go(func() error { return s.logSlowRegions(ctx) })
log.Info("event feed started",
zap.String("namespace", s.changefeed.Namespace),
zap.String("changefeed", s.changefeed.ID))
defer log.Info("event feed exits",
zap.String("namespace", s.changefeed.Namespace),
zap.String("changefeed", s.changefeed.ID))
return g.Wait()
}
// Close closes the client. Must be called after `Run` returns.
func (s *SharedClient) Close() {
s.rangeTaskCh.CloseAndDrain()
s.regionCh.CloseAndDrain()
s.resolveLockTaskCh.CloseAndDrain()
s.errCh.CloseAndDrain()
s.clearMetrics()
for _, rs := range s.stores {
for _, stream := range rs.streams {
stream.requests.CloseAndDrain()
}
}
}
func (s *SharedClient) setTableStopped(rt *subscribedTable) {
log.Info("event feed starts to stop table",
zap.String("namespace", s.changefeed.Namespace),
zap.String("changefeed", s.changefeed.ID),
zap.Any("subscriptionID", rt.subscriptionID))
// Set stopped to true so we can stop handling region events from the table.
// Then send a special singleRegionInfo to regionRouter to deregister the table
// from all TiKV instances.
if rt.stopped.CompareAndSwap(false, true) {
s.regionCh.In() <- regionInfo{subscribedTable: rt}
if rt.rangeLock.Stop() {
s.onTableDrained(rt)
}
}
}
func (s *SharedClient) onTableDrained(rt *subscribedTable) {
log.Info("event feed stop table is finished",
zap.String("namespace", s.changefeed.Namespace),
zap.String("changefeed", s.changefeed.ID),
zap.Any("subscriptionID", rt.subscriptionID))
s.totalSpans.Lock()
defer s.totalSpans.Unlock()
delete(s.totalSpans.v, rt.subscriptionID)
}
func (s *SharedClient) onRegionFail(errInfo regionErrorInfo) {
s.errCh.In() <- errInfo
}
// handleRegions receives regionInfo from regionCh and attch rpcCtx to them,
// then send them to corresponding requestedStore.
func (s *SharedClient) handleRegions(ctx context.Context, eg *errgroup.Group) error {
for {
select {
case <-ctx.Done():
return errors.Trace(ctx.Err())
case region := <-s.regionCh.Out():
if region.isStoped() {
for _, rs := range s.stores {
s.broadcastRequest(rs, region)
}
continue
}
region, ok := s.attachRPCContextForRegion(ctx, region)
// If attachRPCContextForRegion fails, the region will be re-scheduled.
if !ok {
continue
}
store := s.getStore(ctx, eg, region.rpcCtx.Peer.StoreId, region.rpcCtx.Addr)
stream := store.getStream()
stream.requests.In() <- region
s.logRegionDetails("event feed will request a region",
zap.String("namespace", s.changefeed.Namespace),
zap.String("changefeed", s.changefeed.ID),
zap.Uint64("streamID", stream.streamID),
zap.Any("subscriptionID", region.subscribedTable.subscriptionID),
zap.Uint64("regionID", region.verID.GetID()),
zap.Uint64("storeID", store.storeID),
zap.String("addr", store.storeAddr))
}
}
}
func (s *SharedClient) attachRPCContextForRegion(ctx context.Context, region regionInfo) (regionInfo, bool) {
bo := tikv.NewBackoffer(ctx, tikvRequestMaxBackoff)
rpcCtx, err := s.regionCache.GetTiKVRPCContext(bo, region.verID, kvclientv2.ReplicaReadLeader, 0)
if rpcCtx != nil {
region.rpcCtx = rpcCtx
locateTime := time.Since(region.lockedRangeState.Created).Milliseconds()
s.metrics.regionLocateDuration.Observe(float64(locateTime))
return region, true
}
if err != nil {
log.Debug("event feed get RPC context fail",
zap.String("namespace", s.changefeed.Namespace),
zap.String("changefeed", s.changefeed.ID),
zap.Any("subscriptionID", region.subscribedTable.subscriptionID),
zap.Uint64("regionID", region.verID.GetID()),
zap.Error(err))
}
s.onRegionFail(newRegionErrorInfo(region, &rpcCtxUnavailableErr{verID: region.verID}))
return region, false
}
// getStore gets a requestedStore from requestedStores by storeAddr.
func (s *SharedClient) getStore(
ctx context.Context, g *errgroup.Group,
storeID uint64, storeAddr string,
) *requestedStore {
var rs *requestedStore
if rs = s.stores[storeAddr]; rs != nil {
return rs
}
rs = &requestedStore{storeID: storeID, storeAddr: storeAddr}
s.stores[storeAddr] = rs
for i := uint(0); i < s.config.KVClient.GrpcStreamConcurrent; i++ {
stream := newStream(ctx, s, g, rs)
rs.streams = append(rs.streams, stream)
}
return rs
}
func (s *SharedClient) createRegionRequest(region regionInfo) *cdcpb.ChangeDataRequest {
return &cdcpb.ChangeDataRequest{
Header: &cdcpb.Header{ClusterId: s.clusterID, TicdcVersion: version.ReleaseSemver()},
RegionId: region.verID.GetID(),
RequestId: uint64(region.subscribedTable.subscriptionID),
RegionEpoch: region.rpcCtx.Meta.RegionEpoch,
CheckpointTs: region.resolvedTs(),
StartKey: region.span.StartKey,
EndKey: region.span.EndKey,
ExtraOp: kvrpcpb.ExtraOp_ReadOldValue,
FilterLoop: s.filterLoop,
}
}
func (s *SharedClient) broadcastRequest(r *requestedStore, region regionInfo) {
for _, stream := range r.streams {
stream.requests.In() <- region
}
}
func (s *SharedClient) handleRangeTasks(ctx context.Context) error {
g, ctx := errgroup.WithContext(ctx)
g.SetLimit(scanRegionsConcurrency)
for {
select {
case <-ctx.Done():
return ctx.Err()
case task := <-s.rangeTaskCh.Out():
g.Go(func() error { return s.divideSpanAndScheduleRegionRequests(ctx, task.span, task.subscribedTable) })
}
}
}
// divideSpanAndScheduleRegionRequests processes the specified span by dividing it into
// manageable regions and schedules requests to subscribe to these regions.
// 1. Load regions from PD.
// 2. Find the intersection of each region.span and the subscribedTable.span.
// 3. Schedule a region request to subscribe the region.
func (s *SharedClient) divideSpanAndScheduleRegionRequests(
ctx context.Context,
span tablepb.Span,
subscribedTable *subscribedTable,
) error {
// Limit the number of regions loaded at a time to make the load more stable.
limit := 1024
nextSpan := span
backoffBeforeLoad := false
for {
if backoffBeforeLoad {
if err := util.Hang(ctx, loadRegionRetryInterval); err != nil {
return err
}
backoffBeforeLoad = false
}
log.Debug("event feed is going to load regions",
zap.String("namespace", s.changefeed.Namespace),
zap.String("changefeed", s.changefeed.ID),
zap.Any("subscriptionID", subscribedTable.subscriptionID),
zap.Any("span", nextSpan))
backoff := tikv.NewBackoffer(ctx, tikvRequestMaxBackoff)
regions, err := s.regionCache.BatchLoadRegionsWithKeyRange(backoff, nextSpan.StartKey, nextSpan.EndKey, limit)
if err != nil {
log.Warn("event feed load regions failed",
zap.String("namespace", s.changefeed.Namespace),
zap.String("changefeed", s.changefeed.ID),
zap.Any("subscriptionID", subscribedTable.subscriptionID),
zap.Any("span", nextSpan),
zap.Error(err))
backoffBeforeLoad = true
continue
}
regionMetas := make([]*metapb.Region, 0, len(regions))
for _, region := range regions {
if meta := region.GetMeta(); meta != nil {
regionMetas = append(regionMetas, meta)
}
}
regionMetas = regionlock.CutRegionsLeftCoverSpan(regionMetas, nextSpan)
if len(regionMetas) == 0 {
log.Warn("event feed load regions with holes",
zap.String("namespace", s.changefeed.Namespace),
zap.String("changefeed", s.changefeed.ID),
zap.Any("subscriptionID", subscribedTable.subscriptionID),
zap.Any("span", nextSpan))
backoffBeforeLoad = true
continue
}
for _, regionMeta := range regionMetas {
regionSpan := tablepb.Span{StartKey: regionMeta.StartKey, EndKey: regionMeta.EndKey}
// NOTE: the End key return by the PD API will be nil to represent the biggest key.
// So we need to fix it by calling spanz.HackSpan.
regionSpan = spanz.HackSpan(regionSpan)
// Find the intersection of the regionSpan returned by PD and the subscribedTable.span.
// The intersection is the span that needs to be subscribed.
intersectantSpan, err := spanz.Intersect(subscribedTable.span, regionSpan)
if err != nil {
log.Panic("event feed check spans intersect shouldn't fail",
zap.String("namespace", s.changefeed.Namespace),
zap.String("changefeed", s.changefeed.ID),
zap.Any("subscriptionID", subscribedTable.subscriptionID))
}
verID := tikv.NewRegionVerID(regionMeta.Id, regionMeta.RegionEpoch.ConfVer, regionMeta.RegionEpoch.Version)
regionInfo := newRegionInfo(verID, intersectantSpan, nil, subscribedTable)
// Schedule a region request to subscribe the region.
s.scheduleRegionRequest(ctx, regionInfo)
nextSpan.StartKey = regionMeta.EndKey
// If the nextSpan.StartKey is larger than the subscribedTable.span.EndKey,
// it means all span of the subscribedTable have been requested. So we return.
if spanz.EndCompare(nextSpan.StartKey, span.EndKey) >= 0 {
return nil
}
}
}
}
// scheduleRegionRequest locks the region's range and send the region to regionCh,
// which will be handled by handleRegions.
func (s *SharedClient) scheduleRegionRequest(ctx context.Context, region regionInfo) {
lockRangeResult := region.subscribedTable.rangeLock.LockRange(
ctx, region.span.StartKey, region.span.EndKey, region.verID.GetID(), region.verID.GetVer())
if lockRangeResult.Status == regionlock.LockRangeStatusWait {
lockRangeResult = lockRangeResult.WaitFn()
}
switch lockRangeResult.Status {
case regionlock.LockRangeStatusSuccess:
region.lockedRangeState = lockRangeResult.LockedRangeState
lockTime := time.Since(region.lockedRangeState.Created).Milliseconds()
s.metrics.regionLockDuration.Observe(float64(lockTime))
select {
case s.regionCh.In() <- region:
case <-ctx.Done():
}
case regionlock.LockRangeStatusStale:
for _, r := range lockRangeResult.RetryRanges {
s.scheduleRangeRequest(ctx, r, region.subscribedTable)
}
default:
return
}
}
func (s *SharedClient) scheduleRangeRequest(
ctx context.Context, span tablepb.Span,
subscribedTable *subscribedTable,
) {
select {
case s.rangeTaskCh.In() <- rangeTask{span: span, subscribedTable: subscribedTable}:
case <-ctx.Done():
}
}
func (s *SharedClient) handleErrors(ctx context.Context) error {
for {
select {
case <-ctx.Done():
return ctx.Err()
case errInfo := <-s.errCh.Out():
if err := s.doHandleError(ctx, errInfo); err != nil {
return err
}
}
}
}
func (s *SharedClient) doHandleError(ctx context.Context, errInfo regionErrorInfo) error {
if errInfo.subscribedTable.rangeLock.UnlockRange(
errInfo.span.StartKey, errInfo.span.EndKey,
errInfo.verID.GetID(), errInfo.verID.GetVer(), errInfo.resolvedTs()) {
s.onTableDrained(errInfo.subscribedTable)
return nil
}
err := errors.Cause(errInfo.err)
switch eerr := err.(type) {
case *eventError:
innerErr := eerr.err
s.logRegionDetails("cdc region error",
zap.String("namespace", s.changefeed.Namespace),
zap.String("changefeed", s.changefeed.ID),
zap.Any("subscriptionID", errInfo.subscribedTable.subscriptionID),
zap.Stringer("error", innerErr))
if notLeader := innerErr.GetNotLeader(); notLeader != nil {
metricFeedNotLeaderCounter.Inc()
s.regionCache.UpdateLeader(errInfo.verID, notLeader.GetLeader(), errInfo.rpcCtx.AccessIdx)
s.scheduleRegionRequest(ctx, errInfo.regionInfo)
return nil
}
if innerErr.GetEpochNotMatch() != nil {
metricFeedEpochNotMatchCounter.Inc()
s.scheduleRangeRequest(ctx, errInfo.span, errInfo.subscribedTable)
return nil
}
if innerErr.GetRegionNotFound() != nil {
metricFeedRegionNotFoundCounter.Inc()
s.scheduleRangeRequest(ctx, errInfo.span, errInfo.subscribedTable)
return nil
}
if innerErr.GetServerIsBusy() != nil {
metricKvIsBusyCounter.Inc()
s.scheduleRegionRequest(ctx, errInfo.regionInfo)
return nil
}
if duplicated := innerErr.GetDuplicateRequest(); duplicated != nil {
metricFeedDuplicateRequestCounter.Inc()
// TODO(qupeng): It's better to add a new machanism to deregister one region.
return errors.New("duplicate request")
}
if compatibility := innerErr.GetCompatibility(); compatibility != nil {
return cerror.ErrVersionIncompatible.GenWithStackByArgs(compatibility)
}
if mismatch := innerErr.GetClusterIdMismatch(); mismatch != nil {
return cerror.ErrClusterIDMismatch.GenWithStackByArgs(mismatch.Current, mismatch.Request)
}
log.Warn("empty or unknown cdc error",
zap.String("namespace", s.changefeed.Namespace),
zap.String("changefeed", s.changefeed.ID),
zap.Any("subscriptionID", errInfo.subscribedTable.subscriptionID),
zap.Stringer("error", innerErr))
metricFeedUnknownErrorCounter.Inc()
s.scheduleRegionRequest(ctx, errInfo.regionInfo)
return nil
case *rpcCtxUnavailableErr:
metricFeedRPCCtxUnavailable.Inc()
s.scheduleRangeRequest(ctx, errInfo.span, errInfo.subscribedTable)
return nil
case *sendRequestToStoreErr:
metricStoreSendRequestErr.Inc()
bo := tikv.NewBackoffer(ctx, tikvRequestMaxBackoff)
s.regionCache.OnSendFail(bo, errInfo.rpcCtx, regionScheduleReload, err)
s.scheduleRegionRequest(ctx, errInfo.regionInfo)
return nil
default:
// TODO(qupeng): for some errors it's better to just deregister the region from TiKVs.
log.Warn("event feed meets an internal error, fail the changefeed",
zap.String("namespace", s.changefeed.Namespace),
zap.String("changefeed", s.changefeed.ID),
zap.Any("subscriptionID", errInfo.subscribedTable.subscriptionID),
zap.Error(err))
return err
}
}
func (s *SharedClient) handleResolveLockTasks(ctx context.Context) error {
resolveLastRun := make(map[uint64]time.Time)
gcResolveLastRun := func() {
if len(resolveLastRun) > 1024 {
copied := make(map[uint64]time.Time)
now := time.Now()
for regionID, lastRun := range resolveLastRun {
if now.Sub(lastRun) < resolveLockMinInterval {
resolveLastRun[regionID] = lastRun
}
}
resolveLastRun = copied
}
}
doResolve := func(regionID uint64, state *regionlock.LockedRangeState, targetTs uint64) {
if state.ResolvedTs.Load() > targetTs || !state.Initialzied.Load() {
return
}
if lastRun, ok := resolveLastRun[regionID]; ok {
if time.Since(lastRun) < resolveLockMinInterval {
return
}
}
start := time.Now()
defer func() {
s.metrics.lockResolveRunDuration.Observe(float64(time.Since(start).Milliseconds()))
}()
if err := s.lockResolver.Resolve(ctx, regionID, targetTs); err != nil {
log.Warn("event feed resolve lock fail",
zap.String("namespace", s.changefeed.Namespace),
zap.String("changefeed", s.changefeed.ID),
zap.Uint64("regionID", regionID),
zap.Error(err))
}
resolveLastRun[regionID] = time.Now()
}
gcTicker := time.NewTicker(resolveLockMinInterval * 3 / 2)
defer gcTicker.Stop()
for {
var task resolveLockTask
select {
case <-ctx.Done():
return ctx.Err()
case <-gcTicker.C:
gcResolveLastRun()
case task = <-s.resolveLockTaskCh.Out():
s.metrics.lockResolveWaitDuration.Observe(float64(time.Since(task.create).Milliseconds()))
doResolve(task.regionID, task.state, task.targetTs)
}
}
}
func (s *SharedClient) logSlowRegions(ctx context.Context) error {
ticker := time.NewTicker(30 * time.Second)
defer ticker.Stop()
for {
select {
case <-ctx.Done():
return ctx.Err()
case <-ticker.C:
}
log.Info("event feed starts to check locked regions",
zap.String("namespace", s.changefeed.Namespace),
zap.String("changefeed", s.changefeed.ID))
currTime := s.pdClock.CurrentTime()
s.totalSpans.RLock()
slowInitializeRegion := 0
for subscriptionID, rt := range s.totalSpans.v {
attr := rt.rangeLock.IterAll(nil)
ckptTime := oracle.GetTimeFromTS(attr.SlowestRegion.ResolvedTs)
if attr.SlowestRegion.Initialized {
if currTime.Sub(ckptTime) > 2*resolveLockMinInterval {
log.Info("event feed finds a initialized slow region",
zap.String("namespace", s.changefeed.Namespace),
zap.String("changefeed", s.changefeed.ID),
zap.Any("subscriptionID", subscriptionID),
zap.Any("slowRegion", attr.SlowestRegion))
}
} else if currTime.Sub(attr.SlowestRegion.Created) > 10*time.Minute {
slowInitializeRegion += 1
log.Info("event feed initializes a region too slow",
zap.String("namespace", s.changefeed.Namespace),
zap.String("changefeed", s.changefeed.ID),
zap.Any("subscriptionID", subscriptionID),
zap.Any("slowRegion", attr.SlowestRegion))
} else if currTime.Sub(ckptTime) > 10*time.Minute {
log.Info("event feed finds a uninitialized slow region",
zap.String("namespace", s.changefeed.Namespace),
zap.String("changefeed", s.changefeed.ID),
zap.Any("subscriptionID", subscriptionID),
zap.Any("slowRegion", attr.SlowestRegion))
}
if len(attr.UnLockedRanges) > 0 {
log.Info("event feed holes exist",
zap.String("namespace", s.changefeed.Namespace),
zap.String("changefeed", s.changefeed.ID),
zap.Any("subscriptionID", subscriptionID),
zap.Any("holes", attr.UnLockedRanges))
}
}
s.totalSpans.RUnlock()
s.metrics.slowInitializeRegion.Set(float64(slowInitializeRegion))
}
}
func (s *SharedClient) newSubscribedTable(
subID SubscriptionID, span tablepb.Span, startTs uint64,
eventCh chan<- MultiplexingEvent,
) *subscribedTable {
cfName := s.changefeed.String()
rangeLock := regionlock.NewRangeLock(uint64(subID), span.StartKey, span.EndKey, startTs, cfName)
rt := &subscribedTable{
subscriptionID: subID,
span: span,
startTs: startTs,
rangeLock: rangeLock,
eventCh: eventCh,
}
rt.tryResolveLock = func(regionID uint64, state *regionlock.LockedRangeState) {
targetTs := rt.staleLocksTargetTs.Load()
if state.ResolvedTs.Load() < targetTs && state.Initialzied.Load() {
s.resolveLockTaskCh.In() <- resolveLockTask{
regionID: regionID,
targetTs: targetTs,
state: state,
create: time.Now(),
}
}
}
return rt
}
func (r *subscribedTable) resolveStaleLocks(s *SharedClient, targetTs uint64) {
util.MustCompareAndMonotonicIncrease(&r.staleLocksTargetTs, targetTs)
res := r.rangeLock.IterAll(r.tryResolveLock)
log.Warn("event feed finds slow locked ranges",
zap.String("namespace", s.changefeed.Namespace),
zap.String("changefeed", s.changefeed.ID),
zap.Any("subscriptionID", r.subscriptionID),
zap.Any("ranges", res))
}
type sharedClientMetrics struct {
regionLockDuration prometheus.Observer
regionLocateDuration prometheus.Observer
regionConnectDuration prometheus.Observer
batchResolvedSize prometheus.Observer
lockResolveWaitDuration prometheus.Observer
lockResolveRunDuration prometheus.Observer
slowInitializeRegion prometheus.Gauge
}
func (s *SharedClient) initMetrics() {
eventFeedGauge.Inc()
s.metrics.regionLockDuration = regionConnectDuration.
WithLabelValues(s.changefeed.Namespace, s.changefeed.ID, "lock")
s.metrics.regionLocateDuration = regionConnectDuration.
WithLabelValues(s.changefeed.Namespace, s.changefeed.ID, "locate")
s.metrics.regionConnectDuration = regionConnectDuration.
WithLabelValues(s.changefeed.Namespace, s.changefeed.ID, "connect")
s.metrics.lockResolveWaitDuration = lockResolveDuration.
WithLabelValues(s.changefeed.Namespace, s.changefeed.ID, "wait")
s.metrics.lockResolveRunDuration = lockResolveDuration.
WithLabelValues(s.changefeed.Namespace, s.changefeed.ID, "run")
s.metrics.batchResolvedSize = batchResolvedEventSize.
WithLabelValues(s.changefeed.Namespace, s.changefeed.ID)
s.metrics.slowInitializeRegion = slowInitializeRegion.
WithLabelValues(s.changefeed.Namespace, s.changefeed.ID)
}
func (s *SharedClient) clearMetrics() {
eventFeedGauge.Dec()
regionConnectDuration.DeleteLabelValues(s.changefeed.Namespace, s.changefeed.ID, "lock")
regionConnectDuration.DeleteLabelValues(s.changefeed.Namespace, s.changefeed.ID, "locate")
regionConnectDuration.DeleteLabelValues(s.changefeed.Namespace, s.changefeed.ID, "connect")
lockResolveDuration.DeleteLabelValues(s.changefeed.Namespace, s.changefeed.ID, "wait")
lockResolveDuration.DeleteLabelValues(s.changefeed.Namespace, s.changefeed.ID, "run")
batchResolvedEventSize.DeleteLabelValues(s.changefeed.Namespace, s.changefeed.ID)
}
func hashRegionID(regionID uint64, slots int) int {
b := make([]byte, 8)
binary.LittleEndian.PutUint64(b, regionID)
return int(seahash.Sum64(b) % uint64(slots))
}