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:
Dmitry Shulyak 2018-01-18 18:55:17 +02:00 committed by Ivan Daniluk
parent ea55ac0744
commit 0771e7d1b7
13 changed files with 461 additions and 200 deletions

View File

@ -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"

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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)
}

39
geth/txqueue/addrlock.go Normal file
View File

@ -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()
}

View File

@ -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
}

90
geth/txqueue/fake/mock.go Normal file
View File

@ -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)
}

View File

@ -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)
}

View File

@ -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,78 +183,64 @@ 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)
if err != nil {
return emptyHash, err
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(
@ -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)

View File

@ -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{
Address: common.FromAddress(TestConfig.Account1.Address),
}, nil)
password := TestConfig.Account1.Password
key, _ := crypto.GenerateKey()
account := &common.SelectedExtKey{
Address: common.FromAddress(TestConfig.Account1.Address),
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{
Address: common.FromAddress(TestConfig.Account1.Address),
}, nil)
password := TestConfig.Account1.Password
key, _ := crypto.GenerateKey()
account := &common.SelectedExtKey{
Address: common.FromAddress(TestConfig.Account1.Address),
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{
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)
password := "invalid-password"
key, _ := crypto.GenerateKey()
account := &common.SelectedExtKey{
Address: common.FromAddress(TestConfig.Account1.Address),
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))
}

View File

@ -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
}
}

View File

@ -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