Newer
Older
overlay.commit_prospective();
Ok((Some(storage_update), Some(changes_update), Some(overlay.into_committed().collect())))
},
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 = crate::blockchain::tree_route(
self.backend.blockchain(),
BlockId::Hash(last_finalized),
BlockId::Hash(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 = crate::blockchain::tree_route(
self.backend.blockchain(),
BlockId::Hash(best_block),
BlockId::Hash(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 {
// FIXME: #1442 reorganize best block to be the best chain containing
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,
);
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
let notification = FinalityNotification {
header,
hash: finalized_hash,
};
sinks.retain(|sink| sink.unbounded_send(notification.clone()).is_ok());
}
Ok(())
}
fn notify_imported(
&self,
notify_import: (Block::Hash, BlockOrigin, Block::Header, bool, Option<Vec<(Vec<u8>, Option<Vec<u8>>)>>),
) -> error::Result<()> {
let (hash, origin, header, is_new_best, storage_changes) = notify_import;
if let Some(storage_changes) = storage_changes {
// TODO [ToDr] How to handle re-orgs? Should we re-emit all storage changes?
self.storage_notifications.lock()
.trigger(&hash, storage_changes.into_iter());
}
let notification = BlockImportNotification::<Block> {
hash,
origin,
header,
is_new_best,
};
self.import_notification_sinks.lock()
.retain(|sink| sink.unbounded_send(notification.clone()).is_ok());
Ok(())
}
/// Apply auxiliary data insertion into an operation.
pub fn apply_aux<
'a,
'b: 'a,
'c: 'a,
I: IntoIterator<Item=&'a(&'c [u8], &'c [u8])>,
D: IntoIterator<Item=&'a &'b [u8]>,
>(
&self,
operation: &mut ClientImportOperation<Block, Blake2Hasher, B>,
insert: I,
delete: D
) -> error::Result<()> {
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)))
)
}
/// Mark all blocks up to given as finalized in operation. If a
/// justification is provided it is stored with the given finalized
/// block (any other finalized blocks are left unjustified).
pub 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)
}
/// Finalize a block. This will implicitly finalize all blocks up to it and
/// fire finality notifications.
///
/// Pass a flag to indicate whether finality notifications should be propagated.
/// This is usually tied to some synchronization state, where we don't send notifications
/// while performing major synchronization work.
pub fn finalize_block(&self, id: BlockId<Block>, justification: Option<Justification>, notify: bool) -> error::Result<()> {
self.lock_import_and_run(|operation| {
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)
})
/// 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 {
chain: info,
best_queued_hash: None,
best_queued_number: None,
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!("Unknown block {:?}", 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)?;
}
Ok(uncles)
}
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)))
}
/// 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> 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> 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,
Bastian Köcher
committed
fn call_api_at<
R: Encode + Decode + PartialEq,
NC: FnOnce() -> result::Result<R, &'static str> + UnwindSafe,
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::OffchainWorker(_) =>
self.execution_strategies.offchain_worker.get_manager(),
ExecutionContext::Other =>
self.execution_strategies.other.get_manager(),
let mut offchain_extensions = match context {
ExecutionContext::OffchainWorker(ext) => Some(ext),
_ => None,
};
self.executor.contextual_call::<_, _, fn(_,_) -> _,_,_>(
|| core_api.initialize_block(at, &self.prepare_environment_block(at)?),
at,
function,
&args,
changes,
fn runtime_version_at(&self, at: &BlockId<Block>) -> error::Result<RuntimeVersion> {
self.runtime_version_at(at)
}
}
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;
/// Import a checked and validated block. If a justification is provided in
/// `ImportBlock` then `finalized` *must* be true.
fn import_block(
&self,
import_block: ImportBlock<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| ConsensusError::ClientImport(e.to_string()).into())
/// Check block preconditions.
fn check_block(
&self,
hash: Block::Hash,
parent_hash: Block::Hash,
) -> Result<ImportResult, Self::Error> {
match self.block_status(&BlockId::Hash(parent_hash))
.map_err(|e| ConsensusError::ClientImport(e.to_string()))?
BlockStatus::InChainWithState | BlockStatus::Queued => {},
BlockStatus::Unknown | BlockStatus::InChainPruned => return Ok(ImportResult::UnknownParent),
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())
impl<B, E, Block, RA> CurrentHeight for Client<B, E, Block, RA> where
B: backend::Backend<Block, Blake2Hasher>,
E: CallExecutor<Block, Blake2Hasher>,
{
type BlockNumber = <Block::Header as HeaderT>::Number;
fn current_height(&self) -> Self::BlockNumber {
self.backend.blockchain().info().best_number
impl<B, E, Block, RA> BlockNumberToHash for Client<B, E, Block, RA> where
B: backend::Backend<Block, Blake2Hasher>,
E: CallExecutor<Block, Blake2Hasher>,
{
type BlockNumber = <Block::Header as HeaderT>::Number;
type Hash = Block::Hash;
fn block_number_to_hash(&self, n: Self::BlockNumber) -> Option<Self::Hash> {
self.block_hash(n).unwrap_or(None)
}
}
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]>) -> error::Result<StorageEventStream<Block::Hash>> {
Ok(self.storage_notifications.lock().listen(filter_keys))
}
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
/// Implement Longest Chain Select implementation
/// where 'longest' is defined as the highest number of blocks
pub struct LongestChain<B, Block> {
backend: Arc<B>,
import_lock: Arc<Mutex<()>>,
_phantom: PhantomData<Block>
}
impl<B, Block> Clone for LongestChain<B, Block> {
fn clone(&self) -> Self {
let backend = self.backend.clone();
let import_lock = self.import_lock.clone();
LongestChain {
backend,
import_lock,
_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>, import_lock: Arc<Mutex<()>>) -> Self {
LongestChain {
backend,
import_lock,
_phantom: Default::default()
}
}
fn best_block_header(&self) -> error::Result<<Block as BlockT>::Header> {
let info : ChainInfo<Block> = self.backend.blockchain().info();
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547
1548
1549
1550
1551
1552
1553
1554
1555
1556
1557
1558
1559
1560
1561
1562
1563
1564
1565
1566
1567
1568
Ok(self.backend.blockchain().header(BlockId::Hash(info.best_hash))?
.expect("Best block header must always exist"))
}
/// Get the most recent block hash of the best (longest) chains
/// that contain block with the given `target_hash`.
///
/// The search space is always limited to blocks which are in the finalized
/// chain or descendents of it.
///
/// If `maybe_max_block_number` is `Some(max_block_number)`
/// the search is limited to block `numbers <= max_block_number`.
/// in other words as if there were no blocks greater `max_block_number`.
/// Returns `Ok(None)` if `target_hash` is not found in search space.
/// TODO: document time complexity of this, see [#1444](https://github.com/paritytech/substrate/issues/1444)
fn best_containing(
&self,
target_hash: Block::Hash,
maybe_max_number: Option<NumberFor<Block>>
) -> error::Result<Option<Block::Hash>> {
let target_header = {
match self.backend.blockchain().header(BlockId::Hash(target_hash))? {
Some(x) => x,
// target not in blockchain
None => { return Ok(None); },
}
};
if let Some(max_number) = maybe_max_number {
// target outside search range
if target_header.number() > &max_number {
return Ok(None);
}
}
let (leaves, best_already_checked) = {
// ensure no blocks are imported during this code block.
// an import could trigger a reorg which could change the canonical chain.
// we depend on the canonical chain staying the same during this code block.
let _import_lock = self.import_lock.lock();
let info = self.backend.blockchain().info();
1570
1571
1572
1573
1574
1575
1576
1577
1578
1579
1580
1581
1582
1583
1584
1585
1586
1587
1588
1589
1590
1591
1592
1593
1594
1595
1596
1597
1598
1599
1600
1601
1602
1603
1604
1605
1606
1607
1608
1609
1610
1611
1612
1613
1614
1615
1616
1617
1618
1619
1620
1621
1622
1623
1624
1625
1626
1627
1628
1629
1630
1631
1632
1633
1634
1635
1636
1637
1638
1639
1640
1641
1642
1643
1644
1645
1646
1647
1648
1649
1650
let canon_hash = self.backend.blockchain().hash(*target_header.number())?
.ok_or_else(|| error::Error::from(format!("failed to get hash for block number {}", target_header.number())))?;
if canon_hash == target_hash {
// if no block at the given max depth exists fallback to the best block
if let Some(max_number) = maybe_max_number {
if let Some(header) = self.backend.blockchain().hash(max_number)? {
return Ok(Some(header));
}
}
return Ok(Some(info.best_hash));
} else if info.finalized_number >= *target_header.number() {
// header is on a dead fork.
return Ok(None);
}
(self.backend.blockchain().leaves()?, info.best_hash)
};
// for each chain. longest chain first. shortest last
for leaf_hash in leaves {
// ignore canonical chain which we already checked above
if leaf_hash == best_already_checked {
continue;
}
// start at the leaf
let mut current_hash = leaf_hash;
// if search is not restricted then the leaf is the best
let mut best_hash = leaf_hash;
// go backwards entering the search space
// waiting until we are <= max_number
if let Some(max_number) = maybe_max_number {
loop {
let current_header = self.backend.blockchain().header(BlockId::Hash(current_hash.clone()))?
.ok_or_else(|| error::Error::from(format!("failed to get header for hash {}", current_hash)))?;
if current_header.number() <= &max_number {
best_hash = current_header.hash();
break;
}
current_hash = *current_header.parent_hash();
}
}
// go backwards through the chain (via parent links)
loop {
// until we find target
if current_hash == target_hash {
return Ok(Some(best_hash));
}
let current_header = self.backend.blockchain().header(BlockId::Hash(current_hash.clone()))?
.ok_or_else(|| error::Error::from(format!("failed to get header for hash {}", current_hash)))?;
// stop search in this chain once we go below the target's block number
if current_header.number() < target_header.number() {
break;
}
current_hash = *current_header.parent_hash();
}
}
// header may be on a dead fork -- the only leaves that are considered are
// those which can still be finalized.
//
// FIXME #1558 only issue this warning when not on a dead fork
warn!(
"Block {:?} exists in chain but not found when following all \
leaves backwards. Number limit = {:?}",
target_hash,
maybe_max_number,
);
Ok(None)
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> {
LongestChain::best_containing(self, target_hash, maybe_max_number)
.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| {
self.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)
pub(crate) mod tests {
use std::collections::HashMap;
use primitives::blake2_256;
use runtime_primitives::traits::DigestItem as DigestItemT;
use runtime_primitives::generic::DigestItem;
use consensus::{BlockOrigin, SelectChain};
use test_client::{
TestClient, AccountKeyring, client::backend::Backend as TestBackend, TestClientBuilder,
BlockBuilderExt, 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();
1784
1785
1786
1787
1788
1789
1790
1791
1792
1793
1794
1795
1796
1797
1798
1799
1800
1801
1802
1803
1804
1805
1806
1807
1808
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)
}
fn client_initializes_from_genesis_ok() {
let client = test_client::new();
assert_eq!(
client.runtime_api().balance_of(
&BlockId::Number(client.info().chain.best_number),
AccountKeyring::Alice.into()
).unwrap(),
1000
);
assert_eq!(
client.runtime_api().balance_of(
&BlockId::Number(client.info().chain.best_number),
AccountKeyring::Ferdie.into()
).unwrap(),
0
);
}
#[test]
fn block_builder_works_with_no_transactions() {
let client = test_client::new();
let builder = client.new_block(Default::default()).unwrap();
client.import(BlockOrigin::Own, builder.bake().unwrap()).unwrap();
assert_eq!(client.info().chain.best_number, 1);
}
#[test]
fn block_builder_works_with_transactions() {
let client = test_client::new();
let mut builder = client.new_block(Default::default()).unwrap();
builder.push_transfer(Transfer {
from: AccountKeyring::Alice.into(),
to: AccountKeyring::Ferdie.into(),
client.import(BlockOrigin::Own, builder.bake().unwrap()).unwrap();
assert_eq!(client.info().chain.best_number, 1);
assert_ne!(
client.state_at(&BlockId::Number(1)).unwrap().pairs(),
client.state_at(&BlockId::Number(0)).unwrap().pairs()
);
assert_eq!(
client.runtime_api().balance_of(
&BlockId::Number(client.info().chain.best_number),
AccountKeyring::Alice.into()
).unwrap(),
958
);
assert_eq!(
client.runtime_api().balance_of(
&BlockId::Number(client.info().chain.best_number),
AccountKeyring::Ferdie.into()
).unwrap(),
42
);
#[test]
fn block_builder_does_not_include_invalid() {
let client = test_client::new();
let mut builder = client.new_block(Default::default()).unwrap();
builder.push_transfer(Transfer {
from: AccountKeyring::Alice.into(),
to: AccountKeyring::Ferdie.into(),
assert!(builder.push_transfer(Transfer {
from: AccountKeyring::Eve.into(),
to: AccountKeyring::Alice.into(),
client.import(BlockOrigin::Own, builder.bake().unwrap()).unwrap();
assert_eq!(client.info().chain.best_number, 1);
assert_ne!(
client.state_at(&BlockId::Number(1)).unwrap().pairs(),
client.state_at(&BlockId::Number(0)).unwrap().pairs()
);
assert_eq!(client.body(&BlockId::Number(1)).unwrap().unwrap().len(), 1)
}
#[test]
fn best_containing_with_genesis_block() {
// block tree:
// G
let (client, longest_chain_select) = TestClientBuilder::new().build_with_longest_chain();
let genesis_hash = client.info().chain.genesis_hash;
assert_eq!(
genesis_hash.clone(),
longest_chain_select.finality_target(genesis_hash.clone(), None).unwrap().unwrap()
);
}
#[test]
fn best_containing_with_hash_not_found() {
// block tree:
// G
let (client, longest_chain_select) = TestClientBuilder::new().build_with_longest_chain();
let uninserted_block = client.new_block(Default::default()).unwrap().bake().unwrap();
assert_eq!(
None,
longest_chain_select.finality_target(uninserted_block.hash().clone(), None).unwrap()
);
#[test]
fn uncles_with_only_ancestors() {
// block tree:
// G -> A1 -> A2
let client = test_client::new();
// G -> A1
let a1 = client.new_block(Default::default()).unwrap().bake().unwrap();
client.import(BlockOrigin::Own, a1.clone()).unwrap();
// A1 -> A2
let a2 = client.new_block(Default::default()).unwrap().bake().unwrap();
client.import(BlockOrigin::Own, a2.clone()).unwrap();
let v: Vec<H256> = Vec::new();
assert_eq!(v, client.uncles(a2.hash(), 3).unwrap());
}
#[test]
fn uncles_with_multiple_forks() {
// block tree:
// G -> A1 -> A2 -> A3 -> A4 -> A5
// A1 -> B2 -> B3 -> B4
// B2 -> C3
// A1 -> D2
let client = test_client::new();
// G -> A1
let a1 = client.new_block(Default::default()).unwrap().bake().unwrap();
client.import(BlockOrigin::Own, a1.clone()).unwrap();
// A1 -> A2
let a2 = client.new_block_at(&BlockId::Hash(a1.hash()), Default::default()).unwrap().bake().unwrap();
client.import(BlockOrigin::Own, a2.clone()).unwrap();
// A2 -> A3
let a3 = client.new_block_at(&BlockId::Hash(a2.hash()), Default::default()).unwrap().bake().unwrap();
client.import(BlockOrigin::Own, a3.clone()).unwrap();
// A3 -> A4
let a4 = client.new_block_at(&BlockId::Hash(a3.hash()), Default::default()).unwrap().bake().unwrap();
client.import(BlockOrigin::Own, a4.clone()).unwrap();
// A4 -> A5
let a5 = client.new_block_at(&BlockId::Hash(a4.hash()), Default::default()).unwrap().bake().unwrap();
client.import(BlockOrigin::Own, a5.clone()).unwrap();
// A1 -> B2
let mut builder = client.new_block_at(&BlockId::Hash(a1.hash()), Default::default()).unwrap();
// this push is required as otherwise B2 has the same hash as A2 and won't get imported
builder.push_transfer(Transfer {
from: AccountKeyring::Alice.into(),
to: AccountKeyring::Ferdie.into(),
amount: 41,
nonce: 0,
}).unwrap();
let b2 = builder.bake().unwrap();
client.import(BlockOrigin::Own, b2.clone()).unwrap();
// B2 -> B3
let b3 = client.new_block_at(&BlockId::Hash(b2.hash()), Default::default()).unwrap().bake().unwrap();
client.import(BlockOrigin::Own, b3.clone()).unwrap();
// B3 -> B4
let b4 = client.new_block_at(&BlockId::Hash(b3.hash()), Default::default()).unwrap().bake().unwrap();