From 2d1a82dddc6412f65b653dd0a463eae5c588010b Mon Sep 17 00:00:00 2001 From: Farber98 Date: Thu, 14 Nov 2024 21:03:14 -0300 Subject: [PATCH 01/51] refactor so txm owns blockhash assignment --- pkg/solana/chain.go | 42 +--- pkg/solana/chain_test.go | 23 ++- pkg/solana/relay.go | 2 +- pkg/solana/transmitter.go | 24 +-- pkg/solana/transmitter_test.go | 8 +- pkg/solana/txm/pendingtx.go | 105 +++++----- pkg/solana/txm/pendingtx_test.go | 294 ++++++++++++++-------------- pkg/solana/txm/txm.go | 125 +++++++----- pkg/solana/txm/txm_internal_test.go | 93 +++++---- pkg/solana/txm/txm_load_test.go | 26 ++- pkg/solana/txm/txm_race_test.go | 68 +++++-- 11 files changed, 418 insertions(+), 392 deletions(-) diff --git a/pkg/solana/chain.go b/pkg/solana/chain.go index ab901a548..436399853 100644 --- a/pkg/solana/chain.go +++ b/pkg/solana/chain.go @@ -12,6 +12,7 @@ import ( "sync" "time" + "github.com/gagliardetto/solana-go" solanago "github.com/gagliardetto/solana-go" "github.com/gagliardetto/solana-go/programs/system" "github.com/gagliardetto/solana-go/rpc" @@ -527,11 +528,6 @@ func (c *chain) HealthReport() map[string]error { } func (c *chain) sendTx(ctx context.Context, from, to string, amount *big.Int, balanceCheck bool) error { - reader, err := c.Reader() - if err != nil { - return fmt.Errorf("chain unreachable: %w", err) - } - fromKey, err := solanago.PublicKeyFromBase58(from) if err != nil { return fmt.Errorf("failed to parse from key: %w", err) @@ -545,10 +541,6 @@ func (c *chain) sendTx(ctx context.Context, from, to string, amount *big.Int, ba } amountI := amount.Uint64() - blockhash, err := reader.LatestBlockhash(ctx) - if err != nil { - return fmt.Errorf("failed to get latest block hash: %w", err) - } tx, err := solanago.NewTransaction( []solanago.Instruction{ system.NewTransferInstruction( @@ -557,21 +549,24 @@ func (c *chain) sendTx(ctx context.Context, from, to string, amount *big.Int, ba toKey, ).Build(), }, - blockhash.Value.Blockhash, + solana.Hash{}, // Will be set within sendWithRetry txm function. solanago.TransactionPayer(fromKey), ) if err != nil { return fmt.Errorf("failed to create tx: %w", err) } - if balanceCheck { - if err = solanaValidateBalance(ctx, reader, fromKey, amountI, tx.Message.ToBase64()); err != nil { - return fmt.Errorf("failed to validate balance: %w", err) - } + msg := &txm.PendingTx{ + Tx: *tx, + // To perform balanceCheck we need a blockhash. + // Storing values to perform balanceCheck within sendWithRetry txm function before sending tx. + BalanceCheck: balanceCheck, + From: fromKey, + Amount: amountI, } chainTxm := c.TxManager() - err = chainTxm.Enqueue(ctx, "", tx, nil, + err = chainTxm.Enqueue(ctx, "", msg, txm.SetComputeUnitLimit(500), // reduce from default 200K limit - should only take 450 compute units // no fee bumping and no additional fee - makes validating balance accurate txm.SetComputeUnitPriceMax(0), @@ -584,20 +579,3 @@ func (c *chain) sendTx(ctx context.Context, from, to string, amount *big.Int, ba } return nil } - -func solanaValidateBalance(ctx context.Context, reader client.Reader, from solanago.PublicKey, amount uint64, msg string) error { - balance, err := reader.Balance(ctx, from) - if err != nil { - return err - } - - fee, err := reader.GetFeeForMessage(ctx, msg) - if err != nil { - return err - } - - if balance < (amount + fee) { - return fmt.Errorf("balance %d is too low for this transaction to be executed: amount %d + fee %d", balance, amount, fee) - } - return nil -} diff --git a/pkg/solana/chain_test.go b/pkg/solana/chain_test.go index db011d80e..75efc2d14 100644 --- a/pkg/solana/chain_test.go +++ b/pkg/solana/chain_test.go @@ -17,18 +17,20 @@ import ( "github.com/gagliardetto/solana-go/programs/system" "github.com/gagliardetto/solana-go/rpc" "github.com/google/uuid" - "github.com/smartcontractkit/chainlink-common/pkg/config" - "github.com/smartcontractkit/chainlink-common/pkg/logger" - "github.com/smartcontractkit/chainlink-common/pkg/utils/tests" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" "go.uber.org/zap/zapcore" + "github.com/smartcontractkit/chainlink-common/pkg/config" + "github.com/smartcontractkit/chainlink-common/pkg/logger" + "github.com/smartcontractkit/chainlink-common/pkg/utils/tests" + "github.com/smartcontractkit/chainlink-solana/pkg/solana/client" mn "github.com/smartcontractkit/chainlink-solana/pkg/solana/client/multinode" solcfg "github.com/smartcontractkit/chainlink-solana/pkg/solana/config" "github.com/smartcontractkit/chainlink-solana/pkg/solana/fees" + "github.com/smartcontractkit/chainlink-solana/pkg/solana/txm" "github.com/smartcontractkit/chainlink-solana/pkg/solana/txm/mocks" ) @@ -534,11 +536,8 @@ func TestSolanaChain_MultiNode_Txm(t *testing.T) { assert.NoError(t, err) assert.Equal(t, uint64(0), receiverBal) - createTx := func(signer solana.PublicKey, sender solana.PublicKey, receiver solana.PublicKey, amt uint64) *solana.Transaction { - selectedClient, err = testChain.getClient() + createMsgWithTx := func(signer solana.PublicKey, sender solana.PublicKey, receiver solana.PublicKey, amt uint64) *txm.PendingTx { assert.NoError(t, err) - hash, hashErr := selectedClient.LatestBlockhash(tests.Context(t)) - assert.NoError(t, hashErr) tx, txErr := solana.NewTransaction( []solana.Instruction{ system.NewTransferInstruction( @@ -547,15 +546,15 @@ func TestSolanaChain_MultiNode_Txm(t *testing.T) { receiver, ).Build(), }, - hash.Value.Blockhash, + solana.Hash{}, solana.TransactionPayer(signer), ) require.NoError(t, txErr) - return tx + return &txm.PendingTx{Tx: *tx} } // Send funds twice, along with an invalid transaction - require.NoError(t, testChain.txm.Enqueue(tests.Context(t), "test_success", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) + require.NoError(t, testChain.txm.Enqueue(tests.Context(t), "test_success", createMsgWithTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL))) // Wait for new block hash currentBh, err := selectedClient.LatestBlockhash(tests.Context(t)) @@ -576,8 +575,8 @@ NewBlockHash: } } - require.NoError(t, testChain.txm.Enqueue(tests.Context(t), "test_success_2", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) - require.Error(t, testChain.txm.Enqueue(tests.Context(t), "test_invalidSigner", createTx(pubKeyReceiver, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) // cannot sign tx before enqueuing + require.NoError(t, testChain.txm.Enqueue(tests.Context(t), "test_success_2", createMsgWithTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL))) + require.Error(t, testChain.txm.Enqueue(tests.Context(t), "test_invalidSigner", createMsgWithTx(pubKeyReceiver, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL))) // cannot sign tx before enqueuing // wait for all txes to finish ctx, cancel := context.WithCancel(tests.Context(t)) diff --git a/pkg/solana/relay.go b/pkg/solana/relay.go index d98ab0442..fd448e6a5 100644 --- a/pkg/solana/relay.go +++ b/pkg/solana/relay.go @@ -24,7 +24,7 @@ import ( var _ TxManager = (*txm.Txm)(nil) type TxManager interface { - Enqueue(ctx context.Context, accountID string, tx *solana.Transaction, txID *string, txCfgs ...txm.SetTxConfig) error + Enqueue(ctx context.Context, accountID string, msg *txm.PendingTx, txCfgs ...txm.SetTxConfig) error } var _ relaytypes.Relayer = &Relayer{} //nolint:staticcheck diff --git a/pkg/solana/transmitter.go b/pkg/solana/transmitter.go index 951e9633e..014f743e3 100644 --- a/pkg/solana/transmitter.go +++ b/pkg/solana/transmitter.go @@ -3,7 +3,6 @@ package solana import ( "bytes" "context" - "errors" "fmt" "github.com/gagliardetto/solana-go" @@ -11,6 +10,8 @@ import ( "github.com/smartcontractkit/chainlink-common/pkg/logger" "github.com/smartcontractkit/chainlink-common/pkg/utils" + + "github.com/smartcontractkit/chainlink-solana/pkg/solana/txm" ) var _ types.ContractTransmitter = (*Transmitter)(nil) @@ -30,19 +31,6 @@ func (c *Transmitter) Transmit( report types.Report, sigs []types.AttributedOnchainSignature, ) error { - reader, err := c.getReader() - if err != nil { - return fmt.Errorf("error on Transmit.Reader: %w", err) - } - - blockhash, err := reader.LatestBlockhash(ctx) - if err != nil { - return fmt.Errorf("error on Transmit.GetRecentBlockhash: %w", err) - } - if blockhash == nil || blockhash.Value == nil { - return errors.New("nil pointer returned from Transmit.GetRecentBlockhash") - } - // Determine store authority seeds := [][]byte{[]byte("store"), c.stateID.Bytes()} storeAuthority, storeNonce, err := solana.FindProgramAddress(seeds, c.programID) @@ -78,16 +66,20 @@ func (c *Transmitter) Transmit( []solana.Instruction{ solana.NewInstruction(c.programID, accounts, data.Bytes()), }, - blockhash.Value.Blockhash, + solana.Hash{}, // Will be set within sendWithRetry txm function. solana.TransactionPayer(c.transmissionSigner), ) if err != nil { return fmt.Errorf("error on Transmit.NewTransaction: %w", err) } + msg := &txm.PendingTx{ + Tx: *tx, + } + // pass transmit payload to tx manager queue c.lggr.Debugf("Queuing transmit tx: state (%s) + transmissions (%s)", c.stateID.String(), c.transmissionsID.String()) - if err = c.txManager.Enqueue(ctx, c.stateID.String(), tx, nil); err != nil { + if err = c.txManager.Enqueue(ctx, c.stateID.String(), msg); err != nil { return fmt.Errorf("error on Transmit.txManager.Enqueue: %w", err) } return nil diff --git a/pkg/solana/transmitter_test.go b/pkg/solana/transmitter_test.go index 1d058d36a..fde87147a 100644 --- a/pkg/solana/transmitter_test.go +++ b/pkg/solana/transmitter_test.go @@ -5,10 +5,8 @@ import ( "testing" "github.com/gagliardetto/solana-go" - "github.com/gagliardetto/solana-go/rpc" "github.com/smartcontractkit/libocr/offchainreporting2/types" "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" "github.com/smartcontractkit/chainlink-common/pkg/logger" @@ -27,8 +25,9 @@ type verifyTxSize struct { s *solana.PrivateKey } -func (txm verifyTxSize) Enqueue(_ context.Context, _ string, tx *solana.Transaction, txID *string, _ ...txm.SetTxConfig) error { +func (txm verifyTxSize) Enqueue(_ context.Context, _ string, msg *txm.PendingTx, _ ...txm.SetTxConfig) error { // additional components that transaction manager adds to the transaction + tx := &msg.Tx require.NoError(txm.t, fees.SetComputeUnitPrice(tx, 0)) require.NoError(txm.t, fees.SetComputeUnitLimit(tx, 0)) @@ -59,9 +58,6 @@ func TestTransmitter_TxSize(t *testing.T) { } rw := clientmocks.NewReaderWriter(t) - rw.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{}, - }, nil) transmitter := Transmitter{ stateID: mustNewRandomPublicKey(), diff --git a/pkg/solana/txm/pendingtx.go b/pkg/solana/txm/pendingtx.go index ecae7243b..8b7cf4bf0 100644 --- a/pkg/solana/txm/pendingtx.go +++ b/pkg/solana/txm/pendingtx.go @@ -21,7 +21,7 @@ var ( type PendingTxContext interface { // New adds a new tranasction in Broadcasted state to the storage - New(msg pendingTx, sig solana.Signature, cancel context.CancelFunc) error + New(msg PendingTx, sig solana.Signature, cancel context.CancelFunc) error // AddSignature adds a new signature for an existing transaction ID AddSignature(id string, sig solana.Signature) error // Remove removes transaction and related signatures from storage if not in finalized or errored state @@ -46,20 +46,18 @@ type PendingTxContext interface { TrimFinalizedErroredTxs() int } -// finishedTx is used to store info required to track transactions to finality or error -type pendingTx struct { - tx solana.Transaction - cfg TxConfig - signatures []solana.Signature - id string - createTs time.Time - state TxState -} - -// finishedTx is used to store minimal info specifically for finalized or errored transactions for external status checks -type finishedTx struct { - retentionTs time.Time - state TxState +type PendingTx struct { + Tx solana.Transaction + cfg TxConfig + signatures []solana.Signature + UUID string + createTs time.Time + retentionTs time.Time + state TxState + LastValidBlockHeight uint64 // to track expiration + BalanceCheck bool + From solana.PublicKey // to perform balanceCheck + Amount uint64 // to perform balanceCheck } var _ PendingTxContext = &pendingTxContext{} @@ -68,9 +66,9 @@ type pendingTxContext struct { cancelBy map[string]context.CancelFunc sigToID map[solana.Signature]string - broadcastedTxs map[string]pendingTx // transactions that require retry and bumping i.e broadcasted, processed - confirmedTxs map[string]pendingTx // transactions that require monitoring for re-org - finalizedErroredTxs map[string]finishedTx // finalized and errored transactions held onto for status + broadcastedProcessedTxs map[string]PendingTx // broadcasted and processed transactions that may require retry and bumping + confirmedTxs map[string]PendingTx // transactions that require monitoring for re-org + finalizedErroredTxs map[string]PendingTx // finalized and errored transactions held onto for status lock sync.RWMutex } @@ -80,20 +78,20 @@ func newPendingTxContext() *pendingTxContext { cancelBy: map[string]context.CancelFunc{}, sigToID: map[solana.Signature]string{}, - broadcastedTxs: map[string]pendingTx{}, - confirmedTxs: map[string]pendingTx{}, - finalizedErroredTxs: map[string]finishedTx{}, + broadcastedProcessedTxs: map[string]PendingTx{}, + confirmedTxs: map[string]PendingTx{}, + finalizedErroredTxs: map[string]PendingTx{}, } } -func (c *pendingTxContext) New(tx pendingTx, sig solana.Signature, cancel context.CancelFunc) error { +func (c *pendingTxContext) New(tx PendingTx, sig solana.Signature, cancel context.CancelFunc) error { err := c.withReadLock(func() error { // validate signature does not exist if _, exists := c.sigToID[sig]; exists { return ErrSigAlreadyExists } // validate id does not exist - if _, exists := c.broadcastedTxs[tx.id]; exists { + if _, exists := c.broadcastedProcessedTxs[tx.UUID]; exists { return ErrIDAlreadyExists } return nil @@ -107,18 +105,18 @@ func (c *pendingTxContext) New(tx pendingTx, sig solana.Signature, cancel contex if _, exists := c.sigToID[sig]; exists { return "", ErrSigAlreadyExists } - if _, exists := c.broadcastedTxs[tx.id]; exists { + if _, exists := c.broadcastedProcessedTxs[tx.UUID]; exists { return "", ErrIDAlreadyExists } // save cancel func - c.cancelBy[tx.id] = cancel - c.sigToID[sig] = tx.id + c.cancelBy[tx.UUID] = cancel + c.sigToID[sig] = tx.UUID // add signature to tx tx.signatures = append(tx.signatures, sig) tx.createTs = time.Now() tx.state = Broadcasted // save to the broadcasted map since transaction was just broadcasted - c.broadcastedTxs[tx.id] = tx + c.broadcastedProcessedTxs[tx.UUID] = tx return "", nil }) return err @@ -132,7 +130,7 @@ func (c *pendingTxContext) AddSignature(id string, sig solana.Signature) error { } // new signatures should only be added for broadcasted transactions // otherwise, the transaction has transitioned states and no longer needs new signatures to track - if _, exists := c.broadcastedTxs[id]; !exists { + if _, exists := c.broadcastedProcessedTxs[id]; !exists { return ErrTransactionNotFound } return nil @@ -146,15 +144,15 @@ func (c *pendingTxContext) AddSignature(id string, sig solana.Signature) error { if _, exists := c.sigToID[sig]; exists { return "", ErrSigAlreadyExists } - if _, exists := c.broadcastedTxs[id]; !exists { + if _, exists := c.broadcastedProcessedTxs[id]; !exists { return "", ErrTransactionNotFound } c.sigToID[sig] = id - tx := c.broadcastedTxs[id] + tx := c.broadcastedProcessedTxs[id] // save new signature tx.signatures = append(tx.signatures, sig) // save updated tx to broadcasted map - c.broadcastedTxs[id] = tx + c.broadcastedProcessedTxs[id] = tx return "", nil }) return err @@ -169,7 +167,7 @@ func (c *pendingTxContext) Remove(sig solana.Signature) (id string, err error) { if !sigExists { return ErrSigDoesNotExist } - _, broadcastedIDExists := c.broadcastedTxs[id] + _, broadcastedIDExists := c.broadcastedProcessedTxs[id] _, confirmedIDExists := c.confirmedTxs[id] // transcation does not exist in tx maps if !broadcastedIDExists && !confirmedIDExists { @@ -187,10 +185,10 @@ func (c *pendingTxContext) Remove(sig solana.Signature) (id string, err error) { if !sigExists { return id, ErrSigDoesNotExist } - var tx pendingTx - if tempTx, exists := c.broadcastedTxs[id]; exists { + var tx PendingTx + if tempTx, exists := c.broadcastedProcessedTxs[id]; exists { tx = tempTx - delete(c.broadcastedTxs, id) + delete(c.broadcastedProcessedTxs, id) } if tempTx, exists := c.confirmedTxs[id]; exists { tx = tempTx @@ -229,7 +227,7 @@ func (c *pendingTxContext) Expired(sig solana.Signature, confirmationTimeout tim if !exists { return false // return expired = false if timestamp does not exist (likely cleaned up by something else previously) } - if tx, exists := c.broadcastedTxs[id]; exists { + if tx, exists := c.broadcastedProcessedTxs[id]; exists { return time.Since(tx.createTs) > confirmationTimeout } if tx, exists := c.confirmedTxs[id]; exists { @@ -246,7 +244,7 @@ func (c *pendingTxContext) OnProcessed(sig solana.Signature) (string, error) { return ErrSigDoesNotExist } // Transactions should only move to processed from broadcasted - tx, exists := c.broadcastedTxs[id] + tx, exists := c.broadcastedProcessedTxs[id] if !exists { return ErrTransactionNotFound } @@ -266,14 +264,15 @@ func (c *pendingTxContext) OnProcessed(sig solana.Signature) (string, error) { if !sigExists { return id, ErrSigDoesNotExist } - tx, exists := c.broadcastedTxs[id] + tx, exists := c.broadcastedProcessedTxs[id] if !exists { return id, ErrTransactionNotFound } + tx = c.broadcastedProcessedTxs[id] // update tx state to Processed tx.state = Processed // save updated tx back to the broadcasted map - c.broadcastedTxs[id] = tx + c.broadcastedProcessedTxs[id] = tx return id, nil }) } @@ -290,7 +289,7 @@ func (c *pendingTxContext) OnConfirmed(sig solana.Signature) (string, error) { return ErrAlreadyInExpectedState } // Transactions should only move to confirmed from broadcasted/processed - if _, exists := c.broadcastedTxs[id]; !exists { + if _, exists := c.broadcastedProcessedTxs[id]; !exists { return ErrTransactionNotFound } return nil @@ -305,8 +304,7 @@ func (c *pendingTxContext) OnConfirmed(sig solana.Signature) (string, error) { if !sigExists { return id, ErrSigDoesNotExist } - tx, exists := c.broadcastedTxs[id] - if !exists { + if _, exists := c.broadcastedProcessedTxs[id]; !exists { return id, ErrTransactionNotFound } // call cancel func + remove from map to stop the retry/bumping cycle for this transaction @@ -314,12 +312,13 @@ func (c *pendingTxContext) OnConfirmed(sig solana.Signature) (string, error) { cancel() // cancel context delete(c.cancelBy, id) } + tx := c.broadcastedProcessedTxs[id] // update tx state to Confirmed tx.state = Confirmed // move tx to confirmed map c.confirmedTxs[id] = tx // remove tx from broadcasted map - delete(c.broadcastedTxs, id) + delete(c.broadcastedProcessedTxs, id) return id, nil }) } @@ -331,7 +330,7 @@ func (c *pendingTxContext) OnFinalized(sig solana.Signature, retentionTimeout ti return ErrSigDoesNotExist } // Allow transactions to transition from broadcasted, processed, or confirmed state in case there are delays between status checks - _, broadcastedExists := c.broadcastedTxs[id] + _, broadcastedExists := c.broadcastedProcessedTxs[id] _, confirmedExists := c.confirmedTxs[id] if !broadcastedExists && !confirmedExists { return ErrTransactionNotFound @@ -348,9 +347,9 @@ func (c *pendingTxContext) OnFinalized(sig solana.Signature, retentionTimeout ti if !exists { return id, ErrSigDoesNotExist } - var tx, tempTx pendingTx + var tx, tempTx PendingTx var broadcastedExists, confirmedExists bool - if tempTx, broadcastedExists = c.broadcastedTxs[id]; broadcastedExists { + if tempTx, broadcastedExists = c.broadcastedProcessedTxs[id]; broadcastedExists { tx = tempTx } if tempTx, confirmedExists = c.confirmedTxs[id]; confirmedExists { @@ -366,7 +365,7 @@ func (c *pendingTxContext) OnFinalized(sig solana.Signature, retentionTimeout ti delete(c.cancelBy, id) } // delete from broadcasted map, if exists - delete(c.broadcastedTxs, id) + delete(c.broadcastedProcessedTxs, id) // delete from confirmed map, if exists delete(c.confirmedTxs, id) // remove all related signatures from the sigToID map to skip picking up this tx in the confirmation logic @@ -437,7 +436,7 @@ func (c *pendingTxContext) OnError(sig solana.Signature, retentionTimeout time.D } // transaction can transition from any non-finalized state var broadcastedExists, confirmedExists bool - _, broadcastedExists = c.broadcastedTxs[id] + _, broadcastedExists = c.broadcastedProcessedTxs[id] _, confirmedExists = c.confirmedTxs[id] // transcation does not exist in any tx maps if !broadcastedExists && !confirmedExists { @@ -455,9 +454,9 @@ func (c *pendingTxContext) OnError(sig solana.Signature, retentionTimeout time.D if !exists { return "", ErrSigDoesNotExist } - var tx, tempTx pendingTx + var tx, tempTx PendingTx var broadcastedExists, confirmedExists bool - if tempTx, broadcastedExists = c.broadcastedTxs[id]; broadcastedExists { + if tempTx, broadcastedExists = c.broadcastedProcessedTxs[id]; broadcastedExists { tx = tempTx } if tempTx, confirmedExists = c.confirmedTxs[id]; confirmedExists { @@ -473,7 +472,7 @@ func (c *pendingTxContext) OnError(sig solana.Signature, retentionTimeout time.D delete(c.cancelBy, id) } // delete from broadcasted map, if exists - delete(c.broadcastedTxs, id) + delete(c.broadcastedProcessedTxs, id) // delete from confirmed map, if exists delete(c.confirmedTxs, id) // remove all related signatures from the sigToID map to skip picking up this tx in the confirmation logic @@ -497,7 +496,7 @@ func (c *pendingTxContext) OnError(sig solana.Signature, retentionTimeout time.D func (c *pendingTxContext) GetTxState(id string) (TxState, error) { c.lock.RLock() defer c.lock.RUnlock() - if tx, exists := c.broadcastedTxs[id]; exists { + if tx, exists := c.broadcastedProcessedTxs[id]; exists { return tx.state, nil } if tx, exists := c.confirmedTxs[id]; exists { @@ -574,7 +573,7 @@ func newPendingTxContextWithProm(id string) *pendingTxContextWithProm { } } -func (c *pendingTxContextWithProm) New(msg pendingTx, sig solana.Signature, cancel context.CancelFunc) error { +func (c *pendingTxContextWithProm) New(msg PendingTx, sig solana.Signature, cancel context.CancelFunc) error { return c.pendingTx.New(msg, sig, cancel) } diff --git a/pkg/solana/txm/pendingtx_test.go b/pkg/solana/txm/pendingtx_test.go index e7b7fc51e..5747e1f36 100644 --- a/pkg/solana/txm/pendingtx_test.go +++ b/pkg/solana/txm/pendingtx_test.go @@ -39,10 +39,10 @@ func TestPendingTxContext_add_remove_multiple(t *testing.T) { n := 5 for i := 0; i < n; i++ { sig, cancel := newProcess() - msg := pendingTx{id: uuid.NewString()} + msg := PendingTx{UUID: uuid.NewString()} err := txs.New(msg, sig, cancel) assert.NoError(t, err) - ids[sig] = msg.id + ids[sig] = msg.UUID } // cannot add signature for non existent ID @@ -74,17 +74,17 @@ func TestPendingTxContext_new(t *testing.T) { txs := newPendingTxContext() // Create new transaction - msg := pendingTx{id: uuid.NewString()} + msg := PendingTx{UUID: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Check it exists in signature map id, exists := txs.sigToID[sig] require.True(t, exists) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Check it exists in broadcasted map - tx, exists := txs.broadcastedTxs[msg.id] + tx, exists := txs.broadcastedProcessedTxs[msg.UUID] require.True(t, exists) require.Len(t, tx.signatures, 1) require.Equal(t, sig, tx.signatures[0]) @@ -93,11 +93,11 @@ func TestPendingTxContext_new(t *testing.T) { require.Equal(t, Broadcasted, tx.state) // Check it does not exist in confirmed map - _, exists = txs.confirmedTxs[msg.id] + tx, exists = txs.confirmedTxs[msg.UUID] require.False(t, exists) // Check it does not exist in finalized map - _, exists = txs.finalizedErroredTxs[msg.id] + tx, exists = txs.finalizedErroredTxs[msg.UUID] require.False(t, exists) } @@ -111,34 +111,34 @@ func TestPendingTxContext_add_signature(t *testing.T) { sig2 := randomSignature(t) // Create new transaction - msg := pendingTx{id: uuid.NewString()} + msg := PendingTx{UUID: uuid.NewString()} err := txs.New(msg, sig1, cancel) require.NoError(t, err) - err = txs.AddSignature(msg.id, sig2) + err = txs.AddSignature(msg.UUID, sig2) require.NoError(t, err) // Check signature map id, exists := txs.sigToID[sig1] require.True(t, exists) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) id, exists = txs.sigToID[sig2] require.True(t, exists) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Check broadcasted map - tx, exists := txs.broadcastedTxs[msg.id] + tx, exists := txs.broadcastedProcessedTxs[msg.UUID] require.True(t, exists) require.Len(t, tx.signatures, 2) require.Equal(t, sig1, tx.signatures[0]) require.Equal(t, sig2, tx.signatures[1]) // Check confirmed map - _, exists = txs.confirmedTxs[msg.id] + tx, exists = txs.confirmedTxs[msg.UUID] require.False(t, exists) // Check finalized map - _, exists = txs.finalizedErroredTxs[msg.id] + tx, exists = txs.finalizedErroredTxs[msg.UUID] require.False(t, exists) }) @@ -146,11 +146,11 @@ func TestPendingTxContext_add_signature(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := pendingTx{id: uuid.NewString()} + msg := PendingTx{UUID: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) - err = txs.AddSignature(msg.id, sig) + err = txs.AddSignature(msg.UUID, sig) require.ErrorIs(t, err, ErrSigAlreadyExists) }) @@ -159,7 +159,7 @@ func TestPendingTxContext_add_signature(t *testing.T) { sig2 := randomSignature(t) // Create new transaction - msg := pendingTx{id: uuid.NewString()} + msg := PendingTx{UUID: uuid.NewString()} err := txs.New(msg, sig1, cancel) require.NoError(t, err) @@ -172,21 +172,21 @@ func TestPendingTxContext_add_signature(t *testing.T) { sig2 := randomSignature(t) // Create new transaction - msg := pendingTx{id: uuid.NewString()} + msg := PendingTx{UUID: uuid.NewString()} err := txs.New(msg, sig1, cancel) require.NoError(t, err) // Transition to processed state id, err := txs.OnProcessed(sig1) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Transition to confirmed state id, err = txs.OnConfirmed(sig1) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) - err = txs.AddSignature(msg.id, sig2) + err = txs.AddSignature(msg.UUID, sig2) require.ErrorIs(t, err, ErrTransactionNotFound) }) } @@ -201,22 +201,22 @@ func TestPendingTxContext_on_broadcasted_processed(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := pendingTx{id: uuid.NewString()} + msg := PendingTx{UUID: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to processed state id, err := txs.OnProcessed(sig) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Check it exists in signature map id, exists := txs.sigToID[sig] require.True(t, exists) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Check it exists in broadcasted map - tx, exists := txs.broadcastedTxs[msg.id] + tx, exists := txs.broadcastedProcessedTxs[msg.UUID] require.True(t, exists) require.Len(t, tx.signatures, 1) require.Equal(t, sig, tx.signatures[0]) @@ -225,11 +225,11 @@ func TestPendingTxContext_on_broadcasted_processed(t *testing.T) { require.Equal(t, Processed, tx.state) // Check it does not exist in confirmed map - _, exists = txs.confirmedTxs[msg.id] + tx, exists = txs.confirmedTxs[msg.UUID] require.False(t, exists) // Check it does not exist in finalized map - _, exists = txs.finalizedErroredTxs[msg.id] + tx, exists = txs.finalizedErroredTxs[msg.UUID] require.False(t, exists) }) @@ -237,19 +237,19 @@ func TestPendingTxContext_on_broadcasted_processed(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := pendingTx{id: uuid.NewString()} + msg := PendingTx{UUID: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to processed state id, err := txs.OnProcessed(sig) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Transition to confirmed state id, err = txs.OnConfirmed(sig) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Transition back to processed state _, err = txs.OnProcessed(sig) @@ -260,24 +260,24 @@ func TestPendingTxContext_on_broadcasted_processed(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := pendingTx{id: uuid.NewString()} + msg := PendingTx{UUID: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to processed state id, err := txs.OnProcessed(sig) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Transition to confirmed state id, err = txs.OnConfirmed(sig) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Transition to finalized state id, err = txs.OnFinalized(sig, retentionTimeout) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Transition back to processed state _, err = txs.OnProcessed(sig) @@ -288,14 +288,14 @@ func TestPendingTxContext_on_broadcasted_processed(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := pendingTx{id: uuid.NewString()} + msg := PendingTx{UUID: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to errored state id, err := txs.OnError(sig, retentionTimeout, Errored, 0) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Transition back to processed state _, err = txs.OnProcessed(sig) @@ -306,14 +306,14 @@ func TestPendingTxContext_on_broadcasted_processed(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := pendingTx{id: uuid.NewString()} + msg := PendingTx{UUID: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to processed state id, err := txs.OnProcessed(sig) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // No error if OnProcessed called again _, err = txs.OnProcessed(sig) @@ -331,31 +331,31 @@ func TestPendingTxContext_on_confirmed(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := pendingTx{id: uuid.NewString()} + msg := PendingTx{UUID: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to processed state id, err := txs.OnProcessed(sig) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Transition to confirmed state id, err = txs.OnConfirmed(sig) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Check it exists in signature map id, exists := txs.sigToID[sig] require.True(t, exists) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Check it does not exist in broadcasted map - _, exists = txs.broadcastedTxs[msg.id] + _, exists = txs.broadcastedProcessedTxs[msg.UUID] require.False(t, exists) // Check it exists in confirmed map - tx, exists := txs.confirmedTxs[msg.id] + tx, exists := txs.confirmedTxs[msg.UUID] require.True(t, exists) require.Len(t, tx.signatures, 1) require.Equal(t, sig, tx.signatures[0]) @@ -364,7 +364,7 @@ func TestPendingTxContext_on_confirmed(t *testing.T) { require.Equal(t, Confirmed, tx.state) // Check it does not exist in finalized map - _, exists = txs.finalizedErroredTxs[msg.id] + tx, exists = txs.finalizedErroredTxs[msg.UUID] require.False(t, exists) }) @@ -372,24 +372,24 @@ func TestPendingTxContext_on_confirmed(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := pendingTx{id: uuid.NewString()} + msg := PendingTx{UUID: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to processed state id, err := txs.OnProcessed(sig) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Transition to confirmed state id, err = txs.OnConfirmed(sig) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Transition to finalized state id, err = txs.OnFinalized(sig, retentionTimeout) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Transition back to processed state _, err = txs.OnConfirmed(sig) @@ -400,14 +400,14 @@ func TestPendingTxContext_on_confirmed(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := pendingTx{id: uuid.NewString()} + msg := PendingTx{UUID: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to errored state id, err := txs.OnError(sig, retentionTimeout, Errored, 0) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Transition back to confirmed state _, err = txs.OnConfirmed(sig) @@ -418,19 +418,19 @@ func TestPendingTxContext_on_confirmed(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := pendingTx{id: uuid.NewString()} + msg := PendingTx{UUID: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to processed state id, err := txs.OnProcessed(sig) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Transition to confirmed state id, err = txs.OnConfirmed(sig) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // No error if OnConfirmed called again _, err = txs.OnConfirmed(sig) @@ -449,29 +449,29 @@ func TestPendingTxContext_on_finalized(t *testing.T) { sig2 := randomSignature(t) // Create new transaction - msg := pendingTx{id: uuid.NewString()} + msg := PendingTx{UUID: uuid.NewString()} err := txs.New(msg, sig1, cancel) require.NoError(t, err) // Add second signature - err = txs.AddSignature(msg.id, sig2) + err = txs.AddSignature(msg.UUID, sig2) require.NoError(t, err) // Transition to finalized state id, err := txs.OnFinalized(sig1, retentionTimeout) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Check it does not exist in broadcasted map - _, exists := txs.broadcastedTxs[msg.id] + _, exists := txs.broadcastedProcessedTxs[msg.UUID] require.False(t, exists) // Check it does not exist in confirmed map - _, exists = txs.confirmedTxs[msg.id] + _, exists = txs.confirmedTxs[msg.UUID] require.False(t, exists) // Check it exists in finalized map - tx, exists := txs.finalizedErroredTxs[msg.id] + tx, exists := txs.finalizedErroredTxs[msg.UUID] require.True(t, exists) // Check status is Finalized @@ -489,39 +489,39 @@ func TestPendingTxContext_on_finalized(t *testing.T) { sig2 := randomSignature(t) // Create new transaction - msg := pendingTx{id: uuid.NewString()} + msg := PendingTx{UUID: uuid.NewString()} err := txs.New(msg, sig1, cancel) require.NoError(t, err) // Add second signature - err = txs.AddSignature(msg.id, sig2) + err = txs.AddSignature(msg.UUID, sig2) require.NoError(t, err) // Transition to processed state id, err := txs.OnProcessed(sig1) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Transition to confirmed state id, err = txs.OnConfirmed(sig1) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Transition to finalized state id, err = txs.OnFinalized(sig1, retentionTimeout) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Check it does not exist in broadcasted map - _, exists := txs.broadcastedTxs[msg.id] + _, exists := txs.broadcastedProcessedTxs[msg.UUID] require.False(t, exists) // Check it does not exist in confirmed map - _, exists = txs.confirmedTxs[msg.id] + _, exists = txs.confirmedTxs[msg.UUID] require.False(t, exists) // Check it exists in finalized map - tx, exists := txs.finalizedErroredTxs[msg.id] + tx, exists := txs.finalizedErroredTxs[msg.UUID] require.True(t, exists) // Check status is Finalized @@ -538,35 +538,35 @@ func TestPendingTxContext_on_finalized(t *testing.T) { sig1 := randomSignature(t) // Create new transaction - msg := pendingTx{id: uuid.NewString()} + msg := PendingTx{UUID: uuid.NewString()} err := txs.New(msg, sig1, cancel) require.NoError(t, err) // Transition to processed state id, err := txs.OnProcessed(sig1) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Transition to confirmed state id, err = txs.OnConfirmed(sig1) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Transition to finalized state id, err = txs.OnFinalized(sig1, 0*time.Second) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Check it does not exist in broadcasted map - _, exists := txs.broadcastedTxs[msg.id] + _, exists := txs.broadcastedProcessedTxs[msg.UUID] require.False(t, exists) // Check it does not exist in confirmed map - _, exists = txs.confirmedTxs[msg.id] + _, exists = txs.confirmedTxs[msg.UUID] require.False(t, exists) // Check it does not exist in finalized map - _, exists = txs.finalizedErroredTxs[msg.id] + _, exists = txs.finalizedErroredTxs[msg.UUID] require.False(t, exists) // Check sigs do no exist in signature map @@ -578,14 +578,14 @@ func TestPendingTxContext_on_finalized(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := pendingTx{id: uuid.NewString()} + msg := PendingTx{UUID: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to errored state id, err := txs.OnError(sig, retentionTimeout, Errored, 0) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Transition back to confirmed state _, err = txs.OnFinalized(sig, retentionTimeout) @@ -603,25 +603,25 @@ func TestPendingTxContext_on_error(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := pendingTx{id: uuid.NewString()} + msg := PendingTx{UUID: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to errored state id, err := txs.OnError(sig, retentionTimeout, Errored, 0) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Check it does not exist in broadcasted map - _, exists := txs.broadcastedTxs[msg.id] + _, exists := txs.broadcastedProcessedTxs[msg.UUID] require.False(t, exists) // Check it does not exist in confirmed map - _, exists = txs.confirmedTxs[msg.id] + _, exists = txs.confirmedTxs[msg.UUID] require.False(t, exists) // Check it exists in errored map - tx, exists := txs.finalizedErroredTxs[msg.id] + tx, exists := txs.finalizedErroredTxs[msg.UUID] require.True(t, exists) // Check status is Finalized @@ -636,30 +636,30 @@ func TestPendingTxContext_on_error(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := pendingTx{id: uuid.NewString()} + msg := PendingTx{UUID: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to errored state id, err := txs.OnConfirmed(sig) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Transition to errored state id, err = txs.OnError(sig, retentionTimeout, Errored, 0) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Check it does not exist in broadcasted map - _, exists := txs.broadcastedTxs[msg.id] + _, exists := txs.broadcastedProcessedTxs[msg.UUID] require.False(t, exists) // Check it does not exist in confirmed map - _, exists = txs.confirmedTxs[msg.id] + _, exists = txs.confirmedTxs[msg.UUID] require.False(t, exists) // Check it exists in errored map - tx, exists := txs.finalizedErroredTxs[msg.id] + tx, exists := txs.finalizedErroredTxs[msg.UUID] require.True(t, exists) // Check status is Finalized @@ -674,7 +674,7 @@ func TestPendingTxContext_on_error(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := pendingTx{id: uuid.NewString()} + msg := PendingTx{UUID: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) @@ -703,30 +703,30 @@ func TestPendingTxContext_on_error(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := pendingTx{id: uuid.NewString()} + msg := PendingTx{UUID: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to errored state id, err := txs.OnConfirmed(sig) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Transition to errored state id, err = txs.OnError(sig, 0*time.Second, Errored, 0) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Check it does not exist in broadcasted map - _, exists := txs.broadcastedTxs[msg.id] + _, exists := txs.broadcastedProcessedTxs[msg.UUID] require.False(t, exists) // Check it does not exist in confirmed map - _, exists = txs.confirmedTxs[msg.id] + _, exists = txs.confirmedTxs[msg.UUID] require.False(t, exists) // Check it exists in errored map - _, exists = txs.finalizedErroredTxs[msg.id] + _, exists = txs.finalizedErroredTxs[msg.UUID] require.False(t, exists) // Check sigs do no exist in signature map @@ -738,14 +738,14 @@ func TestPendingTxContext_on_error(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := pendingTx{id: uuid.NewString()} + msg := PendingTx{UUID: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to confirmed state id, err := txs.OnFinalized(sig, retentionTimeout) require.NoError(t, err) - require.Equal(t, msg.id, id) + require.Equal(t, msg.UUID, id) // Transition back to confirmed state id, err = txs.OnError(sig, retentionTimeout, Errored, 0) @@ -833,50 +833,50 @@ func TestPendingTxContext_remove(t *testing.T) { erroredSig := randomSignature(t) // Create new broadcasted transaction with extra sig - broadcastedMsg := pendingTx{id: uuid.NewString()} + broadcastedMsg := PendingTx{UUID: uuid.NewString()} err := txs.New(broadcastedMsg, broadcastedSig1, cancel) require.NoError(t, err) - err = txs.AddSignature(broadcastedMsg.id, broadcastedSig2) + err = txs.AddSignature(broadcastedMsg.UUID, broadcastedSig2) require.NoError(t, err) // Create new processed transaction - processedMsg := pendingTx{id: uuid.NewString()} + processedMsg := PendingTx{UUID: uuid.NewString()} err = txs.New(processedMsg, processedSig, cancel) require.NoError(t, err) id, err := txs.OnProcessed(processedSig) require.NoError(t, err) - require.Equal(t, processedMsg.id, id) + require.Equal(t, processedMsg.UUID, id) // Create new confirmed transaction - confirmedMsg := pendingTx{id: uuid.NewString()} + confirmedMsg := PendingTx{UUID: uuid.NewString()} err = txs.New(confirmedMsg, confirmedSig, cancel) require.NoError(t, err) id, err = txs.OnConfirmed(confirmedSig) require.NoError(t, err) - require.Equal(t, confirmedMsg.id, id) + require.Equal(t, confirmedMsg.UUID, id) // Create new finalized transaction - finalizedMsg := pendingTx{id: uuid.NewString()} + finalizedMsg := PendingTx{UUID: uuid.NewString()} err = txs.New(finalizedMsg, finalizedSig, cancel) require.NoError(t, err) id, err = txs.OnFinalized(finalizedSig, retentionTimeout) require.NoError(t, err) - require.Equal(t, finalizedMsg.id, id) + require.Equal(t, finalizedMsg.UUID, id) // Create new errored transaction - erroredMsg := pendingTx{id: uuid.NewString()} + erroredMsg := PendingTx{UUID: uuid.NewString()} err = txs.New(erroredMsg, erroredSig, cancel) require.NoError(t, err) id, err = txs.OnError(erroredSig, retentionTimeout, Errored, 0) require.NoError(t, err) - require.Equal(t, erroredMsg.id, id) + require.Equal(t, erroredMsg.UUID, id) // Remove broadcasted transaction id, err = txs.Remove(broadcastedSig1) require.NoError(t, err) - require.Equal(t, broadcastedMsg.id, id) + require.Equal(t, broadcastedMsg.UUID, id) // Check removed from broadcasted map - _, exists := txs.broadcastedTxs[broadcastedMsg.id] + _, exists := txs.broadcastedProcessedTxs[broadcastedMsg.UUID] require.False(t, exists) // Check all signatures removed from sig map _, exists = txs.sigToID[broadcastedSig1] @@ -887,9 +887,9 @@ func TestPendingTxContext_remove(t *testing.T) { // Remove processed transaction id, err = txs.Remove(processedSig) require.NoError(t, err) - require.Equal(t, processedMsg.id, id) + require.Equal(t, processedMsg.UUID, id) // Check removed from broadcasted map - _, exists = txs.broadcastedTxs[processedMsg.id] + _, exists = txs.broadcastedProcessedTxs[processedMsg.UUID] require.False(t, exists) // Check all signatures removed from sig map _, exists = txs.sigToID[processedSig] @@ -898,9 +898,9 @@ func TestPendingTxContext_remove(t *testing.T) { // Remove confirmed transaction id, err = txs.Remove(confirmedSig) require.NoError(t, err) - require.Equal(t, confirmedMsg.id, id) + require.Equal(t, confirmedMsg.UUID, id) // Check removed from confirmed map - _, exists = txs.confirmedTxs[confirmedMsg.id] + _, exists = txs.confirmedTxs[confirmedMsg.UUID] require.False(t, exists) // Check all signatures removed from sig map _, exists = txs.sigToID[confirmedSig] @@ -924,34 +924,32 @@ func TestPendingTxContext_trim_finalized_errored_txs(t *testing.T) { txs := newPendingTxContext() // Create new finalized transaction with retention ts in the past and add to map - finalizedMsg1 := finishedTx{retentionTs: time.Now().Add(-2 * time.Second)} - finalizedMsg1ID := uuid.NewString() - txs.finalizedErroredTxs[finalizedMsg1ID] = finalizedMsg1 + finalizedMsg1 := PendingTx{UUID: uuid.NewString(), retentionTs: time.Now().Add(-2 * time.Second)} + txs.finalizedErroredTxs[finalizedMsg1.UUID] = finalizedMsg1 // Create new finalized transaction with retention ts in the future and add to map - finalizedMsg2 := finishedTx{retentionTs: time.Now().Add(1 * time.Second)} - finalizedMsg2ID := uuid.NewString() - txs.finalizedErroredTxs[finalizedMsg2ID] = finalizedMsg2 + finalizedMsg2 := PendingTx{UUID: uuid.NewString(), retentionTs: time.Now().Add(1 * time.Second)} + txs.finalizedErroredTxs[finalizedMsg2.UUID] = finalizedMsg2 // Create new finalized transaction with retention ts in the past and add to map - erroredMsg := finishedTx{retentionTs: time.Now().Add(-2 * time.Second)} - erroredMsgID := uuid.NewString() - txs.finalizedErroredTxs[erroredMsgID] = erroredMsg + erroredMsg := PendingTx{UUID: uuid.NewString(), retentionTs: time.Now().Add(-2 * time.Second)} + txs.finalizedErroredTxs[erroredMsg.UUID] = erroredMsg // Delete finalized/errored transactions that have passed the retention period txs.TrimFinalizedErroredTxs() // Check finalized message past retention is deleted - _, exists := txs.finalizedErroredTxs[finalizedMsg1ID] + _, exists := txs.finalizedErroredTxs[finalizedMsg1.UUID] require.False(t, exists) // Check errored message past retention is deleted - _, exists = txs.finalizedErroredTxs[erroredMsgID] + _, exists = txs.finalizedErroredTxs[erroredMsg.UUID] require.False(t, exists) // Check finalized message within retention period still exists - _, exists = txs.finalizedErroredTxs[finalizedMsg2ID] + msg, exists := txs.finalizedErroredTxs[finalizedMsg2.UUID] require.True(t, exists) + require.Equal(t, finalizedMsg2.UUID, msg.UUID) } func TestPendingTxContext_expired(t *testing.T) { @@ -960,16 +958,16 @@ func TestPendingTxContext_expired(t *testing.T) { sig := solana.Signature{} txs := newPendingTxContext() - msg := pendingTx{id: uuid.NewString()} + msg := PendingTx{UUID: uuid.NewString()} err := txs.New(msg, sig, cancel) assert.NoError(t, err) - msg, exists := txs.broadcastedTxs[msg.id] + msg, exists := txs.broadcastedProcessedTxs[msg.UUID] require.True(t, exists) // Set createTs to 10 seconds ago msg.createTs = time.Now().Add(-10 * time.Second) - txs.broadcastedTxs[msg.id] = msg + txs.broadcastedProcessedTxs[msg.UUID] = msg assert.False(t, txs.Expired(sig, 0*time.Second)) // false if timeout 0 assert.True(t, txs.Expired(sig, 5*time.Second)) // expired for 5s lifetime @@ -977,7 +975,7 @@ func TestPendingTxContext_expired(t *testing.T) { id, err := txs.Remove(sig) assert.NoError(t, err) - assert.Equal(t, msg.id, id) + assert.Equal(t, msg.UUID, id) assert.False(t, txs.Expired(sig, 60*time.Second)) // no longer exists, should return false } @@ -989,11 +987,11 @@ func TestPendingTxContext_race(t *testing.T) { var err [2]error go func() { - err[0] = txCtx.New(pendingTx{id: uuid.NewString()}, solana.Signature{}, func() {}) + err[0] = txCtx.New(PendingTx{UUID: uuid.NewString()}, solana.Signature{}, func() {}) wg.Done() }() go func() { - err[1] = txCtx.New(pendingTx{id: uuid.NewString()}, solana.Signature{}, func() {}) + err[1] = txCtx.New(PendingTx{UUID: uuid.NewString()}, solana.Signature{}, func() {}) wg.Done() }() @@ -1003,7 +1001,7 @@ func TestPendingTxContext_race(t *testing.T) { t.Run("add signature", func(t *testing.T) { txCtx := newPendingTxContext() - msg := pendingTx{id: uuid.NewString()} + msg := PendingTx{UUID: uuid.NewString()} createErr := txCtx.New(msg, solana.Signature{}, func() {}) require.NoError(t, createErr) var wg sync.WaitGroup @@ -1011,11 +1009,11 @@ func TestPendingTxContext_race(t *testing.T) { var err [2]error go func() { - err[0] = txCtx.AddSignature(msg.id, solana.Signature{1}) + err[0] = txCtx.AddSignature(msg.UUID, solana.Signature{1}) wg.Done() }() go func() { - err[1] = txCtx.AddSignature(msg.id, solana.Signature{1}) + err[1] = txCtx.AddSignature(msg.UUID, solana.Signature{1}) wg.Done() }() @@ -1025,7 +1023,7 @@ func TestPendingTxContext_race(t *testing.T) { t.Run("remove", func(t *testing.T) { txCtx := newPendingTxContext() - msg := pendingTx{id: uuid.NewString()} + msg := PendingTx{UUID: uuid.NewString()} err := txCtx.New(msg, solana.Signature{}, func() {}) require.NoError(t, err) var wg sync.WaitGroup @@ -1058,56 +1056,56 @@ func TestGetTxState(t *testing.T) { fatallyErroredSig := randomSignature(t) // Create new broadcasted transaction with extra sig - broadcastedMsg := pendingTx{id: uuid.NewString()} + broadcastedMsg := PendingTx{UUID: uuid.NewString()} err := txs.New(broadcastedMsg, broadcastedSig, cancel) require.NoError(t, err) var state TxState // Create new processed transaction - processedMsg := pendingTx{id: uuid.NewString()} + processedMsg := PendingTx{UUID: uuid.NewString()} err = txs.New(processedMsg, processedSig, cancel) require.NoError(t, err) id, err := txs.OnProcessed(processedSig) require.NoError(t, err) - require.Equal(t, processedMsg.id, id) + require.Equal(t, processedMsg.UUID, id) // Check Processed state is returned - state, err = txs.GetTxState(processedMsg.id) + state, err = txs.GetTxState(processedMsg.UUID) require.NoError(t, err) require.Equal(t, Processed, state) // Create new confirmed transaction - confirmedMsg := pendingTx{id: uuid.NewString()} + confirmedMsg := PendingTx{UUID: uuid.NewString()} err = txs.New(confirmedMsg, confirmedSig, cancel) require.NoError(t, err) id, err = txs.OnConfirmed(confirmedSig) require.NoError(t, err) - require.Equal(t, confirmedMsg.id, id) + require.Equal(t, confirmedMsg.UUID, id) // Check Confirmed state is returned - state, err = txs.GetTxState(confirmedMsg.id) + state, err = txs.GetTxState(confirmedMsg.UUID) require.NoError(t, err) require.Equal(t, Confirmed, state) // Create new finalized transaction - finalizedMsg := pendingTx{id: uuid.NewString()} + finalizedMsg := PendingTx{UUID: uuid.NewString()} err = txs.New(finalizedMsg, finalizedSig, cancel) require.NoError(t, err) id, err = txs.OnFinalized(finalizedSig, retentionTimeout) require.NoError(t, err) - require.Equal(t, finalizedMsg.id, id) + require.Equal(t, finalizedMsg.UUID, id) // Check Finalized state is returned - state, err = txs.GetTxState(finalizedMsg.id) + state, err = txs.GetTxState(finalizedMsg.UUID) require.NoError(t, err) require.Equal(t, Finalized, state) // Create new errored transaction - erroredMsg := pendingTx{id: uuid.NewString()} + erroredMsg := PendingTx{UUID: uuid.NewString()} err = txs.New(erroredMsg, erroredSig, cancel) require.NoError(t, err) id, err = txs.OnError(erroredSig, retentionTimeout, Errored, 0) require.NoError(t, err) - require.Equal(t, erroredMsg.id, id) + require.Equal(t, erroredMsg.UUID, id) // Check Errored state is returned - state, err = txs.GetTxState(erroredMsg.id) + state, err = txs.GetTxState(erroredMsg.UUID) require.NoError(t, err) require.Equal(t, Errored, state) diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index 342f54dce..e9d9c1ec2 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -9,6 +9,7 @@ import ( "sync" "time" + "github.com/gagliardetto/solana-go" solanaGo "github.com/gagliardetto/solana-go" "github.com/gagliardetto/solana-go/rpc" "github.com/google/uuid" @@ -50,8 +51,8 @@ var _ loop.Keystore = (SimpleKeystore)(nil) type Txm struct { services.StateMachine lggr logger.Logger - chSend chan pendingTx - chSim chan pendingTx + chSend chan PendingTx + chSim chan PendingTx chStop services.StopChan done sync.WaitGroup cfg config.Config @@ -94,8 +95,8 @@ func NewTxm(chainID string, client internal.Loader[client.ReaderWriter], return &Txm{ lggr: logger.Named(lggr, "Txm"), - chSend: make(chan pendingTx, MaxQueueLen), // queue can support 1000 pending txs - chSim: make(chan pendingTx, MaxQueueLen), // queue can support 1000 pending txs + chSend: make(chan PendingTx, MaxQueueLen), // queue can support 1000 pending txs + chSim: make(chan PendingTx, MaxQueueLen), // queue can support 1000 pending txs chStop: make(chan struct{}), cfg: cfg, txs: newPendingTxContextWithProm(chainID), @@ -159,9 +160,9 @@ func (txm *Txm) run() { } // send tx + signature to simulation queue - msg.tx = tx + msg.Tx = tx msg.signatures = append(msg.signatures, sig) - msg.id = id + msg.UUID = id select { case txm.chSim <- msg: default: @@ -175,11 +176,11 @@ func (txm *Txm) run() { } } -func (txm *Txm) sendWithRetry(ctx context.Context, msg pendingTx) (solanaGo.Transaction, string, solanaGo.Signature, error) { +func (txm *Txm) sendWithRetry(ctx context.Context, msg PendingTx) (solanaGo.Transaction, string, solanaGo.Signature, error) { // get key // fee payer account is index 0 account // https://github.com/gagliardetto/solana-go/blob/main/transaction.go#L252 - key := msg.tx.Message.AccountKeys[0].String() + key := msg.Tx.Message.AccountKeys[0].String() // base compute unit price should only be calculated once // prevent underlying base changing when bumping (could occur with RPC based estimation) @@ -193,7 +194,29 @@ func (txm *Txm) sendWithRetry(ctx context.Context, msg pendingTx) (solanaGo.Tran return fees.ComputeUnitPrice(fee) } - baseTx := msg.tx + // Get client + client, err := txm.client.Get() + if err != nil { + return solanaGo.Transaction{}, "", solanaGo.Signature{}, fmt.Errorf("failed to get client in soltxm.sendWithRetry: %w", err) + } + + // Get blockhash and assign to msg + blockhash, err := client.LatestBlockhash(ctx) + if err != nil { + return solanaGo.Transaction{}, "", solanaGo.Signature{}, fmt.Errorf("failed to get blockhash in soltxm.sendWithRetry: %w", err) + } + msg.Tx.Message.RecentBlockhash = blockhash.Value.Blockhash + msg.LastValidBlockHeight = blockhash.Value.LastValidBlockHeight + + // if requested, validate balance before sending transaction. + if msg.BalanceCheck { + if err = solanaValidateBalance(ctx, client, msg.From, msg.Amount, msg.Tx.Message.ToBase64()); err != nil { + return solanaGo.Transaction{}, "", solanaGo.Signature{}, fmt.Errorf("failed to validate balance: %w", err) + } + } + + // set baseTx as a copy of the transaction + baseTx := msg.Tx // add compute unit limit instruction - static for the transaction // skip if compute unit limit = 0 (otherwise would always fail) @@ -258,7 +281,7 @@ func (txm *Txm) sendWithRetry(ctx context.Context, msg pendingTx) (solanaGo.Tran return solanaGo.Transaction{}, "", solanaGo.Signature{}, fmt.Errorf("failed to save initial signature in signature list: %w", initSetErr) } - txm.lggr.Debugw("tx initial broadcast", "id", msg.id, "fee", getFee(0), "signature", sig) + txm.lggr.Debugw("tx initial broadcast", "id", msg.UUID, "fee", getFee(0), "signature", sig) txm.done.Add(1) // retry with exponential backoff @@ -277,7 +300,7 @@ func (txm *Txm) sendWithRetry(ctx context.Context, msg pendingTx) (solanaGo.Tran case <-ctx.Done(): // stop sending tx after retry tx ctx times out (does not stop confirmation polling for tx) wg.Wait() - txm.lggr.Debugw("stopped tx retry", "id", msg.id, "signatures", sigs.List(), "err", context.Cause(ctx)) + txm.lggr.Debugw("stopped tx retry", "id", msg.UUID, "signatures", sigs.List(), "err", context.Cause(ctx)) return case <-tick: var shouldBump bool @@ -293,7 +316,7 @@ func (txm *Txm) sendWithRetry(ctx context.Context, msg pendingTx) (solanaGo.Tran var retryBuildErr error currentTx, retryBuildErr = buildTx(ctx, baseTx, bumpCount) if retryBuildErr != nil { - txm.lggr.Errorw("failed to build bumped retry tx", "error", retryBuildErr, "id", msg.id) + txm.lggr.Errorw("failed to build bumped retry tx", "error", retryBuildErr, "id", msg.UUID) return // exit func if cannot build tx for retrying } ind := sigs.Allocate() @@ -312,24 +335,24 @@ func (txm *Txm) sendWithRetry(ctx context.Context, msg pendingTx) (solanaGo.Tran // this could occur if endpoint goes down or if ctx cancelled if retrySendErr != nil { if strings.Contains(retrySendErr.Error(), "context canceled") || strings.Contains(retrySendErr.Error(), "context deadline exceeded") { - txm.lggr.Debugw("ctx error on send retry transaction", "error", retrySendErr, "signatures", sigs.List(), "id", msg.id) + txm.lggr.Debugw("ctx error on send retry transaction", "error", retrySendErr, "signatures", sigs.List(), "id", msg.UUID) } else { - txm.lggr.Warnw("failed to send retry transaction", "error", retrySendErr, "signatures", sigs.List(), "id", msg.id) + txm.lggr.Warnw("failed to send retry transaction", "error", retrySendErr, "signatures", sigs.List(), "id", msg.UUID) } return } // save new signature if fee bumped if bump { - if retryStoreErr := txm.txs.AddSignature(msg.id, retrySig); retryStoreErr != nil { - txm.lggr.Warnw("error in adding retry transaction", "error", retryStoreErr, "id", msg.id) + if retryStoreErr := txm.txs.AddSignature(msg.UUID, retrySig); retryStoreErr != nil { + txm.lggr.Warnw("error in adding retry transaction", "error", retryStoreErr, "id", msg.UUID) return } if setErr := sigs.Set(count, retrySig); setErr != nil { // this should never happen txm.lggr.Errorw("INVARIANT VIOLATION", "error", setErr) } - txm.lggr.Debugw("tx rebroadcast with bumped fee", "id", msg.id, "fee", getFee(count), "signatures", sigs.List()) + txm.lggr.Debugw("tx rebroadcast with bumped fee", "id", msg.UUID, "fee", getFee(count), "signatures", sigs.List()) } // prevent locking on waitgroup when ctx is closed @@ -361,7 +384,7 @@ func (txm *Txm) sendWithRetry(ctx context.Context, msg pendingTx) (solanaGo.Tran }(ctx, baseTx, initTx) // return signed tx, id, signature for use in simulation - return initTx, msg.id, sig, nil + return initTx, msg.UUID, sig, nil } // goroutine that polls to confirm implementation @@ -513,6 +536,23 @@ func (txm *Txm) confirm() { } } +func solanaValidateBalance(ctx context.Context, reader client.Reader, from solana.PublicKey, amount uint64, msg string) error { + balance, err := reader.Balance(ctx, from) + if err != nil { + return err + } + + fee, err := reader.GetFeeForMessage(ctx, msg) + if err != nil { + return err + } + + if balance < (amount + fee) { + return fmt.Errorf("balance %d is too low for this transaction to be executed: amount %d + fee %d", balance, amount, fee) + } + return nil +} + // goroutine that simulates tx (use a bounded number of goroutines to pick from queue?) // simulate can cancel the send retry function early in the tx management process // additionally, it can provide reasons for why a tx failed in the logs @@ -526,11 +566,11 @@ func (txm *Txm) simulate() { case <-ctx.Done(): return case msg := <-txm.chSim: - res, err := txm.simulateTx(ctx, &msg.tx) + res, err := txm.simulateTx(ctx, &msg.Tx) if err != nil { // this error can occur if endpoint goes down or if invalid signature (invalid signature should occur further upstream in sendWithRetry) // allow retry to continue in case temporary endpoint failure (if still invalid, confirmation or timeout will cleanup) - txm.lggr.Debugw("failed to simulate tx", "id", msg.id, "signatures", msg.signatures, "error", err) + txm.lggr.Debugw("failed to simulate tx", "id", msg.UUID, "signatures", msg.signatures, "error", err) continue } @@ -540,18 +580,8 @@ func (txm *Txm) simulate() { } // Transaction has to have a signature if simulation succeeded but added check for belt and braces approach - if len(msg.signatures) == 0 { - continue - } - // Process error to determine the corresponding state and type. - // Certain errors can be considered not to be failures during simulation to allow the process to continue - if txState, errType := txm.processError(msg.signatures[0], res.Err, true); errType != NoFailure { - id, err := txm.txs.OnError(msg.signatures[0], txm.cfg.TxRetentionTimeout(), txState, errType) - if err != nil { - txm.lggr.Errorw(fmt.Sprintf("failed to mark transaction as %s", txState.String()), "id", id, "err", err) - } else { - txm.lggr.Debugw(fmt.Sprintf("marking transaction as %s", txState.String()), "id", id, "signature", msg.signatures[0], "error", res.Err) - } + if len(msg.signatures) > 0 { + txm.processSimulationError(msg.UUID, msg.signatures[0], res) } } } @@ -580,24 +610,20 @@ func (txm *Txm) reap() { } // Enqueue enqueues a msg destined for the solana chain. -func (txm *Txm) Enqueue(ctx context.Context, accountID string, tx *solanaGo.Transaction, txID *string, txCfgs ...SetTxConfig) error { +func (txm *Txm) Enqueue(ctx context.Context, accountID string, msg *PendingTx, txCfgs ...SetTxConfig) error { if err := txm.Ready(); err != nil { return fmt.Errorf("error in soltxm.Enqueue: %w", err) } - // validate nil pointer - if tx == nil { - return errors.New("error in soltxm.Enqueue: tx is nil pointer") - } - // validate account keys slice - if len(tx.Message.AccountKeys) == 0 { - return errors.New("error in soltxm.Enqueue: not enough account keys in tx") + // validate msg and tx are not empty + if msg == nil || isEmptyTransactionAccountKeys(msg.Tx) { + return errors.New("error in soltxm.Enqueue: tx or account keys are empty") } // validate expected key exists by trying to sign with it // fee payer account is index 0 account // https://github.com/gagliardetto/solana-go/blob/main/transaction.go#L252 - _, err := txm.ks.Sign(ctx, tx.Message.AccountKeys[0].String(), nil) + _, err := txm.ks.Sign(ctx, msg.Tx.Message.AccountKeys[0].String(), nil) if err != nil { return fmt.Errorf("error in soltxm.Enqueue.GetKey: %w", err) } @@ -617,7 +643,7 @@ func (txm *Txm) Enqueue(ctx context.Context, accountID string, tx *solanaGo.Tran // Perform compute unit limit estimation after storing transaction // If error found during simulation, transaction should be in storage to mark accordingly if cfg.EstimateComputeUnitLimit { - computeUnitLimit, err := txm.EstimateComputeUnitLimit(ctx, tx, id) + computeUnitLimit, err := txm.EstimateComputeUnitLimit(ctx, &msg.Tx) if err != nil { return fmt.Errorf("transaction failed simulation: %w", err) } @@ -627,14 +653,14 @@ func (txm *Txm) Enqueue(ctx context.Context, accountID string, tx *solanaGo.Tran } } - msg := pendingTx{ - tx: *tx, - cfg: cfg, - id: id, + msg.cfg = cfg + // If ID was not set by caller, create one. + if msg.UUID == "" { + msg.UUID = uuid.New().String() } select { - case txm.chSend <- msg: + case txm.chSend <- *msg: default: txm.lggr.Errorw("failed to enqueue tx", "queueFull", len(txm.chSend) == MaxQueueLen, "tx", msg) return fmt.Errorf("failed to enqueue transaction for %s", accountID) @@ -826,3 +852,8 @@ func (txm *Txm) defaultTxConfig() TxConfig { EstimateComputeUnitLimit: txm.cfg.EstimateComputeUnitLimit(), } } + +// isEmptyTransactionAccountKeys validates that a solana tx and its account keys are not empty. +func isEmptyTransactionAccountKeys(tx solana.Transaction) bool { + return len(tx.Signatures) == 0 && len(tx.Message.AccountKeys) == 0 +} diff --git a/pkg/solana/txm/txm_internal_test.go b/pkg/solana/txm/txm_internal_test.go index 418bdbec1..c37b0d310 100644 --- a/pkg/solana/txm/txm_internal_test.go +++ b/pkg/solana/txm/txm_internal_test.go @@ -28,7 +28,6 @@ import ( relayconfig "github.com/smartcontractkit/chainlink-common/pkg/config" "github.com/smartcontractkit/chainlink-common/pkg/logger" "github.com/smartcontractkit/chainlink-common/pkg/types" - commontypes "github.com/smartcontractkit/chainlink-common/pkg/types" "github.com/smartcontractkit/chainlink-common/pkg/utils" bigmath "github.com/smartcontractkit/chainlink-common/pkg/utils/big_math" "github.com/smartcontractkit/chainlink-common/pkg/utils/tests" @@ -127,6 +126,13 @@ func TestTxm(t *testing.T) { cfg := config.NewDefault() cfg.Chain.FeeEstimatorMode = &estimator mc := mocks.NewReaderWriter(t) + blockhash, _ := solana.HashFromBase58("blockhash") + mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + Blockhash: blockhash, + LastValidBlockHeight: uint64(2000), + }, + }, nil) mc.On("GetLatestBlock", mock.Anything).Return(&rpc.GetBlockResult{}, nil).Maybe() mc.On("SlotHeight", mock.Anything).Return(uint64(0), nil).Maybe() @@ -138,6 +144,7 @@ func TestTxm(t *testing.T) { txm := NewTxm(id, loader, nil, cfg, mkey, lggr) require.NoError(t, txm.Start(ctx)) t.Cleanup(func() { require.NoError(t, txm.Close()) }) + t.Cleanup(func() { require.NoError(t, txm.Close()) }) // tracking prom metrics prom := soltxmProm{id: id} @@ -204,7 +211,7 @@ func TestTxm(t *testing.T) { // send tx testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID})) wg.Wait() // no transactions stored inflight txs list @@ -240,7 +247,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID})) wg.Wait() // wait to be picked up and processed // no transactions stored inflight txs list @@ -254,7 +261,6 @@ func TestTxm(t *testing.T) { _, err := txm.GetTransactionStatus(ctx, testTxID) require.Error(t, err) // transaction cleared from storage after finalized should not return status }) - // tx fails simulation (simulation error) t.Run("fail_simulation", func(t *testing.T) { tx, signed := getTx(t, 2, mkey) @@ -272,7 +278,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID})) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared quickly @@ -308,7 +314,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID})) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared after timeout @@ -348,7 +354,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID})) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared after timeout @@ -363,7 +369,6 @@ func TestTxm(t *testing.T) { // panic if sendTx called after context cancelled mc.On("SendTx", mock.Anything, tx).Panic("SendTx should not be called anymore").Maybe() }) - // tx fails simulation with BlockHashNotFound error // txm should continue to finalize tx (in this case it will succeed) t.Run("fail_simulation_blockhashNotFound", func(t *testing.T) { @@ -399,7 +404,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID})) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared after timeout @@ -441,7 +446,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID})) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared after timeout @@ -486,7 +491,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID})) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // inflight txs cleared after timeout @@ -538,7 +543,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID})) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // inflight txs cleared after timeout @@ -576,7 +581,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID})) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // inflight txs cleared after timeout @@ -622,7 +627,7 @@ func TestTxm(t *testing.T) { // send tx testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID})) wg.Wait() // no transactions stored inflight txs list @@ -676,7 +681,7 @@ func TestTxm(t *testing.T) { // send tx - with disabled fee bumping testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, SetFeeBumpPeriod(0))) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID}, SetFeeBumpPeriod(0))) wg.Wait() // no transactions stored inflight txs list @@ -728,7 +733,7 @@ func TestTxm(t *testing.T) { // send tx - with disabled fee bumping and disabled compute unit limit testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, SetFeeBumpPeriod(0), SetComputeUnitLimit(0))) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID}, SetFeeBumpPeriod(0), SetComputeUnitLimit(0))) wg.Wait() // no transactions stored inflight txs list @@ -766,6 +771,13 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { // Enable retention timeout to keep transactions after finality cfg.Chain.TxRetentionTimeout = relayconfig.MustNewDuration(5 * time.Second) mc := mocks.NewReaderWriter(t) + blockhash, err := solana.HashFromBase58("blockhash") + mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + Blockhash: blockhash, + LastValidBlockHeight: uint64(2000), + }, + }, nil) mc.On("GetLatestBlock", mock.Anything).Return(&rpc.GetBlockResult{}, nil).Maybe() computeUnitLimitDefault := fees.ComputeUnitLimit(cfg.ComputeUnitLimitDefault()) @@ -778,6 +790,7 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { txm := NewTxm(id, loader, nil, cfg, mkey, lggr) require.NoError(t, txm.Start(ctx)) t.Cleanup(func() { require.NoError(t, txm.Close()) }) + t.Cleanup(func() { require.NoError(t, txm.Close()) }) // tracking prom metrics prom := soltxmProm{id: id} @@ -836,7 +849,7 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID})) wg.Wait() // wait to be picked up and processed waitFor(t, 5*time.Second, txm, prom, empty) // inflight txs cleared after timeout @@ -965,6 +978,13 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { // Enable retention timeout to keep transactions after finality or error cfg.Chain.TxRetentionTimeout = relayconfig.MustNewDuration(5 * time.Second) mc := mocks.NewReaderWriter(t) + blockhash, _ := solana.HashFromBase58("blockhash") + mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + Blockhash: blockhash, + LastValidBlockHeight: uint64(2000), + }, + }, nil) mc.On("GetLatestBlock", mock.Anything).Return(&rpc.GetBlockResult{}, nil).Maybe() // mock solana keystore @@ -975,6 +995,7 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { txm := NewTxm(id, loader, nil, cfg, mkey, lggr) require.NoError(t, txm.Start(ctx)) t.Cleanup(func() { require.NoError(t, txm.Close()) }) + t.Cleanup(func() { require.NoError(t, txm.Close()) }) // tracking prom metrics prom := soltxmProm{id: id} @@ -1040,7 +1061,7 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { // send tx testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID})) wg.Wait() // no transactions stored inflight txs list @@ -1069,7 +1090,7 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { mc.On("SimulateTx", mock.Anything, mock.Anything, mock.Anything).Return(nil, errors.New("simulation failed")).Once() // tx should NOT be able to queue - assert.Error(t, txm.Enqueue(ctx, t.Name(), tx, nil)) + assert.Error(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx})) }) t.Run("simulation_returns_error", func(t *testing.T) { @@ -1085,11 +1106,7 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { txID := uuid.NewString() // tx should NOT be able to queue - assert.Error(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) - // tx should be stored in-memory and moved to errored state - status, err := txm.GetTransactionStatus(ctx, txID) - require.NoError(t, err) - require.Equal(t, commontypes.Failed, status) + assert.Error(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx})) }) } @@ -1098,6 +1115,13 @@ func TestTxm_Enqueue(t *testing.T) { lggr := logger.Test(t) cfg := config.NewDefault() mc := mocks.NewReaderWriter(t) + blockhash, _ := solana.HashFromBase58("blockhash") + mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + Blockhash: blockhash, + LastValidBlockHeight: uint64(2000), + }, + }, nil) mc.On("SendTx", mock.Anything, mock.Anything).Return(solana.Signature{}, nil).Maybe() mc.On("SimulateTx", mock.Anything, mock.Anything, mock.Anything).Return(&rpc.SimulateTransactionResult{}, nil).Maybe() mc.On("SignatureStatuses", mock.Anything, mock.AnythingOfType("[]solana.Signature")).Return( @@ -1147,7 +1171,7 @@ func TestTxm_Enqueue(t *testing.T) { loader := utils.NewLazyLoad(func() (client.ReaderWriter, error) { return mc, nil }) txm := NewTxm("enqueue_test", loader, nil, cfg, mkey, lggr) - require.ErrorContains(t, txm.Enqueue(ctx, "txmUnstarted", &solana.Transaction{}, nil), "not started") + require.ErrorContains(t, txm.Enqueue(ctx, "txmUnstarted", &PendingTx{}), "not started") require.NoError(t, txm.Start(ctx)) t.Cleanup(func() { require.NoError(t, txm.Close()) }) @@ -1158,31 +1182,16 @@ func TestTxm_Enqueue(t *testing.T) { }{ {"success", tx, false}, {"invalid_key", invalidTx, true}, - {"nil_pointer", nil, true}, {"empty_tx", &solana.Transaction{}, true}, } for _, run := range txs { t.Run(run.name, func(t *testing.T) { if !run.fail { - assert.NoError(t, txm.Enqueue(ctx, run.name, run.tx, nil)) + assert.NoError(t, txm.Enqueue(ctx, run.name, &PendingTx{Tx: *run.tx})) return } - assert.Error(t, txm.Enqueue(ctx, run.name, run.tx, nil)) + assert.Error(t, txm.Enqueue(ctx, run.name, &PendingTx{Tx: *run.tx})) }) } } - -func addSigAndLimitToTx(t *testing.T, keystore SimpleKeystore, pubkey solana.PublicKey, tx solana.Transaction, limit fees.ComputeUnitLimit) *solana.Transaction { - txCopy := tx - // sign tx - txMsg, err := tx.Message.MarshalBinary() - require.NoError(t, err) - sigBytes, err := keystore.Sign(context.Background(), pubkey.String(), txMsg) - require.NoError(t, err) - var sig [64]byte - copy(sig[:], sigBytes) - txCopy.Signatures = append(txCopy.Signatures, sig) - require.NoError(t, fees.SetComputeUnitLimit(&txCopy, limit)) - return &txCopy -} diff --git a/pkg/solana/txm/txm_load_test.go b/pkg/solana/txm/txm_load_test.go index 5d5a8061b..f37e166d2 100644 --- a/pkg/solana/txm/txm_load_test.go +++ b/pkg/solana/txm/txm_load_test.go @@ -1,6 +1,6 @@ //go:build integration -package txm_test +package txm import ( "context" @@ -14,14 +14,13 @@ import ( "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" - "github.com/smartcontractkit/chainlink-common/pkg/services/servicetest" solanaClient "github.com/smartcontractkit/chainlink-solana/pkg/solana/client" "github.com/smartcontractkit/chainlink-solana/pkg/solana/config" - "github.com/smartcontractkit/chainlink-solana/pkg/solana/txm" keyMocks "github.com/smartcontractkit/chainlink-solana/pkg/solana/txm/mocks" relayconfig "github.com/smartcontractkit/chainlink-common/pkg/config" "github.com/smartcontractkit/chainlink-common/pkg/logger" + "github.com/smartcontractkit/chainlink-common/pkg/services/servicetest" "github.com/smartcontractkit/chainlink-common/pkg/utils" "github.com/smartcontractkit/chainlink-common/pkg/utils/tests" ) @@ -72,7 +71,7 @@ func TestTxm_Integration(t *testing.T) { client, err := solanaClient.NewClient(url, cfg, 2*time.Second, lggr) require.NoError(t, err) loader := utils.NewLazyLoad(func() (solanaClient.ReaderWriter, error) { return client, nil }) - txm := txm.NewTxm("localnet", loader, nil, cfg, mkey, lggr) + txm := NewTxm("localnet", loader, nil, cfg, mkey, lggr) // track initial balance initBal, err := client.Balance(ctx, pubKey) @@ -84,9 +83,8 @@ func TestTxm_Integration(t *testing.T) { // already started assert.Error(t, txm.Start(ctx)) - createTx := func(signer solana.PublicKey, sender solana.PublicKey, receiver solana.PublicKey, amt uint64) *solana.Transaction { + createMsgWithTx := func(signer solana.PublicKey, sender solana.PublicKey, receiver solana.PublicKey, amt uint64) *PendingTx { // create transfer tx - hash, err := client.LatestBlockhash(ctx) assert.NoError(t, err) tx, err := solana.NewTransaction( []solana.Instruction{ @@ -96,24 +94,24 @@ func TestTxm_Integration(t *testing.T) { receiver, ).Build(), }, - hash.Value.Blockhash, + solana.Hash{}, solana.TransactionPayer(signer), ) require.NoError(t, err) - return tx + return &PendingTx{Tx: *tx} } // enqueue txs (must pass to move on to load test) - require.NoError(t, txm.Enqueue(ctx, "test_success_0", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) - require.Error(t, txm.Enqueue(ctx, "test_invalidSigner", createTx(pubKeyReceiver, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) // cannot sign tx before enqueuing - require.NoError(t, txm.Enqueue(ctx, "test_invalidReceiver", createTx(pubKey, pubKey, solana.PublicKey{}, solana.LAMPORTS_PER_SOL), nil)) + require.NoError(t, txm.Enqueue(ctx, "test_success_0", createMsgWithTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL))) + require.Error(t, txm.Enqueue(ctx, "test_invalidSigner", createMsgWithTx(pubKeyReceiver, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL))) // cannot sign tx before enqueuing + require.NoError(t, txm.Enqueue(ctx, "test_invalidReceiver", createMsgWithTx(pubKey, pubKey, solana.PublicKey{}, solana.LAMPORTS_PER_SOL))) time.Sleep(500 * time.Millisecond) // pause 0.5s for new blockhash - require.NoError(t, txm.Enqueue(ctx, "test_success_1", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) - require.NoError(t, txm.Enqueue(ctx, "test_txFail", createTx(pubKey, pubKey, pubKeyReceiver, 1000*solana.LAMPORTS_PER_SOL), nil)) + require.NoError(t, txm.Enqueue(ctx, "test_success_1", createMsgWithTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL))) + require.NoError(t, txm.Enqueue(ctx, "test_txFail", createMsgWithTx(pubKey, pubKey, pubKeyReceiver, 1000*solana.LAMPORTS_PER_SOL))) // load test: try to overload txs, confirm, or simulation for i := 0; i < 1000; i++ { - assert.NoError(t, txm.Enqueue(ctx, fmt.Sprintf("load_%d", i), createTx(loadTestKey.PublicKey(), loadTestKey.PublicKey(), loadTestKey.PublicKey(), uint64(i)), nil)) + assert.NoError(t, txm.Enqueue(ctx, fmt.Sprintf("load_%d", i), createMsgWithTx(loadTestKey.PublicKey(), loadTestKey.PublicKey(), loadTestKey.PublicKey(), uint64(i)))) time.Sleep(10 * time.Millisecond) // ~100 txs per second (note: have run 5ms delays for ~200tx/s succesfully) } diff --git a/pkg/solana/txm/txm_race_test.go b/pkg/solana/txm/txm_race_test.go index 42062718f..a25903cac 100644 --- a/pkg/solana/txm/txm_race_test.go +++ b/pkg/solana/txm/txm_race_test.go @@ -8,7 +8,9 @@ import ( "testing" "time" + "github.com/gagliardetto/solana-go" solanaGo "github.com/gagliardetto/solana-go" + "github.com/gagliardetto/solana-go/rpc" "go.uber.org/zap/zapcore" "github.com/smartcontractkit/chainlink-common/pkg/logger" @@ -27,10 +29,10 @@ import ( "github.com/stretchr/testify/require" ) -func NewTestMsg() (msg pendingTx) { +func NewTestMsg() (msg PendingTx) { tx := solanaGo.Transaction{} tx.Message.AccountKeys = append(tx.Message.AccountKeys, solanaGo.PublicKey{}) - msg.tx = tx + msg.Tx = tx return msg } @@ -62,6 +64,7 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { // assemble minimal tx for testing retry msg := NewTestMsg() + blockhash, _ := solana.HashFromBase58("blockhash") testRunner := func(t *testing.T, client solanaClient.ReaderWriter) { // build minimal txm @@ -117,6 +120,12 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { }, nil, ) + client.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + Blockhash: blockhash, + LastValidBlockHeight: uint64(2000), + }, + }, nil) testRunner(t, client) }) @@ -153,6 +162,12 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { }, nil, ) + client.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + Blockhash: blockhash, + LastValidBlockHeight: uint64(2000), + }, + }, nil) testRunner(t, client) }) @@ -200,6 +215,12 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { }, nil, ) + client.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + Blockhash: blockhash, + LastValidBlockHeight: uint64(2000), + }, + }, nil) testRunner(t, client) }) @@ -207,34 +228,39 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { client := clientmocks.NewReaderWriter(t) // client mock - first tx is always successful msg0 := NewTestMsg() - require.NoError(t, fees.SetComputeUnitPrice(&msg0.tx, 0)) - require.NoError(t, fees.SetComputeUnitLimit(&msg0.tx, 200_000)) - msg0.tx.Signatures = make([]solanaGo.Signature, 1) - client.On("SendTx", mock.Anything, &msg0.tx).Return(solanaGo.Signature{1}, nil) + require.NoError(t, fees.SetComputeUnitPrice(&msg0.Tx, 0)) + require.NoError(t, fees.SetComputeUnitLimit(&msg0.Tx, 200_000)) + msg0.Tx.Signatures = make([]solanaGo.Signature, 1) + client.On("SendTx", mock.Anything, &msg0.Tx).Return(solanaGo.Signature{1}, nil) // init bump tx fails, rebroadcast is successful msg1 := NewTestMsg() - require.NoError(t, fees.SetComputeUnitPrice(&msg1.tx, 1)) - require.NoError(t, fees.SetComputeUnitLimit(&msg1.tx, 200_000)) - msg1.tx.Signatures = make([]solanaGo.Signature, 1) - client.On("SendTx", mock.Anything, &msg1.tx).Return(solanaGo.Signature{}, fmt.Errorf("BUMP FAILED")).Once() - client.On("SendTx", mock.Anything, &msg1.tx).Return(solanaGo.Signature{2}, nil) + require.NoError(t, fees.SetComputeUnitPrice(&msg1.Tx, 1)) + require.NoError(t, fees.SetComputeUnitLimit(&msg1.Tx, 200_000)) + msg1.Tx.Signatures = make([]solanaGo.Signature, 1) + client.On("SendTx", mock.Anything, &msg1.Tx).Return(solanaGo.Signature{}, fmt.Errorf("BUMP FAILED")).Once() + client.On("SendTx", mock.Anything, &msg1.Tx).Return(solanaGo.Signature{2}, nil) // init bump tx success, rebroadcast fails msg2 := NewTestMsg() - require.NoError(t, fees.SetComputeUnitPrice(&msg2.tx, 2)) - require.NoError(t, fees.SetComputeUnitLimit(&msg2.tx, 200_000)) - msg2.tx.Signatures = make([]solanaGo.Signature, 1) - client.On("SendTx", mock.Anything, &msg2.tx).Return(solanaGo.Signature{3}, nil).Once() - client.On("SendTx", mock.Anything, &msg2.tx).Return(solanaGo.Signature{}, fmt.Errorf("REBROADCAST FAILED")) + require.NoError(t, fees.SetComputeUnitPrice(&msg2.Tx, 2)) + require.NoError(t, fees.SetComputeUnitLimit(&msg2.Tx, 200_000)) + msg2.Tx.Signatures = make([]solanaGo.Signature, 1) + client.On("SendTx", mock.Anything, &msg2.Tx).Return(solanaGo.Signature{3}, nil).Once() + client.On("SendTx", mock.Anything, &msg2.Tx).Return(solanaGo.Signature{}, fmt.Errorf("REBROADCAST FAILED")) // always successful msg3 := NewTestMsg() - require.NoError(t, fees.SetComputeUnitPrice(&msg3.tx, 4)) - require.NoError(t, fees.SetComputeUnitLimit(&msg3.tx, 200_000)) - msg3.tx.Signatures = make([]solanaGo.Signature, 1) - client.On("SendTx", mock.Anything, &msg3.tx).Return(solanaGo.Signature{4}, nil) - + require.NoError(t, fees.SetComputeUnitPrice(&msg3.Tx, 4)) + require.NoError(t, fees.SetComputeUnitLimit(&msg3.Tx, 200_000)) + msg3.Tx.Signatures = make([]solanaGo.Signature, 1) + client.On("SendTx", mock.Anything, &msg3.Tx).Return(solanaGo.Signature{4}, nil) + client.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + Blockhash: blockhash, + LastValidBlockHeight: uint64(2000), + }, + }, nil) testRunner(t, client) }) } From 50dfef099817d5e3177d70c804cd7697db7692fb Mon Sep 17 00:00:00 2001 From: Farber98 Date: Thu, 14 Nov 2024 21:06:22 -0300 Subject: [PATCH 02/51] lastValidBlockHeight shouldn't be exported --- pkg/solana/txm/pendingtx.go | 2 +- pkg/solana/txm/txm.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/solana/txm/pendingtx.go b/pkg/solana/txm/pendingtx.go index 8b7cf4bf0..219c45c1d 100644 --- a/pkg/solana/txm/pendingtx.go +++ b/pkg/solana/txm/pendingtx.go @@ -54,7 +54,7 @@ type PendingTx struct { createTs time.Time retentionTs time.Time state TxState - LastValidBlockHeight uint64 // to track expiration + lastValidBlockHeight uint64 // to track expiration BalanceCheck bool From solana.PublicKey // to perform balanceCheck Amount uint64 // to perform balanceCheck diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index e9d9c1ec2..c45ebdb2a 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -206,7 +206,7 @@ func (txm *Txm) sendWithRetry(ctx context.Context, msg PendingTx) (solanaGo.Tran return solanaGo.Transaction{}, "", solanaGo.Signature{}, fmt.Errorf("failed to get blockhash in soltxm.sendWithRetry: %w", err) } msg.Tx.Message.RecentBlockhash = blockhash.Value.Blockhash - msg.LastValidBlockHeight = blockhash.Value.LastValidBlockHeight + msg.lastValidBlockHeight = blockhash.Value.LastValidBlockHeight // if requested, validate balance before sending transaction. if msg.BalanceCheck { From 4e545e27c36ea9b505174fb058696b59737f8a5a Mon Sep 17 00:00:00 2001 From: Farber98 Date: Thu, 14 Nov 2024 21:08:55 -0300 Subject: [PATCH 03/51] better comment --- pkg/solana/txm/txm.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index c45ebdb2a..01d812b32 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -194,13 +194,11 @@ func (txm *Txm) sendWithRetry(ctx context.Context, msg PendingTx) (solanaGo.Tran return fees.ComputeUnitPrice(fee) } - // Get client + // Assign blockhash to msg client, err := txm.client.Get() if err != nil { return solanaGo.Transaction{}, "", solanaGo.Signature{}, fmt.Errorf("failed to get client in soltxm.sendWithRetry: %w", err) } - - // Get blockhash and assign to msg blockhash, err := client.LatestBlockhash(ctx) if err != nil { return solanaGo.Transaction{}, "", solanaGo.Signature{}, fmt.Errorf("failed to get blockhash in soltxm.sendWithRetry: %w", err) From 4ded53ca453764f33af578940dc6791f2a6a98e4 Mon Sep 17 00:00:00 2001 From: Farber98 Date: Fri, 15 Nov 2024 17:37:50 -0300 Subject: [PATCH 04/51] refactor sendWithRetry to make it clearer --- pkg/solana/txm/txm.go | 378 +++++++++++++++++++++++------------------- 1 file changed, 206 insertions(+), 172 deletions(-) diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index 01d812b32..b4eb44479 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -176,213 +176,250 @@ func (txm *Txm) run() { } } +// sendWithRetry attempts to send a transaction with exponential backoff retry logic. +// It prepares the transaction, builds and signs it, sends the initial transaction, and starts a retry routine with fee bumping if needed. +// The function returns the signed transaction, its ID, and the initial signature for use in simulation. func (txm *Txm) sendWithRetry(ctx context.Context, msg PendingTx) (solanaGo.Transaction, string, solanaGo.Signature, error) { - // get key - // fee payer account is index 0 account - // https://github.com/gagliardetto/solana-go/blob/main/transaction.go#L252 - key := msg.Tx.Message.AccountKeys[0].String() + // Prepare transaction assigning blockhash and lastValidBlockHeight (for expiration tracking). + // If required, it also performs balanceCheck and sets compute unit limit. + if err := txm.prepareTransaction(ctx, &msg); err != nil { + return solanaGo.Transaction{}, "", solanaGo.Signature{}, err + } - // base compute unit price should only be calculated once - // prevent underlying base changing when bumping (could occur with RPC based estimation) - getFee := func(count int) fees.ComputeUnitPrice { - fee := fees.CalculateFee( - msg.cfg.BaseComputeUnitPrice, - msg.cfg.ComputeUnitPriceMax, - msg.cfg.ComputeUnitPriceMin, - uint(count), //nolint:gosec // reasonable number of bumps should never cause overflow - ) - return fees.ComputeUnitPrice(fee) + // Build and sign initial transaction setting compute unit price + initTx, err := txm.buildTx(ctx, msg, 0) + if err != nil { + return solanaGo.Transaction{}, "", solanaGo.Signature{}, err + } + + // Create timeout context + ctx, cancel := context.WithTimeout(ctx, msg.cfg.Timeout) + + // Send initial transaction + sig, err := txm.sendInitialTx(ctx, initTx, msg, cancel) + if err != nil { + return solanaGo.Transaction{}, "", solanaGo.Signature{}, err + } + + // Initialize signature list with initialTx signature. This list will be used to add new signatures and track retry attempts. + sigs := &signatureList{} + sigs.Allocate() + if initSetErr := sigs.Set(0, sig); initSetErr != nil { + return solanaGo.Transaction{}, "", solanaGo.Signature{}, fmt.Errorf("failed to save initial signature in signature list: %w", initSetErr) } - // Assign blockhash to msg + // Start retry routine + // pass in copy of msg (to build new tx with bumped fee) and broadcasted tx == initTx (to retry tx without bumping) + txm.done.Add(1) + go func() { + defer txm.done.Done() + txm.retryTx(ctx, msg, initTx, sigs) + }() + + // Return signed tx, id, signature for use in simulation + return initTx, msg.UUID, sig, nil +} + +// prepareTransaction sets blockhash and lastValidBlockHeight which will be used to track expiration. +// If required, it also performs balanceCheck and sets compute unit limit. +func (txm *Txm) prepareTransaction(ctx context.Context, msg *PendingTx) error { client, err := txm.client.Get() if err != nil { - return solanaGo.Transaction{}, "", solanaGo.Signature{}, fmt.Errorf("failed to get client in soltxm.sendWithRetry: %w", err) + return fmt.Errorf("failed to get client in sendWithRetry: %w", err) } + + // Assign blockhash blockhash, err := client.LatestBlockhash(ctx) if err != nil { - return solanaGo.Transaction{}, "", solanaGo.Signature{}, fmt.Errorf("failed to get blockhash in soltxm.sendWithRetry: %w", err) + return fmt.Errorf("failed to get blockhash: %w", err) } msg.Tx.Message.RecentBlockhash = blockhash.Value.Blockhash msg.lastValidBlockHeight = blockhash.Value.LastValidBlockHeight - // if requested, validate balance before sending transaction. + // Validate balance if required if msg.BalanceCheck { if err = solanaValidateBalance(ctx, client, msg.From, msg.Amount, msg.Tx.Message.ToBase64()); err != nil { - return solanaGo.Transaction{}, "", solanaGo.Signature{}, fmt.Errorf("failed to validate balance: %w", err) + return fmt.Errorf("failed to validate balance: %w", err) } } - // set baseTx as a copy of the transaction - baseTx := msg.Tx - - // add compute unit limit instruction - static for the transaction - // skip if compute unit limit = 0 (otherwise would always fail) + // Set compute unit limit if msg.cfg.ComputeUnitLimit != 0 { - if computeUnitLimitErr := fees.SetComputeUnitLimit(&baseTx, fees.ComputeUnitLimit(msg.cfg.ComputeUnitLimit)); computeUnitLimitErr != nil { - return solanaGo.Transaction{}, "", solanaGo.Signature{}, fmt.Errorf("failed to add compute unit limit instruction: %w", computeUnitLimitErr) + if err := fees.SetComputeUnitLimit(&msg.Tx, fees.ComputeUnitLimit(msg.cfg.ComputeUnitLimit)); err != nil { + return fmt.Errorf("failed to add compute unit limit instruction: %w", err) } } - buildTx := func(ctx context.Context, base solanaGo.Transaction, retryCount int) (solanaGo.Transaction, error) { - newTx := base // make copy - - // set fee - // fee bumping can be enabled by moving the setting & signing logic to the broadcaster - if computeUnitErr := fees.SetComputeUnitPrice(&newTx, getFee(retryCount)); computeUnitErr != nil { - return solanaGo.Transaction{}, computeUnitErr - } + return nil +} - // sign tx - txMsg, marshalErr := newTx.Message.MarshalBinary() - if marshalErr != nil { - return solanaGo.Transaction{}, fmt.Errorf("error in soltxm.SendWithRetry.MarshalBinary: %w", marshalErr) - } - sigBytes, signErr := txm.ks.Sign(ctx, key, txMsg) - if signErr != nil { - return solanaGo.Transaction{}, fmt.Errorf("error in soltxm.SendWithRetry.Sign: %w", signErr) - } - var finalSig [64]byte - copy(finalSig[:], sigBytes) - newTx.Signatures = append(newTx.Signatures, finalSig) +// buildTx builds and signs the transaction with the appropriate compute unit price. +func (txm *Txm) buildTx(ctx context.Context, msg PendingTx, retryCount int) (solanaGo.Transaction, error) { + // work with a copy + newTx := msg.Tx - return newTx, nil + // Set compute unit price (fee) + fee := fees.ComputeUnitPrice( + fees.CalculateFee( + msg.cfg.BaseComputeUnitPrice, + msg.cfg.ComputeUnitPriceMax, + msg.cfg.ComputeUnitPriceMin, + uint(retryCount), //nolint:gosec // reasonable number of bumps should never cause overflow + )) + if err := fees.SetComputeUnitPrice(&newTx, fee); err != nil { + return solanaGo.Transaction{}, err } - initTx, initBuildErr := buildTx(ctx, baseTx, 0) - if initBuildErr != nil { - return solanaGo.Transaction{}, "", solanaGo.Signature{}, initBuildErr + // Sign transaction + // NOTE: fee payer account is index 0 account. https://github.com/gagliardetto/solana-go/blob/main/transaction.go#L252 + txMsg, err := newTx.Message.MarshalBinary() + if err != nil { + return solanaGo.Transaction{}, fmt.Errorf("error in MarshalBinary: %w", err) + } + sigBytes, err := txm.ks.Sign(ctx, msg.Tx.Message.AccountKeys[0].String(), txMsg) + if err != nil { + return solanaGo.Transaction{}, fmt.Errorf("error in Sign: %w", err) } + var finalSig [64]byte + copy(finalSig[:], sigBytes) + newTx.Signatures = append(newTx.Signatures, finalSig) - // create timeout context - ctx, cancel := context.WithTimeout(ctx, msg.cfg.Timeout) + return newTx, nil +} - // send initial tx (do not retry and exit early if fails) - sig, initSendErr := txm.sendTx(ctx, &initTx) - if initSendErr != nil { - cancel() // cancel context when exiting early - stateTransitionErr := txm.txs.OnPrebroadcastError(msg.id, txm.cfg.TxRetentionTimeout(), Errored, TxFailReject) - return solanaGo.Transaction{}, "", solanaGo.Signature{}, fmt.Errorf("tx failed initial transmit: %w", errors.Join(initSendErr, stateTransitionErr)) +// sendInitialTx sends the initial tx and handles any errors that may occur. It also stores the transaction signature and cancellation function. +func (txm *Txm) sendInitialTx(ctx context.Context, initTx solanaGo.Transaction, msg PendingTx, cancel context.CancelFunc) (solanaGo.Signature, error) { + // Send initial transaction + sig, err := txm.sendTx(ctx, &initTx) + if err != nil { + // do not retry and exit early if fails + cancel() + txm.txs.OnError(sig, txm.cfg.TxRetentionTimeout(), TxFailReject) //nolint // no need to check error since only incrementing metric here + return solanaGo.Signature{}, fmt.Errorf("tx failed initial transmit: %w", err) } - // store tx signature + cancel function - initStoreErr := txm.txs.New(msg, sig, cancel) - if initStoreErr != nil { + // Store tx signature and cancel function + if err := txm.txs.New(msg, sig, cancel); err != nil { cancel() // cancel context when exiting early - return solanaGo.Transaction{}, "", solanaGo.Signature{}, fmt.Errorf("failed to save tx signature (%s) to inflight txs: %w", sig, initStoreErr) - } - - // used for tracking rebroadcasting only in SendWithRetry - var sigs signatureList - sigs.Allocate() - if initSetErr := sigs.Set(0, sig); initSetErr != nil { - return solanaGo.Transaction{}, "", solanaGo.Signature{}, fmt.Errorf("failed to save initial signature in signature list: %w", initSetErr) + return solanaGo.Signature{}, fmt.Errorf("failed to save tx signature (%s) to inflight txs: %w", sig, err) } - txm.lggr.Debugw("tx initial broadcast", "id", msg.UUID, "fee", getFee(0), "signature", sig) + txm.lggr.Debugw("tx initial broadcast", "id", msg.UUID, "fee", msg.cfg.BaseComputeUnitPrice, "signature", sig) + return sig, nil +} - txm.done.Add(1) - // retry with exponential backoff - // until context cancelled by timeout or called externally - // pass in copy of baseTx (used to build new tx with bumped fee) and broadcasted tx == initTx (used to retry tx without bumping) - go func(ctx context.Context, baseTx, currentTx solanaGo.Transaction) { - defer txm.done.Done() - deltaT := 1 // ms - tick := time.After(0) - bumpCount := 0 - bumpTime := time.Now() - var wg sync.WaitGroup +// retryTx contains the logic for retrying the transaction, including exponential backoff and fee bumping. +// Retries until context cancelled by timeout or called externally. +// It uses handleRetry helper function to handle each retry attempt. +func (txm *Txm) retryTx(ctx context.Context, msg PendingTx, currentTx solanaGo.Transaction, sigs *signatureList) { + deltaT := 1 // initial delay in ms + tick := time.After(0) + bumpCount := 0 + bumpTime := time.Now() + var wg sync.WaitGroup - for { - select { - case <-ctx.Done(): - // stop sending tx after retry tx ctx times out (does not stop confirmation polling for tx) - wg.Wait() - txm.lggr.Debugw("stopped tx retry", "id", msg.UUID, "signatures", sigs.List(), "err", context.Cause(ctx)) - return - case <-tick: - var shouldBump bool - // bump if period > 0 and past time - if msg.cfg.FeeBumpPeriod != 0 && time.Since(bumpTime) > msg.cfg.FeeBumpPeriod { - bumpCount++ - bumpTime = time.Now() - shouldBump = true + for { + select { + case <-ctx.Done(): + // stop sending tx after retry tx ctx times out (does not stop confirmation polling for tx) + wg.Wait() + txm.lggr.Debugw("stopped tx retry", "id", msg.UUID, "signatures", sigs.List(), "err", context.Cause(ctx)) + return + case <-tick: + // Determine if we should bump the fee + shouldBump := txm.shouldBumpFee(msg.cfg.FeeBumpPeriod, bumpTime) + if shouldBump { + bumpCount++ + bumpTime = time.Now() + // Build new transaction with bumped fee and replace current tx + var err error + currentTx, err = txm.buildTx(ctx, msg, bumpCount) + if err != nil { + // Exit if unable to build transaction for retrying + txm.lggr.Errorw("failed to build bumped retry tx", "error", err, "id", msg.UUID) + return } - - // if fee should be bumped, build new tx and replace currentTx - if shouldBump { - var retryBuildErr error - currentTx, retryBuildErr = buildTx(ctx, baseTx, bumpCount) - if retryBuildErr != nil { - txm.lggr.Errorw("failed to build bumped retry tx", "error", retryBuildErr, "id", msg.UUID) - return // exit func if cannot build tx for retrying - } - ind := sigs.Allocate() - if ind != bumpCount { - txm.lggr.Errorw("INVARIANT VIOLATION: index (%d) != bumpCount (%d)", ind, bumpCount) - return - } + // allocates space for new signature that will be introduced in handleRetry if needs bumping. + index := sigs.Allocate() + if index != bumpCount { + txm.lggr.Errorw("invariant violation: index does not match bumpCount", "index", index, "bumpCount", bumpCount) + return } + } - // take currentTx and broadcast, if bumped fee -> save signature to list - wg.Add(1) - go func(bump bool, count int, retryTx solanaGo.Transaction) { - defer wg.Done() + // Start a goroutine to handle the retry attempt + // takes currentTx and rebroadcast. If needs bumping it will new signature to already allocated space in signatureList. + wg.Add(1) + go func(bump bool, count int, retryTx solanaGo.Transaction) { + defer wg.Done() + txm.handleRetry(ctx, msg, bump, count, retryTx, sigs) + }(shouldBump, bumpCount, currentTx) + } - retrySig, retrySendErr := txm.sendTx(ctx, &retryTx) - // this could occur if endpoint goes down or if ctx cancelled - if retrySendErr != nil { - if strings.Contains(retrySendErr.Error(), "context canceled") || strings.Contains(retrySendErr.Error(), "context deadline exceeded") { - txm.lggr.Debugw("ctx error on send retry transaction", "error", retrySendErr, "signatures", sigs.List(), "id", msg.UUID) - } else { - txm.lggr.Warnw("failed to send retry transaction", "error", retrySendErr, "signatures", sigs.List(), "id", msg.UUID) - } - return - } + // Update the exponential backoff delay + deltaT = txm.updateBackoffDelay(deltaT) + tick = time.After(time.Duration(deltaT) * time.Millisecond) + } +} - // save new signature if fee bumped - if bump { - if retryStoreErr := txm.txs.AddSignature(msg.UUID, retrySig); retryStoreErr != nil { - txm.lggr.Warnw("error in adding retry transaction", "error", retryStoreErr, "id", msg.UUID) - return - } - if setErr := sigs.Set(count, retrySig); setErr != nil { - // this should never happen - txm.lggr.Errorw("INVARIANT VIOLATION", "error", setErr) - } - txm.lggr.Debugw("tx rebroadcast with bumped fee", "id", msg.UUID, "fee", getFee(count), "signatures", sigs.List()) - } +// shouldBumpFee determines whether the fee should be bumped based on the fee bump period. +func (txm *Txm) shouldBumpFee(feeBumpPeriod time.Duration, lastBumpTime time.Time) bool { + return feeBumpPeriod != 0 && time.Since(lastBumpTime) > feeBumpPeriod +} - // prevent locking on waitgroup when ctx is closed - wait := make(chan struct{}) - go func() { - defer close(wait) - sigs.Wait(count) // wait until bump tx has set the tx signature to compare rebroadcast signatures - }() - select { - case <-ctx.Done(): - return - case <-wait: - } +// updateBackoffDelay updates the exponential backoff delay up to a maximum limit. +func (txm *Txm) updateBackoffDelay(currentDelay int) int { + newDelay := currentDelay * 2 + if newDelay > MaxRetryTimeMs { + return MaxRetryTimeMs + } + return newDelay +} - // this should never happen (should match the signature saved to sigs) - if fetchedSig, fetchErr := sigs.Get(count); fetchErr != nil || retrySig != fetchedSig { - txm.lggr.Errorw("original signature does not match retry signature", "expectedSignatures", sigs.List(), "receivedSignature", retrySig, "error", fetchErr) - } - }(shouldBump, bumpCount, currentTx) - } +// handleRetry handles the logic for each retry attempt, including sending the transaction, updating signatures, and logging. +func (txm *Txm) handleRetry(ctx context.Context, msg PendingTx, bump bool, count int, retryTx solanaGo.Transaction, sigs *signatureList) { + // send retry transaction + retrySig, err := txm.sendTx(ctx, &retryTx) + if err != nil { + // this could occur if endpoint goes down or if ctx cancelled + if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) { + txm.lggr.Debugw("ctx error on send retry transaction", "error", err, "signatures", sigs.List(), "id", msg.UUID) + } else { + txm.lggr.Warnw("failed to send retry transaction", "error", err, "signatures", sigs.List(), "id", msg.UUID) + } + return + } - // exponential increase in wait time, capped at 250ms - deltaT *= 2 - if deltaT > MaxRetryTimeMs { - deltaT = MaxRetryTimeMs - } - tick = time.After(time.Duration(deltaT) * time.Millisecond) + // if bump is true, update signature list and set new signature in space already allocated. + if bump { + if err := txm.txs.AddSignature(msg.UUID, retrySig); err != nil { + txm.lggr.Warnw("error in adding retry transaction", "error", err, "id", msg.UUID) + return + } + if err := sigs.Set(count, retrySig); err != nil { + // this should never happen + txm.lggr.Errorw("INVARIANT VIOLATION: failed to set signature", "error", err, "id", msg.UUID) + return } - }(ctx, baseTx, initTx) + txm.lggr.Debugw("tx rebroadcast with bumped fee", "id", msg.UUID, "retryCount", count, "fee", msg.cfg.BaseComputeUnitPrice, "signatures", sigs.List()) + } - // return signed tx, id, signature for use in simulation - return initTx, msg.UUID, sig, nil + // prevent locking on waitgroup when ctx is closed + wait := make(chan struct{}) + go func() { + defer close(wait) + sigs.Wait(count) // wait until bump tx has set the tx signature to compare rebroadcast signatures + }() + select { + case <-ctx.Done(): + return + case <-wait: + } + + // this should never happen (should match the signature saved to sigs) + if fetchedSig, err := sigs.Get(count); err != nil || retrySig != fetchedSig { + txm.lggr.Errorw("original signature does not match retry signature", "expectedSignatures", sigs.List(), "receivedSignature", retrySig, "error", err) + } } // goroutine that polls to confirm implementation @@ -790,18 +827,15 @@ func (txm *Txm) processError(sig solanaGo.Signature, resErr interface{}, simulat // blockhash not found when simulating, occurs when network bank has not seen the given blockhash or tx is too old // let confirmation process clean up case strings.Contains(errStr, "BlockhashNotFound"): - txm.lggr.Debugw("BlockhashNotFound", logValues...) - // return no failure for this error when simulating to allow later send/retry code to assign a proper blockhash - // in case the one provided by the caller is outdated - if simulation { - return txState, NoFailure - } - return Errored, errType - // transaction will encounter execution error/revert + txm.lggr.Debugw("simulate: BlockhashNotFound", logValues...) + // transaction will encounter execution error/revert, mark as reverted to remove from confirmation + retry case strings.Contains(errStr, "InstructionError"): - txm.lggr.Debugw("InstructionError", logValues...) - return Errored, errType - // transaction is already processed in the chain + _, err := txm.txs.OnError(sig, txm.cfg.TxRetentionTimeout(), TxFailSimRevert) // cancel retry + if err != nil { + logValues = append(logValues, "stateTransitionErr", err) + } + txm.lggr.Debugw("simulate: InstructionError", logValues...) + // transaction is already processed in the chain, letting txm confirmation handle case strings.Contains(errStr, "AlreadyProcessed"): txm.lggr.Debugw("AlreadyProcessed", logValues...) // return no failure for this error when simulating in case there is a race between broadcast and simulation From 9e1be6d8fad6c9db4fcc182ceb0499b844b6a945 Mon Sep 17 00:00:00 2001 From: Farber98 Date: Mon, 18 Nov 2024 19:34:14 -0300 Subject: [PATCH 05/51] confirm loop refactor --- pkg/solana/txm/txm.go | 276 ++++++++++++++++++++++-------------------- 1 file changed, 144 insertions(+), 132 deletions(-) diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index b4eb44479..2c64e2c0d 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -253,6 +253,23 @@ func (txm *Txm) prepareTransaction(ctx context.Context, msg *PendingTx) error { return nil } +func solanaValidateBalance(ctx context.Context, reader client.Reader, from solana.PublicKey, amount uint64, msg string) error { + balance, err := reader.Balance(ctx, from) + if err != nil { + return err + } + + fee, err := reader.GetFeeForMessage(ctx, msg) + if err != nil { + return err + } + + if balance < (amount + fee) { + return fmt.Errorf("balance %d is too low for this transaction to be executed: amount %d + fee %d", balance, amount, fee) + } + return nil +} + // buildTx builds and signs the transaction with the appropriate compute unit price. func (txm *Txm) buildTx(ctx context.Context, msg PendingTx, retryCount int) (solanaGo.Transaction, error) { // work with a copy @@ -435,157 +452,152 @@ func (txm *Txm) confirm() { case <-ctx.Done(): return case <-tick: - // get list of tx signatures to confirm - sigs := txm.txs.ListAll() - - // exit switch if not txs to confirm - if len(sigs) == 0 { - break - } - - // get client client, err := txm.client.Get() if err != nil { - txm.lggr.Errorw("failed to get client in soltxm.confirm", "error", err) - break // exit switch + txm.lggr.Errorw("failed to get client in txm.confirm", "error", err) + return } + txm.processConfirmations(ctx, client) + } + tick = time.After(utils.WithJitter(txm.cfg.ConfirmPollPeriod())) + } +} - // batch sigs no more than MaxSigsToConfirm each - sigsBatch, err := utils.BatchSplit(sigs, MaxSigsToConfirm) - if err != nil { // this should never happen - txm.lggr.Fatalw("failed to batch signatures", "error", err) - break // exit switch - } +func (txm *Txm) processConfirmations(ctx context.Context, client client.ReaderWriter) { + // Get list of transaction signatures to confirm + sigs := txm.txs.ListAll() - // process signatures - processSigs := func(s []solanaGo.Signature, res []*rpc.SignatureStatusesResult) { - // sort signatures and results process successful first - s, res, err := SortSignaturesAndResults(s, res) - if err != nil { - txm.lggr.Errorw("sorting error", "error", err) - return - } + if len(sigs) == 0 { + return + } - for i := 0; i < len(res); i++ { - // if status is nil (sig not found), continue polling - // sig not found could mean invalid tx or not picked up yet - if res[i] == nil { - txm.lggr.Debugw("tx state: not found", - "signature", s[i], - ) - - // check confirm timeout exceeded - if txm.cfg.TxConfirmTimeout() != 0*time.Second && txm.txs.Expired(s[i], txm.cfg.TxConfirmTimeout()) { - id, err := txm.txs.OnError(s[i], txm.cfg.TxRetentionTimeout(), Errored, TxFailDrop) - if err != nil { - txm.lggr.Infow("failed to mark transaction as errored", "id", id, "signature", s[i], "timeoutSeconds", txm.cfg.TxConfirmTimeout(), "error", err) - } else { - txm.lggr.Debugw("failed to find transaction within confirm timeout", "id", id, "signature", s[i], "timeoutSeconds", txm.cfg.TxConfirmTimeout()) - } - } - continue - } - - // if signature has an error, end polling - if res[i].Err != nil { - // Process error to determine the corresponding state and type. - // Skip marking as errored if error considered to not be a failure. - if txState, errType := txm.processError(s[i], res[i].Err, false); errType != NoFailure { - id, err := txm.txs.OnError(s[i], txm.cfg.TxRetentionTimeout(), txState, errType) - if err != nil { - txm.lggr.Infow(fmt.Sprintf("failed to mark transaction as %s", txState.String()), "id", id, "signature", s[i], "error", err) - } else { - txm.lggr.Debugw(fmt.Sprintf("marking transaction as %s", txState.String()), "id", id, "signature", s[i], "error", res[i].Err, "status", res[i].ConfirmationStatus) - } - } - continue - } - - // if signature is processed, keep polling for confirmed or finalized status - if res[i].ConfirmationStatus == rpc.ConfirmationStatusProcessed { - // update transaction state in local memory - id, err := txm.txs.OnProcessed(s[i]) - if err != nil && !errors.Is(err, ErrAlreadyInExpectedState) { - txm.lggr.Errorw("failed to mark transaction as processed", "signature", s[i], "error", err) - } else if err == nil { - txm.lggr.Debugw("marking transaction as processed", "id", id, "signature", s[i]) - } - // check confirm timeout exceeded if TxConfirmTimeout set - if txm.cfg.TxConfirmTimeout() != 0*time.Second && txm.txs.Expired(s[i], txm.cfg.TxConfirmTimeout()) { - id, err := txm.txs.OnError(s[i], txm.cfg.TxRetentionTimeout(), Errored, TxFailDrop) - if err != nil { - txm.lggr.Infow("failed to mark transaction as errored", "id", id, "signature", s[i], "timeoutSeconds", txm.cfg.TxConfirmTimeout(), "error", err) - } else { - txm.lggr.Debugw("tx failed to move beyond 'processed' within confirm timeout", "id", id, "signature", s[i], "timeoutSeconds", txm.cfg.TxConfirmTimeout()) - } - } - continue - } - - // if signature is confirmed, keep polling for finalized status - if res[i].ConfirmationStatus == rpc.ConfirmationStatusConfirmed { - id, err := txm.txs.OnConfirmed(s[i]) - if err != nil && !errors.Is(err, ErrAlreadyInExpectedState) { - txm.lggr.Errorw("failed to mark transaction as confirmed", "id", id, "signature", s[i], "error", err) - } else if err == nil { - txm.lggr.Debugw("marking transaction as confirmed", "id", id, "signature", s[i]) - } - continue - } - - // if signature is finalized, end polling - if res[i].ConfirmationStatus == rpc.ConfirmationStatusFinalized { - id, err := txm.txs.OnFinalized(s[i], txm.cfg.TxRetentionTimeout()) - if err != nil { - txm.lggr.Errorw("failed to mark transaction as finalized", "id", id, "signature", s[i], "error", err) - } else { - txm.lggr.Debugw("marking transaction as finalized", "id", id, "signature", s[i]) - } - continue - } - } - } + // batch sigs no more than MaxSigsToConfirm each + sigsBatch, err := utils.BatchSplit(sigs, MaxSigsToConfirm) + if err != nil { // this should never happen + txm.lggr.Fatalw("failed to batch signatures", "error", err) + return + } - // waitgroup for processing - var wg sync.WaitGroup + var wg sync.WaitGroup + for i := 0; i < len(sigsBatch); i++ { + // fetch signature statuses + statuses, err := client.SignatureStatuses(ctx, sigsBatch[i]) + if err != nil { + txm.lggr.Errorw("failed to get signature statuses in txm.confirm", "error", err) + break // exit for loop + } - // loop through batch - for i := 0; i < len(sigsBatch); i++ { - // fetch signature statuses - statuses, err := client.SignatureStatuses(ctx, sigsBatch[i]) - if err != nil { - txm.lggr.Errorw("failed to get signature statuses in soltxm.confirm", "error", err) - break // exit for loop - } + wg.Add(1) + // nonblocking: process batches as soon as they come in + go func(index int) { + defer wg.Done() + txm.processSignatureStatuses(sigsBatch[i], statuses) + }(i) + } + wg.Wait() // wait for processing to finish +} - wg.Add(1) - // nonblocking: process batches as soon as they come in - go func(index int) { - defer wg.Done() - processSigs(sigsBatch[index], statuses) - }(i) - } - wg.Wait() // wait for processing to finish +func (txm *Txm) processSignatureStatuses(sigs []solanaGo.Signature, res []*rpc.SignatureStatusesResult) { + // Sort signatures and results process successful first + sortedSigs, sortedRes, err := SortSignaturesAndResults(sigs, res) + if err != nil { + txm.lggr.Errorw("sorting error", "error", err) + return + } + + for i := 0; i < len(sortedRes); i++ { + sig, status := sortedSigs[i], sortedRes[i] + // if status is nil (sig not found), continue polling + // sig not found could mean invalid tx or not picked up yet + if status == nil { + txm.handleNotFoundSignatureStatus(sig) + continue + } + + // if signature has an error, end polling + if status.Err != nil { + txm.handleErrorSignatureStatus(sig, status) + continue + } + + switch status.ConfirmationStatus { + case rpc.ConfirmationStatusProcessed: + // if signature is processed, keep polling for confirmed or finalized status + txm.handleProcessedSignatureStatus(sig) + continue + case rpc.ConfirmationStatusConfirmed: + // if signature is confirmed, keep polling for finalized status + txm.handleConfirmedSignatureStatus(sig) + continue + case rpc.ConfirmationStatusFinalized: + // if signature is finalized, end polling + txm.handleFinalizedSignatureStatus(sig) + continue + default: + txm.lggr.Warnw("unknown confirmation status", "signature", sig, "status", status.ConfirmationStatus) + continue } - tick = time.After(utils.WithJitter(txm.cfg.ConfirmPollPeriod())) } } -func solanaValidateBalance(ctx context.Context, reader client.Reader, from solana.PublicKey, amount uint64, msg string) error { - balance, err := reader.Balance(ctx, from) - if err != nil { - return err +func (txm *Txm) handleNotFoundSignatureStatus(sig solanaGo.Signature) { + txm.lggr.Debugw("tx state: not found", "signature", sig) + + // check confirm timeout exceeded + if txm.txs.Expired(sig, txm.cfg.TxConfirmTimeout()) { + id, err := txm.txs.OnError(sig, txm.cfg.TxRetentionTimeout(), TxFailDrop) + if err != nil { + txm.lggr.Infow("failed to mark transaction as errored", "id", id, "signature", sig, "timeoutSeconds", txm.cfg.TxConfirmTimeout(), "error", err) + } else { + txm.lggr.Infow("failed to find transaction within confirm timeout", "id", id, "signature", sig, "timeoutSeconds", txm.cfg.TxConfirmTimeout()) + } } +} - fee, err := reader.GetFeeForMessage(ctx, msg) +func (txm *Txm) handleErrorSignatureStatus(sig solanaGo.Signature, status *rpc.SignatureStatusesResult) { + id, err := txm.txs.OnError(sig, txm.cfg.TxRetentionTimeout(), TxFailRevert) if err != nil { - return err + txm.lggr.Infow("failed to mark transaction as errored", "id", id, "signature", sig, "error", err) + } else { + txm.lggr.Debugw("tx state: failed", "id", id, "signature", sig, "error", status.Err, "status", status.ConfirmationStatus) } +} - if balance < (amount + fee) { - return fmt.Errorf("balance %d is too low for this transaction to be executed: amount %d + fee %d", balance, amount, fee) +func (txm *Txm) handleProcessedSignatureStatus(sig solanaGo.Signature) { + // update transaction state in local memory + id, err := txm.txs.OnProcessed(sig) + if err != nil && !errors.Is(err, ErrAlreadyInExpectedState) { + txm.lggr.Errorw("failed to mark transaction as processed", "signature", sig, "error", err) + } else if err == nil { + txm.lggr.Debugw("marking transaction as processed", "id", id, "signature", sig) + } + // check confirm timeout exceeded if TxConfirmTimeout set + if txm.cfg.TxConfirmTimeout() != 0*time.Second && txm.txs.Expired(sig, txm.cfg.TxConfirmTimeout()) { + id, err := txm.txs.OnError(sig, txm.cfg.TxRetentionTimeout(), TxFailDrop) + if err != nil { + txm.lggr.Infow("failed to mark transaction as errored", "id", id, "signature", sig, "timeoutSeconds", txm.cfg.TxConfirmTimeout(), "error", err) + } else { + txm.lggr.Debugw("tx failed to move beyond 'processed' within confirm timeout", "id", id, "signature", sig, "timeoutSeconds", txm.cfg.TxConfirmTimeout()) + } + } +} + +func (txm *Txm) handleConfirmedSignatureStatus(sig solanaGo.Signature) { + id, err := txm.txs.OnConfirmed(sig) + if err != nil && !errors.Is(err, ErrAlreadyInExpectedState) { + txm.lggr.Errorw("failed to mark transaction as confirmed", "id", id, "signature", sig, "error", err) + } else if err == nil { + txm.lggr.Debugw("marking transaction as confirmed", "id", id, "signature", sig) + } +} + +func (txm *Txm) handleFinalizedSignatureStatus(sig solanaGo.Signature) { + id, err := txm.txs.OnFinalized(sig, txm.cfg.TxRetentionTimeout()) + if err != nil { + txm.lggr.Errorw("failed to mark transaction as finalized", "id", id, "signature", sig, "error", err) + } else { + txm.lggr.Debugw("marking transaction as finalized", "id", id, "signature", sig) } - return nil } // goroutine that simulates tx (use a bounded number of goroutines to pick from queue?) From 7dd20284d0dceb6845eacc581b273f5abd7d12c4 Mon Sep 17 00:00:00 2001 From: Farber98 Date: Mon, 18 Nov 2024 20:58:43 -0300 Subject: [PATCH 06/51] fix infinite loop --- pkg/solana/txm/txm.go | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index 2c64e2c0d..cba421757 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -453,24 +453,24 @@ func (txm *Txm) confirm() { return case <-tick: client, err := txm.client.Get() + // Get list of transaction signatures to confirm + // If no signatures to confirm, we can break loop. + sigs := txm.txs.ListAll() + if len(sigs) == 0 { + break + } + if err != nil { txm.lggr.Errorw("failed to get client in txm.confirm", "error", err) return } - txm.processConfirmations(ctx, client) + txm.processConfirmations(ctx, client, sigs) } tick = time.After(utils.WithJitter(txm.cfg.ConfirmPollPeriod())) } } -func (txm *Txm) processConfirmations(ctx context.Context, client client.ReaderWriter) { - // Get list of transaction signatures to confirm - sigs := txm.txs.ListAll() - - if len(sigs) == 0 { - return - } - +func (txm *Txm) processConfirmations(ctx context.Context, client client.ReaderWriter, sigs []solanaGo.Signature) { // batch sigs no more than MaxSigsToConfirm each sigsBatch, err := utils.BatchSplit(sigs, MaxSigsToConfirm) if err != nil { // this should never happen From 6c675f281500afd7d0dd5061332cddf1b571a943 Mon Sep 17 00:00:00 2001 From: Farber98 Date: Mon, 18 Nov 2024 22:17:49 -0300 Subject: [PATCH 07/51] move accountID inside msg --- pkg/solana/chain.go | 5 ++-- pkg/solana/chain_test.go | 10 +++---- pkg/solana/relay.go | 2 +- pkg/solana/transmitter.go | 5 ++-- pkg/solana/transmitter_test.go | 2 +- pkg/solana/txm/pendingtx.go | 1 + pkg/solana/txm/txm.go | 6 ++--- pkg/solana/txm/txm_internal_test.go | 42 ++++++++++++++--------------- pkg/solana/txm/txm_load_test.go | 18 ++++++------- 9 files changed, 45 insertions(+), 46 deletions(-) diff --git a/pkg/solana/chain.go b/pkg/solana/chain.go index 436399853..a70e2a4a4 100644 --- a/pkg/solana/chain.go +++ b/pkg/solana/chain.go @@ -557,7 +557,8 @@ func (c *chain) sendTx(ctx context.Context, from, to string, amount *big.Int, ba } msg := &txm.PendingTx{ - Tx: *tx, + Tx: *tx, + AccountID: "", // To perform balanceCheck we need a blockhash. // Storing values to perform balanceCheck within sendWithRetry txm function before sending tx. BalanceCheck: balanceCheck, @@ -566,7 +567,7 @@ func (c *chain) sendTx(ctx context.Context, from, to string, amount *big.Int, ba } chainTxm := c.TxManager() - err = chainTxm.Enqueue(ctx, "", msg, + err = chainTxm.Enqueue(ctx, msg, txm.SetComputeUnitLimit(500), // reduce from default 200K limit - should only take 450 compute units // no fee bumping and no additional fee - makes validating balance accurate txm.SetComputeUnitPriceMax(0), diff --git a/pkg/solana/chain_test.go b/pkg/solana/chain_test.go index 75efc2d14..d8cf651c6 100644 --- a/pkg/solana/chain_test.go +++ b/pkg/solana/chain_test.go @@ -536,7 +536,7 @@ func TestSolanaChain_MultiNode_Txm(t *testing.T) { assert.NoError(t, err) assert.Equal(t, uint64(0), receiverBal) - createMsgWithTx := func(signer solana.PublicKey, sender solana.PublicKey, receiver solana.PublicKey, amt uint64) *txm.PendingTx { + createMsgWithTx := func(accountID string, signer solana.PublicKey, sender solana.PublicKey, receiver solana.PublicKey, amt uint64) *txm.PendingTx { assert.NoError(t, err) tx, txErr := solana.NewTransaction( []solana.Instruction{ @@ -550,11 +550,11 @@ func TestSolanaChain_MultiNode_Txm(t *testing.T) { solana.TransactionPayer(signer), ) require.NoError(t, txErr) - return &txm.PendingTx{Tx: *tx} + return &txm.PendingTx{Tx: *tx, AccountID: accountID} } // Send funds twice, along with an invalid transaction - require.NoError(t, testChain.txm.Enqueue(tests.Context(t), "test_success", createMsgWithTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL))) + require.NoError(t, testChain.txm.Enqueue(tests.Context(t), createMsgWithTx("test_success", pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL))) // Wait for new block hash currentBh, err := selectedClient.LatestBlockhash(tests.Context(t)) @@ -575,8 +575,8 @@ NewBlockHash: } } - require.NoError(t, testChain.txm.Enqueue(tests.Context(t), "test_success_2", createMsgWithTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL))) - require.Error(t, testChain.txm.Enqueue(tests.Context(t), "test_invalidSigner", createMsgWithTx(pubKeyReceiver, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL))) // cannot sign tx before enqueuing + require.NoError(t, testChain.txm.Enqueue(tests.Context(t), createMsgWithTx("test_success_2", pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL))) + require.Error(t, testChain.txm.Enqueue(tests.Context(t), createMsgWithTx("test_invalidSigner", pubKeyReceiver, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL))) // cannot sign tx before enqueuing // wait for all txes to finish ctx, cancel := context.WithCancel(tests.Context(t)) diff --git a/pkg/solana/relay.go b/pkg/solana/relay.go index fd448e6a5..db7c681f2 100644 --- a/pkg/solana/relay.go +++ b/pkg/solana/relay.go @@ -24,7 +24,7 @@ import ( var _ TxManager = (*txm.Txm)(nil) type TxManager interface { - Enqueue(ctx context.Context, accountID string, msg *txm.PendingTx, txCfgs ...txm.SetTxConfig) error + Enqueue(ctx context.Context, msg *txm.PendingTx, txCfgs ...txm.SetTxConfig) error } var _ relaytypes.Relayer = &Relayer{} //nolint:staticcheck diff --git a/pkg/solana/transmitter.go b/pkg/solana/transmitter.go index 014f743e3..444aae632 100644 --- a/pkg/solana/transmitter.go +++ b/pkg/solana/transmitter.go @@ -74,12 +74,13 @@ func (c *Transmitter) Transmit( } msg := &txm.PendingTx{ - Tx: *tx, + Tx: *tx, + AccountID: c.stateID.String(), } // pass transmit payload to tx manager queue c.lggr.Debugf("Queuing transmit tx: state (%s) + transmissions (%s)", c.stateID.String(), c.transmissionsID.String()) - if err = c.txManager.Enqueue(ctx, c.stateID.String(), msg); err != nil { + if err = c.txManager.Enqueue(ctx, msg); err != nil { return fmt.Errorf("error on Transmit.txManager.Enqueue: %w", err) } return nil diff --git a/pkg/solana/transmitter_test.go b/pkg/solana/transmitter_test.go index fde87147a..6798ad692 100644 --- a/pkg/solana/transmitter_test.go +++ b/pkg/solana/transmitter_test.go @@ -25,7 +25,7 @@ type verifyTxSize struct { s *solana.PrivateKey } -func (txm verifyTxSize) Enqueue(_ context.Context, _ string, msg *txm.PendingTx, _ ...txm.SetTxConfig) error { +func (txm verifyTxSize) Enqueue(_ context.Context, msg *txm.PendingTx, _ ...txm.SetTxConfig) error { // additional components that transaction manager adds to the transaction tx := &msg.Tx require.NoError(txm.t, fees.SetComputeUnitPrice(tx, 0)) diff --git a/pkg/solana/txm/pendingtx.go b/pkg/solana/txm/pendingtx.go index 219c45c1d..da2f10407 100644 --- a/pkg/solana/txm/pendingtx.go +++ b/pkg/solana/txm/pendingtx.go @@ -48,6 +48,7 @@ type PendingTxContext interface { type PendingTx struct { Tx solana.Transaction + AccountID string cfg TxConfig signatures []solana.Signature UUID string diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index cba421757..2a6682871 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -657,7 +657,7 @@ func (txm *Txm) reap() { } // Enqueue enqueues a msg destined for the solana chain. -func (txm *Txm) Enqueue(ctx context.Context, accountID string, msg *PendingTx, txCfgs ...SetTxConfig) error { +func (txm *Txm) Enqueue(ctx context.Context, msg *PendingTx, txCfgs ...SetTxConfig) error { if err := txm.Ready(); err != nil { return fmt.Errorf("error in soltxm.Enqueue: %w", err) } @@ -709,8 +709,8 @@ func (txm *Txm) Enqueue(ctx context.Context, accountID string, msg *PendingTx, t select { case txm.chSend <- *msg: default: - txm.lggr.Errorw("failed to enqueue tx", "queueFull", len(txm.chSend) == MaxQueueLen, "tx", msg) - return fmt.Errorf("failed to enqueue transaction for %s", accountID) + txm.lggr.Errorw("failed to enqeue tx", "queueFull", len(txm.chSend) == MaxQueueLen, "tx", msg) + return fmt.Errorf("failed to enqueue transaction for %s", msg.AccountID) } return nil } diff --git a/pkg/solana/txm/txm_internal_test.go b/pkg/solana/txm/txm_internal_test.go index c37b0d310..23d030186 100644 --- a/pkg/solana/txm/txm_internal_test.go +++ b/pkg/solana/txm/txm_internal_test.go @@ -1,5 +1,3 @@ -//go:build integration - package txm import ( @@ -211,7 +209,7 @@ func TestTxm(t *testing.T) { // send tx testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID})) + assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()})) wg.Wait() // no transactions stored inflight txs list @@ -247,7 +245,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID})) + assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()})) wg.Wait() // wait to be picked up and processed // no transactions stored inflight txs list @@ -278,7 +276,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID})) + assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()})) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared quickly @@ -314,7 +312,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID})) + assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()})) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared after timeout @@ -354,7 +352,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID})) + assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()})) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared after timeout @@ -404,7 +402,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID})) + assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()})) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared after timeout @@ -446,7 +444,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID})) + assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()})) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared after timeout @@ -491,7 +489,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID})) + assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()})) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // inflight txs cleared after timeout @@ -543,7 +541,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID})) + assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()})) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // inflight txs cleared after timeout @@ -581,7 +579,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID})) + assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()})) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // inflight txs cleared after timeout @@ -627,7 +625,7 @@ func TestTxm(t *testing.T) { // send tx testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID})) + assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()})) wg.Wait() // no transactions stored inflight txs list @@ -681,7 +679,7 @@ func TestTxm(t *testing.T) { // send tx - with disabled fee bumping testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID}, SetFeeBumpPeriod(0))) + assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()}, SetFeeBumpPeriod(0))) wg.Wait() // no transactions stored inflight txs list @@ -733,7 +731,7 @@ func TestTxm(t *testing.T) { // send tx - with disabled fee bumping and disabled compute unit limit testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID}, SetFeeBumpPeriod(0), SetComputeUnitLimit(0))) + assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()}, SetFeeBumpPeriod(0), SetComputeUnitLimit(0))) wg.Wait() // no transactions stored inflight txs list @@ -849,7 +847,7 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID})) + assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()})) wg.Wait() // wait to be picked up and processed waitFor(t, 5*time.Second, txm, prom, empty) // inflight txs cleared after timeout @@ -1061,7 +1059,7 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { // send tx testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx, UUID: testTxID})) + assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()})) wg.Wait() // no transactions stored inflight txs list @@ -1090,7 +1088,7 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { mc.On("SimulateTx", mock.Anything, mock.Anything, mock.Anything).Return(nil, errors.New("simulation failed")).Once() // tx should NOT be able to queue - assert.Error(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx})) + assert.Error(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, AccountID: t.Name()})) }) t.Run("simulation_returns_error", func(t *testing.T) { @@ -1106,7 +1104,7 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { txID := uuid.NewString() // tx should NOT be able to queue - assert.Error(t, txm.Enqueue(ctx, t.Name(), &PendingTx{Tx: *tx})) + assert.Error(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, AccountID: t.Name()})) }) } @@ -1171,7 +1169,7 @@ func TestTxm_Enqueue(t *testing.T) { loader := utils.NewLazyLoad(func() (client.ReaderWriter, error) { return mc, nil }) txm := NewTxm("enqueue_test", loader, nil, cfg, mkey, lggr) - require.ErrorContains(t, txm.Enqueue(ctx, "txmUnstarted", &PendingTx{}), "not started") + require.ErrorContains(t, txm.Enqueue(ctx, &PendingTx{AccountID: "txmUnstarted"}), "not started") require.NoError(t, txm.Start(ctx)) t.Cleanup(func() { require.NoError(t, txm.Close()) }) @@ -1188,10 +1186,10 @@ func TestTxm_Enqueue(t *testing.T) { for _, run := range txs { t.Run(run.name, func(t *testing.T) { if !run.fail { - assert.NoError(t, txm.Enqueue(ctx, run.name, &PendingTx{Tx: *run.tx})) + assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *run.tx, AccountID: run.name})) return } - assert.Error(t, txm.Enqueue(ctx, run.name, &PendingTx{Tx: *run.tx})) + assert.Error(t, txm.Enqueue(ctx, &PendingTx{Tx: *run.tx, AccountID: run.name})) }) } } diff --git a/pkg/solana/txm/txm_load_test.go b/pkg/solana/txm/txm_load_test.go index f37e166d2..3f72369d0 100644 --- a/pkg/solana/txm/txm_load_test.go +++ b/pkg/solana/txm/txm_load_test.go @@ -1,5 +1,3 @@ -//go:build integration - package txm import ( @@ -83,7 +81,7 @@ func TestTxm_Integration(t *testing.T) { // already started assert.Error(t, txm.Start(ctx)) - createMsgWithTx := func(signer solana.PublicKey, sender solana.PublicKey, receiver solana.PublicKey, amt uint64) *PendingTx { + createMsgWithTx := func(accountID string, signer solana.PublicKey, sender solana.PublicKey, receiver solana.PublicKey, amt uint64) *PendingTx { // create transfer tx assert.NoError(t, err) tx, err := solana.NewTransaction( @@ -98,20 +96,20 @@ func TestTxm_Integration(t *testing.T) { solana.TransactionPayer(signer), ) require.NoError(t, err) - return &PendingTx{Tx: *tx} + return &PendingTx{Tx: *tx, AccountID: accountID} } // enqueue txs (must pass to move on to load test) - require.NoError(t, txm.Enqueue(ctx, "test_success_0", createMsgWithTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL))) - require.Error(t, txm.Enqueue(ctx, "test_invalidSigner", createMsgWithTx(pubKeyReceiver, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL))) // cannot sign tx before enqueuing - require.NoError(t, txm.Enqueue(ctx, "test_invalidReceiver", createMsgWithTx(pubKey, pubKey, solana.PublicKey{}, solana.LAMPORTS_PER_SOL))) + require.NoError(t, txm.Enqueue(ctx, createMsgWithTx("test_success_0", pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL))) + require.Error(t, txm.Enqueue(ctx, createMsgWithTx("test_invalidSigner", pubKeyReceiver, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL))) // cannot sign tx before enqueuing + require.NoError(t, txm.Enqueue(ctx, createMsgWithTx("test_invalidReceiver", pubKey, pubKey, solana.PublicKey{}, solana.LAMPORTS_PER_SOL))) time.Sleep(500 * time.Millisecond) // pause 0.5s for new blockhash - require.NoError(t, txm.Enqueue(ctx, "test_success_1", createMsgWithTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL))) - require.NoError(t, txm.Enqueue(ctx, "test_txFail", createMsgWithTx(pubKey, pubKey, pubKeyReceiver, 1000*solana.LAMPORTS_PER_SOL))) + require.NoError(t, txm.Enqueue(ctx, createMsgWithTx("test_success_1", pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL))) + require.NoError(t, txm.Enqueue(ctx, createMsgWithTx("test_txFail", pubKey, pubKey, pubKeyReceiver, 1000*solana.LAMPORTS_PER_SOL))) // load test: try to overload txs, confirm, or simulation for i := 0; i < 1000; i++ { - assert.NoError(t, txm.Enqueue(ctx, fmt.Sprintf("load_%d", i), createMsgWithTx(loadTestKey.PublicKey(), loadTestKey.PublicKey(), loadTestKey.PublicKey(), uint64(i)))) + assert.NoError(t, txm.Enqueue(ctx, createMsgWithTx(fmt.Sprintf("load_%d", i), loadTestKey.PublicKey(), loadTestKey.PublicKey(), loadTestKey.PublicKey(), uint64(i)))) time.Sleep(10 * time.Millisecond) // ~100 txs per second (note: have run 5ms delays for ~200tx/s succesfully) } From b0d9426cdc8dd11a300a458e998744d316f6c17d Mon Sep 17 00:00:00 2001 From: Farber98 Date: Mon, 18 Nov 2024 22:30:53 -0300 Subject: [PATCH 08/51] lint fix --- pkg/solana/txm/txm_internal_test.go | 3 +++ pkg/solana/txm/txm_load_test.go | 8 +++++--- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/pkg/solana/txm/txm_internal_test.go b/pkg/solana/txm/txm_internal_test.go index 23d030186..242645eaa 100644 --- a/pkg/solana/txm/txm_internal_test.go +++ b/pkg/solana/txm/txm_internal_test.go @@ -1,3 +1,5 @@ +//go:build integration + package txm import ( @@ -770,6 +772,7 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { cfg.Chain.TxRetentionTimeout = relayconfig.MustNewDuration(5 * time.Second) mc := mocks.NewReaderWriter(t) blockhash, err := solana.HashFromBase58("blockhash") + require.NoError(t, err) mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ Blockhash: blockhash, diff --git a/pkg/solana/txm/txm_load_test.go b/pkg/solana/txm/txm_load_test.go index 3f72369d0..d371c9311 100644 --- a/pkg/solana/txm/txm_load_test.go +++ b/pkg/solana/txm/txm_load_test.go @@ -1,3 +1,5 @@ +//go:build integration + package txm import ( @@ -84,7 +86,7 @@ func TestTxm_Integration(t *testing.T) { createMsgWithTx := func(accountID string, signer solana.PublicKey, sender solana.PublicKey, receiver solana.PublicKey, amt uint64) *PendingTx { // create transfer tx assert.NoError(t, err) - tx, err := solana.NewTransaction( + tx, txErr := solana.NewTransaction( []solana.Instruction{ system.NewTransferInstruction( amt, @@ -95,7 +97,7 @@ func TestTxm_Integration(t *testing.T) { solana.Hash{}, solana.TransactionPayer(signer), ) - require.NoError(t, err) + require.NoError(t, txErr) return &PendingTx{Tx: *tx, AccountID: accountID} } @@ -110,7 +112,7 @@ func TestTxm_Integration(t *testing.T) { // load test: try to overload txs, confirm, or simulation for i := 0; i < 1000; i++ { assert.NoError(t, txm.Enqueue(ctx, createMsgWithTx(fmt.Sprintf("load_%d", i), loadTestKey.PublicKey(), loadTestKey.PublicKey(), loadTestKey.PublicKey(), uint64(i)))) - time.Sleep(10 * time.Millisecond) // ~100 txs per second (note: have run 5ms delays for ~200tx/s succesfully) + time.Sleep(10 * time.Millisecond) // ~100 txs per second (note: have run 5ms delays for ~200tx/s successfully) } // check to make sure all txs are closed out from inflight list (longest should last MaxConfirmTimeout) From 1b3866552704e4400119f6c284b6235e108c367c Mon Sep 17 00:00:00 2001 From: Farber98 Date: Mon, 18 Nov 2024 22:39:26 -0300 Subject: [PATCH 09/51] base58 does not contain lower l --- pkg/solana/txm/txm_internal_test.go | 11 +++++++---- pkg/solana/txm/txm_race_test.go | 3 ++- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/pkg/solana/txm/txm_internal_test.go b/pkg/solana/txm/txm_internal_test.go index 242645eaa..d50c25f76 100644 --- a/pkg/solana/txm/txm_internal_test.go +++ b/pkg/solana/txm/txm_internal_test.go @@ -126,7 +126,8 @@ func TestTxm(t *testing.T) { cfg := config.NewDefault() cfg.Chain.FeeEstimatorMode = &estimator mc := mocks.NewReaderWriter(t) - blockhash, _ := solana.HashFromBase58("blockhash") + blockhash, err := solana.HashFromBase58("hash") + require.NoError(t, err) mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ Blockhash: blockhash, @@ -771,7 +772,7 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { // Enable retention timeout to keep transactions after finality cfg.Chain.TxRetentionTimeout = relayconfig.MustNewDuration(5 * time.Second) mc := mocks.NewReaderWriter(t) - blockhash, err := solana.HashFromBase58("blockhash") + blockhash, err := solana.HashFromBase58("hash") require.NoError(t, err) mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ @@ -979,7 +980,8 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { // Enable retention timeout to keep transactions after finality or error cfg.Chain.TxRetentionTimeout = relayconfig.MustNewDuration(5 * time.Second) mc := mocks.NewReaderWriter(t) - blockhash, _ := solana.HashFromBase58("blockhash") + blockhash, err := solana.HashFromBase58("hash") + require.NoError(t, err) mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ Blockhash: blockhash, @@ -1116,7 +1118,8 @@ func TestTxm_Enqueue(t *testing.T) { lggr := logger.Test(t) cfg := config.NewDefault() mc := mocks.NewReaderWriter(t) - blockhash, _ := solana.HashFromBase58("blockhash") + blockhash, err := solana.HashFromBase58("hash") + require.NoError(t, err) mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ Blockhash: blockhash, diff --git a/pkg/solana/txm/txm_race_test.go b/pkg/solana/txm/txm_race_test.go index a25903cac..bdd1f3c25 100644 --- a/pkg/solana/txm/txm_race_test.go +++ b/pkg/solana/txm/txm_race_test.go @@ -64,7 +64,8 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { // assemble minimal tx for testing retry msg := NewTestMsg() - blockhash, _ := solana.HashFromBase58("blockhash") + blockhash, err := solana.HashFromBase58("hash") + require.NoError(t, err) testRunner := func(t *testing.T, client solanaClient.ReaderWriter) { // build minimal txm From 6923ddf33878f7d499ef2feb4c9e961829fe21c9 Mon Sep 17 00:00:00 2001 From: Farber98 Date: Mon, 18 Nov 2024 22:52:26 -0300 Subject: [PATCH 10/51] fix hash errors --- pkg/solana/txm/txm_internal_test.go | 8 ++++---- pkg/solana/txm/txm_race_test.go | 3 +-- pkg/solana/txm/utils.go | 17 +++++++++++++++++ 3 files changed, 22 insertions(+), 6 deletions(-) diff --git a/pkg/solana/txm/txm_internal_test.go b/pkg/solana/txm/txm_internal_test.go index d50c25f76..66aa1b566 100644 --- a/pkg/solana/txm/txm_internal_test.go +++ b/pkg/solana/txm/txm_internal_test.go @@ -126,7 +126,7 @@ func TestTxm(t *testing.T) { cfg := config.NewDefault() cfg.Chain.FeeEstimatorMode = &estimator mc := mocks.NewReaderWriter(t) - blockhash, err := solana.HashFromBase58("hash") + blockhash, err := generateRandomHash() require.NoError(t, err) mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ @@ -772,7 +772,7 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { // Enable retention timeout to keep transactions after finality cfg.Chain.TxRetentionTimeout = relayconfig.MustNewDuration(5 * time.Second) mc := mocks.NewReaderWriter(t) - blockhash, err := solana.HashFromBase58("hash") + blockhash, err := generateRandomHash() require.NoError(t, err) mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ @@ -980,7 +980,7 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { // Enable retention timeout to keep transactions after finality or error cfg.Chain.TxRetentionTimeout = relayconfig.MustNewDuration(5 * time.Second) mc := mocks.NewReaderWriter(t) - blockhash, err := solana.HashFromBase58("hash") + blockhash, err := generateRandomHash() require.NoError(t, err) mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ @@ -1118,7 +1118,7 @@ func TestTxm_Enqueue(t *testing.T) { lggr := logger.Test(t) cfg := config.NewDefault() mc := mocks.NewReaderWriter(t) - blockhash, err := solana.HashFromBase58("hash") + blockhash, err := generateRandomHash() require.NoError(t, err) mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ diff --git a/pkg/solana/txm/txm_race_test.go b/pkg/solana/txm/txm_race_test.go index bdd1f3c25..9ef298a78 100644 --- a/pkg/solana/txm/txm_race_test.go +++ b/pkg/solana/txm/txm_race_test.go @@ -8,7 +8,6 @@ import ( "testing" "time" - "github.com/gagliardetto/solana-go" solanaGo "github.com/gagliardetto/solana-go" "github.com/gagliardetto/solana-go/rpc" "go.uber.org/zap/zapcore" @@ -64,7 +63,7 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { // assemble minimal tx for testing retry msg := NewTestMsg() - blockhash, err := solana.HashFromBase58("hash") + blockhash, err := generateRandomHash() require.NoError(t, err) testRunner := func(t *testing.T, client solanaClient.ReaderWriter) { diff --git a/pkg/solana/txm/utils.go b/pkg/solana/txm/utils.go index fef260e3d..f7cb28a32 100644 --- a/pkg/solana/txm/utils.go +++ b/pkg/solana/txm/utils.go @@ -1,6 +1,7 @@ package txm import ( + "crypto/rand" "errors" "fmt" "sort" @@ -9,6 +10,7 @@ import ( "github.com/gagliardetto/solana-go" "github.com/gagliardetto/solana-go/rpc" + "github.com/mr-tron/base58" ) type TxState int @@ -222,3 +224,18 @@ func SetEstimateComputeUnitLimit(v bool) SetTxConfig { cfg.EstimateComputeUnitLimit = v } } + +func generateRandomHash() (solana.Hash, error) { + // Generate 32 random bytes + randomBytes := make([]byte, 32) + _, err := rand.Read(randomBytes) + if err != nil { + return solana.Hash{}, err + } + + // Encode the random bytes in base58 + base58Hash := base58.Encode(randomBytes) + + // Convert the base58 string to a solana.Hash + return solana.HashFromBase58(base58Hash) +} From 462844b910b07122167d7f1580bf16c442786ce4 Mon Sep 17 00:00:00 2001 From: Farber98 Date: Mon, 18 Nov 2024 23:01:43 -0300 Subject: [PATCH 11/51] fix generate random hash --- pkg/solana/txm/utils.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/pkg/solana/txm/utils.go b/pkg/solana/txm/utils.go index f7cb28a32..f7c6c6e61 100644 --- a/pkg/solana/txm/utils.go +++ b/pkg/solana/txm/utils.go @@ -10,7 +10,6 @@ import ( "github.com/gagliardetto/solana-go" "github.com/gagliardetto/solana-go/rpc" - "github.com/mr-tron/base58" ) type TxState int @@ -225,6 +224,7 @@ func SetEstimateComputeUnitLimit(v bool) SetTxConfig { } } +// generateRandomHash generates a random 32-byte hash. func generateRandomHash() (solana.Hash, error) { // Generate 32 random bytes randomBytes := make([]byte, 32) @@ -233,9 +233,9 @@ func generateRandomHash() (solana.Hash, error) { return solana.Hash{}, err } - // Encode the random bytes in base58 - base58Hash := base58.Encode(randomBytes) + // Convert the random bytes to a solana.Hash + var hash solana.Hash + copy(hash[:], randomBytes) - // Convert the base58 string to a solana.Hash - return solana.HashFromBase58(base58Hash) + return hash, nil } From fd785d02f99d376f47e39fadc4418349fa5faaeb Mon Sep 17 00:00:00 2001 From: Farber98 Date: Mon, 18 Nov 2024 23:20:35 -0300 Subject: [PATCH 12/51] remove blockhash as we only need block height --- pkg/solana/txm/txm_internal_test.go | 12 ------------ pkg/solana/txm/txm_race_test.go | 7 ------- pkg/solana/txm/utils.go | 17 ----------------- 3 files changed, 36 deletions(-) diff --git a/pkg/solana/txm/txm_internal_test.go b/pkg/solana/txm/txm_internal_test.go index 66aa1b566..07d0417db 100644 --- a/pkg/solana/txm/txm_internal_test.go +++ b/pkg/solana/txm/txm_internal_test.go @@ -126,11 +126,8 @@ func TestTxm(t *testing.T) { cfg := config.NewDefault() cfg.Chain.FeeEstimatorMode = &estimator mc := mocks.NewReaderWriter(t) - blockhash, err := generateRandomHash() - require.NoError(t, err) mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ - Blockhash: blockhash, LastValidBlockHeight: uint64(2000), }, }, nil) @@ -772,11 +769,8 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { // Enable retention timeout to keep transactions after finality cfg.Chain.TxRetentionTimeout = relayconfig.MustNewDuration(5 * time.Second) mc := mocks.NewReaderWriter(t) - blockhash, err := generateRandomHash() - require.NoError(t, err) mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ - Blockhash: blockhash, LastValidBlockHeight: uint64(2000), }, }, nil) @@ -980,11 +974,8 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { // Enable retention timeout to keep transactions after finality or error cfg.Chain.TxRetentionTimeout = relayconfig.MustNewDuration(5 * time.Second) mc := mocks.NewReaderWriter(t) - blockhash, err := generateRandomHash() - require.NoError(t, err) mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ - Blockhash: blockhash, LastValidBlockHeight: uint64(2000), }, }, nil) @@ -1118,11 +1109,8 @@ func TestTxm_Enqueue(t *testing.T) { lggr := logger.Test(t) cfg := config.NewDefault() mc := mocks.NewReaderWriter(t) - blockhash, err := generateRandomHash() - require.NoError(t, err) mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ - Blockhash: blockhash, LastValidBlockHeight: uint64(2000), }, }, nil) diff --git a/pkg/solana/txm/txm_race_test.go b/pkg/solana/txm/txm_race_test.go index 9ef298a78..c662649ba 100644 --- a/pkg/solana/txm/txm_race_test.go +++ b/pkg/solana/txm/txm_race_test.go @@ -63,9 +63,6 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { // assemble minimal tx for testing retry msg := NewTestMsg() - blockhash, err := generateRandomHash() - require.NoError(t, err) - testRunner := func(t *testing.T, client solanaClient.ReaderWriter) { // build minimal txm loader := utils.NewLazyLoad(func() (solanaClient.ReaderWriter, error) { @@ -122,7 +119,6 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { ) client.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ - Blockhash: blockhash, LastValidBlockHeight: uint64(2000), }, }, nil) @@ -164,7 +160,6 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { ) client.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ - Blockhash: blockhash, LastValidBlockHeight: uint64(2000), }, }, nil) @@ -217,7 +212,6 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { ) client.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ - Blockhash: blockhash, LastValidBlockHeight: uint64(2000), }, }, nil) @@ -257,7 +251,6 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { client.On("SendTx", mock.Anything, &msg3.Tx).Return(solanaGo.Signature{4}, nil) client.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ - Blockhash: blockhash, LastValidBlockHeight: uint64(2000), }, }, nil) diff --git a/pkg/solana/txm/utils.go b/pkg/solana/txm/utils.go index f7c6c6e61..fef260e3d 100644 --- a/pkg/solana/txm/utils.go +++ b/pkg/solana/txm/utils.go @@ -1,7 +1,6 @@ package txm import ( - "crypto/rand" "errors" "fmt" "sort" @@ -223,19 +222,3 @@ func SetEstimateComputeUnitLimit(v bool) SetTxConfig { cfg.EstimateComputeUnitLimit = v } } - -// generateRandomHash generates a random 32-byte hash. -func generateRandomHash() (solana.Hash, error) { - // Generate 32 random bytes - randomBytes := make([]byte, 32) - _, err := rand.Read(randomBytes) - if err != nil { - return solana.Hash{}, err - } - - // Convert the random bytes to a solana.Hash - var hash solana.Hash - copy(hash[:], randomBytes) - - return hash, nil -} From cf958a46a6cb1a01159f66a6a49d39d76f8e8a00 Mon Sep 17 00:00:00 2001 From: Farber98 Date: Tue, 19 Nov 2024 00:19:23 -0300 Subject: [PATCH 13/51] expired tx changes without tests --- docs/relay/README.md | 3 +- pkg/solana/config/config.go | 28 +++++++++++-------- pkg/solana/config/mocks/config.go | 35 +++++++++-------------- pkg/solana/config/toml.go | 7 +++++ pkg/solana/txm/pendingtx.go | 19 +++++++++++++ pkg/solana/txm/txm.go | 43 +++++++++++++++++++++++++++++ pkg/solana/txm/txm_internal_test.go | 4 ++- 7 files changed, 104 insertions(+), 35 deletions(-) diff --git a/docs/relay/README.md b/docs/relay/README.md index f1dbffe81..b54fb12fc 100644 --- a/docs/relay/README.md +++ b/docs/relay/README.md @@ -43,7 +43,8 @@ chainlink nodes solana create --name= --chain-id= --url= Date: Tue, 19 Nov 2024 00:27:39 -0300 Subject: [PATCH 14/51] add maybe to mocks --- pkg/solana/txm/txm_internal_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/solana/txm/txm_internal_test.go b/pkg/solana/txm/txm_internal_test.go index 6f68cba17..207e161b8 100644 --- a/pkg/solana/txm/txm_internal_test.go +++ b/pkg/solana/txm/txm_internal_test.go @@ -132,7 +132,7 @@ func TestTxm(t *testing.T) { }, }, nil) mc.On("GetLatestBlock", mock.Anything).Return(&rpc.GetBlockResult{}, nil).Maybe() - mc.On("SlotHeight", mock.Anything).Return(uint64(0), nil) + mc.On("SlotHeight", mock.Anything).Return(uint64(0), nil).Maybe() // mock solana keystore mkey := keyMocks.NewSimpleKeystore(t) @@ -775,7 +775,7 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { }, }, nil) mc.On("GetLatestBlock", mock.Anything).Return(&rpc.GetBlockResult{}, nil).Maybe() - mc.On("SlotHeight", mock.Anything).Return(uint64(0), nil) + mc.On("SlotHeight", mock.Anything).Return(uint64(0), nil).Maybe() computeUnitLimitDefault := fees.ComputeUnitLimit(cfg.ComputeUnitLimitDefault()) @@ -981,7 +981,7 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { }, }, nil) mc.On("GetLatestBlock", mock.Anything).Return(&rpc.GetBlockResult{}, nil).Maybe() - mc.On("SlotHeight", mock.Anything).Return(uint64(0), nil) + mc.On("SlotHeight", mock.Anything).Return(uint64(0), nil).Maybe() // mock solana keystore mkey := keyMocks.NewSimpleKeystore(t) From a5059932d121ff8f61ab73d45951abfca0959fc3 Mon Sep 17 00:00:00 2001 From: Farber98 Date: Tue, 19 Nov 2024 12:09:47 -0300 Subject: [PATCH 15/51] expiration tests --- pkg/solana/txm/pendingtx.go | 2 + pkg/solana/txm/txm.go | 33 +++++--- pkg/solana/txm/txm_internal_test.go | 126 ++++++++++++++++++++++++++++ 3 files changed, 150 insertions(+), 11 deletions(-) diff --git a/pkg/solana/txm/pendingtx.go b/pkg/solana/txm/pendingtx.go index a48b981fc..7013284af 100644 --- a/pkg/solana/txm/pendingtx.go +++ b/pkg/solana/txm/pendingtx.go @@ -54,6 +54,8 @@ type PendingTx struct { cfg TxConfig signatures []solana.Signature UUID string + IDSetByCaller bool + rebroadcastCount int createTs time.Time retentionTs time.Time state TxState diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index 03cfa9a11..5dc941ecd 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -521,22 +521,31 @@ func (txm *Txm) rebroadcastExpiredTxs(ctx context.Context, client client.ReaderW continue } - rebroadcastTx := &PendingTx{ - Tx: tx.Tx, - UUID: tx.UUID, // same id to handle case where set by caller. Previous ID has already been removed. - BalanceCheck: tx.BalanceCheck, - Amount: tx.Amount, - From: tx.From, + rebroadcastTx := PendingTx{ + Tx: tx.Tx, + BalanceCheck: tx.BalanceCheck, + Amount: tx.Amount, + From: tx.From, + IDSetByCaller: tx.IDSetByCaller, + rebroadcastCount: tx.rebroadcastCount + 1, + } + + // If the ID was set by the caller, we should keep it similar. + if tx.IDSetByCaller { + // Append #tx.rebroadcast count to the end of the ID. + rebroadcastTx.UUID = fmt.Sprintf("%s#%d", tx.UUID, rebroadcastTx.rebroadcastCount) + } else { + rebroadcastTx.UUID = uuid.New().String() } - // Re-enqueue the transaction for rebroadcasting - err = txm.Enqueue(ctx, rebroadcastTx) + // Attempt to rebroadcast the transaction with sendWithRetry + _, _, _, err = txm.sendWithRetry(ctx, rebroadcastTx) if err != nil { - txm.lggr.Errorw("failed to enqueue rebroadcast transaction", "id", tx.UUID, "error", err) + txm.lggr.Errorw("failed to send rebroadcast transaction", "error", err) continue } - txm.lggr.Infow("rebroadcast transaction enqueued", "id", tx.UUID) + txm.lggr.Infow("rebroadcast transaction sent", "previous id", tx.UUID, "new id", rebroadcastTx.UUID) } } @@ -744,9 +753,11 @@ func (txm *Txm) Enqueue(ctx context.Context, msg *PendingTx, txCfgs ...SetTxConf } msg.cfg = cfg - // If ID was not set by caller, create one. if msg.UUID == "" { + // If ID was not set by caller, create one. msg.UUID = uuid.New().String() + } else { + msg.IDSetByCaller = true } select { diff --git a/pkg/solana/txm/txm_internal_test.go b/pkg/solana/txm/txm_internal_test.go index 207e161b8..2ee5a4c5e 100644 --- a/pkg/solana/txm/txm_internal_test.go +++ b/pkg/solana/txm/txm_internal_test.go @@ -1189,3 +1189,129 @@ func TestTxm_Enqueue(t *testing.T) { }) } } + +func addSigAndLimitToTx(t *testing.T, keystore SimpleKeystore, pubkey solana.PublicKey, tx solana.Transaction, limit fees.ComputeUnitLimit) *solana.Transaction { + txCopy := tx + // sign tx + txMsg, err := tx.Message.MarshalBinary() + require.NoError(t, err) + sigBytes, err := keystore.Sign(context.Background(), pubkey.String(), txMsg) + require.NoError(t, err) + var sig [64]byte + copy(sig[:], sigBytes) + txCopy.Signatures = append(txCopy.Signatures, sig) + require.NoError(t, fees.SetComputeUnitLimit(&txCopy, limit)) + return &txCopy +} + +func TestTxm_ExpirationRebroadcast(t *testing.T) { + t.Parallel() + + // Set up configurations + estimator := "fixed" + id := "mocknet-" + estimator + "-" + uuid.NewString() + t.Logf("Starting new iteration: %s", id) + + ctx := tests.Context(t) + lggr := logger.Test(t) + cfg := config.NewDefault() + cfg.Chain.FeeEstimatorMode = &estimator + + // Enable TxExpirationRebroadcast + txExpirationRebroadcast := true + cfg.Chain.TxExpirationRebroadcast = &txExpirationRebroadcast + cfg.Chain.TxConfirmTimeout = relayconfig.MustNewDuration(5 * time.Second) + // Enable retention timeout to keep transactions after finality so we can check. + cfg.Chain.TxRetentionTimeout = relayconfig.MustNewDuration(15 * time.Second) + + mc := mocks.NewReaderWriter(t) + + // Set up LatestBlockhash to return different LastValidBlockHeight values + latestBlockhashCallCount := 0 + mc.On("LatestBlockhash", mock.Anything).Return(func(_ context.Context) (*rpc.GetLatestBlockhashResult, error) { + latestBlockhashCallCount++ + if latestBlockhashCallCount == 1 { + return &rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: uint64(1000), + }, + }, nil + } + return &rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: uint64(2000), + }, + }, nil + }).Maybe() + + // Set up SlotHeight to return a value greater than the initial LastValidBlockHeight + mc.On("SlotHeight", mock.Anything).Return(uint64(1500), nil).Maybe() + mkey := keyMocks.NewSimpleKeystore(t) + mkey.On("Sign", mock.Anything, mock.Anything, mock.Anything).Return([]byte{}, nil) + loader := utils.NewLazyLoad(func() (client.ReaderWriter, error) { return mc, nil }) + txm := NewTxm(id, loader, nil, cfg, mkey, lggr) + require.NoError(t, txm.Start(ctx)) + t.Cleanup(func() { require.NoError(t, txm.Close()) }) + + sig1 := randomSignature(t) + mc.On("SendTx", mock.Anything, mock.Anything).Return(sig1, nil).Maybe() + mc.On("SimulateTx", mock.Anything, mock.Anything, mock.Anything).Return(&rpc.SimulateTransactionResult{}, nil).Maybe() + statuses := map[solana.Signature]func() *rpc.SignatureStatusesResult{} + mc.On("SignatureStatuses", mock.Anything, mock.AnythingOfType("[]solana.Signature")).Return( + func(_ context.Context, sigs []solana.Signature) (out []*rpc.SignatureStatusesResult) { + for i := range sigs { + get, exists := statuses[sigs[i]] + if !exists { + out = append(out, nil) + continue + } + out = append(out, get()) + } + return out + }, nil, + ) + + nowTs := time.Now() + sigStatusCallCount := 0 + var wg sync.WaitGroup + wg.Add(1) + statuses[sig1] = func() *rpc.SignatureStatusesResult { + // If time is less than confirm timeout, return nil. This is when tx should be rebroadcasted + if time.Since(nowTs) < cfg.TxConfirmTimeout()-2*time.Second { + return nil + } else { + sigStatusCallCount++ + if sigStatusCallCount == 1 { + return &rpc.SignatureStatusesResult{ + ConfirmationStatus: rpc.ConfirmationStatusProcessed, + } + } else if sigStatusCallCount == 2 { + return &rpc.SignatureStatusesResult{ + ConfirmationStatus: rpc.ConfirmationStatusConfirmed, + } + } else { + wg.Done() + return &rpc.SignatureStatusesResult{ + ConfirmationStatus: rpc.ConfirmationStatusFinalized, + } + } + } + } + + // Enqueue the transaction + tx, _ := getTx(t, 0, mkey) + expiredTxID := "test" + assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: expiredTxID, AccountID: t.Name()}, SetTimeout(10*time.Second))) + wg.Wait() // Wait for the transaction to be finalized + + // Check that transaction for expiredTxID is not stored in memory + status, err := txm.GetTransactionStatus(ctx, expiredTxID) + require.Error(t, err) + require.Equal(t, types.Unknown, status) + + // Check the transaction status for rebroadcasted txID has been finalized + rebroadcastedTxID := expiredTxID + "#1" + status, err = txm.GetTransactionStatus(ctx, rebroadcastedTxID) + require.NoError(t, err) + require.Equal(t, types.Finalized, status) +} From adc8b1c6490acb2c4106cb983a23b0fbf657aa12 Mon Sep 17 00:00:00 2001 From: Farber98 Date: Tue, 19 Nov 2024 12:35:45 -0300 Subject: [PATCH 16/51] send txes through queue --- docs/relay/README.md | 2 +- pkg/solana/config/config.go | 6 +++--- pkg/solana/txm/txm.go | 8 ++++---- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/relay/README.md b/docs/relay/README.md index b54fb12fc..72057c732 100644 --- a/docs/relay/README.md +++ b/docs/relay/README.md @@ -44,7 +44,7 @@ chainlink nodes solana create --name= --chain-id= --url= Date: Wed, 20 Nov 2024 10:30:00 -0300 Subject: [PATCH 17/51] revert pendingtx leakage of information. overwrite blockhash --- pkg/solana/chain.go | 43 +++-- pkg/solana/chain_test.go | 18 +- pkg/solana/relay.go | 2 +- pkg/solana/transmitter.go | 25 ++- pkg/solana/transmitter_test.go | 9 +- pkg/solana/txm/pendingtx.go | 53 +++-- pkg/solana/txm/pendingtx_test.go | 290 ++++++++++++++-------------- pkg/solana/txm/txm.go | 159 ++++++--------- pkg/solana/txm/txm_internal_test.go | 43 +++-- pkg/solana/txm/txm_load_test.go | 22 ++- pkg/solana/txm/txm_race_test.go | 40 ++-- 11 files changed, 351 insertions(+), 353 deletions(-) diff --git a/pkg/solana/chain.go b/pkg/solana/chain.go index a70e2a4a4..a9c3e10e2 100644 --- a/pkg/solana/chain.go +++ b/pkg/solana/chain.go @@ -12,7 +12,6 @@ import ( "sync" "time" - "github.com/gagliardetto/solana-go" solanago "github.com/gagliardetto/solana-go" "github.com/gagliardetto/solana-go/programs/system" "github.com/gagliardetto/solana-go/rpc" @@ -528,6 +527,11 @@ func (c *chain) HealthReport() map[string]error { } func (c *chain) sendTx(ctx context.Context, from, to string, amount *big.Int, balanceCheck bool) error { + reader, err := c.Reader() + if err != nil { + return fmt.Errorf("chain unreachable: %w", err) + } + fromKey, err := solanago.PublicKeyFromBase58(from) if err != nil { return fmt.Errorf("failed to parse from key: %w", err) @@ -541,6 +545,10 @@ func (c *chain) sendTx(ctx context.Context, from, to string, amount *big.Int, ba } amountI := amount.Uint64() + blockhash, err := reader.LatestBlockhash(ctx) + if err != nil { + return fmt.Errorf("failed to get latest block hash: %w", err) + } tx, err := solanago.NewTransaction( []solanago.Instruction{ system.NewTransferInstruction( @@ -549,25 +557,21 @@ func (c *chain) sendTx(ctx context.Context, from, to string, amount *big.Int, ba toKey, ).Build(), }, - solana.Hash{}, // Will be set within sendWithRetry txm function. + blockhash.Value.Blockhash, // Will be override if needed within sendWithRetry txm function. solanago.TransactionPayer(fromKey), ) if err != nil { return fmt.Errorf("failed to create tx: %w", err) } - msg := &txm.PendingTx{ - Tx: *tx, - AccountID: "", - // To perform balanceCheck we need a blockhash. - // Storing values to perform balanceCheck within sendWithRetry txm function before sending tx. - BalanceCheck: balanceCheck, - From: fromKey, - Amount: amountI, + if balanceCheck { + if err = solanaValidateBalance(ctx, reader, fromKey, amountI, tx.Message.ToBase64()); err != nil { + return fmt.Errorf("failed to validate balance: %w", err) + } } chainTxm := c.TxManager() - err = chainTxm.Enqueue(ctx, msg, + err = chainTxm.Enqueue(ctx, "", tx, nil, txm.SetComputeUnitLimit(500), // reduce from default 200K limit - should only take 450 compute units // no fee bumping and no additional fee - makes validating balance accurate txm.SetComputeUnitPriceMax(0), @@ -580,3 +584,20 @@ func (c *chain) sendTx(ctx context.Context, from, to string, amount *big.Int, ba } return nil } + +func solanaValidateBalance(ctx context.Context, reader client.Reader, from solanago.PublicKey, amount uint64, msg string) error { + balance, err := reader.Balance(ctx, from) + if err != nil { + return err + } + + fee, err := reader.GetFeeForMessage(ctx, msg) + if err != nil { + return err + } + + if balance < (amount + fee) { + return fmt.Errorf("balance %d is too low for this transaction to be executed: amount %d + fee %d", balance, amount, fee) + } + return nil +} diff --git a/pkg/solana/chain_test.go b/pkg/solana/chain_test.go index d8cf651c6..b1e6ff427 100644 --- a/pkg/solana/chain_test.go +++ b/pkg/solana/chain_test.go @@ -30,7 +30,6 @@ import ( mn "github.com/smartcontractkit/chainlink-solana/pkg/solana/client/multinode" solcfg "github.com/smartcontractkit/chainlink-solana/pkg/solana/config" "github.com/smartcontractkit/chainlink-solana/pkg/solana/fees" - "github.com/smartcontractkit/chainlink-solana/pkg/solana/txm" "github.com/smartcontractkit/chainlink-solana/pkg/solana/txm/mocks" ) @@ -536,8 +535,11 @@ func TestSolanaChain_MultiNode_Txm(t *testing.T) { assert.NoError(t, err) assert.Equal(t, uint64(0), receiverBal) - createMsgWithTx := func(accountID string, signer solana.PublicKey, sender solana.PublicKey, receiver solana.PublicKey, amt uint64) *txm.PendingTx { + createTx := func(signer solana.PublicKey, sender solana.PublicKey, receiver solana.PublicKey, amt uint64) *solana.Transaction { + selectedClient, err = testChain.getClient() assert.NoError(t, err) + hash, hashErr := selectedClient.LatestBlockhash(tests.Context(t)) + assert.NoError(t, hashErr) tx, txErr := solana.NewTransaction( []solana.Instruction{ system.NewTransferInstruction( @@ -546,16 +548,15 @@ func TestSolanaChain_MultiNode_Txm(t *testing.T) { receiver, ).Build(), }, - solana.Hash{}, + hash.Value.Blockhash, solana.TransactionPayer(signer), ) require.NoError(t, txErr) - return &txm.PendingTx{Tx: *tx, AccountID: accountID} + return tx } // Send funds twice, along with an invalid transaction - require.NoError(t, testChain.txm.Enqueue(tests.Context(t), createMsgWithTx("test_success", pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL))) - + require.NoError(t, testChain.txm.Enqueue(tests.Context(t), "test_success", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) // Wait for new block hash currentBh, err := selectedClient.LatestBlockhash(tests.Context(t)) require.NoError(t, err) @@ -575,9 +576,8 @@ NewBlockHash: } } - require.NoError(t, testChain.txm.Enqueue(tests.Context(t), createMsgWithTx("test_success_2", pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL))) - require.Error(t, testChain.txm.Enqueue(tests.Context(t), createMsgWithTx("test_invalidSigner", pubKeyReceiver, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL))) // cannot sign tx before enqueuing - + require.NoError(t, testChain.txm.Enqueue(tests.Context(t), "test_success_2", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) + require.Error(t, testChain.txm.Enqueue(tests.Context(t), "test_invalidSigner", createTx(pubKeyReceiver, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) // cannot sign tx before enqueuing // wait for all txes to finish ctx, cancel := context.WithCancel(tests.Context(t)) t.Cleanup(cancel) diff --git a/pkg/solana/relay.go b/pkg/solana/relay.go index db7c681f2..d98ab0442 100644 --- a/pkg/solana/relay.go +++ b/pkg/solana/relay.go @@ -24,7 +24,7 @@ import ( var _ TxManager = (*txm.Txm)(nil) type TxManager interface { - Enqueue(ctx context.Context, msg *txm.PendingTx, txCfgs ...txm.SetTxConfig) error + Enqueue(ctx context.Context, accountID string, tx *solana.Transaction, txID *string, txCfgs ...txm.SetTxConfig) error } var _ relaytypes.Relayer = &Relayer{} //nolint:staticcheck diff --git a/pkg/solana/transmitter.go b/pkg/solana/transmitter.go index 444aae632..5ee1b01b1 100644 --- a/pkg/solana/transmitter.go +++ b/pkg/solana/transmitter.go @@ -3,6 +3,7 @@ package solana import ( "bytes" "context" + "errors" "fmt" "github.com/gagliardetto/solana-go" @@ -10,8 +11,6 @@ import ( "github.com/smartcontractkit/chainlink-common/pkg/logger" "github.com/smartcontractkit/chainlink-common/pkg/utils" - - "github.com/smartcontractkit/chainlink-solana/pkg/solana/txm" ) var _ types.ContractTransmitter = (*Transmitter)(nil) @@ -31,6 +30,19 @@ func (c *Transmitter) Transmit( report types.Report, sigs []types.AttributedOnchainSignature, ) error { + reader, err := c.getReader() + if err != nil { + return fmt.Errorf("error on Transmit.Reader: %w", err) + } + + blockhash, err := reader.LatestBlockhash(ctx) + if err != nil { + return fmt.Errorf("error on Transmit.GetRecentBlockhash: %w", err) + } + if blockhash == nil || blockhash.Value == nil { + return errors.New("nil pointer returned from Transmit.GetRecentBlockhash") + } + // Determine store authority seeds := [][]byte{[]byte("store"), c.stateID.Bytes()} storeAuthority, storeNonce, err := solana.FindProgramAddress(seeds, c.programID) @@ -66,21 +78,16 @@ func (c *Transmitter) Transmit( []solana.Instruction{ solana.NewInstruction(c.programID, accounts, data.Bytes()), }, - solana.Hash{}, // Will be set within sendWithRetry txm function. + blockhash.Value.Blockhash, // Will be override if needed within sendWithRetry txm function. solana.TransactionPayer(c.transmissionSigner), ) if err != nil { return fmt.Errorf("error on Transmit.NewTransaction: %w", err) } - msg := &txm.PendingTx{ - Tx: *tx, - AccountID: c.stateID.String(), - } - // pass transmit payload to tx manager queue c.lggr.Debugf("Queuing transmit tx: state (%s) + transmissions (%s)", c.stateID.String(), c.transmissionsID.String()) - if err = c.txManager.Enqueue(ctx, msg); err != nil { + if err = c.txManager.Enqueue(ctx, c.stateID.String(), tx, nil); err != nil { return fmt.Errorf("error on Transmit.txManager.Enqueue: %w", err) } return nil diff --git a/pkg/solana/transmitter_test.go b/pkg/solana/transmitter_test.go index 6798ad692..d2f0229ca 100644 --- a/pkg/solana/transmitter_test.go +++ b/pkg/solana/transmitter_test.go @@ -5,8 +5,10 @@ import ( "testing" "github.com/gagliardetto/solana-go" + "github.com/gagliardetto/solana-go/rpc" "github.com/smartcontractkit/libocr/offchainreporting2/types" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" "github.com/smartcontractkit/chainlink-common/pkg/logger" @@ -25,9 +27,8 @@ type verifyTxSize struct { s *solana.PrivateKey } -func (txm verifyTxSize) Enqueue(_ context.Context, msg *txm.PendingTx, _ ...txm.SetTxConfig) error { +func (txm verifyTxSize) Enqueue(_ context.Context, _ string, tx *solana.Transaction, txID *string, _ ...txm.SetTxConfig) error { // additional components that transaction manager adds to the transaction - tx := &msg.Tx require.NoError(txm.t, fees.SetComputeUnitPrice(tx, 0)) require.NoError(txm.t, fees.SetComputeUnitLimit(tx, 0)) @@ -58,7 +59,9 @@ func TestTransmitter_TxSize(t *testing.T) { } rw := clientmocks.NewReaderWriter(t) - + rw.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{}, + }, nil) transmitter := Transmitter{ stateID: mustNewRandomPublicKey(), programID: mustNewRandomPublicKey(), diff --git a/pkg/solana/txm/pendingtx.go b/pkg/solana/txm/pendingtx.go index 7013284af..6ec448916 100644 --- a/pkg/solana/txm/pendingtx.go +++ b/pkg/solana/txm/pendingtx.go @@ -21,7 +21,7 @@ var ( type PendingTxContext interface { // New adds a new tranasction in Broadcasted state to the storage - New(msg PendingTx, sig solana.Signature, cancel context.CancelFunc) error + New(msg pendingTx, sig solana.Signature, cancel context.CancelFunc) error // AddSignature adds a new signature for an existing transaction ID AddSignature(id string, sig solana.Signature) error // Remove removes transaction and related signatures from storage if not in finalized or errored state @@ -29,7 +29,7 @@ type PendingTxContext interface { // ListAll returns all of the signatures being tracked for all transactions not yet finalized or errored ListAll() []solana.Signature // ListAllExpiredBroadcastedTxs returns all the expired broadcasted that are in broadcasted state and have expired for given slot height. - ListAllExpiredBroadcastedTxs(currHeight uint64) []PendingTx + ListAllExpiredBroadcastedTxs(currHeight uint64) []pendingTx // Expired returns whether or not confirmation timeout amount of time has passed since creation Expired(sig solana.Signature, confirmationTimeout time.Duration) bool // OnProcessed marks transactions as Processed @@ -48,21 +48,16 @@ type PendingTxContext interface { TrimFinalizedErroredTxs() int } -type PendingTx struct { - Tx solana.Transaction - AccountID string +type pendingTx struct { + tx solana.Transaction cfg TxConfig signatures []solana.Signature - UUID string - IDSetByCaller bool + id string rebroadcastCount int createTs time.Time retentionTs time.Time state TxState lastValidBlockHeight uint64 // to track expiration - BalanceCheck bool - From solana.PublicKey // to perform balanceCheck - Amount uint64 // to perform balanceCheck } var _ PendingTxContext = &pendingTxContext{} @@ -71,9 +66,9 @@ type pendingTxContext struct { cancelBy map[string]context.CancelFunc sigToID map[solana.Signature]string - broadcastedProcessedTxs map[string]PendingTx // broadcasted and processed transactions that may require retry and bumping - confirmedTxs map[string]PendingTx // transactions that require monitoring for re-org - finalizedErroredTxs map[string]PendingTx // finalized and errored transactions held onto for status + broadcastedProcessedTxs map[string]pendingTx // broadcasted and processed transactions that may require retry and bumping + confirmedTxs map[string]pendingTx // transactions that require monitoring for re-org + finalizedErroredTxs map[string]pendingTx // finalized and errored transactions held onto for status lock sync.RWMutex } @@ -83,20 +78,20 @@ func newPendingTxContext() *pendingTxContext { cancelBy: map[string]context.CancelFunc{}, sigToID: map[solana.Signature]string{}, - broadcastedProcessedTxs: map[string]PendingTx{}, - confirmedTxs: map[string]PendingTx{}, - finalizedErroredTxs: map[string]PendingTx{}, + broadcastedProcessedTxs: map[string]pendingTx{}, + confirmedTxs: map[string]pendingTx{}, + finalizedErroredTxs: map[string]pendingTx{}, } } -func (c *pendingTxContext) New(tx PendingTx, sig solana.Signature, cancel context.CancelFunc) error { +func (c *pendingTxContext) New(tx pendingTx, sig solana.Signature, cancel context.CancelFunc) error { err := c.withReadLock(func() error { // validate signature does not exist if _, exists := c.sigToID[sig]; exists { return ErrSigAlreadyExists } // validate id does not exist - if _, exists := c.broadcastedProcessedTxs[tx.UUID]; exists { + if _, exists := c.broadcastedProcessedTxs[tx.id]; exists { return ErrIDAlreadyExists } return nil @@ -110,18 +105,18 @@ func (c *pendingTxContext) New(tx PendingTx, sig solana.Signature, cancel contex if _, exists := c.sigToID[sig]; exists { return "", ErrSigAlreadyExists } - if _, exists := c.broadcastedProcessedTxs[tx.UUID]; exists { + if _, exists := c.broadcastedProcessedTxs[tx.id]; exists { return "", ErrIDAlreadyExists } // save cancel func - c.cancelBy[tx.UUID] = cancel - c.sigToID[sig] = tx.UUID + c.cancelBy[tx.id] = cancel + c.sigToID[sig] = tx.id // add signature to tx tx.signatures = append(tx.signatures, sig) tx.createTs = time.Now() tx.state = Broadcasted // save to the broadcasted map since transaction was just broadcasted - c.broadcastedProcessedTxs[tx.UUID] = tx + c.broadcastedProcessedTxs[tx.id] = tx return "", nil }) return err @@ -190,7 +185,7 @@ func (c *pendingTxContext) Remove(sig solana.Signature) (id string, err error) { if !sigExists { return id, ErrSigDoesNotExist } - var tx PendingTx + var tx pendingTx if tempTx, exists := c.broadcastedProcessedTxs[id]; exists { tx = tempTx delete(c.broadcastedProcessedTxs, id) @@ -221,10 +216,10 @@ func (c *pendingTxContext) ListAll() []solana.Signature { } // ListAllExpiredBroadcastedTxs returns all the expired broadcasted that are in broadcasted state and have expired for given slot height. -func (c *pendingTxContext) ListAllExpiredBroadcastedTxs(currHeight uint64) []PendingTx { +func (c *pendingTxContext) ListAllExpiredBroadcastedTxs(currHeight uint64) []pendingTx { c.lock.RLock() defer c.lock.RUnlock() - broadcastedTxes := make([]PendingTx, 0, len(c.broadcastedProcessedTxs)) // worst case, all of them + broadcastedTxes := make([]pendingTx, 0, len(c.broadcastedProcessedTxs)) // worst case, all of them for _, tx := range c.broadcastedProcessedTxs { if tx.state == Broadcasted && tx.lastValidBlockHeight < currHeight { broadcastedTxes = append(broadcastedTxes, tx) @@ -365,7 +360,7 @@ func (c *pendingTxContext) OnFinalized(sig solana.Signature, retentionTimeout ti if !exists { return id, ErrSigDoesNotExist } - var tx, tempTx PendingTx + var tx, tempTx pendingTx var broadcastedExists, confirmedExists bool if tempTx, broadcastedExists = c.broadcastedProcessedTxs[id]; broadcastedExists { tx = tempTx @@ -472,7 +467,7 @@ func (c *pendingTxContext) OnError(sig solana.Signature, retentionTimeout time.D if !exists { return "", ErrSigDoesNotExist } - var tx, tempTx PendingTx + var tx, tempTx pendingTx var broadcastedExists, confirmedExists bool if tempTx, broadcastedExists = c.broadcastedProcessedTxs[id]; broadcastedExists { tx = tempTx @@ -591,7 +586,7 @@ func newPendingTxContextWithProm(id string) *pendingTxContextWithProm { } } -func (c *pendingTxContextWithProm) New(msg PendingTx, sig solana.Signature, cancel context.CancelFunc) error { +func (c *pendingTxContextWithProm) New(msg pendingTx, sig solana.Signature, cancel context.CancelFunc) error { return c.pendingTx.New(msg, sig, cancel) } @@ -621,7 +616,7 @@ func (c *pendingTxContextWithProm) ListAll() []solana.Signature { return sigs } -func (c *pendingTxContextWithProm) ListAllExpiredBroadcastedTxs(currHeight uint64) []PendingTx { +func (c *pendingTxContextWithProm) ListAllExpiredBroadcastedTxs(currHeight uint64) []pendingTx { return c.pendingTx.ListAllExpiredBroadcastedTxs(currHeight) } diff --git a/pkg/solana/txm/pendingtx_test.go b/pkg/solana/txm/pendingtx_test.go index 5747e1f36..31eeb527f 100644 --- a/pkg/solana/txm/pendingtx_test.go +++ b/pkg/solana/txm/pendingtx_test.go @@ -39,10 +39,10 @@ func TestPendingTxContext_add_remove_multiple(t *testing.T) { n := 5 for i := 0; i < n; i++ { sig, cancel := newProcess() - msg := PendingTx{UUID: uuid.NewString()} + msg := pendingTx{id: uuid.NewString()} err := txs.New(msg, sig, cancel) assert.NoError(t, err) - ids[sig] = msg.UUID + ids[sig] = msg.id } // cannot add signature for non existent ID @@ -74,17 +74,17 @@ func TestPendingTxContext_new(t *testing.T) { txs := newPendingTxContext() // Create new transaction - msg := PendingTx{UUID: uuid.NewString()} + msg := pendingTx{id: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Check it exists in signature map id, exists := txs.sigToID[sig] require.True(t, exists) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Check it exists in broadcasted map - tx, exists := txs.broadcastedProcessedTxs[msg.UUID] + tx, exists := txs.broadcastedProcessedTxs[msg.id] require.True(t, exists) require.Len(t, tx.signatures, 1) require.Equal(t, sig, tx.signatures[0]) @@ -93,11 +93,11 @@ func TestPendingTxContext_new(t *testing.T) { require.Equal(t, Broadcasted, tx.state) // Check it does not exist in confirmed map - tx, exists = txs.confirmedTxs[msg.UUID] + tx, exists = txs.confirmedTxs[msg.id] require.False(t, exists) // Check it does not exist in finalized map - tx, exists = txs.finalizedErroredTxs[msg.UUID] + tx, exists = txs.finalizedErroredTxs[msg.id] require.False(t, exists) } @@ -111,34 +111,34 @@ func TestPendingTxContext_add_signature(t *testing.T) { sig2 := randomSignature(t) // Create new transaction - msg := PendingTx{UUID: uuid.NewString()} + msg := pendingTx{id: uuid.NewString()} err := txs.New(msg, sig1, cancel) require.NoError(t, err) - err = txs.AddSignature(msg.UUID, sig2) + err = txs.AddSignature(msg.id, sig2) require.NoError(t, err) // Check signature map id, exists := txs.sigToID[sig1] require.True(t, exists) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) id, exists = txs.sigToID[sig2] require.True(t, exists) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Check broadcasted map - tx, exists := txs.broadcastedProcessedTxs[msg.UUID] + tx, exists := txs.broadcastedProcessedTxs[msg.id] require.True(t, exists) require.Len(t, tx.signatures, 2) require.Equal(t, sig1, tx.signatures[0]) require.Equal(t, sig2, tx.signatures[1]) // Check confirmed map - tx, exists = txs.confirmedTxs[msg.UUID] + tx, exists = txs.confirmedTxs[msg.id] require.False(t, exists) // Check finalized map - tx, exists = txs.finalizedErroredTxs[msg.UUID] + tx, exists = txs.finalizedErroredTxs[msg.id] require.False(t, exists) }) @@ -146,11 +146,11 @@ func TestPendingTxContext_add_signature(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := PendingTx{UUID: uuid.NewString()} + msg := pendingTx{id: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) - err = txs.AddSignature(msg.UUID, sig) + err = txs.AddSignature(msg.id, sig) require.ErrorIs(t, err, ErrSigAlreadyExists) }) @@ -159,7 +159,7 @@ func TestPendingTxContext_add_signature(t *testing.T) { sig2 := randomSignature(t) // Create new transaction - msg := PendingTx{UUID: uuid.NewString()} + msg := pendingTx{id: uuid.NewString()} err := txs.New(msg, sig1, cancel) require.NoError(t, err) @@ -172,21 +172,21 @@ func TestPendingTxContext_add_signature(t *testing.T) { sig2 := randomSignature(t) // Create new transaction - msg := PendingTx{UUID: uuid.NewString()} + msg := pendingTx{id: uuid.NewString()} err := txs.New(msg, sig1, cancel) require.NoError(t, err) // Transition to processed state id, err := txs.OnProcessed(sig1) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Transition to confirmed state id, err = txs.OnConfirmed(sig1) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) - err = txs.AddSignature(msg.UUID, sig2) + err = txs.AddSignature(msg.id, sig2) require.ErrorIs(t, err, ErrTransactionNotFound) }) } @@ -201,22 +201,22 @@ func TestPendingTxContext_on_broadcasted_processed(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := PendingTx{UUID: uuid.NewString()} + msg := pendingTx{id: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to processed state id, err := txs.OnProcessed(sig) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Check it exists in signature map id, exists := txs.sigToID[sig] require.True(t, exists) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Check it exists in broadcasted map - tx, exists := txs.broadcastedProcessedTxs[msg.UUID] + tx, exists := txs.broadcastedProcessedTxs[msg.id] require.True(t, exists) require.Len(t, tx.signatures, 1) require.Equal(t, sig, tx.signatures[0]) @@ -225,11 +225,11 @@ func TestPendingTxContext_on_broadcasted_processed(t *testing.T) { require.Equal(t, Processed, tx.state) // Check it does not exist in confirmed map - tx, exists = txs.confirmedTxs[msg.UUID] + tx, exists = txs.confirmedTxs[msg.id] require.False(t, exists) // Check it does not exist in finalized map - tx, exists = txs.finalizedErroredTxs[msg.UUID] + tx, exists = txs.finalizedErroredTxs[msg.id] require.False(t, exists) }) @@ -237,19 +237,19 @@ func TestPendingTxContext_on_broadcasted_processed(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := PendingTx{UUID: uuid.NewString()} + msg := pendingTx{id: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to processed state id, err := txs.OnProcessed(sig) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Transition to confirmed state id, err = txs.OnConfirmed(sig) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Transition back to processed state _, err = txs.OnProcessed(sig) @@ -260,24 +260,24 @@ func TestPendingTxContext_on_broadcasted_processed(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := PendingTx{UUID: uuid.NewString()} + msg := pendingTx{id: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to processed state id, err := txs.OnProcessed(sig) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Transition to confirmed state id, err = txs.OnConfirmed(sig) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Transition to finalized state id, err = txs.OnFinalized(sig, retentionTimeout) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Transition back to processed state _, err = txs.OnProcessed(sig) @@ -288,14 +288,14 @@ func TestPendingTxContext_on_broadcasted_processed(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := PendingTx{UUID: uuid.NewString()} + msg := pendingTx{id: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to errored state id, err := txs.OnError(sig, retentionTimeout, Errored, 0) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Transition back to processed state _, err = txs.OnProcessed(sig) @@ -306,14 +306,14 @@ func TestPendingTxContext_on_broadcasted_processed(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := PendingTx{UUID: uuid.NewString()} + msg := pendingTx{id: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to processed state id, err := txs.OnProcessed(sig) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // No error if OnProcessed called again _, err = txs.OnProcessed(sig) @@ -331,31 +331,31 @@ func TestPendingTxContext_on_confirmed(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := PendingTx{UUID: uuid.NewString()} + msg := pendingTx{id: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to processed state id, err := txs.OnProcessed(sig) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Transition to confirmed state id, err = txs.OnConfirmed(sig) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Check it exists in signature map id, exists := txs.sigToID[sig] require.True(t, exists) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Check it does not exist in broadcasted map - _, exists = txs.broadcastedProcessedTxs[msg.UUID] + _, exists = txs.broadcastedProcessedTxs[msg.id] require.False(t, exists) // Check it exists in confirmed map - tx, exists := txs.confirmedTxs[msg.UUID] + tx, exists := txs.confirmedTxs[msg.id] require.True(t, exists) require.Len(t, tx.signatures, 1) require.Equal(t, sig, tx.signatures[0]) @@ -364,7 +364,7 @@ func TestPendingTxContext_on_confirmed(t *testing.T) { require.Equal(t, Confirmed, tx.state) // Check it does not exist in finalized map - tx, exists = txs.finalizedErroredTxs[msg.UUID] + tx, exists = txs.finalizedErroredTxs[msg.id] require.False(t, exists) }) @@ -372,24 +372,24 @@ func TestPendingTxContext_on_confirmed(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := PendingTx{UUID: uuid.NewString()} + msg := pendingTx{id: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to processed state id, err := txs.OnProcessed(sig) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Transition to confirmed state id, err = txs.OnConfirmed(sig) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Transition to finalized state id, err = txs.OnFinalized(sig, retentionTimeout) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Transition back to processed state _, err = txs.OnConfirmed(sig) @@ -400,14 +400,14 @@ func TestPendingTxContext_on_confirmed(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := PendingTx{UUID: uuid.NewString()} + msg := pendingTx{id: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to errored state id, err := txs.OnError(sig, retentionTimeout, Errored, 0) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Transition back to confirmed state _, err = txs.OnConfirmed(sig) @@ -418,19 +418,19 @@ func TestPendingTxContext_on_confirmed(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := PendingTx{UUID: uuid.NewString()} + msg := pendingTx{id: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to processed state id, err := txs.OnProcessed(sig) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Transition to confirmed state id, err = txs.OnConfirmed(sig) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // No error if OnConfirmed called again _, err = txs.OnConfirmed(sig) @@ -449,29 +449,29 @@ func TestPendingTxContext_on_finalized(t *testing.T) { sig2 := randomSignature(t) // Create new transaction - msg := PendingTx{UUID: uuid.NewString()} + msg := pendingTx{id: uuid.NewString()} err := txs.New(msg, sig1, cancel) require.NoError(t, err) // Add second signature - err = txs.AddSignature(msg.UUID, sig2) + err = txs.AddSignature(msg.id, sig2) require.NoError(t, err) // Transition to finalized state id, err := txs.OnFinalized(sig1, retentionTimeout) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Check it does not exist in broadcasted map - _, exists := txs.broadcastedProcessedTxs[msg.UUID] + _, exists := txs.broadcastedProcessedTxs[msg.id] require.False(t, exists) // Check it does not exist in confirmed map - _, exists = txs.confirmedTxs[msg.UUID] + _, exists = txs.confirmedTxs[msg.id] require.False(t, exists) // Check it exists in finalized map - tx, exists := txs.finalizedErroredTxs[msg.UUID] + tx, exists := txs.finalizedErroredTxs[msg.id] require.True(t, exists) // Check status is Finalized @@ -489,39 +489,39 @@ func TestPendingTxContext_on_finalized(t *testing.T) { sig2 := randomSignature(t) // Create new transaction - msg := PendingTx{UUID: uuid.NewString()} + msg := pendingTx{id: uuid.NewString()} err := txs.New(msg, sig1, cancel) require.NoError(t, err) // Add second signature - err = txs.AddSignature(msg.UUID, sig2) + err = txs.AddSignature(msg.id, sig2) require.NoError(t, err) // Transition to processed state id, err := txs.OnProcessed(sig1) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Transition to confirmed state id, err = txs.OnConfirmed(sig1) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Transition to finalized state id, err = txs.OnFinalized(sig1, retentionTimeout) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Check it does not exist in broadcasted map - _, exists := txs.broadcastedProcessedTxs[msg.UUID] + _, exists := txs.broadcastedProcessedTxs[msg.id] require.False(t, exists) // Check it does not exist in confirmed map - _, exists = txs.confirmedTxs[msg.UUID] + _, exists = txs.confirmedTxs[msg.id] require.False(t, exists) // Check it exists in finalized map - tx, exists := txs.finalizedErroredTxs[msg.UUID] + tx, exists := txs.finalizedErroredTxs[msg.id] require.True(t, exists) // Check status is Finalized @@ -538,35 +538,35 @@ func TestPendingTxContext_on_finalized(t *testing.T) { sig1 := randomSignature(t) // Create new transaction - msg := PendingTx{UUID: uuid.NewString()} + msg := pendingTx{id: uuid.NewString()} err := txs.New(msg, sig1, cancel) require.NoError(t, err) // Transition to processed state id, err := txs.OnProcessed(sig1) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Transition to confirmed state id, err = txs.OnConfirmed(sig1) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Transition to finalized state id, err = txs.OnFinalized(sig1, 0*time.Second) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Check it does not exist in broadcasted map - _, exists := txs.broadcastedProcessedTxs[msg.UUID] + _, exists := txs.broadcastedProcessedTxs[msg.id] require.False(t, exists) // Check it does not exist in confirmed map - _, exists = txs.confirmedTxs[msg.UUID] + _, exists = txs.confirmedTxs[msg.id] require.False(t, exists) // Check it does not exist in finalized map - _, exists = txs.finalizedErroredTxs[msg.UUID] + _, exists = txs.finalizedErroredTxs[msg.id] require.False(t, exists) // Check sigs do no exist in signature map @@ -578,14 +578,14 @@ func TestPendingTxContext_on_finalized(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := PendingTx{UUID: uuid.NewString()} + msg := pendingTx{id: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to errored state id, err := txs.OnError(sig, retentionTimeout, Errored, 0) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Transition back to confirmed state _, err = txs.OnFinalized(sig, retentionTimeout) @@ -603,25 +603,25 @@ func TestPendingTxContext_on_error(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := PendingTx{UUID: uuid.NewString()} + msg := pendingTx{id: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to errored state id, err := txs.OnError(sig, retentionTimeout, Errored, 0) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Check it does not exist in broadcasted map - _, exists := txs.broadcastedProcessedTxs[msg.UUID] + _, exists := txs.broadcastedProcessedTxs[msg.id] require.False(t, exists) // Check it does not exist in confirmed map - _, exists = txs.confirmedTxs[msg.UUID] + _, exists = txs.confirmedTxs[msg.id] require.False(t, exists) // Check it exists in errored map - tx, exists := txs.finalizedErroredTxs[msg.UUID] + tx, exists := txs.finalizedErroredTxs[msg.id] require.True(t, exists) // Check status is Finalized @@ -636,30 +636,30 @@ func TestPendingTxContext_on_error(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := PendingTx{UUID: uuid.NewString()} + msg := pendingTx{id: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to errored state id, err := txs.OnConfirmed(sig) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Transition to errored state id, err = txs.OnError(sig, retentionTimeout, Errored, 0) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Check it does not exist in broadcasted map - _, exists := txs.broadcastedProcessedTxs[msg.UUID] + _, exists := txs.broadcastedProcessedTxs[msg.id] require.False(t, exists) // Check it does not exist in confirmed map - _, exists = txs.confirmedTxs[msg.UUID] + _, exists = txs.confirmedTxs[msg.id] require.False(t, exists) // Check it exists in errored map - tx, exists := txs.finalizedErroredTxs[msg.UUID] + tx, exists := txs.finalizedErroredTxs[msg.id] require.True(t, exists) // Check status is Finalized @@ -703,30 +703,30 @@ func TestPendingTxContext_on_error(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := PendingTx{UUID: uuid.NewString()} + msg := pendingTx{id: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to errored state id, err := txs.OnConfirmed(sig) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Transition to errored state id, err = txs.OnError(sig, 0*time.Second, Errored, 0) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Check it does not exist in broadcasted map - _, exists := txs.broadcastedProcessedTxs[msg.UUID] + _, exists := txs.broadcastedProcessedTxs[msg.id] require.False(t, exists) // Check it does not exist in confirmed map - _, exists = txs.confirmedTxs[msg.UUID] + _, exists = txs.confirmedTxs[msg.id] require.False(t, exists) // Check it exists in errored map - _, exists = txs.finalizedErroredTxs[msg.UUID] + _, exists = txs.finalizedErroredTxs[msg.id] require.False(t, exists) // Check sigs do no exist in signature map @@ -738,14 +738,14 @@ func TestPendingTxContext_on_error(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := PendingTx{UUID: uuid.NewString()} + msg := pendingTx{id: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) // Transition to confirmed state id, err := txs.OnFinalized(sig, retentionTimeout) require.NoError(t, err) - require.Equal(t, msg.UUID, id) + require.Equal(t, msg.id, id) // Transition back to confirmed state id, err = txs.OnError(sig, retentionTimeout, Errored, 0) @@ -833,50 +833,50 @@ func TestPendingTxContext_remove(t *testing.T) { erroredSig := randomSignature(t) // Create new broadcasted transaction with extra sig - broadcastedMsg := PendingTx{UUID: uuid.NewString()} + broadcastedMsg := pendingTx{id: uuid.NewString()} err := txs.New(broadcastedMsg, broadcastedSig1, cancel) require.NoError(t, err) - err = txs.AddSignature(broadcastedMsg.UUID, broadcastedSig2) + err = txs.AddSignature(broadcastedMsg.id, broadcastedSig2) require.NoError(t, err) // Create new processed transaction - processedMsg := PendingTx{UUID: uuid.NewString()} + processedMsg := pendingTx{id: uuid.NewString()} err = txs.New(processedMsg, processedSig, cancel) require.NoError(t, err) id, err := txs.OnProcessed(processedSig) require.NoError(t, err) - require.Equal(t, processedMsg.UUID, id) + require.Equal(t, processedMsg.id, id) // Create new confirmed transaction - confirmedMsg := PendingTx{UUID: uuid.NewString()} + confirmedMsg := pendingTx{id: uuid.NewString()} err = txs.New(confirmedMsg, confirmedSig, cancel) require.NoError(t, err) id, err = txs.OnConfirmed(confirmedSig) require.NoError(t, err) - require.Equal(t, confirmedMsg.UUID, id) + require.Equal(t, confirmedMsg.id, id) // Create new finalized transaction - finalizedMsg := PendingTx{UUID: uuid.NewString()} + finalizedMsg := pendingTx{id: uuid.NewString()} err = txs.New(finalizedMsg, finalizedSig, cancel) require.NoError(t, err) id, err = txs.OnFinalized(finalizedSig, retentionTimeout) require.NoError(t, err) - require.Equal(t, finalizedMsg.UUID, id) + require.Equal(t, finalizedMsg.id, id) // Create new errored transaction - erroredMsg := PendingTx{UUID: uuid.NewString()} + erroredMsg := pendingTx{id: uuid.NewString()} err = txs.New(erroredMsg, erroredSig, cancel) require.NoError(t, err) id, err = txs.OnError(erroredSig, retentionTimeout, Errored, 0) require.NoError(t, err) - require.Equal(t, erroredMsg.UUID, id) + require.Equal(t, erroredMsg.id, id) // Remove broadcasted transaction id, err = txs.Remove(broadcastedSig1) require.NoError(t, err) - require.Equal(t, broadcastedMsg.UUID, id) + require.Equal(t, broadcastedMsg.id, id) // Check removed from broadcasted map - _, exists := txs.broadcastedProcessedTxs[broadcastedMsg.UUID] + _, exists := txs.broadcastedProcessedTxs[broadcastedMsg.id] require.False(t, exists) // Check all signatures removed from sig map _, exists = txs.sigToID[broadcastedSig1] @@ -887,9 +887,9 @@ func TestPendingTxContext_remove(t *testing.T) { // Remove processed transaction id, err = txs.Remove(processedSig) require.NoError(t, err) - require.Equal(t, processedMsg.UUID, id) + require.Equal(t, processedMsg.id, id) // Check removed from broadcasted map - _, exists = txs.broadcastedProcessedTxs[processedMsg.UUID] + _, exists = txs.broadcastedProcessedTxs[processedMsg.id] require.False(t, exists) // Check all signatures removed from sig map _, exists = txs.sigToID[processedSig] @@ -898,9 +898,9 @@ func TestPendingTxContext_remove(t *testing.T) { // Remove confirmed transaction id, err = txs.Remove(confirmedSig) require.NoError(t, err) - require.Equal(t, confirmedMsg.UUID, id) + require.Equal(t, confirmedMsg.id, id) // Check removed from confirmed map - _, exists = txs.confirmedTxs[confirmedMsg.UUID] + _, exists = txs.confirmedTxs[confirmedMsg.id] require.False(t, exists) // Check all signatures removed from sig map _, exists = txs.sigToID[confirmedSig] @@ -924,32 +924,32 @@ func TestPendingTxContext_trim_finalized_errored_txs(t *testing.T) { txs := newPendingTxContext() // Create new finalized transaction with retention ts in the past and add to map - finalizedMsg1 := PendingTx{UUID: uuid.NewString(), retentionTs: time.Now().Add(-2 * time.Second)} - txs.finalizedErroredTxs[finalizedMsg1.UUID] = finalizedMsg1 + finalizedMsg1 := pendingTx{id: uuid.NewString(), retentionTs: time.Now().Add(-2 * time.Second)} + txs.finalizedErroredTxs[finalizedMsg1.id] = finalizedMsg1 // Create new finalized transaction with retention ts in the future and add to map - finalizedMsg2 := PendingTx{UUID: uuid.NewString(), retentionTs: time.Now().Add(1 * time.Second)} - txs.finalizedErroredTxs[finalizedMsg2.UUID] = finalizedMsg2 + finalizedMsg2 := pendingTx{id: uuid.NewString(), retentionTs: time.Now().Add(1 * time.Second)} + txs.finalizedErroredTxs[finalizedMsg2.id] = finalizedMsg2 // Create new finalized transaction with retention ts in the past and add to map - erroredMsg := PendingTx{UUID: uuid.NewString(), retentionTs: time.Now().Add(-2 * time.Second)} - txs.finalizedErroredTxs[erroredMsg.UUID] = erroredMsg + erroredMsg := pendingTx{id: uuid.NewString(), retentionTs: time.Now().Add(-2 * time.Second)} + txs.finalizedErroredTxs[erroredMsg.id] = erroredMsg // Delete finalized/errored transactions that have passed the retention period txs.TrimFinalizedErroredTxs() // Check finalized message past retention is deleted - _, exists := txs.finalizedErroredTxs[finalizedMsg1.UUID] + _, exists := txs.finalizedErroredTxs[finalizedMsg1.id] require.False(t, exists) // Check errored message past retention is deleted - _, exists = txs.finalizedErroredTxs[erroredMsg.UUID] + _, exists = txs.finalizedErroredTxs[erroredMsg.id] require.False(t, exists) // Check finalized message within retention period still exists - msg, exists := txs.finalizedErroredTxs[finalizedMsg2.UUID] + msg, exists := txs.finalizedErroredTxs[finalizedMsg2.id] require.True(t, exists) - require.Equal(t, finalizedMsg2.UUID, msg.UUID) + require.Equal(t, finalizedMsg2.id, msg.id) } func TestPendingTxContext_expired(t *testing.T) { @@ -958,16 +958,16 @@ func TestPendingTxContext_expired(t *testing.T) { sig := solana.Signature{} txs := newPendingTxContext() - msg := PendingTx{UUID: uuid.NewString()} + msg := pendingTx{id: uuid.NewString()} err := txs.New(msg, sig, cancel) assert.NoError(t, err) - msg, exists := txs.broadcastedProcessedTxs[msg.UUID] + msg, exists := txs.broadcastedProcessedTxs[msg.id] require.True(t, exists) // Set createTs to 10 seconds ago msg.createTs = time.Now().Add(-10 * time.Second) - txs.broadcastedProcessedTxs[msg.UUID] = msg + txs.broadcastedProcessedTxs[msg.id] = msg assert.False(t, txs.Expired(sig, 0*time.Second)) // false if timeout 0 assert.True(t, txs.Expired(sig, 5*time.Second)) // expired for 5s lifetime @@ -975,7 +975,7 @@ func TestPendingTxContext_expired(t *testing.T) { id, err := txs.Remove(sig) assert.NoError(t, err) - assert.Equal(t, msg.UUID, id) + assert.Equal(t, msg.id, id) assert.False(t, txs.Expired(sig, 60*time.Second)) // no longer exists, should return false } @@ -987,11 +987,11 @@ func TestPendingTxContext_race(t *testing.T) { var err [2]error go func() { - err[0] = txCtx.New(PendingTx{UUID: uuid.NewString()}, solana.Signature{}, func() {}) + err[0] = txCtx.New(pendingTx{id: uuid.NewString()}, solana.Signature{}, func() {}) wg.Done() }() go func() { - err[1] = txCtx.New(PendingTx{UUID: uuid.NewString()}, solana.Signature{}, func() {}) + err[1] = txCtx.New(pendingTx{id: uuid.NewString()}, solana.Signature{}, func() {}) wg.Done() }() @@ -1001,7 +1001,7 @@ func TestPendingTxContext_race(t *testing.T) { t.Run("add signature", func(t *testing.T) { txCtx := newPendingTxContext() - msg := PendingTx{UUID: uuid.NewString()} + msg := pendingTx{id: uuid.NewString()} createErr := txCtx.New(msg, solana.Signature{}, func() {}) require.NoError(t, createErr) var wg sync.WaitGroup @@ -1009,11 +1009,11 @@ func TestPendingTxContext_race(t *testing.T) { var err [2]error go func() { - err[0] = txCtx.AddSignature(msg.UUID, solana.Signature{1}) + err[0] = txCtx.AddSignature(msg.id, solana.Signature{1}) wg.Done() }() go func() { - err[1] = txCtx.AddSignature(msg.UUID, solana.Signature{1}) + err[1] = txCtx.AddSignature(msg.id, solana.Signature{1}) wg.Done() }() @@ -1023,7 +1023,7 @@ func TestPendingTxContext_race(t *testing.T) { t.Run("remove", func(t *testing.T) { txCtx := newPendingTxContext() - msg := PendingTx{UUID: uuid.NewString()} + msg := pendingTx{id: uuid.NewString()} err := txCtx.New(msg, solana.Signature{}, func() {}) require.NoError(t, err) var wg sync.WaitGroup @@ -1056,56 +1056,56 @@ func TestGetTxState(t *testing.T) { fatallyErroredSig := randomSignature(t) // Create new broadcasted transaction with extra sig - broadcastedMsg := PendingTx{UUID: uuid.NewString()} + broadcastedMsg := pendingTx{id: uuid.NewString()} err := txs.New(broadcastedMsg, broadcastedSig, cancel) require.NoError(t, err) var state TxState // Create new processed transaction - processedMsg := PendingTx{UUID: uuid.NewString()} + processedMsg := pendingTx{id: uuid.NewString()} err = txs.New(processedMsg, processedSig, cancel) require.NoError(t, err) id, err := txs.OnProcessed(processedSig) require.NoError(t, err) - require.Equal(t, processedMsg.UUID, id) + require.Equal(t, processedMsg.id, id) // Check Processed state is returned - state, err = txs.GetTxState(processedMsg.UUID) + state, err = txs.GetTxState(processedMsg.id) require.NoError(t, err) require.Equal(t, Processed, state) // Create new confirmed transaction - confirmedMsg := PendingTx{UUID: uuid.NewString()} + confirmedMsg := pendingTx{id: uuid.NewString()} err = txs.New(confirmedMsg, confirmedSig, cancel) require.NoError(t, err) id, err = txs.OnConfirmed(confirmedSig) require.NoError(t, err) - require.Equal(t, confirmedMsg.UUID, id) + require.Equal(t, confirmedMsg.id, id) // Check Confirmed state is returned - state, err = txs.GetTxState(confirmedMsg.UUID) + state, err = txs.GetTxState(confirmedMsg.id) require.NoError(t, err) require.Equal(t, Confirmed, state) // Create new finalized transaction - finalizedMsg := PendingTx{UUID: uuid.NewString()} + finalizedMsg := pendingTx{id: uuid.NewString()} err = txs.New(finalizedMsg, finalizedSig, cancel) require.NoError(t, err) id, err = txs.OnFinalized(finalizedSig, retentionTimeout) require.NoError(t, err) - require.Equal(t, finalizedMsg.UUID, id) + require.Equal(t, finalizedMsg.id, id) // Check Finalized state is returned - state, err = txs.GetTxState(finalizedMsg.UUID) + state, err = txs.GetTxState(finalizedMsg.id) require.NoError(t, err) require.Equal(t, Finalized, state) // Create new errored transaction - erroredMsg := PendingTx{UUID: uuid.NewString()} + erroredMsg := pendingTx{id: uuid.NewString()} err = txs.New(erroredMsg, erroredSig, cancel) require.NoError(t, err) id, err = txs.OnError(erroredSig, retentionTimeout, Errored, 0) require.NoError(t, err) - require.Equal(t, erroredMsg.UUID, id) + require.Equal(t, erroredMsg.id, id) // Check Errored state is returned - state, err = txs.GetTxState(erroredMsg.UUID) + state, err = txs.GetTxState(erroredMsg.id) require.NoError(t, err) require.Equal(t, Errored, state) diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index 3c3904aef..a38aba116 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -9,7 +9,6 @@ import ( "sync" "time" - "github.com/gagliardetto/solana-go" solanaGo "github.com/gagliardetto/solana-go" "github.com/gagliardetto/solana-go/rpc" "github.com/google/uuid" @@ -51,8 +50,8 @@ var _ loop.Keystore = (SimpleKeystore)(nil) type Txm struct { services.StateMachine lggr logger.Logger - chSend chan PendingTx - chSim chan PendingTx + chSend chan pendingTx + chSim chan pendingTx chStop services.StopChan done sync.WaitGroup cfg config.Config @@ -95,8 +94,8 @@ func NewTxm(chainID string, client internal.Loader[client.ReaderWriter], return &Txm{ lggr: logger.Named(lggr, "Txm"), - chSend: make(chan PendingTx, MaxQueueLen), // queue can support 1000 pending txs - chSim: make(chan PendingTx, MaxQueueLen), // queue can support 1000 pending txs + chSend: make(chan pendingTx, MaxQueueLen), // queue can support 1000 pending txs + chSim: make(chan pendingTx, MaxQueueLen), // queue can support 1000 pending txs chStop: make(chan struct{}), cfg: cfg, txs: newPendingTxContextWithProm(chainID), @@ -160,9 +159,9 @@ func (txm *Txm) run() { } // send tx + signature to simulation queue - msg.Tx = tx + msg.tx = tx msg.signatures = append(msg.signatures, sig) - msg.UUID = id + msg.id = id select { case txm.chSim <- msg: default: @@ -179,7 +178,7 @@ func (txm *Txm) run() { // sendWithRetry attempts to send a transaction with exponential backoff retry logic. // It prepares the transaction, builds and signs it, sends the initial transaction, and starts a retry routine with fee bumping if needed. // The function returns the signed transaction, its ID, and the initial signature for use in simulation. -func (txm *Txm) sendWithRetry(ctx context.Context, msg PendingTx) (solanaGo.Transaction, string, solanaGo.Signature, error) { +func (txm *Txm) sendWithRetry(ctx context.Context, msg pendingTx) (solanaGo.Transaction, string, solanaGo.Signature, error) { // Prepare transaction assigning blockhash and lastValidBlockHeight (for expiration tracking). // If required, it also performs balanceCheck and sets compute unit limit. if err := txm.prepareTransaction(ctx, &msg); err != nil { @@ -217,12 +216,12 @@ func (txm *Txm) sendWithRetry(ctx context.Context, msg PendingTx) (solanaGo.Tran }() // Return signed tx, id, signature for use in simulation - return initTx, msg.UUID, sig, nil + return initTx, msg.id, sig, nil } // prepareTransaction sets blockhash and lastValidBlockHeight which will be used to track expiration. // If required, it also performs balanceCheck and sets compute unit limit. -func (txm *Txm) prepareTransaction(ctx context.Context, msg *PendingTx) error { +func (txm *Txm) prepareTransaction(ctx context.Context, msg *pendingTx) error { client, err := txm.client.Get() if err != nil { return fmt.Errorf("failed to get client in sendWithRetry: %w", err) @@ -233,19 +232,12 @@ func (txm *Txm) prepareTransaction(ctx context.Context, msg *PendingTx) error { if err != nil { return fmt.Errorf("failed to get blockhash: %w", err) } - msg.Tx.Message.RecentBlockhash = blockhash.Value.Blockhash + msg.tx.Message.RecentBlockhash = blockhash.Value.Blockhash msg.lastValidBlockHeight = blockhash.Value.LastValidBlockHeight - // Validate balance if required - if msg.BalanceCheck { - if err = solanaValidateBalance(ctx, client, msg.From, msg.Amount, msg.Tx.Message.ToBase64()); err != nil { - return fmt.Errorf("failed to validate balance: %w", err) - } - } - // Set compute unit limit if msg.cfg.ComputeUnitLimit != 0 { - if err := fees.SetComputeUnitLimit(&msg.Tx, fees.ComputeUnitLimit(msg.cfg.ComputeUnitLimit)); err != nil { + if err := fees.SetComputeUnitLimit(&msg.tx, fees.ComputeUnitLimit(msg.cfg.ComputeUnitLimit)); err != nil { return fmt.Errorf("failed to add compute unit limit instruction: %w", err) } } @@ -253,27 +245,10 @@ func (txm *Txm) prepareTransaction(ctx context.Context, msg *PendingTx) error { return nil } -func solanaValidateBalance(ctx context.Context, reader client.Reader, from solana.PublicKey, amount uint64, msg string) error { - balance, err := reader.Balance(ctx, from) - if err != nil { - return err - } - - fee, err := reader.GetFeeForMessage(ctx, msg) - if err != nil { - return err - } - - if balance < (amount + fee) { - return fmt.Errorf("balance %d is too low for this transaction to be executed: amount %d + fee %d", balance, amount, fee) - } - return nil -} - // buildTx builds and signs the transaction with the appropriate compute unit price. -func (txm *Txm) buildTx(ctx context.Context, msg PendingTx, retryCount int) (solanaGo.Transaction, error) { +func (txm *Txm) buildTx(ctx context.Context, msg pendingTx, retryCount int) (solanaGo.Transaction, error) { // work with a copy - newTx := msg.Tx + newTx := msg.tx // Set compute unit price (fee) fee := fees.ComputeUnitPrice( @@ -293,7 +268,7 @@ func (txm *Txm) buildTx(ctx context.Context, msg PendingTx, retryCount int) (sol if err != nil { return solanaGo.Transaction{}, fmt.Errorf("error in MarshalBinary: %w", err) } - sigBytes, err := txm.ks.Sign(ctx, msg.Tx.Message.AccountKeys[0].String(), txMsg) + sigBytes, err := txm.ks.Sign(ctx, msg.tx.Message.AccountKeys[0].String(), txMsg) if err != nil { return solanaGo.Transaction{}, fmt.Errorf("error in Sign: %w", err) } @@ -305,7 +280,7 @@ func (txm *Txm) buildTx(ctx context.Context, msg PendingTx, retryCount int) (sol } // sendInitialTx sends the initial tx and handles any errors that may occur. It also stores the transaction signature and cancellation function. -func (txm *Txm) sendInitialTx(ctx context.Context, initTx solanaGo.Transaction, msg PendingTx, cancel context.CancelFunc) (solanaGo.Signature, error) { +func (txm *Txm) sendInitialTx(ctx context.Context, initTx solanaGo.Transaction, msg pendingTx, cancel context.CancelFunc) (solanaGo.Signature, error) { // Send initial transaction sig, err := txm.sendTx(ctx, &initTx) if err != nil { @@ -321,14 +296,14 @@ func (txm *Txm) sendInitialTx(ctx context.Context, initTx solanaGo.Transaction, return solanaGo.Signature{}, fmt.Errorf("failed to save tx signature (%s) to inflight txs: %w", sig, err) } - txm.lggr.Debugw("tx initial broadcast", "id", msg.UUID, "fee", msg.cfg.BaseComputeUnitPrice, "signature", sig) + txm.lggr.Debugw("tx initial broadcast", "id", msg.id, "fee", msg.cfg.BaseComputeUnitPrice, "signature", sig) return sig, nil } // retryTx contains the logic for retrying the transaction, including exponential backoff and fee bumping. // Retries until context cancelled by timeout or called externally. // It uses handleRetry helper function to handle each retry attempt. -func (txm *Txm) retryTx(ctx context.Context, msg PendingTx, currentTx solanaGo.Transaction, sigs *signatureList) { +func (txm *Txm) retryTx(ctx context.Context, msg pendingTx, currentTx solanaGo.Transaction, sigs *signatureList) { deltaT := 1 // initial delay in ms tick := time.After(0) bumpCount := 0 @@ -340,7 +315,7 @@ func (txm *Txm) retryTx(ctx context.Context, msg PendingTx, currentTx solanaGo.T case <-ctx.Done(): // stop sending tx after retry tx ctx times out (does not stop confirmation polling for tx) wg.Wait() - txm.lggr.Debugw("stopped tx retry", "id", msg.UUID, "signatures", sigs.List(), "err", context.Cause(ctx)) + txm.lggr.Debugw("stopped tx retry", "id", msg.id, "signatures", sigs.List(), "err", context.Cause(ctx)) return case <-tick: // Determine if we should bump the fee @@ -353,7 +328,7 @@ func (txm *Txm) retryTx(ctx context.Context, msg PendingTx, currentTx solanaGo.T currentTx, err = txm.buildTx(ctx, msg, bumpCount) if err != nil { // Exit if unable to build transaction for retrying - txm.lggr.Errorw("failed to build bumped retry tx", "error", err, "id", msg.UUID) + txm.lggr.Errorw("failed to build bumped retry tx", "error", err, "id", msg.id) return } // allocates space for new signature that will be introduced in handleRetry if needs bumping. @@ -394,31 +369,31 @@ func (txm *Txm) updateBackoffDelay(currentDelay int) int { } // handleRetry handles the logic for each retry attempt, including sending the transaction, updating signatures, and logging. -func (txm *Txm) handleRetry(ctx context.Context, msg PendingTx, bump bool, count int, retryTx solanaGo.Transaction, sigs *signatureList) { +func (txm *Txm) handleRetry(ctx context.Context, msg pendingTx, bump bool, count int, retryTx solanaGo.Transaction, sigs *signatureList) { // send retry transaction retrySig, err := txm.sendTx(ctx, &retryTx) if err != nil { // this could occur if endpoint goes down or if ctx cancelled if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) { - txm.lggr.Debugw("ctx error on send retry transaction", "error", err, "signatures", sigs.List(), "id", msg.UUID) + txm.lggr.Debugw("ctx error on send retry transaction", "error", err, "signatures", sigs.List(), "id", msg.id) } else { - txm.lggr.Warnw("failed to send retry transaction", "error", err, "signatures", sigs.List(), "id", msg.UUID) + txm.lggr.Warnw("failed to send retry transaction", "error", err, "signatures", sigs.List(), "id", msg.id) } return } // if bump is true, update signature list and set new signature in space already allocated. if bump { - if err := txm.txs.AddSignature(msg.UUID, retrySig); err != nil { - txm.lggr.Warnw("error in adding retry transaction", "error", err, "id", msg.UUID) + if err := txm.txs.AddSignature(msg.id, retrySig); err != nil { + txm.lggr.Warnw("error in adding retry transaction", "error", err, "id", msg.id) return } if err := sigs.Set(count, retrySig); err != nil { // this should never happen - txm.lggr.Errorw("INVARIANT VIOLATION: failed to set signature", "error", err, "id", msg.UUID) + txm.lggr.Errorw("INVARIANT VIOLATION: failed to set signature", "error", err, "id", msg.id) return } - txm.lggr.Debugw("tx rebroadcast with bumped fee", "id", msg.UUID, "retryCount", count, "fee", msg.cfg.BaseComputeUnitPrice, "signatures", sigs.List()) + txm.lggr.Debugw("tx rebroadcast with bumped fee", "id", msg.id, "retryCount", count, "fee", msg.cfg.BaseComputeUnitPrice, "signatures", sigs.List()) } // prevent locking on waitgroup when ctx is closed @@ -452,7 +427,6 @@ func (txm *Txm) confirm() { case <-ctx.Done(): return case <-tick: - client, err := txm.client.Get() // Get list of transaction signatures to confirm // If no signatures to confirm, we can break loop. sigs := txm.txs.ListAll() @@ -460,10 +434,12 @@ func (txm *Txm) confirm() { break } + client, err := txm.client.Get() if err != nil { txm.lggr.Errorw("failed to get client in txm.confirm", "error", err) - return + break } + if txm.cfg.TxExpirationRebroadcast() { txm.rebroadcastExpiredTxs(ctx, client) } @@ -510,42 +486,32 @@ func (txm *Txm) rebroadcastExpiredTxs(ctx context.Context, client client.ReaderW // Rebroadcast all expired txes for _, tx := range txm.txs.ListAllExpiredBroadcastedTxs(currHeight) { - txm.lggr.Infow("transaction expired, rebroadcasting", "id", tx.UUID, "signature", tx.signatures) + txm.lggr.Infow("transaction expired, rebroadcasting", "id", tx.id, "signature", tx.signatures) if len(tx.signatures) == 0 { // prevent panic, shouldn't happen. - txm.lggr.Errorw("no signatures found for expired transaction", "id", tx.UUID) + txm.lggr.Errorw("no signatures found for expired transaction", "id", tx.id) continue } _, err := txm.txs.Remove(tx.signatures[0]) // only picking signature[0] because remove func removes all remaining signatures. if err != nil { - txm.lggr.Errorw("failed to remove expired transaction", "id", tx.UUID, "error", err) + txm.lggr.Errorw("failed to remove expired transaction", "id", tx.id, "error", err) continue } - rebroadcastTx := &PendingTx{ - Tx: tx.Tx, - BalanceCheck: tx.BalanceCheck, - Amount: tx.Amount, - From: tx.From, - IDSetByCaller: tx.IDSetByCaller, + rebroadcastTx := pendingTx{ + tx: tx.tx, + // new id with the same family as the original tx by appending #rebroadcastCount+1. + id: fmt.Sprintf("%s#%d", tx.id, tx.rebroadcastCount+1), rebroadcastCount: tx.rebroadcastCount + 1, } - // If the ID was set by the caller, we should keep it similar. - if tx.IDSetByCaller { - // Append #tx.rebroadcast count to the end of the ID. - rebroadcastTx.UUID = fmt.Sprintf("%s#%d", tx.UUID, rebroadcastTx.rebroadcastCount) - } else { - rebroadcastTx.UUID = uuid.New().String() - } - - // Re-enqueue the transaction for rebroadcasting - err = txm.Enqueue(ctx, rebroadcastTx) + // call sendWithRetry directly + _, _, _, err = txm.sendWithRetry(ctx, rebroadcastTx) if err != nil { - txm.lggr.Errorw("failed to enqueue rebroadcast transaction", "previous id", tx.UUID, "new id", rebroadcastTx.UUID, "error", err) + txm.lggr.Errorw("failed to rebroadcast transaction", "id", tx.id, "error", err) continue } - txm.lggr.Infow("rebroadcast transaction sent", "previous id", tx.UUID, "new id", rebroadcastTx.UUID) + txm.lggr.Infow("rebroadcast transaction sent", "id", tx.id) } } @@ -665,11 +631,11 @@ func (txm *Txm) simulate() { case <-ctx.Done(): return case msg := <-txm.chSim: - res, err := txm.simulateTx(ctx, &msg.Tx) + res, err := txm.simulateTx(ctx, &msg.tx) if err != nil { // this error can occur if endpoint goes down or if invalid signature (invalid signature should occur further upstream in sendWithRetry) // allow retry to continue in case temporary endpoint failure (if still invalid, confirmation or timeout will cleanup) - txm.lggr.Debugw("failed to simulate tx", "id", msg.UUID, "signatures", msg.signatures, "error", err) + txm.lggr.Debugw("failed to simulate tx", "id", msg.id, "signatures", msg.signatures, "error", err) continue } @@ -680,7 +646,7 @@ func (txm *Txm) simulate() { // Transaction has to have a signature if simulation succeeded but added check for belt and braces approach if len(msg.signatures) > 0 { - txm.processSimulationError(msg.UUID, msg.signatures[0], res) + txm.processSimulationError(msg.id, msg.signatures[0], res) } } } @@ -709,20 +675,24 @@ func (txm *Txm) reap() { } // Enqueue enqueues a msg destined for the solana chain. -func (txm *Txm) Enqueue(ctx context.Context, msg *PendingTx, txCfgs ...SetTxConfig) error { +func (txm *Txm) Enqueue(ctx context.Context, accountID string, tx *solanaGo.Transaction, txID *string, txCfgs ...SetTxConfig) error { if err := txm.Ready(); err != nil { return fmt.Errorf("error in soltxm.Enqueue: %w", err) } - // validate msg and tx are not empty - if msg == nil || isEmptyTransactionAccountKeys(msg.Tx) { - return errors.New("error in soltxm.Enqueue: tx or account keys are empty") + // validate nil pointer + if tx == nil { + return errors.New("error in soltxm.Enqueue: tx is nil pointer") + } + // validate account keys slice + if len(tx.Message.AccountKeys) == 0 { + return errors.New("error in soltxm.Enqueue: not enough account keys in tx") } // validate expected key exists by trying to sign with it // fee payer account is index 0 account // https://github.com/gagliardetto/solana-go/blob/main/transaction.go#L252 - _, err := txm.ks.Sign(ctx, msg.Tx.Message.AccountKeys[0].String(), nil) + _, err := txm.ks.Sign(ctx, tx.Message.AccountKeys[0].String(), nil) if err != nil { return fmt.Errorf("error in soltxm.Enqueue.GetKey: %w", err) } @@ -742,7 +712,7 @@ func (txm *Txm) Enqueue(ctx context.Context, msg *PendingTx, txCfgs ...SetTxConf // Perform compute unit limit estimation after storing transaction // If error found during simulation, transaction should be in storage to mark accordingly if cfg.EstimateComputeUnitLimit { - computeUnitLimit, err := txm.EstimateComputeUnitLimit(ctx, &msg.Tx) + computeUnitLimit, err := txm.EstimateComputeUnitLimit(ctx, tx) if err != nil { return fmt.Errorf("transaction failed simulation: %w", err) } @@ -752,19 +722,23 @@ func (txm *Txm) Enqueue(ctx context.Context, msg *PendingTx, txCfgs ...SetTxConf } } - msg.cfg = cfg - if msg.UUID == "" { - // If ID was not set by caller, create one. - msg.UUID = uuid.New().String() + msg := &pendingTx{ + tx: *tx, + cfg: cfg, + } + + // If ID was not set by caller, create one. + if txID != nil && *txID != "" { + msg.id = *txID } else { - msg.IDSetByCaller = true + msg.id = uuid.New().String() } select { case txm.chSend <- *msg: default: - txm.lggr.Errorw("failed to enqeue tx", "queueFull", len(txm.chSend) == MaxQueueLen, "tx", msg) - return fmt.Errorf("failed to enqueue transaction for %s", msg.AccountID) + txm.lggr.Errorw("failed to enqueue tx", "queueFull", len(txm.chSend) == MaxQueueLen, "tx", msg) + return fmt.Errorf("failed to enqueue transaction for %s", accountID) } return nil } @@ -950,8 +924,3 @@ func (txm *Txm) defaultTxConfig() TxConfig { EstimateComputeUnitLimit: txm.cfg.EstimateComputeUnitLimit(), } } - -// isEmptyTransactionAccountKeys validates that a solana tx and its account keys are not empty. -func isEmptyTransactionAccountKeys(tx solana.Transaction) bool { - return len(tx.Signatures) == 0 && len(tx.Message.AccountKeys) == 0 -} diff --git a/pkg/solana/txm/txm_internal_test.go b/pkg/solana/txm/txm_internal_test.go index 2ee5a4c5e..00e7be8f1 100644 --- a/pkg/solana/txm/txm_internal_test.go +++ b/pkg/solana/txm/txm_internal_test.go @@ -209,7 +209,7 @@ func TestTxm(t *testing.T) { // send tx testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()})) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) wg.Wait() // no transactions stored inflight txs list @@ -245,7 +245,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()})) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) wg.Wait() // wait to be picked up and processed // no transactions stored inflight txs list @@ -276,7 +276,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()})) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared quickly @@ -312,7 +312,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()})) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared after timeout @@ -352,7 +352,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()})) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared after timeout @@ -402,7 +402,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()})) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared after timeout @@ -444,7 +444,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()})) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared after timeout @@ -489,7 +489,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()})) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // inflight txs cleared after timeout @@ -541,7 +541,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()})) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // inflight txs cleared after timeout @@ -579,7 +579,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()})) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // inflight txs cleared after timeout @@ -625,7 +625,7 @@ func TestTxm(t *testing.T) { // send tx testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()})) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) wg.Wait() // no transactions stored inflight txs list @@ -679,7 +679,7 @@ func TestTxm(t *testing.T) { // send tx - with disabled fee bumping testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()}, SetFeeBumpPeriod(0))) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, SetFeeBumpPeriod(0))) wg.Wait() // no transactions stored inflight txs list @@ -731,7 +731,7 @@ func TestTxm(t *testing.T) { // send tx - with disabled fee bumping and disabled compute unit limit testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()}, SetFeeBumpPeriod(0), SetComputeUnitLimit(0))) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, SetFeeBumpPeriod(0), SetComputeUnitLimit(0))) wg.Wait() // no transactions stored inflight txs list @@ -846,7 +846,7 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()})) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) wg.Wait() // wait to be picked up and processed waitFor(t, 5*time.Second, txm, prom, empty) // inflight txs cleared after timeout @@ -1057,7 +1057,7 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { // send tx testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: testTxID, AccountID: t.Name()})) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) wg.Wait() // no transactions stored inflight txs list @@ -1086,7 +1086,7 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { mc.On("SimulateTx", mock.Anything, mock.Anything, mock.Anything).Return(nil, errors.New("simulation failed")).Once() // tx should NOT be able to queue - assert.Error(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, AccountID: t.Name()})) + assert.Error(t, txm.Enqueue(ctx, t.Name(), tx, nil)) }) t.Run("simulation_returns_error", func(t *testing.T) { @@ -1102,7 +1102,7 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { txID := uuid.NewString() // tx should NOT be able to queue - assert.Error(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, AccountID: t.Name()})) + assert.Error(t, txm.Enqueue(ctx, t.Name(), tx, nil)) }) } @@ -1165,7 +1165,7 @@ func TestTxm_Enqueue(t *testing.T) { loader := utils.NewLazyLoad(func() (client.ReaderWriter, error) { return mc, nil }) txm := NewTxm("enqueue_test", loader, nil, cfg, mkey, lggr) - require.ErrorContains(t, txm.Enqueue(ctx, &PendingTx{AccountID: "txmUnstarted"}), "not started") + require.ErrorContains(t, txm.Enqueue(ctx, "txmUnstarted", &solana.Transaction{}, nil), "not started") require.NoError(t, txm.Start(ctx)) t.Cleanup(func() { require.NoError(t, txm.Close()) }) @@ -1176,16 +1176,17 @@ func TestTxm_Enqueue(t *testing.T) { }{ {"success", tx, false}, {"invalid_key", invalidTx, true}, + {"nil_pointer", nil, true}, {"empty_tx", &solana.Transaction{}, true}, } for _, run := range txs { t.Run(run.name, func(t *testing.T) { if !run.fail { - assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *run.tx, AccountID: run.name})) + assert.NoError(t, txm.Enqueue(ctx, run.name, run.tx, nil)) return } - assert.Error(t, txm.Enqueue(ctx, &PendingTx{Tx: *run.tx, AccountID: run.name})) + assert.Error(t, txm.Enqueue(ctx, run.name, run.tx, nil)) }) } } @@ -1301,7 +1302,7 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { // Enqueue the transaction tx, _ := getTx(t, 0, mkey) expiredTxID := "test" - assert.NoError(t, txm.Enqueue(ctx, &PendingTx{Tx: *tx, UUID: expiredTxID, AccountID: t.Name()}, SetTimeout(10*time.Second))) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &expiredTxID), SetTimeout(10*time.Second)) wg.Wait() // Wait for the transaction to be finalized // Check that transaction for expiredTxID is not stored in memory diff --git a/pkg/solana/txm/txm_load_test.go b/pkg/solana/txm/txm_load_test.go index d371c9311..885fa7383 100644 --- a/pkg/solana/txm/txm_load_test.go +++ b/pkg/solana/txm/txm_load_test.go @@ -16,6 +16,7 @@ import ( solanaClient "github.com/smartcontractkit/chainlink-solana/pkg/solana/client" "github.com/smartcontractkit/chainlink-solana/pkg/solana/config" + "github.com/smartcontractkit/chainlink-solana/pkg/solana/txm" keyMocks "github.com/smartcontractkit/chainlink-solana/pkg/solana/txm/mocks" relayconfig "github.com/smartcontractkit/chainlink-common/pkg/config" @@ -71,7 +72,7 @@ func TestTxm_Integration(t *testing.T) { client, err := solanaClient.NewClient(url, cfg, 2*time.Second, lggr) require.NoError(t, err) loader := utils.NewLazyLoad(func() (solanaClient.ReaderWriter, error) { return client, nil }) - txm := NewTxm("localnet", loader, nil, cfg, mkey, lggr) + txm := txm.NewTxm("localnet", loader, nil, cfg, mkey, lggr) // track initial balance initBal, err := client.Balance(ctx, pubKey) @@ -83,8 +84,9 @@ func TestTxm_Integration(t *testing.T) { // already started assert.Error(t, txm.Start(ctx)) - createMsgWithTx := func(accountID string, signer solana.PublicKey, sender solana.PublicKey, receiver solana.PublicKey, amt uint64) *PendingTx { + createTx := func(signer solana.PublicKey, sender solana.PublicKey, receiver solana.PublicKey, amt uint64) *solana.Transaction { // create transfer tx + hash, err := client.LatestBlockhash(ctx) assert.NoError(t, err) tx, txErr := solana.NewTransaction( []solana.Instruction{ @@ -94,24 +96,24 @@ func TestTxm_Integration(t *testing.T) { receiver, ).Build(), }, - solana.Hash{}, + hash.Value.Blockhash, solana.TransactionPayer(signer), ) require.NoError(t, txErr) - return &PendingTx{Tx: *tx, AccountID: accountID} + return tx } // enqueue txs (must pass to move on to load test) - require.NoError(t, txm.Enqueue(ctx, createMsgWithTx("test_success_0", pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL))) - require.Error(t, txm.Enqueue(ctx, createMsgWithTx("test_invalidSigner", pubKeyReceiver, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL))) // cannot sign tx before enqueuing - require.NoError(t, txm.Enqueue(ctx, createMsgWithTx("test_invalidReceiver", pubKey, pubKey, solana.PublicKey{}, solana.LAMPORTS_PER_SOL))) + require.NoError(t, txm.Enqueue(ctx, "test_success_0", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) + require.Error(t, txm.Enqueue(ctx, "test_invalidSigner", createTx(pubKeyReceiver, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) // cannot sign tx before enqueuing + require.NoError(t, txm.Enqueue(ctx, "test_invalidReceiver", createTx(pubKey, pubKey, solana.PublicKey{}, solana.LAMPORTS_PER_SOL), nil)) time.Sleep(500 * time.Millisecond) // pause 0.5s for new blockhash - require.NoError(t, txm.Enqueue(ctx, createMsgWithTx("test_success_1", pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL))) - require.NoError(t, txm.Enqueue(ctx, createMsgWithTx("test_txFail", pubKey, pubKey, pubKeyReceiver, 1000*solana.LAMPORTS_PER_SOL))) + require.NoError(t, txm.Enqueue(ctx, "test_success_1", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) + require.NoError(t, txm.Enqueue(ctx, "test_txFail", createTx(pubKey, pubKey, pubKeyReceiver, 1000*solana.LAMPORTS_PER_SOL), nil)) // load test: try to overload txs, confirm, or simulation for i := 0; i < 1000; i++ { - assert.NoError(t, txm.Enqueue(ctx, createMsgWithTx(fmt.Sprintf("load_%d", i), loadTestKey.PublicKey(), loadTestKey.PublicKey(), loadTestKey.PublicKey(), uint64(i)))) + assert.NoError(t, txm.Enqueue(ctx, fmt.Sprintf("load_%d", i), createTx(loadTestKey.PublicKey(), loadTestKey.PublicKey(), loadTestKey.PublicKey(), uint64(i)), nil)) time.Sleep(10 * time.Millisecond) // ~100 txs per second (note: have run 5ms delays for ~200tx/s successfully) } diff --git a/pkg/solana/txm/txm_race_test.go b/pkg/solana/txm/txm_race_test.go index c662649ba..f008023d0 100644 --- a/pkg/solana/txm/txm_race_test.go +++ b/pkg/solana/txm/txm_race_test.go @@ -28,10 +28,10 @@ import ( "github.com/stretchr/testify/require" ) -func NewTestMsg() (msg PendingTx) { +func NewTestMsg() (msg pendingTx) { tx := solanaGo.Transaction{} tx.Message.AccountKeys = append(tx.Message.AccountKeys, solanaGo.PublicKey{}) - msg.Tx = tx + msg.tx = tx return msg } @@ -222,33 +222,33 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { client := clientmocks.NewReaderWriter(t) // client mock - first tx is always successful msg0 := NewTestMsg() - require.NoError(t, fees.SetComputeUnitPrice(&msg0.Tx, 0)) - require.NoError(t, fees.SetComputeUnitLimit(&msg0.Tx, 200_000)) - msg0.Tx.Signatures = make([]solanaGo.Signature, 1) - client.On("SendTx", mock.Anything, &msg0.Tx).Return(solanaGo.Signature{1}, nil) + require.NoError(t, fees.SetComputeUnitPrice(&msg0.tx, 0)) + require.NoError(t, fees.SetComputeUnitLimit(&msg0.tx, 200_000)) + msg0.tx.Signatures = make([]solanaGo.Signature, 1) + client.On("SendTx", mock.Anything, &msg0.tx).Return(solanaGo.Signature{1}, nil) // init bump tx fails, rebroadcast is successful msg1 := NewTestMsg() - require.NoError(t, fees.SetComputeUnitPrice(&msg1.Tx, 1)) - require.NoError(t, fees.SetComputeUnitLimit(&msg1.Tx, 200_000)) - msg1.Tx.Signatures = make([]solanaGo.Signature, 1) - client.On("SendTx", mock.Anything, &msg1.Tx).Return(solanaGo.Signature{}, fmt.Errorf("BUMP FAILED")).Once() - client.On("SendTx", mock.Anything, &msg1.Tx).Return(solanaGo.Signature{2}, nil) + require.NoError(t, fees.SetComputeUnitPrice(&msg1.tx, 1)) + require.NoError(t, fees.SetComputeUnitLimit(&msg1.tx, 200_000)) + msg1.tx.Signatures = make([]solanaGo.Signature, 1) + client.On("SendTx", mock.Anything, &msg1.tx).Return(solanaGo.Signature{}, fmt.Errorf("BUMP FAILED")).Once() + client.On("SendTx", mock.Anything, &msg1.tx).Return(solanaGo.Signature{2}, nil) // init bump tx success, rebroadcast fails msg2 := NewTestMsg() - require.NoError(t, fees.SetComputeUnitPrice(&msg2.Tx, 2)) - require.NoError(t, fees.SetComputeUnitLimit(&msg2.Tx, 200_000)) - msg2.Tx.Signatures = make([]solanaGo.Signature, 1) - client.On("SendTx", mock.Anything, &msg2.Tx).Return(solanaGo.Signature{3}, nil).Once() - client.On("SendTx", mock.Anything, &msg2.Tx).Return(solanaGo.Signature{}, fmt.Errorf("REBROADCAST FAILED")) + require.NoError(t, fees.SetComputeUnitPrice(&msg2.tx, 2)) + require.NoError(t, fees.SetComputeUnitLimit(&msg2.tx, 200_000)) + msg2.tx.Signatures = make([]solanaGo.Signature, 1) + client.On("SendTx", mock.Anything, &msg2.tx).Return(solanaGo.Signature{3}, nil).Once() + client.On("SendTx", mock.Anything, &msg2.tx).Return(solanaGo.Signature{}, fmt.Errorf("REBROADCAST FAILED")) // always successful msg3 := NewTestMsg() - require.NoError(t, fees.SetComputeUnitPrice(&msg3.Tx, 4)) - require.NoError(t, fees.SetComputeUnitLimit(&msg3.Tx, 200_000)) - msg3.Tx.Signatures = make([]solanaGo.Signature, 1) - client.On("SendTx", mock.Anything, &msg3.Tx).Return(solanaGo.Signature{4}, nil) + require.NoError(t, fees.SetComputeUnitPrice(&msg3.tx, 4)) + require.NoError(t, fees.SetComputeUnitLimit(&msg3.tx, 200_000)) + msg3.tx.Signatures = make([]solanaGo.Signature, 1) + client.On("SendTx", mock.Anything, &msg3.tx).Return(solanaGo.Signature{4}, nil) client.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ LastValidBlockHeight: uint64(2000), From 92a280b431a10f508cfb4bb7ea8fbfdd60dd85ce Mon Sep 17 00:00:00 2001 From: Farber98 Date: Wed, 20 Nov 2024 11:18:17 -0300 Subject: [PATCH 18/51] fix order of confirm loop and not found signature check --- pkg/solana/txm/txm.go | 15 +++------------ pkg/solana/txm/txm_load_test.go | 3 +-- 2 files changed, 4 insertions(+), 14 deletions(-) diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index a38aba116..0084b930a 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -191,10 +191,7 @@ func (txm *Txm) sendWithRetry(ctx context.Context, msg pendingTx) (solanaGo.Tran return solanaGo.Transaction{}, "", solanaGo.Signature{}, err } - // Create timeout context ctx, cancel := context.WithTimeout(ctx, msg.cfg.Timeout) - - // Send initial transaction sig, err := txm.sendInitialTx(ctx, initTx, msg, cancel) if err != nil { return solanaGo.Transaction{}, "", solanaGo.Signature{}, err @@ -207,7 +204,6 @@ func (txm *Txm) sendWithRetry(ctx context.Context, msg pendingTx) (solanaGo.Tran return solanaGo.Transaction{}, "", solanaGo.Signature{}, fmt.Errorf("failed to save initial signature in signature list: %w", initSetErr) } - // Start retry routine // pass in copy of msg (to build new tx with bumped fee) and broadcasted tx == initTx (to retry tx without bumping) txm.done.Add(1) go func() { @@ -226,8 +222,6 @@ func (txm *Txm) prepareTransaction(ctx context.Context, msg *pendingTx) error { if err != nil { return fmt.Errorf("failed to get client in sendWithRetry: %w", err) } - - // Assign blockhash blockhash, err := client.LatestBlockhash(ctx) if err != nil { return fmt.Errorf("failed to get blockhash: %w", err) @@ -235,7 +229,6 @@ func (txm *Txm) prepareTransaction(ctx context.Context, msg *pendingTx) error { msg.tx.Message.RecentBlockhash = blockhash.Value.Blockhash msg.lastValidBlockHeight = blockhash.Value.LastValidBlockHeight - // Set compute unit limit if msg.cfg.ComputeUnitLimit != 0 { if err := fees.SetComputeUnitLimit(&msg.tx, fees.ComputeUnitLimit(msg.cfg.ComputeUnitLimit)); err != nil { return fmt.Errorf("failed to add compute unit limit instruction: %w", err) @@ -439,11 +432,10 @@ func (txm *Txm) confirm() { txm.lggr.Errorw("failed to get client in txm.confirm", "error", err) break } - + txm.processConfirmations(ctx, client, sigs) if txm.cfg.TxExpirationRebroadcast() { txm.rebroadcastExpiredTxs(ctx, client) } - txm.processConfirmations(ctx, client, sigs) } tick = time.After(utils.WithJitter(txm.cfg.ConfirmPollPeriod())) } @@ -504,7 +496,7 @@ func (txm *Txm) rebroadcastExpiredTxs(ctx context.Context, client client.ReaderW rebroadcastCount: tx.rebroadcastCount + 1, } - // call sendWithRetry directly + // call sendWithRetry directly to avoid enqueuing _, _, _, err = txm.sendWithRetry(ctx, rebroadcastTx) if err != nil { txm.lggr.Errorw("failed to rebroadcast transaction", "id", tx.id, "error", err) @@ -560,9 +552,8 @@ func (txm *Txm) processSignatureStatuses(sigs []solanaGo.Signature, res []*rpc.S func (txm *Txm) handleNotFoundSignatureStatus(sig solanaGo.Signature) { txm.lggr.Debugw("tx state: not found", "signature", sig) - // check confirm timeout exceeded - if txm.txs.Expired(sig, txm.cfg.TxConfirmTimeout()) { + if txm.cfg.TxConfirmTimeout() != 0*time.Second && txm.txs.Expired(sig, txm.cfg.TxConfirmTimeout()) { id, err := txm.txs.OnError(sig, txm.cfg.TxRetentionTimeout(), TxFailDrop) if err != nil { txm.lggr.Infow("failed to mark transaction as errored", "id", id, "signature", sig, "timeoutSeconds", txm.cfg.TxConfirmTimeout(), "error", err) diff --git a/pkg/solana/txm/txm_load_test.go b/pkg/solana/txm/txm_load_test.go index 885fa7383..19d02e526 100644 --- a/pkg/solana/txm/txm_load_test.go +++ b/pkg/solana/txm/txm_load_test.go @@ -16,7 +16,6 @@ import ( solanaClient "github.com/smartcontractkit/chainlink-solana/pkg/solana/client" "github.com/smartcontractkit/chainlink-solana/pkg/solana/config" - "github.com/smartcontractkit/chainlink-solana/pkg/solana/txm" keyMocks "github.com/smartcontractkit/chainlink-solana/pkg/solana/txm/mocks" relayconfig "github.com/smartcontractkit/chainlink-common/pkg/config" @@ -72,7 +71,7 @@ func TestTxm_Integration(t *testing.T) { client, err := solanaClient.NewClient(url, cfg, 2*time.Second, lggr) require.NoError(t, err) loader := utils.NewLazyLoad(func() (solanaClient.ReaderWriter, error) { return client, nil }) - txm := txm.NewTxm("localnet", loader, nil, cfg, mkey, lggr) + txm := NewTxm("localnet", loader, nil, cfg, mkey, lggr) // track initial balance initBal, err := client.Balance(ctx, pubKey) From 2598e1982105cf0798c359a07378ffe4864385c6 Mon Sep 17 00:00:00 2001 From: Farber98 Date: Wed, 20 Nov 2024 11:19:52 -0300 Subject: [PATCH 19/51] fix mocks --- pkg/solana/config/mocks/config.go | 45 +++++++++++++++++++++++++++++++ 1 file changed, 45 insertions(+) diff --git a/pkg/solana/config/mocks/config.go b/pkg/solana/config/mocks/config.go index c397a1307..e4a17dd5e 100644 --- a/pkg/solana/config/mocks/config.go +++ b/pkg/solana/config/mocks/config.go @@ -780,6 +780,51 @@ func (_m *Config) TxExpirationRebroadcast() bool { return r0 } +// TxExpirationRebroadcast provides a mock function with given fields: +func (_m *Config) TxExpirationRebroadcast() bool { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for TxExpirationRebroadcast") + } + + var r0 bool + if rf, ok := ret.Get(0).(func() bool); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(bool) + } + + return r0 +} + +// Config_TxExpirationRebroadcast_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'TxExpirationRebroadcast' +type Config_TxExpirationRebroadcast_Call struct { + *mock.Call +} + +// TxExpirationRebroadcast is a helper method to define mock.On call +func (_e *Config_Expecter) TxExpirationRebroadcast() *Config_TxExpirationRebroadcast_Call { + return &Config_TxExpirationRebroadcast_Call{Call: _e.mock.On("TxExpirationRebroadcast")} +} + +func (_c *Config_TxExpirationRebroadcast_Call) Run(run func()) *Config_TxExpirationRebroadcast_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *Config_TxExpirationRebroadcast_Call) Return(_a0 bool) *Config_TxExpirationRebroadcast_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *Config_TxExpirationRebroadcast_Call) RunAndReturn(run func() bool) *Config_TxExpirationRebroadcast_Call { + _c.Call.Return(run) + return _c +} + // TxRetentionTimeout provides a mock function with given fields: func (_m *Config) TxRetentionTimeout() time.Duration { ret := _m.Called() From 42b3da17b94d8df25aa29e4c49a9a058ccb1beac Mon Sep 17 00:00:00 2001 From: Farber98 Date: Wed, 20 Nov 2024 11:31:59 -0300 Subject: [PATCH 20/51] prevent confirmation loop to mark tx as errored when it needs to be rebroadcasted --- pkg/solana/txm/txm.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index 0084b930a..d330a6c67 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -564,6 +564,12 @@ func (txm *Txm) handleNotFoundSignatureStatus(sig solanaGo.Signature) { } func (txm *Txm) handleErrorSignatureStatus(sig solanaGo.Signature, status *rpc.SignatureStatusesResult) { + // if error is BlockhashNotFound and txExpirationRebroadcast is enabled, skip error handling. + // We want to rebroadcast rather than drop tx if blockhash not found and txExpirationRebroadcast is enabled. + if status.Err != nil && status.Err == client.ErrBlockhashNotFound && txm.cfg.TxExpirationRebroadcast() { + return + } + id, err := txm.txs.OnError(sig, txm.cfg.TxRetentionTimeout(), TxFailRevert) if err != nil { txm.lggr.Infow("failed to mark transaction as errored", "id", id, "signature", sig, "error", err) From 89af1f3bb8e1790a34d5c547d599b970c1c2ced4 Mon Sep 17 00:00:00 2001 From: Farber98 Date: Wed, 20 Nov 2024 14:58:40 -0300 Subject: [PATCH 21/51] fix test --- pkg/solana/txm/pendingtx.go | 30 +++- pkg/solana/txm/txm.go | 206 +++++++++++++--------------- pkg/solana/txm/txm_internal_test.go | 26 ++-- 3 files changed, 137 insertions(+), 125 deletions(-) diff --git a/pkg/solana/txm/pendingtx.go b/pkg/solana/txm/pendingtx.go index 6ec448916..074ab26b0 100644 --- a/pkg/solana/txm/pendingtx.go +++ b/pkg/solana/txm/pendingtx.go @@ -28,7 +28,7 @@ type PendingTxContext interface { Remove(sig solana.Signature) (string, error) // ListAll returns all of the signatures being tracked for all transactions not yet finalized or errored ListAll() []solana.Signature - // ListAllExpiredBroadcastedTxs returns all the expired broadcasted that are in broadcasted state and have expired for given slot height. + // ListAllExpiredBroadcastedTxs returns all the txes that are in broadcasted state and have expired for given slot height compared against their lastValidBlockHeight. ListAllExpiredBroadcastedTxs(currHeight uint64) []pendingTx // Expired returns whether or not confirmation timeout amount of time has passed since creation Expired(sig solana.Signature, confirmationTimeout time.Duration) bool @@ -45,7 +45,9 @@ type PendingTxContext interface { // GetTxState returns the transaction state for the provided ID if it exists GetTxState(id string) (TxState, error) // TrimFinalizedErroredTxs removes transactions that have reached their retention time - TrimFinalizedErroredTxs() int + TrimFinalizedErroredTxs() + // GetTxRebroadcastCount returns the number of times a transaction has been rebroadcasted if found. + GetTxRebroadcastCount(id string) (int, error) } type pendingTx struct { @@ -561,6 +563,22 @@ func (c *pendingTxContext) withWriteLock(fn func() (string, error)) (string, err return fn() } +// GetTxRebroadcastCount returns the number of times a transaction has been rebroadcasted if found. +func (c pendingTxContext) GetTxRebroadcastCount(id string) (int, error) { + c.lock.RLock() + defer c.lock.RUnlock() + if tx, exists := c.broadcastedProcessedTxs[id]; exists { + return tx.rebroadcastCount, nil + } + if tx, exists := c.confirmedTxs[id]; exists { + return tx.rebroadcastCount, nil + } + if tx, exists := c.finalizedErroredTxs[id]; exists { + return tx.rebroadcastCount, nil + } + return 0, fmt.Errorf("failed to find transaction for id: %s", id) +} + var _ PendingTxContext = &pendingTxContextWithProm{} type pendingTxContextWithProm struct { @@ -672,6 +690,10 @@ func (c *pendingTxContextWithProm) GetTxState(id string) (TxState, error) { return c.pendingTx.GetTxState(id) } -func (c *pendingTxContextWithProm) TrimFinalizedErroredTxs() int { - return c.pendingTx.TrimFinalizedErroredTxs() +func (c *pendingTxContextWithProm) TrimFinalizedErroredTxs() { + c.pendingTx.TrimFinalizedErroredTxs() +} + +func (c *pendingTxContextWithProm) GetTxRebroadcastCount(id string) (int, error) { + return c.pendingTx.GetTxRebroadcastCount(id) } diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index d330a6c67..9a4557172 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -311,8 +311,8 @@ func (txm *Txm) retryTx(ctx context.Context, msg pendingTx, currentTx solanaGo.T txm.lggr.Debugw("stopped tx retry", "id", msg.id, "signatures", sigs.List(), "err", context.Cause(ctx)) return case <-tick: - // Determine if we should bump the fee - shouldBump := txm.shouldBumpFee(msg.cfg.FeeBumpPeriod, bumpTime) + // determines whether the fee should be bumped based on the fee bump period. + shouldBump := msg.cfg.FeeBumpPeriod != 0 && time.Since(bumpTime) > msg.cfg.FeeBumpPeriod if shouldBump { bumpCount++ bumpTime = time.Now() @@ -341,26 +341,15 @@ func (txm *Txm) retryTx(ctx context.Context, msg pendingTx, currentTx solanaGo.T }(shouldBump, bumpCount, currentTx) } - // Update the exponential backoff delay - deltaT = txm.updateBackoffDelay(deltaT) + // updates the exponential backoff delay up to a maximum limit. + deltaT = deltaT * 2 + if deltaT > MaxRetryTimeMs { + deltaT = MaxRetryTimeMs + } tick = time.After(time.Duration(deltaT) * time.Millisecond) } } -// shouldBumpFee determines whether the fee should be bumped based on the fee bump period. -func (txm *Txm) shouldBumpFee(feeBumpPeriod time.Duration, lastBumpTime time.Time) bool { - return feeBumpPeriod != 0 && time.Since(lastBumpTime) > feeBumpPeriod -} - -// updateBackoffDelay updates the exponential backoff delay up to a maximum limit. -func (txm *Txm) updateBackoffDelay(currentDelay int) int { - newDelay := currentDelay * 2 - if newDelay > MaxRetryTimeMs { - return MaxRetryTimeMs - } - return newDelay -} - // handleRetry handles the logic for each retry attempt, including sending the transaction, updating signatures, and logging. func (txm *Txm) handleRetry(ctx context.Context, msg pendingTx, bump bool, count int, retryTx solanaGo.Transaction, sigs *signatureList) { // send retry transaction @@ -407,7 +396,7 @@ func (txm *Txm) handleRetry(ctx context.Context, msg pendingTx, bump bool, count } } -// goroutine that polls to confirm implementation +// confirm is a goroutine that polls to confirm implementation // cancels the exponential retry once confirmed func (txm *Txm) confirm() { defer txm.done.Done() @@ -420,8 +409,7 @@ func (txm *Txm) confirm() { case <-ctx.Done(): return case <-tick: - // Get list of transaction signatures to confirm - // If no signatures to confirm, we can break loop. + // If no signatures to confirm, we can break loop as there's nothing to process. sigs := txm.txs.ListAll() if len(sigs) == 0 { break @@ -441,8 +429,11 @@ func (txm *Txm) confirm() { } } +// processConfirmations checks the status of transaction signatures on-chain and updates our in-memory state accordingly. +// It splits the signatures into batches, retrieves their statuses with an RPC call, and processes each status accordingly. +// The function handles transitions, managing expiration, errors, and transitions between different states like broadcasted, processed, confirmed, and finalized. +// It also determines when to end polling based on the status of each signature cancelling the exponential retry. func (txm *Txm) processConfirmations(ctx context.Context, client client.ReaderWriter, sigs []solanaGo.Signature) { - // batch sigs no more than MaxSigsToConfirm each sigsBatch, err := utils.BatchSplit(sigs, MaxSigsToConfirm) if err != nil { // this should never happen txm.lggr.Fatalw("failed to batch signatures", "error", err) @@ -451,105 +442,63 @@ func (txm *Txm) processConfirmations(ctx context.Context, client client.ReaderWr var wg sync.WaitGroup for i := 0; i < len(sigsBatch); i++ { - // fetch signature statuses statuses, err := client.SignatureStatuses(ctx, sigsBatch[i]) if err != nil { txm.lggr.Errorw("failed to get signature statuses in txm.confirm", "error", err) - break // exit for loop + break } wg.Add(1) // nonblocking: process batches as soon as they come in go func(index int) { defer wg.Done() - txm.processSignatureStatuses(sigsBatch[i], statuses) - }(i) - } - wg.Wait() // wait for processing to finish -} -func (txm *Txm) rebroadcastExpiredTxs(ctx context.Context, client client.ReaderWriter) { - // Get current slot height to check if txes have expired when compared against their lastValidBlockHeight - currHeight, err := client.SlotHeight(ctx) - if err != nil { - txm.lggr.Errorw("failed to get current slot height", "error", err) - return - } - - // Rebroadcast all expired txes - for _, tx := range txm.txs.ListAllExpiredBroadcastedTxs(currHeight) { - txm.lggr.Infow("transaction expired, rebroadcasting", "id", tx.id, "signature", tx.signatures) - if len(tx.signatures) == 0 { // prevent panic, shouldn't happen. - txm.lggr.Errorw("no signatures found for expired transaction", "id", tx.id) - continue - } - _, err := txm.txs.Remove(tx.signatures[0]) // only picking signature[0] because remove func removes all remaining signatures. - if err != nil { - txm.lggr.Errorw("failed to remove expired transaction", "id", tx.id, "error", err) - continue - } - - rebroadcastTx := pendingTx{ - tx: tx.tx, - // new id with the same family as the original tx by appending #rebroadcastCount+1. - id: fmt.Sprintf("%s#%d", tx.id, tx.rebroadcastCount+1), - rebroadcastCount: tx.rebroadcastCount + 1, - } - - // call sendWithRetry directly to avoid enqueuing - _, _, _, err = txm.sendWithRetry(ctx, rebroadcastTx) - if err != nil { - txm.lggr.Errorw("failed to rebroadcast transaction", "id", tx.id, "error", err) - continue - } - - txm.lggr.Infow("rebroadcast transaction sent", "id", tx.id) - } -} - -func (txm *Txm) processSignatureStatuses(sigs []solanaGo.Signature, res []*rpc.SignatureStatusesResult) { - // Sort signatures and results process successful first - sortedSigs, sortedRes, err := SortSignaturesAndResults(sigs, res) - if err != nil { - txm.lggr.Errorw("sorting error", "error", err) - return - } + // to process successful first + sortedSigs, sortedRes, err := SortSignaturesAndResults(sigsBatch[i], statuses) + if err != nil { + txm.lggr.Errorw("sorting error", "error", err) + return + } - for i := 0; i < len(sortedRes); i++ { - sig, status := sortedSigs[i], sortedRes[i] - // if status is nil (sig not found), continue polling - // sig not found could mean invalid tx or not picked up yet - if status == nil { - txm.handleNotFoundSignatureStatus(sig) - continue - } + for j := 0; j < len(sortedRes); j++ { + sig, status := sortedSigs[j], sortedRes[j] + // sig not found could mean invalid tx or not picked up yet, keep polling + if status == nil { + txm.handleNotFoundSignatureStatus(sig) + continue + } - // if signature has an error, end polling - if status.Err != nil { - txm.handleErrorSignatureStatus(sig, status) - continue - } + // if signature has an error, end polling unless blockhash not found and expiration rebroadcast is enabled + if status.Err != nil { + txm.handleErrorSignatureStatus(sig, status) + continue + } - switch status.ConfirmationStatus { - case rpc.ConfirmationStatusProcessed: - // if signature is processed, keep polling for confirmed or finalized status - txm.handleProcessedSignatureStatus(sig) - continue - case rpc.ConfirmationStatusConfirmed: - // if signature is confirmed, keep polling for finalized status - txm.handleConfirmedSignatureStatus(sig) - continue - case rpc.ConfirmationStatusFinalized: - // if signature is finalized, end polling - txm.handleFinalizedSignatureStatus(sig) - continue - default: - txm.lggr.Warnw("unknown confirmation status", "signature", sig, "status", status.ConfirmationStatus) - continue - } + switch status.ConfirmationStatus { + case rpc.ConfirmationStatusProcessed: + // if signature is processed, keep polling for confirmed or finalized status + txm.handleProcessedSignatureStatus(sig) + continue + case rpc.ConfirmationStatusConfirmed: + // if signature is confirmed, keep polling for finalized status + txm.handleConfirmedSignatureStatus(sig) + continue + case rpc.ConfirmationStatusFinalized: + // if signature is finalized, end polling + txm.handleFinalizedSignatureStatus(sig) + continue + default: + txm.lggr.Warnw("unknown confirmation status", "signature", sig, "status", status.ConfirmationStatus) + continue + } + } + }(i) } + wg.Wait() // wait for processing to finish } +// handleNotFoundSignatureStatus handles the case where a transaction signature is not found on-chain. +// If the confirmation timeout has been exceeded it marks the transaction as errored. func (txm *Txm) handleNotFoundSignatureStatus(sig solanaGo.Signature) { txm.lggr.Debugw("tx state: not found", "signature", sig) // check confirm timeout exceeded @@ -563,9 +512,11 @@ func (txm *Txm) handleNotFoundSignatureStatus(sig solanaGo.Signature) { } } +// handleErrorSignatureStatus handles the case where a transaction signature has an error on-chain. +// If the error is BlockhashNotFound and expiration rebroadcast is enabled, it skips error handling to allow rebroadcasting. +// Otherwise, it marks the transaction as errored. func (txm *Txm) handleErrorSignatureStatus(sig solanaGo.Signature, status *rpc.SignatureStatusesResult) { - // if error is BlockhashNotFound and txExpirationRebroadcast is enabled, skip error handling. - // We want to rebroadcast rather than drop tx if blockhash not found and txExpirationRebroadcast is enabled. + // We want to rebroadcast rather than drop tx if expiration rebroadcast is enabled when blockhash was not found. if status.Err != nil && status.Err == client.ErrBlockhashNotFound && txm.cfg.TxExpirationRebroadcast() { return } @@ -578,6 +529,9 @@ func (txm *Txm) handleErrorSignatureStatus(sig solanaGo.Signature, status *rpc.S } } +// handleProcessedSignatureStatus handles the case where a transaction signature is in the "processed" state on-chain. +// It updates the transaction state in the local memory and checks if the confirmation timeout has been exceeded. +// If the timeout is exceeded, it marks the transaction as errored. func (txm *Txm) handleProcessedSignatureStatus(sig solanaGo.Signature) { // update transaction state in local memory id, err := txm.txs.OnProcessed(sig) @@ -597,6 +551,8 @@ func (txm *Txm) handleProcessedSignatureStatus(sig solanaGo.Signature) { } } +// handleConfirmedSignatureStatus handles the case where a transaction signature is in the "confirmed" state on-chain. +// It updates the transaction state in the local memory. func (txm *Txm) handleConfirmedSignatureStatus(sig solanaGo.Signature) { id, err := txm.txs.OnConfirmed(sig) if err != nil && !errors.Is(err, ErrAlreadyInExpectedState) { @@ -606,6 +562,8 @@ func (txm *Txm) handleConfirmedSignatureStatus(sig solanaGo.Signature) { } } +// handleFinalizedSignatureStatus handles the case where a transaction signature is in the "finalized" state on-chain. +// It updates the transaction state in the local memory. func (txm *Txm) handleFinalizedSignatureStatus(sig solanaGo.Signature) { id, err := txm.txs.OnFinalized(sig, txm.cfg.TxRetentionTimeout()) if err != nil { @@ -615,6 +573,40 @@ func (txm *Txm) handleFinalizedSignatureStatus(sig solanaGo.Signature) { } } +// rebroadcastExpiredTxs attempts to rebroadcast all transactions that are in broadcasted state and have expired. +// An expired tx is one where it's blockhash lastValidBlockHeight is smaller than the current slot height. +// If any error occurs during rebroadcast attempt, they are discarded, and the function continues with the next transaction. +func (txm *Txm) rebroadcastExpiredTxs(ctx context.Context, client client.ReaderWriter) { + currHeight, err := client.SlotHeight(ctx) + if err != nil { + txm.lggr.Errorw("failed to get current slot height", "error", err) + return + } + + // Rebroadcast all expired txes + for _, tx := range txm.txs.ListAllExpiredBroadcastedTxs(currHeight) { + txm.lggr.Infow("transaction expired, rebroadcasting", "id", tx.id, "signature", tx.signatures) + if len(tx.signatures) == 0 { // prevent panic, shouldn't happen. + txm.lggr.Errorw("no signatures found for expired transaction", "id", tx.id) + continue + } + _, err := txm.txs.Remove(tx.signatures[0]) // only picking signature[0]. Remove func removes all related remaining signatures. + if err != nil { + txm.lggr.Errorw("failed to remove expired transaction", "id", tx.id, "error", err) + continue + } + // call sendWithRetry directly to avoid enqueuing + // using same id in case it was set by caller and we need to maintain it. + _, _, _, err = txm.sendWithRetry(ctx, pendingTx{tx: tx.tx, id: tx.id, rebroadcastCount: tx.rebroadcastCount + 1}) + if err != nil { + txm.lggr.Errorw("failed to rebroadcast transaction", "id", tx.id, "error", err) + continue + } + + txm.lggr.Infow("rebroadcast transaction sent", "id", tx.id) + } +} + // goroutine that simulates tx (use a bounded number of goroutines to pick from queue?) // simulate can cancel the send retry function early in the tx management process // additionally, it can provide reasons for why a tx failed in the logs diff --git a/pkg/solana/txm/txm_internal_test.go b/pkg/solana/txm/txm_internal_test.go index 00e7be8f1..9186b6b4b 100644 --- a/pkg/solana/txm/txm_internal_test.go +++ b/pkg/solana/txm/txm_internal_test.go @@ -1277,10 +1277,11 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { var wg sync.WaitGroup wg.Add(1) statuses[sig1] = func() *rpc.SignatureStatusesResult { - // If time is less than confirm timeout, return nil. This is when tx should be rebroadcasted + // first transaction should be rebroadcasted. if time.Since(nowTs) < cfg.TxConfirmTimeout()-2*time.Second { return nil } else { + // second transaction should reach finalization. sigStatusCallCount++ if sigStatusCallCount == 1 { return &rpc.SignatureStatusesResult{ @@ -1299,20 +1300,17 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { } } - // Enqueue the transaction tx, _ := getTx(t, 0, mkey) - expiredTxID := "test" - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &expiredTxID), SetTimeout(10*time.Second)) - wg.Wait() // Wait for the transaction to be finalized - - // Check that transaction for expiredTxID is not stored in memory - status, err := txm.GetTransactionStatus(ctx, expiredTxID) - require.Error(t, err) - require.Equal(t, types.Unknown, status) - - // Check the transaction status for rebroadcasted txID has been finalized - rebroadcastedTxID := expiredTxID + "#1" - status, err = txm.GetTransactionStatus(ctx, rebroadcastedTxID) + txID := "test" + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID), SetTimeout(10*time.Second)) + wg.Wait() + time.Sleep(2 * time.Second) // Sleep to allow for rebroadcasting + + // Check that transaction for txID has been finalized and rebroadcasted + status, err := txm.GetTransactionStatus(ctx, txID) require.NoError(t, err) require.Equal(t, types.Finalized, status) + rebroadcastCount, err := txm.txs.GetTxRebroadcastCount(txID) + require.NoError(t, err) + require.Equal(t, 1, rebroadcastCount) } From 5e8a0da9b3b2461174412ac1d25210f799d2ebd0 Mon Sep 17 00:00:00 2001 From: Farber98 Date: Wed, 20 Nov 2024 15:19:37 -0300 Subject: [PATCH 22/51] fix pointer --- pkg/solana/txm/pendingtx.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/solana/txm/pendingtx.go b/pkg/solana/txm/pendingtx.go index 074ab26b0..47030056f 100644 --- a/pkg/solana/txm/pendingtx.go +++ b/pkg/solana/txm/pendingtx.go @@ -564,7 +564,7 @@ func (c *pendingTxContext) withWriteLock(fn func() (string, error)) (string, err } // GetTxRebroadcastCount returns the number of times a transaction has been rebroadcasted if found. -func (c pendingTxContext) GetTxRebroadcastCount(id string) (int, error) { +func (c *pendingTxContext) GetTxRebroadcastCount(id string) (int, error) { c.lock.RLock() defer c.lock.RUnlock() if tx, exists := c.broadcastedProcessedTxs[id]; exists { From 75c1dcd4cd6669579ce6814ae781de54dd96b7ca Mon Sep 17 00:00:00 2001 From: Farber98 Date: Thu, 21 Nov 2024 09:27:21 -0300 Subject: [PATCH 23/51] add comments --- pkg/solana/txm/txm.go | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index 9a4557172..f491005a4 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -142,6 +142,10 @@ func (txm *Txm) Start(ctx context.Context) error { }) } +// run is a goroutine that continuously processes transactions from the chSend channel. +// It attempts to send each transaction with retry logic and, upon success, enqueues the transaction for simulation. +// If a transaction fails to send, it logs the error and resets the client to handle potential bad RPCs. +// The function runs until the chStop channel signals to stop. func (txm *Txm) run() { defer txm.done.Done() ctx, cancel := txm.chStop.NewCtx() @@ -396,8 +400,8 @@ func (txm *Txm) handleRetry(ctx context.Context, msg pendingTx, bump bool, count } } -// confirm is a goroutine that polls to confirm implementation -// cancels the exponential retry once confirmed +// confirm is a goroutine that continuously polls for transaction confirmations and handles rebroadcasts expired transactions if enabled. +// The function runs until the chStop channel signals to stop. func (txm *Txm) confirm() { defer txm.done.Done() ctx, cancel := txm.chStop.NewCtx() @@ -599,6 +603,7 @@ func (txm *Txm) rebroadcastExpiredTxs(ctx context.Context, client client.ReaderW // using same id in case it was set by caller and we need to maintain it. _, _, _, err = txm.sendWithRetry(ctx, pendingTx{tx: tx.tx, id: tx.id, rebroadcastCount: tx.rebroadcastCount + 1}) if err != nil { + // TODO: add prebroadcast error handling when merged https://github.com/smartcontractkit/chainlink-solana/pull/936 txm.lggr.Errorw("failed to rebroadcast transaction", "id", tx.id, "error", err) continue } From 4ff2d23080743f77871cc935a9b7062b45ec1ceb Mon Sep 17 00:00:00 2001 From: Farber98 Date: Thu, 21 Nov 2024 14:13:35 -0300 Subject: [PATCH 24/51] reduce rpc calls + refactors --- pkg/solana/txm/txm.go | 98 ++++++++++++----------------- pkg/solana/txm/txm_internal_test.go | 48 ++------------ pkg/solana/txm/txm_race_test.go | 21 ------- 3 files changed, 45 insertions(+), 122 deletions(-) diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index f491005a4..8f8d419c9 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -183,24 +183,30 @@ func (txm *Txm) run() { // It prepares the transaction, builds and signs it, sends the initial transaction, and starts a retry routine with fee bumping if needed. // The function returns the signed transaction, its ID, and the initial signature for use in simulation. func (txm *Txm) sendWithRetry(ctx context.Context, msg pendingTx) (solanaGo.Transaction, string, solanaGo.Signature, error) { - // Prepare transaction assigning blockhash and lastValidBlockHeight (for expiration tracking). - // If required, it also performs balanceCheck and sets compute unit limit. - if err := txm.prepareTransaction(ctx, &msg); err != nil { - return solanaGo.Transaction{}, "", solanaGo.Signature{}, err - } - - // Build and sign initial transaction setting compute unit price + // Build and sign initial transaction setting compute unit price and limit initTx, err := txm.buildTx(ctx, msg, 0) if err != nil { return solanaGo.Transaction{}, "", solanaGo.Signature{}, err } + // Send initial transaction ctx, cancel := context.WithTimeout(ctx, msg.cfg.Timeout) - sig, err := txm.sendInitialTx(ctx, initTx, msg, cancel) + sig, err := txm.sendTx(ctx, &initTx) if err != nil { - return solanaGo.Transaction{}, "", solanaGo.Signature{}, err + // Do not retry and exit early if fails + cancel() + txm.txs.OnError(sig, txm.cfg.TxRetentionTimeout(), TxFailReject) //nolint // no need to check error since only incrementing metric here + return solanaGo.Transaction{}, "", solanaGo.Signature{}, fmt.Errorf("tx failed initial transmit: %w", err) + } + + // Store tx signature and cancel function + if err := txm.txs.New(msg, sig, cancel); err != nil { + cancel() // Cancel context when exiting early + return solanaGo.Transaction{}, "", solanaGo.Signature{}, fmt.Errorf("failed to save tx signature (%s) to inflight txs: %w", sig, err) } + txm.lggr.Debugw("tx initial broadcast", "id", msg.id, "fee", msg.cfg.BaseComputeUnitPrice, "signature", sig) + // Initialize signature list with initialTx signature. This list will be used to add new signatures and track retry attempts. sigs := &signatureList{} sigs.Allocate() @@ -219,31 +225,14 @@ func (txm *Txm) sendWithRetry(ctx context.Context, msg pendingTx) (solanaGo.Tran return initTx, msg.id, sig, nil } -// prepareTransaction sets blockhash and lastValidBlockHeight which will be used to track expiration. -// If required, it also performs balanceCheck and sets compute unit limit. -func (txm *Txm) prepareTransaction(ctx context.Context, msg *pendingTx) error { - client, err := txm.client.Get() - if err != nil { - return fmt.Errorf("failed to get client in sendWithRetry: %w", err) - } - blockhash, err := client.LatestBlockhash(ctx) - if err != nil { - return fmt.Errorf("failed to get blockhash: %w", err) - } - msg.tx.Message.RecentBlockhash = blockhash.Value.Blockhash - msg.lastValidBlockHeight = blockhash.Value.LastValidBlockHeight - +// buildTx builds and signs the transaction with the appropriate compute unit price. +func (txm *Txm) buildTx(ctx context.Context, msg pendingTx, retryCount int) (solanaGo.Transaction, error) { + // Set compute unit limit if specified if msg.cfg.ComputeUnitLimit != 0 { if err := fees.SetComputeUnitLimit(&msg.tx, fees.ComputeUnitLimit(msg.cfg.ComputeUnitLimit)); err != nil { - return fmt.Errorf("failed to add compute unit limit instruction: %w", err) + return solanaGo.Transaction{}, fmt.Errorf("failed to add compute unit limit instruction: %w", err) } } - - return nil -} - -// buildTx builds and signs the transaction with the appropriate compute unit price. -func (txm *Txm) buildTx(ctx context.Context, msg pendingTx, retryCount int) (solanaGo.Transaction, error) { // work with a copy newTx := msg.tx @@ -276,27 +265,6 @@ func (txm *Txm) buildTx(ctx context.Context, msg pendingTx, retryCount int) (sol return newTx, nil } -// sendInitialTx sends the initial tx and handles any errors that may occur. It also stores the transaction signature and cancellation function. -func (txm *Txm) sendInitialTx(ctx context.Context, initTx solanaGo.Transaction, msg pendingTx, cancel context.CancelFunc) (solanaGo.Signature, error) { - // Send initial transaction - sig, err := txm.sendTx(ctx, &initTx) - if err != nil { - // do not retry and exit early if fails - cancel() - txm.txs.OnError(sig, txm.cfg.TxRetentionTimeout(), TxFailReject) //nolint // no need to check error since only incrementing metric here - return solanaGo.Signature{}, fmt.Errorf("tx failed initial transmit: %w", err) - } - - // Store tx signature and cancel function - if err := txm.txs.New(msg, sig, cancel); err != nil { - cancel() // cancel context when exiting early - return solanaGo.Signature{}, fmt.Errorf("failed to save tx signature (%s) to inflight txs: %w", sig, err) - } - - txm.lggr.Debugw("tx initial broadcast", "id", msg.id, "fee", msg.cfg.BaseComputeUnitPrice, "signature", sig) - return sig, nil -} - // retryTx contains the logic for retrying the transaction, including exponential backoff and fee bumping. // Retries until context cancelled by timeout or called externally. // It uses handleRetry helper function to handle each retry attempt. @@ -413,9 +381,8 @@ func (txm *Txm) confirm() { case <-ctx.Done(): return case <-tick: - // If no signatures to confirm, we can break loop as there's nothing to process. - sigs := txm.txs.ListAll() - if len(sigs) == 0 { + // If no signatures to confirm and rebroadcast, we can break loop as there's nothing to process. + if txm.InflightTxs() == 0 { break } @@ -424,7 +391,7 @@ func (txm *Txm) confirm() { txm.lggr.Errorw("failed to get client in txm.confirm", "error", err) break } - txm.processConfirmations(ctx, client, sigs) + txm.processConfirmations(ctx, client) if txm.cfg.TxExpirationRebroadcast() { txm.rebroadcastExpiredTxs(ctx, client) } @@ -437,8 +404,8 @@ func (txm *Txm) confirm() { // It splits the signatures into batches, retrieves their statuses with an RPC call, and processes each status accordingly. // The function handles transitions, managing expiration, errors, and transitions between different states like broadcasted, processed, confirmed, and finalized. // It also determines when to end polling based on the status of each signature cancelling the exponential retry. -func (txm *Txm) processConfirmations(ctx context.Context, client client.ReaderWriter, sigs []solanaGo.Signature) { - sigsBatch, err := utils.BatchSplit(sigs, MaxSigsToConfirm) +func (txm *Txm) processConfirmations(ctx context.Context, client client.ReaderWriter) { + sigsBatch, err := utils.BatchSplit(txm.txs.ListAll(), MaxSigsToConfirm) if err != nil { // this should never happen txm.lggr.Fatalw("failed to batch signatures", "error", err) return @@ -586,7 +553,11 @@ func (txm *Txm) rebroadcastExpiredTxs(ctx context.Context, client client.ReaderW txm.lggr.Errorw("failed to get current slot height", "error", err) return } - + blockhash, err := client.LatestBlockhash(ctx) + if err != nil { + txm.lggr.Errorw("failed to get blockhash", "error", err) + return + } // Rebroadcast all expired txes for _, tx := range txm.txs.ListAllExpiredBroadcastedTxs(currHeight) { txm.lggr.Infow("transaction expired, rebroadcasting", "id", tx.id, "signature", tx.signatures) @@ -599,9 +570,17 @@ func (txm *Txm) rebroadcastExpiredTxs(ctx context.Context, client client.ReaderW txm.lggr.Errorw("failed to remove expired transaction", "id", tx.id, "error", err) continue } + // Overwrite blockhash and lastValidBlockHeight with latest values so that the transaction can be rebroadcasted an accepted. + tx.tx.Message.RecentBlockhash = blockhash.Value.Blockhash + rebroadcastTx := pendingTx{ + tx: tx.tx, + id: tx.id, + rebroadcastCount: tx.rebroadcastCount + 1, + lastValidBlockHeight: blockhash.Value.LastValidBlockHeight, + } // call sendWithRetry directly to avoid enqueuing // using same id in case it was set by caller and we need to maintain it. - _, _, _, err = txm.sendWithRetry(ctx, pendingTx{tx: tx.tx, id: tx.id, rebroadcastCount: tx.rebroadcastCount + 1}) + _, _, _, err = txm.sendWithRetry(ctx, rebroadcastTx) if err != nil { // TODO: add prebroadcast error handling when merged https://github.com/smartcontractkit/chainlink-solana/pull/936 txm.lggr.Errorw("failed to rebroadcast transaction", "id", tx.id, "error", err) @@ -891,6 +870,7 @@ func (txm *Txm) processError(sig solanaGo.Signature, resErr interface{}, simulat return } +// InflightTxs returns the number of signatures being tracked for all transactions not yet finalized or errored func (txm *Txm) InflightTxs() int { return len(txm.txs.ListAll()) } diff --git a/pkg/solana/txm/txm_internal_test.go b/pkg/solana/txm/txm_internal_test.go index 9186b6b4b..4812122dc 100644 --- a/pkg/solana/txm/txm_internal_test.go +++ b/pkg/solana/txm/txm_internal_test.go @@ -126,11 +126,6 @@ func TestTxm(t *testing.T) { cfg := config.NewDefault() cfg.Chain.FeeEstimatorMode = &estimator mc := mocks.NewReaderWriter(t) - mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: uint64(2000), - }, - }, nil) mc.On("GetLatestBlock", mock.Anything).Return(&rpc.GetBlockResult{}, nil).Maybe() mc.On("SlotHeight", mock.Anything).Return(uint64(0), nil).Maybe() @@ -769,11 +764,6 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { // Enable retention timeout to keep transactions after finality cfg.Chain.TxRetentionTimeout = relayconfig.MustNewDuration(5 * time.Second) mc := mocks.NewReaderWriter(t) - mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: uint64(2000), - }, - }, nil) mc.On("GetLatestBlock", mock.Anything).Return(&rpc.GetBlockResult{}, nil).Maybe() mc.On("SlotHeight", mock.Anything).Return(uint64(0), nil).Maybe() @@ -975,11 +965,6 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { // Enable retention timeout to keep transactions after finality or error cfg.Chain.TxRetentionTimeout = relayconfig.MustNewDuration(5 * time.Second) mc := mocks.NewReaderWriter(t) - mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: uint64(2000), - }, - }, nil) mc.On("GetLatestBlock", mock.Anything).Return(&rpc.GetBlockResult{}, nil).Maybe() mc.On("SlotHeight", mock.Anything).Return(uint64(0), nil).Maybe() @@ -1111,11 +1096,6 @@ func TestTxm_Enqueue(t *testing.T) { lggr := logger.Test(t) cfg := config.NewDefault() mc := mocks.NewReaderWriter(t) - mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: uint64(2000), - }, - }, nil) mc.On("SendTx", mock.Anything, mock.Anything).Return(solana.Signature{}, nil).Maybe() mc.On("SimulateTx", mock.Anything, mock.Anything, mock.Anything).Return(&rpc.SimulateTransactionResult{}, nil).Maybe() mc.On("SignatureStatuses", mock.Anything, mock.AnythingOfType("[]solana.Signature")).Return( @@ -1207,37 +1187,24 @@ func addSigAndLimitToTx(t *testing.T, keystore SimpleKeystore, pubkey solana.Pub func TestTxm_ExpirationRebroadcast(t *testing.T) { t.Parallel() - // Set up configurations estimator := "fixed" id := "mocknet-" + estimator + "-" + uuid.NewString() t.Logf("Starting new iteration: %s", id) - ctx := tests.Context(t) lggr := logger.Test(t) cfg := config.NewDefault() cfg.Chain.FeeEstimatorMode = &estimator - - // Enable TxExpirationRebroadcast txExpirationRebroadcast := true - cfg.Chain.TxExpirationRebroadcast = &txExpirationRebroadcast + cfg.Chain.TxExpirationRebroadcast = &txExpirationRebroadcast // enable expiration rebroadcast cfg.Chain.TxConfirmTimeout = relayconfig.MustNewDuration(5 * time.Second) - // Enable retention timeout to keep transactions after finality so we can check. - cfg.Chain.TxRetentionTimeout = relayconfig.MustNewDuration(15 * time.Second) + cfg.Chain.TxRetentionTimeout = relayconfig.MustNewDuration(10 * time.Second) // Enable retention to keep transactions after finality and be able to check. mc := mocks.NewReaderWriter(t) - // Set up LatestBlockhash to return different LastValidBlockHeight values - latestBlockhashCallCount := 0 + // First blockhash is set on sender. Second blockhash (the one returned here) is set on txExpirationRebroadcast before rebroadcasting. + // The first one will be invalid as it's initialized in 0 by default. This call will get a valid one greater than slotHeight and go through. mc.On("LatestBlockhash", mock.Anything).Return(func(_ context.Context) (*rpc.GetLatestBlockhashResult, error) { - latestBlockhashCallCount++ - if latestBlockhashCallCount == 1 { - return &rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: uint64(1000), - }, - }, nil - } return &rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ LastValidBlockHeight: uint64(2000), @@ -1245,7 +1212,7 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { }, nil }).Maybe() - // Set up SlotHeight to return a value greater than the initial LastValidBlockHeight + // Set up SlotHeight to return a value greater than 0 so the initial LastValidBlockHeight is invalid. mc.On("SlotHeight", mock.Anything).Return(uint64(1500), nil).Maybe() mkey := keyMocks.NewSimpleKeystore(t) mkey.On("Sign", mock.Anything, mock.Anything, mock.Anything).Return([]byte{}, nil) @@ -1253,7 +1220,6 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { txm := NewTxm(id, loader, nil, cfg, mkey, lggr) require.NoError(t, txm.Start(ctx)) t.Cleanup(func() { require.NoError(t, txm.Close()) }) - sig1 := randomSignature(t) mc.On("SendTx", mock.Anything, mock.Anything).Return(sig1, nil).Maybe() mc.On("SimulateTx", mock.Anything, mock.Anything, mock.Anything).Return(&rpc.SimulateTransactionResult{}, nil).Maybe() @@ -1299,13 +1265,11 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { } } } - tx, _ := getTx(t, 0, mkey) txID := "test" - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID), SetTimeout(10*time.Second)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) // Will create a expired transaction as lastValidBlockHeight is 0 by default. wg.Wait() time.Sleep(2 * time.Second) // Sleep to allow for rebroadcasting - // Check that transaction for txID has been finalized and rebroadcasted status, err := txm.GetTransactionStatus(ctx, txID) require.NoError(t, err) diff --git a/pkg/solana/txm/txm_race_test.go b/pkg/solana/txm/txm_race_test.go index f008023d0..ea175e63b 100644 --- a/pkg/solana/txm/txm_race_test.go +++ b/pkg/solana/txm/txm_race_test.go @@ -9,7 +9,6 @@ import ( "time" solanaGo "github.com/gagliardetto/solana-go" - "github.com/gagliardetto/solana-go/rpc" "go.uber.org/zap/zapcore" "github.com/smartcontractkit/chainlink-common/pkg/logger" @@ -117,11 +116,6 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { }, nil, ) - client.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: uint64(2000), - }, - }, nil) testRunner(t, client) }) @@ -158,11 +152,6 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { }, nil, ) - client.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: uint64(2000), - }, - }, nil) testRunner(t, client) }) @@ -210,11 +199,6 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { }, nil, ) - client.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: uint64(2000), - }, - }, nil) testRunner(t, client) }) @@ -249,11 +233,6 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { require.NoError(t, fees.SetComputeUnitLimit(&msg3.tx, 200_000)) msg3.tx.Signatures = make([]solanaGo.Signature, 1) client.On("SendTx", mock.Anything, &msg3.tx).Return(solanaGo.Signature{4}, nil) - client.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: uint64(2000), - }, - }, nil) testRunner(t, client) }) } From 84e423e374f35b173f12d08f001fbc374593d106 Mon Sep 17 00:00:00 2001 From: Farber98 Date: Thu, 21 Nov 2024 17:53:23 -0300 Subject: [PATCH 25/51] tests + check to save rpc calls --- pkg/solana/txm/pendingtx.go | 21 +- pkg/solana/txm/txm.go | 6 + pkg/solana/txm/txm_internal_test.go | 405 +++++++++++++++++++++++----- 3 files changed, 357 insertions(+), 75 deletions(-) diff --git a/pkg/solana/txm/pendingtx.go b/pkg/solana/txm/pendingtx.go index 47030056f..595716cb7 100644 --- a/pkg/solana/txm/pendingtx.go +++ b/pkg/solana/txm/pendingtx.go @@ -30,6 +30,8 @@ type PendingTxContext interface { ListAll() []solana.Signature // ListAllExpiredBroadcastedTxs returns all the txes that are in broadcasted state and have expired for given slot height compared against their lastValidBlockHeight. ListAllExpiredBroadcastedTxs(currHeight uint64) []pendingTx + // ListAllBroadcastedTxs returns all the txes that are in broadcasted state. + ListAllBroadcastedTxs() []pendingTx // Expired returns whether or not confirmation timeout amount of time has passed since creation Expired(sig solana.Signature, confirmationTimeout time.Duration) bool // OnProcessed marks transactions as Processed @@ -217,7 +219,7 @@ func (c *pendingTxContext) ListAll() []solana.Signature { return maps.Keys(c.sigToID) } -// ListAllExpiredBroadcastedTxs returns all the expired broadcasted that are in broadcasted state and have expired for given slot height. +// ListAllExpiredBroadcastedTxs returns all the txes that are in broadcasted state and have expired for given slot height compared against their lastValidBlockHeight. func (c *pendingTxContext) ListAllExpiredBroadcastedTxs(currHeight uint64) []pendingTx { c.lock.RLock() defer c.lock.RUnlock() @@ -230,6 +232,19 @@ func (c *pendingTxContext) ListAllExpiredBroadcastedTxs(currHeight uint64) []pen return broadcastedTxes } +// ListAllBroadcastedTxs returns all the txes that are in broadcasted state. +func (c *pendingTxContext) ListAllBroadcastedTxs() []pendingTx { + c.lock.RLock() + defer c.lock.RUnlock() + broadcastedTxes := make([]pendingTx, 0, len(c.broadcastedProcessedTxs)) // worst case, all of them + for _, tx := range c.broadcastedProcessedTxs { + if tx.state == Broadcasted { + broadcastedTxes = append(broadcastedTxes, tx) + } + } + return broadcastedTxes +} + // Expired returns if the timeout for trying to confirm a signature has been reached func (c *pendingTxContext) Expired(sig solana.Signature, confirmationTimeout time.Duration) bool { c.lock.RLock() @@ -697,3 +712,7 @@ func (c *pendingTxContextWithProm) TrimFinalizedErroredTxs() { func (c *pendingTxContextWithProm) GetTxRebroadcastCount(id string) (int, error) { return c.pendingTx.GetTxRebroadcastCount(id) } + +func (c *pendingTxContextWithProm) ListAllBroadcastedTxs() []pendingTx { + return c.pendingTx.ListAllBroadcastedTxs() +} diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index 8f8d419c9..5f5150da4 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -392,6 +392,12 @@ func (txm *Txm) confirm() { break } txm.processConfirmations(ctx, client) + + // In case all txes where confirmed and there's nothing to rebroadcast. + // This check saves making 2 RPC calls (slot height + blockhash) when there's nothing to process. + if len(txm.txs.ListAllBroadcastedTxs()) == 0 { + break + } if txm.cfg.TxExpirationRebroadcast() { txm.rebroadcastExpiredTxs(ctx, client) } diff --git a/pkg/solana/txm/txm_internal_test.go b/pkg/solana/txm/txm_internal_test.go index 4812122dc..ea2ec8aab 100644 --- a/pkg/solana/txm/txm_internal_test.go +++ b/pkg/solana/txm/txm_internal_test.go @@ -1187,94 +1187,351 @@ func addSigAndLimitToTx(t *testing.T, keystore SimpleKeystore, pubkey solana.Pub func TestTxm_ExpirationRebroadcast(t *testing.T) { t.Parallel() - // Set up configurations estimator := "fixed" id := "mocknet-" + estimator + "-" + uuid.NewString() - t.Logf("Starting new iteration: %s", id) - ctx := tests.Context(t) - lggr := logger.Test(t) cfg := config.NewDefault() cfg.Chain.FeeEstimatorMode = &estimator - txExpirationRebroadcast := true - cfg.Chain.TxExpirationRebroadcast = &txExpirationRebroadcast // enable expiration rebroadcast cfg.Chain.TxConfirmTimeout = relayconfig.MustNewDuration(5 * time.Second) cfg.Chain.TxRetentionTimeout = relayconfig.MustNewDuration(10 * time.Second) // Enable retention to keep transactions after finality and be able to check. + lggr := logger.Test(t) + ctx := tests.Context(t) - mc := mocks.NewReaderWriter(t) + // Helper function to set up common test environment + setupTxmTest := func( + txExpirationRebroadcast bool, + latestBlockhashFunc func() (*rpc.GetLatestBlockhashResult, error), + slotHeightFunc func() (uint64, error), + sendTxFunc func() (solana.Signature, error), + statuses map[solana.Signature]func() *rpc.SignatureStatusesResult, + ) (*Txm, *mocks.ReaderWriter, *keyMocks.SimpleKeystore) { + cfg.Chain.TxExpirationRebroadcast = &txExpirationRebroadcast + + mc := mocks.NewReaderWriter(t) + if latestBlockhashFunc != nil { + mc.On("LatestBlockhash", mock.Anything).Return( + func(_ context.Context) (*rpc.GetLatestBlockhashResult, error) { + return latestBlockhashFunc() + }, + ).Maybe() + } + if slotHeightFunc != nil { + mc.On("SlotHeight", mock.Anything).Return( + func(_ context.Context) (uint64, error) { + return slotHeightFunc() + }, + ).Maybe() + } + if sendTxFunc != nil { + mc.On("SendTx", mock.Anything, mock.Anything).Return( + func(_ context.Context, _ *solana.Transaction) (solana.Signature, error) { + return sendTxFunc() + }, + ).Maybe() + } - // First blockhash is set on sender. Second blockhash (the one returned here) is set on txExpirationRebroadcast before rebroadcasting. - // The first one will be invalid as it's initialized in 0 by default. This call will get a valid one greater than slotHeight and go through. - mc.On("LatestBlockhash", mock.Anything).Return(func(_ context.Context) (*rpc.GetLatestBlockhashResult, error) { - return &rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: uint64(2000), - }, - }, nil - }).Maybe() - - // Set up SlotHeight to return a value greater than 0 so the initial LastValidBlockHeight is invalid. - mc.On("SlotHeight", mock.Anything).Return(uint64(1500), nil).Maybe() - mkey := keyMocks.NewSimpleKeystore(t) - mkey.On("Sign", mock.Anything, mock.Anything, mock.Anything).Return([]byte{}, nil) - loader := utils.NewLazyLoad(func() (client.ReaderWriter, error) { return mc, nil }) - txm := NewTxm(id, loader, nil, cfg, mkey, lggr) - require.NoError(t, txm.Start(ctx)) - t.Cleanup(func() { require.NoError(t, txm.Close()) }) - sig1 := randomSignature(t) - mc.On("SendTx", mock.Anything, mock.Anything).Return(sig1, nil).Maybe() - mc.On("SimulateTx", mock.Anything, mock.Anything, mock.Anything).Return(&rpc.SimulateTransactionResult{}, nil).Maybe() - statuses := map[solana.Signature]func() *rpc.SignatureStatusesResult{} - mc.On("SignatureStatuses", mock.Anything, mock.AnythingOfType("[]solana.Signature")).Return( - func(_ context.Context, sigs []solana.Signature) (out []*rpc.SignatureStatusesResult) { - for i := range sigs { - get, exists := statuses[sigs[i]] - if !exists { - out = append(out, nil) - continue + mc.On("SimulateTx", mock.Anything, mock.Anything, mock.Anything).Return(&rpc.SimulateTransactionResult{}, nil) + if statuses != nil { + mc.On("SignatureStatuses", mock.Anything, mock.AnythingOfType("[]solana.Signature")).Return( + func(_ context.Context, sigs []solana.Signature) ([]*rpc.SignatureStatusesResult, error) { + var out []*rpc.SignatureStatusesResult + for _, sig := range sigs { + getStatus, exists := statuses[sig] + if !exists { + out = append(out, nil) + } else { + out = append(out, getStatus()) + } + } + return out, nil + }, + ).Maybe() + } + + mkey := keyMocks.NewSimpleKeystore(t) + mkey.On("Sign", mock.Anything, mock.Anything, mock.Anything).Return([]byte{}, nil) + + loader := utils.NewLazyLoad(func() (client.ReaderWriter, error) { return mc, nil }) + txm := NewTxm(id, loader, nil, cfg, mkey, lggr) + require.NoError(t, txm.Start(ctx)) + t.Cleanup(func() { require.NoError(t, txm.Close()) }) + + return txm, mc, mkey + } + + t.Run("WithRebroadcast", func(t *testing.T) { + txExpirationRebroadcast := true + statuses := map[solana.Signature]func() *rpc.SignatureStatusesResult{} + + // Mock SlotHeight to return a value greater than 0 + slotHeightFunc := func() (uint64, error) { + return uint64(1500), nil + } + + // Mock LatestBlockhash to return a valid blockhash greater than slotHeight + latestBlockhashFunc := func() (*rpc.GetLatestBlockhashResult, error) { + return &rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: uint64(2000), + }, + }, nil + } + + sig1 := randomSignature(t) + sendTxFunc := func() (solana.Signature, error) { + return sig1, nil + } + + nowTs := time.Now() + sigStatusCallCount := 0 + var wg sync.WaitGroup + wg.Add(1) + statuses[sig1] = func() *rpc.SignatureStatusesResult { + // First transaction should be rebroadcasted. + if time.Since(nowTs) < cfg.TxConfirmTimeout()-2*time.Second { + return nil + } else { + // Second transaction should reach finalization. + sigStatusCallCount++ + if sigStatusCallCount == 1 { + return &rpc.SignatureStatusesResult{ + ConfirmationStatus: rpc.ConfirmationStatusProcessed, + } + } else if sigStatusCallCount == 2 { + return &rpc.SignatureStatusesResult{ + ConfirmationStatus: rpc.ConfirmationStatusConfirmed, + } + } else { + wg.Done() + return &rpc.SignatureStatusesResult{ + ConfirmationStatus: rpc.ConfirmationStatusFinalized, + } } - out = append(out, get()) } - return out - }, nil, - ) + } - nowTs := time.Now() - sigStatusCallCount := 0 - var wg sync.WaitGroup - wg.Add(1) - statuses[sig1] = func() *rpc.SignatureStatusesResult { - // first transaction should be rebroadcasted. - if time.Since(nowTs) < cfg.TxConfirmTimeout()-2*time.Second { - return nil - } else { - // second transaction should reach finalization. - sigStatusCallCount++ - if sigStatusCallCount == 1 { - return &rpc.SignatureStatusesResult{ - ConfirmationStatus: rpc.ConfirmationStatusProcessed, - } - } else if sigStatusCallCount == 2 { - return &rpc.SignatureStatusesResult{ - ConfirmationStatus: rpc.ConfirmationStatusConfirmed, - } + txm, _, mkey := setupTxmTest(txExpirationRebroadcast, latestBlockhashFunc, slotHeightFunc, sendTxFunc, statuses) + + tx, _ := getTx(t, 0, mkey) + txID := "test-rebroadcast" + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) // Will create an expired transaction as lastValidBlockHeight is 0 by default. + wg.Wait() + time.Sleep(2 * time.Second) // Sleep to allow for rebroadcasting + + // Check that transaction for txID has been finalized and rebroadcasted + status, err := txm.GetTransactionStatus(ctx, txID) + require.NoError(t, err) + require.Equal(t, types.Finalized, status) + rebroadcastCount, err := txm.txs.GetTxRebroadcastCount(txID) + require.NoError(t, err) + require.Equal(t, 1, rebroadcastCount) + }) + + t.Run("WithoutRebroadcast", func(t *testing.T) { + txExpirationRebroadcast := false + statuses := map[solana.Signature]func() *rpc.SignatureStatusesResult{} + + sig1 := randomSignature(t) + sendTxFunc := func() (solana.Signature, error) { + return sig1, nil + } + + nowTs := time.Now() + var wg sync.WaitGroup + wg.Add(1) + statuses[sig1] = func() *rpc.SignatureStatusesResult { + // Transaction remains unconfirmed and should not be rebroadcasted. + if time.Since(nowTs) < cfg.TxConfirmTimeout() { + return nil } else { wg.Done() - return &rpc.SignatureStatusesResult{ - ConfirmationStatus: rpc.ConfirmationStatusFinalized, + return nil + } + } + // No LatestBlockhash nor slotHeight needed because there's no rebroadcast. + txm, _, mkey := setupTxmTest(txExpirationRebroadcast, nil, nil, sendTxFunc, statuses) + + tx, _ := getTx(t, 5, mkey) + txID := "test-no-rebroadcast" + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) + wg.Wait() + time.Sleep(2 * time.Second) // Sleep to ensure no rebroadcast + + // Check that transaction for txID has not been finalized and has not been rebroadcasted + status, err := txm.GetTransactionStatus(ctx, txID) + require.NoError(t, err) + require.Equal(t, types.Failed, status) + rebroadcastCount, err := txm.txs.GetTxRebroadcastCount(txID) + require.NoError(t, err) + require.Equal(t, 0, rebroadcastCount) + }) + + t.Run("WithMultipleRebroadcast", func(t *testing.T) { + txExpirationRebroadcast := true + expectedRebroadcastsCount := 3 + statuses := map[solana.Signature]func() *rpc.SignatureStatusesResult{} + + // Mock SlotHeight to return a value greater than 0 + slotHeightFunc := func() (uint64, error) { + return uint64(1500), nil + } + + // Mock LatestBlockhash to return a invalid blockhash first 2 attempts and a valid blockhash third time + // the third one is valid because it is greater than the slotHeight + callCount := 1 + latestBlockhashFunc := func() (*rpc.GetLatestBlockhashResult, error) { + defer func() { callCount++ }() + if callCount < expectedRebroadcastsCount { + return &rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: uint64(1000), + }, + }, nil + } + return &rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: uint64(2000), + }, + }, nil + } + + sig1 := randomSignature(t) + sendTxFunc := func() (solana.Signature, error) { + return sig1, nil + } + nowTs := time.Now() + sigStatusCallCount := 0 + var wg sync.WaitGroup + wg.Add(1) + statuses[sig1] = func() *rpc.SignatureStatusesResult { + // transaction should be rebroadcasted multiple times. + if time.Since(nowTs) < cfg.TxConfirmTimeout()-2*time.Second { + return nil + } else { + // Second transaction should reach finalization. + sigStatusCallCount++ + if sigStatusCallCount == 1 { + return &rpc.SignatureStatusesResult{ + ConfirmationStatus: rpc.ConfirmationStatusProcessed, + } + } else if sigStatusCallCount == 2 { + return &rpc.SignatureStatusesResult{ + ConfirmationStatus: rpc.ConfirmationStatusConfirmed, + } + } else { + wg.Done() + return &rpc.SignatureStatusesResult{ + ConfirmationStatus: rpc.ConfirmationStatusFinalized, + } } } } - } - tx, _ := getTx(t, 0, mkey) - txID := "test" - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) // Will create a expired transaction as lastValidBlockHeight is 0 by default. - wg.Wait() - time.Sleep(2 * time.Second) // Sleep to allow for rebroadcasting - // Check that transaction for txID has been finalized and rebroadcasted - status, err := txm.GetTransactionStatus(ctx, txID) - require.NoError(t, err) - require.Equal(t, types.Finalized, status) - rebroadcastCount, err := txm.txs.GetTxRebroadcastCount(txID) - require.NoError(t, err) - require.Equal(t, 1, rebroadcastCount) + + txm, _, mkey := setupTxmTest(txExpirationRebroadcast, latestBlockhashFunc, slotHeightFunc, sendTxFunc, statuses) + tx, _ := getTx(t, 0, mkey) + txID := "test-rebroadcast" + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) // Will create an expired transaction as lastValidBlockHeight is 0 by default. + wg.Wait() + time.Sleep(2 * time.Second) // Sleep to allow for rebroadcasting + + // Check that transaction for txID has been finalized and rebroadcasted + status, err := txm.GetTransactionStatus(ctx, txID) + require.NoError(t, err) + require.Equal(t, types.Finalized, status) + rebroadcastCount, err := txm.txs.GetTxRebroadcastCount(txID) + require.NoError(t, err) + require.Equal(t, expectedRebroadcastsCount, rebroadcastCount) + }) + + t.Run("ConfirmedBeforeRebroadcast", func(t *testing.T) { + txExpirationRebroadcast := true + statuses := map[solana.Signature]func() *rpc.SignatureStatusesResult{} + sig1 := randomSignature(t) + sendTxFunc := func() (solana.Signature, error) { + return sig1, nil + } + + var wg sync.WaitGroup + wg.Add(1) + count := 0 + statuses[sig1] = func() *rpc.SignatureStatusesResult { + defer func() { count++ }() + + out := &rpc.SignatureStatusesResult{} + if count == 1 { + out.ConfirmationStatus = rpc.ConfirmationStatusConfirmed + return out + } + if count == 2 { + out.ConfirmationStatus = rpc.ConfirmationStatusFinalized + wg.Done() + return out + } + out.ConfirmationStatus = rpc.ConfirmationStatusProcessed + return out + } + + // No LatestBlockhash nor SlotHeight needed + // Our check will detect there are no rebroadcasts to process saving 2 rpc calls and ending loop. + txm, _, mkey := setupTxmTest(txExpirationRebroadcast, nil, nil, sendTxFunc, statuses) + tx, _ := getTx(t, 0, mkey) + txID := "test-confirmed-before-rebroadcast" + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) + wg.Wait() + time.Sleep(1 * time.Second) // Allow for processing + + // Check that transaction has been finalized without rebroadcast + status, err := txm.GetTransactionStatus(ctx, txID) + require.NoError(t, err) + require.Equal(t, types.Finalized, status) + rebroadcastCount, err := txm.txs.GetTxRebroadcastCount(txID) + require.NoError(t, err) + require.Equal(t, 0, rebroadcastCount) + }) + + t.Run("RebroadcastWithError", func(t *testing.T) { + txExpirationRebroadcast := true + statuses := map[solana.Signature]func() *rpc.SignatureStatusesResult{} + + // Mock SlotHeight to return a value greater than 0 + slotHeightFunc := func() (uint64, error) { + return uint64(1500), nil + } + + // Mock LatestBlockhash to return a valid blockhash greater than slotHeight + latestBlockhashFunc := func() (*rpc.GetLatestBlockhashResult, error) { + return &rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: uint64(2000), + }, + }, nil + } + + sig1 := randomSignature(t) + sendTxFunc := func() (solana.Signature, error) { + return sig1, nil + } + + statuses[sig1] = func() *rpc.SignatureStatusesResult { + // Transaction remains unconfirmed + return nil + } + + txm, _, mkey := setupTxmTest(txExpirationRebroadcast, latestBlockhashFunc, slotHeightFunc, sendTxFunc, statuses) + tx, _ := getTx(t, 0, mkey) + txID := "test-rebroadcast-error" + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) + time.Sleep(2 * time.Second) // Allow for processing + + // TODO: Add check that transaction status is failed due to rebroadcast error when prebroadcast is implemented and we have an error in sendWithRetry + status, err := txm.GetTransactionStatus(ctx, txID) + require.NoError(t, err) + require.Equal(t, types.Pending, status) // TODO: Change to Failed when prebroadcast error is implemented + rebroadcastCount, err := txm.txs.GetTxRebroadcastCount(txID) + require.NoError(t, err) + require.Equal(t, 1, rebroadcastCount) // Attempted to rebroadcast 1 time but encountered error + time.Sleep(2 * time.Second) // Allow for processing + rebroadcastCount, err = txm.txs.GetTxRebroadcastCount(txID) // rebroadcast should still be 1. We should not be rebroadcasting. + require.NoError(t, err) + require.Equal(t, 1, rebroadcastCount) + }) } From 7d8319e6d1fe67bfd2fe48a470dd577964c3f7aa Mon Sep 17 00:00:00 2001 From: Farber98 Date: Fri, 22 Nov 2024 12:19:02 -0300 Subject: [PATCH 26/51] address feedback + remove redundant impl --- pkg/solana/chain.go | 2 +- pkg/solana/transmitter.go | 2 +- pkg/solana/txm/pendingtx.go | 20 +------------------- pkg/solana/txm/txm.go | 4 +++- 4 files changed, 6 insertions(+), 22 deletions(-) diff --git a/pkg/solana/chain.go b/pkg/solana/chain.go index a9c3e10e2..ab901a548 100644 --- a/pkg/solana/chain.go +++ b/pkg/solana/chain.go @@ -557,7 +557,7 @@ func (c *chain) sendTx(ctx context.Context, from, to string, amount *big.Int, ba toKey, ).Build(), }, - blockhash.Value.Blockhash, // Will be override if needed within sendWithRetry txm function. + blockhash.Value.Blockhash, solanago.TransactionPayer(fromKey), ) if err != nil { diff --git a/pkg/solana/transmitter.go b/pkg/solana/transmitter.go index 5ee1b01b1..951e9633e 100644 --- a/pkg/solana/transmitter.go +++ b/pkg/solana/transmitter.go @@ -78,7 +78,7 @@ func (c *Transmitter) Transmit( []solana.Instruction{ solana.NewInstruction(c.programID, accounts, data.Bytes()), }, - blockhash.Value.Blockhash, // Will be override if needed within sendWithRetry txm function. + blockhash.Value.Blockhash, solana.TransactionPayer(c.transmissionSigner), ) if err != nil { diff --git a/pkg/solana/txm/pendingtx.go b/pkg/solana/txm/pendingtx.go index 595716cb7..b04818a73 100644 --- a/pkg/solana/txm/pendingtx.go +++ b/pkg/solana/txm/pendingtx.go @@ -30,8 +30,6 @@ type PendingTxContext interface { ListAll() []solana.Signature // ListAllExpiredBroadcastedTxs returns all the txes that are in broadcasted state and have expired for given slot height compared against their lastValidBlockHeight. ListAllExpiredBroadcastedTxs(currHeight uint64) []pendingTx - // ListAllBroadcastedTxs returns all the txes that are in broadcasted state. - ListAllBroadcastedTxs() []pendingTx // Expired returns whether or not confirmation timeout amount of time has passed since creation Expired(sig solana.Signature, confirmationTimeout time.Duration) bool // OnProcessed marks transactions as Processed @@ -220,6 +218,7 @@ func (c *pendingTxContext) ListAll() []solana.Signature { } // ListAllExpiredBroadcastedTxs returns all the txes that are in broadcasted state and have expired for given slot height compared against their lastValidBlockHeight. +// Passing maxUint64 as currHeight will return all broadcasted txes. func (c *pendingTxContext) ListAllExpiredBroadcastedTxs(currHeight uint64) []pendingTx { c.lock.RLock() defer c.lock.RUnlock() @@ -232,19 +231,6 @@ func (c *pendingTxContext) ListAllExpiredBroadcastedTxs(currHeight uint64) []pen return broadcastedTxes } -// ListAllBroadcastedTxs returns all the txes that are in broadcasted state. -func (c *pendingTxContext) ListAllBroadcastedTxs() []pendingTx { - c.lock.RLock() - defer c.lock.RUnlock() - broadcastedTxes := make([]pendingTx, 0, len(c.broadcastedProcessedTxs)) // worst case, all of them - for _, tx := range c.broadcastedProcessedTxs { - if tx.state == Broadcasted { - broadcastedTxes = append(broadcastedTxes, tx) - } - } - return broadcastedTxes -} - // Expired returns if the timeout for trying to confirm a signature has been reached func (c *pendingTxContext) Expired(sig solana.Signature, confirmationTimeout time.Duration) bool { c.lock.RLock() @@ -712,7 +698,3 @@ func (c *pendingTxContextWithProm) TrimFinalizedErroredTxs() { func (c *pendingTxContextWithProm) GetTxRebroadcastCount(id string) (int, error) { return c.pendingTx.GetTxRebroadcastCount(id) } - -func (c *pendingTxContextWithProm) ListAllBroadcastedTxs() []pendingTx { - return c.pendingTx.ListAllBroadcastedTxs() -} diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index 5f5150da4..fbc9e53be 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -4,6 +4,7 @@ import ( "context" "errors" "fmt" + "math" "math/big" "strings" "sync" @@ -395,7 +396,8 @@ func (txm *Txm) confirm() { // In case all txes where confirmed and there's nothing to rebroadcast. // This check saves making 2 RPC calls (slot height + blockhash) when there's nothing to process. - if len(txm.txs.ListAllBroadcastedTxs()) == 0 { + // Passing MaxUint64 as currHeight to ListAllExpiredBroadcastedTxs will return all broadcasted txs. + if len(txm.txs.ListAllExpiredBroadcastedTxs(math.MaxUint64)) == 0 { break } if txm.cfg.TxExpirationRebroadcast() { From 68f3a3e0cf2c14712b14b0fd43725c2a3f8fa10e Mon Sep 17 00:00:00 2001 From: Farber98 Date: Fri, 22 Nov 2024 12:22:28 -0300 Subject: [PATCH 27/51] iface comment --- pkg/solana/txm/pendingtx.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/solana/txm/pendingtx.go b/pkg/solana/txm/pendingtx.go index b04818a73..f851cd2bb 100644 --- a/pkg/solana/txm/pendingtx.go +++ b/pkg/solana/txm/pendingtx.go @@ -29,6 +29,7 @@ type PendingTxContext interface { // ListAll returns all of the signatures being tracked for all transactions not yet finalized or errored ListAll() []solana.Signature // ListAllExpiredBroadcastedTxs returns all the txes that are in broadcasted state and have expired for given slot height compared against their lastValidBlockHeight. + // Passing maxUint64 as currHeight will return all broadcasted txes. ListAllExpiredBroadcastedTxs(currHeight uint64) []pendingTx // Expired returns whether or not confirmation timeout amount of time has passed since creation Expired(sig solana.Signature, confirmationTimeout time.Duration) bool From 780179f35609a5eb9f30db5f773a565e40f153d2 Mon Sep 17 00:00:00 2001 From: Farber98 Date: Mon, 25 Nov 2024 09:13:37 -0300 Subject: [PATCH 28/51] address feedback on compute unit limit and lastValidBlockHeight assignment --- pkg/solana/chain.go | 2 +- pkg/solana/chain_test.go | 9 ++-- pkg/solana/relay.go | 2 +- pkg/solana/transmitter.go | 2 +- pkg/solana/transmitter_test.go | 2 +- pkg/solana/txm/txm.go | 31 +++++------- pkg/solana/txm/txm_internal_test.go | 78 ++++++++++++++++++----------- pkg/solana/txm/txm_load_test.go | 16 +++--- 8 files changed, 78 insertions(+), 64 deletions(-) diff --git a/pkg/solana/chain.go b/pkg/solana/chain.go index ab901a548..0267e2848 100644 --- a/pkg/solana/chain.go +++ b/pkg/solana/chain.go @@ -571,7 +571,7 @@ func (c *chain) sendTx(ctx context.Context, from, to string, amount *big.Int, ba } chainTxm := c.TxManager() - err = chainTxm.Enqueue(ctx, "", tx, nil, + err = chainTxm.Enqueue(ctx, "", tx, nil, blockhash.Value.LastValidBlockHeight, txm.SetComputeUnitLimit(500), // reduce from default 200K limit - should only take 450 compute units // no fee bumping and no additional fee - makes validating balance accurate txm.SetComputeUnitPriceMax(0), diff --git a/pkg/solana/chain_test.go b/pkg/solana/chain_test.go index b1e6ff427..65f803bac 100644 --- a/pkg/solana/chain_test.go +++ b/pkg/solana/chain_test.go @@ -534,12 +534,13 @@ func TestSolanaChain_MultiNode_Txm(t *testing.T) { receiverBal, err := selectedClient.Balance(tests.Context(t), pubKeyReceiver) assert.NoError(t, err) assert.Equal(t, uint64(0), receiverBal) - + var lastValidBlockHeight uint64 createTx := func(signer solana.PublicKey, sender solana.PublicKey, receiver solana.PublicKey, amt uint64) *solana.Transaction { selectedClient, err = testChain.getClient() assert.NoError(t, err) hash, hashErr := selectedClient.LatestBlockhash(tests.Context(t)) assert.NoError(t, hashErr) + lastValidBlockHeight = hash.Value.LastValidBlockHeight tx, txErr := solana.NewTransaction( []solana.Instruction{ system.NewTransferInstruction( @@ -556,7 +557,7 @@ func TestSolanaChain_MultiNode_Txm(t *testing.T) { } // Send funds twice, along with an invalid transaction - require.NoError(t, testChain.txm.Enqueue(tests.Context(t), "test_success", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) + require.NoError(t, testChain.txm.Enqueue(tests.Context(t), "test_success", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil, lastValidBlockHeight)) // Wait for new block hash currentBh, err := selectedClient.LatestBlockhash(tests.Context(t)) require.NoError(t, err) @@ -576,8 +577,8 @@ NewBlockHash: } } - require.NoError(t, testChain.txm.Enqueue(tests.Context(t), "test_success_2", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) - require.Error(t, testChain.txm.Enqueue(tests.Context(t), "test_invalidSigner", createTx(pubKeyReceiver, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) // cannot sign tx before enqueuing + require.NoError(t, testChain.txm.Enqueue(tests.Context(t), "test_success_2", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil, lastValidBlockHeight)) + require.Error(t, testChain.txm.Enqueue(tests.Context(t), "test_invalidSigner", createTx(pubKeyReceiver, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil, lastValidBlockHeight)) // cannot sign tx before enqueuing // wait for all txes to finish ctx, cancel := context.WithCancel(tests.Context(t)) t.Cleanup(cancel) diff --git a/pkg/solana/relay.go b/pkg/solana/relay.go index d98ab0442..bef9b02c3 100644 --- a/pkg/solana/relay.go +++ b/pkg/solana/relay.go @@ -24,7 +24,7 @@ import ( var _ TxManager = (*txm.Txm)(nil) type TxManager interface { - Enqueue(ctx context.Context, accountID string, tx *solana.Transaction, txID *string, txCfgs ...txm.SetTxConfig) error + Enqueue(ctx context.Context, accountID string, tx *solana.Transaction, txID *string, lastValidBlockHeight uint64, txCfgs ...txm.SetTxConfig) error } var _ relaytypes.Relayer = &Relayer{} //nolint:staticcheck diff --git a/pkg/solana/transmitter.go b/pkg/solana/transmitter.go index 951e9633e..537c72699 100644 --- a/pkg/solana/transmitter.go +++ b/pkg/solana/transmitter.go @@ -87,7 +87,7 @@ func (c *Transmitter) Transmit( // pass transmit payload to tx manager queue c.lggr.Debugf("Queuing transmit tx: state (%s) + transmissions (%s)", c.stateID.String(), c.transmissionsID.String()) - if err = c.txManager.Enqueue(ctx, c.stateID.String(), tx, nil); err != nil { + if err = c.txManager.Enqueue(ctx, c.stateID.String(), tx, nil, blockhash.Value.LastValidBlockHeight); err != nil { return fmt.Errorf("error on Transmit.txManager.Enqueue: %w", err) } return nil diff --git a/pkg/solana/transmitter_test.go b/pkg/solana/transmitter_test.go index d2f0229ca..6924eeef1 100644 --- a/pkg/solana/transmitter_test.go +++ b/pkg/solana/transmitter_test.go @@ -27,7 +27,7 @@ type verifyTxSize struct { s *solana.PrivateKey } -func (txm verifyTxSize) Enqueue(_ context.Context, _ string, tx *solana.Transaction, txID *string, _ ...txm.SetTxConfig) error { +func (txm verifyTxSize) Enqueue(_ context.Context, _ string, tx *solana.Transaction, txID *string, _ uint64, _ ...txm.SetTxConfig) error { // additional components that transaction manager adds to the transaction require.NoError(txm.t, fees.SetComputeUnitPrice(tx, 0)) require.NoError(txm.t, fees.SetComputeUnitLimit(tx, 0)) diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index fbc9e53be..0dc12a2c3 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -4,7 +4,6 @@ import ( "context" "errors" "fmt" - "math" "math/big" "strings" "sync" @@ -228,14 +227,15 @@ func (txm *Txm) sendWithRetry(ctx context.Context, msg pendingTx) (solanaGo.Tran // buildTx builds and signs the transaction with the appropriate compute unit price. func (txm *Txm) buildTx(ctx context.Context, msg pendingTx, retryCount int) (solanaGo.Transaction, error) { + // work with a copy + newTx := msg.tx + // Set compute unit limit if specified if msg.cfg.ComputeUnitLimit != 0 { - if err := fees.SetComputeUnitLimit(&msg.tx, fees.ComputeUnitLimit(msg.cfg.ComputeUnitLimit)); err != nil { + if err := fees.SetComputeUnitLimit(&newTx, fees.ComputeUnitLimit(msg.cfg.ComputeUnitLimit)); err != nil { return solanaGo.Transaction{}, fmt.Errorf("failed to add compute unit limit instruction: %w", err) } } - // work with a copy - newTx := msg.tx // Set compute unit price (fee) fee := fees.ComputeUnitPrice( @@ -393,13 +393,6 @@ func (txm *Txm) confirm() { break } txm.processConfirmations(ctx, client) - - // In case all txes where confirmed and there's nothing to rebroadcast. - // This check saves making 2 RPC calls (slot height + blockhash) when there's nothing to process. - // Passing MaxUint64 as currHeight to ListAllExpiredBroadcastedTxs will return all broadcasted txs. - if len(txm.txs.ListAllExpiredBroadcastedTxs(math.MaxUint64)) == 0 { - break - } if txm.cfg.TxExpirationRebroadcast() { txm.rebroadcastExpiredTxs(ctx, client) } @@ -568,7 +561,7 @@ func (txm *Txm) rebroadcastExpiredTxs(ctx context.Context, client client.ReaderW } // Rebroadcast all expired txes for _, tx := range txm.txs.ListAllExpiredBroadcastedTxs(currHeight) { - txm.lggr.Infow("transaction expired, rebroadcasting", "id", tx.id, "signature", tx.signatures) + txm.lggr.Debugw("transaction expired, rebroadcasting", "id", tx.id, "signature", tx.signatures) if len(tx.signatures) == 0 { // prevent panic, shouldn't happen. txm.lggr.Errorw("no signatures found for expired transaction", "id", tx.id) continue @@ -582,6 +575,7 @@ func (txm *Txm) rebroadcastExpiredTxs(ctx context.Context, client client.ReaderW tx.tx.Message.RecentBlockhash = blockhash.Value.Blockhash rebroadcastTx := pendingTx{ tx: tx.tx, + cfg: tx.cfg, id: tx.id, rebroadcastCount: tx.rebroadcastCount + 1, lastValidBlockHeight: blockhash.Value.LastValidBlockHeight, @@ -595,7 +589,7 @@ func (txm *Txm) rebroadcastExpiredTxs(ctx context.Context, client client.ReaderW continue } - txm.lggr.Infow("rebroadcast transaction sent", "id", tx.id) + txm.lggr.Debugw("rebroadcast transaction sent", "id", tx.id) } } @@ -656,7 +650,7 @@ func (txm *Txm) reap() { } // Enqueue enqueues a msg destined for the solana chain. -func (txm *Txm) Enqueue(ctx context.Context, accountID string, tx *solanaGo.Transaction, txID *string, txCfgs ...SetTxConfig) error { +func (txm *Txm) Enqueue(ctx context.Context, accountID string, tx *solanaGo.Transaction, txID *string, lastValidBlockHeight uint64, txCfgs ...SetTxConfig) error { if err := txm.Ready(); err != nil { return fmt.Errorf("error in soltxm.Enqueue: %w", err) } @@ -703,9 +697,10 @@ func (txm *Txm) Enqueue(ctx context.Context, accountID string, tx *solanaGo.Tran } } - msg := &pendingTx{ - tx: *tx, - cfg: cfg, + msg := pendingTx{ + tx: *tx, + lastValidBlockHeight: lastValidBlockHeight, + cfg: cfg, } // If ID was not set by caller, create one. @@ -716,7 +711,7 @@ func (txm *Txm) Enqueue(ctx context.Context, accountID string, tx *solanaGo.Tran } select { - case txm.chSend <- *msg: + case txm.chSend <- msg: default: txm.lggr.Errorw("failed to enqueue tx", "queueFull", len(txm.chSend) == MaxQueueLen, "tx", msg) return fmt.Errorf("failed to enqueue transaction for %s", accountID) diff --git a/pkg/solana/txm/txm_internal_test.go b/pkg/solana/txm/txm_internal_test.go index ea2ec8aab..c5f1caff6 100644 --- a/pkg/solana/txm/txm_internal_test.go +++ b/pkg/solana/txm/txm_internal_test.go @@ -128,6 +128,7 @@ func TestTxm(t *testing.T) { mc := mocks.NewReaderWriter(t) mc.On("GetLatestBlock", mock.Anything).Return(&rpc.GetBlockResult{}, nil).Maybe() mc.On("SlotHeight", mock.Anything).Return(uint64(0), nil).Maybe() + lastValidBlockHeight := uint64(100) // higher than slotHeight so it's valid // mock solana keystore mkey := keyMocks.NewSimpleKeystore(t) @@ -204,7 +205,7 @@ func TestTxm(t *testing.T) { // send tx testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() // no transactions stored inflight txs list @@ -240,7 +241,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() // wait to be picked up and processed // no transactions stored inflight txs list @@ -271,7 +272,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared quickly @@ -307,7 +308,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared after timeout @@ -347,7 +348,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared after timeout @@ -397,7 +398,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared after timeout @@ -439,7 +440,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared after timeout @@ -484,7 +485,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // inflight txs cleared after timeout @@ -536,7 +537,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // inflight txs cleared after timeout @@ -574,7 +575,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // inflight txs cleared after timeout @@ -620,7 +621,7 @@ func TestTxm(t *testing.T) { // send tx testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() // no transactions stored inflight txs list @@ -674,7 +675,7 @@ func TestTxm(t *testing.T) { // send tx - with disabled fee bumping testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, SetFeeBumpPeriod(0))) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight, SetFeeBumpPeriod(0))) wg.Wait() // no transactions stored inflight txs list @@ -726,7 +727,7 @@ func TestTxm(t *testing.T) { // send tx - with disabled fee bumping and disabled compute unit limit testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, SetFeeBumpPeriod(0), SetComputeUnitLimit(0))) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight, SetFeeBumpPeriod(0), SetComputeUnitLimit(0))) wg.Wait() // no transactions stored inflight txs list @@ -766,6 +767,7 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { mc := mocks.NewReaderWriter(t) mc.On("GetLatestBlock", mock.Anything).Return(&rpc.GetBlockResult{}, nil).Maybe() mc.On("SlotHeight", mock.Anything).Return(uint64(0), nil).Maybe() + lastValidBlockHeight := uint64(100) // higher sthan slotHeight so it's valid. computeUnitLimitDefault := fees.ComputeUnitLimit(cfg.ComputeUnitLimitDefault()) @@ -836,7 +838,7 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() // wait to be picked up and processed waitFor(t, 5*time.Second, txm, prom, empty) // inflight txs cleared after timeout @@ -967,6 +969,7 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { mc := mocks.NewReaderWriter(t) mc.On("GetLatestBlock", mock.Anything).Return(&rpc.GetBlockResult{}, nil).Maybe() mc.On("SlotHeight", mock.Anything).Return(uint64(0), nil).Maybe() + lastValidBlockHeight := uint64(100) // higher sthan slotHeight so it's valid. // mock solana keystore mkey := keyMocks.NewSimpleKeystore(t) @@ -1042,7 +1045,7 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { // send tx testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() // no transactions stored inflight txs list @@ -1071,7 +1074,7 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { mc.On("SimulateTx", mock.Anything, mock.Anything, mock.Anything).Return(nil, errors.New("simulation failed")).Once() // tx should NOT be able to queue - assert.Error(t, txm.Enqueue(ctx, t.Name(), tx, nil)) + assert.Error(t, txm.Enqueue(ctx, t.Name(), tx, nil, lastValidBlockHeight)) }) t.Run("simulation_returns_error", func(t *testing.T) { @@ -1087,7 +1090,7 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { txID := uuid.NewString() // tx should NOT be able to queue - assert.Error(t, txm.Enqueue(ctx, t.Name(), tx, nil)) + assert.Error(t, txm.Enqueue(ctx, t.Name(), tx, nil, lastValidBlockHeight)) }) } @@ -1114,6 +1117,7 @@ func TestTxm_Enqueue(t *testing.T) { invalidKey := solana.PublicKeyFromBytes([]byte{2}) mkey.On("Sign", mock.Anything, validKey.String(), mock.Anything).Return([]byte{1}, nil) mkey.On("Sign", mock.Anything, invalidKey.String(), mock.Anything).Return([]byte{}, relayconfig.KeyNotFoundError{ID: invalidKey.String(), KeyType: "Solana"}) + lastValidBlockHeight := uint64(100) // build txs tx, err := solana.NewTransaction( @@ -1145,7 +1149,7 @@ func TestTxm_Enqueue(t *testing.T) { loader := utils.NewLazyLoad(func() (client.ReaderWriter, error) { return mc, nil }) txm := NewTxm("enqueue_test", loader, nil, cfg, mkey, lggr) - require.ErrorContains(t, txm.Enqueue(ctx, "txmUnstarted", &solana.Transaction{}, nil), "not started") + require.ErrorContains(t, txm.Enqueue(ctx, "txmUnstarted", &solana.Transaction{}, nil, lastValidBlockHeight), "not started") require.NoError(t, txm.Start(ctx)) t.Cleanup(func() { require.NoError(t, txm.Close()) }) @@ -1163,10 +1167,10 @@ func TestTxm_Enqueue(t *testing.T) { for _, run := range txs { t.Run(run.name, func(t *testing.T) { if !run.fail { - assert.NoError(t, txm.Enqueue(ctx, run.name, run.tx, nil)) + assert.NoError(t, txm.Enqueue(ctx, run.name, run.tx, nil, lastValidBlockHeight)) return } - assert.Error(t, txm.Enqueue(ctx, run.name, run.tx, nil)) + assert.Error(t, txm.Enqueue(ctx, run.name, run.tx, nil, lastValidBlockHeight)) }) } } @@ -1313,7 +1317,7 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { tx, _ := getTx(t, 0, mkey) txID := "test-rebroadcast" - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) // Will create an expired transaction as lastValidBlockHeight is 0 by default. + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID, 0)) // Will create an expired transaction as lastValidBlockHeight is 0. This will force the rebroadcast. wg.Wait() time.Sleep(2 * time.Second) // Sleep to allow for rebroadcasting @@ -1352,7 +1356,7 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { tx, _ := getTx(t, 5, mkey) txID := "test-no-rebroadcast" - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID, 0)) // Won't rebroadcast as txExpirationRebroadcast is false. wg.Wait() time.Sleep(2 * time.Second) // Sleep to ensure no rebroadcast @@ -1375,7 +1379,8 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { return uint64(1500), nil } - // Mock LatestBlockhash to return a invalid blockhash first 2 attempts and a valid blockhash third time + // Mock LatestBlockhash to return a invalid blockhash first 2 rebroadcast attempts + // will return a valid blockhash third rebroadcast attempt. // the third one is valid because it is greater than the slotHeight callCount := 1 latestBlockhashFunc := func() (*rpc.GetLatestBlockhashResult, error) { @@ -1429,7 +1434,7 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { txm, _, mkey := setupTxmTest(txExpirationRebroadcast, latestBlockhashFunc, slotHeightFunc, sendTxFunc, statuses) tx, _ := getTx(t, 0, mkey) txID := "test-rebroadcast" - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) // Will create an expired transaction as lastValidBlockHeight is 0 by default. + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID, 0)) // lastValidBlockHeight is 0 to force rebroadcast. wg.Wait() time.Sleep(2 * time.Second) // Sleep to allow for rebroadcasting @@ -1450,6 +1455,21 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { return sig1, nil } + // Mock SlotHeight to return a value greater than 0 + slotHeightFunc := func() (uint64, error) { + return uint64(1500), nil + } + + // Mock LatestBlockhash to return an invalid blockhash less than slotHeight + // We won't use it as there are no rebroadcasts to process. Just to test that we don't need it. + latestBlockhashFunc := func() (*rpc.GetLatestBlockhashResult, error) { + return &rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: uint64(1000), + }, + }, nil + } + var wg sync.WaitGroup wg.Add(1) count := 0 @@ -1470,12 +1490,10 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { return out } - // No LatestBlockhash nor SlotHeight needed - // Our check will detect there are no rebroadcasts to process saving 2 rpc calls and ending loop. - txm, _, mkey := setupTxmTest(txExpirationRebroadcast, nil, nil, sendTxFunc, statuses) + txm, _, mkey := setupTxmTest(txExpirationRebroadcast, latestBlockhashFunc, slotHeightFunc, sendTxFunc, statuses) tx, _ := getTx(t, 0, mkey) txID := "test-confirmed-before-rebroadcast" - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID, 0)) wg.Wait() time.Sleep(1 * time.Second) // Allow for processing @@ -1519,8 +1537,8 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { txm, _, mkey := setupTxmTest(txExpirationRebroadcast, latestBlockhashFunc, slotHeightFunc, sendTxFunc, statuses) tx, _ := getTx(t, 0, mkey) txID := "test-rebroadcast-error" - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) - time.Sleep(2 * time.Second) // Allow for processing + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID, 0)) // lastValidBlockHeight is 0 to force rebroadcast. + time.Sleep(2 * time.Second) // Allow for processing // TODO: Add check that transaction status is failed due to rebroadcast error when prebroadcast is implemented and we have an error in sendWithRetry status, err := txm.GetTransactionStatus(ctx, txID) diff --git a/pkg/solana/txm/txm_load_test.go b/pkg/solana/txm/txm_load_test.go index 19d02e526..b1b2398c9 100644 --- a/pkg/solana/txm/txm_load_test.go +++ b/pkg/solana/txm/txm_load_test.go @@ -82,7 +82,7 @@ func TestTxm_Integration(t *testing.T) { // already started assert.Error(t, txm.Start(ctx)) - + var lastValidBlockHeight uint64 createTx := func(signer solana.PublicKey, sender solana.PublicKey, receiver solana.PublicKey, amt uint64) *solana.Transaction { // create transfer tx hash, err := client.LatestBlockhash(ctx) @@ -98,21 +98,21 @@ func TestTxm_Integration(t *testing.T) { hash.Value.Blockhash, solana.TransactionPayer(signer), ) + lastValidBlockHeight = hash.Value.LastValidBlockHeight require.NoError(t, txErr) return tx } - // enqueue txs (must pass to move on to load test) - require.NoError(t, txm.Enqueue(ctx, "test_success_0", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) - require.Error(t, txm.Enqueue(ctx, "test_invalidSigner", createTx(pubKeyReceiver, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) // cannot sign tx before enqueuing - require.NoError(t, txm.Enqueue(ctx, "test_invalidReceiver", createTx(pubKey, pubKey, solana.PublicKey{}, solana.LAMPORTS_PER_SOL), nil)) + require.NoError(t, txm.Enqueue(ctx, "test_success_0", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil, lastValidBlockHeight)) + require.Error(t, txm.Enqueue(ctx, "test_invalidSigner", createTx(pubKeyReceiver, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil, lastValidBlockHeight)) // cannot sign tx before enqueuing + require.NoError(t, txm.Enqueue(ctx, "test_invalidReceiver", createTx(pubKey, pubKey, solana.PublicKey{}, solana.LAMPORTS_PER_SOL), nil, lastValidBlockHeight)) time.Sleep(500 * time.Millisecond) // pause 0.5s for new blockhash - require.NoError(t, txm.Enqueue(ctx, "test_success_1", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) - require.NoError(t, txm.Enqueue(ctx, "test_txFail", createTx(pubKey, pubKey, pubKeyReceiver, 1000*solana.LAMPORTS_PER_SOL), nil)) + require.NoError(t, txm.Enqueue(ctx, "test_success_1", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil, lastValidBlockHeight)) + require.NoError(t, txm.Enqueue(ctx, "test_txFail", createTx(pubKey, pubKey, pubKeyReceiver, 1000*solana.LAMPORTS_PER_SOL), nil, lastValidBlockHeight)) // load test: try to overload txs, confirm, or simulation for i := 0; i < 1000; i++ { - assert.NoError(t, txm.Enqueue(ctx, fmt.Sprintf("load_%d", i), createTx(loadTestKey.PublicKey(), loadTestKey.PublicKey(), loadTestKey.PublicKey(), uint64(i)), nil)) + assert.NoError(t, txm.Enqueue(ctx, fmt.Sprintf("load_%d", i), createTx(loadTestKey.PublicKey(), loadTestKey.PublicKey(), loadTestKey.PublicKey(), uint64(i)), nil, lastValidBlockHeight)) time.Sleep(10 * time.Millisecond) // ~100 txs per second (note: have run 5ms delays for ~200tx/s successfully) } From 98f0246d9ea91a98b0dcb6aef055de92bf171b3a Mon Sep 17 00:00:00 2001 From: Farber98 Date: Mon, 25 Nov 2024 14:49:30 -0300 Subject: [PATCH 29/51] blockhash assignment inside txm.sendWithRetry --- pkg/solana/chain.go | 2 +- pkg/solana/chain_test.go | 8 +- pkg/solana/relay.go | 2 +- pkg/solana/transmitter.go | 2 +- pkg/solana/transmitter_test.go | 2 +- pkg/solana/txm/txm.go | 43 +++-- pkg/solana/txm/txm_internal_test.go | 255 +++++++++++++++++++--------- pkg/solana/txm/txm_load_test.go | 14 +- pkg/solana/txm/txm_race_test.go | 32 +++- 9 files changed, 245 insertions(+), 115 deletions(-) diff --git a/pkg/solana/chain.go b/pkg/solana/chain.go index 0267e2848..ab901a548 100644 --- a/pkg/solana/chain.go +++ b/pkg/solana/chain.go @@ -571,7 +571,7 @@ func (c *chain) sendTx(ctx context.Context, from, to string, amount *big.Int, ba } chainTxm := c.TxManager() - err = chainTxm.Enqueue(ctx, "", tx, nil, blockhash.Value.LastValidBlockHeight, + err = chainTxm.Enqueue(ctx, "", tx, nil, txm.SetComputeUnitLimit(500), // reduce from default 200K limit - should only take 450 compute units // no fee bumping and no additional fee - makes validating balance accurate txm.SetComputeUnitPriceMax(0), diff --git a/pkg/solana/chain_test.go b/pkg/solana/chain_test.go index 65f803bac..0dd4b0df3 100644 --- a/pkg/solana/chain_test.go +++ b/pkg/solana/chain_test.go @@ -534,13 +534,11 @@ func TestSolanaChain_MultiNode_Txm(t *testing.T) { receiverBal, err := selectedClient.Balance(tests.Context(t), pubKeyReceiver) assert.NoError(t, err) assert.Equal(t, uint64(0), receiverBal) - var lastValidBlockHeight uint64 createTx := func(signer solana.PublicKey, sender solana.PublicKey, receiver solana.PublicKey, amt uint64) *solana.Transaction { selectedClient, err = testChain.getClient() assert.NoError(t, err) hash, hashErr := selectedClient.LatestBlockhash(tests.Context(t)) assert.NoError(t, hashErr) - lastValidBlockHeight = hash.Value.LastValidBlockHeight tx, txErr := solana.NewTransaction( []solana.Instruction{ system.NewTransferInstruction( @@ -557,7 +555,7 @@ func TestSolanaChain_MultiNode_Txm(t *testing.T) { } // Send funds twice, along with an invalid transaction - require.NoError(t, testChain.txm.Enqueue(tests.Context(t), "test_success", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil, lastValidBlockHeight)) + require.NoError(t, testChain.txm.Enqueue(tests.Context(t), "test_success", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) // Wait for new block hash currentBh, err := selectedClient.LatestBlockhash(tests.Context(t)) require.NoError(t, err) @@ -577,8 +575,8 @@ NewBlockHash: } } - require.NoError(t, testChain.txm.Enqueue(tests.Context(t), "test_success_2", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil, lastValidBlockHeight)) - require.Error(t, testChain.txm.Enqueue(tests.Context(t), "test_invalidSigner", createTx(pubKeyReceiver, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil, lastValidBlockHeight)) // cannot sign tx before enqueuing + require.NoError(t, testChain.txm.Enqueue(tests.Context(t), "test_success_2", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) + require.Error(t, testChain.txm.Enqueue(tests.Context(t), "test_invalidSigner", createTx(pubKeyReceiver, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) // cannot sign tx before enqueuing // wait for all txes to finish ctx, cancel := context.WithCancel(tests.Context(t)) t.Cleanup(cancel) diff --git a/pkg/solana/relay.go b/pkg/solana/relay.go index bef9b02c3..d98ab0442 100644 --- a/pkg/solana/relay.go +++ b/pkg/solana/relay.go @@ -24,7 +24,7 @@ import ( var _ TxManager = (*txm.Txm)(nil) type TxManager interface { - Enqueue(ctx context.Context, accountID string, tx *solana.Transaction, txID *string, lastValidBlockHeight uint64, txCfgs ...txm.SetTxConfig) error + Enqueue(ctx context.Context, accountID string, tx *solana.Transaction, txID *string, txCfgs ...txm.SetTxConfig) error } var _ relaytypes.Relayer = &Relayer{} //nolint:staticcheck diff --git a/pkg/solana/transmitter.go b/pkg/solana/transmitter.go index 537c72699..951e9633e 100644 --- a/pkg/solana/transmitter.go +++ b/pkg/solana/transmitter.go @@ -87,7 +87,7 @@ func (c *Transmitter) Transmit( // pass transmit payload to tx manager queue c.lggr.Debugf("Queuing transmit tx: state (%s) + transmissions (%s)", c.stateID.String(), c.transmissionsID.String()) - if err = c.txManager.Enqueue(ctx, c.stateID.String(), tx, nil, blockhash.Value.LastValidBlockHeight); err != nil { + if err = c.txManager.Enqueue(ctx, c.stateID.String(), tx, nil); err != nil { return fmt.Errorf("error on Transmit.txManager.Enqueue: %w", err) } return nil diff --git a/pkg/solana/transmitter_test.go b/pkg/solana/transmitter_test.go index 6924eeef1..d2f0229ca 100644 --- a/pkg/solana/transmitter_test.go +++ b/pkg/solana/transmitter_test.go @@ -27,7 +27,7 @@ type verifyTxSize struct { s *solana.PrivateKey } -func (txm verifyTxSize) Enqueue(_ context.Context, _ string, tx *solana.Transaction, txID *string, _ uint64, _ ...txm.SetTxConfig) error { +func (txm verifyTxSize) Enqueue(_ context.Context, _ string, tx *solana.Transaction, txID *string, _ ...txm.SetTxConfig) error { // additional components that transaction manager adds to the transaction require.NoError(txm.t, fees.SetComputeUnitPrice(tx, 0)) require.NoError(txm.t, fees.SetComputeUnitLimit(tx, 0)) diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index 0dc12a2c3..dd1df171d 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -180,9 +180,26 @@ func (txm *Txm) run() { } // sendWithRetry attempts to send a transaction with exponential backoff retry logic. -// It prepares the transaction, builds and signs it, sends the initial transaction, and starts a retry routine with fee bumping if needed. +// It builds, signs and sends the initial tx with a new valid blockhash, and starts a retry routine with fee bumping if needed. // The function returns the signed transaction, its ID, and the initial signature for use in simulation. func (txm *Txm) sendWithRetry(ctx context.Context, msg pendingTx) (solanaGo.Transaction, string, solanaGo.Signature, error) { + // Assign new blockhash and lastValidBlockHeight to the transaction + // This is essential for tracking transaction rebroadcast + // Only the initial transaction should be sent with the updated blockhash + client, err := txm.client.Get() + if err != nil { + return solanaGo.Transaction{}, "", solanaGo.Signature{}, fmt.Errorf("failed to get client: %w", err) + } + blockhash, err := client.LatestBlockhash(ctx) + if err != nil { + return solanaGo.Transaction{}, "", solanaGo.Signature{}, fmt.Errorf("failed to get latest blockhash: %w", err) + } + if blockhash == nil || blockhash.Value == nil { + return solanaGo.Transaction{}, "", solanaGo.Signature{}, errors.New("nil pointer returned from LatestBlockhash") + } + msg.tx.Message.RecentBlockhash = blockhash.Value.Blockhash + msg.lastValidBlockHeight = blockhash.Value.LastValidBlockHeight + // Build and sign initial transaction setting compute unit price and limit initTx, err := txm.buildTx(ctx, msg, 0) if err != nil { @@ -554,11 +571,6 @@ func (txm *Txm) rebroadcastExpiredTxs(ctx context.Context, client client.ReaderW txm.lggr.Errorw("failed to get current slot height", "error", err) return } - blockhash, err := client.LatestBlockhash(ctx) - if err != nil { - txm.lggr.Errorw("failed to get blockhash", "error", err) - return - } // Rebroadcast all expired txes for _, tx := range txm.txs.ListAllExpiredBroadcastedTxs(currHeight) { txm.lggr.Debugw("transaction expired, rebroadcasting", "id", tx.id, "signature", tx.signatures) @@ -571,17 +583,13 @@ func (txm *Txm) rebroadcastExpiredTxs(ctx context.Context, client client.ReaderW txm.lggr.Errorw("failed to remove expired transaction", "id", tx.id, "error", err) continue } - // Overwrite blockhash and lastValidBlockHeight with latest values so that the transaction can be rebroadcasted an accepted. - tx.tx.Message.RecentBlockhash = blockhash.Value.Blockhash rebroadcastTx := pendingTx{ - tx: tx.tx, - cfg: tx.cfg, - id: tx.id, - rebroadcastCount: tx.rebroadcastCount + 1, - lastValidBlockHeight: blockhash.Value.LastValidBlockHeight, + tx: tx.tx, + cfg: tx.cfg, + id: tx.id, // using same id in case it was set by caller and we need to maintain it. + rebroadcastCount: tx.rebroadcastCount + 1, } // call sendWithRetry directly to avoid enqueuing - // using same id in case it was set by caller and we need to maintain it. _, _, _, err = txm.sendWithRetry(ctx, rebroadcastTx) if err != nil { // TODO: add prebroadcast error handling when merged https://github.com/smartcontractkit/chainlink-solana/pull/936 @@ -650,7 +658,7 @@ func (txm *Txm) reap() { } // Enqueue enqueues a msg destined for the solana chain. -func (txm *Txm) Enqueue(ctx context.Context, accountID string, tx *solanaGo.Transaction, txID *string, lastValidBlockHeight uint64, txCfgs ...SetTxConfig) error { +func (txm *Txm) Enqueue(ctx context.Context, accountID string, tx *solanaGo.Transaction, txID *string, txCfgs ...SetTxConfig) error { if err := txm.Ready(); err != nil { return fmt.Errorf("error in soltxm.Enqueue: %w", err) } @@ -698,9 +706,8 @@ func (txm *Txm) Enqueue(ctx context.Context, accountID string, tx *solanaGo.Tran } msg := pendingTx{ - tx: *tx, - lastValidBlockHeight: lastValidBlockHeight, - cfg: cfg, + tx: *tx, + cfg: cfg, } // If ID was not set by caller, create one. diff --git a/pkg/solana/txm/txm_internal_test.go b/pkg/solana/txm/txm_internal_test.go index c5f1caff6..4c51ce603 100644 --- a/pkg/solana/txm/txm_internal_test.go +++ b/pkg/solana/txm/txm_internal_test.go @@ -128,7 +128,6 @@ func TestTxm(t *testing.T) { mc := mocks.NewReaderWriter(t) mc.On("GetLatestBlock", mock.Anything).Return(&rpc.GetBlockResult{}, nil).Maybe() mc.On("SlotHeight", mock.Anything).Return(uint64(0), nil).Maybe() - lastValidBlockHeight := uint64(100) // higher than slotHeight so it's valid // mock solana keystore mkey := keyMocks.NewSimpleKeystore(t) @@ -165,6 +164,12 @@ func TestTxm(t *testing.T) { // happy path (send => simulate success => tx: nil => tx: processed => tx: confirmed => finalized => done) t.Run("happyPath", func(t *testing.T) { + mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: 100, + Blockhash: solana.Hash{}, + }, + }, nil).Once() sig := randomSignature(t) tx, signed := getTx(t, 0, mkey) var wg sync.WaitGroup @@ -205,7 +210,7 @@ func TestTxm(t *testing.T) { // send tx testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) wg.Wait() // no transactions stored inflight txs list @@ -230,6 +235,12 @@ func TestTxm(t *testing.T) { // fail on initial transmit (RPC immediate rejects) t.Run("fail_initialTx", func(t *testing.T) { + mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: 100, + Blockhash: solana.Hash{}, + }, + }, nil).Once() tx, signed := getTx(t, 1, mkey) var wg sync.WaitGroup wg.Add(1) @@ -241,7 +252,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) wg.Wait() // wait to be picked up and processed // no transactions stored inflight txs list @@ -257,11 +268,16 @@ func TestTxm(t *testing.T) { }) // tx fails simulation (simulation error) t.Run("fail_simulation", func(t *testing.T) { + mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: 100, + Blockhash: solana.Hash{}, + }, + }, nil).Once() tx, signed := getTx(t, 2, mkey) sig := randomSignature(t) var wg sync.WaitGroup wg.Add(1) - mc.On("SendTx", mock.Anything, signed(0, true, computeUnitLimitDefault)).Return(sig, nil) mc.On("SimulateTx", mock.Anything, signed(0, true, computeUnitLimitDefault), mock.Anything).Run(func(mock.Arguments) { wg.Done() @@ -272,7 +288,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared quickly @@ -287,6 +303,12 @@ func TestTxm(t *testing.T) { // tx fails simulation (rpc error, timeout should clean up b/c sig status will be nil) t.Run("fail_simulation_confirmNil", func(t *testing.T) { + mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: 100, + Blockhash: solana.Hash{}, + }, + }, nil).Once() tx, signed := getTx(t, 3, mkey) sig := randomSignature(t) retry0 := randomSignature(t) @@ -295,7 +317,6 @@ func TestTxm(t *testing.T) { retry3 := randomSignature(t) var wg sync.WaitGroup wg.Add(1) - mc.On("SendTx", mock.Anything, signed(0, true, computeUnitLimitDefault)).Return(sig, nil) mc.On("SendTx", mock.Anything, signed(1, true, computeUnitLimitDefault)).Return(retry0, nil) mc.On("SendTx", mock.Anything, signed(2, true, computeUnitLimitDefault)).Return(retry1, nil) @@ -308,7 +329,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared after timeout @@ -327,6 +348,12 @@ func TestTxm(t *testing.T) { // tx fails simulation with an InstructionError (indicates reverted execution) // manager should cancel sending retry immediately + increment reverted prom metric t.Run("fail_simulation_instructionError", func(t *testing.T) { + mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: 100, + Blockhash: solana.Hash{}, + }, + }, nil).Once() tx, signed := getTx(t, 4, mkey) sig := randomSignature(t) var wg sync.WaitGroup @@ -348,7 +375,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared after timeout @@ -366,11 +393,16 @@ func TestTxm(t *testing.T) { // tx fails simulation with BlockHashNotFound error // txm should continue to finalize tx (in this case it will succeed) t.Run("fail_simulation_blockhashNotFound", func(t *testing.T) { + mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: 100, + Blockhash: solana.Hash{}, + }, + }, nil).Once() tx, signed := getTx(t, 5, mkey) sig := randomSignature(t) var wg sync.WaitGroup wg.Add(2) - mc.On("SendTx", mock.Anything, signed(0, true, computeUnitLimitDefault)).Return(sig, nil) mc.On("SimulateTx", mock.Anything, signed(0, true, computeUnitLimitDefault), mock.Anything).Run(func(mock.Arguments) { wg.Done() @@ -398,7 +430,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared after timeout @@ -417,11 +449,16 @@ func TestTxm(t *testing.T) { // tx fails simulation with AlreadyProcessed error // txm should continue to confirm tx (in this case it will revert) t.Run("fail_simulation_alreadyProcessed", func(t *testing.T) { + mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: 100, + Blockhash: solana.Hash{}, + }, + }, nil).Once() tx, signed := getTx(t, 6, mkey) sig := randomSignature(t) var wg sync.WaitGroup wg.Add(2) - mc.On("SendTx", mock.Anything, signed(0, true, computeUnitLimitDefault)).Return(sig, nil) mc.On("SimulateTx", mock.Anything, signed(0, true, computeUnitLimitDefault), mock.Anything).Run(func(mock.Arguments) { wg.Done() @@ -440,7 +477,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared after timeout @@ -458,6 +495,12 @@ func TestTxm(t *testing.T) { // tx passes sim, never passes processed (timeout should cleanup) t.Run("fail_confirm_processed", func(t *testing.T) { + mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: 100, + Blockhash: solana.Hash{}, + }, + }, nil).Once() tx, signed := getTx(t, 7, mkey) sig := randomSignature(t) retry0 := randomSignature(t) @@ -466,7 +509,6 @@ func TestTxm(t *testing.T) { retry3 := randomSignature(t) var wg sync.WaitGroup wg.Add(1) - mc.On("SendTx", mock.Anything, signed(0, true, computeUnitLimitDefault)).Return(sig, nil) mc.On("SendTx", mock.Anything, signed(1, true, computeUnitLimitDefault)).Return(retry0, nil) mc.On("SendTx", mock.Anything, signed(2, true, computeUnitLimitDefault)).Return(retry1, nil) @@ -485,7 +527,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // inflight txs cleared after timeout @@ -503,6 +545,12 @@ func TestTxm(t *testing.T) { // tx passes sim, shows processed, moves to nil (timeout should cleanup) t.Run("fail_confirm_processedToNil", func(t *testing.T) { + mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: 100, + Blockhash: solana.Hash{}, + }, + }, nil).Once() tx, signed := getTx(t, 8, mkey) sig := randomSignature(t) retry0 := randomSignature(t) @@ -511,7 +559,6 @@ func TestTxm(t *testing.T) { retry3 := randomSignature(t) var wg sync.WaitGroup wg.Add(1) - mc.On("SendTx", mock.Anything, signed(0, true, computeUnitLimitDefault)).Return(sig, nil) mc.On("SendTx", mock.Anything, signed(1, true, computeUnitLimitDefault)).Return(retry0, nil) mc.On("SendTx", mock.Anything, signed(2, true, computeUnitLimitDefault)).Return(retry1, nil) @@ -537,7 +584,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // inflight txs cleared after timeout @@ -555,11 +602,16 @@ func TestTxm(t *testing.T) { // tx passes sim, errors on confirm t.Run("fail_confirm_revert", func(t *testing.T) { + mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: 100, + Blockhash: solana.Hash{}, + }, + }, nil).Once() tx, signed := getTx(t, 9, mkey) sig := randomSignature(t) var wg sync.WaitGroup wg.Add(1) - mc.On("SendTx", mock.Anything, signed(0, true, computeUnitLimitDefault)).Return(sig, nil) mc.On("SimulateTx", mock.Anything, signed(0, true, computeUnitLimitDefault), mock.Anything).Run(func(mock.Arguments) { wg.Done() @@ -575,7 +627,7 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // inflight txs cleared after timeout @@ -593,6 +645,12 @@ func TestTxm(t *testing.T) { // tx passes sim, first retried TXs get dropped t.Run("success_retryTx", func(t *testing.T) { + mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: 100, + Blockhash: solana.Hash{}, + }, + }, nil).Once() tx, signed := getTx(t, 10, mkey) sig := randomSignature(t) retry0 := randomSignature(t) @@ -601,7 +659,6 @@ func TestTxm(t *testing.T) { retry3 := randomSignature(t) var wg sync.WaitGroup wg.Add(2) - mc.On("SendTx", mock.Anything, signed(0, true, computeUnitLimitDefault)).Return(sig, nil) mc.On("SendTx", mock.Anything, signed(1, true, computeUnitLimitDefault)).Return(retry0, nil) mc.On("SendTx", mock.Anything, signed(2, true, computeUnitLimitDefault)).Return(retry1, nil) @@ -621,7 +678,7 @@ func TestTxm(t *testing.T) { // send tx testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) wg.Wait() // no transactions stored inflight txs list @@ -640,6 +697,12 @@ func TestTxm(t *testing.T) { // fee bumping disabled t.Run("feeBumpingDisabled", func(t *testing.T) { + mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: 100, + Blockhash: solana.Hash{}, + }, + }, nil).Once() sig := randomSignature(t) tx, signed := getTx(t, 11, mkey) @@ -675,7 +738,7 @@ func TestTxm(t *testing.T) { // send tx - with disabled fee bumping testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight, SetFeeBumpPeriod(0))) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, SetFeeBumpPeriod(0))) wg.Wait() // no transactions stored inflight txs list @@ -700,6 +763,12 @@ func TestTxm(t *testing.T) { // compute unit limit disabled t.Run("computeUnitLimitDisabled", func(t *testing.T) { + mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: 100, + Blockhash: solana.Hash{}, + }, + }, nil).Once() sig := randomSignature(t) tx, signed := getTx(t, 12, mkey) @@ -727,7 +796,7 @@ func TestTxm(t *testing.T) { // send tx - with disabled fee bumping and disabled compute unit limit testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight, SetFeeBumpPeriod(0), SetComputeUnitLimit(0))) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, SetFeeBumpPeriod(0), SetComputeUnitLimit(0))) wg.Wait() // no transactions stored inflight txs list @@ -767,7 +836,6 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { mc := mocks.NewReaderWriter(t) mc.On("GetLatestBlock", mock.Anything).Return(&rpc.GetBlockResult{}, nil).Maybe() mc.On("SlotHeight", mock.Anything).Return(uint64(0), nil).Maybe() - lastValidBlockHeight := uint64(100) // higher sthan slotHeight so it's valid. computeUnitLimitDefault := fees.ComputeUnitLimit(cfg.ComputeUnitLimitDefault()) @@ -799,23 +867,26 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { return out }, nil, ) - - t.Run("happyPath", func(t *testing.T) { - // Test tx is not discarded due to confirm timeout and tracked to finalization - // use unique val across tests to avoid collision during mocking - tx, signed := getTx(t, 1, mkey) - sig := randomSignature(t) - retry0 := randomSignature(t) - retry1 := randomSignature(t) - var wg sync.WaitGroup - wg.Add(2) - - mc.On("SendTx", mock.Anything, signed(0, true, computeUnitLimitDefault)).Return(sig, nil) - mc.On("SendTx", mock.Anything, signed(1, true, computeUnitLimitDefault)).Return(retry0, nil).Maybe() - mc.On("SendTx", mock.Anything, signed(2, true, computeUnitLimitDefault)).Return(retry1, nil).Maybe() - mc.On("SimulateTx", mock.Anything, signed(0, true, computeUnitLimitDefault), mock.Anything).Run(func(mock.Arguments) { - wg.Done() - }).Return(&rpc.SimulateTransactionResult{}, nil).Once() + mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: 100, + Blockhash: solana.Hash{}, + }, + }, nil).Once() + + // Test tx is not discarded due to confirm timeout and tracked to finalization + tx, signed := getTx(t, 7, mkey) + sig := randomSignature(t) + retry0 := randomSignature(t) + retry1 := randomSignature(t) + var wg sync.WaitGroup + wg.Add(2) + mc.On("SendTx", mock.Anything, signed(0, true, computeUnitLimitDefault)).Return(sig, nil) + mc.On("SendTx", mock.Anything, signed(1, true, computeUnitLimitDefault)).Return(retry0, nil).Maybe() + mc.On("SendTx", mock.Anything, signed(2, true, computeUnitLimitDefault)).Return(retry1, nil).Maybe() + mc.On("SimulateTx", mock.Anything, signed(0, true, computeUnitLimitDefault), mock.Anything).Run(func(mock.Arguments) { + wg.Done() + }).Return(&rpc.SimulateTransactionResult{}, nil).Once() // handle signature status calls (initial stays processed, others don't exist) start := time.Now() @@ -836,11 +907,11 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { return } - // tx should be able to queue - testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) - wg.Wait() // wait to be picked up and processed - waitFor(t, 5*time.Second, txm, prom, empty) // inflight txs cleared after timeout + // tx should be able to queue + testTxID := uuid.New().String() + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + wg.Wait() // wait to be picked up and processed + waitFor(t, 5*time.Second, txm, prom, empty) // inflight txs cleared after timeout // panic if sendTx called after context cancelled mc.On("SendTx", mock.Anything, tx).Panic("SendTx should not be called anymore").Maybe() @@ -969,7 +1040,6 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { mc := mocks.NewReaderWriter(t) mc.On("GetLatestBlock", mock.Anything).Return(&rpc.GetBlockResult{}, nil).Maybe() mc.On("SlotHeight", mock.Anything).Return(uint64(0), nil).Maybe() - lastValidBlockHeight := uint64(100) // higher sthan slotHeight so it's valid. // mock solana keystore mkey := keyMocks.NewSimpleKeystore(t) @@ -1001,6 +1071,12 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { ) t.Run("simulation_succeeds", func(t *testing.T) { + mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: 100, + Blockhash: solana.Hash{}, + }, + }, nil).Once() // Test tx is not discarded due to confirm timeout and tracked to finalization // use unique val across tests to avoid collision during mocking tx, signed := getTx(t, 1, mkey) @@ -1045,7 +1121,7 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { // send tx testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) wg.Wait() // no transactions stored inflight txs list @@ -1074,7 +1150,7 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { mc.On("SimulateTx", mock.Anything, mock.Anything, mock.Anything).Return(nil, errors.New("simulation failed")).Once() // tx should NOT be able to queue - assert.Error(t, txm.Enqueue(ctx, t.Name(), tx, nil, lastValidBlockHeight)) + assert.Error(t, txm.Enqueue(ctx, t.Name(), tx, nil)) }) t.Run("simulation_returns_error", func(t *testing.T) { @@ -1090,7 +1166,7 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { txID := uuid.NewString() // tx should NOT be able to queue - assert.Error(t, txm.Enqueue(ctx, t.Name(), tx, nil, lastValidBlockHeight)) + assert.Error(t, txm.Enqueue(ctx, t.Name(), tx, nil)) }) } @@ -1099,6 +1175,12 @@ func TestTxm_Enqueue(t *testing.T) { lggr := logger.Test(t) cfg := config.NewDefault() mc := mocks.NewReaderWriter(t) + mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: 100, + Blockhash: solana.Hash{}, + }, + }, nil).Once() mc.On("SendTx", mock.Anything, mock.Anything).Return(solana.Signature{}, nil).Maybe() mc.On("SimulateTx", mock.Anything, mock.Anything, mock.Anything).Return(&rpc.SimulateTransactionResult{}, nil).Maybe() mc.On("SignatureStatuses", mock.Anything, mock.AnythingOfType("[]solana.Signature")).Return( @@ -1117,7 +1199,6 @@ func TestTxm_Enqueue(t *testing.T) { invalidKey := solana.PublicKeyFromBytes([]byte{2}) mkey.On("Sign", mock.Anything, validKey.String(), mock.Anything).Return([]byte{1}, nil) mkey.On("Sign", mock.Anything, invalidKey.String(), mock.Anything).Return([]byte{}, relayconfig.KeyNotFoundError{ID: invalidKey.String(), KeyType: "Solana"}) - lastValidBlockHeight := uint64(100) // build txs tx, err := solana.NewTransaction( @@ -1149,7 +1230,7 @@ func TestTxm_Enqueue(t *testing.T) { loader := utils.NewLazyLoad(func() (client.ReaderWriter, error) { return mc, nil }) txm := NewTxm("enqueue_test", loader, nil, cfg, mkey, lggr) - require.ErrorContains(t, txm.Enqueue(ctx, "txmUnstarted", &solana.Transaction{}, nil, lastValidBlockHeight), "not started") + require.ErrorContains(t, txm.Enqueue(ctx, "txmUnstarted", &solana.Transaction{}, nil), "not started") require.NoError(t, txm.Start(ctx)) t.Cleanup(func() { require.NoError(t, txm.Close()) }) @@ -1167,10 +1248,10 @@ func TestTxm_Enqueue(t *testing.T) { for _, run := range txs { t.Run(run.name, func(t *testing.T) { if !run.fail { - assert.NoError(t, txm.Enqueue(ctx, run.name, run.tx, nil, lastValidBlockHeight)) + assert.NoError(t, txm.Enqueue(ctx, run.name, run.tx, nil)) return } - assert.Error(t, txm.Enqueue(ctx, run.name, run.tx, nil, lastValidBlockHeight)) + assert.Error(t, txm.Enqueue(ctx, run.name, run.tx, nil)) }) } } @@ -1271,8 +1352,18 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { return uint64(1500), nil } - // Mock LatestBlockhash to return a valid blockhash greater than slotHeight + callCount := 0 latestBlockhashFunc := func() (*rpc.GetLatestBlockhashResult, error) { + defer func() { callCount++ }() + if callCount < 1 { + // To force rebroadcast, first call needs to be smaller than slotHeight + return &rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: uint64(1000), + }, + }, nil + } + // following rebroadcast call will go through because lastValidBlockHeight is bigger than slotHeight return &rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ LastValidBlockHeight: uint64(2000), @@ -1317,7 +1408,7 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { tx, _ := getTx(t, 0, mkey) txID := "test-rebroadcast" - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID, 0)) // Will create an expired transaction as lastValidBlockHeight is 0. This will force the rebroadcast. + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) wg.Wait() time.Sleep(2 * time.Second) // Sleep to allow for rebroadcasting @@ -1334,6 +1425,15 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { txExpirationRebroadcast := false statuses := map[solana.Signature]func() *rpc.SignatureStatusesResult{} + // mocking the call within sendWithRetry. Rebroadcast is off, so we won't compare it against the slotHeight. + latestBlockhashFunc := func() (*rpc.GetLatestBlockhashResult, error) { + return &rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: uint64(2000), + }, + }, nil + } + sig1 := randomSignature(t) sendTxFunc := func() (solana.Signature, error) { return sig1, nil @@ -1351,12 +1451,12 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { return nil } } - // No LatestBlockhash nor slotHeight needed because there's no rebroadcast. - txm, _, mkey := setupTxmTest(txExpirationRebroadcast, nil, nil, sendTxFunc, statuses) + + txm, _, mkey := setupTxmTest(txExpirationRebroadcast, latestBlockhashFunc, nil, sendTxFunc, statuses) tx, _ := getTx(t, 5, mkey) txID := "test-no-rebroadcast" - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID, 0)) // Won't rebroadcast as txExpirationRebroadcast is false. + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) wg.Wait() time.Sleep(2 * time.Second) // Sleep to ensure no rebroadcast @@ -1371,18 +1471,15 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { t.Run("WithMultipleRebroadcast", func(t *testing.T) { txExpirationRebroadcast := true - expectedRebroadcastsCount := 3 statuses := map[solana.Signature]func() *rpc.SignatureStatusesResult{} - // Mock SlotHeight to return a value greater than 0 slotHeightFunc := func() (uint64, error) { return uint64(1500), nil } - - // Mock LatestBlockhash to return a invalid blockhash first 2 rebroadcast attempts - // will return a valid blockhash third rebroadcast attempt. - // the third one is valid because it is greater than the slotHeight - callCount := 1 + // Mock LatestBlockhash to return a invalid blockhash in the first 3 attempts (initial + 2 rebroadcasts) + // the last one is valid because it is greater than the slotHeight + expectedRebroadcastsCount := 3 + callCount := 0 latestBlockhashFunc := func() (*rpc.GetLatestBlockhashResult, error) { defer func() { callCount++ }() if callCount < expectedRebroadcastsCount { @@ -1434,7 +1531,7 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { txm, _, mkey := setupTxmTest(txExpirationRebroadcast, latestBlockhashFunc, slotHeightFunc, sendTxFunc, statuses) tx, _ := getTx(t, 0, mkey) txID := "test-rebroadcast" - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID, 0)) // lastValidBlockHeight is 0 to force rebroadcast. + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) wg.Wait() time.Sleep(2 * time.Second) // Sleep to allow for rebroadcasting @@ -1455,13 +1552,11 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { return sig1, nil } - // Mock SlotHeight to return a value greater than 0 + // Mock LatestBlockhash to return an invalid blockhash less than slotHeight + // We won't use it as there will be no rebroadcasts txes to process. All txes will be confirmed before. slotHeightFunc := func() (uint64, error) { return uint64(1500), nil } - - // Mock LatestBlockhash to return an invalid blockhash less than slotHeight - // We won't use it as there are no rebroadcasts to process. Just to test that we don't need it. latestBlockhashFunc := func() (*rpc.GetLatestBlockhashResult, error) { return &rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ @@ -1493,7 +1588,7 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { txm, _, mkey := setupTxmTest(txExpirationRebroadcast, latestBlockhashFunc, slotHeightFunc, sendTxFunc, statuses) tx, _ := getTx(t, 0, mkey) txID := "test-confirmed-before-rebroadcast" - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID, 0)) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) wg.Wait() time.Sleep(1 * time.Second) // Allow for processing @@ -1510,13 +1605,21 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { txExpirationRebroadcast := true statuses := map[solana.Signature]func() *rpc.SignatureStatusesResult{} - // Mock SlotHeight to return a value greater than 0 + // To force rebroadcast, first call needs to be smaller than slotHeight + // following rebroadcast call will go through because lastValidBlockHeight will be bigger than slotHeight slotHeightFunc := func() (uint64, error) { return uint64(1500), nil } - - // Mock LatestBlockhash to return a valid blockhash greater than slotHeight + callCount := 0 latestBlockhashFunc := func() (*rpc.GetLatestBlockhashResult, error) { + defer func() { callCount++ }() + if callCount < 1 { + return &rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: uint64(1000), + }, + }, nil + } return &rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ LastValidBlockHeight: uint64(2000), @@ -1537,8 +1640,8 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { txm, _, mkey := setupTxmTest(txExpirationRebroadcast, latestBlockhashFunc, slotHeightFunc, sendTxFunc, statuses) tx, _ := getTx(t, 0, mkey) txID := "test-rebroadcast-error" - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID, 0)) // lastValidBlockHeight is 0 to force rebroadcast. - time.Sleep(2 * time.Second) // Allow for processing + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) + time.Sleep(2 * time.Second) // Allow for processing // TODO: Add check that transaction status is failed due to rebroadcast error when prebroadcast is implemented and we have an error in sendWithRetry status, err := txm.GetTransactionStatus(ctx, txID) diff --git a/pkg/solana/txm/txm_load_test.go b/pkg/solana/txm/txm_load_test.go index b1b2398c9..3d4941374 100644 --- a/pkg/solana/txm/txm_load_test.go +++ b/pkg/solana/txm/txm_load_test.go @@ -82,7 +82,6 @@ func TestTxm_Integration(t *testing.T) { // already started assert.Error(t, txm.Start(ctx)) - var lastValidBlockHeight uint64 createTx := func(signer solana.PublicKey, sender solana.PublicKey, receiver solana.PublicKey, amt uint64) *solana.Transaction { // create transfer tx hash, err := client.LatestBlockhash(ctx) @@ -98,21 +97,20 @@ func TestTxm_Integration(t *testing.T) { hash.Value.Blockhash, solana.TransactionPayer(signer), ) - lastValidBlockHeight = hash.Value.LastValidBlockHeight require.NoError(t, txErr) return tx } // enqueue txs (must pass to move on to load test) - require.NoError(t, txm.Enqueue(ctx, "test_success_0", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil, lastValidBlockHeight)) - require.Error(t, txm.Enqueue(ctx, "test_invalidSigner", createTx(pubKeyReceiver, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil, lastValidBlockHeight)) // cannot sign tx before enqueuing - require.NoError(t, txm.Enqueue(ctx, "test_invalidReceiver", createTx(pubKey, pubKey, solana.PublicKey{}, solana.LAMPORTS_PER_SOL), nil, lastValidBlockHeight)) + require.NoError(t, txm.Enqueue(ctx, "test_success_0", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) + require.Error(t, txm.Enqueue(ctx, "test_invalidSigner", createTx(pubKeyReceiver, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) // cannot sign tx before enqueuing + require.NoError(t, txm.Enqueue(ctx, "test_invalidReceiver", createTx(pubKey, pubKey, solana.PublicKey{}, solana.LAMPORTS_PER_SOL), nil)) time.Sleep(500 * time.Millisecond) // pause 0.5s for new blockhash - require.NoError(t, txm.Enqueue(ctx, "test_success_1", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil, lastValidBlockHeight)) - require.NoError(t, txm.Enqueue(ctx, "test_txFail", createTx(pubKey, pubKey, pubKeyReceiver, 1000*solana.LAMPORTS_PER_SOL), nil, lastValidBlockHeight)) + require.NoError(t, txm.Enqueue(ctx, "test_success_1", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) + require.NoError(t, txm.Enqueue(ctx, "test_txFail", createTx(pubKey, pubKey, pubKeyReceiver, 1000*solana.LAMPORTS_PER_SOL), nil)) // load test: try to overload txs, confirm, or simulation for i := 0; i < 1000; i++ { - assert.NoError(t, txm.Enqueue(ctx, fmt.Sprintf("load_%d", i), createTx(loadTestKey.PublicKey(), loadTestKey.PublicKey(), loadTestKey.PublicKey(), uint64(i)), nil, lastValidBlockHeight)) + assert.NoError(t, txm.Enqueue(ctx, fmt.Sprintf("load_%d", i), createTx(loadTestKey.PublicKey(), loadTestKey.PublicKey(), loadTestKey.PublicKey(), uint64(i)), nil)) time.Sleep(10 * time.Millisecond) // ~100 txs per second (note: have run 5ms delays for ~200tx/s successfully) } diff --git a/pkg/solana/txm/txm_race_test.go b/pkg/solana/txm/txm_race_test.go index ea175e63b..6800d9557 100644 --- a/pkg/solana/txm/txm_race_test.go +++ b/pkg/solana/txm/txm_race_test.go @@ -8,7 +8,9 @@ import ( "testing" "time" + "github.com/gagliardetto/solana-go" solanaGo "github.com/gagliardetto/solana-go" + "github.com/gagliardetto/solana-go/rpc" "go.uber.org/zap/zapcore" "github.com/smartcontractkit/chainlink-common/pkg/logger" @@ -83,7 +85,12 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { t.Run("delay in rebroadcasting tx", func(t *testing.T) { client := clientmocks.NewReaderWriter(t) - // client mock + client.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: 100, + Blockhash: solana.Hash{}, + }, + }, nil).Once() txs := map[string]solanaGo.Signature{} var lock sync.RWMutex client.On("SendTx", mock.Anything, mock.Anything).Return( @@ -121,7 +128,12 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { t.Run("delay in broadcasting new tx", func(t *testing.T) { client := clientmocks.NewReaderWriter(t) - // client mock + client.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: 100, + Blockhash: solana.Hash{}, + }, + }, nil).Once() txs := map[string]solanaGo.Signature{} var lock sync.RWMutex client.On("SendTx", mock.Anything, mock.Anything).Return( @@ -157,7 +169,12 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { t.Run("overlapping bumping tx", func(t *testing.T) { client := clientmocks.NewReaderWriter(t) - // client mock + client.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: 100, + Blockhash: solana.Hash{}, + }, + }, nil).Once() txs := map[string]solanaGo.Signature{} var lock sync.RWMutex client.On("SendTx", mock.Anything, mock.Anything).Return( @@ -204,7 +221,14 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { t.Run("bumping tx errors and ctx cleans up waitgroup blocks", func(t *testing.T) { client := clientmocks.NewReaderWriter(t) - // client mock - first tx is always successful + client.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: 100, + Blockhash: solana.Hash{}, + }, + }, nil).Once() + + // first tx is always successful msg0 := NewTestMsg() require.NoError(t, fees.SetComputeUnitPrice(&msg0.tx, 0)) require.NoError(t, fees.SetComputeUnitLimit(&msg0.tx, 200_000)) From cbf55f69bb3a77a2dbbd22c50a93bb15c578262a Mon Sep 17 00:00:00 2001 From: Farber98 Date: Tue, 26 Nov 2024 14:16:44 -0300 Subject: [PATCH 30/51] address feedback --- pkg/solana/txm/txm.go | 3 +- pkg/solana/txm/txm_internal_test.go | 157 ++++++++++------------------ pkg/solana/txm/txm_race_test.go | 79 +++++--------- 3 files changed, 88 insertions(+), 151 deletions(-) diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index dd1df171d..b48556dab 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -506,7 +506,8 @@ func (txm *Txm) handleNotFoundSignatureStatus(sig solanaGo.Signature) { // Otherwise, it marks the transaction as errored. func (txm *Txm) handleErrorSignatureStatus(sig solanaGo.Signature, status *rpc.SignatureStatusesResult) { // We want to rebroadcast rather than drop tx if expiration rebroadcast is enabled when blockhash was not found. - if status.Err != nil && status.Err == client.ErrBlockhashNotFound && txm.cfg.TxExpirationRebroadcast() { + // converting error to string so we are able to check if it contains the error message. + if status.Err != nil && strings.Contains(fmt.Sprintf("%v", status.Err), "BlockhashNotFound") && txm.cfg.TxExpirationRebroadcast() { return } diff --git a/pkg/solana/txm/txm_internal_test.go b/pkg/solana/txm/txm_internal_test.go index 4c51ce603..785d30b81 100644 --- a/pkg/solana/txm/txm_internal_test.go +++ b/pkg/solana/txm/txm_internal_test.go @@ -161,15 +161,14 @@ func TestTxm(t *testing.T) { return out }, nil, ) - + mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: 100, + Blockhash: solana.Hash{}, + }, + }, nil) // happy path (send => simulate success => tx: nil => tx: processed => tx: confirmed => finalized => done) t.Run("happyPath", func(t *testing.T) { - mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: 100, - Blockhash: solana.Hash{}, - }, - }, nil).Once() sig := randomSignature(t) tx, signed := getTx(t, 0, mkey) var wg sync.WaitGroup @@ -235,12 +234,6 @@ func TestTxm(t *testing.T) { // fail on initial transmit (RPC immediate rejects) t.Run("fail_initialTx", func(t *testing.T) { - mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: 100, - Blockhash: solana.Hash{}, - }, - }, nil).Once() tx, signed := getTx(t, 1, mkey) var wg sync.WaitGroup wg.Add(1) @@ -268,12 +261,6 @@ func TestTxm(t *testing.T) { }) // tx fails simulation (simulation error) t.Run("fail_simulation", func(t *testing.T) { - mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: 100, - Blockhash: solana.Hash{}, - }, - }, nil).Once() tx, signed := getTx(t, 2, mkey) sig := randomSignature(t) var wg sync.WaitGroup @@ -303,12 +290,6 @@ func TestTxm(t *testing.T) { // tx fails simulation (rpc error, timeout should clean up b/c sig status will be nil) t.Run("fail_simulation_confirmNil", func(t *testing.T) { - mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: 100, - Blockhash: solana.Hash{}, - }, - }, nil).Once() tx, signed := getTx(t, 3, mkey) sig := randomSignature(t) retry0 := randomSignature(t) @@ -348,12 +329,6 @@ func TestTxm(t *testing.T) { // tx fails simulation with an InstructionError (indicates reverted execution) // manager should cancel sending retry immediately + increment reverted prom metric t.Run("fail_simulation_instructionError", func(t *testing.T) { - mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: 100, - Blockhash: solana.Hash{}, - }, - }, nil).Once() tx, signed := getTx(t, 4, mkey) sig := randomSignature(t) var wg sync.WaitGroup @@ -393,12 +368,6 @@ func TestTxm(t *testing.T) { // tx fails simulation with BlockHashNotFound error // txm should continue to finalize tx (in this case it will succeed) t.Run("fail_simulation_blockhashNotFound", func(t *testing.T) { - mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: 100, - Blockhash: solana.Hash{}, - }, - }, nil).Once() tx, signed := getTx(t, 5, mkey) sig := randomSignature(t) var wg sync.WaitGroup @@ -449,12 +418,6 @@ func TestTxm(t *testing.T) { // tx fails simulation with AlreadyProcessed error // txm should continue to confirm tx (in this case it will revert) t.Run("fail_simulation_alreadyProcessed", func(t *testing.T) { - mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: 100, - Blockhash: solana.Hash{}, - }, - }, nil).Once() tx, signed := getTx(t, 6, mkey) sig := randomSignature(t) var wg sync.WaitGroup @@ -495,12 +458,6 @@ func TestTxm(t *testing.T) { // tx passes sim, never passes processed (timeout should cleanup) t.Run("fail_confirm_processed", func(t *testing.T) { - mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: 100, - Blockhash: solana.Hash{}, - }, - }, nil).Once() tx, signed := getTx(t, 7, mkey) sig := randomSignature(t) retry0 := randomSignature(t) @@ -545,12 +502,6 @@ func TestTxm(t *testing.T) { // tx passes sim, shows processed, moves to nil (timeout should cleanup) t.Run("fail_confirm_processedToNil", func(t *testing.T) { - mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: 100, - Blockhash: solana.Hash{}, - }, - }, nil).Once() tx, signed := getTx(t, 8, mkey) sig := randomSignature(t) retry0 := randomSignature(t) @@ -602,12 +553,6 @@ func TestTxm(t *testing.T) { // tx passes sim, errors on confirm t.Run("fail_confirm_revert", func(t *testing.T) { - mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: 100, - Blockhash: solana.Hash{}, - }, - }, nil).Once() tx, signed := getTx(t, 9, mkey) sig := randomSignature(t) var wg sync.WaitGroup @@ -645,12 +590,6 @@ func TestTxm(t *testing.T) { // tx passes sim, first retried TXs get dropped t.Run("success_retryTx", func(t *testing.T) { - mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: 100, - Blockhash: solana.Hash{}, - }, - }, nil).Once() tx, signed := getTx(t, 10, mkey) sig := randomSignature(t) retry0 := randomSignature(t) @@ -697,12 +636,6 @@ func TestTxm(t *testing.T) { // fee bumping disabled t.Run("feeBumpingDisabled", func(t *testing.T) { - mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: 100, - Blockhash: solana.Hash{}, - }, - }, nil).Once() sig := randomSignature(t) tx, signed := getTx(t, 11, mkey) @@ -763,12 +696,6 @@ func TestTxm(t *testing.T) { // compute unit limit disabled t.Run("computeUnitLimitDisabled", func(t *testing.T) { - mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: 100, - Blockhash: solana.Hash{}, - }, - }, nil).Once() sig := randomSignature(t) tx, signed := getTx(t, 12, mkey) @@ -1069,14 +996,14 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { return out }, nil, ) + mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: 100, + Blockhash: solana.Hash{}, + }, + }, nil) t.Run("simulation_succeeds", func(t *testing.T) { - mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: 100, - Blockhash: solana.Hash{}, - }, - }, nil).Once() // Test tx is not discarded due to confirm timeout and tracked to finalization // use unique val across tests to avoid collision during mocking tx, signed := getTx(t, 1, mkey) @@ -1277,7 +1204,7 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { cfg := config.NewDefault() cfg.Chain.FeeEstimatorMode = &estimator cfg.Chain.TxConfirmTimeout = relayconfig.MustNewDuration(5 * time.Second) - cfg.Chain.TxRetentionTimeout = relayconfig.MustNewDuration(10 * time.Second) // Enable retention to keep transactions after finality and be able to check. + cfg.Chain.TxRetentionTimeout = relayconfig.MustNewDuration(10 * time.Second) // Enable retention to keep transactions after finality and be able to check their statuses. lggr := logger.Test(t) ctx := tests.Context(t) @@ -1314,7 +1241,7 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { ).Maybe() } - mc.On("SimulateTx", mock.Anything, mock.Anything, mock.Anything).Return(&rpc.SimulateTransactionResult{}, nil) + mc.On("SimulateTx", mock.Anything, mock.Anything, mock.Anything).Return(&rpc.SimulateTransactionResult{}, nil).Maybe() if statuses != nil { mc.On("SignatureStatuses", mock.Anything, mock.AnythingOfType("[]solana.Signature")).Return( func(_ context.Context, sigs []solana.Signature) ([]*rpc.SignatureStatusesResult, error) { @@ -1343,6 +1270,9 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { return txm, mc, mkey } + // tracking prom metrics + prom := soltxmProm{id: id} + t.Run("WithRebroadcast", func(t *testing.T) { txExpirationRebroadcast := true statuses := map[solana.Signature]func() *rpc.SignatureStatusesResult{} @@ -1410,7 +1340,12 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { txID := "test-rebroadcast" assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) wg.Wait() - time.Sleep(2 * time.Second) // Sleep to allow for rebroadcasting + waitFor(t, txm.cfg.TxConfirmTimeout(), txm, prom, empty) + + // check prom metric + prom.confirmed++ + prom.finalized++ + prom.assertEqual(t) // Check that transaction for txID has been finalized and rebroadcasted status, err := txm.GetTransactionStatus(ctx, txID) @@ -1458,7 +1393,12 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { txID := "test-no-rebroadcast" assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) wg.Wait() - time.Sleep(2 * time.Second) // Sleep to ensure no rebroadcast + waitFor(t, txm.cfg.TxConfirmTimeout(), txm, prom, empty) + + // check prom metric + prom.drop++ + prom.error++ + prom.assertEqual(t) // Check that transaction for txID has not been finalized and has not been rebroadcasted status, err := txm.GetTransactionStatus(ctx, txID) @@ -1476,7 +1416,7 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { slotHeightFunc := func() (uint64, error) { return uint64(1500), nil } - // Mock LatestBlockhash to return a invalid blockhash in the first 3 attempts (initial + 2 rebroadcasts) + // Mock LatestBlockhash to return an invalid blockhash in the first 3 attempts (initial + 2 rebroadcasts) // the last one is valid because it is greater than the slotHeight expectedRebroadcastsCount := 3 callCount := 0 @@ -1533,7 +1473,12 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { txID := "test-rebroadcast" assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) wg.Wait() - time.Sleep(2 * time.Second) // Sleep to allow for rebroadcasting + waitFor(t, txm.cfg.TxConfirmTimeout(), txm, prom, empty) + + // check prom metric + prom.confirmed++ + prom.finalized++ + prom.assertEqual(t) // Check that transaction for txID has been finalized and rebroadcasted status, err := txm.GetTransactionStatus(ctx, txID) @@ -1590,7 +1535,12 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { txID := "test-confirmed-before-rebroadcast" assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) wg.Wait() - time.Sleep(1 * time.Second) // Allow for processing + waitFor(t, txm.cfg.TxConfirmTimeout(), txm, prom, empty) + + // check prom metric + prom.confirmed++ + prom.finalized++ + prom.assertEqual(t) // Check that transaction has been finalized without rebroadcast status, err := txm.GetTransactionStatus(ctx, txID) @@ -1632,8 +1582,15 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { return sig1, nil } + var wg sync.WaitGroup + wg.Add(1) + count := 0 statuses[sig1] = func() *rpc.SignatureStatusesResult { + defer func() { count++ }() // Transaction remains unconfirmed + if count == 1 { + wg.Done() + } return nil } @@ -1641,18 +1598,20 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { tx, _ := getTx(t, 0, mkey) txID := "test-rebroadcast-error" assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) - time.Sleep(2 * time.Second) // Allow for processing + wg.Wait() + waitFor(t, cfg.TxConfirmTimeout(), txm, prom, empty) + + // check prom metric + prom.drop++ + prom.error++ + prom.assertEqual(t) - // TODO: Add check that transaction status is failed due to rebroadcast error when prebroadcast is implemented and we have an error in sendWithRetry + // Transaction should be moved to failed after trying to rebroadcast and failing to get confirmations status, err := txm.GetTransactionStatus(ctx, txID) require.NoError(t, err) - require.Equal(t, types.Pending, status) // TODO: Change to Failed when prebroadcast error is implemented + require.Equal(t, types.Failed, status) rebroadcastCount, err := txm.txs.GetTxRebroadcastCount(txID) require.NoError(t, err) - require.Equal(t, 1, rebroadcastCount) // Attempted to rebroadcast 1 time but encountered error - time.Sleep(2 * time.Second) // Allow for processing - rebroadcastCount, err = txm.txs.GetTxRebroadcastCount(txID) // rebroadcast should still be 1. We should not be rebroadcasting. - require.NoError(t, err) require.Equal(t, 1, rebroadcastCount) }) } diff --git a/pkg/solana/txm/txm_race_test.go b/pkg/solana/txm/txm_race_test.go index 6800d9557..fd05587d6 100644 --- a/pkg/solana/txm/txm_race_test.go +++ b/pkg/solana/txm/txm_race_test.go @@ -9,7 +9,6 @@ import ( "time" "github.com/gagliardetto/solana-go" - solanaGo "github.com/gagliardetto/solana-go" "github.com/gagliardetto/solana-go/rpc" "go.uber.org/zap/zapcore" @@ -30,8 +29,8 @@ import ( ) func NewTestMsg() (msg pendingTx) { - tx := solanaGo.Transaction{} - tx.Message.AccountKeys = append(tx.Message.AccountKeys, solanaGo.PublicKey{}) + tx := solana.Transaction{} + tx.Message.AccountKeys = append(tx.Message.AccountKeys, solana.PublicKey{}) msg.tx = tx return msg } @@ -82,20 +81,20 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { lastLog := observer.All()[len(observer.All())-1] assert.Contains(t, lastLog.Message, "stopped tx retry") // assert that all retry goroutines exit successfully } + client := clientmocks.NewReaderWriter(t) + client.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ + Value: &rpc.LatestBlockhashResult{ + LastValidBlockHeight: 100, + Blockhash: solana.Hash{}, + }, + }, nil) t.Run("delay in rebroadcasting tx", func(t *testing.T) { - client := clientmocks.NewReaderWriter(t) - client.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: 100, - Blockhash: solana.Hash{}, - }, - }, nil).Once() - txs := map[string]solanaGo.Signature{} + txs := map[string]solana.Signature{} var lock sync.RWMutex client.On("SendTx", mock.Anything, mock.Anything).Return( // build new sig if tx is different - func(_ context.Context, tx *solanaGo.Transaction) solanaGo.Signature { + func(_ context.Context, tx *solana.Transaction) solana.Signature { strTx := tx.String() // if exists, slow down client response to trigger race @@ -117,7 +116,7 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { sig := make([]byte, 16) _, err := rand.Read(sig) require.NoError(t, err) - txs[strTx] = solanaGo.SignatureFromBytes(sig) + txs[strTx] = solana.SignatureFromBytes(sig) return txs[strTx] }, @@ -127,18 +126,11 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { }) t.Run("delay in broadcasting new tx", func(t *testing.T) { - client := clientmocks.NewReaderWriter(t) - client.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: 100, - Blockhash: solana.Hash{}, - }, - }, nil).Once() - txs := map[string]solanaGo.Signature{} + txs := map[string]solana.Signature{} var lock sync.RWMutex client.On("SendTx", mock.Anything, mock.Anything).Return( // build new sig if tx is different - func(_ context.Context, tx *solanaGo.Transaction) solanaGo.Signature { + func(_ context.Context, tx *solana.Transaction) solana.Signature { strTx := tx.String() lock.Lock() @@ -151,7 +143,7 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { sig := make([]byte, 16) _, err := rand.Read(sig) require.NoError(t, err) - txs[strTx] = solanaGo.SignatureFromBytes(sig) + txs[strTx] = solana.SignatureFromBytes(sig) lock.Unlock() // don't lock on delay @@ -168,18 +160,11 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { }) t.Run("overlapping bumping tx", func(t *testing.T) { - client := clientmocks.NewReaderWriter(t) - client.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: 100, - Blockhash: solana.Hash{}, - }, - }, nil).Once() - txs := map[string]solanaGo.Signature{} + txs := map[string]solana.Signature{} var lock sync.RWMutex client.On("SendTx", mock.Anything, mock.Anything).Return( // build new sig if tx is different - func(_ context.Context, tx *solanaGo.Transaction) solanaGo.Signature { + func(_ context.Context, tx *solana.Transaction) solana.Signature { strTx := tx.String() lock.Lock() @@ -192,7 +177,7 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { sig := make([]byte, 16) _, err := rand.Read(sig) require.NoError(t, err) - txs[strTx] = solanaGo.SignatureFromBytes(sig) + txs[strTx] = solana.SignatureFromBytes(sig) triggerDelay := len(txs) == 2 lock.Unlock() @@ -220,43 +205,35 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { }) t.Run("bumping tx errors and ctx cleans up waitgroup blocks", func(t *testing.T) { - client := clientmocks.NewReaderWriter(t) - client.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: 100, - Blockhash: solana.Hash{}, - }, - }, nil).Once() - // first tx is always successful msg0 := NewTestMsg() require.NoError(t, fees.SetComputeUnitPrice(&msg0.tx, 0)) require.NoError(t, fees.SetComputeUnitLimit(&msg0.tx, 200_000)) - msg0.tx.Signatures = make([]solanaGo.Signature, 1) - client.On("SendTx", mock.Anything, &msg0.tx).Return(solanaGo.Signature{1}, nil) + msg0.tx.Signatures = make([]solana.Signature, 1) + client.On("SendTx", mock.Anything, &msg0.tx).Return(solana.Signature{1}, nil) // init bump tx fails, rebroadcast is successful msg1 := NewTestMsg() require.NoError(t, fees.SetComputeUnitPrice(&msg1.tx, 1)) require.NoError(t, fees.SetComputeUnitLimit(&msg1.tx, 200_000)) - msg1.tx.Signatures = make([]solanaGo.Signature, 1) - client.On("SendTx", mock.Anything, &msg1.tx).Return(solanaGo.Signature{}, fmt.Errorf("BUMP FAILED")).Once() - client.On("SendTx", mock.Anything, &msg1.tx).Return(solanaGo.Signature{2}, nil) + msg1.tx.Signatures = make([]solana.Signature, 1) + client.On("SendTx", mock.Anything, &msg1.tx).Return(solana.Signature{}, fmt.Errorf("BUMP FAILED")) + client.On("SendTx", mock.Anything, &msg1.tx).Return(solana.Signature{2}, nil) // init bump tx success, rebroadcast fails msg2 := NewTestMsg() require.NoError(t, fees.SetComputeUnitPrice(&msg2.tx, 2)) require.NoError(t, fees.SetComputeUnitLimit(&msg2.tx, 200_000)) - msg2.tx.Signatures = make([]solanaGo.Signature, 1) - client.On("SendTx", mock.Anything, &msg2.tx).Return(solanaGo.Signature{3}, nil).Once() - client.On("SendTx", mock.Anything, &msg2.tx).Return(solanaGo.Signature{}, fmt.Errorf("REBROADCAST FAILED")) + msg2.tx.Signatures = make([]solana.Signature, 1) + client.On("SendTx", mock.Anything, &msg2.tx).Return(solana.Signature{3}, nil) + client.On("SendTx", mock.Anything, &msg2.tx).Return(solana.Signature{}, fmt.Errorf("REBROADCAST FAILED")) // always successful msg3 := NewTestMsg() require.NoError(t, fees.SetComputeUnitPrice(&msg3.tx, 4)) require.NoError(t, fees.SetComputeUnitLimit(&msg3.tx, 200_000)) - msg3.tx.Signatures = make([]solanaGo.Signature, 1) - client.On("SendTx", mock.Anything, &msg3.tx).Return(solanaGo.Signature{4}, nil) + msg3.tx.Signatures = make([]solana.Signature, 1) + client.On("SendTx", mock.Anything, &msg3.tx).Return(solana.Signature{4}, nil) testRunner(t, client) }) } From 90daf33c9ade93b340459827729f0dec8774b7c1 Mon Sep 17 00:00:00 2001 From: Farber98 Date: Tue, 26 Nov 2024 16:18:49 -0300 Subject: [PATCH 31/51] Merge branch 'develop' into nonevm-706-support-custom-bumping-strategy-rpc-expiration-within-confirmation --- pkg/solana/txm/pendingtx.go | 103 ++++++++++++++--- pkg/solana/txm/pendingtx_test.go | 126 +++++++++++++++++--- pkg/solana/txm/txm.go | 64 +++++++---- pkg/solana/txm/txm_internal_test.go | 172 +++++++++++++++++++++++++--- 4 files changed, 390 insertions(+), 75 deletions(-) diff --git a/pkg/solana/txm/pendingtx.go b/pkg/solana/txm/pendingtx.go index f851cd2bb..86374094c 100644 --- a/pkg/solana/txm/pendingtx.go +++ b/pkg/solana/txm/pendingtx.go @@ -41,16 +41,20 @@ type PendingTxContext interface { OnFinalized(sig solana.Signature, retentionTimeout time.Duration) (string, error) // OnPrebroadcastError adds transaction that has not yet been broadcasted to the finalized/errored map as errored, matches err type using enum OnPrebroadcastError(id string, retentionTimeout time.Duration, txState TxState, errType TxErrType) error + // OnPrebroadcastError adds transaction that has not yet been broadcasted to the finalized/errored map as errored, matches err type using enum + OnPrebroadcastError(id string, retentionTimeout time.Duration, txState TxState, errType TxErrType) error // OnError marks transaction as errored, matches err type using enum, moves it from the broadcasted or confirmed map to finalized/errored map, removes signatures from signature map to stop confirmation checks OnError(sig solana.Signature, retentionTimeout time.Duration, txState TxState, errType TxErrType) (string, error) + OnError(sig solana.Signature, retentionTimeout time.Duration, txState TxState, errType TxErrType) (string, error) // GetTxState returns the transaction state for the provided ID if it exists GetTxState(id string) (TxState, error) // TrimFinalizedErroredTxs removes transactions that have reached their retention time - TrimFinalizedErroredTxs() + TrimFinalizedErroredTxs() int // GetTxRebroadcastCount returns the number of times a transaction has been rebroadcasted if found. GetTxRebroadcastCount(id string) (int, error) } +// finishedTx is used to store info required to track transactions to finality or error type pendingTx struct { tx solana.Transaction cfg TxConfig @@ -58,20 +62,26 @@ type pendingTx struct { id string rebroadcastCount int createTs time.Time - retentionTs time.Time state TxState lastValidBlockHeight uint64 // to track expiration } +// finishedTx is used to store minimal info specifically for finalized or errored transactions for external status checks +type finishedTx struct { + retentionTs time.Time + state TxState + rebroadcastCount int +} + var _ PendingTxContext = &pendingTxContext{} type pendingTxContext struct { cancelBy map[string]context.CancelFunc sigToID map[solana.Signature]string - broadcastedProcessedTxs map[string]pendingTx // broadcasted and processed transactions that may require retry and bumping - confirmedTxs map[string]pendingTx // transactions that require monitoring for re-org - finalizedErroredTxs map[string]pendingTx // finalized and errored transactions held onto for status + broadcastedProcessedTxs map[string]pendingTx // broadcasted and processed transactions that may require retry and bumping + confirmedTxs map[string]pendingTx // transactions that require monitoring for re-org + finalizedErroredTxs map[string]finishedTx // finalized and errored transactions held onto for status lock sync.RWMutex } @@ -83,7 +93,7 @@ func newPendingTxContext() *pendingTxContext { broadcastedProcessedTxs: map[string]pendingTx{}, confirmedTxs: map[string]pendingTx{}, - finalizedErroredTxs: map[string]pendingTx{}, + finalizedErroredTxs: map[string]finishedTx{}, } } @@ -285,7 +295,6 @@ func (c *pendingTxContext) OnProcessed(sig solana.Signature) (string, error) { if !exists { return id, ErrTransactionNotFound } - tx = c.broadcastedProcessedTxs[id] // update tx state to Processed tx.state = Processed // save updated tx back to the broadcasted map @@ -321,7 +330,8 @@ func (c *pendingTxContext) OnConfirmed(sig solana.Signature) (string, error) { if !sigExists { return id, ErrSigDoesNotExist } - if _, exists := c.broadcastedProcessedTxs[id]; !exists { + tx, exists := c.broadcastedProcessedTxs[id] + if !exists { return id, ErrTransactionNotFound } // call cancel func + remove from map to stop the retry/bumping cycle for this transaction @@ -329,7 +339,6 @@ func (c *pendingTxContext) OnConfirmed(sig solana.Signature) (string, error) { cancel() // cancel context delete(c.cancelBy, id) } - tx := c.broadcastedProcessedTxs[id] // update tx state to Confirmed tx.state = Confirmed // move tx to confirmed map @@ -398,8 +407,13 @@ func (c *pendingTxContext) OnFinalized(sig solana.Signature, retentionTimeout ti state: Finalized, retentionTs: time.Now().Add(retentionTimeout), } + finalizedTx := finishedTx{ + state: Finalized, + retentionTs: time.Now().Add(retentionTimeout), + } // move transaction from confirmed to finalized map c.finalizedErroredTxs[id] = finalizedTx + c.finalizedErroredTxs[id] = finalizedTx return id, nil }) } @@ -413,7 +427,7 @@ func (c *pendingTxContext) OnPrebroadcastError(id string, retentionTimeout time. if tx, exists := c.finalizedErroredTxs[id]; exists && tx.state == txState { return ErrAlreadyInExpectedState } - _, broadcastedExists := c.broadcastedTxs[id] + _, broadcastedExists := c.broadcastedProcessedTxs[id] _, confirmedExists := c.confirmedTxs[id] if broadcastedExists || confirmedExists { return ErrIDAlreadyExists @@ -429,7 +443,7 @@ func (c *pendingTxContext) OnPrebroadcastError(id string, retentionTimeout time. if tx, exists := c.finalizedErroredTxs[id]; exists && tx.state == txState { return "", ErrAlreadyInExpectedState } - _, broadcastedExists := c.broadcastedTxs[id] + _, broadcastedExists := c.broadcastedProcessedTxs[id] _, confirmedExists := c.confirmedTxs[id] if broadcastedExists || confirmedExists { return "", ErrIDAlreadyExists @@ -497,6 +511,7 @@ func (c *pendingTxContext) OnError(sig solana.Signature, retentionTimeout time.D delete(c.sigToID, s) } // if retention duration is set to 0, skip adding transaction to the errored map + // if retention duration is set to 0, skip adding transaction to the errored map if retentionTimeout == 0 { return id, nil } @@ -504,8 +519,13 @@ func (c *pendingTxContext) OnError(sig solana.Signature, retentionTimeout time.D state: txState, retentionTs: time.Now().Add(retentionTimeout), } + erroredTx := finishedTx{ + state: txState, + retentionTs: time.Now().Add(retentionTimeout), + } // move transaction from broadcasted to error map c.finalizedErroredTxs[id] = erroredTx + c.finalizedErroredTxs[id] = erroredTx return id, nil }) } @@ -541,6 +561,31 @@ func (c *pendingTxContext) TrimFinalizedErroredTxs() int { return 0 } + _, err = c.withWriteLock(func() (string, error) { + for _, id := range expiredIDs { + delete(c.finalizedErroredTxs, id) + } + return "", nil + }) + if err != nil { + return 0 + } + return len(expiredIDs) +func (c *pendingTxContext) TrimFinalizedErroredTxs() int { + var expiredIDs []string + err := c.withReadLock(func() error { + expiredIDs = make([]string, 0, len(c.finalizedErroredTxs)) + for id, tx := range c.finalizedErroredTxs { + if time.Now().After(tx.retentionTs) { + expiredIDs = append(expiredIDs, id) + } + } + return nil + }) + if err != nil { + return 0 + } + _, err = c.withWriteLock(func() (string, error) { for _, id := range expiredIDs { delete(c.finalizedErroredTxs, id) @@ -590,7 +635,11 @@ type pendingTxContextWithProm struct { type TxErrType int +type TxErrType int + const ( + NoFailure TxErrType = iota + TxFailRevert NoFailure TxErrType = iota TxFailRevert TxFailReject @@ -663,14 +712,39 @@ func (c *pendingTxContextWithProm) OnError(sig solana.Signature, retentionTimeou func (c *pendingTxContextWithProm) OnPrebroadcastError(id string, retentionTimeout time.Duration, txState TxState, errType TxErrType) error { err := c.pendingTx.OnPrebroadcastError(id, retentionTimeout, txState, errType) // err indicates transaction not found so may already be removed +func (c *pendingTxContextWithProm) OnError(sig solana.Signature, retentionTimeout time.Duration, txState TxState, errType TxErrType) (string, error) { + id, err := c.pendingTx.OnError(sig, retentionTimeout, txState, errType) // err indicates transaction not found so may already be removed if err == nil { incrementErrorMetrics(errType, c.chainID) } + return id, err +} + +func (c *pendingTxContextWithProm) OnPrebroadcastError(id string, retentionTimeout time.Duration, txState TxState, errType TxErrType) error { + err := c.pendingTx.OnPrebroadcastError(id, retentionTimeout, txState, errType) // err indicates transaction not found so may already be removed + if err == nil { + incrementErrorMetrics(errType, c.chainID) + } + return err +} + +func incrementErrorMetrics(errType TxErrType, chainID string) { + incrementErrorMetrics(errType, c.chainID) + } return err } func incrementErrorMetrics(errType TxErrType, chainID string) { switch errType { + case NoFailure: + // Return early if no failure identified + return + case TxFailReject: + promSolTxmRejectTxs.WithLabelValues(chainID).Inc() + case TxFailRevert: + promSolTxmRevertTxs.WithLabelValues(chainID).Inc() + case TxFailDrop: + promSolTxmDropTxs.WithLabelValues(chainID).Inc() case NoFailure: // Return early if no failure identified return @@ -682,18 +756,21 @@ func incrementErrorMetrics(errType TxErrType, chainID string) { promSolTxmDropTxs.WithLabelValues(chainID).Inc() case TxFailSimRevert: promSolTxmSimRevertTxs.WithLabelValues(chainID).Inc() + promSolTxmSimRevertTxs.WithLabelValues(chainID).Inc() case TxFailSimOther: promSolTxmSimOtherTxs.WithLabelValues(chainID).Inc() + promSolTxmSimOtherTxs.WithLabelValues(chainID).Inc() } promSolTxmErrorTxs.WithLabelValues(chainID).Inc() + promSolTxmErrorTxs.WithLabelValues(chainID).Inc() } func (c *pendingTxContextWithProm) GetTxState(id string) (TxState, error) { return c.pendingTx.GetTxState(id) } -func (c *pendingTxContextWithProm) TrimFinalizedErroredTxs() { - c.pendingTx.TrimFinalizedErroredTxs() +func (c *pendingTxContextWithProm) TrimFinalizedErroredTxs() int { + return c.pendingTx.TrimFinalizedErroredTxs() } func (c *pendingTxContextWithProm) GetTxRebroadcastCount(id string) (int, error) { diff --git a/pkg/solana/txm/pendingtx_test.go b/pkg/solana/txm/pendingtx_test.go index 31eeb527f..d7944b3d0 100644 --- a/pkg/solana/txm/pendingtx_test.go +++ b/pkg/solana/txm/pendingtx_test.go @@ -93,11 +93,11 @@ func TestPendingTxContext_new(t *testing.T) { require.Equal(t, Broadcasted, tx.state) // Check it does not exist in confirmed map - tx, exists = txs.confirmedTxs[msg.id] + _, exists = txs.confirmedTxs[msg.id] require.False(t, exists) // Check it does not exist in finalized map - tx, exists = txs.finalizedErroredTxs[msg.id] + _, exists = txs.finalizedErroredTxs[msg.id] require.False(t, exists) } @@ -134,11 +134,11 @@ func TestPendingTxContext_add_signature(t *testing.T) { require.Equal(t, sig2, tx.signatures[1]) // Check confirmed map - tx, exists = txs.confirmedTxs[msg.id] + _, exists = txs.confirmedTxs[msg.id] require.False(t, exists) // Check finalized map - tx, exists = txs.finalizedErroredTxs[msg.id] + _, exists = txs.finalizedErroredTxs[msg.id] require.False(t, exists) }) @@ -225,11 +225,11 @@ func TestPendingTxContext_on_broadcasted_processed(t *testing.T) { require.Equal(t, Processed, tx.state) // Check it does not exist in confirmed map - tx, exists = txs.confirmedTxs[msg.id] + _, exists = txs.confirmedTxs[msg.id] require.False(t, exists) // Check it does not exist in finalized map - tx, exists = txs.finalizedErroredTxs[msg.id] + _, exists = txs.finalizedErroredTxs[msg.id] require.False(t, exists) }) @@ -294,6 +294,7 @@ func TestPendingTxContext_on_broadcasted_processed(t *testing.T) { // Transition to errored state id, err := txs.OnError(sig, retentionTimeout, Errored, 0) + id, err := txs.OnError(sig, retentionTimeout, Errored, 0) require.NoError(t, err) require.Equal(t, msg.id, id) @@ -364,7 +365,7 @@ func TestPendingTxContext_on_confirmed(t *testing.T) { require.Equal(t, Confirmed, tx.state) // Check it does not exist in finalized map - tx, exists = txs.finalizedErroredTxs[msg.id] + _, exists = txs.finalizedErroredTxs[msg.id] require.False(t, exists) }) @@ -406,6 +407,7 @@ func TestPendingTxContext_on_confirmed(t *testing.T) { // Transition to errored state id, err := txs.OnError(sig, retentionTimeout, Errored, 0) + id, err := txs.OnError(sig, retentionTimeout, Errored, 0) require.NoError(t, err) require.Equal(t, msg.id, id) @@ -584,6 +586,7 @@ func TestPendingTxContext_on_finalized(t *testing.T) { // Transition to errored state id, err := txs.OnError(sig, retentionTimeout, Errored, 0) + id, err := txs.OnError(sig, retentionTimeout, Errored, 0) require.NoError(t, err) require.Equal(t, msg.id, id) @@ -609,6 +612,7 @@ func TestPendingTxContext_on_error(t *testing.T) { // Transition to errored state id, err := txs.OnError(sig, retentionTimeout, Errored, 0) + id, err := txs.OnError(sig, retentionTimeout, Errored, 0) require.NoError(t, err) require.Equal(t, msg.id, id) @@ -647,6 +651,7 @@ func TestPendingTxContext_on_error(t *testing.T) { // Transition to errored state id, err = txs.OnError(sig, retentionTimeout, Errored, 0) + id, err = txs.OnError(sig, retentionTimeout, Errored, 0) require.NoError(t, err) require.Equal(t, msg.id, id) @@ -674,7 +679,7 @@ func TestPendingTxContext_on_error(t *testing.T) { sig := randomSignature(t) // Create new transaction - msg := PendingTx{UUID: uuid.NewString()} + msg := pendingTx{id: uuid.NewString()} err := txs.New(msg, sig, cancel) require.NoError(t, err) @@ -684,7 +689,7 @@ func TestPendingTxContext_on_error(t *testing.T) { require.Equal(t, msg.id, id) // Check it does not exist in broadcasted map - _, exists := txs.broadcastedTxs[msg.id] + _, exists := txs.broadcastedProcessedTxs[msg.id] require.False(t, exists) // Check it exists in errored map @@ -714,6 +719,7 @@ func TestPendingTxContext_on_error(t *testing.T) { // Transition to errored state id, err = txs.OnError(sig, 0*time.Second, Errored, 0) + id, err = txs.OnError(sig, 0*time.Second, Errored, 0) require.NoError(t, err) require.Equal(t, msg.id, id) @@ -749,6 +755,7 @@ func TestPendingTxContext_on_error(t *testing.T) { // Transition back to confirmed state id, err = txs.OnError(sig, retentionTimeout, Errored, 0) + id, err = txs.OnError(sig, retentionTimeout, Errored, 0) require.Error(t, err) require.Equal(t, "", id) }) @@ -818,6 +825,70 @@ func TestPendingTxContext_on_prebroadcast_error(t *testing.T) { }) } +func TestPendingTxContext_on_prebroadcast_error(t *testing.T) { + t.Parallel() + _, cancel := context.WithCancel(tests.Context(t)) + txs := newPendingTxContext() + retentionTimeout := 5 * time.Second + + t.Run("successfully adds transaction with errored state", func(t *testing.T) { + // Create new transaction + msg := pendingTx{id: uuid.NewString()} + // Transition to errored state + err := txs.OnPrebroadcastError(msg.id, retentionTimeout, Errored, 0) + require.NoError(t, err) + + // Check it exists in errored map + tx, exists := txs.finalizedErroredTxs[msg.id] + require.True(t, exists) + + // Check status is Errored + require.Equal(t, Errored, tx.state) + }) + + t.Run("successfully adds transaction with fatally errored state", func(t *testing.T) { + // Create new transaction + msg := pendingTx{id: uuid.NewString()} + + // Transition to fatally errored state + err := txs.OnPrebroadcastError(msg.id, retentionTimeout, FatallyErrored, 0) + require.NoError(t, err) + + // Check it exists in errored map + tx, exists := txs.finalizedErroredTxs[msg.id] + require.True(t, exists) + + // Check status is Errored + require.Equal(t, FatallyErrored, tx.state) + }) + + t.Run("fails to add transaction to errored map if id exists in another map already", func(t *testing.T) { + sig := randomSignature(t) + + // Create new transaction + msg := pendingTx{id: uuid.NewString()} + // Add transaction to broadcasted map + err := txs.New(msg, sig, cancel) + require.NoError(t, err) + + // Transition to errored state + err = txs.OnPrebroadcastError(msg.id, retentionTimeout, FatallyErrored, 0) + require.ErrorIs(t, err, ErrIDAlreadyExists) + }) + + t.Run("predefined error if transaction already in errored state", func(t *testing.T) { + txID := uuid.NewString() + + // Transition to errored state + err := txs.OnPrebroadcastError(txID, retentionTimeout, Errored, 0) + require.NoError(t, err) + + // Transition back to errored state + err = txs.OnPrebroadcastError(txID, retentionTimeout, Errored, 0) + require.ErrorIs(t, err, ErrAlreadyInExpectedState) + }) +} + func TestPendingTxContext_remove(t *testing.T) { t.Parallel() _, cancel := context.WithCancel(tests.Context(t)) @@ -868,6 +939,7 @@ func TestPendingTxContext_remove(t *testing.T) { err = txs.New(erroredMsg, erroredSig, cancel) require.NoError(t, err) id, err = txs.OnError(erroredSig, retentionTimeout, Errored, 0) + id, err = txs.OnError(erroredSig, retentionTimeout, Errored, 0) require.NoError(t, err) require.Equal(t, erroredMsg.id, id) @@ -924,32 +996,34 @@ func TestPendingTxContext_trim_finalized_errored_txs(t *testing.T) { txs := newPendingTxContext() // Create new finalized transaction with retention ts in the past and add to map - finalizedMsg1 := pendingTx{id: uuid.NewString(), retentionTs: time.Now().Add(-2 * time.Second)} - txs.finalizedErroredTxs[finalizedMsg1.id] = finalizedMsg1 + finalizedMsg1 := finishedTx{retentionTs: time.Now().Add(-2 * time.Second)} + finalizedMsg1ID := uuid.NewString() + txs.finalizedErroredTxs[finalizedMsg1ID] = finalizedMsg1 // Create new finalized transaction with retention ts in the future and add to map - finalizedMsg2 := pendingTx{id: uuid.NewString(), retentionTs: time.Now().Add(1 * time.Second)} - txs.finalizedErroredTxs[finalizedMsg2.id] = finalizedMsg2 + finalizedMsg2 := finishedTx{retentionTs: time.Now().Add(1 * time.Second)} + finalizedMsg2ID := uuid.NewString() + txs.finalizedErroredTxs[finalizedMsg2ID] = finalizedMsg2 // Create new finalized transaction with retention ts in the past and add to map - erroredMsg := pendingTx{id: uuid.NewString(), retentionTs: time.Now().Add(-2 * time.Second)} - txs.finalizedErroredTxs[erroredMsg.id] = erroredMsg + erroredMsg := finishedTx{retentionTs: time.Now().Add(-2 * time.Second)} + erroredMsgID := uuid.NewString() + txs.finalizedErroredTxs[erroredMsgID] = erroredMsg // Delete finalized/errored transactions that have passed the retention period txs.TrimFinalizedErroredTxs() // Check finalized message past retention is deleted - _, exists := txs.finalizedErroredTxs[finalizedMsg1.id] + _, exists := txs.finalizedErroredTxs[finalizedMsg1ID] require.False(t, exists) // Check errored message past retention is deleted - _, exists = txs.finalizedErroredTxs[erroredMsg.id] + _, exists = txs.finalizedErroredTxs[erroredMsgID] require.False(t, exists) // Check finalized message within retention period still exists - msg, exists := txs.finalizedErroredTxs[finalizedMsg2.id] + _, exists = txs.finalizedErroredTxs[finalizedMsg2ID] require.True(t, exists) - require.Equal(t, finalizedMsg2.id, msg.id) } func TestPendingTxContext_expired(t *testing.T) { @@ -1054,6 +1128,7 @@ func TestGetTxState(t *testing.T) { finalizedSig := randomSignature(t) erroredSig := randomSignature(t) fatallyErroredSig := randomSignature(t) + fatallyErroredSig := randomSignature(t) // Create new broadcasted transaction with extra sig broadcastedMsg := pendingTx{id: uuid.NewString()} @@ -1102,6 +1177,7 @@ func TestGetTxState(t *testing.T) { err = txs.New(erroredMsg, erroredSig, cancel) require.NoError(t, err) id, err = txs.OnError(erroredSig, retentionTimeout, Errored, 0) + id, err = txs.OnError(erroredSig, retentionTimeout, Errored, 0) require.NoError(t, err) require.Equal(t, erroredMsg.id, id) // Check Errored state is returned @@ -1121,6 +1197,18 @@ func TestGetTxState(t *testing.T) { require.NoError(t, err) require.Equal(t, FatallyErrored, state) + // Create new fatally errored transaction + fatallyErroredMsg := pendingTx{id: uuid.NewString()} + err = txs.New(fatallyErroredMsg, fatallyErroredSig, cancel) + require.NoError(t, err) + id, err = txs.OnError(fatallyErroredSig, retentionTimeout, FatallyErrored, 0) + require.NoError(t, err) + require.Equal(t, fatallyErroredMsg.id, id) + // Check Errored state is returned + state, err = txs.GetTxState(fatallyErroredMsg.id) + require.NoError(t, err) + require.Equal(t, FatallyErrored, state) + // Check NotFound state is returned if unknown id provided state, err = txs.GetTxState("unknown id") require.Error(t, err) diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index b48556dab..7e0862481 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -208,12 +208,12 @@ func (txm *Txm) sendWithRetry(ctx context.Context, msg pendingTx) (solanaGo.Tran // Send initial transaction ctx, cancel := context.WithTimeout(ctx, msg.cfg.Timeout) - sig, err := txm.sendTx(ctx, &initTx) - if err != nil { + sig, initSendErr := txm.sendTx(ctx, &initTx) + if initSendErr != nil { // Do not retry and exit early if fails cancel() - txm.txs.OnError(sig, txm.cfg.TxRetentionTimeout(), TxFailReject) //nolint // no need to check error since only incrementing metric here - return solanaGo.Transaction{}, "", solanaGo.Signature{}, fmt.Errorf("tx failed initial transmit: %w", err) + stateTransitionErr := txm.txs.OnPrebroadcastError(msg.id, txm.cfg.TxRetentionTimeout(), Errored, TxFailReject) + return solanaGo.Transaction{}, "", solanaGo.Signature{}, fmt.Errorf("tx failed initial transmit: %w", errors.Join(initSendErr, stateTransitionErr)) } // Store tx signature and cancel function @@ -490,13 +490,12 @@ func (txm *Txm) processConfirmations(ctx context.Context, client client.ReaderWr // If the confirmation timeout has been exceeded it marks the transaction as errored. func (txm *Txm) handleNotFoundSignatureStatus(sig solanaGo.Signature) { txm.lggr.Debugw("tx state: not found", "signature", sig) - // check confirm timeout exceeded if txm.cfg.TxConfirmTimeout() != 0*time.Second && txm.txs.Expired(sig, txm.cfg.TxConfirmTimeout()) { - id, err := txm.txs.OnError(sig, txm.cfg.TxRetentionTimeout(), TxFailDrop) + id, err := txm.txs.OnError(sig, txm.cfg.TxRetentionTimeout(), Errored, TxFailDrop) if err != nil { txm.lggr.Infow("failed to mark transaction as errored", "id", id, "signature", sig, "timeoutSeconds", txm.cfg.TxConfirmTimeout(), "error", err) } else { - txm.lggr.Infow("failed to find transaction within confirm timeout", "id", id, "signature", sig, "timeoutSeconds", txm.cfg.TxConfirmTimeout()) + txm.lggr.Debugw("failed to find transaction within confirm timeout", "id", id, "signature", sig, "timeoutSeconds", txm.cfg.TxConfirmTimeout()) } } } @@ -511,11 +510,15 @@ func (txm *Txm) handleErrorSignatureStatus(sig solanaGo.Signature, status *rpc.S return } - id, err := txm.txs.OnError(sig, txm.cfg.TxRetentionTimeout(), TxFailRevert) - if err != nil { - txm.lggr.Infow("failed to mark transaction as errored", "id", id, "signature", sig, "error", err) - } else { - txm.lggr.Debugw("tx state: failed", "id", id, "signature", sig, "error", status.Err, "status", status.ConfirmationStatus) + // Process error to determine the corresponding state and type. + // Skip marking as errored if error considered to not be a failure. + if txState, errType := txm.processError(sig, status.Err, false); errType != NoFailure { + id, err := txm.txs.OnError(sig, txm.cfg.TxRetentionTimeout(), txState, errType) + if err != nil { + txm.lggr.Infow(fmt.Sprintf("failed to mark transaction as %s", txState.String()), "id", id, "signature", sig, "error", err) + } else { + txm.lggr.Debugw(fmt.Sprintf("marking transaction as %s", txState.String()), "id", id, "signature", sig, "error", status.Err, "status", status.ConfirmationStatus) + } } } @@ -532,7 +535,7 @@ func (txm *Txm) handleProcessedSignatureStatus(sig solanaGo.Signature) { } // check confirm timeout exceeded if TxConfirmTimeout set if txm.cfg.TxConfirmTimeout() != 0*time.Second && txm.txs.Expired(sig, txm.cfg.TxConfirmTimeout()) { - id, err := txm.txs.OnError(sig, txm.cfg.TxRetentionTimeout(), TxFailDrop) + id, err := txm.txs.OnError(sig, txm.cfg.TxRetentionTimeout(), Errored, TxFailDrop) if err != nil { txm.lggr.Infow("failed to mark transaction as errored", "id", id, "signature", sig, "timeoutSeconds", txm.cfg.TxConfirmTimeout(), "error", err) } else { @@ -629,8 +632,18 @@ func (txm *Txm) simulate() { } // Transaction has to have a signature if simulation succeeded but added check for belt and braces approach - if len(msg.signatures) > 0 { - txm.processSimulationError(msg.id, msg.signatures[0], res) + if len(msg.signatures) == 0 { + continue + } + // Process error to determine the corresponding state and type. + // Certain errors can be considered not to be failures during simulation to allow the process to continue + if txState, errType := txm.processError(msg.signatures[0], res.Err, true); errType != NoFailure { + id, err := txm.txs.OnError(msg.signatures[0], txm.cfg.TxRetentionTimeout(), txState, errType) + if err != nil { + txm.lggr.Errorw(fmt.Sprintf("failed to mark transaction as %s", txState.String()), "id", id, "err", err) + } else { + txm.lggr.Debugw(fmt.Sprintf("marking transaction as %s", txState.String()), "id", id, "signature", msg.signatures[0], "error", res.Err) + } } } } @@ -696,7 +709,7 @@ func (txm *Txm) Enqueue(ctx context.Context, accountID string, tx *solanaGo.Tran // Perform compute unit limit estimation after storing transaction // If error found during simulation, transaction should be in storage to mark accordingly if cfg.EstimateComputeUnitLimit { - computeUnitLimit, err := txm.EstimateComputeUnitLimit(ctx, tx) + computeUnitLimit, err := txm.EstimateComputeUnitLimit(ctx, tx, id) if err != nil { return fmt.Errorf("transaction failed simulation: %w", err) } @@ -851,15 +864,18 @@ func (txm *Txm) processError(sig solanaGo.Signature, resErr interface{}, simulat // blockhash not found when simulating, occurs when network bank has not seen the given blockhash or tx is too old // let confirmation process clean up case strings.Contains(errStr, "BlockhashNotFound"): - txm.lggr.Debugw("simulate: BlockhashNotFound", logValues...) - // transaction will encounter execution error/revert, mark as reverted to remove from confirmation + retry - case strings.Contains(errStr, "InstructionError"): - _, err := txm.txs.OnError(sig, txm.cfg.TxRetentionTimeout(), TxFailSimRevert) // cancel retry - if err != nil { - logValues = append(logValues, "stateTransitionErr", err) + txm.lggr.Debugw("BlockhashNotFound", logValues...) + // return no failure for this error when simulating to allow later send/retry code to assign a proper blockhash + // in case the one provided by the caller is outdated + if simulation { + return txState, NoFailure } - txm.lggr.Debugw("simulate: InstructionError", logValues...) - // transaction is already processed in the chain, letting txm confirmation handle + return Errored, errType + // transaction will encounter execution error/revert + case strings.Contains(errStr, "InstructionError"): + txm.lggr.Debugw("InstructionError", logValues...) + return Errored, errType + // transaction is already processed in the chain case strings.Contains(errStr, "AlreadyProcessed"): txm.lggr.Debugw("AlreadyProcessed", logValues...) // return no failure for this error when simulating in case there is a race between broadcast and simulation diff --git a/pkg/solana/txm/txm_internal_test.go b/pkg/solana/txm/txm_internal_test.go index 785d30b81..328ed9536 100644 --- a/pkg/solana/txm/txm_internal_test.go +++ b/pkg/solana/txm/txm_internal_test.go @@ -28,6 +28,7 @@ import ( relayconfig "github.com/smartcontractkit/chainlink-common/pkg/config" "github.com/smartcontractkit/chainlink-common/pkg/logger" "github.com/smartcontractkit/chainlink-common/pkg/types" + commontypes "github.com/smartcontractkit/chainlink-common/pkg/types" "github.com/smartcontractkit/chainlink-common/pkg/utils" bigmath "github.com/smartcontractkit/chainlink-common/pkg/utils/big_math" "github.com/smartcontractkit/chainlink-common/pkg/utils/tests" @@ -801,20 +802,41 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { }, }, nil).Once() - // Test tx is not discarded due to confirm timeout and tracked to finalization - tx, signed := getTx(t, 7, mkey) - sig := randomSignature(t) - retry0 := randomSignature(t) - retry1 := randomSignature(t) - var wg sync.WaitGroup - wg.Add(2) - mc.On("SendTx", mock.Anything, signed(0, true, computeUnitLimitDefault)).Return(sig, nil) - mc.On("SendTx", mock.Anything, signed(1, true, computeUnitLimitDefault)).Return(retry0, nil).Maybe() - mc.On("SendTx", mock.Anything, signed(2, true, computeUnitLimitDefault)).Return(retry1, nil).Maybe() - mc.On("SimulateTx", mock.Anything, signed(0, true, computeUnitLimitDefault), mock.Anything).Run(func(mock.Arguments) { - wg.Done() - }).Return(&rpc.SimulateTransactionResult{}, nil).Once() + t.Run("happyPath", func(t *testing.T) { + // Test tx is not discarded due to confirm timeout and tracked to finalization + // use unique val across tests to avoid collision during mocking + tx, signed := getTx(t, 1, mkey) + sig := randomSignature(t) + retry0 := randomSignature(t) + retry1 := randomSignature(t) + var wg sync.WaitGroup + wg.Add(2) + + mc.On("SendTx", mock.Anything, signed(0, true, computeUnitLimitDefault)).Return(sig, nil) + mc.On("SendTx", mock.Anything, signed(1, true, computeUnitLimitDefault)).Return(retry0, nil).Maybe() + mc.On("SendTx", mock.Anything, signed(2, true, computeUnitLimitDefault)).Return(retry1, nil).Maybe() + mc.On("SimulateTx", mock.Anything, signed(0, true, computeUnitLimitDefault), mock.Anything).Run(func(mock.Arguments) { + wg.Done() + }).Return(&rpc.SimulateTransactionResult{}, nil).Once() + // handle signature status calls (initial stays processed, others don't exist) + start := time.Now() + statuses[sig] = func() (out *rpc.SignatureStatusesResult) { + out = &rpc.SignatureStatusesResult{} + // return confirmed status after default confirmation timeout + if time.Since(start) > 1*time.Second && time.Since(start) < 2*time.Second { + out.ConfirmationStatus = rpc.ConfirmationStatusConfirmed + return + } + // return finalized status only after the confirmation timeout + if time.Since(start) >= 2*time.Second { + out.ConfirmationStatus = rpc.ConfirmationStatusFinalized + wg.Done() + return + } + out.ConfirmationStatus = rpc.ConfirmationStatusProcessed + return + } // handle signature status calls (initial stays processed, others don't exist) start := time.Now() statuses[sig] = func() (out *rpc.SignatureStatusesResult) { @@ -834,20 +856,30 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { return } - // tx should be able to queue - testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) - wg.Wait() // wait to be picked up and processed - waitFor(t, 5*time.Second, txm, prom, empty) // inflight txs cleared after timeout + // tx should be able to queue + testTxID := uuid.New().String() + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + wg.Wait() // wait to be picked up and processed + waitFor(t, 5*time.Second, txm, prom, empty) // inflight txs cleared after timeout + // panic if sendTx called after context cancelled + mc.On("SendTx", mock.Anything, tx).Panic("SendTx should not be called anymore").Maybe() // panic if sendTx called after context cancelled mc.On("SendTx", mock.Anything, tx).Panic("SendTx should not be called anymore").Maybe() + // check prom metric + prom.confirmed++ + prom.finalized++ + prom.assertEqual(t) // check prom metric prom.confirmed++ prom.finalized++ prom.assertEqual(t) + // check transaction status which should still be stored + status, err := txm.GetTransactionStatus(ctx, testTxID) + require.NoError(t, err) + require.Equal(t, types.Finalized, status) // check transaction status which should still be stored status, err := txm.GetTransactionStatus(ctx, testTxID) require.NoError(t, err) @@ -855,12 +887,103 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { // Sleep until retention period has passed for transaction and for another reap cycle to run time.Sleep(10 * time.Second) + // Sleep until retention period has passed for transaction and for another reap cycle to run + time.Sleep(10 * time.Second) + + // check if transaction has been purged from memory + status, err = txm.GetTransactionStatus(ctx, testTxID) + require.Error(t, err) + require.Equal(t, types.Unknown, status) + }) + + t.Run("stores error if initial send fails", func(t *testing.T) { + // Test tx is not discarded due to confirm timeout and tracked to finalization + // use unique val across tests to avoid collision during mocking + tx, signed := getTx(t, 2, mkey) + var wg sync.WaitGroup + wg.Add(1) + + mc.On("SendTx", mock.Anything, signed(0, true, computeUnitLimitDefault)).Run(func(mock.Arguments) { + wg.Done() + }).Return(nil, errors.New("failed to send")) + + // tx should be able to queue + testTxID := uuid.NewString() + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + wg.Wait() + waitFor(t, 5*time.Second, txm, prom, empty) // inflight txs cleared after timeout + + // panic if sendTx called after context cancelled + mc.On("SendTx", mock.Anything, tx).Panic("SendTx should not be called anymore").Maybe() + + // check prom metric + prom.error++ + prom.reject++ + prom.assertEqual(t) + + // check transaction status which should still be stored + status, err := txm.GetTransactionStatus(ctx, testTxID) + require.NoError(t, err) + require.Equal(t, types.Failed, status) + + // Sleep until retention period has passed for transaction and for another reap cycle to run + time.Sleep(15 * time.Second) + + // check if transaction has been purged from memory + status, err = txm.GetTransactionStatus(ctx, testTxID) + require.Error(t, err) + require.Equal(t, types.Unknown, status) + }) + + t.Run("stores error if confirmation returns error", func(t *testing.T) { + // Test tx is not discarded due to confirm timeout and tracked to finalization + // use unique val across tests to avoid collision during mocking + tx, signed := getTx(t, 3, mkey) + sig := randomSignature(t) + var wg sync.WaitGroup + wg.Add(2) + + mc.On("SendTx", mock.Anything, signed(0, true, computeUnitLimitDefault)).Return(sig, nil) + mc.On("SimulateTx", mock.Anything, signed(0, true, computeUnitLimitDefault), mock.Anything).Run(func(mock.Arguments) { + wg.Done() + }).Return(&rpc.SimulateTransactionResult{}, nil).Once() + statuses[sig] = func() (out *rpc.SignatureStatusesResult) { + defer wg.Done() + return &rpc.SignatureStatusesResult{Err: errors.New("InstructionError")} + } + + // tx should be able to queue + testTxID := uuid.NewString() + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + wg.Wait() // wait till send tx + waitFor(t, 5*time.Second, txm, prom, empty) // inflight txs cleared after timeout + + // panic if sendTx called after context cancelled + mc.On("SendTx", mock.Anything, tx).Panic("SendTx should not be called anymore").Maybe() + + // check prom metric + prom.error++ + prom.revert++ + prom.assertEqual(t) + + // check transaction status which should still be stored + status, err := txm.GetTransactionStatus(ctx, testTxID) + require.NoError(t, err) + require.Equal(t, types.Failed, status) + + // Sleep until retention period has passed for transaction and for another reap cycle to run + time.Sleep(15 * time.Second) // check if transaction has been purged from memory status, err = txm.GetTransactionStatus(ctx, testTxID) require.Error(t, err) require.Equal(t, types.Unknown, status) }) + // check if transaction has been purged from memory + status, err = txm.GetTransactionStatus(ctx, testTxID) + require.Error(t, err) + require.Equal(t, types.Unknown, status) + }) t.Run("stores error if initial send fails", func(t *testing.T) { // Test tx is not discarded due to confirm timeout and tracked to finalization @@ -1006,6 +1129,7 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { t.Run("simulation_succeeds", func(t *testing.T) { // Test tx is not discarded due to confirm timeout and tracked to finalization // use unique val across tests to avoid collision during mocking + // use unique val across tests to avoid collision during mocking tx, signed := getTx(t, 1, mkey) // add signature and compute unit limit to tx for simulation (excludes compute unit price) simulateTx := addSigAndLimitToTx(t, mkey, solana.PublicKey{}, *tx, MaxComputeUnitLimit) @@ -1071,6 +1195,8 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { // Test tx is not discarded due to confirm timeout and tracked to finalization // use unique val across tests to avoid collision during mocking tx, signed := getTx(t, 2, mkey) + // use unique val across tests to avoid collision during mocking + tx, signed := getTx(t, 2, mkey) sig := randomSignature(t) mc.On("SendTx", mock.Anything, signed(0, true, fees.ComputeUnitLimit(0))).Return(sig, nil).Panic("SendTx should never be called").Maybe() @@ -1084,16 +1210,24 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { // Test tx is not discarded due to confirm timeout and tracked to finalization // use unique val across tests to avoid collision during mocking tx, _ := getTx(t, 3, mkey) + // use unique val across tests to avoid collision during mocking + tx, _ := getTx(t, 3, mkey) // add signature and compute unit limit to tx for simulation (excludes compute unit price) simulateTx := addSigAndLimitToTx(t, mkey, solana.PublicKey{}, *tx, MaxComputeUnitLimit) sig := randomSignature(t) mc.On("SendTx", mock.Anything, mock.Anything).Return(sig, nil).Panic("SendTx should never be called").Maybe() // First simulation before broadcast with max compute unit limit mc.On("SimulateTx", mock.Anything, simulateTx, mock.Anything).Return(&rpc.SimulateTransactionResult{Err: errors.New("InstructionError")}, nil).Once() + mc.On("SimulateTx", mock.Anything, simulateTx, mock.Anything).Return(&rpc.SimulateTransactionResult{Err: errors.New("InstructionError")}, nil).Once() + txID := uuid.NewString() txID := uuid.NewString() // tx should NOT be able to queue - assert.Error(t, txm.Enqueue(ctx, t.Name(), tx, nil)) + assert.Error(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) + // tx should be stored in-memory and moved to errored state + status, err := txm.GetTransactionStatus(ctx, txID) + require.NoError(t, err) + require.Equal(t, commontypes.Failed, status) }) } From 77b28cf5692d929b6f1ecf50cd3230afe283df3e Mon Sep 17 00:00:00 2001 From: Farber98 Date: Tue, 26 Nov 2024 16:36:59 -0300 Subject: [PATCH 32/51] refactors after merge --- pkg/solana/txm/pendingtx.go | 18 +++++++++++------- pkg/solana/txm/txm_internal_test.go | 2 +- 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/pkg/solana/txm/pendingtx.go b/pkg/solana/txm/pendingtx.go index 86374094c..192729ed9 100644 --- a/pkg/solana/txm/pendingtx.go +++ b/pkg/solana/txm/pendingtx.go @@ -404,8 +404,9 @@ func (c *pendingTxContext) OnFinalized(sig solana.Signature, retentionTimeout ti return id, nil } finalizedTx := finishedTx{ - state: Finalized, - retentionTs: time.Now().Add(retentionTimeout), + state: Finalized, + retentionTs: time.Now().Add(retentionTimeout), + rebroadcastCount: tx.rebroadcastCount, } finalizedTx := finishedTx{ state: Finalized, @@ -440,7 +441,8 @@ func (c *pendingTxContext) OnPrebroadcastError(id string, retentionTimeout time. // upgrade to write lock if id does not exist in other maps and is not in expected state already _, err = c.withWriteLock(func() (string, error) { - if tx, exists := c.finalizedErroredTxs[id]; exists && tx.state == txState { + tx, exists := c.finalizedErroredTxs[id] + if exists && tx.state == txState { return "", ErrAlreadyInExpectedState } _, broadcastedExists := c.broadcastedProcessedTxs[id] @@ -449,8 +451,9 @@ func (c *pendingTxContext) OnPrebroadcastError(id string, retentionTimeout time. return "", ErrIDAlreadyExists } erroredTx := finishedTx{ - state: txState, - retentionTs: time.Now().Add(retentionTimeout), + state: txState, + retentionTs: time.Now().Add(retentionTimeout), + rebroadcastCount: tx.rebroadcastCount, } // add transaction to error map c.finalizedErroredTxs[id] = erroredTx @@ -516,8 +519,9 @@ func (c *pendingTxContext) OnError(sig solana.Signature, retentionTimeout time.D return id, nil } erroredTx := finishedTx{ - state: txState, - retentionTs: time.Now().Add(retentionTimeout), + state: txState, + retentionTs: time.Now().Add(retentionTimeout), + rebroadcastCount: tx.rebroadcastCount, } erroredTx := finishedTx{ state: txState, diff --git a/pkg/solana/txm/txm_internal_test.go b/pkg/solana/txm/txm_internal_test.go index 328ed9536..181d4b727 100644 --- a/pkg/solana/txm/txm_internal_test.go +++ b/pkg/solana/txm/txm_internal_test.go @@ -800,7 +800,7 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { LastValidBlockHeight: 100, Blockhash: solana.Hash{}, }, - }, nil).Once() + }, nil) t.Run("happyPath", func(t *testing.T) { // Test tx is not discarded due to confirm timeout and tracked to finalization From 0c4a7d87578881568648248b8fd16389a0e0474b Mon Sep 17 00:00:00 2001 From: Farber98 Date: Tue, 26 Nov 2024 17:45:10 -0300 Subject: [PATCH 33/51] fix interactive rebase --- pkg/solana/transmitter_test.go | 1 + pkg/solana/txm/pendingtx.go | 71 --------------- pkg/solana/txm/pendingtx_test.go | 86 ------------------- pkg/solana/txm/txm_internal_test.go | 129 ---------------------------- 4 files changed, 1 insertion(+), 286 deletions(-) diff --git a/pkg/solana/transmitter_test.go b/pkg/solana/transmitter_test.go index d2f0229ca..1d058d36a 100644 --- a/pkg/solana/transmitter_test.go +++ b/pkg/solana/transmitter_test.go @@ -62,6 +62,7 @@ func TestTransmitter_TxSize(t *testing.T) { rw.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{}, }, nil) + transmitter := Transmitter{ stateID: mustNewRandomPublicKey(), programID: mustNewRandomPublicKey(), diff --git a/pkg/solana/txm/pendingtx.go b/pkg/solana/txm/pendingtx.go index 192729ed9..540a1221c 100644 --- a/pkg/solana/txm/pendingtx.go +++ b/pkg/solana/txm/pendingtx.go @@ -41,11 +41,8 @@ type PendingTxContext interface { OnFinalized(sig solana.Signature, retentionTimeout time.Duration) (string, error) // OnPrebroadcastError adds transaction that has not yet been broadcasted to the finalized/errored map as errored, matches err type using enum OnPrebroadcastError(id string, retentionTimeout time.Duration, txState TxState, errType TxErrType) error - // OnPrebroadcastError adds transaction that has not yet been broadcasted to the finalized/errored map as errored, matches err type using enum - OnPrebroadcastError(id string, retentionTimeout time.Duration, txState TxState, errType TxErrType) error // OnError marks transaction as errored, matches err type using enum, moves it from the broadcasted or confirmed map to finalized/errored map, removes signatures from signature map to stop confirmation checks OnError(sig solana.Signature, retentionTimeout time.Duration, txState TxState, errType TxErrType) (string, error) - OnError(sig solana.Signature, retentionTimeout time.Duration, txState TxState, errType TxErrType) (string, error) // GetTxState returns the transaction state for the provided ID if it exists GetTxState(id string) (TxState, error) // TrimFinalizedErroredTxs removes transactions that have reached their retention time @@ -408,13 +405,8 @@ func (c *pendingTxContext) OnFinalized(sig solana.Signature, retentionTimeout ti retentionTs: time.Now().Add(retentionTimeout), rebroadcastCount: tx.rebroadcastCount, } - finalizedTx := finishedTx{ - state: Finalized, - retentionTs: time.Now().Add(retentionTimeout), - } // move transaction from confirmed to finalized map c.finalizedErroredTxs[id] = finalizedTx - c.finalizedErroredTxs[id] = finalizedTx return id, nil }) } @@ -514,7 +506,6 @@ func (c *pendingTxContext) OnError(sig solana.Signature, retentionTimeout time.D delete(c.sigToID, s) } // if retention duration is set to 0, skip adding transaction to the errored map - // if retention duration is set to 0, skip adding transaction to the errored map if retentionTimeout == 0 { return id, nil } @@ -523,13 +514,8 @@ func (c *pendingTxContext) OnError(sig solana.Signature, retentionTimeout time.D retentionTs: time.Now().Add(retentionTimeout), rebroadcastCount: tx.rebroadcastCount, } - erroredTx := finishedTx{ - state: txState, - retentionTs: time.Now().Add(retentionTimeout), - } // move transaction from broadcasted to error map c.finalizedErroredTxs[id] = erroredTx - c.finalizedErroredTxs[id] = erroredTx return id, nil }) } @@ -565,31 +551,6 @@ func (c *pendingTxContext) TrimFinalizedErroredTxs() int { return 0 } - _, err = c.withWriteLock(func() (string, error) { - for _, id := range expiredIDs { - delete(c.finalizedErroredTxs, id) - } - return "", nil - }) - if err != nil { - return 0 - } - return len(expiredIDs) -func (c *pendingTxContext) TrimFinalizedErroredTxs() int { - var expiredIDs []string - err := c.withReadLock(func() error { - expiredIDs = make([]string, 0, len(c.finalizedErroredTxs)) - for id, tx := range c.finalizedErroredTxs { - if time.Now().After(tx.retentionTs) { - expiredIDs = append(expiredIDs, id) - } - } - return nil - }) - if err != nil { - return 0 - } - _, err = c.withWriteLock(func() (string, error) { for _, id := range expiredIDs { delete(c.finalizedErroredTxs, id) @@ -639,11 +600,7 @@ type pendingTxContextWithProm struct { type TxErrType int -type TxErrType int - const ( - NoFailure TxErrType = iota - TxFailRevert NoFailure TxErrType = iota TxFailRevert TxFailReject @@ -714,16 +671,6 @@ func (c *pendingTxContextWithProm) OnError(sig solana.Signature, retentionTimeou return id, err } -func (c *pendingTxContextWithProm) OnPrebroadcastError(id string, retentionTimeout time.Duration, txState TxState, errType TxErrType) error { - err := c.pendingTx.OnPrebroadcastError(id, retentionTimeout, txState, errType) // err indicates transaction not found so may already be removed -func (c *pendingTxContextWithProm) OnError(sig solana.Signature, retentionTimeout time.Duration, txState TxState, errType TxErrType) (string, error) { - id, err := c.pendingTx.OnError(sig, retentionTimeout, txState, errType) // err indicates transaction not found so may already be removed - if err == nil { - incrementErrorMetrics(errType, c.chainID) - } - return id, err -} - func (c *pendingTxContextWithProm) OnPrebroadcastError(id string, retentionTimeout time.Duration, txState TxState, errType TxErrType) error { err := c.pendingTx.OnPrebroadcastError(id, retentionTimeout, txState, errType) // err indicates transaction not found so may already be removed if err == nil { @@ -732,23 +679,8 @@ func (c *pendingTxContextWithProm) OnPrebroadcastError(id string, retentionTimeo return err } -func incrementErrorMetrics(errType TxErrType, chainID string) { - incrementErrorMetrics(errType, c.chainID) - } - return err -} - func incrementErrorMetrics(errType TxErrType, chainID string) { switch errType { - case NoFailure: - // Return early if no failure identified - return - case TxFailReject: - promSolTxmRejectTxs.WithLabelValues(chainID).Inc() - case TxFailRevert: - promSolTxmRevertTxs.WithLabelValues(chainID).Inc() - case TxFailDrop: - promSolTxmDropTxs.WithLabelValues(chainID).Inc() case NoFailure: // Return early if no failure identified return @@ -760,13 +692,10 @@ func incrementErrorMetrics(errType TxErrType, chainID string) { promSolTxmDropTxs.WithLabelValues(chainID).Inc() case TxFailSimRevert: promSolTxmSimRevertTxs.WithLabelValues(chainID).Inc() - promSolTxmSimRevertTxs.WithLabelValues(chainID).Inc() case TxFailSimOther: promSolTxmSimOtherTxs.WithLabelValues(chainID).Inc() - promSolTxmSimOtherTxs.WithLabelValues(chainID).Inc() } promSolTxmErrorTxs.WithLabelValues(chainID).Inc() - promSolTxmErrorTxs.WithLabelValues(chainID).Inc() } func (c *pendingTxContextWithProm) GetTxState(id string) (TxState, error) { diff --git a/pkg/solana/txm/pendingtx_test.go b/pkg/solana/txm/pendingtx_test.go index d7944b3d0..472651f26 100644 --- a/pkg/solana/txm/pendingtx_test.go +++ b/pkg/solana/txm/pendingtx_test.go @@ -294,7 +294,6 @@ func TestPendingTxContext_on_broadcasted_processed(t *testing.T) { // Transition to errored state id, err := txs.OnError(sig, retentionTimeout, Errored, 0) - id, err := txs.OnError(sig, retentionTimeout, Errored, 0) require.NoError(t, err) require.Equal(t, msg.id, id) @@ -407,7 +406,6 @@ func TestPendingTxContext_on_confirmed(t *testing.T) { // Transition to errored state id, err := txs.OnError(sig, retentionTimeout, Errored, 0) - id, err := txs.OnError(sig, retentionTimeout, Errored, 0) require.NoError(t, err) require.Equal(t, msg.id, id) @@ -586,7 +584,6 @@ func TestPendingTxContext_on_finalized(t *testing.T) { // Transition to errored state id, err := txs.OnError(sig, retentionTimeout, Errored, 0) - id, err := txs.OnError(sig, retentionTimeout, Errored, 0) require.NoError(t, err) require.Equal(t, msg.id, id) @@ -612,7 +609,6 @@ func TestPendingTxContext_on_error(t *testing.T) { // Transition to errored state id, err := txs.OnError(sig, retentionTimeout, Errored, 0) - id, err := txs.OnError(sig, retentionTimeout, Errored, 0) require.NoError(t, err) require.Equal(t, msg.id, id) @@ -651,7 +647,6 @@ func TestPendingTxContext_on_error(t *testing.T) { // Transition to errored state id, err = txs.OnError(sig, retentionTimeout, Errored, 0) - id, err = txs.OnError(sig, retentionTimeout, Errored, 0) require.NoError(t, err) require.Equal(t, msg.id, id) @@ -719,7 +714,6 @@ func TestPendingTxContext_on_error(t *testing.T) { // Transition to errored state id, err = txs.OnError(sig, 0*time.Second, Errored, 0) - id, err = txs.OnError(sig, 0*time.Second, Errored, 0) require.NoError(t, err) require.Equal(t, msg.id, id) @@ -755,7 +749,6 @@ func TestPendingTxContext_on_error(t *testing.T) { // Transition back to confirmed state id, err = txs.OnError(sig, retentionTimeout, Errored, 0) - id, err = txs.OnError(sig, retentionTimeout, Errored, 0) require.Error(t, err) require.Equal(t, "", id) }) @@ -825,70 +818,6 @@ func TestPendingTxContext_on_prebroadcast_error(t *testing.T) { }) } -func TestPendingTxContext_on_prebroadcast_error(t *testing.T) { - t.Parallel() - _, cancel := context.WithCancel(tests.Context(t)) - txs := newPendingTxContext() - retentionTimeout := 5 * time.Second - - t.Run("successfully adds transaction with errored state", func(t *testing.T) { - // Create new transaction - msg := pendingTx{id: uuid.NewString()} - // Transition to errored state - err := txs.OnPrebroadcastError(msg.id, retentionTimeout, Errored, 0) - require.NoError(t, err) - - // Check it exists in errored map - tx, exists := txs.finalizedErroredTxs[msg.id] - require.True(t, exists) - - // Check status is Errored - require.Equal(t, Errored, tx.state) - }) - - t.Run("successfully adds transaction with fatally errored state", func(t *testing.T) { - // Create new transaction - msg := pendingTx{id: uuid.NewString()} - - // Transition to fatally errored state - err := txs.OnPrebroadcastError(msg.id, retentionTimeout, FatallyErrored, 0) - require.NoError(t, err) - - // Check it exists in errored map - tx, exists := txs.finalizedErroredTxs[msg.id] - require.True(t, exists) - - // Check status is Errored - require.Equal(t, FatallyErrored, tx.state) - }) - - t.Run("fails to add transaction to errored map if id exists in another map already", func(t *testing.T) { - sig := randomSignature(t) - - // Create new transaction - msg := pendingTx{id: uuid.NewString()} - // Add transaction to broadcasted map - err := txs.New(msg, sig, cancel) - require.NoError(t, err) - - // Transition to errored state - err = txs.OnPrebroadcastError(msg.id, retentionTimeout, FatallyErrored, 0) - require.ErrorIs(t, err, ErrIDAlreadyExists) - }) - - t.Run("predefined error if transaction already in errored state", func(t *testing.T) { - txID := uuid.NewString() - - // Transition to errored state - err := txs.OnPrebroadcastError(txID, retentionTimeout, Errored, 0) - require.NoError(t, err) - - // Transition back to errored state - err = txs.OnPrebroadcastError(txID, retentionTimeout, Errored, 0) - require.ErrorIs(t, err, ErrAlreadyInExpectedState) - }) -} - func TestPendingTxContext_remove(t *testing.T) { t.Parallel() _, cancel := context.WithCancel(tests.Context(t)) @@ -939,7 +868,6 @@ func TestPendingTxContext_remove(t *testing.T) { err = txs.New(erroredMsg, erroredSig, cancel) require.NoError(t, err) id, err = txs.OnError(erroredSig, retentionTimeout, Errored, 0) - id, err = txs.OnError(erroredSig, retentionTimeout, Errored, 0) require.NoError(t, err) require.Equal(t, erroredMsg.id, id) @@ -1128,7 +1056,6 @@ func TestGetTxState(t *testing.T) { finalizedSig := randomSignature(t) erroredSig := randomSignature(t) fatallyErroredSig := randomSignature(t) - fatallyErroredSig := randomSignature(t) // Create new broadcasted transaction with extra sig broadcastedMsg := pendingTx{id: uuid.NewString()} @@ -1177,7 +1104,6 @@ func TestGetTxState(t *testing.T) { err = txs.New(erroredMsg, erroredSig, cancel) require.NoError(t, err) id, err = txs.OnError(erroredSig, retentionTimeout, Errored, 0) - id, err = txs.OnError(erroredSig, retentionTimeout, Errored, 0) require.NoError(t, err) require.Equal(t, erroredMsg.id, id) // Check Errored state is returned @@ -1197,18 +1123,6 @@ func TestGetTxState(t *testing.T) { require.NoError(t, err) require.Equal(t, FatallyErrored, state) - // Create new fatally errored transaction - fatallyErroredMsg := pendingTx{id: uuid.NewString()} - err = txs.New(fatallyErroredMsg, fatallyErroredSig, cancel) - require.NoError(t, err) - id, err = txs.OnError(fatallyErroredSig, retentionTimeout, FatallyErrored, 0) - require.NoError(t, err) - require.Equal(t, fatallyErroredMsg.id, id) - // Check Errored state is returned - state, err = txs.GetTxState(fatallyErroredMsg.id) - require.NoError(t, err) - require.Equal(t, FatallyErrored, state) - // Check NotFound state is returned if unknown id provided state, err = txs.GetTxState("unknown id") require.Error(t, err) diff --git a/pkg/solana/txm/txm_internal_test.go b/pkg/solana/txm/txm_internal_test.go index 181d4b727..6d7f979fe 100644 --- a/pkg/solana/txm/txm_internal_test.go +++ b/pkg/solana/txm/txm_internal_test.go @@ -138,7 +138,6 @@ func TestTxm(t *testing.T) { txm := NewTxm(id, loader, nil, cfg, mkey, lggr) require.NoError(t, txm.Start(ctx)) t.Cleanup(func() { require.NoError(t, txm.Close()) }) - t.Cleanup(func() { require.NoError(t, txm.Close()) }) // tracking prom metrics prom := soltxmProm{id: id} @@ -775,7 +774,6 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { txm := NewTxm(id, loader, nil, cfg, mkey, lggr) require.NoError(t, txm.Start(ctx)) t.Cleanup(func() { require.NoError(t, txm.Close()) }) - t.Cleanup(func() { require.NoError(t, txm.Close()) }) // tracking prom metrics prom := soltxmProm{id: id} @@ -819,24 +817,6 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { wg.Done() }).Return(&rpc.SimulateTransactionResult{}, nil).Once() - // handle signature status calls (initial stays processed, others don't exist) - start := time.Now() - statuses[sig] = func() (out *rpc.SignatureStatusesResult) { - out = &rpc.SignatureStatusesResult{} - // return confirmed status after default confirmation timeout - if time.Since(start) > 1*time.Second && time.Since(start) < 2*time.Second { - out.ConfirmationStatus = rpc.ConfirmationStatusConfirmed - return - } - // return finalized status only after the confirmation timeout - if time.Since(start) >= 2*time.Second { - out.ConfirmationStatus = rpc.ConfirmationStatusFinalized - wg.Done() - return - } - out.ConfirmationStatus = rpc.ConfirmationStatusProcessed - return - } // handle signature status calls (initial stays processed, others don't exist) start := time.Now() statuses[sig] = func() (out *rpc.SignatureStatusesResult) { @@ -862,31 +842,19 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { wg.Wait() // wait to be picked up and processed waitFor(t, 5*time.Second, txm, prom, empty) // inflight txs cleared after timeout - // panic if sendTx called after context cancelled - mc.On("SendTx", mock.Anything, tx).Panic("SendTx should not be called anymore").Maybe() // panic if sendTx called after context cancelled mc.On("SendTx", mock.Anything, tx).Panic("SendTx should not be called anymore").Maybe() - // check prom metric - prom.confirmed++ - prom.finalized++ - prom.assertEqual(t) // check prom metric prom.confirmed++ prom.finalized++ prom.assertEqual(t) - // check transaction status which should still be stored - status, err := txm.GetTransactionStatus(ctx, testTxID) - require.NoError(t, err) - require.Equal(t, types.Finalized, status) // check transaction status which should still be stored status, err := txm.GetTransactionStatus(ctx, testTxID) require.NoError(t, err) require.Equal(t, types.Finalized, status) - // Sleep until retention period has passed for transaction and for another reap cycle to run - time.Sleep(10 * time.Second) // Sleep until retention period has passed for transaction and for another reap cycle to run time.Sleep(10 * time.Second) @@ -935,95 +903,6 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { require.Equal(t, types.Unknown, status) }) - t.Run("stores error if confirmation returns error", func(t *testing.T) { - // Test tx is not discarded due to confirm timeout and tracked to finalization - // use unique val across tests to avoid collision during mocking - tx, signed := getTx(t, 3, mkey) - sig := randomSignature(t) - var wg sync.WaitGroup - wg.Add(2) - - mc.On("SendTx", mock.Anything, signed(0, true, computeUnitLimitDefault)).Return(sig, nil) - mc.On("SimulateTx", mock.Anything, signed(0, true, computeUnitLimitDefault), mock.Anything).Run(func(mock.Arguments) { - wg.Done() - }).Return(&rpc.SimulateTransactionResult{}, nil).Once() - statuses[sig] = func() (out *rpc.SignatureStatusesResult) { - defer wg.Done() - return &rpc.SignatureStatusesResult{Err: errors.New("InstructionError")} - } - - // tx should be able to queue - testTxID := uuid.NewString() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) - wg.Wait() // wait till send tx - waitFor(t, 5*time.Second, txm, prom, empty) // inflight txs cleared after timeout - - // panic if sendTx called after context cancelled - mc.On("SendTx", mock.Anything, tx).Panic("SendTx should not be called anymore").Maybe() - - // check prom metric - prom.error++ - prom.revert++ - prom.assertEqual(t) - - // check transaction status which should still be stored - status, err := txm.GetTransactionStatus(ctx, testTxID) - require.NoError(t, err) - require.Equal(t, types.Failed, status) - - // Sleep until retention period has passed for transaction and for another reap cycle to run - time.Sleep(15 * time.Second) - - // check if transaction has been purged from memory - status, err = txm.GetTransactionStatus(ctx, testTxID) - require.Error(t, err) - require.Equal(t, types.Unknown, status) - }) - // check if transaction has been purged from memory - status, err = txm.GetTransactionStatus(ctx, testTxID) - require.Error(t, err) - require.Equal(t, types.Unknown, status) - }) - - t.Run("stores error if initial send fails", func(t *testing.T) { - // Test tx is not discarded due to confirm timeout and tracked to finalization - // use unique val across tests to avoid collision during mocking - tx, signed := getTx(t, 2, mkey) - var wg sync.WaitGroup - wg.Add(1) - - mc.On("SendTx", mock.Anything, signed(0, true, computeUnitLimitDefault)).Run(func(mock.Arguments) { - wg.Done() - }).Return(nil, errors.New("failed to send")) - - // tx should be able to queue - testTxID := uuid.NewString() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) - wg.Wait() - waitFor(t, 5*time.Second, txm, prom, empty) // inflight txs cleared after timeout - - // panic if sendTx called after context cancelled - mc.On("SendTx", mock.Anything, tx).Panic("SendTx should not be called anymore").Maybe() - - // check prom metric - prom.error++ - prom.reject++ - prom.assertEqual(t) - - // check transaction status which should still be stored - status, err := txm.GetTransactionStatus(ctx, testTxID) - require.NoError(t, err) - require.Equal(t, types.Failed, status) - - // Sleep until retention period has passed for transaction and for another reap cycle to run - time.Sleep(15 * time.Second) - - // check if transaction has been purged from memory - status, err = txm.GetTransactionStatus(ctx, testTxID) - require.Error(t, err) - require.Equal(t, types.Unknown, status) - }) - t.Run("stores error if confirmation returns error", func(t *testing.T) { // Test tx is not discarded due to confirm timeout and tracked to finalization // use unique val across tests to avoid collision during mocking @@ -1099,7 +978,6 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { txm := NewTxm(id, loader, nil, cfg, mkey, lggr) require.NoError(t, txm.Start(ctx)) t.Cleanup(func() { require.NoError(t, txm.Close()) }) - t.Cleanup(func() { require.NoError(t, txm.Close()) }) // tracking prom metrics prom := soltxmProm{id: id} @@ -1129,7 +1007,6 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { t.Run("simulation_succeeds", func(t *testing.T) { // Test tx is not discarded due to confirm timeout and tracked to finalization // use unique val across tests to avoid collision during mocking - // use unique val across tests to avoid collision during mocking tx, signed := getTx(t, 1, mkey) // add signature and compute unit limit to tx for simulation (excludes compute unit price) simulateTx := addSigAndLimitToTx(t, mkey, solana.PublicKey{}, *tx, MaxComputeUnitLimit) @@ -1195,8 +1072,6 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { // Test tx is not discarded due to confirm timeout and tracked to finalization // use unique val across tests to avoid collision during mocking tx, signed := getTx(t, 2, mkey) - // use unique val across tests to avoid collision during mocking - tx, signed := getTx(t, 2, mkey) sig := randomSignature(t) mc.On("SendTx", mock.Anything, signed(0, true, fees.ComputeUnitLimit(0))).Return(sig, nil).Panic("SendTx should never be called").Maybe() @@ -1210,17 +1085,13 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { // Test tx is not discarded due to confirm timeout and tracked to finalization // use unique val across tests to avoid collision during mocking tx, _ := getTx(t, 3, mkey) - // use unique val across tests to avoid collision during mocking - tx, _ := getTx(t, 3, mkey) // add signature and compute unit limit to tx for simulation (excludes compute unit price) simulateTx := addSigAndLimitToTx(t, mkey, solana.PublicKey{}, *tx, MaxComputeUnitLimit) sig := randomSignature(t) mc.On("SendTx", mock.Anything, mock.Anything).Return(sig, nil).Panic("SendTx should never be called").Maybe() // First simulation before broadcast with max compute unit limit mc.On("SimulateTx", mock.Anything, simulateTx, mock.Anything).Return(&rpc.SimulateTransactionResult{Err: errors.New("InstructionError")}, nil).Once() - mc.On("SimulateTx", mock.Anything, simulateTx, mock.Anything).Return(&rpc.SimulateTransactionResult{Err: errors.New("InstructionError")}, nil).Once() - txID := uuid.NewString() txID := uuid.NewString() // tx should NOT be able to queue assert.Error(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) From 849ac48560f442746cc3af36c76eed7e0e092931 Mon Sep 17 00:00:00 2001 From: Farber98 Date: Tue, 26 Nov 2024 17:56:01 -0300 Subject: [PATCH 34/51] fix whitespace diffs --- pkg/solana/chain_test.go | 3 +++ pkg/solana/txm/txm_internal_test.go | 10 ++++++++++ 2 files changed, 13 insertions(+) diff --git a/pkg/solana/chain_test.go b/pkg/solana/chain_test.go index 0dd4b0df3..fb56db996 100644 --- a/pkg/solana/chain_test.go +++ b/pkg/solana/chain_test.go @@ -534,6 +534,7 @@ func TestSolanaChain_MultiNode_Txm(t *testing.T) { receiverBal, err := selectedClient.Balance(tests.Context(t), pubKeyReceiver) assert.NoError(t, err) assert.Equal(t, uint64(0), receiverBal) + createTx := func(signer solana.PublicKey, sender solana.PublicKey, receiver solana.PublicKey, amt uint64) *solana.Transaction { selectedClient, err = testChain.getClient() assert.NoError(t, err) @@ -556,6 +557,7 @@ func TestSolanaChain_MultiNode_Txm(t *testing.T) { // Send funds twice, along with an invalid transaction require.NoError(t, testChain.txm.Enqueue(tests.Context(t), "test_success", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) + // Wait for new block hash currentBh, err := selectedClient.LatestBlockhash(tests.Context(t)) require.NoError(t, err) @@ -577,6 +579,7 @@ NewBlockHash: require.NoError(t, testChain.txm.Enqueue(tests.Context(t), "test_success_2", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) require.Error(t, testChain.txm.Enqueue(tests.Context(t), "test_invalidSigner", createTx(pubKeyReceiver, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) // cannot sign tx before enqueuing + // wait for all txes to finish ctx, cancel := context.WithCancel(tests.Context(t)) t.Cleanup(cancel) diff --git a/pkg/solana/txm/txm_internal_test.go b/pkg/solana/txm/txm_internal_test.go index 6d7f979fe..5eb1d31a0 100644 --- a/pkg/solana/txm/txm_internal_test.go +++ b/pkg/solana/txm/txm_internal_test.go @@ -259,12 +259,14 @@ func TestTxm(t *testing.T) { _, err := txm.GetTransactionStatus(ctx, testTxID) require.Error(t, err) // transaction cleared from storage after finalized should not return status }) + // tx fails simulation (simulation error) t.Run("fail_simulation", func(t *testing.T) { tx, signed := getTx(t, 2, mkey) sig := randomSignature(t) var wg sync.WaitGroup wg.Add(1) + mc.On("SendTx", mock.Anything, signed(0, true, computeUnitLimitDefault)).Return(sig, nil) mc.On("SimulateTx", mock.Anything, signed(0, true, computeUnitLimitDefault), mock.Anything).Run(func(mock.Arguments) { wg.Done() @@ -298,6 +300,7 @@ func TestTxm(t *testing.T) { retry3 := randomSignature(t) var wg sync.WaitGroup wg.Add(1) + mc.On("SendTx", mock.Anything, signed(0, true, computeUnitLimitDefault)).Return(sig, nil) mc.On("SendTx", mock.Anything, signed(1, true, computeUnitLimitDefault)).Return(retry0, nil) mc.On("SendTx", mock.Anything, signed(2, true, computeUnitLimitDefault)).Return(retry1, nil) @@ -365,6 +368,7 @@ func TestTxm(t *testing.T) { // panic if sendTx called after context cancelled mc.On("SendTx", mock.Anything, tx).Panic("SendTx should not be called anymore").Maybe() }) + // tx fails simulation with BlockHashNotFound error // txm should continue to finalize tx (in this case it will succeed) t.Run("fail_simulation_blockhashNotFound", func(t *testing.T) { @@ -372,6 +376,7 @@ func TestTxm(t *testing.T) { sig := randomSignature(t) var wg sync.WaitGroup wg.Add(2) + mc.On("SendTx", mock.Anything, signed(0, true, computeUnitLimitDefault)).Return(sig, nil) mc.On("SimulateTx", mock.Anything, signed(0, true, computeUnitLimitDefault), mock.Anything).Run(func(mock.Arguments) { wg.Done() @@ -422,6 +427,7 @@ func TestTxm(t *testing.T) { sig := randomSignature(t) var wg sync.WaitGroup wg.Add(2) + mc.On("SendTx", mock.Anything, signed(0, true, computeUnitLimitDefault)).Return(sig, nil) mc.On("SimulateTx", mock.Anything, signed(0, true, computeUnitLimitDefault), mock.Anything).Run(func(mock.Arguments) { wg.Done() @@ -466,6 +472,7 @@ func TestTxm(t *testing.T) { retry3 := randomSignature(t) var wg sync.WaitGroup wg.Add(1) + mc.On("SendTx", mock.Anything, signed(0, true, computeUnitLimitDefault)).Return(sig, nil) mc.On("SendTx", mock.Anything, signed(1, true, computeUnitLimitDefault)).Return(retry0, nil) mc.On("SendTx", mock.Anything, signed(2, true, computeUnitLimitDefault)).Return(retry1, nil) @@ -510,6 +517,7 @@ func TestTxm(t *testing.T) { retry3 := randomSignature(t) var wg sync.WaitGroup wg.Add(1) + mc.On("SendTx", mock.Anything, signed(0, true, computeUnitLimitDefault)).Return(sig, nil) mc.On("SendTx", mock.Anything, signed(1, true, computeUnitLimitDefault)).Return(retry0, nil) mc.On("SendTx", mock.Anything, signed(2, true, computeUnitLimitDefault)).Return(retry1, nil) @@ -557,6 +565,7 @@ func TestTxm(t *testing.T) { sig := randomSignature(t) var wg sync.WaitGroup wg.Add(1) + mc.On("SendTx", mock.Anything, signed(0, true, computeUnitLimitDefault)).Return(sig, nil) mc.On("SimulateTx", mock.Anything, signed(0, true, computeUnitLimitDefault), mock.Anything).Run(func(mock.Arguments) { wg.Done() @@ -598,6 +607,7 @@ func TestTxm(t *testing.T) { retry3 := randomSignature(t) var wg sync.WaitGroup wg.Add(2) + mc.On("SendTx", mock.Anything, signed(0, true, computeUnitLimitDefault)).Return(sig, nil) mc.On("SendTx", mock.Anything, signed(1, true, computeUnitLimitDefault)).Return(retry0, nil) mc.On("SendTx", mock.Anything, signed(2, true, computeUnitLimitDefault)).Return(retry1, nil) From 20a15482292e06a2dfaaa794b0c9c129613f2576 Mon Sep 17 00:00:00 2001 From: Farber98 Date: Tue, 26 Nov 2024 18:07:26 -0300 Subject: [PATCH 35/51] fix import --- pkg/solana/txm/txm_race_test.go | 52 ++++++++++++++++----------------- 1 file changed, 26 insertions(+), 26 deletions(-) diff --git a/pkg/solana/txm/txm_race_test.go b/pkg/solana/txm/txm_race_test.go index fd05587d6..7abde910e 100644 --- a/pkg/solana/txm/txm_race_test.go +++ b/pkg/solana/txm/txm_race_test.go @@ -8,8 +8,8 @@ import ( "testing" "time" - "github.com/gagliardetto/solana-go" - "github.com/gagliardetto/solana-go/rpc" + solanaGo "github.com/gagliardetto/solana-go" + solanaGoRpc "github.com/gagliardetto/solana-go/rpc" "go.uber.org/zap/zapcore" "github.com/smartcontractkit/chainlink-common/pkg/logger" @@ -29,8 +29,8 @@ import ( ) func NewTestMsg() (msg pendingTx) { - tx := solana.Transaction{} - tx.Message.AccountKeys = append(tx.Message.AccountKeys, solana.PublicKey{}) + tx := solanaGo.Transaction{} + tx.Message.AccountKeys = append(tx.Message.AccountKeys, solanaGo.PublicKey{}) msg.tx = tx return msg } @@ -82,19 +82,19 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { assert.Contains(t, lastLog.Message, "stopped tx retry") // assert that all retry goroutines exit successfully } client := clientmocks.NewReaderWriter(t) - client.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ + client.On("LatestBlockhash", mock.Anything).Return(&solanaGoRpc.GetLatestBlockhashResult{ + Value: &solanaGoRpc.LatestBlockhashResult{ LastValidBlockHeight: 100, - Blockhash: solana.Hash{}, + Blockhash: solanaGo.Hash{}, }, }, nil) t.Run("delay in rebroadcasting tx", func(t *testing.T) { - txs := map[string]solana.Signature{} + txs := map[string]solanaGo.Signature{} var lock sync.RWMutex client.On("SendTx", mock.Anything, mock.Anything).Return( // build new sig if tx is different - func(_ context.Context, tx *solana.Transaction) solana.Signature { + func(_ context.Context, tx *solanaGo.Transaction) solanaGo.Signature { strTx := tx.String() // if exists, slow down client response to trigger race @@ -116,7 +116,7 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { sig := make([]byte, 16) _, err := rand.Read(sig) require.NoError(t, err) - txs[strTx] = solana.SignatureFromBytes(sig) + txs[strTx] = solanaGo.SignatureFromBytes(sig) return txs[strTx] }, @@ -126,11 +126,11 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { }) t.Run("delay in broadcasting new tx", func(t *testing.T) { - txs := map[string]solana.Signature{} + txs := map[string]solanaGo.Signature{} var lock sync.RWMutex client.On("SendTx", mock.Anything, mock.Anything).Return( // build new sig if tx is different - func(_ context.Context, tx *solana.Transaction) solana.Signature { + func(_ context.Context, tx *solanaGo.Transaction) solanaGo.Signature { strTx := tx.String() lock.Lock() @@ -143,7 +143,7 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { sig := make([]byte, 16) _, err := rand.Read(sig) require.NoError(t, err) - txs[strTx] = solana.SignatureFromBytes(sig) + txs[strTx] = solanaGo.SignatureFromBytes(sig) lock.Unlock() // don't lock on delay @@ -160,11 +160,11 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { }) t.Run("overlapping bumping tx", func(t *testing.T) { - txs := map[string]solana.Signature{} + txs := map[string]solanaGo.Signature{} var lock sync.RWMutex client.On("SendTx", mock.Anything, mock.Anything).Return( // build new sig if tx is different - func(_ context.Context, tx *solana.Transaction) solana.Signature { + func(_ context.Context, tx *solanaGo.Transaction) solanaGo.Signature { strTx := tx.String() lock.Lock() @@ -177,7 +177,7 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { sig := make([]byte, 16) _, err := rand.Read(sig) require.NoError(t, err) - txs[strTx] = solana.SignatureFromBytes(sig) + txs[strTx] = solanaGo.SignatureFromBytes(sig) triggerDelay := len(txs) == 2 lock.Unlock() @@ -209,31 +209,31 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { msg0 := NewTestMsg() require.NoError(t, fees.SetComputeUnitPrice(&msg0.tx, 0)) require.NoError(t, fees.SetComputeUnitLimit(&msg0.tx, 200_000)) - msg0.tx.Signatures = make([]solana.Signature, 1) - client.On("SendTx", mock.Anything, &msg0.tx).Return(solana.Signature{1}, nil) + msg0.tx.Signatures = make([]solanaGo.Signature, 1) + client.On("SendTx", mock.Anything, &msg0.tx).Return(solanaGo.Signature{1}, nil) // init bump tx fails, rebroadcast is successful msg1 := NewTestMsg() require.NoError(t, fees.SetComputeUnitPrice(&msg1.tx, 1)) require.NoError(t, fees.SetComputeUnitLimit(&msg1.tx, 200_000)) - msg1.tx.Signatures = make([]solana.Signature, 1) - client.On("SendTx", mock.Anything, &msg1.tx).Return(solana.Signature{}, fmt.Errorf("BUMP FAILED")) - client.On("SendTx", mock.Anything, &msg1.tx).Return(solana.Signature{2}, nil) + msg1.tx.Signatures = make([]solanaGo.Signature, 1) + client.On("SendTx", mock.Anything, &msg1.tx).Return(solanaGo.Signature{}, fmt.Errorf("BUMP FAILED")) + client.On("SendTx", mock.Anything, &msg1.tx).Return(solanaGo.Signature{2}, nil) // init bump tx success, rebroadcast fails msg2 := NewTestMsg() require.NoError(t, fees.SetComputeUnitPrice(&msg2.tx, 2)) require.NoError(t, fees.SetComputeUnitLimit(&msg2.tx, 200_000)) - msg2.tx.Signatures = make([]solana.Signature, 1) - client.On("SendTx", mock.Anything, &msg2.tx).Return(solana.Signature{3}, nil) - client.On("SendTx", mock.Anything, &msg2.tx).Return(solana.Signature{}, fmt.Errorf("REBROADCAST FAILED")) + msg2.tx.Signatures = make([]solanaGo.Signature, 1) + client.On("SendTx", mock.Anything, &msg2.tx).Return(solanaGo.Signature{3}, nil) + client.On("SendTx", mock.Anything, &msg2.tx).Return(solanaGo.Signature{}, fmt.Errorf("REBROADCAST FAILED")) // always successful msg3 := NewTestMsg() require.NoError(t, fees.SetComputeUnitPrice(&msg3.tx, 4)) require.NoError(t, fees.SetComputeUnitLimit(&msg3.tx, 200_000)) - msg3.tx.Signatures = make([]solana.Signature, 1) - client.On("SendTx", mock.Anything, &msg3.tx).Return(solana.Signature{4}, nil) + msg3.tx.Signatures = make([]solanaGo.Signature, 1) + client.On("SendTx", mock.Anything, &msg3.tx).Return(solanaGo.Signature{4}, nil) testRunner(t, client) }) } From a4d477027a9866a54bfe15660422205e347fa044 Mon Sep 17 00:00:00 2001 From: Farber98 Date: Tue, 26 Nov 2024 18:10:00 -0300 Subject: [PATCH 36/51] fix mocks --- pkg/solana/config/mocks/config.go | 35 +++++++++++++++++++------------ 1 file changed, 22 insertions(+), 13 deletions(-) diff --git a/pkg/solana/config/mocks/config.go b/pkg/solana/config/mocks/config.go index e4a17dd5e..0ea855b0f 100644 --- a/pkg/solana/config/mocks/config.go +++ b/pkg/solana/config/mocks/config.go @@ -762,22 +762,31 @@ func (_m *Config) TxConfirmTimeout() time.Duration { return r0 } -// TxExpirationRebroadcast provides a mock function with given fields: -func (_m *Config) TxExpirationRebroadcast() bool { - ret := _m.Called() +// Config_TxConfirmTimeout_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'TxConfirmTimeout' +type Config_TxConfirmTimeout_Call struct { + *mock.Call +} - if len(ret) == 0 { - panic("no return value specified for TxExpirationRebroadcast") - } +// TxConfirmTimeout is a helper method to define mock.On call +func (_e *Config_Expecter) TxConfirmTimeout() *Config_TxConfirmTimeout_Call { + return &Config_TxConfirmTimeout_Call{Call: _e.mock.On("TxConfirmTimeout")} +} - var r0 bool - if rf, ok := ret.Get(0).(func() bool); ok { - r0 = rf() - } else { - r0 = ret.Get(0).(bool) - } +func (_c *Config_TxConfirmTimeout_Call) Run(run func()) *Config_TxConfirmTimeout_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} - return r0 +func (_c *Config_TxConfirmTimeout_Call) Return(_a0 time.Duration) *Config_TxConfirmTimeout_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *Config_TxConfirmTimeout_Call) RunAndReturn(run func() time.Duration) *Config_TxConfirmTimeout_Call { + _c.Call.Return(run) + return _c } // TxExpirationRebroadcast provides a mock function with given fields: From 56a64dad5f035dc483e52a724cf563b85560bc3f Mon Sep 17 00:00:00 2001 From: Farber98 Date: Tue, 26 Nov 2024 18:40:48 -0300 Subject: [PATCH 37/51] add on prebroadcaste error --- pkg/solana/txm/txm.go | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index 7e0862481..27f2e641e 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -582,7 +582,8 @@ func (txm *Txm) rebroadcastExpiredTxs(ctx context.Context, client client.ReaderW txm.lggr.Errorw("no signatures found for expired transaction", "id", tx.id) continue } - _, err := txm.txs.Remove(tx.signatures[0]) // only picking signature[0]. Remove func removes all related remaining signatures. + // only picking signature[0]. Remove func removes all related remaining signatures and cancels tx context. + _, err := txm.txs.Remove(tx.signatures[0]) if err != nil { txm.lggr.Errorw("failed to remove expired transaction", "id", tx.id, "error", err) continue @@ -594,10 +595,10 @@ func (txm *Txm) rebroadcastExpiredTxs(ctx context.Context, client client.ReaderW rebroadcastCount: tx.rebroadcastCount + 1, } // call sendWithRetry directly to avoid enqueuing - _, _, _, err = txm.sendWithRetry(ctx, rebroadcastTx) - if err != nil { - // TODO: add prebroadcast error handling when merged https://github.com/smartcontractkit/chainlink-solana/pull/936 - txm.lggr.Errorw("failed to rebroadcast transaction", "id", tx.id, "error", err) + _, _, _, sendErr := txm.sendWithRetry(ctx, rebroadcastTx) + if sendErr != nil { + stateTransitionErr := txm.txs.OnPrebroadcastError(tx.id, txm.cfg.TxRetentionTimeout(), Errored, TxFailReject) + txm.lggr.Errorw("failed to rebroadcast transaction", "id", tx.id, "error", errors.Join(sendErr, stateTransitionErr)) continue } From 9148d7d9b923d8288df0ed8cce5f823018d2e6ca Mon Sep 17 00:00:00 2001 From: Farber98 Date: Wed, 27 Nov 2024 11:55:05 -0300 Subject: [PATCH 38/51] remove rebroadcast count and fix package --- pkg/solana/txm/pendingtx.go | 43 ++++++----------------------- pkg/solana/txm/txm.go | 7 ++--- pkg/solana/txm/txm_internal_test.go | 31 +++++++++------------ pkg/solana/txm/txm_load_test.go | 5 ++-- 4 files changed, 27 insertions(+), 59 deletions(-) diff --git a/pkg/solana/txm/pendingtx.go b/pkg/solana/txm/pendingtx.go index 540a1221c..44895fd40 100644 --- a/pkg/solana/txm/pendingtx.go +++ b/pkg/solana/txm/pendingtx.go @@ -47,8 +47,6 @@ type PendingTxContext interface { GetTxState(id string) (TxState, error) // TrimFinalizedErroredTxs removes transactions that have reached their retention time TrimFinalizedErroredTxs() int - // GetTxRebroadcastCount returns the number of times a transaction has been rebroadcasted if found. - GetTxRebroadcastCount(id string) (int, error) } // finishedTx is used to store info required to track transactions to finality or error @@ -57,7 +55,6 @@ type pendingTx struct { cfg TxConfig signatures []solana.Signature id string - rebroadcastCount int createTs time.Time state TxState lastValidBlockHeight uint64 // to track expiration @@ -65,9 +62,8 @@ type pendingTx struct { // finishedTx is used to store minimal info specifically for finalized or errored transactions for external status checks type finishedTx struct { - retentionTs time.Time - state TxState - rebroadcastCount int + retentionTs time.Time + state TxState } var _ PendingTxContext = &pendingTxContext{} @@ -401,9 +397,8 @@ func (c *pendingTxContext) OnFinalized(sig solana.Signature, retentionTimeout ti return id, nil } finalizedTx := finishedTx{ - state: Finalized, - retentionTs: time.Now().Add(retentionTimeout), - rebroadcastCount: tx.rebroadcastCount, + state: Finalized, + retentionTs: time.Now().Add(retentionTimeout), } // move transaction from confirmed to finalized map c.finalizedErroredTxs[id] = finalizedTx @@ -443,9 +438,8 @@ func (c *pendingTxContext) OnPrebroadcastError(id string, retentionTimeout time. return "", ErrIDAlreadyExists } erroredTx := finishedTx{ - state: txState, - retentionTs: time.Now().Add(retentionTimeout), - rebroadcastCount: tx.rebroadcastCount, + state: txState, + retentionTs: time.Now().Add(retentionTimeout), } // add transaction to error map c.finalizedErroredTxs[id] = erroredTx @@ -510,9 +504,8 @@ func (c *pendingTxContext) OnError(sig solana.Signature, retentionTimeout time.D return id, nil } erroredTx := finishedTx{ - state: txState, - retentionTs: time.Now().Add(retentionTimeout), - rebroadcastCount: tx.rebroadcastCount, + state: txState, + retentionTs: time.Now().Add(retentionTimeout), } // move transaction from broadcasted to error map c.finalizedErroredTxs[id] = erroredTx @@ -575,22 +568,6 @@ func (c *pendingTxContext) withWriteLock(fn func() (string, error)) (string, err return fn() } -// GetTxRebroadcastCount returns the number of times a transaction has been rebroadcasted if found. -func (c *pendingTxContext) GetTxRebroadcastCount(id string) (int, error) { - c.lock.RLock() - defer c.lock.RUnlock() - if tx, exists := c.broadcastedProcessedTxs[id]; exists { - return tx.rebroadcastCount, nil - } - if tx, exists := c.confirmedTxs[id]; exists { - return tx.rebroadcastCount, nil - } - if tx, exists := c.finalizedErroredTxs[id]; exists { - return tx.rebroadcastCount, nil - } - return 0, fmt.Errorf("failed to find transaction for id: %s", id) -} - var _ PendingTxContext = &pendingTxContextWithProm{} type pendingTxContextWithProm struct { @@ -705,7 +682,3 @@ func (c *pendingTxContextWithProm) GetTxState(id string) (TxState, error) { func (c *pendingTxContextWithProm) TrimFinalizedErroredTxs() int { return c.pendingTx.TrimFinalizedErroredTxs() } - -func (c *pendingTxContextWithProm) GetTxRebroadcastCount(id string) (int, error) { - return c.pendingTx.GetTxRebroadcastCount(id) -} diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index 27f2e641e..3ba39f2f5 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -589,10 +589,9 @@ func (txm *Txm) rebroadcastExpiredTxs(ctx context.Context, client client.ReaderW continue } rebroadcastTx := pendingTx{ - tx: tx.tx, - cfg: tx.cfg, - id: tx.id, // using same id in case it was set by caller and we need to maintain it. - rebroadcastCount: tx.rebroadcastCount + 1, + tx: tx.tx, + cfg: tx.cfg, + id: tx.id, // using same id in case it was set by caller and we need to maintain it. } // call sendWithRetry directly to avoid enqueuing _, _, _, sendErr := txm.sendWithRetry(ctx, rebroadcastTx) diff --git a/pkg/solana/txm/txm_internal_test.go b/pkg/solana/txm/txm_internal_test.go index 5eb1d31a0..95017a29e 100644 --- a/pkg/solana/txm/txm_internal_test.go +++ b/pkg/solana/txm/txm_internal_test.go @@ -1362,13 +1362,11 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { prom.finalized++ prom.assertEqual(t) - // Check that transaction for txID has been finalized and rebroadcasted + // Check that transaction for txID has been finalized and rebroadcasted 1 time. status, err := txm.GetTransactionStatus(ctx, txID) require.NoError(t, err) require.Equal(t, types.Finalized, status) - rebroadcastCount, err := txm.txs.GetTxRebroadcastCount(txID) - require.NoError(t, err) - require.Equal(t, 1, rebroadcastCount) + require.Equal(t, 1, callCount-1) // -1 because the first call is not a rebroadcast }) t.Run("WithoutRebroadcast", func(t *testing.T) { @@ -1376,7 +1374,9 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { statuses := map[solana.Signature]func() *rpc.SignatureStatusesResult{} // mocking the call within sendWithRetry. Rebroadcast is off, so we won't compare it against the slotHeight. + callCount := 0 latestBlockhashFunc := func() (*rpc.GetLatestBlockhashResult, error) { + defer func() { callCount++ }() return &rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ LastValidBlockHeight: uint64(2000), @@ -1419,9 +1419,7 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { status, err := txm.GetTransactionStatus(ctx, txID) require.NoError(t, err) require.Equal(t, types.Failed, status) - rebroadcastCount, err := txm.txs.GetTxRebroadcastCount(txID) - require.NoError(t, err) - require.Equal(t, 0, rebroadcastCount) + require.Equal(t, 0, callCount-1) // -1 because the first call is not a rebroadcast }) t.Run("WithMultipleRebroadcast", func(t *testing.T) { @@ -1495,13 +1493,11 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { prom.finalized++ prom.assertEqual(t) - // Check that transaction for txID has been finalized and rebroadcasted + // Check that transaction for txID has been finalized and rebroadcasted multiple times. status, err := txm.GetTransactionStatus(ctx, txID) require.NoError(t, err) require.Equal(t, types.Finalized, status) - rebroadcastCount, err := txm.txs.GetTxRebroadcastCount(txID) - require.NoError(t, err) - require.Equal(t, expectedRebroadcastsCount, rebroadcastCount) + require.Equal(t, expectedRebroadcastsCount, callCount-1) }) t.Run("ConfirmedBeforeRebroadcast", func(t *testing.T) { @@ -1517,7 +1513,10 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { slotHeightFunc := func() (uint64, error) { return uint64(1500), nil } + + callCount := 0 latestBlockhashFunc := func() (*rpc.GetLatestBlockhashResult, error) { + defer func() { callCount++ }() return &rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ LastValidBlockHeight: uint64(1000), @@ -1561,9 +1560,7 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { status, err := txm.GetTransactionStatus(ctx, txID) require.NoError(t, err) require.Equal(t, types.Finalized, status) - rebroadcastCount, err := txm.txs.GetTxRebroadcastCount(txID) - require.NoError(t, err) - require.Equal(t, 0, rebroadcastCount) + require.Equal(t, 0, callCount-1) // -1 because the first call is not a rebroadcast }) t.Run("RebroadcastWithError", func(t *testing.T) { @@ -1621,12 +1618,10 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { prom.error++ prom.assertEqual(t) - // Transaction should be moved to failed after trying to rebroadcast and failing to get confirmations + // Transaction should be moved to failed after trying to rebroadcast 1 time. status, err := txm.GetTransactionStatus(ctx, txID) require.NoError(t, err) require.Equal(t, types.Failed, status) - rebroadcastCount, err := txm.txs.GetTxRebroadcastCount(txID) - require.NoError(t, err) - require.Equal(t, 1, rebroadcastCount) + require.Equal(t, 1, callCount-1) // -1 because the first call is not a rebroadcast }) } diff --git a/pkg/solana/txm/txm_load_test.go b/pkg/solana/txm/txm_load_test.go index 3d4941374..aa3d6aac7 100644 --- a/pkg/solana/txm/txm_load_test.go +++ b/pkg/solana/txm/txm_load_test.go @@ -1,6 +1,6 @@ //go:build integration -package txm +package txm_test import ( "context" @@ -16,6 +16,7 @@ import ( solanaClient "github.com/smartcontractkit/chainlink-solana/pkg/solana/client" "github.com/smartcontractkit/chainlink-solana/pkg/solana/config" + "github.com/smartcontractkit/chainlink-solana/pkg/solana/txm" keyMocks "github.com/smartcontractkit/chainlink-solana/pkg/solana/txm/mocks" relayconfig "github.com/smartcontractkit/chainlink-common/pkg/config" @@ -71,7 +72,7 @@ func TestTxm_Integration(t *testing.T) { client, err := solanaClient.NewClient(url, cfg, 2*time.Second, lggr) require.NoError(t, err) loader := utils.NewLazyLoad(func() (solanaClient.ReaderWriter, error) { return client, nil }) - txm := NewTxm("localnet", loader, nil, cfg, mkey, lggr) + txm := txm.NewTxm("localnet", loader, nil, cfg, mkey, lggr) // track initial balance initBal, err := client.Balance(ctx, pubKey) From caf2cbfb3c289ee5fbf4d129e1c372a95ea23eec Mon Sep 17 00:00:00 2001 From: Farber98 Date: Wed, 27 Nov 2024 12:23:26 -0300 Subject: [PATCH 39/51] improve docs --- docs/relay/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/relay/README.md b/docs/relay/README.md index 72057c732..2191e985a 100644 --- a/docs/relay/README.md +++ b/docs/relay/README.md @@ -44,7 +44,7 @@ chainlink nodes solana create --name= --chain-id= --url= Date: Thu, 5 Dec 2024 12:55:35 -0300 Subject: [PATCH 40/51] fix comparison against blockHeight instead of slotHeight --- pkg/solana/txm/pendingtx.go | 14 +++---- pkg/solana/txm/txm.go | 12 +++--- pkg/solana/txm/txm_internal_test.go | 61 ++++++++++++++++++----------- 3 files changed, 51 insertions(+), 36 deletions(-) diff --git a/pkg/solana/txm/pendingtx.go b/pkg/solana/txm/pendingtx.go index 44895fd40..5808f6239 100644 --- a/pkg/solana/txm/pendingtx.go +++ b/pkg/solana/txm/pendingtx.go @@ -28,9 +28,9 @@ type PendingTxContext interface { Remove(sig solana.Signature) (string, error) // ListAll returns all of the signatures being tracked for all transactions not yet finalized or errored ListAll() []solana.Signature - // ListAllExpiredBroadcastedTxs returns all the txes that are in broadcasted state and have expired for given slot height compared against their lastValidBlockHeight. + // ListAllExpiredBroadcastedTxs returns all the txes that are in broadcasted state and have expired for given block height compared against their lastValidBlockHeight. // Passing maxUint64 as currHeight will return all broadcasted txes. - ListAllExpiredBroadcastedTxs(currHeight uint64) []pendingTx + ListAllExpiredBroadcastedTxs(currBlockHeight uint64) []pendingTx // Expired returns whether or not confirmation timeout amount of time has passed since creation Expired(sig solana.Signature, confirmationTimeout time.Duration) bool // OnProcessed marks transactions as Processed @@ -221,14 +221,14 @@ func (c *pendingTxContext) ListAll() []solana.Signature { return maps.Keys(c.sigToID) } -// ListAllExpiredBroadcastedTxs returns all the txes that are in broadcasted state and have expired for given slot height compared against their lastValidBlockHeight. +// ListAllExpiredBroadcastedTxs returns all the txes that are in broadcasted state and have expired for given block height compared against their lastValidBlockHeight. // Passing maxUint64 as currHeight will return all broadcasted txes. -func (c *pendingTxContext) ListAllExpiredBroadcastedTxs(currHeight uint64) []pendingTx { +func (c *pendingTxContext) ListAllExpiredBroadcastedTxs(currBlockHeight uint64) []pendingTx { c.lock.RLock() defer c.lock.RUnlock() broadcastedTxes := make([]pendingTx, 0, len(c.broadcastedProcessedTxs)) // worst case, all of them for _, tx := range c.broadcastedProcessedTxs { - if tx.state == Broadcasted && tx.lastValidBlockHeight < currHeight { + if tx.state == Broadcasted && tx.lastValidBlockHeight < currBlockHeight { broadcastedTxes = append(broadcastedTxes, tx) } } @@ -623,8 +623,8 @@ func (c *pendingTxContextWithProm) ListAll() []solana.Signature { return sigs } -func (c *pendingTxContextWithProm) ListAllExpiredBroadcastedTxs(currHeight uint64) []pendingTx { - return c.pendingTx.ListAllExpiredBroadcastedTxs(currHeight) +func (c *pendingTxContextWithProm) ListAllExpiredBroadcastedTxs(currBlockHeight uint64) []pendingTx { + return c.pendingTx.ListAllExpiredBroadcastedTxs(currBlockHeight) } func (c *pendingTxContextWithProm) Expired(sig solana.Signature, lifespan time.Duration) bool { diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index 3ba39f2f5..ae2878c7a 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -222,7 +222,7 @@ func (txm *Txm) sendWithRetry(ctx context.Context, msg pendingTx) (solanaGo.Tran return solanaGo.Transaction{}, "", solanaGo.Signature{}, fmt.Errorf("failed to save tx signature (%s) to inflight txs: %w", sig, err) } - txm.lggr.Debugw("tx initial broadcast", "id", msg.id, "fee", msg.cfg.BaseComputeUnitPrice, "signature", sig) + txm.lggr.Debugw("tx initial broadcast", "id", msg.id, "fee", msg.cfg.BaseComputeUnitPrice, "signature", sig, "lastValidBlockHeight", msg.lastValidBlockHeight) // Initialize signature list with initialTx signature. This list will be used to add new signatures and track retry attempts. sigs := &signatureList{} @@ -570,14 +570,14 @@ func (txm *Txm) handleFinalizedSignatureStatus(sig solanaGo.Signature) { // An expired tx is one where it's blockhash lastValidBlockHeight is smaller than the current slot height. // If any error occurs during rebroadcast attempt, they are discarded, and the function continues with the next transaction. func (txm *Txm) rebroadcastExpiredTxs(ctx context.Context, client client.ReaderWriter) { - currHeight, err := client.SlotHeight(ctx) - if err != nil { - txm.lggr.Errorw("failed to get current slot height", "error", err) + currBlockHeight, err := client.GetLatestBlock(ctx) + if err != nil || currBlockHeight == nil || currBlockHeight.BlockHeight == nil { + txm.lggr.Errorw("failed to get current block height", "error", err) return } // Rebroadcast all expired txes - for _, tx := range txm.txs.ListAllExpiredBroadcastedTxs(currHeight) { - txm.lggr.Debugw("transaction expired, rebroadcasting", "id", tx.id, "signature", tx.signatures) + for _, tx := range txm.txs.ListAllExpiredBroadcastedTxs(*currBlockHeight.BlockHeight) { + txm.lggr.Debugw("transaction expired, rebroadcasting", "id", tx.id, "signature", tx.signatures, "lastValidBlockHeight", "currentBlockHeight", *currBlockHeight.BlockHeight, tx.lastValidBlockHeight) if len(tx.signatures) == 0 { // prevent panic, shouldn't happen. txm.lggr.Errorw("no signatures found for expired transaction", "id", tx.id) continue diff --git a/pkg/solana/txm/txm_internal_test.go b/pkg/solana/txm/txm_internal_test.go index 95017a29e..b7680e8ad 100644 --- a/pkg/solana/txm/txm_internal_test.go +++ b/pkg/solana/txm/txm_internal_test.go @@ -1227,7 +1227,7 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { setupTxmTest := func( txExpirationRebroadcast bool, latestBlockhashFunc func() (*rpc.GetLatestBlockhashResult, error), - slotHeightFunc func() (uint64, error), + getLatestBlockFunc func() (*rpc.GetBlockResult, error), sendTxFunc func() (solana.Signature, error), statuses map[solana.Signature]func() *rpc.SignatureStatusesResult, ) (*Txm, *mocks.ReaderWriter, *keyMocks.SimpleKeystore) { @@ -1241,10 +1241,10 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { }, ).Maybe() } - if slotHeightFunc != nil { - mc.On("SlotHeight", mock.Anything).Return( - func(_ context.Context) (uint64, error) { - return slotHeightFunc() + if getLatestBlockFunc != nil { + mc.On("GetLatestBlock", mock.Anything).Return( + func(_ context.Context) (*rpc.GetBlockResult, error) { + return getLatestBlockFunc() }, ).Maybe() } @@ -1292,9 +1292,12 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { txExpirationRebroadcast := true statuses := map[solana.Signature]func() *rpc.SignatureStatusesResult{} - // Mock SlotHeight to return a value greater than 0 - slotHeightFunc := func() (uint64, error) { - return uint64(1500), nil + // Mock getLatestBlock to return a value greater than 0 + getLatestBlockFunc := func() (*rpc.GetBlockResult, error) { + val := uint64(1500) + return &rpc.GetBlockResult{ + BlockHeight: &val, + }, nil } callCount := 0 @@ -1349,7 +1352,7 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { } } - txm, _, mkey := setupTxmTest(txExpirationRebroadcast, latestBlockhashFunc, slotHeightFunc, sendTxFunc, statuses) + txm, _, mkey := setupTxmTest(txExpirationRebroadcast, latestBlockhashFunc, getLatestBlockFunc, sendTxFunc, statuses) tx, _ := getTx(t, 0, mkey) txID := "test-rebroadcast" @@ -1373,7 +1376,7 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { txExpirationRebroadcast := false statuses := map[solana.Signature]func() *rpc.SignatureStatusesResult{} - // mocking the call within sendWithRetry. Rebroadcast is off, so we won't compare it against the slotHeight. + // mocking the call within sendWithRetry. Rebroadcast is off, so we won't compare it against the blockHeight. callCount := 0 latestBlockhashFunc := func() (*rpc.GetLatestBlockhashResult, error) { defer func() { callCount++ }() @@ -1426,9 +1429,14 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { txExpirationRebroadcast := true statuses := map[solana.Signature]func() *rpc.SignatureStatusesResult{} - slotHeightFunc := func() (uint64, error) { - return uint64(1500), nil + // Mock getLatestBlock to return a value greater than 0 + getLatestBlockFunc := func() (*rpc.GetBlockResult, error) { + val := uint64(1500) + return &rpc.GetBlockResult{ + BlockHeight: &val, + }, nil } + // Mock LatestBlockhash to return an invalid blockhash in the first 3 attempts (initial + 2 rebroadcasts) // the last one is valid because it is greater than the slotHeight expectedRebroadcastsCount := 3 @@ -1481,7 +1489,7 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { } } - txm, _, mkey := setupTxmTest(txExpirationRebroadcast, latestBlockhashFunc, slotHeightFunc, sendTxFunc, statuses) + txm, _, mkey := setupTxmTest(txExpirationRebroadcast, latestBlockhashFunc, getLatestBlockFunc, sendTxFunc, statuses) tx, _ := getTx(t, 0, mkey) txID := "test-rebroadcast" assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) @@ -1508,10 +1516,12 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { return sig1, nil } - // Mock LatestBlockhash to return an invalid blockhash less than slotHeight - // We won't use it as there will be no rebroadcasts txes to process. All txes will be confirmed before. - slotHeightFunc := func() (uint64, error) { - return uint64(1500), nil + // Mock getLatestBlock to return a value greater than 0 + getLatestBlockFunc := func() (*rpc.GetBlockResult, error) { + val := uint64(1500) + return &rpc.GetBlockResult{ + BlockHeight: &val, + }, nil } callCount := 0 @@ -1544,7 +1554,7 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { return out } - txm, _, mkey := setupTxmTest(txExpirationRebroadcast, latestBlockhashFunc, slotHeightFunc, sendTxFunc, statuses) + txm, _, mkey := setupTxmTest(txExpirationRebroadcast, latestBlockhashFunc, getLatestBlockFunc, sendTxFunc, statuses) tx, _ := getTx(t, 0, mkey) txID := "test-confirmed-before-rebroadcast" assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) @@ -1567,11 +1577,15 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { txExpirationRebroadcast := true statuses := map[solana.Signature]func() *rpc.SignatureStatusesResult{} - // To force rebroadcast, first call needs to be smaller than slotHeight - // following rebroadcast call will go through because lastValidBlockHeight will be bigger than slotHeight - slotHeightFunc := func() (uint64, error) { - return uint64(1500), nil + // To force rebroadcast, first call needs to be smaller than blockHeight + // following rebroadcast call will go through because lastValidBlockHeight will be bigger than blockHeight + getLatestBlockFunc := func() (*rpc.GetBlockResult, error) { + val := uint64(1500) + return &rpc.GetBlockResult{ + BlockHeight: &val, + }, nil } + callCount := 0 latestBlockhashFunc := func() (*rpc.GetLatestBlockhashResult, error) { defer func() { callCount++ }() @@ -1606,7 +1620,7 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { return nil } - txm, _, mkey := setupTxmTest(txExpirationRebroadcast, latestBlockhashFunc, slotHeightFunc, sendTxFunc, statuses) + txm, _, mkey := setupTxmTest(txExpirationRebroadcast, latestBlockhashFunc, getLatestBlockFunc, sendTxFunc, statuses) tx, _ := getTx(t, 0, mkey) txID := "test-rebroadcast-error" assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) @@ -1624,4 +1638,5 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { require.Equal(t, types.Failed, status) require.Equal(t, 1, callCount-1) // -1 because the first call is not a rebroadcast }) + } From c00494c7cc46db62cdbe2e6dda14107b58690f0e Mon Sep 17 00:00:00 2001 From: Farber98 Date: Thu, 5 Dec 2024 14:20:47 -0300 Subject: [PATCH 41/51] address feedback --- docs/relay/README.md | 2 +- pkg/solana/txm/pendingtx.go | 38 +++--- pkg/solana/txm/pendingtx_test.go | 205 +++++++++++++++++++++++++++---- pkg/solana/txm/txm.go | 8 +- 4 files changed, 206 insertions(+), 47 deletions(-) diff --git a/docs/relay/README.md b/docs/relay/README.md index 2191e985a..07476babb 100644 --- a/docs/relay/README.md +++ b/docs/relay/README.md @@ -45,6 +45,6 @@ chainlink nodes solana create --name= --chain-id= --url= Date: Thu, 5 Dec 2024 14:24:41 -0300 Subject: [PATCH 42/51] fix lint --- pkg/solana/txm/pendingtx.go | 1 - 1 file changed, 1 deletion(-) diff --git a/pkg/solana/txm/pendingtx.go b/pkg/solana/txm/pendingtx.go index 6038e32a4..262b60936 100644 --- a/pkg/solana/txm/pendingtx.go +++ b/pkg/solana/txm/pendingtx.go @@ -210,7 +210,6 @@ func (c *pendingTxContext) ListAllSigs() []solana.Signature { c.lock.RLock() defer c.lock.RUnlock() return maps.Keys(c.sigToID) - } func (c *pendingTxContext) ListAllTxsIDs() []string { From 0e38174d35b6d6ee9f03dba526bd2bf135ad9809 Mon Sep 17 00:00:00 2001 From: Farber98 Date: Thu, 5 Dec 2024 14:38:53 -0300 Subject: [PATCH 43/51] fix log --- pkg/solana/txm/txm.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index f882d9c72..afc3b9213 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -577,7 +577,7 @@ func (txm *Txm) rebroadcastExpiredTxs(ctx context.Context, client client.ReaderW } // Rebroadcast all expired txes for _, tx := range txm.txs.ListAllExpiredBroadcastedTxs(*currBlockHeight.BlockHeight) { - txm.lggr.Debugw("transaction expired, rebroadcasting", "id", tx.id, "signature", tx.signatures, "lastValidBlockHeight", "currentBlockHeight", *currBlockHeight.BlockHeight, tx.lastValidBlockHeight) + txm.lggr.Debugw("transaction expired, rebroadcasting", "id", tx.id, "signature", tx.signatures, "lastValidBlockHeight", tx.lastValidBlockHeight, "currentBlockHeight", *currBlockHeight.BlockHeight) if len(tx.signatures) == 0 { // prevent panic, shouldn't happen. txm.lggr.Errorw("no signatures found for expired transaction", "id", tx.id) continue From b04653cb51dc1bfc3f9b6148acb88bcf83fcc9b8 Mon Sep 17 00:00:00 2001 From: Farber98 Date: Thu, 5 Dec 2024 15:41:41 -0300 Subject: [PATCH 44/51] config for soaks --- pkg/solana/config/config.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/solana/config/config.go b/pkg/solana/config/config.go index 202b8fca8..68d151a73 100644 --- a/pkg/solana/config/config.go +++ b/pkg/solana/config/config.go @@ -18,20 +18,20 @@ var defaultConfigSet = Chain{ TxTimeout: config.MustNewDuration(time.Minute), // timeout for send tx method in client TxRetryTimeout: config.MustNewDuration(10 * time.Second), // duration for tx rebroadcasting to RPC node TxConfirmTimeout: config.MustNewDuration(30 * time.Second), // duration before discarding tx as unconfirmed. Set to 0 to disable discarding tx. - TxExpirationRebroadcast: ptr(false), // to enable rebroadcasting of expired transactions + TxExpirationRebroadcast: ptr(true), // to enable rebroadcasting of expired transactions TxRetentionTimeout: config.MustNewDuration(0 * time.Second), // duration to retain transactions after being marked as finalized or errored. Set to 0 to immediately drop transactions. SkipPreflight: ptr(true), // to enable or disable preflight checks Commitment: ptr(string(rpc.CommitmentConfirmed)), MaxRetries: ptr(int64(0)), // max number of retries (default = 0). when config.MaxRetries < 0), interpreted as MaxRetries = nil and rpc node will do a reasonable number of retries // fee estimator - FeeEstimatorMode: ptr("fixed"), + FeeEstimatorMode: ptr("blockhistory"), ComputeUnitPriceMax: ptr(uint64(1_000)), ComputeUnitPriceMin: ptr(uint64(0)), ComputeUnitPriceDefault: ptr(uint64(0)), FeeBumpPeriod: config.MustNewDuration(3 * time.Second), // set to 0 to disable fee bumping BlockHistoryPollPeriod: config.MustNewDuration(5 * time.Second), - BlockHistorySize: ptr(uint64(1)), // 1: uses latest block; >1: Uses multiple blocks, where n is number of blocks. DISCLAIMER: 1:1 ratio between n and RPC calls. + BlockHistorySize: ptr(uint64(15)), // 1: uses latest block; >1: Uses multiple blocks, where n is number of blocks. DISCLAIMER: 1:1 ratio between n and RPC calls. ComputeUnitLimitDefault: ptr(uint32(200_000)), // set to 0 to disable adding compute unit limit EstimateComputeUnitLimit: ptr(false), // set to false to disable compute unit limit estimation } From d24002158c94cc835b6433f5fa23941b6f778b0a Mon Sep 17 00:00:00 2001 From: Farber98 Date: Fri, 6 Dec 2024 12:32:24 -0300 Subject: [PATCH 45/51] address feedback --- pkg/solana/txm/pendingtx.go | 6 ------ pkg/solana/txm/pendingtx_test.go | 6 ++++-- pkg/solana/txm/txm.go | 21 ++++++--------------- pkg/solana/txm/txm_internal_test.go | 9 ++++----- 4 files changed, 14 insertions(+), 28 deletions(-) diff --git a/pkg/solana/txm/pendingtx.go b/pkg/solana/txm/pendingtx.go index 262b60936..9230b99db 100644 --- a/pkg/solana/txm/pendingtx.go +++ b/pkg/solana/txm/pendingtx.go @@ -212,12 +212,6 @@ func (c *pendingTxContext) ListAllSigs() []solana.Signature { return maps.Keys(c.sigToID) } -func (c *pendingTxContext) ListAllTxsIDs() []string { - c.lock.RLock() - defer c.lock.RUnlock() - return maps.Values(c.sigToID) -} - // ListAllExpiredBroadcastedTxs returns all the txes that are in broadcasted state and have expired for given block height compared against their lastValidBlockHeight. // Passing maxUint64 as currHeight will return all broadcasted txes. func (c *pendingTxContext) ListAllExpiredBroadcastedTxs(currBlockHeight uint64) []pendingTx { diff --git a/pkg/solana/txm/pendingtx_test.go b/pkg/solana/txm/pendingtx_test.go index 531435c8d..10bf4cb0c 100644 --- a/pkg/solana/txm/pendingtx_test.go +++ b/pkg/solana/txm/pendingtx_test.go @@ -48,8 +48,10 @@ func TestPendingTxContext_add_remove_multiple(t *testing.T) { // cannot add signature for non existent ID require.Error(t, txs.AddSignature(uuid.New().String(), solana.Signature{})) - // return list of txsIds - list := txs.ListAllTxsIDs() + list := make([]string, 0, n) + for _, id := range txs.sigToID { + list = append(list, id) + } assert.Equal(t, n, len(list)) // stop all sub processes diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index afc3b9213..ac2bed40f 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -570,24 +570,21 @@ func (txm *Txm) handleFinalizedSignatureStatus(sig solanaGo.Signature) { // An expired tx is one where it's blockhash lastValidBlockHeight is smaller than the current slot height. // If any error occurs during rebroadcast attempt, they are discarded, and the function continues with the next transaction. func (txm *Txm) rebroadcastExpiredTxs(ctx context.Context, client client.ReaderWriter) { - currBlockHeight, err := client.GetLatestBlock(ctx) - if err != nil || currBlockHeight == nil || currBlockHeight.BlockHeight == nil { + currBlock, err := client.GetLatestBlock(ctx) + if err != nil || currBlock == nil || currBlock.BlockHeight == nil { txm.lggr.Errorw("failed to get current block height", "error", err) return } // Rebroadcast all expired txes - for _, tx := range txm.txs.ListAllExpiredBroadcastedTxs(*currBlockHeight.BlockHeight) { - txm.lggr.Debugw("transaction expired, rebroadcasting", "id", tx.id, "signature", tx.signatures, "lastValidBlockHeight", tx.lastValidBlockHeight, "currentBlockHeight", *currBlockHeight.BlockHeight) - if len(tx.signatures) == 0 { // prevent panic, shouldn't happen. - txm.lggr.Errorw("no signatures found for expired transaction", "id", tx.id) - continue - } + for _, tx := range txm.txs.ListAllExpiredBroadcastedTxs(*currBlock.BlockHeight) { + txm.lggr.Debugw("transaction expired, rebroadcasting", "id", tx.id, "signature", tx.signatures, "lastValidBlockHeight", tx.lastValidBlockHeight, "currentBlockHeight", *currBlock.BlockHeight) // Removes all signatures associated to tx and cancels context. _, err := txm.txs.Remove(tx.id) if err != nil { txm.lggr.Errorw("failed to remove expired transaction", "id", tx.id, "error", err) continue } + tx.cfg.BaseComputeUnitPrice = txm.fee.BaseComputeUnitPrice() // update compute unit price (priority fee) for rebroadcast rebroadcastTx := pendingTx{ tx: tx.tx, cfg: tx.cfg, @@ -720,17 +717,11 @@ func (txm *Txm) Enqueue(ctx context.Context, accountID string, tx *solanaGo.Tran } msg := pendingTx{ + id: id, tx: *tx, cfg: cfg, } - // If ID was not set by caller, create one. - if txID != nil && *txID != "" { - msg.id = *txID - } else { - msg.id = uuid.New().String() - } - select { case txm.chSend <- msg: default: diff --git a/pkg/solana/txm/txm_internal_test.go b/pkg/solana/txm/txm_internal_test.go index b7680e8ad..3f8e0c070 100644 --- a/pkg/solana/txm/txm_internal_test.go +++ b/pkg/solana/txm/txm_internal_test.go @@ -978,7 +978,6 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { cfg.Chain.TxRetentionTimeout = relayconfig.MustNewDuration(5 * time.Second) mc := mocks.NewReaderWriter(t) mc.On("GetLatestBlock", mock.Anything).Return(&rpc.GetBlockResult{}, nil).Maybe() - mc.On("SlotHeight", mock.Anything).Return(uint64(0), nil).Maybe() // mock solana keystore mkey := keyMocks.NewSimpleKeystore(t) @@ -1292,7 +1291,7 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { txExpirationRebroadcast := true statuses := map[solana.Signature]func() *rpc.SignatureStatusesResult{} - // Mock getLatestBlock to return a value greater than 0 + // Mock getLatestBlock to return a value greater than 0 for blockHeight getLatestBlockFunc := func() (*rpc.GetBlockResult, error) { val := uint64(1500) return &rpc.GetBlockResult{ @@ -1304,14 +1303,14 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { latestBlockhashFunc := func() (*rpc.GetLatestBlockhashResult, error) { defer func() { callCount++ }() if callCount < 1 { - // To force rebroadcast, first call needs to be smaller than slotHeight + // To force rebroadcast, first call needs to be smaller than blockHeight return &rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ LastValidBlockHeight: uint64(1000), }, }, nil } - // following rebroadcast call will go through because lastValidBlockHeight is bigger than slotHeight + // following rebroadcast call will go through because lastValidBlockHeight is bigger than blockHeight return &rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ LastValidBlockHeight: uint64(2000), @@ -1438,7 +1437,7 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { } // Mock LatestBlockhash to return an invalid blockhash in the first 3 attempts (initial + 2 rebroadcasts) - // the last one is valid because it is greater than the slotHeight + // the last one is valid because it is greater than the blockHeight expectedRebroadcastsCount := 3 callCount := 0 latestBlockhashFunc := func() (*rpc.GetLatestBlockhashResult, error) { From 4389e13fd9965151b37363d17534dba4b0e8c121 Mon Sep 17 00:00:00 2001 From: Farber98 Date: Fri, 6 Dec 2024 12:57:18 -0300 Subject: [PATCH 46/51] remove useless slot height --- pkg/solana/txm/txm_internal_test.go | 1 - 1 file changed, 1 deletion(-) diff --git a/pkg/solana/txm/txm_internal_test.go b/pkg/solana/txm/txm_internal_test.go index 3f8e0c070..3fa8fa54c 100644 --- a/pkg/solana/txm/txm_internal_test.go +++ b/pkg/solana/txm/txm_internal_test.go @@ -772,7 +772,6 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { cfg.Chain.TxRetentionTimeout = relayconfig.MustNewDuration(5 * time.Second) mc := mocks.NewReaderWriter(t) mc.On("GetLatestBlock", mock.Anything).Return(&rpc.GetBlockResult{}, nil).Maybe() - mc.On("SlotHeight", mock.Anything).Return(uint64(0), nil).Maybe() computeUnitLimitDefault := fees.ComputeUnitLimit(cfg.ComputeUnitLimitDefault()) From 50dd10ffa6157904d429fdf1f64de0633c036693 Mon Sep 17 00:00:00 2001 From: Farber98 Date: Tue, 10 Dec 2024 10:35:41 -0300 Subject: [PATCH 47/51] address feedback --- pkg/solana/txm/pendingtx.go | 26 +++++++++++++------------- pkg/solana/txm/txm.go | 6 +++--- 2 files changed, 16 insertions(+), 16 deletions(-) diff --git a/pkg/solana/txm/pendingtx.go b/pkg/solana/txm/pendingtx.go index 9230b99db..13547416d 100644 --- a/pkg/solana/txm/pendingtx.go +++ b/pkg/solana/txm/pendingtx.go @@ -28,9 +28,9 @@ type PendingTxContext interface { Remove(id string) (string, error) // ListAllSigs returns all of the signatures being tracked for all transactions not yet finalized or errored ListAllSigs() []solana.Signature - // ListAllExpiredBroadcastedTxs returns all the txes that are in broadcasted state and have expired for given block height compared against their lastValidBlockHeight. - // Passing maxUint64 as currHeight will return all broadcasted txes. - ListAllExpiredBroadcastedTxs(currBlockHeight uint64) []pendingTx + // ListAllExpiredBroadcastedTxs returns all the txes that are in broadcasted state and have expired for given block number compared against lastValidBlockHeight (last valid block number) + // Passing maxUint64 as currBlockNumber will return all broadcasted txes. + ListAllExpiredBroadcastedTxs(currBlockNumber uint64) []pendingTx // Expired returns whether or not confirmation timeout amount of time has passed since creation Expired(sig solana.Signature, confirmationTimeout time.Duration) bool // OnProcessed marks transactions as Processed @@ -57,7 +57,7 @@ type pendingTx struct { id string createTs time.Time state TxState - lastValidBlockHeight uint64 // to track expiration + lastValidBlockHeight uint64 // to track expiration, equivalent to last valid block number. } // finishedTx is used to store minimal info specifically for finalized or errored transactions for external status checks @@ -212,18 +212,18 @@ func (c *pendingTxContext) ListAllSigs() []solana.Signature { return maps.Keys(c.sigToID) } -// ListAllExpiredBroadcastedTxs returns all the txes that are in broadcasted state and have expired for given block height compared against their lastValidBlockHeight. -// Passing maxUint64 as currHeight will return all broadcasted txes. -func (c *pendingTxContext) ListAllExpiredBroadcastedTxs(currBlockHeight uint64) []pendingTx { +// ListAllExpiredBroadcastedTxs returns all the txes that are in broadcasted state and have expired for given block number compared against lastValidBlockHeight (last valid block number) +// Passing maxUint64 as currBlockNumber will return all broadcasted txes. +func (c *pendingTxContext) ListAllExpiredBroadcastedTxs(currBlockNumber uint64) []pendingTx { c.lock.RLock() defer c.lock.RUnlock() - broadcastedTxes := make([]pendingTx, 0, len(c.broadcastedProcessedTxs)) // worst case, all of them + expiredBroadcastedTxs := make([]pendingTx, 0, len(c.broadcastedProcessedTxs)) // worst case, all of them for _, tx := range c.broadcastedProcessedTxs { - if tx.state == Broadcasted && tx.lastValidBlockHeight < currBlockHeight { - broadcastedTxes = append(broadcastedTxes, tx) + if tx.state == Broadcasted && tx.lastValidBlockHeight < currBlockNumber { + expiredBroadcastedTxs = append(expiredBroadcastedTxs, tx) } } - return broadcastedTxes + return expiredBroadcastedTxs } // Expired returns if the timeout for trying to confirm a signature has been reached @@ -614,8 +614,8 @@ func (c *pendingTxContextWithProm) ListAllSigs() []solana.Signature { return sigs } -func (c *pendingTxContextWithProm) ListAllExpiredBroadcastedTxs(currBlockHeight uint64) []pendingTx { - return c.pendingTx.ListAllExpiredBroadcastedTxs(currBlockHeight) +func (c *pendingTxContextWithProm) ListAllExpiredBroadcastedTxs(currBlockNumber uint64) []pendingTx { + return c.pendingTx.ListAllExpiredBroadcastedTxs(currBlockNumber) } func (c *pendingTxContextWithProm) Expired(sig solana.Signature, lifespan time.Duration) bool { diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index ac2bed40f..8605b1811 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -183,7 +183,7 @@ func (txm *Txm) run() { // It builds, signs and sends the initial tx with a new valid blockhash, and starts a retry routine with fee bumping if needed. // The function returns the signed transaction, its ID, and the initial signature for use in simulation. func (txm *Txm) sendWithRetry(ctx context.Context, msg pendingTx) (solanaGo.Transaction, string, solanaGo.Signature, error) { - // Assign new blockhash and lastValidBlockHeight to the transaction + // Assign new blockhash and lastValidBlockHeight (last valid block number) to the transaction // This is essential for tracking transaction rebroadcast // Only the initial transaction should be sent with the updated blockhash client, err := txm.client.Get() @@ -567,7 +567,7 @@ func (txm *Txm) handleFinalizedSignatureStatus(sig solanaGo.Signature) { } // rebroadcastExpiredTxs attempts to rebroadcast all transactions that are in broadcasted state and have expired. -// An expired tx is one where it's blockhash lastValidBlockHeight is smaller than the current slot height. +// An expired tx is one where it's blockhash lastValidBlockHeight (last valid block number) is smaller than the current block height (block number). // If any error occurs during rebroadcast attempt, they are discarded, and the function continues with the next transaction. func (txm *Txm) rebroadcastExpiredTxs(ctx context.Context, client client.ReaderWriter) { currBlock, err := client.GetLatestBlock(ctx) @@ -575,7 +575,7 @@ func (txm *Txm) rebroadcastExpiredTxs(ctx context.Context, client client.ReaderW txm.lggr.Errorw("failed to get current block height", "error", err) return } - // Rebroadcast all expired txes + // Rebroadcast all expired txes using currBlockHeight (current block number) for _, tx := range txm.txs.ListAllExpiredBroadcastedTxs(*currBlock.BlockHeight) { txm.lggr.Debugw("transaction expired, rebroadcasting", "id", tx.id, "signature", tx.signatures, "lastValidBlockHeight", tx.lastValidBlockHeight, "currentBlockHeight", *currBlock.BlockHeight) // Removes all signatures associated to tx and cancels context. From 2ea0c508e20deea35754831039cb7ab57698347b Mon Sep 17 00:00:00 2001 From: Farber98 Date: Wed, 11 Dec 2024 12:21:33 -0300 Subject: [PATCH 48/51] validate that tx doesn't exist in any of maps when adding new tx --- pkg/solana/txm/pendingtx.go | 15 ++++++++- pkg/solana/txm/pendingtx_test.go | 54 +++++++++++++++++++++++--------- 2 files changed, 54 insertions(+), 15 deletions(-) diff --git a/pkg/solana/txm/pendingtx.go b/pkg/solana/txm/pendingtx.go index 13547416d..b08039ab7 100644 --- a/pkg/solana/txm/pendingtx.go +++ b/pkg/solana/txm/pendingtx.go @@ -96,10 +96,16 @@ func (c *pendingTxContext) New(tx pendingTx, sig solana.Signature, cancel contex if _, exists := c.sigToID[sig]; exists { return ErrSigAlreadyExists } - // validate id does not exist + // Check if ID already exists in any of the maps if _, exists := c.broadcastedProcessedTxs[tx.id]; exists { return ErrIDAlreadyExists } + if _, exists := c.confirmedTxs[tx.id]; exists { + return ErrIDAlreadyExists + } + if _, exists := c.finalizedErroredTxs[tx.id]; exists { + return ErrIDAlreadyExists + } return nil }) if err != nil { @@ -111,9 +117,16 @@ func (c *pendingTxContext) New(tx pendingTx, sig solana.Signature, cancel contex if _, exists := c.sigToID[sig]; exists { return "", ErrSigAlreadyExists } + // Check if ID already exists in any of the maps if _, exists := c.broadcastedProcessedTxs[tx.id]; exists { return "", ErrIDAlreadyExists } + if _, exists := c.confirmedTxs[tx.id]; exists { + return "", ErrIDAlreadyExists + } + if _, exists := c.finalizedErroredTxs[tx.id]; exists { + return "", ErrIDAlreadyExists + } // save cancel func c.cancelBy[tx.id] = cancel c.sigToID[sig] = tx.id diff --git a/pkg/solana/txm/pendingtx_test.go b/pkg/solana/txm/pendingtx_test.go index 10bf4cb0c..a79f9f7aa 100644 --- a/pkg/solana/txm/pendingtx_test.go +++ b/pkg/solana/txm/pendingtx_test.go @@ -78,29 +78,55 @@ func TestPendingTxContext_new(t *testing.T) { // Create new transaction msg := pendingTx{id: uuid.NewString()} err := txs.New(msg, sig, cancel) - require.NoError(t, err) + require.NoError(t, err, "expected no error when adding a new transaction") - // Check it exists in signature map + // Check it exists in signature map and mapped to the correct txID id, exists := txs.sigToID[sig] - require.True(t, exists) - require.Equal(t, msg.id, id) + require.True(t, exists, "signature should exist in sigToID map") + require.Equal(t, msg.id, id, "signature should map to correct transaction ID") - // Check it exists in broadcasted map + // Check it exists in broadcasted map and that sigs match tx, exists := txs.broadcastedProcessedTxs[msg.id] - require.True(t, exists) - require.Len(t, tx.signatures, 1) - require.Equal(t, sig, tx.signatures[0]) + require.True(t, exists, "transaction should exist in broadcastedProcessedTxs map") + require.Len(t, tx.signatures, 1, "transaction should have one signature") + require.Equal(t, sig, tx.signatures[0], "signature should match") // Check status is Broadcasted - require.Equal(t, Broadcasted, tx.state) + require.Equal(t, Broadcasted, tx.state, "transaction state should be Broadcasted") - // Check it does not exist in confirmed map + // Check it does not exist in confirmed nor finalized maps _, exists = txs.confirmedTxs[msg.id] - require.False(t, exists) - - // Check it does not exist in finalized map + require.False(t, exists, "transaction should not exist in confirmedTxs map") _, exists = txs.finalizedErroredTxs[msg.id] - require.False(t, exists) + require.False(t, exists, "transaction should not exist in finalizedErroredTxs map") + + // Attempt to add the same transaction again with the same signature + err = txs.New(msg, sig, cancel) + require.ErrorIs(t, err, ErrSigAlreadyExists, "expected ErrSigAlreadyExists when adding duplicate signature") + + // Attempt to add a new transaction with the same transaction ID but different signature + err = txs.New(pendingTx{id: msg.id}, randomSignature(t), cancel) + require.ErrorIs(t, err, ErrIDAlreadyExists, "expected ErrIDAlreadyExists when adding duplicate transaction ID") + + // Attempt to add a new transaction with a different transaction ID but same signature + err = txs.New(pendingTx{id: uuid.NewString()}, sig, cancel) + require.ErrorIs(t, err, ErrSigAlreadyExists, "expected ErrSigAlreadyExists when adding duplicate signature") + + // Simulate moving the transaction to confirmedTxs map + _, err = txs.OnConfirmed(sig) + require.NoError(t, err, "expected no error when confirming transaction") + + // Attempt to add a new transaction with the same ID (now in confirmedTxs) and new signature + err = txs.New(pendingTx{id: msg.id}, randomSignature(t), cancel) + require.ErrorIs(t, err, ErrIDAlreadyExists, "expected ErrIDAlreadyExists when adding transaction ID that exists in confirmedTxs") + + // Simulate moving the transaction to finalizedErroredTxs map + _, err = txs.OnFinalized(sig, 10*time.Second) + require.NoError(t, err, "expected no error when finalizing transaction") + + // Attempt to add a new transaction with the same ID (now in finalizedErroredTxs) and new signature + err = txs.New(pendingTx{id: msg.id}, randomSignature(t), cancel) + require.ErrorIs(t, err, ErrIDAlreadyExists, "expected ErrIDAlreadyExists when adding transaction ID that exists in finalizedErroredTxs") } func TestPendingTxContext_add_signature(t *testing.T) { From 409fd1ca9557fb20965ff83b71c91eea3be0b4af Mon Sep 17 00:00:00 2001 From: Farber98 Date: Tue, 17 Dec 2024 00:58:55 -0300 Subject: [PATCH 49/51] callers set lastValidBlockheight + get blockhash on expiration + integration tests --- pkg/solana/chain.go | 2 +- pkg/solana/chain_test.go | 13 +- pkg/solana/relay.go | 2 +- pkg/solana/transmitter.go | 2 +- pkg/solana/transmitter_test.go | 2 +- pkg/solana/txm/txm.go | 59 +++--- pkg/solana/txm/txm_integration_test.go | 186 +++++++++++++++++++ pkg/solana/txm/txm_internal_test.go | 247 +++++++++++-------------- pkg/solana/txm/txm_load_test.go | 26 ++- pkg/solana/txm/txm_race_test.go | 8 - 10 files changed, 355 insertions(+), 192 deletions(-) create mode 100644 pkg/solana/txm/txm_integration_test.go diff --git a/pkg/solana/chain.go b/pkg/solana/chain.go index 630248aff..5ed5eb8cb 100644 --- a/pkg/solana/chain.go +++ b/pkg/solana/chain.go @@ -575,7 +575,7 @@ func (c *chain) sendTx(ctx context.Context, from, to string, amount *big.Int, ba } chainTxm := c.TxManager() - err = chainTxm.Enqueue(ctx, "", tx, nil, + err = chainTxm.Enqueue(ctx, "", tx, nil, blockhash.Value.LastValidBlockHeight, txm.SetComputeUnitLimit(500), // reduce from default 200K limit - should only take 450 compute units // no fee bumping and no additional fee - makes validating balance accurate txm.SetComputeUnitPriceMax(0), diff --git a/pkg/solana/chain_test.go b/pkg/solana/chain_test.go index 4ee0cd4ef..c2182e7c1 100644 --- a/pkg/solana/chain_test.go +++ b/pkg/solana/chain_test.go @@ -536,7 +536,7 @@ func TestSolanaChain_MultiNode_Txm(t *testing.T) { assert.NoError(t, err) assert.Equal(t, uint64(0), receiverBal) - createTx := func(signer solana.PublicKey, sender solana.PublicKey, receiver solana.PublicKey, amt uint64) *solana.Transaction { + createTx := func(signer solana.PublicKey, sender solana.PublicKey, receiver solana.PublicKey, amt uint64) (*solana.Transaction, uint64) { selectedClient, err = testChain.getClient() assert.NoError(t, err) hash, hashErr := selectedClient.LatestBlockhash(tests.Context(t)) @@ -553,11 +553,12 @@ func TestSolanaChain_MultiNode_Txm(t *testing.T) { solana.TransactionPayer(signer), ) require.NoError(t, txErr) - return tx + return tx, hash.Value.LastValidBlockHeight } // Send funds twice, along with an invalid transaction - require.NoError(t, testChain.txm.Enqueue(tests.Context(t), "test_success", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) + tx, lastValidBlockHeight := createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL) + require.NoError(t, testChain.txm.Enqueue(tests.Context(t), "test_success", tx, nil, lastValidBlockHeight)) // Wait for new block hash currentBh, err := selectedClient.LatestBlockhash(tests.Context(t)) @@ -578,8 +579,10 @@ NewBlockHash: } } - require.NoError(t, testChain.txm.Enqueue(tests.Context(t), "test_success_2", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) - require.Error(t, testChain.txm.Enqueue(tests.Context(t), "test_invalidSigner", createTx(pubKeyReceiver, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) // cannot sign tx before enqueuing + tx2, lastValidBlockHeight2 := createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL) + require.NoError(t, testChain.txm.Enqueue(tests.Context(t), "test_success_2", tx2, nil, lastValidBlockHeight2)) + tx3, lastValidBlockHeight3 := createTx(pubKeyReceiver, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL) + require.Error(t, testChain.txm.Enqueue(tests.Context(t), "test_invalidSigner", tx3, nil, lastValidBlockHeight3)) // cannot sign tx before enqueuing // wait for all txes to finish ctx, cancel := context.WithCancel(tests.Context(t)) diff --git a/pkg/solana/relay.go b/pkg/solana/relay.go index 1f2fbdffd..1df61b99d 100644 --- a/pkg/solana/relay.go +++ b/pkg/solana/relay.go @@ -24,7 +24,7 @@ import ( var _ TxManager = (*txm.Txm)(nil) type TxManager interface { - Enqueue(ctx context.Context, accountID string, tx *solana.Transaction, txID *string, txCfgs ...txm.SetTxConfig) error + Enqueue(ctx context.Context, accountID string, tx *solana.Transaction, txID *string, lastValidBlockHeight uint64, txCfgs ...txm.SetTxConfig) error } var _ relaytypes.Relayer = &Relayer{} //nolint:staticcheck diff --git a/pkg/solana/transmitter.go b/pkg/solana/transmitter.go index 951e9633e..537c72699 100644 --- a/pkg/solana/transmitter.go +++ b/pkg/solana/transmitter.go @@ -87,7 +87,7 @@ func (c *Transmitter) Transmit( // pass transmit payload to tx manager queue c.lggr.Debugf("Queuing transmit tx: state (%s) + transmissions (%s)", c.stateID.String(), c.transmissionsID.String()) - if err = c.txManager.Enqueue(ctx, c.stateID.String(), tx, nil); err != nil { + if err = c.txManager.Enqueue(ctx, c.stateID.String(), tx, nil, blockhash.Value.LastValidBlockHeight); err != nil { return fmt.Errorf("error on Transmit.txManager.Enqueue: %w", err) } return nil diff --git a/pkg/solana/transmitter_test.go b/pkg/solana/transmitter_test.go index 1d058d36a..f6db01d6c 100644 --- a/pkg/solana/transmitter_test.go +++ b/pkg/solana/transmitter_test.go @@ -27,7 +27,7 @@ type verifyTxSize struct { s *solana.PrivateKey } -func (txm verifyTxSize) Enqueue(_ context.Context, _ string, tx *solana.Transaction, txID *string, _ ...txm.SetTxConfig) error { +func (txm verifyTxSize) Enqueue(_ context.Context, _ string, tx *solana.Transaction, txID *string, _ uint64, _ ...txm.SetTxConfig) error { // additional components that transaction manager adds to the transaction require.NoError(txm.t, fees.SetComputeUnitPrice(tx, 0)) require.NoError(txm.t, fees.SetComputeUnitLimit(tx, 0)) diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index b3f244ebf..187904c3b 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -180,26 +180,9 @@ func (txm *Txm) run() { } // sendWithRetry attempts to send a transaction with exponential backoff retry logic. -// It builds, signs and sends the initial tx with a new valid blockhash, and starts a retry routine with fee bumping if needed. +// It builds, signs, sends the initial tx, and starts a retry routine with fee bumping if needed. // The function returns the signed transaction, its ID, and the initial signature for use in simulation. func (txm *Txm) sendWithRetry(ctx context.Context, msg pendingTx) (solanaGo.Transaction, string, solanaGo.Signature, error) { - // Assign new blockhash and lastValidBlockHeight (last valid block number) to the transaction - // This is essential for tracking transaction rebroadcast - // Only the initial transaction should be sent with the updated blockhash - client, err := txm.client.Get() - if err != nil { - return solanaGo.Transaction{}, "", solanaGo.Signature{}, fmt.Errorf("failed to get client: %w", err) - } - blockhash, err := client.LatestBlockhash(ctx) - if err != nil { - return solanaGo.Transaction{}, "", solanaGo.Signature{}, fmt.Errorf("failed to get latest blockhash: %w", err) - } - if blockhash == nil || blockhash.Value == nil { - return solanaGo.Transaction{}, "", solanaGo.Signature{}, errors.New("nil pointer returned from LatestBlockhash") - } - msg.tx.Message.RecentBlockhash = blockhash.Value.Blockhash - msg.lastValidBlockHeight = blockhash.Value.LastValidBlockHeight - // Build and sign initial transaction setting compute unit price and limit initTx, err := txm.buildTx(ctx, msg, 0) if err != nil { @@ -568,6 +551,7 @@ func (txm *Txm) handleFinalizedSignatureStatus(sig solanaGo.Signature) { // rebroadcastExpiredTxs attempts to rebroadcast all transactions that are in broadcasted state and have expired. // An expired tx is one where it's blockhash lastValidBlockHeight (last valid block number) is smaller than the current block height (block number). +// The function loops through all expired txes, rebroadcasts them with a new blockhash, and updates the lastValidBlockHeight. // If any error occurs during rebroadcast attempt, they are discarded, and the function continues with the next transaction. func (txm *Txm) rebroadcastExpiredTxs(ctx context.Context, client client.ReaderWriter) { currBlock, err := client.GetLatestBlock(ctx) @@ -575,8 +559,15 @@ func (txm *Txm) rebroadcastExpiredTxs(ctx context.Context, client client.ReaderW txm.lggr.Errorw("failed to get current block height", "error", err) return } - // Rebroadcast all expired txes using currBlockHeight (current block number) - for _, tx := range txm.txs.ListAllExpiredBroadcastedTxs(*currBlock.BlockHeight) { + + // Get all expired broadcasted transactions at current block number. Safe to quit if no txes are found. + expiredBroadcastedTxes := txm.txs.ListAllExpiredBroadcastedTxs(*currBlock.BlockHeight) + if len(expiredBroadcastedTxes) == 0 { + return + } + + // Request new blockhash and loop through all expired txes overwriting with new blockhash and rebroadcasting + for _, tx := range expiredBroadcastedTxes { txm.lggr.Debugw("transaction expired, rebroadcasting", "id", tx.id, "signature", tx.signatures, "lastValidBlockHeight", tx.lastValidBlockHeight, "currentBlockHeight", *currBlock.BlockHeight) // Removes all signatures associated to tx and cancels context. _, err := txm.txs.Remove(tx.id) @@ -584,11 +575,24 @@ func (txm *Txm) rebroadcastExpiredTxs(ctx context.Context, client client.ReaderW txm.lggr.Errorw("failed to remove expired transaction", "id", tx.id, "error", err) continue } + + blockhash, err := client.LatestBlockhash(ctx) + if err != nil { + txm.lggr.Errorw("failed to get latest blockhash for rebroadcast", "error", err) + return + } + if blockhash == nil || blockhash.Value == nil { + txm.lggr.Errorw("nil pointer returned from LatestBlockhash for rebroadcast") + return + } + + tx.tx.Message.RecentBlockhash = blockhash.Value.Blockhash tx.cfg.BaseComputeUnitPrice = txm.fee.BaseComputeUnitPrice() // update compute unit price (priority fee) for rebroadcast rebroadcastTx := pendingTx{ - tx: tx.tx, - cfg: tx.cfg, - id: tx.id, // using same id in case it was set by caller and we need to maintain it. + tx: tx.tx, + cfg: tx.cfg, + id: tx.id, // using same id in case it was set by caller and we need to maintain it. + lastValidBlockHeight: blockhash.Value.LastValidBlockHeight, } // call sendWithRetry directly to avoid enqueuing _, _, _, sendErr := txm.sendWithRetry(ctx, rebroadcastTx) @@ -669,7 +673,7 @@ func (txm *Txm) reap() { } // Enqueue enqueues a msg destined for the solana chain. -func (txm *Txm) Enqueue(ctx context.Context, accountID string, tx *solanaGo.Transaction, txID *string, txCfgs ...SetTxConfig) error { +func (txm *Txm) Enqueue(ctx context.Context, accountID string, tx *solanaGo.Transaction, txID *string, txLastValidBlockHeight uint64, txCfgs ...SetTxConfig) error { if err := txm.Ready(); err != nil { return fmt.Errorf("error in soltxm.Enqueue: %w", err) } @@ -717,9 +721,10 @@ func (txm *Txm) Enqueue(ctx context.Context, accountID string, tx *solanaGo.Tran } msg := pendingTx{ - id: id, - tx: *tx, - cfg: cfg, + id: id, + tx: *tx, + cfg: cfg, + lastValidBlockHeight: txLastValidBlockHeight, } select { diff --git a/pkg/solana/txm/txm_integration_test.go b/pkg/solana/txm/txm_integration_test.go new file mode 100644 index 000000000..1beeafe2b --- /dev/null +++ b/pkg/solana/txm/txm_integration_test.go @@ -0,0 +1,186 @@ +//go:build integration + +package txm_test + +import ( + "context" + "testing" + "time" + + "github.com/gagliardetto/solana-go" + "github.com/gagliardetto/solana-go/programs/system" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + "go.uber.org/zap/zapcore" + "go.uber.org/zap/zaptest/observer" + + "github.com/smartcontractkit/chainlink-common/pkg/logger" + "github.com/smartcontractkit/chainlink-common/pkg/services/servicetest" + "github.com/smartcontractkit/chainlink-common/pkg/types" + "github.com/smartcontractkit/chainlink-common/pkg/utils" + + relayconfig "github.com/smartcontractkit/chainlink-common/pkg/config" + + solanaClient "github.com/smartcontractkit/chainlink-solana/pkg/solana/client" + "github.com/smartcontractkit/chainlink-solana/pkg/solana/config" + "github.com/smartcontractkit/chainlink-solana/pkg/solana/txm" + keyMocks "github.com/smartcontractkit/chainlink-solana/pkg/solana/txm/mocks" +) + +func TestTxm_Integration_ExpirationRebroadcast(t *testing.T) { + t.Parallel() + url := solanaClient.SetupLocalSolNode(t) // live validator + + type TestCase struct { + name string + txExpirationRebroadcast bool + useValidBlockHash bool + expectRebroadcast bool + expectTransactionStatus types.TransactionStatus + } + + testCases := []TestCase{ + { + name: "WithRebroadcast", + txExpirationRebroadcast: true, + useValidBlockHash: false, + expectRebroadcast: true, + expectTransactionStatus: types.Finalized, + }, + { + name: "WithoutRebroadcast", + txExpirationRebroadcast: false, + useValidBlockHash: false, + expectRebroadcast: false, + expectTransactionStatus: types.Failed, + }, + { + name: "ConfirmedBeforeRebroadcast", + txExpirationRebroadcast: true, + useValidBlockHash: true, + expectRebroadcast: false, + expectTransactionStatus: types.Finalized, + }, + } + + for _, tc := range testCases { + tc := tc + t.Run(tc.name, func(t *testing.T) { + t.Parallel() + ctx, client, txmInstance, senderPubKey, receiverPubKey, observer := setup(t, url, tc.txExpirationRebroadcast) + + // Record initial balance + initSenderBalance, err := client.Balance(ctx, senderPubKey) + require.NoError(t, err) + const amount = 1 * solana.LAMPORTS_PER_SOL + + // Create and enqueue tx + txID := tc.name + tx, lastValidBlockHeight := createTransaction(ctx, t, client, senderPubKey, receiverPubKey, amount, tc.useValidBlockHash) + require.NoError(t, txmInstance.Enqueue(ctx, "", tx, &txID, lastValidBlockHeight)) + + // Wait for the transaction to reach the expected status + require.Eventually(t, func() bool { + status, statusErr := txmInstance.GetTransactionStatus(ctx, txID) + if statusErr != nil { + return false + } + return status == tc.expectTransactionStatus + }, 60*time.Second, 1*time.Second, "Transaction should eventually reach expected status") + + // Verify balances + finalSenderBalance, err := client.Balance(ctx, senderPubKey) + require.NoError(t, err) + finalReceiverBalance, err := client.Balance(ctx, receiverPubKey) + require.NoError(t, err) + + if tc.expectTransactionStatus == types.Finalized { + require.Less(t, finalSenderBalance, initSenderBalance, "Sender balance should decrease") + require.Equal(t, amount, finalReceiverBalance, "Receiver should receive the transferred amount") + } else { + require.Equal(t, initSenderBalance, finalSenderBalance, "Sender balance should remain the same") + require.Equal(t, uint64(0), finalReceiverBalance, "Receiver should not receive any funds") + } + + // Verify rebroadcast logs + rebroadcastLogs := observer.FilterMessageSnippet("rebroadcast transaction sent").All() + rebroadcastLogs2 := observer.FilterMessageSnippet("transaction expired, rebroadcasting").All() + if tc.expectRebroadcast { + require.NotEmpty(t, rebroadcastLogs, "Expected rebroadcast log message not found") + require.NotEmpty(t, rebroadcastLogs2, "Expected rebroadcast log message not found") + } else { + require.Empty(t, rebroadcastLogs, "Rebroadcast should not occur") + require.Empty(t, rebroadcastLogs2, "Rebroadcast should not occur") + } + }) + } +} + +func setup(t *testing.T, url string, txExpirationRebroadcast bool) (context.Context, *solanaClient.Client, *txm.Txm, solana.PublicKey, solana.PublicKey, *observer.ObservedLogs) { + ctx := context.Background() + + // Generate sender and receiver keys and fund sender account + senderKey, err := solana.NewRandomPrivateKey() + require.NoError(t, err) + senderPubKey := senderKey.PublicKey() + receiverKey, err := solana.NewRandomPrivateKey() + require.NoError(t, err) + receiverPubKey := receiverKey.PublicKey() + solanaClient.FundTestAccounts(t, []solana.PublicKey{senderPubKey}, url) + + // Set up mock keystore with sender key + mkey := keyMocks.NewSimpleKeystore(t) + mkey.On("Sign", mock.Anything, senderPubKey.String(), mock.Anything).Return(func(_ context.Context, _ string, data []byte) []byte { + sig, _ := senderKey.Sign(data) + return sig[:] + }, nil) + + // Set configs + cfg := config.NewDefault() + cfg.Chain.TxExpirationRebroadcast = &txExpirationRebroadcast + cfg.Chain.TxRetentionTimeout = relayconfig.MustNewDuration(10 * time.Second) // to get the finalized tx status + + // Initialize the Solana client and TXM + lggr, obs := logger.TestObserved(t, zapcore.DebugLevel) + client, err := solanaClient.NewClient(url, cfg, 2*time.Second, lggr) + require.NoError(t, err) + loader := utils.NewLazyLoad(func() (solanaClient.ReaderWriter, error) { return client, nil }) + txmInstance := txm.NewTxm("localnet", loader, nil, cfg, mkey, lggr) + servicetest.Run(t, txmInstance) + + return ctx, client, txmInstance, senderPubKey, receiverPubKey, obs +} + +// createTransaction is a helper function to create a transaction based on the test case. +func createTransaction(ctx context.Context, t *testing.T, client *solanaClient.Client, senderPubKey, receiverPubKey solana.PublicKey, amount uint64, useValidBlockHash bool) (*solana.Transaction, uint64) { + var blockhash solana.Hash + var lastValidBlockHeight uint64 + + if useValidBlockHash { + // Get a valid recent blockhash + recentBlockHashResult, err := client.LatestBlockhash(ctx) + require.NoError(t, err) + blockhash = recentBlockHashResult.Value.Blockhash + lastValidBlockHeight = recentBlockHashResult.Value.LastValidBlockHeight + } else { + // Use empty blockhash to simulate expiration + blockhash = solana.Hash{} + lastValidBlockHeight = 0 + } + + // Create the transaction + tx, err := solana.NewTransaction( + []solana.Instruction{ + system.NewTransferInstruction( + amount, + senderPubKey, + receiverPubKey, + ).Build(), + }, + blockhash, + solana.TransactionPayer(senderPubKey), + ) + require.NoError(t, err) + + return tx, lastValidBlockHeight +} diff --git a/pkg/solana/txm/txm_internal_test.go b/pkg/solana/txm/txm_internal_test.go index 8cf94ec82..13c861362 100644 --- a/pkg/solana/txm/txm_internal_test.go +++ b/pkg/solana/txm/txm_internal_test.go @@ -161,12 +161,6 @@ func TestTxm(t *testing.T) { return out }, nil, ) - mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: 100, - Blockhash: solana.Hash{}, - }, - }, nil) // happy path (send => simulate success => tx: nil => tx: processed => tx: confirmed => finalized => done) t.Run("happyPath", func(t *testing.T) { sig := randomSignature(t) @@ -209,7 +203,8 @@ func TestTxm(t *testing.T) { // send tx testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + lastValidBlockHeight := uint64(100) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() // no transactions stored inflight txs list @@ -245,7 +240,8 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + lastValidBlockHeight := uint64(100) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() // wait to be picked up and processed // no transactions stored inflight txs list @@ -277,7 +273,8 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + lastValidBlockHeight := uint64(100) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared quickly @@ -313,7 +310,8 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + lastValidBlockHeight := uint64(100) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared after timeout @@ -353,7 +351,8 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + lastValidBlockHeight := uint64(100) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared after timeout @@ -404,7 +403,8 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + lastValidBlockHeight := uint64(100) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared after timeout @@ -446,7 +446,8 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + lastValidBlockHeight := uint64(100) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // txs cleared after timeout @@ -491,7 +492,8 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + lastValidBlockHeight := uint64(100) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // inflight txs cleared after timeout @@ -543,7 +545,8 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + lastValidBlockHeight := uint64(100) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // inflight txs cleared after timeout @@ -581,7 +584,8 @@ func TestTxm(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + lastValidBlockHeight := uint64(100) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() // wait to be picked up and processed waitFor(t, waitDuration, txm, prom, empty) // inflight txs cleared after timeout @@ -627,7 +631,8 @@ func TestTxm(t *testing.T) { // send tx testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + lastValidBlockHeight := uint64(100) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() // no transactions stored inflight txs list @@ -681,7 +686,8 @@ func TestTxm(t *testing.T) { // send tx - with disabled fee bumping testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, SetFeeBumpPeriod(0))) + lastValidBlockHeight := uint64(100) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight, SetFeeBumpPeriod(0))) wg.Wait() // no transactions stored inflight txs list @@ -733,7 +739,8 @@ func TestTxm(t *testing.T) { // send tx - with disabled fee bumping and disabled compute unit limit testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, SetFeeBumpPeriod(0), SetComputeUnitLimit(0))) + lastValidBlockHeight := uint64(100) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight, SetFeeBumpPeriod(0), SetComputeUnitLimit(0))) wg.Wait() // no transactions stored inflight txs list @@ -802,12 +809,6 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { return out }, nil, ) - mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: 100, - Blockhash: solana.Hash{}, - }, - }, nil) t.Run("happyPath", func(t *testing.T) { // Test tx is not discarded due to confirm timeout and tracked to finalization @@ -847,7 +848,8 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { // tx should be able to queue testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + lastValidBlockHeight := uint64(100) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() // wait to be picked up and processed waitFor(t, 5*time.Second, txm, prom, empty) // inflight txs cleared after timeout @@ -886,7 +888,8 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { // tx should be able to queue testTxID := uuid.NewString() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + lastValidBlockHeight := uint64(100) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() waitFor(t, 5*time.Second, txm, prom, empty) // inflight txs cleared after timeout @@ -931,7 +934,8 @@ func TestTxm_disabled_confirm_timeout_with_retention(t *testing.T) { // tx should be able to queue testTxID := uuid.NewString() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + lastValidBlockHeight := uint64(100) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() // wait till send tx waitFor(t, 5*time.Second, txm, prom, empty) // inflight txs cleared after timeout @@ -1005,12 +1009,6 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { return out }, nil, ) - mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: 100, - Blockhash: solana.Hash{}, - }, - }, nil) t.Run("simulation_succeeds", func(t *testing.T) { // Test tx is not discarded due to confirm timeout and tracked to finalization @@ -1057,7 +1055,8 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { // send tx testTxID := uuid.New().String() - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID)) + lastValidBlockHeight := uint64(100) + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &testTxID, lastValidBlockHeight)) wg.Wait() // no transactions stored inflight txs list @@ -1086,7 +1085,8 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { mc.On("SimulateTx", mock.Anything, mock.Anything, mock.Anything).Return(nil, errors.New("simulation failed")).Once() // tx should NOT be able to queue - assert.Error(t, txm.Enqueue(ctx, t.Name(), tx, nil)) + lastValidBlockHeight := uint64(0) + assert.Error(t, txm.Enqueue(ctx, t.Name(), tx, nil, lastValidBlockHeight)) }) t.Run("simulation_returns_error", func(t *testing.T) { @@ -1101,8 +1101,9 @@ func TestTxm_compute_unit_limit_estimation(t *testing.T) { mc.On("SimulateTx", mock.Anything, simulateTx, mock.Anything).Return(&rpc.SimulateTransactionResult{Err: errors.New("InstructionError")}, nil).Once() txID := uuid.NewString() + lastValidBlockHeight := uint64(100) // tx should NOT be able to queue - assert.Error(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) + assert.Error(t, txm.Enqueue(ctx, t.Name(), tx, &txID, lastValidBlockHeight)) // tx should be stored in-memory and moved to errored state status, err := txm.GetTransactionStatus(ctx, txID) require.NoError(t, err) @@ -1115,12 +1116,6 @@ func TestTxm_Enqueue(t *testing.T) { lggr := logger.Test(t) cfg := config.NewDefault() mc := mocks.NewReaderWriter(t) - mc.On("LatestBlockhash", mock.Anything).Return(&rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: 100, - Blockhash: solana.Hash{}, - }, - }, nil).Once() mc.On("SendTx", mock.Anything, mock.Anything).Return(solana.Signature{}, nil).Maybe() mc.On("SimulateTx", mock.Anything, mock.Anything, mock.Anything).Return(&rpc.SimulateTransactionResult{}, nil).Maybe() mc.On("SignatureStatuses", mock.Anything, mock.AnythingOfType("[]solana.Signature")).Return( @@ -1154,6 +1149,7 @@ func TestTxm_Enqueue(t *testing.T) { ) require.NoError(t, err) + lastValidBlockHeight := uint64(0) invalidTx, err := solana.NewTransaction( []solana.Instruction{ system.NewTransferInstruction( @@ -1170,28 +1166,29 @@ func TestTxm_Enqueue(t *testing.T) { loader := utils.NewLazyLoad(func() (client.ReaderWriter, error) { return mc, nil }) txm := NewTxm("enqueue_test", loader, nil, cfg, mkey, lggr) - require.ErrorContains(t, txm.Enqueue(ctx, "txmUnstarted", &solana.Transaction{}, nil), "not started") + require.ErrorContains(t, txm.Enqueue(ctx, "txmUnstarted", &solana.Transaction{}, nil, lastValidBlockHeight), "not started") require.NoError(t, txm.Start(ctx)) t.Cleanup(func() { require.NoError(t, txm.Close()) }) txs := []struct { - name string - tx *solana.Transaction - fail bool + name string + tx *solana.Transaction + lastValidBlockHeight uint64 + fail bool }{ - {"success", tx, false}, - {"invalid_key", invalidTx, true}, - {"nil_pointer", nil, true}, - {"empty_tx", &solana.Transaction{}, true}, + {"success", tx, 100, false}, + {"invalid_key", invalidTx, 0, true}, + {"nil_pointer", nil, 0, true}, + {"empty_tx", &solana.Transaction{}, 0, true}, } for _, run := range txs { t.Run(run.name, func(t *testing.T) { if !run.fail { - assert.NoError(t, txm.Enqueue(ctx, run.name, run.tx, nil)) + assert.NoError(t, txm.Enqueue(ctx, run.name, run.tx, nil, run.lastValidBlockHeight)) return } - assert.Error(t, txm.Enqueue(ctx, run.name, run.tx, nil)) + assert.Error(t, txm.Enqueue(ctx, run.name, run.tx, nil, run.lastValidBlockHeight)) }) } } @@ -1298,18 +1295,10 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { }, nil } - callCount := 0 + rebroadcastCount := 0 latestBlockhashFunc := func() (*rpc.GetLatestBlockhashResult, error) { - defer func() { callCount++ }() - if callCount < 1 { - // To force rebroadcast, first call needs to be smaller than blockHeight - return &rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: uint64(1000), - }, - }, nil - } - // following rebroadcast call will go through because lastValidBlockHeight is bigger than blockHeight + defer func() { rebroadcastCount++ }() + // rebroadcast call will go through because lastValidBlockHeight is bigger than blockHeight return &rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ LastValidBlockHeight: uint64(2000), @@ -1330,31 +1319,31 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { // First transaction should be rebroadcasted. if time.Since(nowTs) < cfg.TxConfirmTimeout()-2*time.Second { return nil - } else { - // Second transaction should reach finalization. - sigStatusCallCount++ - if sigStatusCallCount == 1 { - return &rpc.SignatureStatusesResult{ - ConfirmationStatus: rpc.ConfirmationStatusProcessed, - } - } else if sigStatusCallCount == 2 { - return &rpc.SignatureStatusesResult{ - ConfirmationStatus: rpc.ConfirmationStatusConfirmed, - } - } else { - wg.Done() - return &rpc.SignatureStatusesResult{ - ConfirmationStatus: rpc.ConfirmationStatusFinalized, - } + } + // Second transaction should reach finalization. + sigStatusCallCount++ + if sigStatusCallCount == 1 { + return &rpc.SignatureStatusesResult{ + ConfirmationStatus: rpc.ConfirmationStatusProcessed, } } + if sigStatusCallCount == 2 { + return &rpc.SignatureStatusesResult{ + ConfirmationStatus: rpc.ConfirmationStatusConfirmed, + } + } + wg.Done() + return &rpc.SignatureStatusesResult{ + ConfirmationStatus: rpc.ConfirmationStatusFinalized, + } } txm, _, mkey := setupTxmTest(txExpirationRebroadcast, latestBlockhashFunc, getLatestBlockFunc, sendTxFunc, statuses) tx, _ := getTx(t, 0, mkey) txID := "test-rebroadcast" - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) + lastValidBlockHeight := uint64(100) // lastValidBlockHeight is smaller than blockHeight + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID, lastValidBlockHeight)) wg.Wait() waitFor(t, txm.cfg.TxConfirmTimeout(), txm, prom, empty) @@ -1367,23 +1356,13 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { status, err := txm.GetTransactionStatus(ctx, txID) require.NoError(t, err) require.Equal(t, types.Finalized, status) - require.Equal(t, 1, callCount-1) // -1 because the first call is not a rebroadcast + require.Equal(t, 1, rebroadcastCount) }) t.Run("WithoutRebroadcast", func(t *testing.T) { txExpirationRebroadcast := false statuses := map[solana.Signature]func() *rpc.SignatureStatusesResult{} - - // mocking the call within sendWithRetry. Rebroadcast is off, so we won't compare it against the blockHeight. - callCount := 0 - latestBlockhashFunc := func() (*rpc.GetLatestBlockhashResult, error) { - defer func() { callCount++ }() - return &rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: uint64(2000), - }, - }, nil - } + rebroadcastCount := 0 sig1 := randomSignature(t) sendTxFunc := func() (solana.Signature, error) { @@ -1397,17 +1376,17 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { // Transaction remains unconfirmed and should not be rebroadcasted. if time.Since(nowTs) < cfg.TxConfirmTimeout() { return nil - } else { - wg.Done() - return nil } + wg.Done() + return nil } - txm, _, mkey := setupTxmTest(txExpirationRebroadcast, latestBlockhashFunc, nil, sendTxFunc, statuses) + txm, _, mkey := setupTxmTest(txExpirationRebroadcast, nil, nil, sendTxFunc, statuses) tx, _ := getTx(t, 5, mkey) txID := "test-no-rebroadcast" - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) + lastValidBlockHeight := uint64(0) // original lastValidBlockHeight is invalid + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID, lastValidBlockHeight)) wg.Wait() waitFor(t, txm.cfg.TxConfirmTimeout(), txm, prom, empty) @@ -1420,7 +1399,7 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { status, err := txm.GetTransactionStatus(ctx, txID) require.NoError(t, err) require.Equal(t, types.Failed, status) - require.Equal(t, 0, callCount-1) // -1 because the first call is not a rebroadcast + require.Equal(t, 0, rebroadcastCount) }) t.Run("WithMultipleRebroadcast", func(t *testing.T) { @@ -1435,13 +1414,12 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { }, nil } - // Mock LatestBlockhash to return an invalid blockhash in the first 3 attempts (initial + 2 rebroadcasts) + // Mock LatestBlockhash to return an invalid blockhash in the first 2 attempts to rebroadcast. // the last one is valid because it is greater than the blockHeight - expectedRebroadcastsCount := 3 - callCount := 0 + rebroadcastCount := 0 latestBlockhashFunc := func() (*rpc.GetLatestBlockhashResult, error) { - defer func() { callCount++ }() - if callCount < expectedRebroadcastsCount { + defer func() { rebroadcastCount++ }() + if rebroadcastCount < 2 { return &rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ LastValidBlockHeight: uint64(1000), @@ -1467,30 +1445,29 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { // transaction should be rebroadcasted multiple times. if time.Since(nowTs) < cfg.TxConfirmTimeout()-2*time.Second { return nil - } else { - // Second transaction should reach finalization. - sigStatusCallCount++ - if sigStatusCallCount == 1 { - return &rpc.SignatureStatusesResult{ - ConfirmationStatus: rpc.ConfirmationStatusProcessed, - } - } else if sigStatusCallCount == 2 { - return &rpc.SignatureStatusesResult{ - ConfirmationStatus: rpc.ConfirmationStatusConfirmed, - } - } else { - wg.Done() - return &rpc.SignatureStatusesResult{ - ConfirmationStatus: rpc.ConfirmationStatusFinalized, - } + } + // Second transaction should reach finalization. + sigStatusCallCount++ + if sigStatusCallCount == 1 { + return &rpc.SignatureStatusesResult{ + ConfirmationStatus: rpc.ConfirmationStatusProcessed, } + } else if sigStatusCallCount == 2 { + return &rpc.SignatureStatusesResult{ + ConfirmationStatus: rpc.ConfirmationStatusConfirmed, + } + } + wg.Done() + return &rpc.SignatureStatusesResult{ + ConfirmationStatus: rpc.ConfirmationStatusFinalized, } } txm, _, mkey := setupTxmTest(txExpirationRebroadcast, latestBlockhashFunc, getLatestBlockFunc, sendTxFunc, statuses) tx, _ := getTx(t, 0, mkey) txID := "test-rebroadcast" - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) + lastValidBlockHeight := uint64(100) // lastValidBlockHeight is smaller than blockHeight + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID, lastValidBlockHeight)) wg.Wait() waitFor(t, txm.cfg.TxConfirmTimeout(), txm, prom, empty) @@ -1503,7 +1480,7 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { status, err := txm.GetTransactionStatus(ctx, txID) require.NoError(t, err) require.Equal(t, types.Finalized, status) - require.Equal(t, expectedRebroadcastsCount, callCount-1) + require.Equal(t, 3, rebroadcastCount) }) t.Run("ConfirmedBeforeRebroadcast", func(t *testing.T) { @@ -1522,9 +1499,9 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { }, nil } - callCount := 0 + rebroadcastCount := 0 latestBlockhashFunc := func() (*rpc.GetLatestBlockhashResult, error) { - defer func() { callCount++ }() + defer func() { rebroadcastCount++ }() return &rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ LastValidBlockHeight: uint64(1000), @@ -1555,7 +1532,8 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { txm, _, mkey := setupTxmTest(txExpirationRebroadcast, latestBlockhashFunc, getLatestBlockFunc, sendTxFunc, statuses) tx, _ := getTx(t, 0, mkey) txID := "test-confirmed-before-rebroadcast" - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) + lastValidBlockHeight := uint64(1500) // original lastValidBlockHeight is valid + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID, lastValidBlockHeight)) wg.Wait() waitFor(t, txm.cfg.TxConfirmTimeout(), txm, prom, empty) @@ -1568,7 +1546,7 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { status, err := txm.GetTransactionStatus(ctx, txID) require.NoError(t, err) require.Equal(t, types.Finalized, status) - require.Equal(t, 0, callCount-1) // -1 because the first call is not a rebroadcast + require.Equal(t, 0, rebroadcastCount) }) t.Run("RebroadcastWithError", func(t *testing.T) { @@ -1584,16 +1562,9 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { }, nil } - callCount := 0 + rebroadcastCount := 0 latestBlockhashFunc := func() (*rpc.GetLatestBlockhashResult, error) { - defer func() { callCount++ }() - if callCount < 1 { - return &rpc.GetLatestBlockhashResult{ - Value: &rpc.LatestBlockhashResult{ - LastValidBlockHeight: uint64(1000), - }, - }, nil - } + defer func() { rebroadcastCount++ }() return &rpc.GetLatestBlockhashResult{ Value: &rpc.LatestBlockhashResult{ LastValidBlockHeight: uint64(2000), @@ -1621,7 +1592,8 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { txm, _, mkey := setupTxmTest(txExpirationRebroadcast, latestBlockhashFunc, getLatestBlockFunc, sendTxFunc, statuses) tx, _ := getTx(t, 0, mkey) txID := "test-rebroadcast-error" - assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID)) + lastValidBlockHeight := uint64(100) // lastValidBlockHeight is smaller than blockHeight + assert.NoError(t, txm.Enqueue(ctx, t.Name(), tx, &txID, lastValidBlockHeight)) wg.Wait() waitFor(t, cfg.TxConfirmTimeout(), txm, prom, empty) @@ -1634,7 +1606,6 @@ func TestTxm_ExpirationRebroadcast(t *testing.T) { status, err := txm.GetTransactionStatus(ctx, txID) require.NoError(t, err) require.Equal(t, types.Failed, status) - require.Equal(t, 1, callCount-1) // -1 because the first call is not a rebroadcast + require.Equal(t, 1, rebroadcastCount) }) - } diff --git a/pkg/solana/txm/txm_load_test.go b/pkg/solana/txm/txm_load_test.go index aa3d6aac7..333c95e23 100644 --- a/pkg/solana/txm/txm_load_test.go +++ b/pkg/solana/txm/txm_load_test.go @@ -83,10 +83,10 @@ func TestTxm_Integration(t *testing.T) { // already started assert.Error(t, txm.Start(ctx)) - createTx := func(signer solana.PublicKey, sender solana.PublicKey, receiver solana.PublicKey, amt uint64) *solana.Transaction { + createTx := func(signer solana.PublicKey, sender solana.PublicKey, receiver solana.PublicKey, amt uint64) (*solana.Transaction, uint64) { // create transfer tx - hash, err := client.LatestBlockhash(ctx) - assert.NoError(t, err) + hash, blockhashErr := client.LatestBlockhash(ctx) + assert.NoError(t, blockhashErr) tx, txErr := solana.NewTransaction( []solana.Instruction{ system.NewTransferInstruction( @@ -99,19 +99,25 @@ func TestTxm_Integration(t *testing.T) { solana.TransactionPayer(signer), ) require.NoError(t, txErr) - return tx + return tx, hash.Value.LastValidBlockHeight } // enqueue txs (must pass to move on to load test) - require.NoError(t, txm.Enqueue(ctx, "test_success_0", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) - require.Error(t, txm.Enqueue(ctx, "test_invalidSigner", createTx(pubKeyReceiver, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) // cannot sign tx before enqueuing - require.NoError(t, txm.Enqueue(ctx, "test_invalidReceiver", createTx(pubKey, pubKey, solana.PublicKey{}, solana.LAMPORTS_PER_SOL), nil)) + tx, lastValidBlockHeight := createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL) + require.NoError(t, txm.Enqueue(ctx, "test_success_0", tx, nil, lastValidBlockHeight)) + tx2, lastValidBlockHeight2 := createTx(pubKeyReceiver, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL) + require.Error(t, txm.Enqueue(ctx, "test_invalidSigner", tx2, nil, lastValidBlockHeight2)) // cannot sign tx before enqueuing + tx3, lastValidBlockHeight3 := createTx(pubKey, pubKey, solana.PublicKey{}, solana.LAMPORTS_PER_SOL) + require.NoError(t, txm.Enqueue(ctx, "test_invalidReceiver", tx3, nil, lastValidBlockHeight3)) time.Sleep(500 * time.Millisecond) // pause 0.5s for new blockhash - require.NoError(t, txm.Enqueue(ctx, "test_success_1", createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL), nil)) - require.NoError(t, txm.Enqueue(ctx, "test_txFail", createTx(pubKey, pubKey, pubKeyReceiver, 1000*solana.LAMPORTS_PER_SOL), nil)) + tx4, lastValidBlockHeight4 := createTx(pubKey, pubKey, pubKeyReceiver, solana.LAMPORTS_PER_SOL) + require.NoError(t, txm.Enqueue(ctx, "test_success_1", tx4, nil, lastValidBlockHeight4)) + tx5, lastValidBlockHeight5 := createTx(pubKey, pubKey, pubKeyReceiver, 1000*solana.LAMPORTS_PER_SOL) + require.NoError(t, txm.Enqueue(ctx, "test_txFail", tx5, nil, lastValidBlockHeight5)) // load test: try to overload txs, confirm, or simulation for i := 0; i < 1000; i++ { - assert.NoError(t, txm.Enqueue(ctx, fmt.Sprintf("load_%d", i), createTx(loadTestKey.PublicKey(), loadTestKey.PublicKey(), loadTestKey.PublicKey(), uint64(i)), nil)) + tx6, lastValidBlockHeight6 := createTx(loadTestKey.PublicKey(), loadTestKey.PublicKey(), loadTestKey.PublicKey(), uint64(i)) + assert.NoError(t, txm.Enqueue(ctx, fmt.Sprintf("load_%d", i), tx6, nil, lastValidBlockHeight6)) time.Sleep(10 * time.Millisecond) // ~100 txs per second (note: have run 5ms delays for ~200tx/s successfully) } diff --git a/pkg/solana/txm/txm_race_test.go b/pkg/solana/txm/txm_race_test.go index 7abde910e..33ec0f7bf 100644 --- a/pkg/solana/txm/txm_race_test.go +++ b/pkg/solana/txm/txm_race_test.go @@ -9,7 +9,6 @@ import ( "time" solanaGo "github.com/gagliardetto/solana-go" - solanaGoRpc "github.com/gagliardetto/solana-go/rpc" "go.uber.org/zap/zapcore" "github.com/smartcontractkit/chainlink-common/pkg/logger" @@ -82,13 +81,6 @@ func TestTxm_SendWithRetry_Race(t *testing.T) { assert.Contains(t, lastLog.Message, "stopped tx retry") // assert that all retry goroutines exit successfully } client := clientmocks.NewReaderWriter(t) - client.On("LatestBlockhash", mock.Anything).Return(&solanaGoRpc.GetLatestBlockhashResult{ - Value: &solanaGoRpc.LatestBlockhashResult{ - LastValidBlockHeight: 100, - Blockhash: solanaGo.Hash{}, - }, - }, nil) - t.Run("delay in rebroadcasting tx", func(t *testing.T) { txs := map[string]solanaGo.Signature{} var lock sync.RWMutex From 1be704a3d7cf85540817640145eb1dfde262ef41 Mon Sep 17 00:00:00 2001 From: Farber98 Date: Tue, 17 Dec 2024 17:18:18 -0300 Subject: [PATCH 50/51] add enq iface comm to help callers --- pkg/solana/relay.go | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/pkg/solana/relay.go b/pkg/solana/relay.go index 1df61b99d..bd6407cca 100644 --- a/pkg/solana/relay.go +++ b/pkg/solana/relay.go @@ -24,6 +24,15 @@ import ( var _ TxManager = (*txm.Txm)(nil) type TxManager interface { + // Enqueue adds a tx to the txm queue for processing and submitting to the Solana network. + // An error is returned if the txm is not ready, if the tx is invalid, or if the queue is full. + // + // Important Notes: + // - The tx must contain at least one account key. The first account will be used to sign the tx (fee payer's public key). + // - txCfgs can be used to set custom tx configurations. + // - If an txID is provided, it will be used to identify the tx. Otherwise, a random UUID will be generated. + // - The caller needs to set the tx.Message.RecentBlockhash and provide the corresponding lastValidBlockHeight. + // These values are obtained from the GetLatestBlockhash RPC call. Enqueue(ctx context.Context, accountID string, tx *solana.Transaction, txID *string, lastValidBlockHeight uint64, txCfgs ...txm.SetTxConfig) error } From 9ee89773227d19bc4aa9ae3f4e2593e856ac8f2d Mon Sep 17 00:00:00 2001 From: Farber98 Date: Tue, 17 Dec 2024 17:54:07 -0300 Subject: [PATCH 51/51] address feedback --- pkg/solana/relay.go | 5 ++--- pkg/solana/txm/txm.go | 26 +++++++++++++------------- pkg/solana/txm/txm_integration_test.go | 15 ++++++++------- 3 files changed, 23 insertions(+), 23 deletions(-) diff --git a/pkg/solana/relay.go b/pkg/solana/relay.go index bd6407cca..fca61ba9f 100644 --- a/pkg/solana/relay.go +++ b/pkg/solana/relay.go @@ -30,9 +30,8 @@ type TxManager interface { // Important Notes: // - The tx must contain at least one account key. The first account will be used to sign the tx (fee payer's public key). // - txCfgs can be used to set custom tx configurations. - // - If an txID is provided, it will be used to identify the tx. Otherwise, a random UUID will be generated. - // - The caller needs to set the tx.Message.RecentBlockhash and provide the corresponding lastValidBlockHeight. - // These values are obtained from the GetLatestBlockhash RPC call. + // - If a txID is provided, it will be used to identify the tx. Otherwise, a random UUID will be generated. + // - The caller needs to set the tx.Message.RecentBlockhash and provide the corresponding lastValidBlockHeight. These values are obtained from the GetLatestBlockhash RPC call. Enqueue(ctx context.Context, accountID string, tx *solana.Transaction, txID *string, lastValidBlockHeight uint64, txCfgs ...txm.SetTxConfig) error } diff --git a/pkg/solana/txm/txm.go b/pkg/solana/txm/txm.go index 187904c3b..3e169d88a 100644 --- a/pkg/solana/txm/txm.go +++ b/pkg/solana/txm/txm.go @@ -566,28 +566,28 @@ func (txm *Txm) rebroadcastExpiredTxs(ctx context.Context, client client.ReaderW return } - // Request new blockhash and loop through all expired txes overwriting with new blockhash and rebroadcasting + blockhash, err := client.LatestBlockhash(ctx) + if err != nil { + txm.lggr.Errorw("failed to getLatestBlockhash for rebroadcast", "error", err) + return + } + if blockhash == nil || blockhash.Value == nil { + txm.lggr.Errorw("nil pointer returned from getLatestBlockhash for rebroadcast") + return + } + + // rebroadcast each expired tx after updating blockhash, lastValidBlockHeight and compute unit price (priority fee) for _, tx := range expiredBroadcastedTxes { txm.lggr.Debugw("transaction expired, rebroadcasting", "id", tx.id, "signature", tx.signatures, "lastValidBlockHeight", tx.lastValidBlockHeight, "currentBlockHeight", *currBlock.BlockHeight) - // Removes all signatures associated to tx and cancels context. + // Removes all signatures associated to prior tx and cancels context. _, err := txm.txs.Remove(tx.id) if err != nil { txm.lggr.Errorw("failed to remove expired transaction", "id", tx.id, "error", err) continue } - blockhash, err := client.LatestBlockhash(ctx) - if err != nil { - txm.lggr.Errorw("failed to get latest blockhash for rebroadcast", "error", err) - return - } - if blockhash == nil || blockhash.Value == nil { - txm.lggr.Errorw("nil pointer returned from LatestBlockhash for rebroadcast") - return - } - tx.tx.Message.RecentBlockhash = blockhash.Value.Blockhash - tx.cfg.BaseComputeUnitPrice = txm.fee.BaseComputeUnitPrice() // update compute unit price (priority fee) for rebroadcast + tx.cfg.BaseComputeUnitPrice = txm.fee.BaseComputeUnitPrice() rebroadcastTx := pendingTx{ tx: tx.tx, cfg: tx.cfg, diff --git a/pkg/solana/txm/txm_integration_test.go b/pkg/solana/txm/txm_integration_test.go index 1beeafe2b..154a42f6a 100644 --- a/pkg/solana/txm/txm_integration_test.go +++ b/pkg/solana/txm/txm_integration_test.go @@ -18,6 +18,7 @@ import ( "github.com/smartcontractkit/chainlink-common/pkg/services/servicetest" "github.com/smartcontractkit/chainlink-common/pkg/types" "github.com/smartcontractkit/chainlink-common/pkg/utils" + "github.com/smartcontractkit/chainlink-common/pkg/utils/tests" relayconfig "github.com/smartcontractkit/chainlink-common/pkg/config" @@ -103,21 +104,21 @@ func TestTxm_Integration_ExpirationRebroadcast(t *testing.T) { } // Verify rebroadcast logs - rebroadcastLogs := observer.FilterMessageSnippet("rebroadcast transaction sent").All() - rebroadcastLogs2 := observer.FilterMessageSnippet("transaction expired, rebroadcasting").All() + rebroadcastLogs := observer.FilterMessageSnippet("rebroadcast transaction sent").Len() + rebroadcastLogs2 := observer.FilterMessageSnippet("transaction expired, rebroadcasting").Len() if tc.expectRebroadcast { - require.NotEmpty(t, rebroadcastLogs, "Expected rebroadcast log message not found") - require.NotEmpty(t, rebroadcastLogs2, "Expected rebroadcast log message not found") + require.Equal(t, 1, rebroadcastLogs, "Expected rebroadcast log message not found") + require.Equal(t, 1, rebroadcastLogs2, "Expected rebroadcast log message not found") } else { - require.Empty(t, rebroadcastLogs, "Rebroadcast should not occur") - require.Empty(t, rebroadcastLogs2, "Rebroadcast should not occur") + require.Equal(t, 0, rebroadcastLogs, "Rebroadcast should not occur") + require.Equal(t, 0, rebroadcastLogs2, "Rebroadcast should not occur") } }) } } func setup(t *testing.T, url string, txExpirationRebroadcast bool) (context.Context, *solanaClient.Client, *txm.Txm, solana.PublicKey, solana.PublicKey, *observer.ObservedLogs) { - ctx := context.Background() + ctx := tests.Context(t) // Generate sender and receiver keys and fund sender account senderKey, err := solana.NewRandomPrivateKey()