diff --git a/docs/rpc/openapi.yaml b/docs/rpc/openapi.yaml
index cfaa1d7eb..51fe3444e 100644
--- a/docs/rpc/openapi.yaml
+++ b/docs/rpc/openapi.yaml
@@ -576,22 +576,22 @@ paths:
- name: block
in: query
required: true
- description: Proposed Block. Must match Rust struct `NakamotoBlock`
+ description: 'Proposed Block. Must match Rust struct `NakamotoBlock`'
- name: tenure_start_block
in: query
required: true
- description: `StacksBlockId` for block at the start of current mining tenure
+ description: '`StacksBlockId` for block at the start of current mining tenure'
schema:
type: string
- name: chain_id
in: query
required: true
- description: Identifies which chain block is for (Mainnet, Testnet, etc.)
+ description: 'Identifies which chain block is for (Mainnet, Testnet, etc.)'
schema:
type: integer
- name: total_burn
in: query
required: true
- description: Total BTC burn so far
+ description: 'Total BTC burn so far'
schema:
type: integer
\ No newline at end of file
diff --git a/stackslib/src/chainstate/nakamoto/miner.rs b/stackslib/src/chainstate/nakamoto/miner.rs
index 62b0da413..ee30337a1 100644
--- a/stackslib/src/chainstate/nakamoto/miner.rs
+++ b/stackslib/src/chainstate/nakamoto/miner.rs
@@ -15,7 +15,7 @@
// along with this program. If not, see .
use std::collections::{HashMap, HashSet};
-use std::convert::From;
+use std::io::{Read, Write};
use std::sync::atomic::{AtomicBool, Ordering};
use std::sync::{Arc, Mutex};
use std::thread::ThreadId;
@@ -30,7 +30,9 @@ use clarity::vm::database::BurnStateDB;
use clarity::vm::errors::Error as InterpreterError;
use clarity::vm::types::TypeSignature;
use serde::Deserialize;
-use stacks_common::codec::{read_next, write_next, StacksMessageCodec};
+use stacks_common::codec::{
+ read_next, write_next, Error as CodecError, StacksMessageCodec, MAX_PAYLOAD_LEN,
+};
use stacks_common::types::chainstate::{
BlockHeaderHash, BurnchainHeaderHash, ConsensusHash, StacksAddress, StacksBlockId, TrieHash,
};
@@ -62,7 +64,7 @@ use crate::chainstate::stacks::miner::{
TransactionProblematic, TransactionResult, TransactionSkipped,
};
use crate::chainstate::stacks::{Error, StacksBlockHeader, *};
-use crate::clarity_vm::clarity::{ClarityConnection, ClarityInstance, Error as clarity_error};
+use crate::clarity_vm::clarity::{ClarityConnection, ClarityInstance};
use crate::core::mempool::*;
use crate::core::*;
use crate::cost_estimates::metrics::CostMetric;
@@ -71,7 +73,7 @@ use crate::monitoring::{
set_last_mined_block_transaction_count, set_last_mined_execution_cost_observed,
};
use crate::net::relay::Relayer;
-use crate::net::Error as net_error;
+use crate::util_lib::db::Error as DBError;
/// Nakamaoto tenure information
pub struct NakamotoTenureInfo {
@@ -502,6 +504,16 @@ impl NakamotoBlockBuilder {
pub fn get_bytes_so_far(&self) -> u64 {
self.bytes_so_far
}
+
+ /// Add tx to block with no safety checks
+ /// For testing purposes only
+ ///
+ /// FIXME: Why does this not work in `nakamoto_integrations.rs` with `#[cfg(test)]`
+ //#[cfg(test)]
+ pub fn mine_tx_no_checks(&mut self, tx: StacksTransaction) {
+ self.bytes_so_far += tx.tx_len();
+ self.txs.push(tx)
+ }
}
impl BlockBuilder for NakamotoBlockBuilder {
@@ -623,3 +635,255 @@ impl BlockBuilder for NakamotoBlockBuilder {
result
}
}
+
+/// This enum is used to supply a `reason_code` for validation
+/// rejection responses. This is serialized as an enum with string
+/// type (in jsonschema terminology).
+#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)]
+pub enum ValidateRejectCode {
+ BadBlockHash,
+ BadTransaction,
+ InvalidBlock,
+ ChainstateError,
+ UnknownParent,
+}
+
+/// A response for block proposal validation
+/// that the stacks-node thinks should be rejected.
+#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)]
+pub struct BlockValidateReject {
+ pub reason: String,
+ pub reason_code: ValidateRejectCode,
+}
+
+/// A response for block proposal validation
+/// that the stacks-node thinks is acceptable.
+#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)]
+pub struct BlockValidateOk {
+ pub block: NakamotoBlock,
+ pub cost: ExecutionCost,
+ pub size: u64,
+}
+
+/// This enum is used for serializing the response to block
+/// proposal validation.
+#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)]
+#[serde(tag = "Result")]
+pub enum BlockValidateResponse {
+ Ok(BlockValidateOk),
+ Reject(BlockValidateReject),
+}
+
+impl From> for BlockValidateResponse {
+ fn from(value: Result) -> Self {
+ match value {
+ Ok(o) => BlockValidateResponse::Ok(o),
+ Err(e) => BlockValidateResponse::Reject(e),
+ }
+ }
+}
+
+impl From for BlockValidateReject {
+ fn from(value: Error) -> Self {
+ BlockValidateReject {
+ reason: format!("Chainstate Error: {value}"),
+ reason_code: ValidateRejectCode::ChainstateError,
+ }
+ }
+}
+
+impl From for BlockValidateReject {
+ fn from(value: DBError) -> Self {
+ Error::from(value).into()
+ }
+}
+
+/// Represents a block proposed to the `v2/block_proposal` endpoint for validation
+#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)]
+pub struct NakamotoBlockProposal {
+ /// Proposed block
+ pub block: NakamotoBlock,
+ // tenure ID -- this is the index block hash of the start block of the last tenure (i.e.
+ // the data we committed to in the block-commit). If this is an epoch 2.x parent, then
+ // this is just the index block hash of the parent Stacks block.
+ pub tenure_start_block: StacksBlockId,
+ /// Identifies which chain block is for (Mainnet, Testnet, etc.)
+ pub chain_id: u32,
+ /// total BTC burn so far
+ pub total_burn: u64,
+}
+
+impl StacksMessageCodec for NakamotoBlockProposal {
+ fn consensus_serialize(&self, fd: &mut W) -> Result<(), CodecError> {
+ write_next(fd, &self.block)?;
+ write_next(fd, &self.tenure_start_block)?;
+ write_next(fd, &self.chain_id)?;
+ write_next(fd, &self.total_burn)
+ }
+
+ fn consensus_deserialize(fd: &mut R) -> Result {
+ Ok(Self {
+ block: read_next(fd)?,
+ tenure_start_block: read_next(fd)?,
+ chain_id: read_next(fd)?,
+ total_burn: read_next(fd)?,
+ })
+ }
+}
+
+impl NakamotoBlockProposal {
+ /// Test this block proposal against the current chain state and
+ /// either accept or reject the proposal
+ ///
+ /// This is done in 2 steps:
+ /// - Static validation of the block, which checks the following:
+ /// - Block is well-formed
+ /// - Transactions are well-formed
+ /// - Miner signature is valid
+ /// - Validation of transactions by executing them agains current chainstate.
+ /// This is resource intensive, and therefore done only if previous checks pass
+ pub fn validate(
+ &self,
+ sortdb: &SortitionDB,
+ chainstate: &mut StacksChainState, // not directly used; used as a handle to open other chainstates
+ ) -> Result {
+ let ts_start = get_epoch_time_ms();
+ // Measure time from start of function
+ let time_elapsed = || get_epoch_time_ms().saturating_sub(ts_start);
+
+ let mainnet = self.chain_id == CHAIN_ID_MAINNET;
+ if self.chain_id != chainstate.chain_id || mainnet != chainstate.mainnet {
+ return Err(BlockValidateReject {
+ reason_code: ValidateRejectCode::InvalidBlock,
+ reason: "Wrong netowrk/chain_id".into(),
+ });
+ }
+
+ let burn_dbconn = sortdb.index_conn();
+ let sort_tip = SortitionDB::get_canonical_sortition_tip(sortdb.conn())?;
+ let mut db_handle = sortdb.index_handle(&sort_tip);
+ let expected_burn =
+ NakamotoChainState::get_expected_burns(&mut db_handle, chainstate.db(), &self.block)?;
+
+ // Static validation checks
+ NakamotoChainState::validate_nakamoto_block_burnchain(
+ &db_handle,
+ expected_burn,
+ &self.block,
+ mainnet,
+ self.chain_id,
+ )?;
+
+ // Validate block txs against chainstate
+ let parent_stacks_header = NakamotoChainState::get_block_header(
+ chainstate.db(),
+ &self.block.header.parent_block_id,
+ )?
+ .ok_or_else(|| BlockValidateReject {
+ reason_code: ValidateRejectCode::InvalidBlock,
+ reason: "Invalid parent block".into(),
+ })?;
+ let tenure_change = self
+ .block
+ .txs
+ .iter()
+ .find(|tx| matches!(tx.payload, TransactionPayload::TenureChange(..)));
+ let coinbase = self
+ .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 mut builder = NakamotoBlockBuilder::new(
+ &parent_stacks_header,
+ &self.block.header.consensus_hash,
+ self.total_burn,
+ tenure_change,
+ coinbase,
+ )?;
+
+ let mut miner_tenure_info =
+ builder.load_tenure_info(chainstate, &burn_dbconn, tenure_cause)?;
+ let mut tenure_tx = builder.tenure_begin(&burn_dbconn, &mut miner_tenure_info)?;
+
+ for (i, tx) in self.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,
+ ASTRules::PrecheckSize,
+ );
+ let err = match tx_result {
+ TransactionResult::Success(_) => Ok(()),
+ TransactionResult::Skipped(s) => Err(format!("tx {i} skipped: {}", s.error)),
+ TransactionResult::ProcessingError(e) => {
+ Err(format!("Error processing tx {i}: {}", e.error))
+ }
+ TransactionResult::Problematic(p) => {
+ Err(format!("Problematic tx {i}: {}", p.error))
+ }
+ };
+ if let Err(reason) = err {
+ warn!(
+ "Rejected block proposal";
+ "reason" => %reason,
+ "tx" => ?tx,
+ );
+ return Err(BlockValidateReject {
+ reason,
+ reason_code: ValidateRejectCode::BadTransaction,
+ });
+ }
+ }
+
+ let mut block = builder.mine_nakamoto_block(&mut tenure_tx);
+ let size = builder.get_bytes_so_far();
+ let cost = builder.tenure_finish(tenure_tx);
+
+ // Clone signatures from block proposal
+ // These have already been validated by `validate_nakamoto_block_burnchain()``
+ block.header.miner_signature = self.block.header.miner_signature.clone();
+ block.header.signer_signature = self.block.header.signer_signature.clone();
+
+ // Assuming `tx_nerkle_root` has been checked we don't need to hash the whole block
+ let expected_block_header_hash = self.block.header.block_hash();
+ let computed_block_header_hash = block.header.block_hash();
+
+ if computed_block_header_hash != expected_block_header_hash {
+ warn!(
+ "Rejected block proposal";
+ "reason" => "Block hash is not as expected",
+ "expected_block_header_hash" => %expected_block_header_hash,
+ "computed_block_header_hash" => %computed_block_header_hash,
+ "expected_block" => %serde_json::to_string(&serde_json::to_value(&self.block).unwrap()).unwrap(),
+ "computed_block" => %serde_json::to_string(&serde_json::to_value(&block).unwrap()).unwrap(),
+ );
+ return Err(BlockValidateReject {
+ reason: "Block hash is not as expected".into(),
+ reason_code: ValidateRejectCode::BadBlockHash,
+ });
+ }
+
+ info!(
+ "Participant: validated anchored block";
+ "block_header_hash" => %computed_block_header_hash,
+ "height" => block.header.chain_length,
+ "tx_count" => block.txs.len(),
+ "parent_stacks_block_id" => %block.header.parent_block_id,
+ "block_size" => size,
+ "execution_cost" => %cost,
+ "validation_time_ms" => time_elapsed(),
+ "tx_fees_microstacks" => block.txs.iter().fold(0, |agg: u64, tx| {
+ agg.saturating_add(tx.get_tx_fee())
+ })
+ );
+
+ Ok(BlockValidateOk { block, cost, size })
+ }
+}
diff --git a/stackslib/src/chainstate/nakamoto/mod.rs b/stackslib/src/chainstate/nakamoto/mod.rs
index c1dacf57a..d3a3e26a0 100644
--- a/stackslib/src/chainstate/nakamoto/mod.rs
+++ b/stackslib/src/chainstate/nakamoto/mod.rs
@@ -87,7 +87,6 @@ use crate::util_lib::db::{
pub mod coordinator;
pub mod miner;
-pub mod proposal;
pub mod tenure;
#[cfg(test)]
@@ -577,13 +576,9 @@ impl NakamotoBlock {
// there is one coinbase.
// go find it.
- self.txs.iter().find(|tx| {
- if let TransactionPayload::Coinbase(..) = &tx.payload {
- true
- } else {
- false
- }
- })
+ self.txs
+ .iter()
+ .find(|tx| matches!(tx.payload, TransactionPayload::Coinbase(..)))
}
/// Get the VRF proof from this block.
@@ -1051,7 +1046,7 @@ impl NakamotoBlock {
.map_err(|e| {
warn!(
"Leader key did not contain a hash160 of the miner signing public key";
- "leader_key" => format!("{:?}", &leader_key),
+ "leader_key" => ?leader_key,
);
e
})?;
@@ -2165,7 +2160,7 @@ impl NakamotoChainState {
total_tenure_cost,
&tenure_tx_fees.to_string(),
&header.parent_block_id,
- if tenure_changed { &1i64 } else { &0 },
+ if tenure_changed { &1i64 } else { &0i64 },
&vrf_proof_bytes.as_ref(),
];
diff --git a/stackslib/src/chainstate/nakamoto/proposal.rs b/stackslib/src/chainstate/nakamoto/proposal.rs
deleted file mode 100644
index 98c1d89d1..000000000
--- a/stackslib/src/chainstate/nakamoto/proposal.rs
+++ /dev/null
@@ -1,319 +0,0 @@
-use std::collections::{HashMap, HashSet};
-use std::convert::From;
-use std::io::{Read, Write};
-use std::sync::atomic::{AtomicBool, Ordering};
-use std::sync::{Arc, Mutex};
-use std::thread::ThreadId;
-use std::{cmp, fs, mem};
-
-use clarity::vm::analysis::{CheckError, CheckErrors};
-use clarity::vm::ast::errors::ParseErrors;
-use clarity::vm::ast::ASTRules;
-use clarity::vm::clarity::TransactionConnection;
-use clarity::vm::costs::ExecutionCost;
-use clarity::vm::database::BurnStateDB;
-use clarity::vm::errors::Error as InterpreterError;
-use clarity::vm::types::TypeSignature;
-use serde::Deserialize;
-use stacks_common::codec::{
- read_next, write_next, Error as CodecError, StacksMessageCodec, MAX_PAYLOAD_LEN,
-};
-use stacks_common::types::chainstate::{
- BlockHeaderHash, BurnchainHeaderHash, ConsensusHash, StacksAddress, StacksBlockId, TrieHash,
-};
-use stacks_common::types::StacksPublicKeyBuffer;
-use stacks_common::util::get_epoch_time_ms;
-use stacks_common::util::hash::{Hash160, MerkleTree, Sha512Trunc256Sum};
-use stacks_common::util::secp256k1::{MessageSignature, Secp256k1PrivateKey};
-use stacks_common::util::vrf::*;
-
-use crate::burnchains::{PrivateKey, PublicKey};
-use crate::chainstate::burn::db::sortdb::{SortitionDB, SortitionDBConn, SortitionHandleTx};
-use crate::chainstate::burn::operations::*;
-use crate::chainstate::burn::*;
-use crate::chainstate::nakamoto::miner::NakamotoBlockBuilder;
-use crate::chainstate::nakamoto::{
- NakamotoBlock, NakamotoBlockHeader, NakamotoChainState, SetupBlockResult,
-};
-use crate::chainstate::stacks::address::StacksAddressExtensions;
-use crate::chainstate::stacks::db::accounts::MinerReward;
-use crate::chainstate::stacks::db::blocks::MemPoolRejection;
-use crate::chainstate::stacks::db::transactions::{
- handle_clarity_runtime_error, ClarityRuntimeTxError,
-};
-use crate::chainstate::stacks::db::{
- ChainstateTx, ClarityTx, MinerRewardInfo, StacksChainState, StacksHeaderInfo,
- MINER_REWARD_MATURITY,
-};
-use crate::chainstate::stacks::events::{StacksTransactionEvent, StacksTransactionReceipt};
-use crate::chainstate::stacks::miner::{
- BlockBuilder, BlockBuilderSettings, BlockLimitFunction, TransactionError,
- TransactionProblematic, TransactionResult, TransactionSkipped,
-};
-use crate::chainstate::stacks::{Error, StacksBlockHeader, *};
-use crate::clarity_vm::clarity::{ClarityConnection, ClarityInstance, Error as clarity_error};
-use crate::core::mempool::*;
-use crate::core::*;
-use crate::cost_estimates::metrics::CostMetric;
-use crate::cost_estimates::CostEstimator;
-use crate::monitoring::{
- set_last_mined_block_transaction_count, set_last_mined_execution_cost_observed,
-};
-use crate::net::relay::Relayer;
-use crate::net::Error as net_error;
-use crate::util_lib::db::Error as DBError;
-
-/// This enum is used to supply a `reason_code` for validation
-/// rejection responses. This is serialized as an enum with string
-/// type (in jsonschema terminology).
-#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)]
-pub enum ValidateRejectCode {
- BadBlockHash,
- BadTransaction,
- InvalidBlock,
- ChainstateError,
- UnknownParent,
-}
-
-/// A response for block proposal validation
-/// that the stacks-node thinks should be rejected.
-#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)]
-pub struct BlockValidateReject {
- pub reason: String,
- pub reason_code: ValidateRejectCode,
-}
-
-/// A response for block proposal validation
-/// that the stacks-node thinks is acceptable.
-#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)]
-pub struct BlockValidateOk {
- pub block: NakamotoBlock,
- pub cost: ExecutionCost,
- pub size: u64,
-}
-
-/// This enum is used for serializing the response to block
-/// proposal validation.
-#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)]
-#[serde(tag = "Result")]
-pub enum BlockValidateResponse {
- Ok(BlockValidateOk),
- Reject(BlockValidateReject),
-}
-
-impl From> for BlockValidateResponse {
- fn from(value: Result) -> Self {
- match value {
- Ok(o) => BlockValidateResponse::Ok(o),
- Err(e) => BlockValidateResponse::Reject(e),
- }
- }
-}
-
-impl From for BlockValidateReject {
- fn from(value: Error) -> Self {
- BlockValidateReject {
- reason: format!("Chainstate Error: {value}"),
- reason_code: ValidateRejectCode::ChainstateError,
- }
- }
-}
-
-impl From for BlockValidateReject {
- fn from(value: DBError) -> Self {
- Error::from(value).into()
- }
-}
-
-/// Represents a block proposed to the `v2/block_proposal` endpoint for validation
-#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)]
-pub struct NakamotoBlockProposal {
- /// Proposed block
- pub block: NakamotoBlock,
- // tenure ID -- this is the index block hash of the start block of the last tenure (i.e.
- // the data we committed to in the block-commit). If this is an epoch 2.x parent, then
- // this is just the index block hash of the parent Stacks block.
- pub tenure_start_block: StacksBlockId,
- /// Identifies which chain block is for (Mainnet, Testnet, etc.)
- pub chain_id: u32,
- /// total BTC burn so far
- pub total_burn: u64,
-}
-
-impl StacksMessageCodec for NakamotoBlockProposal {
- fn consensus_serialize(&self, fd: &mut W) -> Result<(), CodecError> {
- write_next(fd, &self.block)?;
- write_next(fd, &self.tenure_start_block)?;
- write_next(fd, &self.chain_id)?;
- write_next(fd, &self.total_burn)
- }
-
- fn consensus_deserialize(fd: &mut R) -> Result {
- Ok(Self {
- block: read_next(fd)?,
- tenure_start_block: read_next(fd)?,
- chain_id: read_next(fd)?,
- total_burn: read_next(fd)?,
- })
- }
-}
-
-impl NakamotoBlockProposal {
- /// Test this block proposal against the current chain state and
- /// either accept or reject the proposal
- ///
- /// This is done in 2 steps:
- /// - Static validation of the block, which checks the following:
- /// - Block is well-formed
- /// - Transactions are well-formed
- /// - Miner signature is valid
- /// - Validation of transactions by executing them agains current chainstate.
- /// This is resource intensive, and therefore done only if previous checks pass
- pub fn validate(
- &self,
- sortdb: &SortitionDB,
- chainstate: &mut StacksChainState, // not directly used; used as a handle to open other chainstates
- ) -> Result {
- // Time this function
- let ts_start = get_epoch_time_ms();
-
- let mainnet = self.chain_id == CHAIN_ID_MAINNET;
- if self.chain_id != chainstate.chain_id || mainnet != chainstate.mainnet {
- return Err(BlockValidateReject {
- reason_code: ValidateRejectCode::InvalidBlock,
- reason: "Wrong netowrk/chain_id".into(),
- });
- }
-
- let burn_dbconn = sortdb.index_conn();
- let sort_tip = SortitionDB::get_canonical_sortition_tip(sortdb.conn())?;
- let mut db_handle = sortdb.index_handle(&sort_tip);
- // Is this safe?
- let mut _chainstate = chainstate.reopen()?.0;
- let (chainstate_tx, _clarity_instance) = _chainstate.chainstate_tx_begin()?;
- let expected_burn =
- NakamotoChainState::get_expected_burns(&mut db_handle, &chainstate_tx, &self.block)?;
-
- // Static validation checks
- NakamotoChainState::validate_nakamoto_block_burnchain(
- &db_handle,
- expected_burn,
- &self.block,
- mainnet,
- self.chain_id,
- )?;
-
- // Validate block txs against chainstate
- let parent_stacks_header = NakamotoChainState::get_block_header(
- &chainstate_tx,
- &self.block.header.parent_block_id,
- )?
- .ok_or_else(|| BlockValidateReject {
- reason_code: ValidateRejectCode::InvalidBlock,
- reason: "Invalid parent block".into(),
- })?;
- let tenure_change = self
- .block
- .txs
- .iter()
- .find(|tx| matches!(tx.payload, TransactionPayload::TenureChange(..)));
- let coinbase = self
- .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 mut builder = NakamotoBlockBuilder::new_from_parent(
- &self.tenure_start_block,
- &parent_stacks_header,
- &self.block.header.consensus_hash,
- self.total_burn,
- tenure_change,
- coinbase,
- )?;
-
- let mut miner_tenure_info =
- builder.load_tenure_info(chainstate, &burn_dbconn, tenure_cause)?;
- let mut tenure_tx = builder.tenure_begin(&burn_dbconn, &mut miner_tenure_info)?;
-
- for (i, tx) in self.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,
- ASTRules::PrecheckSize,
- );
- let err = match tx_result {
- TransactionResult::Success(_) => Ok(()),
- TransactionResult::Skipped(s) => Err(format!("tx {i} skipped: {}", s.error)),
- TransactionResult::ProcessingError(e) => {
- Err(format!("Error processing tx {i}: {}", e.error))
- }
- TransactionResult::Problematic(p) => {
- Err(format!("Problematic tx {i}: {}", p.error))
- }
- };
- if let Err(reason) = err {
- warn!(
- "Rejected block proposal";
- "reason" => %reason,
- "tx" => ?tx,
- );
- return Err(BlockValidateReject {
- reason,
- reason_code: ValidateRejectCode::BadTransaction,
- });
- }
- }
-
- let mut block = builder.mine_nakamoto_block(&mut tenure_tx);
- let size = builder.get_bytes_so_far();
- let cost = builder.tenure_finish(tenure_tx);
-
- // Clone signatures from block proposal
- // These have already been validated by `validate_nakamoto_block_burnchain()``
- block.header.miner_signature = self.block.header.miner_signature.clone();
- block.header.signer_signature = self.block.header.signer_signature.clone();
-
- // Assuming `tx_nerkle_root` has been checked we don't need to hash the whole block
- let expected_block_header_hash = self.block.header.block_hash();
- let computed_block_header_hash = block.header.block_hash();
-
- if computed_block_header_hash != expected_block_header_hash {
- warn!(
- "Rejected block proposal";
- "reason" => "Block hash is not as expected",
- "expected_block_header_hash" => %expected_block_header_hash,
- "computed_block_header_hash" => %computed_block_header_hash,
- );
- return Err(BlockValidateReject {
- reason: "Block hash is not as expected".into(),
- reason_code: ValidateRejectCode::BadBlockHash,
- });
- }
-
- let ts_end = get_epoch_time_ms();
-
- info!(
- "Participant: validated anchored block";
- "block_header_hash" => %computed_block_header_hash,
- "height" => block.header.chain_length,
- "tx_count" => block.txs.len(),
- "parent_stacks_block_id" => %block.header.parent_block_id,
- "block_size" => size,
- "execution_cost" => %cost,
- "validation_time_ms" => ts_end.saturating_sub(ts_start),
- "tx_fees_microstacks" => block.txs.iter().fold(0, |agg: u64, tx| {
- agg.saturating_add(tx.get_tx_fee())
- })
- );
-
- Ok(BlockValidateOk { block, cost, size })
- }
-}
diff --git a/stackslib/src/chainstate/stacks/boot/pox_4_tests.rs b/stackslib/src/chainstate/stacks/boot/pox_4_tests.rs
index 3fd4ee173..1051a97e0 100644
--- a/stackslib/src/chainstate/stacks/boot/pox_4_tests.rs
+++ b/stackslib/src/chainstate/stacks/boot/pox_4_tests.rs
@@ -507,7 +507,7 @@ fn pox_extend_transition() {
for cycle_number in first_v4_cycle..(first_v4_cycle + 4) {
let cycle_start = burnchain.reward_cycle_to_block_height(cycle_number);
let reward_set_entries = get_reward_set_entries_at(&mut peer, &latest_block, cycle_start);
- info!("----- {cycle_number} -----");
+ //info!("----- {cycle_number} -----");
assert_eq!(reward_set_entries.len(), 1);
assert_eq!(
reward_set_entries[0].reward_address.bytes(),
@@ -607,6 +607,7 @@ fn pox_extend_transition() {
for cycle_number in (first_v4_cycle + 4)..(first_v4_cycle + 10) {
let cycle_start = burnchain.reward_cycle_to_block_height(cycle_number);
let reward_set_entries = get_reward_set_entries_at(&mut peer, &latest_block, cycle_start);
+ info!("----- {cycle_number} -----");
assert_eq!(reward_set_entries.len(), 1);
assert_eq!(
reward_set_entries[0].reward_address.bytes(),
diff --git a/stackslib/src/chainstate/stacks/transaction.rs b/stackslib/src/chainstate/stacks/transaction.rs
index ff9efc772..cf46bb7f4 100644
--- a/stackslib/src/chainstate/stacks/transaction.rs
+++ b/stackslib/src/chainstate/stacks/transaction.rs
@@ -710,7 +710,7 @@ impl StacksTransaction {
&self,
) -> Option<(&CoinbasePayload, Option<&PrincipalData>, Option<&VRFProof>)> {
match &self.payload {
- TransactionPayload::Coinbase(ref payload, ref recipient_opt, ref vrf_proof_opt) => {
+ TransactionPayload::Coinbase(payload, recipient_opt, vrf_proof_opt) => {
Some((payload, recipient_opt.as_ref(), vrf_proof_opt.as_ref()))
}
_ => None,
@@ -720,7 +720,7 @@ impl StacksTransaction {
/// Try to convert to a tenure change payload
pub fn try_as_tenure_change(&self) -> Option<&TenureChangePayload> {
match &self.payload {
- TransactionPayload::TenureChange(ref tc_payload) => Some(tc_payload),
+ TransactionPayload::TenureChange(tc_payload) => Some(tc_payload),
_ => None,
}
}
diff --git a/stackslib/src/net/api/postblock_proposal.rs b/stackslib/src/net/api/postblock_proposal.rs
index 057cc9f8d..345b3ccde 100644
--- a/stackslib/src/net/api/postblock_proposal.rs
+++ b/stackslib/src/net/api/postblock_proposal.rs
@@ -32,7 +32,7 @@ use stacks_common::util::retry::BoundReader;
use crate::burnchains::affirmation::AffirmationMap;
use crate::burnchains::Txid;
use crate::chainstate::burn::db::sortdb::SortitionDB;
-use crate::chainstate::nakamoto::proposal::{BlockValidateResponse, NakamotoBlockProposal};
+use crate::chainstate::nakamoto::miner::{BlockValidateResponse, NakamotoBlockProposal};
use crate::chainstate::nakamoto::NakamotoBlock;
use crate::chainstate::stacks::db::blocks::MINIMUM_TX_FEE_RATE_PER_BYTE;
use crate::chainstate::stacks::db::StacksChainState;
@@ -42,7 +42,7 @@ use crate::chainstate::stacks::{
use crate::core::mempool::MemPoolDB;
use crate::cost_estimates::FeeRateEstimate;
use crate::net::http::{
- parse_json, Error, HttpBadRequest, HttpContentType, HttpNotFound, HttpRequest,
+ http_reason, parse_json, Error, HttpBadRequest, HttpContentType, HttpNotFound, HttpRequest,
HttpRequestContents, HttpRequestPreamble, HttpResponse, HttpResponseContents,
HttpResponsePayload, HttpResponsePreamble, HttpServerError,
};
@@ -162,14 +162,25 @@ impl RPCRequestHandler for RPCBlockProposalRequestHandler {
.take()
.ok_or(NetError::SendError("`block_proposal` not set".into()))?;
- let resp = node.with_node_state(|_network, sortdb, chainstate, _mempool, _rpc_args| {
+ let res = node.with_node_state(|_network, sortdb, chainstate, _mempool, _rpc_args| {
block_proposal.validate(sortdb, chainstate)
});
- let mut preamble = HttpResponsePreamble::ok_json(&preamble);
- preamble.set_canonical_stacks_tip_height(Some(node.canonical_stacks_tip_height()));
- let body = HttpResponseContents::try_from_json(&resp)?;
- Ok((preamble, body))
+ match res {
+ Ok(ok) => {
+ let mut preamble = HttpResponsePreamble::accepted_json(&preamble);
+ preamble.set_canonical_stacks_tip_height(Some(node.canonical_stacks_tip_height()));
+ let body = HttpResponseContents::try_from_json(&ok)?;
+ Ok((preamble, body))
+ }
+ Err(err) => {
+ let code = 400;
+ let mut preamble = HttpResponsePreamble::error_json(code, http_reason(code));
+ preamble.set_canonical_stacks_tip_height(Some(node.canonical_stacks_tip_height()));
+ let body = HttpResponseContents::try_from_json(&err)?;
+ Ok((preamble, body))
+ }
+ }
}
}
diff --git a/stackslib/src/net/http/response.rs b/stackslib/src/net/http/response.rs
index 57a1afb2c..6c2a61066 100644
--- a/stackslib/src/net/http/response.rs
+++ b/stackslib/src/net/http/response.rs
@@ -34,7 +34,7 @@ use crate::net::http::common::{
};
use crate::net::http::request::{HttpRequestContents, HttpRequestPreamble};
use crate::net::http::stream::HttpChunkGenerator;
-use crate::net::http::{write_headers, Error, HttpContentType, HttpVersion};
+use crate::net::http::{http_reason, write_headers, Error, HttpContentType, HttpVersion};
/// HTTP response preamble. This captures all HTTP header information, but in a way that
/// certain fields that nodes rely on are guaranteed to have correct, sensible values.
@@ -185,17 +185,28 @@ impl HttpResponsePreamble {
)
}
- pub fn ok_json(preamble: &HttpRequestPreamble) -> HttpResponsePreamble {
+ pub fn success_2xx_json(
+ preamble: &HttpRequestPreamble,
+ status_code: u16,
+ ) -> HttpResponsePreamble {
HttpResponsePreamble::new(
preamble.version,
- 200,
- "OK".to_string(),
+ status_code,
+ http_reason(status_code).to_string(),
None,
HttpContentType::JSON,
preamble.keep_alive,
)
}
+ pub fn ok_json(preamble: &HttpRequestPreamble) -> HttpResponsePreamble {
+ Self::success_2xx_json(preamble, 200)
+ }
+
+ pub fn accepted_json(preamble: &HttpRequestPreamble) -> HttpResponsePreamble {
+ Self::success_2xx_json(preamble, 202)
+ }
+
pub fn raw_ok_json(version: HttpVersion, keep_alive: bool) -> HttpResponsePreamble {
HttpResponsePreamble::new(
version,
diff --git a/testnet/stacks-node/src/tests/nakamoto_integrations.rs b/testnet/stacks-node/src/tests/nakamoto_integrations.rs
index 3ca0b6949..8d45251bf 100644
--- a/testnet/stacks-node/src/tests/nakamoto_integrations.rs
+++ b/testnet/stacks-node/src/tests/nakamoto_integrations.rs
@@ -18,14 +18,19 @@ use std::sync::{Arc, Mutex};
use std::time::{Duration, Instant};
use std::{env, thread};
+use clarity::vm::ast::ASTRules;
use clarity::vm::costs::ExecutionCost;
use clarity::vm::types::PrincipalData;
use lazy_static::lazy_static;
+//use stacks::burnchains::tests::TestMinerFactory;
use stacks::burnchains::MagicBytes;
use stacks::chainstate::burn::db::sortdb::SortitionDB;
use stacks::chainstate::coordinator::comm::CoordinatorChannels;
+use stacks::chainstate::nakamoto::miner::{NakamotoBlockBuilder, NakamotoBlockProposal};
use stacks::chainstate::nakamoto::NakamotoChainState;
use stacks::chainstate::stacks::db::StacksChainState;
+use stacks::chainstate::stacks::miner::{BlockBuilder, BlockLimitFunction, TransactionResult};
+use stacks::chainstate::stacks::{StacksTransaction, TransactionPayload};
use stacks::core::{
StacksEpoch, StacksEpochId, BLOCK_LIMIT_MAINNET_10, HELIUM_BLOCK_LIMIT_20,
PEER_VERSION_EPOCH_1_0, PEER_VERSION_EPOCH_2_0, PEER_VERSION_EPOCH_2_05,
@@ -33,21 +38,22 @@ use stacks::core::{
PEER_VERSION_EPOCH_2_5, PEER_VERSION_EPOCH_3_0,
};
use stacks_common::address::AddressHashMode;
+use stacks_common::codec::StacksMessageCodec;
use stacks_common::consts::STACKS_EPOCH_MAX;
-use stacks_common::types::chainstate::{StacksAddress, StacksPublicKey};
+use stacks_common::types::chainstate::{StacksAddress, StacksPrivateKey, StacksPublicKey};
use stacks_common::util::hash::to_hex;
use stacks_common::util::secp256k1::Secp256k1PrivateKey;
use super::bitcoin_regtest::BitcoinCoreController;
-use crate::config::{EventKeyType, EventObserverConfig};
+use crate::config::{EventKeyType, EventObserverConfig, InitialBalance};
use crate::mockamoto::signer::SelfSigner;
use crate::neon::{Counters, RunLoopCounter};
use crate::run_loop::boot_nakamoto;
-use crate::tests::make_stacks_transfer;
use crate::tests::neon_integrations::{
get_account, get_pox_info, next_block_and_wait, run_until_burnchain_height, submit_tx,
test_observer, wait_for_runloop,
};
+use crate::tests::{make_stacks_transfer, to_addr};
use crate::{tests, BitcoinRegtestController, BurnchainController, Config, ConfigFile, Keychain};
static POX_4_DEFAULT_STACKER_BALANCE: u64 = 100_000_000_000_000;
@@ -121,9 +127,27 @@ lazy_static! {
];
}
+pub fn add_initial_balances(conf: &mut Config, accounts: usize) -> Vec {
+ (0..accounts)
+ .map(|i| {
+ let privk = StacksPrivateKey::from_seed(&[5, 5, 5, i as u8]);
+
+ conf.initial_balances.push(InitialBalance {
+ address: to_addr(&privk).into(),
+ amount: 100000,
+ });
+ privk
+ })
+ .collect()
+}
+
/// Return a working nakamoto-neon config and the miner's bitcoin address to fund
-pub fn naka_neon_integration_conf(seed: Option<&[u8]>) -> (Config, StacksAddress) {
+pub fn naka_neon_integration_conf(
+ seed: Option<&[u8]>,
+) -> (Config, StacksAddress, Vec) {
let mut conf = super::new_test_conf();
+ let account_keys = add_initial_balances(&mut conf, 10);
+
conf.burnchain.mode = "nakamoto-neon".into();
// tests can override this, but these tests run with epoch 2.05 by default
@@ -138,7 +162,7 @@ pub fn naka_neon_integration_conf(seed: Option<&[u8]>) -> (Config, StacksAddress
let mining_key = Secp256k1PrivateKey::from_seed(&[1]);
conf.miner.mining_key = Some(mining_key);
- conf.miner.self_signing_key = Some(SelfSigner::single_signer());
+ conf.miner.self_signing_key = Some(SelfSigner::from_seed(7));
conf.node.miner = true;
conf.node.wait_time_for_microblocks = 500;
@@ -180,7 +204,7 @@ pub fn naka_neon_integration_conf(seed: Option<&[u8]>) -> (Config, StacksAddress
conf.burnchain.pox_prepare_length = Some(5);
conf.burnchain.pox_reward_length = Some(20);
- (conf, miner_account)
+ (conf, miner_account, account_keys)
}
pub fn next_block_and(
@@ -371,7 +395,7 @@ fn simple_neon_integration() {
return;
}
- let (mut naka_conf, _miner_account) = naka_neon_integration_conf(None);
+ let (mut naka_conf, _miner_account, _account_keys) = naka_neon_integration_conf(None);
let prom_bind = format!("{}:{}", "127.0.0.1", 6000);
naka_conf.node.prometheus_bind = Some(prom_bind.clone());
naka_conf.miner.wait_on_interim_blocks = Duration::from_secs(1000);
@@ -586,7 +610,7 @@ fn mine_multiple_per_tenure_integration() {
return;
}
- let (mut naka_conf, _miner_account) = naka_neon_integration_conf(None);
+ let (mut naka_conf, _miner_account, _account_keys) = naka_neon_integration_conf(None);
let http_origin = format!("http://{}", &naka_conf.node.rpc_bind);
naka_conf.miner.wait_on_interim_blocks = Duration::from_secs(1);
let sender_sk = Secp256k1PrivateKey::new();
@@ -946,3 +970,234 @@ fn correct_burn_outs() {
run_loop_thread.join().unwrap();
}
+
+/// Test `/v2/block_proposal` API endpoint
+///
+/// This endpoint allows miners to propose Nakamoto blocks to a node,
+/// and test if they would be accepted or rejected
+#[test]
+#[ignore]
+fn block_proposal_api_endpoint() {
+ if env::var("BITCOIND_TEST") != Ok("1".into()) {
+ return;
+ }
+
+ let (mut conf, _miner_account, account_keys) = naka_neon_integration_conf(None);
+ let sender_sk = Secp256k1PrivateKey::new();
+ // setup sender + recipient for a test stx transfer
+ let sender_addr = tests::to_addr(&sender_sk);
+ let send_amt = 1000;
+ let send_fee = 100;
+ conf.add_initial_balance(
+ PrincipalData::from(sender_addr.clone()).to_string(),
+ send_amt + send_fee,
+ );
+ let _recipient = PrincipalData::from(StacksAddress::burn_address(false));
+ let stacker_sk = setup_stacker(&mut conf);
+
+ test_observer::spawn();
+ let observer_port = test_observer::EVENT_OBSERVER_PORT;
+ conf.events_observers.insert(EventObserverConfig {
+ endpoint: format!("localhost:{observer_port}"),
+ events_keys: vec![EventKeyType::AnyEvent],
+ });
+
+ let mut btcd_controller = BitcoinCoreController::new(conf.clone());
+ btcd_controller
+ .start_bitcoind()
+ .expect("Failed starting bitcoind");
+ let mut btc_regtest_controller = BitcoinRegtestController::new(conf.clone(), None);
+ btc_regtest_controller.bootstrap_chain(201);
+
+ let mut run_loop = boot_nakamoto::BootRunLoop::new(conf.clone()).unwrap();
+ let run_loop_stopper = run_loop.get_termination_switch();
+ let Counters {
+ blocks_processed,
+ naka_submitted_vrfs: vrfs_submitted,
+ naka_submitted_commits: commits_submitted,
+ ..
+ } = run_loop.counters();
+
+ let coord_channel = run_loop.coordinator_channels();
+
+ let run_loop_thread = thread::spawn(move || run_loop.start(None, 0));
+ wait_for_runloop(&blocks_processed);
+ boot_to_epoch_3(
+ &conf,
+ &blocks_processed,
+ stacker_sk,
+ &mut btc_regtest_controller,
+ );
+
+ info!("Bootstrapped to Epoch-3.0 boundary, starting nakamoto miner");
+
+ let burnchain = conf.get_burnchain();
+ let sortdb = burnchain.open_sortition_db(true).unwrap();
+ let (mut chainstate, _) = StacksChainState::open(
+ conf.is_mainnet(),
+ conf.burnchain.chain_id,
+ &conf.get_chainstate_path_str(),
+ None,
+ )
+ .unwrap();
+
+ let _block_height_pre_3_0 =
+ NakamotoChainState::get_canonical_block_header(chainstate.db(), &sortdb)
+ .unwrap()
+ .unwrap()
+ .stacks_block_height;
+
+ info!("Nakamoto miner started...");
+
+ // first block wakes up the run loop, wait until a key registration has been submitted.
+ next_block_and(&mut btc_regtest_controller, 60, || {
+ let vrf_count = vrfs_submitted.load(Ordering::SeqCst);
+ Ok(vrf_count >= 1)
+ })
+ .unwrap();
+
+ // second block should confirm the VRF register, wait until a block commit is submitted
+ next_block_and(&mut btc_regtest_controller, 60, || {
+ let commits_count = commits_submitted.load(Ordering::SeqCst);
+ Ok(commits_count >= 1)
+ })
+ .unwrap();
+
+ // Mine 15 nakamoto tenures
+ for _ in 0..15 {
+ next_block_and_mine_commit(
+ &mut btc_regtest_controller,
+ 60,
+ &coord_channel,
+ &commits_submitted,
+ )
+ .unwrap();
+ }
+
+ // TODO (hack) instantiate the sortdb in the burnchain
+ _ = btc_regtest_controller.sortdb_mut();
+
+ // Set up test signer
+ let signer = conf.miner.self_signing_key.as_mut().unwrap();
+
+ // ----- Setup boilerplate finished, test block proposal API endpoint -----
+
+ let tip = NakamotoChainState::get_canonical_block_header(chainstate.db(), &sortdb)
+ .unwrap()
+ .unwrap();
+
+ let privk = conf.miner.mining_key.unwrap().clone();
+ let parent_block_id = tip.index_block_hash();
+ // TODO
+ let total_burn = 640000;
+ let tenure_change = None;
+ let coinbase = None;
+
+ let tenure_cause = tenure_change.and_then(|tx: &StacksTransaction| match &tx.payload {
+ TransactionPayload::TenureChange(tc) => Some(tc.cause),
+ _ => None,
+ });
+
+ // Apply both miner/stacker signatures
+ let mut sign = |mut p: NakamotoBlockProposal| {
+ p.block
+ .header
+ .sign_miner(&privk)
+ .expect("Miner failed to sign");
+ signer.sign_nakamoto_block(&mut p.block);
+ p
+ };
+
+ // Put block builder in code block so any database locks expire at the end
+ let block = {
+ let mut builder = NakamotoBlockBuilder::new(
+ &tip,
+ &tip.consensus_hash,
+ total_burn,
+ tenure_change,
+ coinbase,
+ )
+ .expect("Failed to build Nakamoto block");
+
+ let burn_dbconn = btc_regtest_controller.sortdb_ref().index_conn();
+ let mut miner_tenure_info = builder
+ .load_tenure_info(&mut chainstate, &burn_dbconn, tenure_cause)
+ .unwrap();
+ let mut tenure_tx = builder
+ .tenure_begin(&burn_dbconn, &mut miner_tenure_info)
+ .unwrap();
+
+ let tx = make_stacks_transfer(
+ &account_keys[0],
+ 0,
+ 100,
+ &to_addr(&account_keys[1]).into(),
+ 10000,
+ );
+ let tx = StacksTransaction::consensus_deserialize(&mut &tx[..])
+ .expect("Failed to deserialize transaction");
+ let tx_len = tx.tx_len();
+
+ let res = builder.try_mine_tx_with_len(
+ &mut tenure_tx,
+ &tx,
+ tx_len,
+ &BlockLimitFunction::NO_LIMIT_HIT,
+ ASTRules::PrecheckSize,
+ );
+ assert!(
+ matches!(res, TransactionResult::Success(..)),
+ "Transaction failed"
+ );
+ builder.mine_nakamoto_block(&mut tenure_tx)
+ };
+
+ // Construct a valid proposal. Make alterations to this to test failure cases
+ let proposal = NakamotoBlockProposal {
+ block,
+ tenure_start_block: parent_block_id,
+ chain_id: chainstate.chain_id,
+ total_burn,
+ };
+
+ const HTTP_ACCEPTED: u16 = 202;
+ const HTTP_BADREQUEST: u16 = 400;
+ let test_cases = [
+ ("No signature", proposal.clone(), HTTP_BADREQUEST),
+ ("Signed", sign(proposal.clone()), HTTP_ACCEPTED),
+ ];
+
+ // Build HTTP client
+ let client = reqwest::blocking::Client::builder()
+ .timeout(Duration::from_secs(60))
+ .build()
+ .expect("Failed to build reqwest::Client");
+ // Build URL
+ let http_origin = format!("http://{}", &conf.node.rpc_bind);
+ let path = format!("{http_origin}/v2/block_proposal");
+
+ for (test_description, block_proposal, expected_response) in test_cases {
+ eprintln!("test_block_proposal(): {test_description}");
+ eprintln!("{block_proposal:?}");
+
+ // Send POST request
+ let response = client
+ .post(&path)
+ .header("Content-Type", "application/json")
+ .json(&block_proposal)
+ .send()
+ .expect("Failed to POST");
+
+ eprintln!("{response:?}");
+ assert_eq!(response.status().as_u16(), expected_response);
+ }
+
+ // Clean up
+ coord_channel
+ .lock()
+ .expect("Mutex poisoned")
+ .stop_chains_coordinator();
+ run_loop_stopper.store(false, Ordering::SeqCst);
+
+ run_loop_thread.join().unwrap();
+}
diff --git a/testnet/stacks-node/src/tests/neon_integrations.rs b/testnet/stacks-node/src/tests/neon_integrations.rs
index f5e6d91fb..b9de6d1d6 100644
--- a/testnet/stacks-node/src/tests/neon_integrations.rs
+++ b/testnet/stacks-node/src/tests/neon_integrations.rs
@@ -66,14 +66,13 @@ use super::bitcoin_regtest::BitcoinCoreController;
use super::{
make_contract_call, make_contract_publish, make_contract_publish_microblock_only,
make_microblock, make_stacks_transfer, make_stacks_transfer_mblock_only, to_addr, ADDR_4, SK_1,
- SK_2,
+ SK_2, SK_3,
};
use crate::burnchains::bitcoin_regtest_controller::{self, BitcoinRPCRequest, UTXO};
use crate::config::{EventKeyType, EventObserverConfig, FeeEstimatorName, InitialBalance};
use crate::operations::BurnchainOpSigner;
use crate::stacks_common::types::PrivateKey;
use crate::syncctl::PoxSyncWatchdogComms;
-use crate::tests::SK_3;
use crate::util::hash::{MerkleTree, Sha512Trunc256Sum};
use crate::util::secp256k1::MessageSignature;
use crate::{neon, BitcoinRegtestController, BurnchainController, Config, ConfigFile, Keychain};
@@ -742,39 +741,24 @@ pub fn get_block(http_origin: &str, block_id: &StacksBlockId) -> Option RPCPeerInfoData {
+pub fn get_chain_info_result(conf: &Config) -> Result {
let http_origin = format!("http://{}", &conf.node.rpc_bind);
let client = reqwest::blocking::Client::new();
// get the canonical chain tip
- let path = format!("{}/v2/info", &http_origin);
- let tip_info = client
- .get(&path)
- .send()
- .unwrap()
- .json::()
- .unwrap();
-
- tip_info
+ let path = format!("{http_origin}/v2/info");
+ client.get(&path).send().unwrap().json::()
}
pub fn get_chain_info_opt(conf: &Config) -> Option {
- let http_origin = format!("http://{}", &conf.node.rpc_bind);
- let client = reqwest::blocking::Client::new();
-
- // get the canonical chain tip
- let path = format!("{}/v2/info", &http_origin);
- let tip_info_opt = client
- .get(&path)
- .send()
- .unwrap()
- .json::()
- .ok();
-
- tip_info_opt
+ get_chain_info_result(conf).ok()
}
-fn get_tip_anchored_block(conf: &Config) -> (ConsensusHash, StacksBlock) {
+pub fn get_chain_info(conf: &Config) -> RPCPeerInfoData {
+ get_chain_info_result(conf).unwrap()
+}
+
+pub fn get_tip_anchored_block(conf: &Config) -> (ConsensusHash, StacksBlock) {
let tip_info = get_chain_info(conf);
// get the canonical chain tip