minimal state backend trait

make state module public
This commit is contained in:
Robert Habermeier 2017-02-21 12:35:21 +01:00
parent ac9716a5ce
commit 1fba73c15e
11 changed files with 167 additions and 117 deletions

View File

@ -91,7 +91,7 @@ pub struct ExecutedBlock {
uncles: Vec<Header>,
receipts: Vec<Receipt>,
transactions_set: HashSet<H256>,
state: State,
state: State<StateDB>,
traces: Option<Vec<Vec<FlatTrace>>>,
}
@ -106,7 +106,7 @@ pub struct BlockRefMut<'a> {
/// Transaction receipts.
pub receipts: &'a [Receipt],
/// State.
pub state: &'a mut State,
pub state: &'a mut State<StateDB>,
/// Traces.
pub traces: &'a Option<Vec<Vec<FlatTrace>>>,
}
@ -122,14 +122,14 @@ pub struct BlockRef<'a> {
/// Transaction receipts.
pub receipts: &'a [Receipt],
/// State.
pub state: &'a State,
pub state: &'a State<StateDB>,
/// Traces.
pub traces: &'a Option<Vec<Vec<FlatTrace>>>,
}
impl ExecutedBlock {
/// Create a new block from the given `state`.
fn new(state: State, tracing: bool) -> ExecutedBlock {
fn new(state: State<StateDB>, tracing: bool) -> ExecutedBlock {
ExecutedBlock {
header: Default::default(),
transactions: Default::default(),
@ -184,7 +184,7 @@ pub trait IsBlock {
fn header(&self) -> &Header { &self.block().header }
/// Get the final state associated with this object's block.
fn state(&self) -> &State { &self.block().state }
fn state(&self) -> &State<StateDB> { &self.block().state }
/// Get all information on transactions in this block.
fn transactions(&self) -> &[SignedTransaction] { &self.block().transactions }
@ -228,7 +228,7 @@ pub struct ClosedBlock {
block: ExecutedBlock,
uncle_bytes: Bytes,
last_hashes: Arc<LastHashes>,
unclosed_state: State,
unclosed_state: State<StateDB>,
}
/// Just like `ClosedBlock` except that we can't reopen it and it's faster.

View File

@ -656,7 +656,7 @@ impl Client {
/// This will not fail if given BlockId::Latest.
/// Otherwise, this can fail (but may not) if the DB prunes state or the block
/// is unknown.
pub fn state_at(&self, id: BlockId) -> Option<State> {
pub fn state_at(&self, id: BlockId) -> Option<State<StateDB>> {
// fast path for latest state.
match id.clone() {
BlockId::Pending => return self.miner.pending_state().or_else(|| Some(self.state())),
@ -686,7 +686,7 @@ impl Client {
///
/// This will not fail if given BlockId::Latest.
/// Otherwise, this can fail (but may not) if the DB prunes state.
pub fn state_at_beginning(&self, id: BlockId) -> Option<State> {
pub fn state_at_beginning(&self, id: BlockId) -> Option<State<StateDB>> {
// fast path for latest state.
match id {
BlockId::Pending => self.state_at(BlockId::Latest),
@ -698,7 +698,7 @@ impl Client {
}
/// Get a copy of the best block's state.
pub fn state(&self) -> State {
pub fn state(&self) -> State<StateDB> {
let header = self.best_block_header();
State::from_existing(
self.state_db.lock().boxed_clone_canon(&header.hash()),

View File

@ -17,7 +17,7 @@
//! Transaction Execution environment.
use util::*;
use action_params::{ActionParams, ActionValue};
use state::{State, Substate, CleanupMode};
use state::{Backend as StateBackend, State, Substate, CleanupMode};
use engines::Engine;
use types::executed::CallType;
use env_info::EnvInfo;
@ -56,17 +56,17 @@ pub struct TransactOptions {
}
/// Transaction executor.
pub struct Executive<'a> {
state: &'a mut State,
pub struct Executive<'a, B: 'a + StateBackend> {
state: &'a mut State<B>,
info: &'a EnvInfo,
engine: &'a Engine,
vm_factory: &'a Factory,
depth: usize,
}
impl<'a> Executive<'a> {
impl<'a, B: 'a + StateBackend> Executive<'a, B> {
/// Basic constructor.
pub fn new(state: &'a mut State, info: &'a EnvInfo, engine: &'a Engine, vm_factory: &'a Factory) -> Self {
pub fn new(state: &'a mut State<B>, info: &'a EnvInfo, engine: &'a Engine, vm_factory: &'a Factory) -> Self {
Executive {
state: state,
info: info,
@ -77,7 +77,7 @@ impl<'a> Executive<'a> {
}
/// Populates executive from parent properties. Increments executive depth.
pub fn from_parent(state: &'a mut State, info: &'a EnvInfo, engine: &'a Engine, vm_factory: &'a Factory, parent_depth: usize) -> Self {
pub fn from_parent(state: &'a mut State<B>, info: &'a EnvInfo, engine: &'a Engine, vm_factory: &'a Factory, parent_depth: usize) -> Self {
Executive {
state: state,
info: info,
@ -95,7 +95,7 @@ impl<'a> Executive<'a> {
output: OutputPolicy<'any, 'any>,
tracer: &'any mut T,
vm_tracer: &'any mut V
) -> Externalities<'any, T, V> where T: Tracer, V: VMTracer {
) -> Externalities<'any, T, V, B> where T: Tracer, V: VMTracer {
Externalities::new(self.state, self.info, self.engine, self.vm_factory, self.depth, origin_info, substate, output, tracer, vm_tracer)
}

View File

@ -17,7 +17,7 @@
//! Transaction Execution environment.
use util::*;
use action_params::{ActionParams, ActionValue};
use state::{State, Substate};
use state::{Backend as StateBackend, State, Substate};
use engines::Engine;
use env_info::EnvInfo;
use executive::*;
@ -57,8 +57,10 @@ impl OriginInfo {
}
/// Implementation of evm Externalities.
pub struct Externalities<'a, T, V> where T: 'a + Tracer, V: 'a + VMTracer {
state: &'a mut State,
pub struct Externalities<'a, T: 'a, V: 'a, B: 'a>
where T: Tracer, V: VMTracer, B: StateBackend
{
state: &'a mut State<B>,
env_info: &'a EnvInfo,
engine: &'a Engine,
vm_factory: &'a Factory,
@ -71,10 +73,12 @@ pub struct Externalities<'a, T, V> where T: 'a + Tracer, V: 'a + VMTracer {
vm_tracer: &'a mut V,
}
impl<'a, T, V> Externalities<'a, T, V> where T: 'a + Tracer, V: 'a + VMTracer {
#[cfg_attr(feature="dev", allow(too_many_arguments))]
impl<'a, T: 'a, V: 'a, B: 'a> Externalities<'a, T, V, B>
where T: Tracer, V: VMTracer, B: StateBackend
{
/// Basic `Externalities` constructor.
pub fn new(state: &'a mut State,
#[cfg_attr(feature="dev", allow(too_many_arguments))]
pub fn new(state: &'a mut State<B>,
env_info: &'a EnvInfo,
engine: &'a Engine,
vm_factory: &'a Factory,
@ -101,7 +105,9 @@ impl<'a, T, V> Externalities<'a, T, V> where T: 'a + Tracer, V: 'a + VMTracer {
}
}
impl<'a, T, V> Ext for Externalities<'a, T, V> where T: 'a + Tracer, V: 'a + VMTracer {
impl<'a, T: 'a, V: 'a, B: 'a> Ext for Externalities<'a, T, V, B>
where T: Tracer, V: VMTracer, B: StateBackend
{
fn storage_at(&self, key: &H256) -> H256 {
self.state.storage_at(&self.origin_info.address, key)
}
@ -346,7 +352,7 @@ mod tests {
}
struct TestSetup {
state: GuardedTempResult<State>,
state: GuardedTempResult<State<::state_db::StateDB>>,
engine: Arc<Engine>,
sub_state: Substate,
env_info: EnvInfo

View File

@ -139,6 +139,7 @@ pub mod snapshot;
pub mod action_params;
pub mod db;
pub mod verification;
pub mod state;
#[macro_use] pub mod evm;
mod cache_manager;
@ -146,7 +147,6 @@ mod blooms;
mod basic_types;
mod env_info;
mod pod_account;
mod state;
mod state_db;
mod account_db;
mod builtin;

View File

@ -307,7 +307,7 @@ impl Miner {
}
/// Get `Some` `clone()` of the current pending block's state or `None` if we're not sealing.
pub fn pending_state(&self) -> Option<State> {
pub fn pending_state(&self) -> Option<State<::state_db::StateDB>> {
self.sealing_work.lock().queue.peek_last_ref().map(|b| b.block().fields().state.clone())
}

View File

@ -24,7 +24,7 @@ use executive::Executive;
use trace::{NoopTracer, NoopVMTracer};
use action_params::{ActionValue, ActionParams};
use types::executed::CallType;
use state::{State, Substate};
use state::{Backend, State, Substate};
use env_info::EnvInfo;
use pod_state::*;
use account_db::*;

View File

@ -24,10 +24,10 @@
use std::sync::Arc;
use state::Account;
use util::{Address, HashDB, H256};
use util::{Address, AsHashDB, HashDB, H256};
/// State backend. See module docs for more details.
pub trait Backend {
pub trait Backend: Send {
/// Treat the backend as a read-only hashdb.
fn as_hashdb(&self) -> &HashDB;
@ -42,7 +42,7 @@ pub trait Backend {
/// hash collisions.
fn cache_code(&self, hash: H256, code: Arc<Vec<u8>>);
/// Get basic copy of the cached account. Does not include storage.
/// Get basic copy of the cached account. Not required to include storage.
/// Returns 'None' if cache is disabled or if the account is not cached.
fn get_cached_account(&self, addr: &Address) -> Option<Option<Account>>;
@ -55,4 +55,39 @@ pub trait Backend {
/// Get cached code based on hash.
fn get_cached_code(&self, hash: &H256) -> Option<Arc<Vec<u8>>>;
/// Note that an account with the given address is non-null.
fn note_non_null_account(&self, address: &Address);
/// Check whether an account is known to be empty. Returns true if known to be
/// empty, false otherwise.
fn is_known_null(&self, address: &Address) -> bool;
}
/// A raw backend which simply wraps a hashdb and does no caching.
#[derive(Debug, Clone, PartialEq, Eq)]
pub struct NoCache<T>(T);
impl<T> NoCache<T> {
/// Create a new `NoCache` backend.
pub fn new(inner: T) -> Self { NoCache(inner) }
/// Consume the backend, yielding the inner database.
pub fn into_inner(self) -> T { self.0 }
}
impl<T: AsHashDB + Send> Backend for NoCache<T> {
fn as_hashdb(&self) -> &HashDB { self.0.as_hashdb() }
fn as_hashdb_mut(&mut self) -> &mut HashDB { self.0.as_hashdb_mut() }
fn add_to_account_cache(&mut self, _addr: Address, _data: Option<Account>, _modified: bool) {}
fn cache_code(&self, _hash: H256, _code: Arc<Vec<u8>>) {}
fn get_cached_account(&self, _addr: &Address) -> Option<Option<Account>> { None }
fn get_cached<F, U>(&self, _a: &Address, _f: F) -> Option<U>
where F: FnOnce(Option<&mut Account>) -> U
{
None
}
fn get_cached_code(&self, _hash: &H256) -> Option<Arc<Vec<u8>>> { None }
fn note_non_null_account(&self, _address: &Address) {}
fn is_known_null(&self, _address: &Address) -> bool { false }
}

View File

@ -14,6 +14,11 @@
// You should have received a copy of the GNU General Public License
// along with Parity. If not, see <http://www.gnu.org/licenses/>.
//! A mutable state representation suitable to execute transactions.
//! Generic over a `Backend`. Deals with `Account`s.
//! Unconfirmed sub-states are managed with `checkpoint`s which may be canonicalized
//! or rolled back.
use std::cell::{RefCell, RefMut};
use std::collections::hash_map::Entry;
@ -40,6 +45,7 @@ mod substate;
pub mod backend;
pub use self::account::Account;
pub use self::backend::Backend;
pub use self::substate::Substate;
/// Used to return information about an `State::apply` operation.
@ -188,8 +194,8 @@ impl AccountEntry {
/// checkpoint can be discateded with `discard_checkpoint`. All of the orignal
/// backed-up values are moved into a parent checkpoint (if any).
///
pub struct State {
db: StateDB,
pub struct State<B> {
db: B,
root: H256,
cache: RefCell<HashMap<Address, AccountEntry>>,
// The original account is preserved in
@ -205,20 +211,24 @@ enum RequireCache {
Code,
}
/// Mode of dealing with null accounts.
#[derive(PartialEq)]
pub enum CleanupMode<'a> {
/// Create accounts which would be null.
ForceCreate,
/// Don't delete null accounts upon touching, but also don't create them.
NoEmpty,
/// Add encountered null accounts to the provided kill-set, to be deleted later.
KillEmpty(&'a mut HashSet<Address>),
}
const SEC_TRIE_DB_UNWRAP_STR: &'static str = "A state can only be created with valid root. Creating a SecTrieDB with a valid root will not fail. \
Therefore creating a SecTrieDB with this state's root will not fail.";
impl State {
impl<B: Backend> State<B> {
/// Creates new state with empty state root
#[cfg(test)]
pub fn new(mut db: StateDB, account_start_nonce: U256, factories: Factories) -> State {
pub fn new(mut db: B, account_start_nonce: U256, factories: Factories) -> State<B> {
let mut root = H256::new();
{
// init trie and reset root too null
@ -236,7 +246,7 @@ impl State {
}
/// Creates new state with existing state root
pub fn from_existing(db: StateDB, root: H256, account_start_nonce: U256, factories: Factories) -> Result<State, TrieError> {
pub fn from_existing(db: B, root: H256, account_start_nonce: U256, factories: Factories) -> Result<State<B>, TrieError> {
if !db.as_hashdb().contains(&root) {
return Err(TrieError::InvalidStateRoot(root));
}
@ -330,7 +340,7 @@ impl State {
}
/// Destroy the current object and return root and database.
pub fn drop(mut self) -> (H256, StateDB) {
pub fn drop(mut self) -> (H256, B) {
self.propagate_to_global_cache();
(self.root, self.db)
}
@ -422,8 +432,8 @@ impl State {
}
}
// check bloom before any requests to trie
if !self.db.check_non_null_bloom(address) { return H256::zero() }
// check if the account could exist before any requests to trie
if self.db.is_known_null(address) { return H256::zero() }
// account is not found in the global cache, get from the DB and insert into local
let db = self.factories.trie.readonly(self.db.as_hashdb(), &self.root).expect(SEC_TRIE_DB_UNWRAP_STR);
@ -445,6 +455,7 @@ impl State {
|a| a.as_ref().map_or(None, |a| a.code().clone()))
}
/// Get an account's code hash.
pub fn code_hash(&self, a: &Address) -> H256 {
self.ensure_cached(a, RequireCache::None, true,
|a| a.as_ref().map_or(SHA3_EMPTY, |a| a.code_hash()))
@ -538,7 +549,7 @@ impl State {
#[cfg_attr(feature="dev", allow(needless_borrow))]
fn commit_into(
factories: &Factories,
db: &mut StateDB,
db: &mut B,
root: &mut H256,
accounts: &mut HashMap<Address, AccountEntry>
) -> Result<(), Error> {
@ -632,7 +643,7 @@ impl State {
/// Returns a `StateDiff` describing the difference from `orig` to `self`.
/// Consumes self.
pub fn diff_from(&self, orig: State) -> StateDiff {
pub fn diff_from<X: Backend>(&self, orig: State<X>) -> StateDiff {
let pod_state_post = self.to_pod();
let mut state_pre = orig;
state_pre.query_pod(&pod_state_post);
@ -640,7 +651,7 @@ impl State {
}
// load required account data from the databases.
fn update_account_cache(require: RequireCache, account: &mut Account, state_db: &StateDB, db: &HashDB) {
fn update_account_cache(require: RequireCache, account: &mut Account, state_db: &B, db: &HashDB) {
match (account.is_cached(), require) {
(true, _) | (false, RequireCache::None) => {}
(false, require) => {
@ -670,7 +681,7 @@ impl State {
/// Check caches for required data
/// First searches for account in the local, then the shared cache.
/// Populates local cache if nothing found.
fn ensure_cached<F, U>(&self, a: &Address, require: RequireCache, check_bloom: bool, f: F) -> U
fn ensure_cached<F, U>(&self, a: &Address, require: RequireCache, check_null: bool, f: F) -> U
where F: Fn(Option<&Account>) -> U {
// check local cache first
if let Some(ref mut maybe_acc) = self.cache.borrow_mut().get_mut(a) {
@ -692,8 +703,8 @@ impl State {
match result {
Some(r) => r,
None => {
// first check bloom if it is not in database for sure
if check_bloom && !self.db.check_non_null_bloom(a) { return f(None); }
// first check if it is not in database for sure
if check_null && self.db.is_known_null(a) { return f(None); }
// not found in the global cache, get from the DB and insert into local
let db = self.factories.trie.readonly(self.db.as_hashdb(), &self.root).expect(SEC_TRIE_DB_UNWRAP_STR);
@ -727,7 +738,7 @@ impl State {
match self.db.get_cached_account(a) {
Some(acc) => self.insert_cache(a, AccountEntry::new_clean_cached(acc)),
None => {
let maybe_acc = if self.db.check_non_null_bloom(a) {
let maybe_acc = if !self.db.is_known_null(a) {
let db = self.factories.trie.readonly(self.db.as_hashdb(), &self.root).expect(SEC_TRIE_DB_UNWRAP_STR);
match db.get_with(a, Account::from_rlp) {
Ok(acc) => AccountEntry::new_clean(acc),
@ -769,7 +780,7 @@ impl State {
}
// LES state proof implementations.
impl State {
impl<B: Backend> State<B> {
/// Prove an account's existence or nonexistence in the state trie.
/// Returns a merkle proof of the account's trie node with all nodes before `from_level`
/// omitted or an encountered trie error.
@ -815,14 +826,16 @@ impl State {
}
}
impl fmt::Debug for State {
impl<B> fmt::Debug for State<B> {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
write!(f, "{:?}", self.cache.borrow())
}
}
impl Clone for State {
fn clone(&self) -> State {
// TODO: cloning for `State` shouldn't be possible in general; Remove this and use
// checkpoints where possible.
impl Clone for State<StateDB> {
fn clone(&self) -> State<StateDB> {
let cache = {
let mut cache: HashMap<Address, AccountEntry> = HashMap::new();
for (key, val) in self.cache.borrow().iter() {

View File

@ -21,7 +21,7 @@ use util::journaldb::JournalDB;
use util::kvdb::KeyValueDB;
use util::hash::{H256};
use util::hashdb::HashDB;
use state::Account;
use state::{self, Account};
use header::BlockNumber;
use util::{Arc, Address, DBTransaction, UtilError, Mutex, Hashable};
use bloom_journal::{Bloom, BloomJournal};
@ -166,18 +166,6 @@ impl StateDB {
bloom
}
pub fn check_non_null_bloom(&self, address: &Address) -> bool {
trace!(target: "account_bloom", "Check account bloom: {:?}", address);
let bloom = self.account_bloom.lock();
bloom.check(&*address.sha3())
}
pub fn note_non_null_account(&self, address: &Address) {
trace!(target: "account_bloom", "Note account bloom: {:?}", address);
let mut bloom = self.account_bloom.lock();
bloom.set(&*address.sha3());
}
pub fn commit_bloom(batch: &mut DBTransaction, journal: BloomJournal) -> Result<(), UtilError> {
assert!(journal.hash_functions <= 255);
batch.put(COL_ACCOUNT_BLOOM, ACCOUNT_BLOOM_HASHCOUNT_KEY, &[journal.hash_functions as u8]);
@ -306,12 +294,10 @@ impl StateDB {
}
}
/// Returns an interface to HashDB.
pub fn as_hashdb(&self) -> &HashDB {
self.db.as_hashdb()
}
/// Returns an interface to mutable HashDB.
pub fn as_hashdb_mut(&mut self) -> &mut HashDB {
self.db.as_hashdb_mut()
}
@ -366,56 +352,6 @@ impl StateDB {
&*self.db
}
/// Add a local cache entry.
/// The entry will be propagated to the global cache in `sync_cache`.
/// `modified` indicates that the entry was changed since being read from disk or global cache.
/// `data` can be set to an existing (`Some`), or non-existing account (`None`).
pub fn add_to_account_cache(&mut self, addr: Address, data: Option<Account>, modified: bool) {
self.local_cache.push(CacheQueueItem {
address: addr,
account: data,
modified: modified,
})
}
/// Add a global code cache entry. This doesn't need to worry about canonicality because
/// it simply maps hashes to raw code and will always be correct in the absence of
/// hash collisions.
pub fn cache_code(&self, hash: H256, code: Arc<Vec<u8>>) {
let mut cache = self.code_cache.lock();
cache.insert(hash, code);
}
/// Get basic copy of the cached account. Does not include storage.
/// Returns 'None' if cache is disabled or if the account is not cached.
pub fn get_cached_account(&self, addr: &Address) -> Option<Option<Account>> {
let mut cache = self.account_cache.lock();
if !Self::is_allowed(addr, &self.parent_hash, &cache.modifications) {
return None;
}
cache.accounts.get_mut(addr).map(|a| a.as_ref().map(|a| a.clone_basic()))
}
/// Get cached code based on hash.
#[cfg_attr(feature="dev", allow(map_clone))]
pub fn get_cached_code(&self, hash: &H256) -> Option<Arc<Vec<u8>>> {
let mut cache = self.code_cache.lock();
cache.get_mut(hash).map(|code| code.clone())
}
/// Get value from a cached account.
/// Returns 'None' if cache is disabled or if the account is not cached.
pub fn get_cached<F, U>(&self, a: &Address, f: F) -> Option<U>
where F: FnOnce(Option<&mut Account>) -> U {
let mut cache = self.account_cache.lock();
if !Self::is_allowed(a, &self.parent_hash, &cache.modifications) {
return None;
}
cache.accounts.get_mut(a).map(|c| f(c.as_mut()))
}
/// Query how much memory is set aside for the accounts cache (in bytes).
pub fn cache_size(&self) -> usize {
self.cache_size
@ -456,11 +392,71 @@ impl StateDB {
}
}
impl state::Backend for StateDB {
fn as_hashdb(&self) -> &HashDB {
self.db.as_hashdb()
}
fn as_hashdb_mut(&mut self) -> &mut HashDB {
self.db.as_hashdb_mut()
}
fn add_to_account_cache(&mut self, addr: Address, data: Option<Account>, modified: bool) {
self.local_cache.push(CacheQueueItem {
address: addr,
account: data,
modified: modified,
})
}
fn cache_code(&self, hash: H256, code: Arc<Vec<u8>>) {
let mut cache = self.code_cache.lock();
cache.insert(hash, code);
}
fn get_cached_account(&self, addr: &Address) -> Option<Option<Account>> {
let mut cache = self.account_cache.lock();
if !Self::is_allowed(addr, &self.parent_hash, &cache.modifications) {
return None;
}
cache.accounts.get_mut(addr).map(|a| a.as_ref().map(|a| a.clone_basic()))
}
#[cfg_attr(feature="dev", allow(map_clone))]
fn get_cached_code(&self, hash: &H256) -> Option<Arc<Vec<u8>>> {
let mut cache = self.code_cache.lock();
cache.get_mut(hash).map(|code| code.clone())
}
fn get_cached<F, U>(&self, a: &Address, f: F) -> Option<U>
where F: FnOnce(Option<&mut Account>) -> U {
let mut cache = self.account_cache.lock();
if !Self::is_allowed(a, &self.parent_hash, &cache.modifications) {
return None;
}
cache.accounts.get_mut(a).map(|c| f(c.as_mut()))
}
fn note_non_null_account(&self, address: &Address) {
trace!(target: "account_bloom", "Note account bloom: {:?}", address);
let mut bloom = self.account_bloom.lock();
bloom.set(&*address.sha3());
}
fn is_known_null(&self, address: &Address) -> bool {
trace!(target: "account_bloom", "Check account bloom: {:?}", address);
let bloom = self.account_bloom.lock();
!bloom.check(&*address.sha3())
}
}
#[cfg(test)]
mod tests {
use util::{U256, H256, FixedHash, Address, DBTransaction};
use tests::helpers::*;
use state::Account;
use state::{Account, Backend};
use util::log::init_log;
#[test]

View File

@ -349,7 +349,7 @@ pub fn get_temp_state_db() -> GuardedTempResult<StateDB> {
}
}
pub fn get_temp_state() -> GuardedTempResult<State> {
pub fn get_temp_state() -> GuardedTempResult<State<::state_db::StateDB>> {
let temp = RandomTempPath::new();
let journal_db = get_temp_state_db_in(temp.as_path());
@ -365,7 +365,7 @@ pub fn get_temp_state_db_in(path: &Path) -> StateDB {
StateDB::new(journal_db, 5 * 1024 * 1024)
}
pub fn get_temp_state_in(path: &Path) -> State {
pub fn get_temp_state_in(path: &Path) -> State<::state_db::StateDB> {
let journal_db = get_temp_state_db_in(path);
State::new(journal_db, U256::from(0), Default::default())
}