github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/kv/client.go (about)

     1  // Copyright 2020 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  	"fmt"
    19  	"io"
    20  	"math/rand"
    21  	"strconv"
    22  	"sync"
    23  	"sync/atomic"
    24  	"time"
    25  
    26  	"github.com/pingcap/errors"
    27  	"github.com/pingcap/failpoint"
    28  	"github.com/pingcap/kvproto/pkg/cdcpb"
    29  	"github.com/pingcap/kvproto/pkg/kvrpcpb"
    30  	"github.com/pingcap/kvproto/pkg/metapb"
    31  	"github.com/pingcap/log"
    32  	"github.com/pingcap/ticdc/cdc/model"
    33  	"github.com/pingcap/ticdc/pkg/config"
    34  	cerror "github.com/pingcap/ticdc/pkg/errors"
    35  	"github.com/pingcap/ticdc/pkg/regionspan"
    36  	"github.com/pingcap/ticdc/pkg/retry"
    37  	"github.com/pingcap/ticdc/pkg/txnutil"
    38  	"github.com/pingcap/ticdc/pkg/util"
    39  	"github.com/pingcap/ticdc/pkg/version"
    40  	tidbkv "github.com/pingcap/tidb/kv"
    41  	"github.com/pingcap/tidb/store/tikv"
    42  	"github.com/pingcap/tidb/store/tikv/oracle"
    43  	"github.com/prometheus/client_golang/prometheus"
    44  	pd "github.com/tikv/pd/client"
    45  	"go.uber.org/zap"
    46  	"golang.org/x/sync/errgroup"
    47  	"golang.org/x/time/rate"
    48  	"google.golang.org/grpc/codes"
    49  	"google.golang.org/grpc/status"
    50  )
    51  
    52  const (
    53  	dialTimeout           = 10 * time.Second
    54  	tikvRequestMaxBackoff = 20000 // Maximum total sleep time(in ms)
    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  	// The threshold of warning a message is too large. TiKV split events into 6MB per-message.
    67  	warnRecvMsgSizeThreshold = 12 * 1024 * 1024
    68  
    69  	// TiCDC always interacts with region leader, every time something goes wrong,
    70  	// failed region will be reloaded via `BatchLoadRegionsWithKeyRange` API. So we
    71  	// don't need to force reload region any more.
    72  	regionScheduleReload = false
    73  
    74  	// defaultRegionChanSize is the default channel size for region channel, including
    75  	// range request, region request and region error.
    76  	// Note the producer of region error channel, and the consumer of range request
    77  	// channel work in an asynchronous way, the larger channel can decrease the
    78  	// frequency of creating new goroutine.
    79  	defaultRegionChanSize = 128
    80  )
    81  
    82  // time interval to force kv client to terminate gRPC stream and reconnect
    83  var reconnectInterval = 60 * time.Minute
    84  
    85  // hard code switch
    86  // true: use kv client v2, which has a region worker for each stream
    87  // false: use kv client v1, which runs a goroutine for every single region
    88  var enableKVClientV2 = false
    89  
    90  type singleRegionInfo struct {
    91  	verID  tikv.RegionVerID
    92  	span   regionspan.ComparableSpan
    93  	ts     uint64
    94  	rpcCtx *tikv.RPCContext
    95  }
    96  
    97  var (
    98  	metricFeedNotLeaderCounter        = eventFeedErrorCounter.WithLabelValues("NotLeader")
    99  	metricFeedEpochNotMatchCounter    = eventFeedErrorCounter.WithLabelValues("EpochNotMatch")
   100  	metricFeedRegionNotFoundCounter   = eventFeedErrorCounter.WithLabelValues("RegionNotFound")
   101  	metricFeedDuplicateRequestCounter = eventFeedErrorCounter.WithLabelValues("DuplicateRequest")
   102  	metricFeedUnknownErrorCounter     = eventFeedErrorCounter.WithLabelValues("Unknown")
   103  	metricFeedRPCCtxUnavailable       = eventFeedErrorCounter.WithLabelValues("RPCCtxUnavailable")
   104  	metricStoreSendRequestErr         = eventFeedErrorCounter.WithLabelValues("SendRequestToStore")
   105  	metricConnectToStoreErr           = eventFeedErrorCounter.WithLabelValues("ConnectToStore")
   106  )
   107  
   108  var (
   109  	// unreachable error, only used in unit test
   110  	errUnreachable = errors.New("kv client unreachable error")
   111  	// internal error, force the gPRC stream terminate and reconnect
   112  	errReconnect = errors.New("internal error, reconnect all regions")
   113  	logPanic     = log.Panic
   114  )
   115  
   116  func newSingleRegionInfo(verID tikv.RegionVerID, span regionspan.ComparableSpan, ts uint64, rpcCtx *tikv.RPCContext) singleRegionInfo {
   117  	return singleRegionInfo{
   118  		verID:  verID,
   119  		span:   span,
   120  		ts:     ts,
   121  		rpcCtx: rpcCtx,
   122  	}
   123  }
   124  
   125  // partialClone clones part fields of singleRegionInfo, this is used when error
   126  // happens, kv client needs to recover region request from singleRegionInfo
   127  func (s *singleRegionInfo) partialClone() singleRegionInfo {
   128  	sri := singleRegionInfo{
   129  		verID:  s.verID,
   130  		span:   s.span.Clone(),
   131  		ts:     s.ts,
   132  		rpcCtx: &tikv.RPCContext{},
   133  	}
   134  	if s.rpcCtx != nil {
   135  		sri.rpcCtx.Addr = s.rpcCtx.Addr
   136  	}
   137  	return sri
   138  }
   139  
   140  type regionErrorInfo struct {
   141  	singleRegionInfo
   142  	err error
   143  }
   144  
   145  type regionEvent struct {
   146  	changeEvent *cdcpb.Event
   147  	resolvedTs  *cdcpb.ResolvedTs
   148  }
   149  
   150  // A special event that indicates singleEventFeed is closed.
   151  var emptyRegionEvent = regionEvent{}
   152  
   153  type regionFeedState struct {
   154  	sri           singleRegionInfo
   155  	requestID     uint64
   156  	regionEventCh chan regionEvent
   157  	stopped       int32
   158  
   159  	lock           sync.RWMutex
   160  	initialized    bool
   161  	matcher        *matcher
   162  	startFeedTime  time.Time
   163  	lastResolvedTs uint64
   164  }
   165  
   166  func newRegionFeedState(sri singleRegionInfo, requestID uint64) *regionFeedState {
   167  	return &regionFeedState{
   168  		sri:           sri,
   169  		requestID:     requestID,
   170  		regionEventCh: make(chan regionEvent, 16),
   171  		stopped:       0,
   172  	}
   173  }
   174  
   175  func (s *regionFeedState) start() {
   176  	s.startFeedTime = time.Now()
   177  	s.lastResolvedTs = s.sri.ts
   178  	s.matcher = newMatcher()
   179  }
   180  
   181  func (s *regionFeedState) markStopped() {
   182  	atomic.StoreInt32(&s.stopped, 1)
   183  }
   184  
   185  func (s *regionFeedState) isStopped() bool {
   186  	return atomic.LoadInt32(&s.stopped) > 0
   187  }
   188  
   189  func (s *regionFeedState) getLastResolvedTs() uint64 {
   190  	s.lock.RLock()
   191  	defer s.lock.RUnlock()
   192  	return s.lastResolvedTs
   193  }
   194  
   195  func (s *regionFeedState) getRegionSpan() regionspan.ComparableSpan {
   196  	s.lock.RLock()
   197  	defer s.lock.RUnlock()
   198  	return s.sri.span
   199  }
   200  
   201  type syncRegionFeedStateMap struct {
   202  	mu            *sync.Mutex
   203  	regionInfoMap map[uint64]*regionFeedState
   204  }
   205  
   206  func newSyncRegionFeedStateMap() *syncRegionFeedStateMap {
   207  	return &syncRegionFeedStateMap{
   208  		mu:            &sync.Mutex{},
   209  		regionInfoMap: make(map[uint64]*regionFeedState),
   210  	}
   211  }
   212  
   213  func (m *syncRegionFeedStateMap) insert(requestID uint64, state *regionFeedState) bool {
   214  	m.mu.Lock()
   215  	defer m.mu.Unlock()
   216  
   217  	_, ok := m.regionInfoMap[requestID]
   218  	m.regionInfoMap[requestID] = state
   219  	return ok
   220  }
   221  
   222  func (m *syncRegionFeedStateMap) take(requestID uint64) (*regionFeedState, bool) {
   223  	m.mu.Lock()
   224  	defer m.mu.Unlock()
   225  
   226  	state, ok := m.regionInfoMap[requestID]
   227  	if ok {
   228  		delete(m.regionInfoMap, requestID)
   229  	}
   230  	return state, ok
   231  }
   232  
   233  func (m *syncRegionFeedStateMap) takeAll() map[uint64]*regionFeedState {
   234  	m.mu.Lock()
   235  	defer m.mu.Unlock()
   236  
   237  	state := m.regionInfoMap
   238  	m.regionInfoMap = make(map[uint64]*regionFeedState)
   239  	return state
   240  }
   241  
   242  type regionEventFeedLimiters struct {
   243  	sync.Mutex
   244  	// TODO replace with a LRU cache.
   245  	limiters map[uint64]*rate.Limiter
   246  }
   247  
   248  var defaultRegionEventFeedLimiters *regionEventFeedLimiters = &regionEventFeedLimiters{
   249  	limiters: make(map[uint64]*rate.Limiter),
   250  }
   251  
   252  func (rl *regionEventFeedLimiters) getLimiter(regionID uint64) *rate.Limiter {
   253  	var limiter *rate.Limiter
   254  	var ok bool
   255  
   256  	rl.Lock()
   257  	limiter, ok = rl.limiters[regionID]
   258  	if !ok {
   259  		// In most cases, region replica count is 3.
   260  		replicaCount := 3
   261  		limiter = rate.NewLimiter(rate.Every(100*time.Millisecond), replicaCount)
   262  		rl.limiters[regionID] = limiter
   263  	}
   264  	rl.Unlock()
   265  	return limiter
   266  }
   267  
   268  // eventFeedStream stores an EventFeed stream and pointer to the underlying gRPC connection
   269  type eventFeedStream struct {
   270  	client cdcpb.ChangeData_EventFeedClient
   271  	conn   *sharedConn
   272  }
   273  
   274  // CDCKVClient is an interface to receives kv changed logs from TiKV
   275  type CDCKVClient interface {
   276  	EventFeed(
   277  		ctx context.Context,
   278  		span regionspan.ComparableSpan,
   279  		ts uint64,
   280  		enableOldValue bool,
   281  		lockResolver txnutil.LockResolver,
   282  		isPullerInit PullerInitialization,
   283  		eventCh chan<- model.RegionFeedEvent,
   284  	) error
   285  	Close() error
   286  }
   287  
   288  // NewCDCKVClient is the constructor of CDC KV client
   289  var NewCDCKVClient func(
   290  	ctx context.Context,
   291  	pd pd.Client,
   292  	kvStorage tikv.Storage,
   293  	grpcPool GrpcPool,
   294  ) CDCKVClient = NewCDCClient
   295  
   296  // CDCClient to get events from TiKV
   297  type CDCClient struct {
   298  	pd pd.Client
   299  
   300  	clusterID uint64
   301  
   302  	grpcPool GrpcPool
   303  
   304  	regionCache *tikv.RegionCache
   305  	kvStorage   TiKVStorage
   306  
   307  	regionLimiters *regionEventFeedLimiters
   308  }
   309  
   310  // NewCDCClient creates a CDCClient instance
   311  func NewCDCClient(ctx context.Context, pd pd.Client, kvStorage tikv.Storage, grpcPool GrpcPool) (c CDCKVClient) {
   312  	clusterID := pd.GetClusterID(ctx)
   313  
   314  	var store TiKVStorage
   315  	if kvStorage != nil {
   316  		// wrap to TiKVStorage if need.
   317  		if s, ok := kvStorage.(TiKVStorage); ok {
   318  			store = s
   319  		} else {
   320  			store = newStorageWithCurVersionCache(kvStorage, kvStorage.UUID())
   321  		}
   322  	}
   323  
   324  	c = &CDCClient{
   325  		clusterID:      clusterID,
   326  		pd:             pd,
   327  		kvStorage:      store,
   328  		grpcPool:       grpcPool,
   329  		regionCache:    tikv.NewRegionCache(pd),
   330  		regionLimiters: defaultRegionEventFeedLimiters,
   331  	}
   332  	return
   333  }
   334  
   335  // Close CDCClient
   336  func (c *CDCClient) Close() error {
   337  	c.regionCache.Close()
   338  
   339  	return nil
   340  }
   341  
   342  func (c *CDCClient) getRegionLimiter(regionID uint64) *rate.Limiter {
   343  	return c.regionLimiters.getLimiter(regionID)
   344  }
   345  
   346  func (c *CDCClient) newStream(ctx context.Context, addr string, storeID uint64) (stream *eventFeedStream, newStreamErr error) {
   347  	newStreamErr = retry.Do(ctx, func() (err error) {
   348  		var conn *sharedConn
   349  		defer func() {
   350  			if err != nil && conn != nil {
   351  				c.grpcPool.ReleaseConn(conn, addr)
   352  			}
   353  		}()
   354  		conn, err = c.grpcPool.GetConn(addr)
   355  		if err != nil {
   356  			log.Info("get connection to store failed, retry later", zap.String("addr", addr), zap.Error(err))
   357  			return
   358  		}
   359  		err = version.CheckStoreVersion(ctx, c.pd, storeID)
   360  		if err != nil {
   361  			// TODO: we don't close gPRC conn here, let it goes into TransientFailure
   362  			// state. If the store recovers, the gPRC conn can be reused. But if
   363  			// store goes away forever, the conn will be leaked, we need a better
   364  			// connection pool.
   365  			log.Error("check tikv version failed", zap.Error(err), zap.Uint64("storeID", storeID))
   366  			return
   367  		}
   368  		client := cdcpb.NewChangeDataClient(conn.ClientConn)
   369  		var streamClient cdcpb.ChangeData_EventFeedClient
   370  		streamClient, err = client.EventFeed(ctx)
   371  		if err != nil {
   372  			// TODO: we don't close gPRC conn here, let it goes into TransientFailure
   373  			// state. If the store recovers, the gPRC conn can be reused. But if
   374  			// store goes away forever, the conn will be leaked, we need a better
   375  			// connection pool.
   376  			err = cerror.WrapError(cerror.ErrTiKVEventFeed, err)
   377  			log.Info("establish stream to store failed, retry later", zap.String("addr", addr), zap.Error(err))
   378  			return
   379  		}
   380  		stream = &eventFeedStream{
   381  			client: streamClient,
   382  			conn:   conn,
   383  		}
   384  		log.Debug("created stream to store", zap.String("addr", addr))
   385  		return nil
   386  	}, retry.WithBackoffBaseDelay(500), retry.WithMaxTries(8), retry.WithIsRetryableErr(cerror.IsRetryableError))
   387  	return
   388  }
   389  
   390  // PullerInitialization is a workaround to solved cyclic import.
   391  type PullerInitialization interface {
   392  	IsInitialized() bool
   393  }
   394  
   395  // EventFeed divides a EventFeed request on range boundaries and establishes
   396  // a EventFeed to each of the individual region. It streams back result on the
   397  // provided channel.
   398  // The `Start` and `End` field in input span must be memcomparable encoded.
   399  func (c *CDCClient) EventFeed(
   400  	ctx context.Context, span regionspan.ComparableSpan, ts uint64,
   401  	enableOldValue bool,
   402  	lockResolver txnutil.LockResolver,
   403  	isPullerInit PullerInitialization,
   404  	eventCh chan<- model.RegionFeedEvent,
   405  ) error {
   406  	s := newEventFeedSession(ctx, c, c.regionCache, c.kvStorage, span,
   407  		lockResolver, isPullerInit,
   408  		enableOldValue, ts, eventCh)
   409  	return s.eventFeed(ctx, ts)
   410  }
   411  
   412  var currentID uint64 = 0
   413  
   414  func allocID() uint64 {
   415  	return atomic.AddUint64(&currentID, 1)
   416  }
   417  
   418  // used in test only
   419  func currentRequestID() uint64 {
   420  	return atomic.LoadUint64(&currentID)
   421  }
   422  
   423  type eventFeedSession struct {
   424  	client      *CDCClient
   425  	regionCache *tikv.RegionCache
   426  	kvStorage   TiKVStorage
   427  
   428  	lockResolver txnutil.LockResolver
   429  	isPullerInit PullerInitialization
   430  
   431  	// The whole range that is being subscribed.
   432  	totalSpan regionspan.ComparableSpan
   433  
   434  	// The channel to send the processed events.
   435  	eventCh chan<- model.RegionFeedEvent
   436  	// The token based region router, it controls the uninitialized regions with
   437  	// a given size limit.
   438  	regionRouter LimitRegionRouter
   439  	// The channel to put the region that will be sent requests.
   440  	regionCh chan singleRegionInfo
   441  	// The channel to notify that an error is happening, so that the error will be handled and the affected region
   442  	// will be re-requested.
   443  	errCh chan regionErrorInfo
   444  	// The channel to schedule scanning and requesting regions in a specified range.
   445  	requestRangeCh chan rangeRequestTask
   446  
   447  	rangeLock        *regionspan.RegionRangeLock
   448  	enableOldValue   bool
   449  	enableKVClientV2 bool
   450  
   451  	// To identify metrics of different eventFeedSession
   452  	id                string
   453  	regionChSizeGauge prometheus.Gauge
   454  	errChSizeGauge    prometheus.Gauge
   455  	rangeChSizeGauge  prometheus.Gauge
   456  
   457  	streams          map[string]*eventFeedStream
   458  	streamsLock      sync.RWMutex
   459  	streamsCanceller map[string]context.CancelFunc
   460  }
   461  
   462  type rangeRequestTask struct {
   463  	span regionspan.ComparableSpan
   464  	ts   uint64
   465  }
   466  
   467  func newEventFeedSession(
   468  	ctx context.Context,
   469  	client *CDCClient,
   470  	regionCache *tikv.RegionCache,
   471  	kvStorage TiKVStorage,
   472  	totalSpan regionspan.ComparableSpan,
   473  	lockResolver txnutil.LockResolver,
   474  	isPullerInit PullerInitialization,
   475  	enableOldValue bool,
   476  	startTs uint64,
   477  	eventCh chan<- model.RegionFeedEvent,
   478  ) *eventFeedSession {
   479  	id := strconv.FormatUint(allocID(), 10)
   480  	kvClientCfg := config.GetGlobalServerConfig().KVClient
   481  	return &eventFeedSession{
   482  		client:            client,
   483  		regionCache:       regionCache,
   484  		kvStorage:         kvStorage,
   485  		totalSpan:         totalSpan,
   486  		eventCh:           eventCh,
   487  		regionRouter:      NewSizedRegionRouter(ctx, kvClientCfg.RegionScanLimit),
   488  		regionCh:          make(chan singleRegionInfo, defaultRegionChanSize),
   489  		errCh:             make(chan regionErrorInfo, defaultRegionChanSize),
   490  		requestRangeCh:    make(chan rangeRequestTask, defaultRegionChanSize),
   491  		rangeLock:         regionspan.NewRegionRangeLock(totalSpan.Start, totalSpan.End, startTs),
   492  		enableOldValue:    enableOldValue,
   493  		enableKVClientV2:  enableKVClientV2,
   494  		lockResolver:      lockResolver,
   495  		isPullerInit:      isPullerInit,
   496  		id:                id,
   497  		regionChSizeGauge: clientChannelSize.WithLabelValues("region"),
   498  		errChSizeGauge:    clientChannelSize.WithLabelValues("err"),
   499  		rangeChSizeGauge:  clientChannelSize.WithLabelValues("range"),
   500  		streams:           make(map[string]*eventFeedStream),
   501  		streamsCanceller:  make(map[string]context.CancelFunc),
   502  	}
   503  }
   504  
   505  func (s *eventFeedSession) eventFeed(ctx context.Context, ts uint64) error {
   506  	eventFeedGauge.Inc()
   507  	defer eventFeedGauge.Dec()
   508  
   509  	log.Debug("event feed started", zap.Stringer("span", s.totalSpan), zap.Uint64("ts", ts))
   510  
   511  	g, ctx := errgroup.WithContext(ctx)
   512  
   513  	g.Go(func() error {
   514  		return s.dispatchRequest(ctx)
   515  	})
   516  
   517  	g.Go(func() error {
   518  		return s.requestRegionToStore(ctx, g)
   519  	})
   520  
   521  	g.Go(func() error {
   522  		for {
   523  			select {
   524  			case <-ctx.Done():
   525  				return ctx.Err()
   526  			case task := <-s.requestRangeCh:
   527  				s.rangeChSizeGauge.Dec()
   528  				// divideAndSendEventFeedToRegions could be block for some time,
   529  				// since it must wait for the region lock available. In order to
   530  				// consume region range request from `requestRangeCh` as soon as
   531  				// possible, we create a new goroutine to handle it.
   532  				// The sequence of region range we process is not matter, the
   533  				// region lock keeps the region access sequence.
   534  				// Besides the count or frequency of range request is limited,
   535  				// we use ephemeral goroutine instead of permanent goroutine.
   536  				g.Go(func() error {
   537  					return s.divideAndSendEventFeedToRegions(ctx, task.span, task.ts)
   538  				})
   539  			}
   540  		}
   541  	})
   542  
   543  	g.Go(func() error {
   544  		for {
   545  			select {
   546  			case <-ctx.Done():
   547  				return ctx.Err()
   548  			case errInfo := <-s.errCh:
   549  				s.errChSizeGauge.Dec()
   550  				err := s.handleError(ctx, errInfo)
   551  				if err != nil {
   552  					return err
   553  				}
   554  			}
   555  		}
   556  	})
   557  
   558  	g.Go(func() error {
   559  		return s.regionRouter.Run(ctx)
   560  	})
   561  
   562  	s.requestRangeCh <- rangeRequestTask{span: s.totalSpan, ts: ts}
   563  	s.rangeChSizeGauge.Inc()
   564  
   565  	return g.Wait()
   566  }
   567  
   568  // scheduleDivideRegionAndRequest schedules a range to be divided by regions, and these regions will be then scheduled
   569  // to send ChangeData requests.
   570  func (s *eventFeedSession) scheduleDivideRegionAndRequest(ctx context.Context, span regionspan.ComparableSpan, ts uint64) {
   571  	task := rangeRequestTask{span: span, ts: ts}
   572  	select {
   573  	case s.requestRangeCh <- task:
   574  		s.rangeChSizeGauge.Inc()
   575  	case <-ctx.Done():
   576  	}
   577  }
   578  
   579  // scheduleRegionRequest locks the region's range and schedules sending ChangeData request to the region.
   580  // This function is blocking until the region range is locked successfully
   581  func (s *eventFeedSession) scheduleRegionRequest(ctx context.Context, sri singleRegionInfo) {
   582  	handleResult := func(res regionspan.LockRangeResult) {
   583  		switch res.Status {
   584  		case regionspan.LockRangeStatusSuccess:
   585  			sri.ts = res.CheckpointTs
   586  			select {
   587  			case s.regionCh <- sri:
   588  				s.regionChSizeGauge.Inc()
   589  			case <-ctx.Done():
   590  			}
   591  		case regionspan.LockRangeStatusStale:
   592  			log.Info("request expired",
   593  				zap.Uint64("regionID", sri.verID.GetID()),
   594  				zap.Stringer("span", sri.span),
   595  				zap.Reflect("retrySpans", res.RetryRanges))
   596  			for _, r := range res.RetryRanges {
   597  				// This call is always blocking, otherwise if scheduling in a new
   598  				// goroutine, it won't block the caller of `schedulerRegionRequest`.
   599  				s.scheduleDivideRegionAndRequest(ctx, r, sri.ts)
   600  			}
   601  		case regionspan.LockRangeStatusCancel:
   602  			return
   603  		default:
   604  			panic("unreachable")
   605  		}
   606  	}
   607  
   608  	res := s.rangeLock.LockRange(ctx, sri.span.Start, sri.span.End, sri.verID.GetID(), sri.verID.GetVer())
   609  	failpoint.Inject("kvClientMockRangeLock", func(val failpoint.Value) {
   610  		// short sleep to wait region has split
   611  		time.Sleep(time.Second)
   612  		s.rangeLock.UnlockRange(sri.span.Start, sri.span.End, sri.verID.GetID(), sri.verID.GetVer(), sri.ts)
   613  		regionNum := val.(int)
   614  		retryRanges := make([]regionspan.ComparableSpan, 0, regionNum)
   615  		start := []byte("a")
   616  		end := []byte("b1001")
   617  		for i := 0; i < regionNum; i++ {
   618  			span := regionspan.Span{Start: start, End: end}
   619  			retryRanges = append(retryRanges, regionspan.ToComparableSpan(span))
   620  			start = end
   621  			end = []byte(fmt.Sprintf("b%d", 1002+i))
   622  		}
   623  		res = regionspan.LockRangeResult{
   624  			Status:      regionspan.LockRangeStatusStale,
   625  			RetryRanges: retryRanges,
   626  		}
   627  	})
   628  
   629  	if res.Status == regionspan.LockRangeStatusWait {
   630  		res = res.WaitFn()
   631  	}
   632  
   633  	handleResult(res)
   634  }
   635  
   636  // onRegionFail handles a region's failure, which means, unlock the region's range and send the error to the errCh for
   637  // error handling. This function is non blocking even if error channel is full.
   638  // CAUTION: Note that this should only be called in a context that the region has locked it's range.
   639  func (s *eventFeedSession) onRegionFail(ctx context.Context, errorInfo regionErrorInfo, revokeToken bool) error {
   640  	log.Debug("region failed", zap.Uint64("regionID", errorInfo.verID.GetID()), zap.Error(errorInfo.err))
   641  	s.rangeLock.UnlockRange(errorInfo.span.Start, errorInfo.span.End, errorInfo.verID.GetID(), errorInfo.verID.GetVer(), errorInfo.ts)
   642  	if revokeToken {
   643  		s.regionRouter.Release(errorInfo.rpcCtx.Addr)
   644  	}
   645  	select {
   646  	case s.errCh <- errorInfo:
   647  		s.errChSizeGauge.Inc()
   648  	default:
   649  		go func() {
   650  			select {
   651  			case s.errCh <- errorInfo:
   652  				s.errChSizeGauge.Inc()
   653  			case <-ctx.Done():
   654  			}
   655  		}()
   656  	}
   657  	return nil
   658  }
   659  
   660  // requestRegionToStore gets singleRegionInfo from regionRouter, which is a token
   661  // based limiter, sends request to TiKV.
   662  // If the send request to TiKV returns error, fail the region with sendRequestToStoreErr
   663  // and kv client will redispatch the region.
   664  // If initialize gPRC stream with an error, fail the region with connectToStoreErr
   665  // and kv client will also redispatch the region.
   666  func (s *eventFeedSession) requestRegionToStore(
   667  	ctx context.Context,
   668  	g *errgroup.Group,
   669  ) error {
   670  	// Stores pending regions info for each stream. After sending a new request, the region info wil be put to the map,
   671  	// and it will be loaded by the receiver thread when it receives the first response from that region. We need this
   672  	// to pass the region info to the receiver since the region info cannot be inferred from the response from TiKV.
   673  	storePendingRegions := make(map[string]*syncRegionFeedStateMap)
   674  
   675  	var sri singleRegionInfo
   676  	for {
   677  		select {
   678  		case <-ctx.Done():
   679  			return errors.Trace(ctx.Err())
   680  		case sri = <-s.regionRouter.Chan():
   681  		}
   682  		requestID := allocID()
   683  
   684  		extraOp := kvrpcpb.ExtraOp_Noop
   685  		if s.enableOldValue {
   686  			extraOp = kvrpcpb.ExtraOp_ReadOldValue
   687  		}
   688  
   689  		rpcCtx := sri.rpcCtx
   690  		regionID := rpcCtx.Meta.GetId()
   691  		req := &cdcpb.ChangeDataRequest{
   692  			Header: &cdcpb.Header{
   693  				ClusterId:    s.client.clusterID,
   694  				TicdcVersion: version.ReleaseSemver(),
   695  			},
   696  			RegionId:     regionID,
   697  			RequestId:    requestID,
   698  			RegionEpoch:  rpcCtx.Meta.RegionEpoch,
   699  			CheckpointTs: sri.ts,
   700  			StartKey:     sri.span.Start,
   701  			EndKey:       sri.span.End,
   702  			ExtraOp:      extraOp,
   703  		}
   704  
   705  		failpoint.Inject("kvClientPendingRegionDelay", nil)
   706  
   707  		// each TiKV store has an independent pendingRegions.
   708  		var pendingRegions *syncRegionFeedStateMap
   709  
   710  		var err error
   711  		stream, ok := s.getStream(rpcCtx.Addr)
   712  		if ok {
   713  			var ok bool
   714  			pendingRegions, ok = storePendingRegions[rpcCtx.Addr]
   715  			if !ok {
   716  				// Should never happen
   717  				log.Panic("pending regions is not found for store", zap.String("store", rpcCtx.Addr))
   718  			}
   719  		} else {
   720  			// when a new stream is established, always create a new pending
   721  			// regions map, the old map will be used in old `receiveFromStream`
   722  			// and won't be deleted until that goroutine exits.
   723  			pendingRegions = newSyncRegionFeedStateMap()
   724  			storePendingRegions[rpcCtx.Addr] = pendingRegions
   725  			storeID := rpcCtx.Peer.GetStoreId()
   726  			log.Info("creating new stream to store to send request",
   727  				zap.Uint64("regionID", sri.verID.GetID()),
   728  				zap.Uint64("requestID", requestID),
   729  				zap.Uint64("storeID", storeID),
   730  				zap.String("addr", rpcCtx.Addr))
   731  			streamCtx, streamCancel := context.WithCancel(ctx)
   732  			_ = streamCancel // to avoid possible context leak warning from govet
   733  			stream, err = s.client.newStream(streamCtx, rpcCtx.Addr, storeID)
   734  			if err != nil {
   735  				// if get stream failed, maybe the store is down permanently, we should try to relocate the active store
   736  				log.Warn("get grpc stream client failed",
   737  					zap.Uint64("regionID", sri.verID.GetID()),
   738  					zap.Uint64("requestID", requestID),
   739  					zap.Uint64("storeID", storeID),
   740  					zap.String("error", err.Error()))
   741  				if cerror.ErrVersionIncompatible.Equal(err) {
   742  					// It often occurs on rolling update. Sleep 20s to reduce logs.
   743  					delay := 20 * time.Second
   744  					failpoint.Inject("kvClientDelayWhenIncompatible", func() {
   745  						delay = 100 * time.Millisecond
   746  					})
   747  					time.Sleep(delay)
   748  				}
   749  				bo := tikv.NewBackoffer(ctx, tikvRequestMaxBackoff)
   750  				s.client.regionCache.OnSendFail(bo, rpcCtx, regionScheduleReload, err)
   751  				err = s.onRegionFail(ctx, regionErrorInfo{
   752  					singleRegionInfo: sri,
   753  					err:              &connectToStoreErr{},
   754  				}, false /* revokeToken */)
   755  				if err != nil {
   756  					return errors.Trace(err)
   757  				}
   758  				continue
   759  			}
   760  			s.addStream(rpcCtx.Addr, stream, streamCancel)
   761  
   762  			limiter := s.client.getRegionLimiter(regionID)
   763  			g.Go(func() error {
   764  				defer s.deleteStream(rpcCtx.Addr)
   765  				if !s.enableKVClientV2 {
   766  					return s.receiveFromStream(ctx, g, rpcCtx.Addr, getStoreID(rpcCtx), stream.client, pendingRegions, limiter)
   767  				}
   768  				return s.receiveFromStreamV2(ctx, g, rpcCtx.Addr, getStoreID(rpcCtx), stream.client, pendingRegions, limiter)
   769  			})
   770  		}
   771  
   772  		state := newRegionFeedState(sri, requestID)
   773  		pendingRegions.insert(requestID, state)
   774  
   775  		logReq := log.Debug
   776  		if s.isPullerInit.IsInitialized() {
   777  			logReq = log.Info
   778  		}
   779  		logReq("start new request", zap.Reflect("request", req), zap.String("addr", rpcCtx.Addr))
   780  
   781  		err = stream.client.Send(req)
   782  
   783  		// If Send error, the receiver should have received error too or will receive error soon. So we doesn't need
   784  		// to do extra work here.
   785  		if err != nil {
   786  			log.Warn("send request to stream failed",
   787  				zap.String("addr", rpcCtx.Addr),
   788  				zap.Uint64("storeID", getStoreID(rpcCtx)),
   789  				zap.Uint64("regionID", sri.verID.GetID()),
   790  				zap.Uint64("requestID", requestID),
   791  				zap.Error(err))
   792  			err1 := stream.client.CloseSend()
   793  			if err1 != nil {
   794  				log.Warn("failed to close stream", zap.Error(err1))
   795  			}
   796  			// Delete the stream from the map so that the next time the store is accessed, the stream will be
   797  			// re-established.
   798  			s.deleteStream(rpcCtx.Addr)
   799  			// Delete `pendingRegions` from `storePendingRegions` so that the next time a region of this store is
   800  			// requested, it will create a new one. So if the `receiveFromStream` goroutine tries to stop all
   801  			// pending regions, the new pending regions that are requested after reconnecting won't be stopped
   802  			// incorrectly.
   803  			delete(storePendingRegions, rpcCtx.Addr)
   804  
   805  			// Remove the region from pendingRegions. If it's already removed, it should be already retried by
   806  			// `receiveFromStream`, so no need to retry here.
   807  			_, ok := pendingRegions.take(requestID)
   808  			if !ok {
   809  				// since this pending region has been removed, the token has been
   810  				// released in advance, re-add one token here.
   811  				s.regionRouter.Acquire(rpcCtx.Addr)
   812  				continue
   813  			}
   814  
   815  			// Wait for a while and retry sending the request
   816  			time.Sleep(time.Millisecond * time.Duration(rand.Intn(100)))
   817  			err = s.onRegionFail(ctx, regionErrorInfo{
   818  				singleRegionInfo: sri,
   819  				err:              &sendRequestToStoreErr{},
   820  			}, false /* revokeToken */)
   821  			if err != nil {
   822  				return errors.Trace(err)
   823  			}
   824  		} else {
   825  			s.regionRouter.Acquire(rpcCtx.Addr)
   826  		}
   827  	}
   828  }
   829  
   830  // dispatchRequest manages a set of streams and dispatch event feed requests
   831  // to these streams. Streams to each store will be created on need. After
   832  // establishing new stream, a goroutine will be spawned to handle events from
   833  // the stream.
   834  // Regions from `regionCh` will be connected. If any error happens to a
   835  // region, the error will be send to `errCh` and the receiver of `errCh` is
   836  // responsible for handling the error.
   837  func (s *eventFeedSession) dispatchRequest(
   838  	ctx context.Context,
   839  ) error {
   840  	for {
   841  		// Note that when a region is received from the channel, it's range has been already locked.
   842  		var sri singleRegionInfo
   843  		select {
   844  		case <-ctx.Done():
   845  			return ctx.Err()
   846  		case sri = <-s.regionCh:
   847  			s.regionChSizeGauge.Dec()
   848  		}
   849  
   850  		log.Debug("dispatching region", zap.Uint64("regionID", sri.verID.GetID()))
   851  
   852  		// Send a resolved ts to event channel first, for two reasons:
   853  		// 1. Since we have locked the region range, and have maintained correct
   854  		//    checkpoint ts for the range, it is safe to report the resolved ts
   855  		//    to puller at this moment.
   856  		// 2. Before the kv client gets region rpcCtx, sends request to TiKV and
   857  		//    receives the first kv event from TiKV, the region could split or
   858  		//    merge in advance, which should cause the change of resolved ts
   859  		//    distribution in puller, so this resolved ts event is needed.
   860  		// After this resolved ts event is sent, we don't need to send one more
   861  		// resolved ts event when the region starts to work.
   862  		resolvedEv := model.RegionFeedEvent{
   863  			RegionID: sri.verID.GetID(),
   864  			Resolved: &model.ResolvedSpan{
   865  				Span:       sri.span,
   866  				ResolvedTs: sri.ts,
   867  			},
   868  		}
   869  		select {
   870  		case s.eventCh <- resolvedEv:
   871  		case <-ctx.Done():
   872  			return errors.Trace(ctx.Err())
   873  		}
   874  
   875  		rpcCtx, err := s.getRPCContextForRegion(ctx, sri.verID)
   876  		if err != nil {
   877  			return errors.Trace(err)
   878  		}
   879  		if rpcCtx == nil {
   880  			// The region info is invalid. Retry the span.
   881  			log.Info("cannot get rpcCtx, retry span",
   882  				zap.Uint64("regionID", sri.verID.GetID()),
   883  				zap.Stringer("span", sri.span))
   884  			err = s.onRegionFail(ctx, regionErrorInfo{
   885  				singleRegionInfo: sri,
   886  				err: &rpcCtxUnavailableErr{
   887  					verID: sri.verID,
   888  				},
   889  			}, false /* revokeToken */)
   890  			if err != nil {
   891  				return errors.Trace(err)
   892  			}
   893  			continue
   894  		}
   895  		sri.rpcCtx = rpcCtx
   896  		s.regionRouter.AddRegion(sri)
   897  	}
   898  }
   899  
   900  // partialRegionFeed establishes a EventFeed to the region specified by regionInfo.
   901  // It manages lifecycle events of the region in order to maintain the EventFeed
   902  // connection. If any error happens (region split, leader change, etc), the region
   903  // and error info will be sent to `errCh`, and the receiver of `errCh` is
   904  // responsible for handling the error and re-establish the connection to the region.
   905  func (s *eventFeedSession) partialRegionFeed(
   906  	ctx context.Context,
   907  	state *regionFeedState,
   908  	limiter *rate.Limiter,
   909  ) error {
   910  	receiver := state.regionEventCh
   911  	defer func() {
   912  		state.markStopped()
   913  		// Workaround to avoid remaining messages in the channel blocks the receiver thread.
   914  		// TODO: Find a better solution.
   915  		timer := time.After(time.Second * 2)
   916  		for {
   917  			select {
   918  			case <-receiver:
   919  			case <-timer:
   920  				return
   921  			}
   922  		}
   923  	}()
   924  
   925  	ts := state.sri.ts
   926  	maxTs, initialized, err := s.singleEventFeed(ctx, state.sri.verID.GetID(), state.sri.span,
   927  		state.sri.ts, state.sri.rpcCtx.Addr, receiver)
   928  	log.Debug("singleEventFeed quit")
   929  
   930  	if err == nil || errors.Cause(err) == context.Canceled {
   931  		return nil
   932  	}
   933  
   934  	if errors.Cause(err) == errReconnect {
   935  		cancel, ok := s.getStreamCancel(state.sri.rpcCtx.Addr)
   936  		if ok {
   937  			// cancel the stream to trigger strem.Recv with context cancel error
   938  			// Note use context cancel is the only way to terminate a gRPC stream
   939  			cancel()
   940  			// Failover in stream.Recv has 0-100ms delay, the onRegionFail
   941  			// should be called after stream has been deleted. Add a delay here
   942  			// to avoid too frequent region rebuilt.
   943  			time.Sleep(time.Second)
   944  		}
   945  		// if stream is already deleted, just ignore errReconnect
   946  	}
   947  
   948  	failpoint.Inject("kvClientErrUnreachable", func() {
   949  		if err == errUnreachable {
   950  			failpoint.Return(err)
   951  		}
   952  	})
   953  
   954  	if maxTs > ts {
   955  		ts = maxTs
   956  	}
   957  
   958  	regionID := state.sri.verID.GetID()
   959  	log.Info("EventFeed disconnected",
   960  		zap.Uint64("regionID", regionID),
   961  		zap.Uint64("requestID", state.requestID),
   962  		zap.Stringer("span", state.sri.span),
   963  		zap.Uint64("checkpoint", ts),
   964  		zap.String("error", err.Error()))
   965  
   966  	state.sri.ts = ts
   967  
   968  	// We need to ensure when the error is handled, `isStopped` must be set. So set it before sending the error.
   969  	state.markStopped()
   970  
   971  	failpoint.Inject("kvClientSingleFeedProcessDelay", nil)
   972  
   973  	now := time.Now()
   974  	delay := limiter.ReserveN(now, 1).Delay()
   975  	if delay != 0 {
   976  		log.Info("EventFeed retry rate limited",
   977  			zap.Duration("delay", delay), zap.Reflect("regionID", regionID))
   978  		t := time.NewTimer(delay)
   979  		defer t.Stop()
   980  		select {
   981  		case <-t.C:
   982  			// We can proceed.
   983  		case <-ctx.Done():
   984  			return ctx.Err()
   985  		}
   986  	}
   987  
   988  	revokeToken := !initialized
   989  	return s.onRegionFail(ctx, regionErrorInfo{
   990  		singleRegionInfo: state.sri,
   991  		err:              err,
   992  	}, revokeToken)
   993  }
   994  
   995  // divideAndSendEventFeedToRegions split up the input span into spans aligned
   996  // to region boundaries. When region merging happens, it's possible that it
   997  // will produce some overlapping spans.
   998  func (s *eventFeedSession) divideAndSendEventFeedToRegions(
   999  	ctx context.Context, span regionspan.ComparableSpan, ts uint64,
  1000  ) error {
  1001  	limit := 20
  1002  
  1003  	nextSpan := span
  1004  	captureAddr := util.CaptureAddrFromCtx(ctx)
  1005  
  1006  	for {
  1007  		var (
  1008  			regions []*tikv.Region
  1009  			err     error
  1010  		)
  1011  		retryErr := retry.Do(ctx, func() error {
  1012  			scanT0 := time.Now()
  1013  			bo := tikv.NewBackoffer(ctx, tikvRequestMaxBackoff)
  1014  			regions, err = s.regionCache.BatchLoadRegionsWithKeyRange(bo, nextSpan.Start, nextSpan.End, limit)
  1015  			scanRegionsDuration.WithLabelValues(captureAddr).Observe(time.Since(scanT0).Seconds())
  1016  			if err != nil {
  1017  				return cerror.WrapError(cerror.ErrPDBatchLoadRegions, err)
  1018  			}
  1019  			metas := make([]*metapb.Region, 0, len(regions))
  1020  			for _, region := range regions {
  1021  				if region.GetMeta() == nil {
  1022  					err = cerror.ErrMetaNotInRegion.GenWithStackByArgs()
  1023  					log.Warn("batch load region", zap.Stringer("span", nextSpan), zap.Error(err))
  1024  					return err
  1025  				}
  1026  				metas = append(metas, region.GetMeta())
  1027  			}
  1028  			if !regionspan.CheckRegionsLeftCover(metas, nextSpan) {
  1029  				err = cerror.ErrRegionsNotCoverSpan.GenWithStackByArgs(nextSpan, metas)
  1030  				log.Warn("ScanRegions", zap.Stringer("span", nextSpan), zap.Reflect("regions", metas), zap.Error(err))
  1031  				return err
  1032  			}
  1033  			log.Debug("ScanRegions", zap.Stringer("span", nextSpan), zap.Reflect("regions", metas))
  1034  			return nil
  1035  		}, retry.WithBackoffMaxDelay(50), retry.WithMaxTries(100), retry.WithIsRetryableErr(cerror.IsRetryableError))
  1036  		if retryErr != nil {
  1037  			return retryErr
  1038  		}
  1039  
  1040  		for _, tiRegion := range regions {
  1041  			region := tiRegion.GetMeta()
  1042  			partialSpan, err := regionspan.Intersect(s.totalSpan, regionspan.ComparableSpan{Start: region.StartKey, End: region.EndKey})
  1043  			if err != nil {
  1044  				return errors.Trace(err)
  1045  			}
  1046  			log.Debug("get partialSpan", zap.Stringer("span", partialSpan), zap.Uint64("regionID", region.Id))
  1047  
  1048  			nextSpan.Start = region.EndKey
  1049  
  1050  			sri := newSingleRegionInfo(tiRegion.VerID(), partialSpan, ts, nil)
  1051  			s.scheduleRegionRequest(ctx, sri)
  1052  			log.Debug("partialSpan scheduled", zap.Stringer("span", partialSpan), zap.Uint64("regionID", region.Id))
  1053  
  1054  			// return if no more regions
  1055  			if regionspan.EndCompare(nextSpan.Start, span.End) >= 0 {
  1056  				return nil
  1057  			}
  1058  		}
  1059  	}
  1060  }
  1061  
  1062  // handleError handles error returned by a region. If some new EventFeed connection should be established, the region
  1063  // info will be sent to `regionCh`. Note if region channel is full, this function will be blocked.
  1064  // CAUTION: Note that this should only be invoked in a context that the region is not locked, otherwise use onRegionFail
  1065  // instead.
  1066  func (s *eventFeedSession) handleError(ctx context.Context, errInfo regionErrorInfo) error {
  1067  	err := errInfo.err
  1068  	switch eerr := errors.Cause(err).(type) {
  1069  	case *eventError:
  1070  		innerErr := eerr.err
  1071  		if notLeader := innerErr.GetNotLeader(); notLeader != nil {
  1072  			metricFeedNotLeaderCounter.Inc()
  1073  			// TODO: Handle the case that notleader.GetLeader() is nil.
  1074  			s.regionCache.UpdateLeader(errInfo.verID, notLeader.GetLeader().GetStoreId(), errInfo.rpcCtx.AccessIdx)
  1075  		} else if innerErr.GetEpochNotMatch() != nil {
  1076  			// TODO: If only confver is updated, we don't need to reload the region from region cache.
  1077  			metricFeedEpochNotMatchCounter.Inc()
  1078  			s.scheduleDivideRegionAndRequest(ctx, errInfo.span, errInfo.ts)
  1079  			return nil
  1080  		} else if innerErr.GetRegionNotFound() != nil {
  1081  			metricFeedRegionNotFoundCounter.Inc()
  1082  			s.scheduleDivideRegionAndRequest(ctx, errInfo.span, errInfo.ts)
  1083  			return nil
  1084  		} else if duplicatedRequest := innerErr.GetDuplicateRequest(); duplicatedRequest != nil {
  1085  			metricFeedDuplicateRequestCounter.Inc()
  1086  			logPanic("tikv reported duplicated request to the same region, which is not expected",
  1087  				zap.Uint64("regionID", duplicatedRequest.RegionId))
  1088  			return errUnreachable
  1089  		} else if compatibility := innerErr.GetCompatibility(); compatibility != nil {
  1090  			log.Error("tikv reported compatibility error, which is not expected",
  1091  				zap.String("rpcCtx", errInfo.rpcCtx.String()),
  1092  				zap.Stringer("error", compatibility))
  1093  			return cerror.ErrVersionIncompatible.GenWithStackByArgs(compatibility)
  1094  		} else {
  1095  			metricFeedUnknownErrorCounter.Inc()
  1096  			log.Warn("receive empty or unknown error msg", zap.Stringer("error", innerErr))
  1097  		}
  1098  	case *rpcCtxUnavailableErr:
  1099  		metricFeedRPCCtxUnavailable.Inc()
  1100  		s.scheduleDivideRegionAndRequest(ctx, errInfo.span, errInfo.ts)
  1101  		return nil
  1102  	case *connectToStoreErr:
  1103  		metricConnectToStoreErr.Inc()
  1104  	case *sendRequestToStoreErr:
  1105  		metricStoreSendRequestErr.Inc()
  1106  	default:
  1107  		bo := tikv.NewBackoffer(ctx, tikvRequestMaxBackoff)
  1108  		if errInfo.rpcCtx.Meta != nil {
  1109  			s.regionCache.OnSendFail(bo, errInfo.rpcCtx, regionScheduleReload, err)
  1110  		}
  1111  	}
  1112  
  1113  	failpoint.Inject("kvClientRegionReentrantErrorDelay", nil)
  1114  	s.scheduleRegionRequest(ctx, errInfo.singleRegionInfo)
  1115  	return nil
  1116  }
  1117  
  1118  func (s *eventFeedSession) getRPCContextForRegion(ctx context.Context, id tikv.RegionVerID) (*tikv.RPCContext, error) {
  1119  	bo := tikv.NewBackoffer(ctx, tikvRequestMaxBackoff)
  1120  	rpcCtx, err := s.regionCache.GetTiKVRPCContext(bo, id, tidbkv.ReplicaReadLeader, 0)
  1121  	if err != nil {
  1122  		return nil, cerror.WrapError(cerror.ErrGetTiKVRPCContext, err)
  1123  	}
  1124  	return rpcCtx, nil
  1125  }
  1126  
  1127  func (s *eventFeedSession) receiveFromStream(
  1128  	ctx context.Context,
  1129  	g *errgroup.Group,
  1130  	addr string,
  1131  	storeID uint64,
  1132  	stream cdcpb.ChangeData_EventFeedClient,
  1133  	pendingRegions *syncRegionFeedStateMap,
  1134  	limiter *rate.Limiter,
  1135  ) error {
  1136  	// Cancel the pending regions if the stream failed. Otherwise it will remain unhandled in the pendingRegions list
  1137  	// however not registered in the new reconnected stream.
  1138  	defer func() {
  1139  		log.Info("stream to store closed", zap.String("addr", addr), zap.Uint64("storeID", storeID))
  1140  
  1141  		failpoint.Inject("kvClientStreamCloseDelay", nil)
  1142  
  1143  		remainingRegions := pendingRegions.takeAll()
  1144  
  1145  		for _, state := range remainingRegions {
  1146  			err := s.onRegionFail(ctx, regionErrorInfo{
  1147  				singleRegionInfo: state.sri,
  1148  				err:              cerror.ErrPendingRegionCancel.GenWithStackByArgs(),
  1149  			}, true /* revokeToken */)
  1150  			if err != nil {
  1151  				// The only possible is that the ctx is cancelled. Simply return.
  1152  				return
  1153  			}
  1154  		}
  1155  	}()
  1156  
  1157  	captureAddr := util.CaptureAddrFromCtx(ctx)
  1158  	changefeedID := util.ChangefeedIDFromCtx(ctx)
  1159  	metricSendEventBatchResolvedSize := batchResolvedEventSize.WithLabelValues(captureAddr, changefeedID)
  1160  
  1161  	// Each region has it's own goroutine to handle its messages. `regionStates` stores states of these regions.
  1162  	regionStates := make(map[uint64]*regionFeedState)
  1163  
  1164  	for {
  1165  		cevent, err := stream.Recv()
  1166  
  1167  		failpoint.Inject("kvClientStreamRecvError", func(msg failpoint.Value) {
  1168  			errStr := msg.(string)
  1169  			if errStr == io.EOF.Error() {
  1170  				err = io.EOF
  1171  			} else {
  1172  				err = errors.New(errStr)
  1173  			}
  1174  		})
  1175  		if err != nil {
  1176  			if status.Code(errors.Cause(err)) == codes.Canceled {
  1177  				log.Debug(
  1178  					"receive from stream canceled",
  1179  					zap.String("addr", addr),
  1180  					zap.Uint64("storeID", storeID),
  1181  				)
  1182  			} else {
  1183  				log.Warn(
  1184  					"failed to receive from stream",
  1185  					zap.String("addr", addr),
  1186  					zap.Uint64("storeID", storeID),
  1187  					zap.Error(err),
  1188  				)
  1189  			}
  1190  
  1191  			// Use the same delay mechanism as `stream.Send` error handling, since
  1192  			// these two errors often mean upstream store suffers an accident, which
  1193  			// needs time to recover, kv client doesn't need to retry frequently.
  1194  			// TODO: add a better retry backoff or rate limiter
  1195  			time.Sleep(time.Millisecond * time.Duration(rand.Intn(100)))
  1196  
  1197  			// TODO: better to closes the send direction of the stream to notify
  1198  			// the other side, but it is not safe to call CloseSend concurrently
  1199  			// with SendMsg, in future refactor we should refine the recv loop
  1200  			s.deleteStream(addr)
  1201  
  1202  			for _, state := range regionStates {
  1203  				select {
  1204  				case state.regionEventCh <- emptyRegionEvent:
  1205  				case <-ctx.Done():
  1206  					return ctx.Err()
  1207  				}
  1208  			}
  1209  
  1210  			// Do no return error but gracefully stop the goroutine here. Then the whole job will not be canceled and
  1211  			// connection will be retried.
  1212  			return nil
  1213  		}
  1214  
  1215  		size := cevent.Size()
  1216  		if size > warnRecvMsgSizeThreshold {
  1217  			regionCount := 0
  1218  			if cevent.ResolvedTs != nil {
  1219  				regionCount = len(cevent.ResolvedTs.Regions)
  1220  			}
  1221  			log.Warn("change data event size too large",
  1222  				zap.Int("size", size), zap.Int("event length", len(cevent.Events)),
  1223  				zap.Int("resolved region count", regionCount))
  1224  		}
  1225  
  1226  		for _, event := range cevent.Events {
  1227  			err = s.sendRegionChangeEvent(ctx, g, event, regionStates, pendingRegions, addr, limiter)
  1228  			if err != nil {
  1229  				return err
  1230  			}
  1231  		}
  1232  		if cevent.ResolvedTs != nil {
  1233  			metricSendEventBatchResolvedSize.Observe(float64(len(cevent.ResolvedTs.Regions)))
  1234  			err = s.sendResolvedTs(ctx, cevent.ResolvedTs, regionStates, addr)
  1235  			if err != nil {
  1236  				return err
  1237  			}
  1238  		}
  1239  	}
  1240  }
  1241  
  1242  func (s *eventFeedSession) sendRegionChangeEvent(
  1243  	ctx context.Context,
  1244  	g *errgroup.Group,
  1245  	event *cdcpb.Event,
  1246  	regionStates map[uint64]*regionFeedState,
  1247  	pendingRegions *syncRegionFeedStateMap,
  1248  	addr string,
  1249  	limiter *rate.Limiter,
  1250  ) error {
  1251  	state, ok := regionStates[event.RegionId]
  1252  	// Every region's range is locked before sending requests and unlocked after exiting, and the requestID
  1253  	// is allocated while holding the range lock. Therefore the requestID is always incrementing. If a region
  1254  	// is receiving messages with different requestID, only the messages with the larges requestID is valid.
  1255  	isNewSubscription := !ok
  1256  	if ok {
  1257  		if state.requestID < event.RequestId {
  1258  			log.Info("region state entry will be replaced because received message of newer requestID",
  1259  				zap.Uint64("regionID", event.RegionId),
  1260  				zap.Uint64("oldRequestID", state.requestID),
  1261  				zap.Uint64("requestID", event.RequestId),
  1262  				zap.String("addr", addr))
  1263  			isNewSubscription = true
  1264  		} else if state.requestID > event.RequestId {
  1265  			log.Warn("drop event due to event belongs to a stale request",
  1266  				zap.Uint64("regionID", event.RegionId),
  1267  				zap.Uint64("requestID", event.RequestId),
  1268  				zap.Uint64("currRequestID", state.requestID),
  1269  				zap.String("addr", addr))
  1270  			return nil
  1271  		}
  1272  	}
  1273  
  1274  	if isNewSubscription {
  1275  		// It's the first response for this region. If the region is newly connected, the region info should
  1276  		// have been put in `pendingRegions`. So here we load the region info from `pendingRegions` and start
  1277  		// a new goroutine to handle messages from this region.
  1278  		// Firstly load the region info.
  1279  		state, ok = pendingRegions.take(event.RequestId)
  1280  		if !ok {
  1281  			log.Error("received an event but neither pending region nor running region was found",
  1282  				zap.Uint64("regionID", event.RegionId),
  1283  				zap.Uint64("requestID", event.RequestId),
  1284  				zap.String("addr", addr))
  1285  			return cerror.ErrNoPendingRegion.GenWithStackByArgs(event.RegionId, event.RequestId, addr)
  1286  		}
  1287  
  1288  		// Then spawn the goroutine to process messages of this region.
  1289  		regionStates[event.RegionId] = state
  1290  
  1291  		g.Go(func() error {
  1292  			return s.partialRegionFeed(ctx, state, limiter)
  1293  		})
  1294  	} else if state.isStopped() {
  1295  		log.Warn("drop event due to region feed stopped",
  1296  			zap.Uint64("regionID", event.RegionId),
  1297  			zap.Uint64("requestID", event.RequestId),
  1298  			zap.String("addr", addr))
  1299  		return nil
  1300  	}
  1301  
  1302  	select {
  1303  	case state.regionEventCh <- regionEvent{
  1304  		changeEvent: event,
  1305  	}:
  1306  	case <-ctx.Done():
  1307  		return ctx.Err()
  1308  	}
  1309  	return nil
  1310  }
  1311  
  1312  func (s *eventFeedSession) sendResolvedTs(
  1313  	ctx context.Context,
  1314  	resolvedTs *cdcpb.ResolvedTs,
  1315  	regionStates map[uint64]*regionFeedState,
  1316  	addr string,
  1317  ) error {
  1318  	for _, regionID := range resolvedTs.Regions {
  1319  		state, ok := regionStates[regionID]
  1320  		if ok {
  1321  			if state.isStopped() {
  1322  				log.Debug("drop resolved ts due to region feed stopped",
  1323  					zap.Uint64("regionID", regionID),
  1324  					zap.Uint64("requestID", state.requestID),
  1325  					zap.String("addr", addr))
  1326  				continue
  1327  			}
  1328  			select {
  1329  			case state.regionEventCh <- regionEvent{
  1330  				resolvedTs: resolvedTs,
  1331  			}:
  1332  			case <-ctx.Done():
  1333  				return ctx.Err()
  1334  			}
  1335  		}
  1336  	}
  1337  	return nil
  1338  }
  1339  
  1340  // singleEventFeed handles events of a single EventFeed stream.
  1341  // Results will be send to eventCh
  1342  // EventFeed RPC will not return resolved event directly
  1343  // Resolved event is generate while there's not non-match pre-write
  1344  // Return the maximum resolved
  1345  func (s *eventFeedSession) singleEventFeed(
  1346  	ctx context.Context,
  1347  	regionID uint64,
  1348  	span regionspan.ComparableSpan,
  1349  	startTs uint64,
  1350  	storeAddr string,
  1351  	receiverCh <-chan regionEvent,
  1352  ) (lastResolvedTs uint64, initialized bool, err error) {
  1353  	captureAddr := util.CaptureAddrFromCtx(ctx)
  1354  	changefeedID := util.ChangefeedIDFromCtx(ctx)
  1355  	metricReceivedEventSize := eventSize.WithLabelValues(captureAddr, "received")
  1356  	metricDroppedEventSize := eventSize.WithLabelValues(captureAddr, "dropped")
  1357  	metricPullEventInitializedCounter := pullEventCounter.WithLabelValues(cdcpb.Event_INITIALIZED.String(), captureAddr, changefeedID)
  1358  	metricPullEventCommittedCounter := pullEventCounter.WithLabelValues(cdcpb.Event_COMMITTED.String(), captureAddr, changefeedID)
  1359  	metricPullEventCommitCounter := pullEventCounter.WithLabelValues(cdcpb.Event_COMMIT.String(), captureAddr, changefeedID)
  1360  	metricPullEventPrewriteCounter := pullEventCounter.WithLabelValues(cdcpb.Event_PREWRITE.String(), captureAddr, changefeedID)
  1361  	metricPullEventRollbackCounter := pullEventCounter.WithLabelValues(cdcpb.Event_ROLLBACK.String(), captureAddr, changefeedID)
  1362  	metricSendEventResolvedCounter := sendEventCounter.WithLabelValues("native-resolved", captureAddr, changefeedID)
  1363  	metricSendEventCommitCounter := sendEventCounter.WithLabelValues("commit", captureAddr, changefeedID)
  1364  	metricSendEventCommittedCounter := sendEventCounter.WithLabelValues("committed", captureAddr, changefeedID)
  1365  
  1366  	matcher := newMatcher()
  1367  	advanceCheckTicker := time.NewTicker(time.Second * 5)
  1368  	defer advanceCheckTicker.Stop()
  1369  	lastReceivedEventTime := time.Now()
  1370  	startFeedTime := time.Now()
  1371  	lastResolvedTs = startTs
  1372  	handleResolvedTs := func(resolvedTs uint64) error {
  1373  		if !initialized {
  1374  			return nil
  1375  		}
  1376  		if resolvedTs < lastResolvedTs {
  1377  			log.Warn("The resolvedTs is fallen back in kvclient",
  1378  				zap.String("Event Type", "RESOLVED"),
  1379  				zap.Uint64("resolvedTs", resolvedTs),
  1380  				zap.Uint64("lastResolvedTs", lastResolvedTs),
  1381  				zap.Uint64("regionID", regionID))
  1382  			return nil
  1383  		}
  1384  		// emit a checkpointTs
  1385  		revent := model.RegionFeedEvent{
  1386  			RegionID: regionID,
  1387  			Resolved: &model.ResolvedSpan{
  1388  				Span:       span,
  1389  				ResolvedTs: resolvedTs,
  1390  			},
  1391  		}
  1392  		lastResolvedTs = resolvedTs
  1393  
  1394  		select {
  1395  		case s.eventCh <- revent:
  1396  			metricSendEventResolvedCounter.Inc()
  1397  		case <-ctx.Done():
  1398  			return errors.Trace(ctx.Err())
  1399  		}
  1400  		return nil
  1401  	}
  1402  
  1403  	resolveLockInterval := 20 * time.Second
  1404  	failpoint.Inject("kvClientResolveLockInterval", func(val failpoint.Value) {
  1405  		resolveLockInterval = time.Duration(val.(int)) * time.Second
  1406  	})
  1407  
  1408  	for {
  1409  		var event regionEvent
  1410  		var ok bool
  1411  		select {
  1412  		case <-ctx.Done():
  1413  			err = errors.Trace(ctx.Err())
  1414  			return
  1415  		case <-advanceCheckTicker.C:
  1416  			failpoint.Inject("kvClientForceReconnect", func() {
  1417  				log.Warn("kv client reconnect triggered by failpoint")
  1418  				failpoint.Return(lastResolvedTs, initialized, errReconnect)
  1419  			})
  1420  			if time.Since(startFeedTime) < resolveLockInterval {
  1421  				continue
  1422  			}
  1423  			if !s.isPullerInit.IsInitialized() {
  1424  				// Initializing a puller may take a long time, skip resolved lock to save unnecessary overhead.
  1425  				continue
  1426  			}
  1427  			sinceLastEvent := time.Since(lastReceivedEventTime)
  1428  			if sinceLastEvent > resolveLockInterval {
  1429  				log.Warn("region not receiving event from tikv for too long time",
  1430  					zap.Uint64("regionID", regionID), zap.Stringer("span", span), zap.Duration("duration", sinceLastEvent))
  1431  			}
  1432  			if sinceLastEvent > reconnectInterval && initialized {
  1433  				log.Warn("kv client reconnect triggered", zap.Duration("duration", sinceLastEvent))
  1434  				err = errReconnect
  1435  				return
  1436  			}
  1437  			version, err := s.kvStorage.GetCachedCurrentVersion()
  1438  			if err != nil {
  1439  				log.Warn("failed to get current version from PD", zap.Error(err))
  1440  				continue
  1441  			}
  1442  			currentTimeFromPD := oracle.GetTimeFromTS(version.Ver)
  1443  			sinceLastResolvedTs := currentTimeFromPD.Sub(oracle.GetTimeFromTS(lastResolvedTs))
  1444  			if sinceLastResolvedTs > resolveLockInterval && initialized {
  1445  				log.Warn("region not receiving resolved event from tikv or resolved ts is not pushing for too long time, try to resolve lock",
  1446  					zap.Uint64("regionID", regionID), zap.Stringer("span", span),
  1447  					zap.Duration("duration", sinceLastResolvedTs),
  1448  					zap.Duration("lastEvent", sinceLastEvent),
  1449  					zap.Uint64("resolvedTs", lastResolvedTs))
  1450  				maxVersion := oracle.ComposeTS(oracle.GetPhysical(currentTimeFromPD.Add(-10*time.Second)), 0)
  1451  				err = s.lockResolver.Resolve(ctx, regionID, maxVersion)
  1452  				if err != nil {
  1453  					log.Warn("failed to resolve lock", zap.Uint64("regionID", regionID), zap.Error(err))
  1454  					continue
  1455  				}
  1456  			}
  1457  			continue
  1458  		case event, ok = <-receiverCh:
  1459  		}
  1460  
  1461  		if !ok || event == emptyRegionEvent {
  1462  			log.Debug("singleEventFeed closed by error")
  1463  			err = cerror.ErrEventFeedAborted.GenWithStackByArgs()
  1464  			return
  1465  		}
  1466  		var revent model.RegionFeedEvent
  1467  		lastReceivedEventTime = time.Now()
  1468  		if event.changeEvent != nil {
  1469  			metricReceivedEventSize.Observe(float64(event.changeEvent.Event.Size()))
  1470  			switch x := event.changeEvent.Event.(type) {
  1471  			case *cdcpb.Event_Entries_:
  1472  				for _, entry := range x.Entries.GetEntries() {
  1473  					// if a region with kv range [a, z)
  1474  					// and we only want the get [b, c) from this region,
  1475  					// tikv will return all key events in the region although we specified [b, c) int the request.
  1476  					// we can make tikv only return the events about the keys in the specified range.
  1477  					comparableKey := regionspan.ToComparableKey(entry.GetKey())
  1478  					// key for initialized event is nil
  1479  					if !regionspan.KeyInSpan(comparableKey, span) && entry.Type != cdcpb.Event_INITIALIZED {
  1480  						metricDroppedEventSize.Observe(float64(entry.Size()))
  1481  						continue
  1482  					}
  1483  					switch entry.Type {
  1484  					case cdcpb.Event_INITIALIZED:
  1485  						if time.Since(startFeedTime) > 20*time.Second {
  1486  							log.Warn("The time cost of initializing is too much",
  1487  								zap.Duration("timeCost", time.Since(startFeedTime)),
  1488  								zap.Uint64("regionID", regionID))
  1489  						}
  1490  						metricPullEventInitializedCounter.Inc()
  1491  						initialized = true
  1492  						s.regionRouter.Release(storeAddr)
  1493  						cachedEvents := matcher.matchCachedRow(initialized)
  1494  						for _, cachedEvent := range cachedEvents {
  1495  							revent, err = assembleRowEvent(regionID, cachedEvent, s.enableOldValue)
  1496  							if err != nil {
  1497  								return
  1498  							}
  1499  							select {
  1500  							case s.eventCh <- revent:
  1501  								metricSendEventCommitCounter.Inc()
  1502  							case <-ctx.Done():
  1503  								err = errors.Trace(ctx.Err())
  1504  								return
  1505  							}
  1506  						}
  1507  						matcher.matchCachedRollbackRow(initialized)
  1508  					case cdcpb.Event_COMMITTED:
  1509  						metricPullEventCommittedCounter.Inc()
  1510  						revent, err = assembleRowEvent(regionID, entry, s.enableOldValue)
  1511  						if err != nil {
  1512  							return
  1513  						}
  1514  
  1515  						if entry.CommitTs <= lastResolvedTs {
  1516  							logPanic("The CommitTs must be greater than the resolvedTs",
  1517  								zap.String("Event Type", "COMMITTED"),
  1518  								zap.Uint64("CommitTs", entry.CommitTs),
  1519  								zap.Uint64("resolvedTs", lastResolvedTs),
  1520  								zap.Uint64("regionID", regionID))
  1521  							err = errUnreachable
  1522  							return
  1523  						}
  1524  						select {
  1525  						case s.eventCh <- revent:
  1526  							metricSendEventCommittedCounter.Inc()
  1527  						case <-ctx.Done():
  1528  							err = errors.Trace(ctx.Err())
  1529  							return
  1530  						}
  1531  					case cdcpb.Event_PREWRITE:
  1532  						metricPullEventPrewriteCounter.Inc()
  1533  						matcher.putPrewriteRow(entry)
  1534  					case cdcpb.Event_COMMIT:
  1535  						metricPullEventCommitCounter.Inc()
  1536  						if entry.CommitTs <= lastResolvedTs {
  1537  							logPanic("The CommitTs must be greater than the resolvedTs",
  1538  								zap.String("Event Type", "COMMIT"),
  1539  								zap.Uint64("CommitTs", entry.CommitTs),
  1540  								zap.Uint64("resolvedTs", lastResolvedTs),
  1541  								zap.Uint64("regionID", regionID))
  1542  							err = errUnreachable
  1543  							return
  1544  						}
  1545  						ok := matcher.matchRow(entry, initialized)
  1546  						if !ok {
  1547  							if !initialized {
  1548  								matcher.cacheCommitRow(entry)
  1549  								continue
  1550  							}
  1551  							err = cerror.ErrPrewriteNotMatch.GenWithStackByArgs(entry.GetKey(), entry.GetStartTs())
  1552  							return
  1553  						}
  1554  
  1555  						revent, err = assembleRowEvent(regionID, entry, s.enableOldValue)
  1556  						if err != nil {
  1557  							return
  1558  						}
  1559  
  1560  						select {
  1561  						case s.eventCh <- revent:
  1562  							metricSendEventCommitCounter.Inc()
  1563  						case <-ctx.Done():
  1564  							err = errors.Trace(ctx.Err())
  1565  							return
  1566  						}
  1567  					case cdcpb.Event_ROLLBACK:
  1568  						metricPullEventRollbackCounter.Inc()
  1569  						if !initialized {
  1570  							matcher.cacheRollbackRow(entry)
  1571  							continue
  1572  						}
  1573  						matcher.rollbackRow(entry)
  1574  					}
  1575  				}
  1576  			case *cdcpb.Event_Admin_:
  1577  				log.Info("receive admin event", zap.Stringer("event", event.changeEvent))
  1578  			case *cdcpb.Event_Error:
  1579  				err = cerror.WrapError(cerror.ErrEventFeedEventError, &eventError{err: x.Error})
  1580  				return
  1581  			case *cdcpb.Event_ResolvedTs:
  1582  				if err = handleResolvedTs(x.ResolvedTs); err != nil {
  1583  					return
  1584  				}
  1585  			}
  1586  		}
  1587  
  1588  		if event.resolvedTs != nil {
  1589  			if err = handleResolvedTs(event.resolvedTs.Ts); err != nil {
  1590  				return
  1591  			}
  1592  		}
  1593  	}
  1594  }
  1595  
  1596  func (s *eventFeedSession) addStream(storeAddr string, stream *eventFeedStream, cancel context.CancelFunc) {
  1597  	s.streamsLock.Lock()
  1598  	defer s.streamsLock.Unlock()
  1599  	s.streams[storeAddr] = stream
  1600  	s.streamsCanceller[storeAddr] = cancel
  1601  }
  1602  
  1603  func (s *eventFeedSession) deleteStream(storeAddr string) {
  1604  	s.streamsLock.Lock()
  1605  	defer s.streamsLock.Unlock()
  1606  	if stream, ok := s.streams[storeAddr]; ok {
  1607  		s.client.grpcPool.ReleaseConn(stream.conn, storeAddr)
  1608  		delete(s.streams, storeAddr)
  1609  	}
  1610  	if cancel, ok := s.streamsCanceller[storeAddr]; ok {
  1611  		cancel()
  1612  		delete(s.streamsCanceller, storeAddr)
  1613  	}
  1614  }
  1615  
  1616  func (s *eventFeedSession) getStream(storeAddr string) (stream *eventFeedStream, ok bool) {
  1617  	s.streamsLock.RLock()
  1618  	defer s.streamsLock.RUnlock()
  1619  	stream, ok = s.streams[storeAddr]
  1620  	return
  1621  }
  1622  
  1623  func (s *eventFeedSession) getStreamCancel(storeAddr string) (cancel context.CancelFunc, ok bool) {
  1624  	s.streamsLock.RLock()
  1625  	defer s.streamsLock.RUnlock()
  1626  	cancel, ok = s.streamsCanceller[storeAddr]
  1627  	return
  1628  }
  1629  
  1630  func assembleRowEvent(regionID uint64, entry *cdcpb.Event_Row, enableOldValue bool) (model.RegionFeedEvent, error) {
  1631  	var opType model.OpType
  1632  	switch entry.GetOpType() {
  1633  	case cdcpb.Event_Row_DELETE:
  1634  		opType = model.OpTypeDelete
  1635  	case cdcpb.Event_Row_PUT:
  1636  		opType = model.OpTypePut
  1637  	default:
  1638  		return model.RegionFeedEvent{}, cerror.ErrUnknownKVEventType.GenWithStackByArgs(entry.GetOpType(), entry)
  1639  	}
  1640  
  1641  	revent := model.RegionFeedEvent{
  1642  		RegionID: regionID,
  1643  		Val: &model.RawKVEntry{
  1644  			OpType:   opType,
  1645  			Key:      entry.Key,
  1646  			Value:    entry.GetValue(),
  1647  			StartTs:  entry.StartTs,
  1648  			CRTs:     entry.CommitTs,
  1649  			RegionID: regionID,
  1650  		},
  1651  	}
  1652  
  1653  	// when old-value is disabled, it is still possible for the tikv to send a event containing the old value
  1654  	// we need avoid a old-value sent to downstream when old-value is disabled
  1655  	if enableOldValue {
  1656  		revent.Val.OldValue = entry.GetOldValue()
  1657  	}
  1658  	return revent, nil
  1659  }
  1660  
  1661  // eventError wrap cdcpb.Event_Error to implements error interface.
  1662  type eventError struct {
  1663  	err *cdcpb.Error
  1664  }
  1665  
  1666  // Error implement error interface.
  1667  func (e *eventError) Error() string {
  1668  	return e.err.String()
  1669  }
  1670  
  1671  type rpcCtxUnavailableErr struct {
  1672  	verID tikv.RegionVerID
  1673  }
  1674  
  1675  func (e *rpcCtxUnavailableErr) Error() string {
  1676  	return fmt.Sprintf("cannot get rpcCtx for region %v. ver:%v, confver:%v",
  1677  		e.verID.GetID(), e.verID.GetVer(), e.verID.GetConfVer())
  1678  }
  1679  
  1680  type connectToStoreErr struct{}
  1681  
  1682  func (e *connectToStoreErr) Error() string { return "connect to store error" }
  1683  
  1684  type sendRequestToStoreErr struct{}
  1685  
  1686  func (e *sendRequestToStoreErr) Error() string { return "send request to store error" }
  1687  
  1688  func getStoreID(rpcCtx *tikv.RPCContext) uint64 {
  1689  	if rpcCtx != nil && rpcCtx.Peer != nil {
  1690  		return rpcCtx.Peer.GetStoreId()
  1691  	}
  1692  	return 0
  1693  }