zebra/zebrad/tests/common/lightwalletd/wallet_grpc_test.rs

473 lines
16 KiB
Rust
Raw Normal View History

//! Test all gRPC calls a wallet connected to a lightwalletd instance backed by
//! zebra can do.
//!
//! This test requires a cached chain state that is partially synchronized,
//! i.e., it should be a few blocks below the network chain tip height. It also
//! requires a lightwalletd data dir in sync with the cached chain state.
//!
//! Current coverage of all available rpc methods according to
//! `CompactTxStreamer`:
//!
//! - `GetLatestBlock`: Covered.
//! - `GetBlock`: Covered.
//! - `GetBlockRange`: Covered.
//!
//! - `GetTransaction`: Covered.
add(test): test disabled `lightwalletd` mempool gRPCs via zebrad logs (#5016) * add grpc mempool test research * add a config flag for mempool injection of transactions in test * Only copy the inner state directory in the send transactions test * Preload Zcash parameters in some transaction verification tests * Add a block and transaction Hash method to convert from display order bytes * Update test coverage docs * Add debugging output for mempool transaction verification * Test fetching sent mempool transactions using gRPC * Add extra log checks to the send transaction test * Wait for zebrad mempool activation before running gRPC tests * Update send transaction test for lightwalletd not returning mempool transactions * Check zebrad logs instead of disabled lightwalletd gRPCs * Add a debug option that makes RPCs pretend the sync is finished * Remove an unused debug option * Remove unused test code and downgrade some logs * Fix test log checks * Fix some rustdoc warnings * Fix a compilation error due to new function arguments * Make zebrad sync timeouts consistent and remove outdated code * Document how to increase temporary directory space for tests * Stop checking for a log that doesn't always happen * Remove some commented-out code Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com> * Update a comment about run time Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com> * Add new config to new tests from the `main` branch * Add transactions to the list, rather than replacing the list with each new block Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com>
2022-09-06 06:32:33 -07:00
//! - `SendTransaction`: Covered by the send_transaction_test.
//!
//! - `GetTaddressTxids`: Covered.
//! - `GetTaddressBalance`: Covered.
//! - `GetTaddressBalanceStream`: Covered.
//!
add(test): test disabled `lightwalletd` mempool gRPCs via zebrad logs (#5016) * add grpc mempool test research * add a config flag for mempool injection of transactions in test * Only copy the inner state directory in the send transactions test * Preload Zcash parameters in some transaction verification tests * Add a block and transaction Hash method to convert from display order bytes * Update test coverage docs * Add debugging output for mempool transaction verification * Test fetching sent mempool transactions using gRPC * Add extra log checks to the send transaction test * Wait for zebrad mempool activation before running gRPC tests * Update send transaction test for lightwalletd not returning mempool transactions * Check zebrad logs instead of disabled lightwalletd gRPCs * Add a debug option that makes RPCs pretend the sync is finished * Remove an unused debug option * Remove unused test code and downgrade some logs * Fix test log checks * Fix some rustdoc warnings * Fix a compilation error due to new function arguments * Make zebrad sync timeouts consistent and remove outdated code * Document how to increase temporary directory space for tests * Stop checking for a log that doesn't always happen * Remove some commented-out code Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com> * Update a comment about run time Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com> * Add new config to new tests from the `main` branch * Add transactions to the list, rather than replacing the list with each new block Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com>
2022-09-06 06:32:33 -07:00
//! - `GetMempoolTx`: Covered by the send_transaction_test,
//! currently disabled by `lightwalletd`.
//! - `GetMempoolStream`: Covered by the send_transaction_test,
//! currently disabled by `lightwalletd`.
//!
//! - `GetTreeState`: Covered.
//!
//! - `GetAddressUtxos`: Covered.
//! - `GetAddressUtxosStream`: Covered.
//!
//! - `GetLightdInfo`: Covered.
add(test): test disabled `lightwalletd` mempool gRPCs via zebrad logs (#5016) * add grpc mempool test research * add a config flag for mempool injection of transactions in test * Only copy the inner state directory in the send transactions test * Preload Zcash parameters in some transaction verification tests * Add a block and transaction Hash method to convert from display order bytes * Update test coverage docs * Add debugging output for mempool transaction verification * Test fetching sent mempool transactions using gRPC * Add extra log checks to the send transaction test * Wait for zebrad mempool activation before running gRPC tests * Update send transaction test for lightwalletd not returning mempool transactions * Check zebrad logs instead of disabled lightwalletd gRPCs * Add a debug option that makes RPCs pretend the sync is finished * Remove an unused debug option * Remove unused test code and downgrade some logs * Fix test log checks * Fix some rustdoc warnings * Fix a compilation error due to new function arguments * Make zebrad sync timeouts consistent and remove outdated code * Document how to increase temporary directory space for tests * Stop checking for a log that doesn't always happen * Remove some commented-out code Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com> * Update a comment about run time Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com> * Add new config to new tests from the `main` branch * Add transactions to the list, rather than replacing the list with each new block Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com>
2022-09-06 06:32:33 -07:00
//!
//! - `Ping`: Not covered and it will never be. `Ping` is only used for testing
//! purposes.
use color_eyre::eyre::Result;
use hex_literal::hex;
use zebra_chain::{
2022-05-18 19:30:34 -07:00
block::Block,
parameters::Network,
parameters::NetworkUpgrade::{Nu5, Sapling},
serialization::ZcashDeserializeInto,
};
fix(state): Use correct end heights for end of block subtrees during the full sync (#7566) * Avoid manual handling of previous sapling trees by using iterator windows instead * Avoid manual sapling subtree index handling by comparing prev and current subtree indexes instead * Simplify adding notes by using the exact number of remaining notes * Simplify by skipping the first block, because it can't complete a subtree * Re-use existing tree update code * Apply the sapling changes to orchard subtree updates * add a reverse database column family iterator function * Make skipping the lowest tree independent of iteration order * Move new subtree checks into the iterator, rename to end_height * Split subtree calculation into a new method * Split the calculate and write methods * Quickly check the first subtree before running the full upgrade * Do the quick checks every time Zebra runs, and refactor slow check error handling * Do quick checks for orchard as well * Make orchard tree upgrade match sapling upgrade code * Upgrade subtrees in reverse height order * Bump the database patch version so the upgrade runs again * Reset previous subtree upgrade data before doing this one * Add extra checks to subtree calculation to diagnose errors * Use correct heights for subtrees completed at the end of a block * Add even more checks to diagnose issues * Instrument upgrade methods to improve diagnostics * Prevent modification of re-used trees * Debug with subtree positions as well * Fix an off-by-one error with completed subtrees * Fix typos and confusing comments Co-authored-by: Marek <mail@marek.onl> * Fix mistaken previous tree handling and end tree comments * Remove unnecessary subtraction in remaining leaves calc * Log heights when assertions fail * Fix new subtree detection filter * Move new subtree check into a method, cleanup unused code * Remove redundant assertions * Wait for subtree upgrade before testing RPCs * Fix subtree search in quick check * Temporarily upgrade subtrees in forward height order * Clarify some comments * Fix missing test imports * Fix subtree logging * Add a comment about a potential hang with future upgrades * Fix zebrad var ownership * Log more info when add_subtrees.rs fails * cargo fmt --all * Fix unrelated clippy::unnecessary_unwrap * cargo clippy --fix --all-features --all-targets; cargo fmt --all * Stop the quick check depending on tree de-duplication * Refactor waiting for the upgrade into functions * Wait for state upgrades whenever the cached state is updated * Wait for the testnet upgrade in the right place * Fix unused variable * Fix a subtree detection bug and comments * Remove an early reference to reverse direction * Stop skipping subtrees completed at the end of blocks * Actually fix new subtree code --------- Co-authored-by: Marek <mail@marek.onl>
2023-09-19 07:49:36 -07:00
use zebra_state::latest_version_for_adding_subtrees;
use crate::common::{
fix(state): Use correct end heights for end of block subtrees during the full sync (#7566) * Avoid manual handling of previous sapling trees by using iterator windows instead * Avoid manual sapling subtree index handling by comparing prev and current subtree indexes instead * Simplify adding notes by using the exact number of remaining notes * Simplify by skipping the first block, because it can't complete a subtree * Re-use existing tree update code * Apply the sapling changes to orchard subtree updates * add a reverse database column family iterator function * Make skipping the lowest tree independent of iteration order * Move new subtree checks into the iterator, rename to end_height * Split subtree calculation into a new method * Split the calculate and write methods * Quickly check the first subtree before running the full upgrade * Do the quick checks every time Zebra runs, and refactor slow check error handling * Do quick checks for orchard as well * Make orchard tree upgrade match sapling upgrade code * Upgrade subtrees in reverse height order * Bump the database patch version so the upgrade runs again * Reset previous subtree upgrade data before doing this one * Add extra checks to subtree calculation to diagnose errors * Use correct heights for subtrees completed at the end of a block * Add even more checks to diagnose issues * Instrument upgrade methods to improve diagnostics * Prevent modification of re-used trees * Debug with subtree positions as well * Fix an off-by-one error with completed subtrees * Fix typos and confusing comments Co-authored-by: Marek <mail@marek.onl> * Fix mistaken previous tree handling and end tree comments * Remove unnecessary subtraction in remaining leaves calc * Log heights when assertions fail * Fix new subtree detection filter * Move new subtree check into a method, cleanup unused code * Remove redundant assertions * Wait for subtree upgrade before testing RPCs * Fix subtree search in quick check * Temporarily upgrade subtrees in forward height order * Clarify some comments * Fix missing test imports * Fix subtree logging * Add a comment about a potential hang with future upgrades * Fix zebrad var ownership * Log more info when add_subtrees.rs fails * cargo fmt --all * Fix unrelated clippy::unnecessary_unwrap * cargo clippy --fix --all-features --all-targets; cargo fmt --all * Stop the quick check depending on tree de-duplication * Refactor waiting for the upgrade into functions * Wait for state upgrades whenever the cached state is updated * Wait for the testnet upgrade in the right place * Fix unused variable * Fix a subtree detection bug and comments * Remove an early reference to reverse direction * Stop skipping subtrees completed at the end of blocks * Actually fix new subtree code --------- Co-authored-by: Marek <mail@marek.onl>
2023-09-19 07:49:36 -07:00
cached_state::{wait_for_state_version_message, wait_for_state_version_upgrade},
launch::spawn_zebrad_for_rpc,
lightwalletd::{
can_spawn_lightwalletd_for_rpc, spawn_lightwalletd_for_rpc,
sync::wait_for_zebrad_and_lightwalletd_sync,
wallet_grpc::{
connect_to_lightwalletd, Address, AddressList, BlockId, BlockRange, ChainSpec, Empty,
GetAddressUtxosArg, GetSubtreeRootsArg, ShieldedProtocol,
TransparentAddressBlockFilter, TxFilter,
},
},
test_type::TestType::UpdateCachedState,
};
/// The test entry point.
pub async fn run() -> Result<()> {
let _init_guard = zebra_test::init();
// We want a zebra state dir and a lightwalletd data dir in place,
// so `UpdateCachedState` can be used as our test type
let test_type = UpdateCachedState;
// This test is only for the mainnet
let network = Network::Mainnet;
let test_name = "wallet_grpc_test";
// We run these gRPC tests with a network connection, for better test coverage.
let use_internet_connection = true;
if test_type.launches_lightwalletd() && !can_spawn_lightwalletd_for_rpc(test_name, test_type) {
tracing::info!("skipping test due to missing lightwalletd network or cached state");
return Ok(());
}
// Launch zebra with peers and using a predefined zebrad state path.
// As this tests are just queries we can have a live chain where blocks are coming.
let (mut zebrad, zebra_rpc_address) = if let Some(zebrad_and_address) =
spawn_zebrad_for_rpc(network, test_name, test_type, use_internet_connection)?
{
tracing::info!(
?network,
?test_type,
"running gRPC query tests using lightwalletd & zebrad...",
);
zebrad_and_address
} else {
// Skip the test, we don't have the required cached state
return Ok(());
};
let zebra_rpc_address = zebra_rpc_address.expect("lightwalletd test must have RPC port");
fix(state): Use correct end heights for end of block subtrees during the full sync (#7566) * Avoid manual handling of previous sapling trees by using iterator windows instead * Avoid manual sapling subtree index handling by comparing prev and current subtree indexes instead * Simplify adding notes by using the exact number of remaining notes * Simplify by skipping the first block, because it can't complete a subtree * Re-use existing tree update code * Apply the sapling changes to orchard subtree updates * add a reverse database column family iterator function * Make skipping the lowest tree independent of iteration order * Move new subtree checks into the iterator, rename to end_height * Split subtree calculation into a new method * Split the calculate and write methods * Quickly check the first subtree before running the full upgrade * Do the quick checks every time Zebra runs, and refactor slow check error handling * Do quick checks for orchard as well * Make orchard tree upgrade match sapling upgrade code * Upgrade subtrees in reverse height order * Bump the database patch version so the upgrade runs again * Reset previous subtree upgrade data before doing this one * Add extra checks to subtree calculation to diagnose errors * Use correct heights for subtrees completed at the end of a block * Add even more checks to diagnose issues * Instrument upgrade methods to improve diagnostics * Prevent modification of re-used trees * Debug with subtree positions as well * Fix an off-by-one error with completed subtrees * Fix typos and confusing comments Co-authored-by: Marek <mail@marek.onl> * Fix mistaken previous tree handling and end tree comments * Remove unnecessary subtraction in remaining leaves calc * Log heights when assertions fail * Fix new subtree detection filter * Move new subtree check into a method, cleanup unused code * Remove redundant assertions * Wait for subtree upgrade before testing RPCs * Fix subtree search in quick check * Temporarily upgrade subtrees in forward height order * Clarify some comments * Fix missing test imports * Fix subtree logging * Add a comment about a potential hang with future upgrades * Fix zebrad var ownership * Log more info when add_subtrees.rs fails * cargo fmt --all * Fix unrelated clippy::unnecessary_unwrap * cargo clippy --fix --all-features --all-targets; cargo fmt --all * Stop the quick check depending on tree de-duplication * Refactor waiting for the upgrade into functions * Wait for state upgrades whenever the cached state is updated * Wait for the testnet upgrade in the right place * Fix unused variable * Fix a subtree detection bug and comments * Remove an early reference to reverse direction * Stop skipping subtrees completed at the end of blocks * Actually fix new subtree code --------- Co-authored-by: Marek <mail@marek.onl>
2023-09-19 07:49:36 -07:00
// Store the state version message so we can wait for the upgrade later if needed.
let state_version_message = wait_for_state_version_message(&mut zebrad)?;
tracing::info!(
?test_type,
?zebra_rpc_address,
"launched zebrad, waiting for zebrad to open its RPC port..."
);
zebrad.expect_stdout_line_matches(&format!("Opened RPC endpoint at {zebra_rpc_address}"))?;
tracing::info!(
?zebra_rpc_address,
"zebrad opened its RPC port, spawning lightwalletd...",
);
// Launch lightwalletd
let (lightwalletd, lightwalletd_rpc_port) =
spawn_lightwalletd_for_rpc(network, test_name, test_type, zebra_rpc_address)?
.expect("already checked cached state and network requirements");
add(test): test disabled `lightwalletd` mempool gRPCs via zebrad logs (#5016) * add grpc mempool test research * add a config flag for mempool injection of transactions in test * Only copy the inner state directory in the send transactions test * Preload Zcash parameters in some transaction verification tests * Add a block and transaction Hash method to convert from display order bytes * Update test coverage docs * Add debugging output for mempool transaction verification * Test fetching sent mempool transactions using gRPC * Add extra log checks to the send transaction test * Wait for zebrad mempool activation before running gRPC tests * Update send transaction test for lightwalletd not returning mempool transactions * Check zebrad logs instead of disabled lightwalletd gRPCs * Add a debug option that makes RPCs pretend the sync is finished * Remove an unused debug option * Remove unused test code and downgrade some logs * Fix test log checks * Fix some rustdoc warnings * Fix a compilation error due to new function arguments * Make zebrad sync timeouts consistent and remove outdated code * Document how to increase temporary directory space for tests * Stop checking for a log that doesn't always happen * Remove some commented-out code Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com> * Update a comment about run time Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com> * Add new config to new tests from the `main` branch * Add transactions to the list, rather than replacing the list with each new block Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com>
2022-09-06 06:32:33 -07:00
tracing::info!(
?lightwalletd_rpc_port,
"spawned lightwalletd connected to zebrad, waiting for them both to sync...",
add(test): test disabled `lightwalletd` mempool gRPCs via zebrad logs (#5016) * add grpc mempool test research * add a config flag for mempool injection of transactions in test * Only copy the inner state directory in the send transactions test * Preload Zcash parameters in some transaction verification tests * Add a block and transaction Hash method to convert from display order bytes * Update test coverage docs * Add debugging output for mempool transaction verification * Test fetching sent mempool transactions using gRPC * Add extra log checks to the send transaction test * Wait for zebrad mempool activation before running gRPC tests * Update send transaction test for lightwalletd not returning mempool transactions * Check zebrad logs instead of disabled lightwalletd gRPCs * Add a debug option that makes RPCs pretend the sync is finished * Remove an unused debug option * Remove unused test code and downgrade some logs * Fix test log checks * Fix some rustdoc warnings * Fix a compilation error due to new function arguments * Make zebrad sync timeouts consistent and remove outdated code * Document how to increase temporary directory space for tests * Stop checking for a log that doesn't always happen * Remove some commented-out code Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com> * Update a comment about run time Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com> * Add new config to new tests from the `main` branch * Add transactions to the list, rather than replacing the list with each new block Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com>
2022-09-06 06:32:33 -07:00
);
fix(state): Use correct end heights for end of block subtrees during the full sync (#7566) * Avoid manual handling of previous sapling trees by using iterator windows instead * Avoid manual sapling subtree index handling by comparing prev and current subtree indexes instead * Simplify adding notes by using the exact number of remaining notes * Simplify by skipping the first block, because it can't complete a subtree * Re-use existing tree update code * Apply the sapling changes to orchard subtree updates * add a reverse database column family iterator function * Make skipping the lowest tree independent of iteration order * Move new subtree checks into the iterator, rename to end_height * Split subtree calculation into a new method * Split the calculate and write methods * Quickly check the first subtree before running the full upgrade * Do the quick checks every time Zebra runs, and refactor slow check error handling * Do quick checks for orchard as well * Make orchard tree upgrade match sapling upgrade code * Upgrade subtrees in reverse height order * Bump the database patch version so the upgrade runs again * Reset previous subtree upgrade data before doing this one * Add extra checks to subtree calculation to diagnose errors * Use correct heights for subtrees completed at the end of a block * Add even more checks to diagnose issues * Instrument upgrade methods to improve diagnostics * Prevent modification of re-used trees * Debug with subtree positions as well * Fix an off-by-one error with completed subtrees * Fix typos and confusing comments Co-authored-by: Marek <mail@marek.onl> * Fix mistaken previous tree handling and end tree comments * Remove unnecessary subtraction in remaining leaves calc * Log heights when assertions fail * Fix new subtree detection filter * Move new subtree check into a method, cleanup unused code * Remove redundant assertions * Wait for subtree upgrade before testing RPCs * Fix subtree search in quick check * Temporarily upgrade subtrees in forward height order * Clarify some comments * Fix missing test imports * Fix subtree logging * Add a comment about a potential hang with future upgrades * Fix zebrad var ownership * Log more info when add_subtrees.rs fails * cargo fmt --all * Fix unrelated clippy::unnecessary_unwrap * cargo clippy --fix --all-features --all-targets; cargo fmt --all * Stop the quick check depending on tree de-duplication * Refactor waiting for the upgrade into functions * Wait for state upgrades whenever the cached state is updated * Wait for the testnet upgrade in the right place * Fix unused variable * Fix a subtree detection bug and comments * Remove an early reference to reverse direction * Stop skipping subtrees completed at the end of blocks * Actually fix new subtree code --------- Co-authored-by: Marek <mail@marek.onl>
2023-09-19 07:49:36 -07:00
let (_lightwalletd, mut zebrad) = wait_for_zebrad_and_lightwalletd_sync(
lightwalletd,
lightwalletd_rpc_port,
zebrad,
zebra_rpc_address,
test_type,
// We want our queries to include the mempool and network for better coverage
true,
use_internet_connection,
)?;
tracing::info!(
?lightwalletd_rpc_port,
"connecting gRPC client to lightwalletd...",
);
// Connect to the lightwalletd instance
let mut rpc_client = connect_to_lightwalletd(lightwalletd_rpc_port).await?;
// End of the setup and start the tests
tracing::info!(?lightwalletd_rpc_port, "sending gRPC queries...");
// Call `GetLatestBlock`
let block_tip = rpc_client
.get_latest_block(ChainSpec {})
.await?
.into_inner();
// Get `Sapling` activation height.
let sapling_activation_height = Sapling.activation_height(network).unwrap().0 as u64;
// As we are using a pretty much synchronized blockchain, we can assume the tip is above the Nu5 network upgrade
assert!(block_tip.height > Nu5.activation_height(network).unwrap().0 as u64);
2022-05-18 19:30:34 -07:00
// The first block in the mainnet that has sapling and orchard information.
let block_with_trees = 1687107;
// Call `GetBlock` with `block_with_trees`.
let get_block_response = rpc_client
.get_block(BlockId {
height: block_with_trees,
hash: vec![],
})
.await?
.into_inner();
// Make sure we got block `block_with_trees` back
assert_eq!(get_block_response.height, block_with_trees);
// Testing the `trees` field of `GetBlock`.
assert_eq!(
get_block_response
.chain_metadata
.clone()
.unwrap()
.sapling_commitment_tree_size,
1170439
);
assert_eq!(
get_block_response
.chain_metadata
.unwrap()
.orchard_commitment_tree_size,
2
);
// Call `GetBlockRange` with the range starting at block 1 up to block 10
let mut block_range = rpc_client
.get_block_range(BlockRange {
start: Some(BlockId {
2022-05-18 19:30:34 -07:00
height: sapling_activation_height,
hash: vec![],
}),
end: Some(BlockId {
2022-05-18 19:30:34 -07:00
height: sapling_activation_height + 10,
hash: vec![],
}),
})
.await?
.into_inner();
// Make sure the returned Stream of blocks is what we expect
2022-05-18 19:30:34 -07:00
let mut counter = sapling_activation_height;
while let Some(block) = block_range.message().await? {
assert_eq!(block.height, counter);
2022-05-18 19:30:34 -07:00
counter += 1;
}
// Get the first transaction of the first block in the mainnet
let hash = zebra_test::vectors::BLOCK_MAINNET_1_BYTES
.zcash_deserialize_into::<Block>()
.expect("block should deserialize")
.transactions[0]
.hash()
.0
.to_vec();
// Call `GetTransaction` with the transaction hash
let transaction = rpc_client
.get_transaction(TxFilter {
block: None,
index: 0,
hash,
})
.await?
.into_inner();
// Check the height of transactions is 1 as expected
assert_eq!(transaction.height, 1);
// Call `GetTaddressTxids` with a founders reward address that we know exists and have transactions in the first
// few blocks of the mainnet
let mut transactions = rpc_client
.get_taddress_txids(TransparentAddressBlockFilter {
address: "t3Vz22vK5z2LcKEdg16Yv4FFneEL1zg9ojd".to_string(),
range: Some(BlockRange {
start: Some(BlockId {
height: 1,
hash: vec![],
}),
end: Some(BlockId {
height: 10,
hash: vec![],
}),
}),
})
.await?
.into_inner();
let mut counter = 0;
while let Some(_transaction) = transactions.message().await? {
counter += 1;
}
// For the provided address in the first 10 blocks there are 10 transactions in the mainnet
assert_eq!(10, counter);
// Call `GetTaddressBalance` with the ZF funding stream address
let balance = rpc_client
.get_taddress_balance(AddressList {
addresses: vec!["t3dvVE3SQEi7kqNzwrfNePxZ1d4hUyztBA1".to_string()],
})
.await?
.into_inner();
// With ZFND or Major Grants funding stream address, the balance will always be greater than zero,
// because new coins are created in each block
assert!(balance.value_zat > 0);
// Call `GetTaddressBalanceStream` with the ZF funding stream address as a stream argument
let zf_stream_address = Address {
address: "t3dvVE3SQEi7kqNzwrfNePxZ1d4hUyztBA1".to_string(),
};
let balance_zf = rpc_client
.get_taddress_balance_stream(tokio_stream::iter(vec![zf_stream_address.clone()]))
.await?
.into_inner();
// With ZFND funding stream address, the balance will always be greater than zero,
// because new coins are created in each block
assert!(balance_zf.value_zat > 0);
// Call `GetTaddressBalanceStream` with the MG funding stream address as a stream argument
let mg_stream_address = Address {
address: "t3XyYW8yBFRuMnfvm5KLGFbEVz25kckZXym".to_string(),
};
let balance_mg = rpc_client
.get_taddress_balance_stream(tokio_stream::iter(vec![mg_stream_address.clone()]))
.await?
.into_inner();
// With Major Grants funding stream address, the balance will always be greater than zero,
// because new coins are created in each block
assert!(balance_mg.value_zat > 0);
// Call `GetTaddressBalanceStream` with both, the ZFND and the MG funding stream addresses as a stream argument
let balance_both = rpc_client
.get_taddress_balance_stream(tokio_stream::iter(vec![
zf_stream_address,
mg_stream_address,
]))
.await?
.into_inner();
// The result is the sum of the values in both addresses
assert_eq!(
balance_both.value_zat,
balance_zf.value_zat + balance_mg.value_zat
);
let sapling_treestate_init_height = sapling_activation_height + 1;
// Call `GetTreeState`.
let treestate = rpc_client
.get_tree_state(BlockId {
height: sapling_treestate_init_height,
hash: vec![],
})
.await?
.into_inner();
// Check that the network is correct.
assert_eq!(treestate.network, "main");
// Check that the height is correct.
assert_eq!(treestate.height, sapling_treestate_init_height);
// Check that the hash is correct.
assert_eq!(
treestate.hash,
"00000000014d117faa2ea701b24261d364a6c6a62e5bc4bc27335eb9b3c1e2a8"
);
// Check that the time is correct.
assert_eq!(treestate.time, 1540779438);
// Check that the note commitment tree is correct.
assert_eq!(
treestate.sapling_tree,
*zebra_test::vectors::SAPLING_TREESTATE_MAINNET_419201_STRING
);
fix(state): Use correct end heights for end of block subtrees during the full sync (#7566) * Avoid manual handling of previous sapling trees by using iterator windows instead * Avoid manual sapling subtree index handling by comparing prev and current subtree indexes instead * Simplify adding notes by using the exact number of remaining notes * Simplify by skipping the first block, because it can't complete a subtree * Re-use existing tree update code * Apply the sapling changes to orchard subtree updates * add a reverse database column family iterator function * Make skipping the lowest tree independent of iteration order * Move new subtree checks into the iterator, rename to end_height * Split subtree calculation into a new method * Split the calculate and write methods * Quickly check the first subtree before running the full upgrade * Do the quick checks every time Zebra runs, and refactor slow check error handling * Do quick checks for orchard as well * Make orchard tree upgrade match sapling upgrade code * Upgrade subtrees in reverse height order * Bump the database patch version so the upgrade runs again * Reset previous subtree upgrade data before doing this one * Add extra checks to subtree calculation to diagnose errors * Use correct heights for subtrees completed at the end of a block * Add even more checks to diagnose issues * Instrument upgrade methods to improve diagnostics * Prevent modification of re-used trees * Debug with subtree positions as well * Fix an off-by-one error with completed subtrees * Fix typos and confusing comments Co-authored-by: Marek <mail@marek.onl> * Fix mistaken previous tree handling and end tree comments * Remove unnecessary subtraction in remaining leaves calc * Log heights when assertions fail * Fix new subtree detection filter * Move new subtree check into a method, cleanup unused code * Remove redundant assertions * Wait for subtree upgrade before testing RPCs * Fix subtree search in quick check * Temporarily upgrade subtrees in forward height order * Clarify some comments * Fix missing test imports * Fix subtree logging * Add a comment about a potential hang with future upgrades * Fix zebrad var ownership * Log more info when add_subtrees.rs fails * cargo fmt --all * Fix unrelated clippy::unnecessary_unwrap * cargo clippy --fix --all-features --all-targets; cargo fmt --all * Stop the quick check depending on tree de-duplication * Refactor waiting for the upgrade into functions * Wait for state upgrades whenever the cached state is updated * Wait for the testnet upgrade in the right place * Fix unused variable * Fix a subtree detection bug and comments * Remove an early reference to reverse direction * Stop skipping subtrees completed at the end of blocks * Actually fix new subtree code --------- Co-authored-by: Marek <mail@marek.onl>
2023-09-19 07:49:36 -07:00
// Call `GetAddressUtxos` with the ZF funding stream address that will always have utxos
let utxos = rpc_client
.get_address_utxos(GetAddressUtxosArg {
addresses: vec!["t3dvVE3SQEi7kqNzwrfNePxZ1d4hUyztBA1".to_string()],
start_height: 1,
max_entries: 1,
})
.await?
.into_inner();
// As we requested one entry we should get a response of length 1
assert_eq!(utxos.address_utxos.len(), 1);
// Call `GetAddressUtxosStream` with the ZF funding stream address that will always have utxos
let mut utxos_zf = rpc_client
.get_address_utxos_stream(GetAddressUtxosArg {
addresses: vec!["t3dvVE3SQEi7kqNzwrfNePxZ1d4hUyztBA1".to_string()],
start_height: 1,
max_entries: 2,
})
.await?
.into_inner();
let mut counter = 0;
while let Some(_utxos) = utxos_zf.message().await? {
counter += 1;
}
// As we are in a "in sync" chain we know there are more than 2 utxos for this address
// but we will receive the max of 2 from the stream response because we used a limit of 2 `max_entries`.
assert_eq!(2, counter);
// Call `GetLightdInfo`
let lightd_info = rpc_client.get_lightd_info(Empty {}).await?.into_inner();
// Make sure the subversion field is zebra the user agent
assert_eq!(
lightd_info.zcashd_subversion,
zebrad::application::user_agent()
);
// Before we call `z_getsubtreesbyindex`, we might need to wait for a database upgrade.
//
// TODO: this line will hang if the state upgrade finishes before the subtree tests start.
// But that is unlikely with the 25.2 upgrade, because it takes 20+ minutes.
// If it happens for a later upgrade, this code can be moved earlier in the test,
// as long as all the cached states are version 25.2.2 or later.
wait_for_state_version_upgrade(
&mut zebrad,
&state_version_message,
latest_version_for_adding_subtrees(),
)?;
// Call `z_getsubtreesbyindex` separately for...
// ... Sapling.
let mut subtrees = rpc_client
.get_subtree_roots(GetSubtreeRootsArg {
start_index: 0u32,
shielded_protocol: ShieldedProtocol::Sapling.into(),
max_entries: 2u32,
})
.await?
.into_inner();
let mut counter = 0;
while let Some(subtree) = subtrees.message().await? {
match counter {
0 => {
assert_eq!(
subtree.root_hash,
hex!("754bb593ea42d231a7ddf367640f09bbf59dc00f2c1d2003cc340e0c016b5b13")
);
assert_eq!(subtree.completing_block_height, 558822u64);
}
1 => {
assert_eq!(
subtree.root_hash,
hex!("03654c3eacbb9b93e122cf6d77b606eae29610f4f38a477985368197fd68e02d")
);
assert_eq!(subtree.completing_block_height, 670209u64);
}
_ => {
panic!("The response from the `z_getsubtreesbyindex` RPC contains a wrong number of Sapling subtrees.")
}
}
counter += 1;
}
assert_eq!(counter, 2);
// ... Orchard.
let mut subtrees = rpc_client
.get_subtree_roots(GetSubtreeRootsArg {
start_index: 0u32,
shielded_protocol: ShieldedProtocol::Orchard.into(),
max_entries: 2u32,
})
.await?
.into_inner();
let mut counter = 0;
while let Some(subtree) = subtrees.message().await? {
match counter {
0 => {
assert_eq!(
subtree.root_hash,
hex!("d4e323b3ae0cabfb6be4087fec8c66d9a9bbfc354bf1d9588b6620448182063b")
);
assert_eq!(subtree.completing_block_height, 1707429u64);
}
1 => {
assert_eq!(
subtree.root_hash,
hex!("8c47d0ca43f323ac573ee57c90af4ced484682827248ca5f3eead95eb6415a14")
);
assert_eq!(subtree.completing_block_height, 1708132u64);
}
_ => {
panic!("The response from the `z_getsubtreesbyindex` RPC contains a wrong number of Orchard subtrees.")
}
}
counter += 1;
}
assert_eq!(counter, 2);
Ok(())
}