Newer
Older
let retracted = if is_new_best {
let route_from_best = header_metadata::tree_route(
self.backend.blockchain(),
info.best_hash,
parent_hash,
)?;
route_from_best.retracted().iter().rev().map(|e| e.hash.clone()).collect()
} else {
Vec::default()
};
trace!("Imported {}, (#{}), best={}, origin={:?}", hash, import_headers.post().number(), is_new_best, origin);
operation.op.set_block_data(
import_headers.post().clone(),
leaf_state,
)?;
operation.op.insert_aux(aux)?;
if make_notifications {
if finalized {
operation.notify_finalized.push(hash);
operation.notify_imported = Some(ImportSummary {
hash,
origin,
header: import_headers.into_post(),
is_new_best,
storage_changes,
retracted,
})
Ok(ImportResult::imported(is_new_best))
fn block_execution(
&self,
transaction: &B::BlockImportOperation,
import_headers: &PrePostHeader<Block::Header>,
origin: BlockOrigin,
hash: Block::Hash,
body: &[Block::Extrinsic],
) -> error::Result<(
Option<StorageUpdate<B, Block>>,
Option<Option<ChangesUpdate<Block>>>,
Option<(
Vec<(Vec<u8>, Option<Vec<u8>>)>,
Vec<(Vec<u8>, Vec<(Vec<u8>, Option<Vec<u8>>)>)>
)>
where
E: CallExecutor<Block, Blake2Hasher> + Send + Sync + Clone,
{
match transaction.state()? {
Some(transaction_state) => {
let mut overlay = Default::default();
let get_execution_manager = |execution_strategy: ExecutionStrategy| {
match execution_strategy {
ExecutionStrategy::NativeElseWasm => ExecutionManager::NativeElseWasm,
ExecutionStrategy::AlwaysWasm => ExecutionManager::AlwaysWasm(BackendTrustLevel::Trusted),
ExecutionStrategy::NativeWhenPossible => ExecutionManager::NativeWhenPossible,
ExecutionStrategy::Both => ExecutionManager::Both(|wasm_result, native_result| {
let header = import_headers.post();
warn!("Consensus error between wasm and native block execution at block {}", hash);
warn!(" Header {:?}", header);
warn!(" Native result {:?}", native_result);
warn!(" Wasm result {:?}", wasm_result);
telemetry!(SUBSTRATE_INFO; "block.execute.consensus_failure";
"hash" => ?hash,
"origin" => ?origin,
"header" => ?header
);
wasm_result
}),
let encoded_block = <Block as BlockT>::encode_from(
import_headers.pre(),
);
let (_, storage_update, changes_update) = self.executor
.call_at_state::<_, _, NeverNativeValue, fn() -> _>(
transaction_state,
&mut overlay,
"Core_execute_block",
&encoded_block,
match origin {
BlockOrigin::NetworkInitialSync => get_execution_manager(
self.execution_strategies().syncing,
),
_ => get_execution_manager(self.execution_strategies().importing),
},
None,
None,
)?;
overlay.commit_prospective();
let (top, children) = overlay.into_committed();
let children = children.map(|(sk, it)| (sk, it.collect())).collect();
if import_headers.post().state_root() != &storage_update.1 {
return Err(error::Error::InvalidStateRoot);
}
Ok((Some(storage_update.0), Some(changes_update), Some((top.collect(), children))))
},
None => Ok((None, None, None))
}
}
fn apply_finality_with_block_hash(
operation: &mut ClientImportOperation<Block, Blake2Hasher, B>,
block: Block::Hash,
justification: Option<Justification>,
best_block: Block::Hash,
notify: bool,
) -> error::Result<()> {
// find tree route from last finalized to given block.
let last_finalized = self.backend.blockchain().last_finalized()?;
if block == last_finalized {
warn!("Possible safety violation: attempted to re-finalize last finalized block {:?} ", last_finalized);
return Ok(());
}
let route_from_finalized = header_metadata::tree_route(self.backend.blockchain(), last_finalized, block)?;
if let Some(retracted) = route_from_finalized.retracted().get(0) {
warn!("Safety violation: attempted to revert finalized block {:?} which is not in the \
same chain as last finalized {:?}", retracted, last_finalized);
return Err(error::Error::NotInFinalizedChain);
let route_from_best = header_metadata::tree_route(self.backend.blockchain(), best_block, block)?;
// if the block is not a direct ancestor of the current best chain,
// then some other block is the common ancestor.
if route_from_best.common_block().hash != block {
// NOTE: we're setting the finalized block as best block, this might
// be slightly inaccurate since we might have a "better" block
// further along this chain, but since best chain selection logic is
// pluggable we cannot make a better choice here. usages that need
// an accurate "best" block need to go through `SelectChain`
// instead.
operation.op.mark_head(BlockId::Hash(block))?;
let enacted = route_from_finalized.enacted();
assert!(enacted.len() > 0);
for finalize_new in &enacted[..enacted.len() - 1] {
operation.op.mark_finalized(BlockId::Hash(finalize_new.hash), None)?;
assert_eq!(enacted.last().map(|e| e.hash), Some(block));
operation.op.mark_finalized(BlockId::Hash(block), justification)?;
if notify {
// sometimes when syncing, tons of blocks can be finalized at once.
// we'll send notifications spuriously in that case.
const MAX_TO_NOTIFY: usize = 256;
let enacted = route_from_finalized.enacted();
let start = enacted.len() - ::std::cmp::min(enacted.len(), MAX_TO_NOTIFY);
for finalized in &enacted[start..] {
operation.notify_finalized.push(finalized.hash);
}
}
Ok(())
}
fn notify_finalized(
&self,
notify_finalized: Vec<Block::Hash>,
) -> error::Result<()> {
let mut sinks = self.finality_notification_sinks.lock();
for finalized_hash in notify_finalized {
let header = self.header(&BlockId::Hash(finalized_hash))?
.expect("header already known to exist in DB because it is indicated in the tree route; qed");
telemetry!(SUBSTRATE_INFO; "notify.finalized";
"height" => format!("{}", header.number()),
"best" => ?finalized_hash,
);
let notification = FinalityNotification {
header,
hash: finalized_hash,
};
sinks.retain(|sink| sink.unbounded_send(notification.clone()).is_ok());
}
Ok(())
}
fn notify_imported(&self, notify_import: ImportSummary<Block>) -> error::Result<()> {
if let Some(storage_changes) = notify_import.storage_changes {
// TODO [ToDr] How to handle re-orgs? Should we re-emit all storage changes?
self.storage_notifications.lock()
¬ify_import.hash,
storage_changes.0.into_iter(),
storage_changes.1.into_iter().map(|(sk, v)| (sk, v.into_iter())),
);
}
let notification = BlockImportNotification::<Block> {
hash: notify_import.hash,
origin: notify_import.origin,
header: notify_import.header,
is_new_best: notify_import.is_new_best,
retracted: notify_import.retracted,
};
self.import_notification_sinks.lock()
.retain(|sink| sink.unbounded_send(notification.clone()).is_ok());
Ok(())
}
/// Attempts to revert the chain by `n` blocks. Returns the number of blocks that were
/// successfully reverted.
pub fn revert(&self, n: NumberFor<Block>) -> error::Result<NumberFor<Block>> {
Ok(self.backend.revert(n)?)
}
pub fn info(&self) -> ClientInfo<Block> {
let info = self.backend.blockchain().info();
ClientInfo {
used_state_cache_size: self.backend.used_state_cache_size(),
pub fn block_status(&self, id: &BlockId<Block>) -> error::Result<BlockStatus> {
// this can probably be implemented more efficiently
if let BlockId::Hash(ref h) = id {
if self.importing_block.read().as_ref().map_or(false, |importing| h == importing) {
return Ok(BlockStatus::Queued);
}
}
let hash_and_number = match id.clone() {
BlockId::Hash(hash) => self.backend.blockchain().number(hash)?.map(|n| (hash, n)),
BlockId::Number(n) => self.backend.blockchain().hash(n)?.map(|hash| (hash, n)),
};
match hash_and_number {
Some((hash, number)) => {
if self.backend.have_state_at(&hash, number) {
Ok(BlockStatus::InChainWithState)
} else {
Ok(BlockStatus::InChainPruned)
}
}
None => Ok(BlockStatus::Unknown),
}
}
/// Get block header by id.
pub fn header(&self, id: &BlockId<Block>) -> error::Result<Option<<Block as BlockT>::Header>> {
self.backend.blockchain().header(*id)
}
/// Get block body by id.
pub fn body(&self, id: &BlockId<Block>) -> error::Result<Option<Vec<<Block as BlockT>::Extrinsic>>> {
self.backend.blockchain().body(*id)
}
/// Get block justification set by id.
pub fn justification(&self, id: &BlockId<Block>) -> error::Result<Option<Justification>> {
self.backend.blockchain().justification(*id)
}
pub fn block(&self, id: &BlockId<Block>)
-> error::Result<Option<SignedBlock<Block>>>
Ok(match (self.header(id)?, self.body(id)?, self.justification(id)?) {
(Some(header), Some(extrinsics), justification) =>
Some(SignedBlock { block: Block::new(header, extrinsics), justification }),
/// Gets the uncles of the block with `target_hash` going back `max_generation` ancestors.
pub fn uncles(&self, target_hash: Block::Hash, max_generation: NumberFor<Block>) -> error::Result<Vec<Block::Hash>> {
let load_header = |id: Block::Hash| -> error::Result<Block::Header> {
match self.backend.blockchain().header(BlockId::Hash(id))? {
Some(hdr) => Ok(hdr),
None => Err(Error::UnknownBlock(format!("{:?}", id))),
let genesis_hash = self.backend.blockchain().info().genesis_hash;
if genesis_hash == target_hash { return Ok(Vec::new()); }
let mut current_hash = target_hash;
let mut current = load_header(current_hash)?;
let mut ancestor_hash = *current.parent_hash();
let mut ancestor = load_header(ancestor_hash)?;
let mut uncles = Vec::new();
for _generation in 0..max_generation.saturated_into() {
let children = self.backend.blockchain().children(ancestor_hash)?;
uncles.extend(children.into_iter().filter(|h| h != ¤t_hash));
current_hash = ancestor_hash;
if genesis_hash == current_hash { break; }
current = ancestor;
ancestor_hash = *current.parent_hash();
ancestor = load_header(ancestor_hash)?;
}
trace!("Collected {} uncles", uncles.len());
fn changes_trie_config(&self) -> Result<Option<ChangesTrieConfiguration>, Error> {
Ok(self.backend.state_at(BlockId::Number(self.backend.blockchain().info().best_number))?
.storage(well_known_keys::CHANGES_TRIE_CONFIG)
.map_err(|e| error::Error::from_state(Box::new(e)))?
.and_then(|c| Decode::decode(&mut &*c).ok()))
/// Prepare in-memory header that is used in execution environment.
fn prepare_environment_block(&self, parent: &BlockId<Block>) -> error::Result<Block::Header> {
Arkadiy Paronyan
committed
let parent_header = self.backend.blockchain().expect_header(*parent)?;
Ok(<<Block as BlockT>::Header as HeaderT>::new(
self.backend.blockchain().expect_block_number_from_id(parent)? + One::one(),
Default::default(),
Default::default(),
Default::default(),
))
}
impl<B, E, Block, RA> HeaderMetadata<Block> for Client<B, E, Block, RA> where
B: backend::Backend<Block, Blake2Hasher>,
E: CallExecutor<Block, Blake2Hasher>,
Block: BlockT<Hash=H256>,
{
type Error = error::Error;
fn header_metadata(&self, hash: Block::Hash) -> Result<CachedHeaderMetadata<Block>, Self::Error> {
self.backend.blockchain().header_metadata(hash)
}
fn insert_header_metadata(&self, hash: Block::Hash, metadata: CachedHeaderMetadata<Block>) {
self.backend.blockchain().insert_header_metadata(hash, metadata)
}
fn remove_header_metadata(&self, hash: Block::Hash) {
self.backend.blockchain().remove_header_metadata(hash)
}
}
impl<B, E, Block, RA> ProvideUncles<Block> for Client<B, E, Block, RA> where
B: backend::Backend<Block, Blake2Hasher>,
E: CallExecutor<Block, Blake2Hasher>,
Block: BlockT<Hash=H256>,
{
fn uncles(&self, target_hash: Block::Hash, max_generation: NumberFor<Block>) -> error::Result<Vec<Block::Header>> {
Ok(Client::uncles(self, target_hash, max_generation)?
.into_iter()
.filter_map(|hash| Client::header(self, &BlockId::Hash(hash)).unwrap_or(None))
.collect()
)
}
}
impl<B, E, Block, RA> ChainHeaderBackend<Block> for Client<B, E, Block, RA> where
B: backend::Backend<Block, Blake2Hasher>,
E: CallExecutor<Block, Blake2Hasher> + Send + Sync,
Block: BlockT<Hash=H256>,
RA: Send + Sync,
{
fn header(&self, id: BlockId<Block>) -> error::Result<Option<Block::Header>> {
self.backend.blockchain().header(id)
}
fn info(&self) -> blockchain::Info<Block> {
self.backend.blockchain().info()
}
fn status(&self, id: BlockId<Block>) -> error::Result<blockchain::BlockStatus> {
self.backend.blockchain().status(id)
}
fn number(&self, hash: Block::Hash) -> error::Result<Option<<<Block as BlockT>::Header as HeaderT>::Number>> {
self.backend.blockchain().number(hash)
}
fn hash(&self, number: NumberFor<Block>) -> error::Result<Option<Block::Hash>> {
self.backend.blockchain().hash(number)
}
}
impl<B, E, Block, RA> sr_primitives::traits::BlockIdTo<Block> for Client<B, E, Block, RA> where
B: backend::Backend<Block, Blake2Hasher>,
E: CallExecutor<Block, Blake2Hasher> + Send + Sync,
Block: BlockT<Hash=H256>,
RA: Send + Sync,
{
type Error = Error;
fn to_hash(&self, block_id: &BlockId<Block>) -> error::Result<Option<Block::Hash>> {
self.block_hash_from_id(block_id)
}
fn to_number(&self, block_id: &BlockId<Block>) -> error::Result<Option<NumberFor<Block>>> {
self.block_number_from_id(block_id)
}
}
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
impl<B, E, Block, RA> ChainHeaderBackend<Block> for &Client<B, E, Block, RA> where
B: backend::Backend<Block, Blake2Hasher>,
E: CallExecutor<Block, Blake2Hasher> + Send + Sync,
Block: BlockT<Hash=H256>,
RA: Send + Sync,
{
fn header(&self, id: BlockId<Block>) -> error::Result<Option<Block::Header>> {
(**self).backend.blockchain().header(id)
}
fn info(&self) -> blockchain::Info<Block> {
(**self).backend.blockchain().info()
}
fn status(&self, id: BlockId<Block>) -> error::Result<blockchain::BlockStatus> {
(**self).status(id)
}
fn number(&self, hash: Block::Hash) -> error::Result<Option<<<Block as BlockT>::Header as HeaderT>::Number>> {
(**self).number(hash)
}
fn hash(&self, number: NumberFor<Block>) -> error::Result<Option<Block::Hash>> {
(**self).hash(number)
}
}
impl<B, E, Block, RA> ProvideCache<Block> for Client<B, E, Block, RA> where
B: backend::Backend<Block, Blake2Hasher>,
Block: BlockT<Hash=H256>,
{
fn cache(&self) -> Option<Arc<dyn Cache<Block>>> {
self.backend.blockchain().cache()
}
}
impl<B, E, Block, RA> ProvideRuntimeApi for Client<B, E, Block, RA> where
B: backend::Backend<Block, Blake2Hasher>,
E: CallExecutor<Block, Blake2Hasher> + Clone + Send + Sync,
RA: ConstructRuntimeApi<Block, Self>
type Api = <RA as ConstructRuntimeApi<Block, Self>>::RuntimeApi;
fn runtime_api<'a>(&'a self) -> ApiRef<'a, Self::Api> {
RA::construct_runtime_api(self)
}
}
impl<B, E, Block, RA> CallRuntimeAt<Block> for Client<B, E, Block, RA> where
B: backend::Backend<Block, Blake2Hasher>,
E: CallExecutor<Block, Blake2Hasher> + Clone + Send + Sync,
type Error = Error;
Bastian Köcher
committed
fn call_api_at<
Bastian Köcher
committed
NC: FnOnce() -> result::Result<R, String> + UnwindSafe,
C: CoreApi<Block, Error = Error>,
Bastian Köcher
committed
>(
&self,
at: &BlockId<Block>,
function: &'static str,
args: Vec<u8>,
changes: &RefCell<OverlayedChanges>,
initialize_block: InitializeBlock<'a, Block>,
native_call: Option<NC>,
recorder: &Option<Rc<RefCell<ProofRecorder<Block>>>>,
) -> error::Result<NativeOrEncoded<R>> {
let manager = match context {
ExecutionContext::BlockConstruction =>
self.execution_strategies.block_construction.get_manager(),
ExecutionContext::Syncing =>
self.execution_strategies.syncing.get_manager(),
ExecutionContext::Importing =>
self.execution_strategies.importing.get_manager(),
ExecutionContext::OffchainCall(Some((_, capabilities))) if capabilities.has_all() =>
self.execution_strategies.offchain_worker.get_manager(),
ExecutionContext::OffchainCall(_) =>
self.execution_strategies.other.get_manager(),
let capabilities = context.capabilities();
let offchain_extensions = if let ExecutionContext::OffchainCall(Some(ext)) = context {
Some(OffchainExt::new(offchain::LimitedExternalities::new(capabilities, ext.0)))
} else {
None
};
self.executor.contextual_call::<_, fn(_,_) -> _,_,_>(
|| core_api.initialize_block(at, &self.prepare_environment_block(at)?),
at,
function,
&args,
changes,
capabilities.has(offchain::Capability::Keystore),
fn runtime_version_at(&self, at: &BlockId<Block>) -> error::Result<RuntimeVersion> {
self.runtime_version_at(at)
}
}
/// NOTE: only use this implementation when you are sure there are NO consensus-level BlockImport
/// objects. Otherwise, importing blocks directly into the client would be bypassing
/// important verification work.
impl<'a, B, E, Block, RA> consensus::BlockImport<Block> for &'a Client<B, E, Block, RA> where
B: backend::Backend<Block, Blake2Hasher>,
E: CallExecutor<Block, Blake2Hasher> + Clone + Send + Sync,
Block: BlockT<Hash=H256>,
type Error = ConsensusError;
/// Import a checked and validated block. If a justification is provided in
/// `BlockImportParams` then `finalized` *must* be true.
///
/// NOTE: only use this implementation when there are NO consensus-level BlockImport
/// objects. Otherwise, importing blocks directly into the client would be bypassing
/// important verification work.
///
/// If you are not sure that there are no BlockImport objects provided by the consensus
/// algorithm, don't use this function.
&mut self,
import_block: BlockImportParams<Block>,
new_cache: HashMap<CacheKeyId, Vec<u8>>,
) -> Result<ImportResult, Self::Error> {
self.lock_import_and_run(|operation| {
self.apply_block(operation, import_block, new_cache)
}).map_err(|e| {
warn!("Block import error:\n{:?}", e);
ConsensusError::ClientImport(e.to_string()).into()
})
/// Check block preconditions.
fn check_block(
&mut self,
) -> Result<ImportResult, Self::Error> {
let BlockCheckParams { hash, number, parent_hash, allow_missing_state } = block;
if let Some(h) = self.fork_blocks.as_ref().and_then(|x| x.get(&number)) {
if &hash != h {
trace!(
"Rejecting block from known invalid fork. Got {:?}, expected: {:?} at height {}",
hash,
h,
number
);
return Ok(ImportResult::KnownBad);
}
}
match self.block_status(&BlockId::Hash(parent_hash))
.map_err(|e| ConsensusError::ClientImport(e.to_string()))?
BlockStatus::InChainWithState | BlockStatus::Queued => {},
BlockStatus::Unknown => return Ok(ImportResult::UnknownParent),
BlockStatus::InChainPruned if allow_missing_state => {},
BlockStatus::InChainPruned => return Ok(ImportResult::MissingState),
BlockStatus::KnownBad => return Ok(ImportResult::KnownBad),
match self.block_status(&BlockId::Hash(hash))
.map_err(|e| ConsensusError::ClientImport(e.to_string()))?
BlockStatus::InChainWithState | BlockStatus::Queued => return Ok(ImportResult::AlreadyInChain),
BlockStatus::Unknown | BlockStatus::InChainPruned => {},
BlockStatus::KnownBad => return Ok(ImportResult::KnownBad),
Ok(ImportResult::imported(false))
impl<B, E, Block, RA> consensus::BlockImport<Block> for Client<B, E, Block, RA> where
B: backend::Backend<Block, Blake2Hasher>,
E: CallExecutor<Block, Blake2Hasher> + Clone + Send + Sync,
Block: BlockT<Hash=H256>,
{
type Error = ConsensusError;
fn import_block(
&mut self,
import_block: BlockImportParams<Block>,
new_cache: HashMap<CacheKeyId, Vec<u8>>,
) -> Result<ImportResult, Self::Error> {
(&*self).import_block(import_block, new_cache)
}
fn check_block(
&mut self,
) -> Result<ImportResult, Self::Error> {
}
}
impl<B, E, Block, RA> Finalizer<Block, Blake2Hasher, B> for Client<B, E, Block, RA> where
1636
1637
1638
1639
1640
1641
1642
1643
1644
1645
1646
1647
1648
1649
1650
1651
1652
1653
1654
1655
1656
1657
1658
B: backend::Backend<Block, Blake2Hasher>,
E: CallExecutor<Block, Blake2Hasher>,
Block: BlockT<Hash=H256>,
{
fn apply_finality(
&self,
operation: &mut ClientImportOperation<Block, Blake2Hasher, B>,
id: BlockId<Block>,
justification: Option<Justification>,
notify: bool,
) -> error::Result<()> {
let last_best = self.backend.blockchain().info().best_hash;
let to_finalize_hash = self.backend.blockchain().expect_block_hash_from_id(&id)?;
self.apply_finality_with_block_hash(operation, to_finalize_hash, justification, last_best, notify)
}
fn finalize_block(&self, id: BlockId<Block>, justification: Option<Justification>, notify: bool) -> error::Result<()> {
self.lock_import_and_run(|operation| {
self.apply_finality(operation, id, justification, notify)
})
}
}
impl<B, E, Block, RA> Finalizer<Block, Blake2Hasher, B> for &Client<B, E, Block, RA> where
B: backend::Backend<Block, Blake2Hasher>,
E: CallExecutor<Block, Blake2Hasher>,
Block: BlockT<Hash=H256>,
{
fn apply_finality(
&self,
operation: &mut ClientImportOperation<Block, Blake2Hasher, B>,
id: BlockId<Block>,
justification: Option<Justification>,
notify: bool,
) -> error::Result<()> {
(**self).apply_finality(operation, id, justification, notify)
}
fn finalize_block(&self, id: BlockId<Block>, justification: Option<Justification>, notify: bool) -> error::Result<()> {
(**self).finalize_block(id, justification, notify)
}
}
impl<B, E, Block, RA> BlockchainEvents<Block> for Client<B, E, Block, RA>
E: CallExecutor<Block, Blake2Hasher>,
fn import_notification_stream(&self) -> ImportNotifications<Block> {
let (sink, stream) = mpsc::unbounded();
self.import_notification_sinks.lock().push(sink);
stream
fn finality_notification_stream(&self) -> FinalityNotifications<Block> {
let (sink, stream) = mpsc::unbounded();
self.finality_notification_sinks.lock().push(sink);
stream
}
/// Get storage changes event stream.
fn storage_changes_notification_stream(
&self,
filter_keys: Option<&[StorageKey]>,
child_filter_keys: Option<&[(StorageKey, Option<Vec<StorageKey>>)]>,
) -> error::Result<StorageEventStream<Block::Hash>> {
Ok(self.storage_notifications.lock().listen(filter_keys, child_filter_keys))
/// Implement Longest Chain Select implementation
/// where 'longest' is defined as the highest number of blocks
pub struct LongestChain<B, Block> {
backend: Arc<B>,
_phantom: PhantomData<Block>
}
impl<B, Block> Clone for LongestChain<B, Block> {
fn clone(&self) -> Self {
let backend = self.backend.clone();
LongestChain {
backend,
_phantom: Default::default()
}
}
}
impl<B, Block> LongestChain<B, Block>
B: backend::Backend<Block, Blake2Hasher>,
/// Instantiate a new LongestChain for Backend B
pub fn new(backend: Arc<B>) -> Self {
LongestChain {
backend,
_phantom: Default::default()
}
}
fn best_block_header(&self) -> error::Result<<Block as BlockT>::Header> {
let info = self.backend.blockchain().info();
let import_lock = self.backend.get_import_lock();
let best_hash = self.backend.blockchain().best_containing(info.best_hash, None, import_lock)?
.unwrap_or(info.best_hash);
Ok(self.backend.blockchain().header(BlockId::Hash(best_hash))?
.expect("given block hash was fetched from block in db; qed"))
fn leaves(&self) -> Result<Vec<<Block as BlockT>::Hash>, error::Error> {
self.backend.blockchain().leaves()
}
impl<B, Block> SelectChain<Block> for LongestChain<B, Block>
where
B: backend::Backend<Block, Blake2Hasher>,
Block: BlockT<Hash=H256>,
{
fn leaves(&self) -> Result<Vec<<Block as BlockT>::Hash>, ConsensusError> {
LongestChain::leaves(self)
.map_err(|e| ConsensusError::ChainLookup(e.to_string()).into())
}
fn best_chain(&self)
-> Result<<Block as BlockT>::Header, ConsensusError>
{
LongestChain::best_block_header(&self)
.map_err(|e| ConsensusError::ChainLookup(e.to_string()).into())
}
fn finality_target(
&self,
target_hash: Block::Hash,
maybe_max_number: Option<NumberFor<Block>>
) -> Result<Option<Block::Hash>, ConsensusError> {
let import_lock = self.backend.get_import_lock();
self.backend.blockchain().best_containing(target_hash, maybe_max_number, import_lock)
.map_err(|e| ConsensusError::ChainLookup(e.to_string()).into())
}
}
impl<B, E, Block, RA> BlockBody<Block> for Client<B, E, Block, RA>
where
B: backend::Backend<Block, Blake2Hasher>,
E: CallExecutor<Block, Blake2Hasher>,
Block: BlockT<Hash=H256>,
{
fn block_body(&self, id: &BlockId<Block>) -> error::Result<Option<Vec<<Block as BlockT>::Extrinsic>>> {
self.body(id)
}
}
impl<B, E, Block, RA> backend::AuxStore for Client<B, E, Block, RA>
where
B: backend::Backend<Block, Blake2Hasher>,
E: CallExecutor<Block, Blake2Hasher>,
Block: BlockT<Hash=H256>,
{
/// Insert auxiliary data into key-value store.
fn insert_aux<
'a,
'b: 'a,
'c: 'a,
I: IntoIterator<Item=&'a(&'c [u8], &'c [u8])>,
D: IntoIterator<Item=&'a &'b [u8]>,
>(&self, insert: I, delete: D) -> error::Result<()> {
// Import is locked here because we may have other block import
// operations that tries to set aux data. Note that for consensus
// layer, one can always use atomic operations to make sure
// import is only locked once.
self.lock_import_and_run(|operation| {
apply_aux(operation, insert, delete)
}
/// Query auxiliary data from key-value store.
fn get_aux(&self, key: &[u8]) -> error::Result<Option<Vec<u8>>> {
crate::backend::AuxStore::get_aux(&*self.backend, key)
impl<B, E, Block, RA> backend::AuxStore for &Client<B, E, Block, RA>
where
B: backend::Backend<Block, Blake2Hasher>,
E: CallExecutor<Block, Blake2Hasher>,
Block: BlockT<Hash=H256>,
1827
1828
1829
1830
1831
1832
1833
1834
1835
1836
1837
1838
1839
1840
1841
1842
1843
1844
1845
1846
1847
1848
1849
1850
1851
1852
1853
1854
1855
1856
1857
1858
1859
1860
1861
1862
fn insert_aux<
'a,
'b: 'a,
'c: 'a,
I: IntoIterator<Item=&'a(&'c [u8], &'c [u8])>,
D: IntoIterator<Item=&'a &'b [u8]>,
>(&self, insert: I, delete: D) -> error::Result<()> {
(**self).insert_aux(insert, delete)
}
fn get_aux(&self, key: &[u8]) -> error::Result<Option<Vec<u8>>> {
(**self).get_aux(key)
}
}
/// Helper function to apply auxiliary data insertion into an operation.
pub fn apply_aux<'a, 'b: 'a, 'c: 'a, B, Block, H, D, I>(
operation: &mut ClientImportOperation<Block, H, B>,
insert: I,
delete: D
) -> error::Result<()>
where
Block: BlockT,
H: Hasher<Out=Block::Hash>,
B: backend::Backend<Block, H>,
I: IntoIterator<Item=&'a(&'c [u8], &'c [u8])>,
D: IntoIterator<Item=&'a &'b [u8]>,
{
operation.op.insert_aux(
insert.into_iter()
.map(|(k, v)| (k.to_vec(), Some(v.to_vec())))
.chain(delete.into_iter().map(|k| (k.to_vec(), None)))
)
}
/// Utility methods for the client.
pub mod utils {
use super::*;
use std::borrow::Borrow;
/// Returns a function for checking block ancestry, the returned function will
/// return `true` if the given hash (second parameter) is a descendent of the
/// base (first parameter). If the `current` parameter is defined, it should
/// represent the current block `hash` and its `parent hash`, if given the
/// function that's returned will assume that `hash` isn't part of the local DB
/// yet, and all searches in the DB will instead reference the parent.
pub fn is_descendent_of<'a, Block: BlockT<Hash=H256>, T, H: Borrow<H256> + 'a>(
client: &'a T,
current: Option<(H, H)>,
) -> impl Fn(&H256, &H256) -> Result<bool, error::Error> + 'a
where T: ChainHeaderBackend<Block> + HeaderMetadata<Block, Error=error::Error>,
{
move |base, hash| {
if base == hash { return Ok(false); }
let current = current.as_ref().map(|(c, p)| (c.borrow(), p.borrow()));
let mut hash = hash;
if let Some((current_hash, current_parent_hash)) = current {
if base == current_hash { return Ok(false); }
if hash == current_hash {
if base == current_parent_hash {
return Ok(true);
} else {
hash = current_parent_hash;
}
}
}
let ancestor = header_metadata::lowest_common_ancestor(client, *hash, *base)?;
impl<BE, E, B, RA> consensus::block_validation::Chain<B> for Client<BE, E, B, RA>
where BE: backend::Backend<B, Blake2Hasher>,
E: CallExecutor<B, Blake2Hasher>,
B: BlockT<Hash = H256>
{
fn block_status(&self, id: &BlockId<B>) -> Result<BlockStatus, Box<dyn std::error::Error + Send>> {
Client::block_status(self, id).map_err(|e| Box::new(e) as Box<_>)
}
}
pub(crate) mod tests {
use std::collections::HashMap;
use primitives::blake2_256;
use consensus::{BlockOrigin, SelectChain};
use test_client::{
client_db::{Backend, DatabaseSettings, DatabaseSettingsSrc, PruningMode},
runtime::{self, Block, Transfer, RuntimeApi, TestAPI},
/// Returns tuple, consisting of:
/// 1) test client pre-filled with blocks changing balances;
/// 2) roots of changes tries for these blocks
/// 3) test cases in form (begin, end, key, vec![(block, extrinsic)]) that are required to pass
pub fn prepare_client_with_key_changes() -> (
Bastian Köcher
committed
test_client::client::Client<test_client::Backend, test_client::Executor, Block, RuntimeApi>,
Vec<H256>,
Vec<(u64, u64, Vec<u8>, Vec<(u64, u32)>)>,
) {
// prepare block structure
let blocks_transfers = vec![
vec![(AccountKeyring::Alice, AccountKeyring::Dave), (AccountKeyring::Bob, AccountKeyring::Dave)],
vec![(AccountKeyring::Charlie, AccountKeyring::Eve)],
vec![(AccountKeyring::Alice, AccountKeyring::Dave)],
];
// prepare client ang import blocks
let mut local_roots = Vec::new();
let remote_client = TestClientBuilder::new().set_support_changes_trie(true).build();
let mut nonces: HashMap<_, u64> = Default::default();
for (i, block_transfers) in blocks_transfers.into_iter().enumerate() {
let mut builder = remote_client.new_block(Default::default()).unwrap();
for (from, to) in block_transfers {
builder.push_transfer(Transfer {
from: from.into(),
to: to.into(),
amount: 1,
nonce: *nonces.entry(from).and_modify(|n| { *n = *n + 1 }).or_default(),
}).unwrap();
}
remote_client.import(BlockOrigin::Own, builder.bake().unwrap()).unwrap();
let header = remote_client.header(&BlockId::Number(i as u64 + 1)).unwrap().unwrap();
let trie_root = header.digest().log(DigestItem::as_changes_trie_root)
.map(|root| H256::from_slice(root.as_ref()))
.unwrap();
local_roots.push(trie_root);
}
// prepare test cases
let alice = blake2_256(&runtime::system::balance_of_key(AccountKeyring::Alice.into())).to_vec();
let bob = blake2_256(&runtime::system::balance_of_key(AccountKeyring::Bob.into())).to_vec();
let charlie = blake2_256(&runtime::system::balance_of_key(AccountKeyring::Charlie.into())).to_vec();
let dave = blake2_256(&runtime::system::balance_of_key(AccountKeyring::Dave.into())).to_vec();
let eve = blake2_256(&runtime::system::balance_of_key(AccountKeyring::Eve.into())).to_vec();
let ferdie = blake2_256(&runtime::system::balance_of_key(AccountKeyring::Ferdie.into())).to_vec();
1976
1977
1978
1979
1980
1981
1982
1983
1984
1985
1986
1987
1988
1989
1990
1991
1992
1993
1994
1995
1996
1997
1998
1999
2000
let test_cases = vec![
(1, 4, alice.clone(), vec![(4, 0), (1, 0)]),
(1, 3, alice.clone(), vec![(1, 0)]),
(2, 4, alice.clone(), vec![(4, 0)]),
(2, 3, alice.clone(), vec![]),
(1, 4, bob.clone(), vec![(1, 1)]),
(1, 1, bob.clone(), vec![(1, 1)]),
(2, 4, bob.clone(), vec![]),
(1, 4, charlie.clone(), vec![(2, 0)]),
(1, 4, dave.clone(), vec![(4, 0), (1, 1), (1, 0)]),
(1, 1, dave.clone(), vec![(1, 1), (1, 0)]),
(3, 4, dave.clone(), vec![(4, 0)]),
(1, 4, eve.clone(), vec![(2, 0)]),
(1, 1, eve.clone(), vec![]),
(3, 4, eve.clone(), vec![]),
(1, 4, ferdie.clone(), vec![]),
];
(remote_client, local_roots, test_cases)
}