chain: rename BlockHeaderHash to block::Hash

This is the first in a sequence of changes that change the block:: items
to not include Block as a prefix in their name, in accordance with the
Rust API guidelines.
This commit is contained in:
Henry de Valence 2020-08-15 23:20:01 -07:00
parent 04ba696515
commit 61dea90e2f
30 changed files with 216 additions and 221 deletions

View File

@ -11,7 +11,7 @@ pub mod merkle;
#[cfg(test)]
mod tests;
pub use hash::BlockHeaderHash;
pub use hash::Hash;
pub use header::BlockHeader;
pub use height::BlockHeight;
pub use root_hash::RootHash;
@ -80,8 +80,8 @@ impl Block {
}
/// Get the hash for the current block
pub fn hash(&self) -> BlockHeaderHash {
BlockHeaderHash::from(self)
pub fn hash(&self) -> Hash {
Hash::from(self)
}
/// Get the parsed root hash for this block.
@ -102,8 +102,8 @@ impl Block {
}
}
impl<'a> From<&'a Block> for BlockHeaderHash {
fn from(block: &'a Block) -> BlockHeaderHash {
impl<'a> From<&'a Block> for Hash {
fn from(block: &'a Block) -> Hash {
(&block.header).into()
}
}

View File

@ -24,9 +24,9 @@ use super::BlockHeader;
/// more explicit.
#[derive(Copy, Clone, Eq, PartialEq, Hash, Serialize, Deserialize)]
#[cfg_attr(test, derive(Arbitrary))]
pub struct BlockHeaderHash(pub [u8; 32]);
pub struct Hash(pub [u8; 32]);
impl fmt::Debug for BlockHeaderHash {
impl fmt::Debug for Hash {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
f.debug_tuple("BlockHeaderHash")
.field(&hex::encode(&self.0))
@ -34,7 +34,7 @@ impl fmt::Debug for BlockHeaderHash {
}
}
impl<'a> From<&'a BlockHeader> for BlockHeaderHash {
impl<'a> From<&'a BlockHeader> for Hash {
fn from(block_header: &'a BlockHeader) -> Self {
let mut hash_writer = sha256d::Writer::default();
block_header
@ -44,27 +44,27 @@ impl<'a> From<&'a BlockHeader> for BlockHeaderHash {
}
}
impl ZcashSerialize for BlockHeaderHash {
impl ZcashSerialize for Hash {
fn zcash_serialize<W: io::Write>(&self, mut writer: W) -> Result<(), io::Error> {
writer.write_all(&self.0)?;
Ok(())
}
}
impl ZcashDeserialize for BlockHeaderHash {
impl ZcashDeserialize for Hash {
fn zcash_deserialize<R: io::Read>(mut reader: R) -> Result<Self, SerializationError> {
Ok(BlockHeaderHash(reader.read_32_bytes()?))
Ok(Hash(reader.read_32_bytes()?))
}
}
impl std::str::FromStr for BlockHeaderHash {
impl std::str::FromStr for Hash {
type Err = SerializationError;
fn from_str(s: &str) -> Result<Self, Self::Err> {
let mut bytes = [0; 32];
if hex::decode_to_slice(s, &mut bytes[..]).is_err() {
Err(SerializationError::Parse("hex decoding error"))
} else {
Ok(BlockHeaderHash(bytes))
Ok(Hash(bytes))
}
}
}

View File

@ -3,7 +3,7 @@ use chrono::{DateTime, Duration, Utc};
use crate::serialization::ZcashSerialize;
use crate::work::{difficulty::CompactDifficulty, equihash::Solution};
use super::{merkle::MerkleTreeRootHash, BlockHeaderHash, Error};
use super::{merkle::MerkleTreeRootHash, Error, Hash};
/// Block header.
///
@ -26,7 +26,7 @@ pub struct BlockHeader {
/// A SHA-256d hash in internal byte order of the previous blocks
/// header. This ensures no previous block can be changed without
/// also changing this blocks header.
pub previous_block_hash: BlockHeaderHash,
pub previous_block_hash: Hash,
/// A SHA-256d hash in internal byte order. The merkle root is
/// derived from the SHA256d hashes of all transactions included

View File

@ -9,7 +9,7 @@ use crate::work::{difficulty::CompactDifficulty, equihash};
use super::merkle::MerkleTreeRootHash;
use super::Block;
use super::BlockHeader;
use super::BlockHeaderHash;
use super::Hash;
/// The maximum size of a Zcash block, in bytes.
///
@ -70,7 +70,7 @@ impl ZcashDeserialize for BlockHeader {
Ok(BlockHeader {
version,
previous_block_hash: BlockHeaderHash::zcash_deserialize(&mut reader)?,
previous_block_hash: Hash::zcash_deserialize(&mut reader)?,
merkle_root_hash: MerkleTreeRootHash(reader.read_32_bytes()?),
root_bytes: reader.read_32_bytes()?,
// This can't panic, because all u32 values are valid `Utc.timestamp`s

View File

@ -30,7 +30,7 @@ impl Arbitrary for BlockHeader {
(
// version is interpreted as i32 in the spec, so we are limited to i32::MAX here
(4u32..(i32::MAX as u32)),
any::<BlockHeaderHash>(),
any::<Hash>(),
any::<merkle::MerkleTreeRootHash>(),
any::<[u8; 32]>(),
// time is interpreted as u32 in the spec, but rust timestamps are i64

View File

@ -10,9 +10,9 @@ use super::super::{serialize::MAX_BLOCK_BYTES, *};
proptest! {
#[test]
fn blockheaderhash_roundtrip(hash in any::<BlockHeaderHash>()) {
fn blockheaderhash_roundtrip(hash in any::<Hash>()) {
let bytes = hash.zcash_serialize_to_vec()?;
let other_hash: BlockHeaderHash = bytes.zcash_deserialize_into()?;
let other_hash: Hash = bytes.zcash_deserialize_into()?;
prop_assert_eq![hash, other_hash];
}

View File

@ -17,7 +17,7 @@ fn blockheaderhash_debug() {
let mut sha_writer = sha256d::Writer::default();
let _ = sha_writer.write_all(preimage);
let hash = BlockHeaderHash(sha_writer.finish());
let hash = Hash(sha_writer.finish());
assert_eq!(
format!("{:?}", hash),
@ -29,7 +29,7 @@ fn blockheaderhash_debug() {
fn blockheaderhash_from_blockheader() {
let blockheader = generate::block_header();
let hash = BlockHeaderHash::from(&blockheader);
let hash = Hash::from(&blockheader);
assert_eq!(
format!("{:?}", hash),

View File

@ -1,7 +1,7 @@
//! Block difficulty data structures and calculations
//!
//! The block difficulty "target threshold" is stored in the block header as a
//! 32-bit `CompactDifficulty`. The `BlockHeaderHash` must be less than or equal
//! 32-bit `CompactDifficulty`. The `block::Hash` must be less than or equal
//! to the `ExpandedDifficulty` threshold, when represented as a 256-bit integer
//! in little-endian order.
//!
@ -10,7 +10,7 @@
//! block's work value depends on the fixed threshold in the block header, not
//! the actual work represented by the block header hash.
use crate::block::BlockHeaderHash;
use crate::block;
use std::cmp::{Ordering, PartialEq, PartialOrd};
use std::{fmt, ops::Add, ops::AddAssign};
@ -42,7 +42,7 @@ mod tests;
/// consensus-critical, because it is used for the `difficulty_threshold` field,
/// which is:
/// - part of the `BlockHeader`, which is used to create the
/// `BlockHeaderHash`, and
/// `block::Hash`, and
/// - bitwise equal to the median `ExpandedDifficulty` value of recent blocks,
/// when encoded to `CompactDifficulty` using the specified conversion
/// function.
@ -65,12 +65,12 @@ impl fmt::Debug for CompactDifficulty {
/// A 256-bit unsigned "expanded difficulty" value.
///
/// Used as a target threshold for the difficulty of a `BlockHeaderHash`.
/// Used as a target threshold for the difficulty of a `block::Hash`.
///
/// Details:
///
/// The precise bit pattern of an `ExpandedDifficulty` value is
/// consensus-critical, because it is compared with the `BlockHeaderHash`.
/// consensus-critical, because it is compared with the `block::Hash`.
///
/// Note that each `CompactDifficulty` value represents a range of
/// `ExpandedDifficulty` values, because the precision of the
@ -85,7 +85,7 @@ pub struct ExpandedDifficulty(U256);
impl fmt::Debug for ExpandedDifficulty {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
let mut buf = [0; 32];
// Use the same byte order as BlockHeaderHash
// Use the same byte order as block::Hash
self.0.to_little_endian(&mut buf);
f.debug_tuple("ExpandedDifficulty")
.field(&hex::encode(&buf))
@ -253,29 +253,29 @@ impl ExpandedDifficulty {
///
/// Hashes are not used to calculate the difficulties of future blocks, so
/// users of this module should avoid converting hashes into difficulties.
fn from_hash(hash: &BlockHeaderHash) -> ExpandedDifficulty {
fn from_hash(hash: &block::Hash) -> ExpandedDifficulty {
ExpandedDifficulty(U256::from_little_endian(&hash.0))
}
}
impl PartialEq<BlockHeaderHash> for ExpandedDifficulty {
impl PartialEq<block::Hash> for ExpandedDifficulty {
/// Is `self` equal to `other`?
///
/// See `partial_cmp` for details.
fn eq(&self, other: &BlockHeaderHash) -> bool {
fn eq(&self, other: &block::Hash) -> bool {
self.partial_cmp(other) == Some(Ordering::Equal)
}
}
impl PartialOrd<BlockHeaderHash> for ExpandedDifficulty {
/// `BlockHeaderHash`es are compared with `ExpandedDifficulty` thresholds by
impl PartialOrd<block::Hash> for ExpandedDifficulty {
/// `block::Hash`es are compared with `ExpandedDifficulty` thresholds by
/// converting the hash to a 256-bit integer in little-endian order.
fn partial_cmp(&self, other: &BlockHeaderHash) -> Option<Ordering> {
fn partial_cmp(&self, other: &block::Hash) -> Option<Ordering> {
self.partial_cmp(&ExpandedDifficulty::from_hash(other))
}
}
impl PartialEq<ExpandedDifficulty> for BlockHeaderHash {
impl PartialEq<ExpandedDifficulty> for block::Hash {
/// Is `self` equal to `other`?
///
/// See `partial_cmp` for details.
@ -284,8 +284,8 @@ impl PartialEq<ExpandedDifficulty> for BlockHeaderHash {
}
}
impl PartialOrd<ExpandedDifficulty> for BlockHeaderHash {
/// `BlockHeaderHash`es are compared with `ExpandedDifficulty` thresholds by
impl PartialOrd<ExpandedDifficulty> for block::Hash {
/// `block::Hash`es are compared with `ExpandedDifficulty` thresholds by
/// converting the hash to a 256-bit integer in little-endian order.
fn partial_cmp(&self, other: &ExpandedDifficulty) -> Option<Ordering> {
use Ordering::*;

View File

@ -253,7 +253,7 @@ fn block_difficulty() -> Result<(), Report> {
&zebra_test::vectors::BLOCK_MAINNET_10_BYTES[..],
] {
let block = Arc::<Block>::zcash_deserialize(*b)?;
let hash: BlockHeaderHash = block.as_ref().into();
let hash = block.hash();
blockchain.push((block.clone(), block.coinbase_height().unwrap(), hash));
}
@ -339,7 +339,7 @@ fn expanded_order() -> Result<(), Report> {
Ok(())
}
/// Test ExpandedDifficulty and BlockHeaderHash ordering
/// Test ExpandedDifficulty and block::Hash ordering
#[test]
#[spandoc::spandoc]
fn expanded_hash_order() -> Result<(), Report> {
@ -348,8 +348,8 @@ fn expanded_hash_order() -> Result<(), Report> {
let ex_zero = ExpandedDifficulty(U256::zero());
let ex_one = ExpandedDifficulty(U256::one());
let ex_max = ExpandedDifficulty(U256::MAX);
let hash_zero = BlockHeaderHash([0; 32]);
let hash_max = BlockHeaderHash([0xff; 32]);
let hash_zero = block::Hash([0; 32]);
let hash_max = block::Hash([0xff; 32]);
assert_eq!(hash_zero, ex_zero);
assert!(hash_zero < ex_one);
@ -382,8 +382,8 @@ proptest! {
let expanded = compact.to_expanded();
let work = compact.to_work();
let hash_zero = BlockHeaderHash([0; 32]);
let hash_max = BlockHeaderHash([0xff; 32]);
let hash_zero = block::Hash([0; 32]);
let hash_max = block::Hash([0xff; 32]);
let work_zero = Work(0);
let work_max = Work(u128::MAX);
@ -399,20 +399,20 @@ proptest! {
}
}
/// Check that a random ExpandedDifficulty compares correctly with fixed BlockHeaderHashes.
/// Check that a random ExpandedDifficulty compares correctly with fixed block::Hash
#[test]
fn prop_expanded_order(expanded in any::<ExpandedDifficulty>()) {
// TODO: round-trip test, once we have ExpandedDifficulty::to_compact()
let hash_zero = BlockHeaderHash([0; 32]);
let hash_max = BlockHeaderHash([0xff; 32]);
let hash_zero = block::Hash([0; 32]);
let hash_max = block::Hash([0xff; 32]);
prop_assert!(expanded >= hash_zero);
prop_assert!(expanded <= hash_max);
}
/// Check that ExpandedDifficulty compares correctly with a random BlockHeaderHash.
/// Check that ExpandedDifficulty compares correctly with a random block::Hash.
#[test]
fn prop_hash_order(hash in any::<BlockHeaderHash>()) {
fn prop_hash_order(hash in any::<block::Hash>()) {
let ex_zero = ExpandedDifficulty(U256::zero());
let ex_one = ExpandedDifficulty(U256::one());
let ex_max = ExpandedDifficulty(U256::MAX);
@ -422,10 +422,10 @@ proptest! {
prop_assert!(hash >= ex_one || hash == ex_zero);
}
/// Check that a random ExpandedDifficulty and BlockHeaderHash compare correctly.
/// Check that a random ExpandedDifficulty and block::Hash compare correctly.
#[test]
#[allow(clippy::double_comparisons)]
fn prop_expanded_hash_order(expanded in any::<ExpandedDifficulty>(), hash in any::<BlockHeaderHash>()) {
fn prop_expanded_hash_order(expanded in any::<ExpandedDifficulty>(), hash in any::<block::Hash>()) {
prop_assert!(expanded < hash || expanded > hash || expanded == hash);
}

View File

@ -26,7 +26,7 @@ use tokio::time;
use tower::{buffer::Buffer, Service, ServiceExt};
use zebra_chain::block::BlockHeight;
use zebra_chain::block::{Block, BlockHeaderHash};
use zebra_chain::block::{Block, self};
/// A service that verifies blocks.
#[derive(Debug)]
@ -60,7 +60,7 @@ where
+ 'static,
S::Future: Send + 'static,
{
type Response = BlockHeaderHash;
type Response = block::Hash;
type Error = Error;
type Future =
Pin<Box<dyn Future<Output = Result<Self::Response, Self::Error>> + Send + 'static>>;
@ -158,7 +158,7 @@ where
///
/// If there is no block for that hash, returns `Ok(None)`.
/// Returns an error if `state.poll_ready` errors.
async fn get_block(state: &mut S, hash: BlockHeaderHash) -> Result<Option<Arc<Block>>, Report> {
async fn get_block(state: &mut S, hash: block::Hash) -> Result<Option<Arc<Block>>, Report> {
let block = state
.ready_and()
.await
@ -179,7 +179,7 @@ where
/// Returns an error if `state.poll_ready` errors.
async fn await_block(
state: &mut S,
hash: BlockHeaderHash,
hash: block::Hash,
height: BlockHeight,
) -> Result<Arc<Block>, Report> {
loop {
@ -215,9 +215,9 @@ pub fn init<S>(
state_service: S,
) -> impl Service<
Arc<Block>,
Response = BlockHeaderHash,
Response = block::Hash,
Error = Error,
Future = impl Future<Output = Result<BlockHeaderHash, Error>>,
Future = impl Future<Output = Result<block::Hash, Error>>,
> + Send
+ Clone
+ 'static

View File

@ -6,12 +6,12 @@ use chrono::Utc;
use color_eyre::eyre::{eyre, Report};
use once_cell::sync::Lazy;
use zebra_chain::block::Block;
use zebra_chain::block::{Block, self};
use zebra_chain::block::BlockHeader;
use zebra_chain::serialization::{ZcashDeserialize, ZcashDeserializeInto};
use zebra_test::transcript::{TransError, Transcript};
static VALID_BLOCK_TRANSCRIPT: Lazy<Vec<(Arc<Block>, Result<BlockHeaderHash, TransError>)>> =
static VALID_BLOCK_TRANSCRIPT: Lazy<Vec<(Arc<Block>, Result<block::Hash, TransError>)>> =
Lazy::new(|| {
let block: Arc<_> =
Block::zcash_deserialize(&zebra_test::vectors::BLOCK_MAINNET_GENESIS_BYTES[..])
@ -21,7 +21,7 @@ static VALID_BLOCK_TRANSCRIPT: Lazy<Vec<(Arc<Block>, Result<BlockHeaderHash, Tra
vec![(block, hash)]
});
static INVALID_TIME_BLOCK_TRANSCRIPT: Lazy<Vec<(Arc<Block>, Result<BlockHeaderHash, TransError>)>> =
static INVALID_TIME_BLOCK_TRANSCRIPT: Lazy<Vec<(Arc<Block>, Result<block::Hash, TransError>)>> =
Lazy::new(|| {
let mut block: Block =
Block::zcash_deserialize(&zebra_test::vectors::BLOCK_MAINNET_GENESIS_BYTES[..])
@ -41,7 +41,7 @@ static INVALID_TIME_BLOCK_TRANSCRIPT: Lazy<Vec<(Arc<Block>, Result<BlockHeaderHa
});
static INVALID_HEADER_SOLUTION_TRANSCRIPT: Lazy<
Vec<(Arc<Block>, Result<BlockHeaderHash, TransError>)>,
Vec<(Arc<Block>, Result<block::Hash, TransError>)>,
> = Lazy::new(|| {
let mut block: Block =
Block::zcash_deserialize(&zebra_test::vectors::BLOCK_MAINNET_GENESIS_BYTES[..]).unwrap();
@ -52,7 +52,7 @@ static INVALID_HEADER_SOLUTION_TRANSCRIPT: Lazy<
vec![(Arc::new(block), Err(TransError::Any))]
});
static INVALID_COINBASE_TRANSCRIPT: Lazy<Vec<(Arc<Block>, Result<BlockHeaderHash, TransError>)>> =
static INVALID_COINBASE_TRANSCRIPT: Lazy<Vec<(Arc<Block>, Result<block::Hash, TransError>)>> =
Lazy::new(|| {
let header =
BlockHeader::zcash_deserialize(&zebra_test::vectors::DUMMY_HEADER[..]).unwrap();

View File

@ -28,7 +28,7 @@ use tower::{buffer::Buffer, Service, ServiceExt};
use tracing_futures::Instrument;
use zebra_chain::block::BlockHeight;
use zebra_chain::block::{Block, BlockHeaderHash};
use zebra_chain::block::{Block, self};
use zebra_chain::parameters::{Network, NetworkUpgrade::Sapling};
/// The maximum expected gap between blocks.
@ -52,7 +52,7 @@ struct ChainCheckpointVerifier {
/// and `BlockVerifier`.
struct ChainVerifier<BV, S>
where
BV: Service<Arc<Block>, Response = BlockHeaderHash, Error = Error> + Send + Clone + 'static,
BV: Service<Arc<Block>, Response = block::Hash, Error = Error> + Send + Clone + 'static,
BV::Future: Send + 'static,
S: Service<zebra_state::Request, Response = zebra_state::Response, Error = Error>
+ Send
@ -89,7 +89,7 @@ type Error = Box<dyn error::Error + Send + Sync + 'static>;
/// After verification, blocks are added to the underlying state service.
impl<BV, S> Service<Arc<Block>> for ChainVerifier<BV, S>
where
BV: Service<Arc<Block>, Response = BlockHeaderHash, Error = Error> + Send + Clone + 'static,
BV: Service<Arc<Block>, Response = block::Hash, Error = Error> + Send + Clone + 'static,
BV::Future: Send + 'static,
S: Service<zebra_state::Request, Response = zebra_state::Response, Error = Error>
+ Send
@ -97,7 +97,7 @@ where
+ 'static,
S::Future: Send + 'static,
{
type Response = BlockHeaderHash;
type Response = block::Hash;
type Error = Error;
type Future =
Pin<Box<dyn Future<Output = Result<Self::Response, Self::Error>> + Send + 'static>>;
@ -217,9 +217,9 @@ pub async fn init<S>(
state_service: S,
) -> impl Service<
Arc<Block>,
Response = BlockHeaderHash,
Response = block::Hash,
Error = Error,
Future = impl Future<Output = Result<BlockHeaderHash, Error>>,
Future = impl Future<Output = Result<block::Hash, Error>>,
> + Send
+ Clone
+ 'static
@ -276,14 +276,14 @@ pub(crate) fn init_from_verifiers<BV, S>(
initial_tip: Option<Arc<Block>>,
) -> impl Service<
Arc<Block>,
Response = BlockHeaderHash,
Response = block::Hash,
Error = Error,
Future = impl Future<Output = Result<BlockHeaderHash, Error>>,
Future = impl Future<Output = Result<block::Hash, Error>>,
> + Send
+ Clone
+ 'static
where
BV: Service<Arc<Block>, Response = BlockHeaderHash, Error = Error> + Send + Clone + 'static,
BV: Service<Arc<Block>, Response = block::Hash, Error = Error> + Send + Clone + 'static,
BV::Future: Send + 'static,
S: Service<zebra_state::Request, Response = zebra_state::Response, Error = Error>
+ Send

View File

@ -54,9 +54,9 @@ fn verifiers_from_checkpoint_list(
) -> (
impl Service<
Arc<Block>,
Response = BlockHeaderHash,
Response = block::Hash,
Error = Error,
Future = impl Future<Output = Result<BlockHeaderHash, Error>>,
Future = impl Future<Output = Result<block::Hash, Error>>,
> + Send
+ Clone
+ 'static,
@ -89,9 +89,9 @@ fn verifiers_from_network(
) -> (
impl Service<
Arc<Block>,
Response = BlockHeaderHash,
Response = block::Hash,
Error = Error,
Future = impl Future<Output = Result<BlockHeaderHash, Error>>,
Future = impl Future<Output = Result<block::Hash, Error>>,
> + Send
+ Clone
+ 'static,
@ -108,19 +108,19 @@ fn verifiers_from_network(
}
static BLOCK_VERIFY_TRANSCRIPT_GENESIS: Lazy<
Vec<(Arc<Block>, Result<BlockHeaderHash, TransError>)>,
Vec<(Arc<Block>, Result<block::Hash, TransError>)>,
> = Lazy::new(|| {
let block: Arc<_> =
Block::zcash_deserialize(&zebra_test::vectors::BLOCK_MAINNET_GENESIS_BYTES[..])
.unwrap()
.into();
let hash = Ok(block.as_ref().into());
let hash = Ok(block.hash());
vec![(block, hash)]
});
static BLOCK_VERIFY_TRANSCRIPT_GENESIS_FAIL: Lazy<
Vec<(Arc<Block>, Result<BlockHeaderHash, TransError>)>,
Vec<(Arc<Block>, Result<block::Hash, TransError>)>,
> = Lazy::new(|| {
let block: Arc<_> =
Block::zcash_deserialize(&zebra_test::vectors::BLOCK_MAINNET_GENESIS_BYTES[..])
@ -131,23 +131,23 @@ static BLOCK_VERIFY_TRANSCRIPT_GENESIS_FAIL: Lazy<
});
static BLOCK_VERIFY_TRANSCRIPT_GENESIS_TO_BLOCK_1: Lazy<
Vec<(Arc<Block>, Result<BlockHeaderHash, TransError>)>,
Vec<(Arc<Block>, Result<block::Hash, TransError>)>,
> = Lazy::new(|| {
let block0: Arc<_> =
Block::zcash_deserialize(&zebra_test::vectors::BLOCK_MAINNET_GENESIS_BYTES[..])
.unwrap()
.into();
let hash0 = Ok(block0.as_ref().into());
let hash0 = Ok(block0.hash());
let block1: Arc<_> = Block::zcash_deserialize(&zebra_test::vectors::BLOCK_MAINNET_1_BYTES[..])
.unwrap()
.into();
let hash1 = Ok(block1.as_ref().into());
let hash1 = Ok(block1.hash());
vec![(block0, hash0), (block1, hash1)]
});
static NO_COINBASE_TRANSCRIPT: Lazy<Vec<(Arc<Block>, Result<BlockHeaderHash, TransError>)>> =
static NO_COINBASE_TRANSCRIPT: Lazy<Vec<(Arc<Block>, Result<block::Hash, TransError>)>> =
Lazy::new(|| {
let block = block_no_transactions();
@ -161,7 +161,7 @@ static NO_COINBASE_STATE_TRANSCRIPT: Lazy<
)>,
> = Lazy::new(|| {
let block = block_no_transactions();
let hash: BlockHeaderHash = (&block).into();
let hash = block.hash();
vec![(
zebra_state::Request::GetBlock { hash },
@ -179,7 +179,7 @@ static STATE_VERIFY_TRANSCRIPT_GENESIS: Lazy<
Block::zcash_deserialize(&zebra_test::vectors::BLOCK_MAINNET_GENESIS_BYTES[..])
.unwrap()
.into();
let hash: BlockHeaderHash = block.as_ref().into();
let hash = block.hash();
vec![(
zebra_state::Request::GetBlock { hash },
@ -205,14 +205,14 @@ async fn verify_block() -> Result<(), Report> {
let mut checkpoint_data = Vec::new();
let block0 =
Arc::<Block>::zcash_deserialize(&zebra_test::vectors::BLOCK_MAINNET_GENESIS_BYTES[..])?;
let hash0: BlockHeaderHash = block0.as_ref().into();
let hash0 = block0.hash();
checkpoint_data.push((
block0.coinbase_height().expect("test block has height"),
hash0,
));
// Make a checkpoint list containing the genesis block
let checkpoint_list: BTreeMap<BlockHeight, BlockHeaderHash> =
let checkpoint_list: BTreeMap<BlockHeight, block::Hash> =
checkpoint_data.iter().cloned().collect();
let checkpoint_list = CheckpointList::from_list(checkpoint_list).map_err(|e| eyre!(e))?;
@ -365,7 +365,7 @@ async fn continuous_blockchain(restart_height: Option<BlockHeight>) -> Result<()
&zebra_test::vectors::BLOCK_MAINNET_10_BYTES[..],
] {
let block = Arc::<Block>::zcash_deserialize(*b)?;
let hash: BlockHeaderHash = block.as_ref().into();
let hash = block.hash();
blockchain.push((block.clone(), block.coinbase_height().unwrap(), hash));
}
@ -376,12 +376,12 @@ async fn continuous_blockchain(restart_height: Option<BlockHeight>) -> Result<()
&zebra_test::vectors::BLOCK_MAINNET_4_BYTES[..],
] {
let block = Arc::<Block>::zcash_deserialize(*b)?;
let hash: BlockHeaderHash = block.as_ref().into();
let hash = block.hash();
checkpoints.push((block.clone(), block.coinbase_height().unwrap(), hash));
}
// The checkpoint list will contain only blocks 0 and 4
let checkpoint_list: BTreeMap<BlockHeight, BlockHeaderHash> = checkpoints
let checkpoint_list: BTreeMap<BlockHeight, block::Hash> = checkpoints
.iter()
.map(|(_block, height, hash)| (*height, *hash))
.collect();

View File

@ -39,7 +39,8 @@ use tokio::sync::oneshot;
use tower::Service;
use zebra_chain::{
block::{Block, BlockHeaderHash, BlockHeight},
block::BlockHeight,
block::{Block, self},
parameters::Network,
};
@ -53,9 +54,9 @@ struct QueuedBlock {
/// The block data.
block: Arc<Block>,
/// `block`'s cached header hash.
hash: BlockHeaderHash,
hash: block::Hash,
/// The transmitting end of the oneshot channel for this block's result.
tx: oneshot::Sender<Result<BlockHeaderHash, Error>>,
tx: oneshot::Sender<Result<block::Hash, Error>>,
}
/// A list of unverified blocks at a particular height.
@ -94,7 +95,7 @@ pub struct CheckpointVerifier {
checkpoint_list: CheckpointList,
/// The hash of the initial tip, if any.
initial_tip_hash: Option<BlockHeaderHash>,
initial_tip_hash: Option<block::Hash>,
// Queued Blocks
//
@ -151,7 +152,7 @@ impl CheckpointVerifier {
// This function is designed for use in tests.
#[allow(dead_code)]
pub(crate) fn from_list(
list: impl IntoIterator<Item = (BlockHeight, BlockHeaderHash)>,
list: impl IntoIterator<Item = (BlockHeight, block::Hash)>,
initial_tip: Option<Arc<Block>>,
) -> Result<Self, Error> {
Ok(Self::from_checkpoint_list(
@ -295,7 +296,7 @@ impl CheckpointVerifier {
/// Return the most recently verified checkpoint's hash.
///
/// See `previous_checkpoint_height()` for details.
fn previous_checkpoint_hash(&self) -> Progress<BlockHeaderHash> {
fn previous_checkpoint_hash(&self) -> Progress<block::Hash> {
match self.previous_checkpoint_height() {
BeforeGenesis => BeforeGenesis,
InitialTip(_) => self
@ -387,7 +388,7 @@ impl CheckpointVerifier {
fn queue_block(
&mut self,
block: Arc<Block>,
) -> oneshot::Receiver<Result<BlockHeaderHash, Error>> {
) -> oneshot::Receiver<Result<block::Hash, Error>> {
// Set up a oneshot channel to send results
let (tx, rx) = oneshot::channel();
@ -459,7 +460,7 @@ impl CheckpointVerifier {
fn process_height(
&mut self,
height: BlockHeight,
expected_hash: BlockHeaderHash,
expected_hash: block::Hash,
) -> Option<QueuedBlock> {
let mut qblocks = self
.queued
@ -705,7 +706,7 @@ impl Drop for CheckpointVerifier {
///
/// After verification, the block futures resolve to their hashes.
impl Service<Arc<Block>> for CheckpointVerifier {
type Response = BlockHeaderHash;
type Response = block::Hash;
type Error = Error;
type Future =
Pin<Box<dyn Future<Output = Result<Self::Response, Self::Error>> + Send + 'static>>;

View File

@ -17,7 +17,7 @@ use std::{
str::FromStr,
};
use zebra_chain::block::BlockHeaderHash;
use zebra_chain::block;
use zebra_chain::block::BlockHeight;
use zebra_chain::parameters::Network;
@ -38,7 +38,7 @@ type Error = Box<dyn error::Error + Send + Sync + 'static>;
/// This is actually a bijective map, but since it is read-only, we use a
/// BTreeMap, and do the value uniqueness check on initialisation.
#[derive(Clone, Debug, Eq, Hash, PartialEq)]
pub(crate) struct CheckpointList(BTreeMap<BlockHeight, BlockHeaderHash>);
pub(crate) struct CheckpointList(BTreeMap<BlockHeight, block::Hash>);
impl FromStr for CheckpointList {
type Err = Error;
@ -46,11 +46,11 @@ impl FromStr for CheckpointList {
/// Parse a string into a CheckpointList.
///
/// Each line has one checkpoint, consisting of a `BlockHeight` and
/// `BlockHeaderHash`, separated by a single space.
/// `block::Hash`, separated by a single space.
///
/// Assumes that the provided genesis checkpoint is correct.
fn from_str(s: &str) -> Result<Self, Self::Err> {
let mut checkpoint_list: Vec<(BlockHeight, BlockHeaderHash)> = Vec::new();
let mut checkpoint_list: Vec<(BlockHeight, block::Hash)> = Vec::new();
for checkpoint in s.lines() {
let fields = checkpoint.split(' ').collect::<Vec<_>>();
@ -95,14 +95,14 @@ impl CheckpointList {
/// There must be a checkpoint for a genesis block at BlockHeight 0.
/// (All other checkpoints are optional.)
pub(crate) fn from_list(
list: impl IntoIterator<Item = (BlockHeight, BlockHeaderHash)>,
list: impl IntoIterator<Item = (BlockHeight, block::Hash)>,
) -> Result<Self, Error> {
// BTreeMap silently ignores duplicates, so we count the checkpoints
// before adding them to the map
let original_checkpoints: Vec<(BlockHeight, BlockHeaderHash)> = list.into_iter().collect();
let original_checkpoints: Vec<(BlockHeight, block::Hash)> = list.into_iter().collect();
let original_len = original_checkpoints.len();
let checkpoints: BTreeMap<BlockHeight, BlockHeaderHash> =
let checkpoints: BTreeMap<BlockHeight, block::Hash> =
original_checkpoints.into_iter().collect();
// Check that the list starts with the correct genesis block
@ -123,14 +123,14 @@ impl CheckpointList {
Err("checkpoint heights must be unique")?;
}
let block_hashes: HashSet<&BlockHeaderHash> = checkpoints.values().collect();
let block_hashes: HashSet<&block::Hash> = checkpoints.values().collect();
if block_hashes.len() != original_len {
Err("checkpoint hashes must be unique")?;
}
// Make sure all the hashes are valid. In Bitcoin, [0; 32] is the null
// hash. It is also used as the parent hash of genesis blocks.
if block_hashes.contains(&BlockHeaderHash([0; 32])) {
if block_hashes.contains(&block::Hash([0; 32])) {
Err("checkpoint list contains invalid checkpoint hash: found null hash")?;
}
@ -153,7 +153,7 @@ impl CheckpointList {
/// or None if there is no checkpoint at that height.
///
/// See `BTreeMap::get()` for details.
pub fn hash(&self, height: BlockHeight) -> Option<BlockHeaderHash> {
pub fn hash(&self, height: BlockHeight) -> Option<block::Hash> {
self.0.get(&height).cloned()
}

View File

@ -5,7 +5,10 @@ use super::*;
use std::sync::Arc;
use zebra_chain::parameters::{Network, NetworkUpgrade::Sapling};
use zebra_chain::{block::Block, serialization::ZcashDeserialize};
use zebra_chain::{
block::{self, Block},
serialization::ZcashDeserialize,
};
/// Make a checkpoint list containing only the genesis block
#[test]
@ -16,14 +19,14 @@ fn checkpoint_list_genesis() -> Result<(), Error> {
let mut checkpoint_data = Vec::new();
let block =
Arc::<Block>::zcash_deserialize(&zebra_test::vectors::BLOCK_MAINNET_GENESIS_BYTES[..])?;
let hash: BlockHeaderHash = block.as_ref().into();
let hash = block.hash();
checkpoint_data.push((
block.coinbase_height().expect("test block has height"),
hash,
));
// Make a checkpoint list containing the genesis block
let checkpoint_list: BTreeMap<BlockHeight, BlockHeaderHash> =
let checkpoint_list: BTreeMap<BlockHeight, block::Hash> =
checkpoint_data.iter().cloned().collect();
let _ = CheckpointList::from_list(checkpoint_list)?;
@ -44,7 +47,7 @@ fn checkpoint_list_multiple() -> Result<(), Error> {
&zebra_test::vectors::BLOCK_MAINNET_434873_BYTES[..],
] {
let block = Arc::<Block>::zcash_deserialize(*b)?;
let hash: BlockHeaderHash = block.as_ref().into();
let hash = block.hash();
checkpoint_data.push((
block.coinbase_height().expect("test block has height"),
hash,
@ -52,8 +55,7 @@ fn checkpoint_list_multiple() -> Result<(), Error> {
}
// Make a checkpoint list containing all the blocks
let checkpoint_list: BTreeMap<BlockHeight, BlockHeaderHash> =
checkpoint_data.iter().cloned().collect();
let checkpoint_list: BTreeMap<BlockHeight, block::Hash> = checkpoint_data.iter().cloned().collect();
let _ = CheckpointList::from_list(checkpoint_list)?;
Ok(())
@ -77,15 +79,14 @@ fn checkpoint_list_no_genesis_fail() -> Result<(), Error> {
// Parse a non-genesis block
let mut checkpoint_data = Vec::new();
let block = Arc::<Block>::zcash_deserialize(&zebra_test::vectors::BLOCK_MAINNET_1_BYTES[..])?;
let hash: BlockHeaderHash = block.as_ref().into();
let hash = block.hash();
checkpoint_data.push((
block.coinbase_height().expect("test block has height"),
hash,
));
// Make a checkpoint list containing the non-genesis block
let checkpoint_list: BTreeMap<BlockHeight, BlockHeaderHash> =
checkpoint_data.iter().cloned().collect();
let checkpoint_list: BTreeMap<BlockHeight, block::Hash> = checkpoint_data.iter().cloned().collect();
let _ = CheckpointList::from_list(checkpoint_list)
.expect_err("a checkpoint list with no genesis block should fail");
@ -97,11 +98,10 @@ fn checkpoint_list_no_genesis_fail() -> Result<(), Error> {
fn checkpoint_list_null_hash_fail() -> Result<(), Error> {
zebra_test::init();
let checkpoint_data = vec![(BlockHeight(0), BlockHeaderHash([0; 32]))];
let checkpoint_data = vec![(BlockHeight(0), block::Hash([0; 32]))];
// Make a checkpoint list containing the non-genesis block
let checkpoint_list: BTreeMap<BlockHeight, BlockHeaderHash> =
checkpoint_data.iter().cloned().collect();
let checkpoint_list: BTreeMap<BlockHeight, block::Hash> = checkpoint_data.iter().cloned().collect();
let _ = CheckpointList::from_list(checkpoint_list)
.expect_err("a checkpoint list with a null block hash should fail");
@ -113,23 +113,18 @@ fn checkpoint_list_null_hash_fail() -> Result<(), Error> {
fn checkpoint_list_bad_height_fail() -> Result<(), Error> {
zebra_test::init();
let checkpoint_data = vec![(
BlockHeight(BlockHeight::MAX.0 + 1),
BlockHeaderHash([1; 32]),
)];
let checkpoint_data = vec![(BlockHeight(BlockHeight::MAX.0 + 1), block::Hash([1; 32]))];
// Make a checkpoint list containing the non-genesis block
let checkpoint_list: BTreeMap<BlockHeight, BlockHeaderHash> =
checkpoint_data.iter().cloned().collect();
let checkpoint_list: BTreeMap<BlockHeight, block::Hash> = checkpoint_data.iter().cloned().collect();
let _ = CheckpointList::from_list(checkpoint_list).expect_err(
"a checkpoint list with an invalid block height (BlockHeight::MAX + 1) should fail",
);
let checkpoint_data = vec![(BlockHeight(u32::MAX), BlockHeaderHash([1; 32]))];
let checkpoint_data = vec![(BlockHeight(u32::MAX), block::Hash([1; 32]))];
// Make a checkpoint list containing the non-genesis block
let checkpoint_list: BTreeMap<BlockHeight, BlockHeaderHash> =
checkpoint_data.iter().cloned().collect();
let checkpoint_list: BTreeMap<BlockHeight, block::Hash> = checkpoint_data.iter().cloned().collect();
let _ = CheckpointList::from_list(checkpoint_list)
.expect_err("a checkpoint list with an invalid block height (u32::MAX) should fail");
@ -149,7 +144,7 @@ fn checkpoint_list_duplicate_blocks_fail() -> Result<(), Error> {
&zebra_test::vectors::BLOCK_MAINNET_1_BYTES[..],
] {
let block = Arc::<Block>::zcash_deserialize(*b)?;
let hash: BlockHeaderHash = block.as_ref().into();
let hash = block.hash();
checkpoint_data.push((
block.coinbase_height().expect("test block has height"),
hash,
@ -173,7 +168,7 @@ fn checkpoint_list_duplicate_heights_fail() -> Result<(), Error> {
let mut checkpoint_data = Vec::new();
for b in &[&zebra_test::vectors::BLOCK_MAINNET_GENESIS_BYTES[..]] {
let block = Arc::<Block>::zcash_deserialize(*b)?;
let hash: BlockHeaderHash = block.as_ref().into();
let hash = block.hash();
checkpoint_data.push((
block.coinbase_height().expect("test block has height"),
hash,
@ -181,8 +176,8 @@ fn checkpoint_list_duplicate_heights_fail() -> Result<(), Error> {
}
// Then add some fake entries with duplicate heights
checkpoint_data.push((BlockHeight(1), BlockHeaderHash([0xaa; 32])));
checkpoint_data.push((BlockHeight(1), BlockHeaderHash([0xbb; 32])));
checkpoint_data.push((BlockHeight(1), block::Hash([0xaa; 32])));
checkpoint_data.push((BlockHeight(1), block::Hash([0xbb; 32])));
// Make a checkpoint list containing some duplicate blocks
let _ = CheckpointList::from_list(checkpoint_data)
@ -201,7 +196,7 @@ fn checkpoint_list_duplicate_hashes_fail() -> Result<(), Error> {
let mut checkpoint_data = Vec::new();
for b in &[&zebra_test::vectors::BLOCK_MAINNET_GENESIS_BYTES[..]] {
let block = Arc::<Block>::zcash_deserialize(*b)?;
let hash: BlockHeaderHash = block.as_ref().into();
let hash = block.hash();
checkpoint_data.push((
block.coinbase_height().expect("test block has height"),
hash,
@ -209,8 +204,8 @@ fn checkpoint_list_duplicate_hashes_fail() -> Result<(), Error> {
}
// Then add some fake entries with duplicate hashes
checkpoint_data.push((BlockHeight(1), BlockHeaderHash([0xcc; 32])));
checkpoint_data.push((BlockHeight(2), BlockHeaderHash([0xcc; 32])));
checkpoint_data.push((BlockHeight(1), block::Hash([0xcc; 32])));
checkpoint_data.push((BlockHeight(2), block::Hash([0xcc; 32])));
// Make a checkpoint list containing some duplicate blocks
let _ = CheckpointList::from_list(checkpoint_data)

View File

@ -35,10 +35,10 @@ async fn single_item_checkpoint_list() -> Result<(), Report> {
let block0 =
Arc::<Block>::zcash_deserialize(&zebra_test::vectors::BLOCK_MAINNET_GENESIS_BYTES[..])?;
let hash0: BlockHeaderHash = block0.as_ref().into();
let hash0 = block0.hash();
// Make a checkpoint list containing only the genesis block
let genesis_checkpoint_list: BTreeMap<BlockHeight, BlockHeaderHash> =
let genesis_checkpoint_list: BTreeMap<BlockHeight, block::Hash> =
[(block0.coinbase_height().unwrap(), hash0)]
.iter()
.cloned()
@ -114,12 +114,12 @@ async fn multi_item_checkpoint_list() -> Result<(), Report> {
&zebra_test::vectors::BLOCK_MAINNET_1_BYTES[..],
] {
let block = Arc::<Block>::zcash_deserialize(*b)?;
let hash: BlockHeaderHash = block.as_ref().into();
let hash = block.hash();
checkpoint_data.push((block.clone(), block.coinbase_height().unwrap(), hash));
}
// Make a checkpoint list containing all the blocks
let checkpoint_list: BTreeMap<BlockHeight, BlockHeaderHash> = checkpoint_data
let checkpoint_list: BTreeMap<BlockHeight, block::Hash> = checkpoint_data
.iter()
.map(|(_block, height, hash)| (*height, *hash))
.collect();
@ -234,7 +234,7 @@ async fn continuous_blockchain(restart_height: Option<BlockHeight>) -> Result<()
&zebra_test::vectors::BLOCK_MAINNET_10_BYTES[..],
] {
let block = Arc::<Block>::zcash_deserialize(*b)?;
let hash: BlockHeaderHash = block.as_ref().into();
let hash = block.hash();
blockchain.push((block.clone(), block.coinbase_height().unwrap(), hash));
}
@ -246,12 +246,12 @@ async fn continuous_blockchain(restart_height: Option<BlockHeight>) -> Result<()
&zebra_test::vectors::BLOCK_MAINNET_9_BYTES[..],
] {
let block = Arc::<Block>::zcash_deserialize(*b)?;
let hash: BlockHeaderHash = block.as_ref().into();
let hash = block.hash();
checkpoints.push((block.clone(), block.coinbase_height().unwrap(), hash));
}
// The checkpoint list will contain only block 0, 5 and 9
let checkpoint_list: BTreeMap<BlockHeight, BlockHeaderHash> = checkpoints
let checkpoint_list: BTreeMap<BlockHeight, block::Hash> = checkpoints
.iter()
.map(|(_block, height, hash)| (*height, *hash))
.collect();
@ -376,7 +376,7 @@ async fn block_higher_than_max_checkpoint_fail() -> Result<(), Report> {
Arc::<Block>::zcash_deserialize(&zebra_test::vectors::BLOCK_MAINNET_415000_BYTES[..])?;
// Make a checkpoint list containing only the genesis block
let genesis_checkpoint_list: BTreeMap<BlockHeight, BlockHeaderHash> =
let genesis_checkpoint_list: BTreeMap<BlockHeight, block::Hash> =
[(block0.coinbase_height().unwrap(), block0.as_ref().into())]
.iter()
.cloned()
@ -443,7 +443,7 @@ async fn wrong_checkpoint_hash_fail() -> Result<(), Report> {
let good_block0 =
Arc::<Block>::zcash_deserialize(&zebra_test::vectors::BLOCK_MAINNET_GENESIS_BYTES[..])?;
let good_block0_hash: BlockHeaderHash = good_block0.as_ref().into();
let good_block0_hash = good_block0.hash();
// Change the header hash
let mut bad_block0 = good_block0.clone();
let mut bad_block0 = Arc::make_mut(&mut bad_block0);
@ -451,7 +451,7 @@ async fn wrong_checkpoint_hash_fail() -> Result<(), Report> {
let bad_block0: Arc<Block> = bad_block0.clone().into();
// Make a checkpoint list containing the genesis block checkpoint
let genesis_checkpoint_list: BTreeMap<BlockHeight, BlockHeaderHash> =
let genesis_checkpoint_list: BTreeMap<BlockHeight, block::Hash> =
[(good_block0.coinbase_height().unwrap(), good_block0_hash)]
.iter()
.cloned()
@ -627,12 +627,12 @@ async fn checkpoint_drop_cancel() -> Result<(), Report> {
&zebra_test::vectors::BLOCK_MAINNET_434873_BYTES[..],
] {
let block = Arc::<Block>::zcash_deserialize(*b)?;
let hash: BlockHeaderHash = block.as_ref().into();
let hash = block.hash();
checkpoint_data.push((block.clone(), block.coinbase_height().unwrap(), hash));
}
// Make a checkpoint list containing all the blocks
let checkpoint_list: BTreeMap<BlockHeight, BlockHeaderHash> = checkpoint_data
let checkpoint_list: BTreeMap<BlockHeight, block::Hash> = checkpoint_data
.iter()
.map(|(_block, height, hash)| (*height, *hash))
.collect();
@ -719,7 +719,7 @@ async fn hard_coded_mainnet() -> Result<(), Report> {
let block0 =
Arc::<Block>::zcash_deserialize(&zebra_test::vectors::BLOCK_MAINNET_GENESIS_BYTES[..])?;
let hash0: BlockHeaderHash = block0.as_ref().into();
let hash0 = block0.hash();
// Use the hard-coded checkpoint list
let mut checkpoint_verifier = CheckpointVerifier::new(Network::Mainnet, None);

View File

@ -1,15 +1,15 @@
//! Genesis consensus parameters for each Zcash network.
use zebra_chain::{block::BlockHeaderHash, parameters::Network};
use zebra_chain::{block, parameters::Network};
/// The previous block hash for the genesis block.
///
/// All known networks use the Bitcoin `null` value for the parent of the
/// genesis block. (In Bitcoin, `null` is `[0; 32]`.)
pub const GENESIS_PREVIOUS_BLOCK_HASH: BlockHeaderHash = BlockHeaderHash([0; 32]);
pub const GENESIS_PREVIOUS_BLOCK_HASH: block::Hash = block::Hash([0; 32]);
/// Returns the hash for the genesis block in `network`.
pub fn genesis_hash(network: Network) -> BlockHeaderHash {
pub fn genesis_hash(network: Network) -> block::Hash {
match network {
// zcash-cli getblockhash 0 | zebrad revhex
Network::Mainnet => "08ce3d9731b000c08338455c8a4a6bd05da16e26b11daa1b917184ece80f0400",

View File

@ -28,7 +28,7 @@ use tower::Service;
use tracing_futures::Instrument;
use zebra_chain::{
block::{Block, BlockHeaderHash},
block::{Block, self},
serialization::SerializationError,
};
@ -49,7 +49,7 @@ pub(super) enum Handler {
Ping(Nonce),
GetPeers,
GetBlocksByHash {
hashes: HashSet<BlockHeaderHash>,
hashes: HashSet<block::Hash>,
blocks: Vec<Arc<Block>>,
},
FindBlocks,
@ -87,7 +87,7 @@ impl Handler {
},
Message::Block(block),
) => {
if hashes.remove(&BlockHeaderHash::from(block.as_ref())) {
if hashes.remove(&block.hash()) {
blocks.push(block);
if hashes.is_empty() {
Finished(Ok(Response::Blocks(blocks)))
@ -98,7 +98,7 @@ impl Handler {
Finished(Err(PeerError::WrongBlock.into()))
}
}
(FindBlocks, Message::Inv(inv_hashes)) => Finished(Ok(Response::BlockHeaderHashes(
(FindBlocks, Message::Inv(inv_hashes)) => Finished(Ok(Response::BlockHashes(
inv_hashes
.into_iter()
.filter_map(|inv| match inv {
@ -385,7 +385,7 @@ where
.peer_tx
.send(Message::GetBlocks {
block_locator_hashes: known_blocks,
hash_stop: stop.unwrap_or(BlockHeaderHash([0; 32])),
hash_stop: stop.unwrap_or(block::Hash([0; 32])),
})
.await
.map_err(|e| e.into())
@ -502,7 +502,7 @@ where
}
}
}
Response::BlockHeaderHashes(hashes) => {
Response::BlockHashes(hashes) => {
if let Err(e) = self
.peer_tx
.send(Message::Inv(hashes.into_iter().map(Into::into).collect()))

View File

@ -10,7 +10,7 @@ use tokio_util::codec::{Decoder, Encoder};
use zebra_chain::{
block::BlockHeight,
block::{Block, BlockHeaderHash},
block::{Block, self},
parameters::Network,
serialization::{
sha256d, ReadZcashExt, SerializationError as Error, WriteZcashExt, ZcashDeserialize,
@ -486,7 +486,7 @@ impl Codec {
if self.builder.version == Version(reader.read_u32::<LittleEndian>()?) {
Ok(Message::GetBlocks {
block_locator_hashes: Vec::zcash_deserialize(&mut reader)?,
hash_stop: BlockHeaderHash::zcash_deserialize(&mut reader)?,
hash_stop: block::Hash::zcash_deserialize(&mut reader)?,
})
} else {
Err(Error::Parse("getblocks version did not match negotiation"))
@ -506,7 +506,7 @@ impl Codec {
if self.builder.version == Version(reader.read_u32::<LittleEndian>()?) {
Ok(Message::GetHeaders {
block_locator_hashes: Vec::zcash_deserialize(&mut reader)?,
hash_stop: BlockHeaderHash::zcash_deserialize(&mut reader)?,
hash_stop: block::Hash::zcash_deserialize(&mut reader)?,
})
} else {
Err(Error::Parse("getblocks version did not match negotiation"))

View File

@ -8,7 +8,7 @@ use std::io::{Read, Write};
use byteorder::{LittleEndian, ReadBytesExt, WriteBytesExt};
use zebra_chain::block::BlockHeaderHash;
use zebra_chain::block;
use zebra_chain::serialization::{
ReadZcashExt, SerializationError, ZcashDeserialize, ZcashSerialize,
};
@ -30,14 +30,14 @@ pub enum InventoryHash {
/// A hash of a transaction.
Tx(TransactionHash),
/// A hash of a block.
Block(BlockHeaderHash),
Block(block::Hash),
/// A hash of a filtered block.
///
/// The Bitcoin wiki says: Hash of a block header, but only to be used in
/// getdata message. Indicates the reply should be a merkleblock message
/// rather than a block message; this only works if a bloom filter has been
/// set.
FilteredBlock(BlockHeaderHash),
FilteredBlock(block::Hash),
}
impl From<TransactionHash> for InventoryHash {
@ -46,11 +46,11 @@ impl From<TransactionHash> for InventoryHash {
}
}
impl From<BlockHeaderHash> for InventoryHash {
fn from(block: BlockHeaderHash) -> InventoryHash {
impl From<block::Hash> for InventoryHash {
fn from(hash: block::Hash) -> InventoryHash {
// Auto-convert to Block rather than FilteredBlock because filtered
// blocks aren't useful for Zcash.
InventoryHash::Block(block)
InventoryHash::Block(hash)
}
}
@ -75,8 +75,8 @@ impl ZcashDeserialize for InventoryHash {
match code {
0 => Ok(InventoryHash::Error),
1 => Ok(InventoryHash::Tx(TransactionHash(bytes))),
2 => Ok(InventoryHash::Block(BlockHeaderHash(bytes))),
3 => Ok(InventoryHash::FilteredBlock(BlockHeaderHash(bytes))),
2 => Ok(InventoryHash::Block(block::Hash(bytes))),
3 => Ok(InventoryHash::FilteredBlock(block::Hash(bytes))),
_ => Err(SerializationError::Parse("invalid inventory code")),
}
}

View File

@ -5,8 +5,8 @@ use std::{net, sync::Arc};
use chrono::{DateTime, Utc};
use zebra_chain::block::{Block, BlockHeader, BlockHeaderHash};
use zebra_chain::{block::BlockHeight, transaction::Transaction};
use zebra_chain::block::{Block, self};
use zebra_chain::{block::BlockHeight, block::BlockHeader, transaction::Transaction};
use super::inv::InventoryHash;
use super::types::*;
@ -161,12 +161,12 @@ pub enum Message {
// many results.
GetBlocks {
/// Block locators, from newest back to genesis block.
block_locator_hashes: Vec<BlockHeaderHash>,
block_locator_hashes: Vec<block::Hash>,
/// `BlockHeaderHash` of the last desired block.
///
/// Set to zero to get as many blocks as possible (500).
hash_stop: BlockHeaderHash,
hash_stop: block::Hash,
},
/// A `headers` message.
@ -201,12 +201,12 @@ pub enum Message {
// many results.
GetHeaders {
/// Block locators, from newest back to genesis block.
block_locator_hashes: Vec<BlockHeaderHash>,
block_locator_hashes: Vec<block::Hash>,
/// `BlockHeaderHash` of the last desired block header.
/// `block::Hash` of the last desired block header.
///
/// Set to zero to get as many block headers as possible (2000).
hash_stop: BlockHeaderHash,
hash_stop: block::Hash,
},
/// An `inv` message.

View File

@ -1,6 +1,6 @@
use std::collections::HashSet;
use zebra_chain::block::BlockHeaderHash;
use zebra_chain::block;
use super::super::types::Nonce;
@ -42,7 +42,7 @@ pub enum Request {
/// # Returns
///
/// Returns [`Response::Blocks`](super::Response::Blocks).
BlocksByHash(HashSet<BlockHeaderHash>),
BlocksByHash(HashSet<block::Hash>),
/// Request block hashes of subsequent blocks in the chain, giving hashes of
/// known blocks.
@ -50,7 +50,7 @@ pub enum Request {
/// # Returns
///
/// Returns
/// [`Response::BlockHeaderHashes`](super::Response::BlockHeaderHashes).
/// [`Response::BlockHashes`](super::Response::BlockHashes).
///
/// # Warning
///
@ -65,8 +65,8 @@ pub enum Request {
/// `inv` messages will always have exactly one block hash.
FindBlocks {
/// Hashes of known blocks, ordered from highest height to lowest height.
known_blocks: Vec<BlockHeaderHash>,
known_blocks: Vec<block::Hash>,
/// Optionally, the last header to request.
stop: Option<BlockHeaderHash>,
stop: Option<block::Hash>,
},
}

View File

@ -1,5 +1,4 @@
// XXX clean module layout of zebra_chain
use zebra_chain::block::{Block, BlockHeaderHash};
use zebra_chain::block::{Block, self};
use crate::meta_addr::MetaAddr;
use std::sync::Arc;
@ -17,5 +16,5 @@ pub enum Response {
Blocks(Vec<Arc<Block>>),
/// A list of block hashes.
BlockHeaderHashes(Vec<BlockHeaderHash>),
BlockHashes(Vec<block::Hash>),
}

View File

@ -12,7 +12,7 @@ use std::{
task::{Context, Poll},
};
use tower::{buffer::Buffer, Service};
use zebra_chain::block::BlockHeaderHash;
use zebra_chain::block;
mod block_index;
@ -22,7 +22,7 @@ struct InMemoryState {
}
impl InMemoryState {
fn contains(&mut self, _hash: BlockHeaderHash) -> Result<Option<u32>, Error> {
fn contains(&mut self, _hash: block::Hash) -> Result<Option<u32>, Error> {
todo!()
}
}

View File

@ -5,19 +5,19 @@ use std::{
};
use zebra_chain::{
block::BlockHeight,
block::{Block, BlockHeaderHash},
block::{Block, self},
};
#[derive(Default)]
pub(super) struct BlockIndex {
by_hash: HashMap<BlockHeaderHash, Arc<Block>>,
height_map: BTreeMap<BlockHeight, BlockHeaderHash>,
by_hash: HashMap<block::Hash, Arc<Block>>,
height_map: BTreeMap<BlockHeight, block::Hash>,
}
impl BlockIndex {
pub(super) fn insert(
&mut self,
block: impl Into<Arc<Block>>,
) -> Result<BlockHeaderHash, Box<dyn Error + Send + Sync + 'static>> {
) -> Result<block::Hash, Box<dyn Error + Send + Sync + 'static>> {
let block = block.into();
let hash = block.as_ref().into();
let height = block.coinbase_height().unwrap();
@ -32,11 +32,11 @@ impl BlockIndex {
}
}
pub(super) fn get(&self, hash: BlockHeaderHash) -> Option<Arc<Block>> {
pub(super) fn get(&self, hash: block::Hash) -> Option<Arc<Block>> {
self.by_hash.get(&hash).cloned()
}
pub(super) fn get_main_chain_at(&self, height: BlockHeight) -> Option<BlockHeaderHash> {
pub(super) fn get_main_chain_at(&self, height: BlockHeight) -> Option<block::Hash> {
self.height_map.get(&height).cloned()
}

View File

@ -23,7 +23,7 @@ use tower::{Service, ServiceExt};
use zebra_chain::{
block::BlockHeight,
block::{Block, BlockHeaderHash},
block::{Block, self},
parameters::Network,
};
@ -113,19 +113,19 @@ pub enum Request {
/// Get a block from the zebra-state
GetBlock {
/// The hash used to identify the block
hash: BlockHeaderHash,
hash: block::Hash,
},
/// Get a block locator list for the current best chain
GetBlockLocator {
/// The genesis block of the current best chain
genesis: BlockHeaderHash,
genesis: block::Hash,
},
/// Get the block that is the tip of the current chain
GetTip,
/// Ask the state if the given hash is part of the current best chain
GetDepth {
/// The hash to check against the current chain
hash: BlockHeaderHash,
hash: block::Hash,
},
}
@ -136,7 +136,7 @@ pub enum Response {
/// added to the state
Added {
/// The hash of the block that was added
hash: BlockHeaderHash,
hash: block::Hash,
},
/// The response to a `GetBlock` request by hash
Block {
@ -146,12 +146,12 @@ pub enum Response {
/// The response to a `GetBlockLocator` request
BlockLocator {
/// The set of blocks that make up the block locator
block_locator: Vec<BlockHeaderHash>,
block_locator: Vec<block::Hash>,
},
/// The response to a `GetTip` request
Tip {
/// The hash of the block at the tip of the current chain
hash: BlockHeaderHash,
hash: block::Hash,
},
/// The response to a `Contains` request indicating that the given has is in
/// the current best chain

View File

@ -14,7 +14,7 @@ use tracing::instrument;
use zebra_chain::serialization::{SerializationError, ZcashDeserialize, ZcashSerialize};
use zebra_chain::{
block::BlockHeight,
block::{Block, BlockHeaderHash},
block::{Block, self},
parameters::Network,
};
@ -40,9 +40,9 @@ impl SledState {
pub(super) fn insert(
&mut self,
block: impl Into<Arc<Block>> + std::fmt::Debug,
) -> Result<BlockHeaderHash, Error> {
) -> Result<block::Hash, Error> {
let block = block.into();
let hash: BlockHeaderHash = block.as_ref().into();
let hash = block.hash();
let height = block.coinbase_height().unwrap();
let height_map = self.storage.open_tree(b"height_map")?;
@ -58,7 +58,7 @@ impl SledState {
}
#[instrument(skip(self))]
pub(super) fn get(&self, hash: BlockHeaderHash) -> Result<Option<Arc<Block>>, Error> {
pub(super) fn get(&self, hash: block::Hash) -> Result<Option<Arc<Block>>, Error> {
let by_hash = self.storage.open_tree(b"by_hash")?;
let key = &hash.0;
let value = by_hash.get(key)?;
@ -76,7 +76,7 @@ impl SledState {
pub(super) fn get_main_chain_at(
&self,
height: BlockHeight,
) -> Result<Option<BlockHeaderHash>, Error> {
) -> Result<Option<block::Hash>, Error> {
let height_map = self.storage.open_tree(b"height_map")?;
let key = height.0.to_be_bytes();
let value = height_map.get(key)?;
@ -91,7 +91,7 @@ impl SledState {
}
#[instrument(skip(self))]
pub(super) fn get_tip(&self) -> Result<Option<BlockHeaderHash>, Error> {
pub(super) fn get_tip(&self) -> Result<Option<block::Hash>, Error> {
let tree = self.storage.open_tree(b"height_map")?;
let last_entry = tree.iter().values().next_back();
@ -103,7 +103,7 @@ impl SledState {
}
#[instrument(skip(self))]
fn contains(&self, hash: &BlockHeaderHash) -> Result<bool, Error> {
fn contains(&self, hash: &block::Hash) -> Result<bool, Error> {
let by_hash = self.storage.open_tree(b"by_hash")?;
let key = &hash.0;
@ -230,12 +230,12 @@ impl AsRef<[u8]> for BytesHeight {
}
pub(super) enum BlockQuery {
ByHash(BlockHeaderHash),
ByHash(block::Hash),
ByHeight(BlockHeight),
}
impl From<BlockHeaderHash> for BlockQuery {
fn from(hash: BlockHeaderHash) -> Self {
impl From<block::Hash> for BlockQuery {
fn from(hash: block::Hash) -> Self {
Self::ByHash(hash)
}
}

View File

@ -17,7 +17,7 @@ use std::process::Stdio;
use structopt::StructOpt;
use tracing_subscriber::{layer::SubscriberExt, util::SubscriberInitExt};
use zebra_chain::block::BlockHeaderHash;
use zebra_chain::block;
use zebra_chain::block::BlockHeight;
#[cfg(unix)]
@ -140,7 +140,7 @@ fn main() -> Result<()> {
let v: Value = serde_json::from_str(&output)?;
// get the values we are interested in
let hash: BlockHeaderHash = v["hash"].as_str().map(byte_reverse_hex).unwrap().parse()?;
let hash: block::Hash = v["hash"].as_str().map(byte_reverse_hex).unwrap().parse()?;
let height = BlockHeight(v["height"].as_u64().unwrap() as u32);
assert!(height <= BlockHeight::MAX);
assert_eq!(x, height.0);

View File

@ -8,7 +8,7 @@ use tower::{builder::ServiceBuilder, retry::Retry, timeout::Timeout, Service, Se
use tracing_futures::Instrument;
use zebra_chain::{
block::{Block, BlockHeaderHash},
block::{Block, self},
parameters::Network,
};
use zebra_consensus::checkpoint;
@ -31,8 +31,8 @@ const SYNC_RESTART_TIMEOUT: Duration = Duration::from_secs(20);
/// the returned hashes actually extend a chain tip.
#[derive(Debug, Hash, PartialEq, Eq)]
struct CheckedTip {
tip: BlockHeaderHash,
expected_next: BlockHeaderHash,
tip: block::Hash,
expected_next: block::Hash,
}
#[derive(Debug)]
@ -42,7 +42,7 @@ where
ZN::Future: Send,
ZS: Service<zs::Request, Response = zs::Response, Error = Error> + Send + Clone + 'static,
ZS::Future: Send,
ZV: Service<Arc<Block>, Response = BlockHeaderHash, Error = Error> + Send + Clone + 'static,
ZV: Service<Arc<Block>, Response = block::Hash, Error = Error> + Send + Clone + 'static,
ZV::Future: Send,
{
/// Used to perform extendtips requests, with no retry logic (failover is handled using fanout).
@ -52,8 +52,8 @@ where
state: ZS,
verifier: ZV,
prospective_tips: HashSet<CheckedTip>,
pending_blocks: Pin<Box<FuturesUnordered<JoinHandle<Result<BlockHeaderHash, Error>>>>>,
genesis_hash: BlockHeaderHash,
pending_blocks: Pin<Box<FuturesUnordered<JoinHandle<Result<block::Hash, Error>>>>>,
genesis_hash: block::Hash,
}
impl<ZN, ZS, ZV> Syncer<ZN, ZS, ZV>
@ -62,7 +62,7 @@ where
ZN::Future: Send,
ZS: Service<zs::Request, Response = zs::Response, Error = Error> + Send + Clone + 'static,
ZS::Future: Send,
ZV: Service<Arc<Block>, Response = BlockHeaderHash, Error = Error> + Send + Clone + 'static,
ZV: Service<Arc<Block>, Response = block::Hash, Error = Error> + Send + Clone + 'static,
ZV::Future: Send,
{
/// Returns a new syncer instance, using:
@ -197,7 +197,7 @@ where
let mut download_set = HashSet::new();
while let Some(res) = requests.next().await {
match res.map_err::<Report, _>(|e| eyre!(e)) {
Ok(zn::Response::BlockHeaderHashes(hashes)) => {
Ok(zn::Response::BlockHashes(hashes)) => {
let mut first_unknown = None;
for (i, &hash) in hashes.iter().enumerate() {
if !self.state_contains(hash).await? {
@ -277,7 +277,7 @@ where
}
while let Some(res) = responses.next().await {
match res.map_err::<Report, _>(|e| eyre!(e)) {
Ok(zn::Response::BlockHeaderHashes(hashes)) => {
Ok(zn::Response::BlockHashes(hashes)) => {
tracing::debug!(first = ?hashes.first(), len = ?hashes.len());
let unknown_hashes = match hashes.split_first() {
@ -362,7 +362,7 @@ where
}
/// Queue downloads for each block that isn't currently known to our node
async fn request_blocks(&mut self, hashes: Vec<BlockHeaderHash>) -> Result<(), Report> {
async fn request_blocks(&mut self, hashes: Vec<block::Hash>) -> Result<(), Report> {
tracing::debug!(hashes.len = hashes.len(), "requesting blocks");
for hash in hashes.into_iter() {
// TODO: remove this check once the sync service is more reliable
@ -423,7 +423,7 @@ where
///
/// TODO: handle multiple tips in the state.
#[instrument(skip(self))]
async fn state_contains(&mut self, hash: BlockHeaderHash) -> Result<bool, Report> {
async fn state_contains(&mut self, hash: block::Hash) -> Result<bool, Report> {
match self
.state
.ready_and()