github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/causality/conflict_detector.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 causality
    15  
    16  import (
    17  	"sync"
    18  
    19  	"github.com/pingcap/log"
    20  	"github.com/pingcap/tiflow/pkg/causality/internal"
    21  	"github.com/pingcap/tiflow/pkg/chann"
    22  	"go.uber.org/atomic"
    23  	"go.uber.org/zap"
    24  )
    25  
    26  // ConflictDetector implements a logic that dispatches transaction
    27  // to different worker cache channels in a way that transactions
    28  // modifying the same keys are never executed concurrently and
    29  // have their original orders preserved. Transactions in different
    30  // channels can be executed concurrently.
    31  type ConflictDetector[Txn txnEvent] struct {
    32  	// resolvedTxnCaches are used to cache resolved transactions.
    33  	resolvedTxnCaches []txnCache[Txn]
    34  
    35  	// slots are used to find all unfinished transactions
    36  	// conflicting with an incoming transactions.
    37  	slots    *internal.Slots
    38  	numSlots uint64
    39  
    40  	// nextCacheID is used to dispatch transactions round-robin.
    41  	nextCacheID atomic.Int64
    42  
    43  	// Used to run a background goroutine to GC or notify nodes.
    44  	notifiedNodes *chann.DrainableChann[func()]
    45  	wg            sync.WaitGroup
    46  	closeCh       chan struct{}
    47  }
    48  
    49  // NewConflictDetector creates a new ConflictDetector.
    50  func NewConflictDetector[Txn txnEvent](
    51  	numSlots uint64, opt TxnCacheOption,
    52  ) *ConflictDetector[Txn] {
    53  	ret := &ConflictDetector[Txn]{
    54  		resolvedTxnCaches: make([]txnCache[Txn], opt.Count),
    55  		slots:             internal.NewSlots(numSlots),
    56  		numSlots:          numSlots,
    57  		notifiedNodes:     chann.NewAutoDrainChann[func()](),
    58  		closeCh:           make(chan struct{}),
    59  	}
    60  	for i := 0; i < opt.Count; i++ {
    61  		ret.resolvedTxnCaches[i] = newTxnCache[Txn](opt)
    62  	}
    63  
    64  	ret.wg.Add(1)
    65  	go func() {
    66  		defer ret.wg.Done()
    67  		ret.runBackgroundTasks()
    68  	}()
    69  
    70  	return ret
    71  }
    72  
    73  // Add pushes a transaction to the ConflictDetector.
    74  //
    75  // NOTE: if multiple threads access this concurrently,
    76  // Txn.ConflictKeys must be sorted by the slot index.
    77  func (d *ConflictDetector[Txn]) Add(txn Txn) {
    78  	hashes := txn.ConflictKeys()
    79  	node := d.slots.AllocNode(hashes)
    80  	txnWithNotifier := TxnWithNotifier[Txn]{
    81  		TxnEvent: txn,
    82  		PostTxnExecuted: func() {
    83  			// After this transaction is executed, we can remove the node from the graph,
    84  			// and resolve related dependencies for these transacitons which depend on this
    85  			// executed transaction.
    86  			d.slots.Remove(node)
    87  		},
    88  	}
    89  	node.TrySendToTxnCache = func(cacheID int64) bool {
    90  		// Try sending this txn to related cache as soon as all dependencies are resolved.
    91  		return d.sendToCache(txnWithNotifier, cacheID)
    92  	}
    93  	node.RandCacheID = func() int64 { return d.nextCacheID.Add(1) % int64(len(d.resolvedTxnCaches)) }
    94  	node.OnNotified = func(callback func()) { d.notifiedNodes.In() <- callback }
    95  	d.slots.Add(node)
    96  }
    97  
    98  // Close closes the ConflictDetector.
    99  func (d *ConflictDetector[Txn]) Close() {
   100  	close(d.closeCh)
   101  	d.wg.Wait()
   102  }
   103  
   104  func (d *ConflictDetector[Txn]) runBackgroundTasks() {
   105  	defer func() {
   106  		d.notifiedNodes.CloseAndDrain()
   107  	}()
   108  	for {
   109  		select {
   110  		case <-d.closeCh:
   111  			return
   112  		case notifyCallback := <-d.notifiedNodes.Out():
   113  			if notifyCallback != nil {
   114  				notifyCallback()
   115  			}
   116  		}
   117  	}
   118  }
   119  
   120  // sendToCache should not call txn.Callback if it returns an error.
   121  func (d *ConflictDetector[Txn]) sendToCache(txn TxnWithNotifier[Txn], id int64) bool {
   122  	if id < 0 {
   123  		log.Panic("must assign with a valid cacheID", zap.Int64("cacheID", id))
   124  	}
   125  
   126  	// Note OnConflictResolved must be called before add to cache. Otherwise, there will
   127  	// be a data race since the txn may be read before the OnConflictResolved is called.
   128  	txn.TxnEvent.OnConflictResolved()
   129  	cache := d.resolvedTxnCaches[id]
   130  	ok := cache.add(txn)
   131  	return ok
   132  }
   133  
   134  // GetOutChByCacheID returns the output channel by cacheID.
   135  // Note txns in single cache should be executed sequentially.
   136  func (d *ConflictDetector[Txn]) GetOutChByCacheID(id int64) <-chan TxnWithNotifier[Txn] {
   137  	if id < 0 {
   138  		log.Panic("must assign with a valid cacheID", zap.Int64("cacheID", id))
   139  	}
   140  	return d.resolvedTxnCaches[id].out()
   141  }