From 7909892cc38bb028eceee55bb987cad1c18ab2c7 Mon Sep 17 00:00:00 2001 From: amit-momin Date: Fri, 4 Aug 2023 14:50:34 -0500 Subject: [PATCH 1/9] Moved next_nonce logic from keystore to broadcaster --- common/txmgr/broadcaster.go | 137 +++++-- common/txmgr/confirmer.go | 8 +- common/txmgr/mocks/tx_manager.go | 14 +- common/txmgr/nonce_syncer.go | 4 +- common/txmgr/resender.go | 6 +- common/txmgr/txmgr.go | 22 +- common/txmgr/types/client.go | 6 +- common/txmgr/types/keystore.go | 5 +- common/txmgr/types/mocks/key_store.go | 57 +-- .../txmgr/types/mocks/tx_attempt_builder.go | 4 +- common/txmgr/types/mocks/tx_store.go | 68 ++-- common/txmgr/types/tx.go | 4 +- common/txmgr/types/tx_attempt_builder.go | 2 +- common/txmgr/types/tx_store.go | 16 +- common/types/chain.go | 3 +- core/chains/evm/txmgr/broadcaster_test.go | 357 ++++++++++++------ core/chains/evm/txmgr/builder.go | 2 +- core/chains/evm/txmgr/evm_tx_store.go | 19 +- core/chains/evm/txmgr/evm_tx_store_test.go | 30 +- core/chains/evm/txmgr/mocks/evm_tx_store.go | 64 ++-- core/chains/evm/txmgr/models.go | 2 +- core/chains/evm/txmgr/nonce_syncer.go | 65 +--- core/chains/evm/txmgr/nonce_syncer_test.go | 135 ++----- core/chains/evm/txmgr/txmgr_test.go | 40 +- core/chains/evm/types/nonce.go | 4 + core/cmd/eth_keys_commands.go | 12 +- core/cmd/eth_keys_commands_test.go | 8 +- core/cmd/evm_transaction_commands_test.go | 2 + core/internal/cltest/factories.go | 2 - core/scripts/common/vrf/setup-envs/main.go | 10 +- core/services/keystore/eth.go | 93 +---- core/services/keystore/eth_internal_test.go | 5 +- core/services/keystore/eth_test.go | 136 ------- core/services/keystore/keys/ethkey/key.go | 7 - core/services/keystore/keys/ethkey/models.go | 11 +- core/services/keystore/mocks/eth.go | 89 +---- core/services/keystore/orm.go | 21 +- core/services/keystore/orm_test.go | 4 +- .../0199_remove_next_nonce_from_keystore.sql | 9 + core/web/eth_keys_controller.go | 16 +- core/web/eth_keys_controller_test.go | 67 +--- core/web/pipeline_runs_controller_test.go | 2 + core/web/presenters/eth_key.go | 2 - core/web/presenters/eth_key_test.go | 3 - .../BULLETPROOF_TX_MANAGER_ARCHITECTURE.md | 44 +-- testdata/scripts/keys/eth/help.txtar | 2 +- 46 files changed, 618 insertions(+), 1001 deletions(-) create mode 100644 core/store/migrate/migrations/0199_remove_next_nonce_from_keystore.sql diff --git a/common/txmgr/broadcaster.go b/common/txmgr/broadcaster.go index 4f0e92d0bcd..d6f73c607ad 100644 --- a/common/txmgr/broadcaster.go +++ b/common/txmgr/broadcaster.go @@ -60,7 +60,7 @@ type TransmitCheckerFactory[ CHAIN_ID types.ID, ADDR types.Hashable, TX_HASH, BLOCK_HASH types.Hashable, - SEQ types.Sequence, + SEQ types.Sequence[SEQ], FEE feetypes.Fee, ] interface { // BuildChecker builds a new TransmitChecker based on the given spec. @@ -72,7 +72,7 @@ type TransmitChecker[ CHAIN_ID types.ID, ADDR types.Hashable, TX_HASH, BLOCK_HASH types.Hashable, - SEQ types.Sequence, + SEQ types.Sequence[SEQ], FEE feetypes.Fee, ] interface { @@ -102,14 +102,14 @@ type Broadcaster[ ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, - SEQ types.Sequence, + SEQ types.Sequence[SEQ], FEE feetypes.Fee, ] struct { logger logger.Logger txStore txmgrtypes.TransactionStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, SEQ, FEE] client txmgrtypes.TransactionClient[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] txmgrtypes.TxAttemptBuilder[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] - sequenceSyncer SequenceSyncer[ADDR, TX_HASH, BLOCK_HASH] + sequenceSyncer SequenceSyncer[ADDR, TX_HASH, BLOCK_HASH, SEQ] resumeCallback ResumeCallback chainID CHAIN_ID config txmgrtypes.BroadcasterChainConfig @@ -143,6 +143,9 @@ type Broadcaster[ utils.StartStopOnce parseAddr func(string) (ADDR, error) + + sequenceLock *sync.RWMutex + nextSequenceMap map[ADDR]SEQ } func NewBroadcaster[ @@ -151,7 +154,7 @@ func NewBroadcaster[ ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, - SEQ types.Sequence, + SEQ types.Sequence[SEQ], FEE feetypes.Fee, ]( txStore txmgrtypes.TransactionStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, SEQ, FEE], @@ -163,7 +166,7 @@ func NewBroadcaster[ keystore txmgrtypes.KeyStore[ADDR, CHAIN_ID, SEQ], eventBroadcaster pg.EventBroadcaster, txAttemptBuilder txmgrtypes.TxAttemptBuilder[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], - sequenceSyncer SequenceSyncer[ADDR, TX_HASH, BLOCK_HASH], + sequenceSyncer SequenceSyncer[ADDR, TX_HASH, BLOCK_HASH, SEQ], logger logger.Logger, checkerFactory TransmitCheckerFactory[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], autoSyncSequence bool, @@ -186,6 +189,7 @@ func NewBroadcaster[ checkerFactory: checkerFactory, autoSyncSequence: autoSyncSequence, parseAddr: parseAddress, + sequenceLock: &sync.RWMutex{}, } b.processUnstartedTxsImpl = b.processUnstartedTxs @@ -235,6 +239,13 @@ func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) star eb.wg.Add(1) go eb.txInsertTriggerer() + eb.sequenceLock.Lock() + defer eb.sequenceLock.Unlock() + eb.nextSequenceMap, err = eb.loadNextSequenceMap(eb.enabledAddresses) + if err != nil { + return err + } + eb.isStarted = true return nil } @@ -312,6 +323,33 @@ func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) txIn } } +// Load the next sequence map using the tx table or on-chain (if not found in tx table) +func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) loadNextSequenceMap(addresses []ADDR) (map[ADDR]SEQ, error) { + ctx, cancel := eb.chStop.NewCtx() + defer cancel() + + nextSequenceMap := make(map[ADDR]SEQ) + for _, address := range addresses { + // Get the highest sequence from the tx table + // Will need to be incremented since this sequence is already used + seq, err := eb.txStore.FindHighestSequence(address, eb.chainID) + if err != nil { + // Look for nonce on-chain if address not found in TxStore + // Returns the nonce that should be used for the next transaction so no need to increment + seq, err = eb.client.PendingSequenceAt(ctx, address) + if err != nil { + return nil, errors.New("failed to retrieve next sequence from on-chain causing failure to load next sequence map on broadcaster startup") + } + + nextSequenceMap[address] = seq + } else { + nextSequenceMap[address] = seq.Increment() + } + } + + return nextSequenceMap, nil +} + func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) newSequenceSyncBackoff() backoff.Backoff { return backoff.Backoff{ Min: 100 * time.Millisecond, @@ -392,18 +430,28 @@ func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) moni // syncSequence tries to sync the key sequence, retrying indefinitely until success func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) SyncSequence(ctx context.Context, addr ADDR) { sequenceSyncRetryBackoff := eb.newSequenceSyncBackoff() - if err := eb.sequenceSyncer.Sync(ctx, addr); err != nil { + localSequence, err := eb.GetNextSequence(addr) + // Address not found in map so skip sync + if err != nil { + eb.logger.Criticalw("Failed to retrieve local next sequence for address", "address", addr.String(), "err", err) + return + } + + newNextSequence, err := eb.sequenceSyncer.Sync(ctx, addr, localSequence) + if err != nil { // Enter retry loop with backoff var attempt int eb.logger.Errorw("Failed to sync with on-chain sequence", "address", addr.String(), "attempt", attempt, "err", err) + loop: for { select { case <-eb.chStop: return case <-time.After(sequenceSyncRetryBackoff.Duration()): attempt++ - - if err := eb.sequenceSyncer.Sync(ctx, addr); err != nil { + newNextSequence, err = eb.sequenceSyncer.Sync(ctx, addr, localSequence) + _ = newNextSequence + if err != nil { if attempt > 5 { eb.logger.Criticalw("Failed to sync with on-chain sequence", "address", addr.String(), "attempt", attempt, "err", err) eb.SvcErrBuffer.Append(err) @@ -412,10 +460,19 @@ func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) Sync } continue } - return + break loop } } } + if err == nil && localSequence.String() != newNextSequence.String() { + eb.logger.Infow("Fast-forward sequence", "address", addr, "newNextSequence", newNextSequence, "oldNextSequence", localSequence) + // Set new next sequence in the map + err = eb.SetNextSequence(addr, newNextSequence) + if err != nil { + eb.logger.Criticalw("Failed to set new sequence for address", "address", addr.String(), "sequence", newNextSequence.String(), "err", err) + return + } + } } // ProcessUnstartedTxs picks up and handles all txes in the queue @@ -505,16 +562,6 @@ func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) hand return nil, false } -// This function is used to pass the queryer from the txmgr to the keystore. -// It is inevitable we have to pass the queryer because we need the keystate's next sequence to be incremented -// atomically alongside the transition from `in_progress` to `broadcast` so it is ready for the next transaction -func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) incrementNextSequenceAtomic(tx pg.Queryer, etx txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) error { - if err := eb.incrementNextSequence(etx.FromAddress, *etx.Sequence, pg.WithQueryer(tx)); err != nil { - return errors.Wrap(err, "saveUnconfirmed failed") - } - return nil -} - // There can be at most one in_progress transaction per address. // Here we complete the job that we didn't finish last time. func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) handleInProgressTx(ctx context.Context, etx txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], attempt txmgrtypes.TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], initialBroadcastAt time.Time) (error, bool) { @@ -603,9 +650,8 @@ func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) hand // and hand off to the confirmer to get the receipt (or mark as // failed). observeTimeUntilBroadcast(eb.chainID, etx.CreatedAt, time.Now()) - return eb.txStore.UpdateTxAttemptInProgressToBroadcast(&etx, attempt, txmgrtypes.TxAttemptBroadcast, func(tx pg.Queryer) error { - return eb.incrementNextSequenceAtomic(tx, etx) - }), true + err = eb.txStore.UpdateTxAttemptInProgressToBroadcast(&etx, attempt, txmgrtypes.TxAttemptBroadcast, eb.IncrementNextSequence) + return err, true case clienttypes.Underpriced: return eb.tryAgainBumpingGas(ctx, lgr, err, etx, attempt, initialBroadcastAt) case clienttypes.InsufficientFunds: @@ -650,9 +696,8 @@ func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) hand // Despite the error, the RPC node considers the previously sent // transaction to have been accepted. In this case, the right thing to // do is assume success and hand off to Confirmer - return eb.txStore.UpdateTxAttemptInProgressToBroadcast(&etx, attempt, txmgrtypes.TxAttemptBroadcast, func(tx pg.Queryer) error { - return eb.incrementNextSequenceAtomic(tx, etx) - }), true + err = eb.txStore.UpdateTxAttemptInProgressToBroadcast(&etx, attempt, txmgrtypes.TxAttemptBroadcast, eb.IncrementNextSequence) + return err, true } // Either the unknown error prevented the transaction from being mined, or // it has not yet propagated to the mempool, or there is some race on the @@ -679,7 +724,7 @@ func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) next return nil, errors.Wrap(err, "findNextUnstartedTransactionFromAddress failed") } - sequence, err := eb.getNextSequence(etx.FromAddress) + sequence, err := eb.GetNextSequence(etx.FromAddress) if err != nil { return nil, err } @@ -763,12 +808,42 @@ func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) save return eb.txStore.UpdateTxFatalError(ctx, etx) } -func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) getNextSequence(address ADDR) (sequence SEQ, err error) { - return eb.ks.NextSequence(address, eb.chainID) +// Used to get the next usable sequence for a transaction +func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) GetNextSequence(address ADDR) (seq SEQ, err error) { + eb.sequenceLock.Lock() + defer eb.sequenceLock.Unlock() + // Get next sequence from map + seq, exists := eb.nextSequenceMap[address] + if !exists { + return seq, errors.New(fmt.Sprint("address not found in next sequence map: ", address)) + } + return seq, nil +} + +// Used to increment the sequence in the mapping to have the next usable one available for the next transaction +func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) IncrementNextSequence(address ADDR) error { + eb.sequenceLock.Lock() + defer eb.sequenceLock.Unlock() + // Get next sequence from map + seq, exists := eb.nextSequenceMap[address] + if !exists { + return errors.New(fmt.Sprint("address not found in next sequence map: ", address)) + } + eb.nextSequenceMap[address] = seq.Increment() + return nil } -func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) incrementNextSequence(address ADDR, currentSequence SEQ, qopts ...pg.QOpt) error { - return eb.ks.IncrementNextSequence(address, eb.chainID, currentSequence, qopts...) +// Used to set the next sequence explicitly to a certain value +func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) SetNextSequence(address ADDR, seq SEQ) error { + eb.sequenceLock.Lock() + defer eb.sequenceLock.Unlock() + // Get next sequence from map + _, exists := eb.nextSequenceMap[address] + if !exists { + return errors.New(fmt.Sprint("address not found in next sequence map: ", address)) + } + eb.nextSequenceMap[address] = seq + return nil } func observeTimeUntilBroadcast[CHAIN_ID types.ID](chainID CHAIN_ID, createdAt, broadcastAt time.Time) { diff --git a/common/txmgr/confirmer.go b/common/txmgr/confirmer.go index 31bba771410..155e324ad9b 100644 --- a/common/txmgr/confirmer.go +++ b/common/txmgr/confirmer.go @@ -109,7 +109,7 @@ type Confirmer[ TX_HASH types.Hashable, BLOCK_HASH types.Hashable, R txmgrtypes.ChainReceipt[TX_HASH, BLOCK_HASH], - SEQ types.Sequence, + SEQ types.Sequence[SEQ], FEE feetypes.Fee, ] struct { utils.StartStopOnce @@ -145,7 +145,7 @@ func NewConfirmer[ TX_HASH types.Hashable, BLOCK_HASH types.Hashable, R txmgrtypes.ChainReceipt[TX_HASH, BLOCK_HASH], - SEQ types.Sequence, + SEQ types.Sequence[SEQ], FEE feetypes.Fee, ]( txStore txmgrtypes.TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE], @@ -961,7 +961,7 @@ func hasReceiptInLongestChain[ CHAIN_ID types.ID, ADDR types.Hashable, TX_HASH, BLOCK_HASH types.Hashable, - SEQ types.Sequence, + SEQ types.Sequence[SEQ], FEE feetypes.Fee, ](etx txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], head types.Head[BLOCK_HASH]) bool { for { @@ -1118,7 +1118,7 @@ func observeUntilTxConfirmed[ ADDR types.Hashable, TX_HASH, BLOCK_HASH types.Hashable, R txmgrtypes.ChainReceipt[TX_HASH, BLOCK_HASH], - SEQ types.Sequence, + SEQ types.Sequence[SEQ], FEE feetypes.Fee, ](chainID CHAIN_ID, attempts []txmgrtypes.TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], receipts []R) { for _, attempt := range attempts { diff --git a/common/txmgr/mocks/tx_manager.go b/common/txmgr/mocks/tx_manager.go index fe19023bd5c..be593a4954b 100644 --- a/common/txmgr/mocks/tx_manager.go +++ b/common/txmgr/mocks/tx_manager.go @@ -17,7 +17,7 @@ import ( ) // TxManager is an autogenerated mock type for the TxManager type -type TxManager[CHAIN_ID types.ID, HEAD types.Head[BLOCK_HASH], ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, SEQ types.Sequence, FEE feetypes.Fee] struct { +type TxManager[CHAIN_ID types.ID, HEAD types.Head[BLOCK_HASH], ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, SEQ types.Sequence[SEQ], FEE feetypes.Fee] struct { mock.Mock } @@ -137,13 +137,13 @@ func (_m *TxManager[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) Regist _m.Called(fn) } -// Reset provides a mock function with given fields: f, addr, abandon -func (_m *TxManager[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) Reset(f func(), addr ADDR, abandon bool) error { - ret := _m.Called(f, addr, abandon) +// Reset provides a mock function with given fields: addr, abandon +func (_m *TxManager[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) Reset(addr ADDR, abandon bool) error { + ret := _m.Called(addr, abandon) var r0 error - if rf, ok := ret.Get(0).(func(func(), ADDR, bool) error); ok { - r0 = rf(f, addr, abandon) + if rf, ok := ret.Get(0).(func(ADDR, bool) error); ok { + r0 = rf(addr, abandon) } else { r0 = ret.Error(0) } @@ -200,7 +200,7 @@ type mockConstructorTestingTNewTxManager interface { } // NewTxManager creates a new instance of TxManager. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. -func NewTxManager[CHAIN_ID types.ID, HEAD types.Head[BLOCK_HASH], ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, SEQ types.Sequence, FEE feetypes.Fee](t mockConstructorTestingTNewTxManager) *TxManager[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] { +func NewTxManager[CHAIN_ID types.ID, HEAD types.Head[BLOCK_HASH], ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, SEQ types.Sequence[SEQ], FEE feetypes.Fee](t mockConstructorTestingTNewTxManager) *TxManager[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] { mock := &TxManager[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]{} mock.Mock.Test(t) diff --git a/common/txmgr/nonce_syncer.go b/common/txmgr/nonce_syncer.go index 3c96473f800..f3f62d08d3f 100644 --- a/common/txmgr/nonce_syncer.go +++ b/common/txmgr/nonce_syncer.go @@ -6,6 +6,6 @@ import ( "github.com/smartcontractkit/chainlink/v2/common/types" ) -type SequenceSyncer[ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable] interface { - Sync(ctx context.Context, addr ADDR) (err error) +type SequenceSyncer[ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, SEQ types.Sequence[SEQ]] interface { + Sync(ctx context.Context, addr ADDR, localNonce SEQ) (SEQ, error) } diff --git a/common/txmgr/resender.go b/common/txmgr/resender.go index 655de0f1135..2b3324b34ce 100644 --- a/common/txmgr/resender.go +++ b/common/txmgr/resender.go @@ -40,7 +40,7 @@ type Resender[ ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, - SEQ types.Sequence, + SEQ types.Sequence[SEQ], FEE feetypes.Fee, ] struct { txStore txmgrtypes.TransactionStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, SEQ, FEE] @@ -63,7 +63,7 @@ func NewResender[ ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, - SEQ types.Sequence, + SEQ types.Sequence[SEQ], FEE feetypes.Fee, ]( lggr logger.Logger, @@ -207,7 +207,7 @@ func findOldestUnconfirmedAttempt[ CHAIN_ID types.ID, ADDR types.Hashable, TX_HASH, BLOCK_HASH types.Hashable, - SEQ types.Sequence, + SEQ types.Sequence[SEQ], FEE feetypes.Fee, ](attempts []txmgrtypes.TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) (txmgrtypes.TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], bool) { var oldestAttempt txmgrtypes.TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] diff --git a/common/txmgr/txmgr.go b/common/txmgr/txmgr.go index c6388e2a85c..a47743c891a 100644 --- a/common/txmgr/txmgr.go +++ b/common/txmgr/txmgr.go @@ -35,7 +35,7 @@ type TxManager[ ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, - SEQ types.Sequence, + SEQ types.Sequence[SEQ], FEE feetypes.Fee, ] interface { types.HeadTrackable[HEAD, BLOCK_HASH] @@ -45,7 +45,7 @@ type TxManager[ GetForwarderForEOA(eoa ADDR) (forwarder ADDR, err error) RegisterResumeCallback(fn ResumeCallback) SendNativeToken(ctx context.Context, chainID CHAIN_ID, from, to ADDR, value big.Int, gasLimit uint32) (etx txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) - Reset(f func(), addr ADDR, abandon bool) error + Reset(addr ADDR, abandon bool) error } type reset struct { @@ -64,7 +64,7 @@ type Txm[ TX_HASH types.Hashable, BLOCK_HASH types.Hashable, R txmgrtypes.ChainReceipt[TX_HASH, BLOCK_HASH], - SEQ types.Sequence, + SEQ types.Sequence[SEQ], FEE feetypes.Fee, ] struct { utils.StartStopOnce @@ -91,7 +91,7 @@ type Txm[ confirmer *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE] fwdMgr txmgrtypes.ForwarderManager[ADDR] txAttemptBuilder txmgrtypes.TxAttemptBuilder[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] - sequenceSyncer SequenceSyncer[ADDR, TX_HASH, BLOCK_HASH] + sequenceSyncer SequenceSyncer[ADDR, TX_HASH, BLOCK_HASH, SEQ] } func (b *Txm[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) RegisterResumeCallback(fn ResumeCallback) { @@ -108,7 +108,7 @@ func NewTxm[ TX_HASH types.Hashable, BLOCK_HASH types.Hashable, R txmgrtypes.ChainReceipt[TX_HASH, BLOCK_HASH], - SEQ types.Sequence, + SEQ types.Sequence[SEQ], FEE feetypes.Fee, ]( chainId CHAIN_ID, @@ -120,7 +120,7 @@ func NewTxm[ fwdMgr txmgrtypes.ForwarderManager[ADDR], txAttemptBuilder txmgrtypes.TxAttemptBuilder[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], txStore txmgrtypes.TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE], - sequenceSyncer SequenceSyncer[ADDR, TX_HASH, BLOCK_HASH], + sequenceSyncer SequenceSyncer[ADDR, TX_HASH, BLOCK_HASH, SEQ], broadcaster *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], confirmer *Confirmer[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE], resender *Resender[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], @@ -196,13 +196,11 @@ func (b *Txm[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) Start(ctx }) } -// Reset stops Broadcaster/Confirmer, executes callback, then starts them -// again -func (b *Txm[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) Reset(callback func(), addr ADDR, abandon bool) (err error) { +// Reset stops Broadcaster/Confirmer, executes callback, then starts them again +func (b *Txm[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) Reset(addr ADDR, abandon bool) (err error) { ok := b.IfStarted(func() { done := make(chan error) f := func() { - callback() if abandon { err = b.abandon(addr) } @@ -517,7 +515,7 @@ type NullTxManager[ HEAD types.Head[BLOCK_HASH], ADDR types.Hashable, TX_HASH, BLOCK_HASH types.Hashable, - SEQ types.Sequence, + SEQ types.Sequence[SEQ], FEE feetypes.Fee, ] struct { ErrMsg string @@ -546,7 +544,7 @@ func (n *NullTxManager[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) Cre func (n *NullTxManager[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) GetForwarderForEOA(addr ADDR) (fwdr ADDR, err error) { return fwdr, err } -func (n *NullTxManager[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) Reset(f func(), addr ADDR, abandon bool) error { +func (n *NullTxManager[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) Reset(addr ADDR, abandon bool) error { return nil } diff --git a/common/txmgr/types/client.go b/common/txmgr/types/client.go index 6d7f1c55558..c4f445b0560 100644 --- a/common/txmgr/types/client.go +++ b/common/txmgr/types/client.go @@ -19,7 +19,7 @@ type TxmClient[ TX_HASH types.Hashable, BLOCK_HASH types.Hashable, R ChainReceipt[TX_HASH, BLOCK_HASH], - SEQ types.Sequence, + SEQ types.Sequence[SEQ], FEE feetypes.Fee, ] interface { ChainClient[CHAIN_ID, ADDR, SEQ] @@ -38,7 +38,7 @@ type TransactionClient[ ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, - SEQ types.Sequence, + SEQ types.Sequence[SEQ], FEE feetypes.Fee, ] interface { ChainClient[CHAIN_ID, ADDR, SEQ] @@ -79,7 +79,7 @@ type TransactionClient[ type ChainClient[ CHAIN_ID types.ID, ADDR types.Hashable, - SEQ types.Sequence, + SEQ types.Sequence[SEQ], ] interface { ConfiguredChainID() CHAIN_ID PendingSequenceAt(ctx context.Context, addr ADDR) (SEQ, error) diff --git a/common/txmgr/types/keystore.go b/common/txmgr/types/keystore.go index 4793e6579ad..dc37e787a39 100644 --- a/common/txmgr/types/keystore.go +++ b/common/txmgr/types/keystore.go @@ -2,7 +2,6 @@ package types import ( "github.com/smartcontractkit/chainlink/v2/common/types" - "github.com/smartcontractkit/chainlink/v2/core/services/pg" ) // KeyStore encompasses the subset of keystore used by txmgr @@ -14,11 +13,9 @@ type KeyStore[ // Chain ID type CHAIN_ID types.ID, // Chain's sequence type. For example, EVM chains use nonce, bitcoin uses UTXO. - SEQ types.Sequence, + SEQ types.Sequence[SEQ], ] interface { CheckEnabled(address ADDR, chainID CHAIN_ID) error - NextSequence(address ADDR, chainID CHAIN_ID, qopts ...pg.QOpt) (SEQ, error) EnabledAddressesForChain(chainId CHAIN_ID) ([]ADDR, error) - IncrementNextSequence(address ADDR, chainID CHAIN_ID, currentSequence SEQ, qopts ...pg.QOpt) error SubscribeToKeyChanges() (ch chan struct{}, unsub func()) } diff --git a/common/txmgr/types/mocks/key_store.go b/common/txmgr/types/mocks/key_store.go index 90d8040f3e4..ea317990270 100644 --- a/common/txmgr/types/mocks/key_store.go +++ b/common/txmgr/types/mocks/key_store.go @@ -3,14 +3,13 @@ package mocks import ( - pg "github.com/smartcontractkit/chainlink/v2/core/services/pg" mock "github.com/stretchr/testify/mock" types "github.com/smartcontractkit/chainlink/v2/common/types" ) // KeyStore is an autogenerated mock type for the KeyStore type -type KeyStore[ADDR types.Hashable, CHAIN_ID types.ID, SEQ types.Sequence] struct { +type KeyStore[ADDR types.Hashable, CHAIN_ID types.ID, SEQ types.Sequence[SEQ]] struct { mock.Mock } @@ -54,58 +53,6 @@ func (_m *KeyStore[ADDR, CHAIN_ID, SEQ]) EnabledAddressesForChain(chainId CHAIN_ return r0, r1 } -// IncrementNextSequence provides a mock function with given fields: address, chainID, currentSequence, qopts -func (_m *KeyStore[ADDR, CHAIN_ID, SEQ]) IncrementNextSequence(address ADDR, chainID CHAIN_ID, currentSequence SEQ, qopts ...pg.QOpt) error { - _va := make([]interface{}, len(qopts)) - for _i := range qopts { - _va[_i] = qopts[_i] - } - var _ca []interface{} - _ca = append(_ca, address, chainID, currentSequence) - _ca = append(_ca, _va...) - ret := _m.Called(_ca...) - - var r0 error - if rf, ok := ret.Get(0).(func(ADDR, CHAIN_ID, SEQ, ...pg.QOpt) error); ok { - r0 = rf(address, chainID, currentSequence, qopts...) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -// NextSequence provides a mock function with given fields: address, chainID, qopts -func (_m *KeyStore[ADDR, CHAIN_ID, SEQ]) NextSequence(address ADDR, chainID CHAIN_ID, qopts ...pg.QOpt) (SEQ, error) { - _va := make([]interface{}, len(qopts)) - for _i := range qopts { - _va[_i] = qopts[_i] - } - var _ca []interface{} - _ca = append(_ca, address, chainID) - _ca = append(_ca, _va...) - ret := _m.Called(_ca...) - - var r0 SEQ - var r1 error - if rf, ok := ret.Get(0).(func(ADDR, CHAIN_ID, ...pg.QOpt) (SEQ, error)); ok { - return rf(address, chainID, qopts...) - } - if rf, ok := ret.Get(0).(func(ADDR, CHAIN_ID, ...pg.QOpt) SEQ); ok { - r0 = rf(address, chainID, qopts...) - } else { - r0 = ret.Get(0).(SEQ) - } - - if rf, ok := ret.Get(1).(func(ADDR, CHAIN_ID, ...pg.QOpt) error); ok { - r1 = rf(address, chainID, qopts...) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - // SubscribeToKeyChanges provides a mock function with given fields: func (_m *KeyStore[ADDR, CHAIN_ID, SEQ]) SubscribeToKeyChanges() (chan struct{}, func()) { ret := _m.Called() @@ -140,7 +87,7 @@ type mockConstructorTestingTNewKeyStore interface { } // NewKeyStore creates a new instance of KeyStore. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. -func NewKeyStore[ADDR types.Hashable, CHAIN_ID types.ID, SEQ types.Sequence](t mockConstructorTestingTNewKeyStore) *KeyStore[ADDR, CHAIN_ID, SEQ] { +func NewKeyStore[ADDR types.Hashable, CHAIN_ID types.ID, SEQ types.Sequence[SEQ]](t mockConstructorTestingTNewKeyStore) *KeyStore[ADDR, CHAIN_ID, SEQ] { mock := &KeyStore[ADDR, CHAIN_ID, SEQ]{} mock.Mock.Test(t) diff --git a/common/txmgr/types/mocks/tx_attempt_builder.go b/common/txmgr/types/mocks/tx_attempt_builder.go index cda71cfae09..51f61fdf5b9 100644 --- a/common/txmgr/types/mocks/tx_attempt_builder.go +++ b/common/txmgr/types/mocks/tx_attempt_builder.go @@ -16,7 +16,7 @@ import ( ) // TxAttemptBuilder is an autogenerated mock type for the TxAttemptBuilder type -type TxAttemptBuilder[CHAIN_ID types.ID, HEAD types.Head[BLOCK_HASH], ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, SEQ types.Sequence, FEE feetypes.Fee] struct { +type TxAttemptBuilder[CHAIN_ID types.ID, HEAD types.Head[BLOCK_HASH], ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, SEQ types.Sequence[SEQ], FEE feetypes.Fee] struct { mock.Mock } @@ -307,7 +307,7 @@ type mockConstructorTestingTNewTxAttemptBuilder interface { } // NewTxAttemptBuilder creates a new instance of TxAttemptBuilder. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. -func NewTxAttemptBuilder[CHAIN_ID types.ID, HEAD types.Head[BLOCK_HASH], ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, SEQ types.Sequence, FEE feetypes.Fee](t mockConstructorTestingTNewTxAttemptBuilder) *TxAttemptBuilder[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] { +func NewTxAttemptBuilder[CHAIN_ID types.ID, HEAD types.Head[BLOCK_HASH], ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, SEQ types.Sequence[SEQ], FEE feetypes.Fee](t mockConstructorTestingTNewTxAttemptBuilder) *TxAttemptBuilder[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] { mock := &TxAttemptBuilder[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]{} mock.Mock.Test(t) diff --git a/common/txmgr/types/mocks/tx_store.go b/common/txmgr/types/mocks/tx_store.go index dd40a064b9c..ec0f36eef6e 100644 --- a/common/txmgr/types/mocks/tx_store.go +++ b/common/txmgr/types/mocks/tx_store.go @@ -8,8 +8,6 @@ import ( feetypes "github.com/smartcontractkit/chainlink/v2/common/fee/types" mock "github.com/stretchr/testify/mock" - pg "github.com/smartcontractkit/chainlink/v2/core/services/pg" - time "time" txmgrtypes "github.com/smartcontractkit/chainlink/v2/common/txmgr/types" @@ -20,7 +18,7 @@ import ( ) // TxStore is an autogenerated mock type for the TxStore type -type TxStore[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 { +type TxStore[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[SEQ], FEE feetypes.Fee] struct { mock.Mock } @@ -143,6 +141,30 @@ func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) DeleteInPro return r0 } +// FindHighestSequence provides a mock function with given fields: fromAddress, chainId +func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindHighestSequence(fromAddress ADDR, chainId CHAIN_ID) (SEQ, error) { + ret := _m.Called(fromAddress, chainId) + + var r0 SEQ + var r1 error + if rf, ok := ret.Get(0).(func(ADDR, CHAIN_ID) (SEQ, error)); ok { + return rf(fromAddress, chainId) + } + if rf, ok := ret.Get(0).(func(ADDR, CHAIN_ID) SEQ); ok { + r0 = rf(fromAddress, chainId) + } else { + r0 = ret.Get(0).(SEQ) + } + + if rf, ok := ret.Get(1).(func(ADDR, CHAIN_ID) error); ok { + r1 = rf(fromAddress, chainId) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + // FindNextUnstartedTransactionFromAddress provides a mock function with given fields: ctx, etx, fromAddress, chainID func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindNextUnstartedTransactionFromAddress(ctx context.Context, etx *txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], fromAddress ADDR, chainID CHAIN_ID) error { ret := _m.Called(ctx, etx, fromAddress, chainID) @@ -673,41 +695,13 @@ func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) UpdateBroad return r0 } -// UpdateKeyNextSequence provides a mock function with given fields: newNextSequence, currentNextSequence, address, chainID, qopts -func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) UpdateKeyNextSequence(newNextSequence SEQ, currentNextSequence SEQ, address ADDR, chainID CHAIN_ID, qopts ...pg.QOpt) error { - _va := make([]interface{}, len(qopts)) - for _i := range qopts { - _va[_i] = qopts[_i] - } - var _ca []interface{} - _ca = append(_ca, newNextSequence, currentNextSequence, address, chainID) - _ca = append(_ca, _va...) - ret := _m.Called(_ca...) - - var r0 error - if rf, ok := ret.Get(0).(func(SEQ, SEQ, ADDR, CHAIN_ID, ...pg.QOpt) error); ok { - r0 = rf(newNextSequence, currentNextSequence, address, chainID, qopts...) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -// UpdateTxAttemptInProgressToBroadcast provides a mock function with given fields: etx, attempt, NewAttemptState, incrNextSequenceCallback, qopts -func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) UpdateTxAttemptInProgressToBroadcast(etx *txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], attempt txmgrtypes.TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], NewAttemptState txmgrtypes.TxAttemptState, incrNextSequenceCallback func(pg.Queryer) error, qopts ...pg.QOpt) error { - _va := make([]interface{}, len(qopts)) - for _i := range qopts { - _va[_i] = qopts[_i] - } - var _ca []interface{} - _ca = append(_ca, etx, attempt, NewAttemptState, incrNextSequenceCallback) - _ca = append(_ca, _va...) - ret := _m.Called(_ca...) +// UpdateTxAttemptInProgressToBroadcast provides a mock function with given fields: etx, attempt, NewAttemptState, incrementSeqFunc +func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) UpdateTxAttemptInProgressToBroadcast(etx *txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], attempt txmgrtypes.TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], NewAttemptState txmgrtypes.TxAttemptState, incrementSeqFunc func(ADDR) error) error { + ret := _m.Called(etx, attempt, NewAttemptState, incrementSeqFunc) var r0 error - if rf, ok := ret.Get(0).(func(*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], txmgrtypes.TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], txmgrtypes.TxAttemptState, func(pg.Queryer) error, ...pg.QOpt) error); ok { - r0 = rf(etx, attempt, NewAttemptState, incrNextSequenceCallback, qopts...) + if rf, ok := ret.Get(0).(func(*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], txmgrtypes.TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], txmgrtypes.TxAttemptState, func(ADDR) error) error); ok { + r0 = rf(etx, attempt, NewAttemptState, incrementSeqFunc) } else { r0 = ret.Error(0) } @@ -777,7 +771,7 @@ type mockConstructorTestingTNewTxStore interface { } // NewTxStore creates a new instance of TxStore. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. -func NewTxStore[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](t mockConstructorTestingTNewTxStore) *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { +func NewTxStore[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[SEQ], FEE feetypes.Fee](t mockConstructorTestingTNewTxStore) *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { mock := &TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]{} mock.Mock.Test(t) diff --git a/common/txmgr/types/tx.go b/common/txmgr/types/tx.go index d95f07afabc..b42064284f2 100644 --- a/common/txmgr/types/tx.go +++ b/common/txmgr/types/tx.go @@ -156,7 +156,7 @@ type TxAttempt[ CHAIN_ID types.ID, ADDR types.Hashable, TX_HASH, BLOCK_HASH types.Hashable, - SEQ types.Sequence, + SEQ types.Sequence[SEQ], FEE feetypes.Fee, ] struct { ID int64 @@ -182,7 +182,7 @@ type Tx[ CHAIN_ID types.ID, ADDR types.Hashable, TX_HASH, BLOCK_HASH types.Hashable, - SEQ types.Sequence, + SEQ types.Sequence[SEQ], FEE feetypes.Fee, ] struct { ID int64 diff --git a/common/txmgr/types/tx_attempt_builder.go b/common/txmgr/types/tx_attempt_builder.go index 887219c490e..3efcfd01eb1 100644 --- a/common/txmgr/types/tx_attempt_builder.go +++ b/common/txmgr/types/tx_attempt_builder.go @@ -19,7 +19,7 @@ type TxAttemptBuilder[ HEAD types.Head[BLOCK_HASH], // HEAD - chain head type ADDR types.Hashable, // ADDR - chain address type TX_HASH, BLOCK_HASH types.Hashable, // various chain hash types - SEQ types.Sequence, // SEQ - chain sequence type (nonce, utxo, etc) + SEQ types.Sequence[SEQ], // SEQ - chain sequence type (nonce, utxo, etc) FEE feetypes.Fee, // FEE - chain fee type ] interface { // interfaces for running the underlying estimator diff --git a/common/txmgr/types/tx_store.go b/common/txmgr/types/tx_store.go index a17188be441..421ba62e0b1 100644 --- a/common/txmgr/types/tx_store.go +++ b/common/txmgr/types/tx_store.go @@ -27,14 +27,13 @@ type TxStore[ // Represents a onchain receipt object that a chain's RPC returns R ChainReceipt[TX_HASH, BLOCK_HASH], // Represents the sequence type for a chain. For example, nonce for EVM. - SEQ types.Sequence, + SEQ types.Sequence[SEQ], // Represents the chain specific fee type FEE feetypes.Fee, ] interface { UnstartedTxQueuePruner TxHistoryReaper[CHAIN_ID] TransactionStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, SEQ, FEE] - SequenceStore[ADDR, CHAIN_ID, SEQ] // methods for saving & retreiving receipts FindReceiptsPendingConfirmation(ctx context.Context, blockNum int64, chainID CHAIN_ID) (receiptsPlus []ReceiptPlus[R], err error) @@ -52,13 +51,14 @@ type TransactionStore[ CHAIN_ID types.ID, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, - SEQ types.Sequence, + SEQ types.Sequence[SEQ], FEE feetypes.Fee, ] interface { CountUnconfirmedTransactions(ctx context.Context, fromAddress ADDR, chainID CHAIN_ID) (count uint32, err error) CountUnstartedTransactions(ctx context.Context, fromAddress ADDR, chainID CHAIN_ID) (count uint32, err error) CreateTransaction(ctx context.Context, txRequest TxRequest[ADDR, TX_HASH], chainID CHAIN_ID) (tx Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) DeleteInProgressAttempt(ctx context.Context, attempt TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) error + FindHighestSequence(fromAddress ADDR, chainId CHAIN_ID) (SEQ, error) FindTxsRequiringGasBump(ctx context.Context, address ADDR, blockNum, gasBumpThreshold, depth int64, chainID CHAIN_ID) (etxs []*Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) FindTxsRequiringResubmissionDueToInsufficientFunds(ctx context.Context, address ADDR, chainID CHAIN_ID) (etxs []*Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) FindTxAttemptsConfirmedMissingReceipt(ctx context.Context, chainID CHAIN_ID) (attempts []TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) @@ -84,7 +84,7 @@ type TransactionStore[ SaveSentAttempt(ctx context.Context, timeout time.Duration, attempt *TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], broadcastAt time.Time) error SetBroadcastBeforeBlockNum(ctx context.Context, blockNum int64, chainID CHAIN_ID) error UpdateBroadcastAts(ctx context.Context, now time.Time, etxIDs []int64) error - UpdateTxAttemptInProgressToBroadcast(etx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], attempt TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], NewAttemptState TxAttemptState, incrNextSequenceCallback QueryerFunc, qopts ...pg.QOpt) error + UpdateTxAttemptInProgressToBroadcast(etx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], attempt TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], NewAttemptState TxAttemptState, incrementSeqFunc func(address ADDR) error) error UpdateTxsUnconfirmed(ctx context.Context, ids []int64) error UpdateTxUnstartedToInProgress(ctx context.Context, etx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], attempt *TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) error UpdateTxFatalError(ctx context.Context, etx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) error @@ -99,14 +99,6 @@ type UnstartedTxQueuePruner interface { PruneUnstartedTxQueue(ctx context.Context, queueSize uint32, subject uuid.UUID) (n int64, err error) } -type SequenceStore[ - ADDR types.Hashable, - CHAIN_ID types.ID, - SEQ types.Sequence, -] interface { - UpdateKeyNextSequence(newNextSequence, currentNextSequence SEQ, address ADDR, chainID CHAIN_ID, qopts ...pg.QOpt) error -} - // R is the raw unparsed transaction receipt type ReceiptPlus[R any] struct { ID uuid.UUID `db:"pipeline_run_id"` diff --git a/common/types/chain.go b/common/types/chain.go index 800f0d9fdc0..a502b5cf91d 100644 --- a/common/types/chain.go +++ b/common/types/chain.go @@ -4,9 +4,10 @@ import "fmt" // Sequence represents the base type, for any chain's sequence object. // It should be convertible to a string -type Sequence interface { +type Sequence[SEQ any] interface { fmt.Stringer Int64() int64 // needed for numeric sequence confirmation - to be removed with confirmation logic generalization: https://smartcontract-it.atlassian.net/browse/BCI-860 + Increment() SEQ // needed to produce the next usable sequence for a transaction } // ID represents the base type, for any chain's ID. diff --git a/core/chains/evm/txmgr/broadcaster_test.go b/core/chains/evm/txmgr/broadcaster_test.go index 6a0b6c82eeb..f5f506cf6e8 100644 --- a/core/chains/evm/txmgr/broadcaster_test.go +++ b/core/chains/evm/txmgr/broadcaster_test.go @@ -32,6 +32,7 @@ import ( "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/txmgr" + "github.com/smartcontractkit/chainlink/v2/core/chains/evm/types" evmtypes "github.com/smartcontractkit/chainlink/v2/core/chains/evm/types" "github.com/smartcontractkit/chainlink/v2/core/internal/cltest" "github.com/smartcontractkit/chainlink/v2/core/internal/cltest/heavyweight" @@ -69,7 +70,7 @@ func NewTestEthBroadcaster( ge := config.EVM().GasEstimator() estimator := gas.NewWrappedEvmEstimator(gas.NewFixedPriceEstimator(config.EVM().GasEstimator(), ge.BlockHistory(), lggr), ge.EIP1559DynamicFees(), nil) txBuilder := txmgr.NewEvmTxAttemptBuilder(*ethClient.ConfiguredChainID(), ge, keyStore, estimator) - txNonceSyncer := txmgr.NewNonceSyncer(txStore, lggr, ethClient, keyStore) + txNonceSyncer := txmgr.NewNonceSyncer(txStore, lggr, ethClient) ethBroadcaster := txmgr.NewEvmBroadcaster(txStore, txmgr.NewEvmTxmClient(ethClient), txmgr.NewEvmTxmConfig(config.EVM()), txmgr.NewEvmTxmFeeConfig(config.EVM().GasEstimator()), config.EVM().Transactions(), config.Database().Listener(), keyStore, eb, txBuilder, txNonceSyncer, lggr, checkerFactory, nonceAutoSync) // Mark instance as test @@ -92,7 +93,7 @@ func TestEthBroadcaster_Lifecycle(t *testing.T) { cltest.MustInsertRandomKeyReturningState(t, ethKeyStore) estimator := gasmocks.NewEvmFeeEstimator(t) txBuilder := txmgr.NewEvmTxAttemptBuilder(*ethClient.ConfiguredChainID(), evmcfg.EVM().GasEstimator(), ethKeyStore, estimator) - + ethClient.On("PendingNonceAt", mock.Anything, mock.Anything).Return(uint64(0), nil) eb := txmgr.NewEvmBroadcaster( txStore, txmgr.NewEvmTxmClient(ethClient), @@ -145,12 +146,15 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Success(t *testing.T) { cfg := configtest.NewTestGeneralConfig(t) txStore := cltest.NewTestTxStore(t, db, cfg.Database()) ethKeyStore := cltest.NewKeyStore(t, db, cfg.Database()).Eth() - _, fromAddress := cltest.MustInsertRandomKeyReturningState(t, ethKeyStore) + _, fromAddress := cltest.MustInsertRandomKey(t, ethKeyStore) + _, otherAddress := cltest.MustInsertRandomKey(t, ethKeyStore) ethClient := evmtest.NewEthClientMockWithDefaultChain(t) evmcfg := evmtest.NewChainScopedConfig(t, cfg) checkerFactory := &txmgr.CheckerFactory{Client: ethClient} + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(0), nil).Once() + ethClient.On("PendingNonceAt", mock.Anything, otherAddress).Return(uint64(0), nil).Once() eb := NewTestEthBroadcaster(t, txStore, ethClient, ethKeyStore, evmcfg, checkerFactory, false) toAddress := gethCommon.HexToAddress("0x6C03DDA95a2AEd917EeCc6eddD4b9D16E6380411") @@ -170,7 +174,6 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Success(t *testing.T) { }) t.Run("eth_txes exist for a different from address", func(t *testing.T) { - _, otherAddress := cltest.MustInsertRandomKey(t, ethKeyStore) cltest.MustCreateUnstartedTx(t, txStore, otherAddress, toAddress, encodedPayload, gasLimit, value, &cltest.FixtureChainID) retryable, err := eb.ProcessUnstartedTxs(testutils.Context(t), fromAddress) assert.NoError(t, err) @@ -351,6 +354,8 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Success(t *testing.T) { c.EVM[0].GasEstimator.PriceMax = assets.NewWeiI(rnd + 2) }) evmcfg = evmtest.NewChainScopedConfig(t, cfg) + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(3), nil).Once() + ethClient.On("PendingNonceAt", mock.Anything, otherAddress).Return(uint64(1), nil).Once() eb = NewTestEthBroadcaster(t, txStore, ethClient, ethKeyStore, evmcfg, checkerFactory, false) t.Run("sends transactions with type 0x2 in EIP-1559 mode", func(t *testing.T) { @@ -498,7 +503,7 @@ func TestEthBroadcaster_TransmitChecking(t *testing.T) { ethClient := evmtest.NewEthClientMockWithDefaultChain(t) evmcfg := evmtest.NewChainScopedConfig(t, cfg) checkerFactory := &testCheckerFactory{} - + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(0), nil).Once() eb := NewTestEthBroadcaster(t, txStore, ethClient, ethKeyStore, evmcfg, checkerFactory, false) checker := txmgr.TransmitCheckerSpec{ @@ -573,6 +578,9 @@ func TestEthBroadcaster_TransmitChecking(t *testing.T) { func TestEthBroadcaster_ProcessUnstartedEthTxs_OptimisticLockingOnEthTx(t *testing.T) { // non-transactional DB needed because we deliberately test for FK violation cfg, db := heavyweight.FullTestDBV2(t, "eth_broadcaster_optimistic_locking", nil) + eventBroadcaster := cltest.NewEventBroadcaster(t, cfg.Database().URL()) + require.NoError(t, eventBroadcaster.Start(testutils.Context(t))) + t.Cleanup(func() { assert.NoError(t, eventBroadcaster.Close()) }) txStore := cltest.NewTestTxStore(t, db, cfg.Database()) ccfg := evmtest.NewChainScopedConfig(t, cfg) evmcfg := txmgr.NewEvmTxmConfig(ccfg.EVM()) @@ -588,8 +596,8 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_OptimisticLockingOnEthTx(t *testi estimator.On("GetFee", mock.Anything, mock.Anything, mock.Anything, ccfg.EVM().GasEstimator().PriceMaxKey(fromAddress)).Return(gas.EvmFee{Legacy: assets.GWei(32)}, uint32(500), nil).Run(func(_ mock.Arguments) { close(chStartEstimate) <-chBlock - }) - + }).Once() + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(0), nil) eb := txmgr.NewEvmBroadcaster( txStore, txmgr.NewEvmTxmClient(ethClient), @@ -598,7 +606,7 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_OptimisticLockingOnEthTx(t *testi ccfg.EVM().Transactions(), cfg.Database().Listener(), ethKeyStore, - &pg.NullEventBroadcaster{}, + eventBroadcaster, txBuilder, nil, logger.TestLogger(t), @@ -607,6 +615,10 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_OptimisticLockingOnEthTx(t *testi ) eb.XXXTestDisableUnstartedTxAutoProcessing() + // Start instance of broadcaster + require.NoError(t, eb.Start(testutils.Context(t))) + t.Cleanup(func() { assert.NoError(t, eb.Close()) }) + cltest.MustCreateUnstartedGeneratedTx(t, txStore, fromAddress, &cltest.FixtureChainID) go func() { @@ -619,7 +631,6 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_OptimisticLockingOnEthTx(t *testi // Simulate a "PruneQueue" call assert.NoError(t, utils.JustError(db.Exec(`DELETE FROM evm.txes WHERE state = 'unstarted'`))) - close(chBlock) }() @@ -645,7 +656,7 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Success_WithMultiplier(t *testing evmcfg := evmtest.NewChainScopedConfig(t, cfg) ethClient := evmtest.NewEthClientMockWithDefaultChain(t) - + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(0), nil).Once() eb := NewTestEthBroadcaster(t, txStore, ethClient, ethKeyStore, evmcfg, &testCheckerFactory{}, false) ethClient.On("SendTransactionReturnCode", mock.Anything, mock.MatchedBy(func(tx *gethTypes.Transaction) bool { @@ -725,7 +736,7 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_ResumingFromCrash(t *testing.T) { _, fromAddress := cltest.RandomKey{Nonce: nextNonce.Int64()}.MustInsertWithState(t, ethKeyStore) ethClient := evmtest.NewEthClientMockWithDefaultChain(t) - + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(0), nil).Once() eb := NewTestEthBroadcaster(t, txStore, ethClient, ethKeyStore, evmcfg, &testCheckerFactory{}, false) // Crashed right after we commit the database transaction that saved @@ -763,12 +774,10 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_ResumingFromCrash(t *testing.T) { _, fromAddress := cltest.RandomKey{Nonce: nextNonce.Int64()}.MustInsertWithState(t, ethKeyStore) ethClient := evmtest.NewEthClientMockWithDefaultChain(t) - + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(0), nil).Once() eb := NewTestEthBroadcaster(t, txStore, ethClient, ethKeyStore, evmcfg, &testCheckerFactory{}, false) - // Crashed right after we commit the database transaction that saved - // the nonce to the eth_tx so keys.next_nonce has not been - // incremented yet + // Crashed right after we commit the database transaction that saved the nonce to the eth_tx inProgressEthTx := cltest.MustInsertInProgressEthTxWithAttempt(t, txStore, firstNonce, fromAddress) ethClient.On("SendTransactionReturnCode", mock.Anything, mock.MatchedBy(func(tx *gethTypes.Transaction) bool { @@ -801,12 +810,10 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_ResumingFromCrash(t *testing.T) { _, fromAddress := cltest.RandomKey{Nonce: nextNonce.Int64()}.MustInsertWithState(t, ethKeyStore) ethClient := evmtest.NewEthClientMockWithDefaultChain(t) - + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(0), nil).Once() eb := NewTestEthBroadcaster(t, txStore, ethClient, ethKeyStore, evmcfg, &testCheckerFactory{}, false) - // Crashed right after we commit the database transaction that saved - // the nonce to the eth_tx so keys.next_nonce has not been - // incremented yet + // Crashed right after we commit the database transaction that saved the nonce to the eth_tx inProgressEthTx := cltest.MustInsertInProgressEthTxWithAttempt(t, txStore, firstNonce, fromAddress) ethClient.On("SendTransactionReturnCode", mock.Anything, mock.MatchedBy(func(tx *gethTypes.Transaction) bool { @@ -838,12 +845,10 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_ResumingFromCrash(t *testing.T) { _, fromAddress := cltest.RandomKey{Nonce: nextNonce.Int64()}.MustInsertWithState(t, ethKeyStore) ethClient := evmtest.NewEthClientMockWithDefaultChain(t) - + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(0), nil).Once() eb := NewTestEthBroadcaster(t, txStore, ethClient, ethKeyStore, evmcfg, &testCheckerFactory{}, false) - // Crashed right after we commit the database transaction that saved - // the nonce to the eth_tx so keys.next_nonce has not been - // incremented yet + // Crashed right after we commit the database transaction that saved the nonce to the eth_tx inProgressEthTx := cltest.MustInsertInProgressEthTxWithAttempt(t, txStore, firstNonce, fromAddress) ethClient.On("SendTransactionReturnCode", mock.Anything, mock.MatchedBy(func(tx *gethTypes.Transaction) bool { @@ -877,12 +882,10 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_ResumingFromCrash(t *testing.T) { _, fromAddress := cltest.RandomKey{Nonce: nextNonce.Int64()}.MustInsertWithState(t, ethKeyStore) ethClient := evmtest.NewEthClientMockWithDefaultChain(t) - + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(0), nil).Once() eb := NewTestEthBroadcaster(t, txStore, ethClient, ethKeyStore, evmcfg, &testCheckerFactory{}, false) - // Crashed right after we commit the database transaction that saved - // the nonce to the eth_tx so keys.next_nonce has not been - // incremented yet + // Crashed right after we commit the database transaction that saved the nonce to the eth_tx inProgressEthTx := cltest.MustInsertInProgressEthTxWithAttempt(t, txStore, firstNonce, fromAddress) ethClient.On("SendTransactionReturnCode", mock.Anything, mock.MatchedBy(func(tx *gethTypes.Transaction) bool { @@ -920,12 +923,10 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_ResumingFromCrash(t *testing.T) { evmcfg := evmtest.NewChainScopedConfig(t, cfg) ethClient := evmtest.NewEthClientMockWithDefaultChain(t) - + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(0), nil).Once() eb := NewTestEthBroadcaster(t, txStore, ethClient, ethKeyStore, evmcfg, &testCheckerFactory{}, false) - // Crashed right after we commit the database transaction that saved - // the nonce to the eth_tx so keys.next_nonce has not been - // incremented yet + // Crashed right after we commit the database transaction that saved the nonce to the eth_tx inProgressEthTx := cltest.MustInsertInProgressEthTxWithAttempt(t, txStore, firstNonce, fromAddress) require.Len(t, inProgressEthTx.TxAttempts, 1) attempt := inProgressEthTx.TxAttempts[0] @@ -960,8 +961,8 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_ResumingFromCrash(t *testing.T) { }) } -func getLocalNextNonce(t *testing.T, kst keystore.Eth, fromAddress gethCommon.Address) uint64 { - n, err := kst.NextSequence(fromAddress, &cltest.FixtureChainID) +func getLocalNextNonce(t *testing.T, eb *txmgr.Broadcaster, fromAddress gethCommon.Address) uint64 { + n, err := eb.GetNextSequence(fromAddress) require.NoError(t, err) require.NotNil(t, n) return uint64(n) @@ -982,11 +983,11 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Errors(t *testing.T) { txStore := cltest.NewTestTxStore(t, db, cfg.Database()) ethKeyStore := cltest.NewKeyStore(t, db, cfg.Database()).Eth() - _, fromAddress := cltest.MustInsertRandomKeyReturningState(t, ethKeyStore) + _, fromAddress := cltest.MustInsertRandomKey(t, ethKeyStore) evmcfg := evmtest.NewChainScopedConfig(t, cfg) ethClient := evmtest.NewEthClientMockWithDefaultChain(t) - + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(0), nil).Once() eb := NewTestEthBroadcaster(t, txStore, ethClient, ethKeyStore, evmcfg, &testCheckerFactory{}, false) require.NoError(t, utils.JustError(db.Exec(`SET CONSTRAINTS pipeline_runs_pipeline_spec_id_fkey DEFERRED`))) @@ -1021,7 +1022,7 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Errors(t *testing.T) { assert.Len(t, etx1.TxAttempts, 1) // Check that the local nonce was incremented by one - finalNextNonce := getLocalNextNonce(t, ethKeyStore, fromAddress) + finalNextNonce := getLocalNextNonce(t, eb, fromAddress) require.NoError(t, err) require.NotNil(t, finalNextNonce) require.Equal(t, int64(1), int64(finalNextNonce)) @@ -1029,7 +1030,7 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Errors(t *testing.T) { t.Run("geth Client returns an error in the fatal errors category", func(t *testing.T) { fatalErrorExample := "exceeds block gas limit" - localNextNonce := getLocalNextNonce(t, ethKeyStore, fromAddress) + localNextNonce := getLocalNextNonce(t, eb, fromAddress) t.Run("without callback", func(t *testing.T) { etx := cltest.MustCreateUnstartedTx(t, txStore, fromAddress, toAddress, encodedPayload, gasLimit, value, &cltest.FixtureChainID) @@ -1055,12 +1056,12 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Errors(t *testing.T) { assert.Len(t, etx.TxAttempts, 0) // Check that the key had its nonce reset - var nonce int64 - err = db.Get(&nonce, `SELECT next_nonce FROM evm.key_states WHERE address = $1 ORDER BY created_at ASC, id ASC`, fromAddress) + var nonce evmtypes.Nonce + nonce, err = eb.GetNextSequence(fromAddress) require.NoError(t, err) // Saved NextNonce must be the same as before because this transaction // was not accepted by the eth node and never can be - require.Equal(t, int64(localNextNonce), nonce) + require.Equal(t, int64(localNextNonce), int64(nonce)) }) @@ -1125,9 +1126,12 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Errors(t *testing.T) { lggr := logger.TestLogger(t) estimator := gas.NewWrappedEvmEstimator(gas.NewFixedPriceEstimator(evmcfg.EVM().GasEstimator(), evmcfg.EVM().GasEstimator().BlockHistory(), lggr), evmcfg.EVM().GasEstimator().EIP1559DynamicFees(), nil) txBuilder := txmgr.NewEvmTxAttemptBuilder(*ethClient.ConfiguredChainID(), evmcfg.EVM().GasEstimator(), ethKeyStore, estimator) - eb = txmgr.NewEvmBroadcaster(txStore, txmgr.NewEvmTxmClient(ethClient), txmgr.NewEvmTxmConfig(evmcfg.EVM()), txmgr.NewEvmTxmFeeConfig(evmcfg.EVM().GasEstimator()), evmcfg.EVM().Transactions(), evmcfg.Database().Listener(), ethKeyStore, eventBroadcaster, txBuilder, nil, lggr, &testCheckerFactory{}, false) + localNextNonce := getLocalNextNonce(t, eb, fromAddress) + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(localNextNonce), nil).Once() + eb2 := txmgr.NewEvmBroadcaster(txStore, txmgr.NewEvmTxmClient(ethClient), txmgr.NewEvmTxmConfig(evmcfg.EVM()), txmgr.NewEvmTxmFeeConfig(evmcfg.EVM().GasEstimator()), evmcfg.EVM().Transactions(), evmcfg.Database().Listener(), ethKeyStore, eventBroadcaster, txBuilder, nil, lggr, &testCheckerFactory{}, false) + require.NoError(t, err) { - retryable, err := eb.ProcessUnstartedTxs(testutils.Context(t), fromAddress) + retryable, err := eb2.ProcessUnstartedTxs(testutils.Context(t), fromAddress) assert.NoError(t, err) assert.False(t, retryable) } @@ -1140,7 +1144,7 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Errors(t *testing.T) { t.Run("geth Client fails with error indicating that the transaction was too expensive", func(t *testing.T) { TxFeeExceedsCapError := "tx fee (1.10 ether) exceeds the configured cap (1.00 ether)" - localNextNonce := getLocalNextNonce(t, ethKeyStore, fromAddress) + localNextNonce := getLocalNextNonce(t, eb, fromAddress) etx := cltest.MustCreateUnstartedTx(t, txStore, fromAddress, toAddress, encodedPayload, gasLimit, value, &cltest.FixtureChainID) ethClient.On("SendTransactionReturnCode", mock.Anything, mock.MatchedBy(func(tx *gethTypes.Transaction) bool { return tx.Nonce() == localNextNonce @@ -1149,7 +1153,6 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Errors(t *testing.T) { // of multiple RPC nodes, it is possible that it can be accepted by // another node even if the primary one returns "exceeds the configured // cap" - ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(localNextNonce, nil).Once() { retryable, err := eb.ProcessUnstartedTxs(testutils.Context(t), fromAddress) @@ -1172,15 +1175,15 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Errors(t *testing.T) { assert.Equal(t, txmgrtypes.TxAttemptInProgress, attempt.State) // Check that the key had its nonce reset - var nonce int64 - err = db.Get(&nonce, `SELECT next_nonce FROM evm.key_states WHERE address = $1 ORDER BY created_at ASC, id ASC`, fromAddress) + var nonce evmtypes.Nonce + nonce, err = eb.GetNextSequence(fromAddress) require.NoError(t, err) // Saved NextNonce must be the same as before because this transaction // was not accepted by the eth node and never can be - require.Equal(t, int64(localNextNonce), nonce) + require.Equal(t, int64(localNextNonce), int64(nonce)) // On the second try, the tx has been accepted into the mempool - ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(localNextNonce+1, nil).Once() + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(localNextNonce+1), nil).Once() { retryable, err := eb.ProcessUnstartedTxs(testutils.Context(t), fromAddress) @@ -1203,13 +1206,13 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Errors(t *testing.T) { t.Run("eth Client call fails with an unexpected random error, and transaction was not accepted into mempool", func(t *testing.T) { retryableErrorExample := "some unknown error" - localNextNonce := getLocalNextNonce(t, ethKeyStore, fromAddress) + localNextNonce := getLocalNextNonce(t, eb, fromAddress) etx := cltest.MustCreateUnstartedTx(t, txStore, fromAddress, toAddress, encodedPayload, gasLimit, value, &cltest.FixtureChainID) ethClient.On("SendTransactionReturnCode", mock.Anything, mock.MatchedBy(func(tx *gethTypes.Transaction) bool { - return tx.Nonce() == localNextNonce + return tx.Nonce() == uint64(localNextNonce) }), fromAddress).Return(clienttypes.Unknown, errors.New(retryableErrorExample)).Once() // Nonce is the same as localNextNonce, implying that this sent transaction has not been accepted - ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(localNextNonce, nil).Once() + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(localNextNonce), nil).Once() // Do the thing retryable, err := eb.ProcessUnstartedTxs(testutils.Context(t), fromAddress) @@ -1257,10 +1260,10 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Errors(t *testing.T) { t.Run("eth client call fails with an unexpected random error, and the nonce check also subsequently fails", func(t *testing.T) { retryableErrorExample := "some unknown error" - localNextNonce := getLocalNextNonce(t, ethKeyStore, fromAddress) + localNextNonce := getLocalNextNonce(t, eb, fromAddress) etx := cltest.MustCreateUnstartedTx(t, txStore, fromAddress, toAddress, encodedPayload, gasLimit, value, &cltest.FixtureChainID) ethClient.On("SendTransactionReturnCode", mock.Anything, mock.MatchedBy(func(tx *gethTypes.Transaction) bool { - return tx.Nonce() == localNextNonce + return tx.Nonce() == uint64(localNextNonce) }), fromAddress).Return(clienttypes.Unknown, errors.New(retryableErrorExample)).Once() ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(0), errors.New("pending nonce fetch failed")).Once() @@ -1311,13 +1314,13 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Errors(t *testing.T) { t.Run("eth Client call fails with an unexpected random error, and transaction was accepted into mempool", func(t *testing.T) { retryableErrorExample := "some strange RPC returns an unexpected thing" - localNextNonce := getLocalNextNonce(t, ethKeyStore, fromAddress) + localNextNonce := getLocalNextNonce(t, eb, fromAddress) etx := cltest.MustCreateUnstartedTx(t, txStore, fromAddress, toAddress, encodedPayload, gasLimit, value, &cltest.FixtureChainID) ethClient.On("SendTransactionReturnCode", mock.Anything, mock.MatchedBy(func(tx *gethTypes.Transaction) bool { return tx.Nonce() == localNextNonce }), fromAddress).Return(clienttypes.Unknown, errors.New(retryableErrorExample)).Once() // Nonce is one higher than localNextNonce, implying that despite the error, this sent transaction has been accepted into the mempool - ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(localNextNonce+1, nil).Once() + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(localNextNonce+1), nil).Once() // Do the thing retryable, err := eb.ProcessUnstartedTxs(testutils.Context(t), fromAddress) @@ -1343,7 +1346,7 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Errors(t *testing.T) { // configured for the transaction pool. // This is a configuration error by the node operator, since it means they set the base gas level too low. underpricedError := "transaction underpriced" - localNextNonce := getLocalNextNonce(t, ethKeyStore, fromAddress) + localNextNonce := getLocalNextNonce(t, eb, fromAddress) etx := cltest.MustCreateUnstartedTx(t, txStore, fromAddress, toAddress, encodedPayload, gasLimit, value, &cltest.FixtureChainID) // First was underpriced @@ -1393,7 +1396,7 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Errors(t *testing.T) { t.Run("failed to reach node for some reason", func(t *testing.T) { failedToReachNodeError := context.DeadlineExceeded - localNextNonce := getLocalNextNonce(t, ethKeyStore, fromAddress) + localNextNonce := getLocalNextNonce(t, eb, fromAddress) ethClient.On("SendTransactionReturnCode", mock.Anything, mock.MatchedBy(func(tx *gethTypes.Transaction) bool { return tx.Nonce() == localNextNonce @@ -1422,7 +1425,7 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Errors(t *testing.T) { // This happens if parity is rejecting transactions that are not priced high enough to even get into the mempool at all // It should pretend it was accepted into the mempool and hand off to ethConfirmer to bump gas as normal temporarilyUnderpricedError := "There are too many transactions in the queue. Your transaction was dropped due to limit. Try increasing the fee." - localNextNonce := getLocalNextNonce(t, ethKeyStore, fromAddress) + localNextNonce := getLocalNextNonce(t, eb, fromAddress) // Re-use the previously unfinished transaction, no need to insert new @@ -1455,7 +1458,7 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Errors(t *testing.T) { // configured for the transaction pool. // This is a configuration error by the node operator, since it means they set the base gas level too low. underpricedError := "transaction underpriced" - localNextNonce := getLocalNextNonce(t, ethKeyStore, fromAddress) + localNextNonce := getLocalNextNonce(t, eb, fromAddress) // In this scenario the node operator REALLY fucked up and set the bump // to zero (even though that should not be possible due to config // validation) @@ -1463,6 +1466,7 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Errors(t *testing.T) { c.EVM[0].GasEstimator.BumpMin = assets.NewWeiI(0) c.EVM[0].GasEstimator.BumpPercent = ptr[uint16](0) })) + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(localNextNonce), nil).Once() eb2 := NewTestEthBroadcaster(t, txStore, ethClient, ethKeyStore, evmcfg2, &testCheckerFactory{}, false) cltest.MustCreateUnstartedTx(t, txStore, fromAddress, toAddress, encodedPayload, gasLimit, value, &cltest.FixtureChainID) @@ -1483,7 +1487,7 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Errors(t *testing.T) { t.Run("eth tx is left in progress if eth node returns insufficient eth", func(t *testing.T) { insufficientEthError := "insufficient funds for transfer" - localNextNonce := getLocalNextNonce(t, ethKeyStore, fromAddress) + localNextNonce := getLocalNextNonce(t, eb, fromAddress) etx := cltest.MustCreateUnstartedTx(t, txStore, fromAddress, toAddress, encodedPayload, gasLimit, value, &cltest.FixtureChainID) ethClient.On("SendTransactionReturnCode", mock.Anything, mock.MatchedBy(func(tx *gethTypes.Transaction) bool { return tx.Nonce() == localNextNonce @@ -1512,7 +1516,7 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Errors(t *testing.T) { pgtest.MustExec(t, db, `DELETE FROM evm.txes`) t.Run("eth tx is left in progress if nonce is too high", func(t *testing.T) { - localNextNonce := getLocalNextNonce(t, ethKeyStore, fromAddress) + localNextNonce := getLocalNextNonce(t, eb, fromAddress) nonceGapError := "NonceGap, Future nonce. Expected nonce: " + strconv.FormatUint(localNextNonce, 10) etx := cltest.MustCreateUnstartedTx(t, txStore, fromAddress, toAddress, encodedPayload, gasLimit, value, &cltest.FixtureChainID) ethClient.On("SendTransactionReturnCode", mock.Anything, mock.MatchedBy(func(tx *gethTypes.Transaction) bool { @@ -1553,10 +1557,12 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Errors(t *testing.T) { c.EVM[0].GasEstimator.BumpMin = assets.NewWeiI(0) c.EVM[0].GasEstimator.BumpPercent = ptr[uint16](0) })) + localNextNonce := getLocalNextNonce(t, eb, fromAddress) + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(localNextNonce), nil).Once() eb2 := NewTestEthBroadcaster(t, txStore, ethClient, ethKeyStore, evmcfg2, &testCheckerFactory{}, false) cltest.MustCreateUnstartedTx(t, txStore, fromAddress, toAddress, encodedPayload, gasLimit, value, &cltest.FixtureChainID) underpricedError := "transaction underpriced" - localNextNonce := getLocalNextNonce(t, ethKeyStore, fromAddress) + localNextNonce = getLocalNextNonce(t, eb, fromAddress) ethClient.On("SendTransactionReturnCode", mock.Anything, mock.MatchedBy(func(tx *gethTypes.Transaction) bool { return tx.Nonce() == localNextNonce && tx.GasTipCap().Cmp(big.NewInt(1)) == 0 }), fromAddress).Return(clienttypes.Underpriced, errors.New(underpricedError)).Once() @@ -1575,7 +1581,7 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Errors(t *testing.T) { // configured for the transaction pool. // This is a configuration error by the node operator, since it means they set the base gas level too low. underpricedError := "transaction underpriced" - localNextNonce := getLocalNextNonce(t, ethKeyStore, fromAddress) + localNextNonce := getLocalNextNonce(t, eb, fromAddress) cltest.MustCreateUnstartedTx(t, txStore, fromAddress, toAddress, encodedPayload, gasLimit, value, &cltest.FixtureChainID) // Check gas tip cap verification @@ -1583,6 +1589,7 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Errors(t *testing.T) { c.EVM[0].GasEstimator.EIP1559DynamicFees = ptr(true) c.EVM[0].GasEstimator.TipCapDefault = assets.NewWeiI(0) })) + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(localNextNonce, nil).Once() eb2 := NewTestEthBroadcaster(t, txStore, ethClient, ethKeyStore, evmcfg2, &testCheckerFactory{}, false) retryable, err := eb2.ProcessUnstartedTxs(testutils.Context(t), fromAddress) @@ -1596,6 +1603,8 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Errors(t *testing.T) { c.EVM[0].GasEstimator.EIP1559DynamicFees = ptr(true) c.EVM[0].GasEstimator.TipCapDefault = gasTipCapDefault })) + localNextNonce = getLocalNextNonce(t, eb, fromAddress) + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(localNextNonce), nil).Once() eb2 = NewTestEthBroadcaster(t, txStore, ethClient, ethKeyStore, evmcfg2, &testCheckerFactory{}, false) // Second was underpriced but above minimum @@ -1640,11 +1649,11 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_KeystoreErrors(t *testing.T) { kst := ksmocks.NewEth(t) addresses := []gethCommon.Address{fromAddress} - kst.On("EnabledAddressesForChain", &cltest.FixtureChainID).Return(addresses, nil) - next, err := realKeystore.Eth().NextSequence(fromAddress, testutils.FixtureChainID) - require.NoError(t, err) - kst.On("NextSequence", fromAddress, testutils.FixtureChainID, mock.Anything).Return(next, nil).Once() + kst.On("EnabledAddressesForChain", &cltest.FixtureChainID).Return(addresses, nil).Once() + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(0), nil).Once() eb := NewTestEthBroadcaster(t, txStore, ethClient, kst, evmcfg, &testCheckerFactory{}, false) + _, err := eb.GetNextSequence(fromAddress) + require.NoError(t, err) t.Run("tx signing fails", func(t *testing.T) { etx := cltest.MustCreateUnstartedTx(t, txStore, fromAddress, toAddress, encodedPayload, gasLimit, value, &cltest.FixtureChainID) @@ -1670,21 +1679,27 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_KeystoreErrors(t *testing.T) { assert.Len(t, etx.TxAttempts, 0) // Check that the key did not have its nonce incremented - var nonce int64 - err = db.Get(&nonce, `SELECT next_nonce FROM evm.key_states WHERE address = $1 ORDER BY created_at ASC, id ASC`, fromAddress) + var nonce types.Nonce + nonce, err = eb.GetNextSequence(fromAddress) require.NoError(t, err) - require.Equal(t, int64(localNonce), nonce) + require.Equal(t, int64(localNonce), int64(nonce)) }) } func TestEthBroadcaster_GetNextNonce(t *testing.T) { db := pgtest.NewSqlxDB(t) cfg := configtest.NewGeneralConfig(t, nil) + txStore := cltest.NewTestTxStore(t, db, cfg.Database()) + fromAddress := testutils.NewAddress() + evmcfg := evmtest.NewChainScopedConfig(t, cfg) + ethClient := evmtest.NewEthClientMockWithDefaultChain(t) - ethKeyStore := cltest.NewKeyStore(t, db, cfg.Database()).Eth() - keyState, _ := cltest.MustInsertRandomKeyReturningState(t, ethKeyStore) - - nonce := getLocalNextNonce(t, ethKeyStore, keyState.Address.Address()) + kst := ksmocks.NewEth(t) + addresses := []gethCommon.Address{fromAddress} + kst.On("EnabledAddressesForChain", &cltest.FixtureChainID).Return(addresses, nil).Once() + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(0), nil).Once() + eb := NewTestEthBroadcaster(t, txStore, ethClient, kst, evmcfg, &testCheckerFactory{}, false) + nonce := getLocalNextNonce(t, eb, fromAddress) require.NotNil(t, nonce) assert.Equal(t, int64(0), int64(nonce)) } @@ -1692,20 +1707,23 @@ func TestEthBroadcaster_GetNextNonce(t *testing.T) { func TestEthBroadcaster_IncrementNextNonce(t *testing.T) { db := pgtest.NewSqlxDB(t) cfg := configtest.NewGeneralConfig(t, nil) + txStore := cltest.NewTestTxStore(t, db, cfg.Database()) + kst := ksmocks.NewEth(t) + fromAddress := testutils.NewAddress() + evmcfg := evmtest.NewChainScopedConfig(t, cfg) + ethClient := evmtest.NewEthClientMockWithDefaultChain(t) - ethKeyStore := cltest.NewKeyStore(t, db, cfg.Database()).Eth() - keyState, _ := cltest.MustInsertRandomKeyReturningState(t, ethKeyStore) - - // Cannot increment if supplied nonce doesn't match existing - require.Error(t, ethKeyStore.IncrementNextSequence(keyState.Address.Address(), &cltest.FixtureChainID, evmtypes.Nonce(42))) + addresses := []gethCommon.Address{fromAddress} + kst.On("EnabledAddressesForChain", &cltest.FixtureChainID).Return(addresses, nil).Once() + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(0), nil).Once() + eb := NewTestEthBroadcaster(t, txStore, ethClient, kst, evmcfg, &testCheckerFactory{}, false) - require.NoError(t, ethKeyStore.IncrementNextSequence(keyState.Address.Address(), &cltest.FixtureChainID, evmtypes.Nonce(0))) + require.NoError(t, eb.IncrementNextSequence(fromAddress)) // Nonce bumped to 1 - var nonce int64 - err := db.Get(&nonce, `SELECT next_nonce FROM evm.key_states WHERE address = $1 ORDER BY created_at ASC, id ASC`, keyState.Address.Address()) + nonce, err := eb.GetNextSequence(fromAddress) require.NoError(t, err) - require.Equal(t, int64(1), nonce) + require.Equal(t, int64(1), int64(nonce)) } func TestEthBroadcaster_Trigger(t *testing.T) { @@ -1780,6 +1798,10 @@ func TestEthBroadcaster_SyncNonce(t *testing.T) { ethClient := evmtest.NewEthClientMockWithDefaultChain(t) txBuilder := txmgr.NewEvmTxAttemptBuilder(*ethClient.ConfiguredChainID(), ge, kst, estimator) + kst := ksmocks.NewEth(t) + addresses := []gethCommon.Address{fromAddress} + kst.On("EnabledAddressesForChain", &cltest.FixtureChainID).Return(addresses, nil).Once() + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(0), nil).Once() eb := txmgr.NewEvmBroadcaster(txStore, txmgr.NewEvmTxmClient(ethClient), evmTxmCfg, txmgr.NewEvmTxmFeeConfig(ge), evmcfg.EVM().Transactions(), cfg.Database().Listener(), kst, eventBroadcaster, txBuilder, nil, lggr, checkerFactory, false) err := eb.Start(testutils.Context(t)) assert.NoError(t, err) @@ -1789,68 +1811,181 @@ func TestEthBroadcaster_SyncNonce(t *testing.T) { testutils.WaitForLogMessage(t, observed, "Skipping sequence auto-sync") }) - t.Run("when eth node returns nonce, successfully sets nonce", func(t *testing.T) { + t.Run("when nonce syncer returns new nonce, successfully sets nonce", func(t *testing.T) { ethClient := evmtest.NewEthClientMockWithDefaultChain(t) txBuilder := txmgr.NewEvmTxAttemptBuilder(*ethClient.ConfiguredChainID(), ge, kst, estimator) - txNonceSyncer := txmgr.NewNonceSyncer(txStore, lggr, ethClient, kst) + txNonceSyncer := txmgr.NewNonceSyncer(txStore, lggr, ethClient) + kst := ksmocks.NewEth(t) + addresses := []gethCommon.Address{fromAddress} + kst.On("EnabledAddressesForChain", &cltest.FixtureChainID).Return(addresses, nil).Once() + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(0), nil).Once() eb := txmgr.NewEvmBroadcaster(txStore, txmgr.NewEvmTxmClient(ethClient), evmTxmCfg, txmgr.NewEvmTxmFeeConfig(ge), evmcfg.EVM().Transactions(), cfg.Database().Listener(), kst, eventBroadcaster, txBuilder, txNonceSyncer, lggr, checkerFactory, true) - ethClient.On("PendingNonceAt", mock.Anything, mock.MatchedBy(func(account gethCommon.Address) bool { - return account.Hex() == fromAddress.Hex() - })).Return(ethNodeNonce, nil).Once() - + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(ethNodeNonce), nil).Once() require.NoError(t, eb.Start(ctx)) defer func() { assert.NoError(t, eb.Close()) }() - testutils.WaitForLogMessage(t, observed, "Fast-forwarded nonce") + testutils.WaitForLogMessage(t, observed, "Fast-forward sequence") - // Check keyState to make sure it has correct nonce assigned - var nonce int64 - err := db.Get(&nonce, `SELECT next_nonce FROM evm.key_states WHERE address = $1 ORDER BY created_at ASC, id ASC`, fromAddress) + // Check nextSequenceMap to make sure it has correct nonce assigned + nonce, err := eb.GetNextSequence(fromAddress) require.NoError(t, err) - assert.Equal(t, int64(ethNodeNonce), nonce) + assert.Equal(t, strconv.FormatUint(ethNodeNonce, 10), nonce.String()) // The disabled key did not get updated - err = db.Get(&nonce, `SELECT next_nonce FROM evm.key_states WHERE address = $1 ORDER BY created_at ASC, id ASC`, disabledAddress) - require.NoError(t, err) - assert.Equal(t, int64(0), nonce) + _, err = eb.GetNextSequence(disabledAddress) + require.Error(t, err) }) ethNodeNonce++ observed.TakeAll() - t.Run("when eth node returns error, retries and successfully sets nonce", func(t *testing.T) { + t.Run("when nonce syncer returns error, retries and successfully sets nonce", func(t *testing.T) { ethClient := evmtest.NewEthClientMockWithDefaultChain(t) txBuilder := txmgr.NewEvmTxAttemptBuilder(*ethClient.ConfiguredChainID(), ge, kst, estimator) - txNonceSyncer := txmgr.NewNonceSyncer(txStore, lggr, ethClient, kst) + txNonceSyncer := txmgr.NewNonceSyncer(txStore, lggr, ethClient) + + kst := ksmocks.NewEth(t) + addresses := []gethCommon.Address{fromAddress} + kst.On("EnabledAddressesForChain", &cltest.FixtureChainID).Return(addresses, nil).Once() + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(0), nil).Once() + eb := txmgr.NewEvmBroadcaster(txStore, txmgr.NewEvmTxmClient(ethClient), evmTxmCfg, txmgr.NewEvmTxmFeeConfig(evmcfg.EVM().GasEstimator()), evmcfg.EVM().Transactions(), cfg.Database().Listener(), kst, eventBroadcaster, txBuilder, txNonceSyncer, lggr, checkerFactory, true) eb.XXXTestDisableUnstartedTxAutoProcessing() - ethClient.On("PendingNonceAt", mock.Anything, mock.MatchedBy(func(account gethCommon.Address) bool { - return account.Hex() == fromAddress.Hex() - })).Return(uint64(0), errors.New("something exploded")).Once() - ethClient.On("PendingNonceAt", mock.Anything, mock.MatchedBy(func(account gethCommon.Address) bool { - return account.Hex() == fromAddress.Hex() - })).Return(ethNodeNonce, nil).Once() + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(0), errors.New("something exploded")).Once() + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(ethNodeNonce, nil) require.NoError(t, eb.Start(ctx)) defer func() { assert.NoError(t, eb.Close()) }() - testutils.WaitForLogMessage(t, observed, "Fast-forwarded nonce") + testutils.WaitForLogMessage(t, observed, "Fast-forward sequence") // Check keyState to make sure it has correct nonce assigned - var nonce int64 - err := db.Get(&nonce, `SELECT next_nonce FROM evm.key_states WHERE address = $1 ORDER BY created_at ASC, id ASC`, fromAddress) + nonce, err := eb.GetNextSequence(fromAddress) require.NoError(t, err) - assert.Equal(t, int64(ethNodeNonce), nonce) + assert.Equal(t, int64(ethNodeNonce), int64(nonce)) // The disabled key did not get updated - err = db.Get(&nonce, `SELECT next_nonce FROM evm.key_states WHERE address = $1 ORDER BY created_at ASC, id ASC`, disabledAddress) + _, err = eb.GetNextSequence(disabledAddress) + require.Error(t, err) + }) + +} + +func Test_NextNonce(t *testing.T) { + t.Parallel() + + db := pgtest.NewSqlxDB(t) + cfg := configtest.NewTestGeneralConfig(t) + txStore := cltest.NewTestTxStore(t, db, cfg.Database()) + ks := cltest.NewKeyStore(t, db, cfg.Database()).Eth() + + ethClient := evmtest.NewEthClientMockWithDefaultChain(t) + evmcfg := evmtest.NewChainScopedConfig(t, cfg) + checkerFactory := &txmgr.CheckerFactory{Client: ethClient} + randNonce := testutils.NewRandomPositiveInt64() + _, addr1 := cltest.MustInsertRandomKey(t, ks) + ethClient.On("PendingNonceAt", mock.Anything, addr1).Return(uint64(randNonce), nil).Once() + eb := NewTestEthBroadcaster(t, txStore, ethClient, ks, evmcfg, checkerFactory, false) + + cltest.MustInsertRandomKey(t, ks, *utils.NewBig(testutils.FixtureChainID)) + + nonce, err := eb.GetNextSequence(addr1) + require.NoError(t, err) + assert.Equal(t, randNonce, int64(nonce)) + + randAddr1 := utils.RandomAddress() + _, err = eb.GetNextSequence(randAddr1) + require.Error(t, err) + assert.Contains(t, err.Error(), fmt.Sprintf("address not found in next sequence map: %s", randAddr1.Hex())) + + randAddr2 := utils.RandomAddress() + _, err = eb.GetNextSequence(randAddr2) + require.Error(t, err) + assert.Contains(t, err.Error(), fmt.Sprintf("address not found in next sequence map: %s", randAddr2.Hex())) +} + +func Test_IncrementNextNonce(t *testing.T) { + t.Parallel() + + db := pgtest.NewSqlxDB(t) + cfg := configtest.NewTestGeneralConfig(t) + txStore := cltest.NewTestTxStore(t, db, cfg.Database()) + ks := cltest.NewKeyStore(t, db, cfg.Database()).Eth() + + ethClient := evmtest.NewEthClientMockWithDefaultChain(t) + evmcfg := evmtest.NewChainScopedConfig(t, cfg) + checkerFactory := &txmgr.CheckerFactory{Client: ethClient} + randNonce := testutils.NewRandomPositiveInt64() + _, addr1 := cltest.MustInsertRandomKey(t, ks) + ethClient.On("PendingNonceAt", mock.Anything, addr1).Return(uint64(randNonce), nil).Once() + eb := NewTestEthBroadcaster(t, txStore, ethClient, ks, evmcfg, checkerFactory, false) + + err := eb.IncrementNextSequence(addr1) + require.NoError(t, err) + + var nonce types.Nonce + nonce, err = eb.GetNextSequence(addr1) + require.NoError(t, err) + assert.Equal(t, randNonce+1, int64(nonce)) + + err = eb.IncrementNextSequence(addr1) + require.NoError(t, err) + + randAddr1 := utils.RandomAddress() + err = eb.IncrementNextSequence(randAddr1) + require.Error(t, err) + assert.Contains(t, err.Error(), fmt.Sprintf("address not found in next sequence map: %s", randAddr1.Hex())) + + _, err = eb.GetNextSequence(randAddr1) + require.Error(t, err) + assert.Contains(t, err.Error(), fmt.Sprintf("address not found in next sequence map: %s", randAddr1.Hex())) + + randAddr2 := utils.RandomAddress() + err = eb.IncrementNextSequence(randAddr2) + require.Error(t, err) + assert.Contains(t, err.Error(), fmt.Sprintf("address not found in next sequence map: %s", randAddr2.Hex())) + + // verify it didnt get changed by any erroring calls + nonce, err = eb.GetNextSequence(addr1) + require.NoError(t, err) + assert.Equal(t, randNonce+2, int64(nonce)) +} + +func Test_SetNextNonce(t *testing.T) { + t.Parallel() + + db := pgtest.NewSqlxDB(t) + cfg := configtest.NewTestGeneralConfig(t) + txStore := cltest.NewTestTxStore(t, db, cfg.Database()) + ks := cltest.NewKeyStore(t, db, cfg.Database()).Eth() + + ethClient := evmtest.NewEthClientMockWithDefaultChain(t) + evmcfg := evmtest.NewChainScopedConfig(t, cfg) + checkerFactory := &txmgr.CheckerFactory{Client: ethClient} + _, fromAddress := cltest.MustInsertRandomKey(t, ks) + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(0), nil).Once() + eb := NewTestEthBroadcaster(t, txStore, ethClient, ks, evmcfg, checkerFactory, false) + + t.Run("update next nonce", func(t *testing.T) { + nonce, err := eb.GetNextSequence(fromAddress) require.NoError(t, err) - assert.Equal(t, int64(0), nonce) + assert.Equal(t, int64(0), int64(nonce)) + err = eb.SetNextSequence(fromAddress, evmtypes.Nonce(24)) + require.NoError(t, err) + + newNextNonce, err := eb.GetNextSequence(fromAddress) + require.NoError(t, err) + assert.Equal(t, int64(24), int64(newNextNonce)) }) + t.Run("address not found", func(t *testing.T) { + address := utils.RandomAddress() + err := eb.SetNextSequence(address, evmtypes.Nonce(100)) + require.Error(t, err) + }) } type testCheckerFactory struct { diff --git a/core/chains/evm/txmgr/builder.go b/core/chains/evm/txmgr/builder.go index 464cf6f9c59..8d55b5790f8 100644 --- a/core/chains/evm/txmgr/builder.go +++ b/core/chains/evm/txmgr/builder.go @@ -47,7 +47,7 @@ func NewTxm( // create tx attempt builder txAttemptBuilder := NewEvmTxAttemptBuilder(*client.ConfiguredChainID(), fCfg, keyStore, estimator) txStore := NewTxStore(db, lggr, dbConfig) - txNonceSyncer := NewNonceSyncer(txStore, lggr, client, keyStore) + txNonceSyncer := NewNonceSyncer(txStore, lggr, client) txmCfg := NewEvmTxmConfig(chainConfig) // wrap Evm specific config feeCfg := NewEvmTxmFeeConfig(fCfg) // wrap Evm specific config diff --git a/core/chains/evm/txmgr/evm_tx_store.go b/core/chains/evm/txmgr/evm_tx_store.go index 539c77dfee5..69b7fe7622e 100644 --- a/core/chains/evm/txmgr/evm_tx_store.go +++ b/core/chains/evm/txmgr/evm_tx_store.go @@ -957,6 +957,12 @@ func (o *evmTxStore) FindReceiptsPendingConfirmation(ctx context.Context, blockN return } +func (o *evmTxStore) FindHighestSequence(fromAddress common.Address, chainId *big.Int) (nonce evmtypes.Nonce, err error) { + sql := `SELECT MAX(nonce) FROM evm.txes WHERE from_address = $1 and evm_chain_id = $2` + err = o.q.Get(&nonce, sql, fromAddress.String(), chainId.String()) + return +} + // FindTxWithIdempotencyKey returns any broadcast ethtx with the given idempotencyKey and chainID func (o *evmTxStore) FindTxWithIdempotencyKey(ctx context.Context, idempotencyKey string, chainID *big.Int) (etx *Tx, err error) { var cancel context.CancelFunc @@ -1413,11 +1419,8 @@ func (o *evmTxStore) UpdateTxFatalError(ctx context.Context, etx *Tx) error { } // Updates eth attempt from in_progress to broadcast. Also updates the eth tx to unconfirmed. -// Before it updates both tables though it increments the next nonce from the keystore // One of the more complicated signatures. We have to accept variable pg.QOpt and QueryerFunc arguments -func (o *evmTxStore) UpdateTxAttemptInProgressToBroadcast(etx *Tx, attempt TxAttempt, NewAttemptState txmgrtypes.TxAttemptState, incrNextNonceCallback txmgrtypes.QueryerFunc, qopts ...pg.QOpt) error { - qq := o.q.WithOpts(qopts...) - +func (o *evmTxStore) UpdateTxAttemptInProgressToBroadcast(etx *Tx, attempt TxAttempt, NewAttemptState txmgrtypes.TxAttemptState, incrementSeqFunc func(address common.Address) error) error { if etx.BroadcastAt == nil { return errors.New("unconfirmed transaction must have broadcast_at time") } @@ -1435,10 +1438,7 @@ func (o *evmTxStore) UpdateTxAttemptInProgressToBroadcast(etx *Tx, attempt TxAtt } etx.State = txmgr.TxUnconfirmed attempt.State = NewAttemptState - return qq.Transaction(func(tx pg.Queryer) error { - if err := incrNextNonceCallback(tx); err != nil { - return pkgerrors.Wrap(err, "SaveEthTxAttempt failed on incrNextNonceCallback") - } + return o.q.Transaction(func(tx pg.Queryer) error { var dbEtx DbEthTx dbEtx.FromTx(etx) if err := tx.Get(&dbEtx, `UPDATE evm.txes SET state=$1, error=$2, broadcast_at=$3, initial_broadcast_at=$4 WHERE id = $5 RETURNING *`, dbEtx.State, dbEtx.Error, dbEtx.BroadcastAt, dbEtx.InitialBroadcastAt, dbEtx.ID); err != nil { @@ -1450,6 +1450,9 @@ func (o *evmTxStore) UpdateTxAttemptInProgressToBroadcast(etx *Tx, attempt TxAtt if err := tx.Get(&dbAttempt, `UPDATE evm.tx_attempts SET state = $1 WHERE id = $2 RETURNING *`, dbAttempt.State, dbAttempt.ID); err != nil { return pkgerrors.Wrap(err, "SaveEthTxAttempt failed to save eth_tx_attempt") } + if err := incrementSeqFunc(etx.FromAddress); err != nil { + return pkgerrors.Wrap(err, "IncrementNextSequence failed to increment the next sequence in the broadcaster") + } return nil }) } diff --git a/core/chains/evm/txmgr/evm_tx_store_test.go b/core/chains/evm/txmgr/evm_tx_store_test.go index 30f8912d260..728ecc2781b 100644 --- a/core/chains/evm/txmgr/evm_tx_store_test.go +++ b/core/chains/evm/txmgr/evm_tx_store_test.go @@ -24,6 +24,7 @@ import ( "github.com/smartcontractkit/chainlink/v2/core/store/models" "github.com/smartcontractkit/chainlink/v2/core/utils" + "github.com/ethereum/go-ethereum/common" "github.com/google/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" @@ -1210,8 +1211,7 @@ func TestORM_UpdateTxAttemptInProgressToBroadcast(t *testing.T) { i := int16(0) etx.BroadcastAt = &time1 etx.InitialBroadcastAt = &time1 - err := txStore.UpdateTxAttemptInProgressToBroadcast(&etx, attempt, txmgrtypes.TxAttemptBroadcast, func(_ pg.Queryer) error { - // dummy function because tests do not use keystore as source of truth for next nonce number + err := txStore.UpdateTxAttemptInProgressToBroadcast(&etx, attempt, txmgrtypes.TxAttemptBroadcast, func(_ common.Address) error { i++ return nil }) @@ -1365,32 +1365,6 @@ func TestORM_HasInProgressTransaction(t *testing.T) { }) } -func TestORM_UpdateEthKeyNextNonce(t *testing.T) { - t.Parallel() - - db := pgtest.NewSqlxDB(t) - cfg := newTestChainScopedConfig(t) - txStore := cltest.NewTxStore(t, db, cfg.Database()) - ethKeyStore := cltest.NewKeyStore(t, db, cfg.Database()).Eth() - ethClient := evmtest.NewEthClientMockWithDefaultChain(t) - ethKeyState, fromAddress := cltest.MustInsertRandomKeyReturningState(t, ethKeyStore) - - t.Run("update next nonce", func(t *testing.T) { - assert.Equal(t, int64(0), ethKeyState.NextNonce) - err := txStore.UpdateKeyNextSequence(evmtypes.Nonce(24), evmtypes.Nonce(0), fromAddress, ethClient.ConfiguredChainID()) - require.NoError(t, err) - - newNextNonce, err := ethKeyStore.NextSequence(fromAddress, ethClient.ConfiguredChainID()) - require.NoError(t, err) - assert.Equal(t, int64(24), newNextNonce.Int64()) - }) - - t.Run("no rows found", func(t *testing.T) { - err := txStore.UpdateKeyNextSequence(evmtypes.Nonce(100), evmtypes.Nonce(123), fromAddress, ethClient.ConfiguredChainID()) - require.Error(t, err) - }) -} - func TestORM_CountUnconfirmedTransactions(t *testing.T) { t.Parallel() diff --git a/core/chains/evm/txmgr/mocks/evm_tx_store.go b/core/chains/evm/txmgr/mocks/evm_tx_store.go index 8c078ba3d15..6dc95f7aea4 100644 --- a/core/chains/evm/txmgr/mocks/evm_tx_store.go +++ b/core/chains/evm/txmgr/mocks/evm_tx_store.go @@ -14,8 +14,6 @@ import ( mock "github.com/stretchr/testify/mock" - pg "github.com/smartcontractkit/chainlink/v2/core/services/pg" - time "time" types "github.com/smartcontractkit/chainlink/v2/common/txmgr/types" @@ -147,6 +145,30 @@ func (_m *EvmTxStore) DeleteInProgressAttempt(ctx context.Context, attempt types return r0 } +// FindHighestSequence provides a mock function with given fields: fromAddress, chainId +func (_m *EvmTxStore) FindHighestSequence(fromAddress common.Address, chainId *big.Int) (evmtypes.Nonce, error) { + ret := _m.Called(fromAddress, chainId) + + var r0 evmtypes.Nonce + var r1 error + if rf, ok := ret.Get(0).(func(common.Address, *big.Int) (evmtypes.Nonce, error)); ok { + return rf(fromAddress, chainId) + } + if rf, ok := ret.Get(0).(func(common.Address, *big.Int) evmtypes.Nonce); ok { + r0 = rf(fromAddress, chainId) + } else { + r0 = ret.Get(0).(evmtypes.Nonce) + } + + if rf, ok := ret.Get(1).(func(common.Address, *big.Int) error); ok { + r1 = rf(fromAddress, chainId) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + // FindNextUnstartedTransactionFromAddress provides a mock function with given fields: ctx, etx, fromAddress, chainID func (_m *EvmTxStore) FindNextUnstartedTransactionFromAddress(ctx context.Context, etx *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], fromAddress common.Address, chainID *big.Int) error { ret := _m.Called(ctx, etx, fromAddress, chainID) @@ -878,41 +900,13 @@ func (_m *EvmTxStore) UpdateBroadcastAts(ctx context.Context, now time.Time, etx return r0 } -// UpdateKeyNextSequence provides a mock function with given fields: newNextSequence, currentNextSequence, address, chainID, qopts -func (_m *EvmTxStore) UpdateKeyNextSequence(newNextSequence evmtypes.Nonce, currentNextSequence evmtypes.Nonce, address common.Address, chainID *big.Int, qopts ...pg.QOpt) error { - _va := make([]interface{}, len(qopts)) - for _i := range qopts { - _va[_i] = qopts[_i] - } - var _ca []interface{} - _ca = append(_ca, newNextSequence, currentNextSequence, address, chainID) - _ca = append(_ca, _va...) - ret := _m.Called(_ca...) - - var r0 error - if rf, ok := ret.Get(0).(func(evmtypes.Nonce, evmtypes.Nonce, common.Address, *big.Int, ...pg.QOpt) error); ok { - r0 = rf(newNextSequence, currentNextSequence, address, chainID, qopts...) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -// UpdateTxAttemptInProgressToBroadcast provides a mock function with given fields: etx, attempt, NewAttemptState, incrNextSequenceCallback, qopts -func (_m *EvmTxStore) UpdateTxAttemptInProgressToBroadcast(etx *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], attempt types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], NewAttemptState types.TxAttemptState, incrNextSequenceCallback func(pg.Queryer) error, qopts ...pg.QOpt) error { - _va := make([]interface{}, len(qopts)) - for _i := range qopts { - _va[_i] = qopts[_i] - } - var _ca []interface{} - _ca = append(_ca, etx, attempt, NewAttemptState, incrNextSequenceCallback) - _ca = append(_ca, _va...) - ret := _m.Called(_ca...) +// UpdateTxAttemptInProgressToBroadcast provides a mock function with given fields: etx, attempt, NewAttemptState, incrementSeqFunc +func (_m *EvmTxStore) UpdateTxAttemptInProgressToBroadcast(etx *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], attempt types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], NewAttemptState types.TxAttemptState, incrementSeqFunc func(common.Address) error) error { + ret := _m.Called(etx, attempt, NewAttemptState, incrementSeqFunc) var r0 error - if rf, ok := ret.Get(0).(func(*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], types.TxAttemptState, func(pg.Queryer) error, ...pg.QOpt) error); ok { - r0 = rf(etx, attempt, NewAttemptState, incrNextSequenceCallback, qopts...) + if rf, ok := ret.Get(0).(func(*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], types.TxAttemptState, func(common.Address) error) error); ok { + r0 = rf(etx, attempt, NewAttemptState, incrementSeqFunc) } else { r0 = ret.Error(0) } diff --git a/core/chains/evm/txmgr/models.go b/core/chains/evm/txmgr/models.go index e87c54c2a3f..9044c52c9ae 100644 --- a/core/chains/evm/txmgr/models.go +++ b/core/chains/evm/txmgr/models.go @@ -25,7 +25,7 @@ type ( TransactionStore = txmgrtypes.TransactionStore[common.Address, *big.Int, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee] KeyStore = txmgrtypes.KeyStore[common.Address, *big.Int, evmtypes.Nonce] TxAttemptBuilder = txmgrtypes.TxAttemptBuilder[*big.Int, *evmtypes.Head, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee] - NonceSyncer = txmgr.SequenceSyncer[common.Address, common.Hash, common.Hash] + NonceSyncer = txmgr.SequenceSyncer[common.Address, common.Hash, common.Hash, evmtypes.Nonce] TransmitCheckerFactory = txmgr.TransmitCheckerFactory[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee] Txm = txmgr.Txm[*big.Int, *evmtypes.Head, common.Address, common.Hash, common.Hash, *evmtypes.Receipt, evmtypes.Nonce, gas.EvmFee] TxManager = txmgr.TxManager[*big.Int, *evmtypes.Head, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee] diff --git a/core/chains/evm/txmgr/nonce_syncer.go b/core/chains/evm/txmgr/nonce_syncer.go index af982e062ac..dc0d27e6414 100644 --- a/core/chains/evm/txmgr/nonce_syncer.go +++ b/core/chains/evm/txmgr/nonce_syncer.go @@ -10,9 +10,8 @@ import ( "github.com/smartcontractkit/chainlink/v2/common/txmgr" evmclient "github.com/smartcontractkit/chainlink/v2/core/chains/evm/client" - evmtypes "github.com/smartcontractkit/chainlink/v2/core/chains/evm/types" + "github.com/smartcontractkit/chainlink/v2/core/chains/evm/types" "github.com/smartcontractkit/chainlink/v2/core/logger" - "github.com/smartcontractkit/chainlink/v2/core/services/pg" ) // NonceSyncer manages the delicate task of syncing the local nonce with the @@ -22,7 +21,7 @@ import ( // it to our local value. // // Usually the on-chain nonce will be the same as (or lower than) the -// next_nonce in the DB, in which case we do nothing. +// highest sequence in the DB, in which case we do nothing. // // If we are restoring from a backup however, or another wallet has used the // account, the chain nonce might be higher than our local one. In this @@ -47,14 +46,13 @@ import ( // // This gives us re-org protection up to EVM.FinalityDepth deep in the // worst case, which is in line with our other guarantees. -var _ txmgr.SequenceSyncer[common.Address, common.Hash, common.Hash] = &nonceSyncerImpl{} +var _ txmgr.SequenceSyncer[common.Address, common.Hash, common.Hash, types.Nonce] = &nonceSyncerImpl{} type nonceSyncerImpl struct { txStore EvmTxStore client TxmClient chainID *big.Int logger logger.Logger - kst KeyStore } // NewNonceSyncer returns a new syncer @@ -62,7 +60,6 @@ func NewNonceSyncer( txStore EvmTxStore, lggr logger.Logger, ethClient evmclient.Client, - kst KeyStore, ) NonceSyncer { lggr = lggr.Named("NonceSyncer") return &nonceSyncerImpl{ @@ -70,7 +67,6 @@ func NewNonceSyncer( client: NewEvmTxmClient(ethClient), chainID: ethClient.ConfiguredChainID(), logger: lggr, - kst: kst, } } @@ -78,64 +74,33 @@ func NewNonceSyncer( // // This should only be called once, before the EthBroadcaster has started. // Calling it later is not safe and could lead to races. -func (s nonceSyncerImpl) Sync(ctx context.Context, addr common.Address) (err error) { - err = s.fastForwardNonceIfNecessary(ctx, addr) - return errors.Wrap(err, "NonceSyncer#fastForwardNoncesIfNecessary failed") +func (s nonceSyncerImpl) Sync(ctx context.Context, addr common.Address, localNonce types.Nonce) (nonce types.Nonce, err error) { + nonce, err = s.fastForwardNonceIfNecessary(ctx, addr, localNonce) + return nonce, errors.Wrap(err, "NonceSyncer#fastForwardNoncesIfNecessary failed") } -func (s nonceSyncerImpl) fastForwardNonceIfNecessary(ctx context.Context, address common.Address) error { +func (s nonceSyncerImpl) fastForwardNonceIfNecessary(ctx context.Context, address common.Address, localNonce types.Nonce) (types.Nonce, error) { chainNonce, err := s.pendingNonceFromEthClient(ctx, address) if err != nil { - return errors.Wrap(err, "GetNextNonce failed to loadInitialNonceFromEthClient") + return localNonce, errors.Wrap(err, "GetNextNonce failed to loadInitialNonceFromEthClient") } if chainNonce == 0 { - return nil + return localNonce, nil } - - keyNextNonce, err := s.kst.NextSequence(address, s.chainID, pg.WithParentCtx(ctx)) - if err != nil { - return err - } - - localNonce := keyNextNonce - hasInProgressTransaction, err := s.txStore.HasInProgressTransaction(ctx, address, s.chainID) - - if err != nil { - return errors.Wrapf(err, "failed to query for in_progress transaction for address %s", address.String()) - } else if hasInProgressTransaction { - // If we have an 'in_progress' transaction, our keys.next_nonce will be - // one lower than it should because we must have crashed mid-execution. - // The EthBroadcaster will automatically take care of this and - // increment it by one later, for now we just increment by one here. - localNonce++ - } - if chainNonce <= uint64(localNonce) { - return nil + if chainNonce <= localNonce { + return localNonce, nil } s.logger.Warnw(fmt.Sprintf("address %s has been used before, either by an external wallet or a different Chainlink node. "+ "Local nonce is %v but the on-chain nonce for this account was %v. "+ "It's possible that this node was restored from a backup. If so, transactions sent by the previous node will NOT be re-org protected and in rare cases may need to be manually bumped/resubmitted. "+ "Please note that using the chainlink keys with an external wallet is NOT SUPPORTED and can lead to missed or stuck transactions. ", address, localNonce, chainNonce), - "address", address.String(), "keyNextNonce", keyNextNonce, "localNonce", localNonce, "chainNonce", chainNonce) - - // Need to remember to decrement the chain nonce by one to account for in_progress transaction - newNextNonce := int64(chainNonce) - if hasInProgressTransaction { - newNextNonce-- - } + "address", address.String(), "localNonce", localNonce, "chainNonce", chainNonce) - err = s.txStore.UpdateKeyNextSequence(evmtypes.Nonce(newNextNonce), keyNextNonce, address, s.chainID, pg.WithParentCtx(ctx)) - - if errors.Is(err, ErrKeyNotUpdated) { - return errors.Errorf("NonceSyncer#fastForwardNonceIfNecessary optimistic lock failure fastforwarding nonce %v to %v for key %s", localNonce, chainNonce, address.String()) - } else if err == nil { - s.logger.Infow("Fast-forwarded nonce", "address", address, "newNextNonce", newNextNonce, "oldNextNonce", keyNextNonce) - } - return err + return chainNonce, nil } -func (s nonceSyncerImpl) pendingNonceFromEthClient(ctx context.Context, account common.Address) (uint64, error) { +func (s nonceSyncerImpl) pendingNonceFromEthClient(ctx context.Context, account common.Address) (types.Nonce, error) { nextNonce, err := s.client.PendingSequenceAt(ctx, account) - return uint64(nextNonce), errors.WithStack(err) + return nextNonce, errors.WithStack(err) } diff --git a/core/chains/evm/txmgr/nonce_syncer_test.go b/core/chains/evm/txmgr/nonce_syncer_test.go index f2334038969..9b6b1962365 100644 --- a/core/chains/evm/txmgr/nonce_syncer_test.go +++ b/core/chains/evm/txmgr/nonce_syncer_test.go @@ -4,6 +4,7 @@ import ( "testing" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/txmgr" + "github.com/smartcontractkit/chainlink/v2/core/chains/evm/types" "github.com/smartcontractkit/chainlink/v2/core/internal/cltest" "github.com/smartcontractkit/chainlink/v2/core/internal/testutils" configtest "github.com/smartcontractkit/chainlink/v2/core/internal/testutils/configtest/v2" @@ -11,13 +12,10 @@ import ( "github.com/smartcontractkit/chainlink/v2/core/internal/testutils/pgtest" "github.com/smartcontractkit/chainlink/v2/core/logger" - "github.com/ethereum/go-ethereum/common" "github.com/pkg/errors" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" - - "github.com/smartcontractkit/sqlx" ) func Test_NonceSyncer_Sync(t *testing.T) { @@ -25,149 +23,92 @@ func Test_NonceSyncer_Sync(t *testing.T) { t.Run("returns error if PendingNonceAt fails", func(t *testing.T) { db := pgtest.NewSqlxDB(t) + cfg := configtest.NewTestGeneralConfig(t) + txStore := cltest.NewTestTxStore(t, db, cfg.Database()) ethClient := evmtest.NewEthClientMockWithDefaultChain(t) - cfg := configtest.NewGeneralConfig(t, nil) ethKeyStore := cltest.NewKeyStore(t, db, cfg.Database()).Eth() - txStore := cltest.NewTxStore(t, db, cfg.Database()) - - _, from := cltest.MustInsertRandomKey(t, ethKeyStore) - ethClient.On("PendingNonceAt", mock.Anything, mock.MatchedBy(func(addr common.Address) bool { - return from == addr - })).Return(uint64(0), errors.New("something exploded")) + _, from := cltest.MustInsertRandomKey(t, ethKeyStore) - ns := txmgr.NewNonceSyncer(txStore, logger.TestLogger(t), ethClient, ethKeyStore) + ns := txmgr.NewNonceSyncer(txStore, logger.TestLogger(t), ethClient) - sendingKeys := cltest.MustSendingKeyStates(t, ethKeyStore, testutils.FixtureChainID) - err := ns.Sync(testutils.Context(t), sendingKeys[0].Address.Address()) + ethClient.On("PendingNonceAt", mock.Anything, from).Return(uint64(0), errors.New("something exploded")) + _, err := ns.Sync(testutils.Context(t), from, types.Nonce(0)) require.Error(t, err) assert.Contains(t, err.Error(), "something exploded") cltest.AssertCount(t, db, "evm.txes", 0) cltest.AssertCount(t, db, "evm.tx_attempts", 0) - - assertDatabaseNonce(t, db, from, 0) }) t.Run("does nothing if chain nonce reflects local nonce", func(t *testing.T) { db := pgtest.NewSqlxDB(t) + cfg := configtest.NewTestGeneralConfig(t) + txStore := cltest.NewTestTxStore(t, db, cfg.Database()) ethClient := evmtest.NewEthClientMockWithDefaultChain(t) - cfg := configtest.NewGeneralConfig(t, nil) ethKeyStore := cltest.NewKeyStore(t, db, cfg.Database()).Eth() - txStore := cltest.NewTxStore(t, db, cfg.Database()) _, from := cltest.MustInsertRandomKey(t, ethKeyStore) - ethClient.On("PendingNonceAt", mock.Anything, mock.MatchedBy(func(addr common.Address) bool { - return from == addr - })).Return(uint64(0), nil) + ns := txmgr.NewNonceSyncer(txStore, logger.TestLogger(t), ethClient) - ns := txmgr.NewNonceSyncer(txStore, logger.TestLogger(t), ethClient, ethKeyStore) + ethClient.On("PendingNonceAt", mock.Anything, from).Return(uint64(0), nil) - sendingKeys := cltest.MustSendingKeyStates(t, ethKeyStore, testutils.FixtureChainID) - require.NoError(t, ns.Sync(testutils.Context(t), sendingKeys[0].Address.Address())) + nonce, err := ns.Sync(testutils.Context(t), from, 0) + require.Equal(t, nonce.Int64(), int64(0)) + require.NoError(t, err) cltest.AssertCount(t, db, "evm.txes", 0) cltest.AssertCount(t, db, "evm.tx_attempts", 0) - - assertDatabaseNonce(t, db, from, 0) }) t.Run("does nothing if chain nonce is behind local nonce", func(t *testing.T) { db := pgtest.NewSqlxDB(t) - cfg := configtest.NewGeneralConfig(t, nil) - txStore := cltest.NewTxStore(t, db, cfg.Database()) + cfg := configtest.NewTestGeneralConfig(t) + txStore := cltest.NewTestTxStore(t, db, cfg.Database()) + ks := cltest.NewKeyStore(t, db, cfg.Database()).Eth() ethClient := evmtest.NewEthClientMockWithDefaultChain(t) - ethKeyStore := cltest.NewKeyStore(t, db, cfg.Database()).Eth() - - k1, _ := cltest.RandomKey{Nonce: 32}.MustInsert(t, ethKeyStore) - ethClient.On("PendingNonceAt", mock.Anything, mock.MatchedBy(func(addr common.Address) bool { - return k1.Address == addr - })).Return(uint64(31), nil) + _, fromAddress := cltest.RandomKey{Nonce: 32}.MustInsert(t, ks) - ns := txmgr.NewNonceSyncer(txStore, logger.TestLogger(t), ethClient, ethKeyStore) + ns := txmgr.NewNonceSyncer(txStore, logger.TestLogger(t), ethClient) - sendingKeys := cltest.MustSendingKeyStates(t, ethKeyStore, testutils.FixtureChainID) - require.NoError(t, ns.Sync(testutils.Context(t), sendingKeys[0].Address.Address())) + // Used to mock the chain nonce + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(5), nil) + nonce, err := ns.Sync(testutils.Context(t), fromAddress, types.Nonce(32)) + require.Equal(t, nonce.Int64(), int64(32)) + require.NoError(t, err) cltest.AssertCount(t, db, "evm.txes", 0) cltest.AssertCount(t, db, "evm.tx_attempts", 0) - - assertDatabaseNonce(t, db, k1.Address, 32) }) t.Run("fast forwards if chain nonce is ahead of local nonce", func(t *testing.T) { db := pgtest.NewSqlxDB(t) - cfg := configtest.NewGeneralConfig(t, nil) - txStore := cltest.NewTxStore(t, db, cfg.Database()) - + cfg := configtest.NewTestGeneralConfig(t) + txStore := cltest.NewTestTxStore(t, db, cfg.Database()) ethClient := evmtest.NewEthClientMockWithDefaultChain(t) ethKeyStore := cltest.NewKeyStore(t, db, cfg.Database()).Eth() _, key1 := cltest.MustInsertRandomKey(t, ethKeyStore) _, key2 := cltest.RandomKey{Nonce: 32}.MustInsert(t, ethKeyStore) - ethClient.On("PendingNonceAt", mock.Anything, mock.MatchedBy(func(addr common.Address) bool { - // Nothing to do for key2 - return key2 == addr - })).Return(uint64(32), nil) - ethClient.On("PendingNonceAt", mock.Anything, mock.MatchedBy(func(addr common.Address) bool { - // key1 has chain nonce of 5 which is ahead of local nonce 0 - return key1 == addr - })).Return(uint64(5), nil) + key1LocalNonce := types.Nonce(0) + key2LocalNonce := types.Nonce(32) - ns := txmgr.NewNonceSyncer(txStore, logger.TestLogger(t), ethClient, ethKeyStore) + ns := txmgr.NewNonceSyncer(txStore, logger.TestLogger(t), ethClient) - sendingKeys := cltest.MustSendingKeyStates(t, ethKeyStore, testutils.FixtureChainID) - for _, k := range sendingKeys { - require.NoError(t, ns.Sync(testutils.Context(t), k.Address.Address())) - } + // Used to mock the chain nonce + ethClient.On("PendingNonceAt", mock.Anything, key1).Return(uint64(5), nil).Once() + ethClient.On("PendingNonceAt", mock.Anything, key2).Return(uint64(32), nil).Once() - assertDatabaseNonce(t, db, key1, 5) - }) - - t.Run("counts 'in_progress' eth_tx as bumping the local next nonce by 1", func(t *testing.T) { - db := pgtest.NewSqlxDB(t) - cfg := configtest.NewGeneralConfig(t, nil) - txStore := cltest.NewTestTxStore(t, db, cfg.Database()) - ethKeyStore := cltest.NewKeyStore(t, db, cfg.Database()).Eth() - - _, key1 := cltest.MustInsertRandomKey(t, ethKeyStore) - - cltest.MustInsertInProgressEthTxWithAttempt(t, txStore, 1, key1) + syncerNonce, err := ns.Sync(testutils.Context(t), key1, key1LocalNonce) + require.NoError(t, err) + require.Greater(t, syncerNonce, key1LocalNonce) - ethClient := evmtest.NewEthClientMockWithDefaultChain(t) - ethClient.On("PendingNonceAt", mock.Anything, mock.MatchedBy(func(addr common.Address) bool { - // key1 has chain nonce of 1 which is ahead of keys.next_nonce (0) - // by 1, but does not need to change when taking into account the in_progress tx - return key1 == addr - })).Return(uint64(1), nil) - ns := txmgr.NewNonceSyncer(txStore, logger.TestLogger(t), ethClient, ethKeyStore) - - sendingKeys := cltest.MustSendingKeyStates(t, ethKeyStore, testutils.FixtureChainID) - require.NoError(t, ns.Sync(testutils.Context(t), sendingKeys[0].Address.Address())) - assertDatabaseNonce(t, db, key1, 0) - - ethClient = evmtest.NewEthClientMockWithDefaultChain(t) - ethClient.On("PendingNonceAt", mock.Anything, mock.MatchedBy(func(addr common.Address) bool { - // key1 has chain nonce of 2 which is ahead of keys.next_nonce (0) - // by 2, but only ahead by 1 if we count the in_progress tx as +1 - return key1 == addr - })).Return(uint64(2), nil) - ns = txmgr.NewNonceSyncer(txStore, logger.TestLogger(t), ethClient, ethKeyStore) - - require.NoError(t, ns.Sync(testutils.Context(t), sendingKeys[0].Address.Address())) - assertDatabaseNonce(t, db, key1, 1) + syncerNonce, err = ns.Sync(testutils.Context(t), key2, key2LocalNonce) + require.NoError(t, err) + require.Equal(t, syncerNonce, key2LocalNonce) }) } - -func assertDatabaseNonce(t *testing.T, db *sqlx.DB, address common.Address, nonce int64) { - t.Helper() - - var nextNonce int64 - err := db.Get(&nextNonce, `SELECT next_nonce FROM evm.key_states WHERE address = $1`, address) - require.NoError(t, err) - assert.Equal(t, nonce, nextNonce) -} diff --git a/core/chains/evm/txmgr/txmgr_test.go b/core/chains/evm/txmgr/txmgr_test.go index 6865a23634d..ce173344859 100644 --- a/core/chains/evm/txmgr/txmgr_test.go +++ b/core/chains/evm/txmgr/txmgr_test.go @@ -5,7 +5,6 @@ import ( "encoding/json" "fmt" "math/big" - "sync/atomic" "testing" "time" @@ -648,23 +647,6 @@ func TestTxm_Lifecycle(t *testing.T) { unsub.AwaitOrFail(t, 1*time.Second) } -type fnMock struct{ called atomic.Bool } - -func (fm *fnMock) Fn() { - swapped := fm.called.CompareAndSwap(false, true) - if !swapped { - panic("func called more than once") - } -} - -func (fm *fnMock) AssertNotCalled(t *testing.T) { - assert.False(t, fm.called.Load()) -} - -func (fm *fnMock) AssertCalled(t *testing.T) { - assert.True(t, fm.called.Load()) -} - func TestTxm_Reset(t *testing.T) { t.Parallel() @@ -707,35 +689,23 @@ func TestTxm_Reset(t *testing.T) { } t.Run("returns error if not started", func(t *testing.T) { - f := new(fnMock) - - err := txm.Reset(f.Fn, addr, false) + err := txm.Reset(addr, false) require.Error(t, err) assert.EqualError(t, err, "not started") - - f.AssertNotCalled(t) }) require.NoError(t, txm.Start(testutils.Context(t))) defer func() { assert.NoError(t, txm.Close()) }() - t.Run("calls function if started", func(t *testing.T) { - f := new(fnMock) - - err := txm.Reset(f.Fn, addr, false) + t.Run("returns no error if started", func(t *testing.T) { + err := txm.Reset(addr, false) require.NoError(t, err) - - f.AssertCalled(t) }) - t.Run("calls function and deletes relevant evm.txes if abandon=true", func(t *testing.T) { - f := new(fnMock) - - err := txm.Reset(f.Fn, addr, true) + t.Run("deletes relevant evm.txes if abandon=true", func(t *testing.T) { + err := txm.Reset(addr, true) require.NoError(t, err) - f.AssertCalled(t) - var s string err = db.Get(&s, `SELECT error FROM evm.txes WHERE from_address = $1 AND state = 'fatal_error'`, addr) require.NoError(t, err) diff --git a/core/chains/evm/types/nonce.go b/core/chains/evm/types/nonce.go index 0c3256dc545..4d24cf7d251 100644 --- a/core/chains/evm/types/nonce.go +++ b/core/chains/evm/types/nonce.go @@ -17,3 +17,7 @@ func (n Nonce) Int64() int64 { func (n Nonce) String() string { return strconv.FormatInt(n.Int64(), 10) } + +func (n Nonce) Increment() Nonce { + return Nonce(n + 1) +} diff --git a/core/cmd/eth_keys_commands.go b/core/cmd/eth_keys_commands.go index e5f1f025966..c13bea80f2f 100644 --- a/core/cmd/eth_keys_commands.go +++ b/core/cmd/eth_keys_commands.go @@ -41,7 +41,7 @@ func initEthKeysSubCmd(s *Shell) cli.Command { }, { Name: "list", - Usage: "List available Ethereum accounts with their ETH & LINK balances, nonces, and other metadata", + Usage: "List available Ethereum accounts with their ETH & LINK balances and other metadata", Action: s.ListETHKeys, }, { @@ -100,10 +100,6 @@ func initEthKeysSubCmd(s *Shell) cli.Command { Usage: "chain ID of the key", Required: true, }, - cli.Uint64Flag{ - Name: "set-next-nonce, setNextNonce", - Usage: "manually set the next nonce for the key on the given chain. This should not be necessary during normal operation. USE WITH CAUTION: Setting this incorrectly can break your node", - }, cli.BoolFlag{ Name: "enable", Usage: "enable the key for the given chain", @@ -130,7 +126,6 @@ func (p *EthKeyPresenter) ToRow() []string { return []string{ p.Address, p.EVMChainID.String(), - fmt.Sprintf("%d", p.NextNonce), p.EthBalance.String(), p.LinkBalance.String(), fmt.Sprintf("%v", p.Disabled), @@ -140,7 +135,7 @@ func (p *EthKeyPresenter) ToRow() []string { } } -var ethKeysTableHeaders = []string{"Address", "EVM Chain ID", "Next Nonce", "ETH", "LINK", "Disabled", "Created", "Updated", "Max Gas Price Wei"} +var ethKeysTableHeaders = []string{"Address", "EVM Chain ID", "ETH", "LINK", "Disabled", "Created", "Updated", "Max Gas Price Wei"} // RenderTable implements TableRenderer func (p *EthKeyPresenter) RenderTable(rt RendererTable) error { @@ -368,9 +363,6 @@ func (s *Shell) UpdateChainEVMKey(c *cli.Context) (err error) { abandon := c.String("abandon") query.Set("abandon", abandon) - if c.IsSet("set-next-nonce") { - query.Set("nextNonce", c.String("set-next-nonce")) - } if c.IsSet("enable") && c.IsSet("disable") { return s.errorOut(errors.New("cannot set both --enable and --disable simultaneously")) } else if c.Bool("enable") { diff --git a/core/cmd/eth_keys_commands_test.go b/core/cmd/eth_keys_commands_test.go index 30e115e9482..630e76783a2 100644 --- a/core/cmd/eth_keys_commands_test.go +++ b/core/cmd/eth_keys_commands_test.go @@ -90,6 +90,7 @@ func TestShell_ListETHKeys(t *testing.T) { ethClient := newEthMock(t) ethClient.On("BalanceAt", mock.Anything, mock.Anything, mock.Anything).Return(big.NewInt(42), nil) ethClient.On("LINKBalance", mock.Anything, mock.Anything, mock.Anything).Return(assets.NewLinkFromJuels(13), nil) + ethClient.On("PendingNonceAt", mock.Anything, mock.Anything).Return(uint64(0), nil) app := startNewApplicationV2(t, func(c *chainlink.Config, s *chainlink.Secrets) { c.EVM[0].Enabled = ptr(true) c.EVM[0].NonceAutoSync = ptr(false) @@ -114,6 +115,7 @@ func TestShell_ListETHKeys_Error(t *testing.T) { ethClient := newEthMock(t) ethClient.On("BalanceAt", mock.Anything, mock.Anything, mock.Anything).Return(nil, errors.New("fake error")) ethClient.On("LINKBalance", mock.Anything, mock.Anything, mock.Anything).Return(nil, errors.New("fake error")) + ethClient.On("PendingNonceAt", mock.Anything, mock.Anything).Return(uint64(0), nil) app := startNewApplicationV2(t, func(c *chainlink.Config, s *chainlink.Secrets) { c.EVM[0].Enabled = ptr(true) c.EVM[0].NonceAutoSync = ptr(false) @@ -156,7 +158,7 @@ func TestShell_ListETHKeys_Disabled(t *testing.T) { assert.Nil(t, balances[0].LinkBalance) assert.Nil(t, balances[0].MaxGasPriceWei) assert.Equal(t, []string{ - k.Address.String(), "0", "0", "", "0", "false", + k.Address.String(), "0", "", "0", "false", balances[0].UpdatedAt.String(), balances[0].CreatedAt.String(), "", }, balances[0].ToRow()) } @@ -167,6 +169,8 @@ func TestShell_CreateETHKey(t *testing.T) { ethClient := newEthMock(t) ethClient.On("BalanceAt", mock.Anything, mock.Anything, mock.Anything).Return(big.NewInt(42), nil) ethClient.On("LINKBalance", mock.Anything, mock.Anything, mock.Anything).Return(assets.NewLinkFromJuels(42), nil) + ethClient.On("PendingNonceAt", mock.Anything, mock.Anything).Return(uint64(0), nil) + app := startNewApplicationV2(t, func(c *chainlink.Config, s *chainlink.Secrets) { c.EVM[0].Enabled = ptr(true) c.EVM[0].NonceAutoSync = ptr(false) @@ -240,6 +244,7 @@ func TestShell_ImportExportETHKey_NoChains(t *testing.T) { ethClient := newEthMock(t) ethClient.On("BalanceAt", mock.Anything, mock.Anything, mock.Anything).Return(big.NewInt(42), nil) ethClient.On("LINKBalance", mock.Anything, mock.Anything, mock.Anything).Return(assets.NewLinkFromJuels(42), nil) + ethClient.On("PendingNonceAt", mock.Anything, mock.Anything).Return(uint64(0), nil) app := startNewApplicationV2(t, func(c *chainlink.Config, s *chainlink.Secrets) { c.EVM[0].Enabled = ptr(true) c.EVM[0].NonceAutoSync = ptr(false) @@ -343,6 +348,7 @@ func TestShell_ImportExportETHKey_WithChains(t *testing.T) { t.Cleanup(func() { deleteKeyExportFile(t) }) ethClient := newEthMock(t) + ethClient.On("PendingNonceAt", mock.Anything, mock.Anything).Return(uint64(0), nil) app := startNewApplicationV2(t, func(c *chainlink.Config, s *chainlink.Secrets) { c.EVM[0].Enabled = ptr(true) c.EVM[0].NonceAutoSync = ptr(false) diff --git a/core/cmd/evm_transaction_commands_test.go b/core/cmd/evm_transaction_commands_test.go index 187b3a6aece..484b1ccd3da 100644 --- a/core/cmd/evm_transaction_commands_test.go +++ b/core/cmd/evm_transaction_commands_test.go @@ -140,6 +140,7 @@ func TestShell_SendEther_From_Txm(t *testing.T) { ethMock.On("BalanceAt", mock.Anything, key.Address, (*big.Int)(nil)).Return(balance.ToInt(), nil) ethMock.On("SequenceAt", mock.Anything, mock.Anything, mock.Anything).Return(evmtypes.Nonce(0), nil).Maybe() + ethMock.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(0), nil).Once() app := startNewApplicationV2(t, func(c *chainlink.Config, s *chainlink.Secrets) { c.EVM[0].Enabled = ptr(true) @@ -199,6 +200,7 @@ func TestShell_SendEther_From_Txm_WEI(t *testing.T) { ethMock.On("BalanceAt", mock.Anything, key.Address, (*big.Int)(nil)).Return(balance.ToInt(), nil) ethMock.On("SequenceAt", mock.Anything, mock.Anything, mock.Anything).Return(evmtypes.Nonce(0), nil).Maybe() + ethMock.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(0), nil).Once() app := startNewApplicationV2(t, func(c *chainlink.Config, s *chainlink.Secrets) { c.EVM[0].Enabled = ptr(true) diff --git a/core/internal/cltest/factories.go b/core/internal/cltest/factories.go index ca010447ce9..a1a5d9db6be 100644 --- a/core/internal/cltest/factories.go +++ b/core/internal/cltest/factories.go @@ -490,8 +490,6 @@ func (r RandomKey) MustInsert(t testing.TB, keystore keystore.Eth) (ethkey.KeyV2 for _, cid := range r.chainIDs { require.NoError(t, keystore.Add(key.Address, cid.ToInt())) require.NoError(t, keystore.Enable(key.Address, cid.ToInt())) - err := keystore.Reset(key.Address, cid.ToInt(), r.Nonce) - require.NoError(t, err) if r.Disabled { require.NoError(t, keystore.Disable(key.Address, cid.ToInt())) } diff --git a/core/scripts/common/vrf/setup-envs/main.go b/core/scripts/common/vrf/setup-envs/main.go index 8a7b1c8b439..6748408f476 100644 --- a/core/scripts/common/vrf/setup-envs/main.go +++ b/core/scripts/common/vrf/setup-envs/main.go @@ -5,6 +5,11 @@ import ( "encoding/json" "flag" "fmt" + "io" + "math/big" + "os" + "strings" + "github.com/ethereum/go-ethereum/common" "github.com/shopspring/decimal" helpers "github.com/smartcontractkit/chainlink/core/scripts/common" @@ -17,10 +22,6 @@ import ( "github.com/smartcontractkit/chainlink/v2/core/gethwrappers/generated/vrf_coordinator_v2_5" "github.com/smartcontractkit/chainlink/v2/core/web/presenters" "github.com/urfave/cli" - "io" - "math/big" - "os" - "strings" ) func newApp(remoteNodeURL string, writer io.Writer) (*clcmd.Shell, *cli.App) { @@ -377,7 +378,6 @@ func printETHKeyData(ethKeys []presenters.ETHKeyResource) { fmt.Println("Address: ", ethKey.Address) fmt.Println("MaxGasPriceWei: ", ethKey.MaxGasPriceWei) fmt.Println("EthBalance: ", ethKey.EthBalance) - fmt.Println("NextNonce: ", ethKey.NextNonce) fmt.Println("-----------------------------") } } diff --git a/core/services/keystore/eth.go b/core/services/keystore/eth.go index 362baed6afc..0f9083f32a2 100644 --- a/core/services/keystore/eth.go +++ b/core/services/keystore/eth.go @@ -12,7 +12,6 @@ import ( "github.com/ethereum/go-ethereum/core/types" "github.com/pkg/errors" - evmtypes "github.com/smartcontractkit/chainlink/v2/core/chains/evm/types" "github.com/smartcontractkit/chainlink/v2/core/services/keystore/keys/ethkey" "github.com/smartcontractkit/chainlink/v2/core/services/pg" "github.com/smartcontractkit/chainlink/v2/core/utils" @@ -32,10 +31,6 @@ type Eth interface { Enable(address common.Address, chainID *big.Int, qopts ...pg.QOpt) error Disable(address common.Address, chainID *big.Int, qopts ...pg.QOpt) error Add(address common.Address, chainID *big.Int, qopts ...pg.QOpt) error - Reset(address common.Address, chainID *big.Int, nonce int64, qopts ...pg.QOpt) error - - NextSequence(address common.Address, chainID *big.Int, qopts ...pg.QOpt) (evmtypes.Nonce, error) - IncrementNextSequence(address common.Address, chainID *big.Int, currentNonce evmtypes.Nonce, qopts ...pg.QOpt) error EnsureKeys(chainIDs ...*big.Int) error SubscribeToKeyChanges() (ch chan struct{}, unsub func()) @@ -182,51 +177,6 @@ func (ks *eth) Export(id string, password string) ([]byte, error) { return key.ToEncryptedJSON(password, ks.scryptParams) } -// Get the next nonce for the given key and chain. It is safest to always to go the DB for this -func (ks *eth) NextSequence(address common.Address, chainID *big.Int, qopts ...pg.QOpt) (nonce evmtypes.Nonce, err error) { - if !ks.exists(address) { - return evmtypes.Nonce(0), errors.Errorf("key with address %s does not exist", address.String()) - } - nonceVal, err := ks.orm.getNextNonce(address, chainID, qopts...) - if err != nil { - return evmtypes.Nonce(0), errors.Wrap(err, "NextSequence failed") - } - ks.lock.Lock() - defer ks.lock.Unlock() - state, exists := ks.keyStates.KeyIDChainID[address.String()][chainID.String()] - if !exists { - return evmtypes.Nonce(0), errors.Errorf("state not found for address %s, chainID %s", address, chainID.String()) - } - if state.Disabled { - return evmtypes.Nonce(0), errors.Errorf("state is disabled for address %s, chainID %s", address, chainID.String()) - } - // Always clobber the memory nonce with the DB nonce - state.NextNonce = nonceVal - return evmtypes.Nonce(nonceVal), nil -} - -// IncrementNextNonce increments keys.next_nonce by 1 -func (ks *eth) IncrementNextSequence(address common.Address, chainID *big.Int, currentSequence evmtypes.Nonce, qopts ...pg.QOpt) error { - if !ks.exists(address) { - return errors.Errorf("key with address %s does not exist", address.String()) - } - incrementedNonce, err := ks.orm.incrementNextNonce(address, chainID, currentSequence.Int64(), qopts...) - if err != nil { - return errors.Wrap(err, "failed IncrementNextNonce") - } - ks.lock.Lock() - defer ks.lock.Unlock() - state, exists := ks.keyStates.KeyIDChainID[address.String()][chainID.String()] - if !exists { - return errors.Errorf("state not found for address %s, chainID %s", address, chainID.String()) - } - if state.Disabled { - return errors.Errorf("state is disabled for address %s, chainID %s", address, chainID.String()) - } - state.NextNonce = incrementedNonce - return nil -} - func (ks *eth) Add(address common.Address, chainID *big.Int, qopts ...pg.QOpt) error { ks.lock.Lock() defer ks.lock.Unlock() @@ -240,8 +190,8 @@ func (ks *eth) Add(address common.Address, chainID *big.Int, qopts ...pg.QOpt) e // caller must hold lock! func (ks *eth) addKey(address common.Address, chainID *big.Int, qopts ...pg.QOpt) error { state := new(ethkey.State) - sql := `INSERT INTO evm.key_states (address, next_nonce, disabled, evm_chain_id, created_at, updated_at) - VALUES ($1, 0, false, $2, NOW(), NOW()) + sql := `INSERT INTO evm.key_states (address, disabled, evm_chain_id, created_at, updated_at) + VALUES ($1, false, $2, NOW(), NOW()) RETURNING *;` q := ks.orm.q.WithOpts(qopts...) if err := q.Get(state, sql, address, chainID.String()); err != nil { @@ -292,7 +242,7 @@ func (ks *eth) disable(address common.Address, chainID *big.Int, qopts ...pg.QOp state := new(ethkey.State) q := ks.orm.q.WithOpts(qopts...) sql := `UPDATE evm.key_states SET disabled = false, updated_at = NOW() WHERE address = $1 AND evm_chain_id = $2 - RETURNING id, next_nonce, address, evm_chain_id, disabled, created_at, updated_at;` + RETURNING id, address, evm_chain_id, disabled, created_at, updated_at;` if err := q.Get(state, sql, address, chainID.String()); err != nil { return errors.Wrap(err, "failed to enable state") } @@ -302,30 +252,6 @@ func (ks *eth) disable(address common.Address, chainID *big.Int, qopts ...pg.QOp return nil } -// Reset the key/chain nonce to the given one -func (ks *eth) Reset(address common.Address, chainID *big.Int, nonce int64, qopts ...pg.QOpt) error { - q := ks.orm.q.WithOpts(qopts...) - res, err := q.Exec(`UPDATE evm.key_states SET next_nonce = $1 WHERE address = $2 AND evm_chain_id = $3`, nonce, address, chainID.String()) - if err != nil { - return errors.Wrap(err, "failed to reset state") - } - rowsAffected, err := res.RowsAffected() - if err != nil { - return errors.Wrap(err, "failed to get RowsAffected") - } - if rowsAffected == 0 { - return errors.Errorf("key state not found with address %s and chainID %s", address.Hex(), chainID.String()) - } - ks.lock.Lock() - defer ks.lock.Unlock() - state, exists := ks.keyStates.KeyIDChainID[address.Hex()][chainID.String()] - if !exists { - return errors.Errorf("state not found for address %s, chainID %s", address.Hex(), chainID.String()) - } - state.NextNonce = nonce - return nil -} - func (ks *eth) Delete(id string) (ethkey.KeyV2, error) { ks.lock.Lock() defer ks.lock.Unlock() @@ -554,25 +480,24 @@ func (ks *eth) EnabledAddressesForChain(chainID *big.Int) (addresses []common.Ad return } -func (ks *eth) getV1KeysAsV2() (keys []ethkey.KeyV2, nonces []int64, fundings []bool, _ error) { +func (ks *eth) getV1KeysAsV2() (keys []ethkey.KeyV2, fundings []bool, _ error) { v1Keys, err := ks.orm.GetEncryptedV1EthKeys() if err != nil { - return nil, nil, nil, errors.Wrap(err, "failed to get encrypted v1 eth keys") + return nil, nil, errors.Wrap(err, "failed to get encrypted v1 eth keys") } if len(v1Keys) == 0 { - return nil, nil, nil, nil + return nil, nil, nil } for _, keyV1 := range v1Keys { dKey, err := keystore.DecryptKey(keyV1.JSON, ks.password) if err != nil { - return nil, nil, nil, errors.Wrapf(err, "could not decrypt eth key %s", keyV1.Address.Hex()) + return nil, nil, errors.Wrapf(err, "could not decrypt eth key %s", keyV1.Address.Hex()) } keyV2 := ethkey.FromPrivateKey(dKey.PrivateKey) keys = append(keys, keyV2) - nonces = append(nonces, keyV1.NextNonce) fundings = append(fundings, keyV1.IsFunding) } - return keys, nonces, fundings, nil + return keys, fundings, nil } // XXXTestingOnlySetState is only used in tests to manually update a key's state @@ -587,7 +512,7 @@ func (ks *eth) XXXTestingOnlySetState(state ethkey.State) { panic(fmt.Sprintf("key not found with ID %s", state.KeyID())) } *existingState = state - sql := `UPDATE evm.key_states SET address = :address, next_nonce = :next_nonce, is_disabled = :is_disabled, evm_chain_id = :evm_chain_id, updated_at = NOW() + sql := `UPDATE evm.key_states SET address = :address, is_disabled = :is_disabled, evm_chain_id = :evm_chain_id, updated_at = NOW() WHERE address = :address;` _, err := ks.orm.q.NamedExec(sql, state) if err != nil { diff --git a/core/services/keystore/eth_internal_test.go b/core/services/keystore/eth_internal_test.go index da1585bed44..5fe6e6e7698 100644 --- a/core/services/keystore/eth_internal_test.go +++ b/core/services/keystore/eth_internal_test.go @@ -22,15 +22,14 @@ func Test_EthKeyStore(t *testing.T) { t.Run("returns V1 keys as V2", func(t *testing.T) { ethAddress := testutils.NewAddress() - err = utils.JustError(db.Exec(`INSERT INTO keys (address, json, created_at, updated_at, next_nonce, is_funding, deleted_at) VALUES ($1, '{"address":"6fdac88ddfd811d130095373986889ed90e0d622","crypto":{"cipher":"aes-128-ctr","ciphertext":"557f5324e770c3d203751c1f0f7fb5076386c49f5b05e3f20b3abb59758fd3c3","cipherparams":{"iv":"bd9472543fab7cc63027cbcd039daff0"},"kdf":"scrypt","kdfparams":{"dklen":32,"n":2,"p":1,"r":8,"salt":"647b54770a3fda830b4440ae57c44cf7506297295fe4d72b1ff943e3a8ddb94a"},"mac":"0c654ee29ee06b3816fc0040d84ebd648c557144a77ccc55b9568355f53397b3"},"id":"6fdac88d-dfd8-11d1-3009-5373986889ed","version":3}', NOW(), NOW(), 0, false, NULL)`, ethAddress)) + err = utils.JustError(db.Exec(`INSERT INTO keys (address, json, created_at, updated_at, is_funding, deleted_at) VALUES ($1, '{"address":"6fdac88ddfd811d130095373986889ed90e0d622","crypto":{"cipher":"aes-128-ctr","ciphertext":"557f5324e770c3d203751c1f0f7fb5076386c49f5b05e3f20b3abb59758fd3c3","cipherparams":{"iv":"bd9472543fab7cc63027cbcd039daff0"},"kdf":"scrypt","kdfparams":{"dklen":32,"n":2,"p":1,"r":8,"salt":"647b54770a3fda830b4440ae57c44cf7506297295fe4d72b1ff943e3a8ddb94a"},"mac":"0c654ee29ee06b3816fc0040d84ebd648c557144a77ccc55b9568355f53397b3"},"id":"6fdac88d-dfd8-11d1-3009-5373986889ed","version":3}', NOW(), NOW(), false, NULL)`, ethAddress)) require.NoError(t, err) - keys, nonces, fundings, err := ks.(*eth).getV1KeysAsV2() + keys, fundings, err := ks.(*eth).getV1KeysAsV2() require.NoError(t, err) assert.Len(t, keys, 1) assert.Equal(t, fmt.Sprintf("EthKeyV2{PrivateKey: , Address: %s}", keys[0].Address), keys[0].GoString()) - assert.Equal(t, int64(0), nonces[0]) assert.Equal(t, false, fundings[0]) }) } diff --git a/core/services/keystore/eth_test.go b/core/services/keystore/eth_test.go index 1f2dc27b46d..0fec509a325 100644 --- a/core/services/keystore/eth_test.go +++ b/core/services/keystore/eth_test.go @@ -1,7 +1,6 @@ package keystore_test import ( - "database/sql" "fmt" "math/big" "sort" @@ -15,7 +14,6 @@ import ( "github.com/stretchr/testify/require" evmclient "github.com/smartcontractkit/chainlink/v2/core/chains/evm/client" - evmtypes "github.com/smartcontractkit/chainlink/v2/core/chains/evm/types" "github.com/smartcontractkit/chainlink/v2/core/internal/cltest" "github.com/smartcontractkit/chainlink/v2/core/internal/testutils" configtest "github.com/smartcontractkit/chainlink/v2/core/internal/testutils/configtest/v2" @@ -615,140 +613,6 @@ func Test_EthKeyStore_EnsureKeys(t *testing.T) { }) } -func Test_EthKeyStore_Reset(t *testing.T) { - t.Parallel() - - db := pgtest.NewSqlxDB(t) - cfg := configtest.NewTestGeneralConfig(t) - keyStore := cltest.NewKeyStore(t, db, cfg.Database()) - ks := keyStore.Eth() - - k1, addr1 := cltest.MustInsertRandomKey(t, ks) - cltest.MustInsertRandomKey(t, ks) - cltest.MustInsertRandomKey(t, ks, *utils.NewBig(testutils.SimulatedChainID)) - - newNonce := testutils.NewRandomPositiveInt64() - - t.Run("when no state matches address/chain ID", func(t *testing.T) { - addr := utils.RandomAddress() - cid := testutils.NewRandomEVMChainID() - err := ks.Reset(addr, cid, newNonce) - require.Error(t, err) - assert.Contains(t, err.Error(), fmt.Sprintf("key state not found with address %s and chainID %s", addr.Hex(), cid.String())) - }) - t.Run("when no state matches address", func(t *testing.T) { - addr := utils.RandomAddress() - err := ks.Reset(addr, testutils.FixtureChainID, newNonce) - require.Error(t, err) - assert.Contains(t, err.Error(), fmt.Sprintf("key state not found with address %s and chainID 0", addr.Hex())) - }) - t.Run("when no state matches chain ID", func(t *testing.T) { - cid := testutils.NewRandomEVMChainID() - err := ks.Reset(addr1, cid, newNonce) - require.Error(t, err) - assert.Contains(t, err.Error(), fmt.Sprintf("key state not found with address %s and chainID %s", addr1.Hex(), cid.String())) - }) - t.Run("resets key with given address and chain ID to the given nonce", func(t *testing.T) { - err := ks.Reset(k1.Address, testutils.FixtureChainID, newNonce) - assert.NoError(t, err) - - nonce, err := ks.NextSequence(k1.Address, testutils.FixtureChainID) - require.NoError(t, err) - - assert.Equal(t, nonce.Int64(), newNonce) - - state, err := ks.GetState(k1.Address.Hex(), testutils.FixtureChainID) - require.NoError(t, err) - assert.Equal(t, nonce.Int64(), state.NextNonce) - - keys, err := ks.GetAll() - require.NoError(t, err) - require.Len(t, keys, 3) - states, err := ks.GetStatesForKeys(keys) - require.NoError(t, err) - require.Len(t, states, 3) - for _, state = range states { - if state.Address.Address() == k1.Address { - assert.Equal(t, nonce.Int64(), state.NextNonce) - } else { - // the other states didn't get updated - assert.Equal(t, int64(0), state.NextNonce) - } - } - }) -} - -func Test_NextSequence(t *testing.T) { - t.Parallel() - - db := pgtest.NewSqlxDB(t) - cfg := configtest.NewTestGeneralConfig(t) - keyStore := cltest.NewKeyStore(t, db, cfg.Database()) - ks := keyStore.Eth() - randNonce := testutils.NewRandomPositiveInt64() - - _, addr1 := cltest.RandomKey{Nonce: randNonce}.MustInsert(t, ks) - cltest.MustInsertRandomKey(t, ks) - - nonce, err := ks.NextSequence(addr1, testutils.FixtureChainID) - require.NoError(t, err) - assert.Equal(t, randNonce, nonce.Int64()) - - _, err = ks.NextSequence(addr1, testutils.SimulatedChainID) - require.Error(t, err) - assert.Contains(t, err.Error(), fmt.Sprintf("NextSequence failed: key with address %s is not enabled for chain %s: sql: no rows in result set", addr1.Hex(), testutils.SimulatedChainID.String())) - - randAddr1 := utils.RandomAddress() - _, err = ks.NextSequence(randAddr1, testutils.FixtureChainID) - require.Error(t, err) - assert.Contains(t, err.Error(), fmt.Sprintf("key with address %s does not exist", randAddr1.Hex())) - - randAddr2 := utils.RandomAddress() - _, err = ks.NextSequence(randAddr2, testutils.NewRandomEVMChainID()) - require.Error(t, err) - assert.Contains(t, err.Error(), fmt.Sprintf("key with address %s does not exist", randAddr2.Hex())) -} - -func Test_IncrementNextSequence(t *testing.T) { - t.Parallel() - - db := pgtest.NewSqlxDB(t) - cfg := configtest.NewTestGeneralConfig(t) - keyStore := cltest.NewKeyStore(t, db, cfg.Database()) - ks := keyStore.Eth() - randNonce := testutils.NewRandomPositiveInt64() - - _, addr1 := cltest.RandomKey{Nonce: randNonce}.MustInsert(t, ks) - evmAddr1 := addr1 - cltest.MustInsertRandomKey(t, ks) - - err := ks.IncrementNextSequence(evmAddr1, testutils.FixtureChainID, evmtypes.Nonce(randNonce-1)) - assert.ErrorIs(t, err, sql.ErrNoRows) - - err = ks.IncrementNextSequence(evmAddr1, testutils.FixtureChainID, evmtypes.Nonce(randNonce)) - require.NoError(t, err) - var nonce int64 - require.NoError(t, db.Get(&nonce, `SELECT next_nonce FROM evm.key_states WHERE address = $1 AND evm_chain_id = $2`, addr1, testutils.FixtureChainID.String())) - assert.Equal(t, randNonce+1, nonce) - - err = ks.IncrementNextSequence(evmAddr1, testutils.SimulatedChainID, evmtypes.Nonce(randNonce+1)) - assert.ErrorIs(t, err, sql.ErrNoRows) - - randAddr1 := utils.RandomAddress() - err = ks.IncrementNextSequence(randAddr1, testutils.FixtureChainID, evmtypes.Nonce(randNonce+1)) - require.Error(t, err) - assert.Contains(t, err.Error(), fmt.Sprintf("key with address %s does not exist", randAddr1.Hex())) - - randAddr2 := utils.RandomAddress() - err = ks.IncrementNextSequence(randAddr2, testutils.NewRandomEVMChainID(), evmtypes.Nonce(randNonce+1)) - require.Error(t, err) - assert.Contains(t, err.Error(), fmt.Sprintf("key with address %s does not exist", randAddr2.Hex())) - - // verify it didnt get changed by any erroring calls - require.NoError(t, db.Get(&nonce, `SELECT next_nonce FROM evm.key_states WHERE address = $1 AND evm_chain_id = $2`, addr1, testutils.FixtureChainID.String())) - assert.Equal(t, randNonce+1, nonce) -} - func Test_EthKeyStore_Delete(t *testing.T) { t.Parallel() diff --git a/core/services/keystore/keys/ethkey/key.go b/core/services/keystore/keys/ethkey/key.go index 2ca671b2cda..4201251e34f 100644 --- a/core/services/keystore/keys/ethkey/key.go +++ b/core/services/keystore/keys/ethkey/key.go @@ -19,13 +19,6 @@ type Key struct { CreatedAt time.Time `json:"-"` UpdatedAt time.Time `json:"-"` DeletedAt *time.Time `json:"-"` - // This is the nonce that should be used for the next transaction. - // Conceptually equivalent to geth's `PendingNonceAt` but more reliable - // because we have a better view of our own transactions - // NOTE: Be cautious about using this field, it is provided for convenience - // only, can go out of date, and should not be relied upon. The source of - // truth is always the database row for the key. - NextNonce int64 `json:"-"` // IsFunding marks the address as being used for rescuing the node and the pending transactions // Only one key can be IsFunding=true at a time. IsFunding bool diff --git a/core/services/keystore/keys/ethkey/models.go b/core/services/keystore/keys/ethkey/models.go index 1afab408dbe..b90503c3ed6 100644 --- a/core/services/keystore/keys/ethkey/models.go +++ b/core/services/keystore/keys/ethkey/models.go @@ -10,13 +10,10 @@ type State struct { ID int32 Address EIP55Address EVMChainID utils.Big - // NextNonce is used for convenience and rendering in UI but the source of - // truth is always the DB - NextNonce int64 - Disabled bool - CreatedAt time.Time - UpdatedAt time.Time - lastUsed time.Time + Disabled bool + CreatedAt time.Time + UpdatedAt time.Time + lastUsed time.Time } func (s State) KeyID() string { diff --git a/core/services/keystore/mocks/eth.go b/core/services/keystore/mocks/eth.go index bb539d8b7ac..88a5356ac1f 100644 --- a/core/services/keystore/mocks/eth.go +++ b/core/services/keystore/mocks/eth.go @@ -6,15 +6,13 @@ import ( big "math/big" common "github.com/ethereum/go-ethereum/common" - coretypes "github.com/ethereum/go-ethereum/core/types" - ethkey "github.com/smartcontractkit/chainlink/v2/core/services/keystore/keys/ethkey" mock "github.com/stretchr/testify/mock" pg "github.com/smartcontractkit/chainlink/v2/core/services/pg" - types "github.com/smartcontractkit/chainlink/v2/core/chains/evm/types" + types "github.com/ethereum/go-ethereum/core/types" ) // Eth is an autogenerated mock type for the Eth type @@ -465,97 +463,24 @@ func (_m *Eth) Import(keyJSON []byte, password string, chainIDs ...*big.Int) (et return r0, r1 } -// IncrementNextSequence provides a mock function with given fields: address, chainID, currentNonce, qopts -func (_m *Eth) IncrementNextSequence(address common.Address, chainID *big.Int, currentNonce types.Nonce, qopts ...pg.QOpt) error { - _va := make([]interface{}, len(qopts)) - for _i := range qopts { - _va[_i] = qopts[_i] - } - var _ca []interface{} - _ca = append(_ca, address, chainID, currentNonce) - _ca = append(_ca, _va...) - ret := _m.Called(_ca...) - - var r0 error - if rf, ok := ret.Get(0).(func(common.Address, *big.Int, types.Nonce, ...pg.QOpt) error); ok { - r0 = rf(address, chainID, currentNonce, qopts...) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -// NextSequence provides a mock function with given fields: address, chainID, qopts -func (_m *Eth) NextSequence(address common.Address, chainID *big.Int, qopts ...pg.QOpt) (types.Nonce, error) { - _va := make([]interface{}, len(qopts)) - for _i := range qopts { - _va[_i] = qopts[_i] - } - var _ca []interface{} - _ca = append(_ca, address, chainID) - _ca = append(_ca, _va...) - ret := _m.Called(_ca...) - - var r0 types.Nonce - var r1 error - if rf, ok := ret.Get(0).(func(common.Address, *big.Int, ...pg.QOpt) (types.Nonce, error)); ok { - return rf(address, chainID, qopts...) - } - if rf, ok := ret.Get(0).(func(common.Address, *big.Int, ...pg.QOpt) types.Nonce); ok { - r0 = rf(address, chainID, qopts...) - } else { - r0 = ret.Get(0).(types.Nonce) - } - - if rf, ok := ret.Get(1).(func(common.Address, *big.Int, ...pg.QOpt) error); ok { - r1 = rf(address, chainID, qopts...) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - -// Reset provides a mock function with given fields: address, chainID, nonce, qopts -func (_m *Eth) Reset(address common.Address, chainID *big.Int, nonce int64, qopts ...pg.QOpt) error { - _va := make([]interface{}, len(qopts)) - for _i := range qopts { - _va[_i] = qopts[_i] - } - var _ca []interface{} - _ca = append(_ca, address, chainID, nonce) - _ca = append(_ca, _va...) - ret := _m.Called(_ca...) - - var r0 error - if rf, ok := ret.Get(0).(func(common.Address, *big.Int, int64, ...pg.QOpt) error); ok { - r0 = rf(address, chainID, nonce, qopts...) - } else { - r0 = ret.Error(0) - } - - return r0 -} - // SignTx provides a mock function with given fields: fromAddress, tx, chainID -func (_m *Eth) SignTx(fromAddress common.Address, tx *coretypes.Transaction, chainID *big.Int) (*coretypes.Transaction, error) { +func (_m *Eth) SignTx(fromAddress common.Address, tx *types.Transaction, chainID *big.Int) (*types.Transaction, error) { ret := _m.Called(fromAddress, tx, chainID) - var r0 *coretypes.Transaction + var r0 *types.Transaction var r1 error - if rf, ok := ret.Get(0).(func(common.Address, *coretypes.Transaction, *big.Int) (*coretypes.Transaction, error)); ok { + if rf, ok := ret.Get(0).(func(common.Address, *types.Transaction, *big.Int) (*types.Transaction, error)); ok { return rf(fromAddress, tx, chainID) } - if rf, ok := ret.Get(0).(func(common.Address, *coretypes.Transaction, *big.Int) *coretypes.Transaction); ok { + if rf, ok := ret.Get(0).(func(common.Address, *types.Transaction, *big.Int) *types.Transaction); ok { r0 = rf(fromAddress, tx, chainID) } else { if ret.Get(0) != nil { - r0 = ret.Get(0).(*coretypes.Transaction) + r0 = ret.Get(0).(*types.Transaction) } } - if rf, ok := ret.Get(1).(func(common.Address, *coretypes.Transaction, *big.Int) error); ok { + if rf, ok := ret.Get(1).(func(common.Address, *types.Transaction, *big.Int) error); ok { r1 = rf(fromAddress, tx, chainID) } else { r1 = ret.Error(1) diff --git a/core/services/keystore/orm.go b/core/services/keystore/orm.go index 1396b544205..cb80ae7ef74 100644 --- a/core/services/keystore/orm.go +++ b/core/services/keystore/orm.go @@ -2,7 +2,6 @@ package keystore import ( "database/sql" - "math/big" "github.com/smartcontractkit/chainlink/v2/core/logger" "github.com/smartcontractkit/chainlink/v2/core/services/keystore/keys/csakey" @@ -12,7 +11,6 @@ import ( "github.com/smartcontractkit/chainlink/v2/core/services/keystore/keys/vrfkey" "github.com/smartcontractkit/chainlink/v2/core/services/pg" - "github.com/ethereum/go-ethereum/common" "github.com/pkg/errors" "github.com/smartcontractkit/sqlx" ) @@ -67,7 +65,7 @@ func (orm ksORM) getEncryptedKeyRing() (kr encryptedKeyRing, err error) { func (orm ksORM) loadKeyStates() (*keyStates, error) { ks := newKeyStates() var ethkeystates []*ethkey.State - if err := orm.q.Select(ðkeystates, `SELECT id, address, evm_chain_id, next_nonce, disabled, created_at, updated_at FROM evm.key_states`); err != nil { + if err := orm.q.Select(ðkeystates, `SELECT id, address, evm_chain_id, disabled, created_at, updated_at FROM evm.key_states`); err != nil { return ks, errors.Wrap(err, "error loading evm.key_states from DB") } for _, state := range ethkeystates { @@ -76,23 +74,6 @@ func (orm ksORM) loadKeyStates() (*keyStates, error) { return ks, nil } -// getNextNonce returns evm.key_states.next_nonce for the given address -func (orm ksORM) getNextNonce(address common.Address, chainID *big.Int, qopts ...pg.QOpt) (nonce int64, err error) { - q := orm.q.WithOpts(qopts...) - err = q.Get(&nonce, "SELECT next_nonce FROM evm.key_states WHERE address = $1 AND evm_chain_id = $2 AND disabled = false", address, chainID.String()) - if errors.Is(err, sql.ErrNoRows) { - return 0, errors.Wrapf(sql.ErrNoRows, "key with address %s is not enabled for chain %s", address.Hex(), chainID.String()) - } - return nonce, errors.Wrap(err, "failed to load next nonce") -} - -// incrementNextNonce increments evm.key_states.next_nonce by 1 -func (orm ksORM) incrementNextNonce(address common.Address, chainID *big.Int, currentNonce int64, qopts ...pg.QOpt) (incrementedNonce int64, err error) { - q := orm.q.WithOpts(qopts...) - err = q.Get(&incrementedNonce, "UPDATE evm.key_states SET next_nonce = next_nonce + 1, updated_at = NOW() WHERE address = $1 AND next_nonce = $2 AND evm_chain_id = $3 AND disabled = false RETURNING next_nonce", address, currentNonce, chainID.String()) - return incrementedNonce, errors.Wrap(err, "IncrementNextNonce failed to update keys") -} - // ~~~~~~~~~~~~~~~~~~~~ LEGACY FUNCTIONS FOR V1 MIGRATION ~~~~~~~~~~~~~~~~~~~~ func (orm ksORM) GetEncryptedV1CSAKeys() (retrieved []csakey.Key, err error) { diff --git a/core/services/keystore/orm_test.go b/core/services/keystore/orm_test.go index 3264b3b0178..2cfc7b9e26a 100644 --- a/core/services/keystore/orm_test.go +++ b/core/services/keystore/orm_test.go @@ -40,8 +40,8 @@ func Test_ORM(t *testing.T) { utils.JustError(db.Exec(`INSERT INTO csa_keys (public_key, encrypted_private_key, created_at, updated_at) VALUES ($1, '{}', NOW(), NOW())`, csa.PublicKey)), // two per key-type, one deleted and one not deleted - utils.JustError(db.Exec(`INSERT INTO keys (address, json, created_at, updated_at, next_nonce, is_funding, deleted_at) VALUES ($1, '{}', NOW(), NOW(), 0, false, NULL)`, eth1)), - utils.JustError(db.Exec(`INSERT INTO keys (address, json, created_at, updated_at, next_nonce, is_funding, deleted_at) VALUES ($1, '{}', NOW(), NOW(), 0, false, NOW())`, eth2)), + utils.JustError(db.Exec(`INSERT INTO keys (address, json, created_at, updated_at, is_funding, deleted_at) VALUES ($1, '{}', NOW(), NOW(), false, NULL)`, eth1)), + utils.JustError(db.Exec(`INSERT INTO keys (address, json, created_at, updated_at, is_funding, deleted_at) VALUES ($1, '{}', NOW(), NOW(), false, NOW())`, eth2)), utils.JustError(db.Exec(`INSERT INTO encrypted_ocr_key_bundles (id, on_chain_signing_address, off_chain_public_key, encrypted_private_keys, created_at, updated_at, config_public_key, deleted_at) VALUES ($1, $2, $3, '{}', NOW(), NOW(), $4, NULL)`, ocr1, testutils.NewAddress(), utils.NewHash(), utils.NewHash())), utils.JustError(db.Exec(`INSERT INTO encrypted_ocr_key_bundles (id, on_chain_signing_address, off_chain_public_key, encrypted_private_keys, created_at, updated_at, config_public_key, deleted_at) VALUES ($1, $2, $3, '{}', NOW(), NOW(), $4, NOW())`, ocr2, testutils.NewAddress(), utils.NewHash(), utils.NewHash())), utils.JustError(db.Exec(`INSERT INTO encrypted_p2p_keys (peer_id, pub_key, encrypted_priv_key, created_at, updated_at, deleted_at) VALUES ($1, $2, '{}', NOW(), NOW(), NULL)`, p1.Pretty(), utils.NewHash())), diff --git a/core/store/migrate/migrations/0199_remove_next_nonce_from_keystore.sql b/core/store/migrate/migrations/0199_remove_next_nonce_from_keystore.sql new file mode 100644 index 00000000000..07cdfb02452 --- /dev/null +++ b/core/store/migrate/migrations/0199_remove_next_nonce_from_keystore.sql @@ -0,0 +1,9 @@ +-- +goose Up + +ALTER TABLE evm.key_states DROP COLUMN next_nonce; +ALTER TABLE keys DROP COLUMN next_nonce; + +-- +goose Down + +ALTER TABLE evm.key_states ADD next_nonce bigint NOT NULL DEFAULT 0; +ALTER TABLE keys ADD next_nonce bigint NOT NULL DEFAULT 0; \ No newline at end of file diff --git a/core/web/eth_keys_controller.go b/core/web/eth_keys_controller.go index f708f8634ec..28afe8c43bf 100644 --- a/core/web/eth_keys_controller.go +++ b/core/web/eth_keys_controller.go @@ -277,14 +277,6 @@ func (ekc *ETHKeysController) Chain(c *gin.Context) { return } - var nonce int64 = -1 - if nonceStr := c.Query("nextNonce"); nonceStr != "" { - nonce, err = strconv.ParseInt(nonceStr, 10, 64) - if err != nil || nonce < 0 { - jsonAPIError(c, http.StatusBadRequest, errors.Wrapf(err, "invalid value for nonce: expected 0 or positive int, got: %s", nonceStr)) - return - } - } abandon := false if abandonStr := c.Query("abandon"); abandonStr != "" { abandon, err = strconv.ParseBool(abandonStr) @@ -295,13 +287,9 @@ func (ekc *ETHKeysController) Chain(c *gin.Context) { } // Reset the chain - if abandon || nonce >= 0 { + if abandon { var resetErr error - err = chain.TxManager().Reset(func() { - if nonce >= 0 { - resetErr = kst.Reset(address, chain.ID(), nonce) - } - }, address, abandon) + err = chain.TxManager().Reset(address, abandon) err = multierr.Combine(err, resetErr) if err != nil { if strings.Contains(err.Error(), "key state not found with address") { diff --git a/core/web/eth_keys_controller_test.go b/core/web/eth_keys_controller_test.go index 8c21d168f31..98e7b3807de 100644 --- a/core/web/eth_keys_controller_test.go +++ b/core/web/eth_keys_controller_test.go @@ -4,7 +4,6 @@ import ( "math/big" "net/http" "net/url" - "strconv" "testing" "github.com/pkg/errors" @@ -32,6 +31,7 @@ func TestETHKeysController_Index_Success(t *testing.T) { t.Parallel() ethClient := cltest.NewEthMocksWithStartupAssertions(t) + ethClient.On("PendingNonceAt", mock.Anything, mock.Anything).Return(uint64(0), nil) cfg := configtest.NewGeneralConfig(t, func(c *chainlink.Config, s *chainlink.Secrets) { c.EVM[0].NonceAutoSync = ptr(false) c.EVM[0].BalanceMonitor.Enabled = ptr(false) @@ -84,11 +84,11 @@ func TestETHKeysController_Index_Errors(t *testing.T) { t.Parallel() ethClient := cltest.NewEthMocksWithStartupAssertions(t) + ethClient.On("PendingNonceAt", mock.Anything, mock.Anything).Return(uint64(0), nil) cfg := configtest.NewGeneralConfig(t, func(c *chainlink.Config, s *chainlink.Secrets) { c.EVM[0].NonceAutoSync = ptr(false) c.EVM[0].BalanceMonitor.Enabled = ptr(false) }) - app := cltest.NewApplicationWithConfig(t, cfg, ethClient) require.NoError(t, app.KeyStore.Unlock(cltest.Password)) @@ -156,7 +156,7 @@ func TestETHKeysController_Index_NotDev(t *testing.T) { t.Parallel() ethClient := cltest.NewEthMocksWithStartupAssertions(t) - + ethClient.On("PendingNonceAt", mock.Anything, mock.Anything).Return(uint64(0), nil) cfg := configtest.NewGeneralConfig(t, func(c *chainlink.Config, s *chainlink.Secrets) { c.EVM[0].NonceAutoSync = ptr(false) c.EVM[0].BalanceMonitor.Enabled = ptr(false) @@ -251,6 +251,7 @@ func TestETHKeysController_ChainSuccess_UpdateNonce(t *testing.T) { t.Parallel() ethClient := cltest.NewEthMocksWithStartupAssertions(t) + ethClient.On("PendingNonceAt", mock.Anything, mock.Anything).Return(uint64(0), nil) cfg := configtest.NewGeneralConfig(t, func(c *chainlink.Config, s *chainlink.Secrets) { c.EVM[0].NonceAutoSync = ptr(false) c.EVM[0].BalanceMonitor.Enabled = ptr(false) @@ -270,10 +271,8 @@ func TestETHKeysController_ChainSuccess_UpdateNonce(t *testing.T) { chainURL := url.URL{Path: "/v2/keys/evm/chain"} query := chainURL.Query() - nextNonce := 52 query.Set("address", addr.Hex()) query.Set("evmChainID", cltest.FixtureChainID.String()) - query.Set("nextNonce", strconv.Itoa(nextNonce)) chainURL.RawQuery = query.Encode() resp, cleanup := client.Post(chainURL.String(), nil) @@ -287,13 +286,13 @@ func TestETHKeysController_ChainSuccess_UpdateNonce(t *testing.T) { assert.Equal(t, key.ID(), updatedKey.ID) assert.Equal(t, cltest.FixtureChainID.String(), updatedKey.EVMChainID.String()) assert.Equal(t, false, updatedKey.Disabled) - assert.Equal(t, int64(nextNonce), updatedKey.NextNonce) } func TestETHKeysController_ChainSuccess_Disable(t *testing.T) { t.Parallel() ethClient := cltest.NewEthMocksWithStartupAssertions(t) + ethClient.On("PendingNonceAt", mock.Anything, mock.Anything).Return(uint64(0), nil) cfg := configtest.NewGeneralConfig(t, func(c *chainlink.Config, s *chainlink.Secrets) { c.EVM[0].NonceAutoSync = ptr(false) c.EVM[0].BalanceMonitor.Enabled = ptr(false) @@ -330,7 +329,6 @@ func TestETHKeysController_ChainSuccess_Disable(t *testing.T) { assert.Equal(t, key.ID(), updatedKey.ID) assert.Equal(t, cltest.FixtureChainID.String(), updatedKey.EVMChainID.String()) - assert.Equal(t, int64(0), updatedKey.NextNonce) assert.Equal(t, true, updatedKey.Disabled) } @@ -338,6 +336,7 @@ func TestETHKeysController_ChainSuccess_Enable(t *testing.T) { t.Parallel() ethClient := cltest.NewEthMocksWithStartupAssertions(t) + ethClient.On("PendingNonceAt", mock.Anything, mock.Anything).Return(uint64(0), nil) cfg := configtest.NewGeneralConfig(t, func(c *chainlink.Config, s *chainlink.Secrets) { c.EVM[0].NonceAutoSync = ptr(false) c.EVM[0].BalanceMonitor.Enabled = ptr(false) @@ -374,7 +373,6 @@ func TestETHKeysController_ChainSuccess_Enable(t *testing.T) { assert.Equal(t, key.ID(), updatedKey.ID) assert.Equal(t, cltest.FixtureChainID.String(), updatedKey.EVMChainID.String()) - assert.Equal(t, int64(0), updatedKey.NextNonce) assert.Equal(t, false, updatedKey.Disabled) } @@ -382,6 +380,7 @@ func TestETHKeysController_ChainSuccess_ResetWithAbandon(t *testing.T) { t.Parallel() ethClient := cltest.NewEthMocksWithStartupAssertions(t) + ethClient.On("PendingNonceAt", mock.Anything, mock.Anything).Return(uint64(0), nil) cfg := configtest.NewGeneralConfig(t, func(c *chainlink.Config, s *chainlink.Secrets) { c.EVM[0].NonceAutoSync = ptr(false) c.EVM[0].BalanceMonitor.Enabled = ptr(false) @@ -437,7 +436,6 @@ func TestETHKeysController_ChainSuccess_ResetWithAbandon(t *testing.T) { assert.Equal(t, key.ID(), updatedKey.ID) assert.Equal(t, cltest.FixtureChainID.String(), updatedKey.EVMChainID.String()) - assert.Equal(t, int64(0), updatedKey.NextNonce) assert.Equal(t, false, updatedKey.Disabled) var s string @@ -450,6 +448,7 @@ func TestETHKeysController_ChainFailure_InvalidAbandon(t *testing.T) { t.Parallel() ethClient := cltest.NewEthMocksWithStartupAssertions(t) + ethClient.On("PendingNonceAt", mock.Anything, mock.Anything).Return(uint64(0), nil) cfg := configtest.NewGeneralConfig(t, func(c *chainlink.Config, s *chainlink.Secrets) { c.EVM[0].NonceAutoSync = ptr(false) c.EVM[0].BalanceMonitor.Enabled = ptr(false) @@ -467,10 +466,8 @@ func TestETHKeysController_ChainFailure_InvalidAbandon(t *testing.T) { chainURL := url.URL{Path: "/v2/keys/evm/chain"} query := chainURL.Query() - nextNonce := 52 query.Set("address", addr.Hex()) query.Set("evmChainID", cltest.FixtureChainID.String()) - query.Set("nextNonce", strconv.Itoa(nextNonce)) query.Set("abandon", "invalid") chainURL.RawQuery = query.Encode() @@ -484,6 +481,7 @@ func TestETHKeysController_ChainFailure_InvalidEnabled(t *testing.T) { t.Parallel() ethClient := cltest.NewEthMocksWithStartupAssertions(t) + ethClient.On("PendingNonceAt", mock.Anything, mock.Anything).Return(uint64(0), nil) cfg := configtest.NewGeneralConfig(t, func(c *chainlink.Config, s *chainlink.Secrets) { c.EVM[0].NonceAutoSync = ptr(false) c.EVM[0].BalanceMonitor.Enabled = ptr(false) @@ -501,10 +499,8 @@ func TestETHKeysController_ChainFailure_InvalidEnabled(t *testing.T) { chainURL := url.URL{Path: "/v2/keys/evm/chain"} query := chainURL.Query() - nextNonce := 52 query.Set("address", addr.Hex()) query.Set("evmChainID", cltest.FixtureChainID.String()) - query.Set("nextNonce", strconv.Itoa(nextNonce)) query.Set("enabled", "invalid") chainURL.RawQuery = query.Encode() @@ -532,10 +528,8 @@ func TestETHKeysController_ChainFailure_InvalidAddress(t *testing.T) { chainURL := url.URL{Path: "/v2/keys/evm/chain"} query := chainURL.Query() - nextNonce := 52 query.Set("address", "invalid address") query.Set("evmChainID", cltest.FixtureChainID.String()) - query.Set("nextNonce", strconv.Itoa(nextNonce)) chainURL.RawQuery = query.Encode() resp, cleanup := client.Post(chainURL.String(), nil) @@ -562,16 +556,14 @@ func TestETHKeysController_ChainFailure_MissingAddress(t *testing.T) { chainURL := url.URL{Path: "/v2/keys/evm/chain"} query := chainURL.Query() - nextNonce := 52 query.Set("address", testutils.NewAddress().Hex()) query.Set("evmChainID", cltest.FixtureChainID.String()) - query.Set("nextNonce", strconv.Itoa(nextNonce)) chainURL.RawQuery = query.Encode() resp, cleanup := client.Post(chainURL.String(), nil) defer cleanup() - assert.Equal(t, http.StatusNotFound, resp.StatusCode) + assert.Equal(t, http.StatusInternalServerError, resp.StatusCode) } func TestETHKeysController_ChainFailure_InvalidChainID(t *testing.T) { @@ -592,10 +584,8 @@ func TestETHKeysController_ChainFailure_InvalidChainID(t *testing.T) { chainURL := url.URL{Path: "/v2/keys/evm/chain"} query := chainURL.Query() - nextNonce := 52 query.Set("address", testutils.NewAddress().Hex()) query.Set("evmChainID", "bad chain ID") - query.Set("nextNonce", strconv.Itoa(nextNonce)) chainURL.RawQuery = query.Encode() resp, cleanup := client.Post(chainURL.String(), nil) @@ -608,6 +598,7 @@ func TestETHKeysController_ChainFailure_MissingChainID(t *testing.T) { t.Parallel() ethClient := cltest.NewEthMocksWithStartupAssertions(t) + ethClient.On("PendingNonceAt", mock.Anything, mock.Anything).Return(uint64(0), nil) cfg := configtest.NewGeneralConfig(t, func(c *chainlink.Config, s *chainlink.Secrets) { c.EVM[0].NonceAutoSync = ptr(false) c.EVM[0].BalanceMonitor.Enabled = ptr(false) @@ -625,10 +616,8 @@ func TestETHKeysController_ChainFailure_MissingChainID(t *testing.T) { chainURL := url.URL{Path: "/v2/keys/evm/chain"} query := chainURL.Query() - nextNonce := 52 query.Set("address", addr.Hex()) query.Set("evmChainID", "123456789") - query.Set("nextNonce", strconv.Itoa(nextNonce)) chainURL.RawQuery = query.Encode() resp, cleanup := client.Post(chainURL.String(), nil) @@ -637,41 +626,10 @@ func TestETHKeysController_ChainFailure_MissingChainID(t *testing.T) { assert.Equal(t, http.StatusNotFound, resp.StatusCode) } -func TestETHKeysController_ChainFailure_InvalidNonce(t *testing.T) { - t.Parallel() - - ethClient := cltest.NewEthMocksWithStartupAssertions(t) - cfg := configtest.NewGeneralConfig(t, func(c *chainlink.Config, s *chainlink.Secrets) { - c.EVM[0].NonceAutoSync = ptr(false) - c.EVM[0].BalanceMonitor.Enabled = ptr(false) - }) - app := cltest.NewApplicationWithConfig(t, cfg, ethClient) - - require.NoError(t, app.KeyStore.Unlock(cltest.Password)) - - // enabled key - _, addr := cltest.MustInsertRandomKey(t, app.KeyStore.Eth()) - - require.NoError(t, app.Start(testutils.Context(t))) - - client := app.NewHTTPClient(&cltest.User{}) - chainURL := url.URL{Path: "/v2/keys/evm/chain"} - query := chainURL.Query() - - query.Set("address", addr.Hex()) - query.Set("evmChainID", cltest.FixtureChainID.String()) - query.Set("nextNonce", "bad nonce") - - chainURL.RawQuery = query.Encode() - resp, cleanup := client.Post(chainURL.String(), nil) - defer cleanup() - - assert.Equal(t, http.StatusBadRequest, resp.StatusCode) -} - func TestETHKeysController_DeleteSuccess(t *testing.T) { t.Parallel() ethClient := cltest.NewEthMocksWithStartupAssertions(t) + ethClient.On("PendingNonceAt", mock.Anything, mock.Anything).Return(uint64(0), nil) cfg := configtest.NewGeneralConfig(t, func(c *chainlink.Config, s *chainlink.Secrets) { c.EVM[0].NonceAutoSync = ptr(false) c.EVM[0].BalanceMonitor.Enabled = ptr(false) @@ -704,7 +662,6 @@ func TestETHKeysController_DeleteSuccess(t *testing.T) { assert.Equal(t, key0.ID(), deletedKey.ID) assert.Equal(t, cltest.FixtureChainID.String(), deletedKey.EVMChainID.String()) assert.Equal(t, false, deletedKey.Disabled) - assert.Equal(t, int64(0), deletedKey.NextNonce) resp, cleanup2 := client.Get("/v2/keys/evm") defer cleanup2() diff --git a/core/web/pipeline_runs_controller_test.go b/core/web/pipeline_runs_controller_test.go index 8e53384856d..3efd692c9db 100644 --- a/core/web/pipeline_runs_controller_test.go +++ b/core/web/pipeline_runs_controller_test.go @@ -14,6 +14,7 @@ import ( "github.com/google/uuid" "github.com/pelletier/go-toml" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" "github.com/smartcontractkit/chainlink/v2/core/internal/cltest" @@ -253,6 +254,7 @@ func TestPipelineRunsController_ShowRun_InvalidID(t *testing.T) { func setupPipelineRunsControllerTests(t *testing.T) (cltest.HTTPClientCleaner, int32, []int64) { t.Parallel() ethClient := cltest.NewEthMocksWithStartupAssertions(t) + ethClient.On("PendingNonceAt", mock.Anything, mock.Anything).Return(uint64(0), nil) cfg := configtest.NewGeneralConfig(t, func(c *chainlink.Config, s *chainlink.Secrets) { c.OCR.Enabled = ptr(true) c.P2P.V1.Enabled = ptr(true) diff --git a/core/web/presenters/eth_key.go b/core/web/presenters/eth_key.go index d3a8c55c2be..167679513ef 100644 --- a/core/web/presenters/eth_key.go +++ b/core/web/presenters/eth_key.go @@ -14,7 +14,6 @@ type ETHKeyResource struct { JAID EVMChainID utils.Big `json:"evmChainID"` Address string `json:"address"` - NextNonce int64 `json:"nextNonce"` EthBalance *assets.Eth `json:"ethBalance"` LinkBalance *assets.Link `json:"linkBalance"` Disabled bool `json:"disabled"` @@ -42,7 +41,6 @@ func NewETHKeyResource(k ethkey.KeyV2, state ethkey.State, opts ...NewETHKeyOpti r := ÐKeyResource{ JAID: NewJAID(k.Address.Hex()), EVMChainID: state.EVMChainID, - NextNonce: state.NextNonce, Address: k.Address.Hex(), EthBalance: nil, LinkBalance: nil, diff --git a/core/web/presenters/eth_key_test.go b/core/web/presenters/eth_key_test.go index 1518762cc0e..7afb55068fb 100644 --- a/core/web/presenters/eth_key_test.go +++ b/core/web/presenters/eth_key_test.go @@ -31,7 +31,6 @@ func TestETHKeyResource(t *testing.T) { state := ethkey.State{ ID: 1, EVMChainID: *utils.NewBigI(42), - NextNonce: 99, Address: eip55address, CreatedAt: now, UpdatedAt: now, @@ -59,7 +58,6 @@ func TestETHKeyResource(t *testing.T) { "attributes":{ "address":"%s", "evmChainID":"42", - "nextNonce": 99, "ethBalance":"1", "linkBalance":"1", "disabled":true, @@ -89,7 +87,6 @@ func TestETHKeyResource(t *testing.T) { "attributes":{ "address":"%s", "evmChainID":"42", - "nextNonce": 99, "ethBalance":null, "linkBalance":null, "disabled":true, diff --git a/docs/core/BULLETPROOF_TX_MANAGER_ARCHITECTURE.md b/docs/core/BULLETPROOF_TX_MANAGER_ARCHITECTURE.md index 37480a88c4c..6e0ff946ffc 100644 --- a/docs/core/BULLETPROOF_TX_MANAGER_ARCHITECTURE.md +++ b/docs/core/BULLETPROOF_TX_MANAGER_ARCHITECTURE.md @@ -4,28 +4,28 @@ ## Finite state machine -### `eth_txes.state` +### `evm.txes.state` `unstarted` | | v `in_progress` (only one per key) -| \ -| \ -v v -`fatal_error` `unconfirmed` - | ^ - | | - v | - `confirmed` +| \ +| \ +v v +`fatal_error` `unconfirmed` +| ^ +| | +v | +`confirmed` ### `eth_tx_attempts.state` `in_progress` -| ^ -| | -v | +| ^ +| | +v | `broadcast` # Data structures @@ -42,7 +42,7 @@ EB - EthBroadcaster EC - EthConfirmer -`eth_txes` has five possible states: +`evm.txes` has five possible states: - EB ⚫️ `unstarted` - EB 🟠 `in_progress` @@ -57,12 +57,6 @@ EC - EthConfirmer An attempt may have 0 or more `eth_receipts` indicating that the transaction has been mined into a block. This block may or may not exist as part of the canonical longest chain. -`keys` has a field: - -- `next_nonce` - -Which tracks the nonce that is available to use for the next transaction. It is only updated after a successful broadcast has occurred. - # Components BulletproofTxManager is split into three components, each of which has a clearly delineated set of responsibilities. @@ -71,7 +65,7 @@ BulletproofTxManager is split into three components, each of which has a clearly Conceptually, **EthTx** defines the transaction. -**EthTx** is responsible for generating the transaction criteria and inserting the initial `unstarted` row into the `eth_txes` table. +**EthTx** is responsible for generating the transaction criteria and inserting the initial `unstarted` row into the `evm.txes` table. **EthTx** guarantees that the transaction is defined with the following criteria: @@ -87,9 +81,9 @@ EthTx should wait until it's transaction confirms before marking the task as com ## EthBroadcaster -Conceptually, **EthBroadcaster** assigns a nonce to a transaction and ensures that it is valid. It alone controls the `keys.next_nonce` field. +Conceptually, **EthBroadcaster** assigns a nonce to a transaction and ensures that it is valid. It alone maintains the next usable sequence for a transaction. -**EthBroadcaster** monitors `eth_txes` for transactions that need to be broadcast, assigns nonces and ensures that at least one eth node somewhere has placed the transaction into its mempool. +**EthBroadcaster** monitors `evm.txes` for transactions that need to be broadcast, assigns nonces and ensures that at least one eth node somewhere has placed the transaction into its mempool. It does not guarantee eventual confirmation! @@ -97,8 +91,8 @@ A whole host of other things can subsequently go wrong such as transactions bein **EthBroadcaster** makes the following guarantees: -- A gapless, monotonically increasing sequence of nonces for `eth_txes` (scoped to key). -- Transition of `eth_txes` from `unstarted` to either `fatal_error` or `unconfirmed`. +- A gapless, monotonically increasing sequence of nonces for `evm.txes` (scoped to key). +- Transition of `evm.txes` from `unstarted` to either `fatal_error` or `unconfirmed`. - If final state is `fatal_error` then the nonce is unassigned, and it is impossible that this transaction could ever be mined into a block. - If final state is `unconfirmed` then a saved `eth_transaction_attempt` exists. - If final state is `unconfirmed` then an eth node somewhere has accepted this transaction into its mempool at least once. @@ -132,7 +126,7 @@ Find all transactions confirmed within the past `ETH_FINALITY_DEPTH` blocks and **EthConfirmer** makes the following guarantees: - All transactions will eventually be confirmed on the canonical longest chain, unless a reorg occurs that is deeper than `ETH_FINALITY_DEPTH` blocks. -- In the case that an external wallet used the nonce, we will ensure that *a* transaction exists at this nonce up to a depth of `ETH_FINALITY_DEPTH` blocks but it most likely will not be the transaction in our database. +- In the case that an external wallet used the nonce, we will ensure that _a_ transaction exists at this nonce up to a depth of `ETH_FINALITY_DEPTH` blocks but it most likely will not be the transaction in our database. Note that since checking for inclusion in the longest chain can now be done cheaply, without any calls to the eth node, `ETH_FINALITY_DEPTH` can be set to something quite large without penalty (e.g. 50 or 100). diff --git a/testdata/scripts/keys/eth/help.txtar b/testdata/scripts/keys/eth/help.txtar index 0484445580e..76db7cd5ae3 100644 --- a/testdata/scripts/keys/eth/help.txtar +++ b/testdata/scripts/keys/eth/help.txtar @@ -10,7 +10,7 @@ USAGE: COMMANDS: create Create a key in the node's keystore alongside the existing key; to create an original key, just run the node - list List available Ethereum accounts with their ETH & LINK balances, nonces, and other metadata + list List available Ethereum accounts with their ETH & LINK balances and other metadata delete Delete the ETH key by address (irreversible!) import Import an ETH key from a JSON file export Exports an ETH key to a JSON file From 104efefb30ec60a908729bb39c0038a10643e96f Mon Sep 17 00:00:00 2001 From: amit-momin Date: Mon, 2 Oct 2023 17:34:16 -0500 Subject: [PATCH 2/9] Changed sequence Increment method name to Next --- common/txmgr/broadcaster.go | 4 ++-- common/types/chain.go | 2 +- core/chains/evm/types/nonce.go | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/common/txmgr/broadcaster.go b/common/txmgr/broadcaster.go index d6f73c607ad..04399d6d106 100644 --- a/common/txmgr/broadcaster.go +++ b/common/txmgr/broadcaster.go @@ -343,7 +343,7 @@ func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) load nextSequenceMap[address] = seq } else { - nextSequenceMap[address] = seq.Increment() + nextSequenceMap[address] = seq.Next() } } @@ -829,7 +829,7 @@ func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) Incr if !exists { return errors.New(fmt.Sprint("address not found in next sequence map: ", address)) } - eb.nextSequenceMap[address] = seq.Increment() + eb.nextSequenceMap[address] = seq.Next() return nil } diff --git a/common/types/chain.go b/common/types/chain.go index a502b5cf91d..b230ce870d7 100644 --- a/common/types/chain.go +++ b/common/types/chain.go @@ -7,7 +7,7 @@ import "fmt" type Sequence[SEQ any] interface { fmt.Stringer Int64() int64 // needed for numeric sequence confirmation - to be removed with confirmation logic generalization: https://smartcontract-it.atlassian.net/browse/BCI-860 - Increment() SEQ // needed to produce the next usable sequence for a transaction + Next() SEQ // needed to produce the next usable sequence for a transaction } // ID represents the base type, for any chain's ID. diff --git a/core/chains/evm/types/nonce.go b/core/chains/evm/types/nonce.go index 4d24cf7d251..8ed4b50e72d 100644 --- a/core/chains/evm/types/nonce.go +++ b/core/chains/evm/types/nonce.go @@ -18,6 +18,6 @@ func (n Nonce) String() string { return strconv.FormatInt(n.Int64(), 10) } -func (n Nonce) Increment() Nonce { +func (n Nonce) Next() Nonce { return Nonce(n + 1) } From 1b6c86da9c5ced1c990787f4bff4c7e46ac1bb58 Mon Sep 17 00:00:00 2001 From: amit-momin Date: Wed, 4 Oct 2023 12:21:38 -0500 Subject: [PATCH 3/9] Addressed PR feedback --- common/txmgr/broadcaster.go | 63 ++++++++++----------- common/txmgr/types/mocks/tx_store.go | 28 ++++----- common/txmgr/types/tx_store.go | 4 +- common/types/chain.go | 2 +- core/chains/evm/txmgr/evm_tx_store.go | 17 ++++-- core/chains/evm/txmgr/evm_tx_store_test.go | 2 +- core/chains/evm/txmgr/mocks/evm_tx_store.go | 28 ++++----- docs/CHANGELOG.md | 1 + 8 files changed, 75 insertions(+), 70 deletions(-) diff --git a/common/txmgr/broadcaster.go b/common/txmgr/broadcaster.go index 04399d6d106..6847e5afea7 100644 --- a/common/txmgr/broadcaster.go +++ b/common/txmgr/broadcaster.go @@ -332,7 +332,7 @@ func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) load for _, address := range addresses { // Get the highest sequence from the tx table // Will need to be incremented since this sequence is already used - seq, err := eb.txStore.FindHighestSequence(address, eb.chainID) + seq, err := eb.txStore.FindHighestSequence(ctx, address, eb.chainID) if err != nil { // Look for nonce on-chain if address not found in TxStore // Returns the nonce that should be used for the next transaction so no need to increment @@ -427,7 +427,7 @@ func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) moni } } -// syncSequence tries to sync the key sequence, retrying indefinitely until success +// syncSequence tries to sync the key sequence, retrying indefinitely until success or stop signal is sent func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) SyncSequence(ctx context.Context, addr ADDR) { sequenceSyncRetryBackoff := eb.newSequenceSyncBackoff() localSequence, err := eb.GetNextSequence(addr) @@ -437,40 +437,35 @@ func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) Sync return } - newNextSequence, err := eb.sequenceSyncer.Sync(ctx, addr, localSequence) - if err != nil { - // Enter retry loop with backoff - var attempt int - eb.logger.Errorw("Failed to sync with on-chain sequence", "address", addr.String(), "attempt", attempt, "err", err) - loop: - for { - select { - case <-eb.chStop: - return - case <-time.After(sequenceSyncRetryBackoff.Duration()): - attempt++ - newNextSequence, err = eb.sequenceSyncer.Sync(ctx, addr, localSequence) - _ = newNextSequence + // Enter loop with retries + var attempt int + for { + select { + case <-eb.chStop: + return + case <-time.After(sequenceSyncRetryBackoff.Duration()): + attempt++ + newNextSequence, err := eb.sequenceSyncer.Sync(ctx, addr, localSequence) + if err != nil { + if attempt > 5 { + eb.logger.Criticalw("Failed to sync with on-chain sequence", "address", addr.String(), "attempt", attempt, "err", err) + eb.SvcErrBuffer.Append(err) + } else { + eb.logger.Warnw("Failed to sync with on-chain sequence", "address", addr.String(), "attempt", attempt, "err", err) + } + continue + } + // Found new sequence to use from on-chain + if localSequence.String() != newNextSequence.String() { + eb.logger.Infow("Fast-forward sequence", "address", addr, "newNextSequence", newNextSequence, "oldNextSequence", localSequence) + // Set new sequence in the map + err = eb.SetNextSequence(addr, newNextSequence) if err != nil { - if attempt > 5 { - eb.logger.Criticalw("Failed to sync with on-chain sequence", "address", addr.String(), "attempt", attempt, "err", err) - eb.SvcErrBuffer.Append(err) - } else { - eb.logger.Warnw("Failed to sync with on-chain sequence", "address", addr.String(), "attempt", attempt, "err", err) - } - continue + eb.logger.Criticalw("Failed to set new sequence for address", "address", addr.String(), "sequence", newNextSequence.String(), "err", err) } - break loop } - } - } - if err == nil && localSequence.String() != newNextSequence.String() { - eb.logger.Infow("Fast-forward sequence", "address", addr, "newNextSequence", newNextSequence, "oldNextSequence", localSequence) - // Set new next sequence in the map - err = eb.SetNextSequence(addr, newNextSequence) - if err != nil { - eb.logger.Criticalw("Failed to set new sequence for address", "address", addr.String(), "sequence", newNextSequence.String(), "err", err) return + } } } @@ -650,7 +645,7 @@ func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) hand // and hand off to the confirmer to get the receipt (or mark as // failed). observeTimeUntilBroadcast(eb.chainID, etx.CreatedAt, time.Now()) - err = eb.txStore.UpdateTxAttemptInProgressToBroadcast(&etx, attempt, txmgrtypes.TxAttemptBroadcast, eb.IncrementNextSequence) + err = eb.txStore.UpdateTxAttemptInProgressToBroadcast(ctx, &etx, attempt, txmgrtypes.TxAttemptBroadcast, eb.IncrementNextSequence) return err, true case clienttypes.Underpriced: return eb.tryAgainBumpingGas(ctx, lgr, err, etx, attempt, initialBroadcastAt) @@ -696,7 +691,7 @@ func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) hand // Despite the error, the RPC node considers the previously sent // transaction to have been accepted. In this case, the right thing to // do is assume success and hand off to Confirmer - err = eb.txStore.UpdateTxAttemptInProgressToBroadcast(&etx, attempt, txmgrtypes.TxAttemptBroadcast, eb.IncrementNextSequence) + err = eb.txStore.UpdateTxAttemptInProgressToBroadcast(ctx, &etx, attempt, txmgrtypes.TxAttemptBroadcast, eb.IncrementNextSequence) return err, true } // Either the unknown error prevented the transaction from being mined, or diff --git a/common/txmgr/types/mocks/tx_store.go b/common/txmgr/types/mocks/tx_store.go index ec0f36eef6e..6a935392ba6 100644 --- a/common/txmgr/types/mocks/tx_store.go +++ b/common/txmgr/types/mocks/tx_store.go @@ -141,23 +141,23 @@ func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) DeleteInPro return r0 } -// FindHighestSequence provides a mock function with given fields: fromAddress, chainId -func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindHighestSequence(fromAddress ADDR, chainId CHAIN_ID) (SEQ, error) { - ret := _m.Called(fromAddress, chainId) +// FindHighestSequence provides a mock function with given fields: ctx, fromAddress, chainId +func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindHighestSequence(ctx context.Context, fromAddress ADDR, chainId CHAIN_ID) (SEQ, error) { + ret := _m.Called(ctx, fromAddress, chainId) var r0 SEQ var r1 error - if rf, ok := ret.Get(0).(func(ADDR, CHAIN_ID) (SEQ, error)); ok { - return rf(fromAddress, chainId) + if rf, ok := ret.Get(0).(func(context.Context, ADDR, CHAIN_ID) (SEQ, error)); ok { + return rf(ctx, fromAddress, chainId) } - if rf, ok := ret.Get(0).(func(ADDR, CHAIN_ID) SEQ); ok { - r0 = rf(fromAddress, chainId) + if rf, ok := ret.Get(0).(func(context.Context, ADDR, CHAIN_ID) SEQ); ok { + r0 = rf(ctx, fromAddress, chainId) } else { r0 = ret.Get(0).(SEQ) } - if rf, ok := ret.Get(1).(func(ADDR, CHAIN_ID) error); ok { - r1 = rf(fromAddress, chainId) + if rf, ok := ret.Get(1).(func(context.Context, ADDR, CHAIN_ID) error); ok { + r1 = rf(ctx, fromAddress, chainId) } else { r1 = ret.Error(1) } @@ -695,13 +695,13 @@ func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) UpdateBroad return r0 } -// UpdateTxAttemptInProgressToBroadcast provides a mock function with given fields: etx, attempt, NewAttemptState, incrementSeqFunc -func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) UpdateTxAttemptInProgressToBroadcast(etx *txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], attempt txmgrtypes.TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], NewAttemptState txmgrtypes.TxAttemptState, incrementSeqFunc func(ADDR) error) error { - ret := _m.Called(etx, attempt, NewAttemptState, incrementSeqFunc) +// UpdateTxAttemptInProgressToBroadcast provides a mock function with given fields: ctx, etx, attempt, NewAttemptState, incrementSeqFunc +func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) UpdateTxAttemptInProgressToBroadcast(ctx context.Context, etx *txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], attempt txmgrtypes.TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], NewAttemptState txmgrtypes.TxAttemptState, incrementSeqFunc func(ADDR) error) error { + ret := _m.Called(ctx, etx, attempt, NewAttemptState, incrementSeqFunc) var r0 error - if rf, ok := ret.Get(0).(func(*txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], txmgrtypes.TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], txmgrtypes.TxAttemptState, func(ADDR) error) error); ok { - r0 = rf(etx, attempt, NewAttemptState, incrementSeqFunc) + if rf, ok := ret.Get(0).(func(context.Context, *txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], txmgrtypes.TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], txmgrtypes.TxAttemptState, func(ADDR) error) error); ok { + r0 = rf(ctx, etx, attempt, NewAttemptState, incrementSeqFunc) } else { r0 = ret.Error(0) } diff --git a/common/txmgr/types/tx_store.go b/common/txmgr/types/tx_store.go index 421ba62e0b1..50abe90c902 100644 --- a/common/txmgr/types/tx_store.go +++ b/common/txmgr/types/tx_store.go @@ -58,7 +58,7 @@ type TransactionStore[ CountUnstartedTransactions(ctx context.Context, fromAddress ADDR, chainID CHAIN_ID) (count uint32, err error) CreateTransaction(ctx context.Context, txRequest TxRequest[ADDR, TX_HASH], chainID CHAIN_ID) (tx Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) DeleteInProgressAttempt(ctx context.Context, attempt TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) error - FindHighestSequence(fromAddress ADDR, chainId CHAIN_ID) (SEQ, error) + FindHighestSequence(ctx context.Context, fromAddress ADDR, chainId CHAIN_ID) (SEQ, error) FindTxsRequiringGasBump(ctx context.Context, address ADDR, blockNum, gasBumpThreshold, depth int64, chainID CHAIN_ID) (etxs []*Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) FindTxsRequiringResubmissionDueToInsufficientFunds(ctx context.Context, address ADDR, chainID CHAIN_ID) (etxs []*Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) FindTxAttemptsConfirmedMissingReceipt(ctx context.Context, chainID CHAIN_ID) (attempts []TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) @@ -84,7 +84,7 @@ type TransactionStore[ SaveSentAttempt(ctx context.Context, timeout time.Duration, attempt *TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], broadcastAt time.Time) error SetBroadcastBeforeBlockNum(ctx context.Context, blockNum int64, chainID CHAIN_ID) error UpdateBroadcastAts(ctx context.Context, now time.Time, etxIDs []int64) error - UpdateTxAttemptInProgressToBroadcast(etx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], attempt TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], NewAttemptState TxAttemptState, incrementSeqFunc func(address ADDR) error) error + UpdateTxAttemptInProgressToBroadcast(ctx context.Context, etx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], attempt TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], NewAttemptState TxAttemptState, incrementSeqFunc func(address ADDR) error) error UpdateTxsUnconfirmed(ctx context.Context, ids []int64) error UpdateTxUnstartedToInProgress(ctx context.Context, etx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], attempt *TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) error UpdateTxFatalError(ctx context.Context, etx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) error diff --git a/common/types/chain.go b/common/types/chain.go index b230ce870d7..319e963b732 100644 --- a/common/types/chain.go +++ b/common/types/chain.go @@ -7,7 +7,7 @@ import "fmt" type Sequence[SEQ any] interface { fmt.Stringer Int64() int64 // needed for numeric sequence confirmation - to be removed with confirmation logic generalization: https://smartcontract-it.atlassian.net/browse/BCI-860 - Next() SEQ // needed to produce the next usable sequence for a transaction + Next() SEQ // needed to produce the next usable sequence for a transaction } // ID represents the base type, for any chain's ID. diff --git a/core/chains/evm/txmgr/evm_tx_store.go b/core/chains/evm/txmgr/evm_tx_store.go index 69b7fe7622e..8ebe768c751 100644 --- a/core/chains/evm/txmgr/evm_tx_store.go +++ b/core/chains/evm/txmgr/evm_tx_store.go @@ -957,9 +957,14 @@ func (o *evmTxStore) FindReceiptsPendingConfirmation(ctx context.Context, blockN return } -func (o *evmTxStore) FindHighestSequence(fromAddress common.Address, chainId *big.Int) (nonce evmtypes.Nonce, err error) { +func (o *evmTxStore) FindHighestSequence(ctx context.Context, fromAddress common.Address, chainId *big.Int) (nonce evmtypes.Nonce, err error) { + var cancel context.CancelFunc + ctx, cancel = o.mergeContexts(ctx) + defer cancel() + qq := o.q.WithOpts(pg.WithParentCtx(ctx)) + sql := `SELECT MAX(nonce) FROM evm.txes WHERE from_address = $1 and evm_chain_id = $2` - err = o.q.Get(&nonce, sql, fromAddress.String(), chainId.String()) + err = qq.Get(&nonce, sql, fromAddress.String(), chainId.String()) return } @@ -1420,7 +1425,11 @@ func (o *evmTxStore) UpdateTxFatalError(ctx context.Context, etx *Tx) error { // Updates eth attempt from in_progress to broadcast. Also updates the eth tx to unconfirmed. // One of the more complicated signatures. We have to accept variable pg.QOpt and QueryerFunc arguments -func (o *evmTxStore) UpdateTxAttemptInProgressToBroadcast(etx *Tx, attempt TxAttempt, NewAttemptState txmgrtypes.TxAttemptState, incrementSeqFunc func(address common.Address) error) error { +func (o *evmTxStore) UpdateTxAttemptInProgressToBroadcast(ctx context.Context, etx *Tx, attempt TxAttempt, NewAttemptState txmgrtypes.TxAttemptState, incrementSeqFunc func(address common.Address) error) error { + var cancel context.CancelFunc + ctx, cancel = o.mergeContexts(ctx) + defer cancel() + qq := o.q.WithOpts(pg.WithParentCtx(ctx)) if etx.BroadcastAt == nil { return errors.New("unconfirmed transaction must have broadcast_at time") } @@ -1438,7 +1447,7 @@ func (o *evmTxStore) UpdateTxAttemptInProgressToBroadcast(etx *Tx, attempt TxAtt } etx.State = txmgr.TxUnconfirmed attempt.State = NewAttemptState - return o.q.Transaction(func(tx pg.Queryer) error { + return qq.Transaction(func(tx pg.Queryer) error { var dbEtx DbEthTx dbEtx.FromTx(etx) if err := tx.Get(&dbEtx, `UPDATE evm.txes SET state=$1, error=$2, broadcast_at=$3, initial_broadcast_at=$4 WHERE id = $5 RETURNING *`, dbEtx.State, dbEtx.Error, dbEtx.BroadcastAt, dbEtx.InitialBroadcastAt, dbEtx.ID); err != nil { diff --git a/core/chains/evm/txmgr/evm_tx_store_test.go b/core/chains/evm/txmgr/evm_tx_store_test.go index 728ecc2781b..266f3ef64d8 100644 --- a/core/chains/evm/txmgr/evm_tx_store_test.go +++ b/core/chains/evm/txmgr/evm_tx_store_test.go @@ -1211,7 +1211,7 @@ func TestORM_UpdateTxAttemptInProgressToBroadcast(t *testing.T) { i := int16(0) etx.BroadcastAt = &time1 etx.InitialBroadcastAt = &time1 - err := txStore.UpdateTxAttemptInProgressToBroadcast(&etx, attempt, txmgrtypes.TxAttemptBroadcast, func(_ common.Address) error { + err := txStore.UpdateTxAttemptInProgressToBroadcast(testutils.Context(t), &etx, attempt, txmgrtypes.TxAttemptBroadcast, func(_ common.Address) error { i++ return nil }) diff --git a/core/chains/evm/txmgr/mocks/evm_tx_store.go b/core/chains/evm/txmgr/mocks/evm_tx_store.go index 6dc95f7aea4..57012fcfc55 100644 --- a/core/chains/evm/txmgr/mocks/evm_tx_store.go +++ b/core/chains/evm/txmgr/mocks/evm_tx_store.go @@ -145,23 +145,23 @@ func (_m *EvmTxStore) DeleteInProgressAttempt(ctx context.Context, attempt types return r0 } -// FindHighestSequence provides a mock function with given fields: fromAddress, chainId -func (_m *EvmTxStore) FindHighestSequence(fromAddress common.Address, chainId *big.Int) (evmtypes.Nonce, error) { - ret := _m.Called(fromAddress, chainId) +// FindHighestSequence provides a mock function with given fields: ctx, fromAddress, chainId +func (_m *EvmTxStore) FindHighestSequence(ctx context.Context, fromAddress common.Address, chainId *big.Int) (evmtypes.Nonce, error) { + ret := _m.Called(ctx, fromAddress, chainId) var r0 evmtypes.Nonce var r1 error - if rf, ok := ret.Get(0).(func(common.Address, *big.Int) (evmtypes.Nonce, error)); ok { - return rf(fromAddress, chainId) + if rf, ok := ret.Get(0).(func(context.Context, common.Address, *big.Int) (evmtypes.Nonce, error)); ok { + return rf(ctx, fromAddress, chainId) } - if rf, ok := ret.Get(0).(func(common.Address, *big.Int) evmtypes.Nonce); ok { - r0 = rf(fromAddress, chainId) + if rf, ok := ret.Get(0).(func(context.Context, common.Address, *big.Int) evmtypes.Nonce); ok { + r0 = rf(ctx, fromAddress, chainId) } else { r0 = ret.Get(0).(evmtypes.Nonce) } - if rf, ok := ret.Get(1).(func(common.Address, *big.Int) error); ok { - r1 = rf(fromAddress, chainId) + if rf, ok := ret.Get(1).(func(context.Context, common.Address, *big.Int) error); ok { + r1 = rf(ctx, fromAddress, chainId) } else { r1 = ret.Error(1) } @@ -900,13 +900,13 @@ func (_m *EvmTxStore) UpdateBroadcastAts(ctx context.Context, now time.Time, etx return r0 } -// UpdateTxAttemptInProgressToBroadcast provides a mock function with given fields: etx, attempt, NewAttemptState, incrementSeqFunc -func (_m *EvmTxStore) UpdateTxAttemptInProgressToBroadcast(etx *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], attempt types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], NewAttemptState types.TxAttemptState, incrementSeqFunc func(common.Address) error) error { - ret := _m.Called(etx, attempt, NewAttemptState, incrementSeqFunc) +// UpdateTxAttemptInProgressToBroadcast provides a mock function with given fields: ctx, etx, attempt, NewAttemptState, incrementSeqFunc +func (_m *EvmTxStore) UpdateTxAttemptInProgressToBroadcast(ctx context.Context, etx *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], attempt types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], NewAttemptState types.TxAttemptState, incrementSeqFunc func(common.Address) error) error { + ret := _m.Called(ctx, etx, attempt, NewAttemptState, incrementSeqFunc) var r0 error - if rf, ok := ret.Get(0).(func(*types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], types.TxAttemptState, func(common.Address) error) error); ok { - r0 = rf(etx, attempt, NewAttemptState, incrementSeqFunc) + if rf, ok := ret.Get(0).(func(context.Context, *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], types.TxAttemptState, func(common.Address) error) error); ok { + r0 = rf(ctx, etx, attempt, NewAttemptState, incrementSeqFunc) } else { r0 = ret.Error(0) } diff --git a/docs/CHANGELOG.md b/docs/CHANGELOG.md index 24c0cb6845b..05942763d07 100644 --- a/docs/CHANGELOG.md +++ b/docs/CHANGELOG.md @@ -26,6 +26,7 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 - Removed support for sending telemetry to the deprecated Explorer service. All nodes will have to remove `Explorer` related keys from TOML configuration and env vars. - Removed default evmChainID logic where evmChainID was implicitly injected into the jobspecs based on node EVM chainID toml configuration. All newly created jobs(that have evmChainID field) will have to explicitly define evmChainID in the jobspec. - Removed keyset migration that migrated v1 keys to v2 keys. All keys should've been migrated by now, and we don't permit creation of new v1 keys anymore +- Removed the ability to set a next nonce value for an address through CLI All nodes will have to remove the following secret configurations: From 18e113255be49cb5d97ed35a205bfd3b235838e0 Mon Sep 17 00:00:00 2001 From: amit-momin Date: Wed, 4 Oct 2023 14:54:12 -0500 Subject: [PATCH 4/9] Fixed changelog entry --- docs/CHANGELOG.md | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docs/CHANGELOG.md b/docs/CHANGELOG.md index 05942763d07..71b211a8722 100644 --- a/docs/CHANGELOG.md +++ b/docs/CHANGELOG.md @@ -13,6 +13,10 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 - Added new configuration field named `LeaseDuration` for `EVM.NodePool` that will periodically check if internal subscriptions are connected to the "best" (as defined by the `SelectionMode`) node and switch to it if necessary. Setting this value to `0s` will disable this feature. +### Removed + +- Removed the ability to set a next nonce value for an address through CLI + ## 2.6.0 - UNRELEASED ### Added @@ -26,7 +30,6 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 - Removed support for sending telemetry to the deprecated Explorer service. All nodes will have to remove `Explorer` related keys from TOML configuration and env vars. - Removed default evmChainID logic where evmChainID was implicitly injected into the jobspecs based on node EVM chainID toml configuration. All newly created jobs(that have evmChainID field) will have to explicitly define evmChainID in the jobspec. - Removed keyset migration that migrated v1 keys to v2 keys. All keys should've been migrated by now, and we don't permit creation of new v1 keys anymore -- Removed the ability to set a next nonce value for an address through CLI All nodes will have to remove the following secret configurations: From c3df6f33d1fb3587dfa8e9fa6c7a71512b503b47 Mon Sep 17 00:00:00 2001 From: amit-momin Date: Fri, 6 Oct 2023 13:25:50 -0500 Subject: [PATCH 5/9] Addressed PR feedback --- common/txmgr/broadcaster.go | 48 +++++++++++-------- .../{nonce_syncer.go => sequence_syncer.go} | 0 common/txmgr/types/mocks/tx_store.go | 10 ++-- common/txmgr/types/tx_store.go | 2 +- core/chains/evm/txmgr/broadcaster_test.go | 32 ++++--------- core/chains/evm/txmgr/evm_tx_store.go | 5 +- core/chains/evm/txmgr/evm_tx_store_test.go | 8 ++-- core/chains/evm/txmgr/mocks/evm_tx_store.go | 10 ++-- 8 files changed, 53 insertions(+), 62 deletions(-) rename common/txmgr/{nonce_syncer.go => sequence_syncer.go} (100%) diff --git a/common/txmgr/broadcaster.go b/common/txmgr/broadcaster.go index 6847e5afea7..33c0c20a640 100644 --- a/common/txmgr/broadcaster.go +++ b/common/txmgr/broadcaster.go @@ -459,10 +459,7 @@ func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) Sync if localSequence.String() != newNextSequence.String() { eb.logger.Infow("Fast-forward sequence", "address", addr, "newNextSequence", newNextSequence, "oldNextSequence", localSequence) // Set new sequence in the map - err = eb.SetNextSequence(addr, newNextSequence) - if err != nil { - eb.logger.Criticalw("Failed to set new sequence for address", "address", addr.String(), "sequence", newNextSequence.String(), "err", err) - } + eb.SetNextSequence(addr, newNextSequence) } return @@ -645,7 +642,18 @@ func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) hand // and hand off to the confirmer to get the receipt (or mark as // failed). observeTimeUntilBroadcast(eb.chainID, etx.CreatedAt, time.Now()) - err = eb.txStore.UpdateTxAttemptInProgressToBroadcast(ctx, &etx, attempt, txmgrtypes.TxAttemptBroadcast, eb.IncrementNextSequence) + // Check if from_address exists in map to ensure it is valid before broadcasting + var sequence SEQ + sequence, err = eb.GetNextSequence(etx.FromAddress) + if err != nil { + return err, true + } + err = eb.txStore.UpdateTxAttemptInProgressToBroadcast(ctx, &etx, attempt, txmgrtypes.TxAttemptBroadcast) + if err != nil { + return err, true + } + // Increment sequence if successfully broadcasted + eb.IncrementNextSequence(etx.FromAddress, sequence) return err, true case clienttypes.Underpriced: return eb.tryAgainBumpingGas(ctx, lgr, err, etx, attempt, initialBroadcastAt) @@ -691,7 +699,19 @@ func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) hand // Despite the error, the RPC node considers the previously sent // transaction to have been accepted. In this case, the right thing to // do is assume success and hand off to Confirmer - err = eb.txStore.UpdateTxAttemptInProgressToBroadcast(ctx, &etx, attempt, txmgrtypes.TxAttemptBroadcast, eb.IncrementNextSequence) + + // Check if from_address exists in map to ensure it is valid before broadcasting + var sequence SEQ + sequence, err = eb.GetNextSequence(etx.FromAddress) + if err != nil { + return err, true + } + err = eb.txStore.UpdateTxAttemptInProgressToBroadcast(ctx, &etx, attempt, txmgrtypes.TxAttemptBroadcast) + if err != nil { + return err, true + } + // Increment sequence if successfully broadcasted + eb.IncrementNextSequence(etx.FromAddress, sequence) return err, true } // Either the unknown error prevented the transaction from being mined, or @@ -816,29 +836,17 @@ func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) GetN } // Used to increment the sequence in the mapping to have the next usable one available for the next transaction -func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) IncrementNextSequence(address ADDR) error { +func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) IncrementNextSequence(address ADDR, seq SEQ) { eb.sequenceLock.Lock() defer eb.sequenceLock.Unlock() - // Get next sequence from map - seq, exists := eb.nextSequenceMap[address] - if !exists { - return errors.New(fmt.Sprint("address not found in next sequence map: ", address)) - } eb.nextSequenceMap[address] = seq.Next() - return nil } // Used to set the next sequence explicitly to a certain value -func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) SetNextSequence(address ADDR, seq SEQ) error { +func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) SetNextSequence(address ADDR, seq SEQ) { eb.sequenceLock.Lock() defer eb.sequenceLock.Unlock() - // Get next sequence from map - _, exists := eb.nextSequenceMap[address] - if !exists { - return errors.New(fmt.Sprint("address not found in next sequence map: ", address)) - } eb.nextSequenceMap[address] = seq - return nil } func observeTimeUntilBroadcast[CHAIN_ID types.ID](chainID CHAIN_ID, createdAt, broadcastAt time.Time) { diff --git a/common/txmgr/nonce_syncer.go b/common/txmgr/sequence_syncer.go similarity index 100% rename from common/txmgr/nonce_syncer.go rename to common/txmgr/sequence_syncer.go diff --git a/common/txmgr/types/mocks/tx_store.go b/common/txmgr/types/mocks/tx_store.go index 6a935392ba6..f99797c55c3 100644 --- a/common/txmgr/types/mocks/tx_store.go +++ b/common/txmgr/types/mocks/tx_store.go @@ -695,13 +695,13 @@ func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) UpdateBroad return r0 } -// UpdateTxAttemptInProgressToBroadcast provides a mock function with given fields: ctx, etx, attempt, NewAttemptState, incrementSeqFunc -func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) UpdateTxAttemptInProgressToBroadcast(ctx context.Context, etx *txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], attempt txmgrtypes.TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], NewAttemptState txmgrtypes.TxAttemptState, incrementSeqFunc func(ADDR) error) error { - ret := _m.Called(ctx, etx, attempt, NewAttemptState, incrementSeqFunc) +// UpdateTxAttemptInProgressToBroadcast provides a mock function with given fields: ctx, etx, attempt, NewAttemptState +func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) UpdateTxAttemptInProgressToBroadcast(ctx context.Context, etx *txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], attempt txmgrtypes.TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], NewAttemptState txmgrtypes.TxAttemptState) error { + ret := _m.Called(ctx, etx, attempt, NewAttemptState) var r0 error - if rf, ok := ret.Get(0).(func(context.Context, *txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], txmgrtypes.TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], txmgrtypes.TxAttemptState, func(ADDR) error) error); ok { - r0 = rf(ctx, etx, attempt, NewAttemptState, incrementSeqFunc) + if rf, ok := ret.Get(0).(func(context.Context, *txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], txmgrtypes.TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], txmgrtypes.TxAttemptState) error); ok { + r0 = rf(ctx, etx, attempt, NewAttemptState) } else { r0 = ret.Error(0) } diff --git a/common/txmgr/types/tx_store.go b/common/txmgr/types/tx_store.go index 50abe90c902..f2b1dec8991 100644 --- a/common/txmgr/types/tx_store.go +++ b/common/txmgr/types/tx_store.go @@ -84,7 +84,7 @@ type TransactionStore[ SaveSentAttempt(ctx context.Context, timeout time.Duration, attempt *TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], broadcastAt time.Time) error SetBroadcastBeforeBlockNum(ctx context.Context, blockNum int64, chainID CHAIN_ID) error UpdateBroadcastAts(ctx context.Context, now time.Time, etxIDs []int64) error - UpdateTxAttemptInProgressToBroadcast(ctx context.Context, etx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], attempt TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], NewAttemptState TxAttemptState, incrementSeqFunc func(address ADDR) error) error + UpdateTxAttemptInProgressToBroadcast(ctx context.Context, etx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], attempt TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], NewAttemptState TxAttemptState) error UpdateTxsUnconfirmed(ctx context.Context, ids []int64) error UpdateTxUnstartedToInProgress(ctx context.Context, etx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], attempt *TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) error UpdateTxFatalError(ctx context.Context, etx *Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) error diff --git a/core/chains/evm/txmgr/broadcaster_test.go b/core/chains/evm/txmgr/broadcaster_test.go index f5f506cf6e8..8a7d6e0c1a2 100644 --- a/core/chains/evm/txmgr/broadcaster_test.go +++ b/core/chains/evm/txmgr/broadcaster_test.go @@ -1718,10 +1718,12 @@ func TestEthBroadcaster_IncrementNextNonce(t *testing.T) { ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(0), nil).Once() eb := NewTestEthBroadcaster(t, txStore, ethClient, kst, evmcfg, &testCheckerFactory{}, false) - require.NoError(t, eb.IncrementNextSequence(fromAddress)) + nonce, err := eb.GetNextSequence(fromAddress) + require.NoError(t, err) + eb.IncrementNextSequence(fromAddress, nonce) // Nonce bumped to 1 - nonce, err := eb.GetNextSequence(fromAddress) + nonce, err = eb.GetNextSequence(fromAddress) require.NoError(t, err) require.Equal(t, int64(1), int64(nonce)) } @@ -1923,31 +1925,24 @@ func Test_IncrementNextNonce(t *testing.T) { ethClient.On("PendingNonceAt", mock.Anything, addr1).Return(uint64(randNonce), nil).Once() eb := NewTestEthBroadcaster(t, txStore, ethClient, ks, evmcfg, checkerFactory, false) - err := eb.IncrementNextSequence(addr1) + nonce, err := eb.GetNextSequence(addr1) require.NoError(t, err) + eb.IncrementNextSequence(addr1, nonce) - var nonce types.Nonce nonce, err = eb.GetNextSequence(addr1) require.NoError(t, err) assert.Equal(t, randNonce+1, int64(nonce)) - err = eb.IncrementNextSequence(addr1) + eb.IncrementNextSequence(addr1, nonce) + nonce, err = eb.GetNextSequence(addr1) require.NoError(t, err) + assert.Equal(t, randNonce+2, int64(nonce)) randAddr1 := utils.RandomAddress() - err = eb.IncrementNextSequence(randAddr1) - require.Error(t, err) - assert.Contains(t, err.Error(), fmt.Sprintf("address not found in next sequence map: %s", randAddr1.Hex())) - _, err = eb.GetNextSequence(randAddr1) require.Error(t, err) assert.Contains(t, err.Error(), fmt.Sprintf("address not found in next sequence map: %s", randAddr1.Hex())) - randAddr2 := utils.RandomAddress() - err = eb.IncrementNextSequence(randAddr2) - require.Error(t, err) - assert.Contains(t, err.Error(), fmt.Sprintf("address not found in next sequence map: %s", randAddr2.Hex())) - // verify it didnt get changed by any erroring calls nonce, err = eb.GetNextSequence(addr1) require.NoError(t, err) @@ -1973,19 +1968,12 @@ func Test_SetNextNonce(t *testing.T) { nonce, err := eb.GetNextSequence(fromAddress) require.NoError(t, err) assert.Equal(t, int64(0), int64(nonce)) - err = eb.SetNextSequence(fromAddress, evmtypes.Nonce(24)) - require.NoError(t, err) + eb.SetNextSequence(fromAddress, evmtypes.Nonce(24)) newNextNonce, err := eb.GetNextSequence(fromAddress) require.NoError(t, err) assert.Equal(t, int64(24), int64(newNextNonce)) }) - - t.Run("address not found", func(t *testing.T) { - address := utils.RandomAddress() - err := eb.SetNextSequence(address, evmtypes.Nonce(100)) - require.Error(t, err) - }) } type testCheckerFactory struct { diff --git a/core/chains/evm/txmgr/evm_tx_store.go b/core/chains/evm/txmgr/evm_tx_store.go index 8ebe768c751..7f065239d07 100644 --- a/core/chains/evm/txmgr/evm_tx_store.go +++ b/core/chains/evm/txmgr/evm_tx_store.go @@ -1425,7 +1425,7 @@ func (o *evmTxStore) UpdateTxFatalError(ctx context.Context, etx *Tx) error { // Updates eth attempt from in_progress to broadcast. Also updates the eth tx to unconfirmed. // One of the more complicated signatures. We have to accept variable pg.QOpt and QueryerFunc arguments -func (o *evmTxStore) UpdateTxAttemptInProgressToBroadcast(ctx context.Context, etx *Tx, attempt TxAttempt, NewAttemptState txmgrtypes.TxAttemptState, incrementSeqFunc func(address common.Address) error) error { +func (o *evmTxStore) UpdateTxAttemptInProgressToBroadcast(ctx context.Context, etx *Tx, attempt TxAttempt, NewAttemptState txmgrtypes.TxAttemptState) error { var cancel context.CancelFunc ctx, cancel = o.mergeContexts(ctx) defer cancel() @@ -1459,9 +1459,6 @@ func (o *evmTxStore) UpdateTxAttemptInProgressToBroadcast(ctx context.Context, e if err := tx.Get(&dbAttempt, `UPDATE evm.tx_attempts SET state = $1 WHERE id = $2 RETURNING *`, dbAttempt.State, dbAttempt.ID); err != nil { return pkgerrors.Wrap(err, "SaveEthTxAttempt failed to save eth_tx_attempt") } - if err := incrementSeqFunc(etx.FromAddress); err != nil { - return pkgerrors.Wrap(err, "IncrementNextSequence failed to increment the next sequence in the broadcaster") - } return nil }) } diff --git a/core/chains/evm/txmgr/evm_tx_store_test.go b/core/chains/evm/txmgr/evm_tx_store_test.go index 266f3ef64d8..e1bf4880b2f 100644 --- a/core/chains/evm/txmgr/evm_tx_store_test.go +++ b/core/chains/evm/txmgr/evm_tx_store_test.go @@ -24,7 +24,6 @@ import ( "github.com/smartcontractkit/chainlink/v2/core/store/models" "github.com/smartcontractkit/chainlink/v2/core/utils" - "github.com/ethereum/go-ethereum/common" "github.com/google/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" @@ -1211,11 +1210,10 @@ func TestORM_UpdateTxAttemptInProgressToBroadcast(t *testing.T) { i := int16(0) etx.BroadcastAt = &time1 etx.InitialBroadcastAt = &time1 - err := txStore.UpdateTxAttemptInProgressToBroadcast(testutils.Context(t), &etx, attempt, txmgrtypes.TxAttemptBroadcast, func(_ common.Address) error { - i++ - return nil - }) + err := txStore.UpdateTxAttemptInProgressToBroadcast(testutils.Context(t), &etx, attempt, txmgrtypes.TxAttemptBroadcast) require.NoError(t, err) + // Increment sequence + i++ attemptResult, err := txStore.FindTxAttempt(attempt.Hash) require.NoError(t, err) diff --git a/core/chains/evm/txmgr/mocks/evm_tx_store.go b/core/chains/evm/txmgr/mocks/evm_tx_store.go index 57012fcfc55..93726c3f98a 100644 --- a/core/chains/evm/txmgr/mocks/evm_tx_store.go +++ b/core/chains/evm/txmgr/mocks/evm_tx_store.go @@ -900,13 +900,13 @@ func (_m *EvmTxStore) UpdateBroadcastAts(ctx context.Context, now time.Time, etx return r0 } -// UpdateTxAttemptInProgressToBroadcast provides a mock function with given fields: ctx, etx, attempt, NewAttemptState, incrementSeqFunc -func (_m *EvmTxStore) UpdateTxAttemptInProgressToBroadcast(ctx context.Context, etx *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], attempt types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], NewAttemptState types.TxAttemptState, incrementSeqFunc func(common.Address) error) error { - ret := _m.Called(ctx, etx, attempt, NewAttemptState, incrementSeqFunc) +// UpdateTxAttemptInProgressToBroadcast provides a mock function with given fields: ctx, etx, attempt, NewAttemptState +func (_m *EvmTxStore) UpdateTxAttemptInProgressToBroadcast(ctx context.Context, etx *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], attempt types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], NewAttemptState types.TxAttemptState) error { + ret := _m.Called(ctx, etx, attempt, NewAttemptState) var r0 error - if rf, ok := ret.Get(0).(func(context.Context, *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], types.TxAttemptState, func(common.Address) error) error); ok { - r0 = rf(ctx, etx, attempt, NewAttemptState, incrementSeqFunc) + if rf, ok := ret.Get(0).(func(context.Context, *types.Tx[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], types.TxAttempt[*big.Int, common.Address, common.Hash, common.Hash, evmtypes.Nonce, gas.EvmFee], types.TxAttemptState) error); ok { + r0 = rf(ctx, etx, attempt, NewAttemptState) } else { r0 = ret.Error(0) } From bb5c3b6afb26777af0e5cfedb2c25ceea23fb946 Mon Sep 17 00:00:00 2001 From: amit-momin Date: Tue, 10 Oct 2023 12:37:58 -0500 Subject: [PATCH 6/9] Updated to pass the next seq func as a parameter to the broadcaster --- common/txmgr/broadcaster.go | 19 +++++++++++-------- common/txmgr/confirmer.go | 8 ++++---- common/txmgr/mocks/tx_manager.go | 4 ++-- common/txmgr/resender.go | 6 +++--- common/txmgr/sequence_syncer.go | 2 +- common/txmgr/txmgr.go | 8 ++++---- common/txmgr/types/client.go | 6 +++--- common/txmgr/types/keystore.go | 2 +- common/txmgr/types/mocks/key_store.go | 4 ++-- .../txmgr/types/mocks/tx_attempt_builder.go | 4 ++-- common/txmgr/types/mocks/tx_store.go | 4 ++-- common/txmgr/types/tx.go | 4 ++-- common/txmgr/types/tx_attempt_builder.go | 2 +- common/txmgr/types/tx_store.go | 4 ++-- common/types/chain.go | 6 ++++-- core/chains/evm/txmgr/builder.go | 2 +- core/chains/evm/types/nonce.go | 4 ++-- 17 files changed, 47 insertions(+), 42 deletions(-) diff --git a/common/txmgr/broadcaster.go b/common/txmgr/broadcaster.go index 33c0c20a640..a587a3b3c10 100644 --- a/common/txmgr/broadcaster.go +++ b/common/txmgr/broadcaster.go @@ -60,7 +60,7 @@ type TransmitCheckerFactory[ CHAIN_ID types.ID, ADDR types.Hashable, TX_HASH, BLOCK_HASH types.Hashable, - SEQ types.Sequence[SEQ], + SEQ types.Sequence, FEE feetypes.Fee, ] interface { // BuildChecker builds a new TransmitChecker based on the given spec. @@ -72,7 +72,7 @@ type TransmitChecker[ CHAIN_ID types.ID, ADDR types.Hashable, TX_HASH, BLOCK_HASH types.Hashable, - SEQ types.Sequence[SEQ], + SEQ types.Sequence, FEE feetypes.Fee, ] interface { @@ -102,7 +102,7 @@ type Broadcaster[ ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, - SEQ types.Sequence[SEQ], + SEQ types.Sequence, FEE feetypes.Fee, ] struct { logger logger.Logger @@ -144,8 +144,9 @@ type Broadcaster[ parseAddr func(string) (ADDR, error) - sequenceLock *sync.RWMutex - nextSequenceMap map[ADDR]SEQ + sequenceLock *sync.RWMutex + nextSequenceMap map[ADDR]SEQ + generateNextSequence types.GenerateNextSequenceFunc[SEQ] } func NewBroadcaster[ @@ -154,7 +155,7 @@ func NewBroadcaster[ ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, - SEQ types.Sequence[SEQ], + SEQ types.Sequence, FEE feetypes.Fee, ]( txStore txmgrtypes.TransactionStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, SEQ, FEE], @@ -171,6 +172,7 @@ func NewBroadcaster[ checkerFactory TransmitCheckerFactory[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], autoSyncSequence bool, parseAddress func(string) (ADDR, error), + generateNextSequence types.GenerateNextSequenceFunc[SEQ], ) *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] { logger = logger.Named("Broadcaster") b := &Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]{ @@ -193,6 +195,7 @@ func NewBroadcaster[ } b.processUnstartedTxsImpl = b.processUnstartedTxs + b.generateNextSequence = generateNextSequence return b } @@ -343,7 +346,7 @@ func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) load nextSequenceMap[address] = seq } else { - nextSequenceMap[address] = seq.Next() + nextSequenceMap[address] = eb.generateNextSequence(seq) } } @@ -839,7 +842,7 @@ func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) GetN func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) IncrementNextSequence(address ADDR, seq SEQ) { eb.sequenceLock.Lock() defer eb.sequenceLock.Unlock() - eb.nextSequenceMap[address] = seq.Next() + eb.nextSequenceMap[address] = eb.generateNextSequence(seq) } // Used to set the next sequence explicitly to a certain value diff --git a/common/txmgr/confirmer.go b/common/txmgr/confirmer.go index 155e324ad9b..31bba771410 100644 --- a/common/txmgr/confirmer.go +++ b/common/txmgr/confirmer.go @@ -109,7 +109,7 @@ type Confirmer[ TX_HASH types.Hashable, BLOCK_HASH types.Hashable, R txmgrtypes.ChainReceipt[TX_HASH, BLOCK_HASH], - SEQ types.Sequence[SEQ], + SEQ types.Sequence, FEE feetypes.Fee, ] struct { utils.StartStopOnce @@ -145,7 +145,7 @@ func NewConfirmer[ TX_HASH types.Hashable, BLOCK_HASH types.Hashable, R txmgrtypes.ChainReceipt[TX_HASH, BLOCK_HASH], - SEQ types.Sequence[SEQ], + SEQ types.Sequence, FEE feetypes.Fee, ]( txStore txmgrtypes.TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE], @@ -961,7 +961,7 @@ func hasReceiptInLongestChain[ CHAIN_ID types.ID, ADDR types.Hashable, TX_HASH, BLOCK_HASH types.Hashable, - SEQ types.Sequence[SEQ], + SEQ types.Sequence, FEE feetypes.Fee, ](etx txmgrtypes.Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], head types.Head[BLOCK_HASH]) bool { for { @@ -1118,7 +1118,7 @@ func observeUntilTxConfirmed[ ADDR types.Hashable, TX_HASH, BLOCK_HASH types.Hashable, R txmgrtypes.ChainReceipt[TX_HASH, BLOCK_HASH], - SEQ types.Sequence[SEQ], + SEQ types.Sequence, FEE feetypes.Fee, ](chainID CHAIN_ID, attempts []txmgrtypes.TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], receipts []R) { for _, attempt := range attempts { diff --git a/common/txmgr/mocks/tx_manager.go b/common/txmgr/mocks/tx_manager.go index be593a4954b..caf204103cf 100644 --- a/common/txmgr/mocks/tx_manager.go +++ b/common/txmgr/mocks/tx_manager.go @@ -17,7 +17,7 @@ import ( ) // TxManager is an autogenerated mock type for the TxManager type -type TxManager[CHAIN_ID types.ID, HEAD types.Head[BLOCK_HASH], ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, SEQ types.Sequence[SEQ], FEE feetypes.Fee] struct { +type TxManager[CHAIN_ID types.ID, HEAD types.Head[BLOCK_HASH], ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, SEQ types.Sequence, FEE feetypes.Fee] struct { mock.Mock } @@ -200,7 +200,7 @@ type mockConstructorTestingTNewTxManager interface { } // NewTxManager creates a new instance of TxManager. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. -func NewTxManager[CHAIN_ID types.ID, HEAD types.Head[BLOCK_HASH], ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, SEQ types.Sequence[SEQ], FEE feetypes.Fee](t mockConstructorTestingTNewTxManager) *TxManager[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] { +func NewTxManager[CHAIN_ID types.ID, HEAD types.Head[BLOCK_HASH], ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, SEQ types.Sequence, FEE feetypes.Fee](t mockConstructorTestingTNewTxManager) *TxManager[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] { mock := &TxManager[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]{} mock.Mock.Test(t) diff --git a/common/txmgr/resender.go b/common/txmgr/resender.go index 2b3324b34ce..655de0f1135 100644 --- a/common/txmgr/resender.go +++ b/common/txmgr/resender.go @@ -40,7 +40,7 @@ type Resender[ ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, - SEQ types.Sequence[SEQ], + SEQ types.Sequence, FEE feetypes.Fee, ] struct { txStore txmgrtypes.TransactionStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, SEQ, FEE] @@ -63,7 +63,7 @@ func NewResender[ ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, - SEQ types.Sequence[SEQ], + SEQ types.Sequence, FEE feetypes.Fee, ]( lggr logger.Logger, @@ -207,7 +207,7 @@ func findOldestUnconfirmedAttempt[ CHAIN_ID types.ID, ADDR types.Hashable, TX_HASH, BLOCK_HASH types.Hashable, - SEQ types.Sequence[SEQ], + SEQ types.Sequence, FEE feetypes.Fee, ](attempts []txmgrtypes.TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) (txmgrtypes.TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], bool) { var oldestAttempt txmgrtypes.TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] diff --git a/common/txmgr/sequence_syncer.go b/common/txmgr/sequence_syncer.go index f3f62d08d3f..b8f97a5c94b 100644 --- a/common/txmgr/sequence_syncer.go +++ b/common/txmgr/sequence_syncer.go @@ -6,6 +6,6 @@ import ( "github.com/smartcontractkit/chainlink/v2/common/types" ) -type SequenceSyncer[ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, SEQ types.Sequence[SEQ]] interface { +type SequenceSyncer[ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, SEQ types.Sequence] interface { Sync(ctx context.Context, addr ADDR, localNonce SEQ) (SEQ, error) } diff --git a/common/txmgr/txmgr.go b/common/txmgr/txmgr.go index a47743c891a..feb3218bfc6 100644 --- a/common/txmgr/txmgr.go +++ b/common/txmgr/txmgr.go @@ -35,7 +35,7 @@ type TxManager[ ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, - SEQ types.Sequence[SEQ], + SEQ types.Sequence, FEE feetypes.Fee, ] interface { types.HeadTrackable[HEAD, BLOCK_HASH] @@ -64,7 +64,7 @@ type Txm[ TX_HASH types.Hashable, BLOCK_HASH types.Hashable, R txmgrtypes.ChainReceipt[TX_HASH, BLOCK_HASH], - SEQ types.Sequence[SEQ], + SEQ types.Sequence, FEE feetypes.Fee, ] struct { utils.StartStopOnce @@ -108,7 +108,7 @@ func NewTxm[ TX_HASH types.Hashable, BLOCK_HASH types.Hashable, R txmgrtypes.ChainReceipt[TX_HASH, BLOCK_HASH], - SEQ types.Sequence[SEQ], + SEQ types.Sequence, FEE feetypes.Fee, ]( chainId CHAIN_ID, @@ -515,7 +515,7 @@ type NullTxManager[ HEAD types.Head[BLOCK_HASH], ADDR types.Hashable, TX_HASH, BLOCK_HASH types.Hashable, - SEQ types.Sequence[SEQ], + SEQ types.Sequence, FEE feetypes.Fee, ] struct { ErrMsg string diff --git a/common/txmgr/types/client.go b/common/txmgr/types/client.go index c4f445b0560..6d7f1c55558 100644 --- a/common/txmgr/types/client.go +++ b/common/txmgr/types/client.go @@ -19,7 +19,7 @@ type TxmClient[ TX_HASH types.Hashable, BLOCK_HASH types.Hashable, R ChainReceipt[TX_HASH, BLOCK_HASH], - SEQ types.Sequence[SEQ], + SEQ types.Sequence, FEE feetypes.Fee, ] interface { ChainClient[CHAIN_ID, ADDR, SEQ] @@ -38,7 +38,7 @@ type TransactionClient[ ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, - SEQ types.Sequence[SEQ], + SEQ types.Sequence, FEE feetypes.Fee, ] interface { ChainClient[CHAIN_ID, ADDR, SEQ] @@ -79,7 +79,7 @@ type TransactionClient[ type ChainClient[ CHAIN_ID types.ID, ADDR types.Hashable, - SEQ types.Sequence[SEQ], + SEQ types.Sequence, ] interface { ConfiguredChainID() CHAIN_ID PendingSequenceAt(ctx context.Context, addr ADDR) (SEQ, error) diff --git a/common/txmgr/types/keystore.go b/common/txmgr/types/keystore.go index dc37e787a39..9c5b8cfce37 100644 --- a/common/txmgr/types/keystore.go +++ b/common/txmgr/types/keystore.go @@ -13,7 +13,7 @@ type KeyStore[ // Chain ID type CHAIN_ID types.ID, // Chain's sequence type. For example, EVM chains use nonce, bitcoin uses UTXO. - SEQ types.Sequence[SEQ], + SEQ types.Sequence, ] interface { CheckEnabled(address ADDR, chainID CHAIN_ID) error EnabledAddressesForChain(chainId CHAIN_ID) ([]ADDR, error) diff --git a/common/txmgr/types/mocks/key_store.go b/common/txmgr/types/mocks/key_store.go index ea317990270..e4189c1dd00 100644 --- a/common/txmgr/types/mocks/key_store.go +++ b/common/txmgr/types/mocks/key_store.go @@ -9,7 +9,7 @@ import ( ) // KeyStore is an autogenerated mock type for the KeyStore type -type KeyStore[ADDR types.Hashable, CHAIN_ID types.ID, SEQ types.Sequence[SEQ]] struct { +type KeyStore[ADDR types.Hashable, CHAIN_ID types.ID, SEQ types.Sequence] struct { mock.Mock } @@ -87,7 +87,7 @@ type mockConstructorTestingTNewKeyStore interface { } // NewKeyStore creates a new instance of KeyStore. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. -func NewKeyStore[ADDR types.Hashable, CHAIN_ID types.ID, SEQ types.Sequence[SEQ]](t mockConstructorTestingTNewKeyStore) *KeyStore[ADDR, CHAIN_ID, SEQ] { +func NewKeyStore[ADDR types.Hashable, CHAIN_ID types.ID, SEQ types.Sequence](t mockConstructorTestingTNewKeyStore) *KeyStore[ADDR, CHAIN_ID, SEQ] { mock := &KeyStore[ADDR, CHAIN_ID, SEQ]{} mock.Mock.Test(t) diff --git a/common/txmgr/types/mocks/tx_attempt_builder.go b/common/txmgr/types/mocks/tx_attempt_builder.go index 51f61fdf5b9..cda71cfae09 100644 --- a/common/txmgr/types/mocks/tx_attempt_builder.go +++ b/common/txmgr/types/mocks/tx_attempt_builder.go @@ -16,7 +16,7 @@ import ( ) // TxAttemptBuilder is an autogenerated mock type for the TxAttemptBuilder type -type TxAttemptBuilder[CHAIN_ID types.ID, HEAD types.Head[BLOCK_HASH], ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, SEQ types.Sequence[SEQ], FEE feetypes.Fee] struct { +type TxAttemptBuilder[CHAIN_ID types.ID, HEAD types.Head[BLOCK_HASH], ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, SEQ types.Sequence, FEE feetypes.Fee] struct { mock.Mock } @@ -307,7 +307,7 @@ type mockConstructorTestingTNewTxAttemptBuilder interface { } // NewTxAttemptBuilder creates a new instance of TxAttemptBuilder. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. -func NewTxAttemptBuilder[CHAIN_ID types.ID, HEAD types.Head[BLOCK_HASH], ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, SEQ types.Sequence[SEQ], FEE feetypes.Fee](t mockConstructorTestingTNewTxAttemptBuilder) *TxAttemptBuilder[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] { +func NewTxAttemptBuilder[CHAIN_ID types.ID, HEAD types.Head[BLOCK_HASH], ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, SEQ types.Sequence, FEE feetypes.Fee](t mockConstructorTestingTNewTxAttemptBuilder) *TxAttemptBuilder[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE] { mock := &TxAttemptBuilder[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]{} mock.Mock.Test(t) diff --git a/common/txmgr/types/mocks/tx_store.go b/common/txmgr/types/mocks/tx_store.go index f99797c55c3..6864e43fd81 100644 --- a/common/txmgr/types/mocks/tx_store.go +++ b/common/txmgr/types/mocks/tx_store.go @@ -18,7 +18,7 @@ import ( ) // TxStore is an autogenerated mock type for the TxStore type -type TxStore[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[SEQ], FEE feetypes.Fee] struct { +type TxStore[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.Mock } @@ -771,7 +771,7 @@ type mockConstructorTestingTNewTxStore interface { } // NewTxStore creates a new instance of TxStore. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. -func NewTxStore[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[SEQ], FEE feetypes.Fee](t mockConstructorTestingTNewTxStore) *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { +func NewTxStore[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](t mockConstructorTestingTNewTxStore) *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE] { mock := &TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]{} mock.Mock.Test(t) diff --git a/common/txmgr/types/tx.go b/common/txmgr/types/tx.go index b42064284f2..d95f07afabc 100644 --- a/common/txmgr/types/tx.go +++ b/common/txmgr/types/tx.go @@ -156,7 +156,7 @@ type TxAttempt[ CHAIN_ID types.ID, ADDR types.Hashable, TX_HASH, BLOCK_HASH types.Hashable, - SEQ types.Sequence[SEQ], + SEQ types.Sequence, FEE feetypes.Fee, ] struct { ID int64 @@ -182,7 +182,7 @@ type Tx[ CHAIN_ID types.ID, ADDR types.Hashable, TX_HASH, BLOCK_HASH types.Hashable, - SEQ types.Sequence[SEQ], + SEQ types.Sequence, FEE feetypes.Fee, ] struct { ID int64 diff --git a/common/txmgr/types/tx_attempt_builder.go b/common/txmgr/types/tx_attempt_builder.go index 3efcfd01eb1..887219c490e 100644 --- a/common/txmgr/types/tx_attempt_builder.go +++ b/common/txmgr/types/tx_attempt_builder.go @@ -19,7 +19,7 @@ type TxAttemptBuilder[ HEAD types.Head[BLOCK_HASH], // HEAD - chain head type ADDR types.Hashable, // ADDR - chain address type TX_HASH, BLOCK_HASH types.Hashable, // various chain hash types - SEQ types.Sequence[SEQ], // SEQ - chain sequence type (nonce, utxo, etc) + SEQ types.Sequence, // SEQ - chain sequence type (nonce, utxo, etc) FEE feetypes.Fee, // FEE - chain fee type ] interface { // interfaces for running the underlying estimator diff --git a/common/txmgr/types/tx_store.go b/common/txmgr/types/tx_store.go index f2b1dec8991..b6646d02db4 100644 --- a/common/txmgr/types/tx_store.go +++ b/common/txmgr/types/tx_store.go @@ -27,7 +27,7 @@ type TxStore[ // Represents a onchain receipt object that a chain's RPC returns R ChainReceipt[TX_HASH, BLOCK_HASH], // Represents the sequence type for a chain. For example, nonce for EVM. - SEQ types.Sequence[SEQ], + SEQ types.Sequence, // Represents the chain specific fee type FEE feetypes.Fee, ] interface { @@ -51,7 +51,7 @@ type TransactionStore[ CHAIN_ID types.ID, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, - SEQ types.Sequence[SEQ], + SEQ types.Sequence, FEE feetypes.Fee, ] interface { CountUnconfirmedTransactions(ctx context.Context, fromAddress ADDR, chainID CHAIN_ID) (count uint32, err error) diff --git a/common/types/chain.go b/common/types/chain.go index 319e963b732..c2c2011d8de 100644 --- a/common/types/chain.go +++ b/common/types/chain.go @@ -4,12 +4,14 @@ import "fmt" // Sequence represents the base type, for any chain's sequence object. // It should be convertible to a string -type Sequence[SEQ any] interface { +type Sequence interface { fmt.Stringer Int64() int64 // needed for numeric sequence confirmation - to be removed with confirmation logic generalization: https://smartcontract-it.atlassian.net/browse/BCI-860 - Next() SEQ // needed to produce the next usable sequence for a transaction } +// Generate the next usable sequence for a transaction +type GenerateNextSequenceFunc[SEQ Sequence] func(prev SEQ) SEQ + // ID represents the base type, for any chain's ID. // It should be convertible to a string, that can uniquely identify this chain type ID fmt.Stringer diff --git a/core/chains/evm/txmgr/builder.go b/core/chains/evm/txmgr/builder.go index 8d55b5790f8..39781e83f4c 100644 --- a/core/chains/evm/txmgr/builder.go +++ b/core/chains/evm/txmgr/builder.go @@ -130,5 +130,5 @@ func NewEvmBroadcaster( checkerFactory TransmitCheckerFactory, autoSyncNonce bool, ) *Broadcaster { - return txmgr.NewBroadcaster(txStore, client, chainConfig, feeConfig, txConfig, listenerConfig, keystore, eventBroadcaster, txAttemptBuilder, nonceSyncer, logger, checkerFactory, autoSyncNonce, stringToGethAddress) + return txmgr.NewBroadcaster(txStore, client, chainConfig, feeConfig, txConfig, listenerConfig, keystore, eventBroadcaster, txAttemptBuilder, nonceSyncer, logger, checkerFactory, autoSyncNonce, stringToGethAddress, evmtypes.GenerateNextNonce) } diff --git a/core/chains/evm/types/nonce.go b/core/chains/evm/types/nonce.go index 8ed4b50e72d..e9caf98c763 100644 --- a/core/chains/evm/types/nonce.go +++ b/core/chains/evm/types/nonce.go @@ -18,6 +18,6 @@ func (n Nonce) String() string { return strconv.FormatInt(n.Int64(), 10) } -func (n Nonce) Next() Nonce { - return Nonce(n + 1) +func GenerateNextNonce(prev Nonce) Nonce { + return Nonce(prev + 1) } From a840f0033aea70c9b9458020bf4de64703edadfe Mon Sep 17 00:00:00 2001 From: amit-momin Date: Tue, 10 Oct 2023 17:26:43 -0500 Subject: [PATCH 7/9] Removed unnecessary mutex initialization --- common/txmgr/broadcaster.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/common/txmgr/broadcaster.go b/common/txmgr/broadcaster.go index a587a3b3c10..5f7e6d8c9bd 100644 --- a/common/txmgr/broadcaster.go +++ b/common/txmgr/broadcaster.go @@ -144,7 +144,7 @@ type Broadcaster[ parseAddr func(string) (ADDR, error) - sequenceLock *sync.RWMutex + sequenceLock sync.RWMutex nextSequenceMap map[ADDR]SEQ generateNextSequence types.GenerateNextSequenceFunc[SEQ] } @@ -191,7 +191,6 @@ func NewBroadcaster[ checkerFactory: checkerFactory, autoSyncSequence: autoSyncSequence, parseAddr: parseAddress, - sequenceLock: &sync.RWMutex{}, } b.processUnstartedTxsImpl = b.processUnstartedTxs From dfac79729c2f93fa110fb4f8d0577d630123d575 Mon Sep 17 00:00:00 2001 From: amit-momin Date: Wed, 11 Oct 2023 15:44:37 -0500 Subject: [PATCH 8/9] Addressed PR feedback --- common/txmgr/broadcaster.go | 4 +- common/txmgr/sequence_syncer.go | 2 +- core/chains/evm/txmgr/broadcaster_test.go | 51 ++++++++++++++++--- core/chains/evm/txmgr/evm_tx_store.go | 5 +- core/chains/evm/txmgr/nonce_syncer_test.go | 2 +- core/chains/evm/txmgr/txmgr_test.go | 2 - core/web/eth_keys_controller_test.go | 1 - ...ITECTURE.md => TX_MANAGER_ARCHITECTURE.md} | 0 8 files changed, 51 insertions(+), 16 deletions(-) rename docs/core/{BULLETPROOF_TX_MANAGER_ARCHITECTURE.md => TX_MANAGER_ARCHITECTURE.md} (100%) diff --git a/common/txmgr/broadcaster.go b/common/txmgr/broadcaster.go index 5f7e6d8c9bd..3cb86ffa046 100644 --- a/common/txmgr/broadcaster.go +++ b/common/txmgr/broadcaster.go @@ -245,7 +245,7 @@ func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) star defer eb.sequenceLock.Unlock() eb.nextSequenceMap, err = eb.loadNextSequenceMap(eb.enabledAddresses) if err != nil { - return err + return errors.Wrap(err, "Broadcaster: failed to load next sequence map") } eb.isStarted = true @@ -336,7 +336,7 @@ func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) load // Will need to be incremented since this sequence is already used seq, err := eb.txStore.FindHighestSequence(ctx, address, eb.chainID) if err != nil { - // Look for nonce on-chain if address not found in TxStore + // Look for nonce on-chain if no tx found for address in TxStore or if error occurred // Returns the nonce that should be used for the next transaction so no need to increment seq, err = eb.client.PendingSequenceAt(ctx, address) if err != nil { diff --git a/common/txmgr/sequence_syncer.go b/common/txmgr/sequence_syncer.go index b8f97a5c94b..dd4d458dd74 100644 --- a/common/txmgr/sequence_syncer.go +++ b/common/txmgr/sequence_syncer.go @@ -7,5 +7,5 @@ import ( ) type SequenceSyncer[ADDR types.Hashable, TX_HASH types.Hashable, BLOCK_HASH types.Hashable, SEQ types.Sequence] interface { - Sync(ctx context.Context, addr ADDR, localNonce SEQ) (SEQ, error) + Sync(ctx context.Context, addr ADDR, localSequence SEQ) (SEQ, error) } diff --git a/core/chains/evm/txmgr/broadcaster_test.go b/core/chains/evm/txmgr/broadcaster_test.go index 8a7d6e0c1a2..3865604a49d 100644 --- a/core/chains/evm/txmgr/broadcaster_test.go +++ b/core/chains/evm/txmgr/broadcaster_test.go @@ -354,13 +354,12 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Success(t *testing.T) { c.EVM[0].GasEstimator.PriceMax = assets.NewWeiI(rnd + 2) }) evmcfg = evmtest.NewChainScopedConfig(t, cfg) - ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(3), nil).Once() ethClient.On("PendingNonceAt", mock.Anything, otherAddress).Return(uint64(1), nil).Once() eb = NewTestEthBroadcaster(t, txStore, ethClient, ethKeyStore, evmcfg, checkerFactory, false) t.Run("sends transactions with type 0x2 in EIP-1559 mode", func(t *testing.T) { ethClient.On("SendTransactionReturnCode", mock.Anything, mock.MatchedBy(func(tx *gethTypes.Transaction) bool { - return tx.Nonce() == uint64(3) && tx.Value().Cmp(big.NewInt(242)) == 0 + return tx.Nonce() == uint64(343) && tx.Value().Cmp(big.NewInt(242)) == 0 }), fromAddress).Return(clienttypes.Successful, nil).Once() etx := cltest.MustCreateUnstartedTx(t, txStore, fromAddress, toAddress, []byte{42, 42, 0}, gasLimit, big.Int(assets.NewEthValue(242)), &cltest.FixtureChainID) @@ -379,7 +378,7 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Success(t *testing.T) { require.NotNil(t, etx.FromAddress) assert.Equal(t, fromAddress, etx.FromAddress) require.NotNil(t, etx.Sequence) - assert.Equal(t, evmtypes.Nonce(3), *etx.Sequence) + assert.Equal(t, evmtypes.Nonce(343), *etx.Sequence) assert.NotNil(t, etx.BroadcastAt) assert.NotNil(t, etx.InitialBroadcastAt) assert.Len(t, etx.TxAttempts, 1) @@ -411,7 +410,7 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Success(t *testing.T) { }, } ethClient.On("SendTransactionReturnCode", mock.Anything, mock.MatchedBy(func(tx *gethTypes.Transaction) bool { - return tx.Nonce() == uint64(4) && tx.Value().Cmp(big.NewInt(442)) == 0 + return tx.Nonce() == uint64(344) && tx.Value().Cmp(big.NewInt(442)) == 0 }), fromAddress).Return(clienttypes.Successful, nil).Once() ethClient.On("CallContext", mock.Anything, mock.AnythingOfType("*hexutil.Bytes"), "eth_call", mock.MatchedBy(func(callarg map[string]interface{}) bool { if fmt.Sprintf("%s", callarg["value"]) == "0x1ba" { // 442 @@ -444,7 +443,7 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Success(t *testing.T) { t.Run("with unknown error, sends tx as normal", func(t *testing.T) { ethClient.On("SendTransactionReturnCode", mock.Anything, mock.MatchedBy(func(tx *gethTypes.Transaction) bool { - return tx.Nonce() == uint64(5) && tx.Value().Cmp(big.NewInt(542)) == 0 + return tx.Nonce() == uint64(345) && tx.Value().Cmp(big.NewInt(542)) == 0 }), fromAddress).Return(clienttypes.Successful, nil).Once() ethClient.On("CallContext", mock.Anything, mock.AnythingOfType("*hexutil.Bytes"), "eth_call", mock.MatchedBy(func(callarg map[string]interface{}) bool { return fmt.Sprintf("%s", callarg["value"]) == "0x21e" // 542 @@ -1604,7 +1603,6 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Errors(t *testing.T) { c.EVM[0].GasEstimator.TipCapDefault = gasTipCapDefault })) localNextNonce = getLocalNextNonce(t, eb, fromAddress) - ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(localNextNonce), nil).Once() eb2 = NewTestEthBroadcaster(t, txStore, ethClient, ethKeyStore, evmcfg2, &testCheckerFactory{}, false) // Second was underpriced but above minimum @@ -1876,6 +1874,46 @@ func TestEthBroadcaster_SyncNonce(t *testing.T) { } +func Test_LoadSequenceMap(t *testing.T) { + t.Parallel() + t.Run("set next nonce using entries from tx table", func(t *testing.T) { + db := pgtest.NewSqlxDB(t) + cfg := configtest.NewTestGeneralConfig(t) + txStore := cltest.NewTestTxStore(t, db, cfg.Database()) + ks := cltest.NewKeyStore(t, db, cfg.Database()).Eth() + + ethClient := evmtest.NewEthClientMockWithDefaultChain(t) + evmcfg := evmtest.NewChainScopedConfig(t, cfg) + checkerFactory := &txmgr.CheckerFactory{Client: ethClient} + _, fromAddress := cltest.MustInsertRandomKey(t, ks) + cltest.MustInsertUnconfirmedEthTx(t, txStore, int64(0), fromAddress) + cltest.MustInsertUnconfirmedEthTx(t, txStore, int64(1), fromAddress) + eb := NewTestEthBroadcaster(t, txStore, ethClient, ks, evmcfg, checkerFactory, false) + + nonce, err := eb.GetNextSequence(fromAddress) + require.NoError(t, err) + assert.Equal(t, int64(2), int64(nonce)) + }) + + t.Run("set next nonce using client when not found in tx table", func(t *testing.T) { + db := pgtest.NewSqlxDB(t) + cfg := configtest.NewTestGeneralConfig(t) + txStore := cltest.NewTestTxStore(t, db, cfg.Database()) + ks := cltest.NewKeyStore(t, db, cfg.Database()).Eth() + + ethClient := evmtest.NewEthClientMockWithDefaultChain(t) + evmcfg := evmtest.NewChainScopedConfig(t, cfg) + checkerFactory := &txmgr.CheckerFactory{Client: ethClient} + _, fromAddress := cltest.MustInsertRandomKey(t, ks) + ethClient.On("PendingNonceAt", mock.Anything, fromAddress).Return(uint64(10), nil).Once() + eb := NewTestEthBroadcaster(t, txStore, ethClient, ks, evmcfg, checkerFactory, false) + + nonce, err := eb.GetNextSequence(fromAddress) + require.NoError(t, err) + assert.Equal(t, int64(10), int64(nonce)) + }) +} + func Test_NextNonce(t *testing.T) { t.Parallel() @@ -1907,6 +1945,7 @@ func Test_NextNonce(t *testing.T) { _, err = eb.GetNextSequence(randAddr2) require.Error(t, err) assert.Contains(t, err.Error(), fmt.Sprintf("address not found in next sequence map: %s", randAddr2.Hex())) + } func Test_IncrementNextNonce(t *testing.T) { diff --git a/core/chains/evm/txmgr/evm_tx_store.go b/core/chains/evm/txmgr/evm_tx_store.go index 7f065239d07..4a76faa4e60 100644 --- a/core/chains/evm/txmgr/evm_tx_store.go +++ b/core/chains/evm/txmgr/evm_tx_store.go @@ -962,9 +962,8 @@ func (o *evmTxStore) FindHighestSequence(ctx context.Context, fromAddress common ctx, cancel = o.mergeContexts(ctx) defer cancel() qq := o.q.WithOpts(pg.WithParentCtx(ctx)) - - sql := `SELECT MAX(nonce) FROM evm.txes WHERE from_address = $1 and evm_chain_id = $2` - err = qq.Get(&nonce, sql, fromAddress.String(), chainId.String()) + sql := `SELECT nonce FROM evm.txes WHERE from_address = $1 AND evm_chain_id = $2 AND nonce IS NOT NULL ORDER BY nonce DESC LIMIT 1` + err = qq.Get(&nonce, sql, fromAddress, chainId.String()) return } diff --git a/core/chains/evm/txmgr/nonce_syncer_test.go b/core/chains/evm/txmgr/nonce_syncer_test.go index 9b6b1962365..13e5fd02e8c 100644 --- a/core/chains/evm/txmgr/nonce_syncer_test.go +++ b/core/chains/evm/txmgr/nonce_syncer_test.go @@ -28,7 +28,7 @@ func Test_NonceSyncer_Sync(t *testing.T) { ethClient := evmtest.NewEthClientMockWithDefaultChain(t) ethKeyStore := cltest.NewKeyStore(t, db, cfg.Database()).Eth() - _, from := cltest.MustInsertRandomKey(t, ethKeyStore) + _, from := cltest.MustInsertRandomKey(t, ethKeyStore) ns := txmgr.NewNonceSyncer(txStore, logger.TestLogger(t), ethClient) diff --git a/core/chains/evm/txmgr/txmgr_test.go b/core/chains/evm/txmgr/txmgr_test.go index ce173344859..5d2975e2db5 100644 --- a/core/chains/evm/txmgr/txmgr_test.go +++ b/core/chains/evm/txmgr/txmgr_test.go @@ -669,8 +669,6 @@ func TestTxm_Reset(t *testing.T) { } ethClient := evmtest.NewEthClientMockWithDefaultChain(t) - ethClient.On("PendingNonceAt", mock.Anything, addr).Return(uint64(0), nil) - ethClient.On("PendingNonceAt", mock.Anything, addr2).Return(uint64(0), nil) ethClient.On("HeadByNumber", mock.Anything, (*big.Int)(nil)).Return(nil, nil) ethClient.On("BatchCallContextAll", mock.Anything, mock.Anything).Return(nil).Maybe() eventBroadcaster := pgmocks.NewEventBroadcaster(t) diff --git a/core/web/eth_keys_controller_test.go b/core/web/eth_keys_controller_test.go index 98e7b3807de..4d69591999e 100644 --- a/core/web/eth_keys_controller_test.go +++ b/core/web/eth_keys_controller_test.go @@ -336,7 +336,6 @@ func TestETHKeysController_ChainSuccess_Enable(t *testing.T) { t.Parallel() ethClient := cltest.NewEthMocksWithStartupAssertions(t) - ethClient.On("PendingNonceAt", mock.Anything, mock.Anything).Return(uint64(0), nil) cfg := configtest.NewGeneralConfig(t, func(c *chainlink.Config, s *chainlink.Secrets) { c.EVM[0].NonceAutoSync = ptr(false) c.EVM[0].BalanceMonitor.Enabled = ptr(false) diff --git a/docs/core/BULLETPROOF_TX_MANAGER_ARCHITECTURE.md b/docs/core/TX_MANAGER_ARCHITECTURE.md similarity index 100% rename from docs/core/BULLETPROOF_TX_MANAGER_ARCHITECTURE.md rename to docs/core/TX_MANAGER_ARCHITECTURE.md From 93067b0294b93644ffb7f967c4e9b2ad9a0edea2 Mon Sep 17 00:00:00 2001 From: amit-momin Date: Wed, 11 Oct 2023 16:53:12 -0500 Subject: [PATCH 9/9] Changed FindHighestSequence method name to FindLatestSequence --- common/txmgr/broadcaster.go | 2 +- common/txmgr/types/mocks/tx_store.go | 4 ++-- common/txmgr/types/tx_store.go | 2 +- core/chains/evm/txmgr/evm_tx_store.go | 2 +- core/chains/evm/txmgr/mocks/evm_tx_store.go | 4 ++-- 5 files changed, 7 insertions(+), 7 deletions(-) diff --git a/common/txmgr/broadcaster.go b/common/txmgr/broadcaster.go index 3cb86ffa046..6512f67fe0b 100644 --- a/common/txmgr/broadcaster.go +++ b/common/txmgr/broadcaster.go @@ -334,7 +334,7 @@ func (eb *Broadcaster[CHAIN_ID, HEAD, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) load for _, address := range addresses { // Get the highest sequence from the tx table // Will need to be incremented since this sequence is already used - seq, err := eb.txStore.FindHighestSequence(ctx, address, eb.chainID) + seq, err := eb.txStore.FindLatestSequence(ctx, address, eb.chainID) if err != nil { // Look for nonce on-chain if no tx found for address in TxStore or if error occurred // Returns the nonce that should be used for the next transaction so no need to increment diff --git a/common/txmgr/types/mocks/tx_store.go b/common/txmgr/types/mocks/tx_store.go index 6864e43fd81..7384bf82ec0 100644 --- a/common/txmgr/types/mocks/tx_store.go +++ b/common/txmgr/types/mocks/tx_store.go @@ -141,8 +141,8 @@ func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) DeleteInPro return r0 } -// FindHighestSequence provides a mock function with given fields: ctx, fromAddress, chainId -func (_m *TxStore[ADDR, CHAIN_ID, TX_HASH, BLOCK_HASH, R, SEQ, FEE]) FindHighestSequence(ctx context.Context, fromAddress ADDR, chainId CHAIN_ID) (SEQ, error) { +// 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) var r0 SEQ diff --git a/common/txmgr/types/tx_store.go b/common/txmgr/types/tx_store.go index b6646d02db4..059a87d7ab2 100644 --- a/common/txmgr/types/tx_store.go +++ b/common/txmgr/types/tx_store.go @@ -58,7 +58,7 @@ type TransactionStore[ CountUnstartedTransactions(ctx context.Context, fromAddress ADDR, chainID CHAIN_ID) (count uint32, err error) CreateTransaction(ctx context.Context, txRequest TxRequest[ADDR, TX_HASH], chainID CHAIN_ID) (tx Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) DeleteInProgressAttempt(ctx context.Context, attempt TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE]) error - FindHighestSequence(ctx context.Context, fromAddress ADDR, chainId CHAIN_ID) (SEQ, error) + FindLatestSequence(ctx context.Context, fromAddress ADDR, chainId CHAIN_ID) (SEQ, error) FindTxsRequiringGasBump(ctx context.Context, address ADDR, blockNum, gasBumpThreshold, depth int64, chainID CHAIN_ID) (etxs []*Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) FindTxsRequiringResubmissionDueToInsufficientFunds(ctx context.Context, address ADDR, chainID CHAIN_ID) (etxs []*Tx[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) FindTxAttemptsConfirmedMissingReceipt(ctx context.Context, chainID CHAIN_ID) (attempts []TxAttempt[CHAIN_ID, ADDR, TX_HASH, BLOCK_HASH, SEQ, FEE], err error) diff --git a/core/chains/evm/txmgr/evm_tx_store.go b/core/chains/evm/txmgr/evm_tx_store.go index 4a76faa4e60..7b1ef8948c1 100644 --- a/core/chains/evm/txmgr/evm_tx_store.go +++ b/core/chains/evm/txmgr/evm_tx_store.go @@ -957,7 +957,7 @@ func (o *evmTxStore) FindReceiptsPendingConfirmation(ctx context.Context, blockN return } -func (o *evmTxStore) FindHighestSequence(ctx context.Context, fromAddress common.Address, chainId *big.Int) (nonce evmtypes.Nonce, err error) { +func (o *evmTxStore) FindLatestSequence(ctx context.Context, fromAddress common.Address, chainId *big.Int) (nonce evmtypes.Nonce, err error) { var cancel context.CancelFunc ctx, cancel = o.mergeContexts(ctx) defer cancel() diff --git a/core/chains/evm/txmgr/mocks/evm_tx_store.go b/core/chains/evm/txmgr/mocks/evm_tx_store.go index 93726c3f98a..0b33be8178d 100644 --- a/core/chains/evm/txmgr/mocks/evm_tx_store.go +++ b/core/chains/evm/txmgr/mocks/evm_tx_store.go @@ -145,8 +145,8 @@ func (_m *EvmTxStore) DeleteInProgressAttempt(ctx context.Context, attempt types return r0 } -// FindHighestSequence provides a mock function with given fields: ctx, fromAddress, chainId -func (_m *EvmTxStore) FindHighestSequence(ctx context.Context, fromAddress common.Address, chainId *big.Int) (evmtypes.Nonce, error) { +// 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) var r0 evmtypes.Nonce