change(rpc): Return from long polling immediately when the chain tip changes (#5862)

* Add constants and fix comments for mempool timer / state watch

* Add a best_tip_changed() method to trait ChainTip

* Check for chain tip changes using a future, and return on error

* Ignore state changes before the most recent state fetch

* Add a submit old field to the getblocktemplate RPC during long polling

* Make the submit_old field optional, rather than `null` in JSON

* Update some TODOs

* Add long polling snapshot tests, use RON for deserialized coinbase fields

* Clarify some comments

* Simplify the BestTipChanged future implementations

* Fix some comment typos
This commit is contained in:
teor 2022-12-16 01:33:00 +10:00 committed by GitHub
parent f7011a903e
commit 80a6d3cdab
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
20 changed files with 644 additions and 145 deletions

View File

@ -1,10 +1,11 @@
//! Zebra interfaces for access to chain tip information.
use std::sync::Arc;
use std::{future, sync::Arc};
use chrono::{DateTime, Utc};
use futures::{future::BoxFuture, Future, FutureExt};
use crate::{block, parameters::Network, transaction};
use crate::{block, parameters::Network, transaction, BoxError};
mod network_chain_tip_height_estimator;
@ -18,32 +19,65 @@ use network_chain_tip_height_estimator::NetworkChainTipHeightEstimator;
/// An interface for querying the chain tip.
///
/// This trait helps avoid dependencies between:
/// * zebra-chain and tokio
/// * zebra-network and zebra-state
/// * `zebra-chain` and `tokio`
/// * `zebra-network` and `zebra-state`
pub trait ChainTip {
/// Return the height of the best chain tip.
/// Returns the height of the best chain tip.
///
/// Does not mark the best tip as seen.
fn best_tip_height(&self) -> Option<block::Height>;
/// Return the block hash of the best chain tip.
/// Returns the block hash of the best chain tip.
///
/// Does not mark the best tip as seen.
fn best_tip_hash(&self) -> Option<block::Hash>;
/// Return the height and the hash of the best chain tip.
/// Returns the height and the hash of the best chain tip.
///
/// Does not mark the best tip as seen.
fn best_tip_height_and_hash(&self) -> Option<(block::Height, block::Hash)>;
/// Return the block time of the best chain tip.
/// Returns the block time of the best chain tip.
///
/// Does not mark the best tip as seen.
fn best_tip_block_time(&self) -> Option<DateTime<Utc>>;
/// Return the height and the block time of the best chain tip.
///
/// Returns the height and the block time of the best chain tip.
/// Returning both values at the same time guarantees that they refer to the same chain tip.
///
/// Does not mark the best tip as seen.
fn best_tip_height_and_block_time(&self) -> Option<(block::Height, DateTime<Utc>)>;
/// Return the mined transaction IDs of the transactions in the best chain tip block.
/// Returns the mined transaction IDs of the transactions in the best chain tip block.
///
/// All transactions with these mined IDs should be rejected from the mempool,
/// even if their authorizing data is different.
///
/// Does not mark the best tip as seen.
fn best_tip_mined_transaction_ids(&self) -> Arc<[transaction::Hash]>;
/// A future that returns when the best chain tip changes.
/// Can return immediately if the latest value in this [`ChainTip`] has not been seen yet.
///
/// Marks the best tip as seen.
///
/// Returns an error if Zebra is shutting down, or the state has permanently failed.
///
/// See [`tokio::watch::Receiver::changed()`](https://docs.rs/tokio/latest/tokio/sync/watch/struct.Receiver.html#method.changed) for details.
//
// TODO:
// Use async_fn_in_trait or return_position_impl_trait_in_trait when one of them stabilises:
// https://github.com/rust-lang/rust/issues/91611
fn best_tip_changed(&mut self) -> BestTipChanged;
/// Mark the current best tip as seen.
///
/// Later calls to [`ChainTip::best_tip_changed()`] will wait for the next change
/// before returning.
fn mark_best_tip_seen(&mut self);
// Provided methods
//
/// Return an estimate of the network chain tip's height.
///
/// The estimate is calculated based on the current local time, the block time of the best tip
@ -84,7 +118,34 @@ pub trait ChainTip {
}
}
/// A chain tip that is always empty.
/// A future for the [`ChainTip::best_tip_changed()`] method.
/// See that method for details.
pub struct BestTipChanged<'f> {
fut: BoxFuture<'f, Result<(), BoxError>>,
}
impl<'f> BestTipChanged<'f> {
/// Returns a new [`BestTipChanged`] containing `fut`.
pub fn new<Fut>(fut: Fut) -> Self
where
Fut: Future<Output = Result<(), BoxError>> + Send + 'f,
{
Self { fut: Box::pin(fut) }
}
}
impl<'f> Future for BestTipChanged<'f> {
type Output = Result<(), BoxError>;
fn poll(
mut self: std::pin::Pin<&mut Self>,
cx: &mut std::task::Context<'_>,
) -> std::task::Poll<Self::Output> {
self.fut.poll_unpin(cx)
}
}
/// A chain tip that is always empty and never changes.
///
/// Used in production for isolated network connections,
/// and as a mock chain tip in tests.
@ -115,4 +176,12 @@ impl ChainTip for NoChainTip {
fn best_tip_mined_transaction_ids(&self) -> Arc<[transaction::Hash]> {
Arc::new([])
}
/// The [`NoChainTip`] best tip never changes, so this never returns.
fn best_tip_changed(&mut self) -> BestTipChanged {
BestTipChanged::new(future::pending())
}
/// The [`NoChainTip`] best tip never changes, so this does nothing.
fn mark_best_tip_seen(&mut self) {}
}

