From 1a1610664806da907c4d0097626242baccae2568 Mon Sep 17 00:00:00 2001
From: Silas Davis <silas@monax.io>
Date: Sat, 7 Apr 2018 11:22:35 +0100
Subject: [PATCH] Add sufficient tests for v0 and fix issues with SendAndHold.
 Implement access to mempool state and refactor transactor to use this to set
 sequence number optimistically. Swap order of check cache reset and committer
 commit to avoid concurrency issues with mempool access during commits. Make
 transactor accept InputAccount for all signing methods and add InputAddress
 parameter to V0 methods allowing address based signing in addition to private
 key based signing with minimal changes.

Signed-off-by: Silas Davis <silas@monax.io>
---
 Gopkg.lock                        |  14 +-
 Gopkg.toml                        |  12 -
 Makefile                          |   5 +-
 account/account.go                |   4 +-
 account/state/state_cache.go      |  25 +-
 account/state/state_cache_test.go |  18 ++
 consensus/tendermint/abci/app.go  |  15 +-
 core/kernel.go                    |   5 +-
 execution/accounts.go             |  28 ++-
 execution/events/events.go        |   4 +-
 execution/evm/events/events.go    |   1 -
 execution/evm/snative_test.go     |   2 +-
 execution/evm/vm.go               |  10 +-
 execution/execution.go            |  10 +-
 execution/transactor.go           | 103 +++-----
 execution/transactor.go2          | 377 ------------------------------
 execution/transactor_test.go      |   2 +-
 rpc/service.go                    |  82 ++++---
 rpc/tm/methods.go                 |   4 +-
 rpc/v0/client.go                  |  46 +++-
 rpc/v0/integration/v0_test.go     | 175 +++++++++++---
 rpc/v0/json_service.go            |   3 +-
 rpc/v0/methods.go                 |  47 +++-
 rpc/v0/params.go                  |  29 +--
 rpc/v0/server/server.go           |   2 +-
 25 files changed, 400 insertions(+), 623 deletions(-)
 delete mode 100644 execution/transactor.go2

diff --git a/Gopkg.lock b/Gopkg.lock
index ce5d6fd3..99eff5ee 100644
--- a/Gopkg.lock
+++ b/Gopkg.lock
@@ -187,12 +187,6 @@
   revision = "792786c7400a136282c1664665ae0a8db921c6c2"
   version = "v1.0.0"
 
-[[projects]]
-  name = "github.com/powerman/rpc-codec"
-  packages = ["jsonrpc2"]
-  revision = "3e1ab3b635b7b0d5f771028cd45aa9a827fd9f31"
-  version = "v1.0.0"
-
 [[projects]]
   branch = "master"
   name = "github.com/rcrowley/go-metrics"
@@ -360,12 +354,6 @@
   revision = "9831f2c3ac1068a78f50999a30db84270f647af6"
   version = "v1.1"
 
-[[projects]]
-  name = "github.com/ybbus/jsonrpc"
-  packages = ["."]
-  revision = "dd866631e904a5df2067d934985c5def68f391ac"
-  version = "v2.1.2"
-
 [[projects]]
   branch = "master"
   name = "golang.org/x/crypto"
@@ -486,6 +474,6 @@
 [solve-meta]
   analyzer-name = "dep"
   analyzer-version = 1
-  inputs-digest = "658b10b4a49260c9ee26dcf77b0dcbe9e39d5c1604e234c69e9db99c01ba2d0d"
+  inputs-digest = "5db3ac43679cafd5f68104e93a248b62266d313aef2d8ba3bd7f3986a5a99834"
   solver-name = "gps-cdcl"
   solver-version = 1
diff --git a/Gopkg.toml b/Gopkg.toml
index e07ee9d5..0ac9a88a 100644
--- a/Gopkg.toml
+++ b/Gopkg.toml
@@ -71,10 +71,6 @@
   name = "github.com/tendermint/iavl"
   version = "~0.5.0"
 
-[[constraint]]
-  name = "github.com/tendermint/merkleeyes"
-  version = "~0.2.4"
-
 [[constraint]]
   name = "github.com/tendermint/tendermint"
   version = "~0.15.0"
@@ -98,11 +94,3 @@
 [[constraint]]
   name = "gopkg.in/tylerb/graceful.v1"
   version = "1.2.15"
-
-[[constraint]]
-  name = "github.com/powerman/rpc-codec"
-  version = "1.0.0"
-
-[[constraint]]
-  name = "github.com/ybbus/jsonrpc"
-  version = "2.1.2"
diff --git a/Makefile b/Makefile
index 77bcfd8d..b8992062 100644
--- a/Makefile
+++ b/Makefile
@@ -147,8 +147,9 @@ test: check
 .PHONY: test_integration
 test_integration:
 	@go get github.com/monax/bosmarmot/keys/cmd/monax-keys
-	@go test ./keys/integration -tags integration
-	@go test ./rpc/tm/integration -tags integration
+	@go test -tags integration ./keys/integration
+	@go test -tags integration ./rpc/v0/integration
+	@go test -tags integration ./rpc/tm/integration
 
 # Run integration test from bosmarmot (separated from other integration tests so we can
 # make exception when this test fails when we make a breaking change in Burrow)
diff --git a/account/account.go b/account/account.go
index ab6e9874..725eda71 100644
--- a/account/account.go
+++ b/account/account.go
@@ -156,8 +156,8 @@ func (acc *ConcreteAccount) String() string {
 		return "Account{nil}"
 	}
 
-	return fmt.Sprintf("Account{Address: %s; PublicKey: %v Balance: %v; CodeBytes: %v; StorageRoot: 0x%X; Permissions: %s}",
-		acc.Address, acc.PublicKey, acc.Balance, len(acc.Code), acc.StorageRoot, acc.Permissions)
+	return fmt.Sprintf("Account{Address: %s; Sequence: %v; PublicKey: %v Balance: %v; CodeBytes: %v; StorageRoot: 0x%X; Permissions: %s}",
+		acc.Address, acc.Sequence, acc.PublicKey, acc.Balance, len(acc.Code), acc.StorageRoot, acc.Permissions)
 }
 
 // ConcreteAccount
