github.com/matrixorigin/matrixone@v0.7.0/pkg/vm/engine/tae/txn/txnbase/txnmgr.go (about) 1 // Copyright 2021 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 txnbase 16 17 import ( 18 "sync" 19 "sync/atomic" 20 "time" 21 22 "github.com/matrixorigin/matrixone/pkg/common/moerr" 23 "github.com/matrixorigin/matrixone/pkg/container/types" 24 "github.com/matrixorigin/matrixone/pkg/txn/clock" 25 26 "github.com/matrixorigin/matrixone/pkg/logutil" 27 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common" 28 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/iface/txnif" 29 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/sm" 30 ) 31 32 type TxnCommitListener interface { 33 OnBeginPrePrepare(txnif.AsyncTxn) 34 OnEndPrePrepare(txnif.AsyncTxn) 35 } 36 37 type NoopCommitListener struct{} 38 39 func (bl *NoopCommitListener) OnBeginPrePrepare(txn txnif.AsyncTxn) {} 40 func (bl *NoopCommitListener) OnEndPrePrepare(txn txnif.AsyncTxn) {} 41 42 type batchTxnCommitListener struct { 43 listeners []TxnCommitListener 44 } 45 46 func newBatchCommitListener() *batchTxnCommitListener { 47 return &batchTxnCommitListener{ 48 listeners: make([]TxnCommitListener, 0), 49 } 50 } 51 52 func (bl *batchTxnCommitListener) AddTxnCommitListener(l TxnCommitListener) { 53 bl.listeners = append(bl.listeners, l) 54 } 55 56 func (bl *batchTxnCommitListener) OnBeginPrePrepare(txn txnif.AsyncTxn) { 57 for _, l := range bl.listeners { 58 l.OnBeginPrePrepare(txn) 59 } 60 } 61 62 func (bl *batchTxnCommitListener) OnEndPrePrepare(txn txnif.AsyncTxn) { 63 for _, l := range bl.listeners { 64 l.OnEndPrePrepare(txn) 65 } 66 } 67 68 type TxnStoreFactory = func() txnif.TxnStore 69 type TxnFactory = func(*TxnManager, txnif.TxnStore, []byte, types.TS, []byte) txnif.AsyncTxn 70 71 type TxnManager struct { 72 sync.RWMutex 73 common.ClosedState 74 PreparingSM sm.StateMachine 75 IDMap map[string]txnif.AsyncTxn 76 IdAlloc *common.TxnIDAllocator 77 TsAlloc *types.TsAlloctor 78 TxnStoreFactory TxnStoreFactory 79 TxnFactory TxnFactory 80 Exception *atomic.Value 81 CommitListener *batchTxnCommitListener 82 } 83 84 func NewTxnManager(txnStoreFactory TxnStoreFactory, txnFactory TxnFactory, clock clock.Clock) *TxnManager { 85 if txnFactory == nil { 86 txnFactory = DefaultTxnFactory 87 } 88 mgr := &TxnManager{ 89 IDMap: make(map[string]txnif.AsyncTxn), 90 IdAlloc: common.NewTxnIDAllocator(), 91 TsAlloc: types.NewTsAlloctor(clock), 92 TxnStoreFactory: txnStoreFactory, 93 TxnFactory: txnFactory, 94 Exception: new(atomic.Value), 95 CommitListener: newBatchCommitListener(), 96 } 97 pqueue := sm.NewSafeQueue(20000, 1000, mgr.dequeuePreparing) 98 fqueue := sm.NewSafeQueue(20000, 1000, mgr.dequeuePrepared) 99 mgr.PreparingSM = sm.NewStateMachine(new(sync.WaitGroup), mgr, pqueue, fqueue) 100 101 return mgr 102 } 103 104 func (mgr *TxnManager) Init(prevTs types.TS) error { 105 mgr.TsAlloc.SetStart(prevTs) 106 logutil.Info("[INIT]", TxnMgrField(mgr)) 107 return nil 108 } 109 110 func (mgr *TxnManager) StatMaxCommitTS() (ts types.TS) { 111 mgr.RLock() 112 ts = mgr.TsAlloc.Alloc() 113 mgr.RUnlock() 114 return 115 } 116 117 // Note: Replay should always runs in a single thread 118 func (mgr *TxnManager) OnReplayTxn(txn txnif.AsyncTxn) (err error) { 119 mgr.Lock() 120 defer mgr.Unlock() 121 // TODO: idempotent check 122 mgr.IDMap[txn.GetID()] = txn 123 return 124 } 125 126 // StartTxn starts a local transaction initiated by DN 127 func (mgr *TxnManager) StartTxn(info []byte) (txn txnif.AsyncTxn, err error) { 128 if exp := mgr.Exception.Load(); exp != nil { 129 err = exp.(error) 130 logutil.Warnf("StartTxn: %v", err) 131 return 132 } 133 mgr.Lock() 134 defer mgr.Unlock() 135 txnId := mgr.IdAlloc.Alloc() 136 startTs := mgr.TsAlloc.Alloc() 137 138 store := mgr.TxnStoreFactory() 139 txn = mgr.TxnFactory(mgr, store, txnId, startTs, info) 140 store.BindTxn(txn) 141 mgr.IDMap[string(txnId)] = txn 142 return 143 } 144 145 // GetOrCreateTxnWithMeta Get or create a txn initiated by CN 146 func (mgr *TxnManager) GetOrCreateTxnWithMeta( 147 info []byte, 148 id []byte, 149 ts types.TS) (txn txnif.AsyncTxn, err error) { 150 if exp := mgr.Exception.Load(); exp != nil { 151 err = exp.(error) 152 logutil.Warnf("StartTxn: %v", err) 153 return 154 } 155 mgr.Lock() 156 defer mgr.Unlock() 157 txn, ok := mgr.IDMap[string(id)] 158 if !ok { 159 store := mgr.TxnStoreFactory() 160 txn = mgr.TxnFactory(mgr, store, id, ts, info) 161 store.BindTxn(txn) 162 mgr.IDMap[string(id)] = txn 163 } 164 return 165 } 166 167 func (mgr *TxnManager) DeleteTxn(id string) (err error) { 168 mgr.Lock() 169 defer mgr.Unlock() 170 txn := mgr.IDMap[id] 171 if txn == nil { 172 err = moerr.NewTxnNotFoundNoCtx() 173 logutil.Warnf("Txn %s not found", id) 174 return 175 } 176 delete(mgr.IDMap, id) 177 return 178 } 179 180 func (mgr *TxnManager) GetTxnByCtx(ctx []byte) txnif.AsyncTxn { 181 return mgr.GetTxn(IDCtxToID(ctx)) 182 } 183 184 func (mgr *TxnManager) GetTxn(id string) txnif.AsyncTxn { 185 mgr.RLock() 186 defer mgr.RUnlock() 187 return mgr.IDMap[id] 188 } 189 190 func (mgr *TxnManager) EnqueueFlushing(op any) (err error) { 191 _, err = mgr.PreparingSM.EnqueueCheckpoint(op) 192 return 193 } 194 195 func (mgr *TxnManager) OnOpTxn(op *OpTxn) (err error) { 196 _, err = mgr.PreparingSM.EnqueueRecevied(op) 197 return 198 } 199 200 func (mgr *TxnManager) onPrePrepare(op *OpTxn) { 201 // If txn is not trying committing, do nothing 202 if !op.IsTryCommitting() { 203 return 204 } 205 206 mgr.CommitListener.OnBeginPrePrepare(op.Txn) 207 defer mgr.CommitListener.OnEndPrePrepare(op.Txn) 208 // If txn is trying committing, call txn.PrePrepare() 209 now := time.Now() 210 op.Txn.SetError(op.Txn.PrePrepare()) 211 logutil.Debug("[PrePrepare]", TxnField(op.Txn), common.DurationField(time.Since(now))) 212 } 213 214 func (mgr *TxnManager) onPreparCommit(txn txnif.AsyncTxn) { 215 txn.SetError(txn.PrepareCommit()) 216 } 217 218 func (mgr *TxnManager) onPreApplyCommit(txn txnif.AsyncTxn) { 219 if err := txn.PreApplyCommit(); err != nil { 220 txn.SetError(err) 221 mgr.OnException(err) 222 } 223 } 224 225 func (mgr *TxnManager) onPreparRollback(txn txnif.AsyncTxn) { 226 _ = txn.PrepareRollback() 227 } 228 229 func (mgr *TxnManager) onBindPrepareTimeStamp(op *OpTxn) (ts types.TS) { 230 // Replay txn is always prepared 231 if op.IsReplay() { 232 ts = op.Txn.GetPrepareTS() 233 if err := op.Txn.ToPreparingLocked(ts); err != nil { 234 panic(err) 235 } 236 return 237 } 238 239 mgr.Lock() 240 defer mgr.Unlock() 241 242 ts = mgr.TsAlloc.Alloc() 243 244 op.Txn.Lock() 245 defer op.Txn.Unlock() 246 247 if op.Txn.GetError() != nil { 248 op.Op = OpRollback 249 } 250 251 if op.Op == OpRollback { 252 // Should not fail here 253 _ = op.Txn.ToRollbackingLocked(ts) 254 } else { 255 // Should not fail here 256 _ = op.Txn.ToPreparingLocked(ts) 257 } 258 return 259 } 260 261 func (mgr *TxnManager) onPrepare(op *OpTxn, ts types.TS) { 262 //assign txn's prepare timestamp to TxnMvccNode. 263 mgr.onPreparCommit(op.Txn) 264 if op.Txn.GetError() != nil { 265 op.Op = OpRollback 266 op.Txn.Lock() 267 // Should not fail here 268 _ = op.Txn.ToRollbackingLocked(ts) 269 op.Txn.Unlock() 270 mgr.onPreparRollback(op.Txn) 271 } else { 272 // 1. Appending the data into appendableNode of block 273 // 2. Collect redo log,append into WalDriver 274 // TODO::need to handle the error,instead of panic for simplicity 275 mgr.onPreApplyCommit(op.Txn) 276 if op.Txn.GetError() != nil { 277 panic(op.Txn.GetID()) 278 } 279 } 280 } 281 282 func (mgr *TxnManager) onPrepare1PC(op *OpTxn, ts types.TS) { 283 // If Op is not OpCommit, prepare rollback 284 if op.Op != OpCommit { 285 mgr.onPreparRollback(op.Txn) 286 return 287 } 288 mgr.onPrepare(op, ts) 289 } 290 291 func (mgr *TxnManager) onPrepare2PC(op *OpTxn, ts types.TS) { 292 // If Op is not OpPrepare, prepare rollback 293 if op.Op != OpPrepare { 294 mgr.onPreparRollback(op.Txn) 295 return 296 } 297 298 mgr.onPrepare(op, ts) 299 } 300 301 func (mgr *TxnManager) on1PCPrepared(op *OpTxn) { 302 var err error 303 var isAbort bool 304 switch op.Op { 305 case OpCommit: 306 isAbort = false 307 if err = op.Txn.ApplyCommit(); err != nil { 308 panic(err) 309 } 310 case OpRollback: 311 isAbort = true 312 if err = op.Txn.ApplyRollback(); err != nil { 313 mgr.OnException(err) 314 logutil.Warn("[ApplyRollback]", TxnField(op.Txn), common.ErrorField(err)) 315 } 316 } 317 // Here to change the txn state and 318 // broadcast the rollback or commit event to all waiting threads 319 _ = op.Txn.WaitDone(err, isAbort) 320 } 321 322 func (mgr *TxnManager) on2PCPrepared(op *OpTxn) { 323 var err error 324 var isAbort bool 325 switch op.Op { 326 // case OpPrepare: 327 // if err = op.Txn.ToPrepared(); err != nil { 328 // panic(err) 329 // } 330 case OpRollback: 331 isAbort = true 332 if err = op.Txn.ApplyRollback(); err != nil { 333 mgr.OnException(err) 334 logutil.Warn("[ApplyRollback]", TxnField(op.Txn), common.ErrorField(err)) 335 } 336 } 337 // Here to change the txn state and 338 // broadcast the rollback event to all waiting threads 339 _ = op.Txn.WaitDone(err, isAbort) 340 } 341 342 // 1PC and 2PC 343 // dequeuePreparing the commit of 1PC txn and prepare of 2PC txn 344 // must both enter into this queue for conflict check. 345 // OpCommit : the commit of 1PC txn 346 // OpPrepare: the prepare of 2PC txn 347 // OPRollback:the rollback of 2PC or 1PC 348 func (mgr *TxnManager) dequeuePreparing(items ...any) { 349 now := time.Now() 350 for _, item := range items { 351 op := item.(*OpTxn) 352 353 // Idempotent check 354 if state := op.Txn.GetTxnState(false); state != txnif.TxnStateActive { 355 op.Txn.WaitDone(moerr.NewTxnNotActiveNoCtx(txnif.TxnStrState(state)), false) 356 continue 357 } 358 359 // Mainly do : 1. conflict check for 1PC Commit or 2PC Prepare; 360 // 2. push the AppendNode into the MVCCHandle of block 361 mgr.onPrePrepare(op) 362 363 //Before this moment, all mvcc nodes of a txn has been pushed into the MVCCHandle. 364 //1. Allocate a timestamp , set it to txn's prepare timestamp and commit timestamp, 365 // which would be changed in the future if txn is 2PC. 366 //2. Set transaction's state to Preparing or Rollbacking if op.Op is OpRollback. 367 ts := mgr.onBindPrepareTimeStamp(op) 368 369 if op.Txn.Is2PC() { 370 mgr.onPrepare2PC(op, ts) 371 } else { 372 mgr.onPrepare1PC(op, ts) 373 } 374 375 if err := mgr.EnqueueFlushing(op); err != nil { 376 panic(err) 377 } 378 } 379 logutil.Debug("[dequeuePreparing]", 380 common.NameSpaceField("txns"), 381 common.DurationField(time.Since(now)), 382 common.CountField(len(items))) 383 } 384 385 // 1PC and 2PC 386 func (mgr *TxnManager) dequeuePrepared(items ...any) { 387 var err error 388 now := time.Now() 389 for _, item := range items { 390 op := item.(*OpTxn) 391 //Notice that WaitPrepared do nothing when op is OpRollback 392 if err = op.Txn.WaitPrepared(); err != nil { 393 // v0.6 TODO: Error handling 394 panic(err) 395 } 396 397 if op.Is2PC() { 398 mgr.on2PCPrepared(op) 399 } else { 400 mgr.on1PCPrepared(op) 401 } 402 } 403 logutil.Debug("[dequeuePrepared]", 404 common.NameSpaceField("txns"), 405 common.CountField(len(items)), 406 common.DurationField(time.Since(now))) 407 } 408 409 func (mgr *TxnManager) OnException(new error) { 410 old := mgr.Exception.Load() 411 for old == nil { 412 if mgr.Exception.CompareAndSwap(old, new) { 413 break 414 } 415 old = mgr.Exception.Load() 416 } 417 } 418 419 // MinTSForTest is only be used in ut to ensure that 420 // files that have been gc will not be used. 421 func (mgr *TxnManager) MinTSForTest() types.TS { 422 mgr.RLock() 423 defer mgr.RUnlock() 424 minTS := types.MaxTs() 425 for _, txn := range mgr.IDMap { 426 startTS := txn.GetStartTS() 427 if startTS.Less(minTS) { 428 minTS = startTS 429 } 430 } 431 return minTS 432 } 433 434 func (mgr *TxnManager) Start() { 435 mgr.PreparingSM.Start() 436 } 437 438 func (mgr *TxnManager) Stop() { 439 mgr.PreparingSM.Stop() 440 mgr.OnException(common.ErrClose) 441 logutil.Info("[Stop]", TxnMgrField(mgr)) 442 }