decode block rlp less often (#9252)

in total:
- removed 4 redundant rlp deserializations
- avoid 1 redundant block data copy
This commit is contained in:
Marek Kotewicz 2018-08-02 11:20:46 +02:00 committed by André Silva
parent f442665c46
commit b4ae1b6528
17 changed files with 116 additions and 165 deletions

View File

@ -40,7 +40,7 @@ use engines::EthEngine;
use error::{Error, BlockError}; use error::{Error, BlockError};
use ethereum_types::{H256, U256, Address, Bloom}; use ethereum_types::{H256, U256, Address, Bloom};
use factory::Factories; use factory::Factories;
use hash::{keccak, KECCAK_NULL_RLP, KECCAK_EMPTY_LIST_RLP}; use hash::keccak;
use header::{Header, ExtendedHeader}; use header::{Header, ExtendedHeader};
use receipt::{Receipt, TransactionOutcome}; use receipt::{Receipt, TransactionOutcome};
use rlp::{Rlp, RlpStream, Encodable, Decodable, DecoderError, encode_list}; use rlp::{Rlp, RlpStream, Encodable, Decodable, DecoderError, encode_list};
@ -51,7 +51,6 @@ use transaction::{UnverifiedTransaction, SignedTransaction, Error as Transaction
use triehash::ordered_trie_root; use triehash::ordered_trie_root;
use unexpected::{Mismatch, OutOfBounds}; use unexpected::{Mismatch, OutOfBounds};
use verification::PreverifiedBlock; use verification::PreverifiedBlock;
use views::BlockView;
use vm::{EnvInfo, LastHashes}; use vm::{EnvInfo, LastHashes};
/// A block, encoded as it is on the block chain. /// A block, encoded as it is on the block chain.
@ -66,11 +65,6 @@ pub struct Block {
} }
impl Block { impl Block {
/// Returns true if the given bytes form a valid encoding of a block in RLP.
pub fn is_good(b: &[u8]) -> bool {
Rlp::new(b).as_val::<Block>().is_ok()
}
/// Get the RLP-encoding of the block with the seal. /// Get the RLP-encoding of the block with the seal.
pub fn rlp_bytes(&self) -> Bytes { pub fn rlp_bytes(&self) -> Bytes {
let mut block_rlp = RlpStream::new_list(3); let mut block_rlp = RlpStream::new_list(3);
@ -398,26 +392,11 @@ impl<'x> OpenBlock<'x> {
/// Turn this into a `ClosedBlock`. /// Turn this into a `ClosedBlock`.
pub fn close(self) -> Result<ClosedBlock, Error> { pub fn close(self) -> Result<ClosedBlock, Error> {
let mut s = self; let unclosed_state = self.block.state.clone();
let locked = self.close_and_lock()?;
let unclosed_state = s.block.state.clone();
s.engine.on_close_block(&mut s.block)?;
s.block.state.commit()?;
s.block.header.set_transactions_root(ordered_trie_root(s.block.transactions.iter().map(|e| e.rlp_bytes())));
let uncle_bytes = encode_list(&s.block.uncles);
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.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));
Ok(ClosedBlock { Ok(ClosedBlock {
block: s.block, block: locked.block,
unclosed_state, unclosed_state,
}) })
} }
@ -429,18 +408,11 @@ impl<'x> OpenBlock<'x> {
s.engine.on_close_block(&mut s.block)?; s.engine.on_close_block(&mut s.block)?;
s.block.state.commit()?; s.block.state.commit()?;
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())));
s.block.header.set_transactions_root(ordered_trie_root(s.block.transactions.iter().map(|e| e.rlp_bytes()))); let uncle_bytes = encode_list(&s.block.uncles);
} s.block.header.set_uncles_hash(keccak(&uncle_bytes));
if s.block.header.uncles_hash().is_zero() || s.block.header.uncles_hash() == &KECCAK_EMPTY_LIST_RLP {
let uncle_bytes = encode_list(&s.block.uncles);
s.block.header.set_uncles_hash(keccak(&uncle_bytes));
}
if s.block.header.receipts_root().is_zero() || s.block.header.receipts_root() == &KECCAK_NULL_RLP {
s.block.header.set_receipts_root(ordered_trie_root(s.block.receipts.iter().map(|r| r.rlp_bytes())));
}
s.block.header.set_state_root(s.block.state.root().clone()); 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| { s.block.header.set_log_bloom(s.block.receipts.iter().fold(Bloom::zero(), |mut b, r| {
b.accrue_bloom(&r.log_bloom); b.accrue_bloom(&r.log_bloom);
b b
@ -537,18 +509,16 @@ impl LockedBlock {
self, self,
engine: &EthEngine, engine: &EthEngine,
seal: Vec<Bytes>, seal: Vec<Bytes>,
) -> Result<SealedBlock, (Error, LockedBlock)> { ) -> Result<SealedBlock, Error> {
let mut s = self; let mut s = self;
s.block.header.set_seal(seal); s.block.header.set_seal(seal);
s.block.header.compute_hash(); s.block.header.compute_hash();
// TODO: passing state context to avoid engines owning it? // TODO: passing state context to avoid engines owning it?
match engine.verify_local_seal(&s.block.header) { engine.verify_local_seal(&s.block.header)?;
Err(e) => Err((e, s)), Ok(SealedBlock {
_ => Ok(SealedBlock { block: s.block
block: s.block })
}),
}
} }
} }
@ -637,12 +607,11 @@ pub fn enact_verified(
is_epoch_begin: bool, is_epoch_begin: bool,
ancestry: &mut Iterator<Item=ExtendedHeader>, ancestry: &mut Iterator<Item=ExtendedHeader>,
) -> Result<LockedBlock, Error> { ) -> Result<LockedBlock, Error> {
let view = view!(BlockView, &block.bytes);
enact( enact(
block.header, block.header,
block.transactions, block.transactions,
view.uncles(), block.uncles,
engine, engine,
tracing, tracing,
db, db,

View File

@ -73,6 +73,8 @@ use types::filter::Filter;
use types::ancestry_action::AncestryAction; use types::ancestry_action::AncestryAction;
use verification; use verification;
use verification::{PreverifiedBlock, Verifier, BlockQueue}; use verification::{PreverifiedBlock, Verifier, BlockQueue};
use verification::queue::kind::blocks::Unverified;
use verification::queue::kind::BlockLike;
// re-export // re-export
pub use types::blockchain_info::BlockChainInfo; pub use types::blockchain_info::BlockChainInfo;
@ -208,7 +210,7 @@ pub struct Client {
/// Queued ancient blocks, make sure they are imported in order. /// Queued ancient blocks, make sure they are imported in order.
queued_ancient_blocks: Arc<RwLock<( queued_ancient_blocks: Arc<RwLock<(
HashSet<H256>, HashSet<H256>,
VecDeque<(Header, encoded::Block, Bytes)> VecDeque<(Unverified, Bytes)>
)>>, )>>,
ancient_blocks_import_lock: Arc<Mutex<()>>, ancient_blocks_import_lock: Arc<Mutex<()>>,
/// Consensus messages import queue /// Consensus messages import queue
@ -428,7 +430,7 @@ impl Importer {
/// ///
/// The block is guaranteed to be the next best blocks in the /// The block is guaranteed to be the next best blocks in the
/// first block sequence. Does no sealing or transaction validation. /// first block sequence. Does no sealing or transaction validation.
fn import_old_block(&self, header: &Header, block: encoded::Block, receipts_bytes: &[u8], db: &KeyValueDB, chain: &BlockChain) -> Result<(), ::error::Error> { fn import_old_block(&self, unverified: Unverified, receipts_bytes: &[u8], db: &KeyValueDB, chain: &BlockChain) -> Result<(), ::error::Error> {
let receipts = ::rlp::decode_list(receipts_bytes); let receipts = ::rlp::decode_list(receipts_bytes);
let _import_lock = self.import_lock.lock(); let _import_lock = self.import_lock.lock();
@ -436,11 +438,11 @@ impl Importer {
trace_time!("import_old_block"); trace_time!("import_old_block");
// verify the block, passing the chain for updating the epoch verifier. // verify the block, passing the chain for updating the epoch verifier.
let mut rng = OsRng::new()?; let mut rng = OsRng::new()?;
self.ancient_verifier.verify(&mut rng, &header, &chain)?; self.ancient_verifier.verify(&mut rng, &unverified.header, &chain)?;
// Commit results // Commit results
let mut batch = DBTransaction::new(); let mut batch = DBTransaction::new();
chain.insert_unordered_block(&mut batch, block, receipts, None, false, true); chain.insert_unordered_block(&mut batch, encoded::Block::new(unverified.bytes), receipts, None, false, true);
// Final commit to the DB // Final commit to the DB
db.write_buffered(batch); db.write_buffered(batch);
chain.commit(); chain.commit();
@ -1381,22 +1383,15 @@ impl CallContract for Client {
} }
impl ImportBlock for Client { impl ImportBlock for Client {
fn import_block(&self, bytes: Bytes) -> Result<H256, BlockImportError> { fn import_block(&self, unverified: Unverified) -> Result<H256, BlockImportError> {
use verification::queue::kind::BlockLike; if self.chain.read().is_known(&unverified.hash()) {
use verification::queue::kind::blocks::Unverified; bail!(BlockImportErrorKind::Import(ImportErrorKind::AlreadyInChain));
// create unverified block here so the `keccak` calculation can be cached.
let unverified = Unverified::from_rlp(bytes)?;
{
if self.chain.read().is_known(&unverified.hash()) {
bail!(BlockImportErrorKind::Import(ImportErrorKind::AlreadyInChain));
}
let status = self.block_status(BlockId::Hash(unverified.parent_hash()));
if status == BlockStatus::Unknown || status == BlockStatus::Pending {
bail!(BlockImportErrorKind::Block(BlockError::UnknownParent(unverified.parent_hash())));
}
} }
let status = self.block_status(BlockId::Hash(unverified.parent_hash()));
if status == BlockStatus::Unknown || status == BlockStatus::Pending {
bail!(BlockImportErrorKind::Block(BlockError::UnknownParent(unverified.parent_hash())));
}
Ok(self.importer.block_queue.import(unverified)?) Ok(self.importer.block_queue.import(unverified)?)
} }
} }
@ -2027,24 +2022,23 @@ impl IoClient for Client {
}); });
} }
fn queue_ancient_block(&self, block_bytes: Bytes, receipts_bytes: Bytes) -> Result<H256, BlockImportError> { fn queue_ancient_block(&self, unverified: Unverified, receipts_bytes: Bytes) -> Result<H256, BlockImportError> {
trace_time!("queue_ancient_block"); trace_time!("queue_ancient_block");
let header: Header = ::rlp::Rlp::new(&block_bytes).val_at(0)?;
let hash = header.hash();
let hash = unverified.hash();
{ {
// check block order // check block order
if self.chain.read().is_known(&hash) { if self.chain.read().is_known(&hash) {
bail!(BlockImportErrorKind::Import(ImportErrorKind::AlreadyInChain)); bail!(BlockImportErrorKind::Import(ImportErrorKind::AlreadyInChain));
} }
let parent_hash = header.parent_hash(); let parent_hash = unverified.parent_hash();
// NOTE To prevent race condition with import, make sure to check queued blocks first // NOTE To prevent race condition with import, make sure to check queued blocks first
// (and attempt to acquire lock) // (and attempt to acquire lock)
let is_parent_pending = self.queued_ancient_blocks.read().0.contains(parent_hash); let is_parent_pending = self.queued_ancient_blocks.read().0.contains(&parent_hash);
if !is_parent_pending { if !is_parent_pending {
let status = self.block_status(BlockId::Hash(*parent_hash)); let status = self.block_status(BlockId::Hash(parent_hash));
if status == BlockStatus::Unknown || status == BlockStatus::Pending { if status == BlockStatus::Unknown || status == BlockStatus::Pending {
bail!(BlockImportErrorKind::Block(BlockError::UnknownParent(*parent_hash))); bail!(BlockImportErrorKind::Block(BlockError::UnknownParent(parent_hash)));
} }
} }
} }
@ -2053,7 +2047,7 @@ impl IoClient for Client {
{ {
let mut queued = self.queued_ancient_blocks.write(); let mut queued = self.queued_ancient_blocks.write();
queued.0.insert(hash); queued.0.insert(hash);
queued.1.push_back((header, encoded::Block::new(block_bytes), receipts_bytes)); queued.1.push_back((unverified, receipts_bytes));
} }
let queued = self.queued_ancient_blocks.clone(); let queued = self.queued_ancient_blocks.clone();
@ -2065,11 +2059,10 @@ impl IoClient for Client {
let _lock = lock.lock(); let _lock = lock.lock();
for _i in 0..MAX_ANCIENT_BLOCKS_TO_IMPORT { for _i in 0..MAX_ANCIENT_BLOCKS_TO_IMPORT {
let first = queued.write().1.pop_front(); let first = queued.write().1.pop_front();
if let Some((header, block_bytes, receipts_bytes)) = first { if let Some((unverified, receipts_bytes)) = first {
let hash = header.hash(); let hash = unverified.hash();
let result = client.importer.import_old_block( let result = client.importer.import_old_block(
&header, unverified,
block_bytes,
&receipts_bytes, &receipts_bytes,
&**client.db.read().key_value(), &**client.db.read().key_value(),
&*client.chain.read(), &*client.chain.read(),

View File

@ -53,6 +53,7 @@ use spec::Spec;
use types::basic_account::BasicAccount; use types::basic_account::BasicAccount;
use types::pruning_info::PruningInfo; use types::pruning_info::PruningInfo;
use verification::queue::QueueInfo; use verification::queue::QueueInfo;
use verification::queue::kind::blocks::Unverified;
use block::{OpenBlock, SealedBlock, ClosedBlock}; use block::{OpenBlock, SealedBlock, ClosedBlock};
use executive::Executed; use executive::Executed;
use error::CallError; use error::CallError;
@ -280,7 +281,8 @@ impl TestBlockChainClient {
rlp.append(&header); rlp.append(&header);
rlp.append_raw(&txs, 1); rlp.append_raw(&txs, 1);
rlp.append_raw(uncles.as_raw(), 1); rlp.append_raw(uncles.as_raw(), 1);
self.import_block(rlp.as_raw().to_vec()).unwrap(); let unverified = Unverified::from_rlp(rlp.out()).unwrap();
self.import_block(unverified).unwrap();
} }
} }
@ -512,8 +514,8 @@ impl RegistryInfo for TestBlockChainClient {
} }
impl ImportBlock for TestBlockChainClient { impl ImportBlock for TestBlockChainClient {
fn import_block(&self, b: Bytes) -> Result<H256, BlockImportError> { fn import_block(&self, unverified: Unverified) -> Result<H256, BlockImportError> {
let header = view!(BlockView, &b).header(); let header = unverified.header;
let h = header.hash(); let h = header.hash();
let number: usize = header.number() as usize; let number: usize = header.number() as usize;
if number > self.blocks.read().len() { if number > self.blocks.read().len() {
@ -539,7 +541,7 @@ impl ImportBlock for TestBlockChainClient {
*difficulty = *difficulty + header.difficulty().clone(); *difficulty = *difficulty + header.difficulty().clone();
} }
mem::replace(&mut *self.last_hash.write(), h.clone()); mem::replace(&mut *self.last_hash.write(), h.clone());
self.blocks.write().insert(h.clone(), b); self.blocks.write().insert(h.clone(), unverified.bytes);
self.numbers.write().insert(number, h.clone()); self.numbers.write().insert(number, h.clone());
let mut parent_hash = header.parent_hash().clone(); let mut parent_hash = header.parent_hash().clone();
if number > 0 { if number > 0 {
@ -552,7 +554,7 @@ impl ImportBlock for TestBlockChainClient {
} }
} }
else { else {
self.blocks.write().insert(h.clone(), b.to_vec()); self.blocks.write().insert(h.clone(), unverified.bytes);
} }
Ok(h) Ok(h)
} }
@ -856,8 +858,8 @@ impl IoClient for TestBlockChainClient {
self.miner.import_external_transactions(self, txs); self.miner.import_external_transactions(self, txs);
} }
fn queue_ancient_block(&self, b: Bytes, _r: Bytes) -> Result<H256, BlockImportError> { fn queue_ancient_block(&self, unverified: Unverified, _r: Bytes) -> Result<H256, BlockImportError> {
self.import_block(b) self.import_block(unverified)
} }
fn queue_consensus_message(&self, message: Bytes) { fn queue_consensus_message(&self, message: Bytes) {

View File

@ -34,6 +34,7 @@ use receipt::LocalizedReceipt;
use trace::LocalizedTrace; use trace::LocalizedTrace;
use transaction::{self, LocalizedTransaction, SignedTransaction}; use transaction::{self, LocalizedTransaction, SignedTransaction};
use verification::queue::QueueInfo as BlockQueueInfo; use verification::queue::QueueInfo as BlockQueueInfo;
use verification::queue::kind::blocks::Unverified;
use state::StateInfo; use state::StateInfo;
use header::Header; use header::Header;
use engines::EthEngine; use engines::EthEngine;
@ -167,7 +168,7 @@ pub trait RegistryInfo {
/// Provides methods to import block into blockchain /// Provides methods to import block into blockchain
pub trait ImportBlock { pub trait ImportBlock {
/// Import a block into the blockchain. /// Import a block into the blockchain.
fn import_block(&self, bytes: Bytes) -> Result<H256, BlockImportError>; fn import_block(&self, block: Unverified) -> Result<H256, BlockImportError>;
} }
/// Provides `call_contract` method /// Provides `call_contract` method
@ -204,7 +205,7 @@ pub trait IoClient: Sync + Send {
fn queue_transactions(&self, transactions: Vec<Bytes>, peer_id: usize); fn queue_transactions(&self, transactions: Vec<Bytes>, peer_id: usize);
/// Queue block import with transaction receipts. Does no sealing and transaction validation. /// Queue block import with transaction receipts. Does no sealing and transaction validation.
fn queue_ancient_block(&self, block_bytes: Bytes, receipts_bytes: Bytes) -> Result<H256, BlockImportError>; fn queue_ancient_block(&self, block_bytes: Unverified, receipts_bytes: Bytes) -> Result<H256, BlockImportError>;
/// Queue conensus engine message. /// Queue conensus engine message.
fn queue_consensus_message(&self, message: Bytes); fn queue_consensus_message(&self, message: Bytes);

View File

@ -158,6 +158,7 @@ mod tests {
use test_helpers::{generate_dummy_client_with_spec_and_accounts, generate_dummy_client_with_spec_and_data}; use test_helpers::{generate_dummy_client_with_spec_and_accounts, generate_dummy_client_with_spec_and_data};
use types::ids::BlockId; use types::ids::BlockId;
use ethereum_types::Address; use ethereum_types::Address;
use verification::queue::kind::blocks::Unverified;
use super::Multi; use super::Multi;
@ -198,7 +199,7 @@ mod tests {
let sync_client = generate_dummy_client_with_spec_and_data(Spec::new_validator_multi, 0, 0, &[]); let sync_client = generate_dummy_client_with_spec_and_data(Spec::new_validator_multi, 0, 0, &[]);
sync_client.engine().register_client(Arc::downgrade(&sync_client) as _); sync_client.engine().register_client(Arc::downgrade(&sync_client) as _);
for i in 1..4 { for i in 1..4 {
sync_client.import_block(client.block(BlockId::Number(i)).unwrap().into_inner()).unwrap(); sync_client.import_block(Unverified::from_rlp(client.block(BlockId::Number(i)).unwrap().into_inner()).unwrap()).unwrap();
} }
sync_client.flush_queue(); sync_client.flush_queue();
assert_eq!(sync_client.chain_info().best_block_number, 3); assert_eq!(sync_client.chain_info().best_block_number, 3);

View File

@ -458,6 +458,7 @@ mod tests {
use test_helpers::{generate_dummy_client_with_spec_and_accounts, generate_dummy_client_with_spec_and_data}; use test_helpers::{generate_dummy_client_with_spec_and_accounts, generate_dummy_client_with_spec_and_data};
use super::super::ValidatorSet; use super::super::ValidatorSet;
use super::{ValidatorSafeContract, EVENT_NAME_HASH}; use super::{ValidatorSafeContract, EVENT_NAME_HASH};
use verification::queue::kind::blocks::Unverified;
#[test] #[test]
fn fetches_validators() { fn fetches_validators() {
@ -530,7 +531,7 @@ mod tests {
let sync_client = generate_dummy_client_with_spec_and_data(Spec::new_validator_safe_contract, 0, 0, &[]); let sync_client = generate_dummy_client_with_spec_and_data(Spec::new_validator_safe_contract, 0, 0, &[]);
sync_client.engine().register_client(Arc::downgrade(&sync_client) as _); sync_client.engine().register_client(Arc::downgrade(&sync_client) as _);
for i in 1..4 { for i in 1..4 {
sync_client.import_block(client.block(BlockId::Number(i)).unwrap().into_inner()).unwrap(); sync_client.import_block(Unverified::from_rlp(client.block(BlockId::Number(i)).unwrap().into_inner()).unwrap()).unwrap();
} }
sync_client.flush_queue(); sync_client.flush_queue();
assert_eq!(sync_client.chain_info().best_block_number, 3); assert_eq!(sync_client.chain_info().best_block_number, 3);

View File

@ -17,12 +17,12 @@
use std::path::Path; use std::path::Path;
use std::sync::Arc; use std::sync::Arc;
use client::{EvmTestClient, Client, ClientConfig, ChainInfo, ImportBlock}; use client::{EvmTestClient, Client, ClientConfig, ChainInfo, ImportBlock};
use block::Block;
use spec::Genesis; use spec::Genesis;
use ethjson; use ethjson;
use miner::Miner; use miner::Miner;
use io::IoChannel; use io::IoChannel;
use test_helpers; use test_helpers;
use verification::queue::kind::blocks::Unverified;
use super::HookType; use super::HookType;
@ -83,9 +83,9 @@ pub fn json_chain_test<H: FnMut(&str, HookType)>(json_data: &[u8], start_stop_ho
Arc::new(Miner::new_for_tests(&spec, None)), Arc::new(Miner::new_for_tests(&spec, None)),
IoChannel::disconnected(), IoChannel::disconnected(),
).unwrap(); ).unwrap();
for b in &blockchain.blocks_rlp() { for b in blockchain.blocks_rlp() {
if Block::is_good(&b) { if let Ok(block) = Unverified::from_rlp(b) {
let _ = client.import_block(b.clone()); let _ = client.import_block(block);
client.flush_queue(); client.flush_queue();
client.import_verified_blocks(); client.import_verified_blocks();
} }

View File

@ -1148,7 +1148,7 @@ impl miner::MinerService for Miner {
|b| &b.hash() == &block_hash |b| &b.hash() == &block_hash
) { ) {
trace!(target: "miner", "Submitted block {}={}={} with seal {:?}", block_hash, b.hash(), b.header().bare_hash(), seal); trace!(target: "miner", "Submitted block {}={}={} with seal {:?}", block_hash, b.hash(), b.header().bare_hash(), seal);
b.lock().try_seal(&*self.engine, seal).or_else(|(e, _)| { b.lock().try_seal(&*self.engine, seal).or_else(|e| {
warn!(target: "miner", "Mined solution rejected: {}", e); warn!(target: "miner", "Mined solution rejected: {}", e);
Err(ErrorKind::PowInvalid.into()) Err(ErrorKind::PowInvalid.into())
}) })

View File

@ -43,6 +43,7 @@ use blooms_db;
use kvdb::KeyValueDB; use kvdb::KeyValueDB;
use kvdb_rocksdb; use kvdb_rocksdb;
use tempdir::TempDir; use tempdir::TempDir;
use verification::queue::kind::blocks::Unverified;
use encoded; use encoded;
/// Creates test block with corresponding header /// Creates test block with corresponding header
@ -175,7 +176,7 @@ pub fn generate_dummy_client_with_spec_accounts_and_data<F>(test_spec: F, accoun
let b = b.close_and_lock().unwrap().seal(test_engine, vec![]).unwrap(); let b = b.close_and_lock().unwrap().seal(test_engine, vec![]).unwrap();
if let Err(e) = client.import_block(b.rlp_bytes()) { if let Err(e) = client.import_block(Unverified::from_rlp(b.rlp_bytes()).unwrap()) {
panic!("error importing block which is valid by definition: {:?}", e); panic!("error importing block which is valid by definition: {:?}", e);
} }
@ -211,7 +212,7 @@ pub fn push_blocks_to_client(client: &Arc<Client>, timestamp_salt: u64, starting
rolling_block_number = rolling_block_number + 1; rolling_block_number = rolling_block_number + 1;
rolling_timestamp = rolling_timestamp + 10; rolling_timestamp = rolling_timestamp + 10;
if let Err(e) = client.import_block(create_test_block(&header)) { if let Err(e) = client.import_block(Unverified::from_rlp(create_test_block(&header)).unwrap()) {
panic!("error importing block which is valid by definition: {:?}", e); panic!("error importing block which is valid by definition: {:?}", e);
} }
} }
@ -231,7 +232,7 @@ pub fn push_block_with_transactions(client: &Arc<Client>, transactions: &[Signed
} }
let b = b.close_and_lock().unwrap().seal(test_engine, vec![]).unwrap(); let b = b.close_and_lock().unwrap().seal(test_engine, vec![]).unwrap();
if let Err(e) = client.import_block(b.rlp_bytes()) { if let Err(e) = client.import_block(Unverified::from_rlp(b.rlp_bytes()).unwrap()) {
panic!("error importing block which is valid by definition: {:?}", e); panic!("error importing block which is valid by definition: {:?}", e);
} }
@ -253,7 +254,7 @@ pub fn get_test_client_with_blocks(blocks: Vec<Bytes>) -> Arc<Client> {
).unwrap(); ).unwrap();
for block in blocks { for block in blocks {
if let Err(e) = client.import_block(block) { if let Err(e) = client.import_block(Unverified::from_rlp(block).unwrap()) {
panic!("error importing block which is well-formed: {:?}", e); panic!("error importing block which is well-formed: {:?}", e);
} }
} }

View File

@ -35,9 +35,9 @@ use views::BlockView;
use ethkey::KeyPair; use ethkey::KeyPair;
use transaction::{PendingTransaction, Transaction, Action, Condition}; use transaction::{PendingTransaction, Transaction, Action, Condition};
use miner::MinerService; use miner::MinerService;
use rlp::{RlpStream, EMPTY_LIST_RLP};
use tempdir::TempDir; use tempdir::TempDir;
use test_helpers; use test_helpers;
use verification::queue::kind::blocks::Unverified;
#[test] #[test]
fn imports_from_empty() { fn imports_from_empty() {
@ -97,7 +97,7 @@ fn imports_good_block() {
IoChannel::disconnected(), IoChannel::disconnected(),
).unwrap(); ).unwrap();
let good_block = get_good_dummy_block(); let good_block = get_good_dummy_block();
if client.import_block(good_block).is_err() { if client.import_block(Unverified::from_rlp(good_block).unwrap()).is_err() {
panic!("error importing block being good by definition"); panic!("error importing block being good by definition");
} }
client.flush_queue(); client.flush_queue();
@ -107,24 +107,6 @@ fn imports_good_block() {
assert!(!block.into_inner().is_empty()); assert!(!block.into_inner().is_empty());
} }
#[test]
fn fails_to_import_block_with_invalid_rlp() {
use error::{BlockImportError, BlockImportErrorKind};
let client = generate_dummy_client(6);
let mut rlp = RlpStream::new_list(3);
rlp.append_raw(&EMPTY_LIST_RLP, 1); // empty header
rlp.append_raw(&EMPTY_LIST_RLP, 1);
rlp.append_raw(&EMPTY_LIST_RLP, 1);
let invalid_header_block = rlp.out();
match client.import_block(invalid_header_block) {
Err(BlockImportError(BlockImportErrorKind::Decoder(_), _)) => (), // all good
Err(_) => panic!("Should fail with a decoder error"),
Ok(_) => panic!("Should not import block with invalid header"),
}
}
#[test] #[test]
fn query_none_block() { fn query_none_block() {
let db = test_helpers::new_db(); let db = test_helpers::new_db();

View File

@ -33,6 +33,7 @@ use views::BlockView;
use trace::{RewardType, LocalizedTrace}; use trace::{RewardType, LocalizedTrace};
use trace::trace::Action::Reward; use trace::trace::Action::Reward;
use test_helpers; use test_helpers;
use verification::queue::kind::blocks::Unverified;
#[test] #[test]
fn can_trace_block_and_uncle_reward() { fn can_trace_block_and_uncle_reward() {
@ -91,7 +92,7 @@ fn can_trace_block_and_uncle_reward() {
let root_block = root_block.close_and_lock().unwrap().seal(engine, vec![]).unwrap(); let root_block = root_block.close_and_lock().unwrap().seal(engine, vec![]).unwrap();
if let Err(e) = client.import_block(root_block.rlp_bytes()) { if let Err(e) = client.import_block(Unverified::from_rlp(root_block.rlp_bytes()).unwrap()) {
panic!("error importing block which is valid by definition: {:?}", e); panic!("error importing block which is valid by definition: {:?}", e);
} }
@ -120,7 +121,7 @@ fn can_trace_block_and_uncle_reward() {
let parent_block = parent_block.close_and_lock().unwrap().seal(engine, vec![]).unwrap(); let parent_block = parent_block.close_and_lock().unwrap().seal(engine, vec![]).unwrap();
if let Err(e) = client.import_block(parent_block.rlp_bytes()) { if let Err(e) = client.import_block(Unverified::from_rlp(parent_block.rlp_bytes()).unwrap()) {
panic!("error importing block which is valid by definition: {:?}", e); panic!("error importing block which is valid by definition: {:?}", e);
} }
@ -170,7 +171,7 @@ fn can_trace_block_and_uncle_reward() {
let block = block.close_and_lock().unwrap().seal(engine, vec![]).unwrap(); let block = block.close_and_lock().unwrap().seal(engine, vec![]).unwrap();
let res = client.import_block(block.rlp_bytes()); let res = client.import_block(Unverified::from_rlp(block.rlp_bytes()).unwrap());
if res.is_err() { if res.is_err() {
panic!("error importing block: {:#?}", res.err().unwrap()); panic!("error importing block: {:#?}", res.err().unwrap());
} }

View File

@ -23,11 +23,10 @@ use std::cmp;
use heapsize::HeapSizeOf; use heapsize::HeapSizeOf;
use ethereum_types::H256; use ethereum_types::H256;
use rlp::{self, Rlp}; use rlp::{self, Rlp};
use ethcore::views::BlockView;
use ethcore::header::{BlockNumber, Header as BlockHeader}; use ethcore::header::{BlockNumber, Header as BlockHeader};
use ethcore::client::{BlockStatus, BlockId, BlockImportError, BlockImportErrorKind}; use ethcore::client::{BlockStatus, BlockId, BlockImportError, BlockImportErrorKind};
use ethcore::block::Block;
use ethcore::error::{ImportErrorKind, BlockError}; use ethcore::error::{ImportErrorKind, BlockError};
use ethcore::verification::queue::kind::blocks::Unverified;
use sync_io::SyncIo; use sync_io::SyncIo;
use blocks::BlockCollection; use blocks::BlockCollection;
@ -484,18 +483,19 @@ impl BlockDownloader {
let block = block_and_receipts.block; let block = block_and_receipts.block;
let receipts = block_and_receipts.receipts; let receipts = block_and_receipts.receipts;
// Perform basic block verification let block = match Unverified::from_rlp(block) {
if !Block::is_good(&block) { Ok(block) => block,
debug!(target: "sync", "Bad block rlp: {:?}", block); Err(_) => {
bad = true; debug!(target: "sync", "Bad block rlp");
break; bad = true;
} break;
}
let (h, number, parent) = {
let header = view!(BlockView, &block).header_view();
(header.hash(), header.number(), header.parent_hash())
}; };
let h = block.header.hash();
let number = block.header.number();
let parent = *block.header.parent_hash();
if self.target_hash.as_ref().map_or(false, |t| t == &h) { if self.target_hash.as_ref().map_or(false, |t| t == &h) {
self.state = State::Complete; self.state = State::Complete;
trace!(target: "sync", "Sync target reached"); trace!(target: "sync", "Sync target reached");

View File

@ -294,7 +294,7 @@ impl BlockCollection {
let header = view!(HeaderView, &block.header); let header = view!(HeaderView, &block.header);
let block_view = Block::new_from_header_and_body(&header, &body); let block_view = Block::new_from_header_and_body(&header, &body);
drained.push(BlockAndReceipts { drained.push(BlockAndReceipts {
block: block_view.rlp().as_raw().to_vec(), block: block_view.into_inner(),
receipts: block.receipts.clone(), receipts: block.receipts.clone(),
}); });
} }

View File

@ -19,8 +19,9 @@ use block_sync::{BlockDownloaderImportError as DownloaderImportError, DownloadAc
use bytes::Bytes; use bytes::Bytes;
use ethcore::client::{BlockStatus, BlockId, BlockImportError, BlockImportErrorKind}; use ethcore::client::{BlockStatus, BlockId, BlockImportError, BlockImportErrorKind};
use ethcore::error::*; use ethcore::error::*;
use ethcore::header::{BlockNumber, Header as BlockHeader}; use ethcore::header::BlockNumber;
use ethcore::snapshot::{ManifestData, RestorationStatus}; use ethcore::snapshot::{ManifestData, RestorationStatus};
use ethcore::verification::queue::kind::blocks::Unverified;
use ethereum_types::{H256, U256}; use ethereum_types::{H256, U256};
use hash::keccak; use hash::keccak;
use network::PeerId; use network::PeerId;
@ -162,44 +163,43 @@ impl SyncHandler {
peer.difficulty = Some(difficulty); peer.difficulty = Some(difficulty);
} }
} }
let block_rlp = r.at(0)?; let block = Unverified::from_rlp(r.at(0)?.as_raw().to_vec())?;
let header_rlp = block_rlp.at(0)?; let hash = block.header.hash();
let h = keccak(&header_rlp.as_raw()); let number = block.header.number();
trace!(target: "sync", "{} -> NewBlock ({})", peer_id, h); trace!(target: "sync", "{} -> NewBlock ({})", peer_id, hash);
let header: BlockHeader = header_rlp.as_val()?; if number > sync.highest_block.unwrap_or(0) {
if header.number() > sync.highest_block.unwrap_or(0) { sync.highest_block = Some(number);
sync.highest_block = Some(header.number());
} }
let mut unknown = false; let mut unknown = false;
{
if let Some(ref mut peer) = sync.peers.get_mut(&peer_id) { if let Some(ref mut peer) = sync.peers.get_mut(&peer_id) {
peer.latest_hash = header.hash(); peer.latest_hash = hash;
}
} }
let last_imported_number = sync.new_blocks.last_imported_block_number(); let last_imported_number = sync.new_blocks.last_imported_block_number();
if last_imported_number > header.number() && last_imported_number - header.number() > MAX_NEW_BLOCK_AGE { if last_imported_number > number && last_imported_number - number > MAX_NEW_BLOCK_AGE {
trace!(target: "sync", "Ignored ancient new block {:?}", h); trace!(target: "sync", "Ignored ancient new block {:?}", hash);
return Err(DownloaderImportError::Invalid); return Err(DownloaderImportError::Invalid);
} }
match io.chain().import_block(block_rlp.as_raw().to_vec()) { match io.chain().import_block(block) {
Err(BlockImportError(BlockImportErrorKind::Import(ImportErrorKind::AlreadyInChain), _)) => { Err(BlockImportError(BlockImportErrorKind::Import(ImportErrorKind::AlreadyInChain), _)) => {
trace!(target: "sync", "New block already in chain {:?}", h); trace!(target: "sync", "New block already in chain {:?}", hash);
}, },
Err(BlockImportError(BlockImportErrorKind::Import(ImportErrorKind::AlreadyQueued), _)) => { Err(BlockImportError(BlockImportErrorKind::Import(ImportErrorKind::AlreadyQueued), _)) => {
trace!(target: "sync", "New block already queued {:?}", h); trace!(target: "sync", "New block already queued {:?}", hash);
}, },
Ok(_) => { Ok(_) => {
// abort current download of the same block // abort current download of the same block
sync.complete_sync(io); sync.complete_sync(io);
sync.new_blocks.mark_as_known(&header.hash(), header.number()); sync.new_blocks.mark_as_known(&hash, number);
trace!(target: "sync", "New block queued {:?} ({})", h, header.number()); trace!(target: "sync", "New block queued {:?} ({})", hash, number);
}, },
Err(BlockImportError(BlockImportErrorKind::Block(BlockError::UnknownParent(p)), _)) => { Err(BlockImportError(BlockImportErrorKind::Block(BlockError::UnknownParent(p)), _)) => {
unknown = true; unknown = true;
trace!(target: "sync", "New block with unknown parent ({:?}) {:?}", p, h); trace!(target: "sync", "New block with unknown parent ({:?}) {:?}", p, hash);
}, },
Err(e) => { Err(e) => {
debug!(target: "sync", "Bad new block {:?} : {:?}", h, e); debug!(target: "sync", "Bad new block {:?} : {:?}", hash, e);
return Err(DownloaderImportError::Invalid); return Err(DownloaderImportError::Invalid);
} }
}; };
@ -207,7 +207,7 @@ impl SyncHandler {
if sync.state != SyncState::Idle { if sync.state != SyncState::Idle {
trace!(target: "sync", "NewBlock ignored while seeking"); trace!(target: "sync", "NewBlock ignored while seeking");
} else { } else {
trace!(target: "sync", "New unknown block {:?}", h); trace!(target: "sync", "New unknown block {:?}", hash);
//TODO: handle too many unknown blocks //TODO: handle too many unknown blocks
sync.sync_peer(io, peer_id, true); sync.sync_peer(io, peer_id, true);
} }

View File

@ -30,6 +30,7 @@ use ethcore::client::{Mode, DatabaseCompactionProfile, VMType, BlockImportError,
use ethcore::error::{ImportErrorKind, BlockImportErrorKind}; use ethcore::error::{ImportErrorKind, BlockImportErrorKind};
use ethcore::miner::Miner; use ethcore::miner::Miner;
use ethcore::verification::queue::VerifierSettings; use ethcore::verification::queue::VerifierSettings;
use ethcore::verification::queue::kind::blocks::Unverified;
use ethcore_service::ClientService; use ethcore_service::ClientService;
use cache::CacheConfig; use cache::CacheConfig;
use informant::{Informant, FullNodeInformantData, MillisecondDuration}; use informant::{Informant, FullNodeInformantData, MillisecondDuration};
@ -417,8 +418,9 @@ fn execute_import(cmd: ImportBlockchain) -> Result<(), String> {
service.register_io_handler(informant).map_err(|_| "Unable to register informant handler".to_owned())?; service.register_io_handler(informant).map_err(|_| "Unable to register informant handler".to_owned())?;
let do_import = |bytes| { let do_import = |bytes| {
let block = Unverified::from_rlp(bytes).map_err(|_| "Invalid block rlp")?;
while client.queue_info().is_full() { sleep(Duration::from_secs(1)); } while client.queue_info().is_full() { sleep(Duration::from_secs(1)); }
match client.import_block(bytes) { match client.import_block(block) {
Err(BlockImportError(BlockImportErrorKind::Import(ImportErrorKind::AlreadyInChain), _)) => { Err(BlockImportError(BlockImportErrorKind::Import(ImportErrorKind::AlreadyInChain), _)) => {
trace!("Skipping block already in chain."); trace!("Skipping block already in chain.");
} }

View File

@ -43,7 +43,6 @@ extern crate jsonrpc_ipc_server as ipc;
extern crate jsonrpc_pubsub; extern crate jsonrpc_pubsub;
extern crate ethash; extern crate ethash;
#[cfg_attr(test, macro_use)]
extern crate ethcore; extern crate ethcore;
extern crate parity_bytes as bytes; extern crate parity_bytes as bytes;
extern crate parity_crypto as crypto; extern crate parity_crypto as crypto;

View File

@ -20,14 +20,13 @@ use std::sync::Arc;
use ethereum_types::{H256, Address}; use ethereum_types::{H256, Address};
use ethcore::account_provider::AccountProvider; use ethcore::account_provider::AccountProvider;
use ethcore::block::Block;
use ethcore::client::{BlockChainClient, Client, ClientConfig, ChainInfo, ImportBlock}; use ethcore::client::{BlockChainClient, Client, ClientConfig, ChainInfo, ImportBlock};
use ethcore::ethereum; use ethcore::ethereum;
use ethcore::ids::BlockId; use ethcore::ids::BlockId;
use ethcore::miner::Miner; use ethcore::miner::Miner;
use ethcore::spec::{Genesis, Spec}; use ethcore::spec::{Genesis, Spec};
use ethcore::test_helpers; use ethcore::test_helpers;
use ethcore::views::BlockView; use ethcore::verification::queue::kind::blocks::Unverified;
use ethjson::blockchain::BlockChain; use ethjson::blockchain::BlockChain;
use ethjson::state::test::ForkSpec; use ethjson::state::test::ForkSpec;
use io::IoChannel; use io::IoChannel;
@ -85,9 +84,9 @@ impl EthTester {
fn from_chain(chain: &BlockChain) -> Self { fn from_chain(chain: &BlockChain) -> Self {
let tester = Self::from_spec(make_spec(chain)); let tester = Self::from_spec(make_spec(chain));
for b in &chain.blocks_rlp() { for b in chain.blocks_rlp() {
if Block::is_good(&b) { if let Ok(block) = Unverified::from_rlp(b) {
let _ = tester.client.import_block(b.clone()); let _ = tester.client.import_block(block);
tester.client.flush_queue(); tester.client.flush_queue();
tester.client.import_verified_blocks(); tester.client.import_verified_blocks();
} }
@ -423,11 +422,11 @@ fn verify_transaction_counts(name: String, chain: BlockChain) {
let tester = EthTester::from_chain(&chain); let tester = EthTester::from_chain(&chain);
let mut id = 1; let mut id = 1;
for b in chain.blocks_rlp().iter().filter(|b| Block::is_good(b)).map(|b| view!(BlockView, b)) { for b in chain.blocks_rlp().into_iter().filter_map(|b| Unverified::from_rlp(b).ok()) {
let count = b.transactions_count(); let count = b.transactions.len();
let hash = b.hash(); let hash = b.header.hash();
let number = b.header_view().number(); let number = b.header.number();
let (req, res) = by_hash(hash, count, &mut id); let (req, res) = by_hash(hash, count, &mut id);
assert_eq!(tester.handler.handle_request_sync(&req), Some(res)); assert_eq!(tester.handler.handle_request_sync(&req), Some(res));