Snapshot sync part 2 (#2098)

* Split block downloader into a module

* Snapshot sync progress

* Warp sync CLI option

* Increased snapshot chunk and ping timeouts

* Fixed an issue with delayed writes

* Updated bootnodes

* Don't run pending IO tasks on shutdown

* Optional first_block; removed insert_snapshot_block

* Fixing expect calls

* Fixed stalled sync

* style and docs

* Update block_sync.rs

[ci:skip]
This commit is contained in:
Arkadiy Paronyan 2016-10-18 18:16:00 +02:00 committed by GitHub
parent dba2d79b56
commit 487dfb0208
36 changed files with 1347 additions and 528 deletions

View File

@ -164,7 +164,8 @@
"enode://a979fb575495b8d6db44f750317d0f4622bf4c2aa3365d6af7c284339968eef29b69ad0dce72a4d8db5ebb4968de0e3bec910127f134779fbcb0cb6d3331163c@52.16.188.185:30303", "enode://a979fb575495b8d6db44f750317d0f4622bf4c2aa3365d6af7c284339968eef29b69ad0dce72a4d8db5ebb4968de0e3bec910127f134779fbcb0cb6d3331163c@52.16.188.185:30303",
"enode://de471bccee3d042261d52e9bff31458daecc406142b401d4cd848f677479f73104b9fdeb090af9583d3391b7f10cb2ba9e26865dd5fca4fcdc0fb1e3b723c786@54.94.239.50:30303", "enode://de471bccee3d042261d52e9bff31458daecc406142b401d4cd848f677479f73104b9fdeb090af9583d3391b7f10cb2ba9e26865dd5fca4fcdc0fb1e3b723c786@54.94.239.50:30303",
"enode://1118980bf48b0a3640bdba04e0fe78b1add18e1cd99bf22d53daac1fd9972ad650df52176e7c7d89d1114cfef2bc23a2959aa54998a46afcf7d91809f0855082@52.74.57.123:30303", "enode://1118980bf48b0a3640bdba04e0fe78b1add18e1cd99bf22d53daac1fd9972ad650df52176e7c7d89d1114cfef2bc23a2959aa54998a46afcf7d91809f0855082@52.74.57.123:30303",
"enode://248f12bc8b18d5289358085520ac78cd8076485211e6d96ab0bc93d6cd25442db0ce3a937dc404f64f207b0b9aed50e25e98ce32af5ac7cb321ff285b97de485@zero.parity.io:30303" "enode://4cd540b2c3292e17cff39922e864094bf8b0741fcc8c5dcea14957e389d7944c70278d872902e3d0345927f621547efa659013c400865485ab4bfa0c6596936f@zero.parity.io:30303",
"enode://cc92c4c40d612a10c877ca023ef0496c843fbc92b6c6c0d55ce0b863d51d821c4bd70daebb54324a6086374e6dc05708fed39862b275f169cb678e655da9d07d@136.243.154.246:30303"
], ],
"accounts": { "accounts": {
"0000000000000000000000000000000000000001": { "builtin": { "name": "ecrecover", "pricing": { "linear": { "base": 3000, "word": 0 } } } }, "0000000000000000000000000000000000000001": { "builtin": { "name": "ecrecover", "pricing": { "linear": { "base": 3000, "word": 0 } } } },

View File

@ -29,3 +29,12 @@ pub struct BestBlock {
/// Best block uncompressed bytes /// Best block uncompressed bytes
pub block: Bytes, pub block: Bytes,
} }
/// Best ancient block info. If the blockchain has a gap this keeps track of where it starts.
#[derive(Default)]
pub struct BestAncientBlock {
/// Best block hash.
pub hash: H256,
/// Best block number.
pub number: BlockNumber,
}

View File

@ -27,7 +27,8 @@ use log_entry::{LogEntry, LocalizedLogEntry};
use receipt::Receipt; use receipt::Receipt;
use blooms::{Bloom, BloomGroup}; use blooms::{Bloom, BloomGroup};
use blockchain::block_info::{BlockInfo, BlockLocation, BranchBecomingCanonChainData}; use blockchain::block_info::{BlockInfo, BlockLocation, BranchBecomingCanonChainData};
use blockchain::best_block::BestBlock; use blockchain::best_block::{BestBlock, BestAncientBlock};
use types::blockchain_info::BlockChainInfo;
use types::tree_route::TreeRoute; use types::tree_route::TreeRoute;
use blockchain::update::ExtrasUpdate; use blockchain::update::ExtrasUpdate;
use blockchain::{CacheSize, ImportRoute, Config}; use blockchain::{CacheSize, ImportRoute, Config};
@ -43,16 +44,24 @@ pub trait BlockProvider {
/// (though not necessarily a part of the canon chain). /// (though not necessarily a part of the canon chain).
fn is_known(&self, hash: &H256) -> bool; fn is_known(&self, hash: &H256) -> bool;
/// Get the first block which this chain holds. /// Get the first block of the best part of the chain.
/// Return `None` if there is no gap and the first block is the genesis.
/// Any queries of blocks which precede this one are not guaranteed to /// Any queries of blocks which precede this one are not guaranteed to
/// succeed. /// succeed.
fn first_block(&self) -> H256; fn first_block(&self) -> Option<H256>;
/// Get the number of the first block. /// Get the number of the first block.
fn first_block_number(&self) -> BlockNumber { fn first_block_number(&self) -> Option<BlockNumber> {
self.block_number(&self.first_block()).expect("First block always stored; qed") self.first_block().map(|b| self.block_number(&b).expect("First block is always set to an existing block or `None`. Existing block always has a number; qed"))
} }
/// Get the best block of an first block sequence if there is a gap.
fn best_ancient_block(&self) -> Option<H256>;
/// Get the number of the first block.
fn best_ancient_number(&self) -> Option<BlockNumber> {
self.best_ancient_block().map(|h| self.block_number(&h).expect("Ancient block is always set to an existing block or `None`. Existing block always has a number; qed"))
}
/// Get raw block data /// Get raw block data
fn block(&self, hash: &H256) -> Option<Bytes>; fn block(&self, hash: &H256) -> Option<Bytes>;
@ -160,9 +169,14 @@ impl bc::group::BloomGroupDatabase for BlockChain {
pub struct BlockChain { pub struct BlockChain {
// All locks must be captured in the order declared here. // All locks must be captured in the order declared here.
blooms_config: bc::Config, blooms_config: bc::Config,
first_block: H256,
best_block: RwLock<BestBlock>, best_block: RwLock<BestBlock>,
// Stores best block of the first uninterrupted sequence of blocks. `None` if there are no gaps.
// Only updated with `insert_unordered_block`.
best_ancient_block: RwLock<Option<BestAncientBlock>>,
// Stores the last block of the last sequence of blocks. `None` if there are no gaps.
// This is calculated on start and does not get updated.
first_block: Option<H256>,
// block cache // block cache
block_headers: RwLock<HashMap<H256, Bytes>>, block_headers: RwLock<HashMap<H256, Bytes>>,
@ -191,8 +205,16 @@ impl BlockProvider for BlockChain {
self.db.exists_with_cache(db::COL_EXTRA, &self.block_details, hash) self.db.exists_with_cache(db::COL_EXTRA, &self.block_details, hash)
} }
fn first_block(&self) -> H256 { fn first_block(&self) -> Option<H256> {
self.first_block self.first_block.clone()
}
fn best_ancient_block(&self) -> Option<H256> {
self.best_ancient_block.read().as_ref().map(|b| b.hash.clone())
}
fn best_ancient_number(&self) -> Option<BlockNumber> {
self.best_ancient_block.read().as_ref().map(|b| b.number)
} }
/// Get raw block data /// Get raw block data
@ -400,8 +422,9 @@ impl BlockChain {
levels: LOG_BLOOMS_LEVELS, levels: LOG_BLOOMS_LEVELS,
elements_per_index: LOG_BLOOMS_ELEMENTS_PER_INDEX, elements_per_index: LOG_BLOOMS_ELEMENTS_PER_INDEX,
}, },
first_block: H256::zero(), first_block: None,
best_block: RwLock::new(BestBlock::default()), best_block: RwLock::new(BestBlock::default()),
best_ancient_block: RwLock::new(None),
block_headers: RwLock::new(HashMap::new()), block_headers: RwLock::new(HashMap::new()),
block_bodies: RwLock::new(HashMap::new()), block_bodies: RwLock::new(HashMap::new()),
block_details: RwLock::new(HashMap::new()), block_details: RwLock::new(HashMap::new()),
@ -443,7 +466,6 @@ impl BlockChain {
batch.write(db::COL_EXTRA, &header.number(), &hash); batch.write(db::COL_EXTRA, &header.number(), &hash);
batch.put(db::COL_EXTRA, b"best", &hash); batch.put(db::COL_EXTRA, b"best", &hash);
batch.put(db::COL_EXTRA, b"first", &hash);
bc.db.write(batch).expect("Low level database error. Some issue with disk?"); bc.db.write(batch).expect("Low level database error. Some issue with disk?");
hash hash
} }
@ -455,32 +477,45 @@ impl BlockChain {
let best_block_total_difficulty = bc.block_details(&best_block_hash).unwrap().total_difficulty; let best_block_total_difficulty = bc.block_details(&best_block_hash).unwrap().total_difficulty;
let best_block_rlp = bc.block(&best_block_hash).unwrap(); let best_block_rlp = bc.block(&best_block_hash).unwrap();
let raw_first = bc.db.get(db::COL_EXTRA, b"first").unwrap().map_or(Vec::new(), |v| v.to_vec()); let raw_first = bc.db.get(db::COL_EXTRA, b"first").unwrap().map(|v| v.to_vec());
let mut best_ancient = bc.db.get(db::COL_EXTRA, b"ancient").unwrap().map(|h| H256::from_slice(&h));
let best_ancient_number;
if best_ancient.is_none() && best_block_number > 1 && bc.block_hash(1).is_none() {
best_ancient = Some(bc.genesis_hash());
best_ancient_number = Some(0);
} else {
best_ancient_number = best_ancient.as_ref().and_then(|h| bc.block_number(h));
}
// binary search for the first block. // binary search for the first block.
if raw_first.is_empty() { match raw_first {
let (mut f, mut hash) = (best_block_number, best_block_hash); None => {
let mut l = 0; let (mut f, mut hash) = (best_block_number, best_block_hash);
let mut l = best_ancient_number.unwrap_or(0);
loop { loop {
if l >= f { break; } if l >= f { break; }
let step = (f - l) >> 1; let step = (f - l) >> 1;
let m = l + step; let m = l + step;
match bc.block_hash(m) { match bc.block_hash(m) {
Some(h) => { f = m; hash = h }, Some(h) => { f = m; hash = h },
None => { l = m + 1 }, None => { l = m + 1 },
}
} }
}
let mut batch = db.transaction(); if hash != bc.genesis_hash() {
batch.put(db::COL_EXTRA, b"first", &hash); trace!("First block calculated: {:?}", hash);
db.write(batch).expect("Low level database error."); let mut batch = db.transaction();
batch.put(db::COL_EXTRA, b"first", &hash);
bc.first_block = hash; db.write(batch).expect("Low level database error.");
} else { bc.first_block = Some(hash);
bc.first_block = H256::from_slice(&raw_first); }
},
Some(raw_first) => {
bc.first_block = Some(H256::from_slice(&raw_first));
},
} }
// and write them // and write them
@ -491,6 +526,14 @@ impl BlockChain {
hash: best_block_hash, hash: best_block_hash,
block: best_block_rlp, 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 {
hash: hash,
number: number,
});
}
} }
bc bc
@ -644,11 +687,12 @@ impl BlockChain {
/// Inserts a verified, known block from the canonical chain. /// Inserts a verified, known block from the canonical chain.
/// ///
/// Can be performed out-of-order, but care must be taken that the final chain is in a correct state. /// Can be performed out-of-order, but care must be taken that the final chain is in a correct state.
/// This is used by snapshot restoration. /// This is used by snapshot restoration and when downloading missing blocks for the chain gap.
/// /// `is_best` forces the best block to be updated to this block.
/// `is_ancient` forces the best block of the first block sequence to be updated to this block.
/// Supply a dummy parent total difficulty when the parent block may not be in the chain. /// Supply a dummy parent total difficulty when the parent block may not be in the chain.
/// Returns true if the block is disconnected. /// Returns true if the block is disconnected.
pub fn insert_snapshot_block(&self, bytes: &[u8], receipts: Vec<Receipt>, parent_td: Option<U256>, is_best: bool) -> bool { pub fn insert_unordered_block(&self, batch: &mut DBTransaction, bytes: &[u8], receipts: Vec<Receipt>, parent_td: Option<U256>, is_best: bool, is_ancient: bool) -> bool {
let block = BlockView::new(bytes); let block = BlockView::new(bytes);
let header = block.header_view(); let header = block.header_view();
let hash = header.sha3(); let hash = header.sha3();
@ -659,8 +703,6 @@ impl BlockChain {
assert!(self.pending_best_block.read().is_none()); assert!(self.pending_best_block.read().is_none());
let mut batch = self.db.transaction();
let block_rlp = UntrustedRlp::new(bytes); let block_rlp = UntrustedRlp::new(bytes);
let compressed_header = block_rlp.at(0).unwrap().compress(RlpType::Blocks); let compressed_header = block_rlp.at(0).unwrap().compress(RlpType::Blocks);
let compressed_body = UntrustedRlp::new(&Self::block_to_body(bytes)).compress(RlpType::Blocks); let compressed_body = UntrustedRlp::new(&Self::block_to_body(bytes)).compress(RlpType::Blocks);
@ -674,13 +716,13 @@ impl BlockChain {
if let Some(parent_details) = maybe_parent { if let Some(parent_details) = maybe_parent {
// parent known to be in chain. // parent known to be in chain.
let info = BlockInfo { let info = BlockInfo {
hash: hash, hash: hash.clone(),
number: header.number(), number: header.number(),
total_difficulty: parent_details.total_difficulty + header.difficulty(), total_difficulty: parent_details.total_difficulty + header.difficulty(),
location: BlockLocation::CanonChain, location: BlockLocation::CanonChain,
}; };
self.prepare_update(&mut batch, ExtrasUpdate { self.prepare_update(batch, ExtrasUpdate {
block_hashes: self.prepare_block_hashes_update(bytes, &info), block_hashes: self.prepare_block_hashes_update(bytes, &info),
block_details: self.prepare_block_details_update(bytes, &info), block_details: self.prepare_block_details_update(bytes, &info),
block_receipts: self.prepare_block_receipts_update(receipts, &info), block_receipts: self.prepare_block_receipts_update(receipts, &info),
@ -689,7 +731,21 @@ impl BlockChain {
info: info, info: info,
block: bytes block: bytes
}, is_best); }, is_best);
self.db.write(batch).unwrap();
if is_ancient {
let mut best_ancient_block = self.best_ancient_block.write();
let ancient_number = best_ancient_block.as_ref().map_or(0, |b| b.number);
if self.block_hash(header.number() + 1).is_some() {
batch.delete(db::COL_EXTRA, b"ancient");
*best_ancient_block = None;
} else if header.number() > ancient_number {
batch.put(db::COL_EXTRA, b"ancient", &hash);
*best_ancient_block = Some(BestAncientBlock {
hash: hash,
number: header.number(),
});
}
}
false false
} else { } else {
@ -714,7 +770,7 @@ impl BlockChain {
let mut update = HashMap::new(); let mut update = HashMap::new();
update.insert(hash, block_details); update.insert(hash, block_details);
self.prepare_update(&mut batch, ExtrasUpdate { self.prepare_update(batch, ExtrasUpdate {
block_hashes: self.prepare_block_hashes_update(bytes, &info), block_hashes: self.prepare_block_hashes_update(bytes, &info),
block_details: update, block_details: update,
block_receipts: self.prepare_block_receipts_update(receipts, &info), block_receipts: self.prepare_block_receipts_update(receipts, &info),
@ -723,8 +779,6 @@ impl BlockChain {
info: info, info: info,
block: bytes, block: bytes,
}, is_best); }, is_best);
self.db.write(batch).unwrap();
true true
} }
} }
@ -1210,6 +1264,24 @@ impl BlockChain {
body.append_raw(block_rlp.at(2).as_raw(), 1); body.append_raw(block_rlp.at(2).as_raw(), 1);
body.out() body.out()
} }
/// Returns general blockchain information
pub fn chain_info(&self) -> BlockChainInfo {
// ensure data consistencly by locking everything first
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(),
genesis_hash: self.genesis_hash(),
best_block_hash: best_block.hash.clone(),
best_block_number: best_block.number,
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.clone()),
ancient_block_number: best_ancient_block.as_ref().map(|b| b.number),
}
}
} }
#[cfg(test)] #[cfg(test)]

View File

@ -30,7 +30,7 @@ use util::kvdb::*;
// other // other
use io::*; use io::*;
use views::{HeaderView, BodyView}; use views::{HeaderView, BodyView, BlockView};
use error::{ImportError, ExecutionError, CallError, BlockError, ImportResult, Error as EthcoreError}; use error::{ImportError, ExecutionError, CallError, BlockError, ImportResult, Error as EthcoreError};
use header::BlockNumber; use header::BlockNumber;
use state::State; use state::State;
@ -431,6 +431,29 @@ impl Client {
imported imported
} }
/// Import a block with transaction receipts.
/// 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, block_bytes: Bytes, receipts_bytes: Bytes) -> H256 {
let block = BlockView::new(&block_bytes);
let hash = block.header().hash();
let _import_lock = self.import_lock.lock();
{
let _timer = PerfTimer::new("import_old_block");
let chain = self.chain.read();
// Commit results
let receipts = ::rlp::decode(&receipts_bytes);
let mut batch = DBTransaction::new(&self.db.read());
chain.insert_unordered_block(&mut batch, &block_bytes, receipts, None, false, true);
// Final commit to the DB
self.db.read().write_buffered(batch);
chain.commit();
}
self.db.read().flush().expect("DB flush failed.");
hash
}
fn commit_block<B>(&self, block: B, hash: &H256, block_data: &[u8]) -> ImportRoute where B: IsBlock + Drain { fn commit_block<B>(&self, block: B, hash: &H256, block_data: &[u8]) -> ImportRoute where B: IsBlock + Drain {
let number = block.header().number(); let number = block.header().number();
let parent = block.header().parent_hash().clone(); let parent = block.header().parent_hash().clone();
@ -998,6 +1021,20 @@ impl BlockChainClient for Client {
Ok(try!(self.block_queue.import(unverified))) Ok(try!(self.block_queue.import(unverified)))
} }
fn import_block_with_receipts(&self, block_bytes: Bytes, receipts_bytes: Bytes) -> Result<H256, BlockImportError> {
{
// check block order
let header = BlockView::new(&block_bytes).header_view();
if self.chain.read().is_known(&header.hash()) {
return Err(BlockImportError::Import(ImportError::AlreadyInChain));
}
if self.block_status(BlockID::Hash(header.parent_hash())) == BlockStatus::Unknown {
return Err(BlockImportError::Block(BlockError::UnknownParent(header.parent_hash())));
}
}
Ok(self.import_old_block(block_bytes, receipts_bytes))
}
fn queue_info(&self) -> BlockQueueInfo { fn queue_info(&self) -> BlockQueueInfo {
self.block_queue.queue_info() self.block_queue.queue_info()
} }
@ -1007,14 +1044,7 @@ impl BlockChainClient for Client {
} }
fn chain_info(&self) -> BlockChainInfo { fn chain_info(&self) -> BlockChainInfo {
let chain = self.chain.read(); self.chain.read().chain_info()
BlockChainInfo {
total_difficulty: chain.best_block_total_difficulty(),
pending_total_difficulty: chain.best_block_total_difficulty(),
genesis_hash: chain.genesis_hash(),
best_block_hash: chain.best_block_hash(),
best_block_number: From::from(chain.best_block_number())
}
} }
fn additional_params(&self) -> BTreeMap<String, String> { fn additional_params(&self) -> BTreeMap<String, String> {
@ -1146,21 +1176,22 @@ impl MiningBlockChainClient for Client {
} }
fn import_sealed_block(&self, block: SealedBlock) -> ImportResult { fn import_sealed_block(&self, block: SealedBlock) -> ImportResult {
let _import_lock = self.import_lock.lock();
let _timer = PerfTimer::new("import_sealed_block");
let start = precise_time_ns();
let h = block.header().hash(); let h = block.header().hash();
let number = block.header().number(); let start = precise_time_ns();
let route = {
let block_data = block.rlp_bytes(); // scope for self.import_lock
let route = self.commit_block(block, &h, &block_data); let _import_lock = self.import_lock.lock();
trace!(target: "client", "Imported sealed block #{} ({})", number, h); let _timer = PerfTimer::new("import_sealed_block");
self.state_db.lock().sync_cache(&route.enacted, &route.retracted, false);
let number = block.header().number();
let block_data = block.rlp_bytes();
let route = self.commit_block(block, &h, &block_data);
trace!(target: "client", "Imported sealed block #{} ({})", number, h);
self.state_db.lock().sync_cache(&route.enacted, &route.retracted, false);
route
};
let (enacted, retracted) = self.calculate_enacted_retracted(&[route]); let (enacted, retracted) = self.calculate_enacted_retracted(&[route]);
self.miner.chain_new_blocks(self, &[h.clone()], &[], &enacted, &retracted); self.miner.chain_new_blocks(self, &[h.clone()], &[], &enacted, &retracted);
self.notify(|notify| { self.notify(|notify| {
notify.new_blocks( notify.new_blocks(
vec![h.clone()], vec![h.clone()],

View File

@ -560,6 +560,10 @@ impl BlockChainClient for TestBlockChainClient {
Ok(h) Ok(h)
} }
fn import_block_with_receipts(&self, b: Bytes, _r: Bytes) -> Result<H256, BlockImportError> {
self.import_block(b)
}
fn queue_info(&self) -> QueueInfo { fn queue_info(&self) -> QueueInfo {
QueueInfo { QueueInfo {
verified_queue_size: self.queue_size.load(AtomicOrder::Relaxed), verified_queue_size: self.queue_size.load(AtomicOrder::Relaxed),
@ -585,6 +589,10 @@ impl BlockChainClient for TestBlockChainClient {
genesis_hash: self.genesis_hash.clone(), genesis_hash: self.genesis_hash.clone(),
best_block_hash: self.last_hash.read().clone(), best_block_hash: self.last_hash.read().clone(),
best_block_number: self.blocks.read().len() as BlockNumber - 1, best_block_number: self.blocks.read().len() as BlockNumber - 1,
first_block_hash: None,
first_block_number: None,
ancient_block_hash: None,
ancient_block_number: None,
} }
} }

View File

@ -139,6 +139,9 @@ pub trait BlockChainClient : Sync + Send {
/// 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, bytes: Bytes) -> Result<H256, BlockImportError>;
/// Import a block with transaction receipts. Does no sealing and transaction validation.
fn import_block_with_receipts(&self, block_bytes: Bytes, receipts_bytes: Bytes) -> Result<H256, BlockImportError>;
/// Get block queue information. /// Get block queue information.
fn queue_info(&self) -> BlockQueueInfo; fn queue_info(&self) -> BlockQueueInfo;

View File

@ -564,6 +564,7 @@ const POW_VERIFY_RATE: f32 = 0.02;
/// After all chunks have been submitted, we "glue" the chunks together. /// After all chunks have been submitted, we "glue" the chunks together.
pub struct BlockRebuilder { pub struct BlockRebuilder {
chain: BlockChain, chain: BlockChain,
db: Arc<Database>,
rng: OsRng, rng: OsRng,
disconnected: Vec<(u64, H256)>, disconnected: Vec<(u64, H256)>,
best_number: u64, best_number: u64,
@ -571,9 +572,10 @@ pub struct BlockRebuilder {
impl BlockRebuilder { impl BlockRebuilder {
/// Create a new BlockRebuilder. /// Create a new BlockRebuilder.
pub fn new(chain: BlockChain, best_number: u64) -> Result<Self, ::error::Error> { pub fn new(chain: BlockChain, db: Arc<Database>, best_number: u64) -> Result<Self, ::error::Error> {
Ok(BlockRebuilder { Ok(BlockRebuilder {
chain: chain, chain: chain,
db: db,
rng: try!(OsRng::new()), rng: try!(OsRng::new()),
disconnected: Vec::new(), disconnected: Vec::new(),
best_number: best_number, best_number: best_number,
@ -616,15 +618,17 @@ impl BlockRebuilder {
} }
let is_best = cur_number == self.best_number; let is_best = cur_number == self.best_number;
let mut batch = self.db.transaction();
// special-case the first block in each chunk. // special-case the first block in each chunk.
if idx == 3 { if idx == 3 {
if self.chain.insert_snapshot_block(&block_bytes, receipts, Some(parent_total_difficulty), is_best) { if self.chain.insert_unordered_block(&mut batch, &block_bytes, receipts, Some(parent_total_difficulty), is_best, false) {
self.disconnected.push((cur_number, block.header.hash())); self.disconnected.push((cur_number, block.header.hash()));
} }
} else { } else {
self.chain.insert_snapshot_block(&block_bytes, receipts, None, is_best); self.chain.insert_unordered_block(&mut batch, &block_bytes, receipts, None, is_best, false);
} }
self.db.write(batch).expect("Error writing to the DB");
self.chain.commit(); self.chain.commit();
parent_hash = BlockView::new(&block_bytes).hash(); parent_hash = BlockView::new(&block_bytes).hash();

View File

@ -98,7 +98,7 @@ impl Restoration {
.map_err(UtilError::SimpleString))); .map_err(UtilError::SimpleString)));
let chain = BlockChain::new(Default::default(), params.genesis, raw_db.clone()); let chain = BlockChain::new(Default::default(), params.genesis, raw_db.clone());
let blocks = try!(BlockRebuilder::new(chain, manifest.block_number)); let blocks = try!(BlockRebuilder::new(chain, raw_db.clone(), manifest.block_number));
let root = manifest.state_root.clone(); let root = manifest.state_root.clone();
Ok(Restoration { Ok(Restoration {
@ -415,9 +415,14 @@ impl Service {
guard: Guard::new(rest_dir), guard: Guard::new(rest_dir),
}; };
let state_chunks = params.manifest.state_hashes.len();
let block_chunks = params.manifest.block_hashes.len();
*res = Some(try!(Restoration::new(params))); *res = Some(try!(Restoration::new(params)));
*self.status.lock() = RestorationStatus::Ongoing { *self.status.lock() = RestorationStatus::Ongoing {
state_chunks: state_chunks as u32,
block_chunks: block_chunks as u32,
state_chunks_done: self.state_chunks.load(Ordering::SeqCst) as u32, state_chunks_done: self.state_chunks.load(Ordering::SeqCst) as u32,
block_chunks_done: self.block_chunks.load(Ordering::SeqCst) as u32, block_chunks_done: self.block_chunks.load(Ordering::SeqCst) as u32,
}; };
@ -535,7 +540,7 @@ impl SnapshotService for Service {
fn status(&self) -> RestorationStatus { fn status(&self) -> RestorationStatus {
let mut cur_status = self.status.lock(); let mut cur_status = self.status.lock();
if let RestorationStatus::Ongoing { ref mut state_chunks_done, ref mut block_chunks_done } = *cur_status { if let RestorationStatus::Ongoing { ref mut state_chunks_done, ref mut block_chunks_done, .. } = *cur_status {
*state_chunks_done = self.state_chunks.load(Ordering::SeqCst) as u32; *state_chunks_done = self.state_chunks.load(Ordering::SeqCst) as u32;
*block_chunks_done = self.block_chunks.load(Ordering::SeqCst) as u32; *block_chunks_done = self.block_chunks.load(Ordering::SeqCst) as u32;
} }
@ -629,4 +634,4 @@ mod tests {
service.restore_state_chunk(Default::default(), vec![]); service.restore_state_chunk(Default::default(), vec![]);
service.restore_block_chunk(Default::default(), vec![]); service.restore_block_chunk(Default::default(), vec![]);
} }
} }

View File

@ -69,7 +69,7 @@ fn chunk_and_restore(amount: u64) {
// restore it. // restore it.
let new_db = Arc::new(Database::open(&db_cfg, new_path.as_str()).unwrap()); let new_db = Arc::new(Database::open(&db_cfg, new_path.as_str()).unwrap());
let new_chain = BlockChain::new(Default::default(), &genesis, new_db.clone()); let new_chain = BlockChain::new(Default::default(), &genesis, new_db.clone());
let mut rebuilder = BlockRebuilder::new(new_chain, amount).unwrap(); let mut rebuilder = BlockRebuilder::new(new_chain, new_db.clone(), amount).unwrap();
let reader = PackedReader::new(&snapshot_path).unwrap().unwrap(); let reader = PackedReader::new(&snapshot_path).unwrap().unwrap();
let engine = ::engines::NullEngine::new(Default::default(), Default::default()); let engine = ::engines::NullEngine::new(Default::default(), Default::default());
for chunk_hash in &reader.manifest().block_hashes { for chunk_hash in &reader.manifest().block_hashes {

View File

@ -31,5 +31,13 @@ pub struct BlockChainInfo {
/// Best blockchain block hash. /// Best blockchain block hash.
pub best_block_hash: H256, pub best_block_hash: H256,
/// Best blockchain block number. /// Best blockchain block number.
pub best_block_number: BlockNumber pub best_block_number: BlockNumber,
/// Best ancient block hash.
pub ancient_block_hash: Option<H256>,
/// Best ancient block number.
pub ancient_block_number: Option<BlockNumber>,
/// First block on the best sequence.
pub first_block_hash: Option<H256>,
/// Number of the first block on the best sequence.
pub first_block_number: Option<BlockNumber>,
} }

View File

@ -23,6 +23,10 @@ pub enum RestorationStatus {
Inactive, Inactive,
/// Ongoing restoration. /// Ongoing restoration.
Ongoing { Ongoing {
/// Total number of state chunks.
state_chunks: u32,
/// Total number of block chunks.
block_chunks: u32,
/// Number of state chunks completed. /// Number of state chunks completed.
state_chunks_done: u32, state_chunks_done: u32,
/// Number of block chunks completed. /// Number of block chunks completed.

View File

@ -296,7 +296,7 @@ mod tests {
self.blocks.contains_key(hash) self.blocks.contains_key(hash)
} }
fn first_block(&self) -> H256 { fn first_block(&self) -> Option<H256> {
unimplemented!() unimplemented!()
} }
@ -313,6 +313,10 @@ mod tests {
self.block(hash).map(|b| BlockChain::block_to_body(&b)) self.block(hash).map(|b| BlockChain::block_to_body(&b))
} }
fn best_ancient_block(&self) -> Option<H256> {
None
}
/// Get the familial details concerning a block. /// Get the familial details concerning a block.
fn block_details(&self, hash: &H256) -> Option<BlockDetails> { fn block_details(&self, hash: &H256) -> Option<BlockDetails> {
self.blocks.get(hash).map(|bytes| { self.blocks.get(hash).map(|bytes| {

View File

@ -194,7 +194,7 @@ fn execute_import(cmd: ImportBlockchain) -> Result<String, String> {
} }
}; };
let informant = Informant::new(client.clone(), None, None, cmd.logger_config.color); let informant = Informant::new(client.clone(), None, None, None, cmd.logger_config.color);
try!(service.register_io_handler(Arc::new(ImportIoHandler { try!(service.register_io_handler(Arc::new(ImportIoHandler {
info: Arc::new(informant), info: Arc::new(informant),

View File

@ -28,6 +28,7 @@ nat = "any"
id = "0x1" id = "0x1"
bootnodes = [] bootnodes = []
discovery = true discovery = true
warp = true
reserved_only = false reserved_only = false
reserved_peers = "./path_to_file" reserved_peers = "./path_to_file"

View File

@ -13,6 +13,7 @@ disable = true
[network] [network]
disable = false disable = false
warp = false
discovery = true discovery = true
nat = "any" nat = "any"
min_peers = 10 min_peers = 10

View File

@ -106,6 +106,8 @@ usage! {
// -- Networking Options // -- Networking Options
flag_no_network: bool = false, flag_no_network: bool = false,
or |c: &Config| otry!(c.network).disable.clone(), or |c: &Config| otry!(c.network).disable.clone(),
flag_warp: bool = false,
or |c: &Config| otry!(c.network).warp.clone(),
flag_port: u16 = 30303u16, flag_port: u16 = 30303u16,
or |c: &Config| otry!(c.network).port.clone(), or |c: &Config| otry!(c.network).port.clone(),
flag_min_peers: u16 = 25u16, flag_min_peers: u16 = 25u16,
@ -300,6 +302,7 @@ struct Signer {
#[derive(Default, Debug, PartialEq, RustcDecodable)] #[derive(Default, Debug, PartialEq, RustcDecodable)]
struct Network { struct Network {
disable: Option<bool>, disable: Option<bool>,
warp: Option<bool>,
port: Option<u16>, port: Option<u16>,
min_peers: Option<u16>, min_peers: Option<u16>,
max_peers: Option<u16>, max_peers: Option<u16>,
@ -486,6 +489,7 @@ mod tests {
// -- Networking Options // -- Networking Options
flag_no_network: false, flag_no_network: false,
flag_warp: true,
flag_port: 30303u16, flag_port: 30303u16,
flag_min_peers: 25u16, flag_min_peers: 25u16,
flag_max_peers: 50u16, flag_max_peers: 50u16,
@ -643,6 +647,7 @@ mod tests {
}), }),
network: Some(Network { network: Some(Network {
disable: Some(false), disable: Some(false),
warp: Some(false),
port: None, port: None,
min_peers: Some(10), min_peers: Some(10),
max_peers: Some(20), max_peers: Some(20),

View File

@ -67,6 +67,7 @@ Account Options:
Networking Options: Networking Options:
--no-network Disable p2p networking. (default: {flag_no_network}) --no-network Disable p2p networking. (default: {flag_no_network})
--warp Enable syncing from the snapshot over the network. (default: {flag_warp})
--port PORT Override the port on which the node should listen --port PORT Override the port on which the node should listen
(default: {flag_port}). (default: {flag_port}).
--min-peers NUM Try to maintain at least NUM peers (default: {flag_min_peers}). --min-peers NUM Try to maintain at least NUM peers (default: {flag_min_peers}).

View File

@ -89,6 +89,7 @@ impl Configuration {
let compaction = try!(self.args.flag_db_compaction.parse()); let compaction = try!(self.args.flag_db_compaction.parse());
let wal = !self.args.flag_fast_and_loose; let wal = !self.args.flag_fast_and_loose;
let enable_network = self.enable_network(&mode); let enable_network = self.enable_network(&mode);
let warp_sync = self.args.flag_warp;
let geth_compatibility = self.args.flag_geth; let geth_compatibility = self.args.flag_geth;
let signer_port = self.signer_port(); let signer_port = self.signer_port();
let dapps_conf = self.dapps_config(); let dapps_conf = self.dapps_config();
@ -240,6 +241,7 @@ impl Configuration {
wal: wal, wal: wal,
vm_type: vm_type, vm_type: vm_type,
enable_network: enable_network, enable_network: enable_network,
warp_sync: warp_sync,
geth_compatibility: geth_compatibility, geth_compatibility: geth_compatibility,
signer_port: signer_port, signer_port: signer_port,
net_settings: self.network_settings(), net_settings: self.network_settings(),
@ -810,6 +812,7 @@ mod tests {
ipc_conf: Default::default(), ipc_conf: Default::default(),
net_conf: default_network_config(), net_conf: default_network_config(),
network_id: None, network_id: None,
warp_sync: false,
acc_conf: Default::default(), acc_conf: Default::default(),
gas_pricer: Default::default(), gas_pricer: Default::default(),
miner_extras: Default::default(), miner_extras: Default::default(),

View File

@ -26,6 +26,8 @@ use ethsync::{SyncProvider, ManageNetwork};
use util::{Uint, RwLock, Mutex, H256, Colour}; use util::{Uint, RwLock, Mutex, H256, Colour};
use ethcore::client::*; use ethcore::client::*;
use ethcore::views::BlockView; use ethcore::views::BlockView;
use ethcore::snapshot::service::Service as SnapshotService;
use ethcore::snapshot::{RestorationStatus, SnapshotService as SS};
use number_prefix::{binary_prefix, Standalone, Prefixed}; use number_prefix::{binary_prefix, Standalone, Prefixed};
pub struct Informant { pub struct Informant {
@ -35,6 +37,7 @@ pub struct Informant {
last_tick: RwLock<Instant>, last_tick: RwLock<Instant>,
with_color: bool, with_color: bool,
client: Arc<Client>, client: Arc<Client>,
snapshot: Option<Arc<SnapshotService>>,
sync: Option<Arc<SyncProvider>>, sync: Option<Arc<SyncProvider>>,
net: Option<Arc<ManageNetwork>>, net: Option<Arc<ManageNetwork>>,
last_import: Mutex<Instant>, last_import: Mutex<Instant>,
@ -55,7 +58,7 @@ impl MillisecondDuration for Duration {
impl Informant { impl Informant {
/// Make a new instance potentially `with_color` output. /// Make a new instance potentially `with_color` output.
pub fn new(client: Arc<Client>, sync: Option<Arc<SyncProvider>>, net: Option<Arc<ManageNetwork>>, with_color: bool) -> Self { pub fn new(client: Arc<Client>, sync: Option<Arc<SyncProvider>>, net: Option<Arc<ManageNetwork>>, snapshot: Option<Arc<SnapshotService>>, with_color: bool) -> Self {
Informant { Informant {
chain_info: RwLock::new(None), chain_info: RwLock::new(None),
cache_info: RwLock::new(None), cache_info: RwLock::new(None),
@ -63,6 +66,7 @@ impl Informant {
last_tick: RwLock::new(Instant::now()), last_tick: RwLock::new(Instant::now()),
with_color: with_color, with_color: with_color,
client: client, client: client,
snapshot: snapshot,
sync: sync, sync: sync,
net: net, net: net,
last_import: Mutex::new(Instant::now()), last_import: Mutex::new(Instant::now()),
@ -92,8 +96,16 @@ impl Informant {
let sync_status = self.sync.as_ref().map(|s| s.status()); let sync_status = self.sync.as_ref().map(|s| s.status());
let importing = queue_info.unverified_queue_size + queue_info.verified_queue_size > 3 let importing = queue_info.unverified_queue_size + queue_info.verified_queue_size > 3
|| self.sync.as_ref().map_or(false, |s| s.status().is_major_syncing()); || sync_status.map_or(false, |s| s.is_major_syncing());
if !importing && elapsed < Duration::from_secs(30) { let (snapshot_sync, snapshot_current, snapshot_total) = self.snapshot.as_ref().map_or((false, 0, 0), |s|
match s.status() {
RestorationStatus::Ongoing { state_chunks, block_chunks, state_chunks_done, block_chunks_done } =>
(true, state_chunks_done + block_chunks_done, state_chunks + block_chunks),
_ => (false, 0, 0),
}
);
if !importing && !snapshot_sync && elapsed < Duration::from_secs(30) {
return; return;
} }
@ -109,27 +121,33 @@ impl Informant {
info!(target: "import", "{} {} {}", info!(target: "import", "{} {} {}",
match importing { match importing {
true => format!("Syncing {} {} {} {}+{} Qed", true => match snapshot_sync {
paint(White.bold(), format!("{:>8}", format!("#{}", chain_info.best_block_number))), false => format!("Syncing {} {} {} {}+{} Qed",
paint(White.bold(), format!("{}", chain_info.best_block_hash)), paint(White.bold(), format!("{:>8}", format!("#{}", chain_info.best_block_number))),
{ paint(White.bold(), format!("{}", chain_info.best_block_hash)),
let last_report = match *write_report { Some(ref last_report) => last_report.clone(), _ => ClientReport::default() }; {
format!("{} blk/s {} tx/s {} Mgas/s", let last_report = match *write_report { Some(ref last_report) => last_report.clone(), _ => ClientReport::default() };
paint(Yellow.bold(), format!("{:4}", ((report.blocks_imported - last_report.blocks_imported) * 1000) as u64 / elapsed.as_milliseconds())), format!("{} blk/s {} tx/s {} Mgas/s",
paint(Yellow.bold(), format!("{:4}", ((report.transactions_applied - last_report.transactions_applied) * 1000) as u64 / elapsed.as_milliseconds())), paint(Yellow.bold(), format!("{:4}", ((report.blocks_imported - last_report.blocks_imported) * 1000) as u64 / elapsed.as_milliseconds())),
paint(Yellow.bold(), format!("{:3}", ((report.gas_processed - last_report.gas_processed) / From::from(elapsed.as_milliseconds() * 1000)).low_u64())) paint(Yellow.bold(), format!("{:4}", ((report.transactions_applied - last_report.transactions_applied) * 1000) as u64 / elapsed.as_milliseconds())),
) paint(Yellow.bold(), format!("{:3}", ((report.gas_processed - last_report.gas_processed) / From::from(elapsed.as_milliseconds() * 1000)).low_u64()))
}, )
paint(Green.bold(), format!("{:5}", queue_info.unverified_queue_size)), },
paint(Green.bold(), format!("{:5}", queue_info.verified_queue_size)) paint(Green.bold(), format!("{:5}", queue_info.unverified_queue_size)),
), paint(Green.bold(), format!("{:5}", queue_info.verified_queue_size))
),
true => format!("Syncing snapshot {}/{}", snapshot_current, snapshot_total),
},
false => String::new(), false => String::new(),
}, },
match (&sync_status, &network_config) { match (&sync_status, &network_config) {
(&Some(ref sync_info), &Some(ref net_config)) => format!("{}{}/{}/{} peers", (&Some(ref sync_info), &Some(ref net_config)) => format!("{}{}/{}/{} peers",
match importing { match importing {
true => format!("{} ", paint(Green.bold(), format!("{:>8}", format!("#{}", sync_info.last_imported_block_number.unwrap_or(chain_info.best_block_number))))), true => format!("{} ", paint(Green.bold(), format!("{:>8}", format!("#{}", sync_info.last_imported_block_number.unwrap_or(chain_info.best_block_number))))),
false => String::new(), false => match sync_info.last_imported_old_block_number {
Some(number) => format!("{} ", paint(Yellow.bold(), format!("{:>8}", format!("#{}", number)))),
None => String::new(),
}
}, },
paint(Cyan.bold(), format!("{:2}", sync_info.num_active_peers)), paint(Cyan.bold(), format!("{:2}", sync_info.num_active_peers)),
paint(Cyan.bold(), format!("{:2}", sync_info.num_peers)), paint(Cyan.bold(), format!("{:2}", sync_info.num_peers)),

View File

@ -61,4 +61,4 @@ impl IoHandler<ClientIoMessage> for ImportIoHandler {
self.info.tick() self.info.tick()
} }
} }
} }

View File

@ -70,6 +70,7 @@ pub struct RunCmd {
pub ipc_conf: IpcConfiguration, pub ipc_conf: IpcConfiguration,
pub net_conf: NetworkConfiguration, pub net_conf: NetworkConfiguration,
pub network_id: Option<U256>, pub network_id: Option<U256>,
pub warp_sync: bool,
pub acc_conf: AccountsConfig, pub acc_conf: AccountsConfig,
pub gas_pricer: GasPricerConfig, pub gas_pricer: GasPricerConfig,
pub miner_extras: MinerExtras, pub miner_extras: MinerExtras,
@ -171,6 +172,7 @@ pub fn execute(cmd: RunCmd) -> Result<(), String> {
sync_config.subprotocol_name.clone_from_slice(spec.subprotocol_name().as_bytes()); sync_config.subprotocol_name.clone_from_slice(spec.subprotocol_name().as_bytes());
} }
sync_config.fork_block = spec.fork_block(); sync_config.fork_block = spec.fork_block();
sync_config.warp_sync = cmd.warp_sync;
// prepare account provider // prepare account provider
let account_provider = Arc::new(try!(prepare_account_provider(&cmd.dirs, cmd.acc_conf))); let account_provider = Arc::new(try!(prepare_account_provider(&cmd.dirs, cmd.acc_conf)));
@ -231,7 +233,7 @@ pub fn execute(cmd: RunCmd) -> Result<(), String> {
// create sync object // create sync object
let (sync_provider, manage_network, chain_notify) = try!(modules::sync( let (sync_provider, manage_network, chain_notify) = try!(modules::sync(
&mut hypervisor, sync_config, net_conf.into(), client.clone(), snapshot_service, &cmd.logger_config, &mut hypervisor, sync_config, net_conf.into(), client.clone(), snapshot_service.clone(), &cmd.logger_config,
).map_err(|e| format!("Sync error: {}", e))); ).map_err(|e| format!("Sync error: {}", e)));
service.add_notify(chain_notify.clone()); service.add_notify(chain_notify.clone());
@ -287,7 +289,13 @@ pub fn execute(cmd: RunCmd) -> Result<(), String> {
// start signer server // start signer server
let signer_server = try!(signer::start(cmd.signer_conf, signer_deps)); let signer_server = try!(signer::start(cmd.signer_conf, signer_deps));
let informant = Arc::new(Informant::new(service.client(), Some(sync_provider.clone()), Some(manage_network.clone()), cmd.logger_config.color)); let informant = Arc::new(Informant::new(
service.client(),
Some(sync_provider.clone()),
Some(manage_network.clone()),
Some(snapshot_service.clone()),
cmd.logger_config.color
));
let info_notify: Arc<ChainNotify> = informant.clone(); let info_notify: Arc<ChainNotify> = informant.clone();
service.add_notify(info_notify); service.add_notify(info_notify);
let io_handler = Arc::new(ClientIoHandler { let io_handler = Arc::new(ClientIoHandler {

View File

@ -81,7 +81,7 @@ fn restore_using<R: SnapshotReader>(snapshot: Arc<SnapshotService>, reader: &R,
let informant_handle = snapshot.clone(); let informant_handle = snapshot.clone();
::std::thread::spawn(move || { ::std::thread::spawn(move || {
while let RestorationStatus::Ongoing { state_chunks_done, block_chunks_done } = informant_handle.status() { while let RestorationStatus::Ongoing { state_chunks_done, block_chunks_done, .. } = informant_handle.status() {
info!("Processed {}/{} state chunks and {}/{} block chunks.", info!("Processed {}/{} state chunks and {}/{} block chunks.",
state_chunks_done, num_state, block_chunks_done, num_blocks); state_chunks_done, num_state, block_chunks_done, num_blocks);
::std::thread::sleep(Duration::from_secs(5)); ::std::thread::sleep(Duration::from_secs(5));

View File

@ -257,7 +257,7 @@ impl<C, S: ?Sized, M, EM> Eth for EthClient<C, S, M, EM> where
let status = take_weak!(self.sync).status(); let status = take_weak!(self.sync).status();
match status.state { match status.state {
SyncState::Idle => Ok(SyncStatus::None), SyncState::Idle => Ok(SyncStatus::None),
SyncState::Waiting | SyncState::Blocks | SyncState::NewBlocks | SyncState::ChainHead SyncState::Waiting | SyncState::Blocks | SyncState::NewBlocks
| SyncState::SnapshotManifest | SyncState::SnapshotData | SyncState::SnapshotWaiting => { | SyncState::SnapshotManifest | SyncState::SnapshotData | SyncState::SnapshotWaiting => {
let current_block = U256::from(take_weak!(self.client).chain_info().best_block_number); let current_block = U256::from(take_weak!(self.client).chain_info().best_block_number);
let highest_block = U256::from(status.highest_block_number.unwrap_or(status.start_block_number)); let highest_block = U256::from(status.highest_block_number.unwrap_or(status.start_block_number));

View File

@ -51,6 +51,7 @@ impl TestSyncProvider {
mem_used: 0, mem_used: 0,
num_snapshot_chunks: 0, num_snapshot_chunks: 0,
snapshot_chunks_done: 0, snapshot_chunks_done: 0,
last_imported_old_block_number: None,
}), }),
} }
} }

View File

@ -15,7 +15,8 @@
// along with Parity. If not, see <http://www.gnu.org/licenses/>. // along with Parity. If not, see <http://www.gnu.org/licenses/>.
use std::sync::Arc; use std::sync::Arc;
use std::str; use std::collections::HashMap;
use util::Bytes;
use network::{NetworkProtocolHandler, NetworkService, NetworkContext, PeerId, use network::{NetworkProtocolHandler, NetworkService, NetworkContext, PeerId,
NetworkConfiguration as BasicNetworkConfiguration, NonReservedPeerMode, NetworkError}; NetworkConfiguration as BasicNetworkConfiguration, NonReservedPeerMode, NetworkError};
use util::{U256, H256}; use util::{U256, H256};
@ -41,6 +42,8 @@ pub struct SyncConfig {
pub subprotocol_name: [u8; 3], pub subprotocol_name: [u8; 3],
/// Fork block to check /// Fork block to check
pub fork_block: Option<(BlockNumber, H256)>, pub fork_block: Option<(BlockNumber, H256)>,
/// Enable snapshot sync
pub warp_sync: bool,
} }
impl Default for SyncConfig { impl Default for SyncConfig {
@ -50,6 +53,7 @@ impl Default for SyncConfig {
network_id: U256::from(1), network_id: U256::from(1),
subprotocol_name: *b"eth", subprotocol_name: *b"eth",
fork_block: None, fork_block: None,
warp_sync: true,
} }
} }
} }
@ -104,7 +108,12 @@ impl EthSync {
let service = try!(NetworkService::new(try!(network_config.into_basic()))); let service = try!(NetworkService::new(try!(network_config.into_basic())));
let sync = Arc::new(EthSync{ let sync = Arc::new(EthSync{
network: service, network: service,
handler: Arc::new(SyncProtocolHandler { sync: RwLock::new(chain_sync), chain: chain, snapshot_service: snapshot_service }), handler: Arc::new(SyncProtocolHandler {
sync: RwLock::new(chain_sync),
chain: chain,
snapshot_service: snapshot_service,
overlay: RwLock::new(HashMap::new()),
}),
subprotocol_name: config.subprotocol_name, subprotocol_name: config.subprotocol_name,
}); });
@ -122,7 +131,7 @@ impl SyncProvider for EthSync {
/// Get sync peers /// Get sync peers
fn peers(&self) -> Vec<PeerInfo> { fn peers(&self) -> Vec<PeerInfo> {
self.network.with_context_eval(self.subprotocol_name, |context| { self.network.with_context_eval(self.subprotocol_name, |context| {
let sync_io = NetSyncIo::new(context, &*self.handler.chain, &*self.handler.snapshot_service); let sync_io = NetSyncIo::new(context, &*self.handler.chain, &*self.handler.snapshot_service, &self.handler.overlay);
self.handler.sync.write().peers(&sync_io) self.handler.sync.write().peers(&sync_io)
}).unwrap_or(Vec::new()) }).unwrap_or(Vec::new())
} }
@ -135,6 +144,8 @@ struct SyncProtocolHandler {
snapshot_service: Arc<SnapshotService>, snapshot_service: Arc<SnapshotService>,
/// Sync strategy /// Sync strategy
sync: RwLock<ChainSync>, sync: RwLock<ChainSync>,
/// Chain overlay used to cache data such as fork block.
overlay: RwLock<HashMap<BlockNumber, Bytes>>,
} }
impl NetworkProtocolHandler for SyncProtocolHandler { impl NetworkProtocolHandler for SyncProtocolHandler {
@ -143,21 +154,21 @@ impl NetworkProtocolHandler for SyncProtocolHandler {
} }
fn read(&self, io: &NetworkContext, peer: &PeerId, packet_id: u8, data: &[u8]) { fn read(&self, io: &NetworkContext, peer: &PeerId, packet_id: u8, data: &[u8]) {
ChainSync::dispatch_packet(&self.sync, &mut NetSyncIo::new(io, &*self.chain, &*self.snapshot_service), *peer, packet_id, data); ChainSync::dispatch_packet(&self.sync, &mut NetSyncIo::new(io, &*self.chain, &*self.snapshot_service, &self.overlay), *peer, packet_id, data);
} }
fn connected(&self, io: &NetworkContext, peer: &PeerId) { fn connected(&self, io: &NetworkContext, peer: &PeerId) {
self.sync.write().on_peer_connected(&mut NetSyncIo::new(io, &*self.chain, &*self.snapshot_service), *peer); self.sync.write().on_peer_connected(&mut NetSyncIo::new(io, &*self.chain, &*self.snapshot_service, &self.overlay), *peer);
} }
fn disconnected(&self, io: &NetworkContext, peer: &PeerId) { fn disconnected(&self, io: &NetworkContext, peer: &PeerId) {
self.sync.write().on_peer_aborting(&mut NetSyncIo::new(io, &*self.chain, &*self.snapshot_service), *peer); self.sync.write().on_peer_aborting(&mut NetSyncIo::new(io, &*self.chain, &*self.snapshot_service, &self.overlay), *peer);
} }
fn timeout(&self, io: &NetworkContext, _timer: TimerToken) { fn timeout(&self, io: &NetworkContext, _timer: TimerToken) {
self.sync.write().maintain_peers(&mut NetSyncIo::new(io, &*self.chain, &*self.snapshot_service)); self.sync.write().maintain_peers(&mut NetSyncIo::new(io, &*self.chain, &*self.snapshot_service, &self.overlay));
self.sync.write().maintain_sync(&mut NetSyncIo::new(io, &*self.chain, &*self.snapshot_service)); self.sync.write().maintain_sync(&mut NetSyncIo::new(io, &*self.chain, &*self.snapshot_service, &self.overlay));
self.sync.write().propagate_new_transactions(&mut NetSyncIo::new(io, &*self.chain, &*self.snapshot_service)); self.sync.write().propagate_new_transactions(&mut NetSyncIo::new(io, &*self.chain, &*self.snapshot_service, &self.overlay));
} }
} }
@ -171,7 +182,7 @@ impl ChainNotify for EthSync {
_duration: u64) _duration: u64)
{ {
self.network.with_context(self.subprotocol_name, |context| { self.network.with_context(self.subprotocol_name, |context| {
let mut sync_io = NetSyncIo::new(context, &*self.handler.chain, &*self.handler.snapshot_service); let mut sync_io = NetSyncIo::new(context, &*self.handler.chain, &*self.handler.snapshot_service, &self.handler.overlay);
self.handler.sync.write().chain_new_blocks( self.handler.sync.write().chain_new_blocks(
&mut sync_io, &mut sync_io,
&imported, &imported,
@ -239,7 +250,7 @@ impl ManageNetwork for EthSync {
fn stop_network(&self) { fn stop_network(&self) {
self.network.with_context(self.subprotocol_name, |context| { self.network.with_context(self.subprotocol_name, |context| {
let mut sync_io = NetSyncIo::new(context, &*self.handler.chain, &*self.handler.snapshot_service); let mut sync_io = NetSyncIo::new(context, &*self.handler.chain, &*self.handler.snapshot_service, &self.handler.overlay);
self.handler.sync.write().abort(&mut sync_io); self.handler.sync.write().abort(&mut sync_io);
}); });
self.stop(); self.stop();

477
sync/src/block_sync.rs Normal file
View File

@ -0,0 +1,477 @@
// Copyright 2015, 2016 Ethcore (UK) Ltd.
// This file is part of Parity.
// Parity is free software: you can redistribute it and/or modify
// it under the terms of the GNU General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
// Parity is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU General Public License for more details.
// You should have received a copy of the GNU General Public License
// along with Parity. If not, see <http://www.gnu.org/licenses/>.
///
/// Blockchain downloader
///
use util::*;
use rlp::*;
use ethcore::views::{BlockView};
use ethcore::header::{BlockNumber, Header as BlockHeader};
use ethcore::client::{BlockStatus, BlockID, BlockImportError};
use ethcore::block::Block;
use ethcore::error::{ImportError, BlockError};
use sync_io::SyncIo;
use blocks::BlockCollection;
const MAX_HEADERS_TO_REQUEST: usize = 128;
const MAX_BODIES_TO_REQUEST: usize = 128;
const MAX_RECEPITS_TO_REQUEST: usize = 128;
const SUBCHAIN_SIZE: u64 = 256;
const MAX_ROUND_PARENTS: usize = 32;
#[derive(Copy, Clone, Eq, PartialEq, Debug)]
/// Downloader state
pub enum State {
/// No active downloads.
Idle,
/// Downloading subchain heads
ChainHead,
/// Downloading blocks
Blocks,
/// Download is complete
Complete,
}
/// Data that needs to be requested from a peer.
pub enum BlockRequest {
Headers {
start: H256,
count: u64,
skip: u64,
},
Bodies {
hashes: Vec<H256>,
},
Receipts {
hashes: Vec<H256>,
},
}
#[derive(Eq, PartialEq, Debug)]
pub enum BlockDownloaderImportError {
/// Imported data is rejected as invalid.
Invalid,
/// Imported data is valid but rejected cause the downloader does not need it.
Useless,
}
/// Block downloader strategy.
/// Manages state and block data for a block download process.
pub struct BlockDownloader {
/// Downloader state
state: State,
/// Highest block number seen
highest_block: Option<BlockNumber>,
/// Downloaded blocks, holds `H`, `B` and `S`
blocks: BlockCollection,
/// Last impoted block number
last_imported_block: BlockNumber,
/// Last impoted block hash
last_imported_hash: H256,
/// Number of blocks imported this round
imported_this_round: Option<usize>,
/// Block parents imported this round (hash, parent)
round_parents: VecDeque<(H256, H256)>,
/// Do we need to download block recetips.
download_receipts: bool,
/// Sync up to the block with this hash.
target_hash: Option<H256>,
}
impl BlockDownloader {
/// Create a new instance of syncing strategy.
pub fn new(sync_receipts: bool, start_hash: &H256, start_number: BlockNumber) -> BlockDownloader {
BlockDownloader {
state: State::Idle,
highest_block: None,
last_imported_block: start_number,
last_imported_hash: start_hash.clone(),
blocks: BlockCollection::new(sync_receipts),
imported_this_round: None,
round_parents: VecDeque::new(),
download_receipts: sync_receipts,
target_hash: None,
}
}
/// Reset sync. Clear all local downloaded data.
pub fn reset(&mut self) {
self.blocks.clear();
self.state = State::Idle;
}
/// Mark a block as known in the chain
pub fn mark_as_known(&mut self, hash: &H256, number: BlockNumber) {
if number == self.last_imported_block + 1 {
self.last_imported_block = number;
self.last_imported_hash = hash.clone();
}
}
/// Check if download is complete
pub fn is_complete(&self) -> bool {
self.state == State::Complete
}
/// Check if particular block hash is being downloaded
pub fn is_downloading(&self, hash: &H256) -> bool {
self.blocks.is_downloading(hash)
}
/// Set starting sync block
pub fn set_target(&mut self, hash: &H256) {
self.target_hash = Some(hash.clone());
}
/// Set starting sync block
pub fn _set_start(&mut self, hash: &H256, number: BlockNumber) {
self.last_imported_hash = hash.clone();
self.last_imported_block = number;
}
/// Unmark header as being downloaded.
pub fn clear_header_download(&mut self, hash: &H256) {
self.blocks.clear_header_download(hash)
}
/// Unmark block body as being downloaded.
pub fn clear_body_download(&mut self, hashes: &[H256]) {
self.blocks.clear_body_download(hashes)
}
/// Unmark block receipt as being downloaded.
pub fn clear_receipt_download(&mut self, hashes: &[H256]) {
self.blocks.clear_receipt_download(hashes)
}
/// Reset collection for a new sync round with given subchain block hashes.
pub fn reset_to(&mut self, hashes: Vec<H256>) {
self.reset();
self.blocks.reset_to(hashes);
}
/// Returns used heap memory size.
pub fn heap_size(&self) -> usize {
self.blocks.heap_size() + self.round_parents.heap_size_of_children()
}
/// Returns best imported block number.
pub fn last_imported_block_number(&self) -> BlockNumber {
self.last_imported_block
}
/// Add new block headers.
pub fn import_headers(&mut self, io: &mut SyncIo, r: &UntrustedRlp, expected_hash: Option<H256>) -> Result<(), BlockDownloaderImportError> {
let item_count = r.item_count();
if self.state == State::Idle {
trace!(target: "sync", "Ignored unexpected block headers");
return Ok(())
}
if item_count == 0 && (self.state == State::Blocks) {
return Err(BlockDownloaderImportError::Invalid);
}
let mut headers = Vec::new();
let mut hashes = Vec::new();
let mut valid_response = item_count == 0; //empty response is valid
for i in 0..item_count {
let info: BlockHeader = try!(r.val_at(i).map_err(|e| {
trace!(target: "sync", "Error decoding block header RLP: {:?}", e);
BlockDownloaderImportError::Invalid
}));
let number = BlockNumber::from(info.number());
// Check if any of the headers matches the hash we requested
if !valid_response {
if let Some(expected) = expected_hash {
valid_response = expected == info.hash()
}
}
if self.blocks.contains(&info.hash()) {
trace!(target: "sync", "Skipping existing block header {} ({:?})", number, info.hash());
continue;
}
if self.highest_block.as_ref().map_or(true, |n| number > *n) {
self.highest_block = Some(number);
}
let hash = info.hash();
let hdr = try!(r.at(i).map_err(|e| {
trace!(target: "sync", "Error decoding block header RLP: {:?}", e);
BlockDownloaderImportError::Invalid
}));
match io.chain().block_status(BlockID::Hash(hash.clone())) {
BlockStatus::InChain | BlockStatus::Queued => {
match self.state {
State::Blocks => trace!(target: "sync", "Header already in chain {} ({})", number, hash),
_ => trace!(target: "sync", "Header already in chain {} ({}), state = {:?}", number, hash, self.state),
}
headers.push(hdr.as_raw().to_vec());
hashes.push(hash);
},
BlockStatus::Bad => {
return Err(BlockDownloaderImportError::Invalid);
},
BlockStatus::Unknown => {
headers.push(hdr.as_raw().to_vec());
hashes.push(hash);
}
}
}
// Disable the peer for this syncing round if it gives invalid chain
if !valid_response {
trace!(target: "sync", "Invalid headers response");
return Err(BlockDownloaderImportError::Invalid);
}
match self.state {
State::ChainHead => {
if !headers.is_empty() {
// TODO: validate heads better. E.g. check that there is enough distance between blocks.
trace!(target: "sync", "Received {} subchain heads, proceeding to download", headers.len());
self.blocks.reset_to(hashes);
self.state = State::Blocks;
}
},
State::Blocks => {
let count = headers.len();
self.blocks.insert_headers(headers);
trace!(target: "sync", "Inserted {} headers", count);
},
_ => trace!(target: "sync", "Unexpected headers({})", headers.len()),
}
Ok(())
}
/// Called by peer once it has new block bodies
pub fn import_bodies(&mut self, _io: &mut SyncIo, r: &UntrustedRlp) -> Result<(), BlockDownloaderImportError> {
let item_count = r.item_count();
if item_count == 0 {
return Err(BlockDownloaderImportError::Useless);
}
else if self.state != State::Blocks {
trace!(target: "sync", "Ignored unexpected block bodies");
}
else {
let mut bodies = Vec::with_capacity(item_count);
for i in 0..item_count {
let body = try!(r.at(i).map_err(|e| {
trace!(target: "sync", "Error decoding block boides RLP: {:?}", e);
BlockDownloaderImportError::Invalid
}));
bodies.push(body.as_raw().to_vec());
}
if self.blocks.insert_bodies(bodies) != item_count {
trace!(target: "sync", "Deactivating peer for giving invalid block bodies");
return Err(BlockDownloaderImportError::Invalid);
}
}
Ok(())
}
/// Called by peer once it has new block bodies
pub fn import_receipts(&mut self, _io: &mut SyncIo, r: &UntrustedRlp) -> Result<(), BlockDownloaderImportError> {
let item_count = r.item_count();
if item_count == 0 {
return Err(BlockDownloaderImportError::Useless);
}
else if self.state != State::Blocks {
trace!(target: "sync", "Ignored unexpected block receipts");
}
else {
let mut receipts = Vec::with_capacity(item_count);
for i in 0..item_count {
let receipt = try!(r.at(i).map_err(|e| {
trace!(target: "sync", "Error decoding block receipts RLP: {:?}", e);
BlockDownloaderImportError::Invalid
}));
receipts.push(receipt.as_raw().to_vec());
}
if self.blocks.insert_receipts(receipts) != item_count {
trace!(target: "sync", "Deactivating peer for giving invalid block receipts");
return Err(BlockDownloaderImportError::Invalid);
}
}
Ok(())
}
fn start_sync_round(&mut self, io: &mut SyncIo) {
self.state = State::ChainHead;
trace!(target: "sync", "Starting round (last imported count = {:?}, block = {:?}", self.imported_this_round, self.last_imported_block);
// Check if need to retract to find the common block. The problem is that the peers still return headers by hash even
// from the non-canonical part of the tree. So we also retract if nothing has been imported last round.
match self.imported_this_round {
Some(n) if n == 0 && self.last_imported_block > 0 => {
// nothing was imported last round, step back to a previous block
// search parent in last round known parents first
if let Some(&(_, p)) = self.round_parents.iter().find(|&&(h, _)| h == self.last_imported_hash) {
self.last_imported_block -= 1;
self.last_imported_hash = p.clone();
trace!(target: "sync", "Searching common header from the last round {} ({})", self.last_imported_block, self.last_imported_hash);
} else {
match io.chain().block_hash(BlockID::Number(self.last_imported_block - 1)) {
Some(h) => {
self.last_imported_block -= 1;
self.last_imported_hash = h;
trace!(target: "sync", "Searching common header in the blockchain {} ({})", self.last_imported_block, self.last_imported_hash);
}
None => {
debug!(target: "sync", "Could not revert to previous block, last: {} ({})", self.last_imported_block, self.last_imported_hash);
}
}
}
},
_ => (),
}
self.imported_this_round = None;
}
/// Find some headers or blocks to download for a peer.
pub fn request_blocks(&mut self, io: &mut SyncIo) -> Option<BlockRequest> {
match self.state {
State::Idle => {
self.start_sync_round(io);
return self.request_blocks(io);
},
State::ChainHead => {
// Request subchain headers
trace!(target: "sync", "Starting sync with better chain");
// Request MAX_HEADERS_TO_REQUEST - 2 headers apart so that
// MAX_HEADERS_TO_REQUEST would include headers for neighbouring subchains
return Some(BlockRequest::Headers {
start: self.last_imported_hash.clone(),
count: SUBCHAIN_SIZE,
skip: (MAX_HEADERS_TO_REQUEST - 2) as u64,
});
},
State::Blocks => {
// check to see if we need to download any block bodies first
let needed_bodies = self.blocks.needed_bodies(MAX_BODIES_TO_REQUEST, false);
if !needed_bodies.is_empty() {
return Some(BlockRequest::Bodies {
hashes: needed_bodies,
});
}
if self.download_receipts {
let needed_receipts = self.blocks.needed_receipts(MAX_RECEPITS_TO_REQUEST, false);
if !needed_receipts.is_empty() {
return Some(BlockRequest::Receipts {
hashes: needed_receipts,
});
}
}
// find subchain to download
if let Some((h, count)) = self.blocks.needed_headers(MAX_HEADERS_TO_REQUEST, false) {
return Some(BlockRequest::Headers {
start: h,
count: count as u64,
skip: 0,
});
}
},
State::Complete => (),
}
None
}
/// Checks if there are blocks fully downloaded that can be imported into the blockchain and does the import.
pub fn collect_blocks(&mut self, io: &mut SyncIo, allow_out_of_order: bool) -> Result<(), BlockDownloaderImportError> {
let mut bad = false;
let mut imported = HashSet::new();
let blocks = self.blocks.drain();
let count = blocks.len();
for block_and_receipts in blocks {
let block = block_and_receipts.block;
let receipts = block_and_receipts.receipts;
let (h, number, parent) = {
let header = BlockView::new(&block).header_view();
(header.sha3(), header.number(), header.parent_hash())
};
// Perform basic block verification
if !Block::is_good(&block) {
debug!(target: "sync", "Bad block rlp {:?} : {:?}", h, block);
bad = true;
break;
}
if self.target_hash.as_ref().map_or(false, |t| t == &h) {
self.state = State::Complete;
trace!(target: "sync", "Sync target reached");
return Ok(());
}
let result = if let Some(receipts) = receipts {
io.chain().import_block_with_receipts(block, receipts)
} else {
io.chain().import_block(block)
};
match result {
Err(BlockImportError::Import(ImportError::AlreadyInChain)) => {
trace!(target: "sync", "Block already in chain {:?}", h);
self.block_imported(&h, number, &parent);
},
Err(BlockImportError::Import(ImportError::AlreadyQueued)) => {
trace!(target: "sync", "Block already queued {:?}", h);
self.block_imported(&h, number, &parent);
},
Ok(_) => {
trace!(target: "sync", "Block queued {:?}", h);
imported.insert(h.clone());
self.block_imported(&h, number, &parent);
},
Err(BlockImportError::Block(BlockError::UnknownParent(_))) if allow_out_of_order => {
trace!(target: "sync", "Unknown new block parent, restarting sync");
break;
},
Err(e) => {
debug!(target: "sync", "Bad block {:?} : {:?}", h, e);
bad = true;
break;
}
}
}
trace!(target: "sync", "Imported {} of {}", imported.len(), count);
self.imported_this_round = Some(self.imported_this_round.unwrap_or(0) + imported.len());
if bad {
return Err(BlockDownloaderImportError::Invalid);
}
if self.blocks.is_empty() {
// complete sync round
trace!(target: "sync", "Sync round complete");
self.reset();
}
Ok(())
}
fn block_imported(&mut self, hash: &H256, number: BlockNumber, parent: &H256) {
self.last_imported_block = number;
self.last_imported_hash = hash.clone();
self.round_parents.push_back((hash.clone(), parent.clone()));
if self.round_parents.len() > MAX_ROUND_PARENTS {
self.round_parents.pop_front();
}
}
}
//TODO: module tests

View File

@ -25,6 +25,15 @@ known_heap_size!(0, HeaderId);
struct SyncBlock { struct SyncBlock {
header: Bytes, header: Bytes,
body: Option<Bytes>, body: Option<Bytes>,
receipts: Option<Bytes>,
}
/// Block with optional receipt
pub struct BlockAndReceipts {
/// Block data.
pub block: Bytes,
/// Block receipts RLP list.
pub receipts: Option<Bytes>,
} }
impl HeapSizeOf for SyncBlock { impl HeapSizeOf for SyncBlock {
@ -45,6 +54,8 @@ struct HeaderId {
/// the downloaded blocks. /// the downloaded blocks.
#[derive(Default)] #[derive(Default)]
pub struct BlockCollection { pub struct BlockCollection {
/// Does this collection need block receipts.
need_receipts: bool,
/// Heads of subchains to download /// Heads of subchains to download
heads: Vec<H256>, heads: Vec<H256>,
/// Downloaded blocks. /// Downloaded blocks.
@ -53,25 +64,32 @@ pub struct BlockCollection {
parents: HashMap<H256, H256>, parents: HashMap<H256, H256>,
/// Used to map body to header. /// Used to map body to header.
header_ids: HashMap<HeaderId, H256>, header_ids: HashMap<HeaderId, H256>,
/// Used to map receipts root to header.
receipt_ids: HashMap<H256, H256>,
/// First block in `blocks`. /// First block in `blocks`.
head: Option<H256>, head: Option<H256>,
/// Set of block header hashes being downloaded /// Set of block header hashes being downloaded
downloading_headers: HashSet<H256>, downloading_headers: HashSet<H256>,
/// Set of block bodies being downloaded identified by block hash. /// Set of block bodies being downloaded identified by block hash.
downloading_bodies: HashSet<H256>, downloading_bodies: HashSet<H256>,
/// Set of block receipts being downloaded identified by block hash.
downloading_receipts: HashSet<H256>,
} }
impl BlockCollection { impl BlockCollection {
/// Create a new instance. /// Create a new instance.
pub fn new() -> BlockCollection { pub fn new(download_receipts: bool) -> BlockCollection {
BlockCollection { BlockCollection {
need_receipts: download_receipts,
blocks: HashMap::new(), blocks: HashMap::new(),
header_ids: HashMap::new(), header_ids: HashMap::new(),
receipt_ids: HashMap::new(),
heads: Vec::new(), heads: Vec::new(),
parents: HashMap::new(), parents: HashMap::new(),
head: None, head: None,
downloading_headers: HashSet::new(), downloading_headers: HashSet::new(),
downloading_bodies: HashSet::new(), downloading_bodies: HashSet::new(),
downloading_receipts: HashSet::new(),
} }
} }
@ -80,10 +98,12 @@ impl BlockCollection {
self.blocks.clear(); self.blocks.clear();
self.parents.clear(); self.parents.clear();
self.header_ids.clear(); self.header_ids.clear();
self.receipt_ids.clear();
self.heads.clear(); self.heads.clear();
self.head = None; self.head = None;
self.downloading_headers.clear(); self.downloading_headers.clear();
self.downloading_bodies.clear(); self.downloading_bodies.clear();
self.downloading_receipts.clear();
} }
/// Reset collection for a new sync round with given subchain block hashes. /// Reset collection for a new sync round with given subchain block hashes.
@ -108,8 +128,23 @@ impl BlockCollection {
for b in bodies.into_iter() { for b in bodies.into_iter() {
if let Err(e) = self.insert_body(b) { if let Err(e) = self.insert_body(b) {
trace!(target: "sync", "Ignored invalid body: {:?}", e); trace!(target: "sync", "Ignored invalid body: {:?}", e);
} else {
inserted += 1;
} }
else { }
inserted
}
/// Insert a collection of block receipts for previously downloaded headers.
pub fn insert_receipts(&mut self, receipts: Vec<Bytes>) -> usize {
if !self.need_receipts {
return 0;
}
let mut inserted = 0;
for r in receipts.into_iter() {
if let Err(e) = self.insert_receipt(r) {
trace!(target: "sync", "Ignored invalid receipt: {:?}", e);
} else {
inserted += 1; inserted += 1;
} }
} }
@ -147,6 +182,38 @@ impl BlockCollection {
needed_bodies needed_bodies
} }
/// Returns a set of block hashes that require a receipt download. The returned set is marked as being downloaded.
pub fn needed_receipts(&mut self, count: usize, _ignore_downloading: bool) -> Vec<H256> {
if self.head.is_none() || !self.need_receipts {
return Vec::new();
}
let mut needed_receipts: Vec<H256> = Vec::new();
let mut head = self.head;
while head.is_some() && needed_receipts.len() < count {
head = self.parents.get(&head.unwrap()).cloned();
if let Some(head) = head {
match self.blocks.get(&head) {
Some(block) if block.receipts.is_none() && !self.downloading_receipts.contains(&head) => {
self.downloading_receipts.insert(head.clone());
needed_receipts.push(head.clone());
}
_ => (),
}
}
}
for h in self.receipt_ids.values() {
if needed_receipts.len() >= count {
break;
}
if !self.downloading_receipts.contains(h) {
needed_receipts.push(h.clone());
self.downloading_receipts.insert(h.clone());
}
}
needed_receipts
}
/// Returns a set of block hashes that require a header download. The returned set is marked as being downloaded. /// Returns a set of block hashes that require a header download. The returned set is marked as being downloaded.
pub fn needed_headers(&mut self, count: usize, ignore_downloading: bool) -> Option<(H256, usize)> { pub fn needed_headers(&mut self, count: usize, ignore_downloading: bool) -> Option<(H256, usize)> {
// find subchain to download // find subchain to download
@ -163,18 +230,27 @@ impl BlockCollection {
download.map(|h| (h, count)) download.map(|h| (h, count))
} }
/// Unmark a header as being downloaded. /// Unmark header as being downloaded.
pub fn clear_header_download(&mut self, hash: &H256) { pub fn clear_header_download(&mut self, hash: &H256) {
self.downloading_headers.remove(hash); self.downloading_headers.remove(hash);
} }
/// Unmark a block body as being downloaded. /// Unmark block body as being downloaded.
pub fn clear_body_download(&mut self, hash: &H256) { pub fn clear_body_download(&mut self, hashes: &[H256]) {
self.downloading_bodies.remove(hash); for h in hashes {
self.downloading_bodies.remove(h);
}
}
/// Unmark block receipt as being downloaded.
pub fn clear_receipt_download(&mut self, hashes: &[H256]) {
for h in hashes {
self.downloading_receipts.remove(h);
}
} }
/// Get a valid chain of blocks ordered in descending order and ready for importing into blockchain. /// Get a valid chain of blocks ordered in descending order and ready for importing into blockchain.
pub fn drain(&mut self) -> Vec<Bytes> { pub fn drain(&mut self) -> Vec<BlockAndReceipts> {
if self.blocks.is_empty() || self.head.is_none() { if self.blocks.is_empty() || self.head.is_none() {
return Vec::new(); return Vec::new();
} }
@ -188,7 +264,7 @@ impl BlockCollection {
head = self.parents.get(&h).cloned(); head = self.parents.get(&h).cloned();
if let Some(head) = head { if let Some(head) = head {
match self.blocks.get(&head) { match self.blocks.get(&head) {
Some(block) if block.body.is_some() => { Some(block) if block.body.is_some() && (!self.need_receipts || block.receipts.is_some()) => {
blocks.push(block); blocks.push(block);
hashes.push(head); hashes.push(head);
self.head = Some(head); self.head = Some(head);
@ -198,19 +274,24 @@ impl BlockCollection {
} }
} }
for block in blocks.drain(..) { for block in blocks {
let mut block_rlp = RlpStream::new_list(3); let mut block_rlp = RlpStream::new_list(3);
block_rlp.append_raw(&block.header, 1); block_rlp.append_raw(&block.header, 1);
let body = Rlp::new(block.body.as_ref().expect("blocks contains only full blocks; qed")); {
block_rlp.append_raw(body.at(0).as_raw(), 1); let body = Rlp::new(block.body.as_ref().expect("blocks contains only full blocks; qed"));
block_rlp.append_raw(body.at(1).as_raw(), 1); block_rlp.append_raw(body.at(0).as_raw(), 1);
drained.push(block_rlp.out()); block_rlp.append_raw(body.at(1).as_raw(), 1);
}
drained.push(BlockAndReceipts {
block: block_rlp.out(),
receipts: block.receipts.clone(),
});
} }
} }
for h in hashes { for h in hashes {
self.blocks.remove(&h); self.blocks.remove(&h);
} }
trace!("Drained {} blocks, new head :{:?}", drained.len(), self.head); trace!(target: "sync", "Drained {} blocks, new head :{:?}", drained.len(), self.head);
drained drained
} }
@ -241,14 +322,17 @@ impl BlockCollection {
} }
fn insert_body(&mut self, b: Bytes) -> Result<(), NetworkError> { fn insert_body(&mut self, b: Bytes) -> Result<(), NetworkError> {
let body = UntrustedRlp::new(&b); let header_id = {
let tx = try!(body.at(0)); let body = UntrustedRlp::new(&b);
let tx_root = ordered_trie_root(tx.iter().map(|r| r.as_raw().to_vec())); //TODO: get rid of vectors here let tx = try!(body.at(0));
let uncles = try!(body.at(1)).as_raw().sha3(); let tx_root = ordered_trie_root(tx.iter().map(|r| r.as_raw().to_vec())); //TODO: get rid of vectors here
let header_id = HeaderId { let uncles = try!(body.at(1)).as_raw().sha3();
transactions_root: tx_root, HeaderId {
uncles: uncles transactions_root: tx_root,
uncles: uncles
}
}; };
match self.header_ids.get(&header_id).cloned() { match self.header_ids.get(&header_id).cloned() {
Some(h) => { Some(h) => {
self.header_ids.remove(&header_id); self.header_ids.remove(&header_id);
@ -256,7 +340,7 @@ impl BlockCollection {
match self.blocks.get_mut(&h) { match self.blocks.get_mut(&h) {
Some(ref mut block) => { Some(ref mut block) => {
trace!(target: "sync", "Got body {}", h); trace!(target: "sync", "Got body {}", h);
block.body = Some(body.as_raw().to_vec()); block.body = Some(b);
Ok(()) Ok(())
}, },
None => { None => {
@ -266,7 +350,35 @@ impl BlockCollection {
} }
} }
None => { None => {
trace!(target: "sync", "Ignored unknown/stale block body"); trace!(target: "sync", "Ignored unknown/stale block body. tx_root = {:?}, uncles = {:?}", header_id.transactions_root, header_id.uncles);
Err(NetworkError::BadProtocol)
}
}
}
fn insert_receipt(&mut self, r: Bytes) -> Result<(), NetworkError> {
let receipt_root = {
let receipts = UntrustedRlp::new(&r);
ordered_trie_root(receipts.iter().map(|r| r.as_raw().to_vec())) //TODO: get rid of vectors here
};
match self.receipt_ids.get(&receipt_root).cloned() {
Some(h) => {
self.receipt_ids.remove(&receipt_root);
self.downloading_receipts.remove(&h);
match self.blocks.get_mut(&h) {
Some(ref mut block) => {
trace!(target: "sync", "Got receipt {}", h);
block.receipts = Some(r);
Ok(())
},
None => {
warn!("Got receipt with no header {}", h);
Err(NetworkError::BadProtocol)
}
}
}
None => {
trace!(target: "sync", "Ignored unknown/stale block receipt {:?}", receipt_root);
Err(NetworkError::BadProtocol) Err(NetworkError::BadProtocol)
} }
} }
@ -280,7 +392,7 @@ impl BlockCollection {
} }
match self.head { match self.head {
None if hash == self.heads[0] => { None if hash == self.heads[0] => {
trace!("New head {}", hash); trace!(target: "sync", "New head {}", hash);
self.head = Some(info.parent_hash().clone()); self.head = Some(info.parent_hash().clone());
}, },
_ => () _ => ()
@ -289,6 +401,7 @@ impl BlockCollection {
let mut block = SyncBlock { let mut block = SyncBlock {
header: header, header: header,
body: None, body: None,
receipts: None,
}; };
let header_id = HeaderId { let header_id = HeaderId {
transactions_root: info.transactions_root().clone(), transactions_root: info.transactions_root().clone(),
@ -302,8 +415,21 @@ impl BlockCollection {
block.body = Some(body_stream.out()); block.body = Some(body_stream.out());
} }
else { else {
trace!("Queueing body tx_root = {:?}, uncles = {:?}, block = {:?}, number = {}", header_id.transactions_root, header_id.uncles, hash, info.number());
self.header_ids.insert(header_id, hash.clone()); self.header_ids.insert(header_id, hash.clone());
} }
if self.need_receipts {
let receipt_root = info.receipts_root().clone();
if receipt_root == sha3::SHA3_NULL_RLP {
let receipts_stream = RlpStream::new_list(0);
block.receipts = Some(receipts_stream.out());
} else {
if self.receipt_ids.contains_key(&receipt_root) {
warn!(target: "sync", "Duplicate receipt root {:?}, block: {:?}", receipt_root, hash);
}
self.receipt_ids.insert(receipt_root, hash.clone());
}
}
self.parents.insert(info.parent_hash().clone(), hash.clone()); self.parents.insert(info.parent_hash().clone(), hash.clone());
self.blocks.insert(hash.clone(), block); self.blocks.insert(hash.clone(), block);
@ -326,7 +452,7 @@ impl BlockCollection {
Some(next) => { Some(next) => {
h = next.clone(); h = next.clone();
if old_subchains.contains(&h) { if old_subchains.contains(&h) {
trace!("Completed subchain {:?}", s); trace!(target: "sync", "Completed subchain {:?}", s);
break; // reached head of the other subchain, merge by not adding break; // reached head of the other subchain, merge by not adding
} }
}, },
@ -362,7 +488,7 @@ mod test {
#[test] #[test]
fn create_clear() { fn create_clear() {
let mut bc = BlockCollection::new(); let mut bc = BlockCollection::new(false);
assert!(is_empty(&bc)); assert!(is_empty(&bc));
let client = TestBlockChainClient::new(); let client = TestBlockChainClient::new();
client.add_blocks(100, EachBlockWith::Nothing); client.add_blocks(100, EachBlockWith::Nothing);
@ -375,7 +501,7 @@ mod test {
#[test] #[test]
fn insert_headers() { fn insert_headers() {
let mut bc = BlockCollection::new(); let mut bc = BlockCollection::new(false);
assert!(is_empty(&bc)); assert!(is_empty(&bc));
let client = TestBlockChainClient::new(); let client = TestBlockChainClient::new();
let nblocks = 200; let nblocks = 200;
@ -407,7 +533,7 @@ mod test {
assert!(!bc.is_downloading(&hashes[0])); assert!(!bc.is_downloading(&hashes[0]));
assert!(bc.contains(&hashes[0])); assert!(bc.contains(&hashes[0]));
assert_eq!(&bc.drain()[..], &blocks[0..6]); assert_eq!(&bc.drain().into_iter().map(|b| b.block).collect::<Vec<_>>()[..], &blocks[0..6]);
assert!(!bc.contains(&hashes[0])); assert!(!bc.contains(&hashes[0]));
assert_eq!(hashes[5], bc.head.unwrap()); assert_eq!(hashes[5], bc.head.unwrap());
@ -418,7 +544,7 @@ mod test {
bc.insert_headers(headers[10..16].to_vec()); bc.insert_headers(headers[10..16].to_vec());
assert!(bc.drain().is_empty()); assert!(bc.drain().is_empty());
bc.insert_headers(headers[5..10].to_vec()); bc.insert_headers(headers[5..10].to_vec());
assert_eq!(&bc.drain()[..], &blocks[6..16]); assert_eq!(&bc.drain().into_iter().map(|b| b.block).collect::<Vec<_>>()[..], &blocks[6..16]);
assert_eq!(hashes[15], bc.heads[0]); assert_eq!(hashes[15], bc.heads[0]);
bc.insert_headers(headers[15..].to_vec()); bc.insert_headers(headers[15..].to_vec());
@ -428,7 +554,7 @@ mod test {
#[test] #[test]
fn insert_headers_with_gap() { fn insert_headers_with_gap() {
let mut bc = BlockCollection::new(); let mut bc = BlockCollection::new(false);
assert!(is_empty(&bc)); assert!(is_empty(&bc));
let client = TestBlockChainClient::new(); let client = TestBlockChainClient::new();
let nblocks = 200; let nblocks = 200;
@ -450,7 +576,7 @@ mod test {
#[test] #[test]
fn insert_headers_no_gap() { fn insert_headers_no_gap() {
let mut bc = BlockCollection::new(); let mut bc = BlockCollection::new(false);
assert!(is_empty(&bc)); assert!(is_empty(&bc));
let client = TestBlockChainClient::new(); let client = TestBlockChainClient::new();
let nblocks = 200; let nblocks = 200;

File diff suppressed because it is too large Load Diff

View File

@ -48,6 +48,7 @@ extern crate ethcore_ipc as ipc;
mod chain; mod chain;
mod blocks; mod blocks;
mod block_sync;
mod sync_io; mod sync_io;
mod snapshot; mod snapshot;

View File

@ -14,9 +14,13 @@
// You should have received a copy of the GNU General Public License // You should have received a copy of the GNU General Public License
// along with Parity. If not, see <http://www.gnu.org/licenses/>. // along with Parity. If not, see <http://www.gnu.org/licenses/>.
use std::collections::HashMap;
use network::{NetworkContext, PeerId, PacketId, NetworkError, SessionInfo}; use network::{NetworkContext, PeerId, PacketId, NetworkError, SessionInfo};
use util::Bytes;
use ethcore::client::BlockChainClient; use ethcore::client::BlockChainClient;
use ethcore::header::BlockNumber;
use ethcore::snapshot::SnapshotService; use ethcore::snapshot::SnapshotService;
use parking_lot::RwLock;
/// IO interface for the syning handler. /// IO interface for the syning handler.
/// Provides peer connection management and an interface to the blockchain client. /// Provides peer connection management and an interface to the blockchain client.
@ -48,6 +52,8 @@ pub trait SyncIo {
} }
/// Check if the session is expired /// Check if the session is expired
fn is_expired(&self) -> bool; fn is_expired(&self) -> bool;
/// Return sync overlay
fn chain_overlay(&self) -> &RwLock<HashMap<BlockNumber, Bytes>>;
} }
/// Wraps `NetworkContext` and the blockchain client /// Wraps `NetworkContext` and the blockchain client
@ -55,15 +61,20 @@ pub struct NetSyncIo<'s, 'h> where 'h: 's {
network: &'s NetworkContext<'h>, network: &'s NetworkContext<'h>,
chain: &'s BlockChainClient, chain: &'s BlockChainClient,
snapshot_service: &'s SnapshotService, snapshot_service: &'s SnapshotService,
chain_overlay: &'s RwLock<HashMap<BlockNumber, Bytes>>,
} }
impl<'s, 'h> NetSyncIo<'s, 'h> { impl<'s, 'h> NetSyncIo<'s, 'h> {
/// Creates a new instance from the `NetworkContext` and the blockchain client reference. /// Creates a new instance from the `NetworkContext` and the blockchain client reference.
pub fn new(network: &'s NetworkContext<'h>, chain: &'s BlockChainClient, snapshot_service: &'s SnapshotService) -> NetSyncIo<'s, 'h> { pub fn new(network: &'s NetworkContext<'h>,
chain: &'s BlockChainClient,
snapshot_service: &'s SnapshotService,
chain_overlay: &'s RwLock<HashMap<BlockNumber, Bytes>>) -> NetSyncIo<'s, 'h> {
NetSyncIo { NetSyncIo {
network: network, network: network,
chain: chain, chain: chain,
snapshot_service: snapshot_service, snapshot_service: snapshot_service,
chain_overlay: chain_overlay,
} }
} }
} }
@ -89,6 +100,10 @@ impl<'s, 'h> SyncIo for NetSyncIo<'s, 'h> {
self.chain self.chain
} }
fn chain_overlay(&self) -> &RwLock<HashMap<BlockNumber, Bytes>> {
self.chain_overlay
}
fn snapshot_service(&self) -> &SnapshotService { fn snapshot_service(&self) -> &SnapshotService {
self.snapshot_service self.snapshot_service
} }

View File

@ -30,6 +30,7 @@ pub struct TestIo<'p> {
pub queue: &'p mut VecDeque<TestPacket>, pub queue: &'p mut VecDeque<TestPacket>,
pub sender: Option<PeerId>, pub sender: Option<PeerId>,
pub to_disconnect: HashSet<PeerId>, pub to_disconnect: HashSet<PeerId>,
overlay: RwLock<HashMap<BlockNumber, Bytes>>,
} }
impl<'p> TestIo<'p> { impl<'p> TestIo<'p> {
@ -40,6 +41,7 @@ impl<'p> TestIo<'p> {
queue: queue, queue: queue,
sender: sender, sender: sender,
to_disconnect: HashSet::new(), to_disconnect: HashSet::new(),
overlay: RwLock::new(HashMap::new()),
} }
} }
} }
@ -90,6 +92,10 @@ impl<'p> SyncIo for TestIo<'p> {
fn eth_protocol_version(&self, _peer: PeerId) -> u8 { fn eth_protocol_version(&self, _peer: PeerId) -> u8 {
64 64
} }
fn chain_overlay(&self) -> &RwLock<HashMap<BlockNumber, Bytes>> {
&self.overlay
}
} }
pub struct TestPacket { pub struct TestPacket {
@ -149,6 +155,7 @@ impl TestNet {
for client in 0..self.peers.len() { for client in 0..self.peers.len() {
if peer != client { if peer != client {
let mut p = self.peers.get_mut(peer).unwrap(); let mut p = self.peers.get_mut(peer).unwrap();
p.sync.write().restart(&mut TestIo::new(&mut p.chain, &p.snapshot_service, &mut p.queue, Some(client as PeerId)));
p.sync.write().on_peer_connected(&mut TestIo::new(&mut p.chain, &p.snapshot_service, &mut p.queue, Some(client as PeerId)), client as PeerId); p.sync.write().on_peer_connected(&mut TestIo::new(&mut p.chain, &p.snapshot_service, &mut p.queue, Some(client as PeerId)), client as PeerId);
} }
} }

View File

@ -77,7 +77,9 @@ impl SnapshotService for TestSnapshotService {
match *self.restoration_manifest.lock() { match *self.restoration_manifest.lock() {
Some(ref manifest) if self.state_restoration_chunks.lock().len() == manifest.state_hashes.len() && Some(ref manifest) if self.state_restoration_chunks.lock().len() == manifest.state_hashes.len() &&
self.block_restoration_chunks.lock().len() == manifest.block_hashes.len() => RestorationStatus::Inactive, self.block_restoration_chunks.lock().len() == manifest.block_hashes.len() => RestorationStatus::Inactive,
Some(_) => RestorationStatus::Ongoing { Some(ref manifest) => RestorationStatus::Ongoing {
state_chunks: manifest.state_hashes.len() as u32,
block_chunks: manifest.block_hashes.len() as u32,
state_chunks_done: self.state_restoration_chunks.lock().len() as u32, state_chunks_done: self.state_restoration_chunks.lock().len() as u32,
block_chunks_done: self.block_restoration_chunks.lock().len() as u32, block_chunks_done: self.block_restoration_chunks.lock().len() as u32,
}, },
@ -114,7 +116,7 @@ impl SnapshotService for TestSnapshotService {
fn snapshot_sync() { fn snapshot_sync() {
::env_logger::init().ok(); ::env_logger::init().ok();
let mut net = TestNet::new(2); let mut net = TestNet::new(2);
net.peer_mut(0).snapshot_service = Arc::new(TestSnapshotService::new_with_snapshot(16, H256::new(), 1)); net.peer_mut(0).snapshot_service = Arc::new(TestSnapshotService::new_with_snapshot(16, H256::new(), 500000));
net.peer_mut(0).chain.add_blocks(1, EachBlockWith::Nothing); net.peer_mut(0).chain.add_blocks(1, EachBlockWith::Nothing);
net.sync_steps(19); // status + manifest + chunks net.sync_steps(19); // status + manifest + chunks
assert_eq!(net.peer(1).snapshot_service.state_restoration_chunks.lock().len(), net.peer(0).snapshot_service.manifest.as_ref().unwrap().state_hashes.len()); assert_eq!(net.peer(1).snapshot_service.state_restoration_chunks.lock().len(), net.peer(0).snapshot_service.manifest.as_ref().unwrap().state_hashes.len());

View File

@ -101,11 +101,11 @@ impl Worker {
let _ = wait.wait(lock); let _ = wait.wait(lock);
} }
if deleting.load(AtomicOrdering::Acquire) { while !deleting.load(AtomicOrdering::Acquire) {
return; match stealer.steal() {
} chase_lev::Steal::Data(work) => Worker::do_work(work, channel.clone()),
while let chase_lev::Steal::Data(work) = stealer.steal() { _ => break,
Worker::do_work(work, channel.clone()); }
} }
} }
} }

View File

@ -106,10 +106,11 @@ impl<Socket: GenericSocket> GenericConnection<Socket> {
/// Add a packet to send queue. /// Add a packet to send queue.
pub fn send<Message>(&mut self, io: &IoContext<Message>, data: Bytes) where Message: Send + Clone { pub fn send<Message>(&mut self, io: &IoContext<Message>, data: Bytes) where Message: Send + Clone {
if !data.is_empty() { if !data.is_empty() {
trace!(target:"network", "{}: Sending {} bytes", self.token, data.len());
self.send_queue.push_back(Cursor::new(data)); self.send_queue.push_back(Cursor::new(data));
} if !self.interest.is_writable() {
if !self.interest.is_writable() { self.interest.insert(EventSet::writable());
self.interest.insert(EventSet::writable()); }
io.update_registration(self.token).ok(); io.update_registration(self.token).ok();
} }
} }

View File

@ -30,7 +30,7 @@ use node_table::NodeId;
use stats::NetworkStats; use stats::NetworkStats;
use time; use time;
const PING_TIMEOUT_SEC: u64 = 30; const PING_TIMEOUT_SEC: u64 = 65;
const PING_INTERVAL_SEC: u64 = 30; const PING_INTERVAL_SEC: u64 = 30;
/// Peer session over encrypted connection. /// Peer session over encrypted connection.

View File

@ -46,4 +46,4 @@ pub use rustc_serialize::hex::{FromHex, FromHexError};
pub use heapsize::HeapSizeOf; pub use heapsize::HeapSizeOf;
pub use itertools::Itertools; pub use itertools::Itertools;
pub use parking_lot::{Condvar, Mutex, MutexGuard, RwLock, RwLockReadGuard, RwLockWriteGuard}; pub use parking_lot::{Condvar, Mutex, MutexGuard, RwLock, RwLockReadGuard, RwLockWriteGuard};