github.com/matrixorigin/matrixone@v1.2.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 "context" 19 "fmt" 20 "runtime" 21 "sync" 22 "sync/atomic" 23 "time" 24 25 v2 "github.com/matrixorigin/matrixone/pkg/util/metric/v2" 26 "github.com/panjf2000/ants/v2" 27 "go.uber.org/zap" 28 29 "github.com/matrixorigin/matrixone/pkg/common/moerr" 30 "github.com/matrixorigin/matrixone/pkg/common/util" 31 "github.com/matrixorigin/matrixone/pkg/container/types" 32 "github.com/matrixorigin/matrixone/pkg/txn/clock" 33 34 "github.com/matrixorigin/matrixone/pkg/logutil" 35 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common" 36 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/iface/txnif" 37 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/sm" 38 ) 39 40 type TxnCommitListener interface { 41 OnBeginPrePrepare(txnif.AsyncTxn) 42 OnEndPrePrepare(txnif.AsyncTxn) 43 OnEndPrepareWAL(txnif.AsyncTxn) 44 } 45 46 type NoopCommitListener struct{} 47 48 func (bl *NoopCommitListener) OnBeginPrePrepare(txn txnif.AsyncTxn) {} 49 func (bl *NoopCommitListener) OnEndPrePrepare(txn txnif.AsyncTxn) {} 50 51 type batchTxnCommitListener struct { 52 listeners []TxnCommitListener 53 } 54 55 func newBatchCommitListener() *batchTxnCommitListener { 56 return &batchTxnCommitListener{ 57 listeners: make([]TxnCommitListener, 0), 58 } 59 } 60 61 func (bl *batchTxnCommitListener) AddTxnCommitListener(l TxnCommitListener) { 62 bl.listeners = append(bl.listeners, l) 63 } 64 65 func (bl *batchTxnCommitListener) OnBeginPrePrepare(txn txnif.AsyncTxn) { 66 for _, l := range bl.listeners { 67 l.OnBeginPrePrepare(txn) 68 } 69 } 70 71 func (bl *batchTxnCommitListener) OnEndPrePrepare(txn txnif.AsyncTxn) { 72 for _, l := range bl.listeners { 73 l.OnEndPrePrepare(txn) 74 } 75 } 76 func (bl *batchTxnCommitListener) OnEndPrepareWAL(txn txnif.AsyncTxn) { 77 for _, l := range bl.listeners { 78 l.OnEndPrepareWAL(txn) 79 } 80 } 81 82 type TxnStoreFactory = func() txnif.TxnStore 83 type TxnFactory = func(*TxnManager, txnif.TxnStore, []byte, types.TS, types.TS) txnif.AsyncTxn 84 85 type TxnManager struct { 86 sm.ClosedState 87 PreparingSM sm.StateMachine 88 FlushQueue sm.Queue 89 IDMap *sync.Map 90 IdAlloc *common.TxnIDAllocator 91 MaxCommittedTS atomic.Pointer[types.TS] 92 TxnStoreFactory TxnStoreFactory 93 TxnFactory TxnFactory 94 Exception *atomic.Value 95 CommitListener *batchTxnCommitListener 96 ctx context.Context 97 cancel context.CancelFunc 98 wg sync.WaitGroup 99 workers *ants.Pool 100 101 ts struct { 102 mu sync.Mutex 103 allocator *types.TsAlloctor 104 } 105 106 // for debug 107 prevPrepareTS types.TS 108 prevPrepareTSInPreparing types.TS 109 prevPrepareTSInPrepareWAL types.TS 110 } 111 112 func NewTxnManager(txnStoreFactory TxnStoreFactory, txnFactory TxnFactory, clock clock.Clock) *TxnManager { 113 if txnFactory == nil { 114 txnFactory = DefaultTxnFactory 115 } 116 mgr := &TxnManager{ 117 IDMap: new(sync.Map), 118 IdAlloc: common.NewTxnIDAllocator(), 119 TxnStoreFactory: txnStoreFactory, 120 TxnFactory: txnFactory, 121 Exception: new(atomic.Value), 122 CommitListener: newBatchCommitListener(), 123 wg: sync.WaitGroup{}, 124 } 125 mgr.ts.allocator = types.NewTsAlloctor(clock) 126 mgr.initMaxCommittedTS() 127 pqueue := sm.NewSafeQueue(20000, 1000, mgr.dequeuePreparing) 128 prepareWALQueue := sm.NewSafeQueue(20000, 1000, mgr.onPrepareWAL) 129 mgr.FlushQueue = sm.NewSafeQueue(20000, 1000, mgr.dequeuePrepared) 130 mgr.PreparingSM = sm.NewStateMachine(new(sync.WaitGroup), mgr, pqueue, prepareWALQueue) 131 132 mgr.ctx, mgr.cancel = context.WithCancel(context.Background()) 133 mgr.workers, _ = ants.NewPool(runtime.GOMAXPROCS(0)) 134 return mgr 135 } 136 137 func (mgr *TxnManager) initMaxCommittedTS() { 138 now := mgr.Now() 139 mgr.MaxCommittedTS.Store(&now) 140 } 141 142 // Now gets a timestamp under the protect from a inner lock. The lock makes 143 // all timestamps allocated before have been assigned to txn, which means those 144 // txn are visible for the returned timestamp. 145 func (mgr *TxnManager) Now() types.TS { 146 mgr.ts.mu.Lock() 147 defer mgr.ts.mu.Unlock() 148 return mgr.ts.allocator.Alloc() 149 } 150 151 func (mgr *TxnManager) Init(prevTs types.TS) error { 152 logutil.Infof("init ts to %v", prevTs.ToString()) 153 mgr.ts.allocator.SetStart(prevTs) 154 logutil.Debug("[INIT]", TxnMgrField(mgr)) 155 return nil 156 } 157 158 // Note: Replay should always runs in a single thread 159 func (mgr *TxnManager) OnReplayTxn(txn txnif.AsyncTxn) (err error) { 160 mgr.IDMap.Store(txn.GetID(), txn) 161 return 162 } 163 164 // StartTxn starts a local transaction initiated by DN 165 func (mgr *TxnManager) StartTxn(info []byte) (txn txnif.AsyncTxn, err error) { 166 if exp := mgr.Exception.Load(); exp != nil { 167 err = exp.(error) 168 logutil.Warnf("StartTxn: %v", err) 169 return 170 } 171 txnId := mgr.IdAlloc.Alloc() 172 startTs := *mgr.MaxCommittedTS.Load() 173 174 store := mgr.TxnStoreFactory() 175 txn = mgr.TxnFactory(mgr, store, txnId, startTs, types.TS{}) 176 store.BindTxn(txn) 177 mgr.IDMap.Store(util.UnsafeBytesToString(txnId), txn) 178 return 179 } 180 181 func (mgr *TxnManager) StartTxnWithStartTSAndSnapshotTS( 182 info []byte, 183 startTS, snapshotTS types.TS, 184 ) (txn txnif.AsyncTxn, err error) { 185 if exp := mgr.Exception.Load(); exp != nil { 186 err = exp.(error) 187 logutil.Warnf("StartTxn: %v", err) 188 return 189 } 190 store := mgr.TxnStoreFactory() 191 txnId := mgr.IdAlloc.Alloc() 192 txn = mgr.TxnFactory(mgr, store, txnId, startTS, snapshotTS) 193 store.BindTxn(txn) 194 mgr.IDMap.Store(util.UnsafeBytesToString(txnId), txn) 195 return 196 } 197 198 // GetOrCreateTxnWithMeta Get or create a txn initiated by CN 199 func (mgr *TxnManager) GetOrCreateTxnWithMeta( 200 info []byte, 201 id []byte, 202 ts types.TS) (txn txnif.AsyncTxn, err error) { 203 if exp := mgr.Exception.Load(); exp != nil { 204 err = exp.(error) 205 logutil.Warnf("StartTxn: %v", err) 206 return 207 } 208 if _, ok := mgr.IDMap.Load(util.UnsafeBytesToString(id)); !ok { 209 store := mgr.TxnStoreFactory() 210 txn = mgr.TxnFactory(mgr, store, id, ts, ts) 211 store.BindTxn(txn) 212 mgr.IDMap.Store(util.UnsafeBytesToString(id), txn) 213 } 214 return 215 } 216 217 func (mgr *TxnManager) DeleteTxn(id string) (err error) { 218 if _, ok := mgr.IDMap.LoadAndDelete(id); !ok { 219 err = moerr.NewTxnNotFoundNoCtx() 220 logutil.Warnf("Txn %s not found", id) 221 return 222 } 223 return 224 } 225 226 func (mgr *TxnManager) GetTxnByCtx(ctx []byte) txnif.AsyncTxn { 227 return mgr.GetTxn(IDCtxToID(ctx)) 228 } 229 230 func (mgr *TxnManager) GetTxn(id string) txnif.AsyncTxn { 231 if res, ok := mgr.IDMap.Load(id); ok { 232 return res.(txnif.AsyncTxn) 233 } 234 return nil 235 } 236 237 func (mgr *TxnManager) EnqueueFlushing(op any) (err error) { 238 _, err = mgr.PreparingSM.EnqueueCheckpoint(op) 239 return 240 } 241 242 func (mgr *TxnManager) heartbeat(ctx context.Context) { 243 defer mgr.wg.Done() 244 heartbeatTicker := time.NewTicker(time.Millisecond * 2) 245 for { 246 select { 247 case <-mgr.ctx.Done(): 248 return 249 case <-heartbeatTicker.C: 250 op := mgr.newHeartbeatOpTxn(ctx) 251 op.Txn.(*Txn).Add(1) 252 _, err := mgr.PreparingSM.EnqueueRecevied(op) 253 if err != nil { 254 panic(err) 255 } 256 } 257 } 258 } 259 260 func (mgr *TxnManager) newHeartbeatOpTxn(ctx context.Context) *OpTxn { 261 if exp := mgr.Exception.Load(); exp != nil { 262 err := exp.(error) 263 logutil.Warnf("StartTxn: %v", err) 264 return nil 265 } 266 startTs := mgr.Now() 267 txnId := mgr.IdAlloc.Alloc() 268 store := &heartbeatStore{} 269 txn := DefaultTxnFactory(mgr, store, txnId, startTs, types.TS{}) 270 store.BindTxn(txn) 271 return &OpTxn{ 272 ctx: ctx, 273 Txn: txn, 274 Op: OpCommit, 275 } 276 } 277 278 func (mgr *TxnManager) OnOpTxn(op *OpTxn) (err error) { 279 _, err = mgr.PreparingSM.EnqueueRecevied(op) 280 return 281 } 282 283 func (mgr *TxnManager) onPrePrepare(op *OpTxn) { 284 // If txn is not trying committing, do nothing 285 if !op.IsTryCommitting() { 286 return 287 } 288 289 mgr.CommitListener.OnBeginPrePrepare(op.Txn) 290 defer mgr.CommitListener.OnEndPrePrepare(op.Txn) 291 // If txn is trying committing, call txn.PrePrepare() 292 now := time.Now() 293 op.Txn.SetError(op.Txn.PrePrepare(op.ctx)) 294 common.DoIfDebugEnabled(func() { 295 logutil.Debug("[PrePrepare]", TxnField(op.Txn), common.DurationField(time.Since(now))) 296 }) 297 } 298 299 func (mgr *TxnManager) onPreparCommit(txn txnif.AsyncTxn) { 300 txn.SetError(txn.PrepareCommit()) 301 } 302 303 func (mgr *TxnManager) onPreApplyCommit(txn txnif.AsyncTxn) { 304 if err := txn.PreApplyCommit(); err != nil { 305 txn.SetError(err) 306 mgr.OnException(err) 307 } 308 } 309 310 func (mgr *TxnManager) onPreparRollback(txn txnif.AsyncTxn) { 311 _ = txn.PrepareRollback() 312 } 313 314 func (mgr *TxnManager) onBindPrepareTimeStamp(op *OpTxn) (ts types.TS) { 315 // Replay txn is always prepared 316 if op.IsReplay() { 317 ts = op.Txn.GetPrepareTS() 318 if err := op.Txn.ToPreparingLocked(ts); err != nil { 319 panic(err) 320 } 321 return 322 } 323 324 mgr.ts.mu.Lock() 325 defer mgr.ts.mu.Unlock() 326 327 ts = mgr.ts.allocator.Alloc() 328 if !mgr.prevPrepareTS.IsEmpty() { 329 if ts.Less(&mgr.prevPrepareTS) { 330 panic(fmt.Sprintf("timestamp rollback current %v, previous %v", ts.ToString(), mgr.prevPrepareTS.ToString())) 331 } 332 } 333 mgr.prevPrepareTS = ts 334 335 op.Txn.Lock() 336 defer op.Txn.Unlock() 337 338 if op.Txn.GetError() != nil { 339 op.Op = OpRollback 340 } 341 342 if op.Op == OpRollback { 343 // Should not fail here 344 _ = op.Txn.ToRollbackingLocked(ts) 345 } else { 346 // Should not fail here 347 _ = op.Txn.ToPreparingLocked(ts) 348 } 349 return 350 } 351 352 func (mgr *TxnManager) onPrepare(op *OpTxn, ts types.TS) { 353 //assign txn's prepare timestamp to TxnMvccNode. 354 mgr.onPreparCommit(op.Txn) 355 if op.Txn.GetError() != nil { 356 op.Op = OpRollback 357 op.Txn.Lock() 358 // Should not fail here 359 _ = op.Txn.ToRollbackingLocked(ts) 360 op.Txn.Unlock() 361 mgr.onPreparRollback(op.Txn) 362 } else { 363 // 1. Appending the data into appendableNode of block 364 // 2. Collect redo log,append into WalDriver 365 // TODO::need to handle the error,instead of panic for simplicity 366 mgr.onPreApplyCommit(op.Txn) 367 if op.Txn.GetError() != nil { 368 panic(op.Txn.GetID()) 369 } 370 } 371 } 372 373 func (mgr *TxnManager) onPrepare1PC(op *OpTxn, ts types.TS) { 374 // If Op is not OpCommit, prepare rollback 375 if op.Op != OpCommit { 376 mgr.onPreparRollback(op.Txn) 377 return 378 } 379 mgr.onPrepare(op, ts) 380 } 381 382 func (mgr *TxnManager) onPrepare2PC(op *OpTxn, ts types.TS) { 383 // If Op is not OpPrepare, prepare rollback 384 if op.Op != OpPrepare { 385 mgr.onPreparRollback(op.Txn) 386 return 387 } 388 389 mgr.onPrepare(op, ts) 390 } 391 392 func (mgr *TxnManager) on1PCPrepared(op *OpTxn) { 393 var err error 394 var isAbort bool 395 switch op.Op { 396 case OpCommit: 397 isAbort = false 398 if err = op.Txn.ApplyCommit(); err != nil { 399 panic(err) 400 } 401 case OpRollback: 402 isAbort = true 403 if err = op.Txn.ApplyRollback(); err != nil { 404 mgr.OnException(err) 405 logutil.Warn("[ApplyRollback]", TxnField(op.Txn), common.ErrorField(err)) 406 } 407 } 408 mgr.OnCommitTxn(op.Txn) 409 // Here to change the txn state and 410 // broadcast the rollback or commit event to all waiting threads 411 _ = op.Txn.WaitDone(err, isAbort) 412 } 413 func (mgr *TxnManager) OnCommitTxn(txn txnif.AsyncTxn) { 414 new := txn.GetCommitTS() 415 for old := mgr.MaxCommittedTS.Load(); new.Greater(old); old = mgr.MaxCommittedTS.Load() { 416 if mgr.MaxCommittedTS.CompareAndSwap(old, &new) { 417 return 418 } 419 } 420 } 421 func (mgr *TxnManager) on2PCPrepared(op *OpTxn) { 422 var err error 423 var isAbort bool 424 switch op.Op { 425 // case OpPrepare: 426 // if err = op.Txn.ToPrepared(); err != nil { 427 // panic(err) 428 // } 429 case OpRollback: 430 isAbort = true 431 if err = op.Txn.ApplyRollback(); err != nil { 432 mgr.OnException(err) 433 logutil.Warn("[ApplyRollback]", TxnField(op.Txn), common.ErrorField(err)) 434 } 435 } 436 // Here to change the txn state and 437 // broadcast the rollback event to all waiting threads 438 _ = op.Txn.WaitDone(err, isAbort) 439 } 440 441 // 1PC and 2PC 442 // dequeuePreparing the commit of 1PC txn and prepare of 2PC txn 443 // must both enter into this queue for conflict check. 444 // OpCommit : the commit of 1PC txn 445 // OpPrepare: the prepare of 2PC txn 446 // OPRollback:the rollback of 2PC or 1PC 447 func (mgr *TxnManager) dequeuePreparing(items ...any) { 448 now := time.Now() 449 for _, item := range items { 450 op := item.(*OpTxn) 451 store := op.Txn.GetStore() 452 store.TriggerTrace(txnif.TracePreparing) 453 454 // Idempotent check 455 if state := op.Txn.GetTxnState(false); state != txnif.TxnStateActive { 456 op.Txn.WaitDone(moerr.NewTxnNotActiveNoCtx(txnif.TxnStrState(state)), false) 457 continue 458 } 459 460 // Mainly do : 1. conflict check for 1PC Commit or 2PC Prepare; 461 // 2. push the AppendNode into the MVCCHandle of block 462 mgr.onPrePrepare(op) 463 464 //Before this moment, all mvcc nodes of a txn has been pushed into the MVCCHandle. 465 //1. Allocate a timestamp , set it to txn's prepare timestamp and commit timestamp, 466 // which would be changed in the future if txn is 2PC. 467 //2. Set transaction's state to Preparing or Rollbacking if op.Op is OpRollback. 468 ts := mgr.onBindPrepareTimeStamp(op) 469 470 if op.Txn.Is2PC() { 471 mgr.onPrepare2PC(op, ts) 472 } else { 473 mgr.onPrepare1PC(op, ts) 474 } 475 if !op.Txn.IsReplay() { 476 if !mgr.prevPrepareTSInPreparing.IsEmpty() { 477 prepareTS := op.Txn.GetPrepareTS() 478 if prepareTS.Less(&mgr.prevPrepareTSInPreparing) { 479 panic(fmt.Sprintf("timestamp rollback current %v, previous %v", op.Txn.GetPrepareTS().ToString(), mgr.prevPrepareTSInPreparing.ToString())) 480 } 481 } 482 mgr.prevPrepareTSInPreparing = op.Txn.GetPrepareTS() 483 } 484 485 store.TriggerTrace(txnif.TracePrepareWalWait) 486 if err := mgr.EnqueueFlushing(op); err != nil { 487 panic(err) 488 } 489 } 490 common.DoIfDebugEnabled(func() { 491 logutil.Debug("[dequeuePreparing]", 492 common.NameSpaceField("txns"), 493 common.DurationField(time.Since(now)), 494 common.CountField(len(items))) 495 }) 496 } 497 498 func (mgr *TxnManager) onPrepareWAL(items ...any) { 499 now := time.Now() 500 501 for _, item := range items { 502 op := item.(*OpTxn) 503 store := op.Txn.GetStore() 504 store.TriggerTrace(txnif.TracePrepareWal) 505 var t1, t2, t3, t4, t5 time.Time 506 t1 = time.Now() 507 if op.Txn.GetError() == nil && op.Op == OpCommit || op.Op == OpPrepare { 508 if err := op.Txn.PrepareWAL(); err != nil { 509 panic(err) 510 } 511 512 t2 = time.Now() 513 514 if !op.Txn.IsReplay() { 515 if !mgr.prevPrepareTSInPrepareWAL.IsEmpty() { 516 prepareTS := op.Txn.GetPrepareTS() 517 if prepareTS.Less(&mgr.prevPrepareTSInPrepareWAL) { 518 panic(fmt.Sprintf("timestamp rollback current %v, previous %v", op.Txn.GetPrepareTS().ToString(), mgr.prevPrepareTSInPrepareWAL.ToString())) 519 } 520 } 521 mgr.prevPrepareTSInPrepareWAL = op.Txn.GetPrepareTS() 522 } 523 524 mgr.CommitListener.OnEndPrepareWAL(op.Txn) 525 t3 = time.Now() 526 } 527 528 t4 = time.Now() 529 store.TriggerTrace(txnif.TracePreapredWait) 530 if _, err := mgr.FlushQueue.Enqueue(op); err != nil { 531 panic(err) 532 } 533 t5 = time.Now() 534 535 if t5.Sub(t1) > time.Second { 536 logutil.Warn( 537 "SLOW-LOG", 538 zap.String("txn", op.Txn.String()), 539 zap.Duration("prepare-wal-duration", t2.Sub(t1)), 540 zap.Duration("end-prepare-duration", t3.Sub(t2)), 541 zap.Duration("enqueue-flush-duration", t5.Sub(t4)), 542 ) 543 } 544 } 545 common.DoIfDebugEnabled(func() { 546 logutil.Debug("[prepareWAL]", 547 common.NameSpaceField("txns"), 548 common.DurationField(time.Since(now)), 549 common.CountField(len(items))) 550 }) 551 } 552 553 // 1PC and 2PC 554 func (mgr *TxnManager) dequeuePrepared(items ...any) { 555 now := time.Now() 556 for _, item := range items { 557 op := item.(*OpTxn) 558 store := op.Txn.GetStore() 559 store.TriggerTrace(txnif.TracePrepared) 560 mgr.workers.Submit(func() { 561 //Notice that WaitPrepared do nothing when op is OpRollback 562 t0 := time.Now() 563 if err := op.Txn.WaitPrepared(op.ctx); err != nil { 564 // v0.6 TODO: Error handling 565 panic(err) 566 } 567 568 if op.Is2PC() { 569 mgr.on2PCPrepared(op) 570 } else { 571 mgr.on1PCPrepared(op) 572 } 573 dequeuePreparedDuration := time.Since(t0) 574 v2.TxnDequeuePreparedDurationHistogram.Observe(dequeuePreparedDuration.Seconds()) 575 }) 576 } 577 common.DoIfDebugEnabled(func() { 578 logutil.Debug("[dequeuePrepared]", 579 common.NameSpaceField("txns"), 580 common.CountField(len(items)), 581 common.DurationField(time.Since(now))) 582 }) 583 } 584 585 func (mgr *TxnManager) OnException(new error) { 586 old := mgr.Exception.Load() 587 for old == nil { 588 if mgr.Exception.CompareAndSwap(old, new) { 589 break 590 } 591 old = mgr.Exception.Load() 592 } 593 } 594 595 // MinTSForTest is only be used in ut to ensure that 596 // files that have been gc will not be used. 597 func (mgr *TxnManager) MinTSForTest() types.TS { 598 minTS := types.MaxTs() 599 mgr.IDMap.Range(func(key, value any) bool { 600 txn := value.(txnif.AsyncTxn) 601 startTS := txn.GetStartTS() 602 if startTS.Less(&minTS) { 603 minTS = startTS 604 } 605 return true 606 }) 607 return minTS 608 } 609 610 func (mgr *TxnManager) Start(ctx context.Context) { 611 mgr.FlushQueue.Start() 612 mgr.PreparingSM.Start() 613 mgr.wg.Add(1) 614 go mgr.heartbeat(ctx) 615 } 616 617 func (mgr *TxnManager) Stop() { 618 mgr.cancel() 619 mgr.wg.Wait() 620 mgr.PreparingSM.Stop() 621 mgr.FlushQueue.Stop() 622 mgr.OnException(sm.ErrClose) 623 mgr.workers.Release() 624 logutil.Info("[Stop]", TxnMgrField(mgr)) 625 }