github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/causality/internal/node.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 internal 15 16 import ( 17 "sync" 18 "sync/atomic" 19 20 "github.com/google/btree" 21 "github.com/pingcap/log" 22 "go.uber.org/zap" 23 ) 24 25 type ( 26 cacheID = int64 27 ) 28 29 const ( 30 unassigned = cacheID(-2) 31 assignedToAny = cacheID(-1) 32 ) 33 34 var ( 35 nextNodeID = atomic.Int64{} 36 37 // btreeFreeList is a shared free list used by all 38 // btrees in order to lessen the burden of GC. 39 // 40 // Experiment shows increasing the capacity beyond 1024 yields little 41 // performance improvement. 42 btreeFreeList = btree.NewFreeListG[*Node](1024) 43 ) 44 45 // Node is a node in the dependency graph used 46 // in conflict detection. 47 type Node struct { 48 // Immutable fields. 49 id int64 50 sortedDedupKeysHash []uint64 51 52 // Called when all dependencies are resolved. 53 TrySendToTxnCache func(id cacheID) bool 54 // Set the id generator to get a random ID. 55 RandCacheID func() cacheID 56 // Set the callback that the node is notified. 57 OnNotified func(callback func()) 58 59 // Following fields are used for notifying a node's dependers lock-free. 60 totalDependencies int32 61 removedDependencies int32 62 resolvedDependencies int32 63 resolvedList []int64 64 65 // Following fields are protected by `mu`. 66 mu sync.Mutex 67 68 assignedTo cacheID 69 removed bool 70 71 // dependers is an ordered set for all nodes that 72 // conflict with the current node. 73 // 74 // Notes: 75 // (1) An ordered data structure is preferred because 76 // if we can unblock conflicting transactions in the 77 // order that they have come in, the out-of-order-ness 78 // observed downstream will be less than what would have been 79 // if an unordered set were used. 80 // (2) Google's btree package is selected because it seems to be 81 // the most popular production-grade ordered set implementation in Go. 82 dependers *btree.BTreeG[*Node] 83 } 84 85 func (n *Node) nodeID() int64 { 86 return n.id 87 } 88 89 func (n *Node) dependOn(dependencyNodes map[int64]*Node) { 90 resolvedDependencies := int32(0) 91 92 depend := func(target *Node) { 93 if target.id == n.id { 94 log.Panic("node cannot depend on itself") 95 } 96 97 // The target node might be removed or modified in other places, for example 98 // after its corresponding transaction has been executed. 99 target.mu.Lock() 100 defer target.mu.Unlock() 101 102 if target.assignedTo != unassigned { 103 // The target has already been assigned to a cache. 104 // In this case, record the cache ID in `resolvedList`, and this node 105 // probably can be sent to the same cache and executed sequentially. 106 resolvedDependencies = atomic.AddInt32(&n.resolvedDependencies, 1) 107 atomic.StoreInt64(&n.resolvedList[resolvedDependencies-1], target.assignedTo) 108 } 109 110 // Add the node to the target's dependers if the target has not been removed. 111 if target.removed { 112 // The target has already been removed. 113 atomic.AddInt32(&n.removedDependencies, 1) 114 } else if _, exist := target.getOrCreateDependers().ReplaceOrInsert(n); exist { 115 // Should never depend on a target redundantly. 116 log.Panic("should never exist") 117 } 118 } 119 120 // `totalDependencies` and `resolvedList` must be initialized before depending on any targets. 121 n.totalDependencies = int32(len(dependencyNodes)) 122 n.resolvedList = make([]int64, 0, n.totalDependencies) 123 for i := 0; i < int(n.totalDependencies); i++ { 124 n.resolvedList = append(n.resolvedList, unassigned) 125 } 126 127 for _, node := range dependencyNodes { 128 depend(node) 129 } 130 131 n.maybeResolve() 132 } 133 134 func (n *Node) remove() { 135 n.mu.Lock() 136 defer n.mu.Unlock() 137 138 n.removed = true 139 if n.dependers != nil { 140 // `mu` must be holded during accessing dependers. 141 n.dependers.Ascend(func(node *Node) bool { 142 atomic.AddInt32(&node.removedDependencies, 1) 143 node.OnNotified(node.maybeResolve) 144 return true 145 }) 146 n.dependers.Clear(true) 147 n.dependers = nil 148 } 149 } 150 151 // tryAssignTo assigns a node to a cache. Returns `true` on success. 152 func (n *Node) tryAssignTo(cacheID int64) bool { 153 n.mu.Lock() 154 defer n.mu.Unlock() 155 156 if n.assignedTo != unassigned { 157 // Already resolved by some other guys. 158 return true 159 } 160 161 if n.TrySendToTxnCache != nil { 162 ok := n.TrySendToTxnCache(cacheID) 163 if !ok { 164 return false 165 } 166 n.TrySendToTxnCache = nil 167 } 168 n.assignedTo = cacheID 169 170 if n.dependers != nil { 171 // `mu` must be holded during accessing dependers. 172 n.dependers.Ascend(func(node *Node) bool { 173 resolvedDependencies := atomic.AddInt32(&node.resolvedDependencies, 1) 174 atomic.StoreInt64(&node.resolvedList[resolvedDependencies-1], n.assignedTo) 175 node.OnNotified(node.maybeResolve) 176 return true 177 }) 178 } 179 180 return true 181 } 182 183 func (n *Node) maybeResolve() { 184 if cacheID, ok := n.tryResolve(); ok { 185 if cacheID == unassigned { 186 log.Panic("invalid cache ID", zap.Uint64("cacheID", uint64(cacheID))) 187 } 188 189 if cacheID != assignedToAny { 190 n.tryAssignTo(cacheID) 191 return 192 } 193 194 cacheID := n.RandCacheID() 195 if !n.tryAssignTo(cacheID) { 196 // If the cache is full, we need to try to assign to another cache. 197 n.OnNotified(n.maybeResolve) 198 } 199 } 200 } 201 202 // tryResolve try to find a cache to assign the node to. 203 // Returns (_, false) if there is a conflict, 204 // returns (rand, true) if there is no conflict, 205 // returns (N, true) if only cache N can be used. 206 func (n *Node) tryResolve() (int64, bool) { 207 if n.totalDependencies == 0 { 208 // No conflicts, can select any caches. 209 return assignedToAny, true 210 } 211 212 removedDependencies := atomic.LoadInt32(&n.removedDependencies) 213 if removedDependencies == n.totalDependencies { 214 // All dependcies are removed, so assign the node to any cache is fine. 215 return assignedToAny, true 216 } 217 218 resolvedDependencies := atomic.LoadInt32(&n.resolvedDependencies) 219 if resolvedDependencies == n.totalDependencies { 220 firstDep := atomic.LoadInt64(&n.resolvedList[0]) 221 hasDiffDep := false 222 for i := 1; i < int(n.totalDependencies); i++ { 223 curr := atomic.LoadInt64(&n.resolvedList[i]) 224 // In DependOn, depend(nil) set resolvedList[i] to assignedToAny 225 // for these no dependecy keys. 226 if curr == assignedToAny { 227 continue 228 } 229 if firstDep != curr { 230 hasDiffDep = true 231 break 232 } 233 } 234 if !hasDiffDep && firstDep != unassigned { 235 // If all dependency nodes are assigned to the same cache, we can assign 236 // this node to the same cache directly, and they will execute sequentially. 237 // On the other hand, if dependency nodes are assigned to different caches, 238 // This node has to wait all dependency txn executed and all depencecy nodes 239 // are removed. 240 return firstDep, true 241 } 242 } 243 244 return unassigned, false 245 } 246 247 func (n *Node) getOrCreateDependers() *btree.BTreeG[*Node] { 248 if n.dependers == nil { 249 n.dependers = btree.NewWithFreeListG(8, func(a, b *Node) bool { 250 return a.id < b.id 251 }, btreeFreeList) 252 } 253 return n.dependers 254 } 255 256 // dependerCount returns the number of dependers the node has. 257 // NOTE: dependerCount is used for unit tests only. 258 func (n *Node) dependerCount() int { 259 n.mu.Lock() 260 defer n.mu.Unlock() 261 262 if n.dependers == nil { 263 return 0 264 } 265 return n.dependers.Len() 266 } 267 268 // assignedWorkerID returns the cache ID that the node has been assigned to. 269 // NOTE: assignedWorkerID is used for unit tests only. 270 func (n *Node) assignedWorkerID() cacheID { 271 n.mu.Lock() 272 defer n.mu.Unlock() 273 274 return n.assignedTo 275 } 276 277 func genNextNodeID() int64 { 278 return nextNodeID.Add(1) 279 }