Add new line after writing block to hex file. (#10984)

* add new line after writing block to hex file.

* refactor for testability

* correct import

* better error reporting, code formatting

* multiline imports

* docs

* better docs, move type to common types, merge ImportBlocks and ExportBlocks

* tabs over spaces

* correct test imports

* Apply suggestions from code review

Co-Authored-By: David <dvdplm@gmail.com>

* correct typo

* fixed test import
This commit is contained in:
Seun LanLege 2019-09-09 12:46:05 +01:00 committed by David
parent 53e590f54b
commit 80f0e4b58f
8 changed files with 391 additions and 179 deletions

View File

@ -614,7 +614,7 @@ impl BlockChain {
let best_block_rlp = bc.block(&best_block_hash) let best_block_rlp = bc.block(&best_block_hash)
.expect("Best block is from a known block hash; qed"); .expect("Best block is from a known block hash; qed");
// and write them // and write them to the cache.
let mut best_block = bc.best_block.write(); let mut best_block = bc.best_block.write();
*best_block = BestBlock { *best_block = BestBlock {
total_difficulty: best_block_total_difficulty, total_difficulty: best_block_total_difficulty,
@ -877,12 +877,31 @@ impl BlockChain {
} }
} }
/// clears all caches for testing purposes /// clears all caches, re-loads best block from disk for testing purposes
pub fn clear_cache(&self) { pub fn clear_cache(&self) {
self.block_bodies.write().clear(); self.block_bodies.write().clear();
self.block_details.write().clear(); self.block_details.write().clear();
self.block_hashes.write().clear(); self.block_hashes.write().clear();
self.block_headers.write().clear(); self.block_headers.write().clear();
// Fetch best block details from disk
let best_block_hash = self.db.key_value().get(db::COL_EXTRA, b"best")
.expect("Low-level database error when fetching 'best' block. Some issue with disk?")
.as_ref()
.map(|r| H256::from_slice(r))
.unwrap();
let best_block_total_difficulty = self.block_details(&best_block_hash)
.expect("Best block is from a known block hash; a known block hash always comes with a known block detail; qed")
.total_difficulty;
let best_block_rlp = self.block(&best_block_hash)
.expect("Best block is from a known block hash; qed");
// and write them to the cache
let mut best_block = self.best_block.write();
*best_block = BestBlock {
total_difficulty: best_block_total_difficulty,
header: best_block_rlp.decode_header(),
block: best_block_rlp,
};
} }
/// Update the best ancient block to the given hash, after checking that /// Update the best ancient block to the given hash, after checking that

View File

@ -55,6 +55,7 @@ use trace::{
localized::LocalizedTrace, localized::LocalizedTrace,
VMTrace, VMTrace,
}; };
use common_types::data_format::DataFormat;
use vm::{LastHashes, Schedule}; use vm::{LastHashes, Schedule};
use common_types::snapshot::Progress; use common_types::snapshot::Progress;
@ -510,3 +511,29 @@ pub trait ChainNotify: Send + Sync {
// does nothing by default // does nothing by default
} }
} }
/// Provides a method for importing/exporting blocks
pub trait ImportExportBlocks {
/// Export blocks to destination, with the given from, to and format argument.
/// destination could be a file or stdout.
/// If the format is hex, each block is written on a new line.
/// For binary exports, all block data is written to the same line.
fn export_blocks<'a>(
&self,
destination: Box<dyn std::io::Write + 'a>,
from: BlockId,
to: BlockId,
format: Option<DataFormat>
) -> Result<(), String>;
/// Import blocks from destination, with the given format argument
/// Source could be a file or stdout.
/// For hex format imports, it attempts to read the blocks on a line by line basis.
/// For binary format imports, reads the 8 byte RLP header in order to decode the block
/// length to be read.
fn import_blocks<'a>(
&self,
source: Box<dyn std::io::Read + 'a>,
format: Option<DataFormat>
) -> Result<(), String>;
}

View File

