From 633dcf458b0c073728178ecec1cc5ab568cf2648 Mon Sep 17 00:00:00 2001 From: amit-momin Date: Wed, 12 Jun 2024 18:14:11 -0500 Subject: [PATCH 01/23] Added a finalizer component that assesses confirmed transactions for finality --- .changeset/itchy-bugs-clean.md | 5 + common/txmgr/confirmer.go | 5 +- common/txmgr/finalizer.go | 235 ++++++++++++++++++ common/txmgr/test_helpers.go | 2 +- common/txmgr/txmgr.go | 18 +- common/txmgr/types/client.go | 2 + common/txmgr/types/mocks/tx_store.go | 76 +++--- common/txmgr/types/tx.go | 12 + common/txmgr/types/tx_store.go | 3 +- core/chains/evm/txmgr/builder.go | 15 +- core/chains/evm/txmgr/client.go | 4 + core/chains/evm/txmgr/confirmer_test.go | 44 ++++ core/chains/evm/txmgr/evm_tx_store.go | 61 +++-- core/chains/evm/txmgr/evm_tx_store_test.go | 104 ++++++-- core/chains/evm/txmgr/finalizer_test.go | 160 ++++++++++++ core/chains/evm/txmgr/mocks/evm_tx_store.go | 76 +++--- core/chains/evm/txmgr/models.go | 3 +- core/chains/evm/txmgr/txmgr_test.go | 43 +++- core/services/vrf/v2/integration_v2_test.go | 2 +- core/services/vrf/v2/listener_v2_test.go | 2 +- .../0245_add_tx_finalized_column.sql | 15 ++ core/web/testdata/body/health.html | 3 + core/web/testdata/body/health.json | 9 + core/web/testdata/body/health.txt | 1 + testdata/scripts/health/multi-chain.txtar | 10 + 25 files changed, 795 insertions(+), 115 deletions(-) create mode 100644 .changeset/itchy-bugs-clean.md create mode 100644 common/txmgr/finalizer.go create mode 100644 core/chains/evm/txmgr/finalizer_test.go create mode 100644 core/store/migrate/migrations/0245_add_tx_finalized_column.sql diff --git a/.changeset/itchy-bugs-clean.md b/.changeset/itchy-bugs-clean.md new file mode 100644 index 00000000000..a09117f4ed9 --- /dev/null +++ b/.changeset/itchy-bugs-clean.md @@ -0,0 +1,5 @@ +--- +"chainlink": minor +--- + +Added a finalizer component to the TXM to mark transactions as finalized #internal diff --git a/common/txmgr/confirmer.go b/common/txmgr/confirmer.go index a9e30ffff1e..2cdc48bea19 100644 --- a/common/txmgr/confirmer.go +++ b/common/txmgr/confirmer.go @@ -120,7 +120,7 @@ type Confirmer[ services.StateMachine txStore txmgrtypes.TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] lggr logger.SugaredLogger - client txmgrtypes.TxmClient[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE] + client txmgrtypes.TxmClient[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE, HEAD] txmgrtypes.TxAttemptBuilder[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] stuckTxDetector txmgrtypes.StuckTxDetector[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] resumeCallback ResumeCallback @@ -154,7 +154,7 @@ func NewConfirmer[ FEE feetypes.Fee, ]( txStore txmgrtypes.TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE], - client txmgrtypes.TxmClient[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE], + client txmgrtypes.TxmClient[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE, HEAD], chainConfig txmgrtypes.ConfirmerChainConfig, feeConfig txmgrtypes.ConfirmerFeeConfig, txConfig txmgrtypes.ConfirmerTransactionsConfig, @@ -1100,6 +1100,7 @@ func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) mar "txID", etx.ID, "attemptID", attempt.ID, "nReceipts", len(attempt.Receipts), + "finalized", etx.Finalized, "id", "confirmer", } diff --git a/common/txmgr/finalizer.go b/common/txmgr/finalizer.go new file mode 100644 index 00000000000..dc7bc66db8f --- /dev/null +++ b/common/txmgr/finalizer.go @@ -0,0 +1,235 @@ +package txmgr + +import ( + "context" + "errors" + "fmt" + "math/big" + "sync" + + "github.com/smartcontractkit/chainlink-common/pkg/logger" + "github.com/smartcontractkit/chainlink-common/pkg/services" + "github.com/smartcontractkit/chainlink-common/pkg/utils/mailbox" + feetypes "github.com/smartcontractkit/chainlink/v2/common/fee/types" + txmgrtypes "github.com/smartcontractkit/chainlink/v2/common/txmgr/types" + "github.com/smartcontractkit/chainlink/v2/common/types" +) + +type finalizerTxStore[CHAIN_ID types.ID, ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, SEQ types.Sequence, FEE feetypes.Fee] interface { + FindTransactionsByState(ctx context.Context, state txmgrtypes.TxState, chainID CHAIN_ID) ([]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error) + UpdateTxesFinalized(ctx context.Context, txs []int64, chainId CHAIN_ID) error +} + +type finalizerChainClient[BLOCK_HASH types.Hashable, HEAD types.Head[BLOCK_HASH]] interface { + HeadByHash(ctx context.Context, hash BLOCK_HASH) (HEAD, error) +} + +// Finalizer handles processing new finalized blocks and marking transactions as finalized accordingly in the TXM DB +type Finalizer[CHAIN_ID types.ID, ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, SEQ types.Sequence, FEE feetypes.Fee, HEAD types.Head[BLOCK_HASH]] struct { + services.StateMachine + lggr logger.SugaredLogger + chainId CHAIN_ID + txStore finalizerTxStore[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] + client finalizerChainClient[BLOCK_HASH, HEAD] + mb *mailbox.Mailbox[HEAD] + stopCh services.StopChan + wg sync.WaitGroup + initSync sync.Mutex + isStarted bool +} + +func NewFinalizer[ + CHAIN_ID types.ID, + ADDR types.Hashable, + TX_HASH types.Hashable, + BLOCK_HASH types.Hashable, + SEQ types.Sequence, + FEE feetypes.Fee, + HEAD types.Head[BLOCK_HASH], +]( + lggr logger.Logger, + chainId CHAIN_ID, + txStore finalizerTxStore[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], + client finalizerChainClient[BLOCK_HASH, HEAD], +) *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD] { + lggr = logger.Named(lggr, "Finalizer") + return &Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD]{ + txStore: txStore, + lggr: logger.Sugared(lggr), + chainId: chainId, + client: client, + mb: mailbox.NewSingle[HEAD](), + } +} + +// Start is a comment to appease the linter +func (f *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD]) Start(ctx context.Context) error { + return f.StartOnce("Finalizer", func() error { + return f.startInternal(ctx) + }) +} + +func (f *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD]) startInternal(_ context.Context) error { + f.initSync.Lock() + defer f.initSync.Unlock() + if f.isStarted { + return errors.New("Finalizer is already started") + } + + f.stopCh = make(chan struct{}) + f.wg = sync.WaitGroup{} + f.wg.Add(1) + go f.runLoop() + f.isStarted = true + return nil +} + +// Close is a comment to appease the linter +func (f *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD]) Close() error { + return f.StopOnce("Finalizer", func() error { + return f.closeInternal() + }) +} + +func (f *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD]) closeInternal() error { + f.initSync.Lock() + defer f.initSync.Unlock() + if !f.isStarted { + return fmt.Errorf("Finalizer is not started: %w", services.ErrAlreadyStopped) + } + close(f.stopCh) + f.wg.Wait() + f.isStarted = false + return nil +} + +func (f *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD]) Name() string { + return f.lggr.Name() +} + +func (f *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD]) HealthReport() map[string]error { + return map[string]error{f.Name(): f.Healthy()} +} + +func (f *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD]) runLoop() { + defer f.wg.Done() + ctx, cancel := f.stopCh.NewCtx() + defer cancel() + for { + select { + case <-f.mb.Notify(): + for { + if ctx.Err() != nil { + return + } + head, exists := f.mb.Retrieve() + if !exists { + break + } + if err := f.ProcessHead(ctx, head); err != nil { + f.lggr.Errorw("Error processing head", "err", err) + f.SvcErrBuffer.Append(err) + continue + } + } + case <-ctx.Done(): + return + } + } +} + +func (f *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD]) ProcessHead(ctx context.Context, head types.Head[BLOCK_HASH]) error { + ctx, cancel := context.WithTimeout(ctx, processHeadTimeout) + defer cancel() + return f.processHead(ctx, head) +} + +// Determines if any confirmed transactions can be marked as finalized by comparing their receipts against the latest finalized block +func (f *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD]) processHead(ctx context.Context, head types.Head[BLOCK_HASH]) error { + latestFinalizedHead := head.LatestFinalizedHead() + // Cannot determine finality without a finalized head for comparison + if latestFinalizedHead == nil || !latestFinalizedHead.IsValid() { + return fmt.Errorf("failed to find latest finalized head in chain") + } + earliestBlockNumInChain := latestFinalizedHead.EarliestHeadInChain().BlockNumber() + f.lggr.Debugw("processing latest finalized head", "block num", latestFinalizedHead.BlockNumber(), "block hash", latestFinalizedHead.BlockHash(), "earliest block num in chain", earliestBlockNumInChain) + + // Retrieve all confirmed transactions, loaded with attempts and receipts + confirmedTxs, err := f.txStore.FindTransactionsByState(ctx, TxConfirmed, f.chainId) + if err != nil { + return fmt.Errorf("failed to retrieve confirmed transactions: %w", err) + } + + var finalizedTxs []*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] + // Group by block hash transactions whose receipts cannot be validated using the cached heads + receiptBlockHashToTx := make(map[BLOCK_HASH][]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) + // Find transactions with receipt block nums older than the latest finalized block num and block hashes still in chain + for _, tx := range confirmedTxs { + // Only consider transactions not already marked as finalized + if tx.Finalized { + continue + } + receipt := tx.GetReceipt() + if receipt == nil || receipt.IsZero() || receipt.IsUnmined() { + continue + } + // Receipt newer than latest finalized head block num + if receipt.GetBlockNumber().Cmp(big.NewInt(latestFinalizedHead.BlockNumber())) > 0 { + continue + } + // Receipt block num older than earliest head in chain. Validate hash using RPC call later + if receipt.GetBlockNumber().Int64() < earliestBlockNumInChain { + receiptBlockHashToTx[receipt.GetBlockHash()] = append(receiptBlockHashToTx[receipt.GetBlockHash()], tx) + continue + } + blockHashInChain := latestFinalizedHead.HashAtHeight(receipt.GetBlockNumber().Int64()) + // Receipt block hash does not match the block hash in chain. Transaction has been re-org'd out but DB state has not been updated yet + if blockHashInChain.String() != receipt.GetBlockHash().String() { + continue + } + finalizedTxs = append(finalizedTxs, tx) + } + + // Check if block hashes exist for receipts on-chain older than the earliest cached head + // Transactions are grouped by their receipt block hash to minimize the number of RPC calls in case transactions were confirmed in the same block + // This check is only expected to be used in rare cases if there was an issue with the HeadTracker or if the node was down for significant time + var wg sync.WaitGroup + var txMu sync.RWMutex + for receiptBlockHash, txs := range receiptBlockHashToTx { + wg.Add(1) + go func(hash BLOCK_HASH, txs []*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) { + defer wg.Done() + if head, rpcErr := f.client.HeadByHash(ctx, hash); rpcErr == nil && head.IsValid() { + txMu.Lock() + finalizedTxs = append(finalizedTxs, txs...) + txMu.Unlock() + } + }(receiptBlockHash, txs) + } + wg.Wait() + + etxIDs := f.buildTxIdList(finalizedTxs) + + err = f.txStore.UpdateTxesFinalized(ctx, etxIDs, f.chainId) + if err != nil { + return fmt.Errorf("failed to update transactions as finalized: %w", err) + } + return nil +} + +// Build list of transaction IDs +func (f *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD]) buildTxIdList(finalizedTxs []*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) []int64 { + etxIDs := make([]int64, len(finalizedTxs)) + for i, tx := range finalizedTxs { + receipt := tx.GetReceipt() + f.lggr.Debugw("transaction considered finalized", + "sequence", tx.Sequence, + "fromAddress", tx.FromAddress.String(), + "txHash", receipt.GetTxHash().String(), + "receiptBlockNum", receipt.GetBlockNumber().Int64(), + "receiptBlockHash", receipt.GetBlockHash().String(), + ) + etxIDs[i] = tx.ID + } + return etxIDs +} diff --git a/common/txmgr/test_helpers.go b/common/txmgr/test_helpers.go index 3051e0985d8..ef3866815d5 100644 --- a/common/txmgr/test_helpers.go +++ b/common/txmgr/test_helpers.go @@ -10,7 +10,7 @@ import ( // TEST ONLY FUNCTIONS // these need to be exported for the txmgr tests to continue to work -func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) XXXTestSetClient(client txmgrtypes.TxmClient[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) { +func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) XXXTestSetClient(client txmgrtypes.TxmClient[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE, HEAD]) { ec.client = client } diff --git a/common/txmgr/txmgr.go b/common/txmgr/txmgr.go index 3ac0d2e1d68..46d3011e5ff 100644 --- a/common/txmgr/txmgr.go +++ b/common/txmgr/txmgr.go @@ -110,6 +110,7 @@ type Txm[ broadcaster *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] confirmer *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE] tracker *Tracker[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE] + finalizer *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD] fwdMgr txmgrtypes.ForwarderManager[ADDR] txAttemptBuilder txmgrtypes.TxAttemptBuilder[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] newErrorClassifier NewErrorClassifier @@ -145,6 +146,7 @@ func NewTxm[ confirmer *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE], resender *Resender[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE], tracker *Tracker[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE], + finalizer *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD], newErrorClassifierFunc NewErrorClassifier, ) *Txm[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { b := Txm[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]{ @@ -167,6 +169,7 @@ func NewTxm[ resender: resender, tracker: tracker, newErrorClassifier: newErrorClassifierFunc, + finalizer: finalizer, } if txCfg.ResendAfterThreshold() <= 0 { @@ -201,6 +204,10 @@ func (b *Txm[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) Start(ctx return fmt.Errorf("Txm: Tracker failed to start: %w", err) } + if err := ms.Start(ctx, b.finalizer); err != nil { + return fmt.Errorf("Txm: Finalizer failed to start: %w", err) + } + b.logger.Info("Txm starting runLoop") b.wg.Add(1) go b.runLoop() @@ -295,6 +302,7 @@ func (b *Txm[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) HealthRepo services.CopyHealth(report, b.broadcaster.HealthReport()) services.CopyHealth(report, b.confirmer.HealthReport()) services.CopyHealth(report, b.txAttemptBuilder.HealthReport()) + services.CopyHealth(report, b.finalizer.HealthReport()) }) if b.txConfig.ForwardersEnabled() { @@ -417,6 +425,7 @@ func (b *Txm[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) runLoop() case head := <-b.chHeads: b.confirmer.mb.Deliver(head) b.tracker.mb.Deliver(head.BlockNumber()) + b.finalizer.mb.Deliver(head) case reset := <-b.reset: // This check prevents the weird edge-case where you can select // into this block after chStop has already been closed and the @@ -448,6 +457,10 @@ func (b *Txm[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) runLoop() if err != nil && (!errors.Is(err, services.ErrAlreadyStopped) || !errors.Is(err, services.ErrCannotStopUnstarted)) { b.logger.Errorw(fmt.Sprintf("Failed to Close Tracker: %v", err), "err", err) } + err = b.finalizer.Close() + if err != nil && (!errors.Is(err, services.ErrAlreadyStopped) || !errors.Is(err, services.ErrCannotStopUnstarted)) { + b.logger.Errorw(fmt.Sprintf("Failed to Close Finalizer: %v", err), "err", err) + } return case <-keysChanged: // This check prevents the weird edge-case where you can select @@ -646,7 +659,10 @@ func (b *Txm[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) GetTransac // Return unconfirmed for ConfirmedMissingReceipt since a receipt is required to determine if it is finalized return commontypes.Unconfirmed, nil case TxConfirmed: - // TODO: Check for finality and return finalized status + if tx.Finalized { + // Return finalized if tx receipt's block is equal or older than the latest finalized block + return commontypes.Finalized, nil + } // Return unconfirmed if tx receipt's block is newer than the latest finalized block return commontypes.Unconfirmed, nil case TxFatalError: diff --git a/common/txmgr/types/client.go b/common/txmgr/types/client.go index 759b15d6162..ea3ed3af29c 100644 --- a/common/txmgr/types/client.go +++ b/common/txmgr/types/client.go @@ -21,6 +21,7 @@ type TxmClient[ R ChainReceipt[TX_HASH, BLOCK_HASH], SEQ types.Sequence, FEE feetypes.Fee, + HEAD types.Head[BLOCK_HASH], ] interface { ChainClient[CHAIN_ID, ADDR, SEQ] TransactionClient[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] @@ -30,6 +31,7 @@ type TxmClient[ ctx context.Context, attempts []TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], ) (txReceipt []R, txErr []error, err error) + HeadByHash(ctx context.Context, hash BLOCK_HASH) (HEAD, error) } // TransactionClient contains the methods for building, simulating, broadcasting transactions diff --git a/common/txmgr/types/mocks/tx_store.go b/common/txmgr/types/mocks/tx_store.go index 9c2edfa963f..349c714b59e 100644 --- a/common/txmgr/types/mocks/tx_store.go +++ b/common/txmgr/types/mocks/tx_store.go @@ -310,6 +310,36 @@ func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindNextUns return r0, r1 } +// FindTransactionsByState provides a mock function with given fields: ctx, state, chainID +func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTransactionsByState(ctx context.Context, state txmgrtypes.TxState, chainID CHAIN_ID) ([]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error) { + ret := _m.Called(ctx, state, chainID) + + if len(ret) == 0 { + panic("no return value specified for FindTransactionsByState") + } + + var r0 []*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, txmgrtypes.TxState, CHAIN_ID) ([]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error)); ok { + return rf(ctx, state, chainID) + } + if rf, ok := ret.Get(0).(func(context.Context, txmgrtypes.TxState, CHAIN_ID) []*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]); ok { + r0 = rf(ctx, state, chainID) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, txmgrtypes.TxState, CHAIN_ID) error); ok { + r1 = rf(ctx, state, chainID) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + // FindTransactionsConfirmedInBlockRange provides a mock function with given fields: ctx, highBlockNumber, lowBlockNumber, chainID func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTransactionsConfirmedInBlockRange(ctx context.Context, highBlockNumber int64, lowBlockNumber int64, chainID CHAIN_ID) ([]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error) { ret := _m.Called(ctx, highBlockNumber, lowBlockNumber, chainID) @@ -848,34 +878,6 @@ func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) HasInProgre return r0, r1 } -// IsTxFinalized provides a mock function with given fields: ctx, blockHeight, txID, chainID -func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) IsTxFinalized(ctx context.Context, blockHeight int64, txID int64, chainID CHAIN_ID) (bool, error) { - ret := _m.Called(ctx, blockHeight, txID, chainID) - - if len(ret) == 0 { - panic("no return value specified for IsTxFinalized") - } - - var r0 bool - var r1 error - if rf, ok := ret.Get(0).(func(context.Context, int64, int64, CHAIN_ID) (bool, error)); ok { - return rf(ctx, blockHeight, txID, chainID) - } - if rf, ok := ret.Get(0).(func(context.Context, int64, int64, CHAIN_ID) bool); ok { - r0 = rf(ctx, blockHeight, txID, chainID) - } else { - r0 = ret.Get(0).(bool) - } - - if rf, ok := ret.Get(1).(func(context.Context, int64, int64, CHAIN_ID) error); ok { - r1 = rf(ctx, blockHeight, txID, chainID) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - // LoadTxAttempts provides a mock function with given fields: ctx, etx func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) LoadTxAttempts(ctx context.Context, etx *txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) error { ret := _m.Called(ctx, etx) @@ -1230,6 +1232,24 @@ func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) UpdateTxUns return r0 } +// UpdateTxesFinalized provides a mock function with given fields: ctx, etxs, chainId +func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) UpdateTxesFinalized(ctx context.Context, etxs []int64, chainId CHAIN_ID) error { + ret := _m.Called(ctx, etxs, chainId) + + if len(ret) == 0 { + panic("no return value specified for UpdateTxesFinalized") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, []int64, CHAIN_ID) error); ok { + r0 = rf(ctx, etxs, chainId) + } else { + r0 = ret.Error(0) + } + + return r0 +} + // UpdateTxsUnconfirmed provides a mock function with given fields: ctx, ids func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) UpdateTxsUnconfirmed(ctx context.Context, ids []int64) error { ret := _m.Called(ctx, ids) diff --git a/common/txmgr/types/tx.go b/common/txmgr/types/tx.go index d2afbd209d8..72c13b450ff 100644 --- a/common/txmgr/types/tx.go +++ b/common/txmgr/types/tx.go @@ -215,6 +215,7 @@ type Tx[ InitialBroadcastAt *time.Time CreatedAt time.Time State TxState + Finalized bool TxAttempts []TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] `json:"-"` // Marshalled TxMeta // Used for additional context around transactions which you want to log @@ -340,6 +341,17 @@ func (e *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) GetChecker() (Transm return t, nil } +// Returns the transaction's receipt if one exists, otherwise returns nil +func (e *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) GetReceipt() ChainReceipt[TX_HASH, BLOCK_HASH] { + for _, attempt := range e.TxAttempts { + if len(attempt.Receipts) > 0 { + // Transaction will only have one receipt + return attempt.Receipts[0] + } + } + return nil +} + // Provides error classification to external components in a chain agnostic way // Only exposes the error types that could be set in the transaction error field type ErrorClassifier interface { diff --git a/common/txmgr/types/tx_store.go b/common/txmgr/types/tx_store.go index c102fb5912a..495a6d7fbae 100644 --- a/common/txmgr/types/tx_store.go +++ b/common/txmgr/types/tx_store.go @@ -84,6 +84,7 @@ type TransactionStore[ FindTransactionsConfirmedInBlockRange(ctx context.Context, highBlockNumber, lowBlockNumber int64, chainID CHAIN_ID) (etxs []*Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) FindEarliestUnconfirmedBroadcastTime(ctx context.Context, chainID CHAIN_ID) (null.Time, error) FindEarliestUnconfirmedTxAttemptBlock(ctx context.Context, chainID CHAIN_ID) (null.Int, error) + FindTransactionsByState(ctx context.Context, state TxState, chainID CHAIN_ID) (etxs []*Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) GetTxInProgress(ctx context.Context, fromAddress ADDR) (etx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) GetInProgressTxAttempts(ctx context.Context, address ADDR, chainID CHAIN_ID) (attempts []TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) GetAbandonedTransactionsByBatch(ctx context.Context, chainID CHAIN_ID, enabledAddrs []ADDR, offset, limit uint) (txs []*Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) @@ -106,8 +107,8 @@ type TransactionStore[ UpdateTxsUnconfirmed(ctx context.Context, ids []int64) error UpdateTxUnstartedToInProgress(ctx context.Context, etx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], attempt *TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) error UpdateTxFatalError(ctx context.Context, etx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) error + UpdateTxesFinalized(ctx context.Context, etxs []int64, chainId CHAIN_ID) error UpdateTxForRebroadcast(ctx context.Context, etx Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], etxAttempt TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) error - IsTxFinalized(ctx context.Context, blockHeight int64, txID int64, chainID CHAIN_ID) (finalized bool, err error) } type TxHistoryReaper[CHAIN_ID types.ID] interface { diff --git a/core/chains/evm/txmgr/builder.go b/core/chains/evm/txmgr/builder.go index dcf15a4fa23..20a9d7e2555 100644 --- a/core/chains/evm/txmgr/builder.go +++ b/core/chains/evm/txmgr/builder.go @@ -53,11 +53,12 @@ func NewTxm( evmTracker := NewEvmTracker(txStore, keyStore, chainID, lggr) stuckTxDetector := NewStuckTxDetector(lggr, client.ConfiguredChainID(), chainConfig.ChainType(), fCfg.PriceMax(), txConfig.AutoPurge(), estimator, txStore, client) evmConfirmer := NewEvmConfirmer(txStore, txmClient, txmCfg, feeCfg, txConfig, dbConfig, keyStore, txAttemptBuilder, lggr, stuckTxDetector) + evmFinalizer := NewEvmFinalizer(lggr, client.ConfiguredChainID(), txStore, txmClient) var evmResender *Resender if txConfig.ResendAfterThreshold() > 0 { evmResender = NewEvmResender(lggr, txStore, txmClient, evmTracker, keyStore, txmgr.DefaultResenderPollInterval, chainConfig, txConfig) } - txm = NewEvmTxm(chainID, txmCfg, txConfig, keyStore, lggr, checker, fwdMgr, txAttemptBuilder, txStore, evmBroadcaster, evmConfirmer, evmResender, evmTracker) + txm = NewEvmTxm(chainID, txmCfg, txConfig, keyStore, lggr, checker, fwdMgr, txAttemptBuilder, txStore, evmBroadcaster, evmConfirmer, evmResender, evmTracker, evmFinalizer) return txm, nil } @@ -76,8 +77,9 @@ func NewEvmTxm( confirmer *Confirmer, resender *Resender, tracker *Tracker, + finalizer *Finalizer, ) *Txm { - return txmgr.NewTxm(chainId, cfg, txCfg, keyStore, lggr, checkerFactory, fwdMgr, txAttemptBuilder, txStore, broadcaster, confirmer, resender, tracker, client.NewTxError) + return txmgr.NewTxm(chainId, cfg, txCfg, keyStore, lggr, checkerFactory, fwdMgr, txAttemptBuilder, txStore, broadcaster, confirmer, resender, tracker, finalizer, client.NewTxError) } // NewEvmResender creates a new concrete EvmResender @@ -142,3 +144,12 @@ func NewEvmBroadcaster( nonceTracker := NewNonceTracker(logger, txStore, client) return txmgr.NewBroadcaster(txStore, client, chainConfig, feeConfig, txConfig, listenerConfig, keystore, txAttemptBuilder, nonceTracker, logger, checkerFactory, autoSyncNonce) } + +func NewEvmFinalizer( + logger logger.Logger, + chainId *big.Int, + txStore TransactionStore, + client TxmClient, +) *Finalizer { + return txmgr.NewFinalizer(logger, chainId, txStore, client) +} diff --git a/core/chains/evm/txmgr/client.go b/core/chains/evm/txmgr/client.go index 661a180af50..e995080a260 100644 --- a/core/chains/evm/txmgr/client.go +++ b/core/chains/evm/txmgr/client.go @@ -183,3 +183,7 @@ func (c *evmTxmClient) CallContract(ctx context.Context, a TxAttempt, blockNumbe }, blockNumber) return client.ExtractRPCError(errCall) } + +func (c *evmTxmClient) HeadByHash(ctx context.Context, hash common.Hash) (*evmtypes.Head, error) { + return c.client.HeadByHash(ctx, hash) +} diff --git a/core/chains/evm/txmgr/confirmer_test.go b/core/chains/evm/txmgr/confirmer_test.go index 6b107b222a6..73b4c946cb3 100644 --- a/core/chains/evm/txmgr/confirmer_test.go +++ b/core/chains/evm/txmgr/confirmer_test.go @@ -2828,6 +2828,50 @@ func TestEthConfirmer_EnsureConfirmedTransactionsInLongestChain(t *testing.T) { assert.Equal(t, txmgrtypes.TxAttemptBroadcast, attempt.State) assert.Len(t, attempt.Receipts, 1) }) + + t.Run("unconfirms, unfinalizes, and rebroadcasts finalized transactions that have receipts within head height of the chain but not included in the chain", func(t *testing.T) { + nonce := evmtypes.Nonce(8) + broadcast := time.Now() + tx := &txmgr.Tx{ + Sequence: &nonce, + FromAddress: fromAddress, + EncodedPayload: []byte{1, 2, 3}, + State: txmgrcommon.TxConfirmed, + BroadcastAt: &broadcast, + InitialBroadcastAt: &broadcast, + Finalized: true, + } + err := txStore.InsertTx(ctx, tx) + require.NoError(t, err) + etx, err := txStore.FindTxWithAttempts(ctx, tx.ID) + require.NoError(t, err) + attempt := cltest.NewLegacyEthTxAttempt(t, etx.ID) + broadcastBeforeBlockNum := int64(1) + attempt.BroadcastBeforeBlockNum = &broadcastBeforeBlockNum + attempt.State = txmgrtypes.TxAttemptBroadcast + err = txStore.InsertTxAttempt(ctx, &attempt) + require.NoError(t, err) + // Include one within head height but a different block hash + mustInsertEthReceipt(t, txStore, head.Parent.Number, testutils.NewHash(), attempt.Hash) + + ethClient.On("SendTransactionReturnCode", mock.Anything, mock.MatchedBy(func(tx *types.Transaction) bool { + atx, signErr := txmgr.GetGethSignedTx(attempt.SignedRawTx) + require.NoError(t, signErr) + // Keeps gas price and nonce the same + return atx.GasPrice().Cmp(tx.GasPrice()) == 0 && atx.Nonce() == tx.Nonce() + }), fromAddress).Return(commonclient.Successful, nil).Once() + + // Do the thing + require.NoError(t, ec.EnsureConfirmedTransactionsInLongestChain(tests.Context(t), &head)) + + etx, err = txStore.FindTxWithAttempts(ctx, etx.ID) + require.NoError(t, err) + assert.Equal(t, txmgrcommon.TxUnconfirmed, etx.State) + require.Len(t, etx.TxAttempts, 1) + attempt = etx.TxAttempts[0] + assert.Equal(t, txmgrtypes.TxAttemptBroadcast, attempt.State) + assert.Equal(t, false, etx.Finalized) + }) } func TestEthConfirmer_ForceRebroadcast(t *testing.T) { diff --git a/core/chains/evm/txmgr/evm_tx_store.go b/core/chains/evm/txmgr/evm_tx_store.go index fd38eb7a8c9..b7a526a4742 100644 --- a/core/chains/evm/txmgr/evm_tx_store.go +++ b/core/chains/evm/txmgr/evm_tx_store.go @@ -181,6 +181,7 @@ type DbEthTx struct { // InitialBroadcastAt is recorded once, the first ever time this eth_tx is sent CreatedAt time.Time State txmgrtypes.TxState + Finalized bool // Marshalled EvmTxMeta // Used for additional context around transactions which you want to log // at send time. @@ -211,6 +212,7 @@ func (db *DbEthTx) FromTx(tx *Tx) { db.BroadcastAt = tx.BroadcastAt db.CreatedAt = tx.CreatedAt db.State = tx.State + db.Finalized = tx.Finalized db.Meta = tx.Meta db.Subject = tx.Subject db.PipelineTaskRunID = tx.PipelineTaskRunID @@ -245,6 +247,7 @@ func (db DbEthTx) ToTx(tx *Tx) { tx.BroadcastAt = db.BroadcastAt tx.CreatedAt = db.CreatedAt tx.State = db.State + tx.Finalized = db.Finalized tx.Meta = db.Meta tx.Subject = db.Subject tx.PipelineTaskRunID = db.PipelineTaskRunID @@ -529,8 +532,8 @@ func (o *evmTxStore) InsertTx(ctx context.Context, etx *Tx) error { if etx.CreatedAt == (time.Time{}) { etx.CreatedAt = time.Now() } - const insertEthTxSQL = `INSERT INTO evm.txes (nonce, from_address, to_address, encoded_payload, value, gas_limit, error, broadcast_at, initial_broadcast_at, created_at, state, meta, subject, pipeline_task_run_id, min_confirmations, evm_chain_id, transmit_checker, idempotency_key, signal_callback, callback_completed) VALUES ( -:nonce, :from_address, :to_address, :encoded_payload, :value, :gas_limit, :error, :broadcast_at, :initial_broadcast_at, :created_at, :state, :meta, :subject, :pipeline_task_run_id, :min_confirmations, :evm_chain_id, :transmit_checker, :idempotency_key, :signal_callback, :callback_completed + const insertEthTxSQL = `INSERT INTO evm.txes (nonce, from_address, to_address, encoded_payload, value, gas_limit, error, broadcast_at, initial_broadcast_at, created_at, state, meta, subject, pipeline_task_run_id, min_confirmations, evm_chain_id, transmit_checker, idempotency_key, signal_callback, callback_completed, finalized) VALUES ( +:nonce, :from_address, :to_address, :encoded_payload, :value, :gas_limit, :error, :broadcast_at, :initial_broadcast_at, :created_at, :state, :meta, :subject, :pipeline_task_run_id, :min_confirmations, :evm_chain_id, :transmit_checker, :idempotency_key, :signal_callback, :callback_completed, :finalized ) RETURNING *` var dbTx DbEthTx dbTx.FromTx(etx) @@ -1116,11 +1119,13 @@ func updateEthTxAttemptUnbroadcast(ctx context.Context, orm *evmTxStore, attempt return pkgerrors.Wrap(err, "updateEthTxAttemptUnbroadcast failed") } +// Ensure to mark the transaction as not finalized in case there is a finality violation and a "finalized" transaction +// has been considered re-org'd out func updateEthTxUnconfirm(ctx context.Context, orm *evmTxStore, etx Tx) error { if etx.State != txmgr.TxConfirmed { return errors.New("expected eth_tx state to be confirmed") } - _, err := orm.q.ExecContext(ctx, `UPDATE evm.txes SET state = 'unconfirmed' WHERE id = $1`, etx.ID) + _, err := orm.q.ExecContext(ctx, `UPDATE evm.txes SET state = 'unconfirmed', finalized = false WHERE id = $1`, etx.ID) return pkgerrors.Wrap(err, "updateEthTxUnconfirm failed") } @@ -1207,24 +1212,6 @@ AND evm_chain_id = $1`, chainID.String()).Scan(&earliestUnconfirmedTxBlock) return earliestUnconfirmedTxBlock, err } -func (o *evmTxStore) IsTxFinalized(ctx context.Context, blockHeight int64, txID int64, chainID *big.Int) (finalized bool, err error) { - var cancel context.CancelFunc - ctx, cancel = o.stopCh.Ctx(ctx) - defer cancel() - - var count int32 - err = o.q.GetContext(ctx, &count, ` - SELECT COUNT(evm.receipts.receipt) FROM evm.txes - INNER JOIN evm.tx_attempts ON evm.txes.id = evm.tx_attempts.eth_tx_id - INNER JOIN evm.receipts ON evm.tx_attempts.hash = evm.receipts.tx_hash - WHERE evm.receipts.block_number <= ($1 - evm.txes.min_confirmations) - AND evm.txes.id = $2 AND evm.txes.evm_chain_id = $3`, blockHeight, txID, chainID.String()) - if err != nil { - return false, fmt.Errorf("failed to retrieve transaction reciepts: %w", err) - } - return count > 0, nil -} - func (o *evmTxStore) saveAttemptWithNewState(ctx context.Context, attempt TxAttempt, broadcastAt time.Time) error { var dbAttempt DbEthTxAttempt dbAttempt.FromTxAttempt(&attempt) @@ -2059,3 +2046,35 @@ func (o *evmTxStore) UpdateTxAttemptBroadcastBeforeBlockNum(ctx context.Context, _, err := o.q.ExecContext(ctx, sql, blockNum, id) return err } + +// Returns all transaction in a specified state +func (o *evmTxStore) FindTransactionsByState(ctx context.Context, state txmgrtypes.TxState, chainID *big.Int) (txes []*Tx, err error) { + var cancel context.CancelFunc + ctx, cancel = o.stopCh.Ctx(ctx) + defer cancel() + err = o.Transact(ctx, true, func(orm *evmTxStore) error { + sql := "SELECT * FROM evm.txes WHERE state = $1 AND evm_chain_id = $2" + var dbEtxs []DbEthTx + err = o.q.SelectContext(ctx, &dbEtxs, sql, state, chainID.String()) + txes = make([]*Tx, len(dbEtxs)) + dbEthTxsToEvmEthTxPtrs(dbEtxs, txes) + if err = orm.LoadTxesAttempts(ctx, txes); err != nil { + return pkgerrors.Wrapf(err, "failed to load evm.tx_attempts for evm.tx") + } + if err = orm.loadEthTxesAttemptsReceipts(ctx, txes); err != nil { + return pkgerrors.Wrapf(err, "failed to load evm.receipts for evm.tx") + } + return nil + }) + return txes, err +} + +// Mark transactions provided as finalized +func (o *evmTxStore) UpdateTxesFinalized(ctx context.Context, etxIDs []int64, chainId *big.Int) error { + var cancel context.CancelFunc + ctx, cancel = o.stopCh.Ctx(ctx) + defer cancel() + sql := "UPDATE evm.txes SET finalized = true WHERE id = ANY($1) AND evm_chain_id = $2" + _, err := o.q.ExecContext(ctx, sql, pq.Array(etxIDs), chainId.String()) + return err +} diff --git a/core/chains/evm/txmgr/evm_tx_store_test.go b/core/chains/evm/txmgr/evm_tx_store_test.go index afb8de4ca52..09ec271770f 100644 --- a/core/chains/evm/txmgr/evm_tx_store_test.go +++ b/core/chains/evm/txmgr/evm_tx_store_test.go @@ -783,30 +783,6 @@ func TestORM_UpdateTxForRebroadcast(t *testing.T) { }) } -func TestORM_IsTxFinalized(t *testing.T) { - t.Parallel() - - db := pgtest.NewSqlxDB(t) - txStore := cltest.NewTestTxStore(t, db) - ethClient := evmtest.NewEthClientMockWithDefaultChain(t) - - t.Run("confirmed tx not past finality_depth", func(t *testing.T) { - confirmedAddr := cltest.MustGenerateRandomKey(t).Address - tx := mustInsertConfirmedEthTxWithReceipt(t, txStore, confirmedAddr, 123, 1) - finalized, err := txStore.IsTxFinalized(tests.Context(t), 2, tx.ID, ethClient.ConfiguredChainID()) - require.NoError(t, err) - require.False(t, finalized) - }) - - t.Run("confirmed tx past finality_depth", func(t *testing.T) { - confirmedAddr := cltest.MustGenerateRandomKey(t).Address - tx := mustInsertConfirmedEthTxWithReceipt(t, txStore, confirmedAddr, 123, 1) - finalized, err := txStore.IsTxFinalized(tests.Context(t), 10, tx.ID, ethClient.ConfiguredChainID()) - require.NoError(t, err) - require.True(t, finalized) - }) -} - func TestORM_FindTransactionsConfirmedInBlockRange(t *testing.T) { t.Parallel() @@ -1382,7 +1358,7 @@ func TestORM_UpdateTxUnstartedToInProgress(t *testing.T) { evmTxmCfg := txmgr.NewEvmTxmConfig(ccfg.EVM()) ec := evmtest.NewEthClientMockWithDefaultChain(t) txMgr := txmgr.NewEvmTxm(ec.ConfiguredChainID(), evmTxmCfg, ccfg.EVM().Transactions(), nil, logger.Test(t), nil, nil, - nil, txStore, nil, nil, nil, nil) + nil, txStore, nil, nil, nil, nil, nil) err := txMgr.XXXTestAbandon(fromAddress) // mark transaction as abandoned require.NoError(t, err) @@ -1871,3 +1847,81 @@ func AssertCountPerSubject(t *testing.T, txStore txmgr.TestEvmTxStore, expected require.NoError(t, err) require.Equal(t, int(expected), count) } + +func TestORM_FindTransactionsByState(t *testing.T) { + t.Parallel() + + ctx := tests.Context(t) + db := pgtest.NewSqlxDB(t) + txStore := cltest.NewTestTxStore(t, db) + kst := cltest.NewKeyStore(t, db) + _, fromAddress := cltest.MustInsertRandomKey(t, kst.Eth()) + + mustInsertUnstartedTx(t, txStore, fromAddress) + mustInsertInProgressEthTxWithAttempt(t, txStore, 0, fromAddress) + mustInsertUnconfirmedEthTxWithAttemptState(t, txStore, 1, fromAddress, txmgrtypes.TxAttemptBroadcast) + mustInsertConfirmedMissingReceiptEthTxWithLegacyAttempt(t, txStore, 2, 100, time.Now(), fromAddress) + mustInsertConfirmedEthTxWithReceipt(t, txStore, fromAddress, 3, 100) + mustInsertFatalErrorEthTx(t, txStore, fromAddress) + + var txStates []txmgrtypes.TxState + txStates = append(txStates, txmgrcommon.TxUnstarted) + txStates = append(txStates, txmgrcommon.TxInProgress) + txStates = append(txStates, txmgrcommon.TxUnconfirmed) + txStates = append(txStates, txmgrcommon.TxConfirmed) + txStates = append(txStates, txmgrcommon.TxConfirmedMissingReceipt) + txStates = append(txStates, txmgrcommon.TxConfirmed) + + for _, state := range txStates { + txs, err := txStore.FindTransactionsByState(ctx, state, testutils.FixtureChainID) + require.NoError(t, err) + require.Len(t, txs, 1) + } +} + +func TestORM_UpdateTxesFinalized(t *testing.T) { + t.Parallel() + + ctx := tests.Context(t) + db := pgtest.NewSqlxDB(t) + txStore := cltest.NewTestTxStore(t, db) + kst := cltest.NewKeyStore(t, db) + broadcast := time.Now() + _, fromAddress := cltest.MustInsertRandomKey(t, kst.Eth()) + + t.Run("successfully finalizes a confirmed transaction", func(t *testing.T) { + nonce := evmtypes.Nonce(0) + tx := &txmgr.Tx{ + Sequence: &nonce, + FromAddress: fromAddress, + EncodedPayload: []byte{1, 2, 3}, + State: txmgrcommon.TxConfirmed, + BroadcastAt: &broadcast, + InitialBroadcastAt: &broadcast, + } + err := txStore.InsertTx(ctx, tx) + require.NoError(t, err) + err = txStore.UpdateTxesFinalized(ctx, []int64{tx.ID}, testutils.FixtureChainID) + require.NoError(t, err) + etx, err := txStore.FindTxWithAttempts(ctx, tx.ID) + require.NoError(t, err) + require.True(t, etx.Finalized) + }) + t.Run("fails to finalize an unconfirmed transaction", func(t *testing.T) { + nonce := evmtypes.Nonce(1) + tx := &txmgr.Tx{ + Sequence: &nonce, + FromAddress: fromAddress, + EncodedPayload: []byte{1, 2, 3}, + State: txmgrcommon.TxUnconfirmed, + BroadcastAt: &broadcast, + InitialBroadcastAt: &broadcast, + } + err := txStore.InsertTx(ctx, tx) + require.NoError(t, err) + err = txStore.UpdateTxesFinalized(ctx, []int64{tx.ID}, testutils.FixtureChainID) + // Fails due to chk_eth_txes_state_finalized constraint + // Tx Store is poisoned after this + require.ErrorContains(t, err, "chk_eth_txes_state_finalized") + }) +} diff --git a/core/chains/evm/txmgr/finalizer_test.go b/core/chains/evm/txmgr/finalizer_test.go new file mode 100644 index 00000000000..263f77fafc0 --- /dev/null +++ b/core/chains/evm/txmgr/finalizer_test.go @@ -0,0 +1,160 @@ +package txmgr_test + +import ( + "testing" + "time" + + "github.com/ethereum/go-ethereum/common" + "github.com/google/uuid" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + + "github.com/smartcontractkit/chainlink-common/pkg/logger" + "github.com/smartcontractkit/chainlink-common/pkg/utils/tests" + txmgrcommon "github.com/smartcontractkit/chainlink/v2/common/txmgr" + "github.com/smartcontractkit/chainlink/v2/core/chains/evm/testutils" + "github.com/smartcontractkit/chainlink/v2/core/chains/evm/txmgr" + evmtypes "github.com/smartcontractkit/chainlink/v2/core/chains/evm/types" + "github.com/smartcontractkit/chainlink/v2/core/chains/evm/utils" + "github.com/smartcontractkit/chainlink/v2/core/internal/cltest" + "github.com/smartcontractkit/chainlink/v2/core/internal/testutils/pgtest" +) + +func TestFinalizer_MarkTxFinalized(t *testing.T) { + t.Parallel() + ctx := tests.Context(t) + db := pgtest.NewSqlxDB(t) + txStore := cltest.NewTestTxStore(t, db) + ethKeyStore := cltest.NewKeyStore(t, db).Eth() + feeLimit := uint64(10_000) + ethClient := testutils.NewEthClientMockWithDefaultChain(t) + + finalizer := txmgr.NewEvmFinalizer(logger.Test(t), testutils.FixtureChainID, txStore, txmgr.NewEvmTxmClient(ethClient, nil)) + err := finalizer.Start(ctx) + require.NoError(t, err) + + head := &evmtypes.Head{ + Hash: utils.NewHash(), + Number: 100, + Parent: &evmtypes.Head{ + Hash: utils.NewHash(), + Number: 99, + IsFinalized: true, + }, + } + + t.Run("returns not finalized for tx with receipt newer than finalized block", func(t *testing.T) { + idempotencyKey := uuid.New().String() + _, fromAddress := cltest.MustInsertRandomKey(t, ethKeyStore) + nonce := evmtypes.Nonce(0) + broadcast := time.Now() + tx := &txmgr.Tx{ + Sequence: &nonce, + IdempotencyKey: &idempotencyKey, + FromAddress: fromAddress, + EncodedPayload: []byte{1, 2, 3}, + FeeLimit: feeLimit, + State: txmgrcommon.TxConfirmed, + BroadcastAt: &broadcast, + InitialBroadcastAt: &broadcast, + } + attemptHash := insertTxAndAttemptWithIdempotencyKey(t, txStore, tx, idempotencyKey) + // Insert receipt for unfinalized block num + mustInsertEthReceipt(t, txStore, head.Number, head.Hash, attemptHash) + err = finalizer.ProcessHead(ctx, head) + require.NoError(t, err) + tx, err = txStore.FindTxWithIdempotencyKey(ctx, idempotencyKey, testutils.FixtureChainID) + require.NoError(t, err) + require.Equal(t, false, tx.Finalized) + }) + + t.Run("returns not finalized for tx with receipt re-org'd out", func(t *testing.T) { + idempotencyKey := uuid.New().String() + _, fromAddress := cltest.MustInsertRandomKey(t, ethKeyStore) + nonce := evmtypes.Nonce(0) + broadcast := time.Now() + tx := &txmgr.Tx{ + Sequence: &nonce, + IdempotencyKey: &idempotencyKey, + FromAddress: fromAddress, + EncodedPayload: []byte{1, 2, 3}, + FeeLimit: feeLimit, + State: txmgrcommon.TxConfirmed, + BroadcastAt: &broadcast, + InitialBroadcastAt: &broadcast, + } + attemptHash := insertTxAndAttemptWithIdempotencyKey(t, txStore, tx, idempotencyKey) + // Insert receipt for finalized block num + mustInsertEthReceipt(t, txStore, head.Parent.Number, utils.NewHash(), attemptHash) + err = finalizer.ProcessHead(ctx, head) + require.NoError(t, err) + tx, err = txStore.FindTxWithIdempotencyKey(ctx, idempotencyKey, testutils.FixtureChainID) + require.NoError(t, err) + require.Equal(t, false, tx.Finalized) + }) + + t.Run("returns finalized for tx with receipt in a finalized block", func(t *testing.T) { + idempotencyKey := uuid.New().String() + _, fromAddress := cltest.MustInsertRandomKey(t, ethKeyStore) + nonce := evmtypes.Nonce(0) + broadcast := time.Now() + tx := &txmgr.Tx{ + Sequence: &nonce, + IdempotencyKey: &idempotencyKey, + FromAddress: fromAddress, + EncodedPayload: []byte{1, 2, 3}, + FeeLimit: feeLimit, + State: txmgrcommon.TxConfirmed, + BroadcastAt: &broadcast, + InitialBroadcastAt: &broadcast, + Finalized: true, + } + attemptHash := insertTxAndAttemptWithIdempotencyKey(t, txStore, tx, idempotencyKey) + // Insert receipt for finalized block num + mustInsertEthReceipt(t, txStore, head.Parent.Number, head.Parent.Hash, attemptHash) + err = finalizer.ProcessHead(ctx, head) + require.NoError(t, err) + tx, err = txStore.FindTxWithIdempotencyKey(ctx, idempotencyKey, testutils.FixtureChainID) + require.NoError(t, err) + require.Equal(t, true, tx.Finalized) + }) + + t.Run("returns finalized for tx with receipt older than block history depth", func(t *testing.T) { + idempotencyKey := uuid.New().String() + _, fromAddress := cltest.MustInsertRandomKey(t, ethKeyStore) + nonce := evmtypes.Nonce(0) + broadcast := time.Now() + tx := &txmgr.Tx{ + Sequence: &nonce, + IdempotencyKey: &idempotencyKey, + FromAddress: fromAddress, + EncodedPayload: []byte{1, 2, 3}, + FeeLimit: feeLimit, + State: txmgrcommon.TxConfirmed, + BroadcastAt: &broadcast, + InitialBroadcastAt: &broadcast, + } + attemptHash := insertTxAndAttemptWithIdempotencyKey(t, txStore, tx, idempotencyKey) + // Insert receipt for finalized block num + receiptHash := utils.NewHash() + mustInsertEthReceipt(t, txStore, head.Parent.Number-1, receiptHash, attemptHash) + ethClient.On("HeadByHash", mock.Anything, receiptHash).Return(&evmtypes.Head{Number: head.Parent.Number - 1, Hash: receiptHash}, nil) + err = finalizer.ProcessHead(ctx, head) + require.NoError(t, err) + tx, err = txStore.FindTxWithIdempotencyKey(ctx, idempotencyKey, testutils.FixtureChainID) + require.NoError(t, err) + require.Equal(t, true, tx.Finalized) + }) +} + +func insertTxAndAttemptWithIdempotencyKey(t *testing.T, txStore txmgr.TestEvmTxStore, tx *txmgr.Tx, idempotencyKey string) common.Hash { + ctx := tests.Context(t) + err := txStore.InsertTx(ctx, tx) + require.NoError(t, err) + tx, err = txStore.FindTxWithIdempotencyKey(ctx, idempotencyKey, testutils.FixtureChainID) + require.NoError(t, err) + attempt := cltest.NewLegacyEthTxAttempt(t, tx.ID) + err = txStore.InsertTxAttempt(ctx, &attempt) + require.NoError(t, err) + return attempt.Hash +} diff --git a/core/chains/evm/txmgr/mocks/evm_tx_store.go b/core/chains/evm/txmgr/mocks/evm_tx_store.go index 5a699f71bf9..9c1c6eeabeb 100644 --- a/core/chains/evm/txmgr/mocks/evm_tx_store.go +++ b/core/chains/evm/txmgr/mocks/evm_tx_store.go @@ -313,6 +313,36 @@ func (_m *EvmTxStore) FindNextUnstartedTransactionFromAddress(ctx context.Contex return r0, r1 } +// FindTransactionsByState provides a mock function with given fields: ctx, state, chainID +func (_m *EvmTxStore) FindTransactionsByState(ctx context.Context, state types.TxState, chainID *big.Int) ([]*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error) { + ret := _m.Called(ctx, state, chainID) + + if len(ret) == 0 { + panic("no return value specified for FindTransactionsByState") + } + + var r0 []*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee] + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, types.TxState, *big.Int) ([]*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error)); ok { + return rf(ctx, state, chainID) + } + if rf, ok := ret.Get(0).(func(context.Context, types.TxState, *big.Int) []*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]); ok { + r0 = rf(ctx, state, chainID) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, types.TxState, *big.Int) error); ok { + r1 = rf(ctx, state, chainID) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + // FindTransactionsConfirmedInBlockRange provides a mock function with given fields: ctx, highBlockNumber, lowBlockNumber, chainID func (_m *EvmTxStore) FindTransactionsConfirmedInBlockRange(ctx context.Context, highBlockNumber int64, lowBlockNumber int64, chainID *big.Int) ([]*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error) { ret := _m.Called(ctx, highBlockNumber, lowBlockNumber, chainID) @@ -999,34 +1029,6 @@ func (_m *EvmTxStore) HasInProgressTransaction(ctx context.Context, account comm return r0, r1 } -// IsTxFinalized provides a mock function with given fields: ctx, blockHeight, txID, chainID -func (_m *EvmTxStore) IsTxFinalized(ctx context.Context, blockHeight int64, txID int64, chainID *big.Int) (bool, error) { - ret := _m.Called(ctx, blockHeight, txID, chainID) - - if len(ret) == 0 { - panic("no return value specified for IsTxFinalized") - } - - var r0 bool - var r1 error - if rf, ok := ret.Get(0).(func(context.Context, int64, int64, *big.Int) (bool, error)); ok { - return rf(ctx, blockHeight, txID, chainID) - } - if rf, ok := ret.Get(0).(func(context.Context, int64, int64, *big.Int) bool); ok { - r0 = rf(ctx, blockHeight, txID, chainID) - } else { - r0 = ret.Get(0).(bool) - } - - if rf, ok := ret.Get(1).(func(context.Context, int64, int64, *big.Int) error); ok { - r1 = rf(ctx, blockHeight, txID, chainID) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - // LoadTxAttempts provides a mock function with given fields: ctx, etx func (_m *EvmTxStore) LoadTxAttempts(ctx context.Context, etx *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]) error { ret := _m.Called(ctx, etx) @@ -1492,6 +1494,24 @@ func (_m *EvmTxStore) UpdateTxUnstartedToInProgress(ctx context.Context, etx *ty return r0 } +// UpdateTxesFinalized provides a mock function with given fields: ctx, etxs, chainId +func (_m *EvmTxStore) UpdateTxesFinalized(ctx context.Context, etxs []int64, chainId *big.Int) error { + ret := _m.Called(ctx, etxs, chainId) + + if len(ret) == 0 { + panic("no return value specified for UpdateTxesFinalized") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, []int64, *big.Int) error); ok { + r0 = rf(ctx, etxs, chainId) + } else { + r0 = ret.Error(0) + } + + return r0 +} + // UpdateTxsUnconfirmed provides a mock function with given fields: ctx, ids func (_m *EvmTxStore) UpdateTxsUnconfirmed(ctx context.Context, ids []int64) error { ret := _m.Called(ctx, ids) diff --git a/core/chains/evm/txmgr/models.go b/core/chains/evm/txmgr/models.go index f8682ffd500..92e18b820bf 100644 --- a/core/chains/evm/txmgr/models.go +++ b/core/chains/evm/txmgr/models.go @@ -39,9 +39,10 @@ type ( Receipt = dbReceipt // EvmReceipt is the exported DB table model for receipts ReceiptPlus = txmgrtypes.ReceiptPlus[*evmtypes.Receipt] StuckTxDetector = txmgrtypes.StuckTxDetector[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee] - TxmClient = txmgrtypes.TxmClient[*big.Int, common.Address, common.Hash, common.Hash, *evmtypes.Receipt, evmtypes.Nonce, gas.EvmFee] + TxmClient = txmgrtypes.TxmClient[*big.Int, common.Address, common.Hash, common.Hash, *evmtypes.Receipt, evmtypes.Nonce, gas.EvmFee, *evmtypes.Head] TransactionClient = txmgrtypes.TransactionClient[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee] ChainReceipt = txmgrtypes.ChainReceipt[common.Hash, common.Hash] + Finalizer = txmgr.Finalizer[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee, *evmtypes.Head] ) var _ KeyStore = (keystore.Eth)(nil) // check interface in txmgr to avoid circular import diff --git a/core/chains/evm/txmgr/txmgr_test.go b/core/chains/evm/txmgr/txmgr_test.go index 40df5616c99..c4ccc85a6a4 100644 --- a/core/chains/evm/txmgr/txmgr_test.go +++ b/core/chains/evm/txmgr/txmgr_test.go @@ -693,7 +693,7 @@ func TestTxm_GetTransactionStatus(t *testing.T) { require.Equal(t, commontypes.Unconfirmed, state) }) - t.Run("returns unconfirmed for confirmed state", func(t *testing.T) { + t.Run("returns unconfirmed for confirmed state not marked as finalized", func(t *testing.T) { idempotencyKey := uuid.New().String() _, fromAddress := cltest.MustInsertRandomKey(t, ethKeyStore) nonce := evmtypes.Nonce(0) @@ -707,6 +707,7 @@ func TestTxm_GetTransactionStatus(t *testing.T) { State: txmgrcommon.TxConfirmed, BroadcastAt: &broadcast, InitialBroadcastAt: &broadcast, + Finalized: false, // Set to false by default in DB but here for explicitness } err := txStore.InsertTx(ctx, tx) require.NoError(t, err) @@ -715,13 +716,43 @@ func TestTxm_GetTransactionStatus(t *testing.T) { attempt := cltest.NewLegacyEthTxAttempt(t, tx.ID) err = txStore.InsertTxAttempt(ctx, &attempt) require.NoError(t, err) - // Insert receipt for finalized block num - mustInsertEthReceipt(t, txStore, head.Parent.Number, head.ParentHash, attempt.Hash) + // Insert receipt for unfinalized block num + mustInsertEthReceipt(t, txStore, head.Number, head.Hash, attempt.Hash) state, err := txm.GetTransactionStatus(ctx, idempotencyKey) require.NoError(t, err) require.Equal(t, commontypes.Unconfirmed, state) }) + t.Run("returns finalized for confirmed state marked as finalized", func(t *testing.T) { + idempotencyKey := uuid.New().String() + _, fromAddress := cltest.MustInsertRandomKey(t, ethKeyStore) + nonce := evmtypes.Nonce(0) + broadcast := time.Now() + tx := &txmgr.Tx{ + Sequence: &nonce, + IdempotencyKey: &idempotencyKey, + FromAddress: fromAddress, + EncodedPayload: []byte{1, 2, 3}, + FeeLimit: feeLimit, + State: txmgrcommon.TxConfirmed, + BroadcastAt: &broadcast, + InitialBroadcastAt: &broadcast, + Finalized: true, + } + err := txStore.InsertTx(ctx, tx) + require.NoError(t, err) + tx, err = txStore.FindTxWithIdempotencyKey(ctx, idempotencyKey, testutils.FixtureChainID) + require.NoError(t, err) + attempt := cltest.NewLegacyEthTxAttempt(t, tx.ID) + err = txStore.InsertTxAttempt(ctx, &attempt) + require.NoError(t, err) + // Insert receipt for finalized block num + mustInsertEthReceipt(t, txStore, head.Parent.Number, head.Parent.Hash, attempt.Hash) + state, err := txm.GetTransactionStatus(ctx, idempotencyKey) + require.NoError(t, err) + require.Equal(t, commontypes.Finalized, state) + }) + t.Run("returns unconfirmed for confirmed missing receipt state", func(t *testing.T) { idempotencyKey := uuid.New().String() _, fromAddress := cltest.MustInsertRandomKey(t, ethKeyStore) @@ -1018,6 +1049,12 @@ func mustCreateUnstartedTxFromEvmTxRequest(t testing.TB, txStore txmgr.EvmTxStor return tx } +func mustInsertUnstartedTx(t testing.TB, txStore txmgr.TestEvmTxStore, fromAddress common.Address) { + etx := cltest.NewEthTx(fromAddress) + ctx := tests.Context(t) + require.NoError(t, txStore.InsertTx(ctx, &etx)) +} + func txRequestWithStrategy(strategy txmgrtypes.TxStrategy) func(*txmgr.TxRequest) { return func(tx *txmgr.TxRequest) { tx.Strategy = strategy diff --git a/core/services/vrf/v2/integration_v2_test.go b/core/services/vrf/v2/integration_v2_test.go index e9ae908565a..178b555667b 100644 --- a/core/services/vrf/v2/integration_v2_test.go +++ b/core/services/vrf/v2/integration_v2_test.go @@ -142,7 +142,7 @@ func makeTestTxm(t *testing.T, txStore txmgr.TestEvmTxStore, keyStore keystore.M _, _, evmConfig := txmgr.MakeTestConfigs(t) txmConfig := txmgr.NewEvmTxmConfig(evmConfig) txm := txmgr.NewEvmTxm(ec.ConfiguredChainID(), txmConfig, evmConfig.Transactions(), keyStore.Eth(), logger.TestLogger(t), nil, nil, - nil, txStore, nil, nil, nil, nil) + nil, txStore, nil, nil, nil, nil, nil) return txm } diff --git a/core/services/vrf/v2/listener_v2_test.go b/core/services/vrf/v2/listener_v2_test.go index ac59f1fdb69..b7a8710c4f8 100644 --- a/core/services/vrf/v2/listener_v2_test.go +++ b/core/services/vrf/v2/listener_v2_test.go @@ -40,7 +40,7 @@ func makeTestTxm(t *testing.T, txStore txmgr.TestEvmTxStore, keyStore keystore.M ec := evmtest.NewEthClientMockWithDefaultChain(t) txmConfig := txmgr.NewEvmTxmConfig(evmConfig) txm := txmgr.NewEvmTxm(ec.ConfiguredChainID(), txmConfig, evmConfig.Transactions(), keyStore.Eth(), logger.TestLogger(t), nil, nil, - nil, txStore, nil, nil, nil, nil) + nil, txStore, nil, nil, nil, nil, nil) return txm } diff --git a/core/store/migrate/migrations/0245_add_tx_finalized_column.sql b/core/store/migrate/migrations/0245_add_tx_finalized_column.sql new file mode 100644 index 00000000000..cc0b09fd3dd --- /dev/null +++ b/core/store/migrate/migrations/0245_add_tx_finalized_column.sql @@ -0,0 +1,15 @@ +-- +goose Up +-- +goose StatementBegin +ALTER TABLE evm.txes ADD COLUMN finalized boolean NOT NULL DEFAULT false; +ALTER TABLE evm.txes ADD CONSTRAINT chk_eth_txes_state_finalized CHECK ( + state <> 'confirmed'::eth_txes_state AND finalized = false + OR + state = 'confirmed'::eth_txes_state +) NOT VALID; +-- +goose StatementEnd + +-- +goose Down +-- +goose StatementBegin +ALTER TABLE evm.txes DROP CONSTRAINT chk_eth_txes_state_finalized; +ALTER TABLE evm.txes DROP COLUMN finalized; +-- +goose StatementEnd diff --git a/core/web/testdata/body/health.html b/core/web/testdata/body/health.html index 2a1b2227530..90d301bc8b8 100644 --- a/core/web/testdata/body/health.html +++ b/core/web/testdata/body/health.html @@ -63,6 +63,9 @@
Confirmer
+
+ Finalizer +
WrappedEvmEstimator
diff --git a/core/web/testdata/body/health.json b/core/web/testdata/body/health.json index 10415c0abdc..839428a5103 100644 --- a/core/web/testdata/body/health.json +++ b/core/web/testdata/body/health.json @@ -90,6 +90,15 @@ "output": "" } }, + { + "type": "checks", + "id": "EVM.0.Txm.Finalizer", + "attributes": { + "name": "EVM.0.Txm.Finalizer", + "status": "passing", + "output": "" + } + }, { "type": "checks", "id": "EVM.0.Txm.WrappedEvmEstimator", diff --git a/core/web/testdata/body/health.txt b/core/web/testdata/body/health.txt index 09c8cff6c2d..3709b4e15f0 100644 --- a/core/web/testdata/body/health.txt +++ b/core/web/testdata/body/health.txt @@ -9,6 +9,7 @@ ok EVM.0.Txm ok EVM.0.Txm.BlockHistoryEstimator ok EVM.0.Txm.Broadcaster ok EVM.0.Txm.Confirmer +ok EVM.0.Txm.Finalizer ok EVM.0.Txm.WrappedEvmEstimator ok JobSpawner ok Mailbox.Monitor diff --git a/testdata/scripts/health/multi-chain.txtar b/testdata/scripts/health/multi-chain.txtar index 8178f8e8213..825e788586c 100644 --- a/testdata/scripts/health/multi-chain.txtar +++ b/testdata/scripts/health/multi-chain.txtar @@ -74,6 +74,7 @@ ok EVM.1.Txm ok EVM.1.Txm.BlockHistoryEstimator ok EVM.1.Txm.Broadcaster ok EVM.1.Txm.Confirmer +ok EVM.1.Txm.Finalizer ok EVM.1.Txm.WrappedEvmEstimator ok JobSpawner ok Mailbox.Monitor @@ -207,6 +208,15 @@ ok TelemetryManager "output": "" } }, + { + "type": "checks", + "id": "EVM.1.Txm.Finalizer", + "attributes": { + "name": "EVM.1.Txm.Finalizer", + "status": "passing", + "output": "" + } + }, { "type": "checks", "id": "EVM.1.Txm.WrappedEvmEstimator", From af1e9a0671e55fdd2fca063d44c8715900838386 Mon Sep 17 00:00:00 2001 From: amit-momin Date: Mon, 24 Jun 2024 17:12:43 -0500 Subject: [PATCH 02/23] Moved Finalizer component into EVM code and addressed feedback --- common/txmgr/reaper.go | 9 +- common/txmgr/txmgr.go | 8 +- common/txmgr/types/config.go | 8 - common/txmgr/types/finalizer.go | 12 ++ common/txmgr/types/mocks/tx_store.go | 70 +++---- common/txmgr/types/tx_store.go | 4 +- core/chains/evm/txmgr/builder.go | 17 +- core/chains/evm/txmgr/config.go | 1 - core/chains/evm/txmgr/evm_tx_store.go | 21 +- core/chains/evm/txmgr/evm_tx_store_test.go | 17 +- .../chains/evm}/txmgr/finalizer.go | 180 ++++++++++-------- core/chains/evm/txmgr/finalizer_test.go | 15 +- core/chains/evm/txmgr/mocks/evm_tx_store.go | 70 +++---- core/chains/evm/txmgr/models.go | 2 +- core/chains/evm/txmgr/reaper_test.go | 38 ++-- 15 files changed, 240 insertions(+), 232 deletions(-) create mode 100644 common/txmgr/types/finalizer.go rename {common => core/chains/evm}/txmgr/finalizer.go (50%) diff --git a/common/txmgr/reaper.go b/common/txmgr/reaper.go index 932b58f6430..0c797548b16 100644 --- a/common/txmgr/reaper.go +++ b/common/txmgr/reaper.go @@ -14,7 +14,6 @@ import ( // Reaper handles periodic database cleanup for Txm type Reaper[CHAIN_ID types.ID] struct { store txmgrtypes.TxHistoryReaper[CHAIN_ID] - config txmgrtypes.ReaperChainConfig txConfig txmgrtypes.ReaperTransactionsConfig chainID CHAIN_ID log logger.Logger @@ -25,10 +24,9 @@ type Reaper[CHAIN_ID types.ID] struct { } // NewReaper instantiates a new reaper object -func NewReaper[CHAIN_ID types.ID](lggr logger.Logger, store txmgrtypes.TxHistoryReaper[CHAIN_ID], config txmgrtypes.ReaperChainConfig, txConfig txmgrtypes.ReaperTransactionsConfig, chainID CHAIN_ID) *Reaper[CHAIN_ID] { +func NewReaper[CHAIN_ID types.ID](lggr logger.Logger, store txmgrtypes.TxHistoryReaper[CHAIN_ID], txConfig txmgrtypes.ReaperTransactionsConfig, chainID CHAIN_ID) *Reaper[CHAIN_ID] { r := &Reaper[CHAIN_ID]{ store, - config, txConfig, chainID, logger.Named(lggr, "Reaper"), @@ -103,13 +101,12 @@ func (r *Reaper[CHAIN_ID]) ReapTxes(headNum int64) error { r.log.Debug("Transactions.ReaperThreshold set to 0; skipping ReapTxes") return nil } - minBlockNumberToKeep := headNum - int64(r.config.FinalityDepth()) mark := time.Now() timeThreshold := mark.Add(-threshold) - r.log.Debugw(fmt.Sprintf("reaping old txes created before %s", timeThreshold.Format(time.RFC3339)), "ageThreshold", threshold, "timeThreshold", timeThreshold, "minBlockNumberToKeep", minBlockNumberToKeep) + r.log.Debugw(fmt.Sprintf("reaping old txes created before %s", timeThreshold.Format(time.RFC3339)), "ageThreshold", threshold, "timeThreshold", timeThreshold) - if err := r.store.ReapTxHistory(ctx, minBlockNumberToKeep, timeThreshold, r.chainID); err != nil { + if err := r.store.ReapTxHistory(ctx, timeThreshold, r.chainID); err != nil { return err } diff --git a/common/txmgr/txmgr.go b/common/txmgr/txmgr.go index 46d3011e5ff..ff2c9c2c605 100644 --- a/common/txmgr/txmgr.go +++ b/common/txmgr/txmgr.go @@ -110,7 +110,7 @@ type Txm[ broadcaster *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] confirmer *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE] tracker *Tracker[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE] - finalizer *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD] + finalizer txmgrtypes.Finalizer[BLOCK_HASH, HEAD] fwdMgr txmgrtypes.ForwarderManager[ADDR] txAttemptBuilder txmgrtypes.TxAttemptBuilder[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] newErrorClassifier NewErrorClassifier @@ -146,7 +146,7 @@ func NewTxm[ confirmer *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE], resender *Resender[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE], tracker *Tracker[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE], - finalizer *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD], + finalizer txmgrtypes.Finalizer[BLOCK_HASH, HEAD], newErrorClassifierFunc NewErrorClassifier, ) *Txm[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { b := Txm[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]{ @@ -176,7 +176,7 @@ func NewTxm[ b.logger.Info("Resender: Disabled") } if txCfg.ReaperThreshold() > 0 && txCfg.ReaperInterval() > 0 { - b.reaper = NewReaper[CHAIN_ID](lggr, b.txStore, cfg, txCfg, chainId) + b.reaper = NewReaper[CHAIN_ID](lggr, b.txStore, txCfg, chainId) } else { b.logger.Info("TxReaper: Disabled") } @@ -425,7 +425,7 @@ func (b *Txm[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) runLoop() case head := <-b.chHeads: b.confirmer.mb.Deliver(head) b.tracker.mb.Deliver(head.BlockNumber()) - b.finalizer.mb.Deliver(head) + b.finalizer.DeliverHead(head) case reset := <-b.reset: // This check prevents the weird edge-case where you can select // into this block after chStop has already been closed and the diff --git a/common/txmgr/types/config.go b/common/txmgr/types/config.go index 53e35cd4b6e..8b11a45d11d 100644 --- a/common/txmgr/types/config.go +++ b/common/txmgr/types/config.go @@ -5,7 +5,6 @@ import "time" type TransactionManagerChainConfig interface { BroadcasterChainConfig ConfirmerChainConfig - ReaperChainConfig } type TransactionManagerFeeConfig interface { @@ -74,13 +73,6 @@ type ResenderTransactionsConfig interface { MaxInFlight() uint32 } -// ReaperConfig is the config subset used by the reaper -// -//go:generate mockery --quiet --name ReaperChainConfig --structname ReaperConfig --output ./mocks/ --case=underscore -type ReaperChainConfig interface { - FinalityDepth() uint32 -} - type ReaperTransactionsConfig interface { ReaperInterval() time.Duration ReaperThreshold() time.Duration diff --git a/common/txmgr/types/finalizer.go b/common/txmgr/types/finalizer.go new file mode 100644 index 00000000000..d7fd6b7f603 --- /dev/null +++ b/common/txmgr/types/finalizer.go @@ -0,0 +1,12 @@ +package types + +import ( + "github.com/smartcontractkit/chainlink-common/pkg/services" + "github.com/smartcontractkit/chainlink/v2/common/types" +) + +type Finalizer[BLOCK_HASH types.Hashable, HEAD types.Head[BLOCK_HASH]] interface { + // interfaces for running the underlying estimator + services.Service + DeliverHead(head HEAD) bool +} diff --git a/common/txmgr/types/mocks/tx_store.go b/common/txmgr/types/mocks/tx_store.go index 349c714b59e..9087491dc03 100644 --- a/common/txmgr/types/mocks/tx_store.go +++ b/common/txmgr/types/mocks/tx_store.go @@ -196,6 +196,36 @@ func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) DeleteInPro return r0 } +// FindConfirmedTxesAwaitingFinalization provides a mock function with given fields: ctx, chainID +func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindConfirmedTxesAwaitingFinalization(ctx context.Context, chainID CHAIN_ID) ([]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error) { + ret := _m.Called(ctx, chainID) + + if len(ret) == 0 { + panic("no return value specified for FindConfirmedTxesAwaitingFinalization") + } + + var r0 []*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, CHAIN_ID) ([]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error)); ok { + return rf(ctx, chainID) + } + if rf, ok := ret.Get(0).(func(context.Context, CHAIN_ID) []*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]); ok { + r0 = rf(ctx, chainID) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, CHAIN_ID) error); ok { + r1 = rf(ctx, chainID) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + // FindEarliestUnconfirmedBroadcastTime provides a mock function with given fields: ctx, chainID func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindEarliestUnconfirmedBroadcastTime(ctx context.Context, chainID CHAIN_ID) (null.Time, error) { ret := _m.Called(ctx, chainID) @@ -310,36 +340,6 @@ func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindNextUns return r0, r1 } -// FindTransactionsByState provides a mock function with given fields: ctx, state, chainID -func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTransactionsByState(ctx context.Context, state txmgrtypes.TxState, chainID CHAIN_ID) ([]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error) { - ret := _m.Called(ctx, state, chainID) - - if len(ret) == 0 { - panic("no return value specified for FindTransactionsByState") - } - - var r0 []*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] - var r1 error - if rf, ok := ret.Get(0).(func(context.Context, txmgrtypes.TxState, CHAIN_ID) ([]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error)); ok { - return rf(ctx, state, chainID) - } - if rf, ok := ret.Get(0).(func(context.Context, txmgrtypes.TxState, CHAIN_ID) []*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]); ok { - r0 = rf(ctx, state, chainID) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).([]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) - } - } - - if rf, ok := ret.Get(1).(func(context.Context, txmgrtypes.TxState, CHAIN_ID) error); ok { - r1 = rf(ctx, state, chainID) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - // FindTransactionsConfirmedInBlockRange provides a mock function with given fields: ctx, highBlockNumber, lowBlockNumber, chainID func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTransactionsConfirmedInBlockRange(ctx context.Context, highBlockNumber int64, lowBlockNumber int64, chainID CHAIN_ID) ([]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error) { ret := _m.Called(ctx, highBlockNumber, lowBlockNumber, chainID) @@ -980,17 +980,17 @@ func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) PruneUnstar return r0, r1 } -// ReapTxHistory provides a mock function with given fields: ctx, minBlockNumberToKeep, timeThreshold, chainID -func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) ReapTxHistory(ctx context.Context, minBlockNumberToKeep int64, timeThreshold time.Time, chainID CHAIN_ID) error { - ret := _m.Called(ctx, minBlockNumberToKeep, timeThreshold, chainID) +// ReapTxHistory provides a mock function with given fields: ctx, timeThreshold, chainID +func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) ReapTxHistory(ctx context.Context, timeThreshold time.Time, chainID CHAIN_ID) error { + ret := _m.Called(ctx, timeThreshold, chainID) if len(ret) == 0 { panic("no return value specified for ReapTxHistory") } var r0 error - if rf, ok := ret.Get(0).(func(context.Context, int64, time.Time, CHAIN_ID) error); ok { - r0 = rf(ctx, minBlockNumberToKeep, timeThreshold, chainID) + if rf, ok := ret.Get(0).(func(context.Context, time.Time, CHAIN_ID) error); ok { + r0 = rf(ctx, timeThreshold, chainID) } else { r0 = ret.Error(0) } diff --git a/common/txmgr/types/tx_store.go b/common/txmgr/types/tx_store.go index 495a6d7fbae..487b1b0ba82 100644 --- a/common/txmgr/types/tx_store.go +++ b/common/txmgr/types/tx_store.go @@ -84,7 +84,7 @@ type TransactionStore[ FindTransactionsConfirmedInBlockRange(ctx context.Context, highBlockNumber, lowBlockNumber int64, chainID CHAIN_ID) (etxs []*Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) FindEarliestUnconfirmedBroadcastTime(ctx context.Context, chainID CHAIN_ID) (null.Time, error) FindEarliestUnconfirmedTxAttemptBlock(ctx context.Context, chainID CHAIN_ID) (null.Int, error) - FindTransactionsByState(ctx context.Context, state TxState, chainID CHAIN_ID) (etxs []*Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) + FindConfirmedTxesAwaitingFinalization(ctx context.Context, chainID CHAIN_ID) (etxs []*Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) GetTxInProgress(ctx context.Context, fromAddress ADDR) (etx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) GetInProgressTxAttempts(ctx context.Context, address ADDR, chainID CHAIN_ID) (attempts []TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) GetAbandonedTransactionsByBatch(ctx context.Context, chainID CHAIN_ID, enabledAddrs []ADDR, offset, limit uint) (txs []*Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) @@ -112,7 +112,7 @@ type TransactionStore[ } type TxHistoryReaper[CHAIN_ID types.ID] interface { - ReapTxHistory(ctx context.Context, minBlockNumberToKeep int64, timeThreshold time.Time, chainID CHAIN_ID) error + ReapTxHistory(ctx context.Context, timeThreshold time.Time, chainID CHAIN_ID) error } type UnstartedTxQueuePruner interface { diff --git a/core/chains/evm/txmgr/builder.go b/core/chains/evm/txmgr/builder.go index 20a9d7e2555..23d802c521b 100644 --- a/core/chains/evm/txmgr/builder.go +++ b/core/chains/evm/txmgr/builder.go @@ -53,7 +53,7 @@ func NewTxm( evmTracker := NewEvmTracker(txStore, keyStore, chainID, lggr) stuckTxDetector := NewStuckTxDetector(lggr, client.ConfiguredChainID(), chainConfig.ChainType(), fCfg.PriceMax(), txConfig.AutoPurge(), estimator, txStore, client) evmConfirmer := NewEvmConfirmer(txStore, txmClient, txmCfg, feeCfg, txConfig, dbConfig, keyStore, txAttemptBuilder, lggr, stuckTxDetector) - evmFinalizer := NewEvmFinalizer(lggr, client.ConfiguredChainID(), txStore, txmClient) + evmFinalizer := NewEvmFinalizer(lggr, client.ConfiguredChainID(), txStore, client) var evmResender *Resender if txConfig.ResendAfterThreshold() > 0 { evmResender = NewEvmResender(lggr, txStore, txmClient, evmTracker, keyStore, txmgr.DefaultResenderPollInterval, chainConfig, txConfig) @@ -77,7 +77,7 @@ func NewEvmTxm( confirmer *Confirmer, resender *Resender, tracker *Tracker, - finalizer *Finalizer, + finalizer Finalizer, ) *Txm { return txmgr.NewTxm(chainId, cfg, txCfg, keyStore, lggr, checkerFactory, fwdMgr, txAttemptBuilder, txStore, broadcaster, confirmer, resender, tracker, finalizer, client.NewTxError) } @@ -97,8 +97,8 @@ func NewEvmResender( } // NewEvmReaper instantiates a new EVM-specific reaper object -func NewEvmReaper(lggr logger.Logger, store txmgrtypes.TxHistoryReaper[*big.Int], config EvmReaperConfig, txConfig txmgrtypes.ReaperTransactionsConfig, chainID *big.Int) *Reaper { - return txmgr.NewReaper(lggr, store, config, txConfig, chainID) +func NewEvmReaper(lggr logger.Logger, store txmgrtypes.TxHistoryReaper[*big.Int], txConfig txmgrtypes.ReaperTransactionsConfig, chainID *big.Int) *Reaper { + return txmgr.NewReaper(lggr, store, txConfig, chainID) } // NewEvmConfirmer instantiates a new EVM confirmer @@ -144,12 +144,3 @@ func NewEvmBroadcaster( nonceTracker := NewNonceTracker(logger, txStore, client) return txmgr.NewBroadcaster(txStore, client, chainConfig, feeConfig, txConfig, listenerConfig, keystore, txAttemptBuilder, nonceTracker, logger, checkerFactory, autoSyncNonce) } - -func NewEvmFinalizer( - logger logger.Logger, - chainId *big.Int, - txStore TransactionStore, - client TxmClient, -) *Finalizer { - return txmgr.NewFinalizer(logger, chainId, txStore, client) -} diff --git a/core/chains/evm/txmgr/config.go b/core/chains/evm/txmgr/config.go index c34de17369e..57088ddff23 100644 --- a/core/chains/evm/txmgr/config.go +++ b/core/chains/evm/txmgr/config.go @@ -50,7 +50,6 @@ type ( EvmBroadcasterConfig txmgrtypes.BroadcasterChainConfig EvmConfirmerConfig txmgrtypes.ConfirmerChainConfig EvmResenderConfig txmgrtypes.ResenderChainConfig - EvmReaperConfig txmgrtypes.ReaperChainConfig ) var _ EvmTxmConfig = (*evmTxmConfig)(nil) diff --git a/core/chains/evm/txmgr/evm_tx_store.go b/core/chains/evm/txmgr/evm_tx_store.go index b7a526a4742..0b97bf78770 100644 --- a/core/chains/evm/txmgr/evm_tx_store.go +++ b/core/chains/evm/txmgr/evm_tx_store.go @@ -1863,7 +1863,7 @@ id < ( return } -func (o *evmTxStore) ReapTxHistory(ctx context.Context, minBlockNumberToKeep int64, timeThreshold time.Time, chainID *big.Int) error { +func (o *evmTxStore) ReapTxHistory(ctx context.Context, timeThreshold time.Time, chainID *big.Int) error { var cancel context.CancelFunc ctx, cancel = o.stopCh.Ctx(ctx) defer cancel() @@ -1876,17 +1876,17 @@ func (o *evmTxStore) ReapTxHistory(ctx context.Context, minBlockNumberToKeep int res, err := o.q.ExecContext(ctx, ` WITH old_enough_receipts AS ( SELECT tx_hash FROM evm.receipts - WHERE block_number < $1 ORDER BY block_number ASC, id ASC - LIMIT $2 + LIMIT $1 ) DELETE FROM evm.txes USING old_enough_receipts, evm.tx_attempts WHERE evm.tx_attempts.eth_tx_id = evm.txes.id AND evm.tx_attempts.hash = old_enough_receipts.tx_hash -AND evm.txes.created_at < $3 +AND evm.txes.created_at < $2 AND evm.txes.state = 'confirmed' -AND evm_chain_id = $4`, minBlockNumberToKeep, limit, timeThreshold, chainID.String()) +AND evm.txes.finalized = true +AND evm_chain_id = $3`, limit, timeThreshold, chainID.String()) if err != nil { return count, pkgerrors.Wrap(err, "ReapTxes failed to delete old confirmed evm.txes") } @@ -2047,15 +2047,18 @@ func (o *evmTxStore) UpdateTxAttemptBroadcastBeforeBlockNum(ctx context.Context, return err } -// Returns all transaction in a specified state -func (o *evmTxStore) FindTransactionsByState(ctx context.Context, state txmgrtypes.TxState, chainID *big.Int) (txes []*Tx, err error) { +// Returns all confirmed transactions not yet marked as finalized +func (o *evmTxStore) FindConfirmedTxesAwaitingFinalization(ctx context.Context, chainID *big.Int) (txes []*Tx, err error) { var cancel context.CancelFunc ctx, cancel = o.stopCh.Ctx(ctx) defer cancel() err = o.Transact(ctx, true, func(orm *evmTxStore) error { - sql := "SELECT * FROM evm.txes WHERE state = $1 AND evm_chain_id = $2" + sql := "SELECT * FROM evm.txes WHERE state = 'confirmed' AND finalized = false AND evm_chain_id = $1" var dbEtxs []DbEthTx - err = o.q.SelectContext(ctx, &dbEtxs, sql, state, chainID.String()) + err = o.q.SelectContext(ctx, &dbEtxs, sql, chainID.String()) + if len(dbEtxs) == 0 { + return nil + } txes = make([]*Tx, len(dbEtxs)) dbEthTxsToEvmEthTxPtrs(dbEtxs, txes) if err = orm.LoadTxesAttempts(ctx, txes); err != nil { diff --git a/core/chains/evm/txmgr/evm_tx_store_test.go b/core/chains/evm/txmgr/evm_tx_store_test.go index 09ec271770f..b0a0a0379f1 100644 --- a/core/chains/evm/txmgr/evm_tx_store_test.go +++ b/core/chains/evm/txmgr/evm_tx_store_test.go @@ -1864,19 +1864,10 @@ func TestORM_FindTransactionsByState(t *testing.T) { mustInsertConfirmedEthTxWithReceipt(t, txStore, fromAddress, 3, 100) mustInsertFatalErrorEthTx(t, txStore, fromAddress) - var txStates []txmgrtypes.TxState - txStates = append(txStates, txmgrcommon.TxUnstarted) - txStates = append(txStates, txmgrcommon.TxInProgress) - txStates = append(txStates, txmgrcommon.TxUnconfirmed) - txStates = append(txStates, txmgrcommon.TxConfirmed) - txStates = append(txStates, txmgrcommon.TxConfirmedMissingReceipt) - txStates = append(txStates, txmgrcommon.TxConfirmed) - - for _, state := range txStates { - txs, err := txStore.FindTransactionsByState(ctx, state, testutils.FixtureChainID) - require.NoError(t, err) - require.Len(t, txs, 1) - } + txs, err := txStore.FindConfirmedTxesAwaitingFinalization(ctx, testutils.FixtureChainID) + require.NoError(t, err) + require.Len(t, txs, 1) + } func TestORM_UpdateTxesFinalized(t *testing.T) { diff --git a/common/txmgr/finalizer.go b/core/chains/evm/txmgr/finalizer.go similarity index 50% rename from common/txmgr/finalizer.go rename to core/chains/evm/txmgr/finalizer.go index dc7bc66db8f..477607b0ecf 100644 --- a/common/txmgr/finalizer.go +++ b/core/chains/evm/txmgr/finalizer.go @@ -2,116 +2,101 @@ package txmgr import ( "context" - "errors" "fmt" "math/big" "sync" + "time" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/rpc" "github.com/smartcontractkit/chainlink-common/pkg/logger" "github.com/smartcontractkit/chainlink-common/pkg/services" "github.com/smartcontractkit/chainlink-common/pkg/utils/mailbox" - feetypes "github.com/smartcontractkit/chainlink/v2/common/fee/types" - txmgrtypes "github.com/smartcontractkit/chainlink/v2/common/txmgr/types" - "github.com/smartcontractkit/chainlink/v2/common/types" + + evmtypes "github.com/smartcontractkit/chainlink/v2/core/chains/evm/types" ) -type finalizerTxStore[CHAIN_ID types.ID, ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, SEQ types.Sequence, FEE feetypes.Fee] interface { - FindTransactionsByState(ctx context.Context, state txmgrtypes.TxState, chainID CHAIN_ID) ([]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error) - UpdateTxesFinalized(ctx context.Context, txs []int64, chainId CHAIN_ID) error +var _ Finalizer = (*evmFinalizer)(nil) + +// processHeadTimeout represents a sanity limit on how long ProcessHead +// should take to complete +const processHeadTimeout = 10 * time.Minute + +type finalizerTxStore interface { + FindConfirmedTxesAwaitingFinalization(ctx context.Context, chainID *big.Int) ([]*Tx, error) + UpdateTxesFinalized(ctx context.Context, txs []int64, chainId *big.Int) error } -type finalizerChainClient[BLOCK_HASH types.Hashable, HEAD types.Head[BLOCK_HASH]] interface { - HeadByHash(ctx context.Context, hash BLOCK_HASH) (HEAD, error) +type finalizerChainClient interface { + BatchCallContext(ctx context.Context, elems []rpc.BatchElem) error } // Finalizer handles processing new finalized blocks and marking transactions as finalized accordingly in the TXM DB -type Finalizer[CHAIN_ID types.ID, ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, SEQ types.Sequence, FEE feetypes.Fee, HEAD types.Head[BLOCK_HASH]] struct { +type evmFinalizer struct { services.StateMachine - lggr logger.SugaredLogger - chainId CHAIN_ID - txStore finalizerTxStore[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] - client finalizerChainClient[BLOCK_HASH, HEAD] - mb *mailbox.Mailbox[HEAD] - stopCh services.StopChan - wg sync.WaitGroup - initSync sync.Mutex - isStarted bool -} - -func NewFinalizer[ - CHAIN_ID types.ID, - ADDR types.Hashable, - TX_HASH types.Hashable, - BLOCK_HASH types.Hashable, - SEQ types.Sequence, - FEE feetypes.Fee, - HEAD types.Head[BLOCK_HASH], -]( + lggr logger.SugaredLogger + chainId *big.Int + txStore finalizerTxStore + client finalizerChainClient + mb *mailbox.Mailbox[*evmtypes.Head] + stopCh services.StopChan + wg sync.WaitGroup +} + +func NewEvmFinalizer( lggr logger.Logger, - chainId CHAIN_ID, - txStore finalizerTxStore[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], - client finalizerChainClient[BLOCK_HASH, HEAD], -) *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD] { + chainId *big.Int, + txStore finalizerTxStore, + client finalizerChainClient, +) *evmFinalizer { lggr = logger.Named(lggr, "Finalizer") - return &Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD]{ + return &evmFinalizer{ txStore: txStore, lggr: logger.Sugared(lggr), chainId: chainId, client: client, - mb: mailbox.NewSingle[HEAD](), + mb: mailbox.NewSingle[*evmtypes.Head](), } } // Start is a comment to appease the linter -func (f *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD]) Start(ctx context.Context) error { +func (f *evmFinalizer) Start(ctx context.Context) error { return f.StartOnce("Finalizer", func() error { return f.startInternal(ctx) }) } -func (f *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD]) startInternal(_ context.Context) error { - f.initSync.Lock() - defer f.initSync.Unlock() - if f.isStarted { - return errors.New("Finalizer is already started") - } - +func (f *evmFinalizer) startInternal(_ context.Context) error { f.stopCh = make(chan struct{}) f.wg = sync.WaitGroup{} f.wg.Add(1) go f.runLoop() - f.isStarted = true return nil } // Close is a comment to appease the linter -func (f *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD]) Close() error { +func (f *evmFinalizer) Close() error { return f.StopOnce("Finalizer", func() error { return f.closeInternal() }) } -func (f *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD]) closeInternal() error { - f.initSync.Lock() - defer f.initSync.Unlock() - if !f.isStarted { - return fmt.Errorf("Finalizer is not started: %w", services.ErrAlreadyStopped) - } +func (f *evmFinalizer) closeInternal() error { close(f.stopCh) f.wg.Wait() - f.isStarted = false return nil } -func (f *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD]) Name() string { +func (f *evmFinalizer) Name() string { return f.lggr.Name() } -func (f *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD]) HealthReport() map[string]error { +func (f *evmFinalizer) HealthReport() map[string]error { return map[string]error{f.Name(): f.Healthy()} } -func (f *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD]) runLoop() { +func (f *evmFinalizer) runLoop() { defer f.wg.Done() ctx, cancel := f.stopCh.NewCtx() defer cancel() @@ -138,14 +123,18 @@ func (f *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD]) runLoop } } -func (f *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD]) ProcessHead(ctx context.Context, head types.Head[BLOCK_HASH]) error { +func (f *evmFinalizer) DeliverHead(head *evmtypes.Head) bool { + return f.mb.Deliver(head) +} + +func (f *evmFinalizer) ProcessHead(ctx context.Context, head *evmtypes.Head) error { ctx, cancel := context.WithTimeout(ctx, processHeadTimeout) defer cancel() return f.processHead(ctx, head) } // Determines if any confirmed transactions can be marked as finalized by comparing their receipts against the latest finalized block -func (f *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD]) processHead(ctx context.Context, head types.Head[BLOCK_HASH]) error { +func (f *evmFinalizer) processHead(ctx context.Context, head *evmtypes.Head) error { latestFinalizedHead := head.LatestFinalizedHead() // Cannot determine finality without a finalized head for comparison if latestFinalizedHead == nil || !latestFinalizedHead.IsValid() { @@ -155,22 +144,23 @@ func (f *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD]) process f.lggr.Debugw("processing latest finalized head", "block num", latestFinalizedHead.BlockNumber(), "block hash", latestFinalizedHead.BlockHash(), "earliest block num in chain", earliestBlockNumInChain) // Retrieve all confirmed transactions, loaded with attempts and receipts - confirmedTxs, err := f.txStore.FindTransactionsByState(ctx, TxConfirmed, f.chainId) + unfinalizedTxs, err := f.txStore.FindConfirmedTxesAwaitingFinalization(ctx, f.chainId) if err != nil { return fmt.Errorf("failed to retrieve confirmed transactions: %w", err) } - var finalizedTxs []*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] + var finalizedTxs []*Tx // Group by block hash transactions whose receipts cannot be validated using the cached heads - receiptBlockHashToTx := make(map[BLOCK_HASH][]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) + receiptBlockHashToTx := make(map[common.Hash][]*Tx) // Find transactions with receipt block nums older than the latest finalized block num and block hashes still in chain - for _, tx := range confirmedTxs { + for _, tx := range unfinalizedTxs { // Only consider transactions not already marked as finalized if tx.Finalized { continue } receipt := tx.GetReceipt() if receipt == nil || receipt.IsZero() || receipt.IsUnmined() { + f.lggr.AssumptionViolationw("invalid receipt found for confirmed transaction", "tx", tx, "receipt", receipt) continue } // Receipt newer than latest finalized head block num @@ -191,22 +181,14 @@ func (f *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD]) process } // Check if block hashes exist for receipts on-chain older than the earliest cached head - // Transactions are grouped by their receipt block hash to minimize the number of RPC calls in case transactions were confirmed in the same block - // This check is only expected to be used in rare cases if there was an issue with the HeadTracker or if the node was down for significant time - var wg sync.WaitGroup - var txMu sync.RWMutex - for receiptBlockHash, txs := range receiptBlockHashToTx { - wg.Add(1) - go func(hash BLOCK_HASH, txs []*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) { - defer wg.Done() - if head, rpcErr := f.client.HeadByHash(ctx, hash); rpcErr == nil && head.IsValid() { - txMu.Lock() - finalizedTxs = append(finalizedTxs, txs...) - txMu.Unlock() - } - }(receiptBlockHash, txs) + // Transactions are grouped by their receipt block hash to avoid repeat requests on the same hash in case transactions were confirmed in the same block + validatedReceiptTxs, err := f.batchCheckReceiptHashes(ctx, receiptBlockHashToTx, latestFinalizedHead.BlockNumber()) + if err != nil { + // Do not error out to allow transactions that did not need RPC validation to still be marked as finalized + // The transcations failed to be validated will be checked again in the next round + f.lggr.Errorf("failed to validate receipt block hashes over RPC: %v", err) } - wg.Wait() + finalizedTxs = append(finalizedTxs, validatedReceiptTxs...) etxIDs := f.buildTxIdList(finalizedTxs) @@ -217,8 +199,50 @@ func (f *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD]) process return nil } +func (f *evmFinalizer) batchCheckReceiptHashes(ctx context.Context, receiptMap map[common.Hash][]*Tx, latestFinalizedBlockNum int64) ([]*Tx, error) { + if len(receiptMap) == 0 { + return nil, nil + } + var rpcBatchCalls []rpc.BatchElem + for hash := range receiptMap { + elem := rpc.BatchElem{ + Method: "eth_getBlockByHash", + Args: []any{ + hash, + false, + }, + Result: new(evmtypes.Head), + } + rpcBatchCalls = append(rpcBatchCalls, elem) + } + + err := f.client.BatchCallContext(ctx, rpcBatchCalls) + if err != nil { + return nil, fmt.Errorf("get block hash batch call failed: %w", err) + } + var finalizedTxs []*Tx + for _, req := range rpcBatchCalls { + if req.Error != nil { + f.lggr.Debugw("failed to find block by hash", "hash", req.Args[0]) + continue + } + head := req.Result.(*evmtypes.Head) + if head == nil { + f.lggr.Debugw("failed to find block by hash", "hash", req.Args[0]) + continue + } + // Confirmed receipt's block hash exists on-chain still + // Add to finalized list if block num less than or equal to the latest finalized head block num + if head.BlockNumber() <= latestFinalizedBlockNum { + txs := receiptMap[head.BlockHash()] + finalizedTxs = append(finalizedTxs, txs...) + } + } + return finalizedTxs, nil +} + // Build list of transaction IDs -func (f *Finalizer[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE, HEAD]) buildTxIdList(finalizedTxs []*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) []int64 { +func (f *evmFinalizer) buildTxIdList(finalizedTxs []*Tx) []int64 { etxIDs := make([]int64, len(finalizedTxs)) for i, tx := range finalizedTxs { receipt := tx.GetReceipt() diff --git a/core/chains/evm/txmgr/finalizer_test.go b/core/chains/evm/txmgr/finalizer_test.go index 263f77fafc0..a9e7b33dc32 100644 --- a/core/chains/evm/txmgr/finalizer_test.go +++ b/core/chains/evm/txmgr/finalizer_test.go @@ -5,6 +5,7 @@ import ( "time" "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/rpc" "github.com/google/uuid" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" @@ -29,7 +30,7 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { feeLimit := uint64(10_000) ethClient := testutils.NewEthClientMockWithDefaultChain(t) - finalizer := txmgr.NewEvmFinalizer(logger.Test(t), testutils.FixtureChainID, txStore, txmgr.NewEvmTxmClient(ethClient, nil)) + finalizer := txmgr.NewEvmFinalizer(logger.Test(t), testutils.FixtureChainID, txStore, ethClient) err := finalizer.Start(ctx) require.NoError(t, err) @@ -138,7 +139,17 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { // Insert receipt for finalized block num receiptHash := utils.NewHash() mustInsertEthReceipt(t, txStore, head.Parent.Number-1, receiptHash, attemptHash) - ethClient.On("HeadByHash", mock.Anything, receiptHash).Return(&evmtypes.Head{Number: head.Parent.Number - 1, Hash: receiptHash}, nil) + ethClient.On("BatchCallContext", mock.Anything, mock.IsType([]rpc.BatchElem{})).Run(func(args mock.Arguments) { + rpcElements := args.Get(1).([]rpc.BatchElem) + require.Equal(t, 1, len(rpcElements)) + + require.Equal(t, "eth_getBlockByHash", rpcElements[0].Method) + require.Equal(t, receiptHash.String(), rpcElements[0].Args[0].(common.Hash).String()) + require.Equal(t, false, rpcElements[0].Args[1]) + + head := evmtypes.Head{Number: head.Parent.Number - 1, Hash: receiptHash} + rpcElements[0].Result = &head + }).Return(nil).Once() err = finalizer.ProcessHead(ctx, head) require.NoError(t, err) tx, err = txStore.FindTxWithIdempotencyKey(ctx, idempotencyKey, testutils.FixtureChainID) diff --git a/core/chains/evm/txmgr/mocks/evm_tx_store.go b/core/chains/evm/txmgr/mocks/evm_tx_store.go index 9c1c6eeabeb..454d6ab8f20 100644 --- a/core/chains/evm/txmgr/mocks/evm_tx_store.go +++ b/core/chains/evm/txmgr/mocks/evm_tx_store.go @@ -199,6 +199,36 @@ func (_m *EvmTxStore) DeleteInProgressAttempt(ctx context.Context, attempt types return r0 } +// FindConfirmedTxesAwaitingFinalization provides a mock function with given fields: ctx, chainID +func (_m *EvmTxStore) FindConfirmedTxesAwaitingFinalization(ctx context.Context, chainID *big.Int) ([]*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error) { + ret := _m.Called(ctx, chainID) + + if len(ret) == 0 { + panic("no return value specified for FindConfirmedTxesAwaitingFinalization") + } + + var r0 []*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee] + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, *big.Int) ([]*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error)); ok { + return rf(ctx, chainID) + } + if rf, ok := ret.Get(0).(func(context.Context, *big.Int) []*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]); ok { + r0 = rf(ctx, chainID) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, *big.Int) error); ok { + r1 = rf(ctx, chainID) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + // FindEarliestUnconfirmedBroadcastTime provides a mock function with given fields: ctx, chainID func (_m *EvmTxStore) FindEarliestUnconfirmedBroadcastTime(ctx context.Context, chainID *big.Int) (null.Time, error) { ret := _m.Called(ctx, chainID) @@ -313,36 +343,6 @@ func (_m *EvmTxStore) FindNextUnstartedTransactionFromAddress(ctx context.Contex return r0, r1 } -// FindTransactionsByState provides a mock function with given fields: ctx, state, chainID -func (_m *EvmTxStore) FindTransactionsByState(ctx context.Context, state types.TxState, chainID *big.Int) ([]*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error) { - ret := _m.Called(ctx, state, chainID) - - if len(ret) == 0 { - panic("no return value specified for FindTransactionsByState") - } - - var r0 []*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee] - var r1 error - if rf, ok := ret.Get(0).(func(context.Context, types.TxState, *big.Int) ([]*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error)); ok { - return rf(ctx, state, chainID) - } - if rf, ok := ret.Get(0).(func(context.Context, types.TxState, *big.Int) []*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]); ok { - r0 = rf(ctx, state, chainID) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).([]*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]) - } - } - - if rf, ok := ret.Get(1).(func(context.Context, types.TxState, *big.Int) error); ok { - r1 = rf(ctx, state, chainID) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - // FindTransactionsConfirmedInBlockRange provides a mock function with given fields: ctx, highBlockNumber, lowBlockNumber, chainID func (_m *EvmTxStore) FindTransactionsConfirmedInBlockRange(ctx context.Context, highBlockNumber int64, lowBlockNumber int64, chainID *big.Int) ([]*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error) { ret := _m.Called(ctx, highBlockNumber, lowBlockNumber, chainID) @@ -1131,17 +1131,17 @@ func (_m *EvmTxStore) PruneUnstartedTxQueue(ctx context.Context, queueSize uint3 return r0, r1 } -// ReapTxHistory provides a mock function with given fields: ctx, minBlockNumberToKeep, timeThreshold, chainID -func (_m *EvmTxStore) ReapTxHistory(ctx context.Context, minBlockNumberToKeep int64, timeThreshold time.Time, chainID *big.Int) error { - ret := _m.Called(ctx, minBlockNumberToKeep, timeThreshold, chainID) +// ReapTxHistory provides a mock function with given fields: ctx, timeThreshold, chainID +func (_m *EvmTxStore) ReapTxHistory(ctx context.Context, timeThreshold time.Time, chainID *big.Int) error { + ret := _m.Called(ctx, timeThreshold, chainID) if len(ret) == 0 { panic("no return value specified for ReapTxHistory") } var r0 error - if rf, ok := ret.Get(0).(func(context.Context, int64, time.Time, *big.Int) error); ok { - r0 = rf(ctx, minBlockNumberToKeep, timeThreshold, chainID) + if rf, ok := ret.Get(0).(func(context.Context, time.Time, *big.Int) error); ok { + r0 = rf(ctx, timeThreshold, chainID) } else { r0 = ret.Error(0) } diff --git a/core/chains/evm/txmgr/models.go b/core/chains/evm/txmgr/models.go index 92e18b820bf..5361402c469 100644 --- a/core/chains/evm/txmgr/models.go +++ b/core/chains/evm/txmgr/models.go @@ -42,7 +42,7 @@ type ( TxmClient = txmgrtypes.TxmClient[*big.Int, common.Address, common.Hash, common.Hash, *evmtypes.Receipt, evmtypes.Nonce, gas.EvmFee, *evmtypes.Head] TransactionClient = txmgrtypes.TransactionClient[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee] ChainReceipt = txmgrtypes.ChainReceipt[common.Hash, common.Hash] - Finalizer = txmgr.Finalizer[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee, *evmtypes.Head] + Finalizer = txmgrtypes.Finalizer[common.Hash, *evmtypes.Head] ) var _ KeyStore = (keystore.Eth)(nil) // check interface in txmgr to avoid circular import diff --git a/core/chains/evm/txmgr/reaper_test.go b/core/chains/evm/txmgr/reaper_test.go index b3ce48b702c..d1d84939127 100644 --- a/core/chains/evm/txmgr/reaper_test.go +++ b/core/chains/evm/txmgr/reaper_test.go @@ -12,18 +12,17 @@ import ( "github.com/smartcontractkit/chainlink-common/pkg/utils" txmgrtypes "github.com/smartcontractkit/chainlink/v2/common/txmgr/types" - txmgrmocks "github.com/smartcontractkit/chainlink/v2/common/txmgr/types/mocks" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/txmgr" "github.com/smartcontractkit/chainlink/v2/core/internal/cltest" "github.com/smartcontractkit/chainlink/v2/core/internal/testutils/pgtest" ) -func newReaperWithChainID(t *testing.T, db txmgrtypes.TxHistoryReaper[*big.Int], cfg txmgrtypes.ReaperChainConfig, txConfig txmgrtypes.ReaperTransactionsConfig, cid *big.Int) *txmgr.Reaper { - return txmgr.NewEvmReaper(logger.Test(t), db, cfg, txConfig, cid) +func newReaperWithChainID(t *testing.T, db txmgrtypes.TxHistoryReaper[*big.Int], txConfig txmgrtypes.ReaperTransactionsConfig, cid *big.Int) *txmgr.Reaper { + return txmgr.NewEvmReaper(logger.Test(t), db, txConfig, cid) } -func newReaper(t *testing.T, db txmgrtypes.TxHistoryReaper[*big.Int], cfg txmgrtypes.ReaperChainConfig, txConfig txmgrtypes.ReaperTransactionsConfig) *txmgr.Reaper { - return newReaperWithChainID(t, db, cfg, txConfig, &cltest.FixtureChainID) +func newReaper(t *testing.T, db txmgrtypes.TxHistoryReaper[*big.Int], txConfig txmgrtypes.ReaperTransactionsConfig) *txmgr.Reaper { + return newReaperWithChainID(t, db, txConfig, &cltest.FixtureChainID) } type reaperConfig struct { @@ -51,12 +50,9 @@ func TestReaper_ReapTxes(t *testing.T) { oneDayAgo := time.Now().Add(-24 * time.Hour) t.Run("with nothing in the database, doesn't error", func(t *testing.T) { - config := txmgrmocks.NewReaperConfig(t) - config.On("FinalityDepth").Return(uint32(10)) - tc := &reaperConfig{reaperThreshold: 1 * time.Hour} - r := newReaper(t, txStore, config, tc) + r := newReaper(t, txStore, tc) err := r.ReapTxes(42) assert.NoError(t, err) @@ -66,11 +62,10 @@ func TestReaper_ReapTxes(t *testing.T) { mustInsertConfirmedEthTxWithReceipt(t, txStore, from, nonce, 5) t.Run("skips if threshold=0", func(t *testing.T) { - config := txmgrmocks.NewReaperConfig(t) tc := &reaperConfig{reaperThreshold: 0 * time.Second} - r := newReaper(t, txStore, config, tc) + r := newReaper(t, txStore, tc) err := r.ReapTxes(42) assert.NoError(t, err) @@ -79,12 +74,9 @@ func TestReaper_ReapTxes(t *testing.T) { }) t.Run("doesn't touch ethtxes with different chain ID", func(t *testing.T) { - config := txmgrmocks.NewReaperConfig(t) - config.On("FinalityDepth").Return(uint32(10)) - tc := &reaperConfig{reaperThreshold: 1 * time.Hour} - r := newReaperWithChainID(t, txStore, config, tc, big.NewInt(42)) + r := newReaperWithChainID(t, txStore, tc, big.NewInt(42)) err := r.ReapTxes(42) assert.NoError(t, err) @@ -92,13 +84,10 @@ func TestReaper_ReapTxes(t *testing.T) { cltest.AssertCount(t, db, "evm.txes", 1) }) - t.Run("deletes confirmed evm.txes that exceed the age threshold with at least EVM.FinalityDepth blocks above their receipt", func(t *testing.T) { - config := txmgrmocks.NewReaperConfig(t) - config.On("FinalityDepth").Return(uint32(10)) - + t.Run("deletes confirmed evm.txes marked as finalized that exceed the age threshold", func(t *testing.T) { tc := &reaperConfig{reaperThreshold: 1 * time.Hour} - r := newReaper(t, txStore, config, tc) + r := newReaper(t, txStore, tc) err := r.ReapTxes(42) assert.NoError(t, err) @@ -109,9 +98,11 @@ func TestReaper_ReapTxes(t *testing.T) { err = r.ReapTxes(12) assert.NoError(t, err) - // Didn't delete because eth_tx although old enough, was still within EVM.FinalityDepth of the current head + // Didn't delete because eth_tx although old enough, was not marked as finalized cltest.AssertCount(t, db, "evm.txes", 1) + pgtest.MustExec(t, db, `UPDATE evm.txes SET finalized=true`) + err = r.ReapTxes(42) assert.NoError(t, err) // Now it deleted because the eth_tx was past EVM.FinalityDepth @@ -121,12 +112,9 @@ func TestReaper_ReapTxes(t *testing.T) { mustInsertFatalErrorEthTx(t, txStore, from) t.Run("deletes errored evm.txes that exceed the age threshold", func(t *testing.T) { - config := txmgrmocks.NewReaperConfig(t) - config.On("FinalityDepth").Return(uint32(10)) - tc := &reaperConfig{reaperThreshold: 1 * time.Hour} - r := newReaper(t, txStore, config, tc) + r := newReaper(t, txStore, tc) err := r.ReapTxes(42) assert.NoError(t, err) From 8af7d77ae5bd5817cf5208f32722162a048f8aea Mon Sep 17 00:00:00 2001 From: amit-momin Date: Mon, 24 Jun 2024 17:32:37 -0500 Subject: [PATCH 03/23] Fixed linting and renumbered sql migration --- core/chains/evm/txmgr/evm_tx_store_test.go | 1 - core/chains/evm/txmgr/finalizer.go | 2 +- core/chains/evm/txmgr/reaper_test.go | 1 - ..._finalized_column.sql => 0246_add_tx_finalized_column.sql} | 4 ++-- 4 files changed, 3 insertions(+), 5 deletions(-) rename core/store/migrate/migrations/{0245_add_tx_finalized_column.sql => 0246_add_tx_finalized_column.sql} (79%) diff --git a/core/chains/evm/txmgr/evm_tx_store_test.go b/core/chains/evm/txmgr/evm_tx_store_test.go index b0a0a0379f1..dde8e047bfd 100644 --- a/core/chains/evm/txmgr/evm_tx_store_test.go +++ b/core/chains/evm/txmgr/evm_tx_store_test.go @@ -1867,7 +1867,6 @@ func TestORM_FindTransactionsByState(t *testing.T) { txs, err := txStore.FindConfirmedTxesAwaitingFinalization(ctx, testutils.FixtureChainID) require.NoError(t, err) require.Len(t, txs, 1) - } func TestORM_UpdateTxesFinalized(t *testing.T) { diff --git a/core/chains/evm/txmgr/finalizer.go b/core/chains/evm/txmgr/finalizer.go index 477607b0ecf..87a0726ecaa 100644 --- a/core/chains/evm/txmgr/finalizer.go +++ b/core/chains/evm/txmgr/finalizer.go @@ -185,7 +185,7 @@ func (f *evmFinalizer) processHead(ctx context.Context, head *evmtypes.Head) err validatedReceiptTxs, err := f.batchCheckReceiptHashes(ctx, receiptBlockHashToTx, latestFinalizedHead.BlockNumber()) if err != nil { // Do not error out to allow transactions that did not need RPC validation to still be marked as finalized - // The transcations failed to be validated will be checked again in the next round + // The transactions failed to be validated will be checked again in the next round f.lggr.Errorf("failed to validate receipt block hashes over RPC: %v", err) } finalizedTxs = append(finalizedTxs, validatedReceiptTxs...) diff --git a/core/chains/evm/txmgr/reaper_test.go b/core/chains/evm/txmgr/reaper_test.go index d1d84939127..eaa7eecb252 100644 --- a/core/chains/evm/txmgr/reaper_test.go +++ b/core/chains/evm/txmgr/reaper_test.go @@ -62,7 +62,6 @@ func TestReaper_ReapTxes(t *testing.T) { mustInsertConfirmedEthTxWithReceipt(t, txStore, from, nonce, 5) t.Run("skips if threshold=0", func(t *testing.T) { - tc := &reaperConfig{reaperThreshold: 0 * time.Second} r := newReaper(t, txStore, tc) diff --git a/core/store/migrate/migrations/0245_add_tx_finalized_column.sql b/core/store/migrate/migrations/0246_add_tx_finalized_column.sql similarity index 79% rename from core/store/migrate/migrations/0245_add_tx_finalized_column.sql rename to core/store/migrate/migrations/0246_add_tx_finalized_column.sql index cc0b09fd3dd..5f2c5c5ffb2 100644 --- a/core/store/migrate/migrations/0245_add_tx_finalized_column.sql +++ b/core/store/migrate/migrations/0246_add_tx_finalized_column.sql @@ -2,9 +2,9 @@ -- +goose StatementBegin ALTER TABLE evm.txes ADD COLUMN finalized boolean NOT NULL DEFAULT false; ALTER TABLE evm.txes ADD CONSTRAINT chk_eth_txes_state_finalized CHECK ( - state <> 'confirmed'::eth_txes_state AND finalized = false + state <> 'confirmed'::evm.txes_state AND finalized = false OR - state = 'confirmed'::eth_txes_state + state = 'confirmed'::evm.txes_state ) NOT VALID; -- +goose StatementEnd From c809c1abe6849507a369dc305011f265762c29f3 Mon Sep 17 00:00:00 2001 From: amit-momin Date: Tue, 25 Jun 2024 16:22:58 -0500 Subject: [PATCH 04/23] Added limit to Finalizer RPC batch calls --- core/chains/evm/txmgr/builder.go | 2 +- core/chains/evm/txmgr/finalizer.go | 81 +++++++++++++++---------- core/chains/evm/txmgr/finalizer_test.go | 39 +++++++++--- 3 files changed, 81 insertions(+), 41 deletions(-) diff --git a/core/chains/evm/txmgr/builder.go b/core/chains/evm/txmgr/builder.go index 23d802c521b..4e6949dcca7 100644 --- a/core/chains/evm/txmgr/builder.go +++ b/core/chains/evm/txmgr/builder.go @@ -53,7 +53,7 @@ func NewTxm( evmTracker := NewEvmTracker(txStore, keyStore, chainID, lggr) stuckTxDetector := NewStuckTxDetector(lggr, client.ConfiguredChainID(), chainConfig.ChainType(), fCfg.PriceMax(), txConfig.AutoPurge(), estimator, txStore, client) evmConfirmer := NewEvmConfirmer(txStore, txmClient, txmCfg, feeCfg, txConfig, dbConfig, keyStore, txAttemptBuilder, lggr, stuckTxDetector) - evmFinalizer := NewEvmFinalizer(lggr, client.ConfiguredChainID(), txStore, client) + evmFinalizer := NewEvmFinalizer(lggr, client.ConfiguredChainID(), chainConfig.RPCDefaultBatchSize(), txStore, client) var evmResender *Resender if txConfig.ResendAfterThreshold() > 0 { evmResender = NewEvmResender(lggr, txStore, txmClient, evmTracker, keyStore, txmgr.DefaultResenderPollInterval, chainConfig, txConfig) diff --git a/core/chains/evm/txmgr/finalizer.go b/core/chains/evm/txmgr/finalizer.go index 87a0726ecaa..03c160726ac 100644 --- a/core/chains/evm/txmgr/finalizer.go +++ b/core/chains/evm/txmgr/finalizer.go @@ -35,28 +35,31 @@ type finalizerChainClient interface { // Finalizer handles processing new finalized blocks and marking transactions as finalized accordingly in the TXM DB type evmFinalizer struct { services.StateMachine - lggr logger.SugaredLogger - chainId *big.Int - txStore finalizerTxStore - client finalizerChainClient - mb *mailbox.Mailbox[*evmtypes.Head] - stopCh services.StopChan - wg sync.WaitGroup + lggr logger.SugaredLogger + chainId *big.Int + rpcBatchSize int + txStore finalizerTxStore + client finalizerChainClient + mb *mailbox.Mailbox[*evmtypes.Head] + stopCh services.StopChan + wg sync.WaitGroup } func NewEvmFinalizer( lggr logger.Logger, chainId *big.Int, + rpcBatchSize uint32, txStore finalizerTxStore, client finalizerChainClient, ) *evmFinalizer { lggr = logger.Named(lggr, "Finalizer") return &evmFinalizer{ - txStore: txStore, - lggr: logger.Sugared(lggr), - chainId: chainId, - client: client, - mb: mailbox.NewSingle[*evmtypes.Head](), + lggr: logger.Sugared(lggr), + chainId: chainId, + rpcBatchSize: int(rpcBatchSize), + txStore: txStore, + client: client, + mb: mailbox.NewSingle[*evmtypes.Head](), } } @@ -182,7 +185,7 @@ func (f *evmFinalizer) processHead(ctx context.Context, head *evmtypes.Head) err // Check if block hashes exist for receipts on-chain older than the earliest cached head // Transactions are grouped by their receipt block hash to avoid repeat requests on the same hash in case transactions were confirmed in the same block - validatedReceiptTxs, err := f.batchCheckReceiptHashes(ctx, receiptBlockHashToTx, latestFinalizedHead.BlockNumber()) + validatedReceiptTxs, err := f.batchCheckReceiptHashesOnchain(ctx, receiptBlockHashToTx, latestFinalizedHead.BlockNumber()) if err != nil { // Do not error out to allow transactions that did not need RPC validation to still be marked as finalized // The transactions failed to be validated will be checked again in the next round @@ -199,11 +202,13 @@ func (f *evmFinalizer) processHead(ctx context.Context, head *evmtypes.Head) err return nil } -func (f *evmFinalizer) batchCheckReceiptHashes(ctx context.Context, receiptMap map[common.Hash][]*Tx, latestFinalizedBlockNum int64) ([]*Tx, error) { +func (f *evmFinalizer) batchCheckReceiptHashesOnchain(ctx context.Context, receiptMap map[common.Hash][]*Tx, latestFinalizedBlockNum int64) ([]*Tx, error) { if len(receiptMap) == 0 { return nil, nil } - var rpcBatchCalls []rpc.BatchElem + // Group the RPC batch calls in groups of rpcBatchSize + var rpcBatchGroups [][]rpc.BatchElem + var rpcBatch []rpc.BatchElem for hash := range receiptMap { elem := rpc.BatchElem{ Method: "eth_getBlockByHash", @@ -213,29 +218,39 @@ func (f *evmFinalizer) batchCheckReceiptHashes(ctx context.Context, receiptMap m }, Result: new(evmtypes.Head), } - rpcBatchCalls = append(rpcBatchCalls, elem) + rpcBatch = append(rpcBatch, elem) + if len(rpcBatch) >= f.rpcBatchSize { + rpcBatchGroups = append(rpcBatchGroups, rpcBatch) + rpcBatch = []rpc.BatchElem{} + } } - err := f.client.BatchCallContext(ctx, rpcBatchCalls) - if err != nil { - return nil, fmt.Errorf("get block hash batch call failed: %w", err) - } var finalizedTxs []*Tx - for _, req := range rpcBatchCalls { - if req.Error != nil { - f.lggr.Debugw("failed to find block by hash", "hash", req.Args[0]) - continue - } - head := req.Result.(*evmtypes.Head) - if head == nil { - f.lggr.Debugw("failed to find block by hash", "hash", req.Args[0]) + for _, rpcBatch := range rpcBatchGroups { + err := f.client.BatchCallContext(ctx, rpcBatch) + if err != nil { + // Continue if batch RPC call failed so other batches can still be considered for finalization + f.lggr.Debugw("failed to find blocks due to batch call failure") continue } - // Confirmed receipt's block hash exists on-chain still - // Add to finalized list if block num less than or equal to the latest finalized head block num - if head.BlockNumber() <= latestFinalizedBlockNum { - txs := receiptMap[head.BlockHash()] - finalizedTxs = append(finalizedTxs, txs...) + for _, req := range rpcBatch { + if req.Error != nil { + // Continue if particular RPC call failed so other txs can still be considered for finalization + f.lggr.Debugw("failed to find block by hash", "hash", req.Args[0]) + continue + } + head := req.Result.(*evmtypes.Head) + if head == nil { + // Continue if particular RPC call yielded a nil head so other txs can still be considered for finalization + f.lggr.Debugw("failed to find block by hash", "hash", req.Args[0]) + continue + } + // Confirmed receipt's block hash exists on-chain + // Add to finalized list if block num less than or equal to the latest finalized head block num + if head.BlockNumber() <= latestFinalizedBlockNum { + txs := receiptMap[head.BlockHash()] + finalizedTxs = append(finalizedTxs, txs...) + } } } return finalizedTxs, nil diff --git a/core/chains/evm/txmgr/finalizer_test.go b/core/chains/evm/txmgr/finalizer_test.go index a9e7b33dc32..559eaa42485 100644 --- a/core/chains/evm/txmgr/finalizer_test.go +++ b/core/chains/evm/txmgr/finalizer_test.go @@ -29,8 +29,9 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { ethKeyStore := cltest.NewKeyStore(t, db).Eth() feeLimit := uint64(10_000) ethClient := testutils.NewEthClientMockWithDefaultChain(t) + rpcBatchSize := uint32(1) - finalizer := txmgr.NewEvmFinalizer(logger.Test(t), testutils.FixtureChainID, txStore, ethClient) + finalizer := txmgr.NewEvmFinalizer(logger.Test(t), testutils.FixtureChainID, rpcBatchSize, txStore, ethClient) err := finalizer.Start(ctx) require.NoError(t, err) @@ -137,19 +138,43 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { } attemptHash := insertTxAndAttemptWithIdempotencyKey(t, txStore, tx, idempotencyKey) // Insert receipt for finalized block num - receiptHash := utils.NewHash() - mustInsertEthReceipt(t, txStore, head.Parent.Number-1, receiptHash, attemptHash) + receiptHash1 := utils.NewHash() + mustInsertEthReceipt(t, txStore, head.Parent.Number-2, receiptHash1, attemptHash) + idempotencyKey = uuid.New().String() + nonce = evmtypes.Nonce(1) + tx = &txmgr.Tx{ + Sequence: &nonce, + IdempotencyKey: &idempotencyKey, + FromAddress: fromAddress, + EncodedPayload: []byte{1, 2, 3}, + FeeLimit: feeLimit, + State: txmgrcommon.TxConfirmed, + BroadcastAt: &broadcast, + InitialBroadcastAt: &broadcast, + } + attemptHash = insertTxAndAttemptWithIdempotencyKey(t, txStore, tx, idempotencyKey) + // Insert receipt for finalized block num + receiptHash2 := utils.NewHash() + mustInsertEthReceipt(t, txStore, head.Parent.Number-1, receiptHash2, attemptHash) + // Separate batch calls will be made for each tx due to RPC batch size set to 1 when finalizer initialized above ethClient.On("BatchCallContext", mock.Anything, mock.IsType([]rpc.BatchElem{})).Run(func(args mock.Arguments) { rpcElements := args.Get(1).([]rpc.BatchElem) require.Equal(t, 1, len(rpcElements)) require.Equal(t, "eth_getBlockByHash", rpcElements[0].Method) - require.Equal(t, receiptHash.String(), rpcElements[0].Args[0].(common.Hash).String()) require.Equal(t, false, rpcElements[0].Args[1]) - head := evmtypes.Head{Number: head.Parent.Number - 1, Hash: receiptHash} - rpcElements[0].Result = &head - }).Return(nil).Once() + reqHash := rpcElements[0].Args[0].(common.Hash).String() + var headResult evmtypes.Head + if receiptHash1.String() == reqHash { + headResult = evmtypes.Head{Number: head.Parent.Number - 2, Hash: receiptHash1} + } else if receiptHash2.String() == reqHash { + headResult = evmtypes.Head{Number: head.Parent.Number - 1, Hash: receiptHash2} + } else { + require.Fail(t, "unrecognized block hash") + } + rpcElements[0].Result = &headResult + }).Return(nil).Twice() err = finalizer.ProcessHead(ctx, head) require.NoError(t, err) tx, err = txStore.FindTxWithIdempotencyKey(ctx, idempotencyKey, testutils.FixtureChainID) From 4dab78b98ffdb72bcbb60ab83eb385f7cbedab3d Mon Sep 17 00:00:00 2001 From: amit-momin Date: Wed, 26 Jun 2024 12:35:05 -0500 Subject: [PATCH 05/23] Cleaned up unneeded code --- common/txmgr/confirmer.go | 4 +- common/txmgr/test_helpers.go | 2 +- common/txmgr/types/client.go | 2 - .../txmgr/types/mocks/reaper_chain_config.go | 42 ------------------- core/chains/evm/txmgr/finalizer.go | 19 ++------- core/chains/evm/txmgr/models.go | 2 +- 6 files changed, 8 insertions(+), 63 deletions(-) delete mode 100644 common/txmgr/types/mocks/reaper_chain_config.go diff --git a/common/txmgr/confirmer.go b/common/txmgr/confirmer.go index 2cdc48bea19..a5c2af0d4d7 100644 --- a/common/txmgr/confirmer.go +++ b/common/txmgr/confirmer.go @@ -120,7 +120,7 @@ type Confirmer[ services.StateMachine txStore txmgrtypes.TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] lggr logger.SugaredLogger - client txmgrtypes.TxmClient[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE, HEAD] + client txmgrtypes.TxmClient[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE] txmgrtypes.TxAttemptBuilder[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] stuckTxDetector txmgrtypes.StuckTxDetector[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] resumeCallback ResumeCallback @@ -154,7 +154,7 @@ func NewConfirmer[ FEE feetypes.Fee, ]( txStore txmgrtypes.TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE], - client txmgrtypes.TxmClient[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE, HEAD], + client txmgrtypes.TxmClient[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE], chainConfig txmgrtypes.ConfirmerChainConfig, feeConfig txmgrtypes.ConfirmerFeeConfig, txConfig txmgrtypes.ConfirmerTransactionsConfig, diff --git a/common/txmgr/test_helpers.go b/common/txmgr/test_helpers.go index ef3866815d5..3051e0985d8 100644 --- a/common/txmgr/test_helpers.go +++ b/common/txmgr/test_helpers.go @@ -10,7 +10,7 @@ import ( // TEST ONLY FUNCTIONS // these need to be exported for the txmgr tests to continue to work -func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) XXXTestSetClient(client txmgrtypes.TxmClient[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE, HEAD]) { +func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) XXXTestSetClient(client txmgrtypes.TxmClient[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) { ec.client = client } diff --git a/common/txmgr/types/client.go b/common/txmgr/types/client.go index ea3ed3af29c..759b15d6162 100644 --- a/common/txmgr/types/client.go +++ b/common/txmgr/types/client.go @@ -21,7 +21,6 @@ type TxmClient[ R ChainReceipt[TX_HASH, BLOCK_HASH], SEQ types.Sequence, FEE feetypes.Fee, - HEAD types.Head[BLOCK_HASH], ] interface { ChainClient[CHAIN_ID, ADDR, SEQ] TransactionClient[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] @@ -31,7 +30,6 @@ type TxmClient[ ctx context.Context, attempts []TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], ) (txReceipt []R, txErr []error, err error) - HeadByHash(ctx context.Context, hash BLOCK_HASH) (HEAD, error) } // TransactionClient contains the methods for building, simulating, broadcasting transactions diff --git a/common/txmgr/types/mocks/reaper_chain_config.go b/common/txmgr/types/mocks/reaper_chain_config.go deleted file mode 100644 index 30f58c10003..00000000000 --- a/common/txmgr/types/mocks/reaper_chain_config.go +++ /dev/null @@ -1,42 +0,0 @@ -// Code generated by mockery v2.43.2. DO NOT EDIT. - -package mocks - -import mock "github.com/stretchr/testify/mock" - -// ReaperConfig is an autogenerated mock type for the ReaperChainConfig type -type ReaperConfig struct { - mock.Mock -} - -// FinalityDepth provides a mock function with given fields: -func (_m *ReaperConfig) FinalityDepth() uint32 { - ret := _m.Called() - - if len(ret) == 0 { - panic("no return value specified for FinalityDepth") - } - - var r0 uint32 - if rf, ok := ret.Get(0).(func() uint32); ok { - r0 = rf() - } else { - r0 = ret.Get(0).(uint32) - } - - return r0 -} - -// NewReaperConfig creates a new instance of ReaperConfig. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. -// The first argument is typically a *testing.T value. -func NewReaperConfig(t interface { - mock.TestingT - Cleanup(func()) -}) *ReaperConfig { - mock := &ReaperConfig{} - mock.Mock.Test(t) - - t.Cleanup(func() { mock.AssertExpectations(t) }) - - return mock -} diff --git a/core/chains/evm/txmgr/finalizer.go b/core/chains/evm/txmgr/finalizer.go index 03c160726ac..eba6d5a006b 100644 --- a/core/chains/evm/txmgr/finalizer.go +++ b/core/chains/evm/txmgr/finalizer.go @@ -63,29 +63,18 @@ func NewEvmFinalizer( } } -// Start is a comment to appease the linter +// Start the finalizer func (f *evmFinalizer) Start(ctx context.Context) error { - return f.StartOnce("Finalizer", func() error { - return f.startInternal(ctx) - }) -} - -func (f *evmFinalizer) startInternal(_ context.Context) error { + f.lggr.Debugf("started Finalizer with RPC batch size limit: %d", f.rpcBatchSize) f.stopCh = make(chan struct{}) - f.wg = sync.WaitGroup{} f.wg.Add(1) go f.runLoop() return nil } -// Close is a comment to appease the linter +// Close the finalizer func (f *evmFinalizer) Close() error { - return f.StopOnce("Finalizer", func() error { - return f.closeInternal() - }) -} - -func (f *evmFinalizer) closeInternal() error { + f.lggr.Debug("closing Finalizer") close(f.stopCh) f.wg.Wait() return nil diff --git a/core/chains/evm/txmgr/models.go b/core/chains/evm/txmgr/models.go index 5361402c469..557e3957445 100644 --- a/core/chains/evm/txmgr/models.go +++ b/core/chains/evm/txmgr/models.go @@ -39,7 +39,7 @@ type ( Receipt = dbReceipt // EvmReceipt is the exported DB table model for receipts ReceiptPlus = txmgrtypes.ReceiptPlus[*evmtypes.Receipt] StuckTxDetector = txmgrtypes.StuckTxDetector[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee] - TxmClient = txmgrtypes.TxmClient[*big.Int, common.Address, common.Hash, common.Hash, *evmtypes.Receipt, evmtypes.Nonce, gas.EvmFee, *evmtypes.Head] + TxmClient = txmgrtypes.TxmClient[*big.Int, common.Address, common.Hash, common.Hash, *evmtypes.Receipt, evmtypes.Nonce, gas.EvmFee] TransactionClient = txmgrtypes.TransactionClient[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee] ChainReceipt = txmgrtypes.ChainReceipt[common.Hash, common.Hash] Finalizer = txmgrtypes.Finalizer[common.Hash, *evmtypes.Head] From 37807b90f759cd57fc6962a145962710b931ec84 Mon Sep 17 00:00:00 2001 From: amit-momin Date: Fri, 28 Jun 2024 12:48:28 -0500 Subject: [PATCH 06/23] Renumbered sql migration --- ...d_tx_finalized_column.sql => 0247_add_tx_finalized_column.sql} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename core/store/migrate/migrations/{0246_add_tx_finalized_column.sql => 0247_add_tx_finalized_column.sql} (100%) diff --git a/core/store/migrate/migrations/0246_add_tx_finalized_column.sql b/core/store/migrate/migrations/0247_add_tx_finalized_column.sql similarity index 100% rename from core/store/migrate/migrations/0246_add_tx_finalized_column.sql rename to core/store/migrate/migrations/0247_add_tx_finalized_column.sql From 2a47486ca412070df3d299d62ddf8554dd800267 Mon Sep 17 00:00:00 2001 From: amit-momin Date: Mon, 1 Jul 2024 13:28:23 -0500 Subject: [PATCH 07/23] Updated Finalizer to use LatestAndFinalizedBlock method from HeadTracker --- common/txmgr/txmgr.go | 2 +- common/txmgr/types/finalizer.go | 2 +- .../evm/headtracker/simulated_head_tracker.go | 29 ++++++++++++++++ core/chains/evm/txmgr/builder.go | 4 ++- core/chains/evm/txmgr/finalizer.go | 22 ++++++++++--- core/chains/evm/txmgr/finalizer_test.go | 28 +++++++++++++++- core/chains/evm/txmgr/test_helpers.go | 7 ++-- core/chains/evm/txmgr/txmgr_test.go | 33 ++++++++++++------- core/chains/legacyevm/chain.go | 2 +- core/chains/legacyevm/evm_txm.go | 5 ++- .../promreporter/prom_reporter_test.go | 3 +- core/services/vrf/delegate_test.go | 2 +- 12 files changed, 111 insertions(+), 28 deletions(-) diff --git a/common/txmgr/txmgr.go b/common/txmgr/txmgr.go index ff2c9c2c605..1ceb4df3424 100644 --- a/common/txmgr/txmgr.go +++ b/common/txmgr/txmgr.go @@ -425,7 +425,7 @@ func (b *Txm[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) runLoop() case head := <-b.chHeads: b.confirmer.mb.Deliver(head) b.tracker.mb.Deliver(head.BlockNumber()) - b.finalizer.DeliverHead(head) + b.finalizer.DeliverLatestHead(head) case reset := <-b.reset: // This check prevents the weird edge-case where you can select // into this block after chStop has already been closed and the diff --git a/common/txmgr/types/finalizer.go b/common/txmgr/types/finalizer.go index d7fd6b7f603..be3c897d0e2 100644 --- a/common/txmgr/types/finalizer.go +++ b/common/txmgr/types/finalizer.go @@ -8,5 +8,5 @@ import ( type Finalizer[BLOCK_HASH types.Hashable, HEAD types.Head[BLOCK_HASH]] interface { // interfaces for running the underlying estimator services.Service - DeliverHead(head HEAD) bool + DeliverLatestHead(head HEAD) bool } diff --git a/core/chains/evm/headtracker/simulated_head_tracker.go b/core/chains/evm/headtracker/simulated_head_tracker.go index e1e550de992..d4e09690989 100644 --- a/core/chains/evm/headtracker/simulated_head_tracker.go +++ b/core/chains/evm/headtracker/simulated_head_tracker.go @@ -2,6 +2,7 @@ package headtracker import ( "context" + "errors" "fmt" "math/big" @@ -51,3 +52,31 @@ func (ht *simulatedHeadTracker) LatestAndFinalizedBlock(ctx context.Context) (*e return latest, finalizedBlock, nil } + +func (ht *simulatedHeadTracker) LatestChain() *evmtypes.Head { + return nil +} + +func (ht *simulatedHeadTracker) HealthReport() map[string]error { + return nil +} + +func (ht *simulatedHeadTracker) Start(_ context.Context) error { + return nil +} + +func (ht *simulatedHeadTracker) Close() error { + return nil +} + +func (ht *simulatedHeadTracker) Backfill(_ context.Context, _ *evmtypes.Head) error { + return errors.New("unimplemented") +} + +func (ht *simulatedHeadTracker) Name() string { + return "SimulatedHeadTracker" +} + +func (ht *simulatedHeadTracker) Ready() error { + return nil +} \ No newline at end of file diff --git a/core/chains/evm/txmgr/builder.go b/core/chains/evm/txmgr/builder.go index 4e6949dcca7..9614f494482 100644 --- a/core/chains/evm/txmgr/builder.go +++ b/core/chains/evm/txmgr/builder.go @@ -12,6 +12,7 @@ import ( "github.com/smartcontractkit/chainlink/v2/core/chains/evm/config" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/forwarders" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/gas" + httypes "github.com/smartcontractkit/chainlink/v2/core/chains/evm/headtracker/types" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/keystore" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/logpoller" evmtypes "github.com/smartcontractkit/chainlink/v2/core/chains/evm/types" @@ -31,6 +32,7 @@ func NewTxm( logPoller logpoller.LogPoller, keyStore keystore.Eth, estimator gas.EvmFeeEstimator, + headTracker httypes.HeadTracker, ) (txm TxManager, err error, ) { @@ -53,7 +55,7 @@ func NewTxm( evmTracker := NewEvmTracker(txStore, keyStore, chainID, lggr) stuckTxDetector := NewStuckTxDetector(lggr, client.ConfiguredChainID(), chainConfig.ChainType(), fCfg.PriceMax(), txConfig.AutoPurge(), estimator, txStore, client) evmConfirmer := NewEvmConfirmer(txStore, txmClient, txmCfg, feeCfg, txConfig, dbConfig, keyStore, txAttemptBuilder, lggr, stuckTxDetector) - evmFinalizer := NewEvmFinalizer(lggr, client.ConfiguredChainID(), chainConfig.RPCDefaultBatchSize(), txStore, client) + evmFinalizer := NewEvmFinalizer(lggr, client.ConfiguredChainID(), chainConfig.RPCDefaultBatchSize(), txStore, client, headTracker) var evmResender *Resender if txConfig.ResendAfterThreshold() > 0 { evmResender = NewEvmResender(lggr, txStore, txmClient, evmTracker, keyStore, txmgr.DefaultResenderPollInterval, chainConfig, txConfig) diff --git a/core/chains/evm/txmgr/finalizer.go b/core/chains/evm/txmgr/finalizer.go index eba6d5a006b..666a54e7af0 100644 --- a/core/chains/evm/txmgr/finalizer.go +++ b/core/chains/evm/txmgr/finalizer.go @@ -32,14 +32,21 @@ type finalizerChainClient interface { BatchCallContext(ctx context.Context, elems []rpc.BatchElem) error } +type finalizerHeadTracker interface { + LatestAndFinalizedBlock(ctx context.Context) (latest, finalized *evmtypes.Head, err error) +} + // Finalizer handles processing new finalized blocks and marking transactions as finalized accordingly in the TXM DB type evmFinalizer struct { services.StateMachine lggr logger.SugaredLogger chainId *big.Int rpcBatchSize int + txStore finalizerTxStore client finalizerChainClient + headTracker finalizerHeadTracker + mb *mailbox.Mailbox[*evmtypes.Head] stopCh services.StopChan wg sync.WaitGroup @@ -51,6 +58,7 @@ func NewEvmFinalizer( rpcBatchSize uint32, txStore finalizerTxStore, client finalizerChainClient, + headTracker finalizerHeadTracker, ) *evmFinalizer { lggr = logger.Named(lggr, "Finalizer") return &evmFinalizer{ @@ -59,6 +67,7 @@ func NewEvmFinalizer( rpcBatchSize: int(rpcBatchSize), txStore: txStore, client: client, + headTracker: headTracker, mb: mailbox.NewSingle[*evmtypes.Head](), } } @@ -115,22 +124,25 @@ func (f *evmFinalizer) runLoop() { } } -func (f *evmFinalizer) DeliverHead(head *evmtypes.Head) bool { +func (f *evmFinalizer) DeliverLatestHead(head *evmtypes.Head) bool { return f.mb.Deliver(head) } func (f *evmFinalizer) ProcessHead(ctx context.Context, head *evmtypes.Head) error { ctx, cancel := context.WithTimeout(ctx, processHeadTimeout) defer cancel() - return f.processHead(ctx, head) + _, latestFinalizedHead, err := f.headTracker.LatestAndFinalizedBlock(ctx) + if err != nil { + return fmt.Errorf("failed to retrieve latest finalized head: %w", err) + } + return f.processFinalizedHead(ctx, latestFinalizedHead) } // Determines if any confirmed transactions can be marked as finalized by comparing their receipts against the latest finalized block -func (f *evmFinalizer) processHead(ctx context.Context, head *evmtypes.Head) error { - latestFinalizedHead := head.LatestFinalizedHead() +func (f *evmFinalizer) processFinalizedHead(ctx context.Context, latestFinalizedHead *evmtypes.Head) error { // Cannot determine finality without a finalized head for comparison if latestFinalizedHead == nil || !latestFinalizedHead.IsValid() { - return fmt.Errorf("failed to find latest finalized head in chain") + return fmt.Errorf("invalid latestFinalizedHead") } earliestBlockNumInChain := latestFinalizedHead.EarliestHeadInChain().BlockNumber() f.lggr.Debugw("processing latest finalized head", "block num", latestFinalizedHead.BlockNumber(), "block hash", latestFinalizedHead.BlockHash(), "earliest block num in chain", earliestBlockNumInChain) diff --git a/core/chains/evm/txmgr/finalizer_test.go b/core/chains/evm/txmgr/finalizer_test.go index 559eaa42485..24dc97b4cbd 100644 --- a/core/chains/evm/txmgr/finalizer_test.go +++ b/core/chains/evm/txmgr/finalizer_test.go @@ -1,6 +1,7 @@ package txmgr_test import ( + "errors" "testing" "time" @@ -12,7 +13,10 @@ import ( "github.com/smartcontractkit/chainlink-common/pkg/logger" "github.com/smartcontractkit/chainlink-common/pkg/utils/tests" + txmgrcommon "github.com/smartcontractkit/chainlink/v2/common/txmgr" + + "github.com/smartcontractkit/chainlink/v2/core/chains/evm/headtracker" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/testutils" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/txmgr" evmtypes "github.com/smartcontractkit/chainlink/v2/core/chains/evm/types" @@ -30,8 +34,9 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { feeLimit := uint64(10_000) ethClient := testutils.NewEthClientMockWithDefaultChain(t) rpcBatchSize := uint32(1) + ht := headtracker.NewSimulatedHeadTracker(ethClient, true, 0) - finalizer := txmgr.NewEvmFinalizer(logger.Test(t), testutils.FixtureChainID, rpcBatchSize, txStore, ethClient) + finalizer := txmgr.NewEvmFinalizer(logger.Test(t), testutils.FixtureChainID, rpcBatchSize, txStore, ethClient, ht) err := finalizer.Start(ctx) require.NoError(t, err) @@ -63,6 +68,8 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { attemptHash := insertTxAndAttemptWithIdempotencyKey(t, txStore, tx, idempotencyKey) // Insert receipt for unfinalized block num mustInsertEthReceipt(t, txStore, head.Number, head.Hash, attemptHash) + ethClient.On("HeadByNumber", mock.Anything, mock.Anything).Return(head, nil).Once() + ethClient.On("LatestFinalizedBlock", mock.Anything).Return(head.Parent, nil).Once() err = finalizer.ProcessHead(ctx, head) require.NoError(t, err) tx, err = txStore.FindTxWithIdempotencyKey(ctx, idempotencyKey, testutils.FixtureChainID) @@ -88,6 +95,8 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { attemptHash := insertTxAndAttemptWithIdempotencyKey(t, txStore, tx, idempotencyKey) // Insert receipt for finalized block num mustInsertEthReceipt(t, txStore, head.Parent.Number, utils.NewHash(), attemptHash) + ethClient.On("HeadByNumber", mock.Anything, mock.Anything).Return(head, nil).Once() + ethClient.On("LatestFinalizedBlock", mock.Anything).Return(head.Parent, nil).Once() err = finalizer.ProcessHead(ctx, head) require.NoError(t, err) tx, err = txStore.FindTxWithIdempotencyKey(ctx, idempotencyKey, testutils.FixtureChainID) @@ -114,6 +123,8 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { attemptHash := insertTxAndAttemptWithIdempotencyKey(t, txStore, tx, idempotencyKey) // Insert receipt for finalized block num mustInsertEthReceipt(t, txStore, head.Parent.Number, head.Parent.Hash, attemptHash) + ethClient.On("HeadByNumber", mock.Anything, mock.Anything).Return(head, nil).Once() + ethClient.On("LatestFinalizedBlock", mock.Anything).Return(head.Parent, nil).Once() err = finalizer.ProcessHead(ctx, head) require.NoError(t, err) tx, err = txStore.FindTxWithIdempotencyKey(ctx, idempotencyKey, testutils.FixtureChainID) @@ -175,12 +186,27 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { } rpcElements[0].Result = &headResult }).Return(nil).Twice() + ethClient.On("HeadByNumber", mock.Anything, mock.Anything).Return(head, nil).Once() + ethClient.On("LatestFinalizedBlock", mock.Anything).Return(head.Parent, nil).Once() err = finalizer.ProcessHead(ctx, head) require.NoError(t, err) tx, err = txStore.FindTxWithIdempotencyKey(ctx, idempotencyKey, testutils.FixtureChainID) require.NoError(t, err) require.Equal(t, true, tx.Finalized) }) + + t.Run("returns error if failed to retrieve latest head in headtracker", func(t *testing.T) { + ethClient.On("HeadByNumber", mock.Anything, mock.Anything).Return(nil, errors.New("failed to get latest head")).Once() + err = finalizer.ProcessHead(ctx, head) + require.Error(t, err) + }) + + t.Run("returns error if failed to calculate latest finalized head in headtracker", func(t *testing.T) { + ethClient.On("HeadByNumber", mock.Anything, mock.Anything).Return(head, nil).Once() + ethClient.On("LatestFinalizedBlock", mock.Anything).Return(nil, errors.New("failed to calculate latest finalized head")).Once() + err = finalizer.ProcessHead(ctx, head) + require.Error(t, err) + }) } func insertTxAndAttemptWithIdempotencyKey(t *testing.T, txStore txmgr.TestEvmTxStore, tx *txmgr.Tx, idempotencyKey string) common.Hash { diff --git a/core/chains/evm/txmgr/test_helpers.go b/core/chains/evm/txmgr/test_helpers.go index 3b3584a988b..cbcb8773840 100644 --- a/core/chains/evm/txmgr/test_helpers.go +++ b/core/chains/evm/txmgr/test_helpers.go @@ -53,6 +53,7 @@ type TestEvmConfig struct { Threshold uint32 MinAttempts uint32 DetectionApiUrl *url.URL + RpcDefaultBatchSize uint32 } func (e *TestEvmConfig) Transactions() evmconfig.Transactions { @@ -65,6 +66,8 @@ func (e *TestEvmConfig) FinalityDepth() uint32 { return 42 } func (e *TestEvmConfig) ChainType() chaintype.ChainType { return "" } +func (c *TestEvmConfig) RPCDefaultBatchSize() uint32 { return c.RpcDefaultBatchSize } + type TestGasEstimatorConfig struct { bumpThreshold uint64 } @@ -142,7 +145,6 @@ func (a *autoPurgeConfig) Enabled() bool { return false } type MockConfig struct { EvmConfig *TestEvmConfig - RpcDefaultBatchSize uint32 finalityDepth uint32 finalityTagEnabled bool } @@ -156,11 +158,10 @@ func (c *MockConfig) ChainType() chaintype.ChainType { return "" } func (c *MockConfig) FinalityDepth() uint32 { return c.finalityDepth } func (c *MockConfig) SetFinalityDepth(fd uint32) { c.finalityDepth = fd } func (c *MockConfig) FinalityTagEnabled() bool { return c.finalityTagEnabled } -func (c *MockConfig) RPCDefaultBatchSize() uint32 { return c.RpcDefaultBatchSize } func MakeTestConfigs(t *testing.T) (*MockConfig, *TestDatabaseConfig, *TestEvmConfig) { db := &TestDatabaseConfig{defaultQueryTimeout: utils.DefaultQueryTimeout} - ec := &TestEvmConfig{BumpThreshold: 42, MaxInFlight: uint32(42), MaxQueued: uint64(0), ReaperInterval: time.Duration(0), ReaperThreshold: time.Duration(0)} + ec := &TestEvmConfig{BumpThreshold: 42, MaxInFlight: uint32(42), MaxQueued: uint64(0), ReaperInterval: time.Duration(0), ReaperThreshold: time.Duration(0), RpcDefaultBatchSize: uint32(250)} config := &MockConfig{EvmConfig: ec} return config, db, ec } diff --git a/core/chains/evm/txmgr/txmgr_test.go b/core/chains/evm/txmgr/txmgr_test.go index c4ccc85a6a4..b0e80b396bf 100644 --- a/core/chains/evm/txmgr/txmgr_test.go +++ b/core/chains/evm/txmgr/txmgr_test.go @@ -85,7 +85,8 @@ func makeTestEvmTxm( lggr, lp, keyStore, - estimator) + estimator, + ht) } func TestTxm_SendNativeToken_DoesNotSendToZero(t *testing.T) { @@ -489,14 +490,20 @@ func TestTxm_Lifecycle(t *testing.T) { config, dbConfig, evmConfig := txmgr.MakeTestConfigs(t) config.SetFinalityDepth(uint32(42)) - config.RpcDefaultBatchSize = uint32(4) + evmConfig.RpcDefaultBatchSize = uint32(4) evmConfig.ResendAfterThreshold = 1 * time.Hour evmConfig.ReaperThreshold = 1 * time.Hour evmConfig.ReaperInterval = 1 * time.Hour kst.On("EnabledAddressesForChain", mock.Anything, &cltest.FixtureChainID).Return([]common.Address{}, nil) + head := cltest.Head(42) + finalizedHead := cltest.Head(0) + + ethClient.On("HeadByNumber", mock.Anything, mock.Anything).Return(head, nil).Once() + ethClient.On("HeadByNumber", mock.Anything, mock.Anything).Return(finalizedHead, nil).Once() + keyChangeCh := make(chan struct{}) unsub := cltest.NewAwaiter() kst.On("SubscribeToKeyChanges", mock.Anything).Return(keyChangeCh, unsub.ItHappened) @@ -505,7 +512,6 @@ func TestTxm_Lifecycle(t *testing.T) { txm, err := makeTestEvmTxm(t, db, ethClient, estimator, evmConfig, evmConfig.GasEstimator(), evmConfig.Transactions(), dbConfig, dbConfig.Listener(), kst) require.NoError(t, err) - head := cltest.Head(42) // It should not hang or panic txm.OnNewLongestChain(tests.Context(t), head) @@ -607,8 +613,20 @@ func TestTxm_GetTransactionStatus(t *testing.T) { gcfg := configtest.NewTestGeneralConfig(t) cfg := evmtest.NewChainScopedConfig(t, gcfg) + head := &evmtypes.Head{ + Hash: utils.NewHash(), + Number: 100, + Parent: &evmtypes.Head{ + Hash: utils.NewHash(), + Number: 99, + IsFinalized: true, + }, + } + ethClient := evmtest.NewEthClientMockWithDefaultChain(t) ethClient.On("PendingNonceAt", mock.Anything, mock.Anything).Return(uint64(0), nil).Maybe() + ethClient.On("HeadByNumber", mock.Anything, mock.Anything).Return(head, nil).Once() + ethClient.On("HeadByNumber", mock.Anything, mock.Anything).Return(head.Parent, nil).Once() feeEstimator := gasmocks.NewEvmFeeEstimator(t) feeEstimator.On("Start", mock.Anything).Return(nil).Once() feeEstimator.On("Close", mock.Anything).Return(nil).Once() @@ -617,15 +635,6 @@ func TestTxm_GetTransactionStatus(t *testing.T) { require.NoError(t, err) servicetest.Run(t, txm) - head := &evmtypes.Head{ - Hash: utils.NewHash(), - Number: 100, - Parent: &evmtypes.Head{ - Hash: utils.NewHash(), - Number: 99, - IsFinalized: true, - }, - } txm.OnNewLongestChain(ctx, head) t.Run("returns error if transaction not found", func(t *testing.T) { diff --git a/core/chains/legacyevm/chain.go b/core/chains/legacyevm/chain.go index b38cd2c4508..b0f174dcf42 100644 --- a/core/chains/legacyevm/chain.go +++ b/core/chains/legacyevm/chain.go @@ -250,7 +250,7 @@ func newChain(ctx context.Context, cfg *evmconfig.ChainScoped, nodes []*toml.Nod } // note: gas estimator is started as a part of the txm - txm, gasEstimator, err := newEvmTxm(opts.DS, cfg.EVM(), opts.AppConfig.EVMRPCEnabled(), opts.AppConfig.Database(), opts.AppConfig.Database().Listener(), client, l, logPoller, opts) + txm, gasEstimator, err := newEvmTxm(opts.DS, cfg.EVM(), opts.AppConfig.EVMRPCEnabled(), opts.AppConfig.Database(), opts.AppConfig.Database().Listener(), client, l, logPoller, opts, headTracker) if err != nil { return nil, fmt.Errorf("failed to instantiate EvmTxm for chain with ID %s: %w", chainID.String(), err) } diff --git a/core/chains/legacyevm/evm_txm.go b/core/chains/legacyevm/evm_txm.go index cecfd4ffafe..ab116749665 100644 --- a/core/chains/legacyevm/evm_txm.go +++ b/core/chains/legacyevm/evm_txm.go @@ -7,6 +7,7 @@ import ( evmclient "github.com/smartcontractkit/chainlink/v2/core/chains/evm/client" evmconfig "github.com/smartcontractkit/chainlink/v2/core/chains/evm/config" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/gas" + httypes "github.com/smartcontractkit/chainlink/v2/core/chains/evm/headtracker/types" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/logpoller" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/txmgr" "github.com/smartcontractkit/chainlink/v2/core/logger" @@ -22,6 +23,7 @@ func newEvmTxm( lggr logger.Logger, logPoller logpoller.LogPoller, opts ChainRelayExtenderConfig, + headTracker httypes.HeadTracker, ) (txm txmgr.TxManager, estimator gas.EvmFeeEstimator, err error, @@ -63,7 +65,8 @@ func newEvmTxm( lggr, logPoller, opts.KeyStore, - estimator) + estimator, + headTracker) } else { txm = opts.GenTxManager(chainID) } diff --git a/core/services/promreporter/prom_reporter_test.go b/core/services/promreporter/prom_reporter_test.go index b61fa25bdc4..a0a4a247c21 100644 --- a/core/services/promreporter/prom_reporter_test.go +++ b/core/services/promreporter/prom_reporter_test.go @@ -62,7 +62,8 @@ func newLegacyChainContainer(t *testing.T, db *sqlx.DB) legacyevm.LegacyChainCon lggr, lp, keyStore, - estimator) + estimator, + ht) require.NoError(t, err) cfg := configtest.NewGeneralConfig(t, nil) diff --git a/core/services/vrf/delegate_test.go b/core/services/vrf/delegate_test.go index 889b19d0e04..9718dc376a7 100644 --- a/core/services/vrf/delegate_test.go +++ b/core/services/vrf/delegate_test.go @@ -83,7 +83,7 @@ func buildVrfUni(t *testing.T, db *sqlx.DB, cfg chainlink.GeneralConfig) vrfUniv btORM := bridges.NewORM(db) ks := keystore.NewInMemory(db, utils.FastScryptParams, lggr) _, dbConfig, evmConfig := txmgr.MakeTestConfigs(t) - txm, err := txmgr.NewTxm(db, evmConfig, evmConfig.GasEstimator(), evmConfig.Transactions(), nil, dbConfig, dbConfig.Listener(), ec, logger.TestLogger(t), nil, ks.Eth(), nil) + txm, err := txmgr.NewTxm(db, evmConfig, evmConfig.GasEstimator(), evmConfig.Transactions(), nil, dbConfig, dbConfig.Listener(), ec, logger.TestLogger(t), nil, ks.Eth(), nil, nil) orm := headtracker.NewORM(*testutils.FixtureChainID, db) require.NoError(t, orm.IdempotentInsertHead(testutils.Context(t), cltest.Head(51))) jrm := job.NewORM(db, prm, btORM, ks, lggr) From a65977da2940c76e0ae5627a150f92d226e8c1db Mon Sep 17 00:00:00 2001 From: amit-momin Date: Mon, 1 Jul 2024 14:40:58 -0500 Subject: [PATCH 08/23] Fixed health check tests and fixed linting --- core/chains/evm/txmgr/finalizer.go | 34 +++++++++++++++------------ core/chains/evm/txmgr/test_helpers.go | 8 +++---- 2 files changed, 23 insertions(+), 19 deletions(-) diff --git a/core/chains/evm/txmgr/finalizer.go b/core/chains/evm/txmgr/finalizer.go index 666a54e7af0..08e3c8d8f42 100644 --- a/core/chains/evm/txmgr/finalizer.go +++ b/core/chains/evm/txmgr/finalizer.go @@ -43,13 +43,13 @@ type evmFinalizer struct { chainId *big.Int rpcBatchSize int - txStore finalizerTxStore - client finalizerChainClient + txStore finalizerTxStore + client finalizerChainClient headTracker finalizerHeadTracker - mb *mailbox.Mailbox[*evmtypes.Head] - stopCh services.StopChan - wg sync.WaitGroup + mb *mailbox.Mailbox[*evmtypes.Head] + stopCh services.StopChan + wg sync.WaitGroup } func NewEvmFinalizer( @@ -67,26 +67,30 @@ func NewEvmFinalizer( rpcBatchSize: int(rpcBatchSize), txStore: txStore, client: client, - headTracker: headTracker, + headTracker: headTracker, mb: mailbox.NewSingle[*evmtypes.Head](), } } // Start the finalizer func (f *evmFinalizer) Start(ctx context.Context) error { - f.lggr.Debugf("started Finalizer with RPC batch size limit: %d", f.rpcBatchSize) - f.stopCh = make(chan struct{}) - f.wg.Add(1) - go f.runLoop() - return nil + return f.StartOnce("Finalizer", func() error { + f.lggr.Debugf("started Finalizer with RPC batch size limit: %d", f.rpcBatchSize) + f.stopCh = make(chan struct{}) + f.wg.Add(1) + go f.runLoop() + return nil + }) } // Close the finalizer func (f *evmFinalizer) Close() error { - f.lggr.Debug("closing Finalizer") - close(f.stopCh) - f.wg.Wait() - return nil + return f.StopOnce("Finalizer", func() error { + f.lggr.Debug("closing Finalizer") + close(f.stopCh) + f.wg.Wait() + return nil + }) } func (f *evmFinalizer) Name() string { diff --git a/core/chains/evm/txmgr/test_helpers.go b/core/chains/evm/txmgr/test_helpers.go index cbcb8773840..eca0640d662 100644 --- a/core/chains/evm/txmgr/test_helpers.go +++ b/core/chains/evm/txmgr/test_helpers.go @@ -66,7 +66,7 @@ func (e *TestEvmConfig) FinalityDepth() uint32 { return 42 } func (e *TestEvmConfig) ChainType() chaintype.ChainType { return "" } -func (c *TestEvmConfig) RPCDefaultBatchSize() uint32 { return c.RpcDefaultBatchSize } +func (c *TestEvmConfig) RPCDefaultBatchSize() uint32 { return c.RpcDefaultBatchSize } type TestGasEstimatorConfig struct { bumpThreshold uint64 @@ -144,9 +144,9 @@ type autoPurgeConfig struct { func (a *autoPurgeConfig) Enabled() bool { return false } type MockConfig struct { - EvmConfig *TestEvmConfig - finalityDepth uint32 - finalityTagEnabled bool + EvmConfig *TestEvmConfig + finalityDepth uint32 + finalityTagEnabled bool } func (c *MockConfig) EVM() evmconfig.EVM { From a4dc48b3507720ab325c77064c01e8b5e46b876f Mon Sep 17 00:00:00 2001 From: amit-momin Date: Mon, 1 Jul 2024 14:59:14 -0500 Subject: [PATCH 09/23] Fixed lint error --- core/chains/evm/txmgr/test_helpers.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/chains/evm/txmgr/test_helpers.go b/core/chains/evm/txmgr/test_helpers.go index eca0640d662..8d208744329 100644 --- a/core/chains/evm/txmgr/test_helpers.go +++ b/core/chains/evm/txmgr/test_helpers.go @@ -66,7 +66,7 @@ func (e *TestEvmConfig) FinalityDepth() uint32 { return 42 } func (e *TestEvmConfig) ChainType() chaintype.ChainType { return "" } -func (c *TestEvmConfig) RPCDefaultBatchSize() uint32 { return c.RpcDefaultBatchSize } +func (e *TestEvmConfig) RPCDefaultBatchSize() uint32 { return e.RpcDefaultBatchSize } type TestGasEstimatorConfig struct { bumpThreshold uint64 From 599edacf5ef45c328594cd402c08ba403f11b543 Mon Sep 17 00:00:00 2001 From: amit-momin Date: Mon, 1 Jul 2024 15:08:43 -0500 Subject: [PATCH 10/23] Fixed lint error --- core/chains/evm/headtracker/simulated_head_tracker.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/chains/evm/headtracker/simulated_head_tracker.go b/core/chains/evm/headtracker/simulated_head_tracker.go index d4e09690989..62bb4968c2f 100644 --- a/core/chains/evm/headtracker/simulated_head_tracker.go +++ b/core/chains/evm/headtracker/simulated_head_tracker.go @@ -59,15 +59,15 @@ func (ht *simulatedHeadTracker) LatestChain() *evmtypes.Head { func (ht *simulatedHeadTracker) HealthReport() map[string]error { return nil -} +} func (ht *simulatedHeadTracker) Start(_ context.Context) error { return nil -} +} func (ht *simulatedHeadTracker) Close() error { return nil -} +} func (ht *simulatedHeadTracker) Backfill(_ context.Context, _ *evmtypes.Head) error { return errors.New("unimplemented") @@ -79,4 +79,4 @@ func (ht *simulatedHeadTracker) Name() string { func (ht *simulatedHeadTracker) Ready() error { return nil -} \ No newline at end of file +} From 911d34294fa5d8bb00163b760066cc8c1bed2274 Mon Sep 17 00:00:00 2001 From: amit-momin Date: Wed, 3 Jul 2024 16:31:18 -0500 Subject: [PATCH 11/23] Added finalized state to replace finalized column --- .changeset/itchy-bugs-clean.md | 2 +- common/txmgr/confirmer.go | 1 - common/txmgr/models.go | 1 + common/txmgr/txmgr.go | 8 +- common/txmgr/types/mocks/tx_store.go | 6 +- common/txmgr/types/tx.go | 1 - common/txmgr/types/tx_store.go | 2 +- core/chains/evm/txmgr/confirmer_test.go | 44 ------ core/chains/evm/txmgr/evm_tx_store.go | 24 ++-- core/chains/evm/txmgr/evm_tx_store_test.go | 21 +-- core/chains/evm/txmgr/finalizer.go | 8 +- core/chains/evm/txmgr/finalizer_test.go | 9 +- core/chains/evm/txmgr/mocks/evm_tx_store.go | 6 +- core/chains/evm/txmgr/reaper_test.go | 4 +- core/chains/evm/txmgr/txmgr_test.go | 8 +- core/store/migrate/migrate_test.go | 8 ++ .../0247_add_tx_finalized_column.sql | 15 -- .../0247_add_tx_finalized_state.sql | 135 ++++++++++++++++++ 18 files changed, 177 insertions(+), 126 deletions(-) delete mode 100644 core/store/migrate/migrations/0247_add_tx_finalized_column.sql create mode 100644 core/store/migrate/migrations/0247_add_tx_finalized_state.sql diff --git a/.changeset/itchy-bugs-clean.md b/.changeset/itchy-bugs-clean.md index a09117f4ed9..beeed8ace1e 100644 --- a/.changeset/itchy-bugs-clean.md +++ b/.changeset/itchy-bugs-clean.md @@ -2,4 +2,4 @@ "chainlink": minor --- -Added a finalizer component to the TXM to mark transactions as finalized #internal +Introduced finalized transaction state. Added a finalizer component to the TXM to mark transactions as finalized. #internal diff --git a/common/txmgr/confirmer.go b/common/txmgr/confirmer.go index a5c2af0d4d7..a9e30ffff1e 100644 --- a/common/txmgr/confirmer.go +++ b/common/txmgr/confirmer.go @@ -1100,7 +1100,6 @@ func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) mar "txID", etx.ID, "attemptID", attempt.ID, "nReceipts", len(attempt.Receipts), - "finalized", etx.Finalized, "id", "confirmer", } diff --git a/common/txmgr/models.go b/common/txmgr/models.go index dd121a2c7c4..ca5e7d4f251 100644 --- a/common/txmgr/models.go +++ b/common/txmgr/models.go @@ -11,4 +11,5 @@ const ( TxUnconfirmed = txmgrtypes.TxState("unconfirmed") TxConfirmed = txmgrtypes.TxState("confirmed") TxConfirmedMissingReceipt = txmgrtypes.TxState("confirmed_missing_receipt") + TxFinalized = txmgrtypes.TxState("finalized") ) diff --git a/common/txmgr/txmgr.go b/common/txmgr/txmgr.go index 1ceb4df3424..077e663edae 100644 --- a/common/txmgr/txmgr.go +++ b/common/txmgr/txmgr.go @@ -659,12 +659,10 @@ func (b *Txm[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) GetTransac // Return unconfirmed for ConfirmedMissingReceipt since a receipt is required to determine if it is finalized return commontypes.Unconfirmed, nil case TxConfirmed: - if tx.Finalized { - // Return finalized if tx receipt's block is equal or older than the latest finalized block - return commontypes.Finalized, nil - } - // Return unconfirmed if tx receipt's block is newer than the latest finalized block + // Return unconfirmed for confirmed transactions because they are not yet finalized return commontypes.Unconfirmed, nil + case TxFinalized: + return commontypes.Finalized, nil case TxFatalError: // Use an ErrorClassifier to determine if the transaction is considered Fatal txErr := b.newErrorClassifier(tx.GetError()) diff --git a/common/txmgr/types/mocks/tx_store.go b/common/txmgr/types/mocks/tx_store.go index 9087491dc03..e3a2ab7aefa 100644 --- a/common/txmgr/types/mocks/tx_store.go +++ b/common/txmgr/types/mocks/tx_store.go @@ -196,12 +196,12 @@ func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) DeleteInPro return r0 } -// FindConfirmedTxesAwaitingFinalization provides a mock function with given fields: ctx, chainID -func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindConfirmedTxesAwaitingFinalization(ctx context.Context, chainID CHAIN_ID) ([]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error) { +// FindConfirmedTxes provides a mock function with given fields: ctx, chainID +func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindConfirmedTxes(ctx context.Context, chainID CHAIN_ID) ([]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error) { ret := _m.Called(ctx, chainID) if len(ret) == 0 { - panic("no return value specified for FindConfirmedTxesAwaitingFinalization") + panic("no return value specified for FindConfirmedTxes") } var r0 []*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] diff --git a/common/txmgr/types/tx.go b/common/txmgr/types/tx.go index 72c13b450ff..ee5e23a3bf9 100644 --- a/common/txmgr/types/tx.go +++ b/common/txmgr/types/tx.go @@ -215,7 +215,6 @@ type Tx[ InitialBroadcastAt *time.Time CreatedAt time.Time State TxState - Finalized bool TxAttempts []TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] `json:"-"` // Marshalled TxMeta // Used for additional context around transactions which you want to log diff --git a/common/txmgr/types/tx_store.go b/common/txmgr/types/tx_store.go index 487b1b0ba82..72461e76100 100644 --- a/common/txmgr/types/tx_store.go +++ b/common/txmgr/types/tx_store.go @@ -84,7 +84,7 @@ type TransactionStore[ FindTransactionsConfirmedInBlockRange(ctx context.Context, highBlockNumber, lowBlockNumber int64, chainID CHAIN_ID) (etxs []*Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) FindEarliestUnconfirmedBroadcastTime(ctx context.Context, chainID CHAIN_ID) (null.Time, error) FindEarliestUnconfirmedTxAttemptBlock(ctx context.Context, chainID CHAIN_ID) (null.Int, error) - FindConfirmedTxesAwaitingFinalization(ctx context.Context, chainID CHAIN_ID) (etxs []*Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) + FindConfirmedTxes(ctx context.Context, chainID CHAIN_ID) (etxs []*Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) GetTxInProgress(ctx context.Context, fromAddress ADDR) (etx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) GetInProgressTxAttempts(ctx context.Context, address ADDR, chainID CHAIN_ID) (attempts []TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) GetAbandonedTransactionsByBatch(ctx context.Context, chainID CHAIN_ID, enabledAddrs []ADDR, offset, limit uint) (txs []*Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) diff --git a/core/chains/evm/txmgr/confirmer_test.go b/core/chains/evm/txmgr/confirmer_test.go index 73b4c946cb3..6b107b222a6 100644 --- a/core/chains/evm/txmgr/confirmer_test.go +++ b/core/chains/evm/txmgr/confirmer_test.go @@ -2828,50 +2828,6 @@ func TestEthConfirmer_EnsureConfirmedTransactionsInLongestChain(t *testing.T) { assert.Equal(t, txmgrtypes.TxAttemptBroadcast, attempt.State) assert.Len(t, attempt.Receipts, 1) }) - - t.Run("unconfirms, unfinalizes, and rebroadcasts finalized transactions that have receipts within head height of the chain but not included in the chain", func(t *testing.T) { - nonce := evmtypes.Nonce(8) - broadcast := time.Now() - tx := &txmgr.Tx{ - Sequence: &nonce, - FromAddress: fromAddress, - EncodedPayload: []byte{1, 2, 3}, - State: txmgrcommon.TxConfirmed, - BroadcastAt: &broadcast, - InitialBroadcastAt: &broadcast, - Finalized: true, - } - err := txStore.InsertTx(ctx, tx) - require.NoError(t, err) - etx, err := txStore.FindTxWithAttempts(ctx, tx.ID) - require.NoError(t, err) - attempt := cltest.NewLegacyEthTxAttempt(t, etx.ID) - broadcastBeforeBlockNum := int64(1) - attempt.BroadcastBeforeBlockNum = &broadcastBeforeBlockNum - attempt.State = txmgrtypes.TxAttemptBroadcast - err = txStore.InsertTxAttempt(ctx, &attempt) - require.NoError(t, err) - // Include one within head height but a different block hash - mustInsertEthReceipt(t, txStore, head.Parent.Number, testutils.NewHash(), attempt.Hash) - - ethClient.On("SendTransactionReturnCode", mock.Anything, mock.MatchedBy(func(tx *types.Transaction) bool { - atx, signErr := txmgr.GetGethSignedTx(attempt.SignedRawTx) - require.NoError(t, signErr) - // Keeps gas price and nonce the same - return atx.GasPrice().Cmp(tx.GasPrice()) == 0 && atx.Nonce() == tx.Nonce() - }), fromAddress).Return(commonclient.Successful, nil).Once() - - // Do the thing - require.NoError(t, ec.EnsureConfirmedTransactionsInLongestChain(tests.Context(t), &head)) - - etx, err = txStore.FindTxWithAttempts(ctx, etx.ID) - require.NoError(t, err) - assert.Equal(t, txmgrcommon.TxUnconfirmed, etx.State) - require.Len(t, etx.TxAttempts, 1) - attempt = etx.TxAttempts[0] - assert.Equal(t, txmgrtypes.TxAttemptBroadcast, attempt.State) - assert.Equal(t, false, etx.Finalized) - }) } func TestEthConfirmer_ForceRebroadcast(t *testing.T) { diff --git a/core/chains/evm/txmgr/evm_tx_store.go b/core/chains/evm/txmgr/evm_tx_store.go index 0b97bf78770..c1c05dbcebf 100644 --- a/core/chains/evm/txmgr/evm_tx_store.go +++ b/core/chains/evm/txmgr/evm_tx_store.go @@ -181,7 +181,6 @@ type DbEthTx struct { // InitialBroadcastAt is recorded once, the first ever time this eth_tx is sent CreatedAt time.Time State txmgrtypes.TxState - Finalized bool // Marshalled EvmTxMeta // Used for additional context around transactions which you want to log // at send time. @@ -212,7 +211,6 @@ func (db *DbEthTx) FromTx(tx *Tx) { db.BroadcastAt = tx.BroadcastAt db.CreatedAt = tx.CreatedAt db.State = tx.State - db.Finalized = tx.Finalized db.Meta = tx.Meta db.Subject = tx.Subject db.PipelineTaskRunID = tx.PipelineTaskRunID @@ -247,7 +245,6 @@ func (db DbEthTx) ToTx(tx *Tx) { tx.BroadcastAt = db.BroadcastAt tx.CreatedAt = db.CreatedAt tx.State = db.State - tx.Finalized = db.Finalized tx.Meta = db.Meta tx.Subject = db.Subject tx.PipelineTaskRunID = db.PipelineTaskRunID @@ -532,8 +529,8 @@ func (o *evmTxStore) InsertTx(ctx context.Context, etx *Tx) error { if etx.CreatedAt == (time.Time{}) { etx.CreatedAt = time.Now() } - const insertEthTxSQL = `INSERT INTO evm.txes (nonce, from_address, to_address, encoded_payload, value, gas_limit, error, broadcast_at, initial_broadcast_at, created_at, state, meta, subject, pipeline_task_run_id, min_confirmations, evm_chain_id, transmit_checker, idempotency_key, signal_callback, callback_completed, finalized) VALUES ( -:nonce, :from_address, :to_address, :encoded_payload, :value, :gas_limit, :error, :broadcast_at, :initial_broadcast_at, :created_at, :state, :meta, :subject, :pipeline_task_run_id, :min_confirmations, :evm_chain_id, :transmit_checker, :idempotency_key, :signal_callback, :callback_completed, :finalized + const insertEthTxSQL = `INSERT INTO evm.txes (nonce, from_address, to_address, encoded_payload, value, gas_limit, error, broadcast_at, initial_broadcast_at, created_at, state, meta, subject, pipeline_task_run_id, min_confirmations, evm_chain_id, transmit_checker, idempotency_key, signal_callback, callback_completed) VALUES ( +:nonce, :from_address, :to_address, :encoded_payload, :value, :gas_limit, :error, :broadcast_at, :initial_broadcast_at, :created_at, :state, :meta, :subject, :pipeline_task_run_id, :min_confirmations, :evm_chain_id, :transmit_checker, :idempotency_key, :signal_callback, :callback_completed ) RETURNING *` var dbTx DbEthTx dbTx.FromTx(etx) @@ -1119,13 +1116,11 @@ func updateEthTxAttemptUnbroadcast(ctx context.Context, orm *evmTxStore, attempt return pkgerrors.Wrap(err, "updateEthTxAttemptUnbroadcast failed") } -// Ensure to mark the transaction as not finalized in case there is a finality violation and a "finalized" transaction -// has been considered re-org'd out func updateEthTxUnconfirm(ctx context.Context, orm *evmTxStore, etx Tx) error { if etx.State != txmgr.TxConfirmed { - return errors.New("expected eth_tx state to be confirmed") + return errors.New("expected tx state to be confirmed") } - _, err := orm.q.ExecContext(ctx, `UPDATE evm.txes SET state = 'unconfirmed', finalized = false WHERE id = $1`, etx.ID) + _, err := orm.q.ExecContext(ctx, `UPDATE evm.txes SET state = 'unconfirmed' WHERE id = $1`, etx.ID) return pkgerrors.Wrap(err, "updateEthTxUnconfirm failed") } @@ -1884,8 +1879,7 @@ USING old_enough_receipts, evm.tx_attempts WHERE evm.tx_attempts.eth_tx_id = evm.txes.id AND evm.tx_attempts.hash = old_enough_receipts.tx_hash AND evm.txes.created_at < $2 -AND evm.txes.state = 'confirmed' -AND evm.txes.finalized = true +AND evm.txes.state = 'finalized' AND evm_chain_id = $3`, limit, timeThreshold, chainID.String()) if err != nil { return count, pkgerrors.Wrap(err, "ReapTxes failed to delete old confirmed evm.txes") @@ -2047,13 +2041,13 @@ func (o *evmTxStore) UpdateTxAttemptBroadcastBeforeBlockNum(ctx context.Context, return err } -// Returns all confirmed transactions not yet marked as finalized -func (o *evmTxStore) FindConfirmedTxesAwaitingFinalization(ctx context.Context, chainID *big.Int) (txes []*Tx, err error) { +// Returns all confirmed transactions +func (o *evmTxStore) FindConfirmedTxes(ctx context.Context, chainID *big.Int) (txes []*Tx, err error) { var cancel context.CancelFunc ctx, cancel = o.stopCh.Ctx(ctx) defer cancel() err = o.Transact(ctx, true, func(orm *evmTxStore) error { - sql := "SELECT * FROM evm.txes WHERE state = 'confirmed' AND finalized = false AND evm_chain_id = $1" + sql := "SELECT * FROM evm.txes WHERE state = 'confirmed' AND evm_chain_id = $1" var dbEtxs []DbEthTx err = o.q.SelectContext(ctx, &dbEtxs, sql, chainID.String()) if len(dbEtxs) == 0 { @@ -2077,7 +2071,7 @@ func (o *evmTxStore) UpdateTxesFinalized(ctx context.Context, etxIDs []int64, ch var cancel context.CancelFunc ctx, cancel = o.stopCh.Ctx(ctx) defer cancel() - sql := "UPDATE evm.txes SET finalized = true WHERE id = ANY($1) AND evm_chain_id = $2" + sql := "UPDATE evm.txes SET state = 'finalized' WHERE id = ANY($1) AND evm_chain_id = $2" _, err := o.q.ExecContext(ctx, sql, pq.Array(etxIDs), chainId.String()) return err } diff --git a/core/chains/evm/txmgr/evm_tx_store_test.go b/core/chains/evm/txmgr/evm_tx_store_test.go index dde8e047bfd..fee2fc2d377 100644 --- a/core/chains/evm/txmgr/evm_tx_store_test.go +++ b/core/chains/evm/txmgr/evm_tx_store_test.go @@ -1864,7 +1864,7 @@ func TestORM_FindTransactionsByState(t *testing.T) { mustInsertConfirmedEthTxWithReceipt(t, txStore, fromAddress, 3, 100) mustInsertFatalErrorEthTx(t, txStore, fromAddress) - txs, err := txStore.FindConfirmedTxesAwaitingFinalization(ctx, testutils.FixtureChainID) + txs, err := txStore.FindConfirmedTxes(ctx, testutils.FixtureChainID) require.NoError(t, err) require.Len(t, txs, 1) } @@ -1895,23 +1895,6 @@ func TestORM_UpdateTxesFinalized(t *testing.T) { require.NoError(t, err) etx, err := txStore.FindTxWithAttempts(ctx, tx.ID) require.NoError(t, err) - require.True(t, etx.Finalized) - }) - t.Run("fails to finalize an unconfirmed transaction", func(t *testing.T) { - nonce := evmtypes.Nonce(1) - tx := &txmgr.Tx{ - Sequence: &nonce, - FromAddress: fromAddress, - EncodedPayload: []byte{1, 2, 3}, - State: txmgrcommon.TxUnconfirmed, - BroadcastAt: &broadcast, - InitialBroadcastAt: &broadcast, - } - err := txStore.InsertTx(ctx, tx) - require.NoError(t, err) - err = txStore.UpdateTxesFinalized(ctx, []int64{tx.ID}, testutils.FixtureChainID) - // Fails due to chk_eth_txes_state_finalized constraint - // Tx Store is poisoned after this - require.ErrorContains(t, err, "chk_eth_txes_state_finalized") + require.Equal(t, txmgrcommon.TxFinalized, etx.State) }) } diff --git a/core/chains/evm/txmgr/finalizer.go b/core/chains/evm/txmgr/finalizer.go index 08e3c8d8f42..94c54e534fa 100644 --- a/core/chains/evm/txmgr/finalizer.go +++ b/core/chains/evm/txmgr/finalizer.go @@ -24,7 +24,7 @@ var _ Finalizer = (*evmFinalizer)(nil) const processHeadTimeout = 10 * time.Minute type finalizerTxStore interface { - FindConfirmedTxesAwaitingFinalization(ctx context.Context, chainID *big.Int) ([]*Tx, error) + FindConfirmedTxes(ctx context.Context, chainID *big.Int) ([]*Tx, error) UpdateTxesFinalized(ctx context.Context, txs []int64, chainId *big.Int) error } @@ -152,7 +152,7 @@ func (f *evmFinalizer) processFinalizedHead(ctx context.Context, latestFinalized f.lggr.Debugw("processing latest finalized head", "block num", latestFinalizedHead.BlockNumber(), "block hash", latestFinalizedHead.BlockHash(), "earliest block num in chain", earliestBlockNumInChain) // Retrieve all confirmed transactions, loaded with attempts and receipts - unfinalizedTxs, err := f.txStore.FindConfirmedTxesAwaitingFinalization(ctx, f.chainId) + unfinalizedTxs, err := f.txStore.FindConfirmedTxes(ctx, f.chainId) if err != nil { return fmt.Errorf("failed to retrieve confirmed transactions: %w", err) } @@ -162,10 +162,6 @@ func (f *evmFinalizer) processFinalizedHead(ctx context.Context, latestFinalized receiptBlockHashToTx := make(map[common.Hash][]*Tx) // Find transactions with receipt block nums older than the latest finalized block num and block hashes still in chain for _, tx := range unfinalizedTxs { - // Only consider transactions not already marked as finalized - if tx.Finalized { - continue - } receipt := tx.GetReceipt() if receipt == nil || receipt.IsZero() || receipt.IsUnmined() { f.lggr.AssumptionViolationw("invalid receipt found for confirmed transaction", "tx", tx, "receipt", receipt) diff --git a/core/chains/evm/txmgr/finalizer_test.go b/core/chains/evm/txmgr/finalizer_test.go index 24dc97b4cbd..50d46a61c08 100644 --- a/core/chains/evm/txmgr/finalizer_test.go +++ b/core/chains/evm/txmgr/finalizer_test.go @@ -74,7 +74,7 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { require.NoError(t, err) tx, err = txStore.FindTxWithIdempotencyKey(ctx, idempotencyKey, testutils.FixtureChainID) require.NoError(t, err) - require.Equal(t, false, tx.Finalized) + require.Equal(t, txmgrcommon.TxConfirmed, tx.State) }) t.Run("returns not finalized for tx with receipt re-org'd out", func(t *testing.T) { @@ -101,7 +101,7 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { require.NoError(t, err) tx, err = txStore.FindTxWithIdempotencyKey(ctx, idempotencyKey, testutils.FixtureChainID) require.NoError(t, err) - require.Equal(t, false, tx.Finalized) + require.Equal(t, txmgrcommon.TxConfirmed, tx.State) }) t.Run("returns finalized for tx with receipt in a finalized block", func(t *testing.T) { @@ -118,7 +118,6 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { State: txmgrcommon.TxConfirmed, BroadcastAt: &broadcast, InitialBroadcastAt: &broadcast, - Finalized: true, } attemptHash := insertTxAndAttemptWithIdempotencyKey(t, txStore, tx, idempotencyKey) // Insert receipt for finalized block num @@ -129,7 +128,7 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { require.NoError(t, err) tx, err = txStore.FindTxWithIdempotencyKey(ctx, idempotencyKey, testutils.FixtureChainID) require.NoError(t, err) - require.Equal(t, true, tx.Finalized) + require.Equal(t, txmgrcommon.TxFinalized, tx.State) }) t.Run("returns finalized for tx with receipt older than block history depth", func(t *testing.T) { @@ -192,7 +191,7 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { require.NoError(t, err) tx, err = txStore.FindTxWithIdempotencyKey(ctx, idempotencyKey, testutils.FixtureChainID) require.NoError(t, err) - require.Equal(t, true, tx.Finalized) + require.Equal(t, txmgrcommon.TxFinalized, tx.State) }) t.Run("returns error if failed to retrieve latest head in headtracker", func(t *testing.T) { diff --git a/core/chains/evm/txmgr/mocks/evm_tx_store.go b/core/chains/evm/txmgr/mocks/evm_tx_store.go index 454d6ab8f20..a8683505b5c 100644 --- a/core/chains/evm/txmgr/mocks/evm_tx_store.go +++ b/core/chains/evm/txmgr/mocks/evm_tx_store.go @@ -199,12 +199,12 @@ func (_m *EvmTxStore) DeleteInProgressAttempt(ctx context.Context, attempt types return r0 } -// FindConfirmedTxesAwaitingFinalization provides a mock function with given fields: ctx, chainID -func (_m *EvmTxStore) FindConfirmedTxesAwaitingFinalization(ctx context.Context, chainID *big.Int) ([]*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error) { +// FindConfirmedTxes provides a mock function with given fields: ctx, chainID +func (_m *EvmTxStore) FindConfirmedTxes(ctx context.Context, chainID *big.Int) ([]*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error) { ret := _m.Called(ctx, chainID) if len(ret) == 0 { - panic("no return value specified for FindConfirmedTxesAwaitingFinalization") + panic("no return value specified for FindConfirmedTxes") } var r0 []*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee] diff --git a/core/chains/evm/txmgr/reaper_test.go b/core/chains/evm/txmgr/reaper_test.go index eaa7eecb252..a8fe81ae1b9 100644 --- a/core/chains/evm/txmgr/reaper_test.go +++ b/core/chains/evm/txmgr/reaper_test.go @@ -83,7 +83,7 @@ func TestReaper_ReapTxes(t *testing.T) { cltest.AssertCount(t, db, "evm.txes", 1) }) - t.Run("deletes confirmed evm.txes marked as finalized that exceed the age threshold", func(t *testing.T) { + t.Run("deletes finalized evm.txes that exceed the age threshold", func(t *testing.T) { tc := &reaperConfig{reaperThreshold: 1 * time.Hour} r := newReaper(t, txStore, tc) @@ -100,7 +100,7 @@ func TestReaper_ReapTxes(t *testing.T) { // Didn't delete because eth_tx although old enough, was not marked as finalized cltest.AssertCount(t, db, "evm.txes", 1) - pgtest.MustExec(t, db, `UPDATE evm.txes SET finalized=true`) + pgtest.MustExec(t, db, `UPDATE evm.txes SET state='finalized'`) err = r.ReapTxes(42) assert.NoError(t, err) diff --git a/core/chains/evm/txmgr/txmgr_test.go b/core/chains/evm/txmgr/txmgr_test.go index b0e80b396bf..5f932db8720 100644 --- a/core/chains/evm/txmgr/txmgr_test.go +++ b/core/chains/evm/txmgr/txmgr_test.go @@ -702,7 +702,7 @@ func TestTxm_GetTransactionStatus(t *testing.T) { require.Equal(t, commontypes.Unconfirmed, state) }) - t.Run("returns unconfirmed for confirmed state not marked as finalized", func(t *testing.T) { + t.Run("returns unconfirmed for confirmed state", func(t *testing.T) { idempotencyKey := uuid.New().String() _, fromAddress := cltest.MustInsertRandomKey(t, ethKeyStore) nonce := evmtypes.Nonce(0) @@ -716,7 +716,6 @@ func TestTxm_GetTransactionStatus(t *testing.T) { State: txmgrcommon.TxConfirmed, BroadcastAt: &broadcast, InitialBroadcastAt: &broadcast, - Finalized: false, // Set to false by default in DB but here for explicitness } err := txStore.InsertTx(ctx, tx) require.NoError(t, err) @@ -732,7 +731,7 @@ func TestTxm_GetTransactionStatus(t *testing.T) { require.Equal(t, commontypes.Unconfirmed, state) }) - t.Run("returns finalized for confirmed state marked as finalized", func(t *testing.T) { + t.Run("returns finalized for finalized state", func(t *testing.T) { idempotencyKey := uuid.New().String() _, fromAddress := cltest.MustInsertRandomKey(t, ethKeyStore) nonce := evmtypes.Nonce(0) @@ -743,10 +742,9 @@ func TestTxm_GetTransactionStatus(t *testing.T) { FromAddress: fromAddress, EncodedPayload: []byte{1, 2, 3}, FeeLimit: feeLimit, - State: txmgrcommon.TxConfirmed, + State: txmgrcommon.TxFinalized, BroadcastAt: &broadcast, InitialBroadcastAt: &broadcast, - Finalized: true, } err := txStore.InsertTx(ctx, tx) require.NoError(t, err) diff --git a/core/store/migrate/migrate_test.go b/core/store/migrate/migrate_test.go index 8a7d1628a4c..7a03273551d 100644 --- a/core/store/migrate/migrate_test.go +++ b/core/store/migrate/migrate_test.go @@ -600,3 +600,11 @@ func BenchmarkBackfillingRecordsWithMigration202(b *testing.B) { require.NoError(b, err) } } + +func TestRollback_247_TxStateEnumUpdate(t *testing.T) { + _, db := heavyweight.FullTestDBV2(t, nil) + err := goose.DownTo(db.DB, migrationDir, 54) + require.NoError(t, err) + err = goose.UpTo(db.DB, migrationDir, 247) + require.NoError(t, err) +} diff --git a/core/store/migrate/migrations/0247_add_tx_finalized_column.sql b/core/store/migrate/migrations/0247_add_tx_finalized_column.sql deleted file mode 100644 index 5f2c5c5ffb2..00000000000 --- a/core/store/migrate/migrations/0247_add_tx_finalized_column.sql +++ /dev/null @@ -1,15 +0,0 @@ --- +goose Up --- +goose StatementBegin -ALTER TABLE evm.txes ADD COLUMN finalized boolean NOT NULL DEFAULT false; -ALTER TABLE evm.txes ADD CONSTRAINT chk_eth_txes_state_finalized CHECK ( - state <> 'confirmed'::evm.txes_state AND finalized = false - OR - state = 'confirmed'::evm.txes_state -) NOT VALID; --- +goose StatementEnd - --- +goose Down --- +goose StatementBegin -ALTER TABLE evm.txes DROP CONSTRAINT chk_eth_txes_state_finalized; -ALTER TABLE evm.txes DROP COLUMN finalized; --- +goose StatementEnd diff --git a/core/store/migrate/migrations/0247_add_tx_finalized_state.sql b/core/store/migrate/migrations/0247_add_tx_finalized_state.sql new file mode 100644 index 00000000000..dcfe8eec734 --- /dev/null +++ b/core/store/migrate/migrations/0247_add_tx_finalized_state.sql @@ -0,0 +1,135 @@ +-- +goose Up +-- Creating new column and enum instead of just adding new value to the existing enum so the migration changes match the rollback logic +-- Otherwise, migration will complain about mismatching column order + +-- +goose StatementBegin +-- Rename the existing enum with finalized state to mark it as old +ALTER TYPE evm.txes_state RENAME TO txes_state_old; + +-- Create new enum without finalized state +CREATE TYPE evm.txes_state AS ENUM ( + 'unstarted', + 'in_progress', + 'fatal_error', + 'unconfirmed', + 'confirmed_missing_receipt', + 'confirmed', + 'finalized' +); + +-- Add a new state column with the new enum type to the txes table +ALTER TABLE evm.txes ADD COLUMN state_new evm.txes_state; + +-- Copy data from the old column to the new +UPDATE evm.txes SET state_new = state::text::evm.txes_state; + +-- Drop constraints referring to old enum type on the old state column +ALTER TABLE evm.txes ALTER COLUMN state DROP DEFAULT; +ALTER TABLE evm.txes DROP CONSTRAINT chk_eth_txes_fsm; +DROP INDEX IF EXISTS idx_eth_txes_state_from_address_evm_chain_id; +DROP INDEX IF EXISTS idx_eth_txes_min_unconfirmed_nonce_for_key_evm_chain_id; +DROP INDEX IF EXISTS idx_only_one_in_progress_tx_per_account_id_per_evm_chain_id; +DROP INDEX IF EXISTS idx_eth_txes_unstarted_subject_id_evm_chain_id; + +-- Drop the old state column +ALTER TABLE evm.txes DROP state; + +-- Drop the old enum type +DROP TYPE evm.txes_state_old; + +-- Rename the new column name state to replace the old column +ALTER TABLE evm.txes RENAME state_new TO state; + +-- Reset the state column's default +ALTER TABLE evm.txes ALTER COLUMN state SET DEFAULT 'unstarted'::evm.txes_state, ALTER COLUMN state SET NOT NULL; + +-- Recreate constraint with finalized state +ALTER TABLE evm.txes ADD CONSTRAINT chk_eth_txes_fsm CHECK ( + state = 'unstarted'::evm.txes_state AND nonce IS NULL AND error IS NULL AND broadcast_at IS NULL AND initial_broadcast_at IS NULL + OR + state = 'in_progress'::evm.txes_state AND nonce IS NOT NULL AND error IS NULL AND broadcast_at IS NULL AND initial_broadcast_at IS NULL + OR + state = 'fatal_error'::evm.txes_state AND error IS NOT NULL + OR + state = 'unconfirmed'::evm.txes_state AND nonce IS NOT NULL AND error IS NULL AND broadcast_at IS NOT NULL AND initial_broadcast_at IS NOT NULL + OR + state = 'confirmed'::evm.txes_state AND nonce IS NOT NULL AND error IS NULL AND broadcast_at IS NOT NULL AND initial_broadcast_at IS NOT NULL + OR + state = 'confirmed_missing_receipt'::evm.txes_state AND nonce IS NOT NULL AND error IS NULL AND broadcast_at IS NOT NULL AND initial_broadcast_at IS NOT NULL + OR + state = 'finalized'::evm.txes_state AND nonce IS NOT NULL AND error IS NULL AND broadcast_at IS NOT NULL AND initial_broadcast_at IS NOT NULL +) NOT VALID; + +-- Recreate index to include finalized state +CREATE INDEX idx_eth_txes_state_from_address_evm_chain_id ON evm.txes(evm_chain_id, from_address, state) WHERE state <> 'confirmed'::evm.txes_state AND state <> 'finalized'::evm.txes_state; +CREATE INDEX idx_eth_txes_min_unconfirmed_nonce_for_key_evm_chain_id ON evm.txes(evm_chain_id, from_address, nonce) WHERE state = 'unconfirmed'::evm.txes_state; +CREATE UNIQUE INDEX idx_only_one_in_progress_tx_per_account_id_per_evm_chain_id ON evm.txes(evm_chain_id, from_address) WHERE state = 'in_progress'::evm.txes_state; +CREATE INDEX idx_eth_txes_unstarted_subject_id_evm_chain_id ON evm.txes(evm_chain_id, subject, id) WHERE subject IS NOT NULL AND state = 'unstarted'::evm.txes_state; +-- +goose StatementEnd + +-- +goose Down +-- +goose StatementBegin + +-- Rename the existing enum with finalized state to mark it as old +ALTER TYPE evm.txes_state RENAME TO txes_state_old; + +-- Create new enum without finalized state +CREATE TYPE evm.txes_state AS ENUM ( + 'unstarted', + 'in_progress', + 'fatal_error', + 'unconfirmed', + 'confirmed_missing_receipt', + 'confirmed' +); + +-- Add a new state column with the new enum type to the txes table +ALTER TABLE evm.txes ADD COLUMN state_new evm.txes_state; + +-- Update all transactions with finalized state to confirmed in the old state column +UPDATE evm.txes SET state = 'confirmed'::evm.txes_state_old WHERE state = 'finalized'::evm.txes_state_old; + +-- Copy data from the old column to the new +UPDATE evm.txes SET state_new = state::text::evm.txes_state; + +-- Drop constraints referring to old enum type on the old state column +ALTER TABLE evm.txes ALTER COLUMN state DROP DEFAULT; +ALTER TABLE evm.txes DROP CONSTRAINT chk_eth_txes_fsm; +DROP INDEX IF EXISTS idx_eth_txes_state_from_address_evm_chain_id; +DROP INDEX IF EXISTS idx_eth_txes_min_unconfirmed_nonce_for_key_evm_chain_id; +DROP INDEX IF EXISTS idx_only_one_in_progress_tx_per_account_id_per_evm_chain_id; +DROP INDEX IF EXISTS idx_eth_txes_unstarted_subject_id_evm_chain_id; + +-- Drop the old state column +ALTER TABLE evm.txes DROP state; + +-- Drop the old enum type +DROP TYPE evm.txes_state_old; + +-- Rename the new column name state to replace the old column +ALTER TABLE evm.txes RENAME state_new TO state; + +-- Reset the state column's default +ALTER TABLE evm.txes ALTER COLUMN state SET DEFAULT 'unstarted'::evm.txes_state, ALTER COLUMN state SET NOT NULL; + +-- Recereate constraint without finalized state +ALTER TABLE evm.txes ADD CONSTRAINT chk_eth_txes_fsm CHECK ( + state = 'unstarted'::evm.txes_state AND nonce IS NULL AND error IS NULL AND broadcast_at IS NULL AND initial_broadcast_at IS NULL + OR + state = 'in_progress'::evm.txes_state AND nonce IS NOT NULL AND error IS NULL AND broadcast_at IS NULL AND initial_broadcast_at IS NULL + OR + state = 'fatal_error'::evm.txes_state AND error IS NOT NULL + OR + state = 'unconfirmed'::evm.txes_state AND nonce IS NOT NULL AND error IS NULL AND broadcast_at IS NOT NULL AND initial_broadcast_at IS NOT NULL + OR + state = 'confirmed'::evm.txes_state AND nonce IS NOT NULL AND error IS NULL AND broadcast_at IS NOT NULL AND initial_broadcast_at IS NOT NULL + OR + state = 'confirmed_missing_receipt'::evm.txes_state AND nonce IS NOT NULL AND error IS NULL AND broadcast_at IS NOT NULL AND initial_broadcast_at IS NOT NULL +) NOT VALID; + +-- Recreate index with new enum type +CREATE INDEX idx_eth_txes_state_from_address_evm_chain_id ON evm.txes(evm_chain_id, from_address, state) WHERE state <> 'confirmed'::evm.txes_state; +CREATE INDEX idx_eth_txes_min_unconfirmed_nonce_for_key_evm_chain_id ON evm.txes(evm_chain_id, from_address, nonce) WHERE state = 'unconfirmed'::evm.txes_state; +CREATE UNIQUE INDEX idx_only_one_in_progress_tx_per_account_id_per_evm_chain_id ON evm.txes(evm_chain_id, from_address) WHERE state = 'in_progress'::evm.txes_state; +CREATE INDEX idx_eth_txes_unstarted_subject_id_evm_chain_id ON evm.txes(evm_chain_id, subject, id) WHERE subject IS NOT NULL AND state = 'unstarted'::evm.txes_state; +-- +goose StatementEnd From 005d9726ed1b3abb68e7757c7115b5bcd9dc5fc9 Mon Sep 17 00:00:00 2001 From: amit-momin Date: Wed, 10 Jul 2024 15:32:37 -0500 Subject: [PATCH 12/23] Updated finalizer batch RPC validation to use blockByNumber and added filter to DB query --- common/txmgr/types/tx_store.go | 2 +- core/chains/evm/txmgr/evm_tx_store.go | 11 ++-- core/chains/evm/txmgr/evm_tx_store_test.go | 8 +-- core/chains/evm/txmgr/finalizer.go | 62 +++++++++++++--------- core/chains/evm/txmgr/finalizer_test.go | 12 +++-- 5 files changed, 59 insertions(+), 36 deletions(-) diff --git a/common/txmgr/types/tx_store.go b/common/txmgr/types/tx_store.go index 72461e76100..f577ddf8166 100644 --- a/common/txmgr/types/tx_store.go +++ b/common/txmgr/types/tx_store.go @@ -84,7 +84,7 @@ type TransactionStore[ FindTransactionsConfirmedInBlockRange(ctx context.Context, highBlockNumber, lowBlockNumber int64, chainID CHAIN_ID) (etxs []*Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) FindEarliestUnconfirmedBroadcastTime(ctx context.Context, chainID CHAIN_ID) (null.Time, error) FindEarliestUnconfirmedTxAttemptBlock(ctx context.Context, chainID CHAIN_ID) (null.Int, error) - FindConfirmedTxes(ctx context.Context, chainID CHAIN_ID) (etxs []*Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) + FindTxesToMarkFinalized(ctx context.Context, finalizedBlockNum int64, chainID CHAIN_ID) (etxs []*Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) GetTxInProgress(ctx context.Context, fromAddress ADDR) (etx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) GetInProgressTxAttempts(ctx context.Context, address ADDR, chainID CHAIN_ID) (attempts []TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) GetAbandonedTransactionsByBatch(ctx context.Context, chainID CHAIN_ID, enabledAddrs []ADDR, offset, limit uint) (txs []*Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) diff --git a/core/chains/evm/txmgr/evm_tx_store.go b/core/chains/evm/txmgr/evm_tx_store.go index c1c05dbcebf..325401ff063 100644 --- a/core/chains/evm/txmgr/evm_tx_store.go +++ b/core/chains/evm/txmgr/evm_tx_store.go @@ -2041,15 +2041,18 @@ func (o *evmTxStore) UpdateTxAttemptBroadcastBeforeBlockNum(ctx context.Context, return err } -// Returns all confirmed transactions -func (o *evmTxStore) FindConfirmedTxes(ctx context.Context, chainID *big.Int) (txes []*Tx, err error) { +// Returns all confirmed transactions with receipt block nums older than or equal to the finalized block number +func (o *evmTxStore) FindTxesToMarkFinalized(ctx context.Context, finalizedBlockNum int64, chainID *big.Int) (txes []*Tx, err error) { var cancel context.CancelFunc ctx, cancel = o.stopCh.Ctx(ctx) defer cancel() err = o.Transact(ctx, true, func(orm *evmTxStore) error { - sql := "SELECT * FROM evm.txes WHERE state = 'confirmed' AND evm_chain_id = $1" + sql := `SELECT evm.txes.* FROM evm.txes + INNER JOIN evm.tx_attempts ON evm.txes.id = evm.tx_attempts.eth_tx_id + INNER JOIN evm.receipts ON evm.tx_attempts.hash = evm.receipts.tx_hash + WHERE evm.txes.state = 'confirmed' AND evm.receipts.block_number <= $1 AND evm.txes.evm_chain_id = $2` var dbEtxs []DbEthTx - err = o.q.SelectContext(ctx, &dbEtxs, sql, chainID.String()) + err = o.q.SelectContext(ctx, &dbEtxs, sql, finalizedBlockNum, chainID.String()) if len(dbEtxs) == 0 { return nil } diff --git a/core/chains/evm/txmgr/evm_tx_store_test.go b/core/chains/evm/txmgr/evm_tx_store_test.go index fee2fc2d377..be5b63c83ff 100644 --- a/core/chains/evm/txmgr/evm_tx_store_test.go +++ b/core/chains/evm/txmgr/evm_tx_store_test.go @@ -1856,15 +1856,17 @@ func TestORM_FindTransactionsByState(t *testing.T) { txStore := cltest.NewTestTxStore(t, db) kst := cltest.NewKeyStore(t, db) _, fromAddress := cltest.MustInsertRandomKey(t, kst.Eth()) + finalizedBlockNum := int64(100) mustInsertUnstartedTx(t, txStore, fromAddress) mustInsertInProgressEthTxWithAttempt(t, txStore, 0, fromAddress) mustInsertUnconfirmedEthTxWithAttemptState(t, txStore, 1, fromAddress, txmgrtypes.TxAttemptBroadcast) - mustInsertConfirmedMissingReceiptEthTxWithLegacyAttempt(t, txStore, 2, 100, time.Now(), fromAddress) - mustInsertConfirmedEthTxWithReceipt(t, txStore, fromAddress, 3, 100) + mustInsertConfirmedMissingReceiptEthTxWithLegacyAttempt(t, txStore, 2, finalizedBlockNum, time.Now(), fromAddress) + mustInsertConfirmedEthTxWithReceipt(t, txStore, fromAddress, 3, finalizedBlockNum + 1) + mustInsertConfirmedEthTxWithReceipt(t, txStore, fromAddress, 4, finalizedBlockNum) mustInsertFatalErrorEthTx(t, txStore, fromAddress) - txs, err := txStore.FindConfirmedTxes(ctx, testutils.FixtureChainID) + txs, err := txStore.FindTxesToMarkFinalized(ctx, finalizedBlockNum, testutils.FixtureChainID) require.NoError(t, err) require.Len(t, txs, 1) } diff --git a/core/chains/evm/txmgr/finalizer.go b/core/chains/evm/txmgr/finalizer.go index 94c54e534fa..9df979629fd 100644 --- a/core/chains/evm/txmgr/finalizer.go +++ b/core/chains/evm/txmgr/finalizer.go @@ -7,7 +7,7 @@ import ( "sync" "time" - "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/common/hexutil" "github.com/ethereum/go-ethereum/rpc" "github.com/smartcontractkit/chainlink-common/pkg/logger" @@ -19,12 +19,11 @@ import ( var _ Finalizer = (*evmFinalizer)(nil) -// processHeadTimeout represents a sanity limit on how long ProcessHead -// should take to complete +// processHeadTimeout represents a sanity limit on how long ProcessHead should take to complete const processHeadTimeout = 10 * time.Minute type finalizerTxStore interface { - FindConfirmedTxes(ctx context.Context, chainID *big.Int) ([]*Tx, error) + FindTxesToMarkFinalized(ctx context.Context, finalizedBlockNum int64, chainID *big.Int) ([]*Tx, error) UpdateTxesFinalized(ctx context.Context, txs []int64, chainId *big.Int) error } @@ -151,34 +150,38 @@ func (f *evmFinalizer) processFinalizedHead(ctx context.Context, latestFinalized earliestBlockNumInChain := latestFinalizedHead.EarliestHeadInChain().BlockNumber() f.lggr.Debugw("processing latest finalized head", "block num", latestFinalizedHead.BlockNumber(), "block hash", latestFinalizedHead.BlockHash(), "earliest block num in chain", earliestBlockNumInChain) - // Retrieve all confirmed transactions, loaded with attempts and receipts - unfinalizedTxs, err := f.txStore.FindConfirmedTxes(ctx, f.chainId) + // Retrieve all confirmed transactions with receipts older than or equal to the finalized block, loaded with attempts and receipts + unfinalizedTxs, err := f.txStore.FindTxesToMarkFinalized(ctx, latestFinalizedHead.BlockNumber(), f.chainId) if err != nil { return fmt.Errorf("failed to retrieve confirmed transactions: %w", err) } var finalizedTxs []*Tx // Group by block hash transactions whose receipts cannot be validated using the cached heads - receiptBlockHashToTx := make(map[common.Hash][]*Tx) + blockNumToTxesMap := make(map[int64][]*Tx) // Find transactions with receipt block nums older than the latest finalized block num and block hashes still in chain for _, tx := range unfinalizedTxs { receipt := tx.GetReceipt() + // The tx store query ensures transactions have receipts but leaving this check here for a belts and braces approach if receipt == nil || receipt.IsZero() || receipt.IsUnmined() { f.lggr.AssumptionViolationw("invalid receipt found for confirmed transaction", "tx", tx, "receipt", receipt) continue } - // Receipt newer than latest finalized head block num + // The tx store query only returns transactions with receipts older than or equal to the finalized block but leaving this check here for a belts and braces approach if receipt.GetBlockNumber().Cmp(big.NewInt(latestFinalizedHead.BlockNumber())) > 0 { continue } // Receipt block num older than earliest head in chain. Validate hash using RPC call later - if receipt.GetBlockNumber().Int64() < earliestBlockNumInChain { - receiptBlockHashToTx[receipt.GetBlockHash()] = append(receiptBlockHashToTx[receipt.GetBlockHash()], tx) + if receipt.GetBlockNumber().Cmp(big.NewInt(earliestBlockNumInChain)) < 0 { + blockNumToTxesMap[receipt.GetBlockNumber().Int64()] = append(blockNumToTxesMap[receipt.GetBlockNumber().Int64()], tx) continue } blockHashInChain := latestFinalizedHead.HashAtHeight(receipt.GetBlockNumber().Int64()) // Receipt block hash does not match the block hash in chain. Transaction has been re-org'd out but DB state has not been updated yet if blockHashInChain.String() != receipt.GetBlockHash().String() { + // Log a critical error if a transaction is marked as confirmed with a receipt older than the finalized block + // This scenario could potentially point to a re-org'd transaction the Confirmer has lost track of + f.lggr.Criticalw("found confirmed transaction with re-org'd receipt older than finalized block", "tx", tx, "receipt", receipt, "onchainBlockHash", blockHashInChain.String()) continue } finalizedTxs = append(finalizedTxs, tx) @@ -186,7 +189,7 @@ func (f *evmFinalizer) processFinalizedHead(ctx context.Context, latestFinalized // Check if block hashes exist for receipts on-chain older than the earliest cached head // Transactions are grouped by their receipt block hash to avoid repeat requests on the same hash in case transactions were confirmed in the same block - validatedReceiptTxs, err := f.batchCheckReceiptHashesOnchain(ctx, receiptBlockHashToTx, latestFinalizedHead.BlockNumber()) + validatedReceiptTxs, err := f.batchCheckReceiptHashesOnchain(ctx, blockNumToTxesMap) if err != nil { // Do not error out to allow transactions that did not need RPC validation to still be marked as finalized // The transactions failed to be validated will be checked again in the next round @@ -203,18 +206,18 @@ func (f *evmFinalizer) processFinalizedHead(ctx context.Context, latestFinalized return nil } -func (f *evmFinalizer) batchCheckReceiptHashesOnchain(ctx context.Context, receiptMap map[common.Hash][]*Tx, latestFinalizedBlockNum int64) ([]*Tx, error) { - if len(receiptMap) == 0 { +func (f *evmFinalizer) batchCheckReceiptHashesOnchain(ctx context.Context, blockNumToTxesMap map[int64][]*Tx) ([]*Tx, error) { + if len(blockNumToTxesMap) == 0 { return nil, nil } // Group the RPC batch calls in groups of rpcBatchSize var rpcBatchGroups [][]rpc.BatchElem var rpcBatch []rpc.BatchElem - for hash := range receiptMap { + for blockNum := range blockNumToTxesMap { elem := rpc.BatchElem{ - Method: "eth_getBlockByHash", + Method: "eth_getBlockByNumber", Args: []any{ - hash, + hexutil.EncodeBig(big.NewInt(blockNum)), false, }, Result: new(evmtypes.Head), @@ -225,6 +228,9 @@ func (f *evmFinalizer) batchCheckReceiptHashesOnchain(ctx context.Context, recei rpcBatch = []rpc.BatchElem{} } } + if len(rpcBatch) > 0 { + rpcBatchGroups = append(rpcBatchGroups, rpcBatch) + } var finalizedTxs []*Tx for _, rpcBatch := range rpcBatchGroups { @@ -237,20 +243,28 @@ func (f *evmFinalizer) batchCheckReceiptHashesOnchain(ctx context.Context, recei for _, req := range rpcBatch { if req.Error != nil { // Continue if particular RPC call failed so other txs can still be considered for finalization - f.lggr.Debugw("failed to find block by hash", "hash", req.Args[0]) + f.lggr.Debugw("failed to find block by number", "blockNum", req.Args[0]) continue } head := req.Result.(*evmtypes.Head) if head == nil { - // Continue if particular RPC call yielded a nil head so other txs can still be considered for finalization - f.lggr.Debugw("failed to find block by hash", "hash", req.Args[0]) + // Continue if particular RPC call yielded a nil block so other txs can still be considered for finalization + f.lggr.Debugw("failed to find block by number", "blockNum", req.Args[0]) continue } - // Confirmed receipt's block hash exists on-chain - // Add to finalized list if block num less than or equal to the latest finalized head block num - if head.BlockNumber() <= latestFinalizedBlockNum { - txs := receiptMap[head.BlockHash()] - finalizedTxs = append(finalizedTxs, txs...) + txs := blockNumToTxesMap[head.BlockNumber()] + // Check if transaction receipts match the block hash at the given block num + // If they do not, the transactions may have been re-org'd out + // The expectation is for the Confirmer to pick up on these re-orgs and get the transaction included + for _, tx := range txs { + receipt := tx.GetReceipt() + if receipt.GetBlockHash().String() == head.BlockHash().String() { + finalizedTxs = append(finalizedTxs, tx) + } else { + // Log a critical error if a transaction is marked as confirmed with a receipt older than the finalized block + // This scenario could potentially point to a re-org'd transaction the Confirmer has lost track of + f.lggr.Criticalw("found confirmed transaction with re-org'd receipt older than finalized block", "tx", tx, "receipt", receipt, "onchainBlockHash", head.BlockHash().String()) + } } } } diff --git a/core/chains/evm/txmgr/finalizer_test.go b/core/chains/evm/txmgr/finalizer_test.go index 50d46a61c08..a6f0d13f99f 100644 --- a/core/chains/evm/txmgr/finalizer_test.go +++ b/core/chains/evm/txmgr/finalizer_test.go @@ -2,10 +2,12 @@ package txmgr_test import ( "errors" + "math/big" "testing" "time" "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/common/hexutil" "github.com/ethereum/go-ethereum/rpc" "github.com/google/uuid" "github.com/stretchr/testify/mock" @@ -171,14 +173,16 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { rpcElements := args.Get(1).([]rpc.BatchElem) require.Equal(t, 1, len(rpcElements)) - require.Equal(t, "eth_getBlockByHash", rpcElements[0].Method) + require.Equal(t, "eth_getBlockByNumber", rpcElements[0].Method) require.Equal(t, false, rpcElements[0].Args[1]) - reqHash := rpcElements[0].Args[0].(common.Hash).String() + reqBlockNum := rpcElements[0].Args[0].(string) + req1BlockNum := hexutil.EncodeBig(big.NewInt(head.Parent.Number - 2)) + req2BlockNum := hexutil.EncodeBig(big.NewInt(head.Parent.Number - 1)) var headResult evmtypes.Head - if receiptHash1.String() == reqHash { + if req1BlockNum == reqBlockNum { headResult = evmtypes.Head{Number: head.Parent.Number - 2, Hash: receiptHash1} - } else if receiptHash2.String() == reqHash { + } else if req2BlockNum == reqBlockNum { headResult = evmtypes.Head{Number: head.Parent.Number - 1, Hash: receiptHash2} } else { require.Fail(t, "unrecognized block hash") From db94c84b9452f1a487121d55a9ccc10ab11cb99b Mon Sep 17 00:00:00 2001 From: amit-momin Date: Wed, 10 Jul 2024 16:36:38 -0500 Subject: [PATCH 13/23] Updated reaper to reap old confirmed transactions --- common/txmgr/types/mocks/tx_store.go | 60 ++++++++++----------- core/chains/evm/txmgr/evm_tx_store.go | 36 ++++++++++++- core/chains/evm/txmgr/mocks/evm_tx_store.go | 60 ++++++++++----------- core/chains/evm/txmgr/reaper_test.go | 31 +++++++---- 4 files changed, 116 insertions(+), 71 deletions(-) diff --git a/common/txmgr/types/mocks/tx_store.go b/common/txmgr/types/mocks/tx_store.go index e3a2ab7aefa..a6c72b4b97d 100644 --- a/common/txmgr/types/mocks/tx_store.go +++ b/common/txmgr/types/mocks/tx_store.go @@ -196,36 +196,6 @@ func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) DeleteInPro return r0 } -// FindConfirmedTxes provides a mock function with given fields: ctx, chainID -func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindConfirmedTxes(ctx context.Context, chainID CHAIN_ID) ([]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error) { - ret := _m.Called(ctx, chainID) - - if len(ret) == 0 { - panic("no return value specified for FindConfirmedTxes") - } - - var r0 []*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] - var r1 error - if rf, ok := ret.Get(0).(func(context.Context, CHAIN_ID) ([]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error)); ok { - return rf(ctx, chainID) - } - if rf, ok := ret.Get(0).(func(context.Context, CHAIN_ID) []*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]); ok { - r0 = rf(ctx, chainID) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).([]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) - } - } - - if rf, ok := ret.Get(1).(func(context.Context, CHAIN_ID) error); ok { - r1 = rf(ctx, chainID) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - // FindEarliestUnconfirmedBroadcastTime provides a mock function with given fields: ctx, chainID func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindEarliestUnconfirmedBroadcastTime(ctx context.Context, chainID CHAIN_ID) (null.Time, error) { ret := _m.Called(ctx, chainID) @@ -580,6 +550,36 @@ func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxesPen return r0, r1 } +// FindTxesToMarkFinalized provides a mock function with given fields: ctx, finalizedBlockNum, chainID +func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxesToMarkFinalized(ctx context.Context, finalizedBlockNum int64, chainID CHAIN_ID) ([]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error) { + ret := _m.Called(ctx, finalizedBlockNum, chainID) + + if len(ret) == 0 { + panic("no return value specified for FindTxesToMarkFinalized") + } + + var r0 []*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, int64, CHAIN_ID) ([]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error)); ok { + return rf(ctx, finalizedBlockNum, chainID) + } + if rf, ok := ret.Get(0).(func(context.Context, int64, CHAIN_ID) []*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]); ok { + r0 = rf(ctx, finalizedBlockNum, chainID) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, int64, CHAIN_ID) error); ok { + r1 = rf(ctx, finalizedBlockNum, chainID) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + // FindTxesWithAttemptsAndReceiptsByIdsAndState provides a mock function with given fields: ctx, ids, states, chainID func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxesWithAttemptsAndReceiptsByIdsAndState(ctx context.Context, ids []int64, states []txmgrtypes.TxState, chainID *big.Int) ([]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error) { ret := _m.Called(ctx, ids, states, chainID) diff --git a/core/chains/evm/txmgr/evm_tx_store.go b/core/chains/evm/txmgr/evm_tx_store.go index 325401ff063..d8054993d99 100644 --- a/core/chains/evm/txmgr/evm_tx_store.go +++ b/core/chains/evm/txmgr/evm_tx_store.go @@ -1882,7 +1882,7 @@ AND evm.txes.created_at < $2 AND evm.txes.state = 'finalized' AND evm_chain_id = $3`, limit, timeThreshold, chainID.String()) if err != nil { - return count, pkgerrors.Wrap(err, "ReapTxes failed to delete old confirmed evm.txes") + return count, pkgerrors.Wrap(err, "ReapTxes failed to delete old finalized evm.txes") } rowsAffected, err := res.RowsAffected() if err != nil { @@ -1891,7 +1891,7 @@ AND evm_chain_id = $3`, limit, timeThreshold, chainID.String()) return uint(rowsAffected), err }, batchSize) if err != nil { - return pkgerrors.Wrap(err, "TxmReaper#reapEthTxes batch delete of confirmed evm.txes failed") + return pkgerrors.Wrap(err, "TxmReaper#reapEthTxes batch delete of finalized evm.txes failed") } // Delete old 'fatal_error' evm.txes err = sqlutil.Batch(func(_, limit uint) (count uint, err error) { @@ -1912,6 +1912,38 @@ AND evm_chain_id = $2`, timeThreshold, chainID.String()) if err != nil { return pkgerrors.Wrap(err, "TxmReaper#reapEthTxes batch delete of fatally errored evm.txes failed") } + // Delete old 'confirmed' evm.txes that were never finalized + // This query should never result in changes but added just in case transactions slip through the cracks + // to avoid them building up in the DB + err = sqlutil.Batch(func(_, limit uint) (count uint, err error) { + res, err := o.q.ExecContext(ctx, ` +WITH old_enough_receipts AS ( + SELECT tx_hash FROM evm.receipts + ORDER BY block_number ASC, id ASC + LIMIT $1 +) +DELETE FROM evm.txes +USING old_enough_receipts, evm.tx_attempts +WHERE evm.tx_attempts.eth_tx_id = evm.txes.id +AND evm.tx_attempts.hash = old_enough_receipts.tx_hash +AND evm.txes.created_at < $2 +AND evm.txes.state = 'confirmed' +AND evm_chain_id = $3`, limit, timeThreshold, chainID.String()) + if err != nil { + return count, pkgerrors.Wrap(err, "ReapTxes failed to delete old confirmed evm.txes") + } + rowsAffected, err := res.RowsAffected() + if err != nil { + return count, pkgerrors.Wrap(err, "ReapTxes failed to get rows affected") + } + if rowsAffected > 0 { + o.logger.Criticalf("%d confirmed transactions were reaped before being marked as finalized. This should never happen unless the threshold is set too low or the transactions were lost track of", rowsAffected) + } + return uint(rowsAffected), err + }, batchSize) + if err != nil { + return pkgerrors.Wrap(err, "TxmReaper#reapEthTxes batch delete of confirmed evm.txes failed") + } return nil } diff --git a/core/chains/evm/txmgr/mocks/evm_tx_store.go b/core/chains/evm/txmgr/mocks/evm_tx_store.go index a8683505b5c..a74ef19b5b3 100644 --- a/core/chains/evm/txmgr/mocks/evm_tx_store.go +++ b/core/chains/evm/txmgr/mocks/evm_tx_store.go @@ -199,36 +199,6 @@ func (_m *EvmTxStore) DeleteInProgressAttempt(ctx context.Context, attempt types return r0 } -// FindConfirmedTxes provides a mock function with given fields: ctx, chainID -func (_m *EvmTxStore) FindConfirmedTxes(ctx context.Context, chainID *big.Int) ([]*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error) { - ret := _m.Called(ctx, chainID) - - if len(ret) == 0 { - panic("no return value specified for FindConfirmedTxes") - } - - var r0 []*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee] - var r1 error - if rf, ok := ret.Get(0).(func(context.Context, *big.Int) ([]*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error)); ok { - return rf(ctx, chainID) - } - if rf, ok := ret.Get(0).(func(context.Context, *big.Int) []*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]); ok { - r0 = rf(ctx, chainID) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).([]*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]) - } - } - - if rf, ok := ret.Get(1).(func(context.Context, *big.Int) error); ok { - r1 = rf(ctx, chainID) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - // FindEarliestUnconfirmedBroadcastTime provides a mock function with given fields: ctx, chainID func (_m *EvmTxStore) FindEarliestUnconfirmedBroadcastTime(ctx context.Context, chainID *big.Int) (null.Time, error) { ret := _m.Called(ctx, chainID) @@ -701,6 +671,36 @@ func (_m *EvmTxStore) FindTxesPendingCallback(ctx context.Context, blockNum int6 return r0, r1 } +// FindTxesToMarkFinalized provides a mock function with given fields: ctx, finalizedBlockNum, chainID +func (_m *EvmTxStore) FindTxesToMarkFinalized(ctx context.Context, finalizedBlockNum int64, chainID *big.Int) ([]*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error) { + ret := _m.Called(ctx, finalizedBlockNum, chainID) + + if len(ret) == 0 { + panic("no return value specified for FindTxesToMarkFinalized") + } + + var r0 []*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee] + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, int64, *big.Int) ([]*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error)); ok { + return rf(ctx, finalizedBlockNum, chainID) + } + if rf, ok := ret.Get(0).(func(context.Context, int64, *big.Int) []*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]); ok { + r0 = rf(ctx, finalizedBlockNum, chainID) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, int64, *big.Int) error); ok { + r1 = rf(ctx, finalizedBlockNum, chainID) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + // FindTxesWithAttemptsAndReceiptsByIdsAndState provides a mock function with given fields: ctx, ids, states, chainID func (_m *EvmTxStore) FindTxesWithAttemptsAndReceiptsByIdsAndState(ctx context.Context, ids []int64, states []types.TxState, chainID *big.Int) ([]*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error) { ret := _m.Called(ctx, ids, states, chainID) diff --git a/core/chains/evm/txmgr/reaper_test.go b/core/chains/evm/txmgr/reaper_test.go index a8fe81ae1b9..cfaccdf04eb 100644 --- a/core/chains/evm/txmgr/reaper_test.go +++ b/core/chains/evm/txmgr/reaper_test.go @@ -93,18 +93,11 @@ func TestReaper_ReapTxes(t *testing.T) { // Didn't delete because eth_tx was not old enough cltest.AssertCount(t, db, "evm.txes", 1) - pgtest.MustExec(t, db, `UPDATE evm.txes SET created_at=$1`, oneDayAgo) - - err = r.ReapTxes(12) - assert.NoError(t, err) - // Didn't delete because eth_tx although old enough, was not marked as finalized - cltest.AssertCount(t, db, "evm.txes", 1) - - pgtest.MustExec(t, db, `UPDATE evm.txes SET state='finalized'`) + pgtest.MustExec(t, db, `UPDATE evm.txes SET created_at=$1, state='finalized'`, oneDayAgo) err = r.ReapTxes(42) assert.NoError(t, err) - // Now it deleted because the eth_tx was past EVM.FinalityDepth + // Now it deleted because the eth_tx was past the age threshold cltest.AssertCount(t, db, "evm.txes", 0) }) @@ -127,4 +120,24 @@ func TestReaper_ReapTxes(t *testing.T) { // Deleted because it is old enough now cltest.AssertCount(t, db, "evm.txes", 0) }) + + mustInsertConfirmedEthTxWithReceipt(t, txStore, from, 0, 42) + + t.Run("deletes confirmed evm.txes that exceed the age threshold", func(t *testing.T) { + tc := &reaperConfig{reaperThreshold: 1 * time.Hour} + + r := newReaper(t, txStore, tc) + + err := r.ReapTxes(42) + assert.NoError(t, err) + // Didn't delete because eth_tx was not old enough + cltest.AssertCount(t, db, "evm.txes", 1) + + pgtest.MustExec(t, db, `UPDATE evm.txes SET created_at=$1`, oneDayAgo) + + err = r.ReapTxes(42) + assert.NoError(t, err) + // Now it deleted because the eth_tx was past the age threshold + cltest.AssertCount(t, db, "evm.txes", 0) + }) } From ffe0dc4be53dbd177615bfb36a85f91c49e5668f Mon Sep 17 00:00:00 2001 From: amit-momin Date: Wed, 10 Jul 2024 17:09:50 -0500 Subject: [PATCH 14/23] Fixed migration test --- core/store/migrate/migrate_test.go | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/core/store/migrate/migrate_test.go b/core/store/migrate/migrate_test.go index 0f89a3f5840..f4e91f0a2d2 100644 --- a/core/store/migrate/migrate_test.go +++ b/core/store/migrate/migrate_test.go @@ -620,9 +620,12 @@ func BenchmarkBackfillingRecordsWithMigration202(b *testing.B) { } func TestRollback_247_TxStateEnumUpdate(t *testing.T) { + ctx := testutils.Context(t) _, db := heavyweight.FullTestDBV2(t, nil) - err := goose.DownTo(db.DB, migrationDir, 54) + p, err := migrate.NewProvider(ctx, db.DB) + require.NoError(t, err) + _, err = p.DownTo(ctx, 54) require.NoError(t, err) - err = goose.UpTo(db.DB, migrationDir, 247) + _, err = p.UpTo(ctx, 247) require.NoError(t, err) } From 03b92f0a14333cb51fda6590edd5f4e2c146ada8 Mon Sep 17 00:00:00 2001 From: amit-momin Date: Wed, 10 Jul 2024 18:37:03 -0500 Subject: [PATCH 15/23] Fixed lint error --- core/chains/evm/txmgr/evm_tx_store_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/chains/evm/txmgr/evm_tx_store_test.go b/core/chains/evm/txmgr/evm_tx_store_test.go index be5b63c83ff..7460f7b927b 100644 --- a/core/chains/evm/txmgr/evm_tx_store_test.go +++ b/core/chains/evm/txmgr/evm_tx_store_test.go @@ -1862,7 +1862,7 @@ func TestORM_FindTransactionsByState(t *testing.T) { mustInsertInProgressEthTxWithAttempt(t, txStore, 0, fromAddress) mustInsertUnconfirmedEthTxWithAttemptState(t, txStore, 1, fromAddress, txmgrtypes.TxAttemptBroadcast) mustInsertConfirmedMissingReceiptEthTxWithLegacyAttempt(t, txStore, 2, finalizedBlockNum, time.Now(), fromAddress) - mustInsertConfirmedEthTxWithReceipt(t, txStore, fromAddress, 3, finalizedBlockNum + 1) + mustInsertConfirmedEthTxWithReceipt(t, txStore, fromAddress, 3, finalizedBlockNum+1) mustInsertConfirmedEthTxWithReceipt(t, txStore, fromAddress, 4, finalizedBlockNum) mustInsertFatalErrorEthTx(t, txStore, fromAddress) From 21443a34791a53c7c31558fc7df8527dfdb324b5 Mon Sep 17 00:00:00 2001 From: amit-momin Date: Wed, 10 Jul 2024 19:15:40 -0500 Subject: [PATCH 16/23] Changed log level --- core/chains/evm/txmgr/finalizer.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/chains/evm/txmgr/finalizer.go b/core/chains/evm/txmgr/finalizer.go index 9df979629fd..72364f90e3c 100644 --- a/core/chains/evm/txmgr/finalizer.go +++ b/core/chains/evm/txmgr/finalizer.go @@ -179,9 +179,9 @@ func (f *evmFinalizer) processFinalizedHead(ctx context.Context, latestFinalized blockHashInChain := latestFinalizedHead.HashAtHeight(receipt.GetBlockNumber().Int64()) // Receipt block hash does not match the block hash in chain. Transaction has been re-org'd out but DB state has not been updated yet if blockHashInChain.String() != receipt.GetBlockHash().String() { - // Log a critical error if a transaction is marked as confirmed with a receipt older than the finalized block + // Log error if a transaction is marked as confirmed with a receipt older than the finalized block // This scenario could potentially point to a re-org'd transaction the Confirmer has lost track of - f.lggr.Criticalw("found confirmed transaction with re-org'd receipt older than finalized block", "tx", tx, "receipt", receipt, "onchainBlockHash", blockHashInChain.String()) + f.lggr.Errorw("found confirmed transaction with re-org'd receipt older than finalized block", "tx", tx, "receipt", receipt, "onchainBlockHash", blockHashInChain.String()) continue } finalizedTxs = append(finalizedTxs, tx) @@ -261,9 +261,9 @@ func (f *evmFinalizer) batchCheckReceiptHashesOnchain(ctx context.Context, block if receipt.GetBlockHash().String() == head.BlockHash().String() { finalizedTxs = append(finalizedTxs, tx) } else { - // Log a critical error if a transaction is marked as confirmed with a receipt older than the finalized block + // Log error if a transaction is marked as confirmed with a receipt older than the finalized block // This scenario could potentially point to a re-org'd transaction the Confirmer has lost track of - f.lggr.Criticalw("found confirmed transaction with re-org'd receipt older than finalized block", "tx", tx, "receipt", receipt, "onchainBlockHash", head.BlockHash().String()) + f.lggr.Errorw("found confirmed transaction with re-org'd receipt older than finalized block", "tx", tx, "receipt", receipt, "onchainBlockHash", head.BlockHash().String()) } } } From e3a8ab3cc3aabadf59ba06cb78c08a101562d8b7 Mon Sep 17 00:00:00 2001 From: amit-momin Date: Tue, 16 Jul 2024 12:24:35 -0500 Subject: [PATCH 17/23] Renumbered sql migration --- ...add_tx_finalized_state.sql => 0248_add_tx_finalized_state.sql} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename core/store/migrate/migrations/{0247_add_tx_finalized_state.sql => 0248_add_tx_finalized_state.sql} (100%) diff --git a/core/store/migrate/migrations/0247_add_tx_finalized_state.sql b/core/store/migrate/migrations/0248_add_tx_finalized_state.sql similarity index 100% rename from core/store/migrate/migrations/0247_add_tx_finalized_state.sql rename to core/store/migrate/migrations/0248_add_tx_finalized_state.sql From ecba4ca47d41cab9201ebdad46d2fe6cacf72d9d Mon Sep 17 00:00:00 2001 From: amit-momin Date: Wed, 24 Jul 2024 20:14:48 +0200 Subject: [PATCH 18/23] Updated Finalizer to only process on new finalized heads and improved query performance --- common/txmgr/types/tx.go | 11 --- common/txmgr/types/tx_store.go | 2 - core/chains/evm/txmgr/evm_tx_store.go | 44 ++++----- core/chains/evm/txmgr/evm_tx_store_test.go | 10 +- core/chains/evm/txmgr/finalizer.go | 102 +++++++++++---------- core/chains/evm/txmgr/finalizer_test.go | 47 ++++++---- 6 files changed, 117 insertions(+), 99 deletions(-) diff --git a/common/txmgr/types/tx.go b/common/txmgr/types/tx.go index fbea1078fdc..b65f7edf6e5 100644 --- a/common/txmgr/types/tx.go +++ b/common/txmgr/types/tx.go @@ -338,17 +338,6 @@ func (e *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) GetChecker() (Transm return t, nil } -// Returns the transaction's receipt if one exists, otherwise returns nil -func (e *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) GetReceipt() ChainReceipt[TX_HASH, BLOCK_HASH] { - for _, attempt := range e.TxAttempts { - if len(attempt.Receipts) > 0 { - // Transaction will only have one receipt - return attempt.Receipts[0] - } - } - return nil -} - // Provides error classification to external components in a chain agnostic way // Only exposes the error types that could be set in the transaction error field type ErrorClassifier interface { diff --git a/common/txmgr/types/tx_store.go b/common/txmgr/types/tx_store.go index 669fb9f54cd..bb564c6b4a7 100644 --- a/common/txmgr/types/tx_store.go +++ b/common/txmgr/types/tx_store.go @@ -82,7 +82,6 @@ type TransactionStore[ FindTransactionsConfirmedInBlockRange(ctx context.Context, highBlockNumber, lowBlockNumber int64, chainID CHAIN_ID) (etxs []*Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) FindEarliestUnconfirmedBroadcastTime(ctx context.Context, chainID CHAIN_ID) (null.Time, error) FindEarliestUnconfirmedTxAttemptBlock(ctx context.Context, chainID CHAIN_ID) (null.Int, error) - FindTxesToMarkFinalized(ctx context.Context, finalizedBlockNum int64, chainID CHAIN_ID) (etxs []*Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) GetTxInProgress(ctx context.Context, fromAddress ADDR) (etx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) GetInProgressTxAttempts(ctx context.Context, address ADDR, chainID CHAIN_ID) (attempts []TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) GetAbandonedTransactionsByBatch(ctx context.Context, chainID CHAIN_ID, enabledAddrs []ADDR, offset, limit uint) (txs []*Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) @@ -105,7 +104,6 @@ type TransactionStore[ UpdateTxsUnconfirmed(ctx context.Context, ids []int64) error UpdateTxUnstartedToInProgress(ctx context.Context, etx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], attempt *TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) error UpdateTxFatalError(ctx context.Context, etx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) error - UpdateTxesFinalized(ctx context.Context, etxs []int64, chainId CHAIN_ID) error UpdateTxForRebroadcast(ctx context.Context, etx Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], etxAttempt TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) error } diff --git a/core/chains/evm/txmgr/evm_tx_store.go b/core/chains/evm/txmgr/evm_tx_store.go index 262c79e2451..819eab4a18c 100644 --- a/core/chains/evm/txmgr/evm_tx_store.go +++ b/core/chains/evm/txmgr/evm_tx_store.go @@ -44,6 +44,10 @@ type EvmTxStore interface { // redeclare TxStore for mockery txmgrtypes.TxStore[common.Address, *big.Int, common.Hash, common.Hash, *evmtypes.Receipt, evmtypes.Nonce, gas.EvmFee] TxStoreWebApi + + // methods used solely in EVM components + FindConfirmedTxesReceipts(ctx context.Context, finalizedBlockNum int64, chainID *big.Int) (receipts []Receipt, err error) + UpdateTxesFinalized(ctx context.Context, etxIDs []int64, chainId *big.Int) error } // TxStoreWebApi encapsulates the methods that are not used by the txmgr and only used by the various web controllers, readers, or evm specific components @@ -1933,7 +1937,7 @@ AND evm_chain_id = $3`, limit, timeThreshold, chainID.String()) return count, pkgerrors.Wrap(err, "ReapTxes failed to get rows affected") } if rowsAffected > 0 { - o.logger.Criticalf("%d confirmed transactions were reaped before being marked as finalized. This should never happen unless the threshold is set too low or the transactions were lost track of", rowsAffected) + o.logger.Errorf("%d confirmed transactions were reaped before being marked as finalized. This should never happen unless the threshold is set too low or the transactions were lost track of", rowsAffected) } return uint(rowsAffected), err }, batchSize) @@ -2070,39 +2074,37 @@ func (o *evmTxStore) UpdateTxAttemptBroadcastBeforeBlockNum(ctx context.Context, } // Returns all confirmed transactions with receipt block nums older than or equal to the finalized block number -func (o *evmTxStore) FindTxesToMarkFinalized(ctx context.Context, finalizedBlockNum int64, chainID *big.Int) (txes []*Tx, err error) { +func (o *evmTxStore) FindConfirmedTxesReceipts(ctx context.Context, finalizedBlockNum int64, chainID *big.Int) (receipts []Receipt, err error) { var cancel context.CancelFunc ctx, cancel = o.stopCh.Ctx(ctx) defer cancel() err = o.Transact(ctx, true, func(orm *evmTxStore) error { - sql := `SELECT evm.txes.* FROM evm.txes - INNER JOIN evm.tx_attempts ON evm.txes.id = evm.tx_attempts.eth_tx_id - INNER JOIN evm.receipts ON evm.tx_attempts.hash = evm.receipts.tx_hash + sql := `SELECT evm.receipts.* FROM evm.receipts + INNER JOIN evm.tx_attempts ON evm.tx_attempts.hash = evm.receipts.tx_hash + INNER JOIN evm.txes ON evm.txes.id = evm.tx_attempts.eth_tx_id WHERE evm.txes.state = 'confirmed' AND evm.receipts.block_number <= $1 AND evm.txes.evm_chain_id = $2` - var dbEtxs []DbEthTx - err = o.q.SelectContext(ctx, &dbEtxs, sql, finalizedBlockNum, chainID.String()) - if len(dbEtxs) == 0 { + var dbReceipts []dbReceipt + err = o.q.SelectContext(ctx, &dbReceipts, sql, finalizedBlockNum, chainID.String()) + if len(dbReceipts) == 0 { return nil } - txes = make([]*Tx, len(dbEtxs)) - dbEthTxsToEvmEthTxPtrs(dbEtxs, txes) - if err = orm.LoadTxesAttempts(ctx, txes); err != nil { - return pkgerrors.Wrapf(err, "failed to load evm.tx_attempts for evm.tx") - } - if err = orm.loadEthTxesAttemptsReceipts(ctx, txes); err != nil { - return pkgerrors.Wrapf(err, "failed to load evm.receipts for evm.tx") - } + receipts = dbReceipts return nil }) - return txes, err + return receipts, err } -// Mark transactions provided as finalized -func (o *evmTxStore) UpdateTxesFinalized(ctx context.Context, etxIDs []int64, chainId *big.Int) error { +// Mark transactions corresponding to receipt IDs as finalized +func (o *evmTxStore) UpdateTxesFinalized(ctx context.Context, receiptIDs []int64, chainId *big.Int) error { var cancel context.CancelFunc ctx, cancel = o.stopCh.Ctx(ctx) defer cancel() - sql := "UPDATE evm.txes SET state = 'finalized' WHERE id = ANY($1) AND evm_chain_id = $2" - _, err := o.q.ExecContext(ctx, sql, pq.Array(etxIDs), chainId.String()) + sql := ` +UPDATE evm.txes SET state = 'finalized' WHERE evm.txes.evm_chain_id = $1 AND evm.txes.id IN (SELECT evm.txes.id FROM evm.txes + INNER JOIN evm.tx_attempts ON evm.tx_attempts.eth_tx_id = evm.txes.id + INNER JOIN evm.receipts ON evm.receipts.tx_hash = evm.tx_attempts.hash + WHERE evm.receipts.id = ANY($2)) +` + _, err := o.q.ExecContext(ctx, sql, chainId.String(), pq.Array(receiptIDs)) return err } diff --git a/core/chains/evm/txmgr/evm_tx_store_test.go b/core/chains/evm/txmgr/evm_tx_store_test.go index 7460f7b927b..e98b2c9b97e 100644 --- a/core/chains/evm/txmgr/evm_tx_store_test.go +++ b/core/chains/evm/txmgr/evm_tx_store_test.go @@ -1866,9 +1866,9 @@ func TestORM_FindTransactionsByState(t *testing.T) { mustInsertConfirmedEthTxWithReceipt(t, txStore, fromAddress, 4, finalizedBlockNum) mustInsertFatalErrorEthTx(t, txStore, fromAddress) - txs, err := txStore.FindTxesToMarkFinalized(ctx, finalizedBlockNum, testutils.FixtureChainID) + receipts, err := txStore.FindConfirmedTxesReceipts(ctx, finalizedBlockNum, testutils.FixtureChainID) require.NoError(t, err) - require.Len(t, txs, 1) + require.Len(t, receipts, 1) } func TestORM_UpdateTxesFinalized(t *testing.T) { @@ -1893,7 +1893,11 @@ func TestORM_UpdateTxesFinalized(t *testing.T) { } err := txStore.InsertTx(ctx, tx) require.NoError(t, err) - err = txStore.UpdateTxesFinalized(ctx, []int64{tx.ID}, testutils.FixtureChainID) + attempt := newBroadcastLegacyEthTxAttempt(t, tx.ID) + err = txStore.InsertTxAttempt(ctx, &attempt) + require.NoError(t, err) + receipt := mustInsertEthReceipt(t, txStore, 100, testutils.NewHash(), attempt.Hash) + err = txStore.UpdateTxesFinalized(ctx, []int64{receipt.ID}, testutils.FixtureChainID) require.NoError(t, err) etx, err := txStore.FindTxWithAttempts(ctx, tx.ID) require.NoError(t, err) diff --git a/core/chains/evm/txmgr/finalizer.go b/core/chains/evm/txmgr/finalizer.go index 72364f90e3c..8c6c3e8e2b5 100644 --- a/core/chains/evm/txmgr/finalizer.go +++ b/core/chains/evm/txmgr/finalizer.go @@ -23,7 +23,7 @@ var _ Finalizer = (*evmFinalizer)(nil) const processHeadTimeout = 10 * time.Minute type finalizerTxStore interface { - FindTxesToMarkFinalized(ctx context.Context, finalizedBlockNum int64, chainID *big.Int) ([]*Tx, error) + FindConfirmedTxesReceipts(ctx context.Context, finalizedBlockNum int64, chainID *big.Int) ([]Receipt, error) UpdateTxesFinalized(ctx context.Context, txs []int64, chainId *big.Int) error } @@ -49,6 +49,8 @@ type evmFinalizer struct { mb *mailbox.Mailbox[*evmtypes.Head] stopCh services.StopChan wg sync.WaitGroup + + lastProcessedFinalizedBlockNum int64 } func NewEvmFinalizer( @@ -147,73 +149,85 @@ func (f *evmFinalizer) processFinalizedHead(ctx context.Context, latestFinalized if latestFinalizedHead == nil || !latestFinalizedHead.IsValid() { return fmt.Errorf("invalid latestFinalizedHead") } + // Only continue processing if the latestFinalizedHead has not already been processed + // Helps avoid unnecessary processing on every head if blocks are finalized in batches + if latestFinalizedHead.BlockNumber() == f.lastProcessedFinalizedBlockNum { + return nil + } + if latestFinalizedHead.BlockNumber() < f.lastProcessedFinalizedBlockNum { + f.lggr.Errorw("Received finalized block older than one already processed. This should never happen and could be an issue with RPCs.", "lastProcessedFinalizedBlockNum", f.lastProcessedFinalizedBlockNum, "retrievedFinalizedBlockNum", latestFinalizedHead.BlockNumber()) + return nil + } + earliestBlockNumInChain := latestFinalizedHead.EarliestHeadInChain().BlockNumber() f.lggr.Debugw("processing latest finalized head", "block num", latestFinalizedHead.BlockNumber(), "block hash", latestFinalizedHead.BlockHash(), "earliest block num in chain", earliestBlockNumInChain) // Retrieve all confirmed transactions with receipts older than or equal to the finalized block, loaded with attempts and receipts - unfinalizedTxs, err := f.txStore.FindTxesToMarkFinalized(ctx, latestFinalizedHead.BlockNumber(), f.chainId) + unfinalizedReceipts, err := f.txStore.FindConfirmedTxesReceipts(ctx, latestFinalizedHead.BlockNumber(), f.chainId) if err != nil { - return fmt.Errorf("failed to retrieve confirmed transactions: %w", err) + return fmt.Errorf("failed to retrieve receipts for confirmed, unfinalized transactions: %w", err) } - var finalizedTxs []*Tx + var finalizedReceipts []Receipt // Group by block hash transactions whose receipts cannot be validated using the cached heads - blockNumToTxesMap := make(map[int64][]*Tx) + blockNumToReceiptsMap := make(map[int64][]Receipt) // Find transactions with receipt block nums older than the latest finalized block num and block hashes still in chain - for _, tx := range unfinalizedTxs { - receipt := tx.GetReceipt() + for _, receipt := range unfinalizedReceipts { // The tx store query ensures transactions have receipts but leaving this check here for a belts and braces approach - if receipt == nil || receipt.IsZero() || receipt.IsUnmined() { - f.lggr.AssumptionViolationw("invalid receipt found for confirmed transaction", "tx", tx, "receipt", receipt) + if receipt.Receipt.IsZero() || receipt.Receipt.IsUnmined() { + f.lggr.AssumptionViolationw("invalid receipt found for confirmed transaction", "receipt", receipt) continue } // The tx store query only returns transactions with receipts older than or equal to the finalized block but leaving this check here for a belts and braces approach - if receipt.GetBlockNumber().Cmp(big.NewInt(latestFinalizedHead.BlockNumber())) > 0 { + if receipt.BlockNumber > latestFinalizedHead.BlockNumber() { continue } // Receipt block num older than earliest head in chain. Validate hash using RPC call later - if receipt.GetBlockNumber().Cmp(big.NewInt(earliestBlockNumInChain)) < 0 { - blockNumToTxesMap[receipt.GetBlockNumber().Int64()] = append(blockNumToTxesMap[receipt.GetBlockNumber().Int64()], tx) + if receipt.BlockNumber < earliestBlockNumInChain { + blockNumToReceiptsMap[receipt.BlockNumber] = append(blockNumToReceiptsMap[receipt.BlockNumber], receipt) continue } - blockHashInChain := latestFinalizedHead.HashAtHeight(receipt.GetBlockNumber().Int64()) + blockHashInChain := latestFinalizedHead.HashAtHeight(receipt.BlockNumber) // Receipt block hash does not match the block hash in chain. Transaction has been re-org'd out but DB state has not been updated yet - if blockHashInChain.String() != receipt.GetBlockHash().String() { + if blockHashInChain.String() != receipt.BlockHash.String() { // Log error if a transaction is marked as confirmed with a receipt older than the finalized block // This scenario could potentially point to a re-org'd transaction the Confirmer has lost track of - f.lggr.Errorw("found confirmed transaction with re-org'd receipt older than finalized block", "tx", tx, "receipt", receipt, "onchainBlockHash", blockHashInChain.String()) + f.lggr.Errorw("found confirmed transaction with re-org'd receipt older than finalized block", "receipt", receipt, "onchainBlockHash", blockHashInChain.String()) continue } - finalizedTxs = append(finalizedTxs, tx) + finalizedReceipts = append(finalizedReceipts, receipt) } // Check if block hashes exist for receipts on-chain older than the earliest cached head // Transactions are grouped by their receipt block hash to avoid repeat requests on the same hash in case transactions were confirmed in the same block - validatedReceiptTxs, err := f.batchCheckReceiptHashesOnchain(ctx, blockNumToTxesMap) + validatedReceipts, err := f.batchCheckReceiptHashesOnchain(ctx, blockNumToReceiptsMap) if err != nil { // Do not error out to allow transactions that did not need RPC validation to still be marked as finalized // The transactions failed to be validated will be checked again in the next round f.lggr.Errorf("failed to validate receipt block hashes over RPC: %v", err) } - finalizedTxs = append(finalizedTxs, validatedReceiptTxs...) + finalizedReceipts = append(finalizedReceipts, validatedReceipts...) - etxIDs := f.buildTxIdList(finalizedTxs) + receiptIDs := f.buildReceiptIdList(finalizedReceipts) - err = f.txStore.UpdateTxesFinalized(ctx, etxIDs, f.chainId) + err = f.txStore.UpdateTxesFinalized(ctx, receiptIDs, f.chainId) if err != nil { return fmt.Errorf("failed to update transactions as finalized: %w", err) } + // Update lastProcessedFinalizedBlockNum after processing has completed to allow failed processing to retry on subsequent heads + // Does not need to be protected with mutex lock because the Finalizer only runs in a single loop + f.lastProcessedFinalizedBlockNum = latestFinalizedHead.BlockNumber() return nil } -func (f *evmFinalizer) batchCheckReceiptHashesOnchain(ctx context.Context, blockNumToTxesMap map[int64][]*Tx) ([]*Tx, error) { - if len(blockNumToTxesMap) == 0 { +func (f *evmFinalizer) batchCheckReceiptHashesOnchain(ctx context.Context, blockNumToReceiptsMap map[int64][]Receipt) ([]Receipt, error) { + if len(blockNumToReceiptsMap) == 0 { return nil, nil } // Group the RPC batch calls in groups of rpcBatchSize var rpcBatchGroups [][]rpc.BatchElem var rpcBatch []rpc.BatchElem - for blockNum := range blockNumToTxesMap { + for blockNum := range blockNumToReceiptsMap { elem := rpc.BatchElem{ Method: "eth_getBlockByNumber", Args: []any{ @@ -232,7 +246,7 @@ func (f *evmFinalizer) batchCheckReceiptHashesOnchain(ctx context.Context, block rpcBatchGroups = append(rpcBatchGroups, rpcBatch) } - var finalizedTxs []*Tx + var finalizedReceipts []Receipt for _, rpcBatch := range rpcBatchGroups { err := f.client.BatchCallContext(ctx, rpcBatch) if err != nil { @@ -243,47 +257,43 @@ func (f *evmFinalizer) batchCheckReceiptHashesOnchain(ctx context.Context, block for _, req := range rpcBatch { if req.Error != nil { // Continue if particular RPC call failed so other txs can still be considered for finalization - f.lggr.Debugw("failed to find block by number", "blockNum", req.Args[0]) + f.lggr.Errorw("failed to find block by number", "blockNum", req.Args[0], "error", req.Error) continue } - head := req.Result.(*evmtypes.Head) - if head == nil { + head, ok := req.Result.(*evmtypes.Head) + if !ok || !head.IsValid() { // Continue if particular RPC call yielded a nil block so other txs can still be considered for finalization - f.lggr.Debugw("failed to find block by number", "blockNum", req.Args[0]) + f.lggr.Errorw("retrieved nil head for block number", "blockNum", req.Args[0]) continue } - txs := blockNumToTxesMap[head.BlockNumber()] + receipts := blockNumToReceiptsMap[head.BlockNumber()] // Check if transaction receipts match the block hash at the given block num // If they do not, the transactions may have been re-org'd out // The expectation is for the Confirmer to pick up on these re-orgs and get the transaction included - for _, tx := range txs { - receipt := tx.GetReceipt() - if receipt.GetBlockHash().String() == head.BlockHash().String() { - finalizedTxs = append(finalizedTxs, tx) + for _, receipt := range receipts { + if receipt.BlockHash.String() == head.BlockHash().String() { + finalizedReceipts = append(finalizedReceipts, receipt) } else { // Log error if a transaction is marked as confirmed with a receipt older than the finalized block // This scenario could potentially point to a re-org'd transaction the Confirmer has lost track of - f.lggr.Errorw("found confirmed transaction with re-org'd receipt older than finalized block", "tx", tx, "receipt", receipt, "onchainBlockHash", head.BlockHash().String()) + f.lggr.Errorw("found confirmed transaction with re-org'd receipt older than finalized block", "receipt", receipt, "onchainBlockHash", head.BlockHash().String()) } } } } - return finalizedTxs, nil + return finalizedReceipts, nil } // Build list of transaction IDs -func (f *evmFinalizer) buildTxIdList(finalizedTxs []*Tx) []int64 { - etxIDs := make([]int64, len(finalizedTxs)) - for i, tx := range finalizedTxs { - receipt := tx.GetReceipt() +func (f *evmFinalizer) buildReceiptIdList(finalizedReceipts []Receipt) []int64 { + receiptIds := make([]int64, len(finalizedReceipts)) + for i, receipt := range finalizedReceipts { f.lggr.Debugw("transaction considered finalized", - "sequence", tx.Sequence, - "fromAddress", tx.FromAddress.String(), - "txHash", receipt.GetTxHash().String(), - "receiptBlockNum", receipt.GetBlockNumber().Int64(), - "receiptBlockHash", receipt.GetBlockHash().String(), + "txHash", receipt.TxHash.String(), + "receiptBlockNum", receipt.BlockNumber, + "receiptBlockHash", receipt.BlockHash.String(), ) - etxIDs[i] = tx.ID + receiptIds[i] = receipt.ID } - return etxIDs + return receiptIds } diff --git a/core/chains/evm/txmgr/finalizer_test.go b/core/chains/evm/txmgr/finalizer_test.go index a6f0d13f99f..f83a53bf499 100644 --- a/core/chains/evm/txmgr/finalizer_test.go +++ b/core/chains/evm/txmgr/finalizer_test.go @@ -14,6 +14,7 @@ import ( "github.com/stretchr/testify/require" "github.com/smartcontractkit/chainlink-common/pkg/logger" + "github.com/smartcontractkit/chainlink-common/pkg/services/servicetest" "github.com/smartcontractkit/chainlink-common/pkg/utils/tests" txmgrcommon "github.com/smartcontractkit/chainlink/v2/common/txmgr" @@ -38,10 +39,6 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { rpcBatchSize := uint32(1) ht := headtracker.NewSimulatedHeadTracker(ethClient, true, 0) - finalizer := txmgr.NewEvmFinalizer(logger.Test(t), testutils.FixtureChainID, rpcBatchSize, txStore, ethClient, ht) - err := finalizer.Start(ctx) - require.NoError(t, err) - head := &evmtypes.Head{ Hash: utils.NewHash(), Number: 100, @@ -53,6 +50,9 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { } t.Run("returns not finalized for tx with receipt newer than finalized block", func(t *testing.T) { + finalizer := txmgr.NewEvmFinalizer(logger.Test(t), testutils.FixtureChainID, rpcBatchSize, txStore, ethClient, ht) + servicetest.Run(t, finalizer) + idempotencyKey := uuid.New().String() _, fromAddress := cltest.MustInsertRandomKey(t, ethKeyStore) nonce := evmtypes.Nonce(0) @@ -72,7 +72,7 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { mustInsertEthReceipt(t, txStore, head.Number, head.Hash, attemptHash) ethClient.On("HeadByNumber", mock.Anything, mock.Anything).Return(head, nil).Once() ethClient.On("LatestFinalizedBlock", mock.Anything).Return(head.Parent, nil).Once() - err = finalizer.ProcessHead(ctx, head) + err := finalizer.ProcessHead(ctx, head) require.NoError(t, err) tx, err = txStore.FindTxWithIdempotencyKey(ctx, idempotencyKey, testutils.FixtureChainID) require.NoError(t, err) @@ -80,6 +80,9 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { }) t.Run("returns not finalized for tx with receipt re-org'd out", func(t *testing.T) { + finalizer := txmgr.NewEvmFinalizer(logger.Test(t), testutils.FixtureChainID, rpcBatchSize, txStore, ethClient, ht) + servicetest.Run(t, finalizer) + idempotencyKey := uuid.New().String() _, fromAddress := cltest.MustInsertRandomKey(t, ethKeyStore) nonce := evmtypes.Nonce(0) @@ -99,7 +102,7 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { mustInsertEthReceipt(t, txStore, head.Parent.Number, utils.NewHash(), attemptHash) ethClient.On("HeadByNumber", mock.Anything, mock.Anything).Return(head, nil).Once() ethClient.On("LatestFinalizedBlock", mock.Anything).Return(head.Parent, nil).Once() - err = finalizer.ProcessHead(ctx, head) + err := finalizer.ProcessHead(ctx, head) require.NoError(t, err) tx, err = txStore.FindTxWithIdempotencyKey(ctx, idempotencyKey, testutils.FixtureChainID) require.NoError(t, err) @@ -107,6 +110,9 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { }) t.Run("returns finalized for tx with receipt in a finalized block", func(t *testing.T) { + finalizer := txmgr.NewEvmFinalizer(logger.Test(t), testutils.FixtureChainID, rpcBatchSize, txStore, ethClient, ht) + servicetest.Run(t, finalizer) + idempotencyKey := uuid.New().String() _, fromAddress := cltest.MustInsertRandomKey(t, ethKeyStore) nonce := evmtypes.Nonce(0) @@ -126,7 +132,7 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { mustInsertEthReceipt(t, txStore, head.Parent.Number, head.Parent.Hash, attemptHash) ethClient.On("HeadByNumber", mock.Anything, mock.Anything).Return(head, nil).Once() ethClient.On("LatestFinalizedBlock", mock.Anything).Return(head.Parent, nil).Once() - err = finalizer.ProcessHead(ctx, head) + err := finalizer.ProcessHead(ctx, head) require.NoError(t, err) tx, err = txStore.FindTxWithIdempotencyKey(ctx, idempotencyKey, testutils.FixtureChainID) require.NoError(t, err) @@ -134,6 +140,9 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { }) t.Run("returns finalized for tx with receipt older than block history depth", func(t *testing.T) { + finalizer := txmgr.NewEvmFinalizer(logger.Test(t), testutils.FixtureChainID, rpcBatchSize, txStore, ethClient, ht) + servicetest.Run(t, finalizer) + idempotencyKey := uuid.New().String() _, fromAddress := cltest.MustInsertRandomKey(t, ethKeyStore) nonce := evmtypes.Nonce(0) @@ -150,8 +159,8 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { } attemptHash := insertTxAndAttemptWithIdempotencyKey(t, txStore, tx, idempotencyKey) // Insert receipt for finalized block num - receiptHash1 := utils.NewHash() - mustInsertEthReceipt(t, txStore, head.Parent.Number-2, receiptHash1, attemptHash) + receiptBlockHash1 := utils.NewHash() + mustInsertEthReceipt(t, txStore, head.Parent.Number-2, receiptBlockHash1, attemptHash) idempotencyKey = uuid.New().String() nonce = evmtypes.Nonce(1) tx = &txmgr.Tx{ @@ -166,8 +175,8 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { } attemptHash = insertTxAndAttemptWithIdempotencyKey(t, txStore, tx, idempotencyKey) // Insert receipt for finalized block num - receiptHash2 := utils.NewHash() - mustInsertEthReceipt(t, txStore, head.Parent.Number-1, receiptHash2, attemptHash) + receiptBlockHash2 := utils.NewHash() + mustInsertEthReceipt(t, txStore, head.Parent.Number-1, receiptBlockHash2, attemptHash) // Separate batch calls will be made for each tx due to RPC batch size set to 1 when finalizer initialized above ethClient.On("BatchCallContext", mock.Anything, mock.IsType([]rpc.BatchElem{})).Run(func(args mock.Arguments) { rpcElements := args.Get(1).([]rpc.BatchElem) @@ -181,9 +190,9 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { req2BlockNum := hexutil.EncodeBig(big.NewInt(head.Parent.Number - 1)) var headResult evmtypes.Head if req1BlockNum == reqBlockNum { - headResult = evmtypes.Head{Number: head.Parent.Number - 2, Hash: receiptHash1} + headResult = evmtypes.Head{Number: head.Parent.Number - 2, Hash: receiptBlockHash1} } else if req2BlockNum == reqBlockNum { - headResult = evmtypes.Head{Number: head.Parent.Number - 1, Hash: receiptHash2} + headResult = evmtypes.Head{Number: head.Parent.Number - 1, Hash: receiptBlockHash2} } else { require.Fail(t, "unrecognized block hash") } @@ -191,7 +200,7 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { }).Return(nil).Twice() ethClient.On("HeadByNumber", mock.Anything, mock.Anything).Return(head, nil).Once() ethClient.On("LatestFinalizedBlock", mock.Anything).Return(head.Parent, nil).Once() - err = finalizer.ProcessHead(ctx, head) + err := finalizer.ProcessHead(ctx, head) require.NoError(t, err) tx, err = txStore.FindTxWithIdempotencyKey(ctx, idempotencyKey, testutils.FixtureChainID) require.NoError(t, err) @@ -199,15 +208,21 @@ func TestFinalizer_MarkTxFinalized(t *testing.T) { }) t.Run("returns error if failed to retrieve latest head in headtracker", func(t *testing.T) { + finalizer := txmgr.NewEvmFinalizer(logger.Test(t), testutils.FixtureChainID, rpcBatchSize, txStore, ethClient, ht) + servicetest.Run(t, finalizer) + ethClient.On("HeadByNumber", mock.Anything, mock.Anything).Return(nil, errors.New("failed to get latest head")).Once() - err = finalizer.ProcessHead(ctx, head) + err := finalizer.ProcessHead(ctx, head) require.Error(t, err) }) t.Run("returns error if failed to calculate latest finalized head in headtracker", func(t *testing.T) { + finalizer := txmgr.NewEvmFinalizer(logger.Test(t), testutils.FixtureChainID, rpcBatchSize, txStore, ethClient, ht) + servicetest.Run(t, finalizer) + ethClient.On("HeadByNumber", mock.Anything, mock.Anything).Return(head, nil).Once() ethClient.On("LatestFinalizedBlock", mock.Anything).Return(nil, errors.New("failed to calculate latest finalized head")).Once() - err = finalizer.ProcessHead(ctx, head) + err := finalizer.ProcessHead(ctx, head) require.Error(t, err) }) } From 523fc80eb4093f61b5a56b46b2144192160e2242 Mon Sep 17 00:00:00 2001 From: amit-momin Date: Mon, 29 Jul 2024 11:34:28 -0500 Subject: [PATCH 19/23] Fixed mocks --- common/txmgr/types/mocks/tx_store.go | 101 +--------- core/chains/evm/txmgr/mocks/evm_tx_store.go | 192 +++++++++++--------- 2 files changed, 120 insertions(+), 173 deletions(-) diff --git a/common/txmgr/types/mocks/tx_store.go b/common/txmgr/types/mocks/tx_store.go index 542fd0bbe57..0b9c7110660 100644 --- a/common/txmgr/types/mocks/tx_store.go +++ b/common/txmgr/types/mocks/tx_store.go @@ -1156,36 +1156,6 @@ func (_c *TxStore_FindTxesPendingCallback_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HA return _c } -// FindTxesToMarkFinalized provides a mock function with given fields: ctx, finalizedBlockNum, chainID -func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxesToMarkFinalized(ctx context.Context, finalizedBlockNum int64, chainID CHAIN_ID) ([]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error) { - ret := _m.Called(ctx, finalizedBlockNum, chainID) - - if len(ret) == 0 { - panic("no return value specified for FindTxesToMarkFinalized") - } - - var r0 []*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] - var r1 error - if rf, ok := ret.Get(0).(func(context.Context, int64, CHAIN_ID) ([]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error)); ok { - return rf(ctx, finalizedBlockNum, chainID) - } - if rf, ok := ret.Get(0).(func(context.Context, int64, CHAIN_ID) []*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]); ok { - r0 = rf(ctx, finalizedBlockNum, chainID) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).([]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) - } - } - - if rf, ok := ret.Get(1).(func(context.Context, int64, CHAIN_ID) error); ok { - r1 = rf(ctx, finalizedBlockNum, chainID) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - // FindTxesWithAttemptsAndReceiptsByIdsAndState provides a mock function with given fields: ctx, ids, states, chainID func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxesWithAttemptsAndReceiptsByIdsAndState(ctx context.Context, ids []int64, states []txmgrtypes.TxState, chainID *big.Int) ([]*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error) { ret := _m.Called(ctx, ids, states, chainID) @@ -1790,37 +1760,6 @@ func (_c *TxStore_HasInProgressTransaction_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_H return _c } -// TxStore_IsTxFinalized_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'IsTxFinalized' -type TxStore_IsTxFinalized_Call[ADDR types.Hashable, CHAIN_ID types.ID, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, R txmgrtypes.ChainReceipt[TX_HASH, BLOCK_HASH], SEQ types.Sequence, FEE feetypes.Fee] struct { - *mock.Call -} - -// IsTxFinalized is a helper method to define mock.On call -// - ctx context.Context -// - blockHeight int64 -// - txID int64 -// - chainID CHAIN_ID -func (_e *TxStore_Expecter[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) IsTxFinalized(ctx interface{}, blockHeight interface{}, txID interface{}, chainID interface{}) *TxStore_IsTxFinalized_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { - return &TxStore_IsTxFinalized_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]{Call: _e.mock.On("IsTxFinalized", ctx, blockHeight, txID, chainID)} -} - -func (_c *TxStore_IsTxFinalized_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) Run(run func(ctx context.Context, blockHeight int64, txID int64, chainID CHAIN_ID)) *TxStore_IsTxFinalized_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { - _c.Call.Run(func(args mock.Arguments) { - run(args[0].(context.Context), args[1].(int64), args[2].(int64), args[3].(CHAIN_ID)) - }) - return _c -} - -func (_c *TxStore_IsTxFinalized_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) Return(finalized bool, err error) *TxStore_IsTxFinalized_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { - _c.Call.Return(finalized, err) - return _c -} - -func (_c *TxStore_IsTxFinalized_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) RunAndReturn(run func(context.Context, int64, int64, CHAIN_ID) (bool, error)) *TxStore_IsTxFinalized_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { - _c.Call.Return(run) - return _c -} - // LoadTxAttempts provides a mock function with given fields: ctx, etx func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) LoadTxAttempts(ctx context.Context, etx *txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) error { ret := _m.Called(ctx, etx) @@ -2041,6 +1980,11 @@ func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) PruneUnstar return r0, r1 } +// TxStore_PruneUnstartedTxQueue_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'PruneUnstartedTxQueue' +type TxStore_PruneUnstartedTxQueue_Call[ADDR types.Hashable, CHAIN_ID types.ID, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, R txmgrtypes.ChainReceipt[TX_HASH, BLOCK_HASH], SEQ types.Sequence, FEE feetypes.Fee] struct { + *mock.Call +} + // PruneUnstartedTxQueue is a helper method to define mock.On call // - ctx context.Context // - queueSize uint32 @@ -2066,12 +2010,6 @@ func (_c *TxStore_PruneUnstartedTxQueue_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH return _c } -// TxStore_PruneUnstartedTxQueue_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'PruneUnstartedTxQueue' -type TxStore_PruneUnstartedTxQueue_Call[ADDR types.Hashable, CHAIN_ID types.ID, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, R txmgrtypes.ChainReceipt[TX_HASH, BLOCK_HASH], SEQ types.Sequence, FEE feetypes.Fee] struct { - *mock.Call -} - - // ReapTxHistory provides a mock function with given fields: ctx, timeThreshold, chainID func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) ReapTxHistory(ctx context.Context, timeThreshold time.Time, chainID CHAIN_ID) error { ret := _m.Called(ctx, timeThreshold, chainID) @@ -2097,16 +2035,15 @@ type TxStore_ReapTxHistory_Call[ADDR types.Hashable, CHAIN_ID types.ID, TX_HASH // ReapTxHistory is a helper method to define mock.On call // - ctx context.Context -// - minBlockNumberToKeep int64 // - timeThreshold time.Time // - chainID CHAIN_ID -func (_e *TxStore_Expecter[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) ReapTxHistory(ctx interface{}, minBlockNumberToKeep interface{}, timeThreshold interface{}, chainID interface{}) *TxStore_ReapTxHistory_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { - return &TxStore_ReapTxHistory_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]{Call: _e.mock.On("ReapTxHistory", ctx, minBlockNumberToKeep, timeThreshold, chainID)} +func (_e *TxStore_Expecter[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) ReapTxHistory(ctx interface{}, timeThreshold interface{}, chainID interface{}) *TxStore_ReapTxHistory_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { + return &TxStore_ReapTxHistory_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]{Call: _e.mock.On("ReapTxHistory", ctx, timeThreshold, chainID)} } -func (_c *TxStore_ReapTxHistory_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) Run(run func(ctx context.Context, minBlockNumberToKeep int64, timeThreshold time.Time, chainID CHAIN_ID)) *TxStore_ReapTxHistory_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { +func (_c *TxStore_ReapTxHistory_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) Run(run func(ctx context.Context, timeThreshold time.Time, chainID CHAIN_ID)) *TxStore_ReapTxHistory_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { _c.Call.Run(func(args mock.Arguments) { - run(args[0].(context.Context), args[1].(int64), args[2].(time.Time), args[3].(CHAIN_ID)) + run(args[0].(context.Context), args[1].(time.Time), args[2].(CHAIN_ID)) }) return _c } @@ -2116,7 +2053,7 @@ func (_c *TxStore_ReapTxHistory_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ return _c } -func (_c *TxStore_ReapTxHistory_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) RunAndReturn(run func(context.Context, int64, time.Time, CHAIN_ID) error) *TxStore_ReapTxHistory_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { +func (_c *TxStore_ReapTxHistory_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) RunAndReturn(run func(context.Context, time.Time, CHAIN_ID) error) *TxStore_ReapTxHistory_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { _c.Call.Return(run) return _c } @@ -2749,24 +2686,6 @@ func (_c *TxStore_UpdateTxUnstartedToInProgress_Call[ADDR, CHAIN_ID, TX_HASH, BL return _c } -// UpdateTxesFinalized provides a mock function with given fields: ctx, etxs, chainId -func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) UpdateTxesFinalized(ctx context.Context, etxs []int64, chainId CHAIN_ID) error { - ret := _m.Called(ctx, etxs, chainId) - - if len(ret) == 0 { - panic("no return value specified for UpdateTxesFinalized") - } - - var r0 error - if rf, ok := ret.Get(0).(func(context.Context, []int64, CHAIN_ID) error); ok { - r0 = rf(ctx, etxs, chainId) - } else { - r0 = ret.Error(0) - } - - return r0 -} - // UpdateTxsUnconfirmed provides a mock function with given fields: ctx, ids func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) UpdateTxsUnconfirmed(ctx context.Context, ids []int64) error { ret := _m.Called(ctx, ids) diff --git a/core/chains/evm/txmgr/mocks/evm_tx_store.go b/core/chains/evm/txmgr/mocks/evm_tx_store.go index 368010f4ebb..f7112e1edd2 100644 --- a/core/chains/evm/txmgr/mocks/evm_tx_store.go +++ b/core/chains/evm/txmgr/mocks/evm_tx_store.go @@ -444,6 +444,66 @@ func (_c *EvmTxStore_DeleteInProgressAttempt_Call) RunAndReturn(run func(context return _c } +// FindConfirmedTxesReceipts provides a mock function with given fields: ctx, finalizedBlockNum, chainID +func (_m *EvmTxStore) FindConfirmedTxesReceipts(ctx context.Context, finalizedBlockNum int64, chainID *big.Int) ([]txmgr.dbReceipt, error) { + ret := _m.Called(ctx, finalizedBlockNum, chainID) + + if len(ret) == 0 { + panic("no return value specified for FindConfirmedTxesReceipts") + } + + var r0 []txmgr.dbReceipt + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, int64, *big.Int) ([]txmgr.dbReceipt, error)); ok { + return rf(ctx, finalizedBlockNum, chainID) + } + if rf, ok := ret.Get(0).(func(context.Context, int64, *big.Int) []txmgr.dbReceipt); ok { + r0 = rf(ctx, finalizedBlockNum, chainID) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]txmgr.dbReceipt) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, int64, *big.Int) error); ok { + r1 = rf(ctx, finalizedBlockNum, chainID) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// EvmTxStore_FindConfirmedTxesReceipts_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'FindConfirmedTxesReceipts' +type EvmTxStore_FindConfirmedTxesReceipts_Call struct { + *mock.Call +} + +// FindConfirmedTxesReceipts is a helper method to define mock.On call +// - ctx context.Context +// - finalizedBlockNum int64 +// - chainID *big.Int +func (_e *EvmTxStore_Expecter) FindConfirmedTxesReceipts(ctx interface{}, finalizedBlockNum interface{}, chainID interface{}) *EvmTxStore_FindConfirmedTxesReceipts_Call { + return &EvmTxStore_FindConfirmedTxesReceipts_Call{Call: _e.mock.On("FindConfirmedTxesReceipts", ctx, finalizedBlockNum, chainID)} +} + +func (_c *EvmTxStore_FindConfirmedTxesReceipts_Call) Run(run func(ctx context.Context, finalizedBlockNum int64, chainID *big.Int)) *EvmTxStore_FindConfirmedTxesReceipts_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(int64), args[2].(*big.Int)) + }) + return _c +} + +func (_c *EvmTxStore_FindConfirmedTxesReceipts_Call) Return(receipts []txmgr.dbReceipt, err error) *EvmTxStore_FindConfirmedTxesReceipts_Call { + _c.Call.Return(receipts, err) + return _c +} + +func (_c *EvmTxStore_FindConfirmedTxesReceipts_Call) RunAndReturn(run func(context.Context, int64, *big.Int) ([]txmgr.dbReceipt, error)) *EvmTxStore_FindConfirmedTxesReceipts_Call { + _c.Call.Return(run) + return _c +} + // FindEarliestUnconfirmedBroadcastTime provides a mock function with given fields: ctx, chainID func (_m *EvmTxStore) FindEarliestUnconfirmedBroadcastTime(ctx context.Context, chainID *big.Int) (null.Time, error) { ret := _m.Called(ctx, chainID) @@ -1363,36 +1423,6 @@ func (_m *EvmTxStore) FindTxesPendingCallback(ctx context.Context, blockNum int6 return r0, r1 } -// FindTxesToMarkFinalized provides a mock function with given fields: ctx, finalizedBlockNum, chainID -func (_m *EvmTxStore) FindTxesToMarkFinalized(ctx context.Context, finalizedBlockNum int64, chainID *big.Int) ([]*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error) { - ret := _m.Called(ctx, finalizedBlockNum, chainID) - - if len(ret) == 0 { - panic("no return value specified for FindTxesToMarkFinalized") - } - - var r0 []*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee] - var r1 error - if rf, ok := ret.Get(0).(func(context.Context, int64, *big.Int) ([]*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error)); ok { - return rf(ctx, finalizedBlockNum, chainID) - } - if rf, ok := ret.Get(0).(func(context.Context, int64, *big.Int) []*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]); ok { - r0 = rf(ctx, finalizedBlockNum, chainID) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).([]*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]) - } - } - - if rf, ok := ret.Get(1).(func(context.Context, int64, *big.Int) error); ok { - r1 = rf(ctx, finalizedBlockNum, chainID) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - // EvmTxStore_FindTxesPendingCallback_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'FindTxesPendingCallback' type EvmTxStore_FindTxesPendingCallback_Call struct { *mock.Call @@ -2088,37 +2118,6 @@ func (_c *EvmTxStore_HasInProgressTransaction_Call) RunAndReturn(run func(contex return _c } -// EvmTxStore_IsTxFinalized_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'IsTxFinalized' -type EvmTxStore_IsTxFinalized_Call struct { - *mock.Call -} - -// IsTxFinalized is a helper method to define mock.On call -// - ctx context.Context -// - blockHeight int64 -// - txID int64 -// - chainID *big.Int -func (_e *EvmTxStore_Expecter) IsTxFinalized(ctx interface{}, blockHeight interface{}, txID interface{}, chainID interface{}) *EvmTxStore_IsTxFinalized_Call { - return &EvmTxStore_IsTxFinalized_Call{Call: _e.mock.On("IsTxFinalized", ctx, blockHeight, txID, chainID)} -} - -func (_c *EvmTxStore_IsTxFinalized_Call) Run(run func(ctx context.Context, blockHeight int64, txID int64, chainID *big.Int)) *EvmTxStore_IsTxFinalized_Call { - _c.Call.Run(func(args mock.Arguments) { - run(args[0].(context.Context), args[1].(int64), args[2].(int64), args[3].(*big.Int)) - }) - return _c -} - -func (_c *EvmTxStore_IsTxFinalized_Call) Return(finalized bool, err error) *EvmTxStore_IsTxFinalized_Call { - _c.Call.Return(finalized, err) - return _c -} - -func (_c *EvmTxStore_IsTxFinalized_Call) RunAndReturn(run func(context.Context, int64, int64, *big.Int) (bool, error)) *EvmTxStore_IsTxFinalized_Call { - _c.Call.Return(run) - return _c -} - // LoadTxAttempts provides a mock function with given fields: ctx, etx func (_m *EvmTxStore) LoadTxAttempts(ctx context.Context, etx *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]) error { ret := _m.Called(ctx, etx) @@ -2394,16 +2393,15 @@ type EvmTxStore_ReapTxHistory_Call struct { // ReapTxHistory is a helper method to define mock.On call // - ctx context.Context -// - minBlockNumberToKeep int64 // - timeThreshold time.Time // - chainID *big.Int -func (_e *EvmTxStore_Expecter) ReapTxHistory(ctx interface{}, minBlockNumberToKeep interface{}, timeThreshold interface{}, chainID interface{}) *EvmTxStore_ReapTxHistory_Call { - return &EvmTxStore_ReapTxHistory_Call{Call: _e.mock.On("ReapTxHistory", ctx, minBlockNumberToKeep, timeThreshold, chainID)} +func (_e *EvmTxStore_Expecter) ReapTxHistory(ctx interface{}, timeThreshold interface{}, chainID interface{}) *EvmTxStore_ReapTxHistory_Call { + return &EvmTxStore_ReapTxHistory_Call{Call: _e.mock.On("ReapTxHistory", ctx, timeThreshold, chainID)} } -func (_c *EvmTxStore_ReapTxHistory_Call) Run(run func(ctx context.Context, minBlockNumberToKeep int64, timeThreshold time.Time, chainID *big.Int)) *EvmTxStore_ReapTxHistory_Call { +func (_c *EvmTxStore_ReapTxHistory_Call) Run(run func(ctx context.Context, timeThreshold time.Time, chainID *big.Int)) *EvmTxStore_ReapTxHistory_Call { _c.Call.Run(func(args mock.Arguments) { - run(args[0].(context.Context), args[1].(int64), args[2].(time.Time), args[3].(*big.Int)) + run(args[0].(context.Context), args[1].(time.Time), args[2].(*big.Int)) }) return _c } @@ -2413,7 +2411,7 @@ func (_c *EvmTxStore_ReapTxHistory_Call) Return(_a0 error) *EvmTxStore_ReapTxHis return _c } -func (_c *EvmTxStore_ReapTxHistory_Call) RunAndReturn(run func(context.Context, int64, time.Time, *big.Int) error) *EvmTxStore_ReapTxHistory_Call { +func (_c *EvmTxStore_ReapTxHistory_Call) RunAndReturn(run func(context.Context, time.Time, *big.Int) error) *EvmTxStore_ReapTxHistory_Call { _c.Call.Return(run) return _c } @@ -3217,9 +3215,39 @@ func (_m *EvmTxStore) UpdateTxUnstartedToInProgress(ctx context.Context, etx *ty return r0 } -// UpdateTxesFinalized provides a mock function with given fields: ctx, etxs, chainId -func (_m *EvmTxStore) UpdateTxesFinalized(ctx context.Context, etxs []int64, chainId *big.Int) error { - ret := _m.Called(ctx, etxs, chainId) +// EvmTxStore_UpdateTxUnstartedToInProgress_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateTxUnstartedToInProgress' +type EvmTxStore_UpdateTxUnstartedToInProgress_Call struct { + *mock.Call +} + +// UpdateTxUnstartedToInProgress is a helper method to define mock.On call +// - ctx context.Context +// - etx *types.Tx[*big.Int,common.Address,common.Hash,common.Hash,evmtypes.Nonce,gas.EvmFee] +// - attempt *types.TxAttempt[*big.Int,common.Address,common.Hash,common.Hash,evmtypes.Nonce,gas.EvmFee] +func (_e *EvmTxStore_Expecter) UpdateTxUnstartedToInProgress(ctx interface{}, etx interface{}, attempt interface{}) *EvmTxStore_UpdateTxUnstartedToInProgress_Call { + return &EvmTxStore_UpdateTxUnstartedToInProgress_Call{Call: _e.mock.On("UpdateTxUnstartedToInProgress", ctx, etx, attempt)} +} + +func (_c *EvmTxStore_UpdateTxUnstartedToInProgress_Call) Run(run func(ctx context.Context, etx *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], attempt *types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee])) *EvmTxStore_UpdateTxUnstartedToInProgress_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]), args[2].(*types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee])) + }) + return _c +} + +func (_c *EvmTxStore_UpdateTxUnstartedToInProgress_Call) Return(_a0 error) *EvmTxStore_UpdateTxUnstartedToInProgress_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *EvmTxStore_UpdateTxUnstartedToInProgress_Call) RunAndReturn(run func(context.Context, *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], *types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]) error) *EvmTxStore_UpdateTxUnstartedToInProgress_Call { + _c.Call.Return(run) + return _c +} + +// UpdateTxesFinalized provides a mock function with given fields: ctx, etxIDs, chainId +func (_m *EvmTxStore) UpdateTxesFinalized(ctx context.Context, etxIDs []int64, chainId *big.Int) error { + ret := _m.Called(ctx, etxIDs, chainId) if len(ret) == 0 { panic("no return value specified for UpdateTxesFinalized") @@ -3227,7 +3255,7 @@ func (_m *EvmTxStore) UpdateTxesFinalized(ctx context.Context, etxs []int64, cha var r0 error if rf, ok := ret.Get(0).(func(context.Context, []int64, *big.Int) error); ok { - r0 = rf(ctx, etxs, chainId) + r0 = rf(ctx, etxIDs, chainId) } else { r0 = ret.Error(0) } @@ -3235,32 +3263,32 @@ func (_m *EvmTxStore) UpdateTxesFinalized(ctx context.Context, etxs []int64, cha return r0 } -// EvmTxStore_UpdateTxUnstartedToInProgress_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateTxUnstartedToInProgress' -type EvmTxStore_UpdateTxUnstartedToInProgress_Call struct { +// EvmTxStore_UpdateTxesFinalized_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateTxesFinalized' +type EvmTxStore_UpdateTxesFinalized_Call struct { *mock.Call } -// UpdateTxUnstartedToInProgress is a helper method to define mock.On call +// UpdateTxesFinalized is a helper method to define mock.On call // - ctx context.Context -// - etx *types.Tx[*big.Int,common.Address,common.Hash,common.Hash,evmtypes.Nonce,gas.EvmFee] -// - attempt *types.TxAttempt[*big.Int,common.Address,common.Hash,common.Hash,evmtypes.Nonce,gas.EvmFee] -func (_e *EvmTxStore_Expecter) UpdateTxUnstartedToInProgress(ctx interface{}, etx interface{}, attempt interface{}) *EvmTxStore_UpdateTxUnstartedToInProgress_Call { - return &EvmTxStore_UpdateTxUnstartedToInProgress_Call{Call: _e.mock.On("UpdateTxUnstartedToInProgress", ctx, etx, attempt)} +// - etxIDs []int64 +// - chainId *big.Int +func (_e *EvmTxStore_Expecter) UpdateTxesFinalized(ctx interface{}, etxIDs interface{}, chainId interface{}) *EvmTxStore_UpdateTxesFinalized_Call { + return &EvmTxStore_UpdateTxesFinalized_Call{Call: _e.mock.On("UpdateTxesFinalized", ctx, etxIDs, chainId)} } -func (_c *EvmTxStore_UpdateTxUnstartedToInProgress_Call) Run(run func(ctx context.Context, etx *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], attempt *types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee])) *EvmTxStore_UpdateTxUnstartedToInProgress_Call { +func (_c *EvmTxStore_UpdateTxesFinalized_Call) Run(run func(ctx context.Context, etxIDs []int64, chainId *big.Int)) *EvmTxStore_UpdateTxesFinalized_Call { _c.Call.Run(func(args mock.Arguments) { - run(args[0].(context.Context), args[1].(*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]), args[2].(*types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee])) + run(args[0].(context.Context), args[1].([]int64), args[2].(*big.Int)) }) return _c } -func (_c *EvmTxStore_UpdateTxUnstartedToInProgress_Call) Return(_a0 error) *EvmTxStore_UpdateTxUnstartedToInProgress_Call { +func (_c *EvmTxStore_UpdateTxesFinalized_Call) Return(_a0 error) *EvmTxStore_UpdateTxesFinalized_Call { _c.Call.Return(_a0) return _c } -func (_c *EvmTxStore_UpdateTxUnstartedToInProgress_Call) RunAndReturn(run func(context.Context, *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], *types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]) error) *EvmTxStore_UpdateTxUnstartedToInProgress_Call { +func (_c *EvmTxStore_UpdateTxesFinalized_Call) RunAndReturn(run func(context.Context, []int64, *big.Int) error) *EvmTxStore_UpdateTxesFinalized_Call { _c.Call.Return(run) return _c } From 84f8c28cff0ddd345e6698bd50877c13d3a601b8 Mon Sep 17 00:00:00 2001 From: amit-momin Date: Mon, 29 Jul 2024 18:47:45 -0500 Subject: [PATCH 20/23] Updated TxStore method name and fixed mocks --- core/chains/evm/txmgr/evm_tx_store.go | 20 ++--- core/chains/evm/txmgr/evm_tx_store_test.go | 2 +- core/chains/evm/txmgr/finalizer.go | 4 +- core/chains/evm/txmgr/mocks/evm_tx_store.go | 84 +++++++++++---------- core/chains/evm/txmgr/models.go | 2 +- 5 files changed, 57 insertions(+), 55 deletions(-) diff --git a/core/chains/evm/txmgr/evm_tx_store.go b/core/chains/evm/txmgr/evm_tx_store.go index 819eab4a18c..c5f19d790a2 100644 --- a/core/chains/evm/txmgr/evm_tx_store.go +++ b/core/chains/evm/txmgr/evm_tx_store.go @@ -47,7 +47,7 @@ type EvmTxStore interface { // methods used solely in EVM components FindConfirmedTxesReceipts(ctx context.Context, finalizedBlockNum int64, chainID *big.Int) (receipts []Receipt, err error) - UpdateTxesFinalized(ctx context.Context, etxIDs []int64, chainId *big.Int) error + UpdateTxStatesToFinalizedUsingReceiptIds(ctx context.Context, etxIDs []int64, chainId *big.Int) error } // TxStoreWebApi encapsulates the methods that are not used by the txmgr and only used by the various web controllers, readers, or evm specific components @@ -91,7 +91,7 @@ var _ TestEvmTxStore = (*evmTxStore)(nil) // Directly maps to columns of database table "evm.receipts". // Do not modify type unless you // intend to modify the database schema -type dbReceipt struct { +type DbReceipt struct { ID int64 TxHash common.Hash BlockHash common.Hash @@ -101,8 +101,8 @@ type dbReceipt struct { CreatedAt time.Time } -func DbReceiptFromEvmReceipt(evmReceipt *evmtypes.Receipt) dbReceipt { - return dbReceipt{ +func DbReceiptFromEvmReceipt(evmReceipt *evmtypes.Receipt) DbReceipt { + return DbReceipt{ TxHash: evmReceipt.TxHash, BlockHash: evmReceipt.BlockHash, BlockNumber: evmReceipt.BlockNumber.Int64(), @@ -111,7 +111,7 @@ func DbReceiptFromEvmReceipt(evmReceipt *evmtypes.Receipt) dbReceipt { } } -func DbReceiptToEvmReceipt(receipt *dbReceipt) *evmtypes.Receipt { +func DbReceiptToEvmReceipt(receipt *DbReceipt) *evmtypes.Receipt { return &receipt.Receipt } @@ -135,7 +135,7 @@ type dbReceiptPlus struct { FailOnRevert bool `db:"FailOnRevert"` } -func fromDBReceipts(rs []dbReceipt) []*evmtypes.Receipt { +func fromDBReceipts(rs []DbReceipt) []*evmtypes.Receipt { receipts := make([]*evmtypes.Receipt, len(rs)) for i := 0; i < len(rs); i++ { receipts[i] = DbReceiptToEvmReceipt(&rs[i]) @@ -681,7 +681,7 @@ func (o *evmTxStore) loadEthTxesAttemptsReceipts(ctx context.Context, etxs []*Tx attemptHashes = append(attemptHashes, attempt.Hash.Bytes()) } } - var rs []dbReceipt + var rs []DbReceipt if err = o.q.SelectContext(ctx, &rs, `SELECT * FROM evm.receipts WHERE tx_hash = ANY($1)`, pq.Array(attemptHashes)); err != nil { return pkgerrors.Wrap(err, "loadEthTxesAttemptsReceipts failed to load evm.receipts") } @@ -704,7 +704,7 @@ func loadConfirmedAttemptsReceipts(ctx context.Context, q sqlutil.DataSource, at byHash[attempt.Hash.String()] = &attempts[i] hashes = append(hashes, attempt.Hash.Bytes()) } - var rs []dbReceipt + var rs []DbReceipt if err := q.SelectContext(ctx, &rs, `SELECT * FROM evm.receipts WHERE tx_hash = ANY($1)`, pq.Array(hashes)); err != nil { return pkgerrors.Wrap(err, "loadConfirmedAttemptsReceipts failed to load evm.receipts") } @@ -2083,7 +2083,7 @@ func (o *evmTxStore) FindConfirmedTxesReceipts(ctx context.Context, finalizedBlo INNER JOIN evm.tx_attempts ON evm.tx_attempts.hash = evm.receipts.tx_hash INNER JOIN evm.txes ON evm.txes.id = evm.tx_attempts.eth_tx_id WHERE evm.txes.state = 'confirmed' AND evm.receipts.block_number <= $1 AND evm.txes.evm_chain_id = $2` - var dbReceipts []dbReceipt + var dbReceipts []DbReceipt err = o.q.SelectContext(ctx, &dbReceipts, sql, finalizedBlockNum, chainID.String()) if len(dbReceipts) == 0 { return nil @@ -2095,7 +2095,7 @@ func (o *evmTxStore) FindConfirmedTxesReceipts(ctx context.Context, finalizedBlo } // Mark transactions corresponding to receipt IDs as finalized -func (o *evmTxStore) UpdateTxesFinalized(ctx context.Context, receiptIDs []int64, chainId *big.Int) error { +func (o *evmTxStore) UpdateTxStatesToFinalizedUsingReceiptIds(ctx context.Context, receiptIDs []int64, chainId *big.Int) error { var cancel context.CancelFunc ctx, cancel = o.stopCh.Ctx(ctx) defer cancel() diff --git a/core/chains/evm/txmgr/evm_tx_store_test.go b/core/chains/evm/txmgr/evm_tx_store_test.go index e98b2c9b97e..191a0a5fed2 100644 --- a/core/chains/evm/txmgr/evm_tx_store_test.go +++ b/core/chains/evm/txmgr/evm_tx_store_test.go @@ -1897,7 +1897,7 @@ func TestORM_UpdateTxesFinalized(t *testing.T) { err = txStore.InsertTxAttempt(ctx, &attempt) require.NoError(t, err) receipt := mustInsertEthReceipt(t, txStore, 100, testutils.NewHash(), attempt.Hash) - err = txStore.UpdateTxesFinalized(ctx, []int64{receipt.ID}, testutils.FixtureChainID) + err = txStore.UpdateTxStatesToFinalizedUsingReceiptIds(ctx, []int64{receipt.ID}, testutils.FixtureChainID) require.NoError(t, err) etx, err := txStore.FindTxWithAttempts(ctx, tx.ID) require.NoError(t, err) diff --git a/core/chains/evm/txmgr/finalizer.go b/core/chains/evm/txmgr/finalizer.go index 8c6c3e8e2b5..e88929336eb 100644 --- a/core/chains/evm/txmgr/finalizer.go +++ b/core/chains/evm/txmgr/finalizer.go @@ -24,7 +24,7 @@ const processHeadTimeout = 10 * time.Minute type finalizerTxStore interface { FindConfirmedTxesReceipts(ctx context.Context, finalizedBlockNum int64, chainID *big.Int) ([]Receipt, error) - UpdateTxesFinalized(ctx context.Context, txs []int64, chainId *big.Int) error + UpdateTxStatesToFinalizedUsingReceiptIds(ctx context.Context, txs []int64, chainId *big.Int) error } type finalizerChainClient interface { @@ -210,7 +210,7 @@ func (f *evmFinalizer) processFinalizedHead(ctx context.Context, latestFinalized receiptIDs := f.buildReceiptIdList(finalizedReceipts) - err = f.txStore.UpdateTxesFinalized(ctx, receiptIDs, f.chainId) + err = f.txStore.UpdateTxStatesToFinalizedUsingReceiptIds(ctx, receiptIDs, f.chainId) if err != nil { return fmt.Errorf("failed to update transactions as finalized: %w", err) } diff --git a/core/chains/evm/txmgr/mocks/evm_tx_store.go b/core/chains/evm/txmgr/mocks/evm_tx_store.go index f7112e1edd2..5cffa321f9c 100644 --- a/core/chains/evm/txmgr/mocks/evm_tx_store.go +++ b/core/chains/evm/txmgr/mocks/evm_tx_store.go @@ -18,6 +18,8 @@ import ( time "time" + txmgr "github.com/smartcontractkit/chainlink/v2/core/chains/evm/txmgr" + types "github.com/smartcontractkit/chainlink/v2/common/txmgr/types" uuid "github.com/google/uuid" @@ -445,23 +447,23 @@ func (_c *EvmTxStore_DeleteInProgressAttempt_Call) RunAndReturn(run func(context } // FindConfirmedTxesReceipts provides a mock function with given fields: ctx, finalizedBlockNum, chainID -func (_m *EvmTxStore) FindConfirmedTxesReceipts(ctx context.Context, finalizedBlockNum int64, chainID *big.Int) ([]txmgr.dbReceipt, error) { +func (_m *EvmTxStore) FindConfirmedTxesReceipts(ctx context.Context, finalizedBlockNum int64, chainID *big.Int) ([]txmgr.DbReceipt, error) { ret := _m.Called(ctx, finalizedBlockNum, chainID) if len(ret) == 0 { panic("no return value specified for FindConfirmedTxesReceipts") } - var r0 []txmgr.dbReceipt + var r0 []txmgr.DbReceipt var r1 error - if rf, ok := ret.Get(0).(func(context.Context, int64, *big.Int) ([]txmgr.dbReceipt, error)); ok { + if rf, ok := ret.Get(0).(func(context.Context, int64, *big.Int) ([]txmgr.DbReceipt, error)); ok { return rf(ctx, finalizedBlockNum, chainID) } - if rf, ok := ret.Get(0).(func(context.Context, int64, *big.Int) []txmgr.dbReceipt); ok { + if rf, ok := ret.Get(0).(func(context.Context, int64, *big.Int) []txmgr.DbReceipt); ok { r0 = rf(ctx, finalizedBlockNum, chainID) } else { if ret.Get(0) != nil { - r0 = ret.Get(0).([]txmgr.dbReceipt) + r0 = ret.Get(0).([]txmgr.DbReceipt) } } @@ -494,12 +496,12 @@ func (_c *EvmTxStore_FindConfirmedTxesReceipts_Call) Run(run func(ctx context.Co return _c } -func (_c *EvmTxStore_FindConfirmedTxesReceipts_Call) Return(receipts []txmgr.dbReceipt, err error) *EvmTxStore_FindConfirmedTxesReceipts_Call { +func (_c *EvmTxStore_FindConfirmedTxesReceipts_Call) Return(receipts []txmgr.DbReceipt, err error) *EvmTxStore_FindConfirmedTxesReceipts_Call { _c.Call.Return(receipts, err) return _c } -func (_c *EvmTxStore_FindConfirmedTxesReceipts_Call) RunAndReturn(run func(context.Context, int64, *big.Int) ([]txmgr.dbReceipt, error)) *EvmTxStore_FindConfirmedTxesReceipts_Call { +func (_c *EvmTxStore_FindConfirmedTxesReceipts_Call) RunAndReturn(run func(context.Context, int64, *big.Int) ([]txmgr.DbReceipt, error)) *EvmTxStore_FindConfirmedTxesReceipts_Call { _c.Call.Return(run) return _c } @@ -3197,17 +3199,17 @@ func (_c *EvmTxStore_UpdateTxForRebroadcast_Call) RunAndReturn(run func(context. return _c } -// UpdateTxUnstartedToInProgress provides a mock function with given fields: ctx, etx, attempt -func (_m *EvmTxStore) UpdateTxUnstartedToInProgress(ctx context.Context, etx *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], attempt *types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]) error { - ret := _m.Called(ctx, etx, attempt) +// UpdateTxStatesToFinalizedUsingReceiptId provides a mock function with given fields: ctx, etxIDs, chainId +func (_m *EvmTxStore) UpdateTxStatesToFinalizedUsingReceiptId(ctx context.Context, etxIDs []int64, chainId *big.Int) error { + ret := _m.Called(ctx, etxIDs, chainId) if len(ret) == 0 { - panic("no return value specified for UpdateTxUnstartedToInProgress") + panic("no return value specified for UpdateTxStatesToFinalizedUsingReceiptId") } var r0 error - if rf, ok := ret.Get(0).(func(context.Context, *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], *types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]) error); ok { - r0 = rf(ctx, etx, attempt) + if rf, ok := ret.Get(0).(func(context.Context, []int64, *big.Int) error); ok { + r0 = rf(ctx, etxIDs, chainId) } else { r0 = ret.Error(0) } @@ -3215,47 +3217,47 @@ func (_m *EvmTxStore) UpdateTxUnstartedToInProgress(ctx context.Context, etx *ty return r0 } -// EvmTxStore_UpdateTxUnstartedToInProgress_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateTxUnstartedToInProgress' -type EvmTxStore_UpdateTxUnstartedToInProgress_Call struct { +// EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptId_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateTxStatesToFinalizedUsingReceiptId' +type EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptId_Call struct { *mock.Call } -// UpdateTxUnstartedToInProgress is a helper method to define mock.On call +// UpdateTxStatesToFinalizedUsingReceiptId is a helper method to define mock.On call // - ctx context.Context -// - etx *types.Tx[*big.Int,common.Address,common.Hash,common.Hash,evmtypes.Nonce,gas.EvmFee] -// - attempt *types.TxAttempt[*big.Int,common.Address,common.Hash,common.Hash,evmtypes.Nonce,gas.EvmFee] -func (_e *EvmTxStore_Expecter) UpdateTxUnstartedToInProgress(ctx interface{}, etx interface{}, attempt interface{}) *EvmTxStore_UpdateTxUnstartedToInProgress_Call { - return &EvmTxStore_UpdateTxUnstartedToInProgress_Call{Call: _e.mock.On("UpdateTxUnstartedToInProgress", ctx, etx, attempt)} +// - etxIDs []int64 +// - chainId *big.Int +func (_e *EvmTxStore_Expecter) UpdateTxStatesToFinalizedUsingReceiptId(ctx interface{}, etxIDs interface{}, chainId interface{}) *EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptId_Call { + return &EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptId_Call{Call: _e.mock.On("UpdateTxStatesToFinalizedUsingReceiptId", ctx, etxIDs, chainId)} } -func (_c *EvmTxStore_UpdateTxUnstartedToInProgress_Call) Run(run func(ctx context.Context, etx *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], attempt *types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee])) *EvmTxStore_UpdateTxUnstartedToInProgress_Call { +func (_c *EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptId_Call) Run(run func(ctx context.Context, etxIDs []int64, chainId *big.Int)) *EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptId_Call { _c.Call.Run(func(args mock.Arguments) { - run(args[0].(context.Context), args[1].(*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]), args[2].(*types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee])) + run(args[0].(context.Context), args[1].([]int64), args[2].(*big.Int)) }) return _c } -func (_c *EvmTxStore_UpdateTxUnstartedToInProgress_Call) Return(_a0 error) *EvmTxStore_UpdateTxUnstartedToInProgress_Call { +func (_c *EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptId_Call) Return(_a0 error) *EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptId_Call { _c.Call.Return(_a0) return _c } -func (_c *EvmTxStore_UpdateTxUnstartedToInProgress_Call) RunAndReturn(run func(context.Context, *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], *types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]) error) *EvmTxStore_UpdateTxUnstartedToInProgress_Call { +func (_c *EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptId_Call) RunAndReturn(run func(context.Context, []int64, *big.Int) error) *EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptId_Call { _c.Call.Return(run) return _c } -// UpdateTxesFinalized provides a mock function with given fields: ctx, etxIDs, chainId -func (_m *EvmTxStore) UpdateTxesFinalized(ctx context.Context, etxIDs []int64, chainId *big.Int) error { - ret := _m.Called(ctx, etxIDs, chainId) +// UpdateTxUnstartedToInProgress provides a mock function with given fields: ctx, etx, attempt +func (_m *EvmTxStore) UpdateTxUnstartedToInProgress(ctx context.Context, etx *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], attempt *types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]) error { + ret := _m.Called(ctx, etx, attempt) if len(ret) == 0 { - panic("no return value specified for UpdateTxesFinalized") + panic("no return value specified for UpdateTxUnstartedToInProgress") } var r0 error - if rf, ok := ret.Get(0).(func(context.Context, []int64, *big.Int) error); ok { - r0 = rf(ctx, etxIDs, chainId) + if rf, ok := ret.Get(0).(func(context.Context, *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], *types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]) error); ok { + r0 = rf(ctx, etx, attempt) } else { r0 = ret.Error(0) } @@ -3263,32 +3265,32 @@ func (_m *EvmTxStore) UpdateTxesFinalized(ctx context.Context, etxIDs []int64, c return r0 } -// EvmTxStore_UpdateTxesFinalized_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateTxesFinalized' -type EvmTxStore_UpdateTxesFinalized_Call struct { +// EvmTxStore_UpdateTxUnstartedToInProgress_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateTxUnstartedToInProgress' +type EvmTxStore_UpdateTxUnstartedToInProgress_Call struct { *mock.Call } -// UpdateTxesFinalized is a helper method to define mock.On call +// UpdateTxUnstartedToInProgress is a helper method to define mock.On call // - ctx context.Context -// - etxIDs []int64 -// - chainId *big.Int -func (_e *EvmTxStore_Expecter) UpdateTxesFinalized(ctx interface{}, etxIDs interface{}, chainId interface{}) *EvmTxStore_UpdateTxesFinalized_Call { - return &EvmTxStore_UpdateTxesFinalized_Call{Call: _e.mock.On("UpdateTxesFinalized", ctx, etxIDs, chainId)} +// - etx *types.Tx[*big.Int,common.Address,common.Hash,common.Hash,evmtypes.Nonce,gas.EvmFee] +// - attempt *types.TxAttempt[*big.Int,common.Address,common.Hash,common.Hash,evmtypes.Nonce,gas.EvmFee] +func (_e *EvmTxStore_Expecter) UpdateTxUnstartedToInProgress(ctx interface{}, etx interface{}, attempt interface{}) *EvmTxStore_UpdateTxUnstartedToInProgress_Call { + return &EvmTxStore_UpdateTxUnstartedToInProgress_Call{Call: _e.mock.On("UpdateTxUnstartedToInProgress", ctx, etx, attempt)} } -func (_c *EvmTxStore_UpdateTxesFinalized_Call) Run(run func(ctx context.Context, etxIDs []int64, chainId *big.Int)) *EvmTxStore_UpdateTxesFinalized_Call { +func (_c *EvmTxStore_UpdateTxUnstartedToInProgress_Call) Run(run func(ctx context.Context, etx *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], attempt *types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee])) *EvmTxStore_UpdateTxUnstartedToInProgress_Call { _c.Call.Run(func(args mock.Arguments) { - run(args[0].(context.Context), args[1].([]int64), args[2].(*big.Int)) + run(args[0].(context.Context), args[1].(*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]), args[2].(*types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee])) }) return _c } -func (_c *EvmTxStore_UpdateTxesFinalized_Call) Return(_a0 error) *EvmTxStore_UpdateTxesFinalized_Call { +func (_c *EvmTxStore_UpdateTxUnstartedToInProgress_Call) Return(_a0 error) *EvmTxStore_UpdateTxUnstartedToInProgress_Call { _c.Call.Return(_a0) return _c } -func (_c *EvmTxStore_UpdateTxesFinalized_Call) RunAndReturn(run func(context.Context, []int64, *big.Int) error) *EvmTxStore_UpdateTxesFinalized_Call { +func (_c *EvmTxStore_UpdateTxUnstartedToInProgress_Call) RunAndReturn(run func(context.Context, *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], *types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]) error) *EvmTxStore_UpdateTxUnstartedToInProgress_Call { _c.Call.Return(run) return _c } diff --git a/core/chains/evm/txmgr/models.go b/core/chains/evm/txmgr/models.go index 557e3957445..1ba3d193cba 100644 --- a/core/chains/evm/txmgr/models.go +++ b/core/chains/evm/txmgr/models.go @@ -36,7 +36,7 @@ type ( Tx = txmgrtypes.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee] TxMeta = txmgrtypes.TxMeta[common.Address, common.Hash] TxAttempt = txmgrtypes.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee] - Receipt = dbReceipt // EvmReceipt is the exported DB table model for receipts + Receipt = DbReceipt // DbReceipt is the exported DB table model for receipts ReceiptPlus = txmgrtypes.ReceiptPlus[*evmtypes.Receipt] StuckTxDetector = txmgrtypes.StuckTxDetector[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee] TxmClient = txmgrtypes.TxmClient[*big.Int, common.Address, common.Hash, common.Hash, *evmtypes.Receipt, evmtypes.Nonce, gas.EvmFee] From 92a7383fe1b88c67428fcb258d2afc88e038e485 Mon Sep 17 00:00:00 2001 From: amit-momin Date: Mon, 29 Jul 2024 19:01:59 -0500 Subject: [PATCH 21/23] Fixed mock --- core/chains/evm/txmgr/mocks/evm_tx_store.go | 22 ++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/core/chains/evm/txmgr/mocks/evm_tx_store.go b/core/chains/evm/txmgr/mocks/evm_tx_store.go index 5cffa321f9c..b40c0ca8376 100644 --- a/core/chains/evm/txmgr/mocks/evm_tx_store.go +++ b/core/chains/evm/txmgr/mocks/evm_tx_store.go @@ -3199,12 +3199,12 @@ func (_c *EvmTxStore_UpdateTxForRebroadcast_Call) RunAndReturn(run func(context. return _c } -// UpdateTxStatesToFinalizedUsingReceiptId provides a mock function with given fields: ctx, etxIDs, chainId -func (_m *EvmTxStore) UpdateTxStatesToFinalizedUsingReceiptId(ctx context.Context, etxIDs []int64, chainId *big.Int) error { +// UpdateTxStatesToFinalizedUsingReceiptIds provides a mock function with given fields: ctx, etxIDs, chainId +func (_m *EvmTxStore) UpdateTxStatesToFinalizedUsingReceiptIds(ctx context.Context, etxIDs []int64, chainId *big.Int) error { ret := _m.Called(ctx, etxIDs, chainId) if len(ret) == 0 { - panic("no return value specified for UpdateTxStatesToFinalizedUsingReceiptId") + panic("no return value specified for UpdateTxStatesToFinalizedUsingReceiptIds") } var r0 error @@ -3217,32 +3217,32 @@ func (_m *EvmTxStore) UpdateTxStatesToFinalizedUsingReceiptId(ctx context.Contex return r0 } -// EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptId_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateTxStatesToFinalizedUsingReceiptId' -type EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptId_Call struct { +// EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptIds_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateTxStatesToFinalizedUsingReceiptIds' +type EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptIds_Call struct { *mock.Call } -// UpdateTxStatesToFinalizedUsingReceiptId is a helper method to define mock.On call +// UpdateTxStatesToFinalizedUsingReceiptIds is a helper method to define mock.On call // - ctx context.Context // - etxIDs []int64 // - chainId *big.Int -func (_e *EvmTxStore_Expecter) UpdateTxStatesToFinalizedUsingReceiptId(ctx interface{}, etxIDs interface{}, chainId interface{}) *EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptId_Call { - return &EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptId_Call{Call: _e.mock.On("UpdateTxStatesToFinalizedUsingReceiptId", ctx, etxIDs, chainId)} +func (_e *EvmTxStore_Expecter) UpdateTxStatesToFinalizedUsingReceiptIds(ctx interface{}, etxIDs interface{}, chainId interface{}) *EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptIds_Call { + return &EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptIds_Call{Call: _e.mock.On("UpdateTxStatesToFinalizedUsingReceiptIds", ctx, etxIDs, chainId)} } -func (_c *EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptId_Call) Run(run func(ctx context.Context, etxIDs []int64, chainId *big.Int)) *EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptId_Call { +func (_c *EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptIds_Call) Run(run func(ctx context.Context, etxIDs []int64, chainId *big.Int)) *EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptIds_Call { _c.Call.Run(func(args mock.Arguments) { run(args[0].(context.Context), args[1].([]int64), args[2].(*big.Int)) }) return _c } -func (_c *EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptId_Call) Return(_a0 error) *EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptId_Call { +func (_c *EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptIds_Call) Return(_a0 error) *EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptIds_Call { _c.Call.Return(_a0) return _c } -func (_c *EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptId_Call) RunAndReturn(run func(context.Context, []int64, *big.Int) error) *EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptId_Call { +func (_c *EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptIds_Call) RunAndReturn(run func(context.Context, []int64, *big.Int) error) *EvmTxStore_UpdateTxStatesToFinalizedUsingReceiptIds_Call { _c.Call.Return(run) return _c } From e5812da4da9ece64e32abd7bfd1f3d66bc2290d9 Mon Sep 17 00:00:00 2001 From: amit-momin Date: Mon, 29 Jul 2024 19:37:19 -0500 Subject: [PATCH 22/23] Updated TxStore method to exit early --- core/chains/evm/txmgr/evm_tx_store.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/chains/evm/txmgr/evm_tx_store.go b/core/chains/evm/txmgr/evm_tx_store.go index c5f19d790a2..45de437e443 100644 --- a/core/chains/evm/txmgr/evm_tx_store.go +++ b/core/chains/evm/txmgr/evm_tx_store.go @@ -2096,6 +2096,9 @@ func (o *evmTxStore) FindConfirmedTxesReceipts(ctx context.Context, finalizedBlo // Mark transactions corresponding to receipt IDs as finalized func (o *evmTxStore) UpdateTxStatesToFinalizedUsingReceiptIds(ctx context.Context, receiptIDs []int64, chainId *big.Int) error { + if len(receiptIDs) == 0 { + return nil + } var cancel context.CancelFunc ctx, cancel = o.stopCh.Ctx(ctx) defer cancel() From f8d248730e8489017c47c3a427e8d2e6ab5190e5 Mon Sep 17 00:00:00 2001 From: amit-momin Date: Tue, 30 Jul 2024 09:25:28 -0500 Subject: [PATCH 23/23] Removed unused error --- core/chains/evm/txmgr/finalizer.go | 17 ++++++----------- 1 file changed, 6 insertions(+), 11 deletions(-) diff --git a/core/chains/evm/txmgr/finalizer.go b/core/chains/evm/txmgr/finalizer.go index e88929336eb..6d5fb81782c 100644 --- a/core/chains/evm/txmgr/finalizer.go +++ b/core/chains/evm/txmgr/finalizer.go @@ -160,7 +160,7 @@ func (f *evmFinalizer) processFinalizedHead(ctx context.Context, latestFinalized } earliestBlockNumInChain := latestFinalizedHead.EarliestHeadInChain().BlockNumber() - f.lggr.Debugw("processing latest finalized head", "block num", latestFinalizedHead.BlockNumber(), "block hash", latestFinalizedHead.BlockHash(), "earliest block num in chain", earliestBlockNumInChain) + f.lggr.Debugw("processing latest finalized head", "blockNum", latestFinalizedHead.BlockNumber(), "blockHash", latestFinalizedHead.BlockHash(), "earliestBlockNumInChain", earliestBlockNumInChain) // Retrieve all confirmed transactions with receipts older than or equal to the finalized block, loaded with attempts and receipts unfinalizedReceipts, err := f.txStore.FindConfirmedTxesReceipts(ctx, latestFinalizedHead.BlockNumber(), f.chainId) @@ -200,12 +200,7 @@ func (f *evmFinalizer) processFinalizedHead(ctx context.Context, latestFinalized // Check if block hashes exist for receipts on-chain older than the earliest cached head // Transactions are grouped by their receipt block hash to avoid repeat requests on the same hash in case transactions were confirmed in the same block - validatedReceipts, err := f.batchCheckReceiptHashesOnchain(ctx, blockNumToReceiptsMap) - if err != nil { - // Do not error out to allow transactions that did not need RPC validation to still be marked as finalized - // The transactions failed to be validated will be checked again in the next round - f.lggr.Errorf("failed to validate receipt block hashes over RPC: %v", err) - } + validatedReceipts := f.batchCheckReceiptHashesOnchain(ctx, blockNumToReceiptsMap) finalizedReceipts = append(finalizedReceipts, validatedReceipts...) receiptIDs := f.buildReceiptIdList(finalizedReceipts) @@ -220,9 +215,9 @@ func (f *evmFinalizer) processFinalizedHead(ctx context.Context, latestFinalized return nil } -func (f *evmFinalizer) batchCheckReceiptHashesOnchain(ctx context.Context, blockNumToReceiptsMap map[int64][]Receipt) ([]Receipt, error) { +func (f *evmFinalizer) batchCheckReceiptHashesOnchain(ctx context.Context, blockNumToReceiptsMap map[int64][]Receipt) []Receipt { if len(blockNumToReceiptsMap) == 0 { - return nil, nil + return nil } // Group the RPC batch calls in groups of rpcBatchSize var rpcBatchGroups [][]rpc.BatchElem @@ -251,7 +246,7 @@ func (f *evmFinalizer) batchCheckReceiptHashesOnchain(ctx context.Context, block err := f.client.BatchCallContext(ctx, rpcBatch) if err != nil { // Continue if batch RPC call failed so other batches can still be considered for finalization - f.lggr.Debugw("failed to find blocks due to batch call failure") + f.lggr.Errorw("failed to find blocks due to batch call failure", "error", err) continue } for _, req := range rpcBatch { @@ -281,7 +276,7 @@ func (f *evmFinalizer) batchCheckReceiptHashesOnchain(ctx context.Context, block } } } - return finalizedReceipts, nil + return finalizedReceipts } // Build list of transaction IDs