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

     1  // Copyright 2022 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  	"sync"
    18  
    19  	"github.com/pingcap/tiflow/cdc/kv/regionlock"
    20  	"github.com/pingcap/tiflow/cdc/processor/tablepb"
    21  	"github.com/tikv/client-go/v2/tikv"
    22  )
    23  
    24  const (
    25  	stateNormal  uint32 = 0
    26  	stateStopped uint32 = 1
    27  	stateRemoved uint32 = 2
    28  )
    29  
    30  type regionInfo struct {
    31  	verID tikv.RegionVerID
    32  	// The span of the region.
    33  	// Note(dongmen): The span doesn't always represent the whole span of a region.
    34  	// Instead, it is the portion of the region that belongs the subcribed table.
    35  	// Multiple tables can belong to the same region.
    36  	// For instance, consider region-1 with a span of [a, d).
    37  	// It contains 3 tables: t1[a, b), t2[b,c), and t3[c,d).
    38  	// If only table t1 is subscribed to, then the span of interest is [a,b).
    39  	span   tablepb.Span
    40  	rpcCtx *tikv.RPCContext
    41  
    42  	// The table that the region belongs to.
    43  	subscribedTable *subscribedTable
    44  	// The state of the locked range of the region.
    45  	lockedRangeState *regionlock.LockedRangeState
    46  }
    47  
    48  func (s regionInfo) isStoped() bool {
    49  	// lockedRange only nil when the region's subscribedTable is stopped.
    50  	return s.lockedRangeState == nil
    51  }
    52  
    53  func newRegionInfo(
    54  	verID tikv.RegionVerID,
    55  	span tablepb.Span,
    56  	rpcCtx *tikv.RPCContext,
    57  	subscribedTable *subscribedTable,
    58  ) regionInfo {
    59  	return regionInfo{
    60  		verID:           verID,
    61  		span:            span,
    62  		rpcCtx:          rpcCtx,
    63  		subscribedTable: subscribedTable,
    64  	}
    65  }
    66  
    67  func (s regionInfo) resolvedTs() uint64 {
    68  	return s.lockedRangeState.ResolvedTs.Load()
    69  }
    70  
    71  type regionErrorInfo struct {
    72  	regionInfo
    73  	err error
    74  }
    75  
    76  func newRegionErrorInfo(info regionInfo, err error) regionErrorInfo {
    77  	return regionErrorInfo{
    78  		regionInfo: info,
    79  		err:        err,
    80  	}
    81  }
    82  
    83  type regionFeedState struct {
    84  	region    regionInfo
    85  	requestID uint64
    86  	matcher   *matcher
    87  
    88  	// Transform: normal -> stopped -> removed.
    89  	// normal: the region is in replicating.
    90  	// stopped: some error happens.
    91  	// removed: the region is returned into the pending list,
    92  	//   will be re-resolved and re-scheduled later.
    93  	state struct {
    94  		sync.RWMutex
    95  		v uint32
    96  		// All region errors should be handled in region workers.
    97  		// `err` is used to retrieve errors generated outside.
    98  		err error
    99  	}
   100  }
   101  
   102  func newRegionFeedState(region regionInfo, requestID uint64) *regionFeedState {
   103  	return &regionFeedState{
   104  		region:    region,
   105  		requestID: requestID,
   106  	}
   107  }
   108  
   109  func (s *regionFeedState) start() {
   110  	s.matcher = newMatcher()
   111  }
   112  
   113  // mark regionFeedState as stopped with the given error if possible.
   114  func (s *regionFeedState) markStopped(err error) {
   115  	s.state.Lock()
   116  	defer s.state.Unlock()
   117  	if s.state.v == stateNormal {
   118  		s.state.v = stateStopped
   119  		s.state.err = err
   120  	}
   121  }
   122  
   123  // mark regionFeedState as removed if possible.
   124  func (s *regionFeedState) markRemoved() (changed bool) {
   125  	s.state.Lock()
   126  	defer s.state.Unlock()
   127  	if s.state.v == stateStopped {
   128  		s.state.v = stateRemoved
   129  		changed = true
   130  	}
   131  	return
   132  }
   133  
   134  func (s *regionFeedState) isStale() bool {
   135  	s.state.RLock()
   136  	defer s.state.RUnlock()
   137  	return s.state.v == stateStopped || s.state.v == stateRemoved
   138  }
   139  
   140  func (s *regionFeedState) takeError() (err error) {
   141  	s.state.Lock()
   142  	defer s.state.Unlock()
   143  	err = s.state.err
   144  	s.state.err = nil
   145  	return
   146  }
   147  
   148  func (s *regionFeedState) isInitialized() bool {
   149  	return s.region.lockedRangeState.Initialzied.Load()
   150  }
   151  
   152  func (s *regionFeedState) setInitialized() {
   153  	s.region.lockedRangeState.Initialzied.Store(true)
   154  }
   155  
   156  func (s *regionFeedState) getRegionID() uint64 {
   157  	return s.region.verID.GetID()
   158  }
   159  
   160  func (s *regionFeedState) getLastResolvedTs() uint64 {
   161  	return s.region.lockedRangeState.ResolvedTs.Load()
   162  }
   163  
   164  // updateResolvedTs update the resolved ts of the current region feed
   165  func (s *regionFeedState) updateResolvedTs(resolvedTs uint64) {
   166  	state := s.region.lockedRangeState
   167  	for {
   168  		last := state.ResolvedTs.Load()
   169  		if last > resolvedTs {
   170  			return
   171  		}
   172  		if state.ResolvedTs.CompareAndSwap(last, resolvedTs) {
   173  			break
   174  		}
   175  	}
   176  
   177  	if s.region.subscribedTable != nil {
   178  		// When resolvedTs is received, we need to try to resolve the lock of the region.
   179  		// Because the updated resolvedTs may less than the target resolvedTs we want advance to.
   180  		s.region.subscribedTable.tryResolveLock(
   181  			s.region.verID.GetID(),
   182  			state,
   183  		)
   184  	}
   185  }
   186  
   187  func (s *regionFeedState) getRegionInfo() regionInfo {
   188  	return s.region
   189  }
   190  
   191  func (s *regionFeedState) getRegionMeta() (uint64, tablepb.Span, string) {
   192  	return s.region.verID.GetID(), s.region.span, s.region.rpcCtx.Addr
   193  }