Use single codepath for sending transactions to a local and remote nodes (#527)
- new EthereumTransactor that provides higher level API for working with ethereum network, and it is fully conformant with ethclient - new test rpc service that improves flexibility and coverage of txqueue manager tests - run complete transaction sequantially for each address - go-ethereum: Get pending nonce from transaction pool - add a patch with getting nonce from transaction pool
This commit is contained in:
parent
ea55ac0744
commit
0771e7d1b7
|
@ -23,7 +23,7 @@ jobs:
|
|||
# using fork == false may be preferred as it would allow PRs from the origin
|
||||
# to still run, but currently does not work due to a bug
|
||||
if: type != pull_request
|
||||
script: travis_wait 300 make test-e2e networkid=4
|
||||
script: make test-e2e networkid=4
|
||||
cache:
|
||||
directories:
|
||||
- ".ethereumtest/Mainnet"
|
||||
|
|
1
Makefile
1
Makefile
|
@ -113,6 +113,7 @@ mock: ##@other Regenerate mocks
|
|||
mockgen -source=geth/mailservice/mailservice.go -destination=geth/mailservice/mailservice_mock.go -package=mailservice
|
||||
mockgen -source=geth/common/notification.go -destination=geth/common/notification_mock.go -package=common -imports fcm=github.com/NaySoftware/go-fcm
|
||||
mockgen -source=geth/notification/fcm/client.go -destination=geth/notification/fcm/client_mock.go -package=fcm -imports fcm=github.com/NaySoftware/go-fcm
|
||||
mockgen -source=geth/txqueue/fake/txservice.go -destination=geth/txqueue/fake/mock.go -package=fake
|
||||
|
||||
test: test-unit-coverage ##@tests Run basic, short tests during development
|
||||
|
||||
|
|
|
@ -0,0 +1,19 @@
|
|||
diff --git a/internal/ethapi/api.go b/internal/ethapi/api.go
|
||||
index 362379cc..6e12e500 100644
|
||||
--- a/internal/ethapi/api.go
|
||||
+++ b/internal/ethapi/api.go
|
||||
@@ -956,6 +956,14 @@ func (s *PublicTransactionPoolAPI) GetRawTransactionByBlockHashAndIndex(ctx cont
|
||||
|
||||
// GetTransactionCount returns the number of transactions the given address has sent for the given block number
|
||||
func (s *PublicTransactionPoolAPI) GetTransactionCount(ctx context.Context, address common.Address, blockNr rpc.BlockNumber) (*hexutil.Uint64, error) {
|
||||
+ // go-ethereum issue https://github.com/ethereum/go-ethereum/issues/2880
|
||||
+ if blockNr == rpc.PendingBlockNumber {
|
||||
+ nonce, err := s.b.GetPoolNonce(ctx, address)
|
||||
+ if err != nil {
|
||||
+ return nil, err
|
||||
+ }
|
||||
+ return (*hexutil.Uint64)(&nonce), nil
|
||||
+ }
|
||||
state, _, err := s.b.StateAndHeaderByNumber(ctx, blockNr)
|
||||
if state == nil || err != nil {
|
||||
return nil, err
|
|
@ -16,6 +16,7 @@ We try to minimize number and amount of changes in those patches as much as poss
|
|||
- `0005-whisper-delivery.patch` - adds support for logs/traces of Whisper traffic (questionable, nobody used this functionality so far)
|
||||
- `0006-latest-cht.patch` – updates CHT root hashes, should be updated regularly to keep sync fast, until proper Trusted Checkpoint sync is not implemented as part of LES/2 protocol.
|
||||
- `0007-README.patch` — update upstream README.md.
|
||||
- `0008-tx-pool-nonce.patch` - On GetTransactionCount request with PendingBlockNumber get the nonce from transaction pool
|
||||
|
||||
# Updating upstream version
|
||||
|
||||
|
|
|
@ -90,8 +90,6 @@ func (m *StatusBackend) StartNode(config *params.NodeConfig) (<-chan struct{}, e
|
|||
return nil, err
|
||||
}
|
||||
|
||||
m.txQueueManager.Start()
|
||||
|
||||
m.nodeReady = make(chan struct{}, 1)
|
||||
go m.onNodeStart(nodeStarted, m.nodeReady) // waits on nodeStarted, writes to backendReady
|
||||
|
||||
|
@ -102,6 +100,10 @@ func (m *StatusBackend) StartNode(config *params.NodeConfig) (<-chan struct{}, e
|
|||
func (m *StatusBackend) onNodeStart(nodeStarted <-chan struct{}, backendReady chan struct{}) {
|
||||
<-nodeStarted
|
||||
|
||||
// tx queue manager should be started after node is started, it depends
|
||||
// on rpc client being created
|
||||
m.txQueueManager.Start()
|
||||
|
||||
if err := m.registerHandlers(); err != nil {
|
||||
log.Error("Handler registration failed", "err", err)
|
||||
}
|
||||
|
|
|
@ -0,0 +1,39 @@
|
|||
// copy of go-ethereum/internal/ethapi/addrlock.go
|
||||
|
||||
package txqueue
|
||||
|
||||
import (
|
||||
"sync"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
)
|
||||
|
||||
type AddrLocker struct {
|
||||
mu sync.Mutex
|
||||
locks map[common.Address]*sync.Mutex
|
||||
}
|
||||
|
||||
// lock returns the lock of the given address.
|
||||
func (l *AddrLocker) lock(address common.Address) *sync.Mutex {
|
||||
l.mu.Lock()
|
||||
defer l.mu.Unlock()
|
||||
if l.locks == nil {
|
||||
l.locks = make(map[common.Address]*sync.Mutex)
|
||||
}
|
||||
if _, ok := l.locks[address]; !ok {
|
||||
l.locks[address] = new(sync.Mutex)
|
||||
}
|
||||
return l.locks[address]
|
||||
}
|
||||
|
||||
// LockAddr locks an account's mutex. This is used to prevent another tx getting the
|
||||
// same nonce until the lock is released. The mutex prevents the (an identical nonce) from
|
||||
// being read again during the time that the first transaction is being signed.
|
||||
func (l *AddrLocker) LockAddr(address common.Address) {
|
||||
l.lock(address).Lock()
|
||||
}
|
||||
|
||||
// UnlockAddr unlocks the mutex of the given account.
|
||||
func (l *AddrLocker) UnlockAddr(address common.Address) {
|
||||
l.lock(address).Unlock()
|
||||
}
|
|
@ -0,0 +1,93 @@
|
|||
package txqueue
|
||||
|
||||
import (
|
||||
"context"
|
||||
"math/big"
|
||||
|
||||
ethereum "github.com/ethereum/go-ethereum"
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/common/hexutil"
|
||||
"github.com/ethereum/go-ethereum/core/types"
|
||||
"github.com/ethereum/go-ethereum/rlp"
|
||||
"github.com/status-im/status-go/geth/rpc"
|
||||
)
|
||||
|
||||
// EthTransactor provides methods to create transactions for ethereum network.
|
||||
type EthTransactor interface {
|
||||
PendingNonceAt(ctx context.Context, account common.Address) (uint64, error)
|
||||
ethereum.GasEstimator
|
||||
ethereum.GasPricer
|
||||
ethereum.TransactionSender
|
||||
}
|
||||
|
||||
// EthTxClient wraps common API methods that are used to send transaction.
|
||||
type EthTxClient struct {
|
||||
c *rpc.Client
|
||||
}
|
||||
|
||||
func NewEthTxClient(client *rpc.Client) *EthTxClient {
|
||||
return &EthTxClient{c: client}
|
||||
}
|
||||
|
||||
// PendingNonceAt returns the account nonce of the given account in the pending state.
|
||||
// This is the nonce that should be used for the next transaction.
|
||||
func (ec *EthTxClient) PendingNonceAt(ctx context.Context, account common.Address) (uint64, error) {
|
||||
var result hexutil.Uint64
|
||||
err := ec.c.CallContext(ctx, &result, "eth_getTransactionCount", account, "pending")
|
||||
return uint64(result), err
|
||||
}
|
||||
|
||||
// SuggestGasPrice retrieves the currently suggested gas price to allow a timely
|
||||
// execution of a transaction.
|
||||
func (ec *EthTxClient) SuggestGasPrice(ctx context.Context) (*big.Int, error) {
|
||||
var hex hexutil.Big
|
||||
if err := ec.c.CallContext(ctx, &hex, "eth_gasPrice"); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return (*big.Int)(&hex), nil
|
||||
}
|
||||
|
||||
// EstimateGas tries to estimate the gas needed to execute a specific transaction based on
|
||||
// the current pending state of the backend blockchain. There is no guarantee that this is
|
||||
// the true gas limit requirement as other transactions may be added or removed by miners,
|
||||
// but it should provide a basis for setting a reasonable default.
|
||||
func (ec *EthTxClient) EstimateGas(ctx context.Context, msg ethereum.CallMsg) (*big.Int, error) {
|
||||
var hex hexutil.Big
|
||||
err := ec.c.CallContext(ctx, &hex, "eth_estimateGas", toCallArg(msg))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return (*big.Int)(&hex), nil
|
||||
}
|
||||
|
||||
// SendTransaction injects a signed transaction into the pending pool for execution.
|
||||
//
|
||||
// If the transaction was a contract creation use the TransactionReceipt method to get the
|
||||
// contract address after the transaction has been mined.
|
||||
func (ec *EthTxClient) SendTransaction(ctx context.Context, tx *types.Transaction) error {
|
||||
data, err := rlp.EncodeToBytes(tx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return ec.c.CallContext(ctx, nil, "eth_sendRawTransaction", common.ToHex(data))
|
||||
}
|
||||
|
||||
func toCallArg(msg ethereum.CallMsg) interface{} {
|
||||
arg := map[string]interface{}{
|
||||
"from": msg.From,
|
||||
"to": msg.To,
|
||||
}
|
||||
if len(msg.Data) > 0 {
|
||||
arg["data"] = hexutil.Bytes(msg.Data)
|
||||
}
|
||||
if msg.Value != nil {
|
||||
arg["value"] = (*hexutil.Big)(msg.Value)
|
||||
}
|
||||
if msg.Gas != nil {
|
||||
arg["gas"] = (*hexutil.Big)(msg.Gas)
|
||||
}
|
||||
if msg.GasPrice != nil {
|
||||
arg["gasPrice"] = (*hexutil.Big)(msg.GasPrice)
|
||||
}
|
||||
return arg
|
||||
}
|
|
@ -0,0 +1,90 @@
|
|||
// Code generated by MockGen. DO NOT EDIT.
|
||||
// Source: geth/txqueue/fake/txservice.go
|
||||
|
||||
// Package fake is a generated GoMock package.
|
||||
package fake
|
||||
|
||||
import (
|
||||
context "context"
|
||||
common "github.com/ethereum/go-ethereum/common"
|
||||
hexutil "github.com/ethereum/go-ethereum/common/hexutil"
|
||||
rpc "github.com/ethereum/go-ethereum/rpc"
|
||||
gomock "github.com/golang/mock/gomock"
|
||||
big "math/big"
|
||||
reflect "reflect"
|
||||
)
|
||||
|
||||
// MockFakePublicTransactionPoolAPI is a mock of FakePublicTransactionPoolAPI interface
|
||||
type MockFakePublicTransactionPoolAPI struct {
|
||||
ctrl *gomock.Controller
|
||||
recorder *MockFakePublicTransactionPoolAPIMockRecorder
|
||||
}
|
||||
|
||||
// MockFakePublicTransactionPoolAPIMockRecorder is the mock recorder for MockFakePublicTransactionPoolAPI
|
||||
type MockFakePublicTransactionPoolAPIMockRecorder struct {
|
||||
mock *MockFakePublicTransactionPoolAPI
|
||||
}
|
||||
|
||||
// NewMockFakePublicTransactionPoolAPI creates a new mock instance
|
||||
func NewMockFakePublicTransactionPoolAPI(ctrl *gomock.Controller) *MockFakePublicTransactionPoolAPI {
|
||||
mock := &MockFakePublicTransactionPoolAPI{ctrl: ctrl}
|
||||
mock.recorder = &MockFakePublicTransactionPoolAPIMockRecorder{mock}
|
||||
return mock
|
||||
}
|
||||
|
||||
// EXPECT returns an object that allows the caller to indicate expected use
|
||||
func (m *MockFakePublicTransactionPoolAPI) EXPECT() *MockFakePublicTransactionPoolAPIMockRecorder {
|
||||
return m.recorder
|
||||
}
|
||||
|
||||
// GasPrice mocks base method
|
||||
func (m *MockFakePublicTransactionPoolAPI) GasPrice(ctx context.Context) (*big.Int, error) {
|
||||
ret := m.ctrl.Call(m, "GasPrice", ctx)
|
||||
ret0, _ := ret[0].(*big.Int)
|
||||
ret1, _ := ret[1].(error)
|
||||
return ret0, ret1
|
||||
}
|
||||
|
||||
// GasPrice indicates an expected call of GasPrice
|
||||
func (mr *MockFakePublicTransactionPoolAPIMockRecorder) GasPrice(ctx interface{}) *gomock.Call {
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GasPrice", reflect.TypeOf((*MockFakePublicTransactionPoolAPI)(nil).GasPrice), ctx)
|
||||
}
|
||||
|
||||
// EstimateGas mocks base method
|
||||
func (m *MockFakePublicTransactionPoolAPI) EstimateGas(ctx context.Context, args CallArgs) (*hexutil.Big, error) {
|
||||
ret := m.ctrl.Call(m, "EstimateGas", ctx, args)
|
||||
ret0, _ := ret[0].(*hexutil.Big)
|
||||
ret1, _ := ret[1].(error)
|
||||
return ret0, ret1
|
||||
}
|
||||
|
||||
// EstimateGas indicates an expected call of EstimateGas
|
||||
func (mr *MockFakePublicTransactionPoolAPIMockRecorder) EstimateGas(ctx, args interface{}) *gomock.Call {
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "EstimateGas", reflect.TypeOf((*MockFakePublicTransactionPoolAPI)(nil).EstimateGas), ctx, args)
|
||||
}
|
||||
|
||||
// GetTransactionCount mocks base method
|
||||
func (m *MockFakePublicTransactionPoolAPI) GetTransactionCount(ctx context.Context, address common.Address, blockNr rpc.BlockNumber) (*hexutil.Uint64, error) {
|
||||
ret := m.ctrl.Call(m, "GetTransactionCount", ctx, address, blockNr)
|
||||
ret0, _ := ret[0].(*hexutil.Uint64)
|
||||
ret1, _ := ret[1].(error)
|
||||
return ret0, ret1
|
||||
}
|
||||
|
||||
// GetTransactionCount indicates an expected call of GetTransactionCount
|
||||
func (mr *MockFakePublicTransactionPoolAPIMockRecorder) GetTransactionCount(ctx, address, blockNr interface{}) *gomock.Call {
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetTransactionCount", reflect.TypeOf((*MockFakePublicTransactionPoolAPI)(nil).GetTransactionCount), ctx, address, blockNr)
|
||||
}
|
||||
|
||||
// SendRawTransaction mocks base method
|
||||
func (m *MockFakePublicTransactionPoolAPI) SendRawTransaction(ctx context.Context, encodedTx hexutil.Bytes) (common.Hash, error) {
|
||||
ret := m.ctrl.Call(m, "SendRawTransaction", ctx, encodedTx)
|
||||
ret0, _ := ret[0].(common.Hash)
|
||||
ret1, _ := ret[1].(error)
|
||||
return ret0, ret1
|
||||
}
|
||||
|
||||
// SendRawTransaction indicates an expected call of SendRawTransaction
|
||||
func (mr *MockFakePublicTransactionPoolAPIMockRecorder) SendRawTransaction(ctx, encodedTx interface{}) *gomock.Call {
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SendRawTransaction", reflect.TypeOf((*MockFakePublicTransactionPoolAPI)(nil).SendRawTransaction), ctx, encodedTx)
|
||||
}
|
|
@ -0,0 +1,40 @@
|
|||
package fake
|
||||
|
||||
import (
|
||||
"context"
|
||||
"math/big"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/common/hexutil"
|
||||
"github.com/ethereum/go-ethereum/rpc"
|
||||
"github.com/golang/mock/gomock"
|
||||
)
|
||||
|
||||
func NewTestServer(ctrl *gomock.Controller) (*rpc.Server, *MockFakePublicTransactionPoolAPI) {
|
||||
srv := rpc.NewServer()
|
||||
svc := NewMockFakePublicTransactionPoolAPI(ctrl)
|
||||
if err := srv.RegisterName("eth", svc); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return srv, svc
|
||||
}
|
||||
|
||||
// CallArgs copied from module go-ethereum/internal/ethapi
|
||||
type CallArgs struct {
|
||||
From common.Address `json:"from"`
|
||||
To *common.Address `json:"to"`
|
||||
Gas hexutil.Big `json:"gas"`
|
||||
GasPrice hexutil.Big `json:"gasPrice"`
|
||||
Value hexutil.Big `json:"value"`
|
||||
Data hexutil.Bytes `json:"data"`
|
||||
}
|
||||
|
||||
// FakePublicTransactionPoolAPI used to generate mock by mockgen util.
|
||||
// This was done because PublicTransactionPoolAPI is located in internal/ethapi module
|
||||
// and there is no easy way to generate mocks from internal modules.
|
||||
type FakePublicTransactionPoolAPI interface {
|
||||
GasPrice(ctx context.Context) (*big.Int, error)
|
||||
EstimateGas(ctx context.Context, args CallArgs) (*hexutil.Big, error)
|
||||
GetTransactionCount(ctx context.Context, address common.Address, blockNr rpc.BlockNumber) (*hexutil.Uint64, error)
|
||||
SendRawTransaction(ctx context.Context, encodedTx hexutil.Bytes) (common.Hash, error)
|
||||
}
|
|
@ -5,12 +5,10 @@ import (
|
|||
"math/big"
|
||||
"time"
|
||||
|
||||
ethereum "github.com/ethereum/go-ethereum"
|
||||
"github.com/ethereum/go-ethereum/accounts/keystore"
|
||||
gethcommon "github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/common/hexutil"
|
||||
"github.com/ethereum/go-ethereum/core/types"
|
||||
"github.com/ethereum/go-ethereum/les/status"
|
||||
"github.com/ethereum/go-ethereum/rlp"
|
||||
"github.com/pborman/uuid"
|
||||
"github.com/status-im/status-go/geth/common"
|
||||
"github.com/status-im/status-go/geth/log"
|
||||
|
@ -26,6 +24,10 @@ const (
|
|||
|
||||
// SendTxDefaultErrorCode is sent by default, when error is not nil, but type is unknown/unexpected.
|
||||
SendTxDefaultErrorCode = SendTransactionDefaultErrorCode
|
||||
|
||||
defaultGas = 90000
|
||||
|
||||
defaultTimeout = time.Minute
|
||||
)
|
||||
|
||||
// Send transaction response codes
|
||||
|
@ -49,6 +51,8 @@ type Manager struct {
|
|||
nodeManager common.NodeManager
|
||||
accountManager common.AccountManager
|
||||
txQueue *TxQueue
|
||||
ethTxClient EthTransactor
|
||||
addrLock *AddrLocker
|
||||
}
|
||||
|
||||
// NewManager returns a new Manager.
|
||||
|
@ -57,12 +61,14 @@ func NewManager(nodeManager common.NodeManager, accountManager common.AccountMan
|
|||
nodeManager: nodeManager,
|
||||
accountManager: accountManager,
|
||||
txQueue: NewTransactionQueue(),
|
||||
addrLock: &AddrLocker{},
|
||||
}
|
||||
}
|
||||
|
||||
// Start starts accepting new transactions into the queue.
|
||||
func (m *Manager) Start() {
|
||||
log.Info("start Manager")
|
||||
m.ethTxClient = NewEthTxClient(m.nodeManager.RPCClient())
|
||||
m.txQueue.Start()
|
||||
}
|
||||
|
||||
|
@ -157,21 +163,8 @@ func (m *Manager) CompleteTransaction(id common.QueuedTxID, password string) (ge
|
|||
m.NotifyOnQueuedTxReturn(queuedTx, ErrInvalidCompleteTxSender)
|
||||
return gethcommon.Hash{}, ErrInvalidCompleteTxSender
|
||||
}
|
||||
|
||||
config, err := m.nodeManager.NodeConfig()
|
||||
if err != nil {
|
||||
log.Warn("could not get a node config", "err", err)
|
||||
return gethcommon.Hash{}, err
|
||||
}
|
||||
|
||||
// Send the transaction finally.
|
||||
var hash gethcommon.Hash
|
||||
|
||||
if config.UpstreamConfig.Enabled {
|
||||
hash, err = m.completeRemoteTransaction(queuedTx, password)
|
||||
} else {
|
||||
hash, err = m.completeLocalTransaction(queuedTx, password)
|
||||
}
|
||||
hash, err := m.completeTransaction(queuedTx, selectedAccount, password)
|
||||
|
||||
// when incorrect sender tries to complete the account,
|
||||
// notify and keep tx in queue (so that correct sender can complete)
|
||||
|
@ -190,79 +183,65 @@ func (m *Manager) CompleteTransaction(id common.QueuedTxID, password string) (ge
|
|||
return hash, err
|
||||
}
|
||||
|
||||
const cancelTimeout = time.Minute
|
||||
|
||||
func (m *Manager) completeLocalTransaction(queuedTx *common.QueuedTx, password string) (gethcommon.Hash, error) {
|
||||
log.Info("complete transaction using local node", "id", queuedTx.ID)
|
||||
|
||||
les, err := m.nodeManager.LightEthereumService()
|
||||
if err != nil {
|
||||
return gethcommon.Hash{}, err
|
||||
}
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), cancelTimeout)
|
||||
defer cancel()
|
||||
|
||||
return les.StatusBackend.SendTransaction(ctx, status.SendTxArgs(queuedTx.Args), password)
|
||||
}
|
||||
|
||||
func (m *Manager) completeRemoteTransaction(queuedTx *common.QueuedTx, password string) (gethcommon.Hash, error) {
|
||||
log.Info("complete transaction using upstream node", "id", queuedTx.ID)
|
||||
|
||||
func (m *Manager) completeTransaction(queuedTx *common.QueuedTx, selectedAccount *common.SelectedExtKey, password string) (gethcommon.Hash, error) {
|
||||
log.Info("complete transaction", "id", queuedTx.ID)
|
||||
var emptyHash gethcommon.Hash
|
||||
|
||||
config, err := m.nodeManager.NodeConfig()
|
||||
if err != nil {
|
||||
return emptyHash, err
|
||||
}
|
||||
|
||||
selectedAcct, err := m.accountManager.SelectedAccount()
|
||||
_, err = m.accountManager.VerifyAccountPassword(config.KeyStoreDir, selectedAccount.Address.String(), password)
|
||||
if err != nil {
|
||||
log.Warn("failed to verify account", "account", selectedAccount.Address.String(), "error", err.Error())
|
||||
return emptyHash, err
|
||||
}
|
||||
|
||||
_, err = m.accountManager.VerifyAccountPassword(config.KeyStoreDir, selectedAcct.Address.String(), password)
|
||||
if err != nil {
|
||||
log.Warn("failed to verify account", "account", selectedAcct.Address.String(), "error", err.Error())
|
||||
return emptyHash, err
|
||||
}
|
||||
|
||||
// We need to request a new transaction nounce from upstream node.
|
||||
ctx, cancel := context.WithTimeout(context.Background(), cancelTimeout)
|
||||
// update transaction with nonce, gas price and gas estimates
|
||||
ctx, cancel := context.WithTimeout(context.Background(), defaultTimeout)
|
||||
defer cancel()
|
||||
|
||||
var txCount hexutil.Uint
|
||||
client := m.nodeManager.RPCClient()
|
||||
err = client.CallContext(ctx, &txCount, "eth_getTransactionCount", queuedTx.Args.From, "pending")
|
||||
m.addrLock.LockAddr(queuedTx.Args.From)
|
||||
defer m.addrLock.UnlockAddr(queuedTx.Args.From)
|
||||
nonce, err := m.ethTxClient.PendingNonceAt(ctx, queuedTx.Args.From)
|
||||
if err != nil {
|
||||
return emptyHash, err
|
||||
}
|
||||
|
||||
args := queuedTx.Args
|
||||
|
||||
gasPrice := (*big.Int)(args.GasPrice)
|
||||
if args.GasPrice == nil {
|
||||
value, gasPriceErr := m.gasPrice()
|
||||
if gasPriceErr != nil {
|
||||
return emptyHash, gasPriceErr
|
||||
ctx, cancel = context.WithTimeout(context.Background(), defaultTimeout)
|
||||
defer cancel()
|
||||
gasPrice, err = m.ethTxClient.SuggestGasPrice(ctx)
|
||||
if err != nil {
|
||||
return emptyHash, err
|
||||
}
|
||||
|
||||
args.GasPrice = value
|
||||
}
|
||||
|
||||
chainID := big.NewInt(int64(config.NetworkID))
|
||||
nonce := uint64(txCount)
|
||||
gasPrice := (*big.Int)(args.GasPrice)
|
||||
data := []byte(args.Data)
|
||||
value := (*big.Int)(args.Value)
|
||||
toAddr := gethcommon.Address{}
|
||||
if args.To != nil {
|
||||
toAddr = *args.To
|
||||
}
|
||||
|
||||
gas, err := m.estimateGas(args)
|
||||
gas := (*big.Int)(args.Gas)
|
||||
if args.Gas == nil {
|
||||
ctx, cancel = context.WithTimeout(context.Background(), defaultTimeout)
|
||||
defer cancel()
|
||||
gas, err = m.ethTxClient.EstimateGas(ctx, ethereum.CallMsg{
|
||||
From: args.From,
|
||||
To: args.To,
|
||||
GasPrice: gasPrice,
|
||||
Value: value,
|
||||
Data: data,
|
||||
})
|
||||
if err != nil {
|
||||
return emptyHash, err
|
||||
}
|
||||
if gas.Cmp(big.NewInt(defaultGas)) == -1 {
|
||||
log.Info("default gas will be used. estimated gas", gas, "is lower than", defaultGas)
|
||||
gas = big.NewInt(defaultGas)
|
||||
}
|
||||
}
|
||||
|
||||
log.Info(
|
||||
"preparing raw transaction",
|
||||
|
@ -272,90 +251,19 @@ func (m *Manager) completeRemoteTransaction(queuedTx *common.QueuedTx, password
|
|||
"gasPrice", gasPrice,
|
||||
"value", value,
|
||||
)
|
||||
|
||||
tx := types.NewTransaction(nonce, toAddr, value, (*big.Int)(gas), gasPrice, data)
|
||||
signedTx, err := types.SignTx(tx, types.NewEIP155Signer(chainID), selectedAcct.AccountKey.PrivateKey)
|
||||
tx := types.NewTransaction(nonce, toAddr, value, gas, gasPrice, data)
|
||||
signedTx, err := types.SignTx(tx, types.NewEIP155Signer(chainID), selectedAccount.AccountKey.PrivateKey)
|
||||
if err != nil {
|
||||
return emptyHash, err
|
||||
}
|
||||
|
||||
txBytes, err := rlp.EncodeToBytes(signedTx)
|
||||
if err != nil {
|
||||
ctx, cancel = context.WithTimeout(context.Background(), defaultTimeout)
|
||||
defer cancel()
|
||||
if err := m.ethTxClient.SendTransaction(ctx, signedTx); err != nil {
|
||||
return emptyHash, err
|
||||
}
|
||||
|
||||
ctx2, cancel2 := context.WithTimeout(context.Background(), cancelTimeout)
|
||||
defer cancel2()
|
||||
|
||||
if err := client.CallContext(ctx2, nil, "eth_sendRawTransaction", gethcommon.ToHex(txBytes)); err != nil {
|
||||
return emptyHash, err
|
||||
}
|
||||
|
||||
return signedTx.Hash(), nil
|
||||
}
|
||||
|
||||
func (m *Manager) estimateGas(args common.SendTxArgs) (*hexutil.Big, error) {
|
||||
if args.Gas != nil {
|
||||
return args.Gas, nil
|
||||
}
|
||||
|
||||
client := m.nodeManager.RPCClient()
|
||||
ctx, cancel := context.WithTimeout(context.Background(), cancelTimeout)
|
||||
defer cancel()
|
||||
|
||||
var gasPrice hexutil.Big
|
||||
if args.GasPrice != nil {
|
||||
gasPrice = *args.GasPrice
|
||||
}
|
||||
|
||||
var value hexutil.Big
|
||||
if args.Value != nil {
|
||||
value = *args.Value
|
||||
}
|
||||
|
||||
params := struct {
|
||||
From gethcommon.Address `json:"from"`
|
||||
To *gethcommon.Address `json:"to"`
|
||||
Gas hexutil.Big `json:"gas"`
|
||||
GasPrice hexutil.Big `json:"gasPrice"`
|
||||
Value hexutil.Big `json:"value"`
|
||||
Data hexutil.Bytes `json:"data"`
|
||||
}{
|
||||
From: args.From,
|
||||
To: args.To,
|
||||
GasPrice: gasPrice,
|
||||
Value: value,
|
||||
Data: []byte(args.Data),
|
||||
}
|
||||
|
||||
var estimatedGas hexutil.Big
|
||||
if err := client.CallContext(
|
||||
ctx,
|
||||
&estimatedGas,
|
||||
"eth_estimateGas",
|
||||
params,
|
||||
); err != nil {
|
||||
log.Warn("failed to estimate gas", "err", err)
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &estimatedGas, nil
|
||||
}
|
||||
|
||||
func (m *Manager) gasPrice() (*hexutil.Big, error) {
|
||||
client := m.nodeManager.RPCClient()
|
||||
ctx, cancel := context.WithTimeout(context.Background(), cancelTimeout)
|
||||
defer cancel()
|
||||
|
||||
var gasPrice hexutil.Big
|
||||
if err := client.CallContext(ctx, &gasPrice, "eth_gasPrice"); err != nil {
|
||||
log.Warn("failed to get gas price", "err", err)
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &gasPrice, nil
|
||||
}
|
||||
|
||||
// CompleteTransactions instructs backend to complete sending of multiple transactions
|
||||
func (m *Manager) CompleteTransactions(ids []common.QueuedTxID, password string) map[common.QueuedTxID]common.RawCompleteTransactionResult {
|
||||
results := make(map[common.QueuedTxID]common.RawCompleteTransactionResult)
|
||||
|
|
|
@ -2,22 +2,26 @@ package txqueue
|
|||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"math/big"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/accounts/keystore"
|
||||
"github.com/stretchr/testify/suite"
|
||||
|
||||
gethcommon "github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/common/hexutil"
|
||||
"github.com/ethereum/go-ethereum/crypto"
|
||||
gethrpc "github.com/ethereum/go-ethereum/rpc"
|
||||
"github.com/golang/mock/gomock"
|
||||
"github.com/stretchr/testify/suite"
|
||||
|
||||
"github.com/status-im/status-go/geth/common"
|
||||
"github.com/status-im/status-go/geth/params"
|
||||
"github.com/status-im/status-go/geth/rpc"
|
||||
"github.com/status-im/status-go/geth/txqueue/fake"
|
||||
. "github.com/status-im/status-go/testing"
|
||||
)
|
||||
|
||||
var errTxAssumedSent = errors.New("assume tx is done")
|
||||
|
||||
func TestTxQueueTestSuite(t *testing.T) {
|
||||
suite.Run(t, new(TxQueueTestSuite))
|
||||
}
|
||||
|
@ -28,34 +32,61 @@ type TxQueueTestSuite struct {
|
|||
nodeManagerMock *common.MockNodeManager
|
||||
accountManagerMockCtrl *gomock.Controller
|
||||
accountManagerMock *common.MockAccountManager
|
||||
server *gethrpc.Server
|
||||
client *gethrpc.Client
|
||||
txServiceMockCtrl *gomock.Controller
|
||||
txServiceMock *fake.MockFakePublicTransactionPoolAPI
|
||||
}
|
||||
|
||||
func (s *TxQueueTestSuite) SetupTest() {
|
||||
s.nodeManagerMockCtrl = gomock.NewController(s.T())
|
||||
s.accountManagerMockCtrl = gomock.NewController(s.T())
|
||||
s.txServiceMockCtrl = gomock.NewController(s.T())
|
||||
|
||||
s.nodeManagerMock = common.NewMockNodeManager(s.nodeManagerMockCtrl)
|
||||
s.accountManagerMock = common.NewMockAccountManager(s.accountManagerMockCtrl)
|
||||
|
||||
s.server, s.txServiceMock = fake.NewTestServer(s.txServiceMockCtrl)
|
||||
s.client = gethrpc.DialInProc(s.server)
|
||||
rpclient, _ := rpc.NewClient(s.client, params.UpstreamRPCConfig{})
|
||||
s.nodeManagerMock.EXPECT().RPCClient().Return(rpclient)
|
||||
}
|
||||
|
||||
func (s *TxQueueTestSuite) TearDownTest() {
|
||||
s.nodeManagerMockCtrl.Finish()
|
||||
s.accountManagerMockCtrl.Finish()
|
||||
s.txServiceMockCtrl.Finish()
|
||||
s.server.Stop()
|
||||
s.client.Close()
|
||||
}
|
||||
|
||||
func (s *TxQueueTestSuite) setupTransactionPoolAPI(account *common.SelectedExtKey, nonce hexutil.Uint64, gas hexutil.Big, txErr error) {
|
||||
s.txServiceMock.EXPECT().GetTransactionCount(gomock.Any(), account.Address, gethrpc.PendingBlockNumber).Return(&nonce, nil)
|
||||
s.txServiceMock.EXPECT().GasPrice(gomock.Any()).Return(big.NewInt(10), nil)
|
||||
s.txServiceMock.EXPECT().EstimateGas(gomock.Any(), gomock.Any()).Return(&gas, nil)
|
||||
s.txServiceMock.EXPECT().SendRawTransaction(gomock.Any(), gomock.Any()).Return(gethcommon.Hash{}, txErr)
|
||||
}
|
||||
|
||||
func (s *TxQueueTestSuite) setupStatusBackend(account *common.SelectedExtKey, password string, passwordErr error) {
|
||||
nodeConfig, nodeErr := params.NewNodeConfig("/tmp", params.RopstenNetworkID, true)
|
||||
s.nodeManagerMock.EXPECT().NodeConfig().Return(nodeConfig, nodeErr)
|
||||
s.accountManagerMock.EXPECT().SelectedAccount().Return(account, nil)
|
||||
s.accountManagerMock.EXPECT().VerifyAccountPassword(nodeConfig.KeyStoreDir, account.Address.String(), password).Return(
|
||||
nil, passwordErr)
|
||||
}
|
||||
|
||||
func (s *TxQueueTestSuite) TestCompleteTransaction() {
|
||||
s.accountManagerMock.EXPECT().SelectedAccount().Return(&common.SelectedExtKey{
|
||||
password := TestConfig.Account1.Password
|
||||
key, _ := crypto.GenerateKey()
|
||||
account := &common.SelectedExtKey{
|
||||
Address: common.FromAddress(TestConfig.Account1.Address),
|
||||
}, nil)
|
||||
AccountKey: &keystore.Key{PrivateKey: key},
|
||||
}
|
||||
s.setupStatusBackend(account, password, nil)
|
||||
|
||||
s.nodeManagerMock.EXPECT().NodeConfig().Return(
|
||||
params.NewNodeConfig("/tmp", params.RopstenNetworkID, true),
|
||||
)
|
||||
|
||||
// TODO(adam): StatusBackend as an interface would allow a better solution.
|
||||
// As we want to avoid network connection, we mock LES with a known error
|
||||
// and treat as success.
|
||||
s.nodeManagerMock.EXPECT().LightEthereumService().Return(nil, errTxAssumedSent)
|
||||
nonce := hexutil.Uint64(10)
|
||||
gas := hexutil.Big(*big.NewInt(defaultGas + 1))
|
||||
s.setupTransactionPoolAPI(account, nonce, gas, nil)
|
||||
|
||||
txQueueManager := NewManager(s.nodeManagerMock, s.accountManagerMock)
|
||||
|
||||
|
@ -74,38 +105,41 @@ func (s *TxQueueTestSuite) TestCompleteTransaction() {
|
|||
|
||||
txQueueManager.SetTransactionReturnHandler(func(queuedTx *common.QueuedTx, err error) {
|
||||
s.Equal(tx.ID, queuedTx.ID)
|
||||
s.Equal(errTxAssumedSent, err)
|
||||
s.NoError(err)
|
||||
})
|
||||
|
||||
err := txQueueManager.QueueTransaction(tx)
|
||||
s.NoError(err)
|
||||
|
||||
w := make(chan struct{})
|
||||
go func() {
|
||||
_, errCompleteTransaction := txQueueManager.CompleteTransaction(tx.ID, TestConfig.Account1.Password)
|
||||
s.Equal(errTxAssumedSent, errCompleteTransaction)
|
||||
hash, err := txQueueManager.CompleteTransaction(tx.ID, password)
|
||||
s.NoError(err)
|
||||
s.Equal(tx.Hash, hash)
|
||||
close(w)
|
||||
}()
|
||||
|
||||
err = txQueueManager.WaitForTransaction(tx)
|
||||
s.Equal(errTxAssumedSent, err)
|
||||
s.NoError(err)
|
||||
// Check that error is assigned to the transaction.
|
||||
s.Equal(errTxAssumedSent, tx.Err)
|
||||
s.NoError(tx.Err)
|
||||
// Transaction should be already removed from the queue.
|
||||
s.False(txQueueManager.TransactionQueue().Has(tx.ID))
|
||||
s.NoError(WaitClosed(w, time.Second))
|
||||
}
|
||||
|
||||
func (s *TxQueueTestSuite) TestCompleteTransactionMultipleTimes() {
|
||||
s.accountManagerMock.EXPECT().SelectedAccount().Return(&common.SelectedExtKey{
|
||||
password := TestConfig.Account1.Password
|
||||
key, _ := crypto.GenerateKey()
|
||||
account := &common.SelectedExtKey{
|
||||
Address: common.FromAddress(TestConfig.Account1.Address),
|
||||
}, nil)
|
||||
AccountKey: &keystore.Key{PrivateKey: key},
|
||||
}
|
||||
s.setupStatusBackend(account, password, nil)
|
||||
|
||||
s.nodeManagerMock.EXPECT().NodeConfig().Return(
|
||||
params.NewNodeConfig("/tmp", params.RopstenNetworkID, true),
|
||||
)
|
||||
|
||||
// TODO(adam): StatusBackend as an interface would allow a better solution.
|
||||
// As we want to avoid network connection, we mock LES with a known error
|
||||
// and treat as success.
|
||||
s.nodeManagerMock.EXPECT().LightEthereumService().Return(nil, errTxAssumedSent)
|
||||
nonce := hexutil.Uint64(10)
|
||||
gas := hexutil.Big(*big.NewInt(defaultGas + 1))
|
||||
s.setupTransactionPoolAPI(account, nonce, gas, nil)
|
||||
|
||||
txQueueManager := NewManager(s.nodeManagerMock, s.accountManagerMock)
|
||||
|
||||
|
@ -124,36 +158,47 @@ func (s *TxQueueTestSuite) TestCompleteTransactionMultipleTimes() {
|
|||
|
||||
txQueueManager.SetTransactionReturnHandler(func(queuedTx *common.QueuedTx, err error) {
|
||||
s.Equal(tx.ID, queuedTx.ID)
|
||||
s.Equal(errTxAssumedSent, err)
|
||||
s.NoError(err)
|
||||
})
|
||||
|
||||
err := txQueueManager.QueueTransaction(tx)
|
||||
s.NoError(err)
|
||||
|
||||
var wg sync.WaitGroup
|
||||
var mu sync.Mutex
|
||||
completeTxErrors := make(map[error]int)
|
||||
for i := 0; i < 3; i++ {
|
||||
var (
|
||||
wg sync.WaitGroup
|
||||
mu sync.Mutex
|
||||
completedTx int
|
||||
inprogressTx int
|
||||
txCount = 3
|
||||
)
|
||||
for i := 0; i < txCount; i++ {
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
_, errCompleteTransaction := txQueueManager.CompleteTransaction(tx.ID, TestConfig.Account1.Password)
|
||||
_, err := txQueueManager.CompleteTransaction(tx.ID, password)
|
||||
mu.Lock()
|
||||
completeTxErrors[errCompleteTransaction]++
|
||||
if err == nil {
|
||||
completedTx++
|
||||
} else if err == ErrQueuedTxInProgress {
|
||||
inprogressTx++
|
||||
} else {
|
||||
s.Fail("tx failed with unexpected error: ", err.Error())
|
||||
}
|
||||
mu.Unlock()
|
||||
}()
|
||||
}
|
||||
|
||||
err = txQueueManager.WaitForTransaction(tx)
|
||||
s.Equal(errTxAssumedSent, err)
|
||||
s.NoError(err)
|
||||
// Check that error is assigned to the transaction.
|
||||
s.Equal(errTxAssumedSent, tx.Err)
|
||||
s.NoError(tx.Err)
|
||||
// Transaction should be already removed from the queue.
|
||||
s.False(txQueueManager.TransactionQueue().Has(tx.ID))
|
||||
|
||||
// Wait for all CompleteTransaction calls.
|
||||
wg.Wait()
|
||||
s.Equal(completeTxErrors[errTxAssumedSent], 1)
|
||||
s.Equal(1, completedTx, "only 1 tx expected to be completed")
|
||||
s.Equal(txCount-1, inprogressTx, "txs expected to be reported as inprogress")
|
||||
}
|
||||
|
||||
func (s *TxQueueTestSuite) TestAccountMismatch() {
|
||||
|
@ -196,16 +241,13 @@ func (s *TxQueueTestSuite) TestAccountMismatch() {
|
|||
}
|
||||
|
||||
func (s *TxQueueTestSuite) TestInvalidPassword() {
|
||||
s.accountManagerMock.EXPECT().SelectedAccount().Return(&common.SelectedExtKey{
|
||||
password := "invalid-password"
|
||||
key, _ := crypto.GenerateKey()
|
||||
account := &common.SelectedExtKey{
|
||||
Address: common.FromAddress(TestConfig.Account1.Address),
|
||||
}, nil)
|
||||
|
||||
s.nodeManagerMock.EXPECT().NodeConfig().Return(
|
||||
params.NewNodeConfig("/tmp", params.RopstenNetworkID, true),
|
||||
)
|
||||
|
||||
// Set ErrDecrypt error response as expected with a wrong password.
|
||||
s.nodeManagerMock.EXPECT().LightEthereumService().Return(nil, keystore.ErrDecrypt)
|
||||
AccountKey: &keystore.Key{PrivateKey: key},
|
||||
}
|
||||
s.setupStatusBackend(account, password, keystore.ErrDecrypt)
|
||||
|
||||
txQueueManager := NewManager(s.nodeManagerMock, s.accountManagerMock)
|
||||
|
||||
|
@ -233,8 +275,8 @@ func (s *TxQueueTestSuite) TestInvalidPassword() {
|
|||
err := txQueueManager.QueueTransaction(tx)
|
||||
s.NoError(err)
|
||||
|
||||
_, err = txQueueManager.CompleteTransaction(tx.ID, "invalid-password")
|
||||
s.Equal(err, keystore.ErrDecrypt)
|
||||
_, err = txQueueManager.CompleteTransaction(tx.ID, password)
|
||||
s.Equal(err.Error(), keystore.ErrDecrypt.Error())
|
||||
|
||||
// Transaction should stay in the queue as mismatched accounts
|
||||
// is a recoverable error.
|
||||
|
@ -265,9 +307,11 @@ func (s *TxQueueTestSuite) TestDiscardTransaction() {
|
|||
err := txQueueManager.QueueTransaction(tx)
|
||||
s.NoError(err)
|
||||
|
||||
w := make(chan struct{})
|
||||
go func() {
|
||||
discardErr := txQueueManager.DiscardTransaction(tx.ID)
|
||||
s.NoError(discardErr)
|
||||
err := txQueueManager.DiscardTransaction(tx.ID)
|
||||
s.NoError(err)
|
||||
close(w)
|
||||
}()
|
||||
|
||||
err = txQueueManager.WaitForTransaction(tx)
|
||||
|
@ -276,4 +320,5 @@ func (s *TxQueueTestSuite) TestDiscardTransaction() {
|
|||
s.Equal(ErrQueuedTxDiscarded, tx.Err)
|
||||
// Transaction should be already removed from the queue.
|
||||
s.False(txQueueManager.TransactionQueue().Has(tx.ID))
|
||||
s.NoError(WaitClosed(w, time.Second))
|
||||
}
|
||||
|
|
|
@ -23,6 +23,9 @@ var (
|
|||
// ErrNoRemoteURL is returned when network id has no associated url.
|
||||
ErrNoRemoteURL = errors.New("network id requires a remote URL")
|
||||
|
||||
// ErrTimeout is returned when test times out
|
||||
ErrTimeout = errors.New("timeout")
|
||||
|
||||
// TestConfig defines the default config usable at package-level.
|
||||
TestConfig *common.TestConfig
|
||||
|
||||
|
@ -206,3 +209,16 @@ func GetAccount2PKFile() string {
|
|||
return "test-account2.pk"
|
||||
}
|
||||
}
|
||||
|
||||
// WaitClosed used to wait on a channel in tests
|
||||
func WaitClosed(c chan struct{}, d time.Duration) error {
|
||||
timer := time.NewTimer(d)
|
||||
defer timer.Stop()
|
||||
select {
|
||||
case <-c:
|
||||
return nil
|
||||
case <-timer.C:
|
||||
return ErrTimeout
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -976,6 +976,14 @@ func (s *PublicTransactionPoolAPI) GetRawTransactionByBlockHashAndIndex(ctx cont
|
|||
|
||||
// GetTransactionCount returns the number of transactions the given address has sent for the given block number
|
||||
func (s *PublicTransactionPoolAPI) GetTransactionCount(ctx context.Context, address common.Address, blockNr rpc.BlockNumber) (*hexutil.Uint64, error) {
|
||||
// go-ethereum issue https://github.com/ethereum/go-ethereum/issues/2880
|
||||
if blockNr == rpc.PendingBlockNumber {
|
||||
nonce, err := s.b.GetPoolNonce(ctx, address)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return (*hexutil.Uint64)(&nonce), nil
|
||||
}
|
||||
state, _, err := s.b.StateAndHeaderByNumber(ctx, blockNr)
|
||||
if state == nil || err != nil {
|
||||
return nil, err
|
||||
|
@ -1161,7 +1169,6 @@ func (s *PublicTransactionPoolAPI) SendTransactionWithPassphrase(ctx context.Con
|
|||
}
|
||||
// Assemble the transaction and sign with the wallet
|
||||
tx := args.toTransaction()
|
||||
|
||||
var chainID *big.Int
|
||||
if config := s.b.ChainConfig(); config.IsEIP155(s.b.CurrentBlock().Number()) {
|
||||
chainID = config.ChainId
|
||||
|
|
Loading…
Reference in New Issue