Create a `scan_block` function to use across scanning tasks (#7994)

* Wrap `zcash_client_backend::scanning::scan_block`

* Use the new `scan_block` fn

* Use full path for `zcash::primitives::Network`

* Add docs for `scan_block`

* Impl `From` for networks in `zcash_primitives`

* Update zebra-scan/src/tests.rs

Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com>

* Move code from `tests.rs` to `scan.rs`

---------

Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com>
This commit is contained in:
Marek 2023-11-28 22:58:18 +01:00 committed by GitHub
parent 1d241afbaa
commit e00a762856
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 197 additions and 180 deletions

View File

@ -336,3 +336,12 @@ pub(crate) fn transparent_output_address(
None => None, None => None,
} }
} }
impl From<Network> for zcash_primitives::consensus::Network {
fn from(network: Network) -> Self {
match network {
Network::Mainnet => zcash_primitives::consensus::Network::MainNetwork,
Network::Testnet => zcash_primitives::consensus::Network::TestNetwork,
}
}
}

View File

@ -22,6 +22,9 @@ categories = ["cryptography::cryptocurrencies"]
zebra-chain = { path = "../zebra-chain", version = "1.0.0-beta.31" } zebra-chain = { path = "../zebra-chain", version = "1.0.0-beta.31" }
zebra-state = { path = "../zebra-state", version = "1.0.0-beta.31" } zebra-state = { path = "../zebra-state", version = "1.0.0-beta.31" }
zcash_primitives = "0.13.0-rc.1"
zcash_client_backend = "0.10.0-rc.1"
color-eyre = "0.6.2" color-eyre = "0.6.2"
indexmap = { version = "2.0.1", features = ["serde"] } indexmap = { version = "2.0.1", features = ["serde"] }
serde = { version = "1.0.193", features = ["serde_derive"] } serde = { version = "1.0.193", features = ["serde_derive"] }
@ -30,9 +33,6 @@ tower = "0.4.13"
tracing = "0.1.39" tracing = "0.1.39"
[dev-dependencies] [dev-dependencies]
zcash_client_backend = "0.10.0-rc.1"
zcash_primitives = "0.13.0-rc.1"
zcash_note_encryption = "0.4.0" zcash_note_encryption = "0.4.0"
rand = "0.8.5" rand = "0.8.5"

View File

