diff --git a/account/state.go b/account/state.go index dcbba2424d1ba3227682e5d8a564bdb9998845c9..5569fbe636afc94f665f40970fe6267d5a7dac58 100644 --- a/account/state.go +++ b/account/state.go @@ -63,3 +63,11 @@ type StateWriter interface { Updater StorageSetter } + +type IterableStateWriter interface { + StateReader + Updater + StorageSetter + Iterable + StorageIterable +} diff --git a/account/state_cache.go b/account/state_cache.go index 723126a2a1e66b2e31513f8f64f1e2234174f751..0ddbb3cfe82353293837f999fc6a2bbd458625d7 100644 --- a/account/state_cache.go +++ b/account/state_cache.go @@ -17,84 +17,105 @@ package account import ( "fmt" "sort" + "sync" "github.com/hyperledger/burrow/binary" ) -type StateCache struct { - backend StateReader +type StateCache interface { + IterableStateWriter + Sync(state StateWriter) error + Reset(backend StateIterable) + Flush(state IterableStateWriter) error + Backend() StateIterable +} + +type stateCache struct { + sync.RWMutex + backend StateIterable accounts map[Address]*accountInfo } type accountInfo struct { + sync.RWMutex account Account storage map[binary.Word256]binary.Word256 removed bool updated bool } -var _ StateWriter = &StateCache{} - -func NewStateCache(backend StateReader) *StateCache { - return &StateCache{ +// Returns a StateCache that wraps an underlying StateReader to use on a cache miss, can write to an output StateWriter +// via Sync. Goroutine safe for concurrent access. +func NewStateCache(backend StateIterable) StateCache { + return &stateCache{ backend: backend, accounts: make(map[Address]*accountInfo), } } -func (cache *StateCache) GetAccount(address Address) (Account, error) { +func (cache *stateCache) GetAccount(address Address) (Account, error) { accInfo, err := cache.get(address) if err != nil { return nil, err } + accInfo.RLock() + defer accInfo.RUnlock() if accInfo.removed { return nil, nil } - - if accInfo.account == nil { - // fill cache - account, err := cache.backend.GetAccount(address) - if err != nil { - return nil, err - } - accInfo.account = account - } return accInfo.account, nil } -func (cache *StateCache) UpdateAccount(account Account) error { +func (cache *stateCache) UpdateAccount(account Account) error { accInfo, err := cache.get(account.Address()) if err != nil { return err } + accInfo.Lock() + defer accInfo.Unlock() if accInfo.removed { - return fmt.Errorf("UpdateAccount on a removed account %s", account.Address()) + return fmt.Errorf("UpdateAccount on a removed account: %s", account.Address()) } accInfo.account = account accInfo.updated = true return nil } -func (cache *StateCache) RemoveAccount(address Address) error { +func (cache *stateCache) RemoveAccount(address Address) error { accInfo, err := cache.get(address) if err != nil { return err } + accInfo.Lock() + defer accInfo.Unlock() if accInfo.removed { - fmt.Errorf("RemoveAccount on a removed account %s", address) - } else { - accInfo.removed = true + return fmt.Errorf("RemoveAccount on a removed account: %s", address) } + accInfo.removed = true return nil } -func (cache *StateCache) GetStorage(address Address, key binary.Word256) (binary.Word256, error) { +func (cache *stateCache) IterateAccounts(consumer func(Account) (stop bool)) (stopped bool, err error) { + // Try cache first for early exit + cache.RLock() + for _, info := range cache.accounts { + if consumer(info.account) { + return true, nil + } + } + cache.RUnlock() + return cache.backend.IterateAccounts(consumer) +} + +func (cache *stateCache) GetStorage(address Address, key binary.Word256) (binary.Word256, error) { accInfo, err := cache.get(address) if err != nil { return binary.Zero256, err } // Check cache + accInfo.RLock() value, ok := accInfo.storage[key] + accInfo.RUnlock() if ok { return value, nil } else { @@ -103,28 +124,50 @@ func (cache *StateCache) GetStorage(address Address, key binary.Word256) (binary if err != nil { return binary.Zero256, err } + accInfo.Lock() accInfo.storage[key] = value + accInfo.Unlock() return value, nil } } // NOTE: Set value to zero to remove. -func (cache *StateCache) SetStorage(address Address, key binary.Word256, value binary.Word256) error { +func (cache *stateCache) SetStorage(address Address, key binary.Word256, value binary.Word256) error { accInfo, err := cache.get(address) + accInfo.Lock() + defer accInfo.Unlock() if err != nil { return err } if accInfo.removed { - return fmt.Errorf("SetStorage on a removed account %s", address) + return fmt.Errorf("SetStorage on a removed account: %s", address) } accInfo.storage[key] = value accInfo.updated = true return nil } +func (cache *stateCache) IterateStorage(address Address, consumer func(key, value binary.Word256) (stop bool)) (stopped bool, err error) { + accInfo, err := cache.get(address) + if err != nil { + return false, err + } + accInfo.RLock() + // Try cache first for early exit + for key, value := range accInfo.storage { + if consumer(key, value) { + return true, nil + } + } + accInfo.RUnlock() + return cache.backend.IterateStorage(address, consumer) +} + // Syncs changes to the backend in deterministic order. Sends storage updates before updating // the account they belong so that storage values can be taken account of in the update. -func (cache *StateCache) Sync(state StateWriter) error { +func (cache *stateCache) Sync(state StateWriter) error { + cache.Lock() + defer cache.Unlock() var addresses Addresses for address := range cache.accounts { addresses = append(addresses, address) @@ -133,6 +176,7 @@ func (cache *StateCache) Sync(state StateWriter) error { sort.Stable(addresses) for _, address := range addresses { accInfo := cache.accounts[address] + accInfo.RLock() if accInfo.removed { err := state.RemoveAccount(address) if err != nil { @@ -159,23 +203,38 @@ func (cache *StateCache) Sync(state StateWriter) error { return err } } + accInfo.RUnlock() } return nil } // Resets the cache to empty initialising the backing map to the same size as the previous iteration. -func (cache *StateCache) Reset(backend StateReader) { +func (cache *stateCache) Reset(backend StateIterable) { + cache.Lock() + defer cache.Unlock() cache.backend = backend cache.accounts = make(map[Address]*accountInfo, len(cache.accounts)) } -func (cache *StateCache) Backend() StateReader { +// Syncs the StateCache and Resets it to use as the backend StateReader +func (cache *stateCache) Flush(state IterableStateWriter) error { + err := cache.Sync(state) + if err != nil { + return err + } + cache.Reset(state) + return nil +} + +func (cache *stateCache) Backend() StateIterable { return cache.backend } // Get the cache accountInfo item creating it if necessary -func (cache *StateCache) get(address Address) (*accountInfo, error) { +func (cache *stateCache) get(address Address) (*accountInfo, error) { + cache.RLock() accInfo := cache.accounts[address] + cache.RUnlock() if accInfo == nil { account, err := cache.backend.GetAccount(address) if err != nil { @@ -185,7 +244,9 @@ func (cache *StateCache) get(address Address) (*accountInfo, error) { account: account, storage: make(map[binary.Word256]binary.Word256), } + cache.Lock() cache.accounts[address] = accInfo + cache.Unlock() } return accInfo, nil } diff --git a/consensus/tendermint/abci/app.go b/consensus/tendermint/abci/app.go index 9a3a1db25e6ad438580471be84495fa04a684884..dd388df64fc9e3707d59b117b6ad105907d1f5c7 100644 --- a/consensus/tendermint/abci/app.go +++ b/consensus/tendermint/abci/app.go @@ -178,11 +178,6 @@ func (app *abciApp) Commit() abci_types.ResponseCommit { Log: fmt.Sprintf("Could not commit transactions in block to execution state: %s", err), } } - // Just kill the cache - it is badly implemented - app.committer.Reset() - - logging.InfoMsg(app.logger, "Resetting transaction check cache") - app.checker.Reset() // Commit to our blockchain state err = app.blockchain.CommitBlock(time.Unix(int64(app.block.Header.Time), 0), app.block.Hash, appHash) diff --git a/consensus/tendermint/tendermint.go b/consensus/tendermint/tendermint.go index 26c61a73da17da273cf353883f606a35ac912030..7a3af288ad8149f76052c1cabbb4ce467e2763c5 100644 --- a/consensus/tendermint/tendermint.go +++ b/consensus/tendermint/tendermint.go @@ -21,6 +21,7 @@ import ( dbm "github.com/tendermint/tmlibs/db" ) +// Serves as a wrapper around the Tendermint node's closeable resources (database connections) type Node struct { *node.Node closers []interface { @@ -28,6 +29,19 @@ type Node struct { } } +// Since Tendermint doesn't close its DB connections +func (n *Node) DBProvider(ctx *node.DBContext) (dbm.DB, error) { + db := dbm.NewDB(ctx.ID, ctx.Config.DBBackend, ctx.Config.DBDir()) + n.closers = append(n.closers, db) + return db, nil +} + +func (n *Node) Close() { + for _, closer := range n.closers { + closer.Close() + } +} + func NewNode( conf *config.Config, privValidator tm_types.PrivValidator, @@ -74,19 +88,6 @@ func BroadcastTxAsyncFunc(validator *Node, txEncoder txs.Encoder) func(tx txs.Tx } } -// Since Tendermint doesn't close its DB connections -func (n *Node) DBProvider(ctx *node.DBContext) (dbm.DB, error) { - db := dbm.NewDB(ctx.ID, ctx.Config.DBBackend, ctx.Config.DBDir()) - n.closers = append(n.closers, db) - return db, nil -} - -func (n *Node) Close() { - for _, closer := range n.closers { - closer.Close() - } -} - func DeriveGenesisDoc(burrowGenesisDoc *genesis.GenesisDoc) *tm_types.GenesisDoc { validators := make([]tm_types.GenesisValidator, len(burrowGenesisDoc.Validators)) for i, validator := range burrowGenesisDoc.Validators { diff --git a/execution/block_cache.go b/execution/block_cache.go deleted file mode 100644 index 7f1a240306be7945ec627d4043f879a4726278de..0000000000000000000000000000000000000000 --- a/execution/block_cache.go +++ /dev/null @@ -1,384 +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 ( - "bytes" - "fmt" - "sort" - "sync" - - acm "github.com/hyperledger/burrow/account" - . "github.com/hyperledger/burrow/binary" - "github.com/tendermint/merkleeyes/iavl" - dbm "github.com/tendermint/tmlibs/db" - "github.com/tendermint/tmlibs/merkle" -) - -func makeStorage(db dbm.DB, root []byte) merkle.Tree { - storage := iavl.NewIAVLTree(1024, db) - storage.Load(root) - return storage -} - -var _ acm.StateWriter = &BlockCache{} - -var _ acm.StateIterable = &BlockCache{} - -// TODO: BlockCache badly needs a rewrite to remove database sharing with State and make it communicate using the -// Account interfaces like a proper person. As well as other oddities of decoupled storage and account state - -// The blockcache helps prevent unnecessary IAVLTree updates and garbage generation. -type BlockCache struct { - // We currently provide the RPC layer with access to read-only access to BlockCache via the StateIterable interface - // on BatchExecutor. However since read-only operations generate writes to the BlockCache in the current design - // we need a mutex here. Otherwise BlockCache ought to be used within a component that is responsible for serialising - // the operations on the BlockCache. - sync.RWMutex - db dbm.DB - backend *State - accounts map[acm.Address]accountInfo - storages map[acm.Address]map[Word256]storageInfo - names map[string]nameInfo -} - -func NewBlockCache(backend *State) *BlockCache { - return &BlockCache{ - // TODO: This is bad and probably the cause of various panics. Accounts themselves are written - // to the State 'backend' but updates to storage just skip that and write directly to the database - db: backend.db, - backend: backend, - accounts: make(map[acm.Address]accountInfo), - storages: make(map[acm.Address]map[Word256]storageInfo), - names: make(map[string]nameInfo), - } -} - -func (cache *BlockCache) State() *State { - return cache.backend -} - -//------------------------------------- -// BlockCache.account - -func (cache *BlockCache) GetAccount(addr acm.Address) (acm.Account, error) { - acc, _, removed, _ := cache.accounts[addr].unpack() - if removed { - return nil, nil - } else if acc != nil { - return acc, nil - } else { - acc, err := cache.backend.GetAccount(addr) - if err != nil { - return nil, err - } - cache.Lock() - defer cache.Unlock() - cache.accounts[addr] = accountInfo{acc, nil, false, false} - return acc, nil - } -} - -func (cache *BlockCache) UpdateAccount(acc acm.Account) error { - cache.Lock() - defer cache.Unlock() - addr := acc.Address() - _, storage, removed, _ := cache.accounts[addr].unpack() - if removed { - return fmt.Errorf("UpdateAccount on a removed account %s", addr) - } - cache.accounts[addr] = accountInfo{acc, storage, false, true} - return nil -} - -func (cache *BlockCache) RemoveAccount(addr acm.Address) error { - cache.Lock() - defer cache.Unlock() - _, _, removed, _ := cache.accounts[addr].unpack() - if removed { - return fmt.Errorf("RemoveAccount on a removed account %s", addr) - } - cache.accounts[addr] = accountInfo{nil, nil, true, false} - return nil -} - -func (cache *BlockCache) IterateAccounts(consumer func(acm.Account) (stop bool)) (bool, error) { - cache.RLock() - defer cache.RUnlock() - for _, info := range cache.accounts { - if consumer(info.account) { - return true, nil - } - } - return cache.backend.IterateAccounts(consumer) -} - -// BlockCache.account -//------------------------------------- -// BlockCache.storage - -func (cache *BlockCache) GetStorage(addr acm.Address, key Word256) (Word256, error) { - // Check cache - cache.RLock() - info, ok := cache.lookupStorage(addr, key) - cache.RUnlock() - if ok { - return info.value, nil - } - // Get or load storage - cache.RLock() - acc, storage, removed, dirty := cache.accounts[addr].unpack() - cache.RUnlock() - if removed { - return Zero256, fmt.Errorf("GetStorage on a removed account %s", addr) - } - cache.Lock() - defer cache.Unlock() - - if acc != nil && storage == nil { - storage = makeStorage(cache.db, acc.StorageRoot()) - cache.accounts[addr] = accountInfo{acc, storage, false, dirty} - } else if acc == nil { - return Zero256, nil - } - // Load and set cache - _, val, _ := storage.Get(key.Bytes()) - value := LeftPadWord256(val) - cache.setStorage(addr, key, storageInfo{value, false}) - return value, nil -} - -// NOTE: Set value to zero to removed from the trie. -func (cache *BlockCache) SetStorage(addr acm.Address, key Word256, value Word256) error { - cache.Lock() - defer cache.Unlock() - _, _, removed, _ := cache.accounts[addr].unpack() - if removed { - return fmt.Errorf("SetStorage on a removed account %s", addr) - } - cache.setStorage(addr, key, storageInfo{value, true}) - return nil -} - -func (cache *BlockCache) IterateStorage(address acm.Address, consumer func(key, value Word256) (stop bool)) (bool, error) { - cache.RLock() - defer cache.RUnlock() - // Try cache first for early exit - for key, info := range cache.storages[address] { - if consumer(key, info.value) { - return true, nil - } - } - - return cache.backend.IterateStorage(address, consumer) -} - -// BlockCache.storage -//------------------------------------- -// BlockCache.names - -func (cache *BlockCache) GetNameRegEntry(name string) *NameRegEntry { - cache.RLock() - entry, removed, _ := cache.names[name].unpack() - cache.RUnlock() - if removed { - return nil - } else if entry != nil { - return entry - } else { - entry = cache.backend.GetNameRegEntry(name) - cache.Lock() - cache.names[name] = nameInfo{entry, false, false} - cache.Unlock() - return entry - } -} - -func (cache *BlockCache) UpdateNameRegEntry(entry *NameRegEntry) { - cache.Lock() - defer cache.Unlock() - cache.names[entry.Name] = nameInfo{entry, false, true} -} - -func (cache *BlockCache) RemoveNameRegEntry(name string) { - cache.Lock() - defer cache.Unlock() - _, removed, _ := cache.names[name].unpack() - if removed { - panic("RemoveNameRegEntry on a removed entry") - } - cache.names[name] = nameInfo{nil, true, false} -} - -// BlockCache.names -//------------------------------------- - -// CONTRACT the updates are in deterministic order. -func (cache *BlockCache) Sync() { - cache.Lock() - defer cache.Unlock() - // Determine order for storage updates - // The address comes first so it'll be grouped. - storageKeys := make([]Tuple256, 0, len(cache.storages)) - for address, keyInfoMap := range cache.storages { - for key, _ := range keyInfoMap { - storageKeys = append(storageKeys, Tuple256{First: address.Word256(), Second: key}) - } - } - Tuple256Slice(storageKeys).Sort() - - // Update storage for all account/key. - // Later we'll iterate over all the users and save storage + update storage root. - var ( - curAddr acm.Address - curAcc acm.Account - curAccRemoved bool - curStorage merkle.Tree - ) - for _, storageKey := range storageKeys { - addrWord256, key := Tuple256Split(storageKey) - addr := acm.AddressFromWord256(addrWord256) - if addr != curAddr || curAcc == nil { - acc, storage, removed, _ := cache.accounts[addr].unpack() - if !removed && storage == nil { - storage = makeStorage(cache.db, acc.StorageRoot()) - } - curAddr = addr - curAcc = acc - curAccRemoved = removed - curStorage = storage - } - if curAccRemoved { - continue - } - value, dirty := cache.storages[acm.AddressFromWord256(storageKey.First)][storageKey.Second].unpack() - if !dirty { - continue - } - if value.IsZero() { - curStorage.Remove(key.Bytes()) - } else { - curStorage.Set(key.Bytes(), value.Bytes()) - cache.accounts[addr] = accountInfo{curAcc, curStorage, false, true} - } - } - - // Determine order for accounts - addrs := []acm.Address{} - for addr := range cache.accounts { - addrs = append(addrs, addr) - } - sort.Slice(addrs, func(i, j int) bool { - return addrs[i].String() < addrs[j].String() - }) - - // Update or delete accounts. - for _, addr := range addrs { - acc, storage, removed, dirty := cache.accounts[addr].unpack() - if removed { - cache.backend.RemoveAccount(addr) - } else { - if acc == nil { - continue - } - if storage != nil { - newStorageRoot := storage.Save() - if !bytes.Equal(newStorageRoot, acc.StorageRoot()) { - acc = acm.AsMutableAccount(acc).SetStorageRoot(newStorageRoot) - dirty = true - } - } - if dirty { - cache.backend.UpdateAccount(acc) - } - } - } - - // Determine order for names - // note names may be of any length less than some limit - nameStrs := []string{} - for nameStr := range cache.names { - nameStrs = append(nameStrs, nameStr) - } - sort.Strings(nameStrs) - - // Update or delete names. - for _, nameStr := range nameStrs { - entry, removed, dirty := cache.names[nameStr].unpack() - if removed { - removed := cache.backend.RemoveNameRegEntry(nameStr) - if !removed { - panic(fmt.Sprintf("Could not remove namereg entry to be removed: %s", nameStr)) - } - } else { - if entry == nil { - continue - } - if dirty { - cache.backend.UpdateNameRegEntry(entry) - } - } - } -} - -func (cache *BlockCache) lookupStorage(address acm.Address, key Word256) (storageInfo, bool) { - keyInfoMap, ok := cache.storages[address] - if !ok { - return storageInfo{}, false - } - info, ok := keyInfoMap[key] - return info, ok -} - -func (cache *BlockCache) setStorage(address acm.Address, key Word256, info storageInfo) { - keyInfoMap, ok := cache.storages[address] - if !ok { - keyInfoMap = make(map[Word256]storageInfo) - cache.storages[address] = keyInfoMap - } - keyInfoMap[key] = info -} - -//----------------------------------------------------------------------------- - -type accountInfo struct { - account acm.Account - storage merkle.Tree - removed bool - dirty bool -} - -func (accInfo accountInfo) unpack() (acm.Account, merkle.Tree, bool, bool) { - return accInfo.account, accInfo.storage, accInfo.removed, accInfo.dirty -} - -type storageInfo struct { - value Word256 - dirty bool -} - -func (stjInfo storageInfo) unpack() (Word256, bool) { - return stjInfo.value, stjInfo.dirty -} - -type nameInfo struct { - name *NameRegEntry - removed bool - dirty bool -} - -func (nInfo nameInfo) unpack() (*NameRegEntry, bool, bool) { - return nInfo.name, nInfo.removed, nInfo.dirty -} diff --git a/execution/execution.go b/execution/execution.go index f62560d7b1f122fd2bc908729c299e4a16e1e094..abbb51ecc3db70391538ca9d28dba7abe64c8a46 100644 --- a/execution/execution.go +++ b/execution/execution.go @@ -51,15 +51,16 @@ type BatchCommitter interface { } type executor struct { - mtx sync.Mutex - chainID string - tip bcm.Tip - runCall bool - state *State - blockCache *BlockCache - publisher event.Publisher - eventCache *event.Cache - logger logging_types.InfoTraceLogger + sync.Mutex + chainID string + tip bcm.Tip + runCall bool + state *State + stateCache acm.StateCache + nameRegCache *NameRegCache + publisher event.Publisher + eventCache *event.Cache + logger logging_types.InfoTraceLogger } var _ BatchExecutor = (*executor)(nil) @@ -89,54 +90,62 @@ func newExecutor(runCall bool, eventFireable event.Publisher, logger logging_types.InfoTraceLogger) *executor { return &executor{ - chainID: chainID, - tip: tip, - runCall: runCall, - state: state, - blockCache: NewBlockCache(state), - publisher: eventFireable, - eventCache: event.NewEventCache(eventFireable), - logger: logger.With(structure.ComponentKey, "Executor"), + chainID: chainID, + tip: tip, + runCall: runCall, + state: state, + stateCache: acm.NewStateCache(state), + nameRegCache: NewNameRegCache(state), + publisher: eventFireable, + eventCache: event.NewEventCache(eventFireable), + logger: logger.With(structure.ComponentKey, "Executor"), } } // Accounts func (exe *executor) GetAccount(address acm.Address) (acm.Account, error) { - return exe.blockCache.GetAccount(address) + return exe.stateCache.GetAccount(address) } func (exe *executor) UpdateAccount(account acm.Account) error { - return exe.blockCache.UpdateAccount(account) + return exe.stateCache.UpdateAccount(account) } func (exe *executor) RemoveAccount(address acm.Address) error { - return exe.blockCache.RemoveAccount(address) + return exe.stateCache.RemoveAccount(address) } func (exe *executor) IterateAccounts(consumer func(acm.Account) bool) (bool, error) { - return exe.blockCache.IterateAccounts(consumer) + return exe.stateCache.IterateAccounts(consumer) } // Storage func (exe *executor) GetStorage(address acm.Address, key binary.Word256) (binary.Word256, error) { - return exe.blockCache.GetStorage(address, key) + return exe.stateCache.GetStorage(address, key) } func (exe *executor) SetStorage(address acm.Address, key binary.Word256, value binary.Word256) error { - return exe.blockCache.SetStorage(address, key, value) + return exe.stateCache.SetStorage(address, key, value) } func (exe *executor) IterateStorage(address acm.Address, consumer func(key, value binary.Word256) bool) (bool, error) { - return exe.blockCache.IterateStorage(address, consumer) + return exe.stateCache.IterateStorage(address, consumer) } func (exe *executor) Commit() ([]byte, error) { - exe.mtx.Lock() - defer exe.mtx.Unlock() - // sync the cache - exe.blockCache.Sync() + exe.Lock() + defer exe.Unlock() + // flush the caches + err := exe.stateCache.Flush(exe.state) + if err != nil { + return nil, err + } + err = exe.nameRegCache.Flush(exe.state) + if err != nil { + return nil, err + } // save state to disk - err := exe.state.Save() + err = exe.state.Save() if err != nil { return nil, err } @@ -146,7 +155,8 @@ func (exe *executor) Commit() ([]byte, error) { } func (exe *executor) Reset() error { - exe.blockCache = NewBlockCache(exe.state) + exe.stateCache = acm.NewStateCache(exe.state) + exe.nameRegCache = NewNameRegCache(exe.state) exe.eventCache = event.NewEventCache(exe.publisher) return nil } @@ -170,19 +180,19 @@ func (exe *executor) Execute(tx txs.Tx) (err error) { // Exec tx switch tx := tx.(type) { case *txs.SendTx: - accounts, err := getInputs(exe.blockCache, tx.Inputs) + accounts, err := getInputs(exe.stateCache, tx.Inputs) if err != nil { return err } // ensure all inputs have send permissions - if !hasSendPermission(exe.blockCache, accounts, logger) { + if !hasSendPermission(exe.stateCache, accounts, logger) { return fmt.Errorf("at least one input lacks permission for SendTx") } // add outputs to accounts map // if any outputs don't exist, all inputs must have CreateAccount perm - accounts, err = getOrMakeOutputs(exe.blockCache, accounts, tx.Outputs, logger) + accounts, err = getOrMakeOutputs(exe.stateCache, accounts, tx.Outputs, logger) if err != nil { return err } @@ -214,7 +224,7 @@ func (exe *executor) Execute(tx txs.Tx) (err error) { } for _, acc := range accounts { - exe.blockCache.UpdateAccount(acc) + exe.stateCache.UpdateAccount(acc) } // if the exe.eventCache is nil, nothing will happen @@ -234,7 +244,7 @@ func (exe *executor) Execute(tx txs.Tx) (err error) { var outAcc acm.Account // Validate input - inAcc, err := acm.GetMutableAccount(exe.blockCache, tx.Input.Address) + inAcc, err := acm.GetMutableAccount(exe.stateCache, tx.Input.Address) if err != nil { return err } @@ -246,11 +256,11 @@ func (exe *executor) Execute(tx txs.Tx) (err error) { createContract := tx.Address == nil if createContract { - if !hasCreateContractPermission(exe.blockCache, inAcc, logger) { + if !hasCreateContractPermission(exe.stateCache, inAcc, logger) { return fmt.Errorf("account %s does not have CreateContract permission", tx.Input.Address) } } else { - if !hasCallPermission(exe.blockCache, inAcc, logger) { + if !hasCallPermission(exe.stateCache, inAcc, logger) { return fmt.Errorf("account %s does not have Call permission", tx.Input.Address) } } @@ -287,7 +297,7 @@ func (exe *executor) Execute(tx txs.Tx) (err error) { // but that's fine, because the account will be created properly when the create tx runs in the block // and then this won't return nil. otherwise, we take their fee // Note: tx.Address == nil iff createContract so dereference is okay - outAcc, err = exe.blockCache.GetAccount(*tx.Address) + outAcc, err = exe.stateCache.GetAccount(*tx.Address) if err != nil { return err } @@ -308,7 +318,7 @@ func (exe *executor) Execute(tx txs.Tx) (err error) { return err } - exe.blockCache.UpdateAccount(inAcc) + exe.stateCache.UpdateAccount(inAcc) // The logic in runCall MUST NOT return. if exe.runCall { @@ -320,7 +330,7 @@ func (exe *executor) Execute(tx txs.Tx) (err error) { callee acm.MutableAccount = nil // initialized below code []byte = nil ret []byte = nil - txCache = acm.NewStateCache(exe.blockCache) + txCache = acm.NewStateCache(exe.stateCache) params = evm.Params{ BlockHeight: exe.tip.LastBlockHeight(), BlockHash: binary.LeftPadWord256(exe.tip.LastBlockHash()), @@ -389,7 +399,7 @@ func (exe *executor) Execute(tx txs.Tx) (err error) { if createContract { callee.SetCode(ret) } - txCache.Sync(exe.blockCache) + txCache.Sync(exe.stateCache) } CALL_COMPLETE: // err may or may not be nil. @@ -432,14 +442,14 @@ func (exe *executor) Execute(tx txs.Tx) (err error) { "new_sequence", inAcc.Sequence()+1) inAcc.IncSequence() } - exe.blockCache.UpdateAccount(inAcc) + exe.stateCache.UpdateAccount(inAcc) } return nil case *txs.NameTx: // Validate input - inAcc, err := acm.GetMutableAccount(exe.blockCache, tx.Input.Address) + inAcc, err := acm.GetMutableAccount(exe.stateCache, tx.Input.Address) if err != nil { return err } @@ -449,7 +459,7 @@ func (exe *executor) Execute(tx txs.Tx) (err error) { return txs.ErrTxInvalidAddress } // check permission - if !hasNamePermission(exe.blockCache, inAcc, logger) { + if !hasNamePermission(exe.stateCache, inAcc, logger) { return fmt.Errorf("account %s does not have Name permission", tx.Input.Address) } // pubKey should be present in either "inAcc" or "tx.Input" @@ -490,7 +500,10 @@ func (exe *executor) Execute(tx txs.Tx) (err error) { "last_block_height", lastBlockHeight) // check if the name exists - entry := exe.blockCache.GetNameRegEntry(tx.Name) + entry, err := exe.nameRegCache.GetNameRegEntry(tx.Name) + if err != nil { + return err + } if entry != nil { var expired bool @@ -512,7 +525,10 @@ func (exe *executor) Execute(tx txs.Tx) (err error) { // (owners if not expired, anyone if expired) logging.TraceMsg(logger, "Removing NameReg entry (no value and empty data in tx requests this)", "name", entry.Name) - exe.blockCache.RemoveNameRegEntry(entry.Name) + err := exe.nameRegCache.RemoveNameRegEntry(entry.Name) + if err != nil { + return err + } } else { // update the entry by bumping the expiry // and changing the data @@ -544,7 +560,10 @@ func (exe *executor) Execute(tx txs.Tx) (err error) { "credit", credit) } entry.Data = tx.Data - exe.blockCache.UpdateNameRegEntry(entry) + err := exe.nameRegCache.UpdateNameRegEntry(entry) + if err != nil { + return err + } } } else { if expiresIn < txs.MinNameRegistrationPeriod { @@ -560,7 +579,10 @@ func (exe *executor) Execute(tx txs.Tx) (err error) { logging.TraceMsg(logger, "Creating NameReg entry", "name", entry.Name, "expires_in", expiresIn) - exe.blockCache.UpdateNameRegEntry(entry) + err := exe.nameRegCache.UpdateNameRegEntry(entry) + if err != nil { + return err + } } // TODO: something with the value sent? @@ -571,7 +593,7 @@ func (exe *executor) Execute(tx txs.Tx) (err error) { if err != nil { return err } - exe.blockCache.UpdateAccount(inAcc) + exe.stateCache.UpdateAccount(inAcc) // TODO: maybe we want to take funds on error and allow txs in that don't do anythingi? @@ -724,7 +746,7 @@ func (exe *executor) Execute(tx txs.Tx) (err error) { case *txs.PermissionsTx: // Validate input - inAcc, err := acm.GetMutableAccount(exe.blockCache, tx.Input.Address) + inAcc, err := acm.GetMutableAccount(exe.stateCache, tx.Input.Address) if err != nil { return err } @@ -741,7 +763,7 @@ func (exe *executor) Execute(tx txs.Tx) (err error) { permFlag := tx.PermArgs.PermFlag // check permission - if !HasPermission(exe.blockCache, inAcc, permFlag, logger) { + if !HasPermission(exe.stateCache, inAcc, permFlag, logger) { return fmt.Errorf("account %s does not have moderator permission %s (%b)", tx.Input.Address, permission.PermFlagToString(permFlag), permFlag) } @@ -772,24 +794,24 @@ func (exe *executor) Execute(tx txs.Tx) (err error) { // this one doesn't make sense from txs return fmt.Errorf("HasBase is for contracts, not humans. Just look at the blockchain") case permission.SetBase: - permAcc, err = mutatePermissions(exe.blockCache, *tx.PermArgs.Address, + permAcc, err = mutatePermissions(exe.stateCache, *tx.PermArgs.Address, func(perms *ptypes.AccountPermissions) error { return perms.Base.Set(*tx.PermArgs.Permission, *tx.PermArgs.Value) }) case permission.UnsetBase: - permAcc, err = mutatePermissions(exe.blockCache, *tx.PermArgs.Address, + permAcc, err = mutatePermissions(exe.stateCache, *tx.PermArgs.Address, func(perms *ptypes.AccountPermissions) error { return perms.Base.Unset(*tx.PermArgs.Permission) }) case permission.SetGlobal: - permAcc, err = mutatePermissions(exe.blockCache, permission.GlobalPermissionsAddress, + permAcc, err = mutatePermissions(exe.stateCache, permission.GlobalPermissionsAddress, func(perms *ptypes.AccountPermissions) error { return perms.Base.Set(*tx.PermArgs.Permission, *tx.PermArgs.Value) }) case permission.HasRole: return fmt.Errorf("HasRole is for contracts, not humans. Just look at the blockchain") case permission.AddRole: - permAcc, err = mutatePermissions(exe.blockCache, *tx.PermArgs.Address, + permAcc, err = mutatePermissions(exe.stateCache, *tx.PermArgs.Address, func(perms *ptypes.AccountPermissions) error { if !perms.AddRole(*tx.PermArgs.Role) { return fmt.Errorf("role (%s) already exists for account %s", @@ -798,7 +820,7 @@ func (exe *executor) Execute(tx txs.Tx) (err error) { return nil }) case permission.RemoveRole: - permAcc, err = mutatePermissions(exe.blockCache, *tx.PermArgs.Address, + permAcc, err = mutatePermissions(exe.stateCache, *tx.PermArgs.Address, func(perms *ptypes.AccountPermissions) error { if !perms.RmRole(*tx.PermArgs.Role) { return fmt.Errorf("role (%s) does not exist for account %s", @@ -821,9 +843,9 @@ func (exe *executor) Execute(tx txs.Tx) (err error) { if err != nil { return err } - exe.blockCache.UpdateAccount(inAcc) + exe.stateCache.UpdateAccount(inAcc) if permAcc != nil { - exe.blockCache.UpdateAccount(permAcc) + exe.stateCache.UpdateAccount(permAcc) } if exe.eventCache != nil { diff --git a/execution/execution_test.go b/execution/execution_test.go index b1df56840f29bddf8992c4ecdee5b33831b2d829..da8c62104bbbf7c9cc0f46decb57cef772eeaba6 100644 --- a/execution/execution_test.go +++ b/execution/execution_test.go @@ -179,7 +179,7 @@ func TestSendFails(t *testing.T) { // simple send tx should fail tx := txs.NewSendTx() - if err := tx.AddInput(batchCommitter.blockCache, users[0].PublicKey(), 5); err != nil { + if err := tx.AddInput(batchCommitter.stateCache, users[0].PublicKey(), 5); err != nil { t.Fatal(err) } tx.AddOutput(users[1].Address(), 5) @@ -192,7 +192,7 @@ func TestSendFails(t *testing.T) { // simple send tx with call perm should fail tx = txs.NewSendTx() - if err := tx.AddInput(batchCommitter.blockCache, users[2].PublicKey(), 5); err != nil { + if err := tx.AddInput(batchCommitter.stateCache, users[2].PublicKey(), 5); err != nil { t.Fatal(err) } tx.AddOutput(users[4].Address(), 5) @@ -205,7 +205,7 @@ func TestSendFails(t *testing.T) { // simple send tx with create perm should fail tx = txs.NewSendTx() - if err := tx.AddInput(batchCommitter.blockCache, users[3].PublicKey(), 5); err != nil { + if err := tx.AddInput(batchCommitter.stateCache, users[3].PublicKey(), 5); err != nil { t.Fatal(err) } tx.AddOutput(users[4].Address(), 5) @@ -217,11 +217,11 @@ func TestSendFails(t *testing.T) { } // simple send tx to unknown account without create_account perm should fail - acc := getAccount(batchCommitter.blockCache, users[3].Address()) + acc := getAccount(batchCommitter.stateCache, users[3].Address()) acc.MutablePermissions().Base.Set(permission.Send, true) - batchCommitter.blockCache.UpdateAccount(acc) + batchCommitter.stateCache.UpdateAccount(acc) tx = txs.NewSendTx() - if err := tx.AddInput(batchCommitter.blockCache, users[3].PublicKey(), 5); err != nil { + if err := tx.AddInput(batchCommitter.stateCache, users[3].PublicKey(), 5); err != nil { t.Fatal(err) } tx.AddOutput(users[6].Address(), 5) @@ -283,7 +283,7 @@ func TestCallFails(t *testing.T) { address4 := users[4].Address() // simple call tx should fail - tx, _ := txs.NewCallTx(batchCommitter.blockCache, users[0].PublicKey(), &address4, nil, 100, 100, 100) + tx, _ := txs.NewCallTx(batchCommitter.stateCache, users[0].PublicKey(), &address4, nil, 100, 100, 100) tx.Sign(testChainID, users[0]) if err := batchCommitter.Execute(tx); err == nil { t.Fatal("Expected error") @@ -292,7 +292,7 @@ func TestCallFails(t *testing.T) { } // simple call tx with send permission should fail - tx, _ = txs.NewCallTx(batchCommitter.blockCache, users[1].PublicKey(), &address4, nil, 100, 100, 100) + tx, _ = txs.NewCallTx(batchCommitter.stateCache, users[1].PublicKey(), &address4, nil, 100, 100, 100) tx.Sign(testChainID, users[1]) if err := batchCommitter.Execute(tx); err == nil { t.Fatal("Expected error") @@ -301,7 +301,7 @@ func TestCallFails(t *testing.T) { } // simple call tx with create permission should fail - tx, _ = txs.NewCallTx(batchCommitter.blockCache, users[3].PublicKey(), &address4, nil, 100, 100, 100) + tx, _ = txs.NewCallTx(batchCommitter.stateCache, users[3].PublicKey(), &address4, nil, 100, 100, 100) tx.Sign(testChainID, users[3]) if err := batchCommitter.Execute(tx); err == nil { t.Fatal("Expected error") @@ -313,7 +313,7 @@ func TestCallFails(t *testing.T) { // create txs // simple call create tx should fail - tx, _ = txs.NewCallTx(batchCommitter.blockCache, users[0].PublicKey(), nil, nil, 100, 100, 100) + tx, _ = txs.NewCallTx(batchCommitter.stateCache, users[0].PublicKey(), nil, nil, 100, 100, 100) tx.Sign(testChainID, users[0]) if err := batchCommitter.Execute(tx); err == nil { t.Fatal("Expected error") @@ -322,7 +322,7 @@ func TestCallFails(t *testing.T) { } // simple call create tx with send perm should fail - tx, _ = txs.NewCallTx(batchCommitter.blockCache, users[1].PublicKey(), nil, nil, 100, 100, 100) + tx, _ = txs.NewCallTx(batchCommitter.stateCache, users[1].PublicKey(), nil, nil, 100, 100, 100) tx.Sign(testChainID, users[1]) if err := batchCommitter.Execute(tx); err == nil { t.Fatal("Expected error") @@ -331,7 +331,7 @@ func TestCallFails(t *testing.T) { } // simple call create tx with call perm should fail - tx, _ = txs.NewCallTx(batchCommitter.blockCache, users[2].PublicKey(), nil, nil, 100, 100, 100) + tx, _ = txs.NewCallTx(batchCommitter.stateCache, users[2].PublicKey(), nil, nil, 100, 100, 100) tx.Sign(testChainID, users[2]) if err := batchCommitter.Execute(tx); err == nil { t.Fatal("Expected error") @@ -350,7 +350,7 @@ func TestSendPermission(t *testing.T) { // A single input, having the permission, should succeed tx := txs.NewSendTx() - if err := tx.AddInput(batchCommitter.blockCache, users[0].PublicKey(), 5); err != nil { + if err := tx.AddInput(batchCommitter.stateCache, users[0].PublicKey(), 5); err != nil { t.Fatal(err) } tx.AddOutput(users[1].Address(), 5) @@ -361,10 +361,10 @@ func TestSendPermission(t *testing.T) { // Two inputs, one with permission, one without, should fail tx = txs.NewSendTx() - if err := tx.AddInput(batchCommitter.blockCache, users[0].PublicKey(), 5); err != nil { + if err := tx.AddInput(batchCommitter.stateCache, users[0].PublicKey(), 5); err != nil { t.Fatal(err) } - if err := tx.AddInput(batchCommitter.blockCache, users[1].PublicKey(), 5); err != nil { + if err := tx.AddInput(batchCommitter.stateCache, users[1].PublicKey(), 5); err != nil { t.Fatal(err) } tx.AddOutput(users[2].Address(), 10) @@ -402,7 +402,7 @@ func TestCallPermission(t *testing.T) { st.UpdateAccount(simpleAcc) // A single input, having the permission, should succeed - tx, _ := txs.NewCallTx(batchCommitter.blockCache, users[0].PublicKey(), &simpleContractAddr, nil, 100, 100, 100) + tx, _ := txs.NewCallTx(batchCommitter.stateCache, users[0].PublicKey(), &simpleContractAddr, nil, 100, 100, 100) tx.Sign(testChainID, users[0]) if err := batchCommitter.Execute(tx); err != nil { t.Fatal("Transaction failed", err) @@ -423,10 +423,10 @@ func TestCallPermission(t *testing.T) { StorageRoot: Zero256.Bytes(), Permissions: permission.ZeroAccountPermissions, }.MutableAccount() - batchCommitter.blockCache.UpdateAccount(caller1Acc) + batchCommitter.stateCache.UpdateAccount(caller1Acc) // A single input, having the permission, but the contract doesn't have permission - tx, _ = txs.NewCallTx(batchCommitter.blockCache, users[0].PublicKey(), &caller1ContractAddr, nil, 100, 10000, 100) + tx, _ = txs.NewCallTx(batchCommitter.stateCache, users[0].PublicKey(), &caller1ContractAddr, nil, 100, 10000, 100) tx.Sign(testChainID, users[0]) // we need to subscribe to the Call event to detect the exception @@ -441,8 +441,8 @@ func TestCallPermission(t *testing.T) { // A single input, having the permission, and the contract has permission caller1Acc.MutablePermissions().Base.Set(permission.Call, true) - batchCommitter.blockCache.UpdateAccount(caller1Acc) - tx, _ = txs.NewCallTx(batchCommitter.blockCache, users[0].PublicKey(), &caller1ContractAddr, nil, 100, 10000, 100) + batchCommitter.stateCache.UpdateAccount(caller1Acc) + tx, _ = txs.NewCallTx(batchCommitter.stateCache, users[0].PublicKey(), &caller1ContractAddr, nil, 100, 10000, 100) tx.Sign(testChainID, users[0]) // we need to subscribe to the Call event to detect the exception @@ -469,10 +469,10 @@ func TestCallPermission(t *testing.T) { }.MutableAccount() caller1Acc.MutablePermissions().Base.Set(permission.Call, false) caller2Acc.MutablePermissions().Base.Set(permission.Call, true) - batchCommitter.blockCache.UpdateAccount(caller1Acc) - batchCommitter.blockCache.UpdateAccount(caller2Acc) + batchCommitter.stateCache.UpdateAccount(caller1Acc) + batchCommitter.stateCache.UpdateAccount(caller2Acc) - tx, _ = txs.NewCallTx(batchCommitter.blockCache, users[0].PublicKey(), &caller2ContractAddr, nil, 100, 10000, 100) + tx, _ = txs.NewCallTx(batchCommitter.stateCache, users[0].PublicKey(), &caller2ContractAddr, nil, 100, 10000, 100) tx.Sign(testChainID, users[0]) // we need to subscribe to the Call event to detect the exception @@ -488,9 +488,9 @@ func TestCallPermission(t *testing.T) { fmt.Println("\n##### CALL TO CONTRACT CALLING SIMPLE CONTRACT (PASS)") caller1Acc.MutablePermissions().Base.Set(permission.Call, true) - batchCommitter.blockCache.UpdateAccount(caller1Acc) + batchCommitter.stateCache.UpdateAccount(caller1Acc) - tx, _ = txs.NewCallTx(batchCommitter.blockCache, users[0].PublicKey(), &caller2ContractAddr, nil, 100, 10000, 100) + tx, _ = txs.NewCallTx(batchCommitter.stateCache, users[0].PublicKey(), &caller2ContractAddr, nil, 100, 10000, 100) tx.Sign(testChainID, users[0]) // we need to subscribe to the Call event to detect the exception @@ -517,14 +517,14 @@ func TestCreatePermission(t *testing.T) { createCode := wrapContractForCreate(contractCode) // A single input, having the permission, should succeed - tx, _ := txs.NewCallTx(batchCommitter.blockCache, users[0].PublicKey(), nil, createCode, 100, 100, 100) + tx, _ := txs.NewCallTx(batchCommitter.stateCache, users[0].PublicKey(), nil, createCode, 100, 100, 100) tx.Sign(testChainID, users[0]) if err := batchCommitter.Execute(tx); err != nil { t.Fatal("Transaction failed", err) } // ensure the contract is there contractAddr := acm.NewContractAddress(tx.Input.Address, tx.Input.Sequence) - contractAcc := getAccount(batchCommitter.blockCache, contractAddr) + contractAcc := getAccount(batchCommitter.stateCache, contractAddr) if contractAcc == nil { t.Fatalf("failed to create contract %s", contractAddr) } @@ -542,14 +542,14 @@ func TestCreatePermission(t *testing.T) { createFactoryCode := wrapContractForCreate(factoryCode) // A single input, having the permission, should succeed - tx, _ = txs.NewCallTx(batchCommitter.blockCache, users[0].PublicKey(), nil, createFactoryCode, 100, 100, 100) + tx, _ = txs.NewCallTx(batchCommitter.stateCache, users[0].PublicKey(), nil, createFactoryCode, 100, 100, 100) tx.Sign(testChainID, users[0]) if err := batchCommitter.Execute(tx); err != nil { t.Fatal("Transaction failed", err) } // ensure the contract is there contractAddr = acm.NewContractAddress(tx.Input.Address, tx.Input.Sequence) - contractAcc = getAccount(batchCommitter.blockCache, contractAddr) + contractAcc = getAccount(batchCommitter.stateCache, contractAddr) if contractAcc == nil { t.Fatalf("failed to create contract %s", contractAddr) } @@ -562,7 +562,7 @@ func TestCreatePermission(t *testing.T) { fmt.Println("\n###### CALL THE FACTORY (FAIL)") // A single input, having the permission, should succeed - tx, _ = txs.NewCallTx(batchCommitter.blockCache, users[0].PublicKey(), &contractAddr, createCode, 100, 100, 100) + tx, _ = txs.NewCallTx(batchCommitter.stateCache, users[0].PublicKey(), &contractAddr, createCode, 100, 100, 100) tx.Sign(testChainID, users[0]) // we need to subscribe to the Call event to detect the exception _, exception := execTxWaitEvent(t, batchCommitter, tx, evm_events.EventStringAccountCall(contractAddr)) // @@ -575,10 +575,10 @@ func TestCreatePermission(t *testing.T) { fmt.Println("\n###### CALL THE FACTORY (PASS)") contractAcc.MutablePermissions().Base.Set(permission.CreateContract, true) - batchCommitter.blockCache.UpdateAccount(contractAcc) + batchCommitter.stateCache.UpdateAccount(contractAcc) // A single input, having the permission, should succeed - tx, _ = txs.NewCallTx(batchCommitter.blockCache, users[0].PublicKey(), &contractAddr, createCode, 100, 100, 100) + tx, _ = txs.NewCallTx(batchCommitter.stateCache, users[0].PublicKey(), &contractAddr, createCode, 100, 100, 100) tx.Sign(testChainID, users[0]) // we need to subscribe to the Call event to detect the exception _, exception = execTxWaitEvent(t, batchCommitter, tx, evm_events.EventStringAccountCall(contractAddr)) // @@ -601,17 +601,17 @@ func TestCreatePermission(t *testing.T) { }.MutableAccount() contractAcc.MutablePermissions().Base.Set(permission.Call, true) contractAcc.MutablePermissions().Base.Set(permission.CreateContract, true) - batchCommitter.blockCache.UpdateAccount(contractAcc) + batchCommitter.stateCache.UpdateAccount(contractAcc) // this should call the 0 address but not create ... - tx, _ = txs.NewCallTx(batchCommitter.blockCache, users[0].PublicKey(), &contractAddr, createCode, 100, 10000, 100) + tx, _ = txs.NewCallTx(batchCommitter.stateCache, users[0].PublicKey(), &contractAddr, createCode, 100, 10000, 100) tx.Sign(testChainID, users[0]) // we need to subscribe to the Call event to detect the exception _, exception = execTxWaitEvent(t, batchCommitter, tx, evm_events.EventStringAccountCall(acm.Address{})) // if exception != "" { t.Fatal("unexpected exception", exception) } - zeroAcc := getAccount(batchCommitter.blockCache, acm.Address{}) + zeroAcc := getAccount(batchCommitter.stateCache, acm.Address{}) if len(zeroAcc.Code()) != 0 { t.Fatal("the zero account was given code from a CALL!") } @@ -757,7 +757,7 @@ func TestCreateAccountPermission(t *testing.T) { // A single input, having the permission, should succeed tx := txs.NewSendTx() - if err := tx.AddInput(batchCommitter.blockCache, users[0].PublicKey(), 5); err != nil { + if err := tx.AddInput(batchCommitter.stateCache, users[0].PublicKey(), 5); err != nil { t.Fatal(err) } tx.AddOutput(users[6].Address(), 5) @@ -768,10 +768,10 @@ func TestCreateAccountPermission(t *testing.T) { // Two inputs, both with send, one with create, one without, should fail tx = txs.NewSendTx() - if err := tx.AddInput(batchCommitter.blockCache, users[0].PublicKey(), 5); err != nil { + if err := tx.AddInput(batchCommitter.stateCache, users[0].PublicKey(), 5); err != nil { t.Fatal(err) } - if err := tx.AddInput(batchCommitter.blockCache, users[1].PublicKey(), 5); err != nil { + if err := tx.AddInput(batchCommitter.stateCache, users[1].PublicKey(), 5); err != nil { t.Fatal(err) } tx.AddOutput(users[7].Address(), 10) @@ -785,10 +785,10 @@ func TestCreateAccountPermission(t *testing.T) { // Two inputs, both with send, one with create, one without, two ouputs (one known, one unknown) should fail tx = txs.NewSendTx() - if err := tx.AddInput(batchCommitter.blockCache, users[0].PublicKey(), 5); err != nil { + if err := tx.AddInput(batchCommitter.stateCache, users[0].PublicKey(), 5); err != nil { t.Fatal(err) } - if err := tx.AddInput(batchCommitter.blockCache, users[1].PublicKey(), 5); err != nil { + if err := tx.AddInput(batchCommitter.stateCache, users[1].PublicKey(), 5); err != nil { t.Fatal(err) } tx.AddOutput(users[7].Address(), 4) @@ -802,14 +802,14 @@ func TestCreateAccountPermission(t *testing.T) { } // Two inputs, both with send, both with create, should pass - acc := getAccount(batchCommitter.blockCache, users[1].Address()) + acc := getAccount(batchCommitter.stateCache, users[1].Address()) acc.MutablePermissions().Base.Set(permission.CreateAccount, true) - batchCommitter.blockCache.UpdateAccount(acc) + batchCommitter.stateCache.UpdateAccount(acc) tx = txs.NewSendTx() - if err := tx.AddInput(batchCommitter.blockCache, users[0].PublicKey(), 5); err != nil { + if err := tx.AddInput(batchCommitter.stateCache, users[0].PublicKey(), 5); err != nil { t.Fatal(err) } - if err := tx.AddInput(batchCommitter.blockCache, users[1].PublicKey(), 5); err != nil { + if err := tx.AddInput(batchCommitter.stateCache, users[1].PublicKey(), 5); err != nil { t.Fatal(err) } tx.AddOutput(users[7].Address(), 10) @@ -821,10 +821,10 @@ func TestCreateAccountPermission(t *testing.T) { // Two inputs, both with send, both with create, two outputs (one known, one unknown) should pass tx = txs.NewSendTx() - if err := tx.AddInput(batchCommitter.blockCache, users[0].PublicKey(), 5); err != nil { + if err := tx.AddInput(batchCommitter.stateCache, users[0].PublicKey(), 5); err != nil { t.Fatal(err) } - if err := tx.AddInput(batchCommitter.blockCache, users[1].PublicKey(), 5); err != nil { + if err := tx.AddInput(batchCommitter.stateCache, users[1].PublicKey(), 5); err != nil { t.Fatal(err) } tx.AddOutput(users[7].Address(), 7) @@ -838,9 +838,9 @@ func TestCreateAccountPermission(t *testing.T) { //---------------------------------------------------------- // CALL to unknown account - acc = getAccount(batchCommitter.blockCache, users[0].Address()) + acc = getAccount(batchCommitter.stateCache, users[0].Address()) acc.MutablePermissions().Base.Set(permission.Call, true) - batchCommitter.blockCache.UpdateAccount(acc) + batchCommitter.stateCache.UpdateAccount(acc) // call to contract that calls unknown account - without create_account perm // create contract that calls the simple contract @@ -854,10 +854,10 @@ func TestCreateAccountPermission(t *testing.T) { StorageRoot: Zero256.Bytes(), Permissions: permission.ZeroAccountPermissions, }.MutableAccount() - batchCommitter.blockCache.UpdateAccount(caller1Acc) + batchCommitter.stateCache.UpdateAccount(caller1Acc) // A single input, having the permission, but the contract doesn't have permission - txCall, _ := txs.NewCallTx(batchCommitter.blockCache, users[0].PublicKey(), &caller1ContractAddr, nil, 100, 10000, 100) + txCall, _ := txs.NewCallTx(batchCommitter.stateCache, users[0].PublicKey(), &caller1ContractAddr, nil, 100, 10000, 100) txCall.Sign(testChainID, users[0]) // we need to subscribe to the Call event to detect the exception @@ -870,9 +870,9 @@ func TestCreateAccountPermission(t *testing.T) { // NOTE: for a users to be able to CreateAccount, it must be able to send! caller1Acc.MutablePermissions().Base.Set(permission.CreateAccount, true) caller1Acc.MutablePermissions().Base.Set(permission.Call, true) - batchCommitter.blockCache.UpdateAccount(caller1Acc) + batchCommitter.stateCache.UpdateAccount(caller1Acc) // A single input, having the permission, but the contract doesn't have permission - txCall, _ = txs.NewCallTx(batchCommitter.blockCache, users[0].PublicKey(), &caller1ContractAddr, nil, 100, 10000, 100) + txCall, _ = txs.NewCallTx(batchCommitter.stateCache, users[0].PublicKey(), &caller1ContractAddr, nil, 100, 10000, 100) txCall.Sign(testChainID, users[0]) // we need to subscribe to the Call event to detect the exception @@ -916,7 +916,7 @@ func TestSNativeCALL(t *testing.T) { doug.MutablePermissions().Base.Set(permission.Call, true) //doug.Permissions.Base.Set(permission.HasBase, true) - batchCommitter.blockCache.UpdateAccount(doug) + batchCommitter.stateCache.UpdateAccount(doug) fmt.Println("\n#### HasBase") // HasBase @@ -1045,13 +1045,13 @@ func TestSNativeTx(t *testing.T) { snativeArgs := snativePermTestInputTx("setBase", users[3], permission.Bond, false) testSNativeTxExpectFail(t, batchCommitter, snativeArgs) testSNativeTxExpectPass(t, batchCommitter, permission.SetBase, snativeArgs) - acc := getAccount(batchCommitter.blockCache, users[3].Address()) + acc := getAccount(batchCommitter.stateCache, users[3].Address()) if v, _ := acc.MutablePermissions().Base.Get(permission.Bond); v { t.Fatal("expected permission to be set false") } snativeArgs = snativePermTestInputTx("setBase", users[3], permission.CreateContract, true) testSNativeTxExpectPass(t, batchCommitter, permission.SetBase, snativeArgs) - acc = getAccount(batchCommitter.blockCache, users[3].Address()) + acc = getAccount(batchCommitter.stateCache, users[3].Address()) if v, _ := acc.MutablePermissions().Base.Get(permission.CreateContract); !v { t.Fatal("expected permission to be set true") } @@ -1061,7 +1061,7 @@ func TestSNativeTx(t *testing.T) { snativeArgs = snativePermTestInputTx("unsetBase", users[3], permission.CreateContract, false) testSNativeTxExpectFail(t, batchCommitter, snativeArgs) testSNativeTxExpectPass(t, batchCommitter, permission.UnsetBase, snativeArgs) - acc = getAccount(batchCommitter.blockCache, users[3].Address()) + acc = getAccount(batchCommitter.stateCache, users[3].Address()) if v, _ := acc.MutablePermissions().Base.Get(permission.CreateContract); v { t.Fatal("expected permission to be set false") } @@ -1071,7 +1071,7 @@ func TestSNativeTx(t *testing.T) { snativeArgs = snativePermTestInputTx("setGlobal", users[3], permission.CreateContract, true) testSNativeTxExpectFail(t, batchCommitter, snativeArgs) testSNativeTxExpectPass(t, batchCommitter, permission.SetGlobal, snativeArgs) - acc = getAccount(batchCommitter.blockCache, permission.GlobalPermissionsAddress) + acc = getAccount(batchCommitter.stateCache, permission.GlobalPermissionsAddress) if v, _ := acc.MutablePermissions().Base.Get(permission.CreateContract); !v { t.Fatal("expected permission to be set true") } @@ -1081,7 +1081,7 @@ func TestSNativeTx(t *testing.T) { snativeArgs = snativeRoleTestInputTx("addRole", users[3], "chuck") testSNativeTxExpectFail(t, batchCommitter, snativeArgs) testSNativeTxExpectPass(t, batchCommitter, permission.AddRole, snativeArgs) - acc = getAccount(batchCommitter.blockCache, users[3].Address()) + acc = getAccount(batchCommitter.stateCache, users[3].Address()) if v := acc.Permissions().HasRole("chuck"); !v { t.Fatal("expected role to be added") } @@ -1091,7 +1091,7 @@ func TestSNativeTx(t *testing.T) { snativeArgs = snativeRoleTestInputTx("removeRole", users[3], "chuck") testSNativeTxExpectFail(t, batchCommitter, snativeArgs) testSNativeTxExpectPass(t, batchCommitter, permission.RemoveRole, snativeArgs) - acc = getAccount(batchCommitter.blockCache, users[3].Address()) + acc = getAccount(batchCommitter.stateCache, users[3].Address()) if v := acc.Permissions().HasRole("chuck"); v { t.Fatal("expected role to be removed") } @@ -1157,8 +1157,8 @@ func testSNativeCALL(t *testing.T, expectPass bool, batchCommitter *executor, do doug.SetCode(callContractCode(snativeAddress)) dougAddress := doug.Address() - batchCommitter.blockCache.UpdateAccount(doug) - tx, _ := txs.NewCallTx(batchCommitter.blockCache, users[0].PublicKey(), &dougAddress, data, 100, 10000, 100) + batchCommitter.stateCache.UpdateAccount(doug) + tx, _ := txs.NewCallTx(batchCommitter.stateCache, users[0].PublicKey(), &dougAddress, data, 100, 10000, 100) tx.Sign(testChainID, users[0]) fmt.Println("subscribing to", evm_events.EventStringAccountCall(snativeAddress)) ev, exception := execTxWaitEvent(t, batchCommitter, tx, evm_events.EventStringAccountCall(snativeAddress)) @@ -1191,11 +1191,11 @@ func testSNativeTxExpectPass(t *testing.T, batchCommitter *executor, perm ptypes func testSNativeTx(t *testing.T, expectPass bool, batchCommitter *executor, perm ptypes.PermFlag, snativeArgs permission.PermArgs) { if expectPass { - acc := getAccount(batchCommitter.blockCache, users[0].Address()) + acc := getAccount(batchCommitter.stateCache, users[0].Address()) acc.MutablePermissions().Base.Set(perm, true) - batchCommitter.blockCache.UpdateAccount(acc) + batchCommitter.stateCache.UpdateAccount(acc) } - tx, _ := txs.NewPermissionsTx(batchCommitter.blockCache, users[0].PublicKey(), snativeArgs) + tx, _ := txs.NewPermissionsTx(batchCommitter.stateCache, users[0].PublicKey(), snativeArgs) tx.Sign(testChainID, users[0]) err := batchCommitter.Execute(tx) if expectPass { diff --git a/execution/namereg.go b/execution/namereg.go index 4524882bf019942f03258eaa0aa31d4ec14aad0e..50babf2a15b60ae8f6d9143cbc3284c1bed9afd8 100644 --- a/execution/namereg.go +++ b/execution/namereg.go @@ -18,19 +18,34 @@ import "github.com/hyperledger/burrow/account" // NameReg provides a global key value store based on Name, Data pairs that are subject to expiry and ownership by an // account. +type NameRegEntry struct { + // registered name for the entry + Name string + // address that created the entry + Owner account.Address + // data to store under this name + Data string + // block at which this entry expires + Expires uint64 +} type NameRegGetter interface { - GetNameRegEntry(name string) *NameRegEntry + GetNameRegEntry(name string) (*NameRegEntry, error) } -type NameRegIterable interface { +type NameRegUpdater interface { + // Updates the name entry creating it if it does not exist + UpdateNameRegEntry(entry *NameRegEntry) error + // Remove the name entry + RemoveNameRegEntry(name string) error +} + +type NameRegWriter interface { NameRegGetter - IterateNameRegEntries(consumer func(*NameRegEntry) (stop bool)) (stopped bool) + NameRegUpdater } -type NameRegEntry struct { - Name string // registered name for the entry - Owner account.Address // address that created the entry - Data string // data to store under this name - Expires uint64 // block at which this entry expires +type NameRegIterable interface { + NameRegGetter + IterateNameRegEntries(consumer func(*NameRegEntry) (stop bool)) (stopped bool, err error) } diff --git a/execution/namereg_cache.go b/execution/namereg_cache.go new file mode 100644 index 0000000000000000000000000000000000000000..f08224741b0aa4a1f1b1d9a765f90e5b5ad5c0d0 --- /dev/null +++ b/execution/namereg_cache.go @@ -0,0 +1,163 @@ +// 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 ( + "fmt" + "sort" + "sync" +) + +// The NameRegCache helps prevent unnecessary IAVLTree updates and garbage generation. +type NameRegCache struct { + sync.RWMutex + backend NameRegGetter + names map[string]*nameInfo +} + +type nameInfo struct { + sync.RWMutex + entry *NameRegEntry + removed bool + updated bool +} + +var _ NameRegWriter = &NameRegCache{} + +// Returns a NameRegCache that wraps an underlying NameRegCacheGetter to use on a cache miss, can write to an +// output NameRegWriter via Sync. +// Not goroutine safe, use syncStateCache if you need concurrent access +func NewNameRegCache(backend NameRegGetter) *NameRegCache { + return &NameRegCache{ + backend: backend, + names: make(map[string]*nameInfo), + } +} + +func (cache *NameRegCache) GetNameRegEntry(name string) (*NameRegEntry, error) { + nameInfo, err := cache.get(name) + if err != nil { + return nil, err + } + nameInfo.RLock() + defer nameInfo.RUnlock() + if nameInfo.removed { + return nil, nil + } + return nameInfo.entry, nil +} + +func (cache *NameRegCache) UpdateNameRegEntry(entry *NameRegEntry) error { + nameInfo, err := cache.get(entry.Name) + if err != nil { + return err + } + nameInfo.Lock() + defer nameInfo.Unlock() + if nameInfo.removed { + return fmt.Errorf("UpdateNameRegEntry on a removed name: %s", nameInfo.entry.Name) + } + + nameInfo.entry = entry + nameInfo.updated = true + return nil +} + +func (cache *NameRegCache) RemoveNameRegEntry(name string) error { + nameInfo, err := cache.get(name) + if err != nil { + return err + } + nameInfo.Lock() + defer nameInfo.Unlock() + if nameInfo.removed { + return fmt.Errorf("RemoveNameRegEntry on removed name: %s", name) + } + nameInfo.removed = true + return nil +} + +// Writes whatever is in the cache to the output NameRegWriter state. Does not flush the cache, to do that call Reset() +// after Sync or use Flusth if your wish to use the output state as your next backend +func (cache *NameRegCache) Sync(state NameRegWriter) error { + cache.Lock() + defer cache.Unlock() + // Determine order for names + // note names may be of any length less than some limit + var names sort.StringSlice + for nameStr := range cache.names { + names = append(names, nameStr) + } + sort.Stable(names) + + // Update or delete names. + for _, name := range names { + nameInfo := cache.names[name] + nameInfo.RLock() + if nameInfo.removed { + err := state.RemoveNameRegEntry(name) + if err != nil { + return err + } + } else if nameInfo.updated { + err := state.UpdateNameRegEntry(nameInfo.entry) + if err != nil { + return err + } + } + nameInfo.RUnlock() + } + return nil +} + +// Resets the cache to empty initialising the backing map to the same size as the previous iteration. +func (cache *NameRegCache) Reset(backend NameRegGetter) { + cache.backend = backend + cache.names = make(map[string]*nameInfo) +} + +// Syncs the NameRegCache and Resets it to use NameRegWriter as the backend NameRegGetter +func (cache *NameRegCache) Flush(state NameRegWriter) error { + err := cache.Sync(state) + if err != nil { + return err + } + cache.Reset(state) + return nil +} + +func (cache *NameRegCache) Backend() NameRegGetter { + return cache.backend +} + +// Get the cache accountInfo item creating it if necessary +func (cache *NameRegCache) get(name string) (*nameInfo, error) { + cache.RLock() + nmeInfo := cache.names[name] + cache.RUnlock() + if nmeInfo == nil { + entry, err := cache.backend.GetNameRegEntry(name) + if err != nil { + return nil, err + } + nmeInfo = &nameInfo{ + entry: entry, + } + cache.Lock() + cache.names[name] = nmeInfo + cache.Unlock() + } + return nmeInfo, nil +} diff --git a/execution/namereg_cache_test.go b/execution/namereg_cache_test.go new file mode 100644 index 0000000000000000000000000000000000000000..371bf21e9dbdf24b823e574e11e53be8b744285d --- /dev/null +++ b/execution/namereg_cache_test.go @@ -0,0 +1,38 @@ +// 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 ( + "testing" +) + +func TestNewNameRegCache(t *testing.T) { +} + +func TestNameRegCache_GetNameRegEntry(t *testing.T) { +} + +func TestNameRegCache_UpdateNameRegEntry(t *testing.T) { +} + +func TestNameRegCache_RemoveNameRegEntry(t *testing.T) { +} + +func TestNameRegCache_Sync(t *testing.T) { + +} + +func TestNameRegCache_get(t *testing.T) { +} diff --git a/execution/state.go b/execution/state.go index 4564c146bec3016c6bf958b124ebb237a57ebc78..c9917f6c989eb7a3b673cf04e5bdfff41410b666 100644 --- a/execution/state.go +++ b/execution/state.go @@ -371,19 +371,19 @@ func (s *State) IterateStorage(address acm.Address, var _ NameRegIterable = &State{} -func (s *State) GetNameRegEntry(name string) *NameRegEntry { +func (s *State) GetNameRegEntry(name string) (*NameRegEntry, error) { _, valueBytes, _ := s.nameReg.Get([]byte(name)) if valueBytes == nil { - return nil + return nil, nil } - return DecodeNameRegEntry(valueBytes) + return DecodeNameRegEntry(valueBytes), nil } -func (s *State) IterateNameRegEntries(consumer func(*NameRegEntry) (stop bool)) (stopped bool) { +func (s *State) IterateNameRegEntries(consumer func(*NameRegEntry) (stop bool)) (stopped bool, err error) { return s.nameReg.Iterate(func(key []byte, value []byte) (stop bool) { return consumer(DecodeNameRegEntry(value)) - }) + }), nil } func DecodeNameRegEntry(entryBytes []byte) *NameRegEntry { @@ -393,23 +393,23 @@ func DecodeNameRegEntry(entryBytes []byte) *NameRegEntry { return value.(*NameRegEntry) } -func (s *State) UpdateNameRegEntry(entry *NameRegEntry) bool { +func (s *State) UpdateNameRegEntry(entry *NameRegEntry) error { w := new(bytes.Buffer) var n int var err error NameRegEncode(entry, w, &n, &err) - return s.nameReg.Set([]byte(entry.Name), w.Bytes()) + if err != nil { + return err + } + s.nameReg.Set([]byte(entry.Name), w.Bytes()) + return nil } -func (s *State) RemoveNameRegEntry(name string) bool { - _, removed := s.nameReg.Remove([]byte(name)) - return removed +func (s *State) RemoveNameRegEntry(name string) error { + s.nameReg.Remove([]byte(name)) + return nil } -// Set the name reg tree -func (s *State) SetNameReg(nameReg merkle.Tree) { - s.nameReg = nameReg -} func NameRegEncode(o interface{}, w io.Writer, n *int, err *error) { wire.WriteBinary(o.(*NameRegEntry), w, n, err) } diff --git a/execution/state_test.go b/execution/state_test.go index 9a681bbee77935246906378566cee8ae628381db..e1844a998665190c41681964705672c334c5cd75 100644 --- a/execution/state_test.go +++ b/execution/state_test.go @@ -46,7 +46,7 @@ func execTxWithStateAndBlockchain(state *State, tip bcm.Tip, tx txs.Tx) error { if err := exe.Execute(tx); err != nil { return err } else { - exe.blockCache.Sync() + exe.stateCache.Flush(exe.state) return nil } } @@ -346,7 +346,8 @@ func TestNameTxs(t *testing.T) { if err := execTxWithState(state, tx); err != nil { t.Fatal(err) } - entry := state.GetNameRegEntry(name) + entry, err := state.GetNameRegEntry(name) + require.NoError(t, err) validateEntry(t, entry, name, data, testPrivAccounts[0].Address(), startingBlock+numDesiredBlocks) // fail to update it as non-owner, in same block @@ -363,7 +364,8 @@ func TestNameTxs(t *testing.T) { if err := execTxWithStateNewBlock(state, blockchain, tx); err != nil { t.Fatal(err) } - entry = state.GetNameRegEntry(name) + entry, err = state.GetNameRegEntry(name) + require.NoError(t, err) validateEntry(t, entry, name, data, testPrivAccounts[0].Address(), startingBlock+numDesiredBlocks*2) // update it as owner, just to increase expiry, in next block @@ -372,7 +374,8 @@ func TestNameTxs(t *testing.T) { if err := execTxWithStateNewBlock(state, blockchain, tx); err != nil { t.Fatal(err) } - entry = state.GetNameRegEntry(name) + entry, err = state.GetNameRegEntry(name) + require.NoError(t, err) validateEntry(t, entry, name, data, testPrivAccounts[0].Address(), startingBlock+numDesiredBlocks*3) // fail to update it as non-owner @@ -393,7 +396,8 @@ func TestNameTxs(t *testing.T) { if err := execTxWithStateAndBlockchain(state, blockchain, tx); err != nil { t.Fatal(err) } - entry = state.GetNameRegEntry(name) + entry, err = state.GetNameRegEntry(name) + require.NoError(t, err) validateEntry(t, entry, name, data, testPrivAccounts[1].Address(), blockchain.LastBlockHeight()+numDesiredBlocks) // update it as new owner, with new data (longer), but keep the expiry! @@ -406,7 +410,8 @@ func TestNameTxs(t *testing.T) { if err := execTxWithStateAndBlockchain(state, blockchain, tx); err != nil { t.Fatal(err) } - entry = state.GetNameRegEntry(name) + entry, err = state.GetNameRegEntry(name) + require.NoError(t, err) validateEntry(t, entry, name, data, testPrivAccounts[1].Address(), blockchain.LastBlockHeight()+numDesiredBlocks) // test removal @@ -417,7 +422,8 @@ func TestNameTxs(t *testing.T) { if err := execTxWithStateNewBlock(state, blockchain, tx); err != nil { t.Fatal(err) } - entry = state.GetNameRegEntry(name) + entry, err = state.GetNameRegEntry(name) + require.NoError(t, err) if entry != nil { t.Fatal("Expected removed entry to be nil") } @@ -432,7 +438,8 @@ func TestNameTxs(t *testing.T) { if err := execTxWithStateAndBlockchain(state, blockchain, tx); err != nil { t.Fatal(err) } - entry = state.GetNameRegEntry(name) + entry, err = state.GetNameRegEntry(name) + require.NoError(t, err) validateEntry(t, entry, name, data, testPrivAccounts[0].Address(), blockchain.LastBlockHeight()+numDesiredBlocks) // Fast forward for blockchain.Tip().LastBlockHeight() < entry.Expires { @@ -446,7 +453,8 @@ func TestNameTxs(t *testing.T) { if err := execTxWithStateNewBlock(state, blockchain, tx); err != nil { t.Fatal(err) } - entry = state.GetNameRegEntry(name) + entry, err = state.GetNameRegEntry(name) + require.NoError(t, err) if entry != nil { t.Fatal("Expected removed entry to be nil") } @@ -794,7 +802,8 @@ proof-of-work chain as proof of what happened while they were gone ` t.Errorf("Unexpected newAcc0 balance. Expected %v, got %v", acc0.Balance()-entryAmount, newAcc0.Balance()) } - entry := stateNameTx.GetNameRegEntry(entryName) + entry, err := stateNameTx.GetNameRegEntry(entryName) + require.NoError(t, err) if entry == nil { t.Errorf("Expected an entry but got nil") } diff --git a/execution/transactor.go b/execution/transactor.go index e4642112a3d5cc1cc8102200aee34477915f3d1b..eda8d5f33c5885716b200922e75934fecdf656b7 100644 --- a/execution/transactor.go +++ b/execution/transactor.go @@ -60,7 +60,7 @@ type Transactor interface { type transactor struct { txMtx sync.Mutex blockchain blockchain.Blockchain - state acm.StateReader + state acm.StateIterable eventEmitter event.Emitter broadcastTxAsync func(tx txs.Tx, callback func(res *abci_types.Response)) error logger logging_types.InfoTraceLogger @@ -68,7 +68,7 @@ type transactor struct { var _ Transactor = &transactor{} -func NewTransactor(blockchain blockchain.Blockchain, state acm.StateReader, eventEmitter event.Emitter, +func NewTransactor(blockchain blockchain.Blockchain, state acm.StateIterable, eventEmitter event.Emitter, broadcastTxAsync func(tx txs.Tx, callback func(res *abci_types.Response)) error, logger logging_types.InfoTraceLogger) *transactor { diff --git a/rpc/service.go b/rpc/service.go index 517e898bda2d22e83674f02d8cdf6a29c5b37454..d84c7b70b71458ecad30bbf662c97afa378ef886 100644 --- a/rpc/service.go +++ b/rpc/service.go @@ -302,7 +302,10 @@ func (s *service) DumpStorage(address acm.Address) (*ResultDumpStorage, error) { // Name registry func (s *service) GetName(name string) (*ResultGetName, error) { - entry := s.nameReg.GetNameRegEntry(name) + entry, err := s.nameReg.GetNameRegEntry(name) + if err != nil { + return nil, err + } if entry == nil { return nil, fmt.Errorf("name %s not found", name) }