github.com/matrixorigin/matrixone@v0.7.0/pkg/vm/engine/tae/txn/txnbase/impl.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  	"github.com/matrixorigin/matrixone/pkg/common/moerr"
    19  	"github.com/matrixorigin/matrixone/pkg/logutil"
    20  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/iface/txnif"
    21  )
    22  
    23  func (txn *Txn) rollback1PC() (err error) {
    24  	if txn.IsReplay() {
    25  		panic(moerr.NewTAERollbackNoCtx("1pc txn %s should not be called here", txn.String()))
    26  	}
    27  	state := txn.GetTxnState(false)
    28  	if state != txnif.TxnStateActive {
    29  		return moerr.NewTAERollbackNoCtx("unexpected txn status : %s", txnif.TxnStrState(state))
    30  	}
    31  
    32  	txn.Add(1)
    33  	err = txn.Mgr.OnOpTxn(&OpTxn{
    34  		Txn: txn,
    35  		Op:  OpRollback,
    36  	})
    37  	if err != nil {
    38  		_ = txn.PrepareRollback()
    39  		_ = txn.ApplyRollback()
    40  		txn.DoneWithErr(err, true)
    41  	}
    42  	txn.Wait()
    43  	//txn.Status = txnif.TxnStatusRollbacked
    44  	if err = txn.Mgr.DeleteTxn(txn.GetID()); err != nil {
    45  		return
    46  	}
    47  	return txn.Err
    48  }
    49  
    50  func (txn *Txn) commit1PC(_ bool) (err error) {
    51  	state := txn.GetTxnState(false)
    52  	if state != txnif.TxnStateActive {
    53  		logutil.Warnf("unexpected txn state : %s", txnif.TxnStrState(state))
    54  		return moerr.NewTAECommitNoCtx("invalid txn state %s", txnif.TxnStrState(state))
    55  	}
    56  	txn.Add(1)
    57  	err = txn.Mgr.OnOpTxn(&OpTxn{
    58  		Txn: txn,
    59  		Op:  OpCommit,
    60  	})
    61  	// TxnManager is closed
    62  	if err != nil {
    63  		txn.SetError(err)
    64  		txn.Lock()
    65  		_ = txn.ToRollbackingLocked(txn.GetStartTS().Next())
    66  		txn.Unlock()
    67  		_ = txn.PrepareRollback()
    68  		_ = txn.ApplyRollback()
    69  		txn.DoneWithErr(err, true)
    70  	}
    71  	txn.Wait()
    72  	//if txn.Err == nil {
    73  	//txn.Status = txnif.TxnStatusCommitted
    74  	//}
    75  	if err = txn.Mgr.DeleteTxn(txn.GetID()); err != nil {
    76  		return
    77  	}
    78  	return txn.GetError()
    79  }
    80  
    81  func (txn *Txn) rollback2PC() (err error) {
    82  	state := txn.GetTxnState(false)
    83  
    84  	switch state {
    85  	case txnif.TxnStateActive:
    86  		txn.Add(1)
    87  		err = txn.Mgr.OnOpTxn(&OpTxn{
    88  			Txn: txn,
    89  			Op:  OpRollback,
    90  		})
    91  		if err != nil {
    92  			_ = txn.PrepareRollback()
    93  			_ = txn.ApplyRollback()
    94  			_ = txn.ToRollbacking(txn.GetStartTS())
    95  			txn.DoneWithErr(err, true)
    96  		}
    97  		txn.Wait()
    98  
    99  	case txnif.TxnStatePrepared:
   100  		//Notice that at this moment, txn had already appended data into state machine, so
   101  		// we can not just delete the AppendNode from the MVCCHandle, instead ,we should
   102  		// set the state of the AppendNode to Abort to make reader perceive it .
   103  		_ = txn.ApplyRollback()
   104  		txn.DoneWithErr(nil, true)
   105  
   106  	default:
   107  		logutil.Warnf("unexpected txn state : %s", txnif.TxnStrState(state))
   108  		return moerr.NewTAERollbackNoCtx("unexpected txn status : %s", txnif.TxnStrState(state))
   109  	}
   110  
   111  	txn.Mgr.DeleteTxn(txn.GetID())
   112  
   113  	return txn.GetError()
   114  }
   115  
   116  func (txn *Txn) commit2PC(inRecovery bool) (err error) {
   117  	state := txn.GetTxnState(false)
   118  
   119  	switch state {
   120  	//It's a 2PC transaction running on Coordinator
   121  	case txnif.TxnStateCommittingFinished:
   122  		if err = txn.ApplyCommit(); err != nil {
   123  			panic(err)
   124  		}
   125  		txn.DoneWithErr(nil, false)
   126  
   127  		// Skip logging if in recovery
   128  		if !inRecovery {
   129  			//Append a committed log entry into log service asynchronously
   130  			//     for checkpointing the committing log entry
   131  			_, err = txn.LogTxnState(false)
   132  			if err != nil {
   133  				panic(err)
   134  			}
   135  		}
   136  
   137  	//It's a 2PC transaction running on Participant.
   138  	//Notice that Commit must be successful once the commit message arrives,
   139  	//since Committing had succeed.
   140  	case txnif.TxnStatePrepared:
   141  		if err = txn.ApplyCommit(); err != nil {
   142  			panic(err)
   143  		}
   144  		txn.DoneWithErr(nil, false)
   145  
   146  		// Skip logging if in recovery
   147  		if !inRecovery {
   148  			//Append committed log entry ,and wait it synced.
   149  			_, err = txn.LogTxnState(true)
   150  			if err != nil {
   151  				panic(err)
   152  			}
   153  		}
   154  
   155  	default:
   156  		logutil.Warnf("unexpected txn state : %s", txnif.TxnStrState(state))
   157  		return moerr.NewTAECommitNoCtx("invalid txn state %s", txnif.TxnStrState(state))
   158  	}
   159  	txn.Mgr.DeleteTxn(txn.GetID())
   160  
   161  	return txn.GetError()
   162  }
   163  
   164  func (txn *Txn) done1PCWithErr(err error) {
   165  	txn.DoneCond.L.Lock()
   166  	defer txn.DoneCond.L.Unlock()
   167  
   168  	if err != nil {
   169  		txn.ToUnknownLocked()
   170  		txn.SetError(err)
   171  	} else {
   172  		if txn.State == txnif.TxnStatePreparing {
   173  			if err := txn.ToCommittedLocked(); err != nil {
   174  				txn.SetError(err)
   175  			}
   176  		} else {
   177  			if err := txn.ToRollbackedLocked(); err != nil {
   178  				txn.SetError(err)
   179  			}
   180  		}
   181  	}
   182  	txn.WaitGroup.Done()
   183  	txn.DoneCond.Broadcast()
   184  }
   185  
   186  func (txn *Txn) done2PCWithErr(err error, isAbort bool) {
   187  	txn.DoneCond.L.Lock()
   188  	defer txn.DoneCond.L.Unlock()
   189  
   190  	endOfTxn := true
   191  	done := true
   192  
   193  	if err != nil {
   194  		txn.ToUnknownLocked()
   195  		txn.SetError(err)
   196  	} else {
   197  		switch txn.State {
   198  		case txnif.TxnStateRollbacking:
   199  			if err = txn.ToRollbackedLocked(); err != nil {
   200  				panic(err)
   201  			}
   202  		case txnif.TxnStatePreparing:
   203  			endOfTxn = false
   204  			if err = txn.ToPreparedLocked(); err != nil {
   205  				panic(err)
   206  			}
   207  		case txnif.TxnStateCommittingFinished:
   208  			done = false
   209  			if err = txn.ToCommittedLocked(); err != nil {
   210  				panic(err)
   211  			}
   212  		case txnif.TxnStatePrepared:
   213  			done = false
   214  			if isAbort {
   215  				if err = txn.ToRollbackedLocked(); err != nil {
   216  					panic(err)
   217  				}
   218  			} else {
   219  				if err = txn.ToCommittedLocked(); err != nil {
   220  					panic(err)
   221  				}
   222  			}
   223  		}
   224  	}
   225  	if done {
   226  		txn.WaitGroup.Done()
   227  	}
   228  
   229  	if endOfTxn {
   230  		txn.DoneCond.Broadcast()
   231  	}
   232  }