Remove RefCell from Header (#8227)

* Cache RLP and header hashes.

* Refactor header - WiP

* Avoid decoding laster header.

* Pre-compute hashes for Sealed/Locked block.

* Use accrue bloom. Closes ##8241
This commit is contained in:
Tomasz Drwięga 2018-04-03 10:01:28 +02:00 committed by Marek Kotewicz
parent d477670cb9
commit 9f775a7673
20 changed files with 300 additions and 233 deletions

View File

@ -17,8 +17,7 @@
//! Tests for the on-demand service.
use cache::Cache;
use ethcore::encoded;
use ethcore::header::{Header, Seal};
use ethcore::header::Header;
use futures::Future;
use network::{PeerId, NodeId};
use net::*;
@ -148,7 +147,7 @@ fn single_request() {
});
let header = Header::default();
let encoded = encoded::Header::new(header.rlp(Seal::With));
let encoded = header.encoded();
let recv = harness.service.request_raw(
&Context::NoOp,
@ -209,7 +208,7 @@ fn reassign() {
});
let header = Header::default();
let encoded = encoded::Header::new(header.rlp(Seal::With));
let encoded = header.encoded();
let recv = harness.service.request_raw(
&Context::NoOp,
@ -257,7 +256,7 @@ fn partial_response() {
let mut hdr = Header::default();
hdr.set_number(num);
let encoded = encoded::Header::new(hdr.rlp(Seal::With));
let encoded = hdr.encoded();
(hdr, encoded)
};
@ -316,7 +315,7 @@ fn part_bad_part_good() {
let mut hdr = Header::default();
hdr.set_number(num);
let encoded = encoded::Header::new(hdr.rlp(Seal::With));
let encoded = hdr.encoded();
(hdr, encoded)
};
@ -413,7 +412,7 @@ fn back_references() {
});
let header = Header::default();
let encoded = encoded::Header::new(header.rlp(Seal::With));
let encoded = header.encoded();
let recv = harness.service.request_raw(
&Context::NoOp,
@ -470,7 +469,7 @@ fn fill_from_cache() {
});
let header = Header::default();
let encoded = encoded::Header::new(header.rlp(Seal::With));
let encoded = header.encoded();
let recv = harness.service.request_raw(
&Context::NoOp,

View File

@ -31,7 +31,7 @@ use vm::{EnvInfo, LastHashes};
use engines::EthEngine;
use error::{Error, BlockError};
use factory::Factories;
use header::{Header, Seal};
use header::Header;
use receipt::{Receipt, TransactionOutcome};
use state::State;
use state_db::StateDB;
@ -57,10 +57,10 @@ impl Block {
UntrustedRlp::new(b).as_val::<Block>().is_ok()
}
/// Get the RLP-encoding of the block with or without the seal.
pub fn rlp_bytes(&self, seal: Seal) -> Bytes {
/// Get the RLP-encoding of the block with the seal.
pub fn rlp_bytes(&self) -> Bytes {
let mut block_rlp = RlpStream::new_list(3);
self.header.stream_rlp(&mut block_rlp, seal);
block_rlp.append(&self.header);
block_rlp.append_list(&self.transactions);
block_rlp.append_list(&self.uncles);
block_rlp.out()
@ -269,7 +269,6 @@ impl<'x> OpenBlock<'x> {
r.block.header.set_author(author);
r.block.header.set_timestamp_now(parent.timestamp());
r.block.header.set_extra_data(extra_data);
r.block.header.note_dirty();
let gas_floor_target = cmp::max(gas_range_target.0, engine.params().min_gas_limit);
let gas_ceil_target = cmp::max(gas_range_target.1, gas_floor_target);
@ -284,7 +283,9 @@ impl<'x> OpenBlock<'x> {
}
/// Alter the timestamp of the block.
pub fn set_timestamp(&mut self, timestamp: u64) { self.block.header.set_timestamp(timestamp); }
pub fn set_timestamp(&mut self, timestamp: u64) {
self.block.header.set_timestamp(timestamp);
}
/// Removes block gas limit.
pub fn remove_gas_limit(&mut self) {
@ -374,8 +375,11 @@ impl<'x> OpenBlock<'x> {
s.block.header.set_uncles_hash(keccak(&uncle_bytes));
s.block.header.set_state_root(s.block.state.root().clone());
s.block.header.set_receipts_root(ordered_trie_root(s.block.receipts.iter().map(|r| r.rlp_bytes())));
s.block.header.set_log_bloom(s.block.receipts.iter().fold(Bloom::zero(), |mut b, r| {b = &b | &r.log_bloom; b})); //TODO: use |= operator
s.block.header.set_gas_used(s.block.receipts.last().map_or(U256::zero(), |r| r.gas_used));
s.block.header.set_log_bloom(s.block.receipts.iter().fold(Bloom::zero(), |mut b, r| {
b.accrue_bloom(&r.log_bloom);
b
}));
s.block.header.set_gas_used(s.block.receipts.last().map_or_else(U256::zero, |r| r.gas_used));
ClosedBlock {
block: s.block,
@ -395,6 +399,7 @@ impl<'x> OpenBlock<'x> {
if let Err(e) = s.block.state.commit() {
warn!("Encountered error on state commit: {}", e);
}
if s.block.header.transactions_root().is_zero() || s.block.header.transactions_root() == &KECCAK_NULL_RLP {
s.block.header.set_transactions_root(ordered_trie_root(s.block.transactions.iter().map(|e| e.rlp_bytes())));
}
@ -407,8 +412,11 @@ impl<'x> OpenBlock<'x> {
}
s.block.header.set_state_root(s.block.state.root().clone());
s.block.header.set_log_bloom(s.block.receipts.iter().fold(Bloom::zero(), |mut b, r| {b = &b | &r.log_bloom; b})); //TODO: use |= operator
s.block.header.set_gas_used(s.block.receipts.last().map_or(U256::zero(), |r| r.gas_used));
s.block.header.set_log_bloom(s.block.receipts.iter().fold(Bloom::zero(), |mut b, r| {
b.accrue_bloom(&r.log_bloom);
b
}));
s.block.header.set_gas_used(s.block.receipts.last().map_or_else(U256::zero, |r| r.gas_used));
LockedBlock {
block: s.block,
@ -435,7 +443,7 @@ impl<'x> IsBlock for LockedBlock {
impl ClosedBlock {
/// Get the hash of the header without seal arguments.
pub fn hash(&self) -> H256 { self.header().rlp_keccak(Seal::Without) }
pub fn hash(&self) -> H256 { self.header().bare_hash() }
/// Turn this into a `LockedBlock`, unable to be reopened again.
pub fn lock(self) -> LockedBlock {
@ -459,7 +467,7 @@ impl ClosedBlock {
impl LockedBlock {
/// Get the hash of the header without seal arguments.
pub fn hash(&self) -> H256 { self.header().rlp_keccak(Seal::Without) }
pub fn hash(&self) -> H256 { self.header().bare_hash() }
/// Provide a valid seal in order to turn this into a `SealedBlock`.
///
@ -472,6 +480,7 @@ impl LockedBlock {
Mismatch { expected: expected_seal_fields, found: seal.len() }));
}
s.block.header.set_seal(seal);
s.block.header.compute_hash();
Ok(SealedBlock { block: s.block, uncle_bytes: s.uncle_bytes })
}
@ -485,6 +494,7 @@ impl LockedBlock {
) -> Result<SealedBlock, (Error, LockedBlock)> {
let mut s = self;
s.block.header.set_seal(seal);
s.block.header.compute_hash();
// TODO: passing state context to avoid engines owning it?
match engine.verify_local_seal(&s.block.header) {
@ -492,16 +502,6 @@ impl LockedBlock {
_ => Ok(SealedBlock { block: s.block, uncle_bytes: s.uncle_bytes }),
}
}
/// Remove state root from transaction receipts to make them EIP-98 compatible.
pub fn strip_receipts(self) -> LockedBlock {
let mut block = self;
for receipt in &mut block.block.receipts {
receipt.outcome = TransactionOutcome::Unknown;
}
block.block.header.set_receipts_root(ordered_trie_root(block.block.receipts.iter().map(|r| r.rlp_bytes())));
block
}
}
impl Drain for LockedBlock {
@ -515,7 +515,7 @@ impl SealedBlock {
/// Get the RLP-encoding of the block.
pub fn rlp_bytes(&self) -> Bytes {
let mut block_rlp = RlpStream::new_list(3);
self.block.header.stream_rlp(&mut block_rlp, Seal::With);
block_rlp.append(&self.block.header);
block_rlp.append_list(&self.block.transactions);
block_rlp.append_raw(&self.uncle_bytes, 1);
block_rlp.out()
@ -545,6 +545,7 @@ pub fn enact(
last_hashes: Arc<LastHashes>,
factories: Factories,
is_epoch_begin: bool,
strip_receipts: bool,
) -> Result<LockedBlock, Error> {
{
if ::log::max_log_level() >= ::log::LogLevel::Trace {
@ -574,6 +575,12 @@ pub fn enact(
b.push_uncle(u.clone())?;
}
if strip_receipts {
for receipt in &mut b.block.receipts {
receipt.outcome = TransactionOutcome::Unknown;
}
}
Ok(b.close_and_lock())
}
@ -616,6 +623,8 @@ pub fn enact_verified(
last_hashes: Arc<LastHashes>,
factories: Factories,
is_epoch_begin: bool,
// Remove state root from transaction receipts to make them EIP-98 compatible.
strip_receipts: bool,
) -> Result<LockedBlock, Error> {
let view = BlockView::new(&block.bytes);
@ -630,6 +639,7 @@ pub fn enact_verified(
last_hashes,
factories,
is_epoch_begin,
strip_receipts,
)
}

View File

@ -15,8 +15,8 @@
// along with Parity. If not, see <http://www.gnu.org/licenses/>.
use ethereum_types::{H256, U256};
use bytes::Bytes;
use header::BlockNumber;
use encoded;
use header::{Header, BlockNumber};
/// Contains information on a best block that is specific to the consensus engine.
///
@ -24,18 +24,13 @@ use header::BlockNumber;
/// combined difficulty (usually the block with the highest block number).
///
/// Sometimes refered as 'latest block'.
#[derive(Default)]
pub struct BestBlock {
/// Best block hash.
pub hash: H256,
/// Best block number.
pub number: BlockNumber,
/// Best block timestamp.
pub timestamp: u64,
/// Best block decoded header.
pub header: Header,
/// Best block uncompressed bytes.
pub block: encoded::Block,
/// Best block total difficulty.
pub total_difficulty: U256,
/// Best block uncompressed bytes
pub block: Bytes,
}
/// Best ancient block info. If the blockchain has a gap this keeps track of where it starts.

View File

@ -90,11 +90,6 @@ pub trait BlockProvider {
/// Get receipts of block with given hash.
fn block_receipts(&self, hash: &H256) -> Option<BlockReceipts>;
/// Get the partial-header of a block.
fn block_header(&self, hash: &H256) -> Option<Header> {
self.block_header_data(hash).map(|header| header.decode())
}
/// Get the header RLP of a block.
fn block_header_data(&self, hash: &H256) -> Option<encoded::Header>;
@ -115,7 +110,7 @@ pub trait BlockProvider {
/// Get the number of given block's hash.
fn block_number(&self, hash: &H256) -> Option<BlockNumber> {
self.block_details(hash).map(|details| details.number)
self.block_header_data(hash).map(|header| header.number())
}
/// Get transaction with given transaction hash.
@ -144,8 +139,8 @@ pub trait BlockProvider {
}
/// Returns the header of the genesis block.
fn genesis_header(&self) -> Header {
self.block_header(&self.genesis_hash())
fn genesis_header(&self) -> encoded::Header {
self.block_header_data(&self.genesis_hash())
.expect("Genesis header always stored; qed")
}
@ -193,8 +188,8 @@ pub struct BlockChain {
first_block: Option<H256>,
// block cache
block_headers: RwLock<HashMap<H256, Bytes>>,
block_bodies: RwLock<HashMap<H256, Bytes>>,
block_headers: RwLock<HashMap<H256, encoded::Header>>,
block_bodies: RwLock<HashMap<H256, encoded::Body>>,
// extra caches
block_details: RwLock<HashMap<H256, BlockDetails>>,
@ -251,17 +246,15 @@ impl BlockProvider for BlockChain {
{
let read = self.block_headers.read();
if let Some(v) = read.get(hash) {
return Some(encoded::Header::new(v.clone()));
return Some(v.clone());
}
}
// Check if it's the best block
{
let best_block = self.best_block.read();
if &best_block.hash == hash {
return Some(encoded::Header::new(
Rlp::new(&best_block.block).at(0).as_raw().to_vec()
))
if &best_block.header.hash() == hash {
return Some(best_block.header.encoded())
}
}
@ -269,12 +262,12 @@ impl BlockProvider for BlockChain {
let b = self.db.get(db::COL_HEADERS, hash)
.expect("Low level database error. Some issue with disk?")?;
let bytes = decompress(&b, blocks_swapper()).into_vec();
let header = encoded::Header::new(decompress(&b, blocks_swapper()).into_vec());
let mut write = self.block_headers.write();
write.insert(*hash, bytes.clone());
write.insert(*hash, header.clone());
self.cache_man.lock().note_used(CacheId::BlockHeader(*hash));
Some(encoded::Header::new(bytes))
Some(header)
}
/// Get block body data
@ -283,15 +276,15 @@ impl BlockProvider for BlockChain {
{
let read = self.block_bodies.read();
if let Some(v) = read.get(hash) {
return Some(encoded::Body::new(v.clone()));
return Some(v.clone());
}
}
// Check if it's the best block
{
let best_block = self.best_block.read();
if &best_block.hash == hash {
return Some(encoded::Body::new(Self::block_to_body(&best_block.block)));
if &best_block.header.hash() == hash {
return Some(encoded::Body::new(Self::block_to_body(best_block.block.rlp().as_raw())));
}
}
@ -299,12 +292,12 @@ impl BlockProvider for BlockChain {
let b = self.db.get(db::COL_BODIES, hash)
.expect("Low level database error. Some issue with disk?")?;
let bytes = decompress(&b, blocks_swapper()).into_vec();
let body = encoded::Body::new(decompress(&b, blocks_swapper()).into_vec());
let mut write = self.block_bodies.write();
write.insert(*hash, bytes.clone());
write.insert(*hash, body.clone());
self.cache_man.lock().note_used(CacheId::BlockBody(*hash));
Some(encoded::Body::new(bytes))
Some(body)
}
/// Get the familial details concerning a block.
@ -476,7 +469,12 @@ impl BlockChain {
elements_per_index: LOG_BLOOMS_ELEMENTS_PER_INDEX,
},
first_block: None,
best_block: RwLock::new(BestBlock::default()),
best_block: RwLock::new(BestBlock {
// BestBlock will be overwritten anyway.
header: Default::default(),
total_difficulty: Default::default(),
block: encoded::Block::new(genesis.into()),
}),
best_ancient_block: RwLock::new(None),
block_headers: RwLock::new(HashMap::new()),
block_bodies: RwLock::new(HashMap::new()),
@ -527,11 +525,21 @@ impl BlockChain {
{
// Fetch best block details
let best_block_number = bc.block_number(&best_block_hash).unwrap();
let best_block_total_difficulty = bc.block_details(&best_block_hash).unwrap().total_difficulty;
let best_block_rlp = bc.block(&best_block_hash).unwrap().into_inner();
let best_block_timestamp = BlockView::new(&best_block_rlp).header().timestamp();
let best_block_rlp = bc.block(&best_block_hash).unwrap();
// and write them
let mut best_block = bc.best_block.write();
*best_block = BestBlock {
total_difficulty: best_block_total_difficulty,
header: best_block_rlp.decode_header(),
block: best_block_rlp,
};
}
{
let best_block_number = bc.best_block.read().header.number();
// Fetch first and best ancient block details
let raw_first = bc.db.get(db::COL_EXTRA, b"first").unwrap().map(|v| v.into_vec());
let mut best_ancient = bc.db.get(db::COL_EXTRA, b"ancient").unwrap().map(|h| H256::from_slice(&h));
let best_ancient_number;
@ -574,15 +582,6 @@ impl BlockChain {
}
// and write them
let mut best_block = bc.best_block.write();
*best_block = BestBlock {
number: best_block_number,
total_difficulty: best_block_total_difficulty,
hash: best_block_hash,
timestamp: best_block_timestamp,
block: best_block_rlp,
};
if let (Some(hash), Some(number)) = (best_ancient, best_ancient_number) {
let mut best_ancient_block = bc.best_ancient_block.write();
*best_ancient_block = Some(BestAncientBlock {
@ -737,7 +736,6 @@ impl BlockChain {
blocks_blooms: self.prepare_block_blooms_update(bytes, &info),
transactions_addresses: self.prepare_transaction_addresses_update(bytes, &info),
info: info,
timestamp: header.timestamp(),
block: bytes
}, is_best);
@ -786,7 +784,6 @@ impl BlockChain {
blocks_blooms: self.prepare_block_blooms_update(bytes, &info),
transactions_addresses: self.prepare_transaction_addresses_update(bytes, &info),
info: info,
timestamp: header.timestamp(),
block: bytes,
}, is_best);
true
@ -936,7 +933,6 @@ impl BlockChain {
blocks_blooms: self.prepare_block_blooms_update(bytes, &info),
transactions_addresses: self.prepare_transaction_addresses_update(bytes, &info),
info: info.clone(),
timestamp: header.timestamp(),
block: bytes,
}, true);
@ -1028,12 +1024,11 @@ impl BlockChain {
let mut best_block = self.pending_best_block.write();
if is_best && update.info.location != BlockLocation::Branch {
batch.put(db::COL_EXTRA, b"best", &update.info.hash);
let block = encoded::Block::new(update.block.to_vec());
*best_block = Some(BestBlock {
hash: update.info.hash,
number: update.info.number,
total_difficulty: update.info.total_difficulty,
timestamp: update.timestamp,
block: update.block.to_vec(),
header: block.decode_header(),
block,
});
}
@ -1105,8 +1100,9 @@ impl BlockChain {
}
/// Given a block's `parent`, find every block header which represents a valid possible uncle.
pub fn find_uncle_headers(&self, parent: &H256, uncle_generations: usize) -> Option<Vec<Header>> {
self.find_uncle_hashes(parent, uncle_generations).map(|v| v.into_iter().filter_map(|h| self.block_header(&h)).collect())
pub fn find_uncle_headers(&self, parent: &H256, uncle_generations: usize) -> Option<Vec<encoded::Header>> {
self.find_uncle_hashes(parent, uncle_generations)
.map(|v| v.into_iter().filter_map(|h| self.block_header_data(&h)).collect())
}
/// Given a block's `parent`, find every block hash which represents a valid possible uncle.
@ -1307,17 +1303,17 @@ impl BlockChain {
/// Get best block hash.
pub fn best_block_hash(&self) -> H256 {
self.best_block.read().hash
self.best_block.read().header.hash()
}
/// Get best block number.
pub fn best_block_number(&self) -> BlockNumber {
self.best_block.read().number
self.best_block.read().header.number()
}
/// Get best block timestamp.
pub fn best_block_timestamp(&self) -> u64 {
self.best_block.read().timestamp
self.best_block.read().header.timestamp()
}
/// Get best block total difficulty.
@ -1326,10 +1322,8 @@ impl BlockChain {
}
/// Get best block header
pub fn best_block_header(&self) -> encoded::Header {
let block = self.best_block.read();
let raw = BlockView::new(&block.block).header_view().rlp().as_raw().to_vec();
encoded::Header::new(raw)
pub fn best_block_header(&self) -> Header {
self.best_block.read().header.clone()
}
/// Get current cache size.
@ -1402,12 +1396,12 @@ impl BlockChain {
let best_block = self.best_block.read();
let best_ancient_block = self.best_ancient_block.read();
BlockChainInfo {
total_difficulty: best_block.total_difficulty.clone(),
pending_total_difficulty: best_block.total_difficulty.clone(),
total_difficulty: best_block.total_difficulty,
pending_total_difficulty: best_block.total_difficulty,
genesis_hash: self.genesis_hash(),
best_block_hash: best_block.hash,
best_block_number: best_block.number,
best_block_timestamp: best_block.timestamp,
best_block_hash: best_block.header.hash(),
best_block_number: best_block.header.number(),
best_block_timestamp: best_block.header.timestamp(),
first_block_hash: self.first_block(),
first_block_number: From::from(self.first_block_number()),
ancient_block_hash: best_ancient_block.as_ref().map(|b| b.hash),
@ -1539,7 +1533,11 @@ mod tests {
let b4b = b3a.add_block_with_difficulty(9);
let b5b = b4a.add_block_with_difficulty(9);
let uncle_headers = vec![b4b.last().header(), b3b.last().header(), b2b.last().header()];
let uncle_headers = vec![
b4b.last().header().encoded(),
b3b.last().header().encoded(),
b2b.last().header().encoded(),
];
let b4a_hash = b4a.last().hash();
let generator = BlockGenerator::new(
@ -1862,10 +1860,10 @@ mod tests {
assert_eq!(bc.best_block_number(), 2999);
let best_hash = bc.best_block_hash();
let mut block_header = bc.block_header(&best_hash);
let mut block_header = bc.block_header_data(&best_hash);
while !block_header.is_none() {
block_header = bc.block_header(block_header.unwrap().parent_hash());
block_header = bc.block_header_data(&block_header.unwrap().parent_hash());
}
assert!(bc.cache_size().blocks > 1024 * 1024);

View File

@ -9,8 +9,6 @@ use blooms::{BloomGroup, GroupPosition};
pub struct ExtrasUpdate<'a> {
/// Block info.
pub info: BlockInfo,
/// Block timestamp.
pub timestamp: u64,
/// Current block uncompressed rlp bytes
pub block: &'a [u8],
/// Modified block hashes.

View File

@ -362,16 +362,15 @@ impl Importer {
let engine = &*self.engine;
let header = &block.header;
let chain = client.chain.read();
// Check the block isn't so old we won't be able to enact it.
let best_block_number = chain.best_block_number();
let best_block_number = client.chain.read().best_block_number();
if client.pruning_info().earliest_state > header.number() {
warn!(target: "client", "Block import failed for #{} ({})\nBlock is ancient (current best block: #{}).", header.number(), header.hash(), best_block_number);
return Err(());
}
// Check if parent is in chain
let parent = match chain.block_header(header.parent_hash()) {
let parent = match client.block_header_decoded(BlockId::Hash(*header.parent_hash())) {
Some(h) => h,
None => {
warn!(target: "client", "Block import failed for #{} ({}): Parent not found ({}) ", header.number(), header.hash(), header.parent_hash());
@ -379,6 +378,7 @@ impl Importer {
}
};
let chain = client.chain.read();
// Verify Block Family
let verify_family_result = self.verifier.verify_block_family(
header,
@ -408,6 +408,7 @@ impl Importer {
let db = client.state_db.read().boxed_clone_canon(header.parent_hash());
let is_epoch_begin = chain.epoch_transition(parent.number(), *header.parent_hash()).is_some();
let strip_receipts = header.number() < engine.params().validate_receipts_transition;
let enact_result = enact_verified(block,
engine,
client.tracedb.read().tracing_enabled(),
@ -416,15 +417,13 @@ impl Importer {
last_hashes,
client.factories.clone(),
is_epoch_begin,
strip_receipts,
);
let mut locked_block = enact_result.map_err(|e| {
let locked_block = enact_result.map_err(|e| {
warn!(target: "client", "Block import failed for #{} ({})\nError: {:?}", header.number(), header.hash(), e);
})?;
if header.number() < engine.params().validate_receipts_transition && header.receipts_root() != locked_block.block().header().receipts_root() {
locked_block = locked_block.strip_receipts();
}
// Final Verification
if let Err(e) = self.verifier.verify_block_final(header, locked_block.block().header()) {
warn!(target: "client", "Stage 5 block verification failed for #{} ({})\nError: {:?}", header.number(), header.hash(), e);
@ -655,7 +654,7 @@ impl Importer {
fn check_epoch_end<'a>(&self, header: &'a Header, chain: &BlockChain, client: &Client) {
let is_epoch_end = self.engine.is_epoch_end(
header,
&(|hash| chain.block_header(&hash)),
&(|hash| client.block_header_decoded(BlockId::Hash(hash))),
&(|hash| chain.get_pending_transition(hash)), // TODO: limit to current epoch.
);
@ -724,7 +723,7 @@ impl Client {
config.history
};
if !chain.block_header(&chain.best_block_hash()).map_or(true, |h| state_db.journal_db().contains(h.state_root())) {
if !chain.block_header_data(&chain.best_block_hash()).map_or(true, |h| state_db.journal_db().contains(&h.state_root())) {
warn!("State root not found for block #{} ({:x})", chain.best_block_number(), chain.best_block_hash());
}
@ -1000,7 +999,7 @@ impl Client {
let header = self.best_block_header();
State::from_existing(
self.state_db.read().boxed_clone_canon(&header.hash()),
header.state_root(),
*header.state_root(),
self.engine.account_start_nonce(header.number()),
self.factories.clone()
)
@ -1260,6 +1259,23 @@ impl Client {
BlockId::Latest => Some(self.chain.read().best_block_number()),
}
}
/// Retrieve a decoded header given `BlockId`
///
/// This method optimizes access patterns for latest block header
/// to avoid excessive RLP encoding, decoding and hashing.
fn block_header_decoded(&self, id: BlockId) -> Option<Header> {
match id {
BlockId::Latest
=> Some(self.chain.read().best_block_header()),
BlockId::Hash(ref hash) if hash == &self.chain.read().best_block_hash()
=> Some(self.chain.read().best_block_header()),
BlockId::Number(number) if number == self.chain.read().best_block_number()
=> Some(self.chain.read().best_block_header()),
_
=> self.block_header(id).map(|h| h.decode()),
}
}
}
impl snapshot::DatabaseRestore for Client {
@ -1315,16 +1331,14 @@ impl BlockInfo for Client {
Self::block_hash(&chain, id).and_then(|hash| chain.block_header_data(&hash))
}
fn best_block_header(&self) -> encoded::Header {
fn best_block_header(&self) -> Header {
self.chain.read().best_block_header()
}
fn block(&self, id: BlockId) -> Option<encoded::Block> {
let chain = self.chain.read();
Self::block_hash(&chain, id).and_then(|hash| {
chain.block(&hash)
})
Self::block_hash(&chain, id).and_then(|hash| chain.block(&hash))
}
fn code_hash(&self, address: &Address, id: BlockId) -> Option<H256> {
@ -1360,11 +1374,11 @@ impl CallContract for Client {
fn call_contract(&self, block_id: BlockId, address: Address, data: Bytes) -> Result<Bytes, String> {
let state_pruned = || CallError::StatePruned.to_string();
let state = &mut self.state_at(block_id).ok_or_else(&state_pruned)?;
let header = self.block_header(block_id).ok_or_else(&state_pruned)?;
let header = self.block_header_decoded(block_id).ok_or_else(&state_pruned)?;
let transaction = self.contract_call_tx(block_id, address, data);
self.call(&transaction, Default::default(), state, &header.decode())
self.call(&transaction, Default::default(), state, &header)
.map_err(|e| format!("{:?}", e))
.map(|executed| executed.output)
}
@ -1918,8 +1932,8 @@ impl BlockChainClient for Client {
}
fn block_extra_info(&self, id: BlockId) -> Option<BTreeMap<String, String>> {
self.block_header(id)
.map(|header| self.engine.extra_info(&header.decode()))
self.block_header_decoded(id)
.map(|header| self.engine.extra_info(&header))
}
fn uncle_extra_info(&self, id: UncleId) -> Option<BTreeMap<String, String>> {
@ -1973,8 +1987,8 @@ impl ReopenBlock for Client {
for h in uncles {
if !block.uncles().iter().any(|header| header.hash() == h) {
let uncle = chain.block_header(&h).expect("find_uncle_hashes only returns hashes for existing headers; qed");
block.push_uncle(uncle).expect("pushing up to maximum_uncle_count;
let uncle = chain.block_header_data(&h).expect("find_uncle_hashes only returns hashes for existing headers; qed");
block.push_uncle(uncle.decode()).expect("pushing up to maximum_uncle_count;
push_uncle is not ok only if more than maximum_uncle_count is pushed;
so all push_uncle are Ok;
qed");
@ -1991,9 +2005,8 @@ impl PrepareOpenBlock for Client {
fn prepare_open_block(&self, author: Address, gas_range_target: (U256, U256), extra_data: Bytes) -> OpenBlock {
let engine = &*self.engine;
let chain = self.chain.read();
let h = chain.best_block_hash();
let best_header = &chain.block_header(&h)
.expect("h is best block hash: so its header must exist: qed");
let best_header = chain.best_block_header();
let h = best_header.hash();
let is_epoch_begin = chain.epoch_transition(best_header.number(), h).is_some();
let mut open_block = OpenBlock::new(
@ -2001,7 +2014,7 @@ impl PrepareOpenBlock for Client {
self.factories.clone(),
self.tracedb.read().tracing_enabled(),
self.state_db.read().boxed_clone_canon(&h),
best_header,
&best_header,
self.build_last_hashes(&h),
author,
gas_range_target,
@ -2016,7 +2029,7 @@ impl PrepareOpenBlock for Client {
.into_iter()
.take(engine.maximum_uncle_count(open_block.header().number()))
.foreach(|h| {
open_block.push_uncle(h).expect("pushing maximum_uncle_count;
open_block.push_uncle(h.decode()).expect("pushing maximum_uncle_count;
open_block was just created;
push_uncle is not ok only if more than maximum_uncle_count is pushed;
so all push_uncle are Ok;

View File

@ -474,9 +474,10 @@ impl BlockInfo for TestBlockChainClient {
.map(encoded::Header::new)
}
fn best_block_header(&self) -> encoded::Header {
fn best_block_header(&self) -> Header {
self.block_header(BlockId::Hash(self.chain_info().best_block_hash))
.expect("Best block always has header.")
.decode()
}
fn block(&self, id: BlockId) -> Option<encoded::Block> {

View File

@ -121,7 +121,7 @@ pub trait BlockInfo {
fn block_header(&self, id: BlockId) -> Option<encoded::Header>;
/// Get the best block header.
fn best_block_header(&self) -> encoded::Header;
fn best_block_header(&self) -> Header;
/// Get raw block data by block header hash.
fn block(&self, id: BlockId) -> Option<encoded::Block>;

View File

@ -201,7 +201,7 @@ mod tests {
"0000000000000000000000007d577a597b2742b498cb5cf0c26cdcd726d39e6e"
);
// Simulate a misbehaving validator by handling a double proposal.
let header = client.best_block_header().decode();
let header = client.best_block_header();
assert!(client.engine().verify_block_family(&header, &header).is_err());
// Seal a block.
client.engine().step();

View File

@ -20,7 +20,7 @@ use std::sync::{Weak, Arc};
use hash::keccak;
use ethereum_types::{H256, U256, Address, Bloom};
use parking_lot::{Mutex, RwLock};
use parking_lot::RwLock;
use bytes::Bytes;
use memory_cache::MemoryLruCache;
@ -53,19 +53,19 @@ lazy_static! {
// only "first" proofs are such.
struct StateProof {
contract_address: Address,
header: Mutex<Header>,
header: Header,
provider: validator_set::ValidatorSet,
}
impl ::engines::StateDependentProof<EthereumMachine> for StateProof {
fn generate_proof(&self, caller: &Call) -> Result<Vec<u8>, String> {
prove_initial(&self.provider, self.contract_address, &*self.header.lock(), caller)
prove_initial(&self.provider, self.contract_address, &self.header, caller)
}
fn check_proof(&self, machine: &EthereumMachine, proof: &[u8]) -> Result<(), String> {
let (header, state_items) = decode_first_proof(&UntrustedRlp::new(proof))
.map_err(|e| format!("proof incorrectly encoded: {}", e))?;
if &header != &*self.header.lock(){
if &header != &self.header {
return Err("wrong header in proof".into());
}
@ -325,7 +325,7 @@ impl ValidatorSet for ValidatorSafeContract {
debug!(target: "engine", "signalling transition to fresh contract.");
let state_proof = Arc::new(StateProof {
contract_address: self.contract_address,
header: Mutex::new(header.clone()),
header: header.clone(),
provider: validator_set::ValidatorSet::default(),
});
return ::engines::EpochChange::Yes(::engines::Proof::WithState(state_proof as Arc<_>));

View File

@ -17,7 +17,6 @@
//! Block header.
use std::cmp;
use std::cell::RefCell;
use std::time::{SystemTime, UNIX_EPOCH};
use hash::{KECCAK_NULL_RLP, KECCAK_EMPTY_LIST_RLP, keccak};
use heapsize::HeapSizeOf;
@ -28,7 +27,8 @@ use rlp::{UntrustedRlp, RlpStream, Encodable, DecoderError, Decodable};
pub use types::BlockNumber;
/// Semantic boolean for when a seal/signature is included.
pub enum Seal {
#[derive(Debug, Clone, Copy)]
enum Seal {
/// The seal/signature is included.
With,
/// The seal/signature is not included.
@ -75,14 +75,18 @@ pub struct Header {
/// Vector of post-RLP-encoded fields.
seal: Vec<Bytes>,
/// The memoized hash of the RLP representation *including* the seal fields.
hash: RefCell<Option<H256>>,
/// The memoized hash of the RLP representation *without* the seal fields.
bare_hash: RefCell<Option<H256>>,
/// Memoized hash of that header and the seal.
hash: Option<H256>,
}
impl PartialEq for Header {
fn eq(&self, c: &Header) -> bool {
if let (&Some(ref h1), &Some(ref h2)) = (&self.hash, &c.hash) {
if h1 == h2 {
return true
}
}
self.parent_hash == c.parent_hash &&
self.timestamp == c.timestamp &&
self.number == c.number &&
@ -120,51 +124,57 @@ impl Default for Header {
difficulty: U256::default(),
seal: vec![],
hash: RefCell::new(None),
bare_hash: RefCell::new(None),
hash: None,
}
}
}
impl Header {
/// Create a new, default-valued, header.
pub fn new() -> Self {
Self::default()
}
pub fn new() -> Self { Self::default() }
/// Get the parent_hash field of the header.
pub fn parent_hash(&self) -> &H256 { &self.parent_hash }
/// Get the timestamp field of the header.
pub fn timestamp(&self) -> u64 { self.timestamp }
/// Get the number field of the header.
pub fn number(&self) -> BlockNumber { self.number }
/// Get the author field of the header.
pub fn author(&self) -> &Address { &self.author }
/// Get the extra data field of the header.
pub fn extra_data(&self) -> &Bytes { &self.extra_data }
/// Get a mutable reference to extra_data
pub fn extra_data_mut(&mut self) -> &mut Bytes { self.note_dirty(); &mut self.extra_data }
/// Get the state root field of the header.
pub fn state_root(&self) -> &H256 { &self.state_root }
/// Get the receipts root field of the header.
pub fn receipts_root(&self) -> &H256 { &self.receipts_root }
/// Get the log bloom field of the header.
pub fn log_bloom(&self) -> &Bloom { &self.log_bloom }
/// Get the transactions root field of the header.
pub fn transactions_root(&self) -> &H256 { &self.transactions_root }
/// Get the uncles hash field of the header.
pub fn uncles_hash(&self) -> &H256 { &self.uncles_hash }
/// Get the gas used field of the header.
pub fn gas_used(&self) -> &U256 { &self.gas_used }
/// Get the gas limit field of the header.
pub fn gas_limit(&self) -> &U256 { &self.gas_limit }
/// Get the difficulty field of the header.
pub fn difficulty(&self) -> &U256 { &self.difficulty }
/// Get the seal field of the header.
pub fn seal(&self) -> &[Bytes] { &self.seal }
/// Get the seal field with RLP-decoded values as bytes.
pub fn decode_seal<'a, T: ::std::iter::FromIterator<&'a [u8]>>(&'a self) -> Result<T, DecoderError> {
self.seal.iter().map(|rlp| {
@ -172,78 +182,126 @@ impl Header {
}).collect()
}
// TODO: seal_at, set_seal_at &c.
/// Get a mutable reference to extra_data
#[cfg(test)]
pub fn extra_data_mut(&mut self) -> &mut Bytes {
self.hash = None;
&mut self.extra_data
}
/// Set the number field of the header.
pub fn set_parent_hash(&mut self, a: H256) { self.parent_hash = a; self.note_dirty(); }
pub fn set_parent_hash(&mut self, a: H256) {
change_field(&mut self.hash, &mut self.parent_hash, a);
}
/// Set the uncles hash field of the header.
pub fn set_uncles_hash(&mut self, a: H256) { self.uncles_hash = a; self.note_dirty(); }
pub fn set_uncles_hash(&mut self, a: H256) {
change_field(&mut self.hash, &mut self.uncles_hash, a);
}
/// Set the state root field of the header.
pub fn set_state_root(&mut self, a: H256) { self.state_root = a; self.note_dirty(); }
pub fn set_state_root(&mut self, a: H256) {
change_field(&mut self.hash, &mut self.state_root, a);
}
/// Set the transactions root field of the header.
pub fn set_transactions_root(&mut self, a: H256) { self.transactions_root = a; self.note_dirty() }
pub fn set_transactions_root(&mut self, a: H256) {
change_field(&mut self.hash, &mut self.transactions_root, a);
}
/// Set the receipts root field of the header.
pub fn set_receipts_root(&mut self, a: H256) { self.receipts_root = a; self.note_dirty() }
pub fn set_receipts_root(&mut self, a: H256) {
change_field(&mut self.hash, &mut self.receipts_root, a);
}
/// Set the log bloom field of the header.
pub fn set_log_bloom(&mut self, a: Bloom) { self.log_bloom = a; self.note_dirty() }
pub fn set_log_bloom(&mut self, a: Bloom) {
change_field(&mut self.hash, &mut self.log_bloom, a);
}
/// Set the timestamp field of the header.
pub fn set_timestamp(&mut self, a: u64) { self.timestamp = a; self.note_dirty(); }
pub fn set_timestamp(&mut self, a: u64) {
change_field(&mut self.hash, &mut self.timestamp, a);
}
/// Set the timestamp field of the header to the current time.
pub fn set_timestamp_now(&mut self, but_later_than: u64) { let now = SystemTime::now().duration_since(UNIX_EPOCH).unwrap_or_default(); self.timestamp = cmp::max(now.as_secs() as u64, but_later_than + 1); self.note_dirty(); }
pub fn set_timestamp_now(&mut self, but_later_than: u64) {
let now = SystemTime::now().duration_since(UNIX_EPOCH).unwrap_or_default();
self.set_timestamp(cmp::max(now.as_secs() as u64, but_later_than + 1));
}
/// Set the number field of the header.
pub fn set_number(&mut self, a: BlockNumber) { self.number = a; self.note_dirty(); }
pub fn set_number(&mut self, a: BlockNumber) {
change_field(&mut self.hash, &mut self.number, a);
}
/// Set the author field of the header.
pub fn set_author(&mut self, a: Address) { if a != self.author { self.author = a; self.note_dirty(); } }
pub fn set_author(&mut self, a: Address) {
change_field(&mut self.hash, &mut self.author, a);
}
/// Set the extra data field of the header.
pub fn set_extra_data(&mut self, a: Bytes) { if a != self.extra_data { self.extra_data = a; self.note_dirty(); } }
pub fn set_extra_data(&mut self, a: Bytes) {
change_field(&mut self.hash, &mut self.extra_data, a);
}
/// Set the gas used field of the header.
pub fn set_gas_used(&mut self, a: U256) { self.gas_used = a; self.note_dirty(); }
pub fn set_gas_used(&mut self, a: U256) {
change_field(&mut self.hash, &mut self.gas_used, a);
}
/// Set the gas limit field of the header.
pub fn set_gas_limit(&mut self, a: U256) { self.gas_limit = a; self.note_dirty(); }
pub fn set_gas_limit(&mut self, a: U256) {
change_field(&mut self.hash, &mut self.gas_limit, a);
}
/// Set the difficulty field of the header.
pub fn set_difficulty(&mut self, a: U256) { self.difficulty = a; self.note_dirty(); }
/// Set the seal field of the header.
pub fn set_seal(&mut self, a: Vec<Bytes>) { self.seal = a; self.note_dirty(); }
pub fn set_difficulty(&mut self, a: U256) {
change_field(&mut self.hash, &mut self.difficulty, a);
}
/// Get the hash of this header (keccak of the RLP).
/// Set the seal field of the header.
pub fn set_seal(&mut self, a: Vec<Bytes>) {
change_field(&mut self.hash, &mut self.seal, a)
}
/// Get & memoize the hash of this header (keccak of the RLP with seal).
pub fn compute_hash(&mut self) -> H256 {
let hash = self.hash();
self.hash = Some(hash);
hash
}
/// Get the hash of this header (keccak of the RLP with seal).
pub fn hash(&self) -> H256 {
let mut hash = self.hash.borrow_mut();
match &mut *hash {
&mut Some(ref h) => h.clone(),
hash @ &mut None => {
let h = self.rlp_keccak(Seal::With);
*hash = Some(h.clone());
h
}
}
self.hash.unwrap_or_else(|| keccak(self.rlp(Seal::With)))
}
/// Get the hash of the header excluding the seal
pub fn bare_hash(&self) -> H256 {
let mut hash = self.bare_hash.borrow_mut();
match &mut *hash {
&mut Some(ref h) => h.clone(),
hash @ &mut None => {
let h = self.rlp_keccak(Seal::Without);
*hash = Some(h.clone());
h
}
}
keccak(self.rlp(Seal::Without))
}
/// Note that some fields have changed. Resets the memoised hash.
pub fn note_dirty(&self) {
*self.hash.borrow_mut() = None;
*self.bare_hash.borrow_mut() = None;
/// Encode the header, getting a type-safe wrapper around the RLP.
pub fn encoded(&self) -> ::encoded::Header {
::encoded::Header::new(self.rlp(Seal::With))
}
/// Get the RLP representation of this Header.
fn rlp(&self, with_seal: Seal) -> Bytes {
let mut s = RlpStream::new();
self.stream_rlp(&mut s, with_seal);
s.out()
}
// TODO: make these functions traity
/// Place this header into an RLP stream `s`, optionally `with_seal`.
pub fn stream_rlp(&self, s: &mut RlpStream, with_seal: Seal) {
s.begin_list(13 + match with_seal { Seal::With => self.seal.len(), _ => 0 });
fn stream_rlp(&self, s: &mut RlpStream, with_seal: Seal) {
if let Seal::With = with_seal {
s.begin_list(13 + self.seal.len());
} else {
s.begin_list(13);
}
s.append(&self.parent_hash);
s.append(&self.uncles_hash);
s.append(&self.author);
@ -257,29 +315,24 @@ impl Header {
s.append(&self.gas_used);
s.append(&self.timestamp);
s.append(&self.extra_data);
if let Seal::With = with_seal {
for b in &self.seal {
s.append_raw(b, 1);
}
}
}
}
/// Get the RLP of this header, optionally `with_seal`.
pub fn rlp(&self, with_seal: Seal) -> Bytes {
let mut s = RlpStream::new();
self.stream_rlp(&mut s, with_seal);
s.out()
}
/// Get the SHA3 (Keccak) of this header, optionally `with_seal`.
pub fn rlp_keccak(&self, with_seal: Seal) -> H256 { keccak(self.rlp(with_seal)) }
/// Encode the header, getting a type-safe wrapper around the RLP.
pub fn encoded(&self) -> ::encoded::Header {
::encoded::Header::new(self.rlp(Seal::With))
/// Alter value of given field, reset memoised hash if changed.
fn change_field<T>(hash: &mut Option<H256>, field: &mut T, value: T) where T: PartialEq<T> {
if field != &value {
*field = value;
*hash = None;
}
}
impl Decodable for Header {
fn decode(r: &UntrustedRlp) -> Result<Self, DecoderError> {
let mut blockheader = Header {
@ -297,8 +350,7 @@ impl Decodable for Header {
timestamp: cmp::min(r.val_at::<U256>(11)?, u64::max_value().into()).as_u64(),
extra_data: r.val_at(12)?,
seal: vec![],
hash: RefCell::new(Some(keccak(r.as_raw()))),
bare_hash: RefCell::new(None),
hash: keccak(r.as_raw()).into(),
};
for i in 13..r.item_count()? {
@ -376,3 +428,4 @@ mod tests {
assert_eq!(header_rlp, encoded_header);
}
}

View File

@ -653,7 +653,7 @@ impl Miner {
}
fn update_gas_limit<C: BlockChain>(&self, client: &C) {
let gas_limit = client.best_block_header().gas_limit();
let gas_limit = *client.best_block_header().gas_limit();
let mut queue = self.transaction_queue.write();
queue.set_gas_limit(gas_limit);
if let GasLimit::Auto = self.options.tx_queue_gas_limit {
@ -703,7 +703,7 @@ impl Miner {
condition: Option<TransactionCondition>,
transaction_queue: &mut BanningTransactionQueue,
) -> Vec<Result<TransactionImportResult, Error>> {
let best_block_header = client.best_block_header().decode();
let best_block_header = client.best_block_header();
let insertion_time = client.chain_info().best_block_number;
let mut inserted = Vec::with_capacity(transactions.len());

View File

@ -135,7 +135,6 @@ impl AbridgedBlock {
mod tests {
use views::BlockView;
use block::Block;
use header::Seal;
use super::AbridgedBlock;
use transaction::{Action, Transaction};
@ -143,7 +142,7 @@ mod tests {
use bytes::Bytes;
fn encode_block(b: &Block) -> Bytes {
b.rlp_bytes(Seal::With)
b.rlp_bytes()
}
#[test]

View File

@ -28,7 +28,7 @@ use blockchain::{BlockChain, BlockProvider};
use engines::{EthEngine, EpochVerifier, EpochTransition};
use machine::EthereumMachine;
use ids::BlockId;
use header::{Header, Seal};
use header::Header;
use receipt::Receipt;
use snapshot::{Error, ManifestData};
@ -324,7 +324,7 @@ impl Rebuilder for ChunkRebuilder {
transactions: last_rlp.list_at(1)?,
uncles: last_rlp.list_at(2)?,
};
let block_data = block.rlp_bytes(Seal::With);
let block_data = block.rlp_bytes();
let receipts: Vec<Receipt> = last_rlp.list_at(3)?;
{

View File

@ -153,19 +153,19 @@ impl<'a> PowWorker<'a> {
fn write_chunk(&mut self, last: H256) -> Result<(), Error> {
trace!(target: "snapshot", "prepared block chunk with {} blocks", self.rlps.len());
let (last_header, last_details) = self.chain.block_header(&last)
let (last_header, last_details) = self.chain.block_header_data(&last)
.and_then(|n| self.chain.block_details(&last).map(|d| (n, d)))
.ok_or(Error::BlockNotFound(last))?;
let parent_number = last_header.number() - 1;
let parent_hash = last_header.parent_hash();
let parent_total_difficulty = last_details.total_difficulty - *last_header.difficulty();
let parent_total_difficulty = last_details.total_difficulty - last_header.difficulty();
trace!(target: "snapshot", "parent last written block: {}", parent_hash);
let num_entries = self.rlps.len();
let mut rlp_stream = RlpStream::new_list(3 + num_entries);
rlp_stream.append(&parent_number).append(parent_hash).append(&parent_total_difficulty);
rlp_stream.append(&parent_number).append(&parent_hash).append(&parent_total_difficulty);
for pair in self.rlps.drain(..) {
rlp_stream.append_raw(&pair, 1);
@ -220,7 +220,6 @@ impl Rebuilder for PowRebuilder {
/// Feed the rebuilder an uncompressed block chunk.
/// Returns the number of blocks fed or any errors.
fn feed(&mut self, chunk: &[u8], engine: &EthEngine, abort_flag: &AtomicBool) -> Result<(), ::error::Error> {
use header::Seal;
use views::BlockView;
use snapshot::verify_old_block;
use ethereum_types::U256;
@ -251,7 +250,7 @@ impl Rebuilder for PowRebuilder {
let receipts_root = ordered_trie_root(pair.at(1)?.iter().map(|r| r.as_raw()));
let block = abridged_block.to_block(parent_hash, cur_number, receipts_root)?;
let block_bytes = block.rlp_bytes(Seal::With);
let block_bytes = block.rlp_bytes();
let is_best = cur_number == self.best_number;
if is_best {

View File

@ -130,7 +130,7 @@ pub fn take_snapshot<W: SnapshotWriter + Send>(
writer: W,
p: &Progress
) -> Result<(), Error> {
let start_header = chain.block_header(&block_at)
let start_header = chain.block_header_data(&block_at)
.ok_or(Error::InvalidStartingBlock(BlockId::Hash(block_at)))?;
let state_root = start_header.state_root();
let number = start_header.number();
@ -143,7 +143,7 @@ pub fn take_snapshot<W: SnapshotWriter + Send>(
let (state_hashes, block_hashes) = scope(|scope| {
let writer = &writer;
let block_guard = scope.spawn(move || chunk_secondary(chunker, chain, block_at, writer, p));
let state_res = chunk_state(state_db, state_root, writer, p);
let state_res = chunk_state(state_db, &state_root, writer, p);
state_res.and_then(|state_hashes| {
block_guard.join().map(|block_hashes| (state_hashes, block_hashes))
@ -156,7 +156,7 @@ pub fn take_snapshot<W: SnapshotWriter + Send>(
version: snapshot_version,
state_hashes: state_hashes,
block_hashes: block_hashes,
state_root: *state_root,
state_root: state_root,
block_number: number,
block_hash: block_at,
};
@ -486,8 +486,8 @@ pub fn verify_old_block(rng: &mut OsRng, header: &Header, engine: &EthEngine, ch
if always || rng.gen::<f32>() <= POW_VERIFY_RATE {
engine.verify_block_unordered(header)?;
match chain.block_header(header.parent_hash()) {
Some(parent) => engine.verify_block_family(header, &parent),
match chain.block_header_data(header.parent_hash()) {
Some(parent) => engine.verify_block_family(header, &parent.decode()),
None => Ok(()),
}
} else {

View File

@ -362,7 +362,7 @@ fn transaction_proof() {
let mut factories = ::factory::Factories::default();
factories.accountdb = ::account_db::Factory::Plain; // raw state values, no mangled keys.
let root = client.best_block_header().state_root();
let root = *client.best_block_header().state_root();
let mut state = State::from_existing(backend, root, 0.into(), factories.clone()).unwrap();
Executive::new(&mut state, &client.latest_env_info(), test_spec.engine.machine())

View File

@ -211,7 +211,7 @@ fn verify_uncles(header: &Header, bytes: &[u8], bc: &BlockProvider, engine: &Eth
// cB.p^7 -------------/
// cB.p^8
let mut expected_uncle_parent = header.parent_hash().clone();
let uncle_parent = bc.block_header(&uncle.parent_hash()).ok_or_else(|| Error::from(BlockError::UnknownUncleParent(uncle.parent_hash().clone())))?;
let uncle_parent = bc.block_header_data(&uncle.parent_hash()).ok_or_else(|| Error::from(BlockError::UnknownUncleParent(uncle.parent_hash().clone())))?;
for _ in 0..depth {
match bc.block_details(&expected_uncle_parent) {
Some(details) => {
@ -224,6 +224,7 @@ fn verify_uncles(header: &Header, bytes: &[u8], bc: &BlockProvider, engine: &Eth
return Err(From::from(BlockError::UncleParentNotInChain(uncle_parent.hash())));
}
let uncle_parent = uncle_parent.decode();
verify_parent(&uncle, &uncle_parent, engine.params().gas_limit_bound_divisor)?;
engine.verify_block_family(&uncle, &uncle_parent)?;
verified.insert(uncle.hash());
@ -496,8 +497,9 @@ mod tests {
// is fine.
let client = ::client::TestBlockChainClient::default();
let parent = bc.block_header(header.parent_hash())
.ok_or(BlockError::UnknownParent(header.parent_hash().clone()))?;
let parent = bc.block_header_data(header.parent_hash())
.ok_or(BlockError::UnknownParent(header.parent_hash().clone()))?
.decode();
let full_params = FullFamilyParams {
block_bytes: bytes,

View File

@ -30,7 +30,7 @@ use ethcore::block::IsBlock;
use ethcore::client::{MiningBlockChainClient, BlockId, TransactionId, UncleId, StateOrBlock, StateClient, StateInfo, Call, EngineInfo};
use ethcore::ethereum::Ethash;
use ethcore::filter::Filter as EthcoreFilter;
use ethcore::header::{BlockNumber as EthBlockNumber, Seal};
use ethcore::header::{BlockNumber as EthBlockNumber};
use ethcore::log_entry::LogEntry;
use ethcore::miner::MinerService;
use ethcore::snapshot::SnapshotService;
@ -199,7 +199,7 @@ impl<C, SN: ?Sized, S: ?Sized, M, EM, T: StateInfo + 'static> EthClient<C, SN, S
let extra = pending_block.as_ref().map(|b| self.client.engine().extra_info(&b.header));
(pending_block.map(|b| encoded::Block::new(b.rlp_bytes(Seal::Without))), Some(difficulty), extra, true)
(pending_block.map(|b| encoded::Block::new(b.rlp_bytes())), Some(difficulty), extra, true)
},
BlockNumberOrId::Number(num) => {

View File

@ -1974,7 +1974,7 @@ impl ChainSync {
fn propagate_new_hashes(&mut self, chain_info: &BlockChainInfo, io: &mut SyncIo, peers: &[PeerId]) -> usize {
trace!(target: "sync", "Sending NewHashes to {:?}", peers);
let mut sent = 0;
let last_parent = &io.chain().best_block_header().parent_hash();
let last_parent = *io.chain().best_block_header().parent_hash();
for peer_id in peers {
sent += match ChainSync::create_new_hashes_rlp(io.chain(), &last_parent, &chain_info.best_block_hash) {
Some(rlp) => {