github.com/matrixorigin/matrixone@v1.2.0/pkg/txn/service/service.go (about) 1 // Copyright 2021 - 2022 Matrix Origin 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 // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 // See the License for the specific language governing permissions and 13 // limitations under the License. 14 15 package service 16 17 import ( 18 "bytes" 19 "context" 20 "errors" 21 "fmt" 22 "sync" 23 "time" 24 25 "github.com/fagongzi/util/hack" 26 "github.com/matrixorigin/matrixone/pkg/common/log" 27 "github.com/matrixorigin/matrixone/pkg/common/stopper" 28 "github.com/matrixorigin/matrixone/pkg/lockservice" 29 "github.com/matrixorigin/matrixone/pkg/pb/metadata" 30 "github.com/matrixorigin/matrixone/pkg/pb/txn" 31 "github.com/matrixorigin/matrixone/pkg/txn/rpc" 32 "github.com/matrixorigin/matrixone/pkg/txn/storage" 33 "github.com/matrixorigin/matrixone/pkg/txn/util" 34 "go.uber.org/zap" 35 ) 36 37 var _ TxnService = (*service)(nil) 38 39 type service struct { 40 logger *log.MOLogger 41 shard metadata.TNShard 42 storage storage.TxnStorage 43 sender rpc.TxnSender 44 stopper *stopper.Stopper 45 allocator lockservice.LockTableAllocator 46 47 // TxnService maintains a sync.Map in memory to record all running transactions. The metadata for each write 48 // transaction is initialized when the TxnService receives its first write operation and written to the map. 49 // The transaction is removed from the map after it has been committed or aborted. 50 // 51 // When a transaction's Read operation encounters data written by transactions in Committing and Prepared, 52 // it needs to wait for these transactions to reach the final state as it is not sure if the data is visible 53 // for the current read transaction. So we need to keep track of all running write transactions and notify the 54 // blocked Read operation when the transaction is committed or aborted. 55 // 56 // In some cases, after a transaction has been committed or rolled back, a previous write request is received 57 // due to the network, resulting in the transaction information being written back to the map. 58 // We use the zombieTimeout setting to solve this problem, so that when a transaction exceeds the zombieTimeout 59 // threshold in the map, it is cleaned up. 60 transactions sync.Map // string(txn.id) -> txnContext 61 zombieTimeout time.Duration 62 pool sync.Pool 63 recoveryC chan struct{} 64 txnC chan txn.TxnMeta 65 } 66 67 // NewTxnService create TxnService 68 func NewTxnService( 69 shard metadata.TNShard, 70 storage storage.TxnStorage, 71 sender rpc.TxnSender, 72 zombieTimeout time.Duration, 73 allocator lockservice.LockTableAllocator) TxnService { 74 s := &service{ 75 logger: util.GetLogger(), 76 shard: shard, 77 sender: sender, 78 storage: storage, 79 pool: sync.Pool{ 80 New: func() any { 81 return &txnContext{} 82 }}, 83 stopper: stopper.NewStopper(fmt.Sprintf("txn-service-%d-%d", 84 shard.ShardID, 85 shard.ReplicaID), 86 stopper.WithLogger(util.GetLogger().RawLogger())), 87 zombieTimeout: zombieTimeout, 88 recoveryC: make(chan struct{}), 89 txnC: make(chan txn.TxnMeta, 16), 90 allocator: allocator, 91 } 92 if err := s.stopper.RunTask(s.gcZombieTxn); err != nil { 93 s.logger.Fatal("start gc zombie txn failed", 94 zap.Error(err)) 95 } 96 return s 97 } 98 99 func (s *service) Shard() metadata.TNShard { 100 return s.shard 101 } 102 103 func (s *service) Start() error { 104 if err := s.storage.Start(); err != nil { 105 return err 106 } 107 s.startRecovery() 108 return nil 109 } 110 111 func (s *service) Close(destroy bool) error { 112 s.waitRecoveryCompleted() 113 s.stopper.Stop() 114 closer := s.storage.Close 115 if destroy { 116 closer = s.storage.Destroy 117 } 118 // FIXME: all context.TODO() need to use tracing context 119 return errors.Join(closer(context.TODO()), s.sender.Close()) 120 } 121 122 func (s *service) gcZombieTxn(ctx context.Context) { 123 s.logger.Info("gc zombie txn task started") 124 defer s.logger.Info("gc zombie txn task stopped") 125 126 timer := time.NewTicker(s.zombieTimeout) 127 defer timer.Stop() 128 129 var cleanTxns []txn.TxnMeta 130 for { 131 select { 132 case <-ctx.Done(): 133 return 134 case <-timer.C: 135 s.transactions.Range(func(_, value any) bool { 136 txnCtx := value.(*txnContext) 137 txnMeta := txnCtx.getTxn() 138 // if a txn is not a distributed txn coordinator, wait coordinator dnshard. 139 if len(txnMeta.TNShards) == 0 || 140 (len(txnMeta.TNShards) > 0 && s.shard.ShardID != txnMeta.TNShards[0].ShardID) { 141 return true 142 } 143 144 now := time.Now() 145 if now.Sub(txnCtx.createAt) > s.zombieTimeout { 146 cleanTxns = append(cleanTxns, txnMeta) 147 } 148 return true 149 }) 150 for _, txnMeta := range cleanTxns { 151 req := &txn.TxnRequest{ 152 Method: txn.TxnMethod_Rollback, 153 Txn: txnMeta, 154 RollbackRequest: &txn.TxnRollbackRequest{}, 155 } 156 resp := &txn.TxnResponse{} 157 if err := s.Rollback(ctx, req, resp); err != nil || resp.TxnError != nil { 158 txnError := "" 159 if resp.TxnError != nil { 160 txnError = resp.TxnError.DebugString() 161 } 162 s.logger.Error("rollback zombie txn failed", 163 util.TxnField(txnMeta), 164 zap.String("txn-err", txnError), 165 zap.Error(err)) 166 } 167 } 168 cleanTxns = cleanTxns[:0] 169 } 170 } 171 } 172 173 func (s *service) maybeAddTxn(meta txn.TxnMeta) (*txnContext, bool) { 174 id := hack.SliceToString(meta.ID) 175 if v, ok := s.transactions.Load(id); ok { 176 return v.(*txnContext), false 177 } 178 179 txnCtx := s.acquireTxnContext() 180 v, loaded := s.transactions.LoadOrStore(id, txnCtx) 181 if loaded { 182 s.releaseTxnContext(txnCtx) 183 return v.(*txnContext), false 184 } 185 186 // 1. first transaction write request at current DNShard 187 // 2. transaction already committed or aborted, the transaction context will be removed by gcZombieTxn. 188 txnCtx.init(meta, acquireNotifier()) 189 util.LogTxnCreateOn(meta, s.shard) 190 return txnCtx, true 191 } 192 193 func (s *service) removeTxn(txnID []byte) { 194 s.transactions.Delete(hack.SliceToString(txnID)) 195 } 196 197 func (s *service) getTxnContext(txnID []byte) *txnContext { 198 id := hack.SliceToString(txnID) 199 v, ok := s.transactions.Load(id) 200 if !ok { 201 return nil 202 } 203 return v.(*txnContext) 204 } 205 206 func (s *service) validTNShard(tn metadata.TNShard) bool { 207 if !s.shard.Equal(tn) { 208 // DNShard not match, so cn need to fetch latest DNShards from hakeeper. 209 s.logger.Error("DN metadata not match", 210 zap.String("request-dn", tn.DebugString()), 211 zap.String("local-dn", s.shard.DebugString())) 212 return false 213 } 214 return true 215 } 216 217 func (s *service) acquireTxnContext() *txnContext { 218 return s.pool.Get().(*txnContext) 219 } 220 221 func (s *service) releaseTxnContext(txnCtx *txnContext) { 222 txnCtx.resetLocked() 223 s.pool.Put(txnCtx) 224 } 225 226 func (s *service) parallelSendWithRetry( 227 ctx context.Context, 228 requests []txn.TxnRequest, 229 ignoreTxnErrorCodes map[uint16]struct{}) *rpc.SendResult { 230 for { 231 select { 232 case <-ctx.Done(): 233 return nil 234 default: 235 util.LogTxnSendRequests(requests) 236 result, err := s.sender.Send(ctx, requests) 237 if err != nil { 238 util.LogTxnSendRequestsFailed(requests, err) 239 continue 240 } 241 util.LogTxnReceivedResponses(result.Responses) 242 hasError := false 243 for _, resp := range result.Responses { 244 if resp.TxnError != nil { 245 _, ok := ignoreTxnErrorCodes[uint16(resp.TxnError.Code)] 246 if !ok { 247 hasError = true 248 } 249 } 250 } 251 if !hasError { 252 return result 253 } 254 result.Release() 255 } 256 } 257 } 258 259 type txnContext struct { 260 nt *notifier 261 createAt time.Time 262 263 mu struct { 264 sync.RWMutex 265 requests []txn.TxnRequest 266 txn txn.TxnMeta 267 } 268 } 269 270 func (c *txnContext) addWaiter(txnID []byte, w *waiter, waitStatus txn.TxnStatus) bool { 271 c.mu.Lock() 272 defer c.mu.Unlock() 273 274 if !bytes.Equal(c.mu.txn.ID, txnID) { 275 return false 276 } 277 278 util.LogTxnWaiterAdded(c.mu.txn, waitStatus) 279 c.nt.addWaiter(w, waitStatus) 280 return true 281 } 282 283 func (c *txnContext) init(txn txn.TxnMeta, nt *notifier) { 284 c.mu.Lock() 285 defer c.mu.Unlock() 286 287 c.mu.txn = txn 288 c.nt = nt 289 c.createAt = time.Now() 290 } 291 292 func (c *txnContext) getTxn() txn.TxnMeta { 293 c.mu.RLock() 294 defer c.mu.RUnlock() 295 return c.getTxnLocked() 296 } 297 298 func (c *txnContext) updateTxn(txn txn.TxnMeta) { 299 c.mu.Lock() 300 defer c.mu.Unlock() 301 302 c.updateTxnLocked(txn) 303 } 304 305 func (c *txnContext) getTxnLocked() txn.TxnMeta { 306 return c.mu.txn 307 } 308 309 func (c *txnContext) updateTxnLocked(txn txn.TxnMeta) { 310 c.mu.txn = txn 311 util.LogTxnUpdated(c.mu.txn) 312 } 313 314 func (c *txnContext) resetLocked() { 315 c.nt.close(c.mu.txn.Status) 316 c.nt = nil 317 c.mu.requests = c.mu.requests[:0] 318 c.mu.txn = txn.TxnMeta{} 319 } 320 321 func (c *txnContext) changeStatusLocked(status txn.TxnStatus) { 322 if c.mu.txn.Status != status { 323 c.mu.txn.Status = status 324 util.LogTxnUpdated(c.mu.txn) 325 c.nt.notify(status) 326 } 327 }