add(scan): Implement `DeleteKeys` `ScanService` request (#8217)
* handle RemoveKeys command in ScanTask * implements DeleteKeys request * uses spawn_blocking and updates comments * removes spawn_init fn * adds test for new Storage method * adds fake_sapling_results helper function * adds test for scan service DeleteKeys request * adds TODO for unwrap_or_clone * checks the keys sent to the scan task for removal * moves message processing to its own function * adds a timeout for the scanner task response * hide mock() methods behind cfg(test) or feature * adds MAX_REQUEST_KEYS constant * updates test to insert and delete results for a second key * Update zebra-scan/src/init.rs Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com> * test that the expected number of results are in the db * fix unused import lint * fixes unused imports --------- Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com>
This commit is contained in:
parent
768eb90722
commit
80827f5294
|
@ -12,8 +12,8 @@ pub enum Request {
|
||||||
/// TODO: Accept `ViewingKeyWithHash`es and return Ok(()) if successful or an error
|
/// TODO: Accept `ViewingKeyWithHash`es and return Ok(()) if successful or an error
|
||||||
RegisterKeys(Vec<()>),
|
RegisterKeys(Vec<()>),
|
||||||
|
|
||||||
/// TODO: Accept `KeyHash`es and return Ok(`Vec<KeyHash>`) with hashes of deleted keys
|
/// Deletes viewing keys and their results from the database.
|
||||||
DeleteKeys(Vec<()>),
|
DeleteKeys(Vec<String>),
|
||||||
|
|
||||||
/// TODO: Accept `KeyHash`es and return `Transaction`s
|
/// TODO: Accept `KeyHash`es and return `Transaction`s
|
||||||
Results(Vec<()>),
|
Results(Vec<()>),
|
||||||
|
|
|
@ -16,6 +16,9 @@ pub enum Response {
|
||||||
/// Response to Results request
|
/// Response to Results request
|
||||||
Results(Vec<Transaction>),
|
Results(Vec<Transaction>),
|
||||||
|
|
||||||
|
/// Response to DeleteKeys request
|
||||||
|
DeletedKeys,
|
||||||
|
|
||||||
/// Response to SubscribeResults request
|
/// Response to SubscribeResults request
|
||||||
SubscribeResults(mpsc::Receiver<Arc<Transaction>>),
|
SubscribeResults(mpsc::Receiver<Arc<Transaction>>),
|
||||||
}
|
}
|
||||||
|
|
|
@ -17,6 +17,7 @@ categories = ["cryptography::cryptocurrencies"]
|
||||||
[[bin]] # Bin to run the Scanner gRPC server
|
[[bin]] # Bin to run the Scanner gRPC server
|
||||||
name = "scanner-grpc-server"
|
name = "scanner-grpc-server"
|
||||||
path = "src/bin/rpc_server.rs"
|
path = "src/bin/rpc_server.rs"
|
||||||
|
required-features = ["proptest-impl"]
|
||||||
|
|
||||||
[features]
|
[features]
|
||||||
|
|
||||||
|
|
|
@ -2,15 +2,16 @@
|
||||||
|
|
||||||
use tower::ServiceBuilder;
|
use tower::ServiceBuilder;
|
||||||
|
|
||||||
use zebra_scan::service::ScanService;
|
use zebra_scan::{service::ScanService, storage::Storage};
|
||||||
|
|
||||||
#[tokio::main]
|
#[tokio::main]
|
||||||
/// Runs an RPC server with a mock ScanTask
|
/// Runs an RPC server with a mock ScanTask
|
||||||
async fn main() -> Result<(), Box<dyn std::error::Error>> {
|
async fn main() -> Result<(), Box<dyn std::error::Error>> {
|
||||||
let (config, network) = Default::default();
|
let (config, network) = Default::default();
|
||||||
let scan_service = ServiceBuilder::new()
|
|
||||||
.buffer(10)
|
let (scan_service, _cmd_receiver) =
|
||||||
.service(ScanService::new_with_mock_scanner(&config, network));
|
ScanService::new_with_mock_scanner(Storage::new(&config, network, false));
|
||||||
|
let scan_service = ServiceBuilder::new().buffer(10).service(scan_service);
|
||||||
|
|
||||||
// Start the gRPC server.
|
// Start the gRPC server.
|
||||||
zebra_grpc::server::init(scan_service).await?;
|
zebra_grpc::server::init(scan_service).await?;
|
||||||
|
|
|
@ -23,7 +23,7 @@ pub enum ScanTaskCommand {
|
||||||
done_tx: oneshot::Sender<()>,
|
done_tx: oneshot::Sender<()>,
|
||||||
|
|
||||||
/// Key hashes that are to be removed
|
/// Key hashes that are to be removed
|
||||||
key_hashes: Vec<()>,
|
keys: Vec<String>,
|
||||||
},
|
},
|
||||||
|
|
||||||
/// Start sending results for key hashes to `result_sender`
|
/// Start sending results for key hashes to `result_sender`
|
||||||
|
@ -36,25 +36,29 @@ pub enum ScanTaskCommand {
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
|
|
||||||
#[derive(Debug)]
|
#[derive(Debug, Clone)]
|
||||||
/// Scan task handle and command channel sender
|
/// Scan task handle and command channel sender
|
||||||
pub struct ScanTask {
|
pub struct ScanTask {
|
||||||
/// [`JoinHandle`] of scan task
|
/// [`JoinHandle`] of scan task
|
||||||
pub handle: JoinHandle<Result<(), Report>>,
|
pub handle: Arc<JoinHandle<Result<(), Report>>>,
|
||||||
|
|
||||||
/// Task command channel sender
|
/// Task command channel sender
|
||||||
cmd_sender: mpsc::Sender<ScanTaskCommand>,
|
pub cmd_sender: mpsc::Sender<ScanTaskCommand>,
|
||||||
}
|
}
|
||||||
|
|
||||||
impl ScanTask {
|
impl ScanTask {
|
||||||
/// Spawns a new [`ScanTask`] for tests.
|
/// Spawns a new [`ScanTask`] for tests.
|
||||||
pub fn mock() -> Self {
|
#[cfg(any(test, feature = "proptest-impl"))]
|
||||||
let (cmd_sender, _cmd_receiver) = mpsc::channel();
|
pub fn mock() -> (Self, mpsc::Receiver<ScanTaskCommand>) {
|
||||||
|
let (cmd_sender, cmd_receiver) = mpsc::channel();
|
||||||
|
|
||||||
|
(
|
||||||
Self {
|
Self {
|
||||||
handle: tokio::spawn(std::future::pending()),
|
handle: Arc::new(tokio::spawn(std::future::pending())),
|
||||||
cmd_sender,
|
cmd_sender,
|
||||||
}
|
},
|
||||||
|
cmd_receiver,
|
||||||
|
)
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Spawns a new [`ScanTask`].
|
/// Spawns a new [`ScanTask`].
|
||||||
|
@ -64,11 +68,16 @@ impl ScanTask {
|
||||||
state: scan::State,
|
state: scan::State,
|
||||||
chain_tip_change: ChainTipChange,
|
chain_tip_change: ChainTipChange,
|
||||||
) -> Self {
|
) -> Self {
|
||||||
// TODO: Pass `_cmd_receiver` to `scan::start()` to pass it new keys after it's been spawned
|
let (cmd_sender, cmd_receiver) = mpsc::channel();
|
||||||
let (cmd_sender, _cmd_receiver) = mpsc::channel();
|
|
||||||
|
|
||||||
Self {
|
Self {
|
||||||
handle: scan::spawn_init(config, network, state, chain_tip_change),
|
handle: Arc::new(scan::spawn_init(
|
||||||
|
config,
|
||||||
|
network,
|
||||||
|
state,
|
||||||
|
chain_tip_change,
|
||||||
|
cmd_receiver,
|
||||||
|
)),
|
||||||
cmd_sender,
|
cmd_sender,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -80,18 +89,21 @@ impl ScanTask {
|
||||||
) -> Result<(), mpsc::SendError<ScanTaskCommand>> {
|
) -> Result<(), mpsc::SendError<ScanTaskCommand>> {
|
||||||
self.cmd_sender.send(command)
|
self.cmd_sender.send(command)
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
/// Initialize the scanner based on its config, and spawn a task for it.
|
/// Sends a message to the scan task to remove the provided viewing keys.
|
||||||
///
|
pub fn remove_keys(
|
||||||
/// TODO: add a test for this function.
|
&mut self,
|
||||||
pub fn spawn_init(
|
keys: &[String],
|
||||||
config: &Config,
|
) -> Result<oneshot::Receiver<()>, mpsc::SendError<ScanTaskCommand>> {
|
||||||
network: Network,
|
let (done_tx, done_rx) = oneshot::channel();
|
||||||
state: scan::State,
|
|
||||||
chain_tip_change: ChainTipChange,
|
self.send(ScanTaskCommand::RemoveKeys {
|
||||||
) -> JoinHandle<Result<(), Report>> {
|
keys: keys.to_vec(),
|
||||||
scan::spawn_init(config, network, state, chain_tip_change)
|
done_tx,
|
||||||
|
})?;
|
||||||
|
|
||||||
|
Ok(done_rx)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Initialize [`ScanService`] based on its config.
|
/// Initialize [`ScanService`] based on its config.
|
||||||
|
|
|
@ -19,4 +19,4 @@ pub mod service;
|
||||||
pub mod tests;
|
pub mod tests;
|
||||||
|
|
||||||
pub use config::Config;
|
pub use config::Config;
|
||||||
pub use init::{init, spawn_init};
|
pub use init::{init, ScanTask};
|
||||||
|
|
|
@ -2,7 +2,10 @@
|
||||||
|
|
||||||
use std::{
|
use std::{
|
||||||
collections::{BTreeMap, HashMap},
|
collections::{BTreeMap, HashMap},
|
||||||
sync::Arc,
|
sync::{
|
||||||
|
mpsc::{Receiver, TryRecvError},
|
||||||
|
Arc,
|
||||||
|
},
|
||||||
time::Duration,
|
time::Duration,
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -37,8 +40,9 @@ use zebra_chain::{
|
||||||
use zebra_state::{ChainTipChange, SaplingScannedResult, TransactionIndex};
|
use zebra_state::{ChainTipChange, SaplingScannedResult, TransactionIndex};
|
||||||
|
|
||||||
use crate::{
|
use crate::{
|
||||||
|
init::ScanTaskCommand,
|
||||||
storage::{SaplingScanningKey, Storage},
|
storage::{SaplingScanningKey, Storage},
|
||||||
Config,
|
Config, ScanTask,
|
||||||
};
|
};
|
||||||
|
|
||||||
/// The generic state type used by the scanner.
|
/// The generic state type used by the scanner.
|
||||||
|
@ -66,6 +70,7 @@ pub async fn start(
|
||||||
state: State,
|
state: State,
|
||||||
chain_tip_change: ChainTipChange,
|
chain_tip_change: ChainTipChange,
|
||||||
storage: Storage,
|
storage: Storage,
|
||||||
|
cmd_receiver: Receiver<ScanTaskCommand>,
|
||||||
) -> Result<(), Report> {
|
) -> Result<(), Report> {
|
||||||
let network = storage.network();
|
let network = storage.network();
|
||||||
let sapling_activation_height = storage.min_sapling_birthday_height();
|
let sapling_activation_height = storage.min_sapling_birthday_height();
|
||||||
|
@ -102,12 +107,14 @@ pub async fn start(
|
||||||
Ok::<_, Report>((key.clone(), parsed_keys))
|
Ok::<_, Report>((key.clone(), parsed_keys))
|
||||||
})
|
})
|
||||||
.try_collect()?;
|
.try_collect()?;
|
||||||
let parsed_keys = Arc::new(parsed_keys);
|
let mut parsed_keys = Arc::new(parsed_keys);
|
||||||
|
|
||||||
// Give empty states time to verify some blocks before we start scanning.
|
// Give empty states time to verify some blocks before we start scanning.
|
||||||
tokio::time::sleep(INITIAL_WAIT).await;
|
tokio::time::sleep(INITIAL_WAIT).await;
|
||||||
|
|
||||||
loop {
|
loop {
|
||||||
|
parsed_keys = ScanTask::process_msgs(&cmd_receiver, parsed_keys)?;
|
||||||
|
|
||||||
let scanned_height = scan_height_and_store_results(
|
let scanned_height = scan_height_and_store_results(
|
||||||
height,
|
height,
|
||||||
state.clone(),
|
state.clone(),
|
||||||
|
@ -130,6 +137,56 @@ pub async fn start(
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
impl ScanTask {
|
||||||
|
/// Accepts the scan task's `parsed_key` collection and a reference to the command channel receiver
|
||||||
|
///
|
||||||
|
/// Processes messages in the scan task channel, updating `parsed_keys` if required.
|
||||||
|
///
|
||||||
|
/// Returns the updated `parsed_keys`
|
||||||
|
fn process_msgs(
|
||||||
|
cmd_receiver: &Receiver<ScanTaskCommand>,
|
||||||
|
mut parsed_keys: Arc<
|
||||||
|
HashMap<SaplingScanningKey, (Vec<DiversifiableFullViewingKey>, Vec<SaplingIvk>)>,
|
||||||
|
>,
|
||||||
|
) -> Result<
|
||||||
|
Arc<HashMap<SaplingScanningKey, (Vec<DiversifiableFullViewingKey>, Vec<SaplingIvk>)>>,
|
||||||
|
Report,
|
||||||
|
> {
|
||||||
|
loop {
|
||||||
|
let cmd = match cmd_receiver.try_recv() {
|
||||||
|
Ok(cmd) => cmd,
|
||||||
|
|
||||||
|
Err(TryRecvError::Empty) => break,
|
||||||
|
Err(TryRecvError::Disconnected) => {
|
||||||
|
// Return early if the sender has been dropped.
|
||||||
|
return Err(eyre!("command channel disconnected"));
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
match cmd {
|
||||||
|
ScanTaskCommand::RemoveKeys { done_tx, keys } => {
|
||||||
|
// TODO: Replace with Arc::unwrap_or_clone() when it stabilises:
|
||||||
|
// https://github.com/rust-lang/rust/issues/93610
|
||||||
|
let mut updated_parsed_keys =
|
||||||
|
Arc::try_unwrap(parsed_keys).unwrap_or_else(|arc| (*arc).clone());
|
||||||
|
|
||||||
|
for key in keys {
|
||||||
|
updated_parsed_keys.remove(&key);
|
||||||
|
}
|
||||||
|
|
||||||
|
parsed_keys = Arc::new(updated_parsed_keys);
|
||||||
|
|
||||||
|
// Ignore send errors for the done notification
|
||||||
|
let _ = done_tx.send(());
|
||||||
|
}
|
||||||
|
|
||||||
|
_ => continue,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
Ok(parsed_keys)
|
||||||
|
}
|
||||||
|
}
|
||||||
/// Get the block at `height` from `state`, scan it with the keys in `parsed_keys`, and store the
|
/// Get the block at `height` from `state`, scan it with the keys in `parsed_keys`, and store the
|
||||||
/// results in `storage`. If `height` is lower than the `key_birthdays` for that key, skip it.
|
/// results in `storage`. If `height` is lower than the `key_birthdays` for that key, skip it.
|
||||||
///
|
///
|
||||||
|
@ -445,11 +502,12 @@ pub fn spawn_init(
|
||||||
network: Network,
|
network: Network,
|
||||||
state: State,
|
state: State,
|
||||||
chain_tip_change: ChainTipChange,
|
chain_tip_change: ChainTipChange,
|
||||||
|
cmd_receiver: Receiver<ScanTaskCommand>,
|
||||||
) -> JoinHandle<Result<(), Report>> {
|
) -> JoinHandle<Result<(), Report>> {
|
||||||
let config = config.clone();
|
let config = config.clone();
|
||||||
|
|
||||||
// TODO: spawn an entirely new executor here, to avoid timing attacks.
|
// TODO: spawn an entirely new executor here, to avoid timing attacks.
|
||||||
tokio::spawn(init(config, network, state, chain_tip_change).in_current_span())
|
tokio::spawn(init(config, network, state, chain_tip_change, cmd_receiver).in_current_span())
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Initialize the scanner based on its config.
|
/// Initialize the scanner based on its config.
|
||||||
|
@ -460,11 +518,12 @@ pub async fn init(
|
||||||
network: Network,
|
network: Network,
|
||||||
state: State,
|
state: State,
|
||||||
chain_tip_change: ChainTipChange,
|
chain_tip_change: ChainTipChange,
|
||||||
|
cmd_receiver: Receiver<ScanTaskCommand>,
|
||||||
) -> Result<(), Report> {
|
) -> Result<(), Report> {
|
||||||
let storage = tokio::task::spawn_blocking(move || Storage::new(&config, network, false))
|
let storage = tokio::task::spawn_blocking(move || Storage::new(&config, network, false))
|
||||||
.wait_for_panics()
|
.wait_for_panics()
|
||||||
.await;
|
.await;
|
||||||
|
|
||||||
// TODO: add more tasks here?
|
// TODO: add more tasks here?
|
||||||
start(state, chain_tip_change, storage).await
|
start(state, chain_tip_change, storage, cmd_receiver).await
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,6 +1,6 @@
|
||||||
//! [`tower::Service`] for zebra-scan.
|
//! [`tower::Service`] for zebra-scan.
|
||||||
|
|
||||||
use std::{future::Future, pin::Pin, task::Poll};
|
use std::{future::Future, pin::Pin, task::Poll, time::Duration};
|
||||||
|
|
||||||
use futures::future::FutureExt;
|
use futures::future::FutureExt;
|
||||||
use tower::Service;
|
use tower::Service;
|
||||||
|
@ -10,16 +10,25 @@ use zebra_state::ChainTipChange;
|
||||||
|
|
||||||
use crate::{init::ScanTask, scan, storage::Storage, Config, Request, Response};
|
use crate::{init::ScanTask, scan, storage::Storage, Config, Request, Response};
|
||||||
|
|
||||||
|
#[cfg(test)]
|
||||||
|
mod tests;
|
||||||
|
|
||||||
/// Zebra-scan [`tower::Service`]
|
/// Zebra-scan [`tower::Service`]
|
||||||
#[derive(Debug)]
|
#[derive(Debug)]
|
||||||
pub struct ScanService {
|
pub struct ScanService {
|
||||||
/// On-disk storage
|
/// On-disk storage
|
||||||
db: Storage,
|
pub db: Storage,
|
||||||
|
|
||||||
/// Handle to scan task that's responsible for writing results
|
/// Handle to scan task that's responsible for writing results
|
||||||
scan_task: ScanTask,
|
scan_task: ScanTask,
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// A timeout applied to `DeleteKeys` requests.
|
||||||
|
const DELETE_KEY_TIMEOUT: Duration = Duration::from_secs(15);
|
||||||
|
|
||||||
|
/// The maximum number of keys that may be included in a request to the scan service
|
||||||
|
const MAX_REQUEST_KEYS: usize = 1000;
|
||||||
|
|
||||||
impl ScanService {
|
impl ScanService {
|
||||||
/// Create a new [`ScanService`].
|
/// Create a new [`ScanService`].
|
||||||
pub fn new(
|
pub fn new(
|
||||||
|
@ -35,11 +44,15 @@ impl ScanService {
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Create a new [`ScanService`] with a mock `ScanTask`
|
/// Create a new [`ScanService`] with a mock `ScanTask`
|
||||||
pub fn new_with_mock_scanner(config: &Config, network: Network) -> Self {
|
#[cfg(any(test, feature = "proptest-impl"))]
|
||||||
Self {
|
pub fn new_with_mock_scanner(
|
||||||
db: Storage::new(config, network, false),
|
db: Storage,
|
||||||
scan_task: ScanTask::mock(),
|
) -> (
|
||||||
}
|
Self,
|
||||||
|
std::sync::mpsc::Receiver<crate::init::ScanTaskCommand>,
|
||||||
|
) {
|
||||||
|
let (scan_task, cmd_receiver) = ScanTask::mock();
|
||||||
|
(Self { db, scan_task }, cmd_receiver)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -84,10 +97,37 @@ impl Service<Request> for ScanService {
|
||||||
// - send new keys to scan task
|
// - send new keys to scan task
|
||||||
}
|
}
|
||||||
|
|
||||||
Request::DeleteKeys(_key_hashes) => {
|
Request::DeleteKeys(keys) => {
|
||||||
// TODO:
|
let mut db = self.db.clone();
|
||||||
// - delete these keys and their results from db
|
let mut scan_task = self.scan_task.clone();
|
||||||
// - send deleted keys to scan task
|
|
||||||
|
return async move {
|
||||||
|
if keys.len() > MAX_REQUEST_KEYS {
|
||||||
|
return Err(format!(
|
||||||
|
"maximum number of keys per request is {MAX_REQUEST_KEYS}"
|
||||||
|
)
|
||||||
|
.into());
|
||||||
|
}
|
||||||
|
|
||||||
|
// Wait for a message to confirm that the scan task has removed the key up to `DELETE_KEY_TIMEOUT`
|
||||||
|
let remove_keys_result =
|
||||||
|
tokio::time::timeout(DELETE_KEY_TIMEOUT, scan_task.remove_keys(&keys)?)
|
||||||
|
.await
|
||||||
|
.map_err(|_| "timeout waiting for delete keys done notification");
|
||||||
|
|
||||||
|
// Delete the key from the database after either confirmation that it's been removed from the scan task, or
|
||||||
|
// waiting `DELETE_KEY_TIMEOUT`.
|
||||||
|
let delete_key_task = tokio::task::spawn_blocking(move || {
|
||||||
|
db.delete_sapling_results(keys);
|
||||||
|
});
|
||||||
|
|
||||||
|
// Return timeout errors or `RecvError`s, or wait for the key to be deleted from the database.
|
||||||
|
remove_keys_result??;
|
||||||
|
delete_key_task.await?;
|
||||||
|
|
||||||
|
Ok(Response::DeletedKeys)
|
||||||
|
}
|
||||||
|
.boxed();
|
||||||
}
|
}
|
||||||
|
|
||||||
Request::Results(_key_hashes) => {
|
Request::Results(_key_hashes) => {
|
||||||
|
|
|
@ -0,0 +1,79 @@
|
||||||
|
//! Tests for ScanService.
|
||||||
|
|
||||||
|
use tower::{Service, ServiceExt};
|
||||||
|
|
||||||
|
use color_eyre::{eyre::eyre, Result};
|
||||||
|
|
||||||
|
use zebra_chain::{block::Height, parameters::Network};
|
||||||
|
use zebra_node_services::scan_service::{request::Request, response::Response};
|
||||||
|
use zebra_state::TransactionIndex;
|
||||||
|
|
||||||
|
use crate::{
|
||||||
|
init::ScanTaskCommand,
|
||||||
|
service::ScanService,
|
||||||
|
storage::db::tests::{fake_sapling_results, new_test_storage},
|
||||||
|
tests::ZECPAGES_SAPLING_VIEWING_KEY,
|
||||||
|
};
|
||||||
|
|
||||||
|
/// Tests that keys are deleted correctly
|
||||||
|
#[tokio::test]
|
||||||
|
pub async fn scan_service_deletes_keys_correctly() -> Result<()> {
|
||||||
|
let mut db = new_test_storage(Network::Mainnet);
|
||||||
|
|
||||||
|
let zec_pages_sapling_efvk = ZECPAGES_SAPLING_VIEWING_KEY.to_string();
|
||||||
|
|
||||||
|
for fake_result_height in [Height::MIN, Height(1), Height::MAX] {
|
||||||
|
db.insert_sapling_results(
|
||||||
|
&zec_pages_sapling_efvk,
|
||||||
|
fake_result_height,
|
||||||
|
fake_sapling_results([
|
||||||
|
TransactionIndex::MIN,
|
||||||
|
TransactionIndex::from_index(40),
|
||||||
|
TransactionIndex::MAX,
|
||||||
|
]),
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
assert!(
|
||||||
|
!db.sapling_results(&zec_pages_sapling_efvk).is_empty(),
|
||||||
|
"there should be some results for this key in the db"
|
||||||
|
);
|
||||||
|
|
||||||
|
let (mut scan_service, cmd_receiver) = ScanService::new_with_mock_scanner(db);
|
||||||
|
|
||||||
|
let response_fut = scan_service
|
||||||
|
.ready()
|
||||||
|
.await
|
||||||
|
.map_err(|err| eyre!(err))?
|
||||||
|
.call(Request::DeleteKeys(vec![zec_pages_sapling_efvk.clone()]));
|
||||||
|
|
||||||
|
let expected_keys = vec![zec_pages_sapling_efvk.clone()];
|
||||||
|
let cmd_handler_fut = tokio::task::spawn_blocking(move || {
|
||||||
|
let Ok(ScanTaskCommand::RemoveKeys { done_tx, keys }) = cmd_receiver.recv() else {
|
||||||
|
panic!("should successfully receive RemoveKeys message");
|
||||||
|
};
|
||||||
|
|
||||||
|
assert_eq!(keys, expected_keys, "keys should match the request keys");
|
||||||
|
|
||||||
|
done_tx.send(()).expect("send should succeed");
|
||||||
|
});
|
||||||
|
|
||||||
|
// Poll futures
|
||||||
|
let (response, join_result) = tokio::join!(response_fut, cmd_handler_fut);
|
||||||
|
join_result?;
|
||||||
|
|
||||||
|
match response.map_err(|err| eyre!(err))? {
|
||||||
|
Response::DeletedKeys => {}
|
||||||
|
_ => panic!("scan service returned unexpected response variant"),
|
||||||
|
};
|
||||||
|
|
||||||
|
assert!(
|
||||||
|
scan_service
|
||||||
|
.db
|
||||||
|
.sapling_results(&zec_pages_sapling_efvk)
|
||||||
|
.is_empty(),
|
||||||
|
"all results for this key should have been deleted"
|
||||||
|
);
|
||||||
|
|
||||||
|
Ok(())
|
||||||
|
}
|
|
@ -233,6 +233,25 @@ impl Storage {
|
||||||
.write_batch()
|
.write_batch()
|
||||||
.expect("unexpected database write failure");
|
.expect("unexpected database write failure");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Delete the results of sapling scanning `keys`, if they exist
|
||||||
|
pub(crate) fn delete_sapling_results(&mut self, keys: Vec<SaplingScanningKey>) {
|
||||||
|
let mut batch = self.sapling_tx_ids_cf().new_batch_for_writing();
|
||||||
|
|
||||||
|
for key in &keys {
|
||||||
|
let from = SaplingScannedDatabaseIndex::min_for_key(key);
|
||||||
|
let until_strictly_before = SaplingScannedDatabaseIndex::max_for_key(key);
|
||||||
|
|
||||||
|
batch = batch
|
||||||
|
.zs_delete_range(&from, &until_strictly_before)
|
||||||
|
// TODO: convert zs_delete_range() to take std::ops::RangeBounds
|
||||||
|
.zs_delete(&until_strictly_before);
|
||||||
|
}
|
||||||
|
|
||||||
|
batch
|
||||||
|
.write_batch()
|
||||||
|
.expect("unexpected database write failure");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Utility trait for inserting sapling heights into a WriteSaplingTxIdsBatch.
|
/// Utility trait for inserting sapling heights into a WriteSaplingTxIdsBatch.
|
||||||
|
|
|
@ -6,8 +6,9 @@ use zebra_chain::{
|
||||||
block::{Block, Height},
|
block::{Block, Height},
|
||||||
parameters::Network::{self, *},
|
parameters::Network::{self, *},
|
||||||
serialization::ZcashDeserializeInto,
|
serialization::ZcashDeserializeInto,
|
||||||
|
transaction,
|
||||||
};
|
};
|
||||||
use zebra_state::TransactionIndex;
|
use zebra_state::{SaplingScannedResult, TransactionIndex};
|
||||||
|
|
||||||
use crate::{
|
use crate::{
|
||||||
storage::{Storage, INSERT_CONTROL_INTERVAL},
|
storage::{Storage, INSERT_CONTROL_INTERVAL},
|
||||||
|
@ -18,6 +19,9 @@ use crate::{
|
||||||
#[cfg(test)]
|
#[cfg(test)]
|
||||||
mod snapshot;
|
mod snapshot;
|
||||||
|
|
||||||
|
#[cfg(test)]
|
||||||
|
mod vectors;
|
||||||
|
|
||||||
/// Returns an empty `Storage` suitable for testing.
|
/// Returns an empty `Storage` suitable for testing.
|
||||||
pub fn new_test_storage(network: Network) -> Storage {
|
pub fn new_test_storage(network: Network) -> Storage {
|
||||||
Storage::new(&Config::ephemeral(), network, false)
|
Storage::new(&Config::ephemeral(), network, false)
|
||||||
|
@ -74,3 +78,19 @@ pub fn add_fake_results(
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Accepts an iterator of [`TransactionIndex`]es and returns a `BTreeMap` with empty results
|
||||||
|
pub fn fake_sapling_results<T: IntoIterator<Item = TransactionIndex>>(
|
||||||
|
transaction_indexes: T,
|
||||||
|
) -> BTreeMap<TransactionIndex, SaplingScannedResult> {
|
||||||
|
let mut fake_sapling_results = BTreeMap::new();
|
||||||
|
|
||||||
|
for transaction_index in transaction_indexes {
|
||||||
|
fake_sapling_results.insert(
|
||||||
|
transaction_index,
|
||||||
|
SaplingScannedResult::from(transaction::Hash::from([0; 32])),
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
fake_sapling_results
|
||||||
|
}
|
||||||
|
|
|
@ -0,0 +1,67 @@
|
||||||
|
//! Fixed test vectors for the scanner Storage.
|
||||||
|
|
||||||
|
use zebra_chain::{block::Height, parameters::Network};
|
||||||
|
use zebra_state::TransactionIndex;
|
||||||
|
|
||||||
|
use crate::{
|
||||||
|
storage::db::tests::{fake_sapling_results, new_test_storage},
|
||||||
|
tests::ZECPAGES_SAPLING_VIEWING_KEY,
|
||||||
|
};
|
||||||
|
|
||||||
|
/// Tests that keys are deleted correctly
|
||||||
|
#[test]
|
||||||
|
pub fn deletes_keys_and_results_correctly() {
|
||||||
|
let mut db = new_test_storage(Network::Mainnet);
|
||||||
|
|
||||||
|
let zec_pages_sapling_efvk = ZECPAGES_SAPLING_VIEWING_KEY.to_string();
|
||||||
|
|
||||||
|
// Replace the last letter of the zec_pages efvk
|
||||||
|
let fake_efvk = format!(
|
||||||
|
"{}t",
|
||||||
|
&ZECPAGES_SAPLING_VIEWING_KEY[..ZECPAGES_SAPLING_VIEWING_KEY.len() - 1]
|
||||||
|
);
|
||||||
|
|
||||||
|
let efvks = [&zec_pages_sapling_efvk, &fake_efvk];
|
||||||
|
let fake_heights = [Height::MIN, Height(1), Height::MAX];
|
||||||
|
let fake_transaction_indexes = [
|
||||||
|
TransactionIndex::MIN,
|
||||||
|
TransactionIndex::from_index(40),
|
||||||
|
TransactionIndex::MAX,
|
||||||
|
];
|
||||||
|
|
||||||
|
for efvk in efvks {
|
||||||
|
for fake_result_height in fake_heights {
|
||||||
|
db.insert_sapling_results(
|
||||||
|
efvk,
|
||||||
|
fake_result_height,
|
||||||
|
fake_sapling_results(fake_transaction_indexes),
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
let expected_num_entries = fake_heights.len();
|
||||||
|
let expected_num_results_per_entry = fake_transaction_indexes.len();
|
||||||
|
|
||||||
|
for efvk in efvks {
|
||||||
|
assert_eq!(
|
||||||
|
db.sapling_results(efvk).len(),
|
||||||
|
expected_num_entries,
|
||||||
|
"there should be {expected_num_entries} entries for this key in the db"
|
||||||
|
);
|
||||||
|
|
||||||
|
for (_, result) in db.sapling_results(efvk) {
|
||||||
|
assert_eq!(
|
||||||
|
result.len(),
|
||||||
|
expected_num_results_per_entry,
|
||||||
|
"there should be {expected_num_results_per_entry} results for this entry in the db"
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
db.delete_sapling_results(vec![efvk.clone()]);
|
||||||
|
|
||||||
|
assert!(
|
||||||
|
db.sapling_results(efvk).is_empty(),
|
||||||
|
"all results for this key should have been deleted"
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
|
@ -301,17 +301,24 @@ impl StartCmd {
|
||||||
|
|
||||||
#[cfg(feature = "shielded-scan")]
|
#[cfg(feature = "shielded-scan")]
|
||||||
// Spawn never ending scan task only if we have keys to scan for.
|
// Spawn never ending scan task only if we have keys to scan for.
|
||||||
let scan_task_handle = if !config.shielded_scan.sapling_keys_to_scan.is_empty() {
|
let (scan_task_handle, _cmd_sender) =
|
||||||
|
if !config.shielded_scan.sapling_keys_to_scan.is_empty() {
|
||||||
// TODO: log the number of keys and update the scan_task_starts() test
|
// TODO: log the number of keys and update the scan_task_starts() test
|
||||||
info!("spawning shielded scanner with configured viewing keys");
|
info!("spawning shielded scanner with configured viewing keys");
|
||||||
zebra_scan::spawn_init(
|
let scan_task = zebra_scan::ScanTask::spawn(
|
||||||
&config.shielded_scan,
|
&config.shielded_scan,
|
||||||
config.network.network,
|
config.network.network,
|
||||||
state,
|
state,
|
||||||
chain_tip_change,
|
chain_tip_change,
|
||||||
|
);
|
||||||
|
|
||||||
|
(
|
||||||
|
std::sync::Arc::into_inner(scan_task.handle)
|
||||||
|
.expect("should only have one reference here"),
|
||||||
|
Some(scan_task.cmd_sender),
|
||||||
)
|
)
|
||||||
} else {
|
} else {
|
||||||
tokio::spawn(std::future::pending().in_current_span())
|
(tokio::spawn(std::future::pending().in_current_span()), None)
|
||||||
};
|
};
|
||||||
|
|
||||||
#[cfg(not(feature = "shielded-scan"))]
|
#[cfg(not(feature = "shielded-scan"))]
|
||||||
|
|
Loading…
Reference in New Issue