change(rpc): add submitblock RPC method (#5526)

* adds submitblock rpc method

* re-orders imports

* replaces thread::yield_now with async yield_now

* Fix doc warnings and unused variable warnings, add missing docs

* Mark work_id as optional

* Use the same ChainVerifier for downloaded and submitted blocks

* Revert unused changes & minor cleanups

* Document currently-unreachable code

* updates tests and submit_block response for AlreadyVerified error

* Update zebra-rpc/src/methods/get_block_template_rpcs.rs

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

* changes names from BlockVerifier to ChainVerifier and block_verifier to chain_verifier to keep it consistent with naming in zebra-consensus

* move how to run the submit_block test example to acceptance.rs

* updates snapshot tests

* moved acceptance test to a separate file

* removes extra tower::ServiceBuilder::new(), updates docs

* updates vectors and snapshot tests, changes hex decoding error in submit_block method from server error to parse error

* hides errors module in zebra-rpc behind a feature flag and adds docs.

* Updates snapshot test, adds mod docs, moves HexData to its own mod, and removes the unrelated make_server_error refactor for now

* update submit block acceptance test mod doc

Co-authored-by: teor <teor@riseup.net>
Co-authored-by: Alfredo Garcia <oxarbitrage@gmail.com>
Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com>
This commit is contained in:
Arya 2022-11-03 23:57:08 -04:00 committed by GitHub
parent 34313b8857
commit 2f3b05f8e1
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
21 changed files with 638 additions and 23 deletions

1
Cargo.lock generated
View File

@ -5396,6 +5396,7 @@ dependencies = [
"tracing",
"tracing-futures",
"zebra-chain",
"zebra-consensus",
"zebra-network",
"zebra-node-services",
"zebra-state",

View File

@ -7,6 +7,7 @@ edition = "2021"
[features]
default = []
getblocktemplate-rpcs = []
proptest-impl = ["proptest", "proptest-derive", "zebra-chain/proptest-impl", "zebra-state/proptest-impl"]
[dependencies]

View File

@ -93,6 +93,7 @@ where
V: Service<tx::Request, Response = tx::Response, Error = BoxError> + Send + Clone + 'static,
V::Future: Send + 'static,
{
/// Creates a new BlockVerifier
pub fn new(network: Network, state_service: S, transaction_verifier: V) -> Self {
Self {
network,

View File

@ -47,6 +47,7 @@ pub mod chain;
pub mod error;
pub use block::VerifyBlockError;
pub use chain::VerifyChainError;
pub use checkpoint::{
CheckpointList, VerifyCheckpointError, MAX_CHECKPOINT_BYTE_COUNT, MAX_CHECKPOINT_HEIGHT_GAP,
};

View File

@ -9,7 +9,7 @@ edition = "2021"
[features]
default = []
getblocktemplate-rpcs = ["zebra-state/getblocktemplate-rpcs", "zebra-node-services/getblocktemplate-rpcs"]
getblocktemplate-rpcs = ["zebra-state/getblocktemplate-rpcs", "zebra-node-services/getblocktemplate-rpcs", "zebra-consensus/getblocktemplate-rpcs"]
# Test-only features
proptest-impl = ["proptest", "proptest-derive", "zebra-chain/proptest-impl", "zebra-state/proptest-impl"]
@ -43,6 +43,7 @@ proptest = { version = "0.10.1", optional = true }
proptest-derive = { version = "0.3.0", optional = true }
zebra-chain = { path = "../zebra-chain" }
zebra-consensus = { path = "../zebra-consensus" }
zebra-network = { path = "../zebra-network" }
zebra-node-services = { path = "../zebra-node-services" }
zebra-state = { path = "../zebra-state" }
@ -57,5 +58,6 @@ thiserror = "1.0.37"
tokio = { version = "1.21.2", features = ["full", "tracing", "test-util"] }
zebra-chain = { path = "../zebra-chain", features = ["proptest-impl"] }
zebra-consensus = { path = "../zebra-consensus" }
zebra-state = { path = "../zebra-state", features = ["proptest-impl"] }
zebra-test = { path = "../zebra-test/" }

View File

@ -1,17 +1,26 @@
//! RPC methods related to mining only available with `getblocktemplate-rpcs` rust feature.
use std::sync::Arc;
use futures::{FutureExt, TryFutureExt};
use jsonrpc_core::{self, BoxFuture, Error, ErrorCode, Result};
use jsonrpc_derive::rpc;
use tower::{buffer::Buffer, Service, ServiceExt};
use zebra_chain::{amount::Amount, block::Height, chain_tip::ChainTip};
use zebra_chain::{
amount::Amount,
block::Height,
block::{self, Block},
chain_tip::ChainTip,
serialization::ZcashDeserializeInto,
};
use zebra_consensus::{BlockError, VerifyBlockError, VerifyChainError, VerifyCheckpointError};
use zebra_node_services::mempool;
use crate::methods::{
get_block_template_rpcs::types::{
default_roots::DefaultRoots, get_block_template::GetBlockTemplate,
transaction::TransactionTemplate,
default_roots::DefaultRoots, get_block_template::GetBlockTemplate, hex_data::HexData,
submit_block, transaction::TransactionTemplate,
},
GetBlockHash, MISSING_BLOCK_ERROR_CODE,
};
@ -74,10 +83,26 @@ pub trait GetBlockTemplateRpc {
/// This rpc method is available only if zebra is built with `--features getblocktemplate-rpcs`.
#[rpc(name = "getblocktemplate")]
fn get_block_template(&self) -> BoxFuture<Result<GetBlockTemplate>>;
/// Submits block to the node to be validated and committed.
/// Returns the [`submit_block::Response`] for the operation, as a JSON string.
///
/// zcashd reference: [`submitblock`](https://zcash.github.io/rpc/submitblock.html)
///
/// # Parameters
/// - `hexdata` (string, required)
/// - `jsonparametersobject` (string, optional) - currently ignored
/// - holds a single field, workid, that must be included in submissions if provided by the server.
#[rpc(name = "submitblock")]
fn submit_block(
&self,
hex_data: HexData,
_options: Option<submit_block::JsonParameters>,
) -> BoxFuture<Result<submit_block::Response>>;
}
/// RPC method implementations.
pub struct GetBlockTemplateRpcImpl<Mempool, State, Tip>
pub struct GetBlockTemplateRpcImpl<Mempool, State, Tip, ChainVerifier>
where
Mempool: Service<
mempool::Request,
@ -89,7 +114,11 @@ where
Response = zebra_state::ReadResponse,
Error = zebra_state::BoxError,
>,
Tip: ChainTip,
ChainVerifier: Service<Arc<Block>, Response = block::Hash, Error = zebra_consensus::BoxError>
+ Clone
+ Send
+ Sync
+ 'static,
{
// TODO: Add the other fields from the [`Rpc`] struct as-needed
@ -107,9 +136,12 @@ where
/// Allows efficient access to the best tip of the blockchain.
latest_chain_tip: Tip,
/// The chain verifier, used for submitting blocks.
chain_verifier: ChainVerifier,
}
impl<Mempool, State, Tip> GetBlockTemplateRpcImpl<Mempool, State, Tip>
impl<Mempool, State, Tip, ChainVerifier> GetBlockTemplateRpcImpl<Mempool, State, Tip, ChainVerifier>
where
Mempool: Service<
mempool::Request,
@ -125,22 +157,30 @@ where
+ Sync
+ 'static,
Tip: ChainTip + Clone + Send + Sync + 'static,
ChainVerifier: Service<Arc<Block>, Response = block::Hash, Error = zebra_consensus::BoxError>
+ Clone
+ Send
+ Sync
+ 'static,
{
/// Create a new instance of the handler for getblocktemplate RPCs.
pub fn new(
mempool: Buffer<Mempool, mempool::Request>,
state: State,
latest_chain_tip: Tip,
chain_verifier: ChainVerifier,
) -> Self {
Self {
mempool,
state,
latest_chain_tip,
chain_verifier,
}
}
}
impl<Mempool, State, Tip> GetBlockTemplateRpc for GetBlockTemplateRpcImpl<Mempool, State, Tip>
impl<Mempool, State, Tip, ChainVerifier> GetBlockTemplateRpc
for GetBlockTemplateRpcImpl<Mempool, State, Tip, ChainVerifier>
where
Mempool: Service<
mempool::Request,
@ -158,6 +198,12 @@ where
+ 'static,
<State as Service<zebra_state::ReadRequest>>::Future: Send,
Tip: ChainTip + Send + Sync + 'static,
ChainVerifier: Service<Arc<Block>, Response = block::Hash, Error = zebra_consensus::BoxError>
+ Clone
+ Send
+ Sync
+ 'static,
<ChainVerifier as Service<Arc<Block>>>::Future: Send,
{
fn get_block_count(&self) -> Result<u32> {
self.latest_chain_tip
@ -302,6 +348,80 @@ where
}
.boxed()
}
fn submit_block(
&self,
HexData(block_bytes): HexData,
_options: Option<submit_block::JsonParameters>,
) -> BoxFuture<Result<submit_block::Response>> {
let mut chain_verifier = self.chain_verifier.clone();
async move {
let block: Block = match block_bytes.zcash_deserialize_into() {
Ok(block_bytes) => block_bytes,
Err(_) => return Ok(submit_block::ErrorResponse::Rejected.into()),
};
let chain_verifier_response = chain_verifier
.ready()
.await
.map_err(|error| Error {
code: ErrorCode::ServerError(0),
message: error.to_string(),
data: None,
})?
.call(Arc::new(block))
.await;
let chain_error = match chain_verifier_response {
// Currently, this match arm returns `null` (Accepted) for blocks committed
// to any chain, but Accepted is only for blocks in the best chain.
//
// TODO (#5487):
// - Inconclusive: check if the block is on a side-chain
// The difference is important to miners, because they want to mine on the best chain.
Ok(_block_hash) => return Ok(submit_block::Response::Accepted),
// Turns BoxError into Result<VerifyChainError, BoxError>,
// by downcasting from Any to VerifyChainError.
Err(box_error) => box_error
.downcast::<VerifyChainError>()
.map(|boxed_chain_error| *boxed_chain_error),
};
Ok(match chain_error {
Ok(
VerifyChainError::Checkpoint(VerifyCheckpointError::AlreadyVerified { .. })
| VerifyChainError::Block(VerifyBlockError::Block {
source: BlockError::AlreadyInChain(..),
}),
) => submit_block::ErrorResponse::Duplicate,
// Currently, these match arms return Reject for the older duplicate in a queue,
// but queued duplicates should be DuplicateInconclusive.
//
// Optional TODO (#5487):
// - DuplicateInconclusive: turn these non-finalized state duplicate block errors
// into BlockError enum variants, and handle them as DuplicateInconclusive:
// - "block already sent to be committed to the state"
// - "replaced by newer request"
// - keep the older request in the queue,
// and return a duplicate error for the newer request immediately.
// This improves the speed of the RPC response.
//
// Checking the download queues and ChainVerifier buffer for duplicates
// might require architectural changes to Zebra, so we should only do it
// if mining pools really need it.
Ok(_verify_chain_error) => submit_block::ErrorResponse::Rejected,
// This match arm is currently unreachable, but if future changes add extra error types,
// we want to turn them into `Rejected`.
Err(_unknown_error_type) => submit_block::ErrorResponse::Rejected,
}
.into())
}
.boxed()
}
}
/// Given a potentially negative index, find the corresponding `Height`.

View File

@ -2,4 +2,6 @@
pub(crate) mod default_roots;
pub(crate) mod get_block_template;
pub(crate) mod hex_data;
pub(crate) mod submit_block;
pub(crate) mod transaction;

View File

@ -0,0 +1,6 @@
//! Deserializes hex-encoded inputs such as the one required
//! for the `submitblock` RPC method.
/// Deserialize hex-encoded strings to bytes.
#[derive(Debug, PartialEq, Eq, serde::Deserialize)]
pub struct HexData(#[serde(with = "hex")] pub Vec<u8>);

View File

@ -0,0 +1,47 @@
//! Parameter and response types for the `submitblock` RPC.
// Allow doc links to these imports.
#[allow(unused_imports)]
use crate::methods::get_block_template_rpcs::GetBlockTemplateRpc;
/// Optional argument `jsonparametersobject` for `submitblock` RPC request
///
/// See notes for [`GetBlockTemplateRpc::submit_block`] method
#[derive(Debug, serde::Deserialize)]
pub struct JsonParameters {
pub(crate) _work_id: Option<String>,
}
/// Response to a `submitblock` RPC request.
///
/// Zebra never returns "duplicate-invalid", because it does not store invalid blocks.
#[derive(Debug, PartialEq, Eq, serde::Serialize)]
#[serde(rename_all = "kebab-case")]
pub enum ErrorResponse {
/// Block was already committed to the non-finalized or finalized state
Duplicate,
/// Block was already added to the state queue or channel, but not yet committed to the non-finalized state
DuplicateInconclusive,
/// Block was already committed to the non-finalized state, but not on the best chain
Inconclusive,
/// Block rejected as invalid
Rejected,
}
/// Response to a `submitblock` RPC request.
///
/// Zebra never returns "duplicate-invalid", because it does not store invalid blocks.
#[derive(Debug, PartialEq, Eq, serde::Serialize)]
#[serde(untagged)]
pub enum Response {
/// Block was not successfully submitted, return error
ErrorResponse(ErrorResponse),
/// Block successfully submitted, returns null
Accepted,
}
impl From<ErrorResponse> for Response {
fn from(error_response: ErrorResponse) -> Self {
Self::ErrorResponse(error_response)
}
}

View File

@ -46,7 +46,8 @@ async fn test_rpc_response_data_for_network(network: Network) {
let mut mempool: MockService<_, _, _, zebra_node_services::BoxError> =
MockService::build().for_unit_tests();
// Create a populated state service
let (_state, read_state, latest_chain_tip, _chain_tip_change) =
#[cfg_attr(not(feature = "getblocktemplate-rpcs"), allow(unused_variables))]
let (state, read_state, latest_chain_tip, _chain_tip_change) =
zebra_state::populated_state(blocks.clone(), network).await;
// Start snapshots of RPC responses.
@ -56,7 +57,9 @@ async fn test_rpc_response_data_for_network(network: Network) {
// Test getblocktemplate-rpcs snapshots
#[cfg(feature = "getblocktemplate-rpcs")]
get_block_template_rpcs::test_responses(
network,
mempool.clone(),
state,
read_state.clone(),
latest_chain_tip.clone(),
settings.clone(),

View File

@ -8,25 +8,40 @@
use insta::Settings;
use tower::{buffer::Buffer, Service};
use zebra_chain::parameters::Network;
use zebra_node_services::mempool;
use zebra_state::LatestChainTip;
use zebra_test::mock_service::{MockService, PanicAssertion};
use crate::methods::{GetBlockHash, GetBlockTemplateRpc, GetBlockTemplateRpcImpl};
use crate::methods::{
get_block_template_rpcs::types::{hex_data::HexData, submit_block},
GetBlockHash, GetBlockTemplateRpc, GetBlockTemplateRpcImpl,
};
pub async fn test_responses<State>(
pub async fn test_responses<State, ReadState>(
network: Network,
mut mempool: MockService<
mempool::Request,
mempool::Response,
PanicAssertion,
zebra_node_services::BoxError,
>,
read_state: State,
state: State,
read_state: ReadState,
latest_chain_tip: LatestChainTip,
settings: Settings,
) where
State: Service<
zebra_state::Request,
Response = zebra_state::Response,
Error = zebra_state::BoxError,
> + Clone
+ Send
+ Sync
+ 'static,
<State as Service<zebra_state::Request>>::Future: Send,
ReadState: Service<
zebra_state::ReadRequest,
Response = zebra_state::ReadResponse,
Error = zebra_state::BoxError,
@ -34,12 +49,26 @@ pub async fn test_responses<State>(
+ Send
+ Sync
+ 'static,
<State as Service<zebra_state::ReadRequest>>::Future: Send,
<ReadState as Service<zebra_state::ReadRequest>>::Future: Send,
{
let (
chain_verifier,
_transaction_verifier,
_parameter_download_task_handle,
_max_checkpoint_height,
) = zebra_consensus::chain::init(
zebra_consensus::Config::default(),
network,
state.clone(),
true,
)
.await;
let get_block_template_rpc = GetBlockTemplateRpcImpl::new(
Buffer::new(mempool.clone(), 1),
read_state,
latest_chain_tip,
chain_verifier,
);
// `getblockcount`
@ -71,6 +100,14 @@ pub async fn test_responses<State>(
.expect("unexpected error in getblocktemplate RPC call");
snapshot_rpc_getblocktemplate(get_block_template, &settings);
// `submitblock`
let submit_block = get_block_template_rpc
.submit_block(HexData("".into()), None)
.await
.expect("unexpected error in submitblock RPC call");
snapshot_rpc_submit_block_invalid(submit_block, &settings);
}
/// Snapshot `getblockcount` response, using `cargo insta` and JSON serialization.
@ -90,3 +127,13 @@ fn snapshot_rpc_getblocktemplate(
) {
settings.bind(|| insta::assert_json_snapshot!("get_block_template", block_template));
}
/// Snapshot `submitblock` response, using `cargo insta` and JSON serialization.
fn snapshot_rpc_submit_block_invalid(
submit_block_response: submit_block::Response,
settings: &insta::Settings,
) {
settings.bind(|| {
insta::assert_json_snapshot!("snapshot_rpc_submit_block_invalid", submit_block_response)
});
}

View File

@ -0,0 +1,5 @@
---
source: zebra-rpc/src/methods/tests/snapshot/get_block_template_rpcs.rs
expression: submit_block_response
---
"rejected"

View File

@ -0,0 +1,5 @@
---
source: zebra-rpc/src/methods/tests/snapshot/get_block_template_rpcs.rs
expression: submit_block_response
---
"rejected"

View File

@ -632,14 +632,28 @@ async fn rpc_getblockcount() {
// Get a mempool handle
let mut mempool: MockService<_, _, _, BoxError> = MockService::build().for_unit_tests();
// Create a populated state service, the tip will be in `NUMBER_OF_BLOCKS`.
let (_state, read_state, latest_chain_tip, _chain_tip_change) =
let (state, read_state, latest_chain_tip, _chain_tip_change) =
zebra_state::populated_state(blocks.clone(), Mainnet).await;
let (
chain_verifier,
_transaction_verifier,
_parameter_download_task_handle,
_max_checkpoint_height,
) = zebra_consensus::chain::init(
zebra_consensus::Config::default(),
Mainnet,
state.clone(),
true,
)
.await;
// Init RPC
let get_block_template_rpc = get_block_template_rpcs::GetBlockTemplateRpcImpl::new(
Buffer::new(mempool.clone(), 1),
read_state,
latest_chain_tip.clone(),
chain_verifier,
);
// Get the tip height using RPC method `get_block_count`
@ -661,14 +675,28 @@ async fn rpc_getblockcount_empty_state() {
// Get a mempool handle
let mut mempool: MockService<_, _, _, BoxError> = MockService::build().for_unit_tests();
// Create an empty state
let (_state, read_state, latest_chain_tip, _chain_tip_change) =
let (state, read_state, latest_chain_tip, _chain_tip_change) =
zebra_state::init_test_services(Mainnet);
let (
chain_verifier,
_transaction_verifier,
_parameter_download_task_handle,
_max_checkpoint_height,
) = zebra_consensus::chain::init(
zebra_consensus::Config::default(),
Mainnet,
state.clone(),
true,
)
.await;
// Init RPC
let get_block_template_rpc = get_block_template_rpcs::GetBlockTemplateRpcImpl::new(
Buffer::new(mempool.clone(), 1),
read_state,
latest_chain_tip.clone(),
chain_verifier,
);
// Get the tip height using RPC method `get_block_count
@ -696,14 +724,28 @@ async fn rpc_getblockhash() {
let mut mempool: MockService<_, _, _, BoxError> = MockService::build().for_unit_tests();
// Create a populated state service
let (_state, read_state, latest_chain_tip, _chain_tip_change) =
let (state, read_state, latest_chain_tip, _chain_tip_change) =
zebra_state::populated_state(blocks.clone(), Mainnet).await;
let (
chain_verifier,
_transaction_verifier,
_parameter_download_task_handle,
_max_checkpoint_height,
) = zebra_consensus::chain::init(
zebra_consensus::Config::default(),
Mainnet,
state.clone(),
true,
)
.await;
// Init RPC
let get_block_template_rpc = get_block_template_rpcs::GetBlockTemplateRpcImpl::new(
Buffer::new(mempool.clone(), 1),
read_state,
latest_chain_tip.clone(),
tower::ServiceBuilder::new().service(chain_verifier),
);
// Query the hashes using positive indexes
@ -745,14 +787,28 @@ async fn rpc_getblocktemplate() {
let mut mempool: MockService<_, _, _, BoxError> = MockService::build().for_unit_tests();
// Create a populated state service
let (_state, read_state, latest_chain_tip, _chain_tip_change) =
let (state, read_state, latest_chain_tip, _chain_tip_change) =
zebra_state::populated_state(blocks.clone(), Mainnet).await;
let (
chain_verifier,
_transaction_verifier,
_parameter_download_task_handle,
_max_checkpoint_height,
) = zebra_consensus::chain::init(
zebra_consensus::Config::default(),
Mainnet,
state.clone(),
true,
)
.await;
// Init RPC
let get_block_template_rpc = get_block_template_rpcs::GetBlockTemplateRpcImpl::new(
Buffer::new(mempool.clone(), 1),
read_state,
latest_chain_tip.clone(),
tower::ServiceBuilder::new().service(chain_verifier),
);
let get_block_template = tokio::spawn(get_block_template_rpc.get_block_template());
@ -782,3 +838,84 @@ async fn rpc_getblocktemplate() {
mempool.expect_no_requests().await;
}
#[cfg(feature = "getblocktemplate-rpcs")]
#[tokio::test(flavor = "multi_thread")]
async fn rpc_submitblock_errors() {
let _init_guard = zebra_test::init();
// Create a continuous chain of mainnet blocks from genesis
let blocks: Vec<Arc<Block>> = zebra_test::vectors::CONTINUOUS_MAINNET_BLOCKS
.iter()
.map(|(_height, block_bytes)| block_bytes.zcash_deserialize_into().unwrap())
.collect();
let mut mempool: MockService<_, _, _, BoxError> = MockService::build().for_unit_tests();
// Create a populated state service
let (state, read_state, latest_chain_tip, _chain_tip_change) =
zebra_state::populated_state(blocks, Mainnet).await;
// Init RPCs
let _rpc = RpcImpl::new(
"RPC test",
Mainnet,
false,
Buffer::new(mempool.clone(), 1),
Buffer::new(read_state.clone(), 1),
latest_chain_tip.clone(),
);
let (
chain_verifier,
_transaction_verifier,
_parameter_download_task_handle,
_max_checkpoint_height,
) = zebra_consensus::chain::init(
zebra_consensus::Config::default(),
Mainnet,
state.clone(),
true,
)
.await;
// Init RPC
let get_block_template_rpc = get_block_template_rpcs::GetBlockTemplateRpcImpl::new(
Buffer::new(mempool.clone(), 1),
read_state,
latest_chain_tip.clone(),
tower::ServiceBuilder::new().service(chain_verifier),
);
// Try to submit pre-populated blocks and assert that it responds with duplicate.
for (_height, &block_bytes) in zebra_test::vectors::CONTINUOUS_MAINNET_BLOCKS.iter() {
let submit_block_response = get_block_template_rpc
.submit_block(
get_block_template_rpcs::types::hex_data::HexData(block_bytes.into()),
None,
)
.await;
assert_eq!(
submit_block_response,
Ok(get_block_template_rpcs::types::submit_block::ErrorResponse::Duplicate.into())
);
}
let submit_block_response = get_block_template_rpc
.submit_block(
get_block_template_rpcs::types::hex_data::HexData(
zebra_test::vectors::BAD_BLOCK_MAINNET_202_BYTES.to_vec(),
),
None,
)
.await;
assert_eq!(
submit_block_response,
Ok(get_block_template_rpcs::types::submit_block::ErrorResponse::Rejected.into())
);
mempool.expect_no_requests().await;
// See zebrad::tests::acceptance::submit_block for success case.
}

View File

@ -7,17 +7,22 @@
//! See the full list of
//! [Differences between JSON-RPC 1.0 and 2.0.](https://www.simple-is-better.org/rpc/#differences-between-1-0-and-2-0)
use std::panic;
use std::{panic, sync::Arc};
use jsonrpc_core::{Compatibility, MetaIoHandler};
use jsonrpc_http_server::ServerBuilder;
use tokio::task::JoinHandle;
use tower::{buffer::Buffer, Service};
use tracing::*;
use tracing_futures::Instrument;
use zebra_chain::{chain_tip::ChainTip, parameters::Network};
use zebra_node_services::{mempool, BoxError};
use zebra_chain::{
block::{self, Block},
chain_tip::ChainTip,
parameters::Network,
};
use zebra_node_services::mempool;
use crate::{
config::Config,
@ -40,18 +45,23 @@ pub struct RpcServer;
impl RpcServer {
/// Start a new RPC server endpoint
pub fn spawn<Version, Mempool, State, Tip>(
pub fn spawn<Version, Mempool, State, Tip, ChainVerifier>(
config: Config,
app_version: Version,
mempool: Buffer<Mempool, mempool::Request>,
state: State,
#[cfg_attr(not(feature = "getblocktemplate-rpcs"), allow(unused_variables))]
chain_verifier: ChainVerifier,
latest_chain_tip: Tip,
network: Network,
) -> (JoinHandle<()>, JoinHandle<()>)
where
Version: ToString + Clone,
Mempool: tower::Service<mempool::Request, Response = mempool::Response, Error = BoxError>
+ 'static,
Mempool: tower::Service<
mempool::Request,
Response = mempool::Response,
Error = zebra_node_services::BoxError,
> + 'static,
Mempool::Future: Send,
State: Service<
zebra_state::ReadRequest,
@ -63,6 +73,12 @@ impl RpcServer {
+ 'static,
State::Future: Send,
Tip: ChainTip + Clone + Send + Sync + 'static,
ChainVerifier: Service<Arc<Block>, Response = block::Hash, Error = zebra_consensus::BoxError>
+ Clone
+ Send
+ Sync
+ 'static,
<ChainVerifier as Service<Arc<Block>>>::Future: Send,
{
if let Some(listen_addr) = config.listen_addr {
info!("Trying to open RPC endpoint at {}...", listen_addr,);
@ -78,6 +94,7 @@ impl RpcServer {
mempool.clone(),
state.clone(),
latest_chain_tip.clone(),
chain_verifier,
);
io.extend_with(get_block_template_rpc_impl.to_delegate());

View File

@ -46,6 +46,8 @@ fn rpc_server_spawn(parallel_cpu_threads: bool) {
rt.block_on(async {
let mut mempool: MockService<_, _, _, BoxError> = MockService::build().for_unit_tests();
let mut state: MockService<_, _, _, BoxError> = MockService::build().for_unit_tests();
let mut chain_verifier: MockService<_, _, _, BoxError> =
MockService::build().for_unit_tests();
info!("spawning RPC server...");
@ -54,6 +56,7 @@ fn rpc_server_spawn(parallel_cpu_threads: bool) {
"RPC server test",
Buffer::new(mempool.clone(), 1),
Buffer::new(state.clone(), 1),
Buffer::new(chain_verifier.clone(), 1),
NoChainTip,
Mainnet,
);
@ -62,6 +65,7 @@ fn rpc_server_spawn(parallel_cpu_threads: bool) {
mempool.expect_no_requests().await;
state.expect_no_requests().await;
chain_verifier.expect_no_requests().await;
// The server and queue tasks should continue without errors or panics
let rpc_server_task_result = rpc_server_task_handle.now_or_never();
@ -113,6 +117,8 @@ fn rpc_server_spawn_unallocated_port(parallel_cpu_threads: bool) {
rt.block_on(async {
let mut mempool: MockService<_, _, _, BoxError> = MockService::build().for_unit_tests();
let mut state: MockService<_, _, _, BoxError> = MockService::build().for_unit_tests();
let mut chain_verifier: MockService<_, _, _, BoxError> =
MockService::build().for_unit_tests();
info!("spawning RPC server...");
@ -121,6 +127,7 @@ fn rpc_server_spawn_unallocated_port(parallel_cpu_threads: bool) {
"RPC server test",
Buffer::new(mempool.clone(), 1),
Buffer::new(state.clone(), 1),
Buffer::new(chain_verifier.clone(), 1),
NoChainTip,
Mainnet,
);
@ -129,6 +136,7 @@ fn rpc_server_spawn_unallocated_port(parallel_cpu_threads: bool) {
mempool.expect_no_requests().await;
state.expect_no_requests().await;
chain_verifier.expect_no_requests().await;
// The server and queue tasks should continue without errors or panics
let rpc_server_task_result = rpc_server_task_handle.now_or_never();
@ -167,6 +175,8 @@ fn rpc_server_spawn_port_conflict() {
let test_task_handle = rt.spawn(async {
let mut mempool: MockService<_, _, _, BoxError> = MockService::build().for_unit_tests();
let mut state: MockService<_, _, _, BoxError> = MockService::build().for_unit_tests();
let mut chain_verifier: MockService<_, _, _, BoxError> =
MockService::build().for_unit_tests();
info!("spawning RPC server 1...");
@ -175,6 +185,7 @@ fn rpc_server_spawn_port_conflict() {
"RPC server 1 test",
Buffer::new(mempool.clone(), 1),
Buffer::new(state.clone(), 1),
Buffer::new(chain_verifier.clone(), 1),
NoChainTip,
Mainnet,
);
@ -188,6 +199,7 @@ fn rpc_server_spawn_port_conflict() {
"RPC server 2 conflict test",
Buffer::new(mempool.clone(), 1),
Buffer::new(state.clone(), 1),
Buffer::new(chain_verifier.clone(), 1),
NoChainTip,
Mainnet,
);
@ -196,6 +208,7 @@ fn rpc_server_spawn_port_conflict() {
mempool.expect_no_requests().await;
state.expect_no_requests().await;
chain_verifier.expect_no_requests().await;
// Because there is a panic inside a multi-threaded executor,
// we can't depend on the exact behaviour of the other tasks,
@ -263,6 +276,8 @@ fn rpc_server_spawn_port_conflict_parallel_auto() {
let test_task_handle = rt.spawn(async {
let mut mempool: MockService<_, _, _, BoxError> = MockService::build().for_unit_tests();
let mut state: MockService<_, _, _, BoxError> = MockService::build().for_unit_tests();
let mut chain_verifier: MockService<_, _, _, BoxError> =
MockService::build().for_unit_tests();
info!("spawning parallel RPC server 1...");
@ -271,6 +286,7 @@ fn rpc_server_spawn_port_conflict_parallel_auto() {
"RPC server 1 test",
Buffer::new(mempool.clone(), 1),
Buffer::new(state.clone(), 1),
Buffer::new(chain_verifier.clone(), 1),
NoChainTip,
Mainnet,
);
@ -284,6 +300,7 @@ fn rpc_server_spawn_port_conflict_parallel_auto() {
"RPC server 2 conflict test",
Buffer::new(mempool.clone(), 1),
Buffer::new(state.clone(), 1),
Buffer::new(chain_verifier.clone(), 1),
NoChainTip,
Mainnet,
);
@ -292,6 +309,7 @@ fn rpc_server_spawn_port_conflict_parallel_auto() {
mempool.expect_no_requests().await;
state.expect_no_requests().await;
chain_verifier.expect_no_requests().await;
// Because there might be a panic inside a multi-threaded executor,
// we can't depend on the exact behaviour of the other tasks,

View File

@ -181,6 +181,7 @@ impl StartCmd {
app_version(),
mempool.clone(),
read_only_state_service,
chain_verifier.clone(),
latest_chain_tip.clone(),
config.network.network,
);

View File

@ -101,6 +101,14 @@
//! $ cargo test lightwalletd_wallet_grpc_tests --features lightwalletd-grpc-tests -- --ignored --nocapture
//! ```
//!
//! ## Getblocktemplate tests
//!
//! Example of how to run the submit_block test:
//!
//! ```console
//! ZEBRA_CACHED_STATE_DIR=/path/to/zebra/chain cargo test submit_block --features getblocktemplate-rpcs --release -- --ignored --nocapture
//! ```
//!
//! Please refer to the documentation of each test for more information.
//!
//! ## Disk Space for Testing
@ -2157,3 +2165,13 @@ async fn sending_transactions_using_lightwalletd() -> Result<()> {
async fn lightwalletd_wallet_grpc_tests() -> Result<()> {
common::lightwalletd::wallet_grpc_test::run().await
}
/// Test successful submitblock rpc call
///
/// See [`common::getblocktemplate`] for more information.
#[tokio::test]
#[ignore]
#[cfg(feature = "getblocktemplate-rpcs")]
async fn submit_block() -> Result<()> {
common::get_block_template_rpcs::submit_block::run().await
}

View File

@ -0,0 +1,5 @@
//! Acceptance tests for getblocktemplate RPC methods in Zebra.
use super::*;
pub(crate) mod submit_block;

View File

@ -0,0 +1,176 @@
//! Test submitblock RPC method.
//!
//! This test requires a cached chain state that is synchronized past the max checkpoint height,
//! and will sync to the next block without updating the cached chain state.
// TODO: Update this test and the doc to:
//
// This test requires a cached chain state that is partially synchronized close to the
// network chain tip height, and will finish the sync and update the cached chain state.
//
// After finishing the sync, it will get the first 20 blocks in the non-finalized state
// (past the MAX_BLOCK_REORG_HEIGHT) via getblock rpc calls, get the finalized tip height
// of the updated cached state, restart zebra without peers, and submit blocks above the
// finalized tip height.
use std::path::PathBuf;
use color_eyre::eyre::{eyre, Context, Result};
use futures::TryFutureExt;
use indexmap::IndexSet;
use reqwest::Client;
use tower::{Service, ServiceExt};
use zebra_chain::{block::Height, parameters::Network, serialization::ZcashSerialize};
use zebra_state::HashOrHeight;
use zebra_test::args;
use crate::common::{
cached_state::{copy_state_directory, start_state_service_with_cache_dir},
config::{persistent_test_config, testdir},
launch::ZebradTestDirExt,
lightwalletd::random_known_rpc_port_config,
};
use super::cached_state::{load_tip_height_from_state_directory, ZEBRA_CACHED_STATE_DIR};
async fn get_future_block_hex_data(
network: Network,
zebrad_state_path: &PathBuf,
) -> Result<Option<String>> {
tracing::info!(
?zebrad_state_path,
"getting cached sync height from ZEBRA_CACHED_STATE_DIR path"
);
let cached_sync_height =
load_tip_height_from_state_directory(network, zebrad_state_path.as_ref()).await?;
let future_block_height = Height(cached_sync_height.0 + 1);
tracing::info!(
?cached_sync_height,
?future_block_height,
"got cached sync height, copying state dir to tempdir"
);
let copied_state_path = copy_state_directory(network, &zebrad_state_path).await?;
let mut config = persistent_test_config()?;
config.state.debug_stop_at_height = Some(future_block_height.0);
let mut child = copied_state_path
.with_config(&mut config)?
.spawn_child(args!["start"])?
.bypass_test_capture(true);
while child.is_running() {
tokio::task::yield_now().await;
}
let _ = child.kill(true);
let copied_state_path = child.dir.take().unwrap();
let (_read_write_state_service, mut state, _latest_chain_tip, _chain_tip_change) =
start_state_service_with_cache_dir(network, copied_state_path.as_ref()).await?;
let request = zebra_state::ReadRequest::Block(HashOrHeight::Height(future_block_height));
let response = state
.ready()
.and_then(|ready_service| ready_service.call(request))
.map_err(|error| eyre!(error))
.await?;
let block_hex_data = match response {
zebra_state::ReadResponse::Block(Some(block)) => {
hex::encode(block.zcash_serialize_to_vec()?)
}
zebra_state::ReadResponse::Block(None) => {
tracing::info!(
"Reached the end of the finalized chain, state is missing block at {future_block_height:?}",
);
return Ok(None);
}
_ => unreachable!("Incorrect response from state service: {response:?}"),
};
Ok(Some(block_hex_data))
}
#[allow(clippy::print_stderr)]
pub(crate) async fn run() -> Result<(), color_eyre::Report> {
let _init_guard = zebra_test::init();
let mut config = random_known_rpc_port_config(true)?;
let network = config.network.network;
let rpc_address = config.rpc.listen_addr.unwrap();
config.state.cache_dir = match std::env::var_os(ZEBRA_CACHED_STATE_DIR) {
Some(path) => path.into(),
None => {
eprintln!(
"skipped submitblock test, \
set the {ZEBRA_CACHED_STATE_DIR:?} environment variable to run the test",
);
return Ok(());
}
};
// TODO: As part of or as a pre-cursor to issue #5015,
// - Use only original cached state,
// - sync until the tip
// - get first 3 blocks in non-finalized state via getblock rpc calls
// - restart zebra without peers
// - submit block(s) above the finalized tip height
let block_hex_data = get_future_block_hex_data(network, &config.state.cache_dir)
.await?
.expect(
"spawned zebrad in get_future_block_hex_data should live until it gets the next block",
);
// Runs the rest of this test without an internet connection
config.network.initial_mainnet_peers = IndexSet::new();
config.network.initial_testnet_peers = IndexSet::new();
config.mempool.debug_enable_at_height = Some(0);
// We're using the cached state
config.state.ephemeral = false;
let mut child = testdir()?
.with_exact_config(&config)?
.spawn_child(args!["start"])?
.bypass_test_capture(true);
child.expect_stdout_line_matches(&format!("Opened RPC endpoint at {rpc_address}"))?;
// Create an http client
let client = Client::new();
let res = client
.post(format!("http://{}", &rpc_address))
.body(format!(
r#"{{"jsonrpc": "2.0", "method": "submitblock", "params": ["{block_hex_data}"], "id":123 }}"#
))
.header("Content-Type", "application/json")
.send()
.await?;
assert!(res.status().is_success());
let res_text = res.text().await?;
// Test rpc endpoint response
assert!(res_text.contains(r#""result":"null""#));
child.kill(false)?;
let output = child.wait_with_output()?;
let output = output.assert_failure()?;
// [Note on port conflict](#Note on port conflict)
output
.assert_was_killed()
.wrap_err("Possible port conflict. Are there other acceptance tests running?")?;
Ok(())
}

View File

@ -13,6 +13,8 @@ pub mod cached_state;
pub mod check;
pub mod config;
pub mod failure_messages;
#[cfg(feature = "getblocktemplate-rpcs")]
pub mod get_block_template_rpcs;
pub mod launch;
pub mod lightwalletd;
pub mod sync;