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