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

     1  // Copyright 2021 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  	"reflect"
    19  	"runtime"
    20  	"sync"
    21  	"sync/atomic"
    22  	"time"
    23  
    24  	"github.com/pingcap/errors"
    25  	"github.com/pingcap/failpoint"
    26  	"github.com/pingcap/kvproto/pkg/cdcpb"
    27  	"github.com/pingcap/log"
    28  	"github.com/pingcap/ticdc/cdc/model"
    29  	"github.com/pingcap/ticdc/pkg/config"
    30  	cerror "github.com/pingcap/ticdc/pkg/errors"
    31  	"github.com/pingcap/ticdc/pkg/regionspan"
    32  	"github.com/pingcap/ticdc/pkg/util"
    33  	"github.com/pingcap/ticdc/pkg/workerpool"
    34  	"github.com/pingcap/tidb/store/tikv/oracle"
    35  	"github.com/prometheus/client_golang/prometheus"
    36  	"go.uber.org/zap"
    37  	"golang.org/x/sync/errgroup"
    38  	"golang.org/x/time/rate"
    39  )
    40  
    41  var (
    42  	regionWorkerPool          workerpool.WorkerPool
    43  	workerPoolOnce            sync.Once
    44  	regionWorkerInputChanSize = 128
    45  	regionWorkerLowWatermark  = int(float64(regionWorkerInputChanSize) * 0.2)
    46  	regionWorkerHighWatermark = int(float64(regionWorkerInputChanSize) * 0.7)
    47  )
    48  
    49  const (
    50  	minRegionStateBucket = 4
    51  	maxRegionStateBucket = 16
    52  
    53  	maxWorkerPoolSize      = 64
    54  	maxResolvedLockPerLoop = 64
    55  )
    56  
    57  // regionStateManager provides the get/put way like a sync.Map, and it is divided
    58  // into several buckets to reduce lock contention
    59  type regionStateManager struct {
    60  	bucket int
    61  	states []*sync.Map
    62  }
    63  
    64  func newRegionStateManager(bucket int) *regionStateManager {
    65  	if bucket <= 0 {
    66  		bucket = runtime.NumCPU()
    67  		if bucket > maxRegionStateBucket {
    68  			bucket = maxRegionStateBucket
    69  		}
    70  		if bucket < minRegionStateBucket {
    71  			bucket = minRegionStateBucket
    72  		}
    73  	}
    74  	rsm := &regionStateManager{
    75  		bucket: bucket,
    76  		states: make([]*sync.Map, bucket),
    77  	}
    78  	for i := range rsm.states {
    79  		rsm.states[i] = new(sync.Map)
    80  	}
    81  	return rsm
    82  }
    83  
    84  func (rsm *regionStateManager) getBucket(regionID uint64) int {
    85  	return int(regionID) % rsm.bucket
    86  }
    87  
    88  func (rsm *regionStateManager) getState(regionID uint64) (*regionFeedState, bool) {
    89  	bucket := rsm.getBucket(regionID)
    90  	if val, ok := rsm.states[bucket].Load(regionID); ok {
    91  		return val.(*regionFeedState), true
    92  	}
    93  	return nil, false
    94  }
    95  
    96  func (rsm *regionStateManager) setState(regionID uint64, state *regionFeedState) {
    97  	bucket := rsm.getBucket(regionID)
    98  	rsm.states[bucket].Store(regionID, state)
    99  }
   100  
   101  func (rsm *regionStateManager) delState(regionID uint64) {
   102  	bucket := rsm.getBucket(regionID)
   103  	rsm.states[bucket].Delete(regionID)
   104  }
   105  
   106  type regionWorkerMetrics struct {
   107  	// kv events related metrics
   108  	metricReceivedEventSize           prometheus.Observer
   109  	metricDroppedEventSize            prometheus.Observer
   110  	metricPullEventInitializedCounter prometheus.Counter
   111  	metricPullEventPrewriteCounter    prometheus.Counter
   112  	metricPullEventCommitCounter      prometheus.Counter
   113  	metricPullEventCommittedCounter   prometheus.Counter
   114  	metricPullEventRollbackCounter    prometheus.Counter
   115  	metricSendEventResolvedCounter    prometheus.Counter
   116  	metricSendEventCommitCounter      prometheus.Counter
   117  	metricSendEventCommittedCounter   prometheus.Counter
   118  
   119  	// TODO: add region runtime related metrics
   120  }
   121  
   122  /*
   123  `regionWorker` maintains N regions, it runs in background for each gRPC stream,
   124  corresponding to one TiKV store. It receives `regionStatefulEvent` in a channel
   125  from gRPC stream receiving goroutine, processes event as soon as possible and
   126  sends `RegionFeedEvent` to output channel.
   127  Besides the `regionWorker` maintains a background lock resolver, the lock resolver
   128  maintains a resolved-ts based min heap to manager region resolved ts, so it doesn't
   129  need to iterate each region every time when resolving lock.
   130  Note: There exist two locks, one is lock for region states map, the other one is
   131  lock for each region state(each region state has one lock).
   132  `regionWorker` is single routine now, it will be extended to multiple goroutines
   133  for event processing to increase throughput.
   134  */
   135  type regionWorker struct {
   136  	parentCtx context.Context
   137  	session   *eventFeedSession
   138  	limiter   *rate.Limiter
   139  
   140  	inputCh  chan *regionStatefulEvent
   141  	outputCh chan<- model.RegionFeedEvent
   142  	errorCh  chan error
   143  
   144  	// event handlers in region worker
   145  	handles []workerpool.EventHandle
   146  	// how many workers in worker pool will be used for this region worker
   147  	concurrent    int
   148  	statesManager *regionStateManager
   149  
   150  	rtsManager  *regionTsManager
   151  	rtsUpdateCh chan *regionTsInfo
   152  
   153  	metrics *regionWorkerMetrics
   154  
   155  	enableOldValue bool
   156  	storeAddr      string
   157  }
   158  
   159  func newRegionWorker(s *eventFeedSession, limiter *rate.Limiter, addr string) *regionWorker {
   160  	cfg := config.GetGlobalServerConfig().KVClient
   161  	worker := &regionWorker{
   162  		session:        s,
   163  		limiter:        limiter,
   164  		inputCh:        make(chan *regionStatefulEvent, regionWorkerInputChanSize),
   165  		outputCh:       s.eventCh,
   166  		errorCh:        make(chan error, 1),
   167  		statesManager:  newRegionStateManager(-1),
   168  		rtsManager:     newRegionTsManager(),
   169  		rtsUpdateCh:    make(chan *regionTsInfo, 1024),
   170  		enableOldValue: s.enableOldValue,
   171  		storeAddr:      addr,
   172  		concurrent:     cfg.WorkerConcurrent,
   173  	}
   174  	return worker
   175  }
   176  
   177  func (w *regionWorker) initMetrics(ctx context.Context) {
   178  	captureAddr := util.CaptureAddrFromCtx(ctx)
   179  	changefeedID := util.ChangefeedIDFromCtx(ctx)
   180  
   181  	metrics := &regionWorkerMetrics{}
   182  	metrics.metricReceivedEventSize = eventSize.WithLabelValues(captureAddr, "received")
   183  	metrics.metricDroppedEventSize = eventSize.WithLabelValues(captureAddr, "dropped")
   184  	metrics.metricPullEventInitializedCounter = pullEventCounter.WithLabelValues(cdcpb.Event_INITIALIZED.String(), captureAddr, changefeedID)
   185  	metrics.metricPullEventCommittedCounter = pullEventCounter.WithLabelValues(cdcpb.Event_COMMITTED.String(), captureAddr, changefeedID)
   186  	metrics.metricPullEventCommitCounter = pullEventCounter.WithLabelValues(cdcpb.Event_COMMIT.String(), captureAddr, changefeedID)
   187  	metrics.metricPullEventPrewriteCounter = pullEventCounter.WithLabelValues(cdcpb.Event_PREWRITE.String(), captureAddr, changefeedID)
   188  	metrics.metricPullEventRollbackCounter = pullEventCounter.WithLabelValues(cdcpb.Event_ROLLBACK.String(), captureAddr, changefeedID)
   189  	metrics.metricSendEventResolvedCounter = sendEventCounter.WithLabelValues("native-resolved", captureAddr, changefeedID)
   190  	metrics.metricSendEventCommitCounter = sendEventCounter.WithLabelValues("commit", captureAddr, changefeedID)
   191  	metrics.metricSendEventCommittedCounter = sendEventCounter.WithLabelValues("committed", captureAddr, changefeedID)
   192  
   193  	w.metrics = metrics
   194  }
   195  
   196  func (w *regionWorker) getRegionState(regionID uint64) (*regionFeedState, bool) {
   197  	return w.statesManager.getState(regionID)
   198  }
   199  
   200  func (w *regionWorker) setRegionState(regionID uint64, state *regionFeedState) {
   201  	w.statesManager.setState(regionID, state)
   202  }
   203  
   204  func (w *regionWorker) delRegionState(regionID uint64) {
   205  	w.statesManager.delState(regionID)
   206  }
   207  
   208  // checkRegionStateEmpty returns true if there is no region state maintained.
   209  // Note this function is not thread-safe
   210  func (w *regionWorker) checkRegionStateEmpty() (empty bool) {
   211  	empty = true
   212  	for _, states := range w.statesManager.states {
   213  		states.Range(func(_, _ interface{}) bool {
   214  			empty = false
   215  			return false
   216  		})
   217  		if !empty {
   218  			return
   219  		}
   220  	}
   221  	return
   222  }
   223  
   224  // checkShouldExit checks whether the region worker should exit, if should exit
   225  // return an error
   226  func (w *regionWorker) checkShouldExit(addr string) error {
   227  	empty := w.checkRegionStateEmpty()
   228  	// If there is not region maintained by this region worker, exit it and
   229  	// cancel the gRPC stream.
   230  	if empty {
   231  		cancel, ok := w.session.getStreamCancel(addr)
   232  		if ok {
   233  			cancel()
   234  		} else {
   235  			log.Warn("gRPC stream cancel func not found", zap.String("addr", addr))
   236  		}
   237  		return cerror.ErrRegionWorkerExit.GenWithStackByArgs()
   238  	}
   239  	return nil
   240  }
   241  
   242  func (w *regionWorker) handleSingleRegionError(ctx context.Context, err error, state *regionFeedState) error {
   243  	if state.lastResolvedTs > state.sri.ts {
   244  		state.sri.ts = state.lastResolvedTs
   245  	}
   246  	regionID := state.sri.verID.GetID()
   247  	log.Info("single region event feed disconnected",
   248  		zap.Uint64("regionID", regionID),
   249  		zap.Uint64("requestID", state.requestID),
   250  		zap.Stringer("span", state.sri.span),
   251  		zap.Uint64("checkpoint", state.sri.ts),
   252  		zap.String("error", err.Error()))
   253  	// if state is already marked stopped, it must have been or would be processed by `onRegionFail`
   254  	if state.isStopped() {
   255  		return w.checkShouldExit(state.sri.rpcCtx.Addr)
   256  	}
   257  	// We need to ensure when the error is handled, `isStopped` must be set. So set it before sending the error.
   258  	state.markStopped()
   259  	w.delRegionState(regionID)
   260  	failpoint.Inject("kvClientSingleFeedProcessDelay", nil)
   261  	now := time.Now()
   262  	delay := w.limiter.ReserveN(now, 1).Delay()
   263  	if delay != 0 {
   264  		log.Info("EventFeed retry rate limited",
   265  			zap.Duration("delay", delay), zap.Reflect("regionID", regionID))
   266  		t := time.NewTimer(delay)
   267  		defer t.Stop()
   268  		select {
   269  		case <-t.C:
   270  			// We can proceed.
   271  		case <-ctx.Done():
   272  			revokeToken := !state.initialized
   273  			return w.session.onRegionFail(w.parentCtx, regionErrorInfo{
   274  				singleRegionInfo: state.sri,
   275  				err:              err,
   276  			}, revokeToken)
   277  		}
   278  	}
   279  
   280  	failpoint.Inject("kvClientErrUnreachable", func() {
   281  		if err == errUnreachable {
   282  			failpoint.Return(err)
   283  		}
   284  	})
   285  
   286  	// check and cancel gRPC stream before reconnecting region, in case of the
   287  	// scenario that region connects to the same TiKV store again and reuses
   288  	// resource in this region worker by accident.
   289  	retErr := w.checkShouldExit(state.sri.rpcCtx.Addr)
   290  
   291  	revokeToken := !state.initialized
   292  	err2 := w.session.onRegionFail(ctx, regionErrorInfo{
   293  		singleRegionInfo: state.sri,
   294  		err:              err,
   295  	}, revokeToken)
   296  	if err2 != nil {
   297  		return err2
   298  	}
   299  
   300  	return retErr
   301  }
   302  
   303  func (w *regionWorker) resolveLock(ctx context.Context) error {
   304  	// tikv resolved update interval is 1s, use half of the resolck lock interval
   305  	// as lock penalty.
   306  	resolveLockPenalty := 10
   307  	resolveLockInterval := 20 * time.Second
   308  	failpoint.Inject("kvClientResolveLockInterval", func(val failpoint.Value) {
   309  		resolveLockInterval = time.Duration(val.(int)) * time.Second
   310  	})
   311  	advanceCheckTicker := time.NewTicker(time.Second * 5)
   312  	defer advanceCheckTicker.Stop()
   313  
   314  	for {
   315  		select {
   316  		case <-ctx.Done():
   317  			return errors.Trace(ctx.Err())
   318  		case rtsUpdate := <-w.rtsUpdateCh:
   319  			w.rtsManager.Upsert(rtsUpdate)
   320  		case <-advanceCheckTicker.C:
   321  			version, err := w.session.kvStorage.GetCachedCurrentVersion()
   322  			if err != nil {
   323  				log.Warn("failed to get current version from PD", zap.Error(err))
   324  				continue
   325  			}
   326  			currentTimeFromPD := oracle.GetTimeFromTS(version.Ver)
   327  			expired := make([]*regionTsInfo, 0)
   328  			for w.rtsManager.Len() > 0 {
   329  				item := w.rtsManager.Pop()
   330  				sinceLastResolvedTs := currentTimeFromPD.Sub(oracle.GetTimeFromTS(item.ts.resolvedTs))
   331  				// region does not reach resolve lock boundary, put it back
   332  				if sinceLastResolvedTs < resolveLockInterval {
   333  					w.rtsManager.Upsert(item)
   334  					break
   335  				}
   336  				expired = append(expired, item)
   337  				if len(expired) >= maxResolvedLockPerLoop {
   338  					break
   339  				}
   340  			}
   341  			if len(expired) == 0 {
   342  				continue
   343  			}
   344  			maxVersion := oracle.ComposeTS(oracle.GetPhysical(currentTimeFromPD.Add(-10*time.Second)), 0)
   345  			for _, rts := range expired {
   346  				state, ok := w.getRegionState(rts.regionID)
   347  				if !ok || state.isStopped() {
   348  					// state is already deleted or stoppped, just continue,
   349  					// and don't need to push resolved ts back to heap.
   350  					continue
   351  				}
   352  				// recheck resolved ts from region state, which may be larger than that in resolved ts heap
   353  				lastResolvedTs := state.getLastResolvedTs()
   354  				sinceLastResolvedTs := currentTimeFromPD.Sub(oracle.GetTimeFromTS(lastResolvedTs))
   355  				if sinceLastResolvedTs >= resolveLockInterval {
   356  					sinceLastEvent := time.Since(rts.ts.eventTime)
   357  					if sinceLastResolvedTs > reconnectInterval && sinceLastEvent > reconnectInterval {
   358  						log.Warn("kv client reconnect triggered",
   359  							zap.Duration("duration", sinceLastResolvedTs), zap.Duration("since last event", sinceLastResolvedTs))
   360  						return errReconnect
   361  					}
   362  					// Only resolve lock if the resovled-ts keeps unchanged for
   363  					// more than resolveLockPenalty times.
   364  					if rts.ts.penalty < resolveLockPenalty {
   365  						if lastResolvedTs > rts.ts.resolvedTs {
   366  							rts.ts.resolvedTs = lastResolvedTs
   367  							rts.ts.eventTime = time.Now()
   368  							rts.ts.penalty = 0
   369  						}
   370  						w.rtsManager.Upsert(rts)
   371  						continue
   372  					}
   373  					log.Warn("region not receiving resolved event from tikv or resolved ts is not pushing for too long time, try to resolve lock",
   374  						zap.Uint64("regionID", rts.regionID),
   375  						zap.Stringer("span", state.getRegionSpan()),
   376  						zap.Duration("duration", sinceLastResolvedTs),
   377  						zap.Duration("lastEvent", sinceLastEvent),
   378  						zap.Uint64("resolvedTs", lastResolvedTs),
   379  					)
   380  					err = w.session.lockResolver.Resolve(ctx, rts.regionID, maxVersion)
   381  					if err != nil {
   382  						log.Warn("failed to resolve lock", zap.Uint64("regionID", rts.regionID), zap.Error(err))
   383  						continue
   384  					}
   385  				}
   386  				rts.ts.resolvedTs = lastResolvedTs
   387  				w.rtsManager.Upsert(rts)
   388  			}
   389  		}
   390  	}
   391  }
   392  
   393  func (w *regionWorker) processEvent(ctx context.Context, event *regionStatefulEvent) error {
   394  	if event.finishedCounter != nil {
   395  		atomic.AddInt32(event.finishedCounter, -1)
   396  		return nil
   397  	}
   398  	var err error
   399  	event.state.lock.Lock()
   400  	if event.changeEvent != nil {
   401  		w.metrics.metricReceivedEventSize.Observe(float64(event.changeEvent.Event.Size()))
   402  		switch x := event.changeEvent.Event.(type) {
   403  		case *cdcpb.Event_Entries_:
   404  			err = w.handleEventEntry(ctx, x, event.state)
   405  			if err != nil {
   406  				err = w.handleSingleRegionError(ctx, err, event.state)
   407  			}
   408  		case *cdcpb.Event_Admin_:
   409  			log.Info("receive admin event", zap.Stringer("event", event.changeEvent))
   410  		case *cdcpb.Event_Error:
   411  			err = w.handleSingleRegionError(
   412  				ctx,
   413  				cerror.WrapError(cerror.ErrEventFeedEventError, &eventError{err: x.Error}),
   414  				event.state,
   415  			)
   416  		case *cdcpb.Event_ResolvedTs:
   417  			if err = w.handleResolvedTs(ctx, x.ResolvedTs, event.state); err != nil {
   418  				err = w.handleSingleRegionError(ctx, err, event.state)
   419  			}
   420  		}
   421  	}
   422  
   423  	if event.resolvedTs != nil {
   424  		if err = w.handleResolvedTs(ctx, event.resolvedTs.Ts, event.state); err != nil {
   425  			err = w.handleSingleRegionError(ctx, err, event.state)
   426  		}
   427  	}
   428  	event.state.lock.Unlock()
   429  	return err
   430  }
   431  
   432  func (w *regionWorker) initPoolHandles(handleCount int) {
   433  	handles := make([]workerpool.EventHandle, 0, handleCount)
   434  	for i := 0; i < handleCount; i++ {
   435  		poolHandle := regionWorkerPool.RegisterEvent(func(ctx context.Context, eventI interface{}) error {
   436  			event := eventI.(*regionStatefulEvent)
   437  			return w.processEvent(ctx, event)
   438  		}).OnExit(func(err error) {
   439  			w.onHandleExit(err)
   440  		})
   441  		handles = append(handles, poolHandle)
   442  	}
   443  	w.handles = handles
   444  }
   445  
   446  func (w *regionWorker) onHandleExit(err error) {
   447  	select {
   448  	case w.errorCh <- err:
   449  	default:
   450  	}
   451  }
   452  
   453  func (w *regionWorker) eventHandler(ctx context.Context) error {
   454  	preprocess := func(event *regionStatefulEvent, ok bool) (
   455  		exitEventHandler bool,
   456  		skipEvent bool,
   457  	) {
   458  		// event == nil means the region worker should exit and re-establish
   459  		// all existing regions.
   460  		if !ok || event == nil {
   461  			log.Info("region worker closed by error")
   462  			exitEventHandler = true
   463  			return
   464  		}
   465  		if event.state.isStopped() {
   466  			skipEvent = true
   467  		}
   468  		return
   469  	}
   470  	pollEvent := func() (event *regionStatefulEvent, ok bool, err error) {
   471  		select {
   472  		case <-ctx.Done():
   473  			err = errors.Trace(ctx.Err())
   474  		case err = <-w.errorCh:
   475  		case event, ok = <-w.inputCh:
   476  		}
   477  		return
   478  	}
   479  	for {
   480  		event, ok, err := pollEvent()
   481  		if err != nil {
   482  			return err
   483  		}
   484  		exitEventHandler, skipEvent := preprocess(event, ok)
   485  		if exitEventHandler {
   486  			return cerror.ErrRegionWorkerExit.GenWithStackByArgs()
   487  		}
   488  		if skipEvent {
   489  			continue
   490  		}
   491  		// We measure whether the current worker is busy based on the input
   492  		// channel size. If the buffered event count is larger than the high
   493  		// watermark, we send events to worker pool to increase processing
   494  		// throughput. Otherwise we process event in local region worker to
   495  		// ensure low processing latency.
   496  		if len(w.inputCh) < regionWorkerHighWatermark {
   497  			err = w.processEvent(ctx, event)
   498  			if err != nil {
   499  				return err
   500  			}
   501  		} else {
   502  			err = w.handles[int(event.regionID)%w.concurrent].AddEvent(ctx, event)
   503  			if err != nil {
   504  				return err
   505  			}
   506  			// TODO: add events in batch
   507  			for len(w.inputCh) >= regionWorkerLowWatermark {
   508  				event, ok, err = pollEvent()
   509  				if err != nil {
   510  					return err
   511  				}
   512  				exitEventHandler, skipEvent := preprocess(event, ok)
   513  				if exitEventHandler {
   514  					return cerror.ErrRegionWorkerExit.GenWithStackByArgs()
   515  				}
   516  				if skipEvent {
   517  					continue
   518  				}
   519  				err = w.handles[int(event.regionID)%w.concurrent].AddEvent(ctx, event)
   520  				if err != nil {
   521  					return err
   522  				}
   523  			}
   524  			// Principle: events from the same region must be processed linearly.
   525  			//
   526  			// When buffered events exceed high watermark, we start to use worker
   527  			// pool to improve throughtput, and we need a mechanism to quit worker
   528  			// pool when buffered events are less than low watermark, which means
   529  			// we should have a way to know whether events sent to the worker pool
   530  			// are all processed.
   531  			// Send a dummy event to each worker pool handler, after each of these
   532  			// events are processed, we can ensure all events sent to worker pool
   533  			// from this region worker are processed.
   534  			counter := int32(len(w.handles))
   535  			for _, handle := range w.handles {
   536  				err = handle.AddEvent(ctx, &regionStatefulEvent{finishedCounter: &counter})
   537  				if err != nil {
   538  					return err
   539  				}
   540  			}
   541  		checkEventsProcessed:
   542  			for {
   543  				select {
   544  				case <-ctx.Done():
   545  					return errors.Trace(ctx.Err())
   546  				case err = <-w.errorCh:
   547  					return err
   548  				case <-time.After(50 * time.Millisecond):
   549  					if atomic.LoadInt32(&counter) == 0 {
   550  						break checkEventsProcessed
   551  					}
   552  				}
   553  			}
   554  		}
   555  	}
   556  }
   557  
   558  func (w *regionWorker) collectWorkpoolError(ctx context.Context) error {
   559  	cases := make([]reflect.SelectCase, 0, len(w.handles)+1)
   560  	cases = append(cases, reflect.SelectCase{
   561  		Dir:  reflect.SelectRecv,
   562  		Chan: reflect.ValueOf(ctx.Done()),
   563  	})
   564  	for _, handle := range w.handles {
   565  		cases = append(cases, reflect.SelectCase{
   566  			Dir:  reflect.SelectRecv,
   567  			Chan: reflect.ValueOf(handle.ErrCh()),
   568  		})
   569  	}
   570  	idx, value, ok := reflect.Select(cases)
   571  	if idx == 0 {
   572  		return ctx.Err()
   573  	}
   574  	if !ok {
   575  		return nil
   576  	}
   577  	return value.Interface().(error)
   578  }
   579  
   580  func (w *regionWorker) checkErrorReconnect(err error) error {
   581  	if errors.Cause(err) == errReconnect {
   582  		cancel, ok := w.session.getStreamCancel(w.storeAddr)
   583  		if ok {
   584  			// cancel the stream to trigger strem.Recv with context cancel error
   585  			// Note use context cancel is the only way to terminate a gRPC stream
   586  			cancel()
   587  			// Failover in stream.Recv has 0-100ms delay, the onRegionFail
   588  			// should be called after stream has been deleted. Add a delay here
   589  			// to avoid too frequent region rebuilt.
   590  			time.Sleep(time.Second)
   591  		}
   592  		// if stream is already deleted, just ignore errReconnect
   593  		return nil
   594  	}
   595  	return err
   596  }
   597  
   598  func (w *regionWorker) run(parentCtx context.Context) error {
   599  	defer func() {
   600  		for _, h := range w.handles {
   601  			h.Unregister()
   602  		}
   603  	}()
   604  	w.parentCtx = parentCtx
   605  	wg, ctx := errgroup.WithContext(parentCtx)
   606  	w.initMetrics(ctx)
   607  	w.initPoolHandles(w.concurrent)
   608  	wg.Go(func() error {
   609  		return w.checkErrorReconnect(w.resolveLock(ctx))
   610  	})
   611  	wg.Go(func() error {
   612  		return w.eventHandler(ctx)
   613  	})
   614  	wg.Go(func() error {
   615  		return w.collectWorkpoolError(ctx)
   616  	})
   617  	err := wg.Wait()
   618  	// ErrRegionWorkerExit means the region worker exits normally, but we don't
   619  	// need to terminate the other goroutines in errgroup
   620  	if cerror.ErrRegionWorkerExit.Equal(err) {
   621  		return nil
   622  	}
   623  	return err
   624  }
   625  
   626  func (w *regionWorker) handleEventEntry(
   627  	ctx context.Context,
   628  	x *cdcpb.Event_Entries_,
   629  	state *regionFeedState,
   630  ) error {
   631  	regionID := state.sri.verID.GetID()
   632  	for _, entry := range x.Entries.GetEntries() {
   633  		// if a region with kv range [a, z)
   634  		// and we only want the get [b, c) from this region,
   635  		// tikv will return all key events in the region although we specified [b, c) int the request.
   636  		// we can make tikv only return the events about the keys in the specified range.
   637  		comparableKey := regionspan.ToComparableKey(entry.GetKey())
   638  		// key for initialized event is nil
   639  		if !regionspan.KeyInSpan(comparableKey, state.sri.span) && entry.Type != cdcpb.Event_INITIALIZED {
   640  			w.metrics.metricDroppedEventSize.Observe(float64(entry.Size()))
   641  			continue
   642  		}
   643  		switch entry.Type {
   644  		case cdcpb.Event_INITIALIZED:
   645  			if time.Since(state.startFeedTime) > 20*time.Second {
   646  				log.Warn("The time cost of initializing is too much",
   647  					zap.Duration("timeCost", time.Since(state.startFeedTime)),
   648  					zap.Uint64("regionID", regionID))
   649  			}
   650  			w.metrics.metricPullEventInitializedCounter.Inc()
   651  
   652  			state.initialized = true
   653  			w.session.regionRouter.Release(state.sri.rpcCtx.Addr)
   654  			cachedEvents := state.matcher.matchCachedRow(state.initialized)
   655  			for _, cachedEvent := range cachedEvents {
   656  				revent, err := assembleRowEvent(regionID, cachedEvent, w.enableOldValue)
   657  				if err != nil {
   658  					return errors.Trace(err)
   659  				}
   660  				select {
   661  				case w.outputCh <- revent:
   662  					w.metrics.metricSendEventCommitCounter.Inc()
   663  				case <-ctx.Done():
   664  					return errors.Trace(ctx.Err())
   665  				}
   666  			}
   667  			state.matcher.matchCachedRollbackRow(state.initialized)
   668  		case cdcpb.Event_COMMITTED:
   669  			w.metrics.metricPullEventCommittedCounter.Inc()
   670  			revent, err := assembleRowEvent(regionID, entry, w.enableOldValue)
   671  			if err != nil {
   672  				return errors.Trace(err)
   673  			}
   674  
   675  			if entry.CommitTs <= state.lastResolvedTs {
   676  				logPanic("The CommitTs must be greater than the resolvedTs",
   677  					zap.String("Event Type", "COMMITTED"),
   678  					zap.Uint64("CommitTs", entry.CommitTs),
   679  					zap.Uint64("resolvedTs", state.lastResolvedTs),
   680  					zap.Uint64("regionID", regionID))
   681  				return errUnreachable
   682  			}
   683  			select {
   684  			case w.outputCh <- revent:
   685  				w.metrics.metricSendEventCommittedCounter.Inc()
   686  			case <-ctx.Done():
   687  				return errors.Trace(ctx.Err())
   688  			}
   689  		case cdcpb.Event_PREWRITE:
   690  			w.metrics.metricPullEventPrewriteCounter.Inc()
   691  			state.matcher.putPrewriteRow(entry)
   692  		case cdcpb.Event_COMMIT:
   693  			w.metrics.metricPullEventCommitCounter.Inc()
   694  			if entry.CommitTs <= state.lastResolvedTs {
   695  				logPanic("The CommitTs must be greater than the resolvedTs",
   696  					zap.String("Event Type", "COMMIT"),
   697  					zap.Uint64("CommitTs", entry.CommitTs),
   698  					zap.Uint64("resolvedTs", state.lastResolvedTs),
   699  					zap.Uint64("regionID", regionID))
   700  				return errUnreachable
   701  			}
   702  			ok := state.matcher.matchRow(entry, state.initialized)
   703  			if !ok {
   704  				if !state.initialized {
   705  					state.matcher.cacheCommitRow(entry)
   706  					continue
   707  				}
   708  				return cerror.ErrPrewriteNotMatch.GenWithStackByArgs(entry.GetKey(), entry.GetStartTs())
   709  			}
   710  
   711  			revent, err := assembleRowEvent(regionID, entry, w.enableOldValue)
   712  			if err != nil {
   713  				return errors.Trace(err)
   714  			}
   715  
   716  			select {
   717  			case w.outputCh <- revent:
   718  				w.metrics.metricSendEventCommitCounter.Inc()
   719  			case <-ctx.Done():
   720  				return errors.Trace(ctx.Err())
   721  			}
   722  		case cdcpb.Event_ROLLBACK:
   723  			w.metrics.metricPullEventRollbackCounter.Inc()
   724  			if !state.initialized {
   725  				state.matcher.cacheRollbackRow(entry)
   726  				continue
   727  			}
   728  			state.matcher.rollbackRow(entry)
   729  		}
   730  	}
   731  	return nil
   732  }
   733  
   734  func (w *regionWorker) handleResolvedTs(
   735  	ctx context.Context,
   736  	resolvedTs uint64,
   737  	state *regionFeedState,
   738  ) error {
   739  	if !state.initialized {
   740  		return nil
   741  	}
   742  	regionID := state.sri.verID.GetID()
   743  	if resolvedTs < state.lastResolvedTs {
   744  		log.Warn("The resolvedTs is fallen back in kvclient",
   745  			zap.String("Event Type", "RESOLVED"),
   746  			zap.Uint64("resolvedTs", resolvedTs),
   747  			zap.Uint64("lastResolvedTs", state.lastResolvedTs),
   748  			zap.Uint64("regionID", regionID))
   749  		return nil
   750  	}
   751  	// emit a checkpointTs
   752  	revent := model.RegionFeedEvent{
   753  		RegionID: regionID,
   754  		Resolved: &model.ResolvedSpan{
   755  			Span:       state.sri.span,
   756  			ResolvedTs: resolvedTs,
   757  		},
   758  	}
   759  	state.lastResolvedTs = resolvedTs
   760  	// Send resolved ts update in non blocking way, since we can re-query real
   761  	// resolved ts from region state even if resolved ts update is discarded.
   762  	select {
   763  	case w.rtsUpdateCh <- &regionTsInfo{regionID: regionID, ts: newResolvedTsItem(resolvedTs)}:
   764  	default:
   765  	}
   766  
   767  	select {
   768  	case w.outputCh <- revent:
   769  		w.metrics.metricSendEventResolvedCounter.Inc()
   770  	case <-ctx.Done():
   771  		return errors.Trace(ctx.Err())
   772  	}
   773  	return nil
   774  }
   775  
   776  // evictAllRegions is used when gRPC stream meets error and re-establish, notify
   777  // all existing regions to re-establish
   778  func (w *regionWorker) evictAllRegions() error {
   779  	var err error
   780  	for _, states := range w.statesManager.states {
   781  		states.Range(func(_, value interface{}) bool {
   782  			state := value.(*regionFeedState)
   783  			state.lock.Lock()
   784  			// if state is marked as stopped, it must have been or would be processed by `onRegionFail`
   785  			if state.isStopped() {
   786  				state.lock.Unlock()
   787  				return true
   788  			}
   789  			state.markStopped()
   790  			w.delRegionState(state.sri.verID.GetID())
   791  			singleRegionInfo := state.sri.partialClone()
   792  			if state.lastResolvedTs > singleRegionInfo.ts {
   793  				singleRegionInfo.ts = state.lastResolvedTs
   794  			}
   795  			revokeToken := !state.initialized
   796  			state.lock.Unlock()
   797  			// since the context used in region worker will be cancelled after
   798  			// region worker exits, we must use the parent context to prevent
   799  			// regionErrorInfo loss.
   800  			err = w.session.onRegionFail(w.parentCtx, regionErrorInfo{
   801  				singleRegionInfo: singleRegionInfo,
   802  				err:              cerror.ErrEventFeedAborted.FastGenByArgs(),
   803  			}, revokeToken)
   804  			return err == nil
   805  		})
   806  	}
   807  	return err
   808  }
   809  
   810  func getWorkerPoolSize() (size int) {
   811  	cfg := config.GetGlobalServerConfig().KVClient
   812  	if cfg.WorkerPoolSize > 0 {
   813  		size = cfg.WorkerPoolSize
   814  	} else {
   815  		size = runtime.NumCPU() * 2
   816  	}
   817  	if size > maxWorkerPoolSize {
   818  		size = maxWorkerPoolSize
   819  	}
   820  	return
   821  }
   822  
   823  // InitWorkerPool initializs workerpool once, the workerpool must be initialized
   824  // before any kv event is received.
   825  func InitWorkerPool() {
   826  	if !enableKVClientV2 {
   827  		return
   828  	}
   829  	workerPoolOnce.Do(func() {
   830  		size := getWorkerPoolSize()
   831  		regionWorkerPool = workerpool.NewDefaultWorkerPool(size)
   832  	})
   833  }
   834  
   835  // RunWorkerPool runs the worker pool used by the region worker in kv client v2
   836  // It must be running before region worker starts to work
   837  func RunWorkerPool(ctx context.Context) error {
   838  	if !enableKVClientV2 {
   839  		return nil
   840  	}
   841  	InitWorkerPool()
   842  	errg, ctx := errgroup.WithContext(ctx)
   843  	errg.Go(func() error {
   844  		return errors.Trace(regionWorkerPool.Run(ctx))
   845  	})
   846  	return errg.Wait()
   847  }