github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/kv/shared_client.go (about)

     1  // Copyright 2023 PingCAP, Inc.
     2  //
     3  // Licensed under the Apache License, Version 2.0 (the "License");
     4  // you may not use this file except in compliance with the License.
     5  // You may obtain a copy of the License at
     6  //
     7  //     http://www.apache.org/licenses/LICENSE-2.0
     8  //
     9  // Unless required by applicable law or agreed to in writing, software
    10  // distributed under the License is distributed on an "AS IS" BASIS,
    11  // See the License for the specific language governing permissions and
    12  // limitations under the License.
    13  
    14  package kv
    15  
    16  import (
    17  	"context"
    18  	"encoding/binary"
    19  	"fmt"
    20  	"sync"
    21  	"sync/atomic"
    22  	"time"
    23  
    24  	"blainsmith.com/go/seahash"
    25  	"github.com/pingcap/errors"
    26  	"github.com/pingcap/kvproto/pkg/cdcpb"
    27  	"github.com/pingcap/kvproto/pkg/kvrpcpb"
    28  	"github.com/pingcap/kvproto/pkg/metapb"
    29  	"github.com/pingcap/log"
    30  	"github.com/pingcap/tiflow/cdc/kv/regionlock"
    31  	"github.com/pingcap/tiflow/cdc/kv/sharedconn"
    32  	"github.com/pingcap/tiflow/cdc/model"
    33  	"github.com/pingcap/tiflow/cdc/processor/tablepb"
    34  	"github.com/pingcap/tiflow/pkg/chann"
    35  	"github.com/pingcap/tiflow/pkg/config"
    36  	cerror "github.com/pingcap/tiflow/pkg/errors"
    37  	"github.com/pingcap/tiflow/pkg/pdutil"
    38  	"github.com/pingcap/tiflow/pkg/spanz"
    39  	"github.com/pingcap/tiflow/pkg/txnutil"
    40  	"github.com/pingcap/tiflow/pkg/util"
    41  	"github.com/pingcap/tiflow/pkg/version"
    42  	"github.com/prometheus/client_golang/prometheus"
    43  	kvclientv2 "github.com/tikv/client-go/v2/kv"
    44  	"github.com/tikv/client-go/v2/oracle"
    45  	"github.com/tikv/client-go/v2/tikv"
    46  	pd "github.com/tikv/pd/client"
    47  	"go.uber.org/zap"
    48  	"golang.org/x/sync/errgroup"
    49  )
    50  
    51  const (
    52  	dialTimeout = 10 * time.Second
    53  	// Maximum total sleep time(in ms), 20 seconds.
    54  	tikvRequestMaxBackoff = 20000
    55  
    56  	// TiCDC may open numerous gRPC streams,
    57  	// with 65535 bytes window size, 10K streams takes about 27GB memory.
    58  	//
    59  	// 65535 bytes, the initial window size in http2 spec.
    60  	grpcInitialWindowSize = (1 << 16) - 1
    61  	// 8 MB The value for initial window size on a connection
    62  	grpcInitialConnWindowSize = 1 << 23
    63  	// 256 MB The maximum message size the client can receive
    64  	grpcMaxCallRecvMsgSize = 1 << 28
    65  
    66  	// TiCDC always interacts with region leader, every time something goes wrong,
    67  	// failed region will be reloaded via `BatchLoadRegionsWithKeyRange` API. So we
    68  	// don't need to force reload region anymore.
    69  	regionScheduleReload = false
    70  
    71  	scanRegionsConcurrency = 1024
    72  
    73  	loadRegionRetryInterval time.Duration  = 100 * time.Millisecond
    74  	resolveLockMinInterval  time.Duration  = 10 * time.Second
    75  	invalidSubscriptionID   SubscriptionID = SubscriptionID(0)
    76  )
    77  
    78  var (
    79  	// To generate an ID for a new subscription. And the subscription ID will also be used as
    80  	// `RequestId` in region requests of the table.
    81  	subscriptionIDGen atomic.Uint64
    82  	// To generate a streamID in `newStream`.
    83  	streamIDGen atomic.Uint64
    84  )
    85  
    86  var (
    87  	// unreachable error, only used in unit test
    88  	errUnreachable = errors.New("kv client unreachable error")
    89  	logPanic       = log.Panic
    90  )
    91  
    92  var (
    93  	metricFeedNotLeaderCounter        = eventFeedErrorCounter.WithLabelValues("NotLeader")
    94  	metricFeedEpochNotMatchCounter    = eventFeedErrorCounter.WithLabelValues("EpochNotMatch")
    95  	metricFeedRegionNotFoundCounter   = eventFeedErrorCounter.WithLabelValues("RegionNotFound")
    96  	metricFeedDuplicateRequestCounter = eventFeedErrorCounter.WithLabelValues("DuplicateRequest")
    97  	metricFeedUnknownErrorCounter     = eventFeedErrorCounter.WithLabelValues("Unknown")
    98  	metricFeedRPCCtxUnavailable       = eventFeedErrorCounter.WithLabelValues("RPCCtxUnavailable")
    99  	metricStoreSendRequestErr         = eventFeedErrorCounter.WithLabelValues("SendRequestToStore")
   100  	metricKvIsBusyCounter             = eventFeedErrorCounter.WithLabelValues("KvIsBusy")
   101  )
   102  
   103  type eventError struct {
   104  	err *cdcpb.Error
   105  }
   106  
   107  // Error implement error interface.
   108  func (e *eventError) Error() string {
   109  	return e.err.String()
   110  }
   111  
   112  type rpcCtxUnavailableErr struct {
   113  	verID tikv.RegionVerID
   114  }
   115  
   116  func (e *rpcCtxUnavailableErr) Error() string {
   117  	return fmt.Sprintf("cannot get rpcCtx for region %v. ver:%v, confver:%v",
   118  		e.verID.GetID(), e.verID.GetVer(), e.verID.GetConfVer())
   119  }
   120  
   121  type sendRequestToStoreErr struct{}
   122  
   123  func (e *sendRequestToStoreErr) Error() string { return "send request to store error" }
   124  
   125  // SubscriptionID comes from `SharedClient.AllocSubscriptionID`.
   126  type SubscriptionID uint64
   127  
   128  // MultiplexingEvent wrap a region event with
   129  // SubscriptionID to indicate which subscription it belongs to.
   130  type MultiplexingEvent struct {
   131  	model.RegionFeedEvent
   132  	SubscriptionID SubscriptionID
   133  	Start          time.Time
   134  }
   135  
   136  // newMultiplexingEvent creates a new MultiplexingEvent.
   137  func newMultiplexingEvent(e model.RegionFeedEvent, table *subscribedTable) MultiplexingEvent {
   138  	return MultiplexingEvent{
   139  		RegionFeedEvent: e,
   140  		SubscriptionID:  table.subscriptionID,
   141  		Start:           time.Now(),
   142  	}
   143  }
   144  
   145  // SharedClient is shared by many tables to pull events from TiKV.
   146  // All exported Methods are thread-safe.
   147  type SharedClient struct {
   148  	changefeed model.ChangeFeedID
   149  	config     *config.ServerConfig
   150  	metrics    sharedClientMetrics
   151  
   152  	clusterID  uint64
   153  	filterLoop bool
   154  
   155  	pd           pd.Client
   156  	grpcPool     *sharedconn.ConnAndClientPool
   157  	regionCache  *tikv.RegionCache
   158  	pdClock      pdutil.Clock
   159  	lockResolver txnutil.LockResolver
   160  
   161  	totalSpans struct {
   162  		sync.RWMutex
   163  		v map[SubscriptionID]*subscribedTable
   164  	}
   165  
   166  	workers []*sharedRegionWorker
   167  	// Note: stores is only motified in handleRegion goroutine,
   168  	// so it is not protected by a lock.
   169  	stores map[string]*requestedStore
   170  
   171  	// rangeTaskCh is used to receive range tasks.
   172  	// The tasks will be handled in `handleRangeTask` goroutine.
   173  	rangeTaskCh *chann.DrainableChann[rangeTask]
   174  	// regionCh is used to receive region tasks have been locked in rangeLock.
   175  	// The region will be handled in `handleRegions` goroutine.
   176  	regionCh *chann.DrainableChann[regionInfo]
   177  	// resolveLockTaskCh is used to receive resolve lock tasks.
   178  	// The tasks will be handled in `handleResolveLockTasks` goroutine.
   179  	resolveLockTaskCh *chann.DrainableChann[resolveLockTask]
   180  	errCh             *chann.DrainableChann[regionErrorInfo]
   181  
   182  	logRegionDetails func(msg string, fields ...zap.Field)
   183  }
   184  
   185  type resolveLockTask struct {
   186  	regionID uint64
   187  	targetTs uint64
   188  	state    *regionlock.LockedRangeState
   189  	create   time.Time
   190  }
   191  
   192  // rangeTask represents a task to subscribe a range span of a table.
   193  // It can be a part of a table or a whole table, it also can be a part of a region.
   194  type rangeTask struct {
   195  	span            tablepb.Span
   196  	subscribedTable *subscribedTable
   197  }
   198  
   199  // requestedStore represents a store that has been connected.
   200  // A store may have multiple streams.
   201  type requestedStore struct {
   202  	storeID   uint64
   203  	storeAddr string
   204  	// Use to select a stream to send request.
   205  	nextStream atomic.Uint32
   206  	streams    []*requestedStream
   207  }
   208  
   209  func (rs *requestedStore) getStream() *requestedStream {
   210  	index := rs.nextStream.Add(1) % uint32(len(rs.streams))
   211  	return rs.streams[index]
   212  }
   213  
   214  // subscribedTable represents a table to subscribe.
   215  // It contains the span of the table, the startTs of the table, and the output event channel.
   216  type subscribedTable struct {
   217  	subscriptionID SubscriptionID
   218  	startTs        model.Ts
   219  
   220  	// The whole span of the table.
   221  	span tablepb.Span
   222  	// The range lock of the table,
   223  	// it is used to prevent duplicate requests to the same region range,
   224  	// and it also used to calculate this table's resolvedTs.
   225  	rangeLock *regionlock.RangeLock
   226  	// The output event channel of the table.
   227  	eventCh chan<- MultiplexingEvent
   228  
   229  	// To handle table removing.
   230  	stopped atomic.Bool
   231  
   232  	// To handle stale lock resolvings.
   233  	tryResolveLock     func(regionID uint64, state *regionlock.LockedRangeState)
   234  	staleLocksTargetTs atomic.Uint64
   235  
   236  	lastAdvanceTime atomic.Int64
   237  }
   238  
   239  // NewSharedClient creates a client.
   240  func NewSharedClient(
   241  	changefeed model.ChangeFeedID,
   242  	cfg *config.ServerConfig,
   243  	filterLoop bool,
   244  	pd pd.Client,
   245  	grpcPool *sharedconn.ConnAndClientPool,
   246  	regionCache *tikv.RegionCache,
   247  	pdClock pdutil.Clock,
   248  	lockResolver txnutil.LockResolver,
   249  ) *SharedClient {
   250  	s := &SharedClient{
   251  		changefeed: changefeed,
   252  		config:     cfg,
   253  		clusterID:  0,
   254  		filterLoop: filterLoop,
   255  
   256  		pd:           pd,
   257  		grpcPool:     grpcPool,
   258  		regionCache:  regionCache,
   259  		pdClock:      pdClock,
   260  		lockResolver: lockResolver,
   261  
   262  		rangeTaskCh:       chann.NewAutoDrainChann[rangeTask](),
   263  		regionCh:          chann.NewAutoDrainChann[regionInfo](),
   264  		resolveLockTaskCh: chann.NewAutoDrainChann[resolveLockTask](),
   265  		errCh:             chann.NewAutoDrainChann[regionErrorInfo](),
   266  
   267  		stores: make(map[string]*requestedStore),
   268  	}
   269  	s.totalSpans.v = make(map[SubscriptionID]*subscribedTable)
   270  	if cfg.Debug.Puller.LogRegionDetails {
   271  		s.logRegionDetails = log.Info
   272  	} else {
   273  		s.logRegionDetails = log.Debug
   274  	}
   275  
   276  	s.initMetrics()
   277  	return s
   278  }
   279  
   280  // AllocSubscriptionID gets an ID can be used in `Subscribe`.
   281  func (s *SharedClient) AllocSubscriptionID() SubscriptionID {
   282  	return SubscriptionID(subscriptionIDGen.Add(1))
   283  }
   284  
   285  // Subscribe the given table span.
   286  // NOTE: `span.TableID` must be set correctly.
   287  // It new a subscribedTable and store it in `s.totalSpans`,
   288  // and send a rangeTask to `s.rangeTaskCh`.
   289  // The rangeTask will be handled in `handleRangeTasks` goroutine.
   290  func (s *SharedClient) Subscribe(subID SubscriptionID, span tablepb.Span, startTs uint64, eventCh chan<- MultiplexingEvent) {
   291  	if span.TableID == 0 {
   292  		log.Panic("event feed subscribe with zero tablepb.Span.TableID",
   293  			zap.String("namespace", s.changefeed.Namespace),
   294  			zap.String("changefeed", s.changefeed.ID))
   295  	}
   296  
   297  	rt := s.newSubscribedTable(subID, span, startTs, eventCh)
   298  	s.totalSpans.Lock()
   299  	s.totalSpans.v[subID] = rt
   300  	s.totalSpans.Unlock()
   301  
   302  	s.rangeTaskCh.In() <- rangeTask{span: span, subscribedTable: rt}
   303  	log.Info("event feed subscribes table success",
   304  		zap.String("namespace", s.changefeed.Namespace),
   305  		zap.String("changefeed", s.changefeed.ID),
   306  		zap.Any("subscriptionID", rt.subscriptionID),
   307  		zap.String("span", rt.span.String()))
   308  }
   309  
   310  // Unsubscribe the given table span. All covered regions will be deregistered asynchronously.
   311  // NOTE: `span.TableID` must be set correctly.
   312  func (s *SharedClient) Unsubscribe(subID SubscriptionID) {
   313  	// NOTE: `subID` is cleared from `s.totalSpans` in `onTableDrained`.
   314  	s.totalSpans.Lock()
   315  	rt := s.totalSpans.v[subID]
   316  	s.totalSpans.Unlock()
   317  	if rt != nil {
   318  		s.setTableStopped(rt)
   319  	}
   320  
   321  	log.Info("event feed unsubscribes table",
   322  		zap.String("namespace", s.changefeed.Namespace),
   323  		zap.String("changefeed", s.changefeed.ID),
   324  		zap.Any("subscriptionID", rt.subscriptionID),
   325  		zap.Bool("exists", rt != nil))
   326  }
   327  
   328  // ResolveLock is a function. If outsider subscribers find a span resolved timestamp is
   329  // advanced slowly or stopped, they can try to resolve locks in the given span.
   330  func (s *SharedClient) ResolveLock(subID SubscriptionID, targetTs uint64) {
   331  	s.totalSpans.Lock()
   332  	rt := s.totalSpans.v[subID]
   333  	s.totalSpans.Unlock()
   334  	if rt != nil {
   335  		rt.resolveStaleLocks(s, targetTs)
   336  	}
   337  }
   338  
   339  // RegionCount returns subscribed region count for the span.
   340  func (s *SharedClient) RegionCount(subID SubscriptionID) uint64 {
   341  	s.totalSpans.RLock()
   342  	defer s.totalSpans.RUnlock()
   343  	if rt := s.totalSpans.v[subID]; rt != nil {
   344  		return uint64(rt.rangeLock.Len())
   345  	}
   346  	return 0
   347  }
   348  
   349  // Run the client.
   350  func (s *SharedClient) Run(ctx context.Context) error {
   351  	s.clusterID = s.pd.GetClusterID(ctx)
   352  
   353  	g, ctx := errgroup.WithContext(ctx)
   354  	s.workers = make([]*sharedRegionWorker, 0, s.config.KVClient.WorkerConcurrent)
   355  	for i := uint(0); i < s.config.KVClient.WorkerConcurrent; i++ {
   356  		worker := newSharedRegionWorker(s)
   357  		g.Go(func() error { return worker.run(ctx) })
   358  		s.workers = append(s.workers, worker)
   359  	}
   360  
   361  	g.Go(func() error { return s.handleRangeTasks(ctx) })
   362  	g.Go(func() error { return s.handleRegions(ctx, g) })
   363  	g.Go(func() error { return s.handleErrors(ctx) })
   364  	g.Go(func() error { return s.handleResolveLockTasks(ctx) })
   365  	g.Go(func() error { return s.logSlowRegions(ctx) })
   366  
   367  	log.Info("event feed started",
   368  		zap.String("namespace", s.changefeed.Namespace),
   369  		zap.String("changefeed", s.changefeed.ID))
   370  	defer log.Info("event feed exits",
   371  		zap.String("namespace", s.changefeed.Namespace),
   372  		zap.String("changefeed", s.changefeed.ID))
   373  	return g.Wait()
   374  }
   375  
   376  // Close closes the client. Must be called after `Run` returns.
   377  func (s *SharedClient) Close() {
   378  	s.rangeTaskCh.CloseAndDrain()
   379  	s.regionCh.CloseAndDrain()
   380  	s.resolveLockTaskCh.CloseAndDrain()
   381  	s.errCh.CloseAndDrain()
   382  	s.clearMetrics()
   383  
   384  	for _, rs := range s.stores {
   385  		for _, stream := range rs.streams {
   386  			stream.requests.CloseAndDrain()
   387  		}
   388  	}
   389  }
   390  
   391  func (s *SharedClient) setTableStopped(rt *subscribedTable) {
   392  	log.Info("event feed starts to stop table",
   393  		zap.String("namespace", s.changefeed.Namespace),
   394  		zap.String("changefeed", s.changefeed.ID),
   395  		zap.Any("subscriptionID", rt.subscriptionID))
   396  
   397  	// Set stopped to true so we can stop handling region events from the table.
   398  	// Then send a special singleRegionInfo to regionRouter to deregister the table
   399  	// from all TiKV instances.
   400  	if rt.stopped.CompareAndSwap(false, true) {
   401  		s.regionCh.In() <- regionInfo{subscribedTable: rt}
   402  		if rt.rangeLock.Stop() {
   403  			s.onTableDrained(rt)
   404  		}
   405  	}
   406  }
   407  
   408  func (s *SharedClient) onTableDrained(rt *subscribedTable) {
   409  	log.Info("event feed stop table is finished",
   410  		zap.String("namespace", s.changefeed.Namespace),
   411  		zap.String("changefeed", s.changefeed.ID),
   412  		zap.Any("subscriptionID", rt.subscriptionID))
   413  
   414  	s.totalSpans.Lock()
   415  	defer s.totalSpans.Unlock()
   416  	delete(s.totalSpans.v, rt.subscriptionID)
   417  }
   418  
   419  func (s *SharedClient) onRegionFail(errInfo regionErrorInfo) {
   420  	s.errCh.In() <- errInfo
   421  }
   422  
   423  // handleRegions receives regionInfo from regionCh and attch rpcCtx to them,
   424  // then send them to corresponding requestedStore.
   425  func (s *SharedClient) handleRegions(ctx context.Context, eg *errgroup.Group) error {
   426  	for {
   427  		select {
   428  		case <-ctx.Done():
   429  			return errors.Trace(ctx.Err())
   430  		case region := <-s.regionCh.Out():
   431  			if region.isStoped() {
   432  				for _, rs := range s.stores {
   433  					s.broadcastRequest(rs, region)
   434  				}
   435  				continue
   436  			}
   437  
   438  			region, ok := s.attachRPCContextForRegion(ctx, region)
   439  			// If attachRPCContextForRegion fails, the region will be re-scheduled.
   440  			if !ok {
   441  				continue
   442  			}
   443  
   444  			store := s.getStore(ctx, eg, region.rpcCtx.Peer.StoreId, region.rpcCtx.Addr)
   445  			stream := store.getStream()
   446  			stream.requests.In() <- region
   447  
   448  			s.logRegionDetails("event feed will request a region",
   449  				zap.String("namespace", s.changefeed.Namespace),
   450  				zap.String("changefeed", s.changefeed.ID),
   451  				zap.Uint64("streamID", stream.streamID),
   452  				zap.Any("subscriptionID", region.subscribedTable.subscriptionID),
   453  				zap.Uint64("regionID", region.verID.GetID()),
   454  				zap.Uint64("storeID", store.storeID),
   455  				zap.String("addr", store.storeAddr))
   456  		}
   457  	}
   458  }
   459  
   460  func (s *SharedClient) attachRPCContextForRegion(ctx context.Context, region regionInfo) (regionInfo, bool) {
   461  	bo := tikv.NewBackoffer(ctx, tikvRequestMaxBackoff)
   462  	rpcCtx, err := s.regionCache.GetTiKVRPCContext(bo, region.verID, kvclientv2.ReplicaReadLeader, 0)
   463  	if rpcCtx != nil {
   464  		region.rpcCtx = rpcCtx
   465  		locateTime := time.Since(region.lockedRangeState.Created).Milliseconds()
   466  		s.metrics.regionLocateDuration.Observe(float64(locateTime))
   467  		return region, true
   468  	}
   469  	if err != nil {
   470  		log.Debug("event feed get RPC context fail",
   471  			zap.String("namespace", s.changefeed.Namespace),
   472  			zap.String("changefeed", s.changefeed.ID),
   473  			zap.Any("subscriptionID", region.subscribedTable.subscriptionID),
   474  			zap.Uint64("regionID", region.verID.GetID()),
   475  			zap.Error(err))
   476  	}
   477  	s.onRegionFail(newRegionErrorInfo(region, &rpcCtxUnavailableErr{verID: region.verID}))
   478  	return region, false
   479  }
   480  
   481  // getStore gets a requestedStore from requestedStores by storeAddr.
   482  func (s *SharedClient) getStore(
   483  	ctx context.Context, g *errgroup.Group,
   484  	storeID uint64, storeAddr string,
   485  ) *requestedStore {
   486  	var rs *requestedStore
   487  	if rs = s.stores[storeAddr]; rs != nil {
   488  		return rs
   489  	}
   490  	rs = &requestedStore{storeID: storeID, storeAddr: storeAddr}
   491  	s.stores[storeAddr] = rs
   492  	for i := uint(0); i < s.config.KVClient.GrpcStreamConcurrent; i++ {
   493  		stream := newStream(ctx, s, g, rs)
   494  		rs.streams = append(rs.streams, stream)
   495  	}
   496  
   497  	return rs
   498  }
   499  
   500  func (s *SharedClient) createRegionRequest(region regionInfo) *cdcpb.ChangeDataRequest {
   501  	return &cdcpb.ChangeDataRequest{
   502  		Header:       &cdcpb.Header{ClusterId: s.clusterID, TicdcVersion: version.ReleaseSemver()},
   503  		RegionId:     region.verID.GetID(),
   504  		RequestId:    uint64(region.subscribedTable.subscriptionID),
   505  		RegionEpoch:  region.rpcCtx.Meta.RegionEpoch,
   506  		CheckpointTs: region.resolvedTs(),
   507  		StartKey:     region.span.StartKey,
   508  		EndKey:       region.span.EndKey,
   509  		ExtraOp:      kvrpcpb.ExtraOp_ReadOldValue,
   510  		FilterLoop:   s.filterLoop,
   511  	}
   512  }
   513  
   514  func (s *SharedClient) broadcastRequest(r *requestedStore, region regionInfo) {
   515  	for _, stream := range r.streams {
   516  		stream.requests.In() <- region
   517  	}
   518  }
   519  
   520  func (s *SharedClient) handleRangeTasks(ctx context.Context) error {
   521  	g, ctx := errgroup.WithContext(ctx)
   522  	g.SetLimit(scanRegionsConcurrency)
   523  	for {
   524  		select {
   525  		case <-ctx.Done():
   526  			return ctx.Err()
   527  		case task := <-s.rangeTaskCh.Out():
   528  			g.Go(func() error { return s.divideSpanAndScheduleRegionRequests(ctx, task.span, task.subscribedTable) })
   529  		}
   530  	}
   531  }
   532  
   533  // divideSpanAndScheduleRegionRequests processes the specified span by dividing it into
   534  // manageable regions and schedules requests to subscribe to these regions.
   535  // 1. Load regions from PD.
   536  // 2. Find the intersection of each region.span and the subscribedTable.span.
   537  // 3. Schedule a region request to subscribe the region.
   538  func (s *SharedClient) divideSpanAndScheduleRegionRequests(
   539  	ctx context.Context,
   540  	span tablepb.Span,
   541  	subscribedTable *subscribedTable,
   542  ) error {
   543  	// Limit the number of regions loaded at a time to make the load more stable.
   544  	limit := 1024
   545  	nextSpan := span
   546  	backoffBeforeLoad := false
   547  	for {
   548  		if backoffBeforeLoad {
   549  			if err := util.Hang(ctx, loadRegionRetryInterval); err != nil {
   550  				return err
   551  			}
   552  			backoffBeforeLoad = false
   553  		}
   554  		log.Debug("event feed is going to load regions",
   555  			zap.String("namespace", s.changefeed.Namespace),
   556  			zap.String("changefeed", s.changefeed.ID),
   557  			zap.Any("subscriptionID", subscribedTable.subscriptionID),
   558  			zap.Any("span", nextSpan))
   559  
   560  		backoff := tikv.NewBackoffer(ctx, tikvRequestMaxBackoff)
   561  		regions, err := s.regionCache.BatchLoadRegionsWithKeyRange(backoff, nextSpan.StartKey, nextSpan.EndKey, limit)
   562  		if err != nil {
   563  			log.Warn("event feed load regions failed",
   564  				zap.String("namespace", s.changefeed.Namespace),
   565  				zap.String("changefeed", s.changefeed.ID),
   566  				zap.Any("subscriptionID", subscribedTable.subscriptionID),
   567  				zap.Any("span", nextSpan),
   568  				zap.Error(err))
   569  			backoffBeforeLoad = true
   570  			continue
   571  		}
   572  
   573  		regionMetas := make([]*metapb.Region, 0, len(regions))
   574  		for _, region := range regions {
   575  			if meta := region.GetMeta(); meta != nil {
   576  				regionMetas = append(regionMetas, meta)
   577  			}
   578  		}
   579  		regionMetas = regionlock.CutRegionsLeftCoverSpan(regionMetas, nextSpan)
   580  		if len(regionMetas) == 0 {
   581  			log.Warn("event feed load regions with holes",
   582  				zap.String("namespace", s.changefeed.Namespace),
   583  				zap.String("changefeed", s.changefeed.ID),
   584  				zap.Any("subscriptionID", subscribedTable.subscriptionID),
   585  				zap.Any("span", nextSpan))
   586  			backoffBeforeLoad = true
   587  			continue
   588  		}
   589  
   590  		for _, regionMeta := range regionMetas {
   591  			regionSpan := tablepb.Span{StartKey: regionMeta.StartKey, EndKey: regionMeta.EndKey}
   592  			// NOTE: the End key return by the PD API will be nil to represent the biggest key.
   593  			// So we need to fix it by calling spanz.HackSpan.
   594  			regionSpan = spanz.HackSpan(regionSpan)
   595  
   596  			// Find the intersection of the regionSpan returned by PD and the subscribedTable.span.
   597  			// The intersection is the span that needs to be subscribed.
   598  			intersectantSpan, err := spanz.Intersect(subscribedTable.span, regionSpan)
   599  			if err != nil {
   600  				log.Panic("event feed check spans intersect shouldn't fail",
   601  					zap.String("namespace", s.changefeed.Namespace),
   602  					zap.String("changefeed", s.changefeed.ID),
   603  					zap.Any("subscriptionID", subscribedTable.subscriptionID))
   604  			}
   605  
   606  			verID := tikv.NewRegionVerID(regionMeta.Id, regionMeta.RegionEpoch.ConfVer, regionMeta.RegionEpoch.Version)
   607  			regionInfo := newRegionInfo(verID, intersectantSpan, nil, subscribedTable)
   608  
   609  			// Schedule a region request to subscribe the region.
   610  			s.scheduleRegionRequest(ctx, regionInfo)
   611  
   612  			nextSpan.StartKey = regionMeta.EndKey
   613  			// If the nextSpan.StartKey is larger than the subscribedTable.span.EndKey,
   614  			// it means all span of the subscribedTable have been requested. So we return.
   615  			if spanz.EndCompare(nextSpan.StartKey, span.EndKey) >= 0 {
   616  				return nil
   617  			}
   618  		}
   619  	}
   620  }
   621  
   622  // scheduleRegionRequest locks the region's range and send the region to regionCh,
   623  // which will be handled by handleRegions.
   624  func (s *SharedClient) scheduleRegionRequest(ctx context.Context, region regionInfo) {
   625  	lockRangeResult := region.subscribedTable.rangeLock.LockRange(
   626  		ctx, region.span.StartKey, region.span.EndKey, region.verID.GetID(), region.verID.GetVer())
   627  
   628  	if lockRangeResult.Status == regionlock.LockRangeStatusWait {
   629  		lockRangeResult = lockRangeResult.WaitFn()
   630  	}
   631  
   632  	switch lockRangeResult.Status {
   633  	case regionlock.LockRangeStatusSuccess:
   634  		region.lockedRangeState = lockRangeResult.LockedRangeState
   635  		lockTime := time.Since(region.lockedRangeState.Created).Milliseconds()
   636  		s.metrics.regionLockDuration.Observe(float64(lockTime))
   637  		select {
   638  		case s.regionCh.In() <- region:
   639  		case <-ctx.Done():
   640  		}
   641  	case regionlock.LockRangeStatusStale:
   642  		for _, r := range lockRangeResult.RetryRanges {
   643  			s.scheduleRangeRequest(ctx, r, region.subscribedTable)
   644  		}
   645  	default:
   646  		return
   647  	}
   648  }
   649  
   650  func (s *SharedClient) scheduleRangeRequest(
   651  	ctx context.Context, span tablepb.Span,
   652  	subscribedTable *subscribedTable,
   653  ) {
   654  	select {
   655  	case s.rangeTaskCh.In() <- rangeTask{span: span, subscribedTable: subscribedTable}:
   656  	case <-ctx.Done():
   657  	}
   658  }
   659  
   660  func (s *SharedClient) handleErrors(ctx context.Context) error {
   661  	for {
   662  		select {
   663  		case <-ctx.Done():
   664  			return ctx.Err()
   665  		case errInfo := <-s.errCh.Out():
   666  			if err := s.doHandleError(ctx, errInfo); err != nil {
   667  				return err
   668  			}
   669  		}
   670  	}
   671  }
   672  
   673  func (s *SharedClient) doHandleError(ctx context.Context, errInfo regionErrorInfo) error {
   674  	if errInfo.subscribedTable.rangeLock.UnlockRange(
   675  		errInfo.span.StartKey, errInfo.span.EndKey,
   676  		errInfo.verID.GetID(), errInfo.verID.GetVer(), errInfo.resolvedTs()) {
   677  		s.onTableDrained(errInfo.subscribedTable)
   678  		return nil
   679  	}
   680  
   681  	err := errors.Cause(errInfo.err)
   682  	switch eerr := err.(type) {
   683  	case *eventError:
   684  		innerErr := eerr.err
   685  		s.logRegionDetails("cdc region error",
   686  			zap.String("namespace", s.changefeed.Namespace),
   687  			zap.String("changefeed", s.changefeed.ID),
   688  			zap.Any("subscriptionID", errInfo.subscribedTable.subscriptionID),
   689  			zap.Stringer("error", innerErr))
   690  
   691  		if notLeader := innerErr.GetNotLeader(); notLeader != nil {
   692  			metricFeedNotLeaderCounter.Inc()
   693  			s.regionCache.UpdateLeader(errInfo.verID, notLeader.GetLeader(), errInfo.rpcCtx.AccessIdx)
   694  			s.scheduleRegionRequest(ctx, errInfo.regionInfo)
   695  			return nil
   696  		}
   697  		if innerErr.GetEpochNotMatch() != nil {
   698  			metricFeedEpochNotMatchCounter.Inc()
   699  			s.scheduleRangeRequest(ctx, errInfo.span, errInfo.subscribedTable)
   700  			return nil
   701  		}
   702  		if innerErr.GetRegionNotFound() != nil {
   703  			metricFeedRegionNotFoundCounter.Inc()
   704  			s.scheduleRangeRequest(ctx, errInfo.span, errInfo.subscribedTable)
   705  			return nil
   706  		}
   707  		if innerErr.GetServerIsBusy() != nil {
   708  			metricKvIsBusyCounter.Inc()
   709  			s.scheduleRegionRequest(ctx, errInfo.regionInfo)
   710  			return nil
   711  		}
   712  		if duplicated := innerErr.GetDuplicateRequest(); duplicated != nil {
   713  			metricFeedDuplicateRequestCounter.Inc()
   714  			// TODO(qupeng): It's better to add a new machanism to deregister one region.
   715  			return errors.New("duplicate request")
   716  		}
   717  		if compatibility := innerErr.GetCompatibility(); compatibility != nil {
   718  			return cerror.ErrVersionIncompatible.GenWithStackByArgs(compatibility)
   719  		}
   720  		if mismatch := innerErr.GetClusterIdMismatch(); mismatch != nil {
   721  			return cerror.ErrClusterIDMismatch.GenWithStackByArgs(mismatch.Current, mismatch.Request)
   722  		}
   723  
   724  		log.Warn("empty or unknown cdc error",
   725  			zap.String("namespace", s.changefeed.Namespace),
   726  			zap.String("changefeed", s.changefeed.ID),
   727  			zap.Any("subscriptionID", errInfo.subscribedTable.subscriptionID),
   728  			zap.Stringer("error", innerErr))
   729  		metricFeedUnknownErrorCounter.Inc()
   730  		s.scheduleRegionRequest(ctx, errInfo.regionInfo)
   731  		return nil
   732  	case *rpcCtxUnavailableErr:
   733  		metricFeedRPCCtxUnavailable.Inc()
   734  		s.scheduleRangeRequest(ctx, errInfo.span, errInfo.subscribedTable)
   735  		return nil
   736  	case *sendRequestToStoreErr:
   737  		metricStoreSendRequestErr.Inc()
   738  		bo := tikv.NewBackoffer(ctx, tikvRequestMaxBackoff)
   739  		s.regionCache.OnSendFail(bo, errInfo.rpcCtx, regionScheduleReload, err)
   740  		s.scheduleRegionRequest(ctx, errInfo.regionInfo)
   741  		return nil
   742  	default:
   743  		// TODO(qupeng): for some errors it's better to just deregister the region from TiKVs.
   744  		log.Warn("event feed meets an internal error, fail the changefeed",
   745  			zap.String("namespace", s.changefeed.Namespace),
   746  			zap.String("changefeed", s.changefeed.ID),
   747  			zap.Any("subscriptionID", errInfo.subscribedTable.subscriptionID),
   748  			zap.Error(err))
   749  		return err
   750  	}
   751  }
   752  
   753  func (s *SharedClient) handleResolveLockTasks(ctx context.Context) error {
   754  	resolveLastRun := make(map[uint64]time.Time)
   755  
   756  	gcResolveLastRun := func() {
   757  		if len(resolveLastRun) > 1024 {
   758  			copied := make(map[uint64]time.Time)
   759  			now := time.Now()
   760  			for regionID, lastRun := range resolveLastRun {
   761  				if now.Sub(lastRun) < resolveLockMinInterval {
   762  					resolveLastRun[regionID] = lastRun
   763  				}
   764  			}
   765  			resolveLastRun = copied
   766  		}
   767  	}
   768  
   769  	doResolve := func(regionID uint64, state *regionlock.LockedRangeState, targetTs uint64) {
   770  		if state.ResolvedTs.Load() > targetTs || !state.Initialzied.Load() {
   771  			return
   772  		}
   773  		if lastRun, ok := resolveLastRun[regionID]; ok {
   774  			if time.Since(lastRun) < resolveLockMinInterval {
   775  				return
   776  			}
   777  		}
   778  		start := time.Now()
   779  		defer func() {
   780  			s.metrics.lockResolveRunDuration.Observe(float64(time.Since(start).Milliseconds()))
   781  		}()
   782  
   783  		if err := s.lockResolver.Resolve(ctx, regionID, targetTs); err != nil {
   784  			log.Warn("event feed resolve lock fail",
   785  				zap.String("namespace", s.changefeed.Namespace),
   786  				zap.String("changefeed", s.changefeed.ID),
   787  				zap.Uint64("regionID", regionID),
   788  				zap.Error(err))
   789  		}
   790  		resolveLastRun[regionID] = time.Now()
   791  	}
   792  
   793  	gcTicker := time.NewTicker(resolveLockMinInterval * 3 / 2)
   794  	defer gcTicker.Stop()
   795  	for {
   796  		var task resolveLockTask
   797  		select {
   798  		case <-ctx.Done():
   799  			return ctx.Err()
   800  		case <-gcTicker.C:
   801  			gcResolveLastRun()
   802  		case task = <-s.resolveLockTaskCh.Out():
   803  			s.metrics.lockResolveWaitDuration.Observe(float64(time.Since(task.create).Milliseconds()))
   804  			doResolve(task.regionID, task.state, task.targetTs)
   805  		}
   806  	}
   807  }
   808  
   809  func (s *SharedClient) logSlowRegions(ctx context.Context) error {
   810  	ticker := time.NewTicker(30 * time.Second)
   811  	defer ticker.Stop()
   812  	for {
   813  		select {
   814  		case <-ctx.Done():
   815  			return ctx.Err()
   816  		case <-ticker.C:
   817  		}
   818  		log.Info("event feed starts to check locked regions",
   819  			zap.String("namespace", s.changefeed.Namespace),
   820  			zap.String("changefeed", s.changefeed.ID))
   821  
   822  		currTime := s.pdClock.CurrentTime()
   823  		s.totalSpans.RLock()
   824  		slowInitializeRegion := 0
   825  		for subscriptionID, rt := range s.totalSpans.v {
   826  			attr := rt.rangeLock.IterAll(nil)
   827  			ckptTime := oracle.GetTimeFromTS(attr.SlowestRegion.ResolvedTs)
   828  			if attr.SlowestRegion.Initialized {
   829  				if currTime.Sub(ckptTime) > 2*resolveLockMinInterval {
   830  					log.Info("event feed finds a initialized slow region",
   831  						zap.String("namespace", s.changefeed.Namespace),
   832  						zap.String("changefeed", s.changefeed.ID),
   833  						zap.Any("subscriptionID", subscriptionID),
   834  						zap.Any("slowRegion", attr.SlowestRegion))
   835  				}
   836  			} else if currTime.Sub(attr.SlowestRegion.Created) > 10*time.Minute {
   837  				slowInitializeRegion += 1
   838  				log.Info("event feed initializes a region too slow",
   839  					zap.String("namespace", s.changefeed.Namespace),
   840  					zap.String("changefeed", s.changefeed.ID),
   841  					zap.Any("subscriptionID", subscriptionID),
   842  					zap.Any("slowRegion", attr.SlowestRegion))
   843  			} else if currTime.Sub(ckptTime) > 10*time.Minute {
   844  				log.Info("event feed finds a uninitialized slow region",
   845  					zap.String("namespace", s.changefeed.Namespace),
   846  					zap.String("changefeed", s.changefeed.ID),
   847  					zap.Any("subscriptionID", subscriptionID),
   848  					zap.Any("slowRegion", attr.SlowestRegion))
   849  			}
   850  			if len(attr.UnLockedRanges) > 0 {
   851  				log.Info("event feed holes exist",
   852  					zap.String("namespace", s.changefeed.Namespace),
   853  					zap.String("changefeed", s.changefeed.ID),
   854  					zap.Any("subscriptionID", subscriptionID),
   855  					zap.Any("holes", attr.UnLockedRanges))
   856  			}
   857  		}
   858  		s.totalSpans.RUnlock()
   859  		s.metrics.slowInitializeRegion.Set(float64(slowInitializeRegion))
   860  	}
   861  }
   862  
   863  func (s *SharedClient) newSubscribedTable(
   864  	subID SubscriptionID, span tablepb.Span, startTs uint64,
   865  	eventCh chan<- MultiplexingEvent,
   866  ) *subscribedTable {
   867  	cfName := s.changefeed.String()
   868  	rangeLock := regionlock.NewRangeLock(uint64(subID), span.StartKey, span.EndKey, startTs, cfName)
   869  
   870  	rt := &subscribedTable{
   871  		subscriptionID: subID,
   872  		span:           span,
   873  		startTs:        startTs,
   874  		rangeLock:      rangeLock,
   875  		eventCh:        eventCh,
   876  	}
   877  
   878  	rt.tryResolveLock = func(regionID uint64, state *regionlock.LockedRangeState) {
   879  		targetTs := rt.staleLocksTargetTs.Load()
   880  		if state.ResolvedTs.Load() < targetTs && state.Initialzied.Load() {
   881  			s.resolveLockTaskCh.In() <- resolveLockTask{
   882  				regionID: regionID,
   883  				targetTs: targetTs,
   884  				state:    state,
   885  				create:   time.Now(),
   886  			}
   887  		}
   888  	}
   889  	return rt
   890  }
   891  
   892  func (r *subscribedTable) resolveStaleLocks(s *SharedClient, targetTs uint64) {
   893  	util.MustCompareAndMonotonicIncrease(&r.staleLocksTargetTs, targetTs)
   894  	res := r.rangeLock.IterAll(r.tryResolveLock)
   895  	s.logRegionDetails("event feed finds slow locked ranges",
   896  		zap.String("namespace", s.changefeed.Namespace),
   897  		zap.String("changefeed", s.changefeed.ID),
   898  		zap.Any("subscriptionID", r.subscriptionID),
   899  		zap.Any("ranges", res))
   900  }
   901  
   902  type sharedClientMetrics struct {
   903  	regionLockDuration      prometheus.Observer
   904  	regionLocateDuration    prometheus.Observer
   905  	regionConnectDuration   prometheus.Observer
   906  	batchResolvedSize       prometheus.Observer
   907  	lockResolveWaitDuration prometheus.Observer
   908  	lockResolveRunDuration  prometheus.Observer
   909  	slowInitializeRegion    prometheus.Gauge
   910  }
   911  
   912  func (s *SharedClient) initMetrics() {
   913  	eventFeedGauge.Inc()
   914  
   915  	s.metrics.regionLockDuration = regionConnectDuration.
   916  		WithLabelValues(s.changefeed.Namespace, s.changefeed.ID, "lock")
   917  	s.metrics.regionLocateDuration = regionConnectDuration.
   918  		WithLabelValues(s.changefeed.Namespace, s.changefeed.ID, "locate")
   919  	s.metrics.regionConnectDuration = regionConnectDuration.
   920  		WithLabelValues(s.changefeed.Namespace, s.changefeed.ID, "connect")
   921  
   922  	s.metrics.lockResolveWaitDuration = lockResolveDuration.
   923  		WithLabelValues(s.changefeed.Namespace, s.changefeed.ID, "wait")
   924  	s.metrics.lockResolveRunDuration = lockResolveDuration.
   925  		WithLabelValues(s.changefeed.Namespace, s.changefeed.ID, "run")
   926  
   927  	s.metrics.batchResolvedSize = batchResolvedEventSize.
   928  		WithLabelValues(s.changefeed.Namespace, s.changefeed.ID)
   929  
   930  	s.metrics.slowInitializeRegion = slowInitializeRegion.
   931  		WithLabelValues(s.changefeed.Namespace, s.changefeed.ID)
   932  }
   933  
   934  func (s *SharedClient) clearMetrics() {
   935  	eventFeedGauge.Dec()
   936  
   937  	regionConnectDuration.DeleteLabelValues(s.changefeed.Namespace, s.changefeed.ID, "lock")
   938  	regionConnectDuration.DeleteLabelValues(s.changefeed.Namespace, s.changefeed.ID, "locate")
   939  	regionConnectDuration.DeleteLabelValues(s.changefeed.Namespace, s.changefeed.ID, "connect")
   940  
   941  	lockResolveDuration.DeleteLabelValues(s.changefeed.Namespace, s.changefeed.ID, "wait")
   942  	lockResolveDuration.DeleteLabelValues(s.changefeed.Namespace, s.changefeed.ID, "run")
   943  
   944  	batchResolvedEventSize.DeleteLabelValues(s.changefeed.Namespace, s.changefeed.ID)
   945  }
   946  
   947  func hashRegionID(regionID uint64, slots int) int {
   948  	b := make([]byte, 8)
   949  	binary.LittleEndian.PutUint64(b, regionID)
   950  	return int(seahash.Sum64(b) % uint64(slots))
   951  }