@ -1,10 +1,21 @@
//! The scan task. //! The scan task.
use std::time::Duration; use std::{sync::Arc, time::Duration};
use color_eyre::{eyre::eyre, Report}; use color_eyre::{eyre::eyre, Report};
use tower::{buffer::Buffer, util::BoxService, Service, ServiceExt}; use tower::{buffer::Buffer, util::BoxService, Service, ServiceExt};
use tracing::info; use tracing::info;
use zcash_client_backend::{
data_api::ScannedBlock,
proto::compact_formats::{
ChainMetadata, CompactBlock, CompactSaplingOutput, CompactSaplingSpend, CompactTx,
},
scanning::{ScanError, ScanningKey},
};
use zcash_primitives::zip32::AccountId;
use zebra_chain::{
block::Block, parameters::Network, serialization::ZcashSerialize, transaction::Transaction,
};
use crate::storage::Storage; use crate::storage::Storage;
@ -56,3 +67,128 @@ pub async fn start(mut state: State, storage: Storage) -> Result<(), Report> {
tokio::time::sleep(CHECK_INTERVAL).await; tokio::time::sleep(CHECK_INTERVAL).await;
} }
} }
/// Returns transactions belonging to any of the given [`ScanningKey`]s.
///
/// TODO:
/// - Remove the `sapling_tree_size` parameter or turn it into an `Option` once we have access to
/// Zebra's state, and we can retrieve the tree size ourselves.
/// - Add prior block metadata once we have access to Zebra's state.
pub fn scan_block<K: ScanningKey>(
network: Network,
block: Arc<Block>,
sapling_tree_size: u32,
scanning_keys: &[&K],
) -> Result<ScannedBlock<K::Nf>, ScanError> {
// TODO: Implement a check that returns early when the block height is below the Sapling
// activation height.
let network: zcash_primitives::consensus::Network = network.into();
let chain_metadata = ChainMetadata {
sapling_commitment_tree_size: sapling_tree_size,
// Orchard is not supported at the moment so the tree size can be 0.
orchard_commitment_tree_size: 0,
};
// Use a dummy `AccountId` as we don't use accounts yet.
let dummy_account = AccountId::from(0);
let scanning_keys: Vec<_> = scanning_keys
.iter()
.map(|key| (&dummy_account, key))
.collect();
zcash_client_backend::scanning::scan_block(
&network,
block_to_compact(block, chain_metadata),
&scanning_keys,
// Ignore whether notes are change from a viewer's own spends for now.
&[],
// Ignore previous blocks for now.
None,
)
}
/// Converts a zebra block and meta data into a compact block.
pub fn block_to_compact(block: Arc<Block>, chain_metadata: ChainMetadata) -> CompactBlock {
CompactBlock {
height: block
.coinbase_height()
.expect("verified block should have a valid height")
.0
.into(),
hash: block.hash().bytes_in_display_order().to_vec(),
prev_hash: block
.header
.previous_block_hash
.bytes_in_display_order()
.to_vec(),
time: block
.header
.time
.timestamp()
.try_into()
.expect("unsigned 32-bit times should work until 2105"),
header: block
.header
.zcash_serialize_to_vec()
.expect("verified block should serialize"),
vtx: block
.transactions
.iter()
.cloned()
.enumerate()
.map(transaction_to_compact)
.collect(),
chain_metadata: Some(chain_metadata),
// The protocol version is used for the gRPC wire format, so it isn't needed here.
proto_version: 0,
}
}
/// Converts a zebra transaction into a compact transaction.
fn transaction_to_compact((index, tx): (usize, Arc<Transaction>)) -> CompactTx {
CompactTx {
index: index
.try_into()
.expect("tx index in block should fit in u64"),
hash: tx.hash().bytes_in_display_order().to_vec(),
// `fee` is not checked by the `scan_block` function. It is allowed to be unset.
// <https://docs.rs/zcash_client_backend/latest/zcash_client_backend/proto/compact_formats/struct.CompactTx.html#structfield.fee>
fee: 0,
spends: tx
.sapling_nullifiers()
.map(|nf| CompactSaplingSpend {
nf: <[u8; 32]>::from(*nf).to_vec(),
})
.collect(),
// > output encodes the cmu field, ephemeralKey field, and a 52-byte prefix of the encCiphertext field of a Sapling Output
//
// <https://docs.rs/zcash_client_backend/latest/zcash_client_backend/proto/compact_formats/struct.CompactSaplingOutput.html>
outputs: tx
.sapling_outputs()
.map(|output| CompactSaplingOutput {
cmu: output.cm_u.to_bytes().to_vec(),
ephemeral_key: output
.ephemeral_key
.zcash_serialize_to_vec()
.expect("verified output should serialize successfully"),
ciphertext: output
.enc_ciphertext
.zcash_serialize_to_vec()
.expect("verified output should serialize successfully")
.into_iter()
.take(52)
.collect(),
})
.collect(),
// `actions` is not checked by the `scan_block` function.
actions: vec![],
}
}

View File