@ -15,98 +15,129 @@
// along with Parity Ethereum. If not, see <http://www.gnu.org/licenses/>. // along with Parity Ethereum. If not, see <http://www.gnu.org/licenses/>.
use std::cmp; use std::cmp;
use std::collections::{HashSet, BTreeMap, VecDeque}; use std::collections::{BTreeMap, HashSet, VecDeque};
use std::sync::atomic::{AtomicUsize, AtomicBool, Ordering as AtomicOrdering}; use std::io::{BufRead, BufReader};
use std::str::from_utf8;
use std::sync::{Arc, Weak}; use std::sync::{Arc, Weak};
use std::time::{Instant, Duration}; use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering as AtomicOrdering};
use std::time::{Duration, Instant};
use account_state::state::StateInfo; use ansi_term::Colour;
use blockchain::{BlockReceipts, BlockChain, BlockChainDB, BlockProvider, TreeRoute, TransactionAddress, ExtrasInsert, BlockNumberKey};
use bytes::Bytes; use bytes::Bytes;
use call_contract::{CallContract, RegistryInfo}; use bytes::ToPretty;
use ethcore_miner::pool::VerifiedTransaction; use ethereum_types::{Address, H256, H264, U256};
use ethereum_types::{H256, H264, Address, U256};
use evm::Schedule;
use hash::keccak; use hash::keccak;
use io::IoChannel; use hash_db::EMPTY_PREFIX;
use itertools::Itertools; use itertools::Itertools;
use journaldb; use kvdb::{DBTransaction, DBValue, KeyValueDB};
use kvdb::{DBValue, KeyValueDB, DBTransaction};
use parking_lot::{Mutex, RwLock}; use parking_lot::{Mutex, RwLock};
use rand::rngs::OsRng; use rand::rngs::OsRng;
use trie::{TrieSpec, TrieFactory, Trie}; use rlp::PayloadInfo;
use vm::{EnvInfo, LastHashes, CreateContractAddress}; use rustc_hex::FromHex;
use hash_db::EMPTY_PREFIX; use trie::{Trie, TrieFactory, TrieSpec};
use block::{LockedBlock, Drain, ClosedBlock, OpenBlock, enact_verified, SealedBlock};
use client::ancient_import::AncientVerifier; use account_state::State;
use account_state::state::StateInfo;
use block::{ClosedBlock, Drain, enact_verified, LockedBlock, OpenBlock, SealedBlock};
use blockchain::{
BlockChain,
BlockChainDB,
BlockNumberKey,
BlockProvider,
BlockReceipts,
ExtrasInsert,
TransactionAddress,
TreeRoute
};
// re-export
pub use blockchain::CacheSize as BlockChainCacheSize;
use call_contract::{CallContract, RegistryInfo};
use client::{ use client::{
ReopenBlock, PrepareOpenBlock, ImportSealedBlock, BroadcastProposalBlock, bad_blocks, BlockProducer, BroadcastProposalBlock, Call,
Call, BlockProducer, SealedBlockImporter, EngineInfo, ClientConfig, EngineInfo, ImportSealedBlock, PrepareOpenBlock,
ClientConfig, bad_blocks, ReopenBlock, SealedBlockImporter,
}; };
use client::ancient_import::AncientVerifier;
use client_traits::{ use client_traits::{
BlockInfo, ScheduleInfo, StateClient, BlockChainReset, AccountData,
Nonce, Balance, ChainInfo, TransactionInfo, ImportBlock, BadBlocks,
AccountData, BlockChain as BlockChainTrait, BlockChainClient, Balance,
IoClient, BadBlocks, ProvingBlockChainClient, SnapshotClient, BlockChain as BlockChainTrait,
DatabaseRestore, SnapshotWriter, Tick, ChainNotify, BlockChainClient,
BlockChainReset,
BlockInfo,
ChainInfo,
ChainNotify,
DatabaseRestore,
ImportBlock,
ImportExportBlocks,
IoClient,
Nonce,
ProvingBlockChainClient,
ScheduleInfo,
SnapshotClient,
SnapshotWriter,
StateClient,
StateOrBlock, StateOrBlock,
Tick,
TransactionInfo
}; };
use db::{keys::BlockDetails, Readable, Writable};
use engine::Engine; use engine::Engine;
use ethcore_miner::pool::VerifiedTransaction;
use ethtrie::Layout;
use evm::Schedule;
use executive_state;
use io::IoChannel;
use journaldb;
use machine::{ use machine::{
executed::Executed, executed::Executed,
executive::{Executive, TransactOptions, contract_address}, executive::{contract_address, Executive, TransactOptions},
transaction_ext::Transaction, transaction_ext::Transaction,
}; };
use trie_vm_factories::{Factories, VmFactory};
use miner::{Miner, MinerService}; use miner::{Miner, MinerService};
use snapshot; use snapshot;
use spec::Spec; use spec::Spec;
use account_state::State;
use executive_state;
use state_db::StateDB; use state_db::StateDB;
use trace::{self, TraceDB, ImportRequest as TraceImportRequest, LocalizedTrace, Database as TraceDatabase}; use trace::{self, Database as TraceDatabase, ImportRequest as TraceImportRequest, LocalizedTrace, TraceDB};
use trie_vm_factories::{Factories, VmFactory};
use types::{ use types::{
ancestry_action::AncestryAction, ancestry_action::AncestryAction,
BlockNumber,
block::PreverifiedBlock, block::PreverifiedBlock,
block_status::BlockStatus, block_status::BlockStatus,
blockchain_info::BlockChainInfo, blockchain_info::BlockChainInfo,
chain_notify::{NewBlocks, ChainRoute, ChainMessageType}, BlockNumber,
call_analytics::CallAnalytics,
chain_notify::{ChainMessageType, ChainRoute, NewBlocks},
client_types::ClientReport, client_types::ClientReport,
import_route::ImportRoute, client_types::Mode,
io_message::ClientIoMessage,
encoded, encoded,
engines::{ engines::{
ForkChoice,
SealingState,
MAX_UNCLE_AGE,
epoch::{PendingTransition, Transition as EpochTransition}, epoch::{PendingTransition, Transition as EpochTransition},
ForkChoice,
machine::{AuxiliaryData, Call as MachineCall}, machine::{AuxiliaryData, Call as MachineCall},
MAX_UNCLE_AGE,
SealingState,
}, },
errors::{EngineError, ExecutionError, BlockError, EthcoreError, SnapshotError, ImportError, EthcoreResult}, errors::{BlockError, EngineError, EthcoreError, EthcoreResult, ExecutionError, ImportError, SnapshotError},
ids::{BlockId, TransactionId, UncleId, TraceId},
transaction::{self, LocalizedTransaction, UnverifiedTransaction, SignedTransaction, Action, CallError},
filter::Filter, filter::Filter,
log_entry::LocalizedLogEntry,
receipt::{Receipt, LocalizedReceipt},
header::Header, header::Header,
ids::{BlockId, TraceId, TransactionId, UncleId},
import_route::ImportRoute,
io_message::ClientIoMessage,
log_entry::LocalizedLogEntry,
pruning_info::PruningInfo,
receipt::{LocalizedReceipt, Receipt},
snapshot::{Progress, Snapshotting}, snapshot::{Progress, Snapshotting},
trace_filter::Filter as TraceFilter, trace_filter::Filter as TraceFilter,
pruning_info::PruningInfo, transaction::{self, Action, CallError, LocalizedTransaction, SignedTransaction, UnverifiedTransaction},
call_analytics::CallAnalytics,
client_types::Mode,
verification::{Unverified, VerificationQueueInfo as BlockQueueInfo}, verification::{Unverified, VerificationQueueInfo as BlockQueueInfo},
}; };
use types::data_format::DataFormat;
use verification::queue::kind::BlockLike; use verification::{BlockQueue, Verifier};
use verification::{Verifier, BlockQueue};
use verification; use verification;
use ansi_term::Colour; use verification::queue::kind::BlockLike;
use ethtrie::Layout; use vm::{CreateContractAddress, EnvInfo, LastHashes};
// re-export
pub use blockchain::CacheSize as BlockChainCacheSize;
use db::{Writable, Readable, keys::BlockDetails};
use_contract!(registry, "res/contracts/registrar.json"); use_contract!(registry, "res/contracts/registrar.json");
@ -2555,6 +2586,116 @@ impl SnapshotClient for Client {
} }
impl ImportExportBlocks for Client {
fn export_blocks<'a>(
&self,
mut out: Box<dyn std::io::Write + 'a>,
from: BlockId,
to: BlockId,
format: Option<DataFormat>
) -> Result<(), String> {
let from = self.block_number(from).ok_or("Starting block could not be found")?;
let to = self.block_number(to).ok_or("End block could not be found")?;
let format = format.unwrap_or_default();
for i in from..=to {
if i % 10000 == 0 {
info!("#{}", i);
}
let b = self.block(BlockId::Number(i)).ok_or("Error exporting incomplete chain")?.into_inner();
match format {
DataFormat::Binary => {
out.write(&b).map_err(|e| format!("Couldn't write to stream. Cause: {}", e))?;
}
DataFormat::Hex => {
out.write_fmt(format_args!("{}\n", b.pretty())).map_err(|e| format!("Couldn't write to stream. Cause: {}", e))?;
}
}
}
Ok(())
}
fn import_blocks<'a>(
&self,
mut source: Box<dyn std::io::Read + 'a>,
format: Option<DataFormat>
) -> Result<(), String> {
const READAHEAD_BYTES: usize = 8;
let mut first_bytes: Vec<u8> = vec![0; READAHEAD_BYTES];
let mut first_read = 0;
let format = match format {
Some(format) => format,
None => {
first_read = source.read(&mut first_bytes).map_err(|_| "Error reading from the file/stream.")?;
match first_bytes[0] {
0xf9 => DataFormat::Binary,
_ => DataFormat::Hex,
}
}
};
let do_import = |bytes: Vec<u8>| {
let block = Unverified::from_rlp(bytes).map_err(|_| "Invalid block rlp")?;
let number = block.header.number();
while self.queue_info().is_full() { std::thread::sleep(Duration::from_secs(1)); }
match self.import_block(block) {
Err(EthcoreError::Import(ImportError::AlreadyInChain)) => {
trace!("Skipping block #{}: already in chain.", number);
}
Err(e) => {
return Err(format!("Cannot import block #{}: {:?}", number, e));
},
Ok(_) => {},
}
Ok(())
};
match format {
DataFormat::Binary => {
loop {
let (mut bytes, n) = if first_read > 0 {
(first_bytes.clone(), first_read)
} else {
let mut bytes = vec![0; READAHEAD_BYTES];
let n = source.read(&mut bytes)
.map_err(|err| format!("Error reading from the file/stream: {:?}", err))?;
(bytes, n)
};
if n == 0 { break; }
first_read = 0;
let s = PayloadInfo::from(&bytes)
.map_err(|e| format!("Invalid RLP in the file/stream: {:?}", e))?.total();
bytes.resize(s, 0);
source.read_exact(&mut bytes[n..])
.map_err(|err| format!("Error reading from the file/stream: {:?}", err))?;
do_import(bytes)?;
}
}
DataFormat::Hex => {
for line in BufReader::new(source).lines() {
let s = line
.map_err(|err| format!("Error reading from the file/stream: {:?}", err))?;
let s = if first_read > 0 {
from_utf8(&first_bytes)
.map_err(|err| format!("Invalid UTF-8: {:?}", err))?
.to_owned() + &(s[..])
} else {
s
};
first_read = 0;
let bytes = s.from_hex()
.map_err(|err| format!("Invalid hex in file/stream: {:?}", err))?;
do_import(bytes)?;
}
}
};
self.flush_queue();
Ok(())
}
}
/// Returns `LocalizedReceipt` given `LocalizedTransaction` /// Returns `LocalizedReceipt` given `LocalizedTransaction`
/// and a vector of receipts from given block up to transaction index. /// and a vector of receipts from given block up to transaction index.
fn transaction_receipt( fn transaction_receipt(
@ -2634,26 +2775,28 @@ impl IoChannelQueue {
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use ethereum_types::{H256, Address}; use std::sync::Arc;
use std::sync::atomic::{AtomicBool, Ordering};
use std::thread;
use std::time::Duration;
use ethereum_types::{Address, H256};
use hash::keccak;
use kvdb::DBTransaction;
use blockchain::{ExtrasInsert, BlockProvider};
use client_traits::{BlockChainClient, ChainInfo}; use client_traits::{BlockChainClient, ChainInfo};
use ethkey::KeyPair;
use types::{ use types::{
encoded, encoded,
engines::ForkChoice, engines::ForkChoice,
ids::{BlockId, TransactionId}, ids::{BlockId, TransactionId},
log_entry::{LogEntry, LocalizedLogEntry}, log_entry::{LocalizedLogEntry, LogEntry},
receipt::{Receipt, LocalizedReceipt, TransactionOutcome}, receipt::{LocalizedReceipt, Receipt, TransactionOutcome},
transaction::{Transaction, LocalizedTransaction, Action}, transaction::{Action, LocalizedTransaction, Transaction},
}; };
use test_helpers::{generate_dummy_client, generate_dummy_client_with_data, generate_dummy_client_with_spec_and_data, get_good_dummy_block_hash}; use test_helpers::{generate_dummy_client, generate_dummy_client_with_data, generate_dummy_client_with_spec_and_data, get_good_dummy_block_hash};
use std::thread;
use std::time::Duration;
use std::sync::Arc;
use std::sync::atomic::{AtomicBool, Ordering};
use kvdb::DBTransaction;
use blockchain::{BlockProvider, ExtrasInsert};
use hash::keccak;
use super::transaction_receipt; use super::transaction_receipt;
use ethkey::KeyPair;
#[test] #[test]
fn should_not_cache_details_before_commit() { fn should_not_cache_details_before_commit() {

View File

@ -14,7 +14,7 @@
// 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 Ethereum. If not, see <http://www.gnu.org/licenses/>. // along with Parity Ethereum. If not, see <http://www.gnu.org/licenses/>.
use std::str::FromStr; use std::str::{FromStr, from_utf8};
use std::sync::Arc; use std::sync::Arc;
use ethereum_types::{U256, Address}; use ethereum_types::{U256, Address};
@ -23,6 +23,7 @@ use hash::keccak;
use io::IoChannel; use io::IoChannel;
use tempdir::TempDir; use tempdir::TempDir;
use types::{ use types::{
data_format::DataFormat,
ids::BlockId, ids::BlockId,
transaction::{PendingTransaction, Transaction, Action, Condition}, transaction::{PendingTransaction, Transaction, Action, Condition},
filter::Filter, filter::Filter,
@ -34,7 +35,7 @@ use types::{
use client::{Client, ClientConfig, PrepareOpenBlock, ImportSealedBlock}; use client::{Client, ClientConfig, PrepareOpenBlock, ImportSealedBlock};
use client_traits::{ use client_traits::{
BlockInfo, BlockChainClient, BlockChainReset, ChainInfo, BlockInfo, BlockChainClient, BlockChainReset, ChainInfo,
ImportBlock, Tick, ImportExportBlocks, Tick, ImportBlock
}; };
use spec; use spec;
use machine::executive::{Executive, TransactOptions}; use machine::executive::{Executive, TransactOptions};
@ -45,6 +46,7 @@ use test_helpers::{
generate_dummy_client, push_blocks_to_client, get_test_client_with_blocks, get_good_dummy_block_seq, generate_dummy_client, push_blocks_to_client, get_test_client_with_blocks, get_good_dummy_block_seq,
generate_dummy_client_with_data, get_good_dummy_block, get_bad_state_dummy_block generate_dummy_client_with_data, get_good_dummy_block, get_bad_state_dummy_block
}; };
use rustc_hex::ToHex;
#[test] #[test]
fn imports_from_empty() { fn imports_from_empty() {
@ -387,3 +389,79 @@ fn reset_blockchain() {
assert!(client.block_header(BlockId::Number(15)).is_some()); assert!(client.block_header(BlockId::Number(15)).is_some());
} }
#[test]
fn import_export_hex() {
let client = get_test_client_with_blocks(get_good_dummy_block_seq(19));
let block_rlps = (15..20)
.filter_map(|num| client.block(BlockId::Number(num)))
.map(|header| {
header.raw().to_hex()
})
.collect::<Vec<_>>();
let mut out = Vec::new();
client.export_blocks(
Box::new(&mut out),
BlockId::Number(15),
BlockId::Number(20),
Some(DataFormat::Hex)
).unwrap();
let written = from_utf8(&out)
.unwrap()
.split("\n")
// last line is empty, ignore it.
.take(5)
.collect::<Vec<_>>();
assert_eq!(block_rlps, written);
assert!(client.reset(5).is_ok());
client.chain().clear_cache();
assert!(client.block_header(BlockId::Number(20)).is_none());
assert!(client.block_header(BlockId::Number(19)).is_none());
assert!(client.block_header(BlockId::Number(18)).is_none());
assert!(client.block_header(BlockId::Number(17)).is_none());
assert!(client.block_header(BlockId::Number(16)).is_none());
client.import_blocks(Box::new(&*out), Some(DataFormat::Hex)).unwrap();
assert!(client.block_header(BlockId::Number(20)).is_some());
assert!(client.block_header(BlockId::Number(19)).is_some());
assert!(client.block_header(BlockId::Number(18)).is_some());
assert!(client.block_header(BlockId::Number(17)).is_some());
assert!(client.block_header(BlockId::Number(16)).is_some());
}
#[test]
fn import_export_binary() {
let client = get_test_client_with_blocks(get_good_dummy_block_seq(19));
let mut out = Vec::new();
client.export_blocks(
Box::new(&mut out),
BlockId::Number(15),
BlockId::Number(20),
Some(DataFormat::Binary)
).unwrap();
assert!(client.reset(5).is_ok());
client.chain().clear_cache();
assert!(client.block_header(BlockId::Number(20)).is_none());
assert!(client.block_header(BlockId::Number(19)).is_none());
assert!(client.block_header(BlockId::Number(18)).is_none());
assert!(client.block_header(BlockId::Number(17)).is_none());
assert!(client.block_header(BlockId::Number(16)).is_none());
client.import_blocks(Box::new(&*out), Some(DataFormat::Binary)).unwrap();
assert!(client.block_header(BlockId::Number(19)).is_some());
assert!(client.block_header(BlockId::Number(18)).is_some());
assert!(client.block_header(BlockId::Number(20)).is_some());
assert!(client.block_header(BlockId::Number(17)).is_some());
assert!(client.block_header(BlockId::Number(16)).is_some());
}

View File

@ -0,0 +1,43 @@
// Copyright 2015-2019 Parity Technologies (UK) Ltd.
// This file is part of Parity Ethereum.
// Parity Ethereum 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 Ethereum 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 Ethereum. If not, see <http://www.gnu.org/licenses/>.
//! Data format for importing/exporting blocks from disk
use std::str::FromStr;
/// Format for importing/exporting blocks
#[derive(Debug, PartialEq)]
pub enum DataFormat {
Hex,
Binary,
}
impl Default for DataFormat {
fn default() -> Self {
DataFormat::Binary
}
}
impl FromStr for DataFormat {
type Err = String;
fn from_str(s: &str) -> Result<Self, Self::Err> {
match s {
"binary" | "bin" => Ok(DataFormat::Binary),
"hex" => Ok(DataFormat::Hex),
x => Err(format!("Invalid format: {}", x))
}
}
}

View File

@ -84,6 +84,7 @@ pub mod trace_filter;
pub mod transaction; pub mod transaction;
pub mod tree_route; pub mod tree_route;
pub mod verification; pub mod verification;
pub mod data_format;
/// Type for block number. /// Type for block number.
pub type BlockNumber = u64; pub type BlockNumber = u64;

View File

@ -14,7 +14,7 @@
// 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 Ethereum. If not, see <http://www.gnu.org/licenses/>. // along with Parity Ethereum. If not, see <http://www.gnu.org/licenses/>.
use std::str::{FromStr, from_utf8}; use std::str::from_utf8;
use std::{io, fs}; use std::{io, fs};
use std::io::{BufReader, BufRead}; use std::io::{BufReader, BufRead};
use std::time::{Instant, Duration}; use std::time::{Instant, Duration};
@ -26,7 +26,7 @@ use hash::{keccak, KECCAK_NULL_RLP};
use ethereum_types::{U256, H256, Address}; use ethereum_types::{U256, H256, Address};
use bytes::ToPretty; use bytes::ToPretty;
use rlp::PayloadInfo; use rlp::PayloadInfo;
use client_traits::{BlockInfo, BlockChainReset, Nonce, Balance, BlockChainClient, ImportBlock}; use client_traits::{BlockChainReset, Nonce, Balance, BlockChainClient, ImportExportBlocks};
use ethcore::{ use ethcore::{
client::{DatabaseCompactionProfile, VMType}, client::{DatabaseCompactionProfile, VMType},
miner::Miner, miner::Miner,
@ -45,34 +45,10 @@ use types::{
ids::BlockId, ids::BlockId,
errors::{ImportError, EthcoreError}, errors::{ImportError, EthcoreError},
client_types::Mode, client_types::Mode,
verification::Unverified,
}; };
use types::data_format::DataFormat;
use verification::queue::VerifierSettings; use verification::queue::VerifierSettings;
#[derive(Debug, PartialEq)]
pub enum DataFormat {
Hex,
Binary,
}
impl Default for DataFormat {
fn default() -> Self {
DataFormat::Binary
}
}
impl FromStr for DataFormat {
type Err = String;
fn from_str(s: &str) -> Result<Self, Self::Err> {
match s {
"binary" | "bin" => Ok(DataFormat::Binary),
"hex" => Ok(DataFormat::Hex),
x => Err(format!("Invalid format: {}", x))
}
}
}
#[derive(Debug, PartialEq)] #[derive(Debug, PartialEq)]
pub enum BlockchainCmd { pub enum BlockchainCmd {
Kill(KillBlockchain), Kill(KillBlockchain),
@ -412,27 +388,11 @@ fn execute_import(cmd: ImportBlockchain) -> Result<(), String> {
let client = service.client(); let client = service.client();
let mut instream: Box<dyn io::Read> = match cmd.file_path { let instream: Box<dyn io::Read> = match cmd.file_path {
Some(f) => Box::new(fs::File::open(&f).map_err(|_| format!("Cannot open given file: {}", f))?), Some(f) => Box::new(fs::File::open(&f).map_err(|_| format!("Cannot open given file: {}", f))?),
None => Box::new(io::stdin()), None => Box::new(io::stdin()),
}; };
const READAHEAD_BYTES: usize = 8;
let mut first_bytes: Vec<u8> = vec![0; READAHEAD_BYTES];
let mut first_read = 0;
let format = match cmd.format {
Some(format) => format,
None => {
first_read = instream.read(&mut first_bytes).map_err(|_| "Error reading from the file/stream.")?;
match first_bytes[0] {
0xf9 => DataFormat::Binary,
_ => DataFormat::Hex,
}
}
};
let informant = Arc::new(Informant::new( let informant = Arc::new(Informant::new(
FullNodeInformantData { FullNodeInformantData {
client: client.clone(), client: client.clone(),
@ -446,49 +406,7 @@ fn execute_import(cmd: ImportBlockchain) -> Result<(), String> {
service.register_io_handler(informant).map_err(|_| "Unable to register informant handler".to_owned())?; service.register_io_handler(informant).map_err(|_| "Unable to register informant handler".to_owned())?;
let do_import = |bytes| { client.import_blocks(instream, cmd.format)?;
let block = Unverified::from_rlp(bytes).map_err(|_| "Invalid block rlp")?;
while client.queue_info().is_full() { sleep(Duration::from_secs(1)); }
match client.import_block(block) {
Err(EthcoreError::Import(ImportError::AlreadyInChain)) => {
trace!("Skipping block already in chain.");
}
Err(e) => {
return Err(format!("Cannot import block: {:?}", e));
},
Ok(_) => {},
}
Ok(())
};
match format {
DataFormat::Binary => {
loop {
let mut bytes = if first_read > 0 {first_bytes.clone()} else {vec![0; READAHEAD_BYTES]};
let n = if first_read > 0 {
first_read
} else {
instream.read(&mut bytes).map_err(|_| "Error reading from the file/stream.")?
};
if n == 0 { break; }
first_read = 0;
let s = PayloadInfo::from(&bytes).map_err(|e| format!("Invalid RLP in the file/stream: {:?}", e))?.total();
bytes.resize(s, 0);
instream.read_exact(&mut bytes[n..]).map_err(|_| "Error reading from the file/stream.")?;
do_import(bytes)?;
}
}
DataFormat::Hex => {
for line in BufReader::new(instream).lines() {
let s = line.map_err(|_| "Error reading from the file/stream.")?;
let s = if first_read > 0 {from_utf8(&first_bytes).unwrap().to_owned() + &(s[..])} else {s};
first_read = 0;
let bytes = s.from_hex().map_err(|_| "Invalid hex in file/stream.")?;
do_import(bytes)?;
}
}
}
client.flush_queue();
// save user defaults // save user defaults
user_defaults.pruning = algorithm; user_defaults.pruning = algorithm;
@ -617,32 +535,14 @@ fn execute_export(cmd: ExportBlockchain) -> Result<(), String> {
false, false,
cmd.max_round_blocks_to_import, cmd.max_round_blocks_to_import,
)?; )?;
let format = cmd.format.unwrap_or_default();
let client = service.client(); let client = service.client();
let mut out: Box<dyn io::Write> = match cmd.file_path { let out: Box<dyn io::Write> = match cmd.file_path {
Some(f) => Box::new(fs::File::create(&f).map_err(|_| format!("Cannot write to file given: {}", f))?), Some(f) => Box::new(fs::File::create(&f).map_err(|_| format!("Cannot write to file given: {}", f))?),
None => Box::new(io::stdout()), None => Box::new(io::stdout()),
}; };
let from = client.block_number(cmd.from_block).ok_or("From block could not be found")?; client.export_blocks(out, cmd.from_block, cmd.to_block, cmd.format)?;
let to = client.block_number(cmd.to_block).ok_or("To block could not be found")?;
for i in from..(to + 1) {
if i % 10000 == 0 {
info!("#{}", i);
}
let b = client.block(BlockId::Number(i)).ok_or("Error exporting incomplete chain")?.into_inner();
match format {
DataFormat::Binary => {
out.write(&b).map_err(|e| format!("Couldn't write to stream. Cause: {}", e))?;
}
DataFormat::Hex => {
out.write_fmt(format_args!("{}", b.pretty())).map_err(|e| format!("Couldn't write to stream. Cause: {}", e))?;
}
}
}
info!("Export completed."); info!("Export completed.");
Ok(()) Ok(())

View File

@ -48,7 +48,8 @@ use ethcore_private_tx::{ProviderConfig, EncryptorConfig};
use secretstore::{NodeSecretKey, Configuration as SecretStoreConfiguration, ContractAddress as SecretStoreContractAddress}; use secretstore::{NodeSecretKey, Configuration as SecretStoreConfiguration, ContractAddress as SecretStoreContractAddress};
use updater::{UpdatePolicy, UpdateFilter, ReleaseTrack}; use updater::{UpdatePolicy, UpdateFilter, ReleaseTrack};
use run::RunCmd; use run::RunCmd;
use blockchain::{BlockchainCmd, ImportBlockchain, ExportBlockchain, KillBlockchain, ExportState, DataFormat, ResetBlockchain}; use types::data_format::DataFormat;
use blockchain::{BlockchainCmd, ImportBlockchain, ExportBlockchain, KillBlockchain, ExportState, ResetBlockchain};
use export_hardcoded_sync::ExportHsyncCmd; use export_hardcoded_sync::ExportHsyncCmd;
use presale::ImportWallet; use presale::ImportWallet;
use account::{AccountCmd, NewAccount, ListAccounts, ImportAccounts, ImportFromGethAccounts}; use account::{AccountCmd, NewAccount, ListAccounts, ImportAccounts, ImportFromGethAccounts};
@ -1201,9 +1202,9 @@ mod tests {
use parity_rpc::NetworkSettings; use parity_rpc::NetworkSettings;
use updater::{UpdatePolicy, UpdateFilter, ReleaseTrack}; use updater::{UpdatePolicy, UpdateFilter, ReleaseTrack};
use types::ids::BlockId; use types::ids::BlockId;
use types::data_format::DataFormat;
use account::{AccountCmd, NewAccount, ImportAccounts, ListAccounts}; use account::{AccountCmd, NewAccount, ImportAccounts, ListAccounts};
use blockchain::{BlockchainCmd, ImportBlockchain, ExportBlockchain, DataFormat, ExportState}; use blockchain::{BlockchainCmd, ImportBlockchain, ExportBlockchain, ExportState};
use cli::Args; use cli::Args;
use dir::{Directories, default_hypervisor_path}; use dir::{Directories, default_hypervisor_path};
use helpers::{default_network_config}; use helpers::{default_network_config};