Multithreaded snapshot creation (#9239)

* Add Progress to Snapshot Secondary chunks creation

* Use half of CPUs to multithread snapshot creation

* Use env var to define number of threads

* info to debug logs

* Add Snapshot threads as CLI option

* Randomize chunks per thread

* Remove randomness, add debugging

* Add warning

* Add tracing

* Use parity-common fix seek branch

* Fix log

* Fix tests

* Fix tests

* PR Grumbles

* PR Grumble II

* Update Cargo.lock

* PR Grumbles

* Default snapshot threads to half number of CPUs

* Fix default snapshot threads // min 1
This commit is contained in:
Nicolas Gotchac 2018-09-13 12:58:49 +02:00 committed by Afri Schoedon
parent ef4a61c769
commit 4ddd69cc55
12 changed files with 454 additions and 338 deletions

623
Cargo.lock generated

File diff suppressed because it is too large Load Diff

View File

@ -1150,7 +1150,8 @@ impl Client {
}, },
}; };
snapshot::take_snapshot(&*self.engine, &self.chain.read(), start_hash, db.as_hashdb(), writer, p)?; let processing_threads = self.config.snapshot.processing_threads;
snapshot::take_snapshot(&*self.engine, &self.chain.read(), start_hash, db.as_hashdb(), writer, p, processing_threads)?;
Ok(()) Ok(())
} }

View File

@ -19,6 +19,7 @@ use std::fmt::{Display, Formatter, Error as FmtError};
use verification::{VerifierType, QueueConfig}; use verification::{VerifierType, QueueConfig};
use journaldb; use journaldb;
use snapshot::SnapshotConfiguration;
pub use std::time::Duration; pub use std::time::Duration;
pub use blockchain::Config as BlockChainConfig; pub use blockchain::Config as BlockChainConfig;
@ -120,6 +121,8 @@ pub struct ClientConfig {
pub check_seal: bool, pub check_seal: bool,
/// Maximal number of transactions queued for verification in a separate thread. /// Maximal number of transactions queued for verification in a separate thread.
pub transaction_verification_queue_size: usize, pub transaction_verification_queue_size: usize,
/// Snapshot configuration
pub snapshot: SnapshotConfiguration,
} }
impl Default for ClientConfig { impl Default for ClientConfig {
@ -144,6 +147,7 @@ impl Default for ClientConfig {
history_mem: 32 * mb, history_mem: 32 * mb,
check_seal: true, check_seal: true,
transaction_verification_queue_size: 8192, transaction_verification_queue_size: 8192,
snapshot: Default::default(),
} }
} }
} }

View File

@ -30,7 +30,7 @@ use machine::EthereumMachine;
use ids::BlockId; use ids::BlockId;
use header::Header; use header::Header;
use receipt::Receipt; use receipt::Receipt;
use snapshot::{Error, ManifestData}; use snapshot::{Error, ManifestData, Progress};
use itertools::{Position, Itertools}; use itertools::{Position, Itertools};
use rlp::{RlpStream, Rlp}; use rlp::{RlpStream, Rlp};
@ -59,6 +59,7 @@ impl SnapshotComponents for PoaSnapshot {
chain: &BlockChain, chain: &BlockChain,
block_at: H256, block_at: H256,
sink: &mut ChunkSink, sink: &mut ChunkSink,
_progress: &Progress,
preferred_size: usize, preferred_size: usize,
) -> Result<(), Error> { ) -> Result<(), Error> {
let number = chain.block_number(&block_at) let number = chain.block_number(&block_at)

View File

@ -22,7 +22,7 @@ use std::sync::Arc;
use blockchain::{BlockChain, BlockChainDB}; use blockchain::{BlockChain, BlockChainDB};
use engines::EthEngine; use engines::EthEngine;
use snapshot::{Error, ManifestData}; use snapshot::{Error, ManifestData, Progress};
use ethereum_types::H256; use ethereum_types::H256;
@ -49,6 +49,7 @@ pub trait SnapshotComponents: Send {
chain: &BlockChain, chain: &BlockChain,
block_at: H256, block_at: H256,
chunk_sink: &mut ChunkSink, chunk_sink: &mut ChunkSink,
progress: &Progress,
preferred_size: usize, preferred_size: usize,
) -> Result<(), Error>; ) -> Result<(), Error>;

View File

