From fcc4ef5acd3b63cedabef80a3365895b30a11de1 Mon Sep 17 00:00:00 2001 From: Farber98 Date: Wed, 31 Jul 2024 10:54:43 -0300 Subject: [PATCH 01/46] inject trimmed HT API with LatestAndFinalizedBlock --- common/txmgr/confirmer.go | 19 ++++++++++++++++--- core/chains/evm/txmgr/builder.go | 11 +++++++++-- core/chains/legacyevm/chain.go | 2 +- core/chains/legacyevm/evm_txm.go | 6 +++++- core/cmd/shell_local.go | 9 ++++++++- .../promreporter/prom_reporter_test.go | 3 ++- core/services/vrf/delegate_test.go | 2 +- 7 files changed, 42 insertions(+), 10 deletions(-) diff --git a/common/txmgr/confirmer.go b/common/txmgr/confirmer.go index a9e30ffff1e..e23a8e3d6aa 100644 --- a/common/txmgr/confirmer.go +++ b/common/txmgr/confirmer.go @@ -27,6 +27,7 @@ import ( iutils "github.com/smartcontractkit/chainlink/v2/common/internal/utils" 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" ) const ( @@ -102,6 +103,10 @@ var ( }, []string{"chainID"}) ) +type confirmerHeadTracker interface { + LatestAndFinalizedBlock(ctx context.Context) (latest, finalized *evmtypes.Head, err error) +} + // Confirmer is a broad service which performs four different tasks in sequence on every new longest chain // Step 1: Mark that all currently pending transaction attempts were broadcast before this block // Step 2: Check pending transactions for receipts @@ -141,6 +146,7 @@ type Confirmer[ nConsecutiveBlocksChainTooShort int isReceiptNil func(R) bool + headTracker confirmerHeadTracker } func NewConfirmer[ @@ -164,6 +170,7 @@ func NewConfirmer[ lggr logger.Logger, isReceiptNil func(R) bool, stuckTxDetector txmgrtypes.StuckTxDetector[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], + headTracker confirmerHeadTracker, ) *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { lggr = logger.Named(lggr, "Confirmer") return &Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]{ @@ -181,6 +188,7 @@ func NewConfirmer[ mb: mailbox.NewSingle[HEAD](), isReceiptNil: isReceiptNil, stuckTxDetector: stuckTxDetector, + headTracker: headTracker, } } @@ -297,6 +305,11 @@ func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) pro return fmt.Errorf("CheckConfirmedMissingReceipt failed: %w", err) } + _, latestFinalizedHead, err := ec.headTracker.LatestAndFinalizedBlock(ctx) + if err != nil { + return fmt.Errorf("failed to retrieve latest finalized head: %w", err) + } + if err := ec.CheckForReceipts(ctx, head.BlockNumber()); err != nil { return fmt.Errorf("CheckForReceipts failed: %w", err) } @@ -326,7 +339,7 @@ func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) pro if ec.resumeCallback != nil { mark = time.Now() - if err := ec.ResumePendingTaskRuns(ctx, head); err != nil { + if err := ec.ResumePendingTaskRuns(ctx, head, latestFinalizedHead); err != nil { return fmt.Errorf("ResumePendingTaskRuns failed: %w", err) } @@ -1184,8 +1197,8 @@ func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) sen } // ResumePendingTaskRuns issues callbacks to task runs that are pending waiting for receipts -func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) ResumePendingTaskRuns(ctx context.Context, head types.Head[BLOCK_HASH]) error { - receiptsPlus, err := ec.txStore.FindTxesPendingCallback(ctx, head.BlockNumber(), ec.chainID) +func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) ResumePendingTaskRuns(ctx context.Context, head types.Head[BLOCK_HASH], latestFinalizedHead *evmtypes.Head) error { + receiptsPlus, err := ec.txStore.FindTxesPendingCallback(ctx, latestFinalizedHead.BlockNumber(), ec.chainID) if err != nil { return err diff --git a/core/chains/evm/txmgr/builder.go b/core/chains/evm/txmgr/builder.go index dcf15a4fa23..77b8cd260f8 100644 --- a/core/chains/evm/txmgr/builder.go +++ b/core/chains/evm/txmgr/builder.go @@ -1,6 +1,7 @@ package txmgr import ( + "context" "math/big" "time" @@ -17,6 +18,10 @@ import ( evmtypes "github.com/smartcontractkit/chainlink/v2/core/chains/evm/types" ) +type latestAndFinalizedBlockHeadTracker interface { + LatestAndFinalizedBlock(ctx context.Context) (latest, finalized *evmtypes.Head, err error) +} + // NewTxm constructs the necessary dependencies for the EvmTxm (broadcaster, confirmer, etc) and returns a new EvmTxManager func NewTxm( ds sqlutil.DataSource, @@ -31,6 +36,7 @@ func NewTxm( logPoller logpoller.LogPoller, keyStore keystore.Eth, estimator gas.EvmFeeEstimator, + headTracker latestAndFinalizedBlockHeadTracker, ) (txm TxManager, err error, ) { @@ -52,7 +58,7 @@ func NewTxm( evmBroadcaster := NewEvmBroadcaster(txStore, txmClient, txmCfg, feeCfg, txConfig, listenerConfig, keyStore, txAttemptBuilder, lggr, checker, chainConfig.NonceAutoSync()) 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) + evmConfirmer := NewEvmConfirmer(txStore, txmClient, txmCfg, feeCfg, txConfig, dbConfig, keyStore, txAttemptBuilder, lggr, stuckTxDetector, headTracker) var evmResender *Resender if txConfig.ResendAfterThreshold() > 0 { evmResender = NewEvmResender(lggr, txStore, txmClient, evmTracker, keyStore, txmgr.DefaultResenderPollInterval, chainConfig, txConfig) @@ -111,8 +117,9 @@ func NewEvmConfirmer( txAttemptBuilder TxAttemptBuilder, lggr logger.Logger, stuckTxDetector StuckTxDetector, + headTracker latestAndFinalizedBlockHeadTracker, ) *Confirmer { - return txmgr.NewConfirmer(txStore, client, chainConfig, feeConfig, txConfig, dbConfig, keystore, txAttemptBuilder, lggr, func(r *evmtypes.Receipt) bool { return r == nil }, stuckTxDetector) + return txmgr.NewConfirmer(txStore, client, chainConfig, feeConfig, txConfig, dbConfig, keystore, txAttemptBuilder, lggr, func(r *evmtypes.Receipt) bool { return r == nil }, stuckTxDetector, headTracker) } // NewEvmTracker instantiates a new EVM tracker for abandoned transactions 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..7505e344327 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,9 @@ func newEvmTxm( lggr, logPoller, opts.KeyStore, - estimator) + estimator, + headTracker, + ) } else { txm = opts.GenTxManager(chainID) } diff --git a/core/cmd/shell_local.go b/core/cmd/shell_local.go index 3ecd3eed402..81fc936a13d 100644 --- a/core/cmd/shell_local.go +++ b/core/cmd/shell_local.go @@ -34,10 +34,12 @@ import ( "github.com/jmoiron/sqlx" cutils "github.com/smartcontractkit/chainlink-common/pkg/utils" + "github.com/smartcontractkit/chainlink-common/pkg/utils/mailbox" "github.com/smartcontractkit/chainlink/v2/core/build" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/assets" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/gas" + "github.com/smartcontractkit/chainlink/v2/core/chains/evm/headtracker" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/txmgr" evmtypes "github.com/smartcontractkit/chainlink/v2/core/chains/evm/types" ubig "github.com/smartcontractkit/chainlink/v2/core/chains/evm/utils/big" @@ -668,8 +670,13 @@ func (s *Shell) RebroadcastTransactions(c *cli.Context) (err error) { cfg := txmgr.NewEvmTxmConfig(chain.Config().EVM()) feeCfg := txmgr.NewEvmTxmFeeConfig(chain.Config().EVM().GasEstimator()) stuckTxDetector := txmgr.NewStuckTxDetector(lggr, ethClient.ConfiguredChainID(), "", assets.NewWei(assets.NewEth(100).ToInt()), chain.Config().EVM().Transactions().AutoPurge(), nil, orm, ethClient) + htConfig := chain.Config().EVM().HeadTracker() + hb := headtracker.NewHeadBroadcaster(lggr) + htORM := headtracker.NewORM(*chain.ID(), db) + hs := headtracker.NewHeadSaver(lggr, htORM, chain.Config().EVM(), htConfig) + ht := headtracker.NewHeadTracker(lggr, ethClient, chain.Config().EVM(), htConfig, hb, hs, mailbox.NewMonitor(app.ID().String(), lggr)) ec := txmgr.NewEvmConfirmer(orm, txmgr.NewEvmTxmClient(ethClient, chain.Config().EVM().NodePool().Errors()), - cfg, feeCfg, chain.Config().EVM().Transactions(), app.GetConfig().Database(), keyStore.Eth(), txBuilder, chain.Logger(), stuckTxDetector) + cfg, feeCfg, chain.Config().EVM().Transactions(), app.GetConfig().Database(), keyStore.Eth(), txBuilder, chain.Logger(), stuckTxDetector, ht) totalNonces := endingNonce - beginningNonce + 1 nonces := make([]evmtypes.Nonce, totalNonces) for i := int64(0); i < totalNonces; i++ { 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 18665f52b18c683a906878f574a5099fc890a8dd Mon Sep 17 00:00:00 2001 From: Farber98 Date: Wed, 31 Jul 2024 10:55:05 -0300 Subject: [PATCH 02/46] types changes --- common/txmgr/types/tx.go | 2 +- common/txmgr/types/tx_store.go | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/common/txmgr/types/tx.go b/common/txmgr/types/tx.go index d2afbd209d8..6fd0bf3296c 100644 --- a/common/txmgr/types/tx.go +++ b/common/txmgr/types/tx.go @@ -85,7 +85,7 @@ type TxRequest[ADDR types.Hashable, TX_HASH types.Hashable] struct { // Pipeline variables - if you aren't calling this from chain tx task within // the pipeline, you don't need these variables - MinConfirmations clnull.Uint32 + MinConfirmations clnull.Uint32 // deprecated PipelineTaskRunID *uuid.UUID Strategy TxStrategy diff --git a/common/txmgr/types/tx_store.go b/common/txmgr/types/tx_store.go index c102fb5912a..58909d9897a 100644 --- a/common/txmgr/types/tx_store.go +++ b/common/txmgr/types/tx_store.go @@ -35,8 +35,8 @@ type TxStore[ TxHistoryReaper[CHAIN_ID] TransactionStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, SEQ, FEE] - // Find confirmed txes beyond the minConfirmations param that require callback but have not yet been signaled - FindTxesPendingCallback(ctx context.Context, blockNum int64, chainID CHAIN_ID) (receiptsPlus []ReceiptPlus[R], err error) + // Find confirmed txes before the LatestFinalizedBlockNum param that require callback but have not yet been signaled + FindTxesPendingCallback(ctx context.Context, latestFinalizedBlockNum int64, chainID CHAIN_ID) (receiptsPlus []ReceiptPlus[R], err error) // Update tx to mark that its callback has been signaled UpdateTxCallbackCompleted(ctx context.Context, pipelineTaskRunRid uuid.UUID, chainId CHAIN_ID) error SaveFetchedReceipts(ctx context.Context, r []R, state TxState, errorMsg *string, chainID CHAIN_ID) error From c7f79c62e08ab17cd832e5d512ab0354a1803ccc Mon Sep 17 00:00:00 2001 From: Farber98 Date: Wed, 31 Jul 2024 10:55:56 -0300 Subject: [PATCH 03/46] tests --- core/chains/evm/txmgr/confirmer_test.go | 119 ++++++++++++--------- core/chains/evm/txmgr/evm_tx_store.go | 12 +-- core/chains/evm/txmgr/evm_tx_store_test.go | 57 +++------- core/chains/evm/txmgr/txmgr_test.go | 4 +- 4 files changed, 91 insertions(+), 101 deletions(-) diff --git a/core/chains/evm/txmgr/confirmer_test.go b/core/chains/evm/txmgr/confirmer_test.go index 6b107b222a6..27b7d4f1aa4 100644 --- a/core/chains/evm/txmgr/confirmer_test.go +++ b/core/chains/evm/txmgr/confirmer_test.go @@ -34,6 +34,7 @@ import ( evmconfig "github.com/smartcontractkit/chainlink/v2/core/chains/evm/config" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/gas" gasmocks "github.com/smartcontractkit/chainlink/v2/core/chains/evm/gas/mocks" + "github.com/smartcontractkit/chainlink/v2/core/chains/evm/headtracker" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/keystore" ksmocks "github.com/smartcontractkit/chainlink/v2/core/chains/evm/keystore/mocks" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/testutils" @@ -131,7 +132,8 @@ func TestEthConfirmer_Lifecycle(t *testing.T) { feeEstimator := gas.NewEvmFeeEstimator(lggr, newEst, ge.EIP1559DynamicFees(), ge) txBuilder := txmgr.NewEvmTxAttemptBuilder(*ethClient.ConfiguredChainID(), ge, ethKeyStore, feeEstimator) stuckTxDetector := txmgr.NewStuckTxDetector(lggr, testutils.FixtureChainID, "", assets.NewWei(assets.NewEth(100).ToInt()), config.EVM().Transactions().AutoPurge(), feeEstimator, txStore, ethClient) - ec := txmgr.NewEvmConfirmer(txStore, txmgr.NewEvmTxmClient(ethClient, nil), txmgr.NewEvmTxmConfig(config.EVM()), txmgr.NewEvmTxmFeeConfig(ge), config.EVM().Transactions(), gconfig.Database(), ethKeyStore, txBuilder, lggr, stuckTxDetector) + ht := headtracker.NewSimulatedHeadTracker(ethClient, true, 0) + ec := txmgr.NewEvmConfirmer(txStore, txmgr.NewEvmTxmClient(ethClient, nil), txmgr.NewEvmTxmConfig(config.EVM()), txmgr.NewEvmTxmFeeConfig(ge), config.EVM().Transactions(), gconfig.Database(), ethKeyStore, txBuilder, lggr, stuckTxDetector, ht) ctx := tests.Context(t) // Can't close unstarted instance @@ -145,19 +147,27 @@ func TestEthConfirmer_Lifecycle(t *testing.T) { // Can't start an already started instance err = ec.Start(ctx) require.Error(t, err) + + latestFinalizedHead := evmtypes.Head{ + Number: 8, + Hash: testutils.NewHash(), + Parent: nil, + IsFinalized: true, // We are guaranteed to receive a latestFinalizedHead. + } + head := evmtypes.Head{ Hash: testutils.NewHash(), Number: 10, Parent: &evmtypes.Head{ Hash: testutils.NewHash(), Number: 9, - Parent: &evmtypes.Head{ - Number: 8, - Hash: testutils.NewHash(), - Parent: nil, - }, + Parent: &latestFinalizedHead, }, } + + ethClient.On("HeadByNumber", mock.Anything, (*big.Int)(nil)).Return(&head, nil).Once() + ethClient.On("LatestFinalizedBlock", mock.Anything).Return(&latestFinalizedHead, nil).Once() + err = ec.ProcessHead(ctx, &head) require.NoError(t, err) // Can successfully close once @@ -1651,8 +1661,9 @@ func TestEthConfirmer_RebroadcastWhereNecessary_WithConnectivityCheck(t *testing addresses := []gethCommon.Address{fromAddress} kst.On("EnabledAddressesForChain", mock.Anything, &cltest.FixtureChainID).Return(addresses, nil).Maybe() stuckTxDetector := txmgr.NewStuckTxDetector(lggr, testutils.FixtureChainID, "", assets.NewWei(assets.NewEth(100).ToInt()), ccfg.EVM().Transactions().AutoPurge(), feeEstimator, txStore, ethClient) + ht := headtracker.NewSimulatedHeadTracker(ethClient, true, 0) // Create confirmer with necessary state - ec := txmgr.NewEvmConfirmer(txStore, txmgr.NewEvmTxmClient(ethClient, nil), ccfg.EVM(), txmgr.NewEvmTxmFeeConfig(ccfg.EVM().GasEstimator()), ccfg.EVM().Transactions(), cfg.Database(), kst, txBuilder, lggr, stuckTxDetector) + ec := txmgr.NewEvmConfirmer(txStore, txmgr.NewEvmTxmClient(ethClient, nil), ccfg.EVM(), txmgr.NewEvmTxmFeeConfig(ccfg.EVM().GasEstimator()), ccfg.EVM().Transactions(), cfg.Database(), kst, txBuilder, lggr, stuckTxDetector, ht) servicetest.Run(t, ec) currentHead := int64(30) oldEnough := int64(15) @@ -1700,7 +1711,8 @@ func TestEthConfirmer_RebroadcastWhereNecessary_WithConnectivityCheck(t *testing addresses := []gethCommon.Address{fromAddress} kst.On("EnabledAddressesForChain", mock.Anything, &cltest.FixtureChainID).Return(addresses, nil).Maybe() stuckTxDetector := txmgr.NewStuckTxDetector(lggr, testutils.FixtureChainID, "", assets.NewWei(assets.NewEth(100).ToInt()), ccfg.EVM().Transactions().AutoPurge(), feeEstimator, txStore, ethClient) - ec := txmgr.NewEvmConfirmer(txStore, txmgr.NewEvmTxmClient(ethClient, nil), ccfg.EVM(), txmgr.NewEvmTxmFeeConfig(ccfg.EVM().GasEstimator()), ccfg.EVM().Transactions(), cfg.Database(), kst, txBuilder, lggr, stuckTxDetector) + ht := headtracker.NewSimulatedHeadTracker(ethClient, true, 0) + ec := txmgr.NewEvmConfirmer(txStore, txmgr.NewEvmTxmClient(ethClient, nil), ccfg.EVM(), txmgr.NewEvmTxmFeeConfig(ccfg.EVM().GasEstimator()), ccfg.EVM().Transactions(), cfg.Database(), kst, txBuilder, lggr, stuckTxDetector, ht) servicetest.Run(t, ec) currentHead := int64(30) oldEnough := int64(15) @@ -2945,67 +2957,62 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { evmcfg := evmtest.NewChainScopedConfig(t, config) + latestFinalizedHead := evmtypes.Head{ + Number: 8, + Hash: testutils.NewHash(), + Parent: nil, + IsFinalized: true, // We are guaranteed to receive a latestFinalizedHead. + } + head := evmtypes.Head{ Hash: testutils.NewHash(), Number: 10, Parent: &evmtypes.Head{ Hash: testutils.NewHash(), Number: 9, - Parent: &evmtypes.Head{ - Number: 8, - Hash: testutils.NewHash(), - Parent: nil, - }, + Parent: &latestFinalizedHead, }, } - - minConfirmations := int64(2) - pgtest.MustExec(t, db, `SET CONSTRAINTS fk_pipeline_runs_pruning_key DEFERRED`) pgtest.MustExec(t, db, `SET CONSTRAINTS pipeline_runs_pipeline_spec_id_fkey DEFERRED`) - t.Run("doesn't process task runs that are not suspended (possibly already previously resumed)", func(t *testing.T) { - ec := newEthConfirmer(t, txStore, ethClient, config, evmcfg, ethKeyStore, func(context.Context, uuid.UUID, interface{}, error) error { - t.Fatal("No value expected") - return nil - }) + ec := newEthConfirmer(t, txStore, ethClient, config, evmcfg, ethKeyStore, func(context.Context, uuid.UUID, interface{}, error) error { + t.Fatal("No value expected") + return nil + }) + t.Run("doesn't process task runs that are not suspended (possibly already previously resumed)", func(t *testing.T) { run := cltest.MustInsertPipelineRun(t, db) tr := cltest.MustInsertUnfinishedPipelineTaskRun(t, db, run.ID) etx := cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, 1, 1, fromAddress) - mustInsertEthReceipt(t, txStore, head.Number-minConfirmations, head.Hash, etx.TxAttempts[0].Hash) + mustInsertEthReceipt(t, txStore, latestFinalizedHead.BlockNumber(), head.Hash, etx.TxAttempts[0].Hash) // Setting both signal_callback and callback_completed to TRUE to simulate a completed pipeline task // It would only be in a state past suspended if the resume callback was called and callback_completed was set to TRUE - pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, min_confirmations = $2, signal_callback = TRUE, callback_completed = TRUE WHERE id = $3`, &tr.ID, minConfirmations, etx.ID) + pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE, callback_completed = TRUE WHERE id = $2`, &tr.ID, etx.ID) - err := ec.ResumePendingTaskRuns(tests.Context(t), &head) + err := ec.ResumePendingTaskRuns(tests.Context(t), &head, &latestFinalizedHead) require.NoError(t, err) }) - t.Run("doesn't process task runs where the receipt is younger than minConfirmations", func(t *testing.T) { - ec := newEthConfirmer(t, txStore, ethClient, config, evmcfg, ethKeyStore, func(context.Context, uuid.UUID, interface{}, error) error { - t.Fatal("No value expected") - return nil - }) - + t.Run("doesn't process task runs where the receipt is before LatestFinalizedBlockNum", func(t *testing.T) { run := cltest.MustInsertPipelineRun(t, db) tr := cltest.MustInsertUnfinishedPipelineTaskRun(t, db, run.ID) etx := cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, 2, 1, fromAddress) mustInsertEthReceipt(t, txStore, head.Number, head.Hash, etx.TxAttempts[0].Hash) - pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, min_confirmations = $2, signal_callback = TRUE WHERE id = $3`, &tr.ID, minConfirmations, etx.ID) + pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE WHERE id = $2`, &tr.ID, etx.ID) - err := ec.ResumePendingTaskRuns(tests.Context(t), &head) + err := ec.ResumePendingTaskRuns(tests.Context(t), &head, &latestFinalizedHead) require.NoError(t, err) }) - t.Run("processes eth_txes with receipts older than minConfirmations", func(t *testing.T) { + t.Run("processes eth_txes with receipts after LatestFinalizedBlockNum", func(t *testing.T) { ch := make(chan interface{}) nonce := evmtypes.Nonce(3) var err error - ec := newEthConfirmer(t, txStore, ethClient, config, evmcfg, ethKeyStore, func(ctx context.Context, id uuid.UUID, value interface{}, thisErr error) error { + ec = newEthConfirmer(t, txStore, ethClient, config, evmcfg, ethKeyStore, func(ctx context.Context, id uuid.UUID, value interface{}, thisErr error) error { err = thisErr ch <- value return nil @@ -3017,15 +3024,15 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { etx := cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, int64(nonce), 1, fromAddress) pgtest.MustExec(t, db, `UPDATE evm.txes SET meta='{"FailOnRevert": true}'`) - receipt := mustInsertEthReceipt(t, txStore, head.Number-minConfirmations, head.Hash, etx.TxAttempts[0].Hash) + receipt := mustInsertEthReceipt(t, txStore, latestFinalizedHead.BlockNumber(), head.Hash, etx.TxAttempts[0].Hash) - pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, min_confirmations = $2, signal_callback = TRUE WHERE id = $3`, &tr.ID, minConfirmations, etx.ID) + pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE WHERE id = $2`, &tr.ID, etx.ID) done := make(chan struct{}) t.Cleanup(func() { <-done }) go func() { defer close(done) - err2 := ec.ResumePendingTaskRuns(tests.Context(t), &head) + err2 := ec.ResumePendingTaskRuns(tests.Context(t), &head, &latestFinalizedHead) if !assert.NoError(t, err2) { return } @@ -3051,14 +3058,14 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { pgtest.MustExec(t, db, `DELETE FROM pipeline_runs`) - t.Run("processes eth_txes with receipt older than minConfirmations that reverted", func(t *testing.T) { + t.Run("processes eth_txes with receipt before LatestFinalizedBlockNum that reverted", func(t *testing.T) { type data struct { value any error } ch := make(chan data) nonce := evmtypes.Nonce(4) - ec := newEthConfirmer(t, txStore, ethClient, config, evmcfg, ethKeyStore, func(ctx context.Context, id uuid.UUID, value interface{}, err error) error { + ec = newEthConfirmer(t, txStore, ethClient, config, evmcfg, ethKeyStore, func(ctx context.Context, id uuid.UUID, value interface{}, err error) error { ch <- data{value, err} return nil }) @@ -3071,15 +3078,15 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { pgtest.MustExec(t, db, `UPDATE evm.txes SET meta='{"FailOnRevert": true}'`) // receipt is not passed through as a value since it reverted and caused an error - mustInsertRevertedEthReceipt(t, txStore, head.Number-minConfirmations, head.Hash, etx.TxAttempts[0].Hash) + mustInsertRevertedEthReceipt(t, txStore, latestFinalizedHead.BlockNumber()-2, head.Hash, etx.TxAttempts[0].Hash) - pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, min_confirmations = $2, signal_callback = TRUE WHERE id = $3`, &tr.ID, minConfirmations, etx.ID) + pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE WHERE id = $2`, &tr.ID, etx.ID) done := make(chan struct{}) t.Cleanup(func() { <-done }) go func() { defer close(done) - err2 := ec.ResumePendingTaskRuns(tests.Context(t), &head) + err2 := ec.ResumePendingTaskRuns(tests.Context(t), &head, &latestFinalizedHead) if !assert.NoError(t, err2) { return } @@ -3105,7 +3112,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { t.Run("does not mark callback complete if callback fails", func(t *testing.T) { nonce := evmtypes.Nonce(5) - ec := newEthConfirmer(t, txStore, ethClient, config, evmcfg, ethKeyStore, func(context.Context, uuid.UUID, interface{}, error) error { + ec = newEthConfirmer(t, txStore, ethClient, config, evmcfg, ethKeyStore, func(context.Context, uuid.UUID, interface{}, error) error { return errors.New("error") }) @@ -3113,10 +3120,10 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { tr := cltest.MustInsertUnfinishedPipelineTaskRun(t, db, run.ID) etx := cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, int64(nonce), 1, fromAddress) - mustInsertEthReceipt(t, txStore, head.Number-minConfirmations, head.Hash, etx.TxAttempts[0].Hash) - pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, min_confirmations = $2, signal_callback = TRUE WHERE id = $3`, &tr.ID, minConfirmations, etx.ID) + mustInsertEthReceipt(t, txStore, latestFinalizedHead.BlockNumber(), head.Hash, etx.TxAttempts[0].Hash) + pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE WHERE id = $2`, &tr.ID, etx.ID) - err := ec.ResumePendingTaskRuns(tests.Context(t), &head) + err := ec.ResumePendingTaskRuns(tests.Context(t), &head, &latestFinalizedHead) require.Error(t, err) // Retrieve Tx to check if callback completed flag was left unchanged @@ -3158,7 +3165,8 @@ func TestEthConfirmer_ProcessStuckTransactions(t *testing.T) { ge := evmcfg.EVM().GasEstimator() txBuilder := txmgr.NewEvmTxAttemptBuilder(*ethClient.ConfiguredChainID(), ge, ethKeyStore, feeEstimator) stuckTxDetector := txmgr.NewStuckTxDetector(lggr, testutils.FixtureChainID, "", assets.NewWei(assets.NewEth(100).ToInt()), evmcfg.EVM().Transactions().AutoPurge(), feeEstimator, txStore, ethClient) - ec := txmgr.NewEvmConfirmer(txStore, txmgr.NewEvmTxmClient(ethClient, nil), txmgr.NewEvmTxmConfig(evmcfg.EVM()), txmgr.NewEvmTxmFeeConfig(ge), evmcfg.EVM().Transactions(), cfg.Database(), ethKeyStore, txBuilder, lggr, stuckTxDetector) + ht := headtracker.NewSimulatedHeadTracker(ethClient, true, 0) + ec := txmgr.NewEvmConfirmer(txStore, txmgr.NewEvmTxmClient(ethClient, nil), txmgr.NewEvmTxmConfig(evmcfg.EVM()), txmgr.NewEvmTxmFeeConfig(ge), evmcfg.EVM().Transactions(), cfg.Database(), ethKeyStore, txBuilder, lggr, stuckTxDetector, ht) servicetest.Run(t, ec) ctx := tests.Context(t) @@ -3172,9 +3180,12 @@ func TestEthConfirmer_ProcessStuckTransactions(t *testing.T) { tx := mustInsertUnconfirmedTxWithBroadcastAttempts(t, txStore, nonce, fromAddress, autoPurgeMinAttempts, blockNum-int64(autoPurgeThreshold), marketGasPrice.Add(oneGwei)) head := evmtypes.Head{ - Hash: testutils.NewHash(), - Number: blockNum, + Hash: testutils.NewHash(), + Number: blockNum, + IsFinalized: true, } + ethClient.On("HeadByNumber", mock.Anything, (*big.Int)(nil)).Return(&head, nil).Once() + ethClient.On("LatestFinalizedBlock", mock.Anything).Return(&head, nil).Once() ethClient.On("SequenceAt", mock.Anything, mock.Anything, mock.Anything).Return(evmtypes.Nonce(0), nil).Once() ethClient.On("BatchCallContext", mock.Anything, mock.Anything).Return(nil).Once() @@ -3196,9 +3207,12 @@ func TestEthConfirmer_ProcessStuckTransactions(t *testing.T) { require.Equal(t, bumpedFee.Legacy, latestAttempt.TxFee.Legacy) head = evmtypes.Head{ - Hash: testutils.NewHash(), - Number: blockNum + 1, + Hash: testutils.NewHash(), + Number: blockNum + 1, + IsFinalized: true, } + ethClient.On("HeadByNumber", mock.Anything, (*big.Int)(nil)).Return(&head, nil).Once() + ethClient.On("LatestFinalizedBlock", mock.Anything).Return(&head, nil).Once() ethClient.On("SequenceAt", mock.Anything, mock.Anything, mock.Anything).Return(evmtypes.Nonce(1), nil) ethClient.On("BatchCallContext", mock.Anything, mock.MatchedBy(func(b []rpc.BatchElem) bool { return len(b) == 4 && cltest.BatchElemMatchesParams(b[0], latestAttempt.Hash, "eth_getTransactionReceipt") @@ -3237,7 +3251,8 @@ func newEthConfirmer(t testing.TB, txStore txmgr.EvmTxStore, ethClient client.Cl }, ge.EIP1559DynamicFees(), ge) txBuilder := txmgr.NewEvmTxAttemptBuilder(*ethClient.ConfiguredChainID(), ge, ks, estimator) stuckTxDetector := txmgr.NewStuckTxDetector(lggr, testutils.FixtureChainID, "", assets.NewWei(assets.NewEth(100).ToInt()), config.EVM().Transactions().AutoPurge(), estimator, txStore, ethClient) - ec := txmgr.NewEvmConfirmer(txStore, txmgr.NewEvmTxmClient(ethClient, nil), txmgr.NewEvmTxmConfig(config.EVM()), txmgr.NewEvmTxmFeeConfig(ge), config.EVM().Transactions(), gconfig.Database(), ks, txBuilder, lggr, stuckTxDetector) + ht := headtracker.NewSimulatedHeadTracker(ethClient, true, 0) + ec := txmgr.NewEvmConfirmer(txStore, txmgr.NewEvmTxmClient(ethClient, nil), txmgr.NewEvmTxmConfig(config.EVM()), txmgr.NewEvmTxmFeeConfig(ge), config.EVM().Transactions(), gconfig.Database(), ks, txBuilder, lggr, stuckTxDetector, ht) ec.SetResumeCallback(fn) servicetest.Run(t, ec) return ec diff --git a/core/chains/evm/txmgr/evm_tx_store.go b/core/chains/evm/txmgr/evm_tx_store.go index fd38eb7a8c9..f743c385dd1 100644 --- a/core/chains/evm/txmgr/evm_tx_store.go +++ b/core/chains/evm/txmgr/evm_tx_store.go @@ -214,7 +214,6 @@ func (db *DbEthTx) FromTx(tx *Tx) { db.Meta = tx.Meta db.Subject = tx.Subject db.PipelineTaskRunID = tx.PipelineTaskRunID - db.MinConfirmations = tx.MinConfirmations db.TransmitChecker = tx.TransmitChecker db.InitialBroadcastAt = tx.InitialBroadcastAt db.SignalCallback = tx.SignalCallback @@ -248,7 +247,6 @@ func (db DbEthTx) ToTx(tx *Tx) { tx.Meta = db.Meta tx.Subject = db.Subject tx.PipelineTaskRunID = db.PipelineTaskRunID - tx.MinConfirmations = db.MinConfirmations tx.ChainID = db.EVMChainID.ToInt() tx.TransmitChecker = db.TransmitChecker tx.InitialBroadcastAt = db.InitialBroadcastAt @@ -1015,7 +1013,7 @@ WHERE evm.tx_attempts.state = 'in_progress' AND evm.txes.from_address = $1 AND e } // Find confirmed txes requiring callback but have not yet been signaled -func (o *evmTxStore) FindTxesPendingCallback(ctx context.Context, blockNum int64, chainID *big.Int) (receiptsPlus []ReceiptPlus, err error) { +func (o *evmTxStore) FindTxesPendingCallback(ctx context.Context, latestFinalizedBlockNum int64, chainID *big.Int) (receiptsPlus []ReceiptPlus, err error) { var rs []dbReceiptPlus var cancel context.CancelFunc @@ -1026,8 +1024,8 @@ func (o *evmTxStore) FindTxesPendingCallback(ctx context.Context, blockNum int64 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.pipeline_task_run_id IS NOT NULL AND evm.txes.signal_callback = TRUE AND evm.txes.callback_completed = FALSE - AND evm.receipts.block_number <= ($1 - evm.txes.min_confirmations) AND evm.txes.evm_chain_id = $2 - `, blockNum, chainID.String()) + AND evm.receipts.block_number <= $1 AND evm.txes.evm_chain_id = $2 + `, latestFinalizedBlockNum, chainID.String()) if err != nil { return nil, fmt.Errorf("failed to retrieve transactions pending pipeline resume callback: %w", err) } @@ -1834,10 +1832,10 @@ func (o *evmTxStore) CreateTransaction(ctx context.Context, txRequest TxRequest, err = orm.q.GetContext(ctx, &dbEtx, ` INSERT INTO evm.txes (from_address, to_address, encoded_payload, value, gas_limit, state, created_at, meta, subject, evm_chain_id, min_confirmations, pipeline_task_run_id, transmit_checker, idempotency_key, signal_callback) VALUES ( -$1,$2,$3,$4,$5,'unstarted',NOW(),$6,$7,$8,$9,$10,$11,$12,$13 +$1,$2,$3,$4,$5,'unstarted',NOW(),$6,$7,$8,NULL,$9,$10,$11,$12 ) RETURNING "txes".* -`, txRequest.FromAddress, txRequest.ToAddress, txRequest.EncodedPayload, assets.Eth(txRequest.Value), txRequest.FeeLimit, txRequest.Meta, txRequest.Strategy.Subject(), chainID.String(), txRequest.MinConfirmations, txRequest.PipelineTaskRunID, txRequest.Checker, txRequest.IdempotencyKey, txRequest.SignalCallback) +`, txRequest.FromAddress, txRequest.ToAddress, txRequest.EncodedPayload, assets.Eth(txRequest.Value), txRequest.FeeLimit, txRequest.Meta, txRequest.Strategy.Subject(), chainID.String(), txRequest.PipelineTaskRunID, txRequest.Checker, txRequest.IdempotencyKey, txRequest.SignalCallback) if err != nil { return pkgerrors.Wrap(err, "CreateEthTransaction failed to insert evm tx") } diff --git a/core/chains/evm/txmgr/evm_tx_store_test.go b/core/chains/evm/txmgr/evm_tx_store_test.go index afb8de4ca52..26cc98e6ef5 100644 --- a/core/chains/evm/txmgr/evm_tx_store_test.go +++ b/core/chains/evm/txmgr/evm_tx_store_test.go @@ -628,22 +628,23 @@ func TestORM_FindTxesPendingCallback(t *testing.T) { pgtest.MustExec(t, db, `SET CONSTRAINTS fk_pipeline_runs_pruning_key DEFERRED`) pgtest.MustExec(t, db, `SET CONSTRAINTS pipeline_runs_pipeline_spec_id_fkey DEFERRED`) + latestFinalizedHead := evmtypes.Head{ + Number: 8, + Hash: utils.NewHash(), + Parent: nil, + IsFinalized: true, + } + head := evmtypes.Head{ Hash: utils.NewHash(), Number: 10, Parent: &evmtypes.Head{ Hash: utils.NewHash(), Number: 9, - Parent: &evmtypes.Head{ - Number: 8, - Hash: utils.NewHash(), - Parent: nil, - }, + Parent: &latestFinalizedHead, }, } - minConfirmations := int64(2) - // Suspended run waiting for callback run1 := cltest.MustInsertPipelineRun(t, db) tr1 := cltest.MustInsertUnfinishedPipelineTaskRun(t, db, run1.ID) @@ -651,8 +652,8 @@ func TestORM_FindTxesPendingCallback(t *testing.T) { etx1 := cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, 3, 1, fromAddress) pgtest.MustExec(t, db, `UPDATE evm.txes SET meta='{"FailOnRevert": true}'`) attempt1 := etx1.TxAttempts[0] - mustInsertEthReceipt(t, txStore, head.Number-minConfirmations, head.Hash, attempt1.Hash) - pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, min_confirmations = $2, signal_callback = TRUE WHERE id = $3`, &tr1.ID, minConfirmations, etx1.ID) + mustInsertEthReceipt(t, txStore, latestFinalizedHead.BlockNumber(), head.Hash, attempt1.Hash) + pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE WHERE id = $2`, &tr1.ID, etx1.ID) // Callback to pipeline service completed. Should be ignored run2 := cltest.MustInsertPipelineRunWithStatus(t, db, 0, pipeline.RunStatusCompleted, 0) @@ -660,10 +661,10 @@ func TestORM_FindTxesPendingCallback(t *testing.T) { etx2 := cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, 4, 1, fromAddress) pgtest.MustExec(t, db, `UPDATE evm.txes SET meta='{"FailOnRevert": false}'`) attempt2 := etx2.TxAttempts[0] - mustInsertEthReceipt(t, txStore, head.Number-minConfirmations, head.Hash, attempt2.Hash) - pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, min_confirmations = $2, signal_callback = TRUE, callback_completed = TRUE WHERE id = $3`, &tr2.ID, minConfirmations, etx2.ID) + mustInsertEthReceipt(t, txStore, latestFinalizedHead.BlockNumber()-2, head.Hash, attempt2.Hash) + pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE, callback_completed = TRUE WHERE id = $2`, &tr2.ID, etx2.ID) - // Suspended run younger than minConfirmations. Should be ignored + // Suspend run after LatestFinalizedBlockNum. Should be ignored run3 := cltest.MustInsertPipelineRun(t, db) tr3 := cltest.MustInsertUnfinishedPipelineTaskRun(t, db, run3.ID) pgtest.MustExec(t, db, `UPDATE pipeline_runs SET state = 'suspended' WHERE id = $1`, run3.ID) @@ -671,20 +672,18 @@ func TestORM_FindTxesPendingCallback(t *testing.T) { pgtest.MustExec(t, db, `UPDATE evm.txes SET meta='{"FailOnRevert": false}'`) attempt3 := etx3.TxAttempts[0] mustInsertEthReceipt(t, txStore, head.Number, head.Hash, attempt3.Hash) - pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, min_confirmations = $2, signal_callback = TRUE WHERE id = $3`, &tr3.ID, minConfirmations, etx3.ID) + pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE WHERE id = $2`, &tr3.ID, etx3.ID) // Tx not marked for callback. Should be ignore etx4 := cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, 6, 1, fromAddress) attempt4 := etx4.TxAttempts[0] mustInsertEthReceipt(t, txStore, head.Number, head.Hash, attempt4.Hash) - pgtest.MustExec(t, db, `UPDATE evm.txes SET min_confirmations = $1 WHERE id = $2`, minConfirmations, etx4.ID) // Unconfirmed Tx without receipts. Should be ignored - etx5 := cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, 7, 1, fromAddress) - pgtest.MustExec(t, db, `UPDATE evm.txes SET min_confirmations = $1 WHERE id = $2`, minConfirmations, etx5.ID) + cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, 7, 1, fromAddress) // Search evm.txes table for tx requiring callback - receiptsPlus, err := txStore.FindTxesPendingCallback(tests.Context(t), head.Number, ethClient.ConfiguredChainID()) + receiptsPlus, err := txStore.FindTxesPendingCallback(tests.Context(t), latestFinalizedHead.BlockNumber(), ethClient.ConfiguredChainID()) require.NoError(t, err) assert.Len(t, receiptsPlus, 1) assert.Equal(t, tr1.ID, receiptsPlus[0].ID) @@ -783,30 +782,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() diff --git a/core/chains/evm/txmgr/txmgr_test.go b/core/chains/evm/txmgr/txmgr_test.go index 40df5616c99..93a00682523 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) { @@ -612,6 +613,7 @@ func TestTxm_GetTransactionStatus(t *testing.T) { feeEstimator := gasmocks.NewEvmFeeEstimator(t) feeEstimator.On("Start", mock.Anything).Return(nil).Once() feeEstimator.On("Close", mock.Anything).Return(nil).Once() + ethClient.On("HeadByNumber", mock.Anything, (*big.Int)(nil)).Return(nil, nil) feeEstimator.On("OnNewLongestChain", mock.Anything, mock.Anything).Once() txm, err := makeTestEvmTxm(t, db, ethClient, feeEstimator, cfg.EVM(), cfg.EVM().GasEstimator(), cfg.EVM().Transactions(), gcfg.Database(), gcfg.Database().Listener(), ethKeyStore) require.NoError(t, err) From 2df05f72c211c8a5e7298d802a9bb606083b0f10 Mon Sep 17 00:00:00 2001 From: Farber98 Date: Wed, 31 Jul 2024 11:00:29 -0300 Subject: [PATCH 04/46] mocks --- common/txmgr/types/mocks/tx_store.go | 20 ++++++++++---------- core/chains/evm/txmgr/mocks/evm_tx_store.go | 20 ++++++++++---------- 2 files changed, 20 insertions(+), 20 deletions(-) diff --git a/common/txmgr/types/mocks/tx_store.go b/common/txmgr/types/mocks/tx_store.go index ee166638e34..0682259feda 100644 --- a/common/txmgr/types/mocks/tx_store.go +++ b/common/txmgr/types/mocks/tx_store.go @@ -1096,9 +1096,9 @@ func (_c *TxStore_FindTxesByMetaFieldAndStates_Call[ADDR, CHAIN_ID, TX_HASH, BLO return _c } -// FindTxesPendingCallback provides a mock function with given fields: ctx, blockNum, chainID -func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxesPendingCallback(ctx context.Context, blockNum int64, chainID CHAIN_ID) ([]txmgrtypes.ReceiptPlus[R], error) { - ret := _m.Called(ctx, blockNum, chainID) +// FindTxesPendingCallback provides a mock function with given fields: ctx, latestFinalizedBlockNum, chainID +func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxesPendingCallback(ctx context.Context, latestFinalizedBlockNum int64, chainID CHAIN_ID) ([]txmgrtypes.ReceiptPlus[R], error) { + ret := _m.Called(ctx, latestFinalizedBlockNum, chainID) if len(ret) == 0 { panic("no return value specified for FindTxesPendingCallback") @@ -1107,10 +1107,10 @@ func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxesPen var r0 []txmgrtypes.ReceiptPlus[R] var r1 error if rf, ok := ret.Get(0).(func(context.Context, int64, CHAIN_ID) ([]txmgrtypes.ReceiptPlus[R], error)); ok { - return rf(ctx, blockNum, chainID) + return rf(ctx, latestFinalizedBlockNum, chainID) } if rf, ok := ret.Get(0).(func(context.Context, int64, CHAIN_ID) []txmgrtypes.ReceiptPlus[R]); ok { - r0 = rf(ctx, blockNum, chainID) + r0 = rf(ctx, latestFinalizedBlockNum, chainID) } else { if ret.Get(0) != nil { r0 = ret.Get(0).([]txmgrtypes.ReceiptPlus[R]) @@ -1118,7 +1118,7 @@ func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxesPen } if rf, ok := ret.Get(1).(func(context.Context, int64, CHAIN_ID) error); ok { - r1 = rf(ctx, blockNum, chainID) + r1 = rf(ctx, latestFinalizedBlockNum, chainID) } else { r1 = ret.Error(1) } @@ -1133,13 +1133,13 @@ type TxStore_FindTxesPendingCallback_Call[ADDR types.Hashable, CHAIN_ID types.ID // FindTxesPendingCallback is a helper method to define mock.On call // - ctx context.Context -// - blockNum int64 +// - latestFinalizedBlockNum int64 // - chainID CHAIN_ID -func (_e *TxStore_Expecter[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxesPendingCallback(ctx interface{}, blockNum interface{}, chainID interface{}) *TxStore_FindTxesPendingCallback_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { - return &TxStore_FindTxesPendingCallback_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]{Call: _e.mock.On("FindTxesPendingCallback", ctx, blockNum, chainID)} +func (_e *TxStore_Expecter[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxesPendingCallback(ctx interface{}, latestFinalizedBlockNum interface{}, chainID interface{}) *TxStore_FindTxesPendingCallback_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { + return &TxStore_FindTxesPendingCallback_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]{Call: _e.mock.On("FindTxesPendingCallback", ctx, latestFinalizedBlockNum, chainID)} } -func (_c *TxStore_FindTxesPendingCallback_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) Run(run func(ctx context.Context, blockNum int64, chainID CHAIN_ID)) *TxStore_FindTxesPendingCallback_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { +func (_c *TxStore_FindTxesPendingCallback_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) Run(run func(ctx context.Context, latestFinalizedBlockNum int64, chainID CHAIN_ID)) *TxStore_FindTxesPendingCallback_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].(CHAIN_ID)) }) diff --git a/core/chains/evm/txmgr/mocks/evm_tx_store.go b/core/chains/evm/txmgr/mocks/evm_tx_store.go index b28e55ec324..e1cbe8e4dd0 100644 --- a/core/chains/evm/txmgr/mocks/evm_tx_store.go +++ b/core/chains/evm/txmgr/mocks/evm_tx_store.go @@ -1333,9 +1333,9 @@ func (_c *EvmTxStore_FindTxesByMetaFieldAndStates_Call) RunAndReturn(run func(co return _c } -// FindTxesPendingCallback provides a mock function with given fields: ctx, blockNum, chainID -func (_m *EvmTxStore) FindTxesPendingCallback(ctx context.Context, blockNum int64, chainID *big.Int) ([]types.ReceiptPlus[*evmtypes.Receipt], error) { - ret := _m.Called(ctx, blockNum, chainID) +// FindTxesPendingCallback provides a mock function with given fields: ctx, latestFinalizedBlockNum, chainID +func (_m *EvmTxStore) FindTxesPendingCallback(ctx context.Context, latestFinalizedBlockNum int64, chainID *big.Int) ([]types.ReceiptPlus[*evmtypes.Receipt], error) { + ret := _m.Called(ctx, latestFinalizedBlockNum, chainID) if len(ret) == 0 { panic("no return value specified for FindTxesPendingCallback") @@ -1344,10 +1344,10 @@ func (_m *EvmTxStore) FindTxesPendingCallback(ctx context.Context, blockNum int6 var r0 []types.ReceiptPlus[*evmtypes.Receipt] var r1 error if rf, ok := ret.Get(0).(func(context.Context, int64, *big.Int) ([]types.ReceiptPlus[*evmtypes.Receipt], error)); ok { - return rf(ctx, blockNum, chainID) + return rf(ctx, latestFinalizedBlockNum, chainID) } if rf, ok := ret.Get(0).(func(context.Context, int64, *big.Int) []types.ReceiptPlus[*evmtypes.Receipt]); ok { - r0 = rf(ctx, blockNum, chainID) + r0 = rf(ctx, latestFinalizedBlockNum, chainID) } else { if ret.Get(0) != nil { r0 = ret.Get(0).([]types.ReceiptPlus[*evmtypes.Receipt]) @@ -1355,7 +1355,7 @@ func (_m *EvmTxStore) FindTxesPendingCallback(ctx context.Context, blockNum int6 } if rf, ok := ret.Get(1).(func(context.Context, int64, *big.Int) error); ok { - r1 = rf(ctx, blockNum, chainID) + r1 = rf(ctx, latestFinalizedBlockNum, chainID) } else { r1 = ret.Error(1) } @@ -1370,13 +1370,13 @@ type EvmTxStore_FindTxesPendingCallback_Call struct { // FindTxesPendingCallback is a helper method to define mock.On call // - ctx context.Context -// - blockNum int64 +// - latestFinalizedBlockNum int64 // - chainID *big.Int -func (_e *EvmTxStore_Expecter) FindTxesPendingCallback(ctx interface{}, blockNum interface{}, chainID interface{}) *EvmTxStore_FindTxesPendingCallback_Call { - return &EvmTxStore_FindTxesPendingCallback_Call{Call: _e.mock.On("FindTxesPendingCallback", ctx, blockNum, chainID)} +func (_e *EvmTxStore_Expecter) FindTxesPendingCallback(ctx interface{}, latestFinalizedBlockNum interface{}, chainID interface{}) *EvmTxStore_FindTxesPendingCallback_Call { + return &EvmTxStore_FindTxesPendingCallback_Call{Call: _e.mock.On("FindTxesPendingCallback", ctx, latestFinalizedBlockNum, chainID)} } -func (_c *EvmTxStore_FindTxesPendingCallback_Call) Run(run func(ctx context.Context, blockNum int64, chainID *big.Int)) *EvmTxStore_FindTxesPendingCallback_Call { +func (_c *EvmTxStore_FindTxesPendingCallback_Call) Run(run func(ctx context.Context, latestFinalizedBlockNum int64, chainID *big.Int)) *EvmTxStore_FindTxesPendingCallback_Call { _c.Call.Run(func(args mock.Arguments) { run(args[0].(context.Context), args[1].(int64), args[2].(*big.Int)) }) From df0eeefef4da6dd5622e1d9c093c05611c4f21d3 Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Wed, 31 Jul 2024 17:26:56 -0500 Subject: [PATCH 05/46] refactor --- common/txmgr/confirmer.go | 31 ++++++++++++------------- core/chains/evm/txmgr/builder.go | 9 ++----- core/chains/evm/txmgr/confirmer_test.go | 10 ++++---- 3 files changed, 22 insertions(+), 28 deletions(-) diff --git a/common/txmgr/confirmer.go b/common/txmgr/confirmer.go index e23a8e3d6aa..b3377a88da5 100644 --- a/common/txmgr/confirmer.go +++ b/common/txmgr/confirmer.go @@ -5,6 +5,7 @@ import ( "encoding/hex" "errors" "fmt" + "github.com/smartcontractkit/chainlink/v2/core/chains/evm/logpoller" "sort" "strconv" "sync" @@ -27,7 +28,6 @@ import ( iutils "github.com/smartcontractkit/chainlink/v2/common/internal/utils" 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" ) const ( @@ -103,10 +103,6 @@ var ( }, []string{"chainID"}) ) -type confirmerHeadTracker interface { - LatestAndFinalizedBlock(ctx context.Context) (latest, finalized *evmtypes.Head, err error) -} - // Confirmer is a broad service which performs four different tasks in sequence on every new longest chain // Step 1: Mark that all currently pending transaction attempts were broadcast before this block // Step 2: Check pending transactions for receipts @@ -146,7 +142,7 @@ type Confirmer[ nConsecutiveBlocksChainTooShort int isReceiptNil func(R) bool - headTracker confirmerHeadTracker + headTracker logpoller.HeadTracker } func NewConfirmer[ @@ -170,7 +166,7 @@ func NewConfirmer[ lggr logger.Logger, isReceiptNil func(R) bool, stuckTxDetector txmgrtypes.StuckTxDetector[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], - headTracker confirmerHeadTracker, + headTracker logpoller.HeadTracker, ) *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { lggr = logger.Named(lggr, "Confirmer") return &Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]{ @@ -305,11 +301,6 @@ func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) pro return fmt.Errorf("CheckConfirmedMissingReceipt failed: %w", err) } - _, latestFinalizedHead, err := ec.headTracker.LatestAndFinalizedBlock(ctx) - if err != nil { - return fmt.Errorf("failed to retrieve latest finalized head: %w", err) - } - if err := ec.CheckForReceipts(ctx, head.BlockNumber()); err != nil { return fmt.Errorf("CheckForReceipts failed: %w", err) } @@ -339,7 +330,16 @@ func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) pro if ec.resumeCallback != nil { mark = time.Now() - if err := ec.ResumePendingTaskRuns(ctx, head, latestFinalizedHead); err != nil { + + headBlockNumber := head.BlockNumber() + _, latestFinalizedHead, err := ec.headTracker.LatestAndFinalizedBlock(ctx) + if err == nil { + headBlockNumber = latestFinalizedHead.BlockNumber() + } else { + ec.lggr.Errorw("failed to retrieve latest finalized head, use default head number", "error", err, "headNum", headBlockNumber, "id", "confirmer") + } + + if err = ec.ResumePendingTaskRuns(ctx, headBlockNumber); err != nil { return fmt.Errorf("ResumePendingTaskRuns failed: %w", err) } @@ -1197,9 +1197,8 @@ func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) sen } // ResumePendingTaskRuns issues callbacks to task runs that are pending waiting for receipts -func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) ResumePendingTaskRuns(ctx context.Context, head types.Head[BLOCK_HASH], latestFinalizedHead *evmtypes.Head) error { - receiptsPlus, err := ec.txStore.FindTxesPendingCallback(ctx, latestFinalizedHead.BlockNumber(), ec.chainID) - +func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) ResumePendingTaskRuns(ctx context.Context, headBlockNumber int64) error { + receiptsPlus, err := ec.txStore.FindTxesPendingCallback(ctx, headBlockNumber, ec.chainID) if err != nil { return err } diff --git a/core/chains/evm/txmgr/builder.go b/core/chains/evm/txmgr/builder.go index 77b8cd260f8..5378ccae505 100644 --- a/core/chains/evm/txmgr/builder.go +++ b/core/chains/evm/txmgr/builder.go @@ -1,7 +1,6 @@ package txmgr import ( - "context" "math/big" "time" @@ -18,10 +17,6 @@ import ( evmtypes "github.com/smartcontractkit/chainlink/v2/core/chains/evm/types" ) -type latestAndFinalizedBlockHeadTracker interface { - LatestAndFinalizedBlock(ctx context.Context) (latest, finalized *evmtypes.Head, err error) -} - // NewTxm constructs the necessary dependencies for the EvmTxm (broadcaster, confirmer, etc) and returns a new EvmTxManager func NewTxm( ds sqlutil.DataSource, @@ -36,7 +31,7 @@ func NewTxm( logPoller logpoller.LogPoller, keyStore keystore.Eth, estimator gas.EvmFeeEstimator, - headTracker latestAndFinalizedBlockHeadTracker, + headTracker logpoller.HeadTracker, ) (txm TxManager, err error, ) { @@ -117,7 +112,7 @@ func NewEvmConfirmer( txAttemptBuilder TxAttemptBuilder, lggr logger.Logger, stuckTxDetector StuckTxDetector, - headTracker latestAndFinalizedBlockHeadTracker, + headTracker logpoller.HeadTracker, ) *Confirmer { return txmgr.NewConfirmer(txStore, client, chainConfig, feeConfig, txConfig, dbConfig, keystore, txAttemptBuilder, lggr, func(r *evmtypes.Receipt) bool { return r == nil }, stuckTxDetector, headTracker) } diff --git a/core/chains/evm/txmgr/confirmer_test.go b/core/chains/evm/txmgr/confirmer_test.go index 27b7d4f1aa4..0b8127b19e4 100644 --- a/core/chains/evm/txmgr/confirmer_test.go +++ b/core/chains/evm/txmgr/confirmer_test.go @@ -2991,7 +2991,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { // It would only be in a state past suspended if the resume callback was called and callback_completed was set to TRUE pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE, callback_completed = TRUE WHERE id = $2`, &tr.ID, etx.ID) - err := ec.ResumePendingTaskRuns(tests.Context(t), &head, &latestFinalizedHead) + err := ec.ResumePendingTaskRuns(tests.Context(t), latestFinalizedHead.BlockNumber()) require.NoError(t, err) }) @@ -3004,7 +3004,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE WHERE id = $2`, &tr.ID, etx.ID) - err := ec.ResumePendingTaskRuns(tests.Context(t), &head, &latestFinalizedHead) + err := ec.ResumePendingTaskRuns(tests.Context(t), latestFinalizedHead.BlockNumber()) require.NoError(t, err) }) @@ -3032,7 +3032,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { t.Cleanup(func() { <-done }) go func() { defer close(done) - err2 := ec.ResumePendingTaskRuns(tests.Context(t), &head, &latestFinalizedHead) + err2 := ec.ResumePendingTaskRuns(tests.Context(t), latestFinalizedHead.BlockNumber()) if !assert.NoError(t, err2) { return } @@ -3086,7 +3086,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { t.Cleanup(func() { <-done }) go func() { defer close(done) - err2 := ec.ResumePendingTaskRuns(tests.Context(t), &head, &latestFinalizedHead) + err2 := ec.ResumePendingTaskRuns(tests.Context(t), latestFinalizedHead.BlockNumber()) if !assert.NoError(t, err2) { return } @@ -3123,7 +3123,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { mustInsertEthReceipt(t, txStore, latestFinalizedHead.BlockNumber(), head.Hash, etx.TxAttempts[0].Hash) pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE WHERE id = $2`, &tr.ID, etx.ID) - err := ec.ResumePendingTaskRuns(tests.Context(t), &head, &latestFinalizedHead) + err := ec.ResumePendingTaskRuns(tests.Context(t), latestFinalizedHead.BlockNumber()) require.Error(t, err) // Retrieve Tx to check if callback completed flag was left unchanged From 2ae25faa6eff3b332d883512a3dc7645f32f986e Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Wed, 31 Jul 2024 19:57:02 -0500 Subject: [PATCH 06/46] update goland format to fix lint, add changeset file --- .changeset/tidy-planets-kneel.md | 5 +++++ common/txmgr/confirmer.go | 3 ++- 2 files changed, 7 insertions(+), 1 deletion(-) create mode 100644 .changeset/tidy-planets-kneel.md diff --git a/.changeset/tidy-planets-kneel.md b/.changeset/tidy-planets-kneel.md new file mode 100644 index 00000000000..3734b33f6fc --- /dev/null +++ b/.changeset/tidy-planets-kneel.md @@ -0,0 +1,5 @@ +--- +"chainlink": patch +--- + +Remove dependency of MinConfirmations in EVM TXM code diff --git a/common/txmgr/confirmer.go b/common/txmgr/confirmer.go index b3377a88da5..95b5816bfc1 100644 --- a/common/txmgr/confirmer.go +++ b/common/txmgr/confirmer.go @@ -5,12 +5,13 @@ import ( "encoding/hex" "errors" "fmt" - "github.com/smartcontractkit/chainlink/v2/core/chains/evm/logpoller" "sort" "strconv" "sync" "time" + "github.com/smartcontractkit/chainlink/v2/core/chains/evm/logpoller" + "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" "go.uber.org/multierr" From eec43807b8365021baa9058818700ab8f44b53a7 Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Wed, 31 Jul 2024 20:02:04 -0500 Subject: [PATCH 07/46] update changeset --- .changeset/tidy-planets-kneel.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.changeset/tidy-planets-kneel.md b/.changeset/tidy-planets-kneel.md index 3734b33f6fc..514b7c8bcbf 100644 --- a/.changeset/tidy-planets-kneel.md +++ b/.changeset/tidy-planets-kneel.md @@ -2,4 +2,4 @@ "chainlink": patch --- -Remove dependency of MinConfirmations in EVM TXM code +Remove dependency of MinConfirmations in EVM TXM code #update #deprecation_notice From 82156bf15fcaba1fcc93362835661d7077ac8e6e Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Thu, 1 Aug 2024 13:27:30 -0500 Subject: [PATCH 08/46] update --- common/txmgr/confirmer.go | 24 ++++++++++++------------ core/chains/evm/txmgr/builder.go | 9 +++++++-- 2 files changed, 19 insertions(+), 14 deletions(-) diff --git a/common/txmgr/confirmer.go b/common/txmgr/confirmer.go index 95b5816bfc1..fe524d952fe 100644 --- a/common/txmgr/confirmer.go +++ b/common/txmgr/confirmer.go @@ -10,7 +10,7 @@ import ( "sync" "time" - "github.com/smartcontractkit/chainlink/v2/core/chains/evm/logpoller" + evmtypes "github.com/smartcontractkit/chainlink/v2/core/chains/evm/types" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" @@ -104,6 +104,10 @@ var ( }, []string{"chainID"}) ) +type confirmerHeadTracker interface { + LatestAndFinalizedBlock(ctx context.Context) (latest, finalized *evmtypes.Head, err error) +} + // Confirmer is a broad service which performs four different tasks in sequence on every new longest chain // Step 1: Mark that all currently pending transaction attempts were broadcast before this block // Step 2: Check pending transactions for receipts @@ -143,7 +147,7 @@ type Confirmer[ nConsecutiveBlocksChainTooShort int isReceiptNil func(R) bool - headTracker logpoller.HeadTracker + headTracker confirmerHeadTracker } func NewConfirmer[ @@ -167,7 +171,7 @@ func NewConfirmer[ lggr logger.Logger, isReceiptNil func(R) bool, stuckTxDetector txmgrtypes.StuckTxDetector[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], - headTracker logpoller.HeadTracker, + headTracker confirmerHeadTracker, ) *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { lggr = logger.Named(lggr, "Confirmer") return &Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]{ @@ -331,16 +335,12 @@ func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) pro if ec.resumeCallback != nil { mark = time.Now() - - headBlockNumber := head.BlockNumber() _, latestFinalizedHead, err := ec.headTracker.LatestAndFinalizedBlock(ctx) - if err == nil { - headBlockNumber = latestFinalizedHead.BlockNumber() - } else { - ec.lggr.Errorw("failed to retrieve latest finalized head, use default head number", "error", err, "headNum", headBlockNumber, "id", "confirmer") + if err != nil { + return fmt.Errorf("failed to retrieve latest finalized head: %w", err) } - if err = ec.ResumePendingTaskRuns(ctx, headBlockNumber); err != nil { + if err = ec.ResumePendingTaskRuns(ctx, latestFinalizedHead.BlockNumber()); err != nil { return fmt.Errorf("ResumePendingTaskRuns failed: %w", err) } @@ -1198,8 +1198,8 @@ func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) sen } // ResumePendingTaskRuns issues callbacks to task runs that are pending waiting for receipts -func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) ResumePendingTaskRuns(ctx context.Context, headBlockNumber int64) error { - receiptsPlus, err := ec.txStore.FindTxesPendingCallback(ctx, headBlockNumber, ec.chainID) +func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) ResumePendingTaskRuns(ctx context.Context, latestFinalizedBlockNum int64) error { + receiptsPlus, err := ec.txStore.FindTxesPendingCallback(ctx, latestFinalizedBlockNum, ec.chainID) if err != nil { return err } diff --git a/core/chains/evm/txmgr/builder.go b/core/chains/evm/txmgr/builder.go index 5378ccae505..77b8cd260f8 100644 --- a/core/chains/evm/txmgr/builder.go +++ b/core/chains/evm/txmgr/builder.go @@ -1,6 +1,7 @@ package txmgr import ( + "context" "math/big" "time" @@ -17,6 +18,10 @@ import ( evmtypes "github.com/smartcontractkit/chainlink/v2/core/chains/evm/types" ) +type latestAndFinalizedBlockHeadTracker interface { + LatestAndFinalizedBlock(ctx context.Context) (latest, finalized *evmtypes.Head, err error) +} + // NewTxm constructs the necessary dependencies for the EvmTxm (broadcaster, confirmer, etc) and returns a new EvmTxManager func NewTxm( ds sqlutil.DataSource, @@ -31,7 +36,7 @@ func NewTxm( logPoller logpoller.LogPoller, keyStore keystore.Eth, estimator gas.EvmFeeEstimator, - headTracker logpoller.HeadTracker, + headTracker latestAndFinalizedBlockHeadTracker, ) (txm TxManager, err error, ) { @@ -112,7 +117,7 @@ func NewEvmConfirmer( txAttemptBuilder TxAttemptBuilder, lggr logger.Logger, stuckTxDetector StuckTxDetector, - headTracker logpoller.HeadTracker, + headTracker latestAndFinalizedBlockHeadTracker, ) *Confirmer { return txmgr.NewConfirmer(txStore, client, chainConfig, feeConfig, txConfig, dbConfig, keystore, txAttemptBuilder, lggr, func(r *evmtypes.Receipt) bool { return r == nil }, stuckTxDetector, headTracker) } From 92b597439bb5bee2a8e72bf0bb7e7336485a713d Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Thu, 1 Aug 2024 14:05:48 -0500 Subject: [PATCH 09/46] fix test --- core/chains/evm/txmgr/txmgr_test.go | 1 - 1 file changed, 1 deletion(-) diff --git a/core/chains/evm/txmgr/txmgr_test.go b/core/chains/evm/txmgr/txmgr_test.go index 93a00682523..a9f3e1d549a 100644 --- a/core/chains/evm/txmgr/txmgr_test.go +++ b/core/chains/evm/txmgr/txmgr_test.go @@ -613,7 +613,6 @@ func TestTxm_GetTransactionStatus(t *testing.T) { feeEstimator := gasmocks.NewEvmFeeEstimator(t) feeEstimator.On("Start", mock.Anything).Return(nil).Once() feeEstimator.On("Close", mock.Anything).Return(nil).Once() - ethClient.On("HeadByNumber", mock.Anything, (*big.Int)(nil)).Return(nil, nil) feeEstimator.On("OnNewLongestChain", mock.Anything, mock.Anything).Once() txm, err := makeTestEvmTxm(t, db, ethClient, feeEstimator, cfg.EVM(), cfg.EVM().GasEstimator(), cfg.EVM().Transactions(), gcfg.Database(), gcfg.Database().Listener(), ethKeyStore) require.NoError(t, err) From 9c28bbb7de7513af0af1224af06106cd98040933 Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Thu, 1 Aug 2024 16:03:01 -0500 Subject: [PATCH 10/46] more test fix --- core/chains/evm/txmgr/confirmer_test.go | 8 -------- 1 file changed, 8 deletions(-) diff --git a/core/chains/evm/txmgr/confirmer_test.go b/core/chains/evm/txmgr/confirmer_test.go index 0b8127b19e4..e510246c452 100644 --- a/core/chains/evm/txmgr/confirmer_test.go +++ b/core/chains/evm/txmgr/confirmer_test.go @@ -164,10 +164,6 @@ func TestEthConfirmer_Lifecycle(t *testing.T) { Parent: &latestFinalizedHead, }, } - - ethClient.On("HeadByNumber", mock.Anything, (*big.Int)(nil)).Return(&head, nil).Once() - ethClient.On("LatestFinalizedBlock", mock.Anything).Return(&latestFinalizedHead, nil).Once() - err = ec.ProcessHead(ctx, &head) require.NoError(t, err) // Can successfully close once @@ -3184,8 +3180,6 @@ func TestEthConfirmer_ProcessStuckTransactions(t *testing.T) { Number: blockNum, IsFinalized: true, } - ethClient.On("HeadByNumber", mock.Anything, (*big.Int)(nil)).Return(&head, nil).Once() - ethClient.On("LatestFinalizedBlock", mock.Anything).Return(&head, nil).Once() ethClient.On("SequenceAt", mock.Anything, mock.Anything, mock.Anything).Return(evmtypes.Nonce(0), nil).Once() ethClient.On("BatchCallContext", mock.Anything, mock.Anything).Return(nil).Once() @@ -3211,8 +3205,6 @@ func TestEthConfirmer_ProcessStuckTransactions(t *testing.T) { Number: blockNum + 1, IsFinalized: true, } - ethClient.On("HeadByNumber", mock.Anything, (*big.Int)(nil)).Return(&head, nil).Once() - ethClient.On("LatestFinalizedBlock", mock.Anything).Return(&head, nil).Once() ethClient.On("SequenceAt", mock.Anything, mock.Anything, mock.Anything).Return(evmtypes.Nonce(1), nil) ethClient.On("BatchCallContext", mock.Anything, mock.MatchedBy(func(b []rpc.BatchElem) bool { return len(b) == 4 && cltest.BatchElemMatchesParams(b[0], latestAttempt.Hash, "eth_getTransactionReceipt") From cdb12d3c89abab5e8d658672c760bfc1a6856712 Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Thu, 1 Aug 2024 18:40:35 -0500 Subject: [PATCH 11/46] fix integration test, and added important comments --- common/txmgr/confirmer.go | 24 ++++++++++++++++++++---- common/txmgr/types/config.go | 1 + 2 files changed, 21 insertions(+), 4 deletions(-) diff --git a/common/txmgr/confirmer.go b/common/txmgr/confirmer.go index fe524d952fe..003d26ab919 100644 --- a/common/txmgr/confirmer.go +++ b/common/txmgr/confirmer.go @@ -335,12 +335,28 @@ func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) pro if ec.resumeCallback != nil { mark = time.Now() - _, latestFinalizedHead, err := ec.headTracker.LatestAndFinalizedBlock(ctx) - if err != nil { - return fmt.Errorf("failed to retrieve latest finalized head: %w", err) + + // backward compatibility purpose: + // Since fallback.toml have not yet updated, LatestAndFinalizedBlock() is not going to work properly due to + // calculateLatestFinalized() checks the related tags, and we will always get 0 for finalized block height, causing + // integration test (TestIntegration_AsyncEthTx) to fail. + // + // HOWEVER, we are assuming the current head is finalized head, because we removed the minConfirmation check + // if this is an issue, then we will have to merge this PR after BCI-3573 and BCI-3730 + + headNumber := head.BlockNumber() + if ec.chainConfig.FinalityTagEnabled() { + _, latestFinalizedBlock, err := ec.headTracker.LatestAndFinalizedBlock(ctx) + if err != nil { + return fmt.Errorf("failed to retrieve latest finalized head: %w", err) + } + + headNumber = latestFinalizedBlock.BlockNumber() } - if err = ec.ResumePendingTaskRuns(ctx, latestFinalizedHead.BlockNumber()); err != nil { + // TODO Once BCI-3574 is merged we can update the chainConfig interface to remove FinalityTagEnabled, and remove the + // head.BlockNumber() + if err := ec.ResumePendingTaskRuns(ctx, headNumber); err != nil { return fmt.Errorf("ResumePendingTaskRuns failed: %w", err) } diff --git a/common/txmgr/types/config.go b/common/txmgr/types/config.go index 4d9af5f0673..013169184e9 100644 --- a/common/txmgr/types/config.go +++ b/common/txmgr/types/config.go @@ -51,6 +51,7 @@ type ConfirmerFeeConfig interface { } type ConfirmerChainConfig interface { + FinalityTagEnabled() bool RPCDefaultBatchSize() uint32 FinalityDepth() uint32 } From d30eb3a9d39a1ab7b70b635b11496f468a3641a6 Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Thu, 1 Aug 2024 18:51:30 -0500 Subject: [PATCH 12/46] fix commit signing --- common/txmgr/confirmer.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/txmgr/confirmer.go b/common/txmgr/confirmer.go index 003d26ab919..f3e31788484 100644 --- a/common/txmgr/confirmer.go +++ b/common/txmgr/confirmer.go @@ -340,7 +340,7 @@ func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) pro // Since fallback.toml have not yet updated, LatestAndFinalizedBlock() is not going to work properly due to // calculateLatestFinalized() checks the related tags, and we will always get 0 for finalized block height, causing // integration test (TestIntegration_AsyncEthTx) to fail. - // + // HOWEVER, we are assuming the current head is finalized head, because we removed the minConfirmation check // if this is an issue, then we will have to merge this PR after BCI-3573 and BCI-3730 From fe3eb8b3b493ceae6242cb760f09eb9b0fea553f Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Thu, 1 Aug 2024 23:59:33 -0500 Subject: [PATCH 13/46] update comments --- common/txmgr/confirmer.go | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/common/txmgr/confirmer.go b/common/txmgr/confirmer.go index f3e31788484..9159a8b147a 100644 --- a/common/txmgr/confirmer.go +++ b/common/txmgr/confirmer.go @@ -336,14 +336,12 @@ func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) pro if ec.resumeCallback != nil { mark = time.Now() - // backward compatibility purpose: - // Since fallback.toml have not yet updated, LatestAndFinalizedBlock() is not going to work properly due to - // calculateLatestFinalized() checks the related tags, and we will always get 0 for finalized block height, causing - // integration test (TestIntegration_AsyncEthTx) to fail. - - // HOWEVER, we are assuming the current head is finalized head, because we removed the minConfirmation check - // if this is an issue, then we will have to merge this PR after BCI-3573 and BCI-3730 + // LatestAndFinalizedBlock() is returning only the (latest block - finalityDepth - finalityOffset) and we will have + // 0 for the first $(finalityDepth) number of blocks, and this becomes an issue for integration test (TestIntegration_AsyncEthTx) + // Temporarily made the block number to be the current head, however we removed the minConfirmation offset when fetching pending + // task/blocks and this can be optimistic + // TODO update this after BCI-3573 and BCI-3730 is merged headNumber := head.BlockNumber() if ec.chainConfig.FinalityTagEnabled() { _, latestFinalizedBlock, err := ec.headTracker.LatestAndFinalizedBlock(ctx) @@ -352,6 +350,8 @@ func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) pro } headNumber = latestFinalizedBlock.BlockNumber() + } else { + headNumber = headNumber - int64(ec.chainConfig.FinalityDepth()) } // TODO Once BCI-3574 is merged we can update the chainConfig interface to remove FinalityTagEnabled, and remove the From 8b34faabd30fe7e1f4a9cb28bb76d99078fda89f Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Fri, 2 Aug 2024 00:35:13 -0500 Subject: [PATCH 14/46] rephrase comments --- common/txmgr/confirmer.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/common/txmgr/confirmer.go b/common/txmgr/confirmer.go index 9159a8b147a..b85a1f0df3b 100644 --- a/common/txmgr/confirmer.go +++ b/common/txmgr/confirmer.go @@ -336,10 +336,11 @@ func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) pro if ec.resumeCallback != nil { mark = time.Now() - // LatestAndFinalizedBlock() is returning only the (latest block - finalityDepth - finalityOffset) and we will have - // 0 for the first $(finalityDepth) number of blocks, and this becomes an issue for integration test (TestIntegration_AsyncEthTx) + // There might be some dependency here with BCI-3573 and BCI-3730. Since we haven't enabled finality tag, + // LatestAndFinalizedBlock() is returning 0 from calculateLatestFinalized() for the first $(finalityDepth+finalityOffset) number of blocks + // and this becomes an issue for integration test (TestIntegration_AsyncEthTx) // Temporarily made the block number to be the current head, however we removed the minConfirmation offset when fetching pending - // task/blocks and this can be optimistic + // task/blocks and this can be optimistic, so not sure if we want to merge this PR first among the three // TODO update this after BCI-3573 and BCI-3730 is merged headNumber := head.BlockNumber() From ab755c1c631d01ce12c456a6eee59d51197cdeea Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Fri, 2 Aug 2024 00:38:27 -0500 Subject: [PATCH 15/46] undo typo --- common/txmgr/confirmer.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/common/txmgr/confirmer.go b/common/txmgr/confirmer.go index b85a1f0df3b..abd91d15749 100644 --- a/common/txmgr/confirmer.go +++ b/common/txmgr/confirmer.go @@ -351,8 +351,6 @@ func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) pro } headNumber = latestFinalizedBlock.BlockNumber() - } else { - headNumber = headNumber - int64(ec.chainConfig.FinalityDepth()) } // TODO Once BCI-3574 is merged we can update the chainConfig interface to remove FinalityTagEnabled, and remove the From c32d0612862f08314ed756c4682d4700c169b4b5 Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Fri, 2 Aug 2024 00:54:17 -0500 Subject: [PATCH 16/46] small refactor --- common/txmgr/confirmer.go | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/common/txmgr/confirmer.go b/common/txmgr/confirmer.go index abd91d15749..b2b00a127e5 100644 --- a/common/txmgr/confirmer.go +++ b/common/txmgr/confirmer.go @@ -343,19 +343,18 @@ func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) pro // task/blocks and this can be optimistic, so not sure if we want to merge this PR first among the three // TODO update this after BCI-3573 and BCI-3730 is merged - headNumber := head.BlockNumber() - if ec.chainConfig.FinalityTagEnabled() { - _, latestFinalizedBlock, err := ec.headTracker.LatestAndFinalizedBlock(ctx) - if err != nil { - return fmt.Errorf("failed to retrieve latest finalized head: %w", err) - } - - headNumber = latestFinalizedBlock.BlockNumber() + var latestFinalizedHeadNum int64 + if !ec.chainConfig.FinalityTagEnabled() { + latestFinalizedHeadNum = head.BlockNumber() + } else if _, latestFinalizedBlock, err := ec.headTracker.LatestAndFinalizedBlock(ctx); err != nil { + return fmt.Errorf("failed to retrieve latest finalized head: %w", err) + } else { + latestFinalizedHeadNum = latestFinalizedBlock.BlockNumber() } // TODO Once BCI-3574 is merged we can update the chainConfig interface to remove FinalityTagEnabled, and remove the // head.BlockNumber() - if err := ec.ResumePendingTaskRuns(ctx, headNumber); err != nil { + if err := ec.ResumePendingTaskRuns(ctx, latestFinalizedHeadNum); err != nil { return fmt.Errorf("ResumePendingTaskRuns failed: %w", err) } From be447e81822d52a9751b1737bcaf95aaf2c3a76f Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Fri, 2 Aug 2024 00:57:26 -0500 Subject: [PATCH 17/46] update comments --- common/txmgr/confirmer.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/common/txmgr/confirmer.go b/common/txmgr/confirmer.go index b2b00a127e5..1ae02372820 100644 --- a/common/txmgr/confirmer.go +++ b/common/txmgr/confirmer.go @@ -339,8 +339,10 @@ func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) pro // There might be some dependency here with BCI-3573 and BCI-3730. Since we haven't enabled finality tag, // LatestAndFinalizedBlock() is returning 0 from calculateLatestFinalized() for the first $(finalityDepth+finalityOffset) number of blocks // and this becomes an issue for integration test (TestIntegration_AsyncEthTx) - // Temporarily made the block number to be the current head, however we removed the minConfirmation offset when fetching pending - // task/blocks and this can be optimistic, so not sure if we want to merge this PR first among the three + // + // Therefore, line 349, temporarily made the block number to be the current head when finality tag disabled , + // however we removed the minConfirmation offset when fetching pending task/blocks and this can be optimistic, + // wonder if we want to merge this PR the last among the three of them in BCI-3572 // TODO update this after BCI-3573 and BCI-3730 is merged var latestFinalizedHeadNum int64 From c9033db9eb24672e66f309dace28197b7e694739 Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Fri, 2 Aug 2024 01:13:45 -0500 Subject: [PATCH 18/46] fix --- common/txmgr/confirmer.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/common/txmgr/confirmer.go b/common/txmgr/confirmer.go index ba3873a4edb..1f6b8af7daa 100644 --- a/common/txmgr/confirmer.go +++ b/common/txmgr/confirmer.go @@ -338,11 +338,11 @@ func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) pro // There might be some dependency here with BCI-3573 and BCI-3730. Since we haven't enabled finality tag, // LatestAndFinalizedBlock() is returning 0 from calculateLatestFinalized() for the first $(finalityDepth+finalityOffset) number of blocks - // and this becomes an issue for integration test (TestIntegration_AsyncEthTx) + // and this becomes an issue for integration test (TestIntegration_AsyncEthTx). // // Therefore, line 349, temporarily made the block number to be the current head when finality tag disabled , // however we removed the minConfirmation offset when fetching pending task/blocks and this can be optimistic, - // wonder if we want to merge this PR the last among the three of them in BCI-3572 + // wonder if we want to merge this PR the last among the three of them in BCI-3572. // TODO update this after BCI-3573 and BCI-3730 is merged var latestFinalizedHeadNum int64 From a4557a4e30a6ae0c03699e33f4a195b0e4e5c038 Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Fri, 2 Aug 2024 11:00:24 -0500 Subject: [PATCH 19/46] remove multiline comments --- common/txmgr/confirmer.go | 8 -------- 1 file changed, 8 deletions(-) diff --git a/common/txmgr/confirmer.go b/common/txmgr/confirmer.go index 1f6b8af7daa..c19f5efad6b 100644 --- a/common/txmgr/confirmer.go +++ b/common/txmgr/confirmer.go @@ -336,14 +336,6 @@ func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) pro if ec.resumeCallback != nil { mark = time.Now() - // There might be some dependency here with BCI-3573 and BCI-3730. Since we haven't enabled finality tag, - // LatestAndFinalizedBlock() is returning 0 from calculateLatestFinalized() for the first $(finalityDepth+finalityOffset) number of blocks - // and this becomes an issue for integration test (TestIntegration_AsyncEthTx). - // - // Therefore, line 349, temporarily made the block number to be the current head when finality tag disabled , - // however we removed the minConfirmation offset when fetching pending task/blocks and this can be optimistic, - // wonder if we want to merge this PR the last among the three of them in BCI-3572. - // TODO update this after BCI-3573 and BCI-3730 is merged var latestFinalizedHeadNum int64 if !ec.chainConfig.FinalityTagEnabled() { From 2a0c8f72e0398ccaaed2e6a7f57af5577e9dec58 Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Fri, 2 Aug 2024 13:25:57 -0500 Subject: [PATCH 20/46] Address PR comments --- common/txmgr/confirmer.go | 24 +----------------------- core/chains/evm/txmgr/builder.go | 6 ++---- core/chains/evm/txmgr/confirmer_test.go | 16 +++++----------- core/chains/evm/txmgr/evm_tx_store.go | 2 ++ core/chains/legacyevm/chain.go | 2 +- core/chains/legacyevm/evm_txm.go | 3 --- core/cmd/shell_local.go | 10 +--------- 7 files changed, 12 insertions(+), 51 deletions(-) diff --git a/common/txmgr/confirmer.go b/common/txmgr/confirmer.go index c19f5efad6b..115d390a9c6 100644 --- a/common/txmgr/confirmer.go +++ b/common/txmgr/confirmer.go @@ -10,8 +10,6 @@ import ( "sync" "time" - evmtypes "github.com/smartcontractkit/chainlink/v2/core/chains/evm/types" - "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" "go.uber.org/multierr" @@ -104,10 +102,6 @@ var ( }, []string{"chainID"}) ) -type confirmerHeadTracker interface { - LatestAndFinalizedBlock(ctx context.Context) (latest, finalized *evmtypes.Head, err error) -} - // Confirmer is a broad service which performs four different tasks in sequence on every new longest chain // Step 1: Mark that all currently pending transaction attempts were broadcast before this block // Step 2: Check pending transactions for receipts @@ -147,7 +141,6 @@ type Confirmer[ nConsecutiveBlocksChainTooShort int isReceiptNil func(R) bool - headTracker confirmerHeadTracker } func NewConfirmer[ @@ -171,7 +164,6 @@ func NewConfirmer[ lggr logger.Logger, isReceiptNil func(R) bool, stuckTxDetector txmgrtypes.StuckTxDetector[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], - headTracker confirmerHeadTracker, ) *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { lggr = logger.Named(lggr, "Confirmer") return &Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]{ @@ -189,7 +181,6 @@ func NewConfirmer[ mb: mailbox.NewSingle[HEAD](), isReceiptNil: isReceiptNil, stuckTxDetector: stuckTxDetector, - headTracker: headTracker, } } @@ -335,20 +326,7 @@ func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) pro if ec.resumeCallback != nil { mark = time.Now() - - // TODO update this after BCI-3573 and BCI-3730 is merged - var latestFinalizedHeadNum int64 - if !ec.chainConfig.FinalityTagEnabled() { - latestFinalizedHeadNum = head.BlockNumber() - } else if _, latestFinalizedBlock, err := ec.headTracker.LatestAndFinalizedBlock(ctx); err != nil { - return fmt.Errorf("failed to retrieve latest finalized head: %w", err) - } else { - latestFinalizedHeadNum = latestFinalizedBlock.BlockNumber() - } - - // TODO Once BCI-3574 is merged we can update the chainConfig interface to remove FinalityTagEnabled, and remove the - // head.BlockNumber() - if err := ec.ResumePendingTaskRuns(ctx, latestFinalizedHeadNum); err != nil { + if err := ec.ResumePendingTaskRuns(ctx, head.BlockNumber()); err != nil { return fmt.Errorf("ResumePendingTaskRuns failed: %w", err) } diff --git a/core/chains/evm/txmgr/builder.go b/core/chains/evm/txmgr/builder.go index 1c8c097ad5e..09a86b058f0 100644 --- a/core/chains/evm/txmgr/builder.go +++ b/core/chains/evm/txmgr/builder.go @@ -37,7 +37,6 @@ func NewTxm( logPoller logpoller.LogPoller, keyStore keystore.Eth, estimator gas.EvmFeeEstimator, - headTracker latestAndFinalizedBlockHeadTracker, ) (txm TxManager, err error, ) { @@ -59,7 +58,7 @@ func NewTxm( evmBroadcaster := NewEvmBroadcaster(txStore, txmClient, txmCfg, feeCfg, txConfig, listenerConfig, keyStore, txAttemptBuilder, lggr, checker, chainConfig.NonceAutoSync(), chainConfig.ChainType()) 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, headTracker) + evmConfirmer := NewEvmConfirmer(txStore, txmClient, txmCfg, feeCfg, txConfig, dbConfig, keyStore, txAttemptBuilder, lggr, stuckTxDetector) var evmResender *Resender if txConfig.ResendAfterThreshold() > 0 { evmResender = NewEvmResender(lggr, txStore, txmClient, evmTracker, keyStore, txmgr.DefaultResenderPollInterval, chainConfig, txConfig) @@ -118,9 +117,8 @@ func NewEvmConfirmer( txAttemptBuilder TxAttemptBuilder, lggr logger.Logger, stuckTxDetector StuckTxDetector, - headTracker latestAndFinalizedBlockHeadTracker, ) *Confirmer { - return txmgr.NewConfirmer(txStore, client, chainConfig, feeConfig, txConfig, dbConfig, keystore, txAttemptBuilder, lggr, func(r *evmtypes.Receipt) bool { return r == nil }, stuckTxDetector, headTracker) + return txmgr.NewConfirmer(txStore, client, chainConfig, feeConfig, txConfig, dbConfig, keystore, txAttemptBuilder, lggr, func(r *evmtypes.Receipt) bool { return r == nil }, stuckTxDetector) } // NewEvmTracker instantiates a new EVM tracker for abandoned transactions diff --git a/core/chains/evm/txmgr/confirmer_test.go b/core/chains/evm/txmgr/confirmer_test.go index e510246c452..6f45e613a48 100644 --- a/core/chains/evm/txmgr/confirmer_test.go +++ b/core/chains/evm/txmgr/confirmer_test.go @@ -34,7 +34,6 @@ import ( evmconfig "github.com/smartcontractkit/chainlink/v2/core/chains/evm/config" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/gas" gasmocks "github.com/smartcontractkit/chainlink/v2/core/chains/evm/gas/mocks" - "github.com/smartcontractkit/chainlink/v2/core/chains/evm/headtracker" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/keystore" ksmocks "github.com/smartcontractkit/chainlink/v2/core/chains/evm/keystore/mocks" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/testutils" @@ -132,8 +131,7 @@ func TestEthConfirmer_Lifecycle(t *testing.T) { feeEstimator := gas.NewEvmFeeEstimator(lggr, newEst, ge.EIP1559DynamicFees(), ge) txBuilder := txmgr.NewEvmTxAttemptBuilder(*ethClient.ConfiguredChainID(), ge, ethKeyStore, feeEstimator) stuckTxDetector := txmgr.NewStuckTxDetector(lggr, testutils.FixtureChainID, "", assets.NewWei(assets.NewEth(100).ToInt()), config.EVM().Transactions().AutoPurge(), feeEstimator, txStore, ethClient) - ht := headtracker.NewSimulatedHeadTracker(ethClient, true, 0) - ec := txmgr.NewEvmConfirmer(txStore, txmgr.NewEvmTxmClient(ethClient, nil), txmgr.NewEvmTxmConfig(config.EVM()), txmgr.NewEvmTxmFeeConfig(ge), config.EVM().Transactions(), gconfig.Database(), ethKeyStore, txBuilder, lggr, stuckTxDetector, ht) + ec := txmgr.NewEvmConfirmer(txStore, txmgr.NewEvmTxmClient(ethClient, nil), txmgr.NewEvmTxmConfig(config.EVM()), txmgr.NewEvmTxmFeeConfig(ge), config.EVM().Transactions(), gconfig.Database(), ethKeyStore, txBuilder, lggr, stuckTxDetector) ctx := tests.Context(t) // Can't close unstarted instance @@ -1657,9 +1655,8 @@ func TestEthConfirmer_RebroadcastWhereNecessary_WithConnectivityCheck(t *testing addresses := []gethCommon.Address{fromAddress} kst.On("EnabledAddressesForChain", mock.Anything, &cltest.FixtureChainID).Return(addresses, nil).Maybe() stuckTxDetector := txmgr.NewStuckTxDetector(lggr, testutils.FixtureChainID, "", assets.NewWei(assets.NewEth(100).ToInt()), ccfg.EVM().Transactions().AutoPurge(), feeEstimator, txStore, ethClient) - ht := headtracker.NewSimulatedHeadTracker(ethClient, true, 0) // Create confirmer with necessary state - ec := txmgr.NewEvmConfirmer(txStore, txmgr.NewEvmTxmClient(ethClient, nil), ccfg.EVM(), txmgr.NewEvmTxmFeeConfig(ccfg.EVM().GasEstimator()), ccfg.EVM().Transactions(), cfg.Database(), kst, txBuilder, lggr, stuckTxDetector, ht) + ec := txmgr.NewEvmConfirmer(txStore, txmgr.NewEvmTxmClient(ethClient, nil), ccfg.EVM(), txmgr.NewEvmTxmFeeConfig(ccfg.EVM().GasEstimator()), ccfg.EVM().Transactions(), cfg.Database(), kst, txBuilder, lggr, stuckTxDetector) servicetest.Run(t, ec) currentHead := int64(30) oldEnough := int64(15) @@ -1707,8 +1704,7 @@ func TestEthConfirmer_RebroadcastWhereNecessary_WithConnectivityCheck(t *testing addresses := []gethCommon.Address{fromAddress} kst.On("EnabledAddressesForChain", mock.Anything, &cltest.FixtureChainID).Return(addresses, nil).Maybe() stuckTxDetector := txmgr.NewStuckTxDetector(lggr, testutils.FixtureChainID, "", assets.NewWei(assets.NewEth(100).ToInt()), ccfg.EVM().Transactions().AutoPurge(), feeEstimator, txStore, ethClient) - ht := headtracker.NewSimulatedHeadTracker(ethClient, true, 0) - ec := txmgr.NewEvmConfirmer(txStore, txmgr.NewEvmTxmClient(ethClient, nil), ccfg.EVM(), txmgr.NewEvmTxmFeeConfig(ccfg.EVM().GasEstimator()), ccfg.EVM().Transactions(), cfg.Database(), kst, txBuilder, lggr, stuckTxDetector, ht) + ec := txmgr.NewEvmConfirmer(txStore, txmgr.NewEvmTxmClient(ethClient, nil), ccfg.EVM(), txmgr.NewEvmTxmFeeConfig(ccfg.EVM().GasEstimator()), ccfg.EVM().Transactions(), cfg.Database(), kst, txBuilder, lggr, stuckTxDetector) servicetest.Run(t, ec) currentHead := int64(30) oldEnough := int64(15) @@ -3161,8 +3157,7 @@ func TestEthConfirmer_ProcessStuckTransactions(t *testing.T) { ge := evmcfg.EVM().GasEstimator() txBuilder := txmgr.NewEvmTxAttemptBuilder(*ethClient.ConfiguredChainID(), ge, ethKeyStore, feeEstimator) stuckTxDetector := txmgr.NewStuckTxDetector(lggr, testutils.FixtureChainID, "", assets.NewWei(assets.NewEth(100).ToInt()), evmcfg.EVM().Transactions().AutoPurge(), feeEstimator, txStore, ethClient) - ht := headtracker.NewSimulatedHeadTracker(ethClient, true, 0) - ec := txmgr.NewEvmConfirmer(txStore, txmgr.NewEvmTxmClient(ethClient, nil), txmgr.NewEvmTxmConfig(evmcfg.EVM()), txmgr.NewEvmTxmFeeConfig(ge), evmcfg.EVM().Transactions(), cfg.Database(), ethKeyStore, txBuilder, lggr, stuckTxDetector, ht) + ec := txmgr.NewEvmConfirmer(txStore, txmgr.NewEvmTxmClient(ethClient, nil), txmgr.NewEvmTxmConfig(evmcfg.EVM()), txmgr.NewEvmTxmFeeConfig(ge), evmcfg.EVM().Transactions(), cfg.Database(), ethKeyStore, txBuilder, lggr, stuckTxDetector) servicetest.Run(t, ec) ctx := tests.Context(t) @@ -3243,8 +3238,7 @@ func newEthConfirmer(t testing.TB, txStore txmgr.EvmTxStore, ethClient client.Cl }, ge.EIP1559DynamicFees(), ge) txBuilder := txmgr.NewEvmTxAttemptBuilder(*ethClient.ConfiguredChainID(), ge, ks, estimator) stuckTxDetector := txmgr.NewStuckTxDetector(lggr, testutils.FixtureChainID, "", assets.NewWei(assets.NewEth(100).ToInt()), config.EVM().Transactions().AutoPurge(), estimator, txStore, ethClient) - ht := headtracker.NewSimulatedHeadTracker(ethClient, true, 0) - ec := txmgr.NewEvmConfirmer(txStore, txmgr.NewEvmTxmClient(ethClient, nil), txmgr.NewEvmTxmConfig(config.EVM()), txmgr.NewEvmTxmFeeConfig(ge), config.EVM().Transactions(), gconfig.Database(), ks, txBuilder, lggr, stuckTxDetector, ht) + ec := txmgr.NewEvmConfirmer(txStore, txmgr.NewEvmTxmClient(ethClient, nil), txmgr.NewEvmTxmConfig(config.EVM()), txmgr.NewEvmTxmFeeConfig(ge), config.EVM().Transactions(), gconfig.Database(), ks, txBuilder, lggr, stuckTxDetector) ec.SetResumeCallback(fn) servicetest.Run(t, ec) return ec diff --git a/core/chains/evm/txmgr/evm_tx_store.go b/core/chains/evm/txmgr/evm_tx_store.go index d7f523c1860..ed081b5de3f 100644 --- a/core/chains/evm/txmgr/evm_tx_store.go +++ b/core/chains/evm/txmgr/evm_tx_store.go @@ -1017,6 +1017,8 @@ func (o *evmTxStore) FindTxesPendingCallback(ctx context.Context, latestFinalize var cancel context.CancelFunc ctx, cancel = o.stopCh.Ctx(ctx) defer cancel() + + // TODO update the query to use tx finality state instead of use block range after Finalizer PR https://github.com/smartcontractkit/chainlink/pull/13638 merged err = o.q.SelectContext(ctx, &rs, ` SELECT evm.txes.pipeline_task_run_id, evm.receipts.receipt, COALESCE((evm.txes.meta->>'FailOnRevert')::boolean, false) "FailOnRevert" FROM evm.txes INNER JOIN evm.tx_attempts ON evm.txes.id = evm.tx_attempts.eth_tx_id diff --git a/core/chains/legacyevm/chain.go b/core/chains/legacyevm/chain.go index 68ff8d4e111..129c0318820 100644 --- a/core/chains/legacyevm/chain.go +++ b/core/chains/legacyevm/chain.go @@ -247,7 +247,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, headTracker) + txm, gasEstimator, err := newEvmTxm(opts.DS, cfg.EVM(), opts.AppConfig.EVMRPCEnabled(), opts.AppConfig.Database(), opts.AppConfig.Database().Listener(), client, l, logPoller, opts) 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 7505e344327..a6401831dcc 100644 --- a/core/chains/legacyevm/evm_txm.go +++ b/core/chains/legacyevm/evm_txm.go @@ -7,7 +7,6 @@ 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" @@ -23,7 +22,6 @@ func newEvmTxm( lggr logger.Logger, logPoller logpoller.LogPoller, opts ChainRelayExtenderConfig, - headTracker httypes.HeadTracker, ) (txm txmgr.TxManager, estimator gas.EvmFeeEstimator, err error, @@ -66,7 +64,6 @@ func newEvmTxm( logPoller, opts.KeyStore, estimator, - headTracker, ) } else { txm = opts.GenTxManager(chainID) diff --git a/core/cmd/shell_local.go b/core/cmd/shell_local.go index bbe7f46e1a5..9c0b325d865 100644 --- a/core/cmd/shell_local.go +++ b/core/cmd/shell_local.go @@ -34,12 +34,9 @@ import ( "github.com/jmoiron/sqlx" cutils "github.com/smartcontractkit/chainlink-common/pkg/utils" - "github.com/smartcontractkit/chainlink-common/pkg/utils/mailbox" - "github.com/smartcontractkit/chainlink/v2/core/build" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/assets" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/gas" - "github.com/smartcontractkit/chainlink/v2/core/chains/evm/headtracker" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/txmgr" evmtypes "github.com/smartcontractkit/chainlink/v2/core/chains/evm/types" ubig "github.com/smartcontractkit/chainlink/v2/core/chains/evm/utils/big" @@ -670,13 +667,8 @@ func (s *Shell) RebroadcastTransactions(c *cli.Context) (err error) { cfg := txmgr.NewEvmTxmConfig(chain.Config().EVM()) feeCfg := txmgr.NewEvmTxmFeeConfig(chain.Config().EVM().GasEstimator()) stuckTxDetector := txmgr.NewStuckTxDetector(lggr, ethClient.ConfiguredChainID(), "", assets.NewWei(assets.NewEth(100).ToInt()), chain.Config().EVM().Transactions().AutoPurge(), nil, orm, ethClient) - htConfig := chain.Config().EVM().HeadTracker() - hb := headtracker.NewHeadBroadcaster(lggr) - htORM := headtracker.NewORM(*chain.ID(), db) - hs := headtracker.NewHeadSaver(lggr, htORM, chain.Config().EVM(), htConfig) - ht := headtracker.NewHeadTracker(lggr, ethClient, chain.Config().EVM(), htConfig, hb, hs, mailbox.NewMonitor(app.ID().String(), lggr)) ec := txmgr.NewEvmConfirmer(orm, txmgr.NewEvmTxmClient(ethClient, chain.Config().EVM().NodePool().Errors()), - cfg, feeCfg, chain.Config().EVM().Transactions(), app.GetConfig().Database(), keyStore.Eth(), txBuilder, chain.Logger(), stuckTxDetector, ht) + cfg, feeCfg, chain.Config().EVM().Transactions(), app.GetConfig().Database(), keyStore.Eth(), txBuilder, chain.Logger(), stuckTxDetector) totalNonces := endingNonce - beginningNonce + 1 nonces := make([]evmtypes.Nonce, totalNonces) for i := int64(0); i < totalNonces; i++ { From 1370e51e14627aa4ccec1815316ae2a7edc98cbc Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Fri, 2 Aug 2024 14:03:45 -0500 Subject: [PATCH 21/46] fix test, update mock --- common/txmgr/confirmer.go | 4 +- common/txmgr/types/config.go | 1 - common/txmgr/types/mocks/tx_store.go | 20 ++++---- common/txmgr/types/tx_store.go | 4 +- core/chains/evm/txmgr/confirmer_test.go | 55 +++++---------------- core/chains/evm/txmgr/evm_tx_store.go | 4 +- core/chains/evm/txmgr/mocks/evm_tx_store.go | 20 ++++---- core/chains/evm/txmgr/txmgr_test.go | 3 +- 8 files changed, 40 insertions(+), 71 deletions(-) diff --git a/common/txmgr/confirmer.go b/common/txmgr/confirmer.go index 115d390a9c6..643bd134dd1 100644 --- a/common/txmgr/confirmer.go +++ b/common/txmgr/confirmer.go @@ -1188,8 +1188,8 @@ func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) sen } // ResumePendingTaskRuns issues callbacks to task runs that are pending waiting for receipts -func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) ResumePendingTaskRuns(ctx context.Context, latestFinalizedBlockNum int64) error { - receiptsPlus, err := ec.txStore.FindTxesPendingCallback(ctx, latestFinalizedBlockNum, ec.chainID) +func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) ResumePendingTaskRuns(ctx context.Context, headNum int64) error { + receiptsPlus, err := ec.txStore.FindTxesPendingCallback(ctx, headNum, ec.chainID) if err != nil { return err } diff --git a/common/txmgr/types/config.go b/common/txmgr/types/config.go index 013169184e9..4d9af5f0673 100644 --- a/common/txmgr/types/config.go +++ b/common/txmgr/types/config.go @@ -51,7 +51,6 @@ type ConfirmerFeeConfig interface { } type ConfirmerChainConfig interface { - FinalityTagEnabled() bool RPCDefaultBatchSize() uint32 FinalityDepth() uint32 } diff --git a/common/txmgr/types/mocks/tx_store.go b/common/txmgr/types/mocks/tx_store.go index 0682259feda..ee166638e34 100644 --- a/common/txmgr/types/mocks/tx_store.go +++ b/common/txmgr/types/mocks/tx_store.go @@ -1096,9 +1096,9 @@ func (_c *TxStore_FindTxesByMetaFieldAndStates_Call[ADDR, CHAIN_ID, TX_HASH, BLO return _c } -// FindTxesPendingCallback provides a mock function with given fields: ctx, latestFinalizedBlockNum, chainID -func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxesPendingCallback(ctx context.Context, latestFinalizedBlockNum int64, chainID CHAIN_ID) ([]txmgrtypes.ReceiptPlus[R], error) { - ret := _m.Called(ctx, latestFinalizedBlockNum, chainID) +// FindTxesPendingCallback provides a mock function with given fields: ctx, blockNum, chainID +func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxesPendingCallback(ctx context.Context, blockNum int64, chainID CHAIN_ID) ([]txmgrtypes.ReceiptPlus[R], error) { + ret := _m.Called(ctx, blockNum, chainID) if len(ret) == 0 { panic("no return value specified for FindTxesPendingCallback") @@ -1107,10 +1107,10 @@ func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxesPen var r0 []txmgrtypes.ReceiptPlus[R] var r1 error if rf, ok := ret.Get(0).(func(context.Context, int64, CHAIN_ID) ([]txmgrtypes.ReceiptPlus[R], error)); ok { - return rf(ctx, latestFinalizedBlockNum, chainID) + return rf(ctx, blockNum, chainID) } if rf, ok := ret.Get(0).(func(context.Context, int64, CHAIN_ID) []txmgrtypes.ReceiptPlus[R]); ok { - r0 = rf(ctx, latestFinalizedBlockNum, chainID) + r0 = rf(ctx, blockNum, chainID) } else { if ret.Get(0) != nil { r0 = ret.Get(0).([]txmgrtypes.ReceiptPlus[R]) @@ -1118,7 +1118,7 @@ func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxesPen } if rf, ok := ret.Get(1).(func(context.Context, int64, CHAIN_ID) error); ok { - r1 = rf(ctx, latestFinalizedBlockNum, chainID) + r1 = rf(ctx, blockNum, chainID) } else { r1 = ret.Error(1) } @@ -1133,13 +1133,13 @@ type TxStore_FindTxesPendingCallback_Call[ADDR types.Hashable, CHAIN_ID types.ID // FindTxesPendingCallback is a helper method to define mock.On call // - ctx context.Context -// - latestFinalizedBlockNum int64 +// - blockNum int64 // - chainID CHAIN_ID -func (_e *TxStore_Expecter[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxesPendingCallback(ctx interface{}, latestFinalizedBlockNum interface{}, chainID interface{}) *TxStore_FindTxesPendingCallback_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { - return &TxStore_FindTxesPendingCallback_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]{Call: _e.mock.On("FindTxesPendingCallback", ctx, latestFinalizedBlockNum, chainID)} +func (_e *TxStore_Expecter[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxesPendingCallback(ctx interface{}, blockNum interface{}, chainID interface{}) *TxStore_FindTxesPendingCallback_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { + return &TxStore_FindTxesPendingCallback_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]{Call: _e.mock.On("FindTxesPendingCallback", ctx, blockNum, chainID)} } -func (_c *TxStore_FindTxesPendingCallback_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) Run(run func(ctx context.Context, latestFinalizedBlockNum int64, chainID CHAIN_ID)) *TxStore_FindTxesPendingCallback_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { +func (_c *TxStore_FindTxesPendingCallback_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) Run(run func(ctx context.Context, blockNum int64, chainID CHAIN_ID)) *TxStore_FindTxesPendingCallback_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].(CHAIN_ID)) }) diff --git a/common/txmgr/types/tx_store.go b/common/txmgr/types/tx_store.go index 3546c1af6f2..cc80cb092e3 100644 --- a/common/txmgr/types/tx_store.go +++ b/common/txmgr/types/tx_store.go @@ -33,8 +33,8 @@ type TxStore[ TxHistoryReaper[CHAIN_ID] TransactionStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, SEQ, FEE] - // Find confirmed txes before the LatestFinalizedBlockNum param that require callback but have not yet been signaled - FindTxesPendingCallback(ctx context.Context, latestFinalizedBlockNum int64, chainID CHAIN_ID) (receiptsPlus []ReceiptPlus[R], err error) + // Find confirmed txes before the blockNum param that require callback but have not yet been signaled + FindTxesPendingCallback(ctx context.Context, blockNum int64, chainID CHAIN_ID) (receiptsPlus []ReceiptPlus[R], err error) // Update tx to mark that its callback has been signaled UpdateTxCallbackCompleted(ctx context.Context, pipelineTaskRunRid uuid.UUID, chainId CHAIN_ID) error SaveFetchedReceipts(ctx context.Context, r []R, state TxState, errorMsg *string, chainID CHAIN_ID) error diff --git a/core/chains/evm/txmgr/confirmer_test.go b/core/chains/evm/txmgr/confirmer_test.go index 6f45e613a48..c22419efd39 100644 --- a/core/chains/evm/txmgr/confirmer_test.go +++ b/core/chains/evm/txmgr/confirmer_test.go @@ -145,21 +145,13 @@ func TestEthConfirmer_Lifecycle(t *testing.T) { // Can't start an already started instance err = ec.Start(ctx) require.Error(t, err) - - latestFinalizedHead := evmtypes.Head{ - Number: 8, - Hash: testutils.NewHash(), - Parent: nil, - IsFinalized: true, // We are guaranteed to receive a latestFinalizedHead. - } - head := evmtypes.Head{ Hash: testutils.NewHash(), Number: 10, Parent: &evmtypes.Head{ Hash: testutils.NewHash(), - Number: 9, - Parent: &latestFinalizedHead, + Number: 8, + Parent: nil, }, } err = ec.ProcessHead(ctx, &head) @@ -2948,21 +2940,13 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { ethClient := testutils.NewEthClientMockWithDefaultChain(t) evmcfg := evmtest.NewChainScopedConfig(t, config) - - latestFinalizedHead := evmtypes.Head{ - Number: 8, - Hash: testutils.NewHash(), - Parent: nil, - IsFinalized: true, // We are guaranteed to receive a latestFinalizedHead. - } - head := evmtypes.Head{ Hash: testutils.NewHash(), Number: 10, Parent: &evmtypes.Head{ Hash: testutils.NewHash(), Number: 9, - Parent: &latestFinalizedHead, + Parent: nil, }, } pgtest.MustExec(t, db, `SET CONSTRAINTS fk_pipeline_runs_pruning_key DEFERRED`) @@ -2978,29 +2962,16 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { tr := cltest.MustInsertUnfinishedPipelineTaskRun(t, db, run.ID) etx := cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, 1, 1, fromAddress) - mustInsertEthReceipt(t, txStore, latestFinalizedHead.BlockNumber(), head.Hash, etx.TxAttempts[0].Hash) + mustInsertEthReceipt(t, txStore, head.Number, head.Hash, etx.TxAttempts[0].Hash) // Setting both signal_callback and callback_completed to TRUE to simulate a completed pipeline task // It would only be in a state past suspended if the resume callback was called and callback_completed was set to TRUE pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE, callback_completed = TRUE WHERE id = $2`, &tr.ID, etx.ID) - err := ec.ResumePendingTaskRuns(tests.Context(t), latestFinalizedHead.BlockNumber()) - require.NoError(t, err) - }) - - t.Run("doesn't process task runs where the receipt is before LatestFinalizedBlockNum", func(t *testing.T) { - run := cltest.MustInsertPipelineRun(t, db) - tr := cltest.MustInsertUnfinishedPipelineTaskRun(t, db, run.ID) - - etx := cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, 2, 1, fromAddress) - mustInsertEthReceipt(t, txStore, head.Number, head.Hash, etx.TxAttempts[0].Hash) - - pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE WHERE id = $2`, &tr.ID, etx.ID) - - err := ec.ResumePendingTaskRuns(tests.Context(t), latestFinalizedHead.BlockNumber()) + err := ec.ResumePendingTaskRuns(tests.Context(t), head.Number) require.NoError(t, err) }) - t.Run("processes eth_txes with receipts after LatestFinalizedBlockNum", func(t *testing.T) { + t.Run("processes eth_txes with receipts after headNum", func(t *testing.T) { ch := make(chan interface{}) nonce := evmtypes.Nonce(3) var err error @@ -3016,7 +2987,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { etx := cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, int64(nonce), 1, fromAddress) pgtest.MustExec(t, db, `UPDATE evm.txes SET meta='{"FailOnRevert": true}'`) - receipt := mustInsertEthReceipt(t, txStore, latestFinalizedHead.BlockNumber(), head.Hash, etx.TxAttempts[0].Hash) + receipt := mustInsertEthReceipt(t, txStore, head.Number, head.Hash, etx.TxAttempts[0].Hash) pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE WHERE id = $2`, &tr.ID, etx.ID) @@ -3024,7 +2995,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { t.Cleanup(func() { <-done }) go func() { defer close(done) - err2 := ec.ResumePendingTaskRuns(tests.Context(t), latestFinalizedHead.BlockNumber()) + err2 := ec.ResumePendingTaskRuns(tests.Context(t), head.Number) if !assert.NoError(t, err2) { return } @@ -3050,7 +3021,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { pgtest.MustExec(t, db, `DELETE FROM pipeline_runs`) - t.Run("processes eth_txes with receipt before LatestFinalizedBlockNum that reverted", func(t *testing.T) { + t.Run("processes eth_txes with receipt before latest head that reverted", func(t *testing.T) { type data struct { value any error @@ -3070,7 +3041,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { pgtest.MustExec(t, db, `UPDATE evm.txes SET meta='{"FailOnRevert": true}'`) // receipt is not passed through as a value since it reverted and caused an error - mustInsertRevertedEthReceipt(t, txStore, latestFinalizedHead.BlockNumber()-2, head.Hash, etx.TxAttempts[0].Hash) + mustInsertRevertedEthReceipt(t, txStore, head.Number-2, head.Hash, etx.TxAttempts[0].Hash) pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE WHERE id = $2`, &tr.ID, etx.ID) @@ -3078,7 +3049,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { t.Cleanup(func() { <-done }) go func() { defer close(done) - err2 := ec.ResumePendingTaskRuns(tests.Context(t), latestFinalizedHead.BlockNumber()) + err2 := ec.ResumePendingTaskRuns(tests.Context(t), head.Number) if !assert.NoError(t, err2) { return } @@ -3112,10 +3083,10 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { tr := cltest.MustInsertUnfinishedPipelineTaskRun(t, db, run.ID) etx := cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, int64(nonce), 1, fromAddress) - mustInsertEthReceipt(t, txStore, latestFinalizedHead.BlockNumber(), head.Hash, etx.TxAttempts[0].Hash) + mustInsertEthReceipt(t, txStore, head.Number, head.Hash, etx.TxAttempts[0].Hash) pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE WHERE id = $2`, &tr.ID, etx.ID) - err := ec.ResumePendingTaskRuns(tests.Context(t), latestFinalizedHead.BlockNumber()) + err := ec.ResumePendingTaskRuns(tests.Context(t), head.Number) require.Error(t, err) // Retrieve Tx to check if callback completed flag was left unchanged diff --git a/core/chains/evm/txmgr/evm_tx_store.go b/core/chains/evm/txmgr/evm_tx_store.go index ed081b5de3f..b5ed1fa35de 100644 --- a/core/chains/evm/txmgr/evm_tx_store.go +++ b/core/chains/evm/txmgr/evm_tx_store.go @@ -1011,7 +1011,7 @@ WHERE evm.tx_attempts.state = 'in_progress' AND evm.txes.from_address = $1 AND e } // Find confirmed txes requiring callback but have not yet been signaled -func (o *evmTxStore) FindTxesPendingCallback(ctx context.Context, latestFinalizedBlockNum int64, chainID *big.Int) (receiptsPlus []ReceiptPlus, err error) { +func (o *evmTxStore) FindTxesPendingCallback(ctx context.Context, blockNum int64, chainID *big.Int) (receiptsPlus []ReceiptPlus, err error) { var rs []dbReceiptPlus var cancel context.CancelFunc @@ -1025,7 +1025,7 @@ func (o *evmTxStore) FindTxesPendingCallback(ctx context.Context, latestFinalize INNER JOIN evm.receipts ON evm.tx_attempts.hash = evm.receipts.tx_hash WHERE evm.txes.pipeline_task_run_id IS NOT NULL AND evm.txes.signal_callback = TRUE AND evm.txes.callback_completed = FALSE AND evm.receipts.block_number <= $1 AND evm.txes.evm_chain_id = $2 - `, latestFinalizedBlockNum, chainID.String()) + `, blockNum, chainID.String()) if err != nil { return nil, fmt.Errorf("failed to retrieve transactions pending pipeline resume callback: %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 e1cbe8e4dd0..b28e55ec324 100644 --- a/core/chains/evm/txmgr/mocks/evm_tx_store.go +++ b/core/chains/evm/txmgr/mocks/evm_tx_store.go @@ -1333,9 +1333,9 @@ func (_c *EvmTxStore_FindTxesByMetaFieldAndStates_Call) RunAndReturn(run func(co return _c } -// FindTxesPendingCallback provides a mock function with given fields: ctx, latestFinalizedBlockNum, chainID -func (_m *EvmTxStore) FindTxesPendingCallback(ctx context.Context, latestFinalizedBlockNum int64, chainID *big.Int) ([]types.ReceiptPlus[*evmtypes.Receipt], error) { - ret := _m.Called(ctx, latestFinalizedBlockNum, chainID) +// FindTxesPendingCallback provides a mock function with given fields: ctx, blockNum, chainID +func (_m *EvmTxStore) FindTxesPendingCallback(ctx context.Context, blockNum int64, chainID *big.Int) ([]types.ReceiptPlus[*evmtypes.Receipt], error) { + ret := _m.Called(ctx, blockNum, chainID) if len(ret) == 0 { panic("no return value specified for FindTxesPendingCallback") @@ -1344,10 +1344,10 @@ func (_m *EvmTxStore) FindTxesPendingCallback(ctx context.Context, latestFinaliz var r0 []types.ReceiptPlus[*evmtypes.Receipt] var r1 error if rf, ok := ret.Get(0).(func(context.Context, int64, *big.Int) ([]types.ReceiptPlus[*evmtypes.Receipt], error)); ok { - return rf(ctx, latestFinalizedBlockNum, chainID) + return rf(ctx, blockNum, chainID) } if rf, ok := ret.Get(0).(func(context.Context, int64, *big.Int) []types.ReceiptPlus[*evmtypes.Receipt]); ok { - r0 = rf(ctx, latestFinalizedBlockNum, chainID) + r0 = rf(ctx, blockNum, chainID) } else { if ret.Get(0) != nil { r0 = ret.Get(0).([]types.ReceiptPlus[*evmtypes.Receipt]) @@ -1355,7 +1355,7 @@ func (_m *EvmTxStore) FindTxesPendingCallback(ctx context.Context, latestFinaliz } if rf, ok := ret.Get(1).(func(context.Context, int64, *big.Int) error); ok { - r1 = rf(ctx, latestFinalizedBlockNum, chainID) + r1 = rf(ctx, blockNum, chainID) } else { r1 = ret.Error(1) } @@ -1370,13 +1370,13 @@ type EvmTxStore_FindTxesPendingCallback_Call struct { // FindTxesPendingCallback is a helper method to define mock.On call // - ctx context.Context -// - latestFinalizedBlockNum int64 +// - blockNum int64 // - chainID *big.Int -func (_e *EvmTxStore_Expecter) FindTxesPendingCallback(ctx interface{}, latestFinalizedBlockNum interface{}, chainID interface{}) *EvmTxStore_FindTxesPendingCallback_Call { - return &EvmTxStore_FindTxesPendingCallback_Call{Call: _e.mock.On("FindTxesPendingCallback", ctx, latestFinalizedBlockNum, chainID)} +func (_e *EvmTxStore_Expecter) FindTxesPendingCallback(ctx interface{}, blockNum interface{}, chainID interface{}) *EvmTxStore_FindTxesPendingCallback_Call { + return &EvmTxStore_FindTxesPendingCallback_Call{Call: _e.mock.On("FindTxesPendingCallback", ctx, blockNum, chainID)} } -func (_c *EvmTxStore_FindTxesPendingCallback_Call) Run(run func(ctx context.Context, latestFinalizedBlockNum int64, chainID *big.Int)) *EvmTxStore_FindTxesPendingCallback_Call { +func (_c *EvmTxStore_FindTxesPendingCallback_Call) Run(run func(ctx context.Context, blockNum int64, chainID *big.Int)) *EvmTxStore_FindTxesPendingCallback_Call { _c.Call.Run(func(args mock.Arguments) { run(args[0].(context.Context), args[1].(int64), args[2].(*big.Int)) }) diff --git a/core/chains/evm/txmgr/txmgr_test.go b/core/chains/evm/txmgr/txmgr_test.go index a9f3e1d549a..40df5616c99 100644 --- a/core/chains/evm/txmgr/txmgr_test.go +++ b/core/chains/evm/txmgr/txmgr_test.go @@ -85,8 +85,7 @@ func makeTestEvmTxm( lggr, lp, keyStore, - estimator, - ht) + estimator) } func TestTxm_SendNativeToken_DoesNotSendToZero(t *testing.T) { From 77d5a555398f2dd40f0493d2baba22b998ba4a98 Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Fri, 2 Aug 2024 14:14:30 -0500 Subject: [PATCH 22/46] fix lint, and remove unused --- core/chains/evm/txmgr/builder.go | 5 ----- core/services/vrf/delegate_test.go | 2 +- 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/core/chains/evm/txmgr/builder.go b/core/chains/evm/txmgr/builder.go index 09a86b058f0..8234d55b960 100644 --- a/core/chains/evm/txmgr/builder.go +++ b/core/chains/evm/txmgr/builder.go @@ -1,7 +1,6 @@ package txmgr import ( - "context" "math/big" "time" @@ -19,10 +18,6 @@ import ( evmtypes "github.com/smartcontractkit/chainlink/v2/core/chains/evm/types" ) -type latestAndFinalizedBlockHeadTracker interface { - LatestAndFinalizedBlock(ctx context.Context) (latest, finalized *evmtypes.Head, err error) -} - // NewTxm constructs the necessary dependencies for the EvmTxm (broadcaster, confirmer, etc) and returns a new EvmTxManager func NewTxm( ds sqlutil.DataSource, diff --git a/core/services/vrf/delegate_test.go b/core/services/vrf/delegate_test.go index 9718dc376a7..889b19d0e04 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, nil) + txm, err := txmgr.NewTxm(db, evmConfig, evmConfig.GasEstimator(), evmConfig.Transactions(), nil, dbConfig, dbConfig.Listener(), ec, logger.TestLogger(t), nil, ks.Eth(), 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 424253b70afc1a175e226f915f4b75b53e361885 Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Fri, 2 Aug 2024 14:17:12 -0500 Subject: [PATCH 23/46] fix test --- core/services/promreporter/prom_reporter_test.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/services/promreporter/prom_reporter_test.go b/core/services/promreporter/prom_reporter_test.go index a0a4a247c21..b61fa25bdc4 100644 --- a/core/services/promreporter/prom_reporter_test.go +++ b/core/services/promreporter/prom_reporter_test.go @@ -62,8 +62,7 @@ func newLegacyChainContainer(t *testing.T, db *sqlx.DB) legacyevm.LegacyChainCon lggr, lp, keyStore, - estimator, - ht) + estimator) require.NoError(t, err) cfg := configtest.NewGeneralConfig(t, nil) From 0783aab2024cd5327743f42f910172b233ccbbd6 Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Fri, 2 Aug 2024 21:01:45 -0500 Subject: [PATCH 24/46] update tests --- common/txmgr/types/tx_store.go | 2 +- core/chains/evm/txmgr/evm_tx_store_test.go | 23 ++++++++++------------ 2 files changed, 11 insertions(+), 14 deletions(-) diff --git a/common/txmgr/types/tx_store.go b/common/txmgr/types/tx_store.go index cc80cb092e3..e786593a60a 100644 --- a/common/txmgr/types/tx_store.go +++ b/common/txmgr/types/tx_store.go @@ -33,7 +33,7 @@ type TxStore[ TxHistoryReaper[CHAIN_ID] TransactionStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, SEQ, FEE] - // Find confirmed txes before the blockNum param that require callback but have not yet been signaled + // Find confirmed txes with finalized state that require callback but have not yet been signaled FindTxesPendingCallback(ctx context.Context, blockNum int64, chainID CHAIN_ID) (receiptsPlus []ReceiptPlus[R], err error) // Update tx to mark that its callback has been signaled UpdateTxCallbackCompleted(ctx context.Context, pipelineTaskRunRid uuid.UUID, chainId CHAIN_ID) error diff --git a/core/chains/evm/txmgr/evm_tx_store_test.go b/core/chains/evm/txmgr/evm_tx_store_test.go index 26cc98e6ef5..c4933947b62 100644 --- a/core/chains/evm/txmgr/evm_tx_store_test.go +++ b/core/chains/evm/txmgr/evm_tx_store_test.go @@ -628,20 +628,17 @@ func TestORM_FindTxesPendingCallback(t *testing.T) { pgtest.MustExec(t, db, `SET CONSTRAINTS fk_pipeline_runs_pruning_key DEFERRED`) pgtest.MustExec(t, db, `SET CONSTRAINTS pipeline_runs_pipeline_spec_id_fkey DEFERRED`) - latestFinalizedHead := evmtypes.Head{ - Number: 8, - Hash: utils.NewHash(), - Parent: nil, - IsFinalized: true, - } - head := evmtypes.Head{ Hash: utils.NewHash(), Number: 10, Parent: &evmtypes.Head{ Hash: utils.NewHash(), Number: 9, - Parent: &latestFinalizedHead, + Parent: &evmtypes.Head{ + Number: 8, + Hash: utils.NewHash(), + Parent: nil, + }, }, } @@ -652,7 +649,7 @@ func TestORM_FindTxesPendingCallback(t *testing.T) { etx1 := cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, 3, 1, fromAddress) pgtest.MustExec(t, db, `UPDATE evm.txes SET meta='{"FailOnRevert": true}'`) attempt1 := etx1.TxAttempts[0] - mustInsertEthReceipt(t, txStore, latestFinalizedHead.BlockNumber(), head.Hash, attempt1.Hash) + mustInsertEthReceipt(t, txStore, head.BlockNumber(), head.Hash, attempt1.Hash) pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE WHERE id = $2`, &tr1.ID, etx1.ID) // Callback to pipeline service completed. Should be ignored @@ -661,10 +658,10 @@ func TestORM_FindTxesPendingCallback(t *testing.T) { etx2 := cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, 4, 1, fromAddress) pgtest.MustExec(t, db, `UPDATE evm.txes SET meta='{"FailOnRevert": false}'`) attempt2 := etx2.TxAttempts[0] - mustInsertEthReceipt(t, txStore, latestFinalizedHead.BlockNumber()-2, head.Hash, attempt2.Hash) + mustInsertEthReceipt(t, txStore, head.BlockNumber()-2, head.Hash, attempt2.Hash) pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE, callback_completed = TRUE WHERE id = $2`, &tr2.ID, etx2.ID) - // Suspend run after LatestFinalizedBlockNum. Should be ignored + // Suspend run after head block number. Should be ignored run3 := cltest.MustInsertPipelineRun(t, db) tr3 := cltest.MustInsertUnfinishedPipelineTaskRun(t, db, run3.ID) pgtest.MustExec(t, db, `UPDATE pipeline_runs SET state = 'suspended' WHERE id = $1`, run3.ID) @@ -683,9 +680,9 @@ func TestORM_FindTxesPendingCallback(t *testing.T) { cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, 7, 1, fromAddress) // Search evm.txes table for tx requiring callback - receiptsPlus, err := txStore.FindTxesPendingCallback(tests.Context(t), latestFinalizedHead.BlockNumber(), ethClient.ConfiguredChainID()) + receiptsPlus, err := txStore.FindTxesPendingCallback(tests.Context(t), head.BlockNumber(), ethClient.ConfiguredChainID()) require.NoError(t, err) - assert.Len(t, receiptsPlus, 1) + assert.Len(t, receiptsPlus, 2) assert.Equal(t, tr1.ID, receiptsPlus[0].ID) } From 3c9b9651bc7dcf66d36f1e4247cc29374cbb124c Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Fri, 2 Aug 2024 23:36:10 -0500 Subject: [PATCH 25/46] cleanup more test --- core/chains/evm/txmgr/confirmer_test.go | 42 ++++++++++++++----------- 1 file changed, 24 insertions(+), 18 deletions(-) diff --git a/core/chains/evm/txmgr/confirmer_test.go b/core/chains/evm/txmgr/confirmer_test.go index c22419efd39..1f336d2bbd7 100644 --- a/core/chains/evm/txmgr/confirmer_test.go +++ b/core/chains/evm/txmgr/confirmer_test.go @@ -150,8 +150,12 @@ func TestEthConfirmer_Lifecycle(t *testing.T) { Number: 10, Parent: &evmtypes.Head{ Hash: testutils.NewHash(), - Number: 8, - Parent: nil, + Number: 9, + Parent: &evmtypes.Head{ + Number: 8, + Hash: testutils.NewHash(), + Parent: nil, + }, }, } err = ec.ProcessHead(ctx, &head) @@ -2946,18 +2950,22 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { Parent: &evmtypes.Head{ Hash: testutils.NewHash(), Number: 9, - Parent: nil, + Parent: &evmtypes.Head{ + Number: 8, + Hash: testutils.NewHash(), + Parent: nil, + }, }, } pgtest.MustExec(t, db, `SET CONSTRAINTS fk_pipeline_runs_pruning_key DEFERRED`) pgtest.MustExec(t, db, `SET CONSTRAINTS pipeline_runs_pipeline_spec_id_fkey DEFERRED`) - ec := newEthConfirmer(t, txStore, ethClient, config, evmcfg, ethKeyStore, func(context.Context, uuid.UUID, interface{}, error) error { - t.Fatal("No value expected") - return nil - }) - t.Run("doesn't process task runs that are not suspended (possibly already previously resumed)", func(t *testing.T) { + ec := newEthConfirmer(t, txStore, ethClient, config, evmcfg, ethKeyStore, func(context.Context, uuid.UUID, interface{}, error) error { + t.Fatal("No value expected") + return nil + }) + run := cltest.MustInsertPipelineRun(t, db) tr := cltest.MustInsertUnfinishedPipelineTaskRun(t, db, run.ID) @@ -2971,11 +2979,11 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { require.NoError(t, err) }) - t.Run("processes eth_txes with receipts after headNum", func(t *testing.T) { + t.Run("processes eth_txes with receipts older than minConfirmations", func(t *testing.T) { ch := make(chan interface{}) nonce := evmtypes.Nonce(3) var err error - ec = newEthConfirmer(t, txStore, ethClient, config, evmcfg, ethKeyStore, func(ctx context.Context, id uuid.UUID, value interface{}, thisErr error) error { + ec := newEthConfirmer(t, txStore, ethClient, config, evmcfg, ethKeyStore, func(ctx context.Context, id uuid.UUID, value interface{}, thisErr error) error { err = thisErr ch <- value return nil @@ -3028,7 +3036,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { } ch := make(chan data) nonce := evmtypes.Nonce(4) - ec = newEthConfirmer(t, txStore, ethClient, config, evmcfg, ethKeyStore, func(ctx context.Context, id uuid.UUID, value interface{}, err error) error { + ec := newEthConfirmer(t, txStore, ethClient, config, evmcfg, ethKeyStore, func(ctx context.Context, id uuid.UUID, value interface{}, err error) error { ch <- data{value, err} return nil }) @@ -3075,7 +3083,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { t.Run("does not mark callback complete if callback fails", func(t *testing.T) { nonce := evmtypes.Nonce(5) - ec = newEthConfirmer(t, txStore, ethClient, config, evmcfg, ethKeyStore, func(context.Context, uuid.UUID, interface{}, error) error { + ec := newEthConfirmer(t, txStore, ethClient, config, evmcfg, ethKeyStore, func(context.Context, uuid.UUID, interface{}, error) error { return errors.New("error") }) @@ -3142,9 +3150,8 @@ func TestEthConfirmer_ProcessStuckTransactions(t *testing.T) { tx := mustInsertUnconfirmedTxWithBroadcastAttempts(t, txStore, nonce, fromAddress, autoPurgeMinAttempts, blockNum-int64(autoPurgeThreshold), marketGasPrice.Add(oneGwei)) head := evmtypes.Head{ - Hash: testutils.NewHash(), - Number: blockNum, - IsFinalized: true, + Hash: testutils.NewHash(), + Number: blockNum, } ethClient.On("SequenceAt", mock.Anything, mock.Anything, mock.Anything).Return(evmtypes.Nonce(0), nil).Once() ethClient.On("BatchCallContext", mock.Anything, mock.Anything).Return(nil).Once() @@ -3167,9 +3174,8 @@ func TestEthConfirmer_ProcessStuckTransactions(t *testing.T) { require.Equal(t, bumpedFee.Legacy, latestAttempt.TxFee.Legacy) head = evmtypes.Head{ - Hash: testutils.NewHash(), - Number: blockNum + 1, - IsFinalized: true, + Hash: testutils.NewHash(), + Number: blockNum + 1, } ethClient.On("SequenceAt", mock.Anything, mock.Anything, mock.Anything).Return(evmtypes.Nonce(1), nil) ethClient.On("BatchCallContext", mock.Anything, mock.MatchedBy(func(b []rpc.BatchElem) bool { From 5818d5ce3b6c413aecc72ce46974baebaa1bf19f Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Fri, 2 Aug 2024 23:38:27 -0500 Subject: [PATCH 26/46] formatting --- core/chains/evm/txmgr/confirmer_test.go | 1 + core/chains/evm/txmgr/evm_tx_store_test.go | 2 +- core/chains/legacyevm/evm_txm.go | 3 +-- core/cmd/shell_local.go | 1 + 4 files changed, 4 insertions(+), 3 deletions(-) diff --git a/core/chains/evm/txmgr/confirmer_test.go b/core/chains/evm/txmgr/confirmer_test.go index 1f336d2bbd7..62a9428e6b6 100644 --- a/core/chains/evm/txmgr/confirmer_test.go +++ b/core/chains/evm/txmgr/confirmer_test.go @@ -2944,6 +2944,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { ethClient := testutils.NewEthClientMockWithDefaultChain(t) evmcfg := evmtest.NewChainScopedConfig(t, config) + head := evmtypes.Head{ Hash: testutils.NewHash(), Number: 10, diff --git a/core/chains/evm/txmgr/evm_tx_store_test.go b/core/chains/evm/txmgr/evm_tx_store_test.go index c4933947b62..5355376d31c 100644 --- a/core/chains/evm/txmgr/evm_tx_store_test.go +++ b/core/chains/evm/txmgr/evm_tx_store_test.go @@ -680,7 +680,7 @@ func TestORM_FindTxesPendingCallback(t *testing.T) { cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, 7, 1, fromAddress) // Search evm.txes table for tx requiring callback - receiptsPlus, err := txStore.FindTxesPendingCallback(tests.Context(t), head.BlockNumber(), ethClient.ConfiguredChainID()) + receiptsPlus, err := txStore.FindTxesPendingCallback(tests.Context(t), head.Number, ethClient.ConfiguredChainID()) require.NoError(t, err) assert.Len(t, receiptsPlus, 2) assert.Equal(t, tr1.ID, receiptsPlus[0].ID) diff --git a/core/chains/legacyevm/evm_txm.go b/core/chains/legacyevm/evm_txm.go index a6401831dcc..cecfd4ffafe 100644 --- a/core/chains/legacyevm/evm_txm.go +++ b/core/chains/legacyevm/evm_txm.go @@ -63,8 +63,7 @@ func newEvmTxm( lggr, logPoller, opts.KeyStore, - estimator, - ) + estimator) } else { txm = opts.GenTxManager(chainID) } diff --git a/core/cmd/shell_local.go b/core/cmd/shell_local.go index 9c0b325d865..e19cc485d8b 100644 --- a/core/cmd/shell_local.go +++ b/core/cmd/shell_local.go @@ -34,6 +34,7 @@ import ( "github.com/jmoiron/sqlx" cutils "github.com/smartcontractkit/chainlink-common/pkg/utils" + "github.com/smartcontractkit/chainlink/v2/core/build" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/assets" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/gas" From daab2d63e2ebd9f49f652ea911078e86cad09577 Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Sat, 3 Aug 2024 12:42:24 -0500 Subject: [PATCH 27/46] rename --- core/chains/evm/txmgr/confirmer_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/chains/evm/txmgr/confirmer_test.go b/core/chains/evm/txmgr/confirmer_test.go index 62a9428e6b6..36ca77140e7 100644 --- a/core/chains/evm/txmgr/confirmer_test.go +++ b/core/chains/evm/txmgr/confirmer_test.go @@ -3030,7 +3030,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { pgtest.MustExec(t, db, `DELETE FROM pipeline_runs`) - t.Run("processes eth_txes with receipt before latest head that reverted", func(t *testing.T) { + t.Run("processes eth_txes with receipt before current head that reverted", func(t *testing.T) { type data struct { value any error From ca6a2d48fb38edbfbc418b7aa01d7b94f8a9fef7 Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Tue, 6 Aug 2024 15:59:41 -0500 Subject: [PATCH 28/46] update after finalizer merged --- common/txmgr/confirmer.go | 6 ++--- common/txmgr/types/mocks/tx_store.go | 29 ++++++++++----------- common/txmgr/types/tx_store.go | 2 +- core/chains/evm/txmgr/confirmer_test.go | 8 +++--- core/chains/evm/txmgr/evm_tx_store.go | 7 +++-- core/chains/evm/txmgr/evm_tx_store_test.go | 2 +- core/chains/evm/txmgr/mocks/evm_tx_store.go | 29 ++++++++++----------- 7 files changed, 40 insertions(+), 43 deletions(-) diff --git a/common/txmgr/confirmer.go b/common/txmgr/confirmer.go index 643bd134dd1..35c1117c2e0 100644 --- a/common/txmgr/confirmer.go +++ b/common/txmgr/confirmer.go @@ -326,7 +326,7 @@ func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) pro if ec.resumeCallback != nil { mark = time.Now() - if err := ec.ResumePendingTaskRuns(ctx, head.BlockNumber()); err != nil { + if err := ec.ResumePendingTaskRuns(ctx); err != nil { return fmt.Errorf("ResumePendingTaskRuns failed: %w", err) } @@ -1188,8 +1188,8 @@ func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) sen } // ResumePendingTaskRuns issues callbacks to task runs that are pending waiting for receipts -func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) ResumePendingTaskRuns(ctx context.Context, headNum int64) error { - receiptsPlus, err := ec.txStore.FindTxesPendingCallback(ctx, headNum, ec.chainID) +func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) ResumePendingTaskRuns(ctx context.Context) error { + receiptsPlus, err := ec.txStore.FindTxesPendingCallback(ctx, ec.chainID) if err != nil { return err } diff --git a/common/txmgr/types/mocks/tx_store.go b/common/txmgr/types/mocks/tx_store.go index 0b9c7110660..84eb25c8114 100644 --- a/common/txmgr/types/mocks/tx_store.go +++ b/common/txmgr/types/mocks/tx_store.go @@ -1096,9 +1096,9 @@ func (_c *TxStore_FindTxesByMetaFieldAndStates_Call[ADDR, CHAIN_ID, TX_HASH, BLO return _c } -// FindTxesPendingCallback provides a mock function with given fields: ctx, blockNum, chainID -func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxesPendingCallback(ctx context.Context, blockNum int64, chainID CHAIN_ID) ([]txmgrtypes.ReceiptPlus[R], error) { - ret := _m.Called(ctx, blockNum, chainID) +// FindTxesPendingCallback provides a mock function with given fields: ctx, chainID +func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxesPendingCallback(ctx context.Context, chainID CHAIN_ID) ([]txmgrtypes.ReceiptPlus[R], error) { + ret := _m.Called(ctx, chainID) if len(ret) == 0 { panic("no return value specified for FindTxesPendingCallback") @@ -1106,19 +1106,19 @@ func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxesPen var r0 []txmgrtypes.ReceiptPlus[R] var r1 error - if rf, ok := ret.Get(0).(func(context.Context, int64, CHAIN_ID) ([]txmgrtypes.ReceiptPlus[R], error)); ok { - return rf(ctx, blockNum, chainID) + if rf, ok := ret.Get(0).(func(context.Context, CHAIN_ID) ([]txmgrtypes.ReceiptPlus[R], error)); ok { + return rf(ctx, chainID) } - if rf, ok := ret.Get(0).(func(context.Context, int64, CHAIN_ID) []txmgrtypes.ReceiptPlus[R]); ok { - r0 = rf(ctx, blockNum, chainID) + if rf, ok := ret.Get(0).(func(context.Context, CHAIN_ID) []txmgrtypes.ReceiptPlus[R]); ok { + r0 = rf(ctx, chainID) } else { if ret.Get(0) != nil { r0 = ret.Get(0).([]txmgrtypes.ReceiptPlus[R]) } } - if rf, ok := ret.Get(1).(func(context.Context, int64, CHAIN_ID) error); ok { - r1 = rf(ctx, blockNum, chainID) + if rf, ok := ret.Get(1).(func(context.Context, CHAIN_ID) error); ok { + r1 = rf(ctx, chainID) } else { r1 = ret.Error(1) } @@ -1133,15 +1133,14 @@ type TxStore_FindTxesPendingCallback_Call[ADDR types.Hashable, CHAIN_ID types.ID // FindTxesPendingCallback is a helper method to define mock.On call // - ctx context.Context -// - blockNum int64 // - chainID CHAIN_ID -func (_e *TxStore_Expecter[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxesPendingCallback(ctx interface{}, blockNum interface{}, chainID interface{}) *TxStore_FindTxesPendingCallback_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { - return &TxStore_FindTxesPendingCallback_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]{Call: _e.mock.On("FindTxesPendingCallback", ctx, blockNum, chainID)} +func (_e *TxStore_Expecter[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxesPendingCallback(ctx interface{}, chainID interface{}) *TxStore_FindTxesPendingCallback_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { + return &TxStore_FindTxesPendingCallback_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]{Call: _e.mock.On("FindTxesPendingCallback", ctx, chainID)} } -func (_c *TxStore_FindTxesPendingCallback_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) Run(run func(ctx context.Context, blockNum int64, chainID CHAIN_ID)) *TxStore_FindTxesPendingCallback_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { +func (_c *TxStore_FindTxesPendingCallback_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) Run(run func(ctx context.Context, chainID CHAIN_ID)) *TxStore_FindTxesPendingCallback_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].(CHAIN_ID)) + run(args[0].(context.Context), args[1].(CHAIN_ID)) }) return _c } @@ -1151,7 +1150,7 @@ func (_c *TxStore_FindTxesPendingCallback_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HA return _c } -func (_c *TxStore_FindTxesPendingCallback_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) RunAndReturn(run func(context.Context, int64, CHAIN_ID) ([]txmgrtypes.ReceiptPlus[R], error)) *TxStore_FindTxesPendingCallback_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { +func (_c *TxStore_FindTxesPendingCallback_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) RunAndReturn(run func(context.Context, CHAIN_ID) ([]txmgrtypes.ReceiptPlus[R], error)) *TxStore_FindTxesPendingCallback_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { _c.Call.Return(run) return _c } diff --git a/common/txmgr/types/tx_store.go b/common/txmgr/types/tx_store.go index 67130fc4e38..2138ce38409 100644 --- a/common/txmgr/types/tx_store.go +++ b/common/txmgr/types/tx_store.go @@ -34,7 +34,7 @@ type TxStore[ TransactionStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, SEQ, FEE] // Find confirmed txes with finalized state that require callback but have not yet been signaled - FindTxesPendingCallback(ctx context.Context, blockNum int64, chainID CHAIN_ID) (receiptsPlus []ReceiptPlus[R], err error) + FindTxesPendingCallback(ctx context.Context, chainID CHAIN_ID) (receiptsPlus []ReceiptPlus[R], err error) // Update tx to mark that its callback has been signaled UpdateTxCallbackCompleted(ctx context.Context, pipelineTaskRunRid uuid.UUID, chainId CHAIN_ID) error SaveFetchedReceipts(ctx context.Context, r []R, state TxState, errorMsg *string, chainID CHAIN_ID) error diff --git a/core/chains/evm/txmgr/confirmer_test.go b/core/chains/evm/txmgr/confirmer_test.go index 36ca77140e7..31cb67d3e64 100644 --- a/core/chains/evm/txmgr/confirmer_test.go +++ b/core/chains/evm/txmgr/confirmer_test.go @@ -2976,7 +2976,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { // It would only be in a state past suspended if the resume callback was called and callback_completed was set to TRUE pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE, callback_completed = TRUE WHERE id = $2`, &tr.ID, etx.ID) - err := ec.ResumePendingTaskRuns(tests.Context(t), head.Number) + err := ec.ResumePendingTaskRuns(tests.Context(t)) require.NoError(t, err) }) @@ -3004,7 +3004,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { t.Cleanup(func() { <-done }) go func() { defer close(done) - err2 := ec.ResumePendingTaskRuns(tests.Context(t), head.Number) + err2 := ec.ResumePendingTaskRuns(tests.Context(t)) if !assert.NoError(t, err2) { return } @@ -3058,7 +3058,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { t.Cleanup(func() { <-done }) go func() { defer close(done) - err2 := ec.ResumePendingTaskRuns(tests.Context(t), head.Number) + err2 := ec.ResumePendingTaskRuns(tests.Context(t)) if !assert.NoError(t, err2) { return } @@ -3095,7 +3095,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { mustInsertEthReceipt(t, txStore, head.Number, head.Hash, etx.TxAttempts[0].Hash) pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE WHERE id = $2`, &tr.ID, etx.ID) - err := ec.ResumePendingTaskRuns(tests.Context(t), head.Number) + err := ec.ResumePendingTaskRuns(tests.Context(t)) require.Error(t, err) // Retrieve Tx to check if callback completed flag was left unchanged diff --git a/core/chains/evm/txmgr/evm_tx_store.go b/core/chains/evm/txmgr/evm_tx_store.go index 65329f9ecba..2712ab25311 100644 --- a/core/chains/evm/txmgr/evm_tx_store.go +++ b/core/chains/evm/txmgr/evm_tx_store.go @@ -1015,21 +1015,20 @@ WHERE evm.tx_attempts.state = 'in_progress' AND evm.txes.from_address = $1 AND e } // Find confirmed txes requiring callback but have not yet been signaled -func (o *evmTxStore) FindTxesPendingCallback(ctx context.Context, blockNum int64, chainID *big.Int) (receiptsPlus []ReceiptPlus, err error) { +func (o *evmTxStore) FindTxesPendingCallback(ctx context.Context, chainID *big.Int) (receiptsPlus []ReceiptPlus, err error) { var rs []dbReceiptPlus var cancel context.CancelFunc ctx, cancel = o.stopCh.Ctx(ctx) defer cancel() - // TODO update the query to use tx finality state instead of use block range after Finalizer PR https://github.com/smartcontractkit/chainlink/pull/13638 merged err = o.q.SelectContext(ctx, &rs, ` SELECT evm.txes.pipeline_task_run_id, evm.receipts.receipt, COALESCE((evm.txes.meta->>'FailOnRevert')::boolean, false) "FailOnRevert" 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.pipeline_task_run_id IS NOT NULL AND evm.txes.signal_callback = TRUE AND evm.txes.callback_completed = FALSE - AND evm.receipts.block_number <= $1 AND evm.txes.evm_chain_id = $2 - `, blockNum, chainID.String()) + AND evm.txes.state = 'confirmed' AND evm.txes.evm_chain_id = $1 + `, chainID.String()) if err != nil { return nil, fmt.Errorf("failed to retrieve transactions pending pipeline resume callback: %w", err) } diff --git a/core/chains/evm/txmgr/evm_tx_store_test.go b/core/chains/evm/txmgr/evm_tx_store_test.go index b57f2e1b0e5..c7f1ddd788a 100644 --- a/core/chains/evm/txmgr/evm_tx_store_test.go +++ b/core/chains/evm/txmgr/evm_tx_store_test.go @@ -680,7 +680,7 @@ func TestORM_FindTxesPendingCallback(t *testing.T) { cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, 7, 1, fromAddress) // Search evm.txes table for tx requiring callback - receiptsPlus, err := txStore.FindTxesPendingCallback(tests.Context(t), head.Number, ethClient.ConfiguredChainID()) + receiptsPlus, err := txStore.FindTxesPendingCallback(tests.Context(t), ethClient.ConfiguredChainID()) require.NoError(t, err) assert.Len(t, receiptsPlus, 2) assert.Equal(t, tr1.ID, receiptsPlus[0].ID) diff --git a/core/chains/evm/txmgr/mocks/evm_tx_store.go b/core/chains/evm/txmgr/mocks/evm_tx_store.go index b40c0ca8376..59f7eb40903 100644 --- a/core/chains/evm/txmgr/mocks/evm_tx_store.go +++ b/core/chains/evm/txmgr/mocks/evm_tx_store.go @@ -1395,9 +1395,9 @@ func (_c *EvmTxStore_FindTxesByMetaFieldAndStates_Call) RunAndReturn(run func(co return _c } -// FindTxesPendingCallback provides a mock function with given fields: ctx, blockNum, chainID -func (_m *EvmTxStore) FindTxesPendingCallback(ctx context.Context, blockNum int64, chainID *big.Int) ([]types.ReceiptPlus[*evmtypes.Receipt], error) { - ret := _m.Called(ctx, blockNum, chainID) +// FindTxesPendingCallback provides a mock function with given fields: ctx, chainID +func (_m *EvmTxStore) FindTxesPendingCallback(ctx context.Context, chainID *big.Int) ([]types.ReceiptPlus[*evmtypes.Receipt], error) { + ret := _m.Called(ctx, chainID) if len(ret) == 0 { panic("no return value specified for FindTxesPendingCallback") @@ -1405,19 +1405,19 @@ func (_m *EvmTxStore) FindTxesPendingCallback(ctx context.Context, blockNum int6 var r0 []types.ReceiptPlus[*evmtypes.Receipt] var r1 error - if rf, ok := ret.Get(0).(func(context.Context, int64, *big.Int) ([]types.ReceiptPlus[*evmtypes.Receipt], error)); ok { - return rf(ctx, blockNum, chainID) + if rf, ok := ret.Get(0).(func(context.Context, *big.Int) ([]types.ReceiptPlus[*evmtypes.Receipt], error)); ok { + return rf(ctx, chainID) } - if rf, ok := ret.Get(0).(func(context.Context, int64, *big.Int) []types.ReceiptPlus[*evmtypes.Receipt]); ok { - r0 = rf(ctx, blockNum, chainID) + if rf, ok := ret.Get(0).(func(context.Context, *big.Int) []types.ReceiptPlus[*evmtypes.Receipt]); ok { + r0 = rf(ctx, chainID) } else { if ret.Get(0) != nil { r0 = ret.Get(0).([]types.ReceiptPlus[*evmtypes.Receipt]) } } - if rf, ok := ret.Get(1).(func(context.Context, int64, *big.Int) error); ok { - r1 = rf(ctx, blockNum, chainID) + if rf, ok := ret.Get(1).(func(context.Context, *big.Int) error); ok { + r1 = rf(ctx, chainID) } else { r1 = ret.Error(1) } @@ -1432,15 +1432,14 @@ type EvmTxStore_FindTxesPendingCallback_Call struct { // FindTxesPendingCallback is a helper method to define mock.On call // - ctx context.Context -// - blockNum int64 // - chainID *big.Int -func (_e *EvmTxStore_Expecter) FindTxesPendingCallback(ctx interface{}, blockNum interface{}, chainID interface{}) *EvmTxStore_FindTxesPendingCallback_Call { - return &EvmTxStore_FindTxesPendingCallback_Call{Call: _e.mock.On("FindTxesPendingCallback", ctx, blockNum, chainID)} +func (_e *EvmTxStore_Expecter) FindTxesPendingCallback(ctx interface{}, chainID interface{}) *EvmTxStore_FindTxesPendingCallback_Call { + return &EvmTxStore_FindTxesPendingCallback_Call{Call: _e.mock.On("FindTxesPendingCallback", ctx, chainID)} } -func (_c *EvmTxStore_FindTxesPendingCallback_Call) Run(run func(ctx context.Context, blockNum int64, chainID *big.Int)) *EvmTxStore_FindTxesPendingCallback_Call { +func (_c *EvmTxStore_FindTxesPendingCallback_Call) Run(run func(ctx context.Context, chainID *big.Int)) *EvmTxStore_FindTxesPendingCallback_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].(*big.Int)) }) return _c } @@ -1450,7 +1449,7 @@ func (_c *EvmTxStore_FindTxesPendingCallback_Call) Return(receiptsPlus []types.R return _c } -func (_c *EvmTxStore_FindTxesPendingCallback_Call) RunAndReturn(run func(context.Context, int64, *big.Int) ([]types.ReceiptPlus[*evmtypes.Receipt], error)) *EvmTxStore_FindTxesPendingCallback_Call { +func (_c *EvmTxStore_FindTxesPendingCallback_Call) RunAndReturn(run func(context.Context, *big.Int) ([]types.ReceiptPlus[*evmtypes.Receipt], error)) *EvmTxStore_FindTxesPendingCallback_Call { _c.Call.Return(run) return _c } From 09bf8ca4ad66dda2db3335dbb0ad080a7eade941 Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Thu, 8 Aug 2024 11:20:36 -0500 Subject: [PATCH 29/46] address comments --- common/txmgr/types/tx.go | 3 ++- core/chains/evm/txmgr/evm_tx_store.go | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/common/txmgr/types/tx.go b/common/txmgr/types/tx.go index c3e9e4fd51b..bcbd909ca81 100644 --- a/common/txmgr/types/tx.go +++ b/common/txmgr/types/tx.go @@ -83,8 +83,9 @@ type TxRequest[ADDR types.Hashable, TX_HASH types.Hashable] struct { // Pipeline variables - if you aren't calling this from chain tx task within // the pipeline, you don't need these variables - MinConfirmations clnull.Uint32 // deprecated PipelineTaskRunID *uuid.UUID + // deprecated + MinConfirmations clnull.Uint32 Strategy TxStrategy diff --git a/core/chains/evm/txmgr/evm_tx_store.go b/core/chains/evm/txmgr/evm_tx_store.go index 2712ab25311..ae9c543e766 100644 --- a/core/chains/evm/txmgr/evm_tx_store.go +++ b/core/chains/evm/txmgr/evm_tx_store.go @@ -1027,7 +1027,7 @@ func (o *evmTxStore) FindTxesPendingCallback(ctx context.Context, chainID *big.I 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.pipeline_task_run_id IS NOT NULL AND evm.txes.signal_callback = TRUE AND evm.txes.callback_completed = FALSE - AND evm.txes.state = 'confirmed' AND evm.txes.evm_chain_id = $1 + AND evm.txes.state = 'finalized' AND evm.txes.evm_chain_id = $1 `, chainID.String()) if err != nil { return nil, fmt.Errorf("failed to retrieve transactions pending pipeline resume callback: %w", err) From 67a5f8ad943696dd93e933d32049d6ecb9fe26fc Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Thu, 8 Aug 2024 16:50:58 -0500 Subject: [PATCH 30/46] fix unit tests --- core/chains/evm/txmgr/evm_tx_store_test.go | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/core/chains/evm/txmgr/evm_tx_store_test.go b/core/chains/evm/txmgr/evm_tx_store_test.go index c7f1ddd788a..4daa790de61 100644 --- a/core/chains/evm/txmgr/evm_tx_store_test.go +++ b/core/chains/evm/txmgr/evm_tx_store_test.go @@ -649,7 +649,7 @@ func TestORM_FindTxesPendingCallback(t *testing.T) { etx1 := cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, 3, 1, fromAddress) pgtest.MustExec(t, db, `UPDATE evm.txes SET meta='{"FailOnRevert": true}'`) attempt1 := etx1.TxAttempts[0] - mustInsertEthReceipt(t, txStore, head.BlockNumber(), head.Hash, attempt1.Hash) + receipt1 := mustInsertEthReceipt(t, txStore, head.BlockNumber(), head.Hash, attempt1.Hash) pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE WHERE id = $2`, &tr1.ID, etx1.ID) // Callback to pipeline service completed. Should be ignored @@ -668,7 +668,7 @@ func TestORM_FindTxesPendingCallback(t *testing.T) { etx3 := cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, 5, 1, fromAddress) pgtest.MustExec(t, db, `UPDATE evm.txes SET meta='{"FailOnRevert": false}'`) attempt3 := etx3.TxAttempts[0] - mustInsertEthReceipt(t, txStore, head.Number, head.Hash, attempt3.Hash) + receipt3 := mustInsertEthReceipt(t, txStore, head.Number, head.Hash, attempt3.Hash) pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE WHERE id = $2`, &tr3.ID, etx3.ID) // Tx not marked for callback. Should be ignore @@ -679,6 +679,12 @@ func TestORM_FindTxesPendingCallback(t *testing.T) { // Unconfirmed Tx without receipts. Should be ignored cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, 7, 1, fromAddress) + // update txes state to be 'finalized' using finalizer + err := txStore.UpdateTxStatesToFinalizedUsingReceiptIds(tests.Context(t), []int64{receipt1.ID}, testutils.FixtureChainID) + require.NoError(t, err) + err = txStore.UpdateTxStatesToFinalizedUsingReceiptIds(tests.Context(t), []int64{receipt3.ID}, testutils.FixtureChainID) + require.NoError(t, err) + // Search evm.txes table for tx requiring callback receiptsPlus, err := txStore.FindTxesPendingCallback(tests.Context(t), ethClient.ConfiguredChainID()) require.NoError(t, err) From cdcbb99275a2608f73b4510d124d87baf7d1e14e Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Thu, 8 Aug 2024 18:51:58 -0500 Subject: [PATCH 31/46] fix test attempt --- core/chains/evm/txmgr/confirmer_test.go | 2 ++ core/chains/evm/txmgr/evm_tx_store.go | 7 ++++--- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/core/chains/evm/txmgr/confirmer_test.go b/core/chains/evm/txmgr/confirmer_test.go index 31cb67d3e64..bba0fac25c8 100644 --- a/core/chains/evm/txmgr/confirmer_test.go +++ b/core/chains/evm/txmgr/confirmer_test.go @@ -2999,6 +2999,8 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { receipt := mustInsertEthReceipt(t, txStore, head.Number, head.Hash, etx.TxAttempts[0].Hash) pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE WHERE id = $2`, &tr.ID, etx.ID) + err = txStore.UpdateTxStatesToFinalizedUsingReceiptIds(tests.Context(t), []int64{receipt.ID}, testutils.FixtureChainID) + require.NoError(t, err) done := make(chan struct{}) t.Cleanup(func() { <-done }) diff --git a/core/chains/evm/txmgr/evm_tx_store.go b/core/chains/evm/txmgr/evm_tx_store.go index ae9c543e766..88913f3501f 100644 --- a/core/chains/evm/txmgr/evm_tx_store.go +++ b/core/chains/evm/txmgr/evm_tx_store.go @@ -133,6 +133,7 @@ type dbReceiptPlus struct { ID uuid.UUID `db:"pipeline_task_run_id"` Receipt evmtypes.Receipt `db:"receipt"` FailOnRevert bool `db:"FailOnRevert"` + State string `db:"state"` } func fromDBReceipts(rs []DbReceipt) []*evmtypes.Receipt { @@ -1023,11 +1024,11 @@ func (o *evmTxStore) FindTxesPendingCallback(ctx context.Context, chainID *big.I defer cancel() err = o.q.SelectContext(ctx, &rs, ` - SELECT evm.txes.pipeline_task_run_id, evm.receipts.receipt, COALESCE((evm.txes.meta->>'FailOnRevert')::boolean, false) "FailOnRevert" FROM evm.txes + SELECT evm.txes.state, evm.txes.pipeline_task_run_id, evm.receipts.receipt, COALESCE((evm.txes.meta->>'FailOnRevert')::boolean, false) "FailOnRevert" 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.pipeline_task_run_id IS NOT NULL AND evm.txes.signal_callback = TRUE AND evm.txes.callback_completed = FALSE - AND evm.txes.state = 'finalized' AND evm.txes.evm_chain_id = $1 + AND evm.txes.evm_chain_id = $1 `, chainID.String()) if err != nil { return nil, fmt.Errorf("failed to retrieve transactions pending pipeline resume callback: %w", err) @@ -1094,7 +1095,7 @@ func (o *evmTxStore) FindTxWithSequence(ctx context.Context, fromAddress common. err = o.Transact(ctx, true, func(orm *evmTxStore) error { var dbEtx DbEthTx err = orm.q.GetContext(ctx, &dbEtx, ` -SELECT * FROM evm.txes WHERE from_address = $1 AND nonce = $2 AND state IN ('confirmed', 'confirmed_missing_receipt', 'unconfirmed') +SELECT * FROM evm.txes WHERE from_address = $1 AND nonce = $2 AND state IN ('confirmed', 'confirmed_missing_receipt', 'unconfirmed', 'finalized') `, fromAddress, nonce.Int64()) if err != nil { return pkgerrors.Wrap(err, "FindEthTxWithNonce failed to load evm.txes") From 82fde8d675ecdee1415974178d484521843e842a Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Thu, 8 Aug 2024 18:56:31 -0500 Subject: [PATCH 32/46] update test --- core/chains/evm/txmgr/confirmer_test.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/chains/evm/txmgr/confirmer_test.go b/core/chains/evm/txmgr/confirmer_test.go index bba0fac25c8..7ef610202fa 100644 --- a/core/chains/evm/txmgr/confirmer_test.go +++ b/core/chains/evm/txmgr/confirmer_test.go @@ -2980,7 +2980,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { require.NoError(t, err) }) - t.Run("processes eth_txes with receipts older than minConfirmations", func(t *testing.T) { + t.Run("processes eth_txes that's finalized", func(t *testing.T) { ch := make(chan interface{}) nonce := evmtypes.Nonce(3) var err error @@ -2999,6 +2999,8 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { receipt := mustInsertEthReceipt(t, txStore, head.Number, head.Hash, etx.TxAttempts[0].Hash) pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE WHERE id = $2`, &tr.ID, etx.ID) + + // set tx state to be finalized err = txStore.UpdateTxStatesToFinalizedUsingReceiptIds(tests.Context(t), []int64{receipt.ID}, testutils.FixtureChainID) require.NoError(t, err) From acc3a8a7c4c9354f0b4d6c14a2d4650d021c1df4 Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Thu, 8 Aug 2024 19:15:32 -0500 Subject: [PATCH 33/46] refactor TransactionStore interface --- common/txmgr/confirmer.go | 2 +- common/txmgr/types/mocks/tx_store.go | 82 ++++++++++++++++++--- common/txmgr/types/tx_store.go | 5 +- core/chains/evm/txmgr/confirmer_test.go | 6 +- core/chains/evm/txmgr/evm_tx_store.go | 26 ++++++- core/chains/evm/txmgr/evm_tx_store_test.go | 4 +- core/chains/evm/txmgr/mocks/evm_tx_store.go | 82 ++++++++++++++++++--- 7 files changed, 176 insertions(+), 31 deletions(-) diff --git a/common/txmgr/confirmer.go b/common/txmgr/confirmer.go index 35c1117c2e0..5c328a5bca0 100644 --- a/common/txmgr/confirmer.go +++ b/common/txmgr/confirmer.go @@ -1141,7 +1141,7 @@ func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) For ec.lggr.Infof("ForceRebroadcast: will rebroadcast transactions for all sequences between %v and %v", seqs[0], seqs[len(seqs)-1]) for _, seq := range seqs { - etx, err := ec.txStore.FindTxWithSequence(ctx, address, seq) + etx, err := ec.txStore.FindTxWithSequenceForRebroadcast(ctx, address, seq) if err != nil { return fmt.Errorf("ForceRebroadcast failed: %w", err) } diff --git a/common/txmgr/types/mocks/tx_store.go b/common/txmgr/types/mocks/tx_store.go index 84eb25c8114..acdb86e9c5a 100644 --- a/common/txmgr/types/mocks/tx_store.go +++ b/common/txmgr/types/mocks/tx_store.go @@ -555,6 +555,66 @@ func (_c *TxStore_FindEarliestUnconfirmedTxAttemptBlock_Call[ADDR, CHAIN_ID, TX_ return _c } +// FindFinalizedTxWithSequence provides a mock function with given fields: ctx, fromAddress, seq +func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindFinalizedTxWithSequence(ctx context.Context, fromAddress ADDR, seq SEQ) (*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error) { + ret := _m.Called(ctx, fromAddress, seq) + + if len(ret) == 0 { + panic("no return value specified for FindFinalizedTxWithSequence") + } + + 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, ADDR, SEQ) (*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error)); ok { + return rf(ctx, fromAddress, seq) + } + if rf, ok := ret.Get(0).(func(context.Context, ADDR, SEQ) *txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]); ok { + r0 = rf(ctx, fromAddress, seq) + } 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, ADDR, SEQ) error); ok { + r1 = rf(ctx, fromAddress, seq) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// TxStore_FindFinalizedTxWithSequence_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'FindFinalizedTxWithSequence' +type TxStore_FindFinalizedTxWithSequence_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 +} + +// FindFinalizedTxWithSequence is a helper method to define mock.On call +// - ctx context.Context +// - fromAddress ADDR +// - seq SEQ +func (_e *TxStore_Expecter[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindFinalizedTxWithSequence(ctx interface{}, fromAddress interface{}, seq interface{}) *TxStore_FindFinalizedTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { + return &TxStore_FindFinalizedTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]{Call: _e.mock.On("FindFinalizedTxWithSequence", ctx, fromAddress, seq)} +} + +func (_c *TxStore_FindFinalizedTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) Run(run func(ctx context.Context, fromAddress ADDR, seq SEQ)) *TxStore_FindFinalizedTxWithSequence_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].(ADDR), args[2].(SEQ)) + }) + return _c +} + +func (_c *TxStore_FindFinalizedTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) Return(etx *txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) *TxStore_FindFinalizedTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { + _c.Call.Return(etx, err) + return _c +} + +func (_c *TxStore_FindFinalizedTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) RunAndReturn(run func(context.Context, ADDR, SEQ) (*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error)) *TxStore_FindFinalizedTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { + _c.Call.Return(run) + return _c +} + // FindLatestSequence provides a mock function with given fields: ctx, fromAddress, chainId func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindLatestSequence(ctx context.Context, fromAddress ADDR, chainId CHAIN_ID) (SEQ, error) { ret := _m.Called(ctx, fromAddress, chainId) @@ -974,12 +1034,12 @@ func (_c *TxStore_FindTxWithIdempotencyKey_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_H return _c } -// FindTxWithSequence provides a mock function with given fields: ctx, fromAddress, seq -func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxWithSequence(ctx context.Context, fromAddress ADDR, seq SEQ) (*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error) { +// FindTxWithSequenceForRebroadcast provides a mock function with given fields: ctx, fromAddress, seq +func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxWithSequenceForRebroadcast(ctx context.Context, fromAddress ADDR, seq SEQ) (*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error) { ret := _m.Called(ctx, fromAddress, seq) if len(ret) == 0 { - panic("no return value specified for FindTxWithSequence") + panic("no return value specified for FindTxWithSequenceForRebroadcast") } var r0 *txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] @@ -1004,32 +1064,32 @@ func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxWithS return r0, r1 } -// TxStore_FindTxWithSequence_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'FindTxWithSequence' -type TxStore_FindTxWithSequence_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 { +// TxStore_FindTxWithSequenceForRebroadcast_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'FindTxWithSequenceForRebroadcast' +type TxStore_FindTxWithSequenceForRebroadcast_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 } -// FindTxWithSequence is a helper method to define mock.On call +// FindTxWithSequenceForRebroadcast is a helper method to define mock.On call // - ctx context.Context // - fromAddress ADDR // - seq SEQ -func (_e *TxStore_Expecter[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxWithSequence(ctx interface{}, fromAddress interface{}, seq interface{}) *TxStore_FindTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { - return &TxStore_FindTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]{Call: _e.mock.On("FindTxWithSequence", ctx, fromAddress, seq)} +func (_e *TxStore_Expecter[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxWithSequenceForRebroadcast(ctx interface{}, fromAddress interface{}, seq interface{}) *TxStore_FindTxWithSequenceForRebroadcast_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { + return &TxStore_FindTxWithSequenceForRebroadcast_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]{Call: _e.mock.On("FindTxWithSequenceForRebroadcast", ctx, fromAddress, seq)} } -func (_c *TxStore_FindTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) Run(run func(ctx context.Context, fromAddress ADDR, seq SEQ)) *TxStore_FindTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { +func (_c *TxStore_FindTxWithSequenceForRebroadcast_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) Run(run func(ctx context.Context, fromAddress ADDR, seq SEQ)) *TxStore_FindTxWithSequenceForRebroadcast_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].(ADDR), args[2].(SEQ)) }) return _c } -func (_c *TxStore_FindTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) Return(etx *txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) *TxStore_FindTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { +func (_c *TxStore_FindTxWithSequenceForRebroadcast_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) Return(etx *txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) *TxStore_FindTxWithSequenceForRebroadcast_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { _c.Call.Return(etx, err) return _c } -func (_c *TxStore_FindTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) RunAndReturn(run func(context.Context, ADDR, SEQ) (*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error)) *TxStore_FindTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { +func (_c *TxStore_FindTxWithSequenceForRebroadcast_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) RunAndReturn(run func(context.Context, ADDR, SEQ) (*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error)) *TxStore_FindTxWithSequenceForRebroadcast_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { _c.Call.Return(run) return _c } diff --git a/common/txmgr/types/tx_store.go b/common/txmgr/types/tx_store.go index 2138ce38409..c415629238a 100644 --- a/common/txmgr/types/tx_store.go +++ b/common/txmgr/types/tx_store.go @@ -76,8 +76,9 @@ type TransactionStore[ FindTxAttemptsRequiringResend(ctx context.Context, olderThan time.Time, maxInFlightTransactions uint32, chainID CHAIN_ID, address ADDR) (attempts []TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) // Search for Tx using the idempotencyKey and chainID FindTxWithIdempotencyKey(ctx context.Context, idempotencyKey string, chainID CHAIN_ID) (tx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) - // Search for Tx using the fromAddress and sequence - FindTxWithSequence(ctx context.Context, fromAddress ADDR, seq SEQ) (etx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) + // Search for Tx for rebroadcast using the fromAddress and sequence + FindTxWithSequenceForRebroadcast(ctx context.Context, fromAddress ADDR, seq SEQ) (etx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) + FindFinalizedTxWithSequence(ctx context.Context, fromAddress ADDR, seq SEQ) (etx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) FindNextUnstartedTransactionFromAddress(ctx context.Context, fromAddress ADDR, chainID CHAIN_ID) (*Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error) 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) diff --git a/core/chains/evm/txmgr/confirmer_test.go b/core/chains/evm/txmgr/confirmer_test.go index 7ef610202fa..1da66f72fdf 100644 --- a/core/chains/evm/txmgr/confirmer_test.go +++ b/core/chains/evm/txmgr/confirmer_test.go @@ -3013,7 +3013,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { return } // Retrieve Tx to check if callback completed flag was set to true - updateTx, err3 := txStore.FindTxWithSequence(tests.Context(t), fromAddress, nonce) + updateTx, err3 := txStore.FindFinalizedTxWithSequence(tests.Context(t), fromAddress, nonce) if assert.NoError(t, err3) { assert.Equal(t, true, updateTx.CallbackCompleted) } @@ -3067,7 +3067,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { return } // Retrieve Tx to check if callback completed flag was set to true - updateTx, err3 := txStore.FindTxWithSequence(tests.Context(t), fromAddress, nonce) + updateTx, err3 := txStore.FindTxWithSequenceForRebroadcast(tests.Context(t), fromAddress, nonce) if assert.NoError(t, err3) { assert.Equal(t, true, updateTx.CallbackCompleted) } @@ -3103,7 +3103,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { require.Error(t, err) // Retrieve Tx to check if callback completed flag was left unchanged - updateTx, err := txStore.FindTxWithSequence(tests.Context(t), fromAddress, nonce) + updateTx, err := txStore.FindTxWithSequenceForRebroadcast(tests.Context(t), fromAddress, nonce) require.NoError(t, err) require.Equal(t, false, updateTx.CallbackCompleted) }) diff --git a/core/chains/evm/txmgr/evm_tx_store.go b/core/chains/evm/txmgr/evm_tx_store.go index 88913f3501f..9a3e2891841 100644 --- a/core/chains/evm/txmgr/evm_tx_store.go +++ b/core/chains/evm/txmgr/evm_tx_store.go @@ -1087,7 +1087,7 @@ func (o *evmTxStore) FindTxWithIdempotencyKey(ctx context.Context, idempotencyKe } // FindTxWithSequence returns any broadcast ethtx with the given nonce -func (o *evmTxStore) FindTxWithSequence(ctx context.Context, fromAddress common.Address, nonce evmtypes.Nonce) (etx *Tx, err error) { +func (o *evmTxStore) FindTxWithSequenceForRebroadcast(ctx context.Context, fromAddress common.Address, nonce evmtypes.Nonce) (etx *Tx, err error) { var cancel context.CancelFunc ctx, cancel = o.stopCh.Ctx(ctx) defer cancel() @@ -1110,6 +1110,30 @@ SELECT * FROM evm.txes WHERE from_address = $1 AND nonce = $2 AND state IN ('con return } +// FindFinalizedTxWithSequence returns finalized ethtx with the given nonce +func (o *evmTxStore) FindFinalizedTxWithSequence(ctx context.Context, fromAddress common.Address, nonce evmtypes.Nonce) (etx *Tx, err error) { + var cancel context.CancelFunc + ctx, cancel = o.stopCh.Ctx(ctx) + defer cancel() + etx = new(Tx) + err = o.Transact(ctx, true, func(orm *evmTxStore) error { + var dbEtx DbEthTx + err = orm.q.GetContext(ctx, &dbEtx, ` +SELECT * FROM evm.txes WHERE from_address = $1 AND nonce = $2 AND state = 'finalized' +`, fromAddress, nonce.Int64()) + if err != nil { + return pkgerrors.Wrap(err, "FindEthTxWithNonce failed to load evm.txes") + } + dbEtx.ToTx(etx) + err = orm.loadTxAttemptsAtomic(ctx, etx) + return pkgerrors.Wrap(err, "FindEthTxWithNonce failed to load evm.tx_attempts") + }) + if errors.Is(err, sql.ErrNoRows) { + return nil, nil + } + return +} + func updateEthTxAttemptUnbroadcast(ctx context.Context, orm *evmTxStore, attempt TxAttempt) error { if attempt.State != txmgrtypes.TxAttemptBroadcast { return errors.New("expected eth_tx_attempt to be broadcast") diff --git a/core/chains/evm/txmgr/evm_tx_store_test.go b/core/chains/evm/txmgr/evm_tx_store_test.go index 4daa790de61..0c5db83c4f4 100644 --- a/core/chains/evm/txmgr/evm_tx_store_test.go +++ b/core/chains/evm/txmgr/evm_tx_store_test.go @@ -730,7 +730,7 @@ func TestORM_FindTxWithSequence(t *testing.T) { _, fromAddress := cltest.MustInsertRandomKeyReturningState(t, ethKeyStore) t.Run("returns nil if no results", func(t *testing.T) { - etx, err := txStore.FindTxWithSequence(tests.Context(t), fromAddress, evmtypes.Nonce(777)) + etx, err := txStore.FindTxWithSequenceForRebroadcast(tests.Context(t), fromAddress, evmtypes.Nonce(777)) require.NoError(t, err) assert.Nil(t, etx) }) @@ -739,7 +739,7 @@ func TestORM_FindTxWithSequence(t *testing.T) { etx := cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, 777, 1, fromAddress) require.Equal(t, evmtypes.Nonce(777), *etx.Sequence) - res, err := txStore.FindTxWithSequence(tests.Context(t), fromAddress, evmtypes.Nonce(777)) + res, err := txStore.FindTxWithSequenceForRebroadcast(tests.Context(t), fromAddress, evmtypes.Nonce(777)) require.NoError(t, err) assert.Equal(t, etx.Sequence, res.Sequence) }) diff --git a/core/chains/evm/txmgr/mocks/evm_tx_store.go b/core/chains/evm/txmgr/mocks/evm_tx_store.go index 59f7eb40903..ecc502c6340 100644 --- a/core/chains/evm/txmgr/mocks/evm_tx_store.go +++ b/core/chains/evm/txmgr/mocks/evm_tx_store.go @@ -620,6 +620,66 @@ func (_c *EvmTxStore_FindEarliestUnconfirmedTxAttemptBlock_Call) RunAndReturn(ru return _c } +// FindFinalizedTxWithSequence provides a mock function with given fields: ctx, fromAddress, seq +func (_m *EvmTxStore) FindFinalizedTxWithSequence(ctx context.Context, fromAddress common.Address, seq evmtypes.Nonce) (*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error) { + ret := _m.Called(ctx, fromAddress, seq) + + if len(ret) == 0 { + panic("no return value specified for FindFinalizedTxWithSequence") + } + + 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, common.Address, evmtypes.Nonce) (*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error)); ok { + return rf(ctx, fromAddress, seq) + } + if rf, ok := ret.Get(0).(func(context.Context, common.Address, evmtypes.Nonce) *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]); ok { + r0 = rf(ctx, fromAddress, seq) + } 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, common.Address, evmtypes.Nonce) error); ok { + r1 = rf(ctx, fromAddress, seq) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// EvmTxStore_FindFinalizedTxWithSequence_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'FindFinalizedTxWithSequence' +type EvmTxStore_FindFinalizedTxWithSequence_Call struct { + *mock.Call +} + +// FindFinalizedTxWithSequence is a helper method to define mock.On call +// - ctx context.Context +// - fromAddress common.Address +// - seq evmtypes.Nonce +func (_e *EvmTxStore_Expecter) FindFinalizedTxWithSequence(ctx interface{}, fromAddress interface{}, seq interface{}) *EvmTxStore_FindFinalizedTxWithSequence_Call { + return &EvmTxStore_FindFinalizedTxWithSequence_Call{Call: _e.mock.On("FindFinalizedTxWithSequence", ctx, fromAddress, seq)} +} + +func (_c *EvmTxStore_FindFinalizedTxWithSequence_Call) Run(run func(ctx context.Context, fromAddress common.Address, seq evmtypes.Nonce)) *EvmTxStore_FindFinalizedTxWithSequence_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(common.Address), args[2].(evmtypes.Nonce)) + }) + return _c +} + +func (_c *EvmTxStore_FindFinalizedTxWithSequence_Call) Return(etx *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], err error) *EvmTxStore_FindFinalizedTxWithSequence_Call { + _c.Call.Return(etx, err) + return _c +} + +func (_c *EvmTxStore_FindFinalizedTxWithSequence_Call) RunAndReturn(run func(context.Context, common.Address, evmtypes.Nonce) (*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error)) *EvmTxStore_FindFinalizedTxWithSequence_Call { + _c.Call.Return(run) + return _c +} + // FindLatestSequence provides a mock function with given fields: ctx, fromAddress, chainId func (_m *EvmTxStore) FindLatestSequence(ctx context.Context, fromAddress common.Address, chainId *big.Int) (evmtypes.Nonce, error) { ret := _m.Called(ctx, fromAddress, chainId) @@ -1273,12 +1333,12 @@ func (_c *EvmTxStore_FindTxWithIdempotencyKey_Call) RunAndReturn(run func(contex return _c } -// FindTxWithSequence provides a mock function with given fields: ctx, fromAddress, seq -func (_m *EvmTxStore) FindTxWithSequence(ctx context.Context, fromAddress common.Address, seq evmtypes.Nonce) (*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error) { +// FindTxWithSequenceForRebroadcast provides a mock function with given fields: ctx, fromAddress, seq +func (_m *EvmTxStore) FindTxWithSequenceForRebroadcast(ctx context.Context, fromAddress common.Address, seq evmtypes.Nonce) (*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error) { ret := _m.Called(ctx, fromAddress, seq) if len(ret) == 0 { - panic("no return value specified for FindTxWithSequence") + panic("no return value specified for FindTxWithSequenceForRebroadcast") } var r0 *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee] @@ -1303,32 +1363,32 @@ func (_m *EvmTxStore) FindTxWithSequence(ctx context.Context, fromAddress common return r0, r1 } -// EvmTxStore_FindTxWithSequence_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'FindTxWithSequence' -type EvmTxStore_FindTxWithSequence_Call struct { +// EvmTxStore_FindTxWithSequenceForRebroadcast_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'FindTxWithSequenceForRebroadcast' +type EvmTxStore_FindTxWithSequenceForRebroadcast_Call struct { *mock.Call } -// FindTxWithSequence is a helper method to define mock.On call +// FindTxWithSequenceForRebroadcast is a helper method to define mock.On call // - ctx context.Context // - fromAddress common.Address // - seq evmtypes.Nonce -func (_e *EvmTxStore_Expecter) FindTxWithSequence(ctx interface{}, fromAddress interface{}, seq interface{}) *EvmTxStore_FindTxWithSequence_Call { - return &EvmTxStore_FindTxWithSequence_Call{Call: _e.mock.On("FindTxWithSequence", ctx, fromAddress, seq)} +func (_e *EvmTxStore_Expecter) FindTxWithSequenceForRebroadcast(ctx interface{}, fromAddress interface{}, seq interface{}) *EvmTxStore_FindTxWithSequenceForRebroadcast_Call { + return &EvmTxStore_FindTxWithSequenceForRebroadcast_Call{Call: _e.mock.On("FindTxWithSequenceForRebroadcast", ctx, fromAddress, seq)} } -func (_c *EvmTxStore_FindTxWithSequence_Call) Run(run func(ctx context.Context, fromAddress common.Address, seq evmtypes.Nonce)) *EvmTxStore_FindTxWithSequence_Call { +func (_c *EvmTxStore_FindTxWithSequenceForRebroadcast_Call) Run(run func(ctx context.Context, fromAddress common.Address, seq evmtypes.Nonce)) *EvmTxStore_FindTxWithSequenceForRebroadcast_Call { _c.Call.Run(func(args mock.Arguments) { run(args[0].(context.Context), args[1].(common.Address), args[2].(evmtypes.Nonce)) }) return _c } -func (_c *EvmTxStore_FindTxWithSequence_Call) Return(etx *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], err error) *EvmTxStore_FindTxWithSequence_Call { +func (_c *EvmTxStore_FindTxWithSequenceForRebroadcast_Call) Return(etx *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], err error) *EvmTxStore_FindTxWithSequenceForRebroadcast_Call { _c.Call.Return(etx, err) return _c } -func (_c *EvmTxStore_FindTxWithSequence_Call) RunAndReturn(run func(context.Context, common.Address, evmtypes.Nonce) (*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error)) *EvmTxStore_FindTxWithSequence_Call { +func (_c *EvmTxStore_FindTxWithSequenceForRebroadcast_Call) RunAndReturn(run func(context.Context, common.Address, evmtypes.Nonce) (*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error)) *EvmTxStore_FindTxWithSequenceForRebroadcast_Call { _c.Call.Return(run) return _c } From e0573cba9c3814f499cf66d3da5f833b88305ef7 Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Thu, 8 Aug 2024 19:24:22 -0500 Subject: [PATCH 34/46] revert some temp changes --- core/chains/evm/txmgr/evm_tx_store.go | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/core/chains/evm/txmgr/evm_tx_store.go b/core/chains/evm/txmgr/evm_tx_store.go index 9a3e2891841..a3e16eadf49 100644 --- a/core/chains/evm/txmgr/evm_tx_store.go +++ b/core/chains/evm/txmgr/evm_tx_store.go @@ -133,7 +133,6 @@ type dbReceiptPlus struct { ID uuid.UUID `db:"pipeline_task_run_id"` Receipt evmtypes.Receipt `db:"receipt"` FailOnRevert bool `db:"FailOnRevert"` - State string `db:"state"` } func fromDBReceipts(rs []DbReceipt) []*evmtypes.Receipt { @@ -1086,7 +1085,7 @@ func (o *evmTxStore) FindTxWithIdempotencyKey(ctx context.Context, idempotencyKe return } -// FindTxWithSequence returns any broadcast ethtx with the given nonce +// FindTxWithSequenceForRebroadcast returns any broadcast ethtx with the given nonce func (o *evmTxStore) FindTxWithSequenceForRebroadcast(ctx context.Context, fromAddress common.Address, nonce evmtypes.Nonce) (etx *Tx, err error) { var cancel context.CancelFunc ctx, cancel = o.stopCh.Ctx(ctx) @@ -1095,7 +1094,7 @@ func (o *evmTxStore) FindTxWithSequenceForRebroadcast(ctx context.Context, fromA err = o.Transact(ctx, true, func(orm *evmTxStore) error { var dbEtx DbEthTx err = orm.q.GetContext(ctx, &dbEtx, ` -SELECT * FROM evm.txes WHERE from_address = $1 AND nonce = $2 AND state IN ('confirmed', 'confirmed_missing_receipt', 'unconfirmed', 'finalized') +SELECT * FROM evm.txes WHERE from_address = $1 AND nonce = $2 AND state IN ('confirmed', 'confirmed_missing_receipt', 'unconfirmed') `, fromAddress, nonce.Int64()) if err != nil { return pkgerrors.Wrap(err, "FindEthTxWithNonce failed to load evm.txes") From c24408b99d3b4e3e31c68d1dbdb481ae798a576d Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Thu, 8 Aug 2024 19:25:31 -0500 Subject: [PATCH 35/46] one more --- core/chains/evm/txmgr/evm_tx_store.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/chains/evm/txmgr/evm_tx_store.go b/core/chains/evm/txmgr/evm_tx_store.go index a3e16eadf49..9b4bac70eb8 100644 --- a/core/chains/evm/txmgr/evm_tx_store.go +++ b/core/chains/evm/txmgr/evm_tx_store.go @@ -1027,7 +1027,7 @@ func (o *evmTxStore) FindTxesPendingCallback(ctx context.Context, chainID *big.I 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.pipeline_task_run_id IS NOT NULL AND evm.txes.signal_callback = TRUE AND evm.txes.callback_completed = FALSE - AND evm.txes.evm_chain_id = $1 + AND evm.txes.state = 'finalized' AND evm.txes.evm_chain_id = $1 `, chainID.String()) if err != nil { return nil, fmt.Errorf("failed to retrieve transactions pending pipeline resume callback: %w", err) From bc148c9d2f1c38922810bbb81bb8dff1c646969d Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Thu, 8 Aug 2024 19:50:54 -0500 Subject: [PATCH 36/46] revert --- common/txmgr/confirmer.go | 2 +- common/txmgr/types/mocks/tx_store.go | 82 +++------------------ common/txmgr/types/tx_store.go | 5 +- core/chains/evm/txmgr/confirmer_test.go | 6 +- core/chains/evm/txmgr/evm_tx_store.go | 30 +------- core/chains/evm/txmgr/evm_tx_store_test.go | 4 +- core/chains/evm/txmgr/mocks/evm_tx_store.go | 82 +++------------------ 7 files changed, 33 insertions(+), 178 deletions(-) diff --git a/common/txmgr/confirmer.go b/common/txmgr/confirmer.go index 5c328a5bca0..35c1117c2e0 100644 --- a/common/txmgr/confirmer.go +++ b/common/txmgr/confirmer.go @@ -1141,7 +1141,7 @@ func (ec *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) For ec.lggr.Infof("ForceRebroadcast: will rebroadcast transactions for all sequences between %v and %v", seqs[0], seqs[len(seqs)-1]) for _, seq := range seqs { - etx, err := ec.txStore.FindTxWithSequenceForRebroadcast(ctx, address, seq) + etx, err := ec.txStore.FindTxWithSequence(ctx, address, seq) if err != nil { return fmt.Errorf("ForceRebroadcast failed: %w", err) } diff --git a/common/txmgr/types/mocks/tx_store.go b/common/txmgr/types/mocks/tx_store.go index acdb86e9c5a..84eb25c8114 100644 --- a/common/txmgr/types/mocks/tx_store.go +++ b/common/txmgr/types/mocks/tx_store.go @@ -555,66 +555,6 @@ func (_c *TxStore_FindEarliestUnconfirmedTxAttemptBlock_Call[ADDR, CHAIN_ID, TX_ return _c } -// FindFinalizedTxWithSequence provides a mock function with given fields: ctx, fromAddress, seq -func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindFinalizedTxWithSequence(ctx context.Context, fromAddress ADDR, seq SEQ) (*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error) { - ret := _m.Called(ctx, fromAddress, seq) - - if len(ret) == 0 { - panic("no return value specified for FindFinalizedTxWithSequence") - } - - 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, ADDR, SEQ) (*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error)); ok { - return rf(ctx, fromAddress, seq) - } - if rf, ok := ret.Get(0).(func(context.Context, ADDR, SEQ) *txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]); ok { - r0 = rf(ctx, fromAddress, seq) - } 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, ADDR, SEQ) error); ok { - r1 = rf(ctx, fromAddress, seq) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - -// TxStore_FindFinalizedTxWithSequence_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'FindFinalizedTxWithSequence' -type TxStore_FindFinalizedTxWithSequence_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 -} - -// FindFinalizedTxWithSequence is a helper method to define mock.On call -// - ctx context.Context -// - fromAddress ADDR -// - seq SEQ -func (_e *TxStore_Expecter[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindFinalizedTxWithSequence(ctx interface{}, fromAddress interface{}, seq interface{}) *TxStore_FindFinalizedTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { - return &TxStore_FindFinalizedTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]{Call: _e.mock.On("FindFinalizedTxWithSequence", ctx, fromAddress, seq)} -} - -func (_c *TxStore_FindFinalizedTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) Run(run func(ctx context.Context, fromAddress ADDR, seq SEQ)) *TxStore_FindFinalizedTxWithSequence_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].(ADDR), args[2].(SEQ)) - }) - return _c -} - -func (_c *TxStore_FindFinalizedTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) Return(etx *txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) *TxStore_FindFinalizedTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { - _c.Call.Return(etx, err) - return _c -} - -func (_c *TxStore_FindFinalizedTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) RunAndReturn(run func(context.Context, ADDR, SEQ) (*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error)) *TxStore_FindFinalizedTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { - _c.Call.Return(run) - return _c -} - // FindLatestSequence provides a mock function with given fields: ctx, fromAddress, chainId func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindLatestSequence(ctx context.Context, fromAddress ADDR, chainId CHAIN_ID) (SEQ, error) { ret := _m.Called(ctx, fromAddress, chainId) @@ -1034,12 +974,12 @@ func (_c *TxStore_FindTxWithIdempotencyKey_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_H return _c } -// FindTxWithSequenceForRebroadcast provides a mock function with given fields: ctx, fromAddress, seq -func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxWithSequenceForRebroadcast(ctx context.Context, fromAddress ADDR, seq SEQ) (*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error) { +// FindTxWithSequence provides a mock function with given fields: ctx, fromAddress, seq +func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxWithSequence(ctx context.Context, fromAddress ADDR, seq SEQ) (*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error) { ret := _m.Called(ctx, fromAddress, seq) if len(ret) == 0 { - panic("no return value specified for FindTxWithSequenceForRebroadcast") + panic("no return value specified for FindTxWithSequence") } var r0 *txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] @@ -1064,32 +1004,32 @@ func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxWithS return r0, r1 } -// TxStore_FindTxWithSequenceForRebroadcast_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'FindTxWithSequenceForRebroadcast' -type TxStore_FindTxWithSequenceForRebroadcast_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 { +// TxStore_FindTxWithSequence_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'FindTxWithSequence' +type TxStore_FindTxWithSequence_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 } -// FindTxWithSequenceForRebroadcast is a helper method to define mock.On call +// FindTxWithSequence is a helper method to define mock.On call // - ctx context.Context // - fromAddress ADDR // - seq SEQ -func (_e *TxStore_Expecter[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxWithSequenceForRebroadcast(ctx interface{}, fromAddress interface{}, seq interface{}) *TxStore_FindTxWithSequenceForRebroadcast_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { - return &TxStore_FindTxWithSequenceForRebroadcast_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]{Call: _e.mock.On("FindTxWithSequenceForRebroadcast", ctx, fromAddress, seq)} +func (_e *TxStore_Expecter[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindTxWithSequence(ctx interface{}, fromAddress interface{}, seq interface{}) *TxStore_FindTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { + return &TxStore_FindTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]{Call: _e.mock.On("FindTxWithSequence", ctx, fromAddress, seq)} } -func (_c *TxStore_FindTxWithSequenceForRebroadcast_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) Run(run func(ctx context.Context, fromAddress ADDR, seq SEQ)) *TxStore_FindTxWithSequenceForRebroadcast_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { +func (_c *TxStore_FindTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) Run(run func(ctx context.Context, fromAddress ADDR, seq SEQ)) *TxStore_FindTxWithSequence_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].(ADDR), args[2].(SEQ)) }) return _c } -func (_c *TxStore_FindTxWithSequenceForRebroadcast_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) Return(etx *txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) *TxStore_FindTxWithSequenceForRebroadcast_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { +func (_c *TxStore_FindTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) Return(etx *txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) *TxStore_FindTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { _c.Call.Return(etx, err) return _c } -func (_c *TxStore_FindTxWithSequenceForRebroadcast_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) RunAndReturn(run func(context.Context, ADDR, SEQ) (*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error)) *TxStore_FindTxWithSequenceForRebroadcast_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { +func (_c *TxStore_FindTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) RunAndReturn(run func(context.Context, ADDR, SEQ) (*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error)) *TxStore_FindTxWithSequence_Call[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { _c.Call.Return(run) return _c } diff --git a/common/txmgr/types/tx_store.go b/common/txmgr/types/tx_store.go index c415629238a..2138ce38409 100644 --- a/common/txmgr/types/tx_store.go +++ b/common/txmgr/types/tx_store.go @@ -76,9 +76,8 @@ type TransactionStore[ FindTxAttemptsRequiringResend(ctx context.Context, olderThan time.Time, maxInFlightTransactions uint32, chainID CHAIN_ID, address ADDR) (attempts []TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) // Search for Tx using the idempotencyKey and chainID FindTxWithIdempotencyKey(ctx context.Context, idempotencyKey string, chainID CHAIN_ID) (tx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) - // Search for Tx for rebroadcast using the fromAddress and sequence - FindTxWithSequenceForRebroadcast(ctx context.Context, fromAddress ADDR, seq SEQ) (etx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) - FindFinalizedTxWithSequence(ctx context.Context, fromAddress ADDR, seq SEQ) (etx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) + // Search for Tx using the fromAddress and sequence + FindTxWithSequence(ctx context.Context, fromAddress ADDR, seq SEQ) (etx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) FindNextUnstartedTransactionFromAddress(ctx context.Context, fromAddress ADDR, chainID CHAIN_ID) (*Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], error) 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) diff --git a/core/chains/evm/txmgr/confirmer_test.go b/core/chains/evm/txmgr/confirmer_test.go index 1da66f72fdf..7ef610202fa 100644 --- a/core/chains/evm/txmgr/confirmer_test.go +++ b/core/chains/evm/txmgr/confirmer_test.go @@ -3013,7 +3013,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { return } // Retrieve Tx to check if callback completed flag was set to true - updateTx, err3 := txStore.FindFinalizedTxWithSequence(tests.Context(t), fromAddress, nonce) + updateTx, err3 := txStore.FindTxWithSequence(tests.Context(t), fromAddress, nonce) if assert.NoError(t, err3) { assert.Equal(t, true, updateTx.CallbackCompleted) } @@ -3067,7 +3067,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { return } // Retrieve Tx to check if callback completed flag was set to true - updateTx, err3 := txStore.FindTxWithSequenceForRebroadcast(tests.Context(t), fromAddress, nonce) + updateTx, err3 := txStore.FindTxWithSequence(tests.Context(t), fromAddress, nonce) if assert.NoError(t, err3) { assert.Equal(t, true, updateTx.CallbackCompleted) } @@ -3103,7 +3103,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { require.Error(t, err) // Retrieve Tx to check if callback completed flag was left unchanged - updateTx, err := txStore.FindTxWithSequenceForRebroadcast(tests.Context(t), fromAddress, nonce) + updateTx, err := txStore.FindTxWithSequence(tests.Context(t), fromAddress, nonce) require.NoError(t, err) require.Equal(t, false, updateTx.CallbackCompleted) }) diff --git a/core/chains/evm/txmgr/evm_tx_store.go b/core/chains/evm/txmgr/evm_tx_store.go index 9b4bac70eb8..86ac6c0475f 100644 --- a/core/chains/evm/txmgr/evm_tx_store.go +++ b/core/chains/evm/txmgr/evm_tx_store.go @@ -1023,7 +1023,7 @@ func (o *evmTxStore) FindTxesPendingCallback(ctx context.Context, chainID *big.I defer cancel() err = o.q.SelectContext(ctx, &rs, ` - SELECT evm.txes.state, evm.txes.pipeline_task_run_id, evm.receipts.receipt, COALESCE((evm.txes.meta->>'FailOnRevert')::boolean, false) "FailOnRevert" FROM evm.txes + SELECT evm.txes.pipeline_task_run_id, evm.receipts.receipt, COALESCE((evm.txes.meta->>'FailOnRevert')::boolean, false) "FailOnRevert" 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.pipeline_task_run_id IS NOT NULL AND evm.txes.signal_callback = TRUE AND evm.txes.callback_completed = FALSE @@ -1085,8 +1085,8 @@ func (o *evmTxStore) FindTxWithIdempotencyKey(ctx context.Context, idempotencyKe return } -// FindTxWithSequenceForRebroadcast returns any broadcast ethtx with the given nonce -func (o *evmTxStore) FindTxWithSequenceForRebroadcast(ctx context.Context, fromAddress common.Address, nonce evmtypes.Nonce) (etx *Tx, err error) { +// FindTxWithSequence returns any broadcast ethtx with the given nonce +func (o *evmTxStore) FindTxWithSequence(ctx context.Context, fromAddress common.Address, nonce evmtypes.Nonce) (etx *Tx, err error) { var cancel context.CancelFunc ctx, cancel = o.stopCh.Ctx(ctx) defer cancel() @@ -1109,30 +1109,6 @@ SELECT * FROM evm.txes WHERE from_address = $1 AND nonce = $2 AND state IN ('con return } -// FindFinalizedTxWithSequence returns finalized ethtx with the given nonce -func (o *evmTxStore) FindFinalizedTxWithSequence(ctx context.Context, fromAddress common.Address, nonce evmtypes.Nonce) (etx *Tx, err error) { - var cancel context.CancelFunc - ctx, cancel = o.stopCh.Ctx(ctx) - defer cancel() - etx = new(Tx) - err = o.Transact(ctx, true, func(orm *evmTxStore) error { - var dbEtx DbEthTx - err = orm.q.GetContext(ctx, &dbEtx, ` -SELECT * FROM evm.txes WHERE from_address = $1 AND nonce = $2 AND state = 'finalized' -`, fromAddress, nonce.Int64()) - if err != nil { - return pkgerrors.Wrap(err, "FindEthTxWithNonce failed to load evm.txes") - } - dbEtx.ToTx(etx) - err = orm.loadTxAttemptsAtomic(ctx, etx) - return pkgerrors.Wrap(err, "FindEthTxWithNonce failed to load evm.tx_attempts") - }) - if errors.Is(err, sql.ErrNoRows) { - return nil, nil - } - return -} - func updateEthTxAttemptUnbroadcast(ctx context.Context, orm *evmTxStore, attempt TxAttempt) error { if attempt.State != txmgrtypes.TxAttemptBroadcast { return errors.New("expected eth_tx_attempt to be broadcast") diff --git a/core/chains/evm/txmgr/evm_tx_store_test.go b/core/chains/evm/txmgr/evm_tx_store_test.go index 0c5db83c4f4..4daa790de61 100644 --- a/core/chains/evm/txmgr/evm_tx_store_test.go +++ b/core/chains/evm/txmgr/evm_tx_store_test.go @@ -730,7 +730,7 @@ func TestORM_FindTxWithSequence(t *testing.T) { _, fromAddress := cltest.MustInsertRandomKeyReturningState(t, ethKeyStore) t.Run("returns nil if no results", func(t *testing.T) { - etx, err := txStore.FindTxWithSequenceForRebroadcast(tests.Context(t), fromAddress, evmtypes.Nonce(777)) + etx, err := txStore.FindTxWithSequence(tests.Context(t), fromAddress, evmtypes.Nonce(777)) require.NoError(t, err) assert.Nil(t, etx) }) @@ -739,7 +739,7 @@ func TestORM_FindTxWithSequence(t *testing.T) { etx := cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, 777, 1, fromAddress) require.Equal(t, evmtypes.Nonce(777), *etx.Sequence) - res, err := txStore.FindTxWithSequenceForRebroadcast(tests.Context(t), fromAddress, evmtypes.Nonce(777)) + res, err := txStore.FindTxWithSequence(tests.Context(t), fromAddress, evmtypes.Nonce(777)) require.NoError(t, err) assert.Equal(t, etx.Sequence, res.Sequence) }) diff --git a/core/chains/evm/txmgr/mocks/evm_tx_store.go b/core/chains/evm/txmgr/mocks/evm_tx_store.go index ecc502c6340..59f7eb40903 100644 --- a/core/chains/evm/txmgr/mocks/evm_tx_store.go +++ b/core/chains/evm/txmgr/mocks/evm_tx_store.go @@ -620,66 +620,6 @@ func (_c *EvmTxStore_FindEarliestUnconfirmedTxAttemptBlock_Call) RunAndReturn(ru return _c } -// FindFinalizedTxWithSequence provides a mock function with given fields: ctx, fromAddress, seq -func (_m *EvmTxStore) FindFinalizedTxWithSequence(ctx context.Context, fromAddress common.Address, seq evmtypes.Nonce) (*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error) { - ret := _m.Called(ctx, fromAddress, seq) - - if len(ret) == 0 { - panic("no return value specified for FindFinalizedTxWithSequence") - } - - 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, common.Address, evmtypes.Nonce) (*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error)); ok { - return rf(ctx, fromAddress, seq) - } - if rf, ok := ret.Get(0).(func(context.Context, common.Address, evmtypes.Nonce) *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee]); ok { - r0 = rf(ctx, fromAddress, seq) - } 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, common.Address, evmtypes.Nonce) error); ok { - r1 = rf(ctx, fromAddress, seq) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - -// EvmTxStore_FindFinalizedTxWithSequence_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'FindFinalizedTxWithSequence' -type EvmTxStore_FindFinalizedTxWithSequence_Call struct { - *mock.Call -} - -// FindFinalizedTxWithSequence is a helper method to define mock.On call -// - ctx context.Context -// - fromAddress common.Address -// - seq evmtypes.Nonce -func (_e *EvmTxStore_Expecter) FindFinalizedTxWithSequence(ctx interface{}, fromAddress interface{}, seq interface{}) *EvmTxStore_FindFinalizedTxWithSequence_Call { - return &EvmTxStore_FindFinalizedTxWithSequence_Call{Call: _e.mock.On("FindFinalizedTxWithSequence", ctx, fromAddress, seq)} -} - -func (_c *EvmTxStore_FindFinalizedTxWithSequence_Call) Run(run func(ctx context.Context, fromAddress common.Address, seq evmtypes.Nonce)) *EvmTxStore_FindFinalizedTxWithSequence_Call { - _c.Call.Run(func(args mock.Arguments) { - run(args[0].(context.Context), args[1].(common.Address), args[2].(evmtypes.Nonce)) - }) - return _c -} - -func (_c *EvmTxStore_FindFinalizedTxWithSequence_Call) Return(etx *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], err error) *EvmTxStore_FindFinalizedTxWithSequence_Call { - _c.Call.Return(etx, err) - return _c -} - -func (_c *EvmTxStore_FindFinalizedTxWithSequence_Call) RunAndReturn(run func(context.Context, common.Address, evmtypes.Nonce) (*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error)) *EvmTxStore_FindFinalizedTxWithSequence_Call { - _c.Call.Return(run) - return _c -} - // FindLatestSequence provides a mock function with given fields: ctx, fromAddress, chainId func (_m *EvmTxStore) FindLatestSequence(ctx context.Context, fromAddress common.Address, chainId *big.Int) (evmtypes.Nonce, error) { ret := _m.Called(ctx, fromAddress, chainId) @@ -1333,12 +1273,12 @@ func (_c *EvmTxStore_FindTxWithIdempotencyKey_Call) RunAndReturn(run func(contex return _c } -// FindTxWithSequenceForRebroadcast provides a mock function with given fields: ctx, fromAddress, seq -func (_m *EvmTxStore) FindTxWithSequenceForRebroadcast(ctx context.Context, fromAddress common.Address, seq evmtypes.Nonce) (*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error) { +// FindTxWithSequence provides a mock function with given fields: ctx, fromAddress, seq +func (_m *EvmTxStore) FindTxWithSequence(ctx context.Context, fromAddress common.Address, seq evmtypes.Nonce) (*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error) { ret := _m.Called(ctx, fromAddress, seq) if len(ret) == 0 { - panic("no return value specified for FindTxWithSequenceForRebroadcast") + panic("no return value specified for FindTxWithSequence") } var r0 *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee] @@ -1363,32 +1303,32 @@ func (_m *EvmTxStore) FindTxWithSequenceForRebroadcast(ctx context.Context, from return r0, r1 } -// EvmTxStore_FindTxWithSequenceForRebroadcast_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'FindTxWithSequenceForRebroadcast' -type EvmTxStore_FindTxWithSequenceForRebroadcast_Call struct { +// EvmTxStore_FindTxWithSequence_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'FindTxWithSequence' +type EvmTxStore_FindTxWithSequence_Call struct { *mock.Call } -// FindTxWithSequenceForRebroadcast is a helper method to define mock.On call +// FindTxWithSequence is a helper method to define mock.On call // - ctx context.Context // - fromAddress common.Address // - seq evmtypes.Nonce -func (_e *EvmTxStore_Expecter) FindTxWithSequenceForRebroadcast(ctx interface{}, fromAddress interface{}, seq interface{}) *EvmTxStore_FindTxWithSequenceForRebroadcast_Call { - return &EvmTxStore_FindTxWithSequenceForRebroadcast_Call{Call: _e.mock.On("FindTxWithSequenceForRebroadcast", ctx, fromAddress, seq)} +func (_e *EvmTxStore_Expecter) FindTxWithSequence(ctx interface{}, fromAddress interface{}, seq interface{}) *EvmTxStore_FindTxWithSequence_Call { + return &EvmTxStore_FindTxWithSequence_Call{Call: _e.mock.On("FindTxWithSequence", ctx, fromAddress, seq)} } -func (_c *EvmTxStore_FindTxWithSequenceForRebroadcast_Call) Run(run func(ctx context.Context, fromAddress common.Address, seq evmtypes.Nonce)) *EvmTxStore_FindTxWithSequenceForRebroadcast_Call { +func (_c *EvmTxStore_FindTxWithSequence_Call) Run(run func(ctx context.Context, fromAddress common.Address, seq evmtypes.Nonce)) *EvmTxStore_FindTxWithSequence_Call { _c.Call.Run(func(args mock.Arguments) { run(args[0].(context.Context), args[1].(common.Address), args[2].(evmtypes.Nonce)) }) return _c } -func (_c *EvmTxStore_FindTxWithSequenceForRebroadcast_Call) Return(etx *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], err error) *EvmTxStore_FindTxWithSequenceForRebroadcast_Call { +func (_c *EvmTxStore_FindTxWithSequence_Call) Return(etx *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], err error) *EvmTxStore_FindTxWithSequence_Call { _c.Call.Return(etx, err) return _c } -func (_c *EvmTxStore_FindTxWithSequenceForRebroadcast_Call) RunAndReturn(run func(context.Context, common.Address, evmtypes.Nonce) (*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error)) *EvmTxStore_FindTxWithSequenceForRebroadcast_Call { +func (_c *EvmTxStore_FindTxWithSequence_Call) RunAndReturn(run func(context.Context, common.Address, evmtypes.Nonce) (*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], error)) *EvmTxStore_FindTxWithSequence_Call { _c.Call.Return(run) return _c } From 2b195c7ee6737c5eb707d93078c2027cf9cae46d Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Thu, 8 Aug 2024 19:57:36 -0500 Subject: [PATCH 37/46] Amit comments, more work needed --- common/txmgr/types/tx_store.go | 2 +- core/chains/evm/txmgr/evm_tx_store_test.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/common/txmgr/types/tx_store.go b/common/txmgr/types/tx_store.go index 2138ce38409..68aa384596a 100644 --- a/common/txmgr/types/tx_store.go +++ b/common/txmgr/types/tx_store.go @@ -33,7 +33,7 @@ type TxStore[ TxHistoryReaper[CHAIN_ID] TransactionStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, SEQ, FEE] - // Find confirmed txes with finalized state that require callback but have not yet been signaled + // Find finalized txes that require callback but have not yet been signaled FindTxesPendingCallback(ctx context.Context, chainID CHAIN_ID) (receiptsPlus []ReceiptPlus[R], err error) // Update tx to mark that its callback has been signaled UpdateTxCallbackCompleted(ctx context.Context, pipelineTaskRunRid uuid.UUID, chainId CHAIN_ID) error diff --git a/core/chains/evm/txmgr/evm_tx_store_test.go b/core/chains/evm/txmgr/evm_tx_store_test.go index 4daa790de61..2b240941172 100644 --- a/core/chains/evm/txmgr/evm_tx_store_test.go +++ b/core/chains/evm/txmgr/evm_tx_store_test.go @@ -661,7 +661,7 @@ func TestORM_FindTxesPendingCallback(t *testing.T) { mustInsertEthReceipt(t, txStore, head.BlockNumber()-2, head.Hash, attempt2.Hash) pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE, callback_completed = TRUE WHERE id = $2`, &tr2.ID, etx2.ID) - // Suspend run after head block number. Should be ignored + // Suspend not yet finalized run. Should be ignored run3 := cltest.MustInsertPipelineRun(t, db) tr3 := cltest.MustInsertUnfinishedPipelineTaskRun(t, db, run3.ID) pgtest.MustExec(t, db, `UPDATE pipeline_runs SET state = 'suspended' WHERE id = $1`, run3.ID) From 755593cfdb3db664a2b823b67880a6842a9e9b17 Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Thu, 8 Aug 2024 21:33:58 -0500 Subject: [PATCH 38/46] fix test --- core/chains/evm/txmgr/confirmer_test.go | 71 +++++++------------------ core/chains/evm/txmgr/evm_tx_store.go | 13 +++++ core/internal/cltest/factories.go | 20 +++++++ 3 files changed, 51 insertions(+), 53 deletions(-) diff --git a/core/chains/evm/txmgr/confirmer_test.go b/core/chains/evm/txmgr/confirmer_test.go index 7ef610202fa..70a3a4c08f7 100644 --- a/core/chains/evm/txmgr/confirmer_test.go +++ b/core/chains/evm/txmgr/confirmer_test.go @@ -2980,61 +2980,25 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { require.NoError(t, err) }) - t.Run("processes eth_txes that's finalized", func(t *testing.T) { - ch := make(chan interface{}) - nonce := evmtypes.Nonce(3) - var err error - ec := newEthConfirmer(t, txStore, ethClient, config, evmcfg, ethKeyStore, func(ctx context.Context, id uuid.UUID, value interface{}, thisErr error) error { - err = thisErr - ch <- value + t.Run("doesn't process task runs where the receipt is not finalized", func(t *testing.T) { + ec := newEthConfirmer(t, txStore, ethClient, config, evmcfg, ethKeyStore, func(context.Context, uuid.UUID, interface{}, error) error { + t.Fatal("No value expected") return nil }) run := cltest.MustInsertPipelineRun(t, db) tr := cltest.MustInsertUnfinishedPipelineTaskRun(t, db, run.ID) - pgtest.MustExec(t, db, `UPDATE pipeline_runs SET state = 'suspended' WHERE id = $1`, run.ID) - etx := cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, int64(nonce), 1, fromAddress) - pgtest.MustExec(t, db, `UPDATE evm.txes SET meta='{"FailOnRevert": true}'`) - receipt := mustInsertEthReceipt(t, txStore, head.Number, head.Hash, etx.TxAttempts[0].Hash) + etx := cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, 2, 1, fromAddress) + mustInsertEthReceipt(t, txStore, head.Number, head.Hash, etx.TxAttempts[0].Hash) - pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE WHERE id = $2`, &tr.ID, etx.ID) + pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE, callback_completed = TRUE WHERE id = $2`, &tr.ID, etx.ID) - // set tx state to be finalized - err = txStore.UpdateTxStatesToFinalizedUsingReceiptIds(tests.Context(t), []int64{receipt.ID}, testutils.FixtureChainID) + err := ec.ResumePendingTaskRuns(tests.Context(t)) require.NoError(t, err) - - done := make(chan struct{}) - t.Cleanup(func() { <-done }) - go func() { - defer close(done) - err2 := ec.ResumePendingTaskRuns(tests.Context(t)) - if !assert.NoError(t, err2) { - return - } - // Retrieve Tx to check if callback completed flag was set to true - updateTx, err3 := txStore.FindTxWithSequence(tests.Context(t), fromAddress, nonce) - if assert.NoError(t, err3) { - assert.Equal(t, true, updateTx.CallbackCompleted) - } - }() - - select { - case data := <-ch: - assert.NoError(t, err) - - require.IsType(t, &evmtypes.Receipt{}, data) - r := data.(*evmtypes.Receipt) - require.Equal(t, receipt.TxHash, r.TxHash) - - case <-time.After(time.Second): - t.Fatal("no value received") - } }) - pgtest.MustExec(t, db, `DELETE FROM pipeline_runs`) - - t.Run("processes eth_txes with receipt before current head that reverted", func(t *testing.T) { + t.Run("processes eth_txes with receipt for finalized tx", func(t *testing.T) { type data struct { value any error @@ -3050,7 +3014,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { tr := cltest.MustInsertUnfinishedPipelineTaskRun(t, db, run.ID) pgtest.MustExec(t, db, `UPDATE pipeline_runs SET state = 'suspended' WHERE id = $1`, run.ID) - etx := cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, int64(nonce), 1, fromAddress) + etx := cltest.MustInsertFinalizedEthTxWithLegacyAttempt(t, txStore, int64(nonce), 1, fromAddress) pgtest.MustExec(t, db, `UPDATE evm.txes SET meta='{"FailOnRevert": true}'`) // receipt is not passed through as a value since it reverted and caused an error @@ -3067,10 +3031,10 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { return } // Retrieve Tx to check if callback completed flag was set to true - updateTx, err3 := txStore.FindTxWithSequence(tests.Context(t), fromAddress, nonce) - if assert.NoError(t, err3) { - assert.Equal(t, true, updateTx.CallbackCompleted) - } + txs, err := txStore.FindTxesByFromAddressAndNonce(tests.Context(t), fromAddress, int64(nonce)) + assert.Nil(t, err) + assert.Equal(t, 1, len(txs)) + assert.Equal(t, true, txs[0].CallbackCompleted) }() select { @@ -3095,7 +3059,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { run := cltest.MustInsertPipelineRun(t, db) tr := cltest.MustInsertUnfinishedPipelineTaskRun(t, db, run.ID) - etx := cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, int64(nonce), 1, fromAddress) + etx := cltest.MustInsertFinalizedEthTxWithLegacyAttempt(t, txStore, int64(nonce), 1, fromAddress) mustInsertEthReceipt(t, txStore, head.Number, head.Hash, etx.TxAttempts[0].Hash) pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE WHERE id = $2`, &tr.ID, etx.ID) @@ -3103,9 +3067,10 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { require.Error(t, err) // Retrieve Tx to check if callback completed flag was left unchanged - updateTx, err := txStore.FindTxWithSequence(tests.Context(t), fromAddress, nonce) - require.NoError(t, err) - require.Equal(t, false, updateTx.CallbackCompleted) + txs, err := txStore.FindTxesByFromAddressAndNonce(tests.Context(t), fromAddress, int64(nonce)) + assert.Nil(t, err) + assert.Equal(t, 1, len(txs)) + assert.Equal(t, false, txs[0].CallbackCompleted) }) } diff --git a/core/chains/evm/txmgr/evm_tx_store.go b/core/chains/evm/txmgr/evm_tx_store.go index 86ac6c0475f..418bd3b9135 100644 --- a/core/chains/evm/txmgr/evm_tx_store.go +++ b/core/chains/evm/txmgr/evm_tx_store.go @@ -76,6 +76,7 @@ type TestEvmTxStore interface { GetAllTxAttempts(ctx context.Context) (attempts []TxAttempt, err error) CountTxesByStateAndSubject(ctx context.Context, state txmgrtypes.TxState, subject uuid.UUID) (count int, err error) FindTxesByFromAddressAndState(ctx context.Context, fromAddress common.Address, state string) (txes []*Tx, err error) + FindTxesByFromAddressAndNonce(ctx context.Context, fromAddress common.Address, nonce int64) (txes []*Tx, err error) UpdateTxAttemptBroadcastBeforeBlockNum(ctx context.Context, id int64, blockNum uint) error } @@ -2063,6 +2064,18 @@ func (o *evmTxStore) FindTxesByFromAddressAndState(ctx context.Context, fromAddr return txes, err } +func (o *evmTxStore) FindTxesByFromAddressAndNonce(ctx context.Context, fromAddress common.Address, nonce int64) (txes []*Tx, err error) { + var cancel context.CancelFunc + ctx, cancel = o.stopCh.Ctx(ctx) + defer cancel() + sql := "SELECT * FROM evm.txes WHERE from_address = $1 AND nonce = $2" + var dbEtxs []DbEthTx + err = o.q.SelectContext(ctx, &dbEtxs, sql, fromAddress, nonce) + txes = make([]*Tx, len(dbEtxs)) + dbEthTxsToEvmEthTxPtrs(dbEtxs, txes) + return txes, err +} + func (o *evmTxStore) UpdateTxAttemptBroadcastBeforeBlockNum(ctx context.Context, id int64, blockNum uint) error { var cancel context.CancelFunc ctx, cancel = o.stopCh.Ctx(ctx) diff --git a/core/internal/cltest/factories.go b/core/internal/cltest/factories.go index c488dca94a9..1dc318c3d12 100644 --- a/core/internal/cltest/factories.go +++ b/core/internal/cltest/factories.go @@ -220,6 +220,26 @@ func MustInsertConfirmedEthTxWithLegacyAttempt(t *testing.T, txStore txmgr.TestE return etx } +func MustInsertFinalizedEthTxWithLegacyAttempt(t *testing.T, txStore txmgr.TestEvmTxStore, nonce int64, broadcastBeforeBlockNum int64, fromAddress common.Address) txmgr.Tx { + timeNow := time.Now() + etx := NewEthTx(fromAddress) + ctx := testutils.Context(t) + + etx.BroadcastAt = &timeNow + etx.InitialBroadcastAt = &timeNow + n := evmtypes.Nonce(nonce) + etx.Sequence = &n + etx.State = txmgrcommon.TxFinalized + etx.MinConfirmations.SetValid(6) + require.NoError(t, txStore.InsertTx(ctx, &etx)) + attempt := NewLegacyEthTxAttempt(t, etx.ID) + attempt.BroadcastBeforeBlockNum = &broadcastBeforeBlockNum + attempt.State = txmgrtypes.TxAttemptBroadcast + require.NoError(t, txStore.InsertTxAttempt(ctx, &attempt)) + etx.TxAttempts = append(etx.TxAttempts, attempt) + return etx +} + func NewLegacyEthTxAttempt(t *testing.T, etxID int64) txmgr.TxAttempt { gasPrice := assets.NewWeiI(1) return txmgr.TxAttempt{ From f8463ac51194bfb1deb035e38b7f50450bca9664 Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Thu, 8 Aug 2024 21:39:46 -0500 Subject: [PATCH 39/46] add one deleted test back --- core/chains/evm/txmgr/confirmer_test.go | 52 ++++++++++++++++++++++++- 1 file changed, 51 insertions(+), 1 deletion(-) diff --git a/core/chains/evm/txmgr/confirmer_test.go b/core/chains/evm/txmgr/confirmer_test.go index 70a3a4c08f7..d5bb335d3d9 100644 --- a/core/chains/evm/txmgr/confirmer_test.go +++ b/core/chains/evm/txmgr/confirmer_test.go @@ -2998,7 +2998,57 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { require.NoError(t, err) }) - t.Run("processes eth_txes with receipt for finalized tx", func(t *testing.T) { + t.Run("processes eth_txes with receipts for finalized tx", func(t *testing.T) { + ch := make(chan interface{}) + nonce := evmtypes.Nonce(3) + var err error + ec := newEthConfirmer(t, txStore, ethClient, config, evmcfg, ethKeyStore, func(ctx context.Context, id uuid.UUID, value interface{}, thisErr error) error { + err = thisErr + ch <- value + return nil + }) + + run := cltest.MustInsertPipelineRun(t, db) + tr := cltest.MustInsertUnfinishedPipelineTaskRun(t, db, run.ID) + pgtest.MustExec(t, db, `UPDATE pipeline_runs SET state = 'suspended' WHERE id = $1`, run.ID) + + etx := cltest.MustInsertFinalizedEthTxWithLegacyAttempt(t, txStore, int64(nonce), 1, fromAddress) + pgtest.MustExec(t, db, `UPDATE evm.txes SET meta='{"FailOnRevert": true}'`) + receipt := mustInsertEthReceipt(t, txStore, head.Number, head.Hash, etx.TxAttempts[0].Hash) + + pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, signal_callback = TRUE WHERE id = $2`, &tr.ID, etx.ID) + + done := make(chan struct{}) + t.Cleanup(func() { <-done }) + go func() { + defer close(done) + err2 := ec.ResumePendingTaskRuns(tests.Context(t)) + if !assert.NoError(t, err2) { + return + } + // Retrieve Tx to check if callback completed flag was set to true + txs, err := txStore.FindTxesByFromAddressAndNonce(tests.Context(t), fromAddress, int64(nonce)) + assert.Nil(t, err) + assert.Equal(t, 1, len(txs)) + assert.Equal(t, true, txs[0].CallbackCompleted) + }() + + select { + case data := <-ch: + assert.NoError(t, err) + + require.IsType(t, &evmtypes.Receipt{}, data) + r := data.(*evmtypes.Receipt) + require.Equal(t, receipt.TxHash, r.TxHash) + + case <-time.After(time.Second): + t.Fatal("no value received") + } + }) + + pgtest.MustExec(t, db, `DELETE FROM pipeline_runs`) + + t.Run("processes eth_txes with receipt for finalized tx that reverted", func(t *testing.T) { type data struct { value any error From e2332fb0b5648a77ab5e3f7a940e915299ba9f59 Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Thu, 8 Aug 2024 23:09:32 -0500 Subject: [PATCH 40/46] fix lint --- core/chains/evm/txmgr/confirmer_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/chains/evm/txmgr/confirmer_test.go b/core/chains/evm/txmgr/confirmer_test.go index d5bb335d3d9..ca3032a0636 100644 --- a/core/chains/evm/txmgr/confirmer_test.go +++ b/core/chains/evm/txmgr/confirmer_test.go @@ -3027,8 +3027,8 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { return } // Retrieve Tx to check if callback completed flag was set to true - txs, err := txStore.FindTxesByFromAddressAndNonce(tests.Context(t), fromAddress, int64(nonce)) - assert.Nil(t, err) + txs, err3 := txStore.FindTxesByFromAddressAndNonce(tests.Context(t), fromAddress, int64(nonce)) + assert.Nil(t, err3) assert.Equal(t, 1, len(txs)) assert.Equal(t, true, txs[0].CallbackCompleted) }() From b45fb5d55e055a0e94d80a53353fd4c45e2d5796 Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Fri, 9 Aug 2024 14:15:38 -0500 Subject: [PATCH 41/46] fix test attempt --- core/internal/features/features_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/internal/features/features_test.go b/core/internal/features/features_test.go index 046f21b7f7d..8c5cad29d9a 100644 --- a/core/internal/features/features_test.go +++ b/core/internal/features/features_test.go @@ -473,6 +473,8 @@ func setupAppForEthTx(t *testing.T, operatorContracts OperatorContracts) (app *c cfg := configtest.NewGeneralConfigSimulated(t, func(c *chainlink.Config, s *chainlink.Secrets) { c.Database.Listener.FallbackPollInterval = commonconfig.MustNewDuration(100 * time.Millisecond) + depth := uint32(0) + c.EVM[0].FinalityDepth = &depth }) app = cltest.NewApplicationWithConfigV2AndKeyOnSimulatedBlockchain(t, cfg, b, lggr) b.Commit() From 07f5cc3d19d04a84eab9006ab66f4c43e2d287f0 Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Fri, 9 Aug 2024 16:17:44 -0500 Subject: [PATCH 42/46] revert --- core/internal/features/features_test.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/internal/features/features_test.go b/core/internal/features/features_test.go index 8c5cad29d9a..046f21b7f7d 100644 --- a/core/internal/features/features_test.go +++ b/core/internal/features/features_test.go @@ -473,8 +473,6 @@ func setupAppForEthTx(t *testing.T, operatorContracts OperatorContracts) (app *c cfg := configtest.NewGeneralConfigSimulated(t, func(c *chainlink.Config, s *chainlink.Secrets) { c.Database.Listener.FallbackPollInterval = commonconfig.MustNewDuration(100 * time.Millisecond) - depth := uint32(0) - c.EVM[0].FinalityDepth = &depth }) app = cltest.NewApplicationWithConfigV2AndKeyOnSimulatedBlockchain(t, cfg, b, lggr) b.Commit() From 8b45996cc03dc7a918d6f7ec92527ba54c592f50 Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Fri, 9 Aug 2024 21:23:53 -0500 Subject: [PATCH 43/46] fix test --- core/chains/evm/client/simulated_backend_client.go | 10 +++++++++- core/internal/features/features_test.go | 4 ++++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/core/chains/evm/client/simulated_backend_client.go b/core/chains/evm/client/simulated_backend_client.go index 7dfd39f444c..8136a9d1594 100644 --- a/core/chains/evm/client/simulated_backend_client.go +++ b/core/chains/evm/client/simulated_backend_client.go @@ -703,7 +703,15 @@ func (c *SimulatedBackendClient) ethGetHeaderByNumber(ctx context.Context, resul } func (c *SimulatedBackendClient) LatestFinalizedBlock(ctx context.Context) (*evmtypes.Head, error) { - block := c.b.Blockchain().CurrentFinalBlock() + var block *types.Header + if blk := c.b.Blockchain().CurrentFinalBlock(); blk != nil { + block = blk + } else if blk = c.b.Blockchain().CurrentBlock(); blk != nil { + block = blk + } else { + return nil, fmt.Errorf("SimulatedBackendClient failed to find latest finalized block") + } + return &evmtypes.Head{ EVMChainID: ubig.NewI(c.chainId.Int64()), Hash: block.Hash(), diff --git a/core/internal/features/features_test.go b/core/internal/features/features_test.go index 046f21b7f7d..b62457387d2 100644 --- a/core/internal/features/features_test.go +++ b/core/internal/features/features_test.go @@ -473,6 +473,10 @@ func setupAppForEthTx(t *testing.T, operatorContracts OperatorContracts) (app *c cfg := configtest.NewGeneralConfigSimulated(t, func(c *chainlink.Config, s *chainlink.Secrets) { c.Database.Listener.FallbackPollInterval = commonconfig.MustNewDuration(100 * time.Millisecond) + finalityTagEnabled := true + FinalityTagBypass := false + c.EVM[0].HeadTracker.FinalityTagBypass = &FinalityTagBypass + c.EVM[0].FinalityTagEnabled = &finalityTagEnabled }) app = cltest.NewApplicationWithConfigV2AndKeyOnSimulatedBlockchain(t, cfg, b, lggr) b.Commit() From ae734e9deedb3cacb1a3b004a14136a43eca3845 Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Sun, 11 Aug 2024 21:45:42 -0500 Subject: [PATCH 44/46] fix race condition for feature test, and fix simulated backend endpoint --- .../evm/client/simulated_backend_client.go | 23 ++++++++----------- core/internal/features/features_test.go | 7 ++++++ 2 files changed, 16 insertions(+), 14 deletions(-) diff --git a/core/chains/evm/client/simulated_backend_client.go b/core/chains/evm/client/simulated_backend_client.go index 8136a9d1594..cbcc1b14940 100644 --- a/core/chains/evm/client/simulated_backend_client.go +++ b/core/chains/evm/client/simulated_backend_client.go @@ -703,22 +703,17 @@ func (c *SimulatedBackendClient) ethGetHeaderByNumber(ctx context.Context, resul } func (c *SimulatedBackendClient) LatestFinalizedBlock(ctx context.Context) (*evmtypes.Head, error) { - var block *types.Header - if blk := c.b.Blockchain().CurrentFinalBlock(); blk != nil { - block = blk - } else if blk = c.b.Blockchain().CurrentBlock(); blk != nil { - block = blk - } else { - return nil, fmt.Errorf("SimulatedBackendClient failed to find latest finalized block") + if block := c.b.Blockchain().CurrentFinalBlock(); block != nil { + return &evmtypes.Head{ + EVMChainID: ubig.NewI(c.chainId.Int64()), + Hash: block.Hash(), + Number: block.Number.Int64(), + ParentHash: block.ParentHash, + Timestamp: time.Unix(int64(block.Time), 0), + }, nil } - return &evmtypes.Head{ - EVMChainID: ubig.NewI(c.chainId.Int64()), - Hash: block.Hash(), - Number: block.Number.Int64(), - ParentHash: block.ParentHash, - Timestamp: time.Unix(int64(block.Time), 0), - }, nil + return nil, fmt.Errorf("SimulatedBackendClient failed to find latest finalized block") } func (c *SimulatedBackendClient) ethGetLogs(ctx context.Context, result interface{}, args ...interface{}) error { diff --git a/core/internal/features/features_test.go b/core/internal/features/features_test.go index b62457387d2..142e1824079 100644 --- a/core/internal/features/features_test.go +++ b/core/internal/features/features_test.go @@ -508,6 +508,7 @@ func TestIntegration_AsyncEthTx(t *testing.T) { t.Parallel() operatorContracts := setupOperatorContracts(t) b := operatorContracts.sim + b.Blockchain().SetFinalized(b.Blockchain().CurrentBlock()) t.Run("with FailOnRevert enabled, run succeeds when transaction is successful", func(t *testing.T) { app, sendingAddr, o := setupAppForEthTx(t, operatorContracts) @@ -536,6 +537,8 @@ observationSource = """ testutils.WaitForLogMessage(t, o, "Sending transaction") b.Commit() // Needs at least two confirmations + time.Sleep(1 * time.Second) + b.Blockchain().SetFinalized(b.Blockchain().CurrentBlock()) b.Commit() // Needs at least two confirmations b.Commit() // Needs at least two confirmations testutils.WaitForLogMessage(t, o, "Resume run success") @@ -582,6 +585,8 @@ observationSource = """ testutils.WaitForLogMessage(t, o, "Sending transaction") b.Commit() // Needs at least two confirmations + time.Sleep(1 * time.Second) + b.Blockchain().SetFinalized(b.Blockchain().CurrentBlock()) b.Commit() // Needs at least two confirmations b.Commit() // Needs at least two confirmations testutils.WaitForLogMessage(t, o, "Resume run success") @@ -620,6 +625,8 @@ observationSource = """ testutils.WaitForLogMessage(t, o, "Sending transaction") b.Commit() // Needs at least two confirmations + time.Sleep(1 * time.Second) + b.Blockchain().SetFinalized(b.Blockchain().CurrentBlock()) b.Commit() // Needs at least two confirmations b.Commit() // Needs at least two confirmations testutils.WaitForLogMessage(t, o, "Resume run success") From f02505470abb84388435a6750b6e3be9beeb43fa Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Sun, 11 Aug 2024 21:47:08 -0500 Subject: [PATCH 45/46] add minConfirmation field in db --- core/chains/evm/txmgr/evm_tx_store.go | 1 + 1 file changed, 1 insertion(+) diff --git a/core/chains/evm/txmgr/evm_tx_store.go b/core/chains/evm/txmgr/evm_tx_store.go index 418bd3b9135..4075ef52a73 100644 --- a/core/chains/evm/txmgr/evm_tx_store.go +++ b/core/chains/evm/txmgr/evm_tx_store.go @@ -217,6 +217,7 @@ func (db *DbEthTx) FromTx(tx *Tx) { db.Meta = tx.Meta db.Subject = tx.Subject db.PipelineTaskRunID = tx.PipelineTaskRunID + db.MinConfirmations = tx.MinConfirmations db.TransmitChecker = tx.TransmitChecker db.InitialBroadcastAt = tx.InitialBroadcastAt db.SignalCallback = tx.SignalCallback From 85d8a9204a43551355454909f3ca0d604ef74d5e Mon Sep 17 00:00:00 2001 From: Joe Huang Date: Mon, 12 Aug 2024 10:26:20 -0500 Subject: [PATCH 46/46] add comment --- core/internal/features/features_test.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/core/internal/features/features_test.go b/core/internal/features/features_test.go index 142e1824079..b9a951ef2f1 100644 --- a/core/internal/features/features_test.go +++ b/core/internal/features/features_test.go @@ -536,8 +536,8 @@ observationSource = """ assert.Equal(t, []*string(nil), run.Errors) testutils.WaitForLogMessage(t, o, "Sending transaction") - b.Commit() // Needs at least two confirmations - time.Sleep(1 * time.Second) + b.Commit() // Needs at least two confirmations + time.Sleep(10 * time.Millisecond) // wait for finalizer to process confirmed tx b.Blockchain().SetFinalized(b.Blockchain().CurrentBlock()) b.Commit() // Needs at least two confirmations b.Commit() // Needs at least two confirmations @@ -584,8 +584,8 @@ observationSource = """ assert.Equal(t, []*string(nil), run.Errors) testutils.WaitForLogMessage(t, o, "Sending transaction") - b.Commit() // Needs at least two confirmations - time.Sleep(1 * time.Second) + b.Commit() // Needs at least two confirmations + time.Sleep(10 * time.Millisecond) // wait for finalizer to process confirmed tx b.Blockchain().SetFinalized(b.Blockchain().CurrentBlock()) b.Commit() // Needs at least two confirmations b.Commit() // Needs at least two confirmations @@ -625,7 +625,7 @@ observationSource = """ testutils.WaitForLogMessage(t, o, "Sending transaction") b.Commit() // Needs at least two confirmations - time.Sleep(1 * time.Second) + time.Sleep(10 * time.Millisecond) b.Blockchain().SetFinalized(b.Blockchain().CurrentBlock()) b.Commit() // Needs at least two confirmations b.Commit() // Needs at least two confirmations