@ -6,18 +6,15 @@
use std::sync::Arc; use std::sync::Arc;
use zcash_client_backend::{ use zcash_client_backend::{
data_api::BlockMetadata,
encoding::decode_extended_full_viewing_key, encoding::decode_extended_full_viewing_key,
proto::compact_formats::{ proto::compact_formats::{
self as compact, ChainMetadata, CompactBlock, CompactSaplingOutput, CompactSaplingSpend, ChainMetadata, CompactBlock, CompactSaplingOutput, CompactSaplingSpend, CompactTx,
CompactTx,
}, },
scanning::scan_block,
}; };
use zcash_note_encryption::Domain; use zcash_note_encryption::Domain;
use zcash_primitives::{ use zcash_primitives::{
block::BlockHash, block::BlockHash,
consensus::{BlockHeight, Network}, consensus::BlockHeight,
constants::{mainnet::HRP_SAPLING_EXTENDED_FULL_VIEWING_KEY, SPENDING_KEY_GENERATOR}, constants::{mainnet::HRP_SAPLING_EXTENDED_FULL_VIEWING_KEY, SPENDING_KEY_GENERATOR},
memo::MemoBytes, memo::MemoBytes,
sapling::{ sapling::{
@ -35,24 +32,21 @@ use rand::{rngs::OsRng, RngCore};
use ff::{Field, PrimeField}; use ff::{Field, PrimeField};
use group::GroupEncoding; use group::GroupEncoding;
use zebra_chain::{ use zebra_chain::{
block::Block, block::Block, chain_tip::ChainTip, parameters::Network, serialization::ZcashDeserializeInto,
chain_tip::ChainTip, transaction::Hash,
serialization::{ZcashDeserializeInto, ZcashSerialize},
transaction::{Hash, Transaction},
}; };
/// Prove that Zebra blocks can be scanned using the `zcash_client_backend::scanning::scan_block` function: use crate::scan::{block_to_compact, scan_block};
/// - Populates the state with a continuous chain of mainnet blocks from genesis.
/// - Scan the chain from the tip going backwards down to genesis. /// Scans a continuous chain of Mainnet blocks from tip to genesis.
/// - Verifies that no relevant transaction is found in the chain when scanning for a fake account's nullifier. ///
/// Also verifies that no relevant transaction is found in the chain when scanning for a fake
/// account's nullifier.
#[tokio::test] #[tokio::test]
async fn scanning_from_populated_zebra_state() -> Result<()> { async fn scanning_from_populated_zebra_state() -> Result<()> {
let account = AccountId::from(12); let network = Network::default();
let vks: Vec<(&AccountId, &SaplingIvk)> = vec![];
let nf = Nullifier([7; 32]);
let network = zebra_chain::parameters::Network::default();
// Create a continuous chain of mainnet blocks from genesis // Create a continuous chain of mainnet blocks from genesis
let blocks: Vec<Arc<Block>> = zebra_test::vectors::CONTINUOUS_MAINNET_BLOCKS let blocks: Vec<Arc<Block>> = zebra_test::vectors::CONTINUOUS_MAINNET_BLOCKS
@ -75,41 +69,32 @@ async fn scanning_from_populated_zebra_state() -> Result<()> {
// TODO: Accessing the state database directly is ok in the tests, but not in production code. // TODO: Accessing the state database directly is ok in the tests, but not in production code.
// Use `Request::Block` if the code is copied to production. // Use `Request::Block` if the code is copied to production.
while let Some(block) = db.block(height.into()) { while let Some(block) = db.block(height.into()) {
// We fake the sapling tree size to 1 because we are not in Sapling heights. // We use a dummy size of the Sapling note commitment tree. We can't set the size to zero
let sapling_tree_size = 1; // because the underlying scanning function would return
let orchard_tree_size = db // `zcash_client_backeng::scanning::ScanError::TreeSizeUnknown`.
.orchard_tree_by_hash_or_height(height.into()) let sapling_commitment_tree_size = 1;
.expect("each state block must have a sapling tree")
.count(); let orchard_commitment_tree_size = 0;
let chain_metadata = ChainMetadata { let chain_metadata = ChainMetadata {
sapling_commitment_tree_size: sapling_tree_size sapling_commitment_tree_size,
.try_into() orchard_commitment_tree_size,
.expect("sapling position is limited to u32::MAX"),
orchard_commitment_tree_size: orchard_tree_size
.try_into()
.expect("orchard position is limited to u32::MAX"),
}; };
let compact_block = block_to_compact(block, chain_metadata); let compact_block = block_to_compact(block.clone(), chain_metadata);
let res = scan_block( let res =
&zcash_primitives::consensus::MainNetwork, scan_block::<SaplingIvk>(network, block, sapling_commitment_tree_size, &[]).unwrap();
compact_block.clone(),
&vks[..],
&[(account, nf)],
None,
)
.unwrap();
transactions_found += res.transactions().len(); transactions_found += res.transactions().len();
transactions_scanned += compact_block.vtx.len(); transactions_scanned += compact_block.vtx.len();
blocks_scanned += 1; blocks_scanned += 1;
// scan backwards
if height.is_min() { if height.is_min() {
break; break;
} }
// scan backwards
height = height.previous()?; height = height.previous()?;
} }
@ -150,7 +135,7 @@ async fn scanning_from_fake_generated_blocks() -> Result<()> {
// The fake block function will have our transaction and a random one. // The fake block function will have our transaction and a random one.
assert_eq!(cb.vtx.len(), 2); assert_eq!(cb.vtx.len(), 2);
let res = scan_block( let res = zcash_client_backend::scanning::scan_block(
&zcash_primitives::consensus::MainNetwork, &zcash_primitives::consensus::MainNetwork,
cb.clone(), cb.clone(),
&vks[..], &vks[..],
@ -185,12 +170,9 @@ async fn scanning_zecpages_from_populated_zebra_state() -> Result<()> {
) )
.unwrap(); .unwrap();
let account = AccountId::from(1);
// Build a vector of viewing keys `vks` to scan for. // Build a vector of viewing keys `vks` to scan for.
let fvk = efvk.fvk; let fvk = efvk.fvk;
let ivk = fvk.vk.ivk(); let ivk = fvk.vk.ivk();
let vks: Vec<(&AccountId, &SaplingIvk)> = vec![(&account, &ivk)];
let network = zebra_chain::parameters::Network::Mainnet; let network = zebra_chain::parameters::Network::Mainnet;
@ -213,54 +195,22 @@ async fn scanning_zecpages_from_populated_zebra_state() -> Result<()> {
let mut transactions_scanned = 0; let mut transactions_scanned = 0;
let mut blocks_scanned = 0; let mut blocks_scanned = 0;
while let Some(block) = db.block(height.into()) { while let Some(block) = db.block(height.into()) {
// zcash_client_backend doesn't support scanning the genesis block, but that's ok, because // We use a dummy size of the Sapling note commitment tree. We can't set the size to zero
// Sapling activates at height 419,200. So we'll never scan these blocks in production code. // because the underlying scanning function would return
let sapling_tree_size = if height.is_min() { // `zcash_client_backeng::scanning::ScanError::TreeSizeUnknown`.
1 let sapling_commitment_tree_size = 1;
} else {
db.sapling_tree_by_hash_or_height(height.into())
.expect("each state block must have a sapling tree")
.count()
};
let orchard_tree_size = db let orchard_commitment_tree_size = 0;
.orchard_tree_by_hash_or_height(height.into())
.expect("each state block must have a orchard tree")
.count();
let chain_metadata = ChainMetadata { let chain_metadata = ChainMetadata {
sapling_commitment_tree_size: sapling_tree_size sapling_commitment_tree_size,
.try_into() orchard_commitment_tree_size,
.expect("sapling position is limited to u32::MAX"),
orchard_commitment_tree_size: orchard_tree_size
.try_into()
.expect("orchard position is limited to u32::MAX"),
}; };
let block_metadata = if height.is_min() { let compact_block = block_to_compact(block.clone(), chain_metadata);
None
} else {
Some(BlockMetadata::from_parts(
height.previous()?.0.into(),
BlockHash(block.header.previous_block_hash.0),
db.sapling_tree_by_hash_or_height(block.header.previous_block_hash.into())
.expect("each state block must have a sapling tree")
.count()
.try_into()
.expect("sapling position is limited to u32::MAX"),
))
};
let compact_block = block_to_compact(block, chain_metadata); let res = scan_block(network, block, sapling_commitment_tree_size, &[&ivk])
.expect("scanning block for the ZECpages viewing key should work");
let res = scan_block(
&zcash_primitives::consensus::MainNetwork,
compact_block.clone(),
&vks[..],
&[],
block_metadata.as_ref(),
)
.expect("scanning block for the ZECpages viewing key should work");
transactions_found += res.transactions().len(); transactions_found += res.transactions().len();
transactions_scanned += compact_block.vtx.len(); transactions_scanned += compact_block.vtx.len();
@ -322,7 +272,7 @@ async fn scanning_fake_blocks_store_key_and_results() -> Result<()> {
); );
// Scan with our key // Scan with our key
let res = scan_block( let res = zcash_client_backend::scanning::scan_block(
&zcash_primitives::consensus::MainNetwork, &zcash_primitives::consensus::MainNetwork,
cb.clone(), cb.clone(),
&vks[..], &vks[..],
@ -347,89 +297,6 @@ async fn scanning_fake_blocks_store_key_and_results() -> Result<()> {
Ok(()) Ok(())
} }
/// Convert a zebra block and meta data into a compact block.
fn block_to_compact(block: Arc<Block>, chain_metadata: ChainMetadata) -> CompactBlock {
CompactBlock {
height: block
.coinbase_height()
.expect("verified block should have a valid height")
.0
.into(),
hash: block.hash().bytes_in_display_order().to_vec(),
prev_hash: block
.header
.previous_block_hash
.bytes_in_display_order()
.to_vec(),
time: block
.header
.time
.timestamp()
.try_into()
.expect("unsigned 32-bit times should work until 2105"),
header: block
.header
.zcash_serialize_to_vec()
.expect("verified block should serialize"),
vtx: block
.transactions
.iter()
.cloned()
.enumerate()
.map(transaction_to_compact)
.collect(),
chain_metadata: Some(chain_metadata),
// The protocol version is used for the gRPC wire format, so it isn't needed here.
proto_version: 0,
}
}
/// Convert a zebra transaction into a compact transaction.
fn transaction_to_compact((index, tx): (usize, Arc<Transaction>)) -> CompactTx {
CompactTx {
index: index
.try_into()
.expect("tx index in block should fit in u64"),
hash: tx.hash().bytes_in_display_order().to_vec(),
// `fee` is not checked by the `scan_block` function. It is allowed to be unset.
// <https://docs.rs/zcash_client_backend/latest/zcash_client_backend/proto/compact_formats/struct.CompactTx.html#structfield.fee>
fee: 0,
spends: tx
.sapling_nullifiers()
.map(|nf| CompactSaplingSpend {
nf: <[u8; 32]>::from(*nf).to_vec(),
})
.collect(),
// > output encodes the cmu field, ephemeralKey field, and a 52-byte prefix of the encCiphertext field of a Sapling Output
//
// <https://docs.rs/zcash_client_backend/latest/zcash_client_backend/proto/compact_formats/struct.CompactSaplingOutput.html>
outputs: tx
.sapling_outputs()
.map(|output| CompactSaplingOutput {
cmu: output.cm_u.to_bytes().to_vec(),
ephemeral_key: output
.ephemeral_key
.zcash_serialize_to_vec()
.expect("verified output should serialize successfully"),
ciphertext: output
.enc_ciphertext
.zcash_serialize_to_vec()
.expect("verified output should serialize successfully")
.into_iter()
.take(52)
.collect(),
})
.collect(),
// `actions` is not checked by the `scan_block` function.
actions: vec![],
}
}
/// Create a fake compact block with provided fake account data. /// Create a fake compact block with provided fake account data.
// This is a copy of zcash_primitives `fake_compact_block` where the `value` argument was changed to // This is a copy of zcash_primitives `fake_compact_block` where the `value` argument was changed to
// be a number for easier conversion: // be a number for easier conversion:
@ -448,18 +315,23 @@ fn fake_compact_block(
// Create a fake Note for the account // Create a fake Note for the account
let mut rng = OsRng; let mut rng = OsRng;
let rseed = generate_random_rseed(&Network::TestNetwork, height, &mut rng); let rseed = generate_random_rseed(
&zcash_primitives::consensus::Network::TestNetwork,
height,
&mut rng,
);
let note = Note::from_parts(to, NoteValue::from_raw(value), rseed); let note = Note::from_parts(to, NoteValue::from_raw(value), rseed);
let encryptor = sapling_note_encryption::<_, Network>( let encryptor = sapling_note_encryption::<_, zcash_primitives::consensus::Network>(
Some(dfvk.fvk().ovk), Some(dfvk.fvk().ovk),
note.clone(), note.clone(),
MemoBytes::empty(), MemoBytes::empty(),
&mut rng, &mut rng,
); );
let cmu = note.cmu().to_bytes().to_vec(); let cmu = note.cmu().to_bytes().to_vec();
let ephemeral_key = SaplingDomain::<Network>::epk_bytes(encryptor.epk()) let ephemeral_key =
.0 SaplingDomain::<zcash_primitives::consensus::Network>::epk_bytes(encryptor.epk())
.to_vec(); .0
.to_vec();
let enc_ciphertext = encryptor.encrypt_note_plaintext(); let enc_ciphertext = encryptor.encrypt_note_plaintext();
// Create a fake CompactBlock containing the note // Create a fake CompactBlock containing the note
@ -503,7 +375,7 @@ fn fake_compact_block(
cb.vtx.push(tx); cb.vtx.push(tx);
} }
cb.chain_metadata = initial_sapling_tree_size.map(|s| compact::ChainMetadata { cb.chain_metadata = initial_sapling_tree_size.map(|s| ChainMetadata {
sapling_commitment_tree_size: s + cb sapling_commitment_tree_size: s + cb
.vtx .vtx
.iter() .iter()