diff --git a/account/state/state_cache.go b/account/state/state_cache.go
index 04107211..76661f90 100644
--- a/account/state/state_cache.go
+++ b/account/state/state_cache.go
@@ -32,6 +32,8 @@ type Cache interface {
 
 type stateCache struct {
 	sync.RWMutex
+	name string
+
 	backend  Reader
 	accounts map[acm.Address]*accountInfo
 }
@@ -44,13 +46,25 @@ type accountInfo struct {
 	updated bool
 }
 
+type CacheOption func(*stateCache)
+
 // Returns a Cache that wraps an underlying Reader to use on a cache miss, can write to an output Writer
 // via Sync. Goroutine safe for concurrent access.
-func NewCache(backend Reader) Cache {
-	return &stateCache{
+func NewCache(backend Reader, options ...CacheOption) Cache {
+	cache := &stateCache{
 		backend:  backend,
 		accounts: make(map[acm.Address]*accountInfo),
 	}
+	for _, option := range options {
+		option(cache)
+	}
+	return cache
+}
+
+func Name(name string) CacheOption {
+	return func(cache *stateCache) {
+		cache.name = name
+	}
 }
 
 func (cache *stateCache) GetAccount(address acm.Address) (acm.Account, error) {
@@ -233,6 +247,13 @@ func (cache *stateCache) Flush(state IterableWriter) error {
 	return nil
 }
 
+func (cache *stateCache) String() string {
+	if cache.name == "" {
+		return fmt.Sprintf("StateCache{Length: %v}", len(cache.accounts))
+	}
+	return fmt.Sprintf("StateCache{Name: %v; Length: %v}", cache.name, len(cache.accounts))
+}
+
 // Get the cache accountInfo item creating it if necessary
 func (cache *stateCache) get(address acm.Address) (*accountInfo, error) {
 	cache.RLock()
diff --git a/account/state/state_cache_test.go b/account/state/state_cache_test.go
index 2c414708..af02081f 100644
--- a/account/state/state_cache_test.go
+++ b/account/state/state_cache_test.go
@@ -3,6 +3,8 @@ package state
 import (
 	"testing"
 
+	"fmt"
+
 	acm "github.com/hyperledger/burrow/account"
 	"github.com/hyperledger/burrow/binary"
 	"github.com/hyperledger/burrow/execution/evm/asm"
@@ -31,6 +33,22 @@ func TestStateCache_GetAccount(t *testing.T) {
 	assert.Equal(t, acm.AsConcreteAccount(acc), acm.AsConcreteAccount(accOut))
 }
 
+func TestStateCache_Miss(t *testing.T) {
+	readBackend := testAccounts()
+	cache := NewCache(readBackend)
+
+	acc1Address := addressOf("acc1")
+	acc1, err := cache.GetAccount(acc1Address)
+	require.NoError(t, err)
+	fmt.Println(acc1)
+
+	acc1Exp := readBackend.Accounts[acc1Address]
+	assert.Equal(t, acc1Exp, acc1)
+	acc8, err := cache.GetAccount(addressOf("acc8"))
+	require.NoError(t, err)
+	assert.Nil(t, acc8)
+}
+
 func TestStateCache_UpdateAccount(t *testing.T) {
 	// Build backend states for read and write
 	backend := NewCache(NewMemoryState())
diff --git a/consensus/tendermint/abci/app.go b/consensus/tendermint/abci/app.go
index 1bc0700c..d23bde68 100644
--- a/consensus/tendermint/abci/app.go
+++ b/consensus/tendermint/abci/app.go
@@ -92,7 +92,7 @@ func (app *abciApp) CheckTx(txBytes []byte) abci_types.ResponseCheckTx {
 			"creates_contract", receipt.CreatesContract)
 		return abci_types.ResponseCheckTx{
 			Code: codes.EncodingErrorCode,
-			Log:  fmt.Sprintf("Could not execute transaction: %s, error: %v", tx, err),
+			Log:  fmt.Sprintf("CheckTx could not execute transaction: %s, error: %v", tx, err),
 		}
 	}
 
@@ -142,7 +142,7 @@ func (app *abciApp) DeliverTx(txBytes []byte) abci_types.ResponseDeliverTx {
 			"creates_contract", receipt.CreatesContract)
 		return abci_types.ResponseDeliverTx{
 			Code: codes.TxExecutionErrorCode,
-			Log:  fmt.Sprintf("Could not execute transaction: %s, error: %s", tx, err),
+			Log:  fmt.Sprintf("DeliverTx could not execute transaction: %s, error: %s", tx, err),
 		}
 	}
 
@@ -177,18 +177,21 @@ func (app *abciApp) Commit() abci_types.ResponseCommit {
 		"last_block_time", tip.LastBlockTime(),
 		"last_block_hash", tip.LastBlockHash())
 
-	err := app.checker.Reset()
+	// Commit state before resetting check cache so that the emptied cache servicing some RPC requests will fall through
+	// to committed state
+	appHash, err := app.committer.Commit()
 	if err != nil {
 		return abci_types.ResponseCommit{
 			Code: codes.CommitErrorCode,
-			Log:  fmt.Sprintf("Could not reset check cache during commit: %s", err),
+			Log:  fmt.Sprintf("Could not commit transactions in block to execution state: %s", err),
 		}
 	}
-	appHash, err := app.committer.Commit()
+
+	err = app.checker.Reset()
 	if err != nil {
 		return abci_types.ResponseCommit{
 			Code: codes.CommitErrorCode,
-			Log:  fmt.Sprintf("Could not commit transactions in block to execution state: %s", err),
+			Log:  fmt.Sprintf("Could not reset check cache during commit: %s", err),
 		}
 	}
 
diff --git a/core/kernel.go b/core/kernel.go
index a2f6dc8e..70fc3a71 100644
--- a/core/kernel.go
+++ b/core/kernel.go
@@ -101,10 +101,11 @@ func NewKernel(ctx context.Context, keyClient keys.KeyClient, privValidator tm_t
 		return nil, err
 	}
 	txCodec := txs.NewGoWireCodec()
-	transactor := execution.NewTransactor(blockchain, state, emitter, tendermint.BroadcastTxAsyncFunc(tmNode, txCodec),
+	transactor := execution.NewTransactor(blockchain, emitter, tendermint.BroadcastTxAsyncFunc(tmNode, txCodec),
 		logger)
 
-	service := rpc.NewService(ctx, state, checker, state, emitter, blockchain, keyClient, transactor,
+	nameReg := state
+	service := rpc.NewService(ctx, state, nameReg, checker, emitter, blockchain, keyClient, transactor,
 		query.NewNodeView(tmNode, txCodec), logger)
 
 	launchers := []process.Launcher{
diff --git a/execution/accounts.go b/execution/accounts.go
index ec9953d4..05fe03e2 100644
--- a/execution/accounts.go
+++ b/execution/accounts.go
@@ -9,7 +9,7 @@ import (
 )
 
 type Accounts struct {
-	state.Iterable
+	state.Reader
 	keyClient keys.KeyClient
 }
 
@@ -18,25 +18,30 @@ type SigningAccount struct {
 	acm.Signer
 }
 
-func NewAccounts(iterable state.Iterable, keyClient keys.KeyClient) *Accounts {
+func NewAccounts(reader state.Reader, keyClient keys.KeyClient) *Accounts {
 	return &Accounts{
-		Iterable:  iterable,
+		Reader:    reader,
 		keyClient: keyClient,
 	}
 }
 
 func (accs *Accounts) SigningAccount(address acm.Address) (*SigningAccount, error) {
 	signer := keys.Signer(accs.keyClient, address)
-	account, err := accs.GetAccount(address)
+	account, err := state.GetMutableAccount(accs.Reader, address)
 	if err != nil {
 		return nil, err
 	}
 	// If the account is unknown to us return a zeroed account
-	if account != nil {
+	if account == nil {
 		account = acm.ConcreteAccount{
 			Address: address,
-		}.Account()
+		}.MutableAccount()
 	}
+	pubKey, err := accs.keyClient.PublicKey(address)
+	if err != nil {
+		return nil, err
+	}
+	account.SetPublicKey(pubKey)
 	return &SigningAccount{
 		Account: account,
 		Signer:  signer,
@@ -51,17 +56,18 @@ func (accs *Accounts) SigningAccountFromPrivateKey(privateKeyBytes []byte) (*Sig
 	if err != nil {
 		return nil, err
 	}
-	account, err := accs.GetAccount(privateAccount.Address())
+	account, err := state.GetMutableAccount(accs, privateAccount.Address())
 	if err != nil {
 		return nil, err
 	}
-	// If the account is unknown to us return a zeroed account
-	if account != nil {
+	// If the account is unknown to us return zeroed account for the address derived from the private key
+	if account == nil {
 		account = acm.ConcreteAccount{
 			Address: privateAccount.Address(),
-			PublicKey: privateAccount.PublicKey(),
-		}.Account()
+		}.MutableAccount()
 	}
+	// Set the public key in case it was not known previously (needed for signing with an unseen account)
+	account.SetPublicKey(privateAccount.PublicKey())
 	return &SigningAccount{
 		Account: account,
 		Signer:  privateAccount,
diff --git a/execution/events/events.go b/execution/events/events.go
index 5315a741..6f8c394f 100644
--- a/execution/events/events.go
+++ b/execution/events/events.go
@@ -29,11 +29,11 @@ type EventDataTx struct {
 
 // For re-use
 var sendTxQuery = event.NewQueryBuilder().
-	AndEquals(event.MessageTypeKey, reflect.TypeOf(EventDataTx{}).String()).
+	AndEquals(event.MessageTypeKey, reflect.TypeOf(&EventDataTx{}).String()).
 	AndEquals(event.TxTypeKey, reflect.TypeOf(&txs.SendTx{}).String())
 
 var callTxQuery = event.NewQueryBuilder().
-	AndEquals(event.MessageTypeKey, reflect.TypeOf(EventDataTx{}).String()).
+	AndEquals(event.MessageTypeKey, reflect.TypeOf(&EventDataTx{}).String()).
 	AndEquals(event.TxTypeKey, reflect.TypeOf(&txs.CallTx{}).String())
 
 type eventDataTx struct {
diff --git a/execution/evm/events/events.go b/execution/evm/events/events.go
index 719e3354..a9308bb5 100644
--- a/execution/evm/events/events.go
+++ b/execution/evm/events/events.go
@@ -99,7 +99,6 @@ func SubscribeLogEvent(ctx context.Context, subscribable event.Subscribable, sub
 }
 
 func PublishAccountCall(publisher event.Publisher, address acm.Address, eventDataCall *EventDataCall) error {
-	fmt.Printf("%v: %v\n", eventDataCall.StackDepth, eventDataCall.Return[31])
 	return event.PublishWithEventID(publisher, EventStringAccountCall(address), eventDataCall,
 		map[string]interface{}{
 			"address":           address,
diff --git a/execution/evm/snative_test.go b/execution/evm/snative_test.go
index 39c31e72..7c25aaf7 100644
--- a/execution/evm/snative_test.go
+++ b/execution/evm/snative_test.go
@@ -123,7 +123,7 @@ func funcIDFromHex(t *testing.T, hexString string) abi.FunctionSelector {
 		t.Fatalf("FunctionSelector must be 4 bytes but '%s' is %v bytes", hexString,
 			len(bs))
 	}
-	return firstFourBytes(bs)
+	return abi.FirstFourBytes(bs)
 }
 
 func permFlagToWord256(permFlag ptypes.PermFlag) Word256 {
diff --git a/execution/evm/vm.go b/execution/evm/vm.go
index ea29e487..5597fe03 100644
--- a/execution/evm/vm.go
+++ b/execution/evm/vm.go
@@ -156,8 +156,6 @@ func HasPermission(stateWriter state.Writer, acc acm.Account, perm ptypes.PermFl
 }
 
 func (vm *VM) fireCallEvent(exception *string, output *[]byte, callerAddress, calleeAddress acm.Address, input []byte, value uint64, gas *uint64) {
-	ret := make([]byte, len(*output))
-	copy(ret, *output)
 	// fire the post call event (including exception if applicable)
 	if vm.publisher != nil {
 		events.PublishAccountCall(vm.publisher, calleeAddress, &events.EventDataCall{
@@ -168,11 +166,11 @@ func (vm *VM) fireCallEvent(exception *string, output *[]byte, callerAddress, ca
 				Value:  value,
 				Gas:    *gas,
 			},
-			Origin:    vm.origin,
-			TxHash:    vm.txHash,
+			Origin:     vm.origin,
+			TxHash:     vm.txHash,
 			StackDepth: vm.stackDepth,
-			Return:    ret,
-			Exception: *exception,
+			Return:     *output,
+			Exception:  *exception,
 		})
 	}
 }
diff --git a/execution/execution.go b/execution/execution.go
index 58277636..de404f23 100644
--- a/execution/execution.go
+++ b/execution/execution.go
@@ -53,7 +53,7 @@ type BatchCommitter interface {
 }
 
 type executor struct {
-	sync.Mutex
+	sync.RWMutex
 	chainID      string
 	tip          bcm.Tip
 	runCall      bool
@@ -115,11 +115,15 @@ func newExecutor(runCall bool,
 
 // Accounts
 func (exe *executor) GetAccount(address acm.Address) (acm.Account, error) {
+	exe.RLock()
+	defer exe.RUnlock()
 	return exe.stateCache.GetAccount(address)
 }
 
 // Storage
 func (exe *executor) GetStorage(address acm.Address, key binary.Word256) (binary.Word256, error) {
+	exe.RLock()
+	defer exe.RUnlock()
 	return exe.stateCache.GetStorage(address, key)
 }
 
@@ -151,6 +155,8 @@ func (exe *executor) Commit() (hash []byte, err error) {
 }
 
 func (exe *executor) Reset() error {
+	exe.Lock()
+	defer exe.Unlock()
 	exe.stateCache.Reset(exe.state)
 	exe.nameRegCache.Reset(exe.state)
 	return nil
@@ -224,7 +230,6 @@ func (exe *executor) Execute(tx txs.Tx) (err error) {
 			exe.stateCache.UpdateAccount(acc)
 		}
 
-		// if the exe.eventCache is nil, nothing will happen
 		if exe.eventCache != nil {
 			for _, i := range tx.Inputs {
 				events.PublishAccountInput(exe.eventCache, i.Address, txHash, tx, nil, "")
@@ -251,6 +256,7 @@ func (exe *executor) Execute(tx txs.Tx) (err error) {
 			return txs.ErrTxInvalidAddress
 		}
 
+		// Calling a nil destination is defined as requesting contract creation
 		createContract := tx.Address == nil
 		if createContract {
 			if !hasCreateContractPermission(exe.stateCache, inAcc, logger) {
diff --git a/execution/transactor.go b/execution/transactor.go
index 0acb2fd3..d2cfa6f4 100644
--- a/execution/transactor.go
+++ b/execution/transactor.go
@@ -18,7 +18,6 @@ import (
 	"context"
 	"fmt"
 	"runtime/debug"
-	"sync"
 	"time"
 
 	acm "github.com/hyperledger/burrow/account"
@@ -51,21 +50,18 @@ type SequencedAddressableSigner interface {
 
 // Transactor is the controller/middleware for the v0 RPC
 type Transactor struct {
-	sync.Mutex
 	tip              blockchain.Tip
-	state            state.Iterable
 	eventEmitter     event.Emitter
 	broadcastTxAsync func(tx txs.Tx, callback func(res *abci_types.Response)) error
 	logger           *logging.Logger
 }
 
-func NewTransactor(tip blockchain.Tip, state state.Iterable, eventEmitter event.Emitter,
+func NewTransactor(tip blockchain.Tip, eventEmitter event.Emitter,
 	broadcastTxAsync func(tx txs.Tx, callback func(res *abci_types.Response)) error,
 	logger *logging.Logger) *Transactor {
 
 	return &Transactor{
 		tip:              tip,
-		state:            state,
 		eventEmitter:     eventEmitter,
 		broadcastTxAsync: broadcastTxAsync,
 		logger:           logger.With(structure.ComponentKey, "Transactor"),
@@ -74,14 +70,16 @@ func NewTransactor(tip blockchain.Tip, state state.Iterable, eventEmitter event.
 
 // Run a contract's code on an isolated and unpersisted state
 // Cannot be used to create new contracts
-func (trans *Transactor) Call(fromAddress, toAddress acm.Address, data []byte) (call *Call, err error) {
+func (trans *Transactor) Call(reader state.Reader, fromAddress, toAddress acm.Address,
+	data []byte) (call *Call, err error) {
+
 	if evm.RegisteredNativeContract(toAddress.Word256()) {
 		return nil, fmt.Errorf("attempt to call native contract at address "+
 			"%X, but native contracts can not be called directly. Use a deployed "+
 			"contract that calls the native function instead", toAddress)
 	}
 	// This was being run against CheckTx cache, need to understand the reasoning
-	callee, err := state.GetMutableAccount(trans.state, toAddress)
+	callee, err := state.GetMutableAccount(reader, toAddress)
 	if err != nil {
 		return nil, err
 	}
@@ -89,7 +87,7 @@ func (trans *Transactor) Call(fromAddress, toAddress acm.Address, data []byte) (
 		return nil, fmt.Errorf("account %s does not exist", toAddress)
 	}
 	caller := acm.ConcreteAccount{Address: fromAddress}.MutableAccount()
-	txCache := state.NewCache(trans.state)
+	txCache := state.NewCache(reader)
 	params := vmParams(trans.tip)
 
 	vmach := evm.NewVM(txCache, params, caller.Address(), nil, trans.logger.WithScope("Call"))
@@ -111,11 +109,11 @@ func (trans *Transactor) Call(fromAddress, toAddress acm.Address, data []byte) (
 
 // Run the given code on an isolated and unpersisted state
 // Cannot be used to create new contracts.
-func (trans *Transactor) CallCode(fromAddress acm.Address, code, data []byte) (*Call, error) {
+func (trans *Transactor) CallCode(reader state.Reader, fromAddress acm.Address, code, data []byte) (*Call, error) {
 	// This was being run against CheckTx cache, need to understand the reasoning
 	callee := acm.ConcreteAccount{Address: fromAddress}.MutableAccount()
 	caller := acm.ConcreteAccount{Address: fromAddress}.MutableAccount()
-	txCache := state.NewCache(trans.state)
+	txCache := state.NewCache(reader)
 	params := vmParams(trans.tip)
 
 	vmach := evm.NewVM(txCache, params, caller.Address(), nil, trans.logger.WithScope("CallCode"))
@@ -168,8 +166,6 @@ func (trans *Transactor) BroadcastTx(tx txs.Tx) (*txs.Receipt, error) {
 // Orders calls to BroadcastTx using lock (waits for response from core before releasing)
 func (trans *Transactor) Transact(inputAccount SequencedAddressableSigner, address *acm.Address, data []byte, gasLimit,
 	fee uint64) (*txs.Receipt, error) {
-	trans.Lock()
-	defer trans.Unlock()
 	// TODO: [Silas] we should consider revising this method and removing fee, or
 	// possibly adding an amount parameter. It is non-sensical to just be able to
 	// set the fee. Our support of fees in general is questionable since at the
@@ -203,7 +199,6 @@ func (trans *Transactor) Transact(inputAccount SequencedAddressableSigner, addre
 	return trans.BroadcastTx(tx)
 }
 
-
 func (trans *Transactor) TransactAndHold(inputAccount SequencedAddressableSigner, address *acm.Address, data []byte, gasLimit,
 	fee uint64) (*evm_events.EventDataCall, error) {
 
@@ -244,54 +239,28 @@ func (trans *Transactor) TransactAndHold(inputAccount SequencedAddressableSigner
 	}
 }
 
-func (trans *Transactor) Send(privKey []byte, toAddress acm.Address, amount uint64) (*txs.Receipt, error) {
-	if len(privKey) != 64 {
-		return nil, fmt.Errorf("Private key is not of the right length: %d\n",
-			len(privKey))
-	}
-
-	pk := &[64]byte{}
-	copy(pk[:], privKey)
-	trans.Lock()
-	defer trans.Unlock()
-	pa, err := acm.GeneratePrivateAccountFromPrivateKeyBytes(privKey)
-	if err != nil {
-		return nil, err
-	}
-	acc, err := trans.state.GetAccount(pa.Address())
-	if err != nil {
-		return nil, err
-	}
-	sequence := uint64(1)
-	if acc != nil {
-		sequence = acc.Sequence() + uint64(1)
-	}
-
+func (trans *Transactor) Send(inputAccount SequencedAddressableSigner, toAddress acm.Address, amount uint64) (*txs.Receipt, error) {
 	tx := txs.NewSendTx()
 
 	txInput := &txs.TxInput{
-		Address:   pa.Address(),
+		Address:   inputAccount.Address(),
 		Amount:    amount,
-		Sequence:  sequence,
-		PublicKey: pa.PublicKey(),
+		Sequence:  inputAccount.Sequence() + 1,
+		PublicKey: inputAccount.PublicKey(),
 	}
-
 	tx.Inputs = append(tx.Inputs, txInput)
-
 	txOutput := &txs.TxOutput{Address: toAddress, Amount: amount}
-
 	tx.Outputs = append(tx.Outputs, txOutput)
 
-	// Got ourselves a tx.
-	txS, errS := trans.SignTx(tx, []acm.AddressableSigner{pa})
-	if errS != nil {
-		return nil, errS
+	err := tx.Sign(trans.tip.ChainID(), inputAccount)
+	if err != nil {
+		return nil, err
 	}
-	return trans.BroadcastTx(txS)
+	return trans.BroadcastTx(tx)
 }
 
-func (trans *Transactor) SendAndHold(privKey []byte, toAddress acm.Address, amount uint64) (*txs.Receipt, error) {
-	receipt, err := trans.Send(privKey, toAddress, amount)
+func (trans *Transactor) SendAndHold(inputAccount SequencedAddressableSigner, toAddress acm.Address, amount uint64) (*txs.Receipt, error) {
+	receipt, err := trans.Send(inputAccount, toAddress, amount)
 	if err != nil {
 		return nil, err
 	}
@@ -313,17 +282,12 @@ func (trans *Transactor) SendAndHold(privKey []byte, toAddress acm.Address, amou
 	timer := time.NewTimer(BlockingTimeoutSeconds * time.Second)
 	defer timer.Stop()
 
-	pa, err := acm.GeneratePrivateAccountFromPrivateKeyBytes(privKey)
-	if err != nil {
-		return nil, err
-	}
-
 	select {
 	case <-timer.C:
 		return nil, fmt.Errorf("transaction timed out TxHash: %X", receipt.TxHash)
 	case sendTx := <-wc:
 		// This is a double check - we subscribed to this tx's hash so something has gone wrong if the amounts don't match
-		if sendTx.Inputs[0].Address == pa.Address() && sendTx.Inputs[0].Amount == amount {
+		if sendTx.Inputs[0].Address == inputAccount.Address() && sendTx.Inputs[0].Amount == amount {
 			return receipt, nil
 		}
 		return nil, fmt.Errorf("received SendTx but hash doesn't seem to match what we subscribed to, "+
@@ -331,31 +295,16 @@ func (trans *Transactor) SendAndHold(privKey []byte, toAddress acm.Address, amou
 	}
 }
 
-func (trans *Transactor) TransactNameReg(privKey []byte, name, data string, amount, fee uint64) (*txs.Receipt, error) {
-	if len(privKey) != 64 {
-		return nil, fmt.Errorf("Private key is not of the right length: %d\n", len(privKey))
-	}
-	trans.Lock()
-	defer trans.Unlock()
-	pa, err := acm.GeneratePrivateAccountFromPrivateKeyBytes(privKey)
-	if err != nil {
-		return nil, err
-	}
-	acc, err := trans.state.GetAccount(pa.Address())
+func (trans *Transactor) TransactNameReg(inputAccount SequencedAddressableSigner, name, data string, amount,
+	fee uint64) (*txs.Receipt, error) {
+
+	// Formulate and sign
+	tx := txs.NewNameTxWithSequence(inputAccount.PublicKey(), name, data, amount, fee, inputAccount.Sequence()+1)
+	err := tx.Sign(trans.tip.ChainID(), inputAccount)
 	if err != nil {
 		return nil, err
 	}
-	sequence := uint64(1)
-	if acc == nil {
-		sequence = acc.Sequence() + uint64(1)
-	}
-	tx := txs.NewNameTxWithSequence(pa.PublicKey(), name, data, amount, fee, sequence)
-	// Got ourselves a tx.
-	txS, errS := trans.SignTx(tx, []acm.AddressableSigner{pa})
-	if errS != nil {
-		return nil, errS
-	}
-	return trans.BroadcastTx(txS)
+	return trans.BroadcastTx(tx)
 }
 
 // Sign a transaction
diff --git a/execution/transactor.go2 b/execution/transactor.go2
deleted file mode 100644
index 9cdd63a2..00000000
--- a/execution/transactor.go2
+++ /dev/null
@@ -1,377 +0,0 @@
-// Copyright 2017 Monax Industries Limited
-//
-// Licensed under the Apache License, Version 2.0 (the "License");
-// you may not use this file except in compliance with the License.
-// You may obtain a copy of the License at
-//
-//    http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-package execution
-
-import (
-	"context"
-	"fmt"
-	"runtime/debug"
-	"sync"
-	"time"
-
-	acm "github.com/hyperledger/burrow/account"
-	"github.com/hyperledger/burrow/account/state"
-	"github.com/hyperledger/burrow/binary"
-	"github.com/hyperledger/burrow/blockchain"
-	"github.com/hyperledger/burrow/consensus/tendermint/codes"
-	"github.com/hyperledger/burrow/event"
-	exe_events "github.com/hyperledger/burrow/execution/events"
-	"github.com/hyperledger/burrow/execution/evm"
-	evm_events "github.com/hyperledger/burrow/execution/evm/events"
-	"github.com/hyperledger/burrow/logging"
-	"github.com/hyperledger/burrow/logging/structure"
-	"github.com/hyperledger/burrow/txs"
-	abci_types "github.com/tendermint/abci/types"
-	"github.com/tendermint/go-wire"
-)
-
-const BlockingTimeoutSeconds = 30
-
-type Call struct {
-	Return  []byte
-	GasUsed uint64
-}
-
-type SequencedAddressableSigner interface {
-	acm.AddressableSigner
-	Sequence() uint64
-}
-
-// Transactor is the controller/middleware for the v0 RPC
-type Transactor struct {
-	sync.Mutex
-	tip              blockchain.Tip
-	eventEmitter     event.Emitter
-	broadcastTxAsync func(tx txs.Tx, callback func(res *abci_types.Response)) error
-	logger           *logging.Logger
-}
-
-func NewTransactor(tip blockchain.Tip, eventEmitter event.Emitter,
-	broadcastTxAsync func(tx txs.Tx, callback func(res *abci_types.Response)) error,
-	logger *logging.Logger) *Transactor {
-
-	return &Transactor{
-		tip:              tip,
-		eventEmitter:     eventEmitter,
-		broadcastTxAsync: broadcastTxAsync,
-		logger:           logger.With(structure.ComponentKey, "Transactor"),
-	}
-}
-
-// Run a contract's code on an isolated and unpersisted state
-// Cannot be used to create new contracts
-func (trans *Transactor) Call(iterable state.Iterable, fromAddress, toAddress acm.Address,
-	data []byte) (call *Call, err error) {
-
-	if evm.RegisteredNativeContract(toAddress.Word256()) {
-		return nil, fmt.Errorf("attempt to call native contract at address "+
-			"%X, but native contracts can not be called directly. Use a deployed "+
-			"contract that calls the native function instead", toAddress)
-	}
-	// This was being run against CheckTx cache, need to understand the reasoning
-	callee, err := state.GetMutableAccount(iterable, toAddress)
-	if err != nil {
-		return nil, err
-	}
-	if callee == nil {
-		return nil, fmt.Errorf("account %s does not exist", toAddress)
-	}
-	caller := acm.ConcreteAccount{Address: fromAddress}.MutableAccount()
-	txCache := state.NewCache(iterable)
-	params := vmParams(trans.tip)
-
-	vmach := evm.NewVM(txCache, params, caller.Address(), nil, trans.logger.WithScope("Call"))
-	vmach.SetPublisher(trans.eventEmitter)
-
-	gas := params.GasLimit
-	defer func() {
-		if r := recover(); r != nil {
-			err = fmt.Errorf("panic from VM in simulated call: %v\n%s", r, debug.Stack())
-		}
-	}()
-	ret, err := vmach.Call(caller, callee, callee.Code(), data, 0, &gas)
-	if err != nil {
-		return nil, err
-	}
-	gasUsed := params.GasLimit - gas
-	return &Call{Return: ret, GasUsed: gasUsed}, nil
-}
-
-// Run the given code on an isolated and unpersisted state
-// Cannot be used to create new contracts.
-func (trans *Transactor) CallCode(iterable state.Iterable, fromAddress acm.Address, code, data []byte) (*Call, error) {
-	// This was being run against CheckTx cache, need to understand the reasoning
-	callee := acm.ConcreteAccount{Address: fromAddress}.MutableAccount()
-	caller := acm.ConcreteAccount{Address: fromAddress}.MutableAccount()
-	txCache := state.NewCache(iterable)
-	params := vmParams(trans.tip)
-
-	vmach := evm.NewVM(txCache, params, caller.Address(), nil, trans.logger.WithScope("CallCode"))
-	gas := params.GasLimit
-	ret, err := vmach.Call(caller, callee, code, data, 0, &gas)
-	if err != nil {
-		return nil, err
-	}
-	gasUsed := params.GasLimit - gas
-	return &Call{Return: ret, GasUsed: gasUsed}, nil
-}
-
-func (trans *Transactor) BroadcastTxAsync(tx txs.Tx, callback func(res *abci_types.Response)) error {
-	return trans.broadcastTxAsync(tx, callback)
-}
-
-// Broadcast a transaction.
-func (trans *Transactor) BroadcastTx(tx txs.Tx) (*txs.Receipt, error) {
-	trans.logger.Trace.Log("method", "BroadcastTx",
-		"tx_hash", tx.Hash(trans.tip.ChainID()),
-		"tx", tx.String())
-	responseCh := make(chan *abci_types.Response, 1)
-	err := trans.BroadcastTxAsync(tx, func(res *abci_types.Response) {
-		responseCh <- res
-	})
-
-	if err != nil {
-		return nil, err
-	}
-	response := <-responseCh
-	checkTxResponse := response.GetCheckTx()
-	if checkTxResponse == nil {
-		return nil, fmt.Errorf("application did not return CheckTx response")
-	}
-
-	switch checkTxResponse.Code {
-	case codes.TxExecutionSuccessCode:
-		receipt := new(txs.Receipt)
-		err := wire.ReadBinaryBytes(checkTxResponse.Data, receipt)
-		if err != nil {
-			return nil, fmt.Errorf("could not deserialise transaction receipt: %s", err)
-		}
-		return receipt, nil
-	default:
-		return nil, fmt.Errorf("error returned by Tendermint in BroadcastTxSync "+
-			"ABCI code: %v, ABCI log: %v", checkTxResponse.Code, checkTxResponse.Log)
-	}
-}
-
-// Orders calls to BroadcastTx using lock (waits for response from core before releasing)
-func (trans *Transactor) Transact(inputAccount SequencedAddressableSigner, address *acm.Address, data []byte, gasLimit,
-	fee uint64) (*txs.Receipt, error) {
-	trans.Lock()
-	defer trans.Unlock()
-	// TODO: [Silas] we should consider revising this method and removing fee, or
-	// possibly adding an amount parameter. It is non-sensical to just be able to
-	// set the fee. Our support of fees in general is questionable since at the
-	// moment all we do is deduct the fee effectively leaking token. It is possible
-	// someone may be using the sending of native token to payable functions but
-	// they can be served by broadcasting a token.
-
-	// We hard-code the amount to be equal to the fee which means the CallTx we
-	// generate transfers 0 value, which is the most sensible default since in
-	// recent solidity compilers the EVM generated will throw an error if value
-	// is transferred to a non-payable function.
-	txInput := &txs.TxInput{
-		Address:   inputAccount.Address(),
-		Amount:    fee,
-		Sequence:  inputAccount.Sequence() + 1,
-		PublicKey: inputAccount.PublicKey(),
-	}
-	tx := &txs.CallTx{
-		Input:    txInput,
-		Address:  address,
-		GasLimit: gasLimit,
-		Fee:      fee,
-		Data:     data,
-	}
-
-	// Got ourselves a tx.
-	err := tx.Sign(trans.tip.ChainID(), inputAccount)
-	if err != nil {
-		return nil, err
-	}
-	return trans.BroadcastTx(tx)
-}
-
-func (trans *Transactor) TransactAndHold(inputAccount SequencedAddressableSigner, privKey []byte,
-	address *acm.Address, data []byte, gasLimit, fee uint64) (*evm_events.EventDataCall, error) {
-
-	receipt, err := trans.Transact(inputAccount, address, data, gasLimit, fee)
-	if err != nil {
-		return nil, err
-	}
-
-	// We want non-blocking on the first event received (but buffer the value),
-	// after which we want to block (and then discard the value - see below)
-	wc := make(chan *evm_events.EventDataCall, 1)
-
-	subID, err := event.GenerateSubscriptionID()
-	if err != nil {
-		return nil, err
-	}
-
-	err = evm_events.SubscribeAccountCall(context.Background(), trans.eventEmitter, subID, receipt.ContractAddress,
-		receipt.TxHash, wc)
-	if err != nil {
-		return nil, err
-	}
-	// Will clean up callback goroutine and subscription in pubsub
-	defer trans.eventEmitter.UnsubscribeAll(context.Background(), subID)
-
-	timer := time.NewTimer(BlockingTimeoutSeconds * time.Second)
-	defer timer.Stop()
-
-	select {
-	case <-timer.C:
-		return nil, fmt.Errorf("transaction timed out TxHash: %X", receipt.TxHash)
-	case eventDataCall := <-wc:
-		if eventDataCall.Exception != "" {
-			return nil, fmt.Errorf("error when transacting: " + eventDataCall.Exception)
-		} else {
-			return eventDataCall, nil
-		}
-	}
-}
-
-func (trans *Transactor) Send(inputAccount SequencedAddressableSigner, toAddress acm.Address, amount uint64) (*txs.Receipt, error) {
-	if len(privKey) != 64 {
-		return nil, fmt.Errorf("Private key is not of the right length: %d\n",
-			len(privKey))
-	}
-
-	pk := &[64]byte{}
-	copy(pk[:], privKey)
-	trans.Lock()
-	defer trans.Unlock()
-	pa, err := acm.GeneratePrivateAccountFromPrivateKeyBytes(privKey)
-	if err != nil {
-		return nil, err
-	}
-	acc, err := trans.state.GetAccount(pa.Address())
-	if err != nil {
-		return nil, err
-	}
-	sequence := uint64(1)
-	if acc != nil {
-		sequence = acc.Sequence() + uint64(1)
-	}
-
-	tx := txs.NewSendTx()
-
-	txInput := &txs.TxInput{
-		Address:   pa.Address(),
-		Amount:    amount,
-		Sequence:  sequence,
-		PublicKey: pa.PublicKey(),
-	}
-
-	tx.Inputs = append(tx.Inputs, txInput)
-
-	txOutput := &txs.TxOutput{Address: toAddress, Amount: amount}
-
-	tx.Outputs = append(tx.Outputs, txOutput)
-
-	// Got ourselves a tx.
-	txS, errS := trans.SignTx(tx, []acm.AddressableSigner{pa})
-	if errS != nil {
-		return nil, errS
-	}
-	return trans.BroadcastTx(txS)
-}
-
-func (trans *Transactor) SendAndHold(privKey []byte, toAddress acm.Address, amount uint64) (*txs.Receipt, error) {
-	receipt, err := trans.Send(privKey, toAddress, amount)
-	if err != nil {
-		return nil, err
-	}
-
-	wc := make(chan *txs.SendTx)
-
-	subID, err := event.GenerateSubscriptionID()
-	if err != nil {
-		return nil, err
-	}
-
-	err = exe_events.SubscribeAccountOutputSendTx(context.Background(), trans.eventEmitter, subID, toAddress,
-		receipt.TxHash, wc)
-	if err != nil {
-		return nil, err
-	}
-	defer trans.eventEmitter.UnsubscribeAll(context.Background(), subID)
-
-	timer := time.NewTimer(BlockingTimeoutSeconds * time.Second)
-	defer timer.Stop()
-
-	pa, err := acm.GeneratePrivateAccountFromPrivateKeyBytes(privKey)
-	if err != nil {
-		return nil, err
-	}
-
-	select {
-	case <-timer.C:
-		return nil, fmt.Errorf("transaction timed out TxHash: %X", receipt.TxHash)
-	case sendTx := <-wc:
-		// This is a double check - we subscribed to this tx's hash so something has gone wrong if the amounts don't match
-		if sendTx.Inputs[0].Address == pa.Address() && sendTx.Inputs[0].Amount == amount {
-			return receipt, nil
-		}
-		return nil, fmt.Errorf("received SendTx but hash doesn't seem to match what we subscribed to, "+
-			"received SendTx: %v which does not match receipt on sending: %v", sendTx, receipt)
-	}
-}
-
-func (trans *Transactor) TransactNameReg(privKey []byte, name, data string, amount, fee uint64) (*txs.Receipt, error) {
-	if len(privKey) != 64 {
-		return nil, fmt.Errorf("Private key is not of the right length: %d\n", len(privKey))
-	}
-	trans.Lock()
-	defer trans.Unlock()
-	pa, err := acm.GeneratePrivateAccountFromPrivateKeyBytes(privKey)
-	if err != nil {
-		return nil, err
-	}
-	acc, err := trans.state.GetAccount(pa.Address())
-	if err != nil {
-		return nil, err
-	}
-	sequence := uint64(1)
-	if acc == nil {
-		sequence = acc.Sequence() + uint64(1)
-	}
-	tx := txs.NewNameTxWithSequence(pa.PublicKey(), name, data, amount, fee, sequence)
-	// Got ourselves a tx.
-	txS, errS := trans.SignTx(tx, []acm.AddressableSigner{pa})
-	if errS != nil {
-		return nil, errS
-	}
-	return trans.BroadcastTx(txS)
-}
-
-// Sign a transaction
-func (trans *Transactor) SignTx(tx txs.Tx, signingAccounts []acm.AddressableSigner) (txs.Tx, error) {
-	// more checks?
-	err := tx.Sign(trans.tip.ChainID(), signingAccounts...)
-	if err != nil {
-		return nil, err
-	}
-	return tx, nil
-}
-
-func vmParams(tip blockchain.Tip) evm.Params {
-	return evm.Params{
-		BlockHeight: tip.LastBlockHeight(),
-		BlockHash:   binary.LeftPadWord256(tip.LastBlockHash()),
-		BlockTime:   tip.LastBlockTime().Unix(),
-		GasLimit:    GasLimit,
-	}
-}
diff --git a/execution/transactor_test.go b/execution/transactor_test.go
index 7b0cdbfd..5f0689a5 100644
--- a/execution/transactor_test.go
+++ b/execution/transactor_test.go
@@ -24,7 +24,7 @@ type testTransactor struct {
 func newTestTransactor(txProcessor func(tx txs.Tx) (*types.Response, error)) testTransactor {
 	st := state.NewMemoryState()
 	emitter := event.NewEmitter(logger)
-	trans := NewTransactor(blockchain.NewTip(testChainID, time.Time{}, nil), st,
+	trans := NewTransactor(blockchain.NewTip(testChainID, time.Time{}, nil),
 		emitter, func(tx txs.Tx, callback func(res *types.Response)) error {
 			res, err := txProcessor(tx)
 			if err != nil {
diff --git a/rpc/service.go b/rpc/service.go
index 9f975053..c7d9cf52 100644
--- a/rpc/service.go
+++ b/rpc/service.go
@@ -37,36 +37,37 @@ import (
 
 // Magic! Should probably be configurable, but not shouldn't be so huge we
 // end up DoSing ourselves.
-const MaxBlockLookback = 100
+const MaxBlockLookback = 1000
 
 // Base service that provides implementation for all underlying RPC methods
 type Service struct {
-	ctx          context.Context
-	committed    *execution.Accounts
-	mempool      *execution.Accounts
-	subscribable event.Subscribable
-	nameReg      execution.NameRegIterable
-	blockchain   bcm.Blockchain
-	transactor   *execution.Transactor
-	nodeView     query.NodeView
-	logger       *logging.Logger
+	ctx             context.Context
+	committedState  state.Iterable
+	nameReg         execution.NameRegIterable
+	accounts        *execution.Accounts
+	mempoolAccounts *execution.Accounts
+	subscribable    event.Subscribable
+	blockchain      bcm.Blockchain
+	transactor      *execution.Transactor
+	nodeView        query.NodeView
+	logger          *logging.Logger
 }
 
-func NewService(ctx context.Context, committedState state.Iterable, mempoolState state.Iterable,
-	nameReg execution.NameRegIterable, subscribable event.Subscribable, blockchain bcm.Blockchain,
-	keyClient keys.KeyClient, transactor *execution.Transactor, nodeView query.NodeView,
-	logger *logging.Logger) *Service {
+func NewService(ctx context.Context, committedState state.Iterable, nameReg execution.NameRegIterable,
+	checker state.Reader, subscribable event.Subscribable, blockchain bcm.Blockchain, keyClient keys.KeyClient,
+	transactor *execution.Transactor, nodeView query.NodeView, logger *logging.Logger) *Service {
 
 	return &Service{
-		ctx:          ctx,
-		committed:    execution.NewAccounts(committedState, keyClient),
-		mempool:      execution.NewAccounts(mempoolState, keyClient),
-		nameReg:      nameReg,
-		subscribable: subscribable,
-		blockchain:   blockchain,
-		transactor:   transactor,
-		nodeView:     nodeView,
-		logger:       logger.With(structure.ComponentKey, "Service"),
+		ctx:             ctx,
+		committedState:  committedState,
+		accounts:        execution.NewAccounts(committedState, keyClient),
+		mempoolAccounts: execution.NewAccounts(checker, keyClient),
+		nameReg:         nameReg,
+		subscribable:    subscribable,
+		blockchain:      blockchain,
+		transactor:      transactor,
+		nodeView:        nodeView,
+		logger:          logger.With(structure.ComponentKey, "Service"),
 	}
 }
 
@@ -79,18 +80,27 @@ func NewSubscribableService(subscribable event.Subscribable, logger *logging.Log
 	}
 }
 
-// Transacting...
-
+// Get a Transactor providing methods for delegating signing and the core BroadcastTx function for publishing
+// transactions to the network
 func (s *Service) Transactor() *execution.Transactor {
 	return s.transactor
 }
 
-func (s *Service) Committed() *execution.Accounts {
-	return s.committed
+// By providing certain methods on the Transactor (such as Transact, Send, etc) with the (non-final) MempoolAccounts
+// rather than the committed (final) Accounts state the transactor can assign a sequence number based on all of the txs
+// it has seen since the last block - provided these transactions are successfully committed (via DeliverTx) then
+// subsequent transactions will have valid sequence numbers. This allows Burrow to coordinate sequencing and signing
+// for a key it holds or is provided - it is down to the key-holder to manage the mutual information between transactions
+// concurrent within a new block window.
+
+// Get the latest committed account state and signing accounts
+func (s *Service) Accounts() *execution.Accounts {
+	return s.accounts
 }
 
-func (s *Service) Mempool() *execution.Accounts {
-	return s.mempool
+// Get pending account state residing in the mempool
+func (s *Service) MempoolAccounts() *execution.Accounts {
+	return s.mempoolAccounts
 }
 
 func (s *Service) ListUnconfirmedTxs(maxTxs int) (*ResultListUnconfirmedTxs, error) {
@@ -215,7 +225,7 @@ func (s *Service) Genesis() (*ResultGenesis, error) {
 
 // Accounts
 func (s *Service) GetAccount(address acm.Address) (*ResultGetAccount, error) {
-	acc, err := s.committed.GetAccount(address)
+	acc, err := s.accounts.GetAccount(address)
 	if err != nil {
 		return nil, err
 	}
@@ -227,7 +237,7 @@ func (s *Service) GetAccount(address acm.Address) (*ResultGetAccount, error) {
 
 func (s *Service) ListAccounts(predicate func(acm.Account) bool) (*ResultListAccounts, error) {
 	accounts := make([]*acm.ConcreteAccount, 0)
-	s.committed.IterateAccounts(func(account acm.Account) (stop bool) {
+	s.committedState.IterateAccounts(func(account acm.Account) (stop bool) {
 		if predicate(account) {
 			accounts = append(accounts, acm.AsConcreteAccount(account))
 		}
@@ -241,7 +251,7 @@ func (s *Service) ListAccounts(predicate func(acm.Account) bool) (*ResultListAcc
 }
 
 func (s *Service) GetStorage(address acm.Address, key []byte) (*ResultGetStorage, error) {
-	account, err := s.committed.GetAccount(address)
+	account, err := s.accounts.GetAccount(address)
 	if err != nil {
 		return nil, err
 	}
@@ -249,7 +259,7 @@ func (s *Service) GetStorage(address acm.Address, key []byte) (*ResultGetStorage
 		return nil, fmt.Errorf("UnknownAddress: %s", address)
 	}
 
-	value, err := s.committed.GetStorage(address, binary.LeftPadWord256(key))
+	value, err := s.accounts.GetStorage(address, binary.LeftPadWord256(key))
 	if err != nil {
 		return nil, err
 	}
@@ -260,7 +270,7 @@ func (s *Service) GetStorage(address acm.Address, key []byte) (*ResultGetStorage
 }
 
 func (s *Service) DumpStorage(address acm.Address) (*ResultDumpStorage, error) {
-	account, err := s.committed.GetAccount(address)
+	account, err := s.accounts.GetAccount(address)
 	if err != nil {
 		return nil, err
 	}
@@ -268,7 +278,7 @@ func (s *Service) DumpStorage(address acm.Address) (*ResultDumpStorage, error) {
 		return nil, fmt.Errorf("UnknownAddress: %X", address)
 	}
 	var storageItems []StorageItem
-	s.committed.IterateStorage(address, func(key, value binary.Word256) (stop bool) {
+	s.committedState.IterateStorage(address, func(key, value binary.Word256) (stop bool) {
 		storageItems = append(storageItems, StorageItem{Key: key.UnpadLeft(), Value: value.UnpadLeft()})
 		return
 	})
@@ -279,7 +289,7 @@ func (s *Service) DumpStorage(address acm.Address) (*ResultDumpStorage, error) {
 }
 
 func (s *Service) GetAccountHumanReadable(address acm.Address) (*ResultGetAccountHumanReadable, error) {
-	acc, err := s.committed.GetAccount(address)
+	acc, err := s.accounts.GetAccount(address)
 	if err != nil {
 		return nil, err
 	}
diff --git a/rpc/tm/methods.go b/rpc/tm/methods.go
index 0b8e355c..27753d43 100644
--- a/rpc/tm/methods.go
+++ b/rpc/tm/methods.go
@@ -80,7 +80,7 @@ func GetRoutes(service *rpc.Service, logger *logging.Logger) map[string]*gorpc.R
 
 		// Simulated call
 		Call: gorpc.NewRPCFunc(func(fromAddress, toAddress acm.Address, data []byte) (*rpc.ResultCall, error) {
-			call, err := service.Transactor().Call(fromAddress, toAddress, data)
+			call, err := service.Transactor().Call(service.Accounts(), fromAddress, toAddress, data)
 			if err != nil {
 				return nil, err
 			}
@@ -88,7 +88,7 @@ func GetRoutes(service *rpc.Service, logger *logging.Logger) map[string]*gorpc.R
 		}, "fromAddress,toAddress,data"),
 
 		CallCode: gorpc.NewRPCFunc(func(fromAddress acm.Address, code, data []byte) (*rpc.ResultCall, error) {
-			call, err := service.Transactor().CallCode(fromAddress, code, data)
+			call, err := service.Transactor().CallCode(service.Accounts(), fromAddress, code, data)
 			if err != nil {
 				return nil, err
 			}
diff --git a/rpc/v0/client.go b/rpc/v0/client.go
index 351fd130..5bf27c5f 100644
--- a/rpc/v0/client.go
+++ b/rpc/v0/client.go
@@ -7,6 +7,7 @@ import (
 	"net/http"
 	"time"
 
+	"github.com/hyperledger/burrow/execution"
 	"github.com/hyperledger/burrow/execution/evm/events"
 	"github.com/hyperledger/burrow/rpc"
 	"github.com/hyperledger/burrow/txs"
@@ -30,39 +31,66 @@ func NewV0Client(url string) *V0Client {
 		url:   url,
 		codec: NewTCodec(),
 		client: &http.Client{
-			Timeout: 10 * time.Second,
+			Timeout: 1000 * time.Second,
 		},
 	}
 }
 
 func (vc *V0Client) Transact(param TransactParam) (*txs.Receipt, error) {
 	receipt := new(txs.Receipt)
-	err := vc.Call(TRANSACT, param, receipt)
+	err := vc.CallMethod(TRANSACT, param, receipt)
 	if err != nil {
 		return nil, err
 	}
 	return receipt, nil
 }
 
-func (vc *V0Client) TransactAndHold2(param TransactParam) (*events.EventDataCall, error) {
+func (vc *V0Client) TransactAndHold(param TransactParam) (*events.EventDataCall, error) {
 	eventDataCall := new(events.EventDataCall)
-	err := vc.Call(TRANSACT_AND_HOLD+"2", param, eventDataCall)
+	err := vc.CallMethod(TRANSACT_AND_HOLD, param, eventDataCall)
 	if err != nil {
 		return nil, err
 	}
 	return eventDataCall, nil
 }
 
-func (vc *V0Client) TransactAndHold(param TransactParam) (*events.EventDataCall, error) {
-	eventDataCall := new(events.EventDataCall)
-	err := vc.Call(TRANSACT_AND_HOLD, param, eventDataCall)
+func (vc *V0Client) Send(param SendParam) (*txs.Receipt, error) {
+	receipt := new(txs.Receipt)
+	err := vc.CallMethod(SEND, param, receipt)
 	if err != nil {
 		return nil, err
 	}
-	return eventDataCall, nil
+	return receipt, nil
+}
+
+func (vc *V0Client) SendAndHold(param SendParam) (*txs.Receipt, error) {
+	receipt := new(txs.Receipt)
+	err := vc.CallMethod(SEND_AND_HOLD, param, receipt)
+	if err != nil {
+		return nil, err
+	}
+	return receipt, nil
+}
+
+func (vc *V0Client) Call(param CallParam) (*execution.Call, error) {
+	call := new(execution.Call)
+	err := vc.CallMethod(CALL, param, call)
+	if err != nil {
+		return nil, err
+	}
+	return call, nil
+}
+
+func (vc *V0Client) CallCode(param CallCodeParam) (*execution.Call, error) {
+	call := new(execution.Call)
+	err := vc.CallMethod(CALL_CODE, param, call)
+	if err != nil {
+		return nil, err
+	}
+	return call, nil
 }
 
-func (vc *V0Client) Call(method string, param interface{}, result interface{}) error {
+func (vc *V0Client) CallMethod(method string, param interface{}, result interface{}) error {
 	// Marhsal into JSONRPC request object
 	bs, err := vc.codec.EncodeBytes(param)
 	if err != nil {
diff --git a/rpc/v0/integration/v0_test.go b/rpc/v0/integration/v0_test.go
index ede0a128..2a0eaee4 100644
--- a/rpc/v0/integration/v0_test.go
+++ b/rpc/v0/integration/v0_test.go
@@ -18,31 +18,78 @@
 package integration
 
 import (
-	"testing"
-
 	"encoding/hex"
+	"testing"
 
+	"github.com/hyperledger/burrow/account"
+	"github.com/hyperledger/burrow/binary"
 	"github.com/hyperledger/burrow/execution/evm/abi"
 	"github.com/hyperledger/burrow/rpc/v0"
 	"github.com/stretchr/testify/assert"
 	"github.com/stretchr/testify/require"
-	"github.com/hyperledger/burrow/binary"
 )
 
-func TestTransact(t *testing.T) {
+func TestTransactCallNoCode(t *testing.T) {
 	cli := v0.NewV0Client("http://localhost:1337/rpc")
 
-	address := privateAccounts[1].Address()
-	receipt, err := cli.Transact(v0.TransactParam{
-		PrivKey:  privateAccounts[0].PrivateKey().RawBytes(),
-		Address:  address.Bytes(),
-		Data:     []byte{},
-		Fee:      2,
-		GasLimit: 10000,
-	})
-	require.NoError(t, err)
-	assert.False(t, receipt.CreatesContract)
-	assert.Equal(t, address, receipt.ContractAddress)
+	// Flip flops between sending private key and input address to test private key and address based signing
+	privKey, inputAddress := privKeyInputAddressAlternator(privateAccounts[0])
+	toAddress := privateAccounts[2].Address()
+
+	for i := 0; i < 1000; i++ {
+		receipt, err := cli.Transact(v0.TransactParam{
+			PrivKey:      privKey(i),
+			InputAddress: inputAddress(i),
+			Address:      toAddress.Bytes(),
+			Data:         []byte{},
+			Fee:          2,
+			GasLimit:     10000 + uint64(i),
+		})
+		require.NoError(t, err)
+		assert.False(t, receipt.CreatesContract)
+		assert.Equal(t, toAddress, receipt.ContractAddress)
+	}
+}
+
+func TestTransactCreate(t *testing.T) {
+	cli := v0.NewV0Client("http://localhost:1337/rpc")
+
+	// Flip flops between sending private key and input address to test private key and address based signing
+	privKey, inputAddress := privKeyInputAddressAlternator(privateAccounts[0])
+	for i := 0; i < 1000; i++ {
+		bc, err := hex.DecodeString(strangeLoopBytecode)
+		require.NoError(t, err)
+		create, err := cli.Transact(v0.TransactParam{
+			PrivKey:      privKey(i),
+			InputAddress: inputAddress(i),
+			Address:      nil,
+			Data:         bc,
+			Fee:          2,
+			GasLimit:     10000,
+		})
+		require.NoError(t, err)
+		assert.True(t, create.CreatesContract)
+	}
+}
+
+func BenchmarkTransactCreateContract(b *testing.B) {
+	cli := v0.NewV0Client("http://localhost:1337/rpc")
+
+	privKey, inputAddress := privKeyInputAddressAlternator(privateAccounts[0])
+	for i := 0; i < b.N; i++ {
+		bc, err := hex.DecodeString(strangeLoopBytecode)
+		require.NoError(b, err)
+		create, err := cli.Transact(v0.TransactParam{
+			PrivKey:      privKey(i),
+			InputAddress: inputAddress(i),
+			Address:      nil,
+			Data:         bc,
+			Fee:          2,
+			GasLimit:     10000,
+		})
+		require.NoError(b, err)
+		assert.True(b, create.CreatesContract)
+	}
 }
 
 func TestTransactAndHold(t *testing.T) {
@@ -50,26 +97,82 @@ func TestTransactAndHold(t *testing.T) {
 
 	bc, err := hex.DecodeString(strangeLoopBytecode)
 	require.NoError(t, err)
-	create, err := cli.TransactAndHold(v0.TransactParam{
-		PrivKey:  privateAccounts[0].PrivateKey().RawBytes(),
-		Address:  nil,
-		Data:     bc,
-		Fee:      2,
-		GasLimit: 10000,
-	})
-	require.NoError(t, err)
-	assert.Equal(t, 0, create.StackDepth)
-	functionID := abi.FunctionID("UpsieDownsie()")
-	call, err := cli.TransactAndHold2(v0.TransactParam{
-		PrivKey:  privateAccounts[0].PrivateKey().RawBytes(),
-		Address:  create.CallData.Callee.Bytes(),
-		Data:     functionID[:],
-		Fee:      2,
-		GasLimit: 10000,
-	})
-	require.NoError(t, err)
-	depth := binary.Uint64FromWord256(binary.LeftPadWord256(call.Return))
-	// Would give 23 if taken from wrong frame
-	assert.Equal(t, 18, int(depth))
+
+	privKey, inputAddress := privKeyInputAddressAlternator(privateAccounts[0])
+
+	for i := 0; i < 2; i++ {
+		create, err := cli.TransactAndHold(v0.TransactParam{
+			PrivKey:      privKey(i),
+			InputAddress: inputAddress(i),
+			Address:      nil,
+			Data:         bc,
+			Fee:          2,
+			GasLimit:     10000,
+		})
+		require.NoError(t, err)
+		assert.Equal(t, 0, create.StackDepth)
+		functionID := abi.FunctionID("UpsieDownsie()")
+		call, err := cli.TransactAndHold(v0.TransactParam{
+			PrivKey:      privKey(i),
+			InputAddress: inputAddress(i),
+			Address:      create.CallData.Callee.Bytes(),
+			Data:         functionID[:],
+			Fee:          2,
+			GasLimit:     10000,
+		})
+		require.NoError(t, err)
+		depth := binary.Uint64FromWord256(binary.LeftPadWord256(call.Return))
+		// Would give 23 if taken from wrong frame
+		assert.Equal(t, 18, int(depth))
+	}
+}
+
+func TestSend(t *testing.T) {
+	cli := v0.NewV0Client("http://localhost:1337/rpc")
+
+	privKey, inputAddress := privKeyInputAddressAlternator(privateAccounts[0])
+	for i := 0; i < 1000; i++ {
+		send, err := cli.Send(v0.SendParam{
+			PrivKey:      privKey(i),
+			InputAddress: inputAddress(i),
+			Amount:       2003,
+			ToAddress:    privateAccounts[3].Address().Bytes(),
+		})
+		require.NoError(t, err)
+		assert.Equal(t, false, send.CreatesContract)
+	}
 }
 
+func TestSendAndHold(t *testing.T) {
+	cli := v0.NewV0Client("http://localhost:1337/rpc")
+
+	privKey, inputAddress := privKeyInputAddressAlternator(privateAccounts[0])
+
+	for i := 0; i < 2; i++ {
+		send, err := cli.SendAndHold(v0.SendParam{
+			PrivKey:      privKey(i),
+			InputAddress: inputAddress(i),
+			Amount:       2003,
+			ToAddress:    privateAccounts[3].Address().Bytes(),
+		})
+		require.NoError(t, err)
+		assert.Equal(t, false, send.CreatesContract)
+	}
+}
+
+// Returns a pair of functions that mutually exclusively return the private key bytes or input address bytes of a
+// private account in the same iteration of a loop indexed by an int
+func privKeyInputAddressAlternator(privateAccount account.PrivateAccount) (func(int) []byte, func(int) []byte) {
+	privKey := privateAccount.PrivateKey().RawBytes()
+	inputAddress := privateAccount.Address().Bytes()
+	return alternator(privKey, 0), alternator(inputAddress, 1)
+}
+
+func alternator(ret []byte, res int) func(int) []byte {
+	return func(i int) []byte {
+		if i%2 == res {
+			return ret
+		}
+		return nil
+	}
+}
diff --git a/rpc/v0/json_service.go b/rpc/v0/json_service.go
index cac693d2..d075eedd 100644
--- a/rpc/v0/json_service.go
+++ b/rpc/v0/json_service.go
@@ -52,8 +52,7 @@ func NewJSONServer(service server.HttpService) *JsonRpcServer {
 }
 
 // Start adds the rpc path to the router.
-func (jrs *JsonRpcServer) Start(config *server.ServerConfig,
-	router *gin.Engine) {
+func (jrs *JsonRpcServer) Start(config *server.ServerConfig, router *gin.Engine) {
 	router.POST(config.HTTP.JsonRpcEndpoint, jrs.handleFunc)
 	jrs.running = true
 }
diff --git a/rpc/v0/methods.go b/rpc/v0/methods.go
index 77848f4a..c17c7e61 100644
--- a/rpc/v0/methods.go
+++ b/rpc/v0/methods.go
@@ -15,6 +15,8 @@
 package v0
 
 import (
+	"fmt"
+
 	acm "github.com/hyperledger/burrow/account"
 	"github.com/hyperledger/burrow/execution"
 	"github.com/hyperledger/burrow/logging"
@@ -172,7 +174,7 @@ func GetMethods(codec rpc.Codec, service *rpc.Service, logger *logging.Logger) m
 			if err != nil {
 				return nil, rpc.INVALID_PARAMS, err
 			}
-			call, err := service.Transactor().Call(from, address, param.Data)
+			call, err := service.Transactor().Call(service.MempoolAccounts(), from, address, param.Data)
 			if err != nil {
 				return nil, rpc.INTERNAL_ERROR, err
 			}
@@ -188,7 +190,7 @@ func GetMethods(codec rpc.Codec, service *rpc.Service, logger *logging.Logger) m
 			if err != nil {
 				return nil, rpc.INVALID_PARAMS, err
 			}
-			call, err := service.Transactor().CallCode(from, param.Code, param.Data)
+			call, err := service.Transactor().CallCode(service.MempoolAccounts(), from, param.Code, param.Data)
 			if err != nil {
 				return nil, rpc.INTERNAL_ERROR, err
 			}
@@ -229,7 +231,8 @@ func GetMethods(codec rpc.Codec, service *rpc.Service, logger *logging.Logger) m
 			if err != nil {
 				return nil, rpc.INVALID_PARAMS, err
 			}
-			inputAccount, err := service.Mempool().SigningAccountFromPrivateKey(param.PrivKey)
+			// Use mempool state so that transact can generate a run of sequence numbers when formulating transactions
+			inputAccount, err := signingAccount(service.MempoolAccounts(), param.PrivKey, param.InputAddress)
 			if err != nil {
 				return nil, rpc.INVALID_PARAMS, err
 			}
@@ -249,7 +252,7 @@ func GetMethods(codec rpc.Codec, service *rpc.Service, logger *logging.Logger) m
 			if err != nil {
 				return nil, rpc.INVALID_PARAMS, err
 			}
-			inputAccount, err := service.Mempool().SigningAccountFromPrivateKey(param.PrivKey)
+			inputAccount, err := signingAccount(service.MempoolAccounts(), param.PrivKey, param.InputAddress)
 			if err != nil {
 				return nil, rpc.INVALID_PARAMS, err
 			}
@@ -269,7 +272,12 @@ func GetMethods(codec rpc.Codec, service *rpc.Service, logger *logging.Logger) m
 			if err != nil {
 				return nil, rpc.INVALID_PARAMS, err
 			}
-			receipt, err := service.Transactor().Send(param.PrivKey, toAddress, param.Amount)
+			// Run Send against mempool state
+			inputAccount, err := signingAccount(service.MempoolAccounts(), param.PrivKey, param.InputAddress)
+			if err != nil {
+				return nil, rpc.INVALID_PARAMS, err
+			}
+			receipt, err := service.Transactor().Send(inputAccount, toAddress, param.Amount)
 			if err != nil {
 				return nil, rpc.INTERNAL_ERROR, err
 			}
@@ -285,7 +293,12 @@ func GetMethods(codec rpc.Codec, service *rpc.Service, logger *logging.Logger) m
 			if err != nil {
 				return nil, rpc.INVALID_PARAMS, err
 			}
-			rec, err := service.Transactor().SendAndHold(param.PrivKey, toAddress, param.Amount)
+			// Run Send against mempool state
+			inputAccount, err := signingAccount(service.MempoolAccounts(), param.PrivKey, param.InputAddress)
+			if err != nil {
+				return nil, rpc.INVALID_PARAMS, err
+			}
+			rec, err := service.Transactor().SendAndHold(inputAccount, toAddress, param.Amount)
 			if err != nil {
 				return nil, rpc.INTERNAL_ERROR, err
 			}
@@ -297,7 +310,11 @@ func GetMethods(codec rpc.Codec, service *rpc.Service, logger *logging.Logger) m
 			if err != nil {
 				return nil, rpc.INVALID_PARAMS, err
 			}
-			receipt, err := service.Transactor().TransactNameReg(param.PrivKey, param.Name, param.Data, param.Amount, param.Fee)
+			inputAccount, err := service.MempoolAccounts().SigningAccountFromPrivateKey(param.PrivKey)
+			if err != nil {
+				return nil, rpc.INVALID_PARAMS, err
+			}
+			receipt, err := service.Transactor().TransactNameReg(inputAccount, param.Name, param.Data, param.Amount, param.Fee)
 			if err != nil {
 				return nil, rpc.INTERNAL_ERROR, err
 			}
@@ -426,3 +443,19 @@ func GetMethods(codec rpc.Codec, service *rpc.Service, logger *logging.Logger) m
 		},
 	}
 }
+
+// Gets signing account from onr of private key or address - failing if both are provided
+func signingAccount(accounts *execution.Accounts, privKey, addressBytes []byte) (*execution.SigningAccount, error) {
+	if len(addressBytes) > 0 {
+		if len(privKey) > 0 {
+			return nil, fmt.Errorf("privKey and address provided but only one or the other should be given")
+		}
+		address, err := acm.AddressFromBytes(addressBytes)
+		if err != nil {
+			return nil, err
+		}
+		return accounts.SigningAccount(address)
+	}
+
+	return accounts.SigningAccountFromPrivateKey(privKey)
+}
diff --git a/rpc/v0/params.go b/rpc/v0/params.go
index f3cc8efc..2635308e 100644
--- a/rpc/v0/params.go
+++ b/rpc/v0/params.go
@@ -88,18 +88,20 @@ type (
 	// Used when sending a transaction to be created and signed on the server
 	// (using the private key). This only uses the standard key type for now.
 	TransactParam struct {
-		PrivKey  []byte `json:"priv_key"`
-		Data     []byte `json:"data"`
-		Address  []byte `json:"address"`
-		Fee      uint64 `json:"fee"`
-		GasLimit uint64 `json:"gas_limit"`
+		PrivKey      []byte `json:"priv_key"`
+		InputAddress []byte `json:"input_account"`
+		Data         []byte `json:"data"`
+		Address      []byte `json:"address"`
+		Fee          uint64 `json:"fee"`
+		GasLimit     uint64 `json:"gas_limit"`
 	}
 
 	// Used when sending a 'Send' transaction.
 	SendParam struct {
-		PrivKey   []byte `json:"priv_key"`
-		ToAddress []byte `json:"to_address"`
-		Amount    uint64 `json:"amount"`
+		PrivKey      []byte `json:"priv_key"`
+		InputAddress []byte `json:"input_account"`
+		ToAddress    []byte `json:"to_address"`
+		Amount       uint64 `json:"amount"`
 	}
 
 	NameRegEntryParam struct {
@@ -109,10 +111,11 @@ type (
 	// Used when sending a namereg transaction to be created and signed on the server
 	// (using the private key). This only uses the standard key type for now.
 	TransactNameRegParam struct {
-		PrivKey []byte `json:"priv_key"`
-		Name    string `json:"name"`
-		Data    string `json:"data"`
-		Fee     uint64 `json:"fee"`
-		Amount  uint64 `json:"amount"`
+		PrivKey      []byte `json:"priv_key"`
+		InputAddress []byte `json:"input_account"`
+		Name         string `json:"name"`
+		Data         string `json:"data"`
+		Fee          uint64 `json:"fee"`
+		Amount       uint64 `json:"amount"`
 	}
 )
diff --git a/rpc/v0/server/server.go b/rpc/v0/server/server.go
index 9edc900f..5a624001 100644
--- a/rpc/v0/server/server.go
+++ b/rpc/v0/server/server.go
@@ -64,7 +64,7 @@ type ServeProcess struct {
 // Initializes all the servers and starts listening for connections.
 func (serveProcess *ServeProcess) Start() error {
 	router := gin.New()
-
+	gin.SetMode(gin.ReleaseMode)
 	config := serveProcess.config
 
 	ch := NewCORSMiddleware(config.CORS)
-- 
GitLab