From b4ae1b6528fb13821ea6524cc67fded852f776e2 Mon Sep 17 00:00:00 2001 From: Marek Kotewicz Date: Thu, 2 Aug 2018 11:20:46 +0200 Subject: [PATCH] decode block rlp less often (#9252) in total: - removed 4 redundant rlp deserializations - avoid 1 redundant block data copy --- ethcore/src/block.rs | 59 +++++-------------- ethcore/src/client/client.rs | 55 ++++++++--------- ethcore/src/client/test_client.rs | 16 ++--- ethcore/src/client/traits.rs | 5 +- ethcore/src/engines/validator_set/multi.rs | 3 +- .../engines/validator_set/safe_contract.rs | 3 +- ethcore/src/json_tests/chain.rs | 8 +-- ethcore/src/miner/miner.rs | 2 +- ethcore/src/test_helpers.rs | 9 +-- ethcore/src/tests/client.rs | 22 +------ ethcore/src/tests/trace.rs | 7 ++- ethcore/sync/src/block_sync.rs | 24 ++++---- ethcore/sync/src/blocks.rs | 2 +- ethcore/sync/src/chain/handler.rs | 44 +++++++------- parity/blockchain.rs | 4 +- rpc/src/lib.rs | 1 - rpc/src/v1/tests/eth.rs | 17 +++--- 17 files changed, 116 insertions(+), 165 deletions(-) diff --git a/ethcore/src/block.rs b/ethcore/src/block.rs index 6f1ba7a2f..9fd3957fb 100644 --- a/ethcore/src/block.rs +++ b/ethcore/src/block.rs @@ -40,7 +40,7 @@ use engines::EthEngine; use error::{Error, BlockError}; use ethereum_types::{H256, U256, Address, Bloom}; use factory::Factories; -use hash::{keccak, KECCAK_NULL_RLP, KECCAK_EMPTY_LIST_RLP}; +use hash::keccak; use header::{Header, ExtendedHeader}; use receipt::{Receipt, TransactionOutcome}; 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 unexpected::{Mismatch, OutOfBounds}; use verification::PreverifiedBlock; -use views::BlockView; use vm::{EnvInfo, LastHashes}; /// A block, encoded as it is on the block chain. @@ -66,11 +65,6 @@ pub struct 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::().is_ok() - } - /// Get the RLP-encoding of the block with the seal. pub fn rlp_bytes(&self) -> Bytes { let mut block_rlp = RlpStream::new_list(3); @@ -398,26 +392,11 @@ impl<'x> OpenBlock<'x> { /// Turn this into a `ClosedBlock`. pub fn close(self) -> Result { - let mut s = self; - - 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)); + let unclosed_state = self.block.state.clone(); + let locked = self.close_and_lock()?; Ok(ClosedBlock { - block: s.block, + block: locked.block, unclosed_state, }) } @@ -429,18 +408,11 @@ impl<'x> OpenBlock<'x> { s.engine.on_close_block(&mut s.block)?; 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()))); - } - 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_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 @@ -537,18 +509,16 @@ impl LockedBlock { self, engine: &EthEngine, seal: Vec, - ) -> Result { + ) -> Result { 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) { - Err(e) => Err((e, s)), - _ => Ok(SealedBlock { - block: s.block - }), - } + engine.verify_local_seal(&s.block.header)?; + Ok(SealedBlock { + block: s.block + }) } } @@ -637,12 +607,11 @@ pub fn enact_verified( is_epoch_begin: bool, ancestry: &mut Iterator, ) -> Result { - let view = view!(BlockView, &block.bytes); enact( block.header, block.transactions, - view.uncles(), + block.uncles, engine, tracing, db, diff --git a/ethcore/src/client/client.rs b/ethcore/src/client/client.rs index d69803980..02af5ff0a 100644 --- a/ethcore/src/client/client.rs +++ b/ethcore/src/client/client.rs @@ -73,6 +73,8 @@ use types::filter::Filter; use types::ancestry_action::AncestryAction; use verification; use verification::{PreverifiedBlock, Verifier, BlockQueue}; +use verification::queue::kind::blocks::Unverified; +use verification::queue::kind::BlockLike; // re-export 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: Arc, - VecDeque<(Header, encoded::Block, Bytes)> + VecDeque<(Unverified, Bytes)> )>>, ancient_blocks_import_lock: Arc>, /// Consensus messages import queue @@ -428,7 +430,7 @@ impl Importer { /// /// The block is guaranteed to be the next best blocks in the /// 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 _import_lock = self.import_lock.lock(); @@ -436,11 +438,11 @@ impl Importer { trace_time!("import_old_block"); // verify the block, passing the chain for updating the epoch verifier. let mut rng = OsRng::new()?; - self.ancient_verifier.verify(&mut rng, &header, &chain)?; + self.ancient_verifier.verify(&mut rng, &unverified.header, &chain)?; // Commit results 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 db.write_buffered(batch); chain.commit(); @@ -1381,22 +1383,15 @@ impl CallContract for Client { } impl ImportBlock for Client { - fn import_block(&self, bytes: Bytes) -> Result { - use verification::queue::kind::BlockLike; - use verification::queue::kind::blocks::Unverified; - - // 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()))); - } + fn import_block(&self, unverified: Unverified) -> Result { + 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()))); + } + 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 { + fn queue_ancient_block(&self, unverified: Unverified, receipts_bytes: Bytes) -> Result { 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 if self.chain.read().is_known(&hash) { 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 // (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 { - 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 { - 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(); 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(); @@ -2065,11 +2059,10 @@ impl IoClient for Client { let _lock = lock.lock(); for _i in 0..MAX_ANCIENT_BLOCKS_TO_IMPORT { let first = queued.write().1.pop_front(); - if let Some((header, block_bytes, receipts_bytes)) = first { - let hash = header.hash(); + if let Some((unverified, receipts_bytes)) = first { + let hash = unverified.hash(); let result = client.importer.import_old_block( - &header, - block_bytes, + unverified, &receipts_bytes, &**client.db.read().key_value(), &*client.chain.read(), diff --git a/ethcore/src/client/test_client.rs b/ethcore/src/client/test_client.rs index 7a3dfcd00..627d844ae 100644 --- a/ethcore/src/client/test_client.rs +++ b/ethcore/src/client/test_client.rs @@ -53,6 +53,7 @@ use spec::Spec; use types::basic_account::BasicAccount; use types::pruning_info::PruningInfo; use verification::queue::QueueInfo; +use verification::queue::kind::blocks::Unverified; use block::{OpenBlock, SealedBlock, ClosedBlock}; use executive::Executed; use error::CallError; @@ -280,7 +281,8 @@ impl TestBlockChainClient { rlp.append(&header); rlp.append_raw(&txs, 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 { - fn import_block(&self, b: Bytes) -> Result { - let header = view!(BlockView, &b).header(); + fn import_block(&self, unverified: Unverified) -> Result { + let header = unverified.header; let h = header.hash(); let number: usize = header.number() as usize; if number > self.blocks.read().len() { @@ -539,7 +541,7 @@ impl ImportBlock for TestBlockChainClient { *difficulty = *difficulty + header.difficulty().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()); let mut parent_hash = header.parent_hash().clone(); if number > 0 { @@ -552,7 +554,7 @@ impl ImportBlock for TestBlockChainClient { } } else { - self.blocks.write().insert(h.clone(), b.to_vec()); + self.blocks.write().insert(h.clone(), unverified.bytes); } Ok(h) } @@ -856,8 +858,8 @@ impl IoClient for TestBlockChainClient { self.miner.import_external_transactions(self, txs); } - fn queue_ancient_block(&self, b: Bytes, _r: Bytes) -> Result { - self.import_block(b) + fn queue_ancient_block(&self, unverified: Unverified, _r: Bytes) -> Result { + self.import_block(unverified) } fn queue_consensus_message(&self, message: Bytes) { diff --git a/ethcore/src/client/traits.rs b/ethcore/src/client/traits.rs index 65bf00921..3f5595f61 100644 --- a/ethcore/src/client/traits.rs +++ b/ethcore/src/client/traits.rs @@ -34,6 +34,7 @@ use receipt::LocalizedReceipt; use trace::LocalizedTrace; use transaction::{self, LocalizedTransaction, SignedTransaction}; use verification::queue::QueueInfo as BlockQueueInfo; +use verification::queue::kind::blocks::Unverified; use state::StateInfo; use header::Header; use engines::EthEngine; @@ -167,7 +168,7 @@ pub trait RegistryInfo { /// Provides methods to import block into blockchain pub trait ImportBlock { /// Import a block into the blockchain. - fn import_block(&self, bytes: Bytes) -> Result; + fn import_block(&self, block: Unverified) -> Result; } /// Provides `call_contract` method @@ -204,7 +205,7 @@ pub trait IoClient: Sync + Send { fn queue_transactions(&self, transactions: Vec, peer_id: usize); /// Queue block import with transaction receipts. Does no sealing and transaction validation. - fn queue_ancient_block(&self, block_bytes: Bytes, receipts_bytes: Bytes) -> Result; + fn queue_ancient_block(&self, block_bytes: Unverified, receipts_bytes: Bytes) -> Result; /// Queue conensus engine message. fn queue_consensus_message(&self, message: Bytes); diff --git a/ethcore/src/engines/validator_set/multi.rs b/ethcore/src/engines/validator_set/multi.rs index a23208cd2..24fb2d890 100644 --- a/ethcore/src/engines/validator_set/multi.rs +++ b/ethcore/src/engines/validator_set/multi.rs @@ -158,6 +158,7 @@ mod tests { use test_helpers::{generate_dummy_client_with_spec_and_accounts, generate_dummy_client_with_spec_and_data}; use types::ids::BlockId; use ethereum_types::Address; + use verification::queue::kind::blocks::Unverified; 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, &[]); sync_client.engine().register_client(Arc::downgrade(&sync_client) as _); 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(); assert_eq!(sync_client.chain_info().best_block_number, 3); diff --git a/ethcore/src/engines/validator_set/safe_contract.rs b/ethcore/src/engines/validator_set/safe_contract.rs index a7f4f2c73..adaa63f0b 100644 --- a/ethcore/src/engines/validator_set/safe_contract.rs +++ b/ethcore/src/engines/validator_set/safe_contract.rs @@ -458,6 +458,7 @@ mod tests { use test_helpers::{generate_dummy_client_with_spec_and_accounts, generate_dummy_client_with_spec_and_data}; use super::super::ValidatorSet; use super::{ValidatorSafeContract, EVENT_NAME_HASH}; + use verification::queue::kind::blocks::Unverified; #[test] 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, &[]); sync_client.engine().register_client(Arc::downgrade(&sync_client) as _); 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(); assert_eq!(sync_client.chain_info().best_block_number, 3); diff --git a/ethcore/src/json_tests/chain.rs b/ethcore/src/json_tests/chain.rs index 2d643e75f..83a940fcb 100644 --- a/ethcore/src/json_tests/chain.rs +++ b/ethcore/src/json_tests/chain.rs @@ -17,12 +17,12 @@ use std::path::Path; use std::sync::Arc; use client::{EvmTestClient, Client, ClientConfig, ChainInfo, ImportBlock}; -use block::Block; use spec::Genesis; use ethjson; use miner::Miner; use io::IoChannel; use test_helpers; +use verification::queue::kind::blocks::Unverified; use super::HookType; @@ -83,9 +83,9 @@ pub fn json_chain_test(json_data: &[u8], start_stop_ho Arc::new(Miner::new_for_tests(&spec, None)), IoChannel::disconnected(), ).unwrap(); - for b in &blockchain.blocks_rlp() { - if Block::is_good(&b) { - let _ = client.import_block(b.clone()); + for b in blockchain.blocks_rlp() { + if let Ok(block) = Unverified::from_rlp(b) { + let _ = client.import_block(block); client.flush_queue(); client.import_verified_blocks(); } diff --git a/ethcore/src/miner/miner.rs b/ethcore/src/miner/miner.rs index 233e28139..38acf9e1e 100644 --- a/ethcore/src/miner/miner.rs +++ b/ethcore/src/miner/miner.rs @@ -1148,7 +1148,7 @@ impl miner::MinerService for Miner { |b| &b.hash() == &block_hash ) { 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); Err(ErrorKind::PowInvalid.into()) }) diff --git a/ethcore/src/test_helpers.rs b/ethcore/src/test_helpers.rs index 408714571..c873db5d1 100644 --- a/ethcore/src/test_helpers.rs +++ b/ethcore/src/test_helpers.rs @@ -43,6 +43,7 @@ use blooms_db; use kvdb::KeyValueDB; use kvdb_rocksdb; use tempdir::TempDir; +use verification::queue::kind::blocks::Unverified; use encoded; /// Creates test block with corresponding header @@ -175,7 +176,7 @@ pub fn generate_dummy_client_with_spec_accounts_and_data(test_spec: F, accoun 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); } @@ -211,7 +212,7 @@ pub fn push_blocks_to_client(client: &Arc, timestamp_salt: u64, starting rolling_block_number = rolling_block_number + 1; 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); } } @@ -231,7 +232,7 @@ pub fn push_block_with_transactions(client: &Arc, transactions: &[Signed } 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); } @@ -253,7 +254,7 @@ pub fn get_test_client_with_blocks(blocks: Vec) -> Arc { ).unwrap(); 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); } } diff --git a/ethcore/src/tests/client.rs b/ethcore/src/tests/client.rs index a7629313d..4031d30b0 100644 --- a/ethcore/src/tests/client.rs +++ b/ethcore/src/tests/client.rs @@ -35,9 +35,9 @@ use views::BlockView; use ethkey::KeyPair; use transaction::{PendingTransaction, Transaction, Action, Condition}; use miner::MinerService; -use rlp::{RlpStream, EMPTY_LIST_RLP}; use tempdir::TempDir; use test_helpers; +use verification::queue::kind::blocks::Unverified; #[test] fn imports_from_empty() { @@ -97,7 +97,7 @@ fn imports_good_block() { IoChannel::disconnected(), ).unwrap(); 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"); } client.flush_queue(); @@ -107,24 +107,6 @@ fn imports_good_block() { 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] fn query_none_block() { let db = test_helpers::new_db(); diff --git a/ethcore/src/tests/trace.rs b/ethcore/src/tests/trace.rs index fd7c932cb..24ef37800 100644 --- a/ethcore/src/tests/trace.rs +++ b/ethcore/src/tests/trace.rs @@ -33,6 +33,7 @@ use views::BlockView; use trace::{RewardType, LocalizedTrace}; use trace::trace::Action::Reward; use test_helpers; +use verification::queue::kind::blocks::Unverified; #[test] 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(); - 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); } @@ -120,7 +121,7 @@ fn can_trace_block_and_uncle_reward() { 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); } @@ -170,7 +171,7 @@ fn can_trace_block_and_uncle_reward() { 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() { panic!("error importing block: {:#?}", res.err().unwrap()); } diff --git a/ethcore/sync/src/block_sync.rs b/ethcore/sync/src/block_sync.rs index cc04fb04d..588bfc0c7 100644 --- a/ethcore/sync/src/block_sync.rs +++ b/ethcore/sync/src/block_sync.rs @@ -23,11 +23,10 @@ use std::cmp; use heapsize::HeapSizeOf; use ethereum_types::H256; use rlp::{self, Rlp}; -use ethcore::views::BlockView; use ethcore::header::{BlockNumber, Header as BlockHeader}; use ethcore::client::{BlockStatus, BlockId, BlockImportError, BlockImportErrorKind}; -use ethcore::block::Block; use ethcore::error::{ImportErrorKind, BlockError}; +use ethcore::verification::queue::kind::blocks::Unverified; use sync_io::SyncIo; use blocks::BlockCollection; @@ -484,18 +483,19 @@ impl BlockDownloader { let block = block_and_receipts.block; let receipts = block_and_receipts.receipts; - // Perform basic block verification - if !Block::is_good(&block) { - debug!(target: "sync", "Bad block rlp: {:?}", block); - bad = true; - break; - } - - let (h, number, parent) = { - let header = view!(BlockView, &block).header_view(); - (header.hash(), header.number(), header.parent_hash()) + let block = match Unverified::from_rlp(block) { + Ok(block) => block, + Err(_) => { + debug!(target: "sync", "Bad block rlp"); + bad = true; + break; + } }; + 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) { self.state = State::Complete; trace!(target: "sync", "Sync target reached"); diff --git a/ethcore/sync/src/blocks.rs b/ethcore/sync/src/blocks.rs index df7d7a3bf..248180b28 100644 --- a/ethcore/sync/src/blocks.rs +++ b/ethcore/sync/src/blocks.rs @@ -294,7 +294,7 @@ impl BlockCollection { let header = view!(HeaderView, &block.header); let block_view = Block::new_from_header_and_body(&header, &body); drained.push(BlockAndReceipts { - block: block_view.rlp().as_raw().to_vec(), + block: block_view.into_inner(), receipts: block.receipts.clone(), }); } diff --git a/ethcore/sync/src/chain/handler.rs b/ethcore/sync/src/chain/handler.rs index 136ff3395..8547be7b3 100644 --- a/ethcore/sync/src/chain/handler.rs +++ b/ethcore/sync/src/chain/handler.rs @@ -19,8 +19,9 @@ use block_sync::{BlockDownloaderImportError as DownloaderImportError, DownloadAc use bytes::Bytes; use ethcore::client::{BlockStatus, BlockId, BlockImportError, BlockImportErrorKind}; use ethcore::error::*; -use ethcore::header::{BlockNumber, Header as BlockHeader}; +use ethcore::header::BlockNumber; use ethcore::snapshot::{ManifestData, RestorationStatus}; +use ethcore::verification::queue::kind::blocks::Unverified; use ethereum_types::{H256, U256}; use hash::keccak; use network::PeerId; @@ -162,44 +163,43 @@ impl SyncHandler { peer.difficulty = Some(difficulty); } } - let block_rlp = r.at(0)?; - let header_rlp = block_rlp.at(0)?; - let h = keccak(&header_rlp.as_raw()); - trace!(target: "sync", "{} -> NewBlock ({})", peer_id, h); - let header: BlockHeader = header_rlp.as_val()?; - if header.number() > sync.highest_block.unwrap_or(0) { - sync.highest_block = Some(header.number()); + let block = Unverified::from_rlp(r.at(0)?.as_raw().to_vec())?; + let hash = block.header.hash(); + let number = block.header.number(); + trace!(target: "sync", "{} -> NewBlock ({})", peer_id, hash); + if number > sync.highest_block.unwrap_or(0) { + sync.highest_block = Some(number); } let mut unknown = false; - { - if let Some(ref mut peer) = sync.peers.get_mut(&peer_id) { - peer.latest_hash = header.hash(); - } + + if let Some(ref mut peer) = sync.peers.get_mut(&peer_id) { + peer.latest_hash = hash; } + 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 { - trace!(target: "sync", "Ignored ancient new block {:?}", h); + if last_imported_number > number && last_imported_number - number > MAX_NEW_BLOCK_AGE { + trace!(target: "sync", "Ignored ancient new block {:?}", hash); 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), _)) => { - trace!(target: "sync", "New block already in chain {:?}", h); + trace!(target: "sync", "New block already in chain {:?}", hash); }, Err(BlockImportError(BlockImportErrorKind::Import(ImportErrorKind::AlreadyQueued), _)) => { - trace!(target: "sync", "New block already queued {:?}", h); + trace!(target: "sync", "New block already queued {:?}", hash); }, Ok(_) => { // abort current download of the same block sync.complete_sync(io); - sync.new_blocks.mark_as_known(&header.hash(), header.number()); - trace!(target: "sync", "New block queued {:?} ({})", h, header.number()); + sync.new_blocks.mark_as_known(&hash, number); + trace!(target: "sync", "New block queued {:?} ({})", hash, number); }, Err(BlockImportError(BlockImportErrorKind::Block(BlockError::UnknownParent(p)), _)) => { unknown = true; - trace!(target: "sync", "New block with unknown parent ({:?}) {:?}", p, h); + trace!(target: "sync", "New block with unknown parent ({:?}) {:?}", p, hash); }, Err(e) => { - debug!(target: "sync", "Bad new block {:?} : {:?}", h, e); + debug!(target: "sync", "Bad new block {:?} : {:?}", hash, e); return Err(DownloaderImportError::Invalid); } }; @@ -207,7 +207,7 @@ impl SyncHandler { if sync.state != SyncState::Idle { trace!(target: "sync", "NewBlock ignored while seeking"); } else { - trace!(target: "sync", "New unknown block {:?}", h); + trace!(target: "sync", "New unknown block {:?}", hash); //TODO: handle too many unknown blocks sync.sync_peer(io, peer_id, true); } diff --git a/parity/blockchain.rs b/parity/blockchain.rs index 21af2968e..cc92419da 100644 --- a/parity/blockchain.rs +++ b/parity/blockchain.rs @@ -30,6 +30,7 @@ use ethcore::client::{Mode, DatabaseCompactionProfile, VMType, BlockImportError, use ethcore::error::{ImportErrorKind, BlockImportErrorKind}; use ethcore::miner::Miner; use ethcore::verification::queue::VerifierSettings; +use ethcore::verification::queue::kind::blocks::Unverified; use ethcore_service::ClientService; use cache::CacheConfig; 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())?; 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)); } - match client.import_block(bytes) { + match client.import_block(block) { Err(BlockImportError(BlockImportErrorKind::Import(ImportErrorKind::AlreadyInChain), _)) => { trace!("Skipping block already in chain."); } diff --git a/rpc/src/lib.rs b/rpc/src/lib.rs index 96926700c..2e731cd34 100644 --- a/rpc/src/lib.rs +++ b/rpc/src/lib.rs @@ -43,7 +43,6 @@ extern crate jsonrpc_ipc_server as ipc; extern crate jsonrpc_pubsub; extern crate ethash; -#[cfg_attr(test, macro_use)] extern crate ethcore; extern crate parity_bytes as bytes; extern crate parity_crypto as crypto; diff --git a/rpc/src/v1/tests/eth.rs b/rpc/src/v1/tests/eth.rs index 6e3b9855d..217e03290 100644 --- a/rpc/src/v1/tests/eth.rs +++ b/rpc/src/v1/tests/eth.rs @@ -20,14 +20,13 @@ use std::sync::Arc; use ethereum_types::{H256, Address}; use ethcore::account_provider::AccountProvider; -use ethcore::block::Block; use ethcore::client::{BlockChainClient, Client, ClientConfig, ChainInfo, ImportBlock}; use ethcore::ethereum; use ethcore::ids::BlockId; use ethcore::miner::Miner; use ethcore::spec::{Genesis, Spec}; use ethcore::test_helpers; -use ethcore::views::BlockView; +use ethcore::verification::queue::kind::blocks::Unverified; use ethjson::blockchain::BlockChain; use ethjson::state::test::ForkSpec; use io::IoChannel; @@ -85,9 +84,9 @@ impl EthTester { fn from_chain(chain: &BlockChain) -> Self { let tester = Self::from_spec(make_spec(chain)); - for b in &chain.blocks_rlp() { - if Block::is_good(&b) { - let _ = tester.client.import_block(b.clone()); + for b in chain.blocks_rlp() { + if let Ok(block) = Unverified::from_rlp(b) { + let _ = tester.client.import_block(block); tester.client.flush_queue(); tester.client.import_verified_blocks(); } @@ -423,11 +422,11 @@ fn verify_transaction_counts(name: String, chain: BlockChain) { let tester = EthTester::from_chain(&chain); let mut id = 1; - for b in chain.blocks_rlp().iter().filter(|b| Block::is_good(b)).map(|b| view!(BlockView, b)) { - let count = b.transactions_count(); + for b in chain.blocks_rlp().into_iter().filter_map(|b| Unverified::from_rlp(b).ok()) { + let count = b.transactions.len(); - let hash = b.hash(); - let number = b.header_view().number(); + let hash = b.header.hash(); + let number = b.header.number(); let (req, res) = by_hash(hash, count, &mut id); assert_eq!(tester.handler.handle_request_sync(&req), Some(res));