@ -28,7 +28,7 @@ use std::sync::Arc;
use blockchain::{BlockChain, BlockChainDB, BlockProvider}; use blockchain::{BlockChain, BlockChainDB, BlockProvider};
use engines::EthEngine; use engines::EthEngine;
use snapshot::{Error, ManifestData}; use snapshot::{Error, ManifestData, Progress};
use snapshot::block::AbridgedBlock; use snapshot::block::AbridgedBlock;
use ethereum_types::H256; use ethereum_types::H256;
use kvdb::KeyValueDB; use kvdb::KeyValueDB;
@ -65,6 +65,7 @@ impl SnapshotComponents for PowSnapshot {
chain: &BlockChain, chain: &BlockChain,
block_at: H256, block_at: H256,
chunk_sink: &mut ChunkSink, chunk_sink: &mut ChunkSink,
progress: &Progress,
preferred_size: usize, preferred_size: usize,
) -> Result<(), Error> { ) -> Result<(), Error> {
PowWorker { PowWorker {
@ -72,6 +73,7 @@ impl SnapshotComponents for PowSnapshot {
rlps: VecDeque::new(), rlps: VecDeque::new(),
current_hash: block_at, current_hash: block_at,
writer: chunk_sink, writer: chunk_sink,
progress: progress,
preferred_size: preferred_size, preferred_size: preferred_size,
}.chunk_all(self.blocks) }.chunk_all(self.blocks)
} }
@ -96,6 +98,7 @@ struct PowWorker<'a> {
rlps: VecDeque<Bytes>, rlps: VecDeque<Bytes>,
current_hash: H256, current_hash: H256,
writer: &'a mut ChunkSink<'a>, writer: &'a mut ChunkSink<'a>,
progress: &'a Progress,
preferred_size: usize, preferred_size: usize,
} }
@ -138,6 +141,7 @@ impl<'a> PowWorker<'a> {
last = self.current_hash; last = self.current_hash;
self.current_hash = block.header_view().parent_hash(); self.current_hash = block.header_view().parent_hash();
self.progress.blocks.fetch_add(1, Ordering::SeqCst);
} }
if loaded_size != 0 { if loaded_size != 0 {

View File

@ -20,6 +20,7 @@
//! https://wiki.parity.io/Warp-Sync-Snapshot-Format //! https://wiki.parity.io/Warp-Sync-Snapshot-Format
use std::collections::{HashMap, HashSet}; use std::collections::{HashMap, HashSet};
use std::cmp;
use std::sync::Arc; use std::sync::Arc;
use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering}; use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering};
use hash::{keccak, KECCAK_NULL_RLP, KECCAK_EMPTY}; use hash::{keccak, KECCAK_NULL_RLP, KECCAK_EMPTY};
@ -43,6 +44,7 @@ use trie::{Trie, TrieMut};
use ethtrie::{TrieDB, TrieDBMut}; use ethtrie::{TrieDB, TrieDBMut};
use rlp::{RlpStream, Rlp}; use rlp::{RlpStream, Rlp};
use bloom_journal::Bloom; use bloom_journal::Bloom;
use num_cpus;
use self::io::SnapshotWriter; use self::io::SnapshotWriter;
@ -88,6 +90,28 @@ const MAX_CHUNK_SIZE: usize = PREFERRED_CHUNK_SIZE / 4 * 5;
const MIN_SUPPORTED_STATE_CHUNK_VERSION: u64 = 1; const MIN_SUPPORTED_STATE_CHUNK_VERSION: u64 = 1;
// current state chunk version. // current state chunk version.
const STATE_CHUNK_VERSION: u64 = 2; const STATE_CHUNK_VERSION: u64 = 2;
/// number of snapshot subparts, must be a power of 2 in [1; 256]
const SNAPSHOT_SUBPARTS: usize = 16;
/// Maximum number of snapshot subparts (must be a multiple of `SNAPSHOT_SUBPARTS`)
const MAX_SNAPSHOT_SUBPARTS: usize = 256;
/// Configuration for the Snapshot service
#[derive(Debug, Clone, PartialEq)]
pub struct SnapshotConfiguration {
/// If `true`, no periodic snapshots will be created
pub no_periodic: bool,
/// Number of threads for creating snapshots
pub processing_threads: usize,
}
impl Default for SnapshotConfiguration {
fn default() -> Self {
SnapshotConfiguration {
no_periodic: false,
processing_threads: ::std::cmp::max(1, num_cpus::get() / 2),
}
}
}
/// A progress indicator for snapshots. /// A progress indicator for snapshots.
#[derive(Debug, Default)] #[derive(Debug, Default)]
@ -130,7 +154,8 @@ pub fn take_snapshot<W: SnapshotWriter + Send>(
block_at: H256, block_at: H256,
state_db: &HashDB<KeccakHasher>, state_db: &HashDB<KeccakHasher>,
writer: W, writer: W,
p: &Progress p: &Progress,
processing_threads: usize,
) -> Result<(), Error> { ) -> Result<(), Error> {
let start_header = chain.block_header_data(&block_at) let start_header = chain.block_header_data(&block_at)
.ok_or(Error::InvalidStartingBlock(BlockId::Hash(block_at)))?; .ok_or(Error::InvalidStartingBlock(BlockId::Hash(block_at)))?;
@ -142,17 +167,45 @@ pub fn take_snapshot<W: SnapshotWriter + Send>(
let writer = Mutex::new(writer); let writer = Mutex::new(writer);
let chunker = engine.snapshot_components().ok_or(Error::SnapshotsUnsupported)?; let chunker = engine.snapshot_components().ok_or(Error::SnapshotsUnsupported)?;
let snapshot_version = chunker.current_version(); let snapshot_version = chunker.current_version();
let (state_hashes, block_hashes) = scope(|scope| { let (state_hashes, block_hashes) = scope(|scope| -> Result<(Vec<H256>, Vec<H256>), Error> {
let writer = &writer; let writer = &writer;
let block_guard = scope.spawn(move || chunk_secondary(chunker, chain, block_at, writer, p)); let block_guard = scope.spawn(move || chunk_secondary(chunker, chain, block_at, writer, p));
let state_res = chunk_state(state_db, &state_root, writer, p);
state_res.and_then(|state_hashes| { // The number of threads must be between 1 and SNAPSHOT_SUBPARTS
block_guard.join().map(|block_hashes| (state_hashes, block_hashes)) assert!(processing_threads >= 1, "Cannot use less than 1 threads for creating snapshots");
}) let num_threads: usize = cmp::min(processing_threads, SNAPSHOT_SUBPARTS);
info!(target: "snapshot", "Using {} threads for Snapshot creation.", num_threads);
let mut state_guards = Vec::with_capacity(num_threads as usize);
for thread_idx in 0..num_threads {
let state_guard = scope.spawn(move || -> Result<Vec<H256>, Error> {
let mut chunk_hashes = Vec::new();
for part in (thread_idx..SNAPSHOT_SUBPARTS).step_by(num_threads) {
debug!(target: "snapshot", "Chunking part {} in thread {}", part, thread_idx);
let mut hashes = chunk_state(state_db, &state_root, writer, p, Some(part))?;
chunk_hashes.append(&mut hashes);
}
Ok(chunk_hashes)
});
state_guards.push(state_guard);
}
let block_hashes = block_guard.join()?;
let mut state_hashes = Vec::new();
for guard in state_guards {
let part_state_hashes = guard.join()?;
state_hashes.extend(part_state_hashes);
}
debug!(target: "snapshot", "Took a snapshot of {} accounts", p.accounts.load(Ordering::SeqCst));
Ok((state_hashes, block_hashes))
})?; })?;
info!("produced {} state chunks and {} block chunks.", state_hashes.len(), block_hashes.len()); info!(target: "snapshot", "produced {} state chunks and {} block chunks.", state_hashes.len(), block_hashes.len());
let manifest_data = ManifestData { let manifest_data = ManifestData {
version: snapshot_version, version: snapshot_version,
@ -200,6 +253,7 @@ pub fn chunk_secondary<'a>(mut chunker: Box<SnapshotComponents>, chain: &'a Bloc
chain, chain,
start_hash, start_hash,
&mut chunk_sink, &mut chunk_sink,
progress,
PREFERRED_CHUNK_SIZE, PREFERRED_CHUNK_SIZE,
)?; )?;
} }
@ -263,10 +317,12 @@ impl<'a> StateChunker<'a> {
/// Walk the given state database starting from the given root, /// Walk the given state database starting from the given root,
/// creating chunks and writing them out. /// creating chunks and writing them out.
/// `part` is a number between 0 and 15, which describe which part of
/// the tree should be chunked.
/// ///
/// Returns a list of hashes of chunks created, or any error it may /// Returns a list of hashes of chunks created, or any error it may
/// have encountered. /// have encountered.
pub fn chunk_state<'a>(db: &HashDB<KeccakHasher>, root: &H256, writer: &Mutex<SnapshotWriter + 'a>, progress: &'a Progress) -> Result<Vec<H256>, Error> { pub fn chunk_state<'a>(db: &HashDB<KeccakHasher>, root: &H256, writer: &Mutex<SnapshotWriter + 'a>, progress: &'a Progress, part: Option<usize>) -> Result<Vec<H256>, Error> {
let account_trie = TrieDB::new(db, &root)?; let account_trie = TrieDB::new(db, &root)?;
let mut chunker = StateChunker { let mut chunker = StateChunker {
@ -281,11 +337,33 @@ pub fn chunk_state<'a>(db: &HashDB<KeccakHasher>, root: &H256, writer: &Mutex<Sn
let mut used_code = HashSet::new(); let mut used_code = HashSet::new();
// account_key here is the address' hash. // account_key here is the address' hash.
for item in account_trie.iter()? { let mut account_iter = account_trie.iter()?;
let mut seek_to = None;
if let Some(part) = part {
assert!(part < 16, "Wrong chunk state part number (must be <16) in snapshot creation.");
let part_offset = MAX_SNAPSHOT_SUBPARTS / SNAPSHOT_SUBPARTS;
let mut seek_from = vec![0; 32];
seek_from[0] = (part * part_offset) as u8;
account_iter.seek(&seek_from)?;
// Set the upper-bound, except for the last part
if part < SNAPSHOT_SUBPARTS - 1 {
seek_to = Some(((part + 1) * part_offset) as u8)
}
}
for item in account_iter {
let (account_key, account_data) = item?; let (account_key, account_data) = item?;
let account = ::rlp::decode(&*account_data)?;
let account_key_hash = H256::from_slice(&account_key); let account_key_hash = H256::from_slice(&account_key);
if seek_to.map_or(false, |seek_to| account_key[0] >= seek_to) {
break;
}
let account = ::rlp::decode(&*account_data)?;
let account_db = AccountDB::from_hash(db, account_key_hash); let account_db = AccountDB::from_hash(db, account_key_hash);
let fat_rlps = account::to_fat_rlps(&account_key_hash, &account, &account_db, &mut used_code, PREFERRED_CHUNK_SIZE - chunker.chunk_size(), PREFERRED_CHUNK_SIZE)?; let fat_rlps = account::to_fat_rlps(&account_key_hash, &account, &account_db, &mut used_code, PREFERRED_CHUNK_SIZE - chunker.chunk_size(), PREFERRED_CHUNK_SIZE)?;

View File

@ -22,7 +22,7 @@ use hash::{KECCAK_NULL_RLP, keccak};
use basic_account::BasicAccount; use basic_account::BasicAccount;
use snapshot::account; use snapshot::account;
use snapshot::{chunk_state, Error as SnapshotError, Progress, StateRebuilder}; use snapshot::{chunk_state, Error as SnapshotError, Progress, StateRebuilder, SNAPSHOT_SUBPARTS};
use snapshot::io::{PackedReader, PackedWriter, SnapshotReader, SnapshotWriter}; use snapshot::io::{PackedReader, PackedWriter, SnapshotReader, SnapshotWriter};
use super::helpers::{compare_dbs, StateProducer}; use super::helpers::{compare_dbs, StateProducer};
@ -53,7 +53,11 @@ fn snap_and_restore() {
let state_root = producer.state_root(); let state_root = producer.state_root();
let writer = Mutex::new(PackedWriter::new(&snap_file).unwrap()); let writer = Mutex::new(PackedWriter::new(&snap_file).unwrap());
let state_hashes = chunk_state(&old_db, &state_root, &writer, &Progress::default()).unwrap(); let mut state_hashes = Vec::new();
for part in 0..SNAPSHOT_SUBPARTS {
let mut hashes = chunk_state(&old_db, &state_root, &writer, &Progress::default(), Some(part)).unwrap();
state_hashes.append(&mut hashes);
}
writer.into_inner().finish(::snapshot::ManifestData { writer.into_inner().finish(::snapshot::ManifestData {
version: 2, version: 2,
@ -164,7 +168,7 @@ fn checks_flag() {
let state_root = producer.state_root(); let state_root = producer.state_root();
let writer = Mutex::new(PackedWriter::new(&snap_file).unwrap()); let writer = Mutex::new(PackedWriter::new(&snap_file).unwrap());
let state_hashes = chunk_state(&old_db, &state_root, &writer, &Progress::default()).unwrap(); let state_hashes = chunk_state(&old_db, &state_root, &writer, &Progress::default(), None).unwrap();
writer.into_inner().finish(::snapshot::ManifestData { writer.into_inner().finish(::snapshot::ManifestData {
version: 2, version: 2,

View File

@ -874,6 +874,10 @@ usage! {
"--no-periodic-snapshot", "--no-periodic-snapshot",
"Disable automated snapshots which usually occur once every 10000 blocks.", "Disable automated snapshots which usually occur once every 10000 blocks.",
ARG arg_snapshot_threads: (Option<usize>) = None, or |c: &Config| c.snapshots.as_ref()?.processing_threads,
"--snapshot-threads=[NUM]",
"Enables multiple threads for snapshots creation.",
["Whisper Options"] ["Whisper Options"]
FLAG flag_whisper: (bool) = false, or |c: &Config| c.whisper.as_ref()?.enabled, FLAG flag_whisper: (bool) = false, or |c: &Config| c.whisper.as_ref()?.enabled,
"--whisper", "--whisper",
@ -1355,6 +1359,7 @@ struct Footprint {
#[serde(deny_unknown_fields)] #[serde(deny_unknown_fields)]
struct Snapshots { struct Snapshots {
disable_periodic: Option<bool>, disable_periodic: Option<bool>,
processing_threads: Option<usize>,
} }
#[derive(Default, Debug, PartialEq, Deserialize)] #[derive(Default, Debug, PartialEq, Deserialize)]
@ -1788,6 +1793,7 @@ mod tests {
arg_export_state_at: "latest".into(), arg_export_state_at: "latest".into(),
arg_snapshot_at: "latest".into(), arg_snapshot_at: "latest".into(),
flag_no_periodic_snapshot: false, flag_no_periodic_snapshot: false,
arg_snapshot_threads: None,
// -- Light options. // -- Light options.
arg_on_demand_retry_count: Some(15), arg_on_demand_retry_count: Some(15),
@ -2046,6 +2052,7 @@ mod tests {
}), }),
snapshots: Some(Snapshots { snapshots: Some(Snapshots {
disable_periodic: Some(true), disable_periodic: Some(true),
processing_threads: None,
}), }),
misc: Some(Misc { misc: Some(Misc {
logging: Some("own_tx=trace".into()), logging: Some("own_tx=trace".into()),

View File

@ -30,8 +30,10 @@ use sync::{NetworkConfiguration, validate_node_url, self};
use ethcore::ethstore::ethkey::{Secret, Public}; use ethcore::ethstore::ethkey::{Secret, Public};
use ethcore::client::{VMType}; use ethcore::client::{VMType};
use ethcore::miner::{stratum, MinerOptions}; use ethcore::miner::{stratum, MinerOptions};
use ethcore::snapshot::SnapshotConfiguration;
use ethcore::verification::queue::VerifierSettings; use ethcore::verification::queue::VerifierSettings;
use miner::pool; use miner::pool;
use num_cpus;
use rpc::{IpcConfiguration, HttpConfiguration, WsConfiguration}; use rpc::{IpcConfiguration, HttpConfiguration, WsConfiguration};
use parity_rpc::NetworkSettings; use parity_rpc::NetworkSettings;
@ -125,6 +127,7 @@ impl Configuration {
let update_policy = self.update_policy()?; let update_policy = self.update_policy()?;
let logger_config = self.logger_config(); let logger_config = self.logger_config();
let ws_conf = self.ws_config()?; let ws_conf = self.ws_config()?;
let snapshot_conf = self.snapshot_config()?;
let http_conf = self.http_config()?; let http_conf = self.http_config()?;
let ipc_conf = self.ipc_config()?; let ipc_conf = self.ipc_config()?;
let net_conf = self.net_config()?; let net_conf = self.net_config()?;
@ -298,6 +301,7 @@ impl Configuration {
file_path: self.args.arg_snapshot_file.clone(), file_path: self.args.arg_snapshot_file.clone(),
kind: snapshot::Kind::Take, kind: snapshot::Kind::Take,
block_at: to_block_id(&self.args.arg_snapshot_at)?, block_at: to_block_id(&self.args.arg_snapshot_at)?,
snapshot_conf: snapshot_conf,
}; };
Cmd::Snapshot(snapshot_cmd) Cmd::Snapshot(snapshot_cmd)
} else if self.args.cmd_restore { } else if self.args.cmd_restore {
@ -314,6 +318,7 @@ impl Configuration {
file_path: self.args.arg_restore_file.clone(), file_path: self.args.arg_restore_file.clone(),
kind: snapshot::Kind::Restore, kind: snapshot::Kind::Restore,
block_at: to_block_id("latest")?, // unimportant. block_at: to_block_id("latest")?, // unimportant.
snapshot_conf: snapshot_conf,
}; };
Cmd::Snapshot(restore_cmd) Cmd::Snapshot(restore_cmd)
} else if self.args.cmd_export_hardcoded_sync { } else if self.args.cmd_export_hardcoded_sync {
@ -349,6 +354,7 @@ impl Configuration {
gas_price_percentile: self.args.arg_gas_price_percentile, gas_price_percentile: self.args.arg_gas_price_percentile,
poll_lifetime: self.args.arg_poll_lifetime, poll_lifetime: self.args.arg_poll_lifetime,
ws_conf: ws_conf, ws_conf: ws_conf,
snapshot_conf: snapshot_conf,
http_conf: http_conf, http_conf: http_conf,
ipc_conf: ipc_conf, ipc_conf: ipc_conf,
net_conf: net_conf, net_conf: net_conf,
@ -374,7 +380,6 @@ impl Configuration {
private_tx_enabled, private_tx_enabled,
name: self.args.arg_identity, name: self.args.arg_identity,
custom_bootnodes: self.args.arg_bootnodes.is_some(), custom_bootnodes: self.args.arg_bootnodes.is_some(),
no_periodic_snapshot: self.args.flag_no_periodic_snapshot,
check_seal: !self.args.flag_no_seal_check, check_seal: !self.args.flag_no_seal_check,
download_old_blocks: !self.args.flag_no_ancient_blocks, download_old_blocks: !self.args.flag_no_ancient_blocks,
verifier_settings: verifier_settings, verifier_settings: verifier_settings,
@ -892,6 +897,18 @@ impl Configuration {
Ok((provider_conf, encryptor_conf, self.args.flag_private_enabled)) Ok((provider_conf, encryptor_conf, self.args.flag_private_enabled))
} }
fn snapshot_config(&self) -> Result<SnapshotConfiguration, String> {
let conf = SnapshotConfiguration {
no_periodic: self.args.flag_no_periodic_snapshot,
processing_threads: match self.args.arg_snapshot_threads {
Some(threads) if threads > 0 => threads,
_ => ::std::cmp::max(1, num_cpus::get() / 2),
},
};
Ok(conf)
}
fn network_settings(&self) -> Result<NetworkSettings, String> { fn network_settings(&self) -> Result<NetworkSettings, String> {
let http_conf = self.http_config()?; let http_conf = self.http_config()?;
let net_addresses = self.net_addresses()?; let net_addresses = self.net_addresses()?;
@ -1400,7 +1417,7 @@ mod tests {
name: "".into(), name: "".into(),
custom_bootnodes: false, custom_bootnodes: false,
fat_db: Default::default(), fat_db: Default::default(),
no_periodic_snapshot: false, snapshot_conf: Default::default(),
stratum: None, stratum: None,
check_seal: true, check_seal: true,
download_old_blocks: true, download_old_blocks: true,

View File

@ -25,7 +25,7 @@ use ethcore::account_provider::{AccountProvider, AccountProviderSettings};
use ethcore::client::{BlockId, CallContract, Client, Mode, DatabaseCompactionProfile, VMType, BlockChainClient, BlockInfo}; use ethcore::client::{BlockId, CallContract, Client, Mode, DatabaseCompactionProfile, VMType, BlockChainClient, BlockInfo};
use ethcore::ethstore::ethkey; use ethcore::ethstore::ethkey;
use ethcore::miner::{stratum, Miner, MinerService, MinerOptions}; use ethcore::miner::{stratum, Miner, MinerService, MinerOptions};
use ethcore::snapshot; use ethcore::snapshot::{self, SnapshotConfiguration};
use ethcore::spec::{SpecParams, OptimizeFor}; use ethcore::spec::{SpecParams, OptimizeFor};
use ethcore::verification::queue::VerifierSettings; use ethcore::verification::queue::VerifierSettings;
use ethcore_logger::{Config as LogConfig, RotatingLogger}; use ethcore_logger::{Config as LogConfig, RotatingLogger};
@ -119,7 +119,7 @@ pub struct RunCmd {
pub name: String, pub name: String,
pub custom_bootnodes: bool, pub custom_bootnodes: bool,
pub stratum: Option<stratum::Options>, pub stratum: Option<stratum::Options>,
pub no_periodic_snapshot: bool, pub snapshot_conf: SnapshotConfiguration,
pub check_seal: bool, pub check_seal: bool,
pub download_old_blocks: bool, pub download_old_blocks: bool,
pub verifier_settings: VerifierSettings, pub verifier_settings: VerifierSettings,
@ -539,6 +539,7 @@ fn execute_impl<Cr, Rr>(cmd: RunCmd, logger: Arc<RotatingLogger>, on_client_rq:
client_config.queue.verifier_settings = cmd.verifier_settings; client_config.queue.verifier_settings = cmd.verifier_settings;
client_config.transaction_verification_queue_size = ::std::cmp::max(2048, txpool_size / 4); client_config.transaction_verification_queue_size = ::std::cmp::max(2048, txpool_size / 4);
client_config.snapshot = cmd.snapshot_conf.clone();
// set up bootnodes // set up bootnodes
let mut net_conf = cmd.net_conf; let mut net_conf = cmd.net_conf;
@ -786,7 +787,7 @@ fn execute_impl<Cr, Rr>(cmd: RunCmd, logger: Arc<RotatingLogger>, on_client_rq:
}); });
// the watcher must be kept alive. // the watcher must be kept alive.
let watcher = match cmd.no_periodic_snapshot { let watcher = match cmd.snapshot_conf.no_periodic {
true => None, true => None,
false => { false => {
let sync = sync_provider.clone(); let sync = sync_provider.clone();

View File

@ -22,7 +22,7 @@ use std::sync::Arc;
use hash::keccak; use hash::keccak;
use ethcore::account_provider::AccountProvider; use ethcore::account_provider::AccountProvider;
use ethcore::snapshot::{Progress, RestorationStatus, SnapshotService as SS}; use ethcore::snapshot::{Progress, RestorationStatus, SnapshotConfiguration, SnapshotService as SS};
use ethcore::snapshot::io::{SnapshotReader, PackedReader, PackedWriter}; use ethcore::snapshot::io::{SnapshotReader, PackedReader, PackedWriter};
use ethcore::snapshot::service::Service as SnapshotService; use ethcore::snapshot::service::Service as SnapshotService;
use ethcore::client::{Mode, DatabaseCompactionProfile, VMType}; use ethcore::client::{Mode, DatabaseCompactionProfile, VMType};
@ -62,6 +62,7 @@ pub struct SnapshotCommand {
pub file_path: Option<String>, pub file_path: Option<String>,
pub kind: Kind, pub kind: Kind,
pub block_at: BlockId, pub block_at: BlockId,
pub snapshot_conf: SnapshotConfiguration,
} }
// helper for reading chunks from arbitrary reader and feeding them into the // helper for reading chunks from arbitrary reader and feeding them into the
@ -165,7 +166,7 @@ impl SnapshotCommand {
execute_upgrades(&self.dirs.base, &db_dirs, algorithm, &self.compaction)?; execute_upgrades(&self.dirs.base, &db_dirs, algorithm, &self.compaction)?;
// prepare client config // prepare client config
let client_config = to_client_config( let mut client_config = to_client_config(
&self.cache_config, &self.cache_config,
spec.name.to_lowercase(), spec.name.to_lowercase(),
Mode::Active, Mode::Active,
@ -180,6 +181,8 @@ impl SnapshotCommand {
true, true,
); );
client_config.snapshot = self.snapshot_conf;
let restoration_db_handler = db::restoration_db_handler(&client_path, &client_config); let restoration_db_handler = db::restoration_db_handler(&client_path, &client_config);
let client_db = restoration_db_handler.open(&client_path) let client_db = restoration_db_handler.open(&client_path)
.map_err(|e| format!("Failed to open database {:?}", e))?; .map_err(|e| format!("Failed to open database {:?}", e))?;