View File

@ -3,11 +3,19 @@
use std::sync::Arc;
use chrono::{DateTime, Utc};
use futures::{future, FutureExt, TryFutureExt};
use tokio::sync::watch;
use crate::{block, chain_tip::ChainTip, parameters::Network, transaction};
use crate::{
block,
chain_tip::{BestTipChanged, ChainTip},
parameters::Network,
transaction,
};
/// A sender to sets the values read by a [`MockChainTip`].
//
// Update `best_tip_changed()` for each new field that is added to MockChainTipSender.
pub struct MockChainTipSender {
/// A sender that sets the `best_tip_height` of a [`MockChainTip`].
best_tip_height: watch::Sender<Option<block::Height>>,
@ -112,6 +120,40 @@ impl ChainTip for MockChainTip {
.map(|tip_height| (estimated_distance, tip_height))
})
}
/// Returns when any sender channel changes.
/// Returns an error if any sender was dropped.
///
/// Marks the changed channel as seen when the returned future completes.
//
// Update this method when each new mock field is added.
fn best_tip_changed(&mut self) -> BestTipChanged {
// A future that returns when the first watch channel has changed
let select_changed = future::select_all([
// Erase the differing future types for each channel, and map their error types
BestTipChanged::new(self.best_tip_height.changed().err_into()),
BestTipChanged::new(self.best_tip_hash.changed().err_into()),
BestTipChanged::new(self.best_tip_block_time.changed().err_into()),
BestTipChanged::new(
self.estimated_distance_to_network_chain_tip
.changed()
.err_into(),
),
])
// Map the select result to the expected type, dropping the unused channels
.map(|(changed_result, _changed_index, _remaining_futures)| changed_result);
BestTipChanged::new(select_changed)
}
/// Marks all sender channels as seen.
fn mark_best_tip_seen(&mut self) {
self.best_tip_height.borrow_and_update();
self.best_tip_hash.borrow_and_update();
self.best_tip_block_time.borrow_and_update();
self.estimated_distance_to_network_chain_tip
.borrow_and_update();
}
}
impl MockChainTipSender {

View File

@ -70,11 +70,12 @@ zebra-script = { path = "../zebra-script" }
zebra-state = { path = "../zebra-state" }
[dev-dependencies]
insta = { version = "1.23.0", features = ["redactions", "json"] }
insta = { version = "1.23.0", features = ["redactions", "json", "ron"] }
proptest = "0.10.1"
proptest-derive = "0.3.0"
thiserror = "1.0.37"
thiserror = "1.0.37"
tokio = { version = "1.23.0", features = ["full", "tracing", "test-util"] }
zebra-chain = { path = "../zebra-chain", features = ["proptest-impl"] }

View File

@ -22,7 +22,9 @@ use zebra_state::{ReadRequest, ReadResponse};
use crate::methods::{
best_chain_tip_height,
get_block_template_rpcs::{
constants::DEFAULT_SOLUTION_RATE_WINDOW_SIZE,
constants::{
DEFAULT_SOLUTION_RATE_WINDOW_SIZE, GET_BLOCK_TEMPLATE_MEMPOOL_LONG_POLL_INTERVAL,
},
get_block_template::{
check_block_template_parameters, check_miner_address, check_synced_to_tip,
fetch_mempool_transactions, fetch_state_tip_and_local_time,
@ -321,7 +323,7 @@ where
// Clone Services
let mempool = self.mempool.clone();
let latest_chain_tip = self.latest_chain_tip.clone();
let mut latest_chain_tip = self.latest_chain_tip.clone();
let sync_status = self.sync_status.clone();
let state = self.state.clone();
@ -347,33 +349,38 @@ where
// The loop returns the server long poll ID,
// which should be different to the client long poll ID.
let (server_long_poll_id, chain_tip_and_local_time, mempool_txs) = loop {
let (server_long_poll_id, chain_tip_and_local_time, mempool_txs, submit_old) = loop {
// Check if we are synced to the tip.
// The result of this check can change during long polling.
//
// TODO:
// - add `async changed()` methods to ChainTip and ChainSyncStatus
// (using `changed() -> Changed` and `impl Future<()> for Changed`)
// Optional TODO:
// - add `async changed()` method to ChainSyncStatus (like `ChainTip`)
check_synced_to_tip(network, latest_chain_tip.clone(), sync_status.clone())?;
// We're just about to fetch state data, then maybe wait for any changes.
// Mark all the changes before the fetch as seen.
// Changes are also ignored in any clones made after the mark.
latest_chain_tip.mark_best_tip_seen();
// Fetch the state data and local time for the block template:
// - if the tip block hash changes, we must return from long polling,
// - if the local clock changes on testnet, we might return from long polling
//
// We always return after 90 minutes on mainnet, even if we have the same response,
// because the max time has been reached.
//
// TODO: timeout and exit the loop when max time is reached
let chain_tip_and_local_time =
fetch_state_tip_and_local_time(state.clone()).await?;
// Fetch the mempool data for the block template:
// - if the mempool transactions change, we might return from long polling.
//
// TODO:
// - add a `MempoolChange` type with an `async changed()` method.
// - if we are long polling, pause between state and mempool,
// to allow transactions to re-verify (only works after PR #5841)
// If the chain fork has just changed, miners want to get the new block as fast
// as possible, rather than wait for transactions to re-verify. This increases
// miner profits (and any delays can cause chain forks). So we don't wait between
// the chain tip changing and getting mempool transactions.
//
// Optional TODO:
// - add a `MempoolChange` type with an `async changed()` method (like `ChainTip`)
let mempool_txs = fetch_mempool_transactions(mempool.clone()).await?;
// - Long poll ID calculation
@ -390,14 +397,53 @@ where
// - the server long poll ID is different to the client long poll ID, or
// - the previous loop iteration waited until the max time.
if Some(&server_long_poll_id) != client_long_poll_id.as_ref() || max_time_reached {
break (server_long_poll_id, chain_tip_and_local_time, mempool_txs);
let mut submit_old = client_long_poll_id
.as_ref()
.map(|old_long_poll_id| server_long_poll_id.submit_old(old_long_poll_id));
// On testnet, the max time changes the block difficulty, so old shares are
// invalid. On mainnet, this means there has been 90 minutes without a new
// block or mempool transaction, which is very unlikely. So the miner should
// probably reset anyway.
if max_time_reached {
submit_old = Some(false);
}
break (
server_long_poll_id,
chain_tip_and_local_time,
mempool_txs,
submit_old,
);
}
// This duration can be slightly lower than needed, if cur_time was clamped
// to min_time. In that case the wait is very long, and it's ok to return early.
// - Polling wait conditions
//
// It can also be zero if cur_time was clamped to max_time.
// In that case, we want to wait for another change, and ignore this timeout.
// TODO: when we're happy with this code, split it into a function.
//
// Periodically check the mempool for changes.
//
// Optional TODO:
// Remove this polling wait if we switch to using futures to detect sync status
// and mempool changes.
let wait_for_mempool_request = tokio::time::sleep(Duration::from_secs(
GET_BLOCK_TEMPLATE_MEMPOOL_LONG_POLL_INTERVAL,
));
// Return immediately if the chain tip has changed.
let wait_for_best_tip_change = latest_chain_tip.best_tip_changed();
// Wait for the maximum block time to elapse. This can change the block header
// on testnet. (On mainnet it can happen due to a network disconnection, or a
// rapid drop in hash rate.)
//
// This duration might be slightly lower than the actual maximum,
// if cur_time was clamped to min_time. In that case the wait is very long,
// and it's ok to return early.
//
// It can also be zero if cur_time was clamped to max_time. In that case,
// we want to wait for another change, and ignore this timeout. So we use an
// `OptionFuture::None`.
let duration_until_max_time = chain_tip_and_local_time
.max_time
.saturating_duration_since(chain_tip_and_local_time.cur_time);
@ -408,16 +454,70 @@ where
}
.into();
// TODO: remove this polling wait after we've switched to
// using futures to detect state tip, sync status, and mempool changes
let temp_wait_before_requests = tokio::time::sleep(Duration::from_secs(5));
// Optional TODO:
// `zcashd` generates the next coinbase transaction while waiting for changes.
// When Zebra supports shielded coinbase, we might want to do this in parallel.
// But the coinbase value depends on the selected transactions, so this needs
// further analysis to check if it actually saves us any time.
// TODO: change logging to debug after testing
tokio::select! {
// Poll the futures in the same order as they are listed here.
// Poll the futures in the listed order, for efficiency.
// We put the most frequent conditions first.
biased;
// TODO: change logging to debug after testing
// This timer elapses every few seconds
_elapsed = wait_for_mempool_request => {
tracing::info!(
max_time = ?chain_tip_and_local_time.max_time,
cur_time = ?chain_tip_and_local_time.cur_time,
?server_long_poll_id,
?client_long_poll_id,
GET_BLOCK_TEMPLATE_MEMPOOL_LONG_POLL_INTERVAL,
"checking for a new mempool change after waiting a few seconds"
);
}
// The state changes after around a target block interval (75s)
tip_changed_result = wait_for_best_tip_change => {
match tip_changed_result {
Ok(()) => {
tracing::info!(
max_time = ?chain_tip_and_local_time.max_time,
cur_time = ?chain_tip_and_local_time.cur_time,
?server_long_poll_id,
?client_long_poll_id,
"returning from long poll because state has changed"
);
}
Err(recv_error) => {
// This log should stay at info when the others go to debug,
// it will help with debugging.
tracing::info!(
?recv_error,
max_time = ?chain_tip_and_local_time.max_time,
cur_time = ?chain_tip_and_local_time.cur_time,
?server_long_poll_id,
?client_long_poll_id,
"returning from long poll due to a state error.\
Is Zebra shutting down?"
);
return Err(Error {
code: ErrorCode::ServerError(0),
message: recv_error.to_string(),
data: None,
});
}
}
}
// The max time does not elapse during normal operation on mainnet,
// and it rarely elapses on testnet.
Some(_elapsed) = wait_for_max_time => {
// This log should stay at info when the others go to debug,
// it's very rare.
tracing::info!(
max_time = ?chain_tip_and_local_time.max_time,
cur_time = ?chain_tip_and_local_time.cur_time,
@ -428,16 +528,6 @@ where
max_time_reached = true;
}
_elapsed = temp_wait_before_requests => {
tracing::info!(
max_time = ?chain_tip_and_local_time.max_time,
cur_time = ?chain_tip_and_local_time.cur_time,
?server_long_poll_id,
?client_long_poll_id,
"checking long poll inputs again after waiting 5 seconds"
);
}
}
};
@ -482,6 +572,7 @@ where
coinbase_txn,
&mempool_txs,
default_roots,
submit_old,
);
Ok(response)

View File

@ -2,6 +2,18 @@
use jsonrpc_core::ErrorCode;
/// When long polling, the amount of time we wait between mempool queries.
/// (And sync status queries, which we do right before mempool queries.)
///
/// State tip changes make long polling return immediately. But miners can re-use old work
/// with an old set of transactions, so they don't need to know about mempool changes immediately.
///
/// Sync status changes are rare, and the blocks they download cause a chain tip change anyway.
///
/// `zcashd` waits 10 seconds between checking the state
/// <https://github.com/zcash/zcash/blob/420f8dfe38fd6b2465a665324366c2ae14aa98f4/src/rpc/mining.cpp#L626>
pub const GET_BLOCK_TEMPLATE_MEMPOOL_LONG_POLL_INTERVAL: u64 = 5;
/// A range of valid block template nonces, that goes from `u32::MIN` to `u32::MAX` as a string.
pub const GET_BLOCK_TEMPLATE_NONCE_RANGE_FIELD: &str = "00000000ffffffff";

View File

@ -135,7 +135,7 @@ pub struct GetBlockTemplate {
/// Zebra adjusts the minimum and current times for testnet minimum difficulty blocks,
/// so we need to tell miners what the maximum valid time is.
///
/// This field is not in the Zcash RPC reference yet.
/// This field is not in `zcashd` or the Zcash RPC reference yet.
///
/// Currently, some miners just use `min_time` or `cur_time`. Others calculate `max_time` from the
/// fixed 90 minute consensus rule, or a smaller fixed interval (like 1000s).
@ -143,6 +143,22 @@ pub struct GetBlockTemplate {
/// a significant drop in the hash rate, or after the testnet minimum difficulty interval.
#[serde(rename = "maxtime")]
pub max_time: DateTime32,
/// > only relevant for long poll responses:
/// > indicates if work received prior to this response remains potentially valid (default)
/// > and should have its shares submitted;
/// > if false, the miner may wish to discard its share queue
///
/// <https://en.bitcoin.it/wiki/BIP_0022#Optional:_Long_Polling>
///
/// This field is not in `zcashd` or the Zcash RPC reference yet.
///
/// In Zebra, `submit_old` is `false` when the tip block changed or max time is reached,
/// and `true` if only the mempool transactions have changed.
#[serde(skip_serializing_if = "Option::is_none")]
#[serde(default)]
#[serde(rename = "submitold")]
pub submit_old: Option<bool>,
}
impl GetBlockTemplate {
@ -156,6 +172,7 @@ impl GetBlockTemplate {
coinbase_txn: TransactionTemplate<amount::NegativeOrZero>,
mempool_txs: &[VerifiedUnminedTx],
default_roots: DefaultRoots,
submit_old: Option<bool>,
) -> Self {
// Convert transactions into TransactionTemplates
let mempool_txs = mempool_txs.iter().map(Into::into).collect();
@ -212,6 +229,8 @@ impl GetBlockTemplate {
height: next_block_height.0,
max_time: chain_tip_and_local_time.max_time,
submit_old,
}
}
}

View File

@ -187,6 +187,23 @@ pub struct LongPollId {
pub mempool_transaction_content_checksum: u32,
}
impl LongPollId {
/// Returns `true` if shares using `old_long_poll_id` can be submitted in response to the
/// template for `self`:
/// <https://en.bitcoin.it/wiki/BIP_0022#Optional:_Long_Polling>
///
/// Old shares may be valid if only the mempool transactions have changed,
/// because newer transactions don't have to be included in the old shares.
///
/// But if the chain tip has changed, the block header has changed, so old shares are invalid.
/// (And if the max time has changed on testnet, the block header has changed.)
pub fn submit_old(&self, old_long_poll_id: &LongPollId) -> bool {
self.tip_height == old_long_poll_id.tip_height
&& self.tip_hash_checksum == old_long_poll_id.tip_hash_checksum
&& self.max_timestamp == old_long_poll_id.max_timestamp
}
}
/// Update `checksum` from `item`, so changes in `item` are likely to also change `checksum`.
///
/// This checksum is not cryptographically secure.

View File

@ -29,7 +29,11 @@ use crate::methods::{
get_block_template_rpcs::{
self,
types::{
get_block_template::GetBlockTemplate, get_mining_info, hex_data::HexData, submit_block,
get_block_template::{self, GetBlockTemplate},
get_mining_info,
hex_data::HexData,
long_poll::{LongPollId, LONG_POLL_ID_LENGTH},
submit_block,
},
},
tests::utils::fake_history_tree,
@ -144,6 +148,8 @@ pub async fn test_responses<State, ReadState>(
.expect("We should have a success response");
snapshot_rpc_getnetworksolps(get_network_sol_ps, &settings);
// `getblocktemplate`
// get a new empty state
let new_read_state = MockService::build().for_unit_tests();
@ -161,11 +167,13 @@ pub async fn test_responses<State, ReadState>(
mock_sync_status,
);
// `getblocktemplate`
// Basic variant (default mode and no extra features)
// Fake the ChainInfo response
let response_read_state = new_read_state.clone();
tokio::spawn(async move {
new_read_state
response_read_state
.clone()
.expect_request_that(|req| matches!(req, ReadRequest::ChainInfo))
.await
@ -199,9 +207,65 @@ pub async fn test_responses<State, ReadState>(
.zcash_deserialize_into()
.expect("coinbase bytes are valid");
snapshot_rpc_getblocktemplate(get_block_template, coinbase_tx, &settings);
snapshot_rpc_getblocktemplate("basic", get_block_template, coinbase_tx, &settings);
// long polling feature with submit old field
let long_poll_id: LongPollId = "0"
.repeat(LONG_POLL_ID_LENGTH)
.parse()
.expect("unexpected invalid LongPollId");
// Fake the ChainInfo response
let response_read_state = new_read_state.clone();
tokio::spawn(async move {
response_read_state
.clone()
.expect_request_that(|req| matches!(req, ReadRequest::ChainInfo))
.await
.respond(ReadResponse::ChainInfo(GetBlockTemplateChainInfo {
expected_difficulty: CompactDifficulty::from(ExpandedDifficulty::from(U256::one())),
tip_height: fake_tip_height,
tip_hash: fake_tip_hash,
cur_time: fake_cur_time,
min_time: fake_min_time,
max_time: fake_max_time,
history_tree: fake_history_tree(network),
}));
});
let get_block_template = tokio::spawn(
get_block_template_rpc.get_block_template(
get_block_template::JsonParameters {
long_poll_id: long_poll_id.into(),
..Default::default()
}
.into(),
),
);
mempool
.expect_request(mempool::Request::FullTransactions)
.await
.respond(mempool::Response::FullTransactions(vec![]));
let get_block_template = get_block_template
.await
.expect("unexpected panic in getblocktemplate RPC task")
.expect("unexpected error in getblocktemplate RPC call");
let coinbase_tx: Transaction = get_block_template
.coinbase_txn
.data
.as_ref()
.zcash_deserialize_into()
.expect("coinbase bytes are valid");
snapshot_rpc_getblocktemplate("long_poll", get_block_template, coinbase_tx, &settings);
// `submitblock`
let submit_block = get_block_template_rpc
.submit_block(HexData("".into()), None)
.await
@ -222,12 +286,20 @@ fn snapshot_rpc_getblockhash(block_hash: GetBlockHash, settings: &insta::Setting
/// Snapshot `getblocktemplate` response, using `cargo insta` and JSON serialization.
fn snapshot_rpc_getblocktemplate(
variant: &'static str,
block_template: GetBlockTemplate,
coinbase_tx: Transaction,
settings: &insta::Settings,
) {
settings.bind(|| insta::assert_json_snapshot!("get_block_template", block_template));
settings.bind(|| insta::assert_json_snapshot!("get_block_template.coinbase_tx", coinbase_tx));
settings.bind(|| {
insta::assert_json_snapshot!(format!("get_block_template_{variant}"), block_template)
});
settings.bind(|| {
insta::assert_ron_snapshot!(
format!("get_block_template_{variant}.coinbase_tx"),
coinbase_tx
)
});
}
/// Snapshot `submitblock` response, using `cargo insta` and JSON serialization.

View File

@ -1,42 +0,0 @@
---
source: zebra-rpc/src/methods/tests/snapshot/get_block_template_rpcs.rs
expression: coinbase_tx
---
{
"V5": {
"network_upgrade": "Nu5",
"lock_time": {
"Height": 0
},
"expiry_height": 1687105,
"inputs": [
{
"Coinbase": {
"height": 1687105,
"data": [],
"sequence": 0
}
}
],
"outputs": [
{
"value": 15625000,
"lock_script": "a914d45cb1adffb5215a42720532a076f02c7c778c9087"
},
{
"value": 21875000,
"lock_script": "a91469a9f95a98fe581b6eb52841ef4806dc4402eb9087"
},
{
"value": 25000000,
"lock_script": "a914931fec54c1fea86e574462cc32013f5400b8912987"
},
{
"value": 250000000,
"lock_script": "a914adadadadadadadadadadadadadadadadadadadad87"
}
],
"sapling_shielded_data": null,
"orchard_shielded_data": null
}
}

View File

@ -1,42 +0,0 @@
---
source: zebra-rpc/src/methods/tests/snapshot/get_block_template_rpcs.rs
expression: coinbase_tx
---
{
"V5": {
"network_upgrade": "Nu5",
"lock_time": {
"Height": 0
},
"expiry_height": 1842421,
"inputs": [
{
"Coinbase": {
"height": 1842421,
"data": [],
"sequence": 0
}
}
],
"outputs": [
{
"value": 15625000,
"lock_script": "a9140c0bcca02f3cba01a5d7423ac3903d40586399eb87"
},
{
"value": 21875000,
"lock_script": "a9144e3f0d9a33a2721604cbae2de8d9171e21f8fbe487"
},
{
"value": 25000000,
"lock_script": "a91471e1df05024288a00802de81e08c437859586c8787"
},
{
"value": 250000000,
"lock_script": "a914adadadadadadadadadadadadadadadadadadadad87"
}
],
"sapling_shielded_data": null,
"orchard_shielded_data": null
}
}

View File

@ -0,0 +1,36 @@
---
source: zebra-rpc/src/methods/tests/snapshot/get_block_template_rpcs.rs
expression: coinbase_tx
---
V5(
network_upgrade: Nu5,
lock_time: Height(Height(0)),
expiry_height: Height(1687105),
inputs: [
Coinbase(
height: Height(1687105),
data: CoinbaseData([]),
sequence: 0,
),
],
outputs: [
Output(
value: 15625000,
lock_script: Script("a914d45cb1adffb5215a42720532a076f02c7c778c9087"),
),
Output(
value: 21875000,
lock_script: Script("a91469a9f95a98fe581b6eb52841ef4806dc4402eb9087"),
),
Output(
value: 25000000,
lock_script: Script("a914931fec54c1fea86e574462cc32013f5400b8912987"),
),
Output(
value: 250000000,
lock_script: Script("a914adadadadadadadadadadadadadadadadadadadad87"),
),
],
sapling_shielded_data: None,
orchard_shielded_data: None,
)

View File

@ -0,0 +1,36 @@
---
source: zebra-rpc/src/methods/tests/snapshot/get_block_template_rpcs.rs
expression: coinbase_tx
---
V5(
network_upgrade: Nu5,
lock_time: Height(Height(0)),
expiry_height: Height(1842421),
inputs: [
Coinbase(
height: Height(1842421),
data: CoinbaseData([]),
sequence: 0,
),
],
outputs: [
Output(
value: 15625000,
lock_script: Script("a9140c0bcca02f3cba01a5d7423ac3903d40586399eb87"),
),
Output(
value: 21875000,
lock_script: Script("a9144e3f0d9a33a2721604cbae2de8d9171e21f8fbe487"),
),
Output(
value: 25000000,
lock_script: Script("a91471e1df05024288a00802de81e08c437859586c8787"),
),
Output(
value: 250000000,
lock_script: Script("a914adadadadadadadadadadadadadadadadadadadad87"),
),
],
sapling_shielded_data: None,
orchard_shielded_data: None,
)

View File

@ -0,0 +1,36 @@
---
source: zebra-rpc/src/methods/tests/snapshot/get_block_template_rpcs.rs
expression: coinbase_tx
---
V5(
network_upgrade: Nu5,
lock_time: Height(Height(0)),
expiry_height: Height(1687105),
inputs: [
Coinbase(
height: Height(1687105),
data: CoinbaseData([]),
sequence: 0,
),
],
outputs: [
Output(
value: 15625000,
lock_script: Script("a914d45cb1adffb5215a42720532a076f02c7c778c9087"),
),
Output(
value: 21875000,
lock_script: Script("a91469a9f95a98fe581b6eb52841ef4806dc4402eb9087"),
),
Output(
value: 25000000,
lock_script: Script("a914931fec54c1fea86e574462cc32013f5400b8912987"),
),
Output(
value: 250000000,
lock_script: Script("a914adadadadadadadadadadadadadadadadadadadad87"),
),
],
sapling_shielded_data: None,
orchard_shielded_data: None,
)

View File

@ -0,0 +1,36 @@
---
source: zebra-rpc/src/methods/tests/snapshot/get_block_template_rpcs.rs
expression: coinbase_tx
---
V5(
network_upgrade: Nu5,
lock_time: Height(Height(0)),
expiry_height: Height(1842421),
inputs: [
Coinbase(
height: Height(1842421),
data: CoinbaseData([]),
sequence: 0,
),
],
outputs: [
Output(
value: 15625000,
lock_script: Script("a9140c0bcca02f3cba01a5d7423ac3903d40586399eb87"),
),
Output(
value: 21875000,
lock_script: Script("a9144e3f0d9a33a2721604cbae2de8d9171e21f8fbe487"),
),
Output(
value: 25000000,
lock_script: Script("a91471e1df05024288a00802de81e08c437859586c8787"),
),
Output(
value: 250000000,
lock_script: Script("a914adadadadadadadadadadadadadadadadadadadad87"),
),
],
sapling_shielded_data: None,
orchard_shielded_data: None,
)

View File

@ -0,0 +1,44 @@
---
source: zebra-rpc/src/methods/tests/snapshot/get_block_template_rpcs.rs
expression: block_template
---
{
"capabilities": [],
"version": 4,
"previousblockhash": "0000000000d723156d9b65ffcf4984da7a19675ed7e2f06d9e5d5188af087bf8",
"blockcommitmentshash": "fe03d8236b0835c758f59d279230ebaee2128754413103b9edb17c07451c2c82",
"lightclientroothash": "fe03d8236b0835c758f59d279230ebaee2128754413103b9edb17c07451c2c82",
"finalsaplingroothash": "fe03d8236b0835c758f59d279230ebaee2128754413103b9edb17c07451c2c82",
"defaultroots": {
"merkleroot": "6b370584714ab567c9c014ce72d325ab6c5927e181ac891acb35e6d4b6cc19a1",
"chainhistoryroot": "94470fa66ebd1a5fdb109a5aa3f3204f14de3a42135e71aa7f4c44055847e0b5",
"authdataroot": "0dbb78de9fdcd494307971e36dd049fc82d0ee9ee53aec8fd2a54dc0e426289b",
"blockcommitmentshash": "fe03d8236b0835c758f59d279230ebaee2128754413103b9edb17c07451c2c82"
},
"transactions": [],
"coinbasetxn": {
"data": "050000800a27a726b4d0d6c20000000041be1900010000000000000000000000000000000000000000000000000000000000000000ffffffff040341be190000000004286bee000000000017a914d45cb1adffb5215a42720532a076f02c7c778c908738c94d010000000017a91469a9f95a98fe581b6eb52841ef4806dc4402eb908740787d010000000017a914931fec54c1fea86e574462cc32013f5400b891298780b2e60e0000000017a914adadadadadadadadadadadadadadadadadadadad87000000",
"hash": "6b370584714ab567c9c014ce72d325ab6c5927e181ac891acb35e6d4b6cc19a1",
"authdigest": "0dbb78de9fdcd494307971e36dd049fc82d0ee9ee53aec8fd2a54dc0e426289b",
"depends": [],
"fee": 0,
"sigops": 0,
"required": true
},
"longpollid": "00016871043eab7f731654008728000000000000000000",
"target": "0000000000000000000000000000000000000000000000000000000000000001",
"mintime": 1654008606,
"mutable": [
"time",
"transactions",
"prevblock"
],
"noncerange": "00000000ffffffff",
"sigoplimit": 20000,
"sizelimit": 2000000,
"curtime": 1654008617,
"bits": "01010000",
"height": 1687105,
"maxtime": 1654008728,
"submitold": false
}

View File

@ -0,0 +1,44 @@
---
source: zebra-rpc/src/methods/tests/snapshot/get_block_template_rpcs.rs
expression: block_template
---
{
"capabilities": [],
"version": 4,
"previousblockhash": "0000000000d723156d9b65ffcf4984da7a19675ed7e2f06d9e5d5188af087bf8",
"blockcommitmentshash": "cb1f1c6a5ad5ff9c4a170e3b747a24f3aec79817adba9a9451f19914481bb422",
"lightclientroothash": "cb1f1c6a5ad5ff9c4a170e3b747a24f3aec79817adba9a9451f19914481bb422",
"finalsaplingroothash": "cb1f1c6a5ad5ff9c4a170e3b747a24f3aec79817adba9a9451f19914481bb422",
"defaultroots": {
"merkleroot": "623400cc122baa015d3a4209f5903ebe215170c7e6e74831dce8372c5fd5b3cc",
"chainhistoryroot": "03bc75f00c307a05aed2023819e18c2672cbe15fbd3200944997def141967387",
"authdataroot": "a44375f0c0dd5ba612bd7b0efd77683cde8edf5055aff9fbfda443cc8d46bd3e",
"blockcommitmentshash": "cb1f1c6a5ad5ff9c4a170e3b747a24f3aec79817adba9a9451f19914481bb422"
},
"transactions": [],
"coinbasetxn": {
"data": "050000800a27a726b4d0d6c200000000f51c1c00010000000000000000000000000000000000000000000000000000000000000000ffffffff0403f51c1c0000000004286bee000000000017a9140c0bcca02f3cba01a5d7423ac3903d40586399eb8738c94d010000000017a9144e3f0d9a33a2721604cbae2de8d9171e21f8fbe48740787d010000000017a91471e1df05024288a00802de81e08c437859586c878780b2e60e0000000017a914adadadadadadadadadadadadadadadadadadadad87000000",
"hash": "623400cc122baa015d3a4209f5903ebe215170c7e6e74831dce8372c5fd5b3cc",
"authdigest": "a44375f0c0dd5ba612bd7b0efd77683cde8edf5055aff9fbfda443cc8d46bd3e",
"depends": [],
"fee": 0,
"sigops": 0,
"required": true
},
"longpollid": "00018424203eab7f731654008728000000000000000000",
"target": "0000000000000000000000000000000000000000000000000000000000000001",
"mintime": 1654008606,
"mutable": [
"time",
"transactions",
"prevblock"
],
"noncerange": "00000000ffffffff",
"sigoplimit": 20000,
"sizelimit": 2000000,
"curtime": 1654008617,
"bits": "01010000",
"height": 1842421,
"maxtime": 1654008728,
"submitold": false
}

View File

@ -8,17 +8,13 @@
use std::{fmt, sync::Arc};
use chrono::{DateTime, Utc};
use futures::TryFutureExt;
use tokio::sync::watch;
use tracing::{field, instrument};
#[cfg(any(test, feature = "proptest-impl"))]
use proptest_derive::Arbitrary;
#[cfg(any(test, feature = "proptest-impl"))]
use zebra_chain::serialization::arbitrary::datetime_full;
use zebra_chain::{
block,
chain_tip::ChainTip,
chain_tip::{BestTipChanged, ChainTip},
parameters::{Network, NetworkUpgrade},
transaction::{self, Transaction},
};
@ -29,6 +25,12 @@ use crate::{
use TipAction::*;
#[cfg(any(test, feature = "proptest-impl"))]
use proptest_derive::Arbitrary;
#[cfg(any(test, feature = "proptest-impl"))]
use zebra_chain::serialization::arbitrary::datetime_full;
#[cfg(test)]
mod tests;
@ -314,6 +316,8 @@ impl LatestChainTip {
/// A single read lock is acquired to clone `T`, and then released after the clone.
/// See the performance note on [`WatchReceiver::with_watch_data`].
///
/// Does not mark the watched data as seen.
///
/// # Correctness
///
/// To avoid deadlocks, see the correctness note on [`WatchReceiver::with_watch_data`].
@ -387,6 +391,19 @@ impl ChainTip for LatestChainTip {
self.with_chain_tip_block(|block| block.transaction_hashes.clone())
.unwrap_or_else(|| Arc::new([]))
}
/// Returns when the state tip changes.
///
/// Marks the state tip as seen when the returned future completes.
#[instrument(skip(self))]
fn best_tip_changed(&mut self) -> BestTipChanged {
BestTipChanged::new(self.receiver.changed().err_into())
}
/// Mark the current best state tip as seen.
fn mark_best_tip_seen(&mut self) {
self.receiver.mark_as_seen();
}
}
/// A chain tip change monitor.

View File

@ -44,6 +44,8 @@ where
/// This helper method is a shorter way to borrow the value from the [`watch::Receiver`] and
/// extract some information from it.
///
/// Does not mark the watched data as seen.
///
/// # Performance
///
/// A single read lock is acquired to clone `T`, and then released after the
@ -88,7 +90,9 @@ where
}
/// Returns a clone of the watch data in the channel.
/// This helps avoid deadlocks.
/// Cloning the watched data helps avoid deadlocks.
///
/// Does not mark the watched data as seen.
///
/// See `with_watch_data()` for details.
pub fn cloned_watch_data(&self) -> T {
@ -96,7 +100,14 @@ where
}
/// Calls [`watch::Receiver::changed`] and returns the result.
///
/// Marks the watched data as seen.
pub async fn changed(&mut self) -> Result<(), watch::error::RecvError> {
self.receiver.changed().await
}
/// Marks the watched data as seen.
pub fn mark_as_seen(&mut self) {
self.receiver.borrow_and_update();
}
}