github.com/matrixorigin/matrixone@v0.7.0/pkg/txn/service/service_cn_handler.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  	"math"
    21  	"time"
    22  
    23  	"github.com/matrixorigin/matrixone/pkg/common/moerr"
    24  	"github.com/matrixorigin/matrixone/pkg/pb/timestamp"
    25  	"github.com/matrixorigin/matrixone/pkg/pb/txn"
    26  	"github.com/matrixorigin/matrixone/pkg/txn/util"
    27  	"go.uber.org/zap"
    28  )
    29  
    30  var (
    31  	rollbackIngoreErrorCodes = map[uint16]struct{}{
    32  		moerr.ErrTxnNotFound: {},
    33  	}
    34  
    35  	prepareIngoreErrorCodes = map[uint16]struct{}{
    36  		moerr.ErrTxnNotFound: {},
    37  	}
    38  )
    39  
    40  func (s *service) Read(ctx context.Context, request *txn.TxnRequest, response *txn.TxnResponse) error {
    41  	s.waitRecoveryCompleted()
    42  
    43  	util.LogTxnHandleRequest(s.logger, request)
    44  	defer util.LogTxnHandleResult(s.logger, response)
    45  
    46  	response.CNOpResponse = &txn.CNOpResponse{}
    47  	s.checkCNRequest(request)
    48  	if !s.validDNShard(request.GetTargetDN()) {
    49  		response.TxnError = txn.WrapError(moerr.NewDNShardNotFound(ctx, "", request.GetTargetDN().ShardID), 0)
    50  		return nil
    51  	}
    52  
    53  	s.waitClockTo(request.Txn.SnapshotTS)
    54  
    55  	// We do not write transaction information to sync.Map during read operations because commit and abort
    56  	// for read-only transactions are not sent to the DN node, so there is no way to clean up the transaction
    57  	// information in sync.Map.
    58  	result, err := s.storage.Read(ctx, request.Txn, request.CNRequest.OpCode, request.CNRequest.Payload)
    59  	if err != nil {
    60  		util.LogTxnReadFailed(s.logger, request.Txn, err)
    61  		response.TxnError = txn.WrapError(err, moerr.ErrTAERead)
    62  		return nil
    63  	}
    64  	defer result.Release()
    65  
    66  	if len(result.WaitTxns()) > 0 {
    67  		util.LogTxnReadBlockedByUncommittedTxns(s.logger, request.Txn, result.WaitTxns())
    68  		waiters := make([]*waiter, 0, len(result.WaitTxns()))
    69  		for _, txnID := range result.WaitTxns() {
    70  			txnCtx := s.getTxnContext(txnID)
    71  			// The transaction can not found, it means the concurrent transaction to be waited for has already
    72  			// been committed or aborted.
    73  			if txnCtx == nil {
    74  				continue
    75  			}
    76  
    77  			w := acquireWaiter()
    78  			// txn has been committed or aborted between call s.getTxnContext and txnCtx.addWaiter
    79  			if !txnCtx.addWaiter(txnID, w, txn.TxnStatus_Committed) {
    80  				w.close()
    81  				continue
    82  			}
    83  
    84  			waiters = append(waiters, w)
    85  		}
    86  
    87  		for _, w := range waiters {
    88  			if err != nil {
    89  				w.close()
    90  				continue
    91  			}
    92  
    93  			// If no error occurs, then it must have waited until the final state of the transaction, not caring
    94  			// whether the final state is committed or aborted.
    95  			_, err = w.wait(ctx)
    96  			w.close()
    97  		}
    98  
    99  		if err != nil {
   100  			util.LogTxnWaitUncommittedTxnsFailed(s.logger, request.Txn, result.WaitTxns(), err)
   101  			response.TxnError = txn.WrapError(err, moerr.ErrWaitTxn)
   102  			return nil
   103  		}
   104  	}
   105  
   106  	data, err := result.Read()
   107  	if err != nil {
   108  		util.LogTxnReadFailed(s.logger, request.Txn, err)
   109  		response.TxnError = txn.WrapError(err, moerr.ErrTAERead)
   110  		return nil
   111  	}
   112  
   113  	response.CNOpResponse.Payload = data
   114  	txnMeta := request.Txn
   115  	response.Txn = &txnMeta
   116  	return nil
   117  }
   118  
   119  func (s *service) Write(ctx context.Context, request *txn.TxnRequest, response *txn.TxnResponse) error {
   120  	s.waitRecoveryCompleted()
   121  
   122  	util.LogTxnHandleRequest(s.logger, request)
   123  	defer util.LogTxnHandleResult(s.logger, response)
   124  
   125  	response.CNOpResponse = &txn.CNOpResponse{}
   126  	s.checkCNRequest(request)
   127  	if !s.validDNShard(request.GetTargetDN()) {
   128  		response.TxnError = txn.WrapError(moerr.NewDNShardNotFound(ctx, "", request.GetTargetDN().ShardID), 0)
   129  		return nil
   130  	}
   131  
   132  	txnID := request.Txn.ID
   133  	txnCtx, _ := s.maybeAddTxn(request.Txn)
   134  
   135  	// only commit and rollback can held write Lock
   136  	if !txnCtx.mu.TryRLock() {
   137  		util.LogTxnNotFoundOn(s.logger, request.Txn, s.shard)
   138  		response.TxnError = txn.WrapError(moerr.NewDNShardNotFound(ctx, "", request.GetTargetDN().ShardID), 0)
   139  		return nil
   140  	}
   141  	defer txnCtx.mu.RUnlock()
   142  
   143  	newTxn := txnCtx.getTxnLocked()
   144  	if !bytes.Equal(newTxn.ID, txnID) {
   145  		util.LogTxnNotFoundOn(s.logger, request.Txn, s.shard)
   146  		response.TxnError = txn.WrapError(moerr.NewTxnNotFound(ctx), 0)
   147  		return nil
   148  	}
   149  
   150  	response.Txn = &newTxn
   151  	if newTxn.Status != txn.TxnStatus_Active {
   152  		util.LogTxnWriteOnInvalidStatus(s.logger, newTxn)
   153  		response.TxnError = txn.WrapError(moerr.NewTxnNotActive(ctx, ""), 0)
   154  		return nil
   155  	}
   156  
   157  	data, err := s.storage.Write(ctx, request.Txn, request.CNRequest.OpCode, request.CNRequest.Payload)
   158  	if err != nil {
   159  		util.LogTxnWriteFailed(s.logger, newTxn, err)
   160  		response.TxnError = txn.WrapError(err, moerr.ErrTAEWrite)
   161  		return nil
   162  	}
   163  
   164  	response.CNOpResponse.Payload = data
   165  	return nil
   166  }
   167  
   168  func (s *service) Commit(ctx context.Context, request *txn.TxnRequest, response *txn.TxnResponse) error {
   169  	s.waitRecoveryCompleted()
   170  
   171  	util.LogTxnHandleRequest(s.logger, request)
   172  	defer util.LogTxnHandleResult(s.logger, response)
   173  
   174  	response.CommitResponse = &txn.TxnCommitResponse{}
   175  	if !s.validDNShard(request.GetTargetDN()) {
   176  		response.TxnError = txn.WrapError(moerr.NewDNShardNotFound(ctx, "", request.GetTargetDN().ShardID), 0)
   177  		return nil
   178  	}
   179  
   180  	if len(request.Txn.DNShards) == 0 {
   181  		s.logger.Fatal("commit with empty dn shards")
   182  	}
   183  
   184  	txnID := request.Txn.ID
   185  	txnCtx := s.getTxnContext(txnID)
   186  	if txnCtx == nil {
   187  		util.LogTxnNotFoundOn(s.logger, request.Txn, s.shard)
   188  		response.TxnError = txn.WrapError(moerr.NewDNShardNotFound(ctx, "", request.GetTargetDN().ShardID), 0)
   189  		return nil
   190  	}
   191  
   192  	// block all other concurrent read and write operations.
   193  	txnCtx.mu.Lock()
   194  	defer txnCtx.mu.Unlock()
   195  
   196  	newTxn := txnCtx.getTxnLocked()
   197  	if !bytes.Equal(newTxn.ID, txnID) {
   198  		util.LogTxnNotFoundOn(s.logger, request.Txn, s.shard)
   199  		response.TxnError = txn.WrapError(moerr.NewDNShardNotFound(ctx, "", request.GetTargetDN().ShardID), 0)
   200  		return nil
   201  	}
   202  
   203  	cleanTxnContext := true
   204  	defer func() {
   205  		// remove txnCtx, commit can only execute once.
   206  		s.removeTxn(txnID)
   207  		if cleanTxnContext {
   208  			s.releaseTxnContext(txnCtx)
   209  		}
   210  	}()
   211  
   212  	response.Txn = &newTxn
   213  	if newTxn.Status != txn.TxnStatus_Active {
   214  		util.LogTxnCommitOnInvalidStatus(s.logger, newTxn)
   215  		response.TxnError = txn.WrapError(moerr.NewTxnNotActive(ctx, ""), 0)
   216  		return nil
   217  	}
   218  
   219  	newTxn.DNShards = request.Txn.DNShards
   220  	changeStatus := func(status txn.TxnStatus) {
   221  		newTxn.Status = status
   222  		txnCtx.changeStatusLocked(status)
   223  	}
   224  
   225  	// fast path: write in only one DNShard.
   226  	if len(newTxn.DNShards) == 1 {
   227  		newTxn.CommitTS, _ = s.rt.Clock().Now()
   228  		txnCtx.updateTxnLocked(newTxn)
   229  
   230  		util.LogTxnStart1PCCommit(s.logger, newTxn)
   231  		if err := s.storage.Commit(ctx, newTxn); err != nil {
   232  			util.LogTxnStart1PCCommitFailed(s.logger, newTxn, err)
   233  			response.TxnError = txn.WrapError(err, moerr.ErrTAECommit)
   234  			changeStatus(txn.TxnStatus_Aborted)
   235  		} else {
   236  			changeStatus(txn.TxnStatus_Committed)
   237  			util.LogTxn1PCCommitCompleted(s.logger, newTxn)
   238  		}
   239  		return nil
   240  	}
   241  
   242  	util.LogTxnStart2PCCommit(s.logger, newTxn)
   243  
   244  	// slow path. 2pc transaction.
   245  	// 1. send prepare request to all DNShards.
   246  	// 2. start async commit task if all prepare succeed.
   247  	// 3. response to client txn committed.
   248  	for _, dn := range newTxn.DNShards {
   249  		txnCtx.mu.requests = append(txnCtx.mu.requests, txn.TxnRequest{
   250  			Txn:            newTxn,
   251  			Method:         txn.TxnMethod_Prepare,
   252  			PrepareRequest: &txn.TxnPrepareRequest{DNShard: dn},
   253  		})
   254  	}
   255  
   256  	// unlock and lock here, because the prepare request will be sent to the current TxnService, it
   257  	// will need to get the Lock when processing the Prepare.
   258  	txnCtx.mu.Unlock()
   259  	// FIXME: txnCtx.mu.requests without lock, is it safe?
   260  	util.LogTxnSendRequests(s.logger, txnCtx.mu.requests)
   261  	result, err := s.sender.Send(ctx, txnCtx.mu.requests)
   262  	txnCtx.mu.Lock()
   263  	if err != nil {
   264  		util.LogTxnParallelPrepareFailed(s.logger, newTxn, err)
   265  
   266  		changeStatus(txn.TxnStatus_Aborted)
   267  		response.TxnError = txn.WrapError(moerr.NewRpcError(ctx, err.Error()), 0)
   268  		s.startAsyncRollbackTask(newTxn)
   269  		return nil
   270  	}
   271  
   272  	defer result.Release()
   273  
   274  	// get latest txn metadata
   275  	newTxn = txnCtx.getTxnLocked()
   276  	newTxn.CommitTS = newTxn.PreparedTS
   277  
   278  	hasError := false
   279  	var txnErr *txn.TxnError
   280  	for idx, resp := range result.Responses {
   281  		if resp.TxnError != nil {
   282  			txnErr = resp.TxnError
   283  			hasError = true
   284  			util.LogTxnPrepareFailedOn(s.logger, newTxn, newTxn.DNShards[idx], txnErr)
   285  			continue
   286  		}
   287  
   288  		if resp.Txn.PreparedTS.IsEmpty() {
   289  			s.logger.Fatal("missing prepared timestamp",
   290  				zap.String("target-dn-shard", newTxn.DNShards[idx].DebugString()),
   291  				util.TxnIDFieldWithID(newTxn.ID))
   292  		}
   293  
   294  		util.LogTxnPrepareCompletedOn(s.logger, newTxn, newTxn.DNShards[idx], resp.Txn.PreparedTS)
   295  		if newTxn.CommitTS.Less(resp.Txn.PreparedTS) {
   296  			newTxn.CommitTS = resp.Txn.PreparedTS
   297  		}
   298  	}
   299  	if hasError {
   300  		changeStatus(txn.TxnStatus_Aborted)
   301  		response.TxnError = txnErr
   302  		s.startAsyncRollbackTask(newTxn)
   303  		return nil
   304  	}
   305  
   306  	util.LogTxnParallelPrepareCompleted(s.logger, newTxn)
   307  
   308  	// All DNShards prepared means the transaction is committed
   309  	cleanTxnContext = false
   310  	txnCtx.updateTxnLocked(newTxn)
   311  	return s.startAsyncCommitTask(txnCtx)
   312  }
   313  
   314  func (s *service) Rollback(ctx context.Context, request *txn.TxnRequest, response *txn.TxnResponse) error {
   315  	s.waitRecoveryCompleted()
   316  
   317  	util.LogTxnHandleRequest(s.logger, request)
   318  	defer util.LogTxnHandleResult(s.logger, response)
   319  
   320  	response.RollbackResponse = &txn.TxnRollbackResponse{}
   321  	if !s.validDNShard(request.GetTargetDN()) {
   322  		response.TxnError = txn.WrapError(moerr.NewDNShardNotFound(ctx, "", request.GetTargetDN().ShardID), 0)
   323  		return nil
   324  	}
   325  
   326  	if len(request.Txn.DNShards) == 0 {
   327  		s.logger.Fatal("rollback with empty dn shards")
   328  	}
   329  
   330  	txnID := request.Txn.ID
   331  	txnCtx := s.getTxnContext(txnID)
   332  	if txnCtx == nil {
   333  		util.LogTxnNotFoundOn(s.logger, request.Txn, s.shard)
   334  		response.TxnError = txn.WrapError(moerr.NewTxnNotFound(ctx), 0)
   335  		return nil
   336  	}
   337  
   338  	txnCtx.mu.Lock()
   339  	defer txnCtx.mu.Unlock()
   340  
   341  	newTxn := txnCtx.getTxnLocked()
   342  	if !bytes.Equal(newTxn.ID, txnID) {
   343  		util.LogTxnNotFoundOn(s.logger, request.Txn, s.shard)
   344  		response.TxnError = txn.WrapError(moerr.NewTxnNotFound(ctx), 0)
   345  		return nil
   346  	}
   347  
   348  	response.Txn = &newTxn
   349  	newTxn.DNShards = request.Txn.DNShards
   350  	s.startAsyncRollbackTask(newTxn)
   351  
   352  	response.Txn.Status = txn.TxnStatus_Aborted
   353  	return nil
   354  }
   355  
   356  func (s *service) startAsyncRollbackTask(txnMeta txn.TxnMeta) {
   357  	err := s.stopper.RunTask(func(ctx context.Context) {
   358  		util.LogTxnStartAsyncRollback(s.logger, txnMeta)
   359  
   360  		requests := make([]txn.TxnRequest, 0, len(txnMeta.DNShards))
   361  		for _, dn := range txnMeta.DNShards {
   362  			requests = append(requests, txn.TxnRequest{
   363  				Txn:                    txnMeta,
   364  				Method:                 txn.TxnMethod_RollbackDNShard,
   365  				RollbackDNShardRequest: &txn.TxnRollbackDNShardRequest{DNShard: dn},
   366  			})
   367  		}
   368  
   369  		s.parallelSendWithRetry(ctx, txnMeta, requests, rollbackIngoreErrorCodes)
   370  		util.LogTxnRollbackCompleted(s.logger, txnMeta)
   371  	})
   372  	if err != nil {
   373  		s.logger.Error("start rollback task failed",
   374  			zap.Error(err),
   375  			util.TxnIDFieldWithID(txnMeta.ID))
   376  	}
   377  }
   378  
   379  func (s *service) Debug(ctx context.Context, request *txn.TxnRequest, response *txn.TxnResponse) error {
   380  	data, err := s.storage.Debug(ctx, request.Txn, request.CNRequest.OpCode, request.CNRequest.Payload)
   381  	if err != nil {
   382  		response.TxnError = txn.WrapError(err, moerr.ErrTAEDebug)
   383  		return nil
   384  	}
   385  	response.CNOpResponse = &txn.CNOpResponse{
   386  		Payload: data,
   387  	}
   388  	return nil
   389  }
   390  
   391  func (s *service) startAsyncCommitTask(txnCtx *txnContext) error {
   392  	return s.stopper.RunTask(func(ctx context.Context) {
   393  		txnCtx.mu.Lock()
   394  		defer txnCtx.mu.Unlock()
   395  
   396  		txnMeta := txnCtx.getTxnLocked()
   397  		util.LogTxnStartAsyncCommit(s.logger, txnMeta)
   398  
   399  		if txnMeta.Status != txn.TxnStatus_Committing {
   400  			for {
   401  				err := s.storage.Committing(ctx, txnMeta)
   402  				if err == nil {
   403  					txnCtx.changeStatusLocked(txn.TxnStatus_Committing)
   404  					break
   405  				}
   406  				util.LogTxnCommittingFailed(s.logger, txnMeta, err)
   407  				// TODO: make config
   408  				time.Sleep(time.Second)
   409  			}
   410  		}
   411  
   412  		util.LogTxnCommittingCompleted(s.logger, txnMeta)
   413  
   414  		requests := make([]txn.TxnRequest, 0, len(txnMeta.DNShards)-1)
   415  		for _, dn := range txnMeta.DNShards[1:] {
   416  			requests = append(requests, txn.TxnRequest{
   417  				Txn:                  txnMeta,
   418  				Method:               txn.TxnMethod_CommitDNShard,
   419  				CommitDNShardRequest: &txn.TxnCommitDNShardRequest{DNShard: dn},
   420  			})
   421  		}
   422  
   423  		// no timeout, keep retry until TxnService.Close
   424  		ctx, cancel := context.WithTimeout(ctx, time.Duration(math.MaxInt64))
   425  		defer cancel()
   426  
   427  		if result := s.parallelSendWithRetry(ctx, txnMeta, requests, rollbackIngoreErrorCodes); result != nil {
   428  			result.Release()
   429  			if s.logger.Enabled(zap.DebugLevel) {
   430  				s.logger.Debug("other dnshards committed",
   431  					util.TxnIDFieldWithID(txnMeta.ID))
   432  			}
   433  
   434  			if err := s.storage.Commit(ctx, txnMeta); err != nil {
   435  				s.logger.Fatal("commit failed after prepared",
   436  					util.TxnIDFieldWithID(txnMeta.ID),
   437  					zap.Error(err))
   438  			}
   439  
   440  			if s.logger.Enabled(zap.DebugLevel) {
   441  				s.logger.Debug("coordinator dnshard committed, txn committed",
   442  					util.TxnIDFieldWithID(txnMeta.ID))
   443  			}
   444  
   445  			txnCtx.changeStatusLocked(txn.TxnStatus_Committed)
   446  			s.releaseTxnContext(txnCtx)
   447  		}
   448  	})
   449  }
   450  
   451  func (s *service) checkCNRequest(request *txn.TxnRequest) {
   452  	if request.CNRequest == nil {
   453  		s.logger.Fatal("missing CNRequest")
   454  	}
   455  }
   456  
   457  func (s *service) waitClockTo(ts timestamp.Timestamp) {
   458  	for {
   459  		now, _ := s.rt.Clock().Now()
   460  		if now.GreaterEq(ts) {
   461  			return
   462  		}
   463  		time.Sleep(time.Duration(ts.PhysicalTime + 1 - now.PhysicalTime))
   464  	}
   465  }