diff --git a/CHANGELOG.md b/CHANGELOG.md
index da077c5051d..b7e14fb57ca 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -13,6 +13,7 @@ and this project adheres to the versioning scheme outlined in the [README.md](RE
- Add `stackerdb_timeout_secs` to miner config for limiting duration of StackerDB HTTP requests.
- When determining a global transaction replay set, the state evaluator now uses a longest-common-prefix algorithm to find a replay set in the case where a single replay set has less than 70% of signer weight.
- New endpoints /v3/tenures/blocks/, /v3/tenures/blocks/hash, /v3/tenures/blocks/height allowing retrieving the list of stacks blocks from a burn block
+- New authenticated endpoint /v3/block/replay to replay the execution of any Nakamoto block in the chain (useful for validation, simulation, getting events...)
- Creates epoch 3.3 and costs-4 in preparation for a hardfork to activate Clarity 4
- Adds support for new Clarity 4 builtins (not activated until epoch 3.3):
- `contract-hash?`
diff --git a/docs/rpc/components/examples/block-replay.example.json b/docs/rpc/components/examples/block-replay.example.json
new file mode 100644
index 00000000000..b6c79c99f2d
--- /dev/null
+++ b/docs/rpc/components/examples/block-replay.example.json
@@ -0,0 +1,108 @@
+{
+ "block_hash": "732f57eefc4dbfb015c9988d9943c47273d25fbe039220d53f311b307609c83f",
+ "block_id": "856f6b08f338164df7422f66337c8ce916b6b0301fcaa09de06c61cfb79e2a45",
+ "block_height": 123,
+ "consensus_hash": "33dffda027e2ca3aaf278855c59a8a0b2d2dd51f",
+ "fees": 1,
+ "miner_signature": "004b2878d7639060c4d183b1338447c06ceb0ad55424856c550a4c988401fdf8cf1fe6e8c05cc0e3ff8d4383590bf700cb5dd1a8bb3269f7125f6e0704b66eade8",
+ "parent_block_id": "3ac36fc1acfc86ba80ea27cd26017c675f75bc07fb042814b72e74cd7d331503",
+ "signer_signature": [
+ "00d4e08331db614d18d7b5af53cf9bc03add9c7a2dcb6f0448721de7ea98f662cf7dc43ee73e14d18dfae3d3d349ff67e0fd773a446fb8c949c93ae4676f4d34bc",
+ "01619c6e69bad5b43e11bae3eb4d4626e5cf19d595923b0b5d2053e8883a266b41315fdaefd1eca03c5c0580b0f7fd28053c3f34eb0a12220b61392d342f5afb0d",
+ "0078fa352e0e4d2d94b25d4070ae24a819f268b2260a1e4d0d867415dbdc39e2cf75e57de69375794073e22a75873a5e1ca33ed96eadd8086415e934f697b78fdb",
+ "00b8d9b0d0cdfabe3c65237801e714185777f60507c773fcd2a72ed00b9d4c59cb5ab96e0e8d545bd562b5ca3de6db1d3b9fccd8f41c3bfa7de3528deb1acd30d6"
+ ],
+ "state_index_root": "839b826290027e5b92de415495be7bab2eab2ad4e2f8c371a1a773ae552fedba",
+ "timestamp": 1758284349,
+ "transactions": [
+ {
+ "data": {
+ "anchor_mode": "OnChainOnly",
+ "auth": {
+ "Standard": {
+ "Singlesig": {
+ "hash_mode": "P2PKH",
+ "key_encoding": "Compressed",
+ "nonce": 99,
+ "signature": "01e29229b386e1f69ffd91e339c878246235ec1cd4771b42a7f45e1ed108643bc9417d43dd96a02c93314ef4cf5bcbcc5642df2e1f5a177333ff983c8719d80661",
+ "signer": "2965a4e6e4226868fa3ae88b2b9bb9e937d77fba",
+ "tx_fee": 1
+ }
+ }
+ },
+ "chain_id": 2147483648,
+ "payload": {
+ "TokenTransfer": [
+ {
+ "Standard": [
+ 26,
+ [
+ 189,
+ 65,
+ 200,
+ 147,
+ 188,
+ 192,
+ 157,
+ 152,
+ 224,
+ 211,
+ 77,
+ 255,
+ 135,
+ 190,
+ 175,
+ 153,
+ 88,
+ 51,
+ 140,
+ 222
+ ]
+ ]
+ },
+ 1,
+ "00000000000000000000000000000000000000000000000000000000000000000000"
+ ]
+ },
+ "post_condition_mode": "Deny",
+ "post_conditions": [],
+ "version": "Testnet"
+ },
+ "events": [
+ {
+ "committed": true,
+ "event_index": 0,
+ "stx_transfer_event": {
+ "amount": "1",
+ "memo": "00000000000000000000000000000000000000000000000000000000000000000000",
+ "recipient": "ST2YM3J4KQK09V670TD6ZZ1XYNYCNGCWCVTASN5VM",
+ "sender": "STMPB976WGH6GT7T7BM8PAWVQ7MKFNVZQAXS4BFS"
+ },
+ "txid": "0xf14dd7dec56405fd7dac69c3080fb569fae4c49c591f9ad0e5cf5c797add9005",
+ "type": "stx_transfer_event"
+ }
+ ],
+ "execution_cost": {
+ "read_count": 0,
+ "read_length": 0,
+ "runtime": 0,
+ "write_count": 0,
+ "write_length": 0
+ },
+ "hex": "808000000004002965a4e6e4226868fa3ae88b2b9bb9e937d77fba000000000000006300000000000000010001e29229b386e1f69ffd91e339c878246235ec1cd4771b42a7f45e1ed108643bc9417d43dd96a02c93314ef4cf5bcbcc5642df2e1f5a177333ff983c8719d8066101020000000000051abd41c893bcc09d98e0d34dff87beaf9958338cde000000000000000100000000000000000000000000000000000000000000000000000000000000000000",
+ "result": {
+ "Response": {
+ "committed": true,
+ "data": {
+ "Bool": true
+ }
+ }
+ },
+ "stx_burned": 0,
+ "tx_index": 0,
+ "txid": "f14dd7dec56405fd7dac69c3080fb569fae4c49c591f9ad0e5cf5c797add9005"
+ }
+ ],
+ "tx_merkle_root": "a68e3c76471d9e66b71a14165c4c9a2b980c51efb5b313425cffcef7172d6080",
+ "valid_merkle_root": true
+}
\ No newline at end of file
diff --git a/docs/rpc/components/schemas/block-replay.schema.yaml b/docs/rpc/components/schemas/block-replay.schema.yaml
new file mode 100644
index 00000000000..c8e9802bb21
--- /dev/null
+++ b/docs/rpc/components/schemas/block-replay.schema.yaml
@@ -0,0 +1,78 @@
+type: object
+properties:
+ block_hash:
+ type: string
+ description: Hash of the block
+ pattern: "^[0-9a-f]{64}$"
+ block_id:
+ type: string
+ description: Block ID (index block hash)
+ pattern: "^[0-9a-f]{64}$"
+ block_height:
+ type: integer
+ description: Height of the Stacks block
+ format: uint64
+ consensus_hash:
+ type: string
+ description: Consensus hash of the tenure
+ pattern: "^[0-9a-f]{40}$"
+ fees:
+ type: integer
+ description: total fees for the block
+ miner_signature:
+ type: string
+ description: Uncompressed signature of the miner
+ pattern: "^[0-9a-f]{130}$"
+ parent_block_id:
+ type: string
+ description: Parent Block ID (index block hash)
+ pattern: "^[0-9a-f]{64}$"
+ signer_signature:
+ type: array
+ items:
+ type: string
+ description: Uncompressed signature of the signer
+ pattern: "^[0-9a-f]{130}$"
+ state_index_root:
+ type: string
+ pattern: "^[0-9a-f]{64}$"
+ description: block state index root computed from the MARF (got from the original block)
+ timestamp:
+ type: integer
+ tx_merkle_root:
+ type: string
+ description: merkle_root of the included transactions
+ valid_merkle_root:
+ type: boolean
+ description: does the merkle_root matches the chain block and the simulated one?
+ transactions:
+ type: array
+ items:
+ type: object
+ properties:
+ data:
+ type: object
+ description: JSON representation of the transaction payload
+ events:
+ type: array
+ items:
+ type: object
+ description: JSON representation of the transaction events
+ execution_cost:
+ type: object
+ description: costs accounting for the transaction
+ hex:
+ type: string
+ description: hexadecimal representation of the transaction body
+ result:
+ type: object
+ description: Clarity value representing the transaction result
+ stx_burned:
+ type: integer
+ description: number of burned stx
+ tx_index:
+ type: integer
+ description: index of the transaction in the array of transactions
+ txid:
+ type: string
+ description: transaction id
\ No newline at end of file
diff --git a/docs/rpc/openapi.yaml b/docs/rpc/openapi.yaml
index d1a4abcc852..b24645c8de7 100644
--- a/docs/rpc/openapi.yaml
+++ b/docs/rpc/openapi.yaml
@@ -159,6 +159,8 @@ components:
$ref: ./components/schemas/get-stacker-set.schema.yaml
TenureBlocks:
$ref: ./components/schemas/tenure-blocks.schema.yaml
+ BlockReplay:
+ $ref: ./components/schemas/block-replay.schema.yaml
paths:
/v2/transactions:
@@ -2197,3 +2199,36 @@ paths:
$ref: "#/components/responses/Unauthorized"
"500":
$ref: "#/components/responses/InternalServerError"
+
+ /v3/blocks/replay/{block_id}:
+ get:
+ summary: Replay mining of a block and returns its content
+ tags:
+ - Blocks
+ security: []
+ operationId: blockReplay
+ description: |
+ Replay the mining of a block (no data is written in the MARF) and returns its content.
+ parameters:
+ - name: block_id
+ in: path
+ description: The block ID hash
+ required: true
+ schema:
+ type: string
+ pattern: "^[0-9a-f]{64}$"
+ responses:
+ "200":
+ description: Content of the replayed block
+ content:
+ application/json:
+ schema:
+ $ref: "#/components/schemas/BlockReplay"
+ example:
+ $ref: "./components/examples/block-replay.example.json"
+ "400":
+ $ref: "#/components/responses/BadRequest"
+ "404":
+ $ref: "#/components/responses/NotFound"
+ "500":
+ $ref: "#/components/responses/InternalServerError"
diff --git a/stackslib/src/net/api/blockreplay.rs b/stackslib/src/net/api/blockreplay.rs
new file mode 100644
index 00000000000..2d3fe50e44c
--- /dev/null
+++ b/stackslib/src/net/api/blockreplay.rs
@@ -0,0 +1,453 @@
+// Copyright (C) 2025 Stacks Open Internet Foundation
+//
+// This program is free software: you can redistribute it and/or modify
+// it under the terms of the GNU General Public License as published by
+// the Free Software Foundation, either version 3 of the License, or
+// (at your option) any later version.
+//
+// This program is distributed in the hope that it will be useful,
+// but WITHOUT ANY WARRANTY; without even the implied warranty of
+// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+// GNU General Public License for more details.
+//
+// You should have received a copy of the GNU General Public License
+// along with this program. If not, see .
+
+use clarity::vm::costs::ExecutionCost;
+use clarity::vm::Value;
+use regex::{Captures, Regex};
+use stacks_common::codec::StacksMessageCodec;
+use stacks_common::types::chainstate::{BlockHeaderHash, ConsensusHash, StacksBlockId, TrieHash};
+use stacks_common::types::net::PeerHost;
+use stacks_common::util::hash::Sha512Trunc256Sum;
+use stacks_common::util::secp256k1::MessageSignature;
+
+use crate::burnchains::Txid;
+use crate::chainstate::burn::db::sortdb::SortitionDB;
+use crate::chainstate::nakamoto::miner::NakamotoBlockBuilder;
+use crate::chainstate::nakamoto::{NakamotoBlock, NakamotoChainState};
+use crate::chainstate::stacks::db::StacksChainState;
+use crate::chainstate::stacks::events::{StacksTransactionReceipt, TransactionOrigin};
+use crate::chainstate::stacks::miner::{BlockBuilder, BlockLimitFunction, TransactionResult};
+use crate::chainstate::stacks::{Error as ChainError, StacksTransaction, TransactionPayload};
+use crate::net::http::{
+ parse_json, Error, HttpNotFound, HttpRequest, HttpRequestContents, HttpRequestPreamble,
+ HttpResponse, HttpResponseContents, HttpResponsePayload, HttpResponsePreamble, HttpServerError,
+};
+use crate::net::httpcore::{RPCRequestHandler, StacksHttpResponse};
+use crate::net::{Error as NetError, StacksHttpRequest, StacksNodeState};
+
+#[derive(Clone)]
+pub struct RPCNakamotoBlockReplayRequestHandler {
+ pub block_id: Option,
+ pub auth: Option,
+}
+
+impl RPCNakamotoBlockReplayRequestHandler {
+ pub fn new(auth: Option) -> Self {
+ Self {
+ block_id: None,
+ auth,
+ }
+ }
+
+ pub fn block_replay(
+ &self,
+ sortdb: &SortitionDB,
+ chainstate: &mut StacksChainState,
+ ) -> Result {
+ let Some(block_id) = &self.block_id else {
+ return Err(ChainError::InvalidStacksBlock("block_id is None".into()));
+ };
+
+ let Some((tenure_id, parent_block_id)) = chainstate
+ .nakamoto_blocks_db()
+ .get_tenure_and_parent_block_id(&block_id)?
+ else {
+ return Err(ChainError::NoSuchBlockError);
+ };
+
+ let staging_db_path = chainstate.get_nakamoto_staging_blocks_path()?;
+ let db_conn = StacksChainState::open_nakamoto_staging_blocks(&staging_db_path, false)?;
+ let rowid = db_conn
+ .conn()
+ .get_nakamoto_block_rowid(&block_id)?
+ .ok_or(ChainError::NoSuchBlockError)?;
+
+ let mut blob_fd = match db_conn.open_nakamoto_block(rowid, false).map_err(|e| {
+ let msg = format!("Failed to open Nakamoto block {}: {:?}", &block_id, &e);
+ warn!("{}", &msg);
+ msg
+ }) {
+ Ok(blob_fd) => blob_fd,
+ Err(e) => return Err(ChainError::InvalidStacksBlock(e)),
+ };
+
+ let block = match NakamotoBlock::consensus_deserialize(&mut blob_fd).map_err(|e| {
+ let msg = format!("Failed to read Nakamoto block {}: {:?}", &block_id, &e);
+ warn!("{}", &msg);
+ msg
+ }) {
+ Ok(block) => block,
+ Err(e) => return Err(ChainError::InvalidStacksBlock(e)),
+ };
+
+ let burn_dbconn = match sortdb.index_handle_at_block(chainstate, &parent_block_id) {
+ Ok(burn_dbconn) => burn_dbconn,
+ Err(_) => return Err(ChainError::NoSuchBlockError),
+ };
+
+ let tenure_change = block
+ .txs
+ .iter()
+ .find(|tx| matches!(tx.payload, TransactionPayload::TenureChange(..)));
+ let coinbase = block
+ .txs
+ .iter()
+ .find(|tx| matches!(tx.payload, TransactionPayload::Coinbase(..)));
+ let tenure_cause = tenure_change.and_then(|tx| match &tx.payload {
+ TransactionPayload::TenureChange(tc) => Some(tc.cause),
+ _ => None,
+ });
+
+ let parent_stacks_header_opt =
+ match NakamotoChainState::get_block_header(chainstate.db(), &parent_block_id) {
+ Ok(parent_stacks_header_opt) => parent_stacks_header_opt,
+ Err(e) => return Err(e),
+ };
+
+ let Some(parent_stacks_header) = parent_stacks_header_opt else {
+ return Err(ChainError::InvalidStacksBlock(
+ "Invalid Parent Block".into(),
+ ));
+ };
+
+ let mut builder = match NakamotoBlockBuilder::new(
+ &parent_stacks_header,
+ &block.header.consensus_hash,
+ block.header.burn_spent,
+ tenure_change,
+ coinbase,
+ block.header.pox_treatment.len(),
+ None,
+ None,
+ ) {
+ Ok(builder) => builder,
+ Err(e) => return Err(e),
+ };
+
+ let mut miner_tenure_info =
+ match builder.load_ephemeral_tenure_info(chainstate, &burn_dbconn, tenure_cause) {
+ Ok(miner_tenure_info) => miner_tenure_info,
+ Err(e) => return Err(e),
+ };
+
+ let burn_chain_height = miner_tenure_info.burn_tip_height;
+ let mut tenure_tx = match builder.tenure_begin(&burn_dbconn, &mut miner_tenure_info) {
+ Ok(tenure_tx) => tenure_tx,
+ Err(e) => return Err(e),
+ };
+
+ let mut block_fees: u128 = 0;
+ let mut txs_receipts = vec![];
+
+ for (i, tx) in block.txs.iter().enumerate() {
+ let tx_len = tx.tx_len();
+
+ let tx_result = builder.try_mine_tx_with_len(
+ &mut tenure_tx,
+ tx,
+ tx_len,
+ &BlockLimitFunction::NO_LIMIT_HIT,
+ None,
+ );
+ let err = match tx_result {
+ TransactionResult::Success(tx_result) => {
+ txs_receipts.push(tx_result.receipt);
+ Ok(())
+ }
+ _ => Err(format!("Problematic tx {i}")),
+ };
+ if let Err(reason) = err {
+ let txid = tx.txid();
+ return Err(ChainError::InvalidStacksTransaction(
+ format!("Unable to replay transaction {txid}: {reason}").into(),
+ false,
+ ));
+ }
+
+ block_fees += tx.get_tx_fee() as u128;
+ }
+
+ let replayed_block = builder.mine_nakamoto_block(&mut tenure_tx, burn_chain_height);
+
+ tenure_tx.rollback_block();
+
+ let tx_merkle_root = block.header.tx_merkle_root.clone();
+
+ let mut rpc_replayed_block =
+ RPCReplayedBlock::from_block(block, block_fees, tenure_id, parent_block_id);
+
+ for receipt in &txs_receipts {
+ let transaction = RPCReplayedBlockTransaction::from_receipt(receipt);
+ rpc_replayed_block.transactions.push(transaction);
+ }
+
+ rpc_replayed_block.valid_merkle_root =
+ tx_merkle_root == replayed_block.header.tx_merkle_root;
+
+ Ok(rpc_replayed_block)
+ }
+}
+
+#[derive(Debug, PartialEq, Clone, Serialize, Deserialize)]
+pub struct RPCReplayedBlockTransaction {
+ /// transaction id
+ pub txid: Txid,
+ /// index of transaction in the block
+ pub tx_index: u32,
+ /// body (headers + payload) of transaction
+ pub data: Option,
+ /// hex representation of the transaction body
+ pub hex: String,
+ /// result of transaction execution (clarity value)
+ pub result: Value,
+ /// amount of burned stx
+ pub stx_burned: u128,
+ /// execution cost infos
+ pub execution_cost: ExecutionCost,
+ /// generated events
+ pub events: Vec,
+}
+
+impl RPCReplayedBlockTransaction {
+ pub fn from_receipt(receipt: &StacksTransactionReceipt) -> Self {
+ let events = receipt
+ .events
+ .iter()
+ .enumerate()
+ .map(|(event_index, event)| {
+ event
+ .json_serialize(event_index, &receipt.transaction.txid(), true)
+ .unwrap()
+ })
+ .collect();
+
+ let transaction_data = match &receipt.transaction {
+ TransactionOrigin::Stacks(stacks) => Some(stacks.clone()),
+ TransactionOrigin::Burn(_) => None,
+ };
+
+ let txid = receipt.transaction.txid();
+
+ Self {
+ txid,
+ tx_index: receipt.tx_index,
+ data: transaction_data,
+ hex: receipt.transaction.serialize_to_dbstring(),
+ result: receipt.result.clone(),
+ stx_burned: receipt.stx_burned,
+ execution_cost: receipt.execution_cost.clone(),
+ events,
+ }
+ }
+}
+
+#[derive(Debug, PartialEq, Clone, Serialize, Deserialize)]
+pub struct RPCReplayedBlock {
+ /// block id (index_block_hash)
+ pub block_id: StacksBlockId,
+ /// block hash
+ pub block_hash: BlockHeaderHash,
+ /// height of the block
+ pub block_height: u64,
+ /// index_block_hash of the parent
+ pub parent_block_id: StacksBlockId,
+ /// consensus hash of the tenure containing the block
+ pub consensus_hash: ConsensusHash,
+ /// total fees for the transactions in the block
+ pub fees: u128,
+ /// merkle tree root hash of the included transactions
+ pub tx_merkle_root: Sha512Trunc256Sum,
+ /// state index of the MARF
+ pub state_index_root: TrieHash,
+ /// block timestamp
+ pub timestamp: u64,
+ /// signature of the miner
+ pub miner_signature: MessageSignature,
+ /// list of signers signatures
+ pub signer_signature: Vec,
+ /// the list of block transactions
+ pub transactions: Vec,
+ /// check if the computed merkle tree root hash matches the one from the original block
+ pub valid_merkle_root: bool,
+}
+
+impl RPCReplayedBlock {
+ pub fn from_block(
+ block: NakamotoBlock,
+ block_fees: u128,
+ tenure_id: ConsensusHash,
+ parent_block_id: StacksBlockId,
+ ) -> Self {
+ let block_id = block.block_id();
+ let block_hash = block.header.block_hash();
+
+ Self {
+ block_id,
+ block_hash,
+ block_height: block.header.chain_length,
+ parent_block_id,
+ consensus_hash: tenure_id,
+ fees: block_fees,
+ tx_merkle_root: block.header.tx_merkle_root,
+ state_index_root: block.header.state_index_root,
+ timestamp: block.header.timestamp,
+ miner_signature: block.header.miner_signature,
+ signer_signature: block.header.signer_signature,
+ transactions: vec![],
+ valid_merkle_root: false,
+ }
+ }
+}
+
+/// Decode the HTTP request
+impl HttpRequest for RPCNakamotoBlockReplayRequestHandler {
+ fn verb(&self) -> &'static str {
+ "GET"
+ }
+
+ fn path_regex(&self) -> Regex {
+ Regex::new(r#"^/v3/blocks/replay/(?P[0-9a-f]{64})$"#).unwrap()
+ }
+
+ fn metrics_identifier(&self) -> &str {
+ "/v3/blocks/replay/:block_id"
+ }
+
+ /// Try to decode this request.
+ /// There's nothing to load here, so just make sure the request is well-formed.
+ fn try_parse_request(
+ &mut self,
+ preamble: &HttpRequestPreamble,
+ captures: &Captures,
+ query: Option<&str>,
+ _body: &[u8],
+ ) -> Result {
+ // If no authorization is set, then the block replay endpoint is not enabled
+ let Some(password) = &self.auth else {
+ return Err(Error::Http(400, "Bad Request.".into()));
+ };
+ let Some(auth_header) = preamble.headers.get("authorization") else {
+ return Err(Error::Http(401, "Unauthorized".into()));
+ };
+ if auth_header != password {
+ return Err(Error::Http(401, "Unauthorized".into()));
+ }
+ if preamble.get_content_length() != 0 {
+ return Err(Error::DecodeError(
+ "Invalid Http request: expected 0-length body".to_string(),
+ ));
+ }
+
+ let block_id_str = captures
+ .name("block_id")
+ .ok_or_else(|| {
+ Error::DecodeError("Failed to match path to block ID group".to_string())
+ })?
+ .as_str();
+
+ let block_id = StacksBlockId::from_hex(block_id_str)
+ .map_err(|_| Error::DecodeError("Invalid path: unparseable block id".to_string()))?;
+
+ self.block_id = Some(block_id);
+
+ Ok(HttpRequestContents::new().query_string(query))
+ }
+}
+
+impl RPCRequestHandler for RPCNakamotoBlockReplayRequestHandler {
+ /// Reset internal state
+ fn restart(&mut self) {
+ self.block_id = None;
+ }
+
+ /// Make the response
+ fn try_handle_request(
+ &mut self,
+ preamble: HttpRequestPreamble,
+ _contents: HttpRequestContents,
+ node: &mut StacksNodeState,
+ ) -> Result<(HttpResponsePreamble, HttpResponseContents), NetError> {
+ let Some(block_id) = &self.block_id else {
+ return Err(NetError::SendError("Missing `block_id`".into()));
+ };
+
+ let replayed_block_res =
+ node.with_node_state(|_network, sortdb, chainstate, _mempool, _rpc_args| {
+ self.block_replay(sortdb, chainstate)
+ });
+
+ // start loading up the block
+ let replayed_block = match replayed_block_res {
+ Ok(replayed_block) => replayed_block,
+ Err(ChainError::NoSuchBlockError) => {
+ return StacksHttpResponse::new_error(
+ &preamble,
+ &HttpNotFound::new(format!("No such block {block_id}\n")),
+ )
+ .try_into_contents()
+ .map_err(NetError::from)
+ }
+ Err(e) => {
+ // nope -- error trying to check
+ let msg = format!("Failed to load block {}: {:?}\n", &block_id, &e);
+ warn!("{}", &msg);
+ return StacksHttpResponse::new_error(&preamble, &HttpServerError::new(msg))
+ .try_into_contents()
+ .map_err(NetError::from);
+ }
+ };
+
+ let preamble = HttpResponsePreamble::ok_json(&preamble);
+ let body = HttpResponseContents::try_from_json(&replayed_block)?;
+ Ok((preamble, body))
+ }
+}
+
+impl StacksHttpRequest {
+ /// Make a new block_replay request to this endpoint
+ pub fn new_block_replay(host: PeerHost, block_id: &StacksBlockId) -> StacksHttpRequest {
+ StacksHttpRequest::new_for_peer(
+ host,
+ "GET".into(),
+ format!("/v3/blocks/replay/{block_id}"),
+ HttpRequestContents::new(),
+ )
+ .expect("FATAL: failed to construct request from infallible data")
+ }
+}
+
+/// Decode the HTTP response
+impl HttpResponse for RPCNakamotoBlockReplayRequestHandler {
+ /// Decode this response from a byte stream. This is called by the client to decode this
+ /// message
+ fn try_parse_response(
+ &self,
+ preamble: &HttpResponsePreamble,
+ body: &[u8],
+ ) -> Result {
+ let rpc_replayed_block: RPCReplayedBlock = parse_json(preamble, body)?;
+ Ok(HttpResponsePayload::try_from_json(rpc_replayed_block)?)
+ }
+}
+
+impl StacksHttpResponse {
+ pub fn decode_replayed_block(self) -> Result {
+ let contents = self.get_http_payload_ok()?;
+ let response_json: serde_json::Value = contents.try_into()?;
+ let replayed_block: RPCReplayedBlock = serde_json::from_value(response_json)
+ .map_err(|_e| Error::DecodeError("Failed to decode JSON".to_string()))?;
+ Ok(replayed_block)
+ }
+}
diff --git a/stackslib/src/net/api/mod.rs b/stackslib/src/net/api/mod.rs
index 2f46069c4e6..a5777a751d9 100644
--- a/stackslib/src/net/api/mod.rs
+++ b/stackslib/src/net/api/mod.rs
@@ -17,6 +17,7 @@ use crate::net::http::Error;
use crate::net::httpcore::StacksHttp;
use crate::net::Error as NetError;
+pub mod blockreplay;
pub mod callreadonly;
pub mod fastcallreadonly;
pub mod get_tenures_fork_info;
@@ -74,6 +75,9 @@ impl StacksHttp {
/// Register all RPC methods.
/// Put your new RPC method handlers here.
pub fn register_rpc_methods(&mut self) {
+ self.register_rpc_endpoint(blockreplay::RPCNakamotoBlockReplayRequestHandler::new(
+ self.auth_token.clone(),
+ ));
self.register_rpc_endpoint(callreadonly::RPCCallReadOnlyRequestHandler::new(
self.maximum_call_argument_size,
self.read_only_call_limit.clone(),
diff --git a/stackslib/src/net/api/tests/blockreplay.rs b/stackslib/src/net/api/tests/blockreplay.rs
new file mode 100644
index 00000000000..31e727727b1
--- /dev/null
+++ b/stackslib/src/net/api/tests/blockreplay.rs
@@ -0,0 +1,181 @@
+// Copyright (C) 2013-2020 Blockstack PBC, a public benefit corporation
+// Copyright (C) 2020-2025 Stacks Open Internet Foundation
+//
+// This program is free software: you can redistribute it and/or modify
+// it under the terms of the GNU General Public License as published by
+// the Free Software Foundation, either version 3 of the License, or
+// (at your option) any later version.
+//
+// This program is distributed in the hope that it will be useful,
+// but WITHOUT ANY WARRANTY; without even the implied warranty of
+// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+// GNU General Public License for more details.
+//
+// You should have received a copy of the GNU General Public License
+// along with this program. If not, see .
+
+use std::net::{IpAddr, Ipv4Addr, SocketAddr};
+
+use stacks_common::types::chainstate::StacksBlockId;
+
+use crate::chainstate::stacks::Error as ChainError;
+use crate::net::api::blockreplay;
+use crate::net::api::tests::TestRPC;
+use crate::net::connection::ConnectionOptions;
+use crate::net::httpcore::{StacksHttp, StacksHttpRequest};
+use crate::net::test::TestEventObserver;
+use crate::net::ProtocolFamily;
+
+#[test]
+fn test_try_parse_request() {
+ let addr = SocketAddr::new(IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), 33333);
+ let mut http = StacksHttp::new(addr.clone(), &ConnectionOptions::default());
+
+ let mut request = StacksHttpRequest::new_block_replay(addr.into(), &StacksBlockId([0x01; 32]));
+
+ // add the authorization header
+ request.add_header("authorization".into(), "password".into());
+
+ let bytes = request.try_serialize().unwrap();
+
+ debug!("Request:\n{}\n", std::str::from_utf8(&bytes).unwrap());
+
+ let (parsed_preamble, offset) = http.read_preamble(&bytes).unwrap();
+
+ let mut handler =
+ blockreplay::RPCNakamotoBlockReplayRequestHandler::new(Some("password".into()));
+
+ let mut parsed_request = http
+ .handle_try_parse_request(
+ &mut handler,
+ &parsed_preamble.expect_request(),
+ &bytes[offset..],
+ )
+ .unwrap();
+ assert_eq!(handler.block_id, Some(StacksBlockId([0x01; 32])));
+
+ // parsed request consumes headers that would not be in a constructed request
+ parsed_request.clear_headers();
+ parsed_request.add_header("authorization".into(), "password".into());
+
+ let (preamble, contents) = parsed_request.destruct();
+
+ assert_eq!(&preamble, request.preamble());
+}
+
+#[test]
+fn test_block_reply_errors() {
+ let mut handler =
+ blockreplay::RPCNakamotoBlockReplayRequestHandler::new(Some("password".into()));
+
+ let test_observer = TestEventObserver::new();
+ let mut rpc_test = TestRPC::setup_nakamoto(function_name!(), &test_observer);
+
+ let sort_db = rpc_test.peer_1.sortdb.take().unwrap();
+ let chainstate = rpc_test.peer_1.chainstate();
+
+ let err = handler.block_replay(&sort_db, chainstate).err().unwrap();
+
+ assert!(matches!(err, ChainError::InvalidStacksBlock(_)));
+ assert_eq!(err.to_string(), "block_id is None");
+
+ handler.block_id = Some(StacksBlockId([0x01; 32]));
+
+ let err = handler.block_replay(&sort_db, chainstate).err().unwrap();
+
+ assert!(matches!(err, ChainError::NoSuchBlockError));
+ assert_eq!(err.to_string(), "No such Stacks block");
+}
+
+#[test]
+fn test_try_make_response() {
+ let addr = SocketAddr::new(IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), 33333);
+
+ let test_observer = TestEventObserver::new();
+ let rpc_test = TestRPC::setup_nakamoto(function_name!(), &test_observer);
+
+ let nakamoto_consensus_hash = rpc_test.consensus_hash.clone();
+
+ let mut requests = vec![];
+
+ // query existing, non-empty Nakamoto block
+ let mut request =
+ StacksHttpRequest::new_block_replay(addr.clone().into(), &rpc_test.canonical_tip);
+ // add the authorization header
+ request.add_header("authorization".into(), "password".into());
+ requests.push(request);
+
+ // query non-existent block
+ let mut request =
+ StacksHttpRequest::new_block_replay(addr.clone().into(), &StacksBlockId([0x01; 32]));
+ // add the authorization header
+ request.add_header("authorization".into(), "password".into());
+ requests.push(request);
+
+ // unauthenticated request
+ let request =
+ StacksHttpRequest::new_block_replay(addr.clone().into(), &StacksBlockId([0x00; 32]));
+ requests.push(request);
+
+ let mut responses = rpc_test.run(requests);
+
+ // got the Nakamoto tip
+ let response = responses.remove(0);
+
+ debug!(
+ "Response:\n{}\n",
+ std::str::from_utf8(&response.try_serialize().unwrap()).unwrap()
+ );
+
+ let resp = response.decode_replayed_block().unwrap();
+
+ let tip_block = test_observer.get_blocks().last().unwrap().clone();
+
+ assert_eq!(resp.consensus_hash, nakamoto_consensus_hash);
+ assert_eq!(resp.consensus_hash, tip_block.metadata.consensus_hash);
+
+ assert_eq!(resp.block_hash, tip_block.block.block_hash);
+ assert_eq!(resp.block_id, tip_block.metadata.index_block_hash());
+ assert_eq!(resp.parent_block_id, tip_block.parent);
+
+ assert_eq!(resp.block_height, tip_block.metadata.stacks_block_height);
+
+ assert!(resp.valid_merkle_root);
+
+ assert_eq!(resp.transactions.len(), tip_block.receipts.len());
+
+ for tx_index in 0..resp.transactions.len() {
+ assert_eq!(
+ resp.transactions[tx_index].txid,
+ tip_block.receipts[tx_index].transaction.txid()
+ );
+ assert_eq!(
+ resp.transactions[tx_index].events.len(),
+ tip_block.receipts[tx_index].events.len()
+ );
+ assert_eq!(
+ resp.transactions[tx_index].result,
+ tip_block.receipts[tx_index].result
+ );
+ }
+
+ // got a failure (404)
+ let response = responses.remove(0);
+ debug!(
+ "Response:\n{}\n",
+ std::str::from_utf8(&response.try_serialize().unwrap()).unwrap()
+ );
+
+ let (preamble, body) = response.destruct();
+ assert_eq!(preamble.status_code, 404);
+
+ // got another failure (401 this time)
+ let response = responses.remove(0);
+ debug!(
+ "Response:\n{}\n",
+ std::str::from_utf8(&response.try_serialize().unwrap()).unwrap()
+ );
+
+ let (preamble, body) = response.destruct();
+ assert_eq!(preamble.status_code, 401);
+}
diff --git a/stackslib/src/net/api/tests/mod.rs b/stackslib/src/net/api/tests/mod.rs
index c7b6285665a..97a60a6b43a 100644
--- a/stackslib/src/net/api/tests/mod.rs
+++ b/stackslib/src/net/api/tests/mod.rs
@@ -59,6 +59,7 @@ use crate::net::{
UrlString,
};
+mod blockreplay;
mod callreadonly;
mod fastcallreadonly;
mod get_tenures_fork_info;