explorer: introducing explorer service

This commit introduces the ExplorerService, a dedicated service layer for the Explorer application that acts as a bridge between the JSON-RPC and database layers. It encapsulates application logic, orchestrates operations, and transforms database data into structured responses suitable for RPC callers, as well as other use cases.

As the Explorer application is growing in complexity, we’ve observed that blending RPC functionality with the database layer (ExplorerDb) introduces functionality into these layers that is not cohesively aligned. To address this, ExplorerService refines the application’s architecture by clearly defining and organizing application-specific responsibilities. This ensures that transformations, operation orchestration, contract source code processing, gas calculations, and other operations are performed within their appropriate layers, fostering a more cohesive and maintainable architecture over time.
This commit is contained in:
kalm
2024-12-23 05:53:26 -08:00
parent 0f67f19701
commit 95244396e4
7 changed files with 313 additions and 284 deletions

View File

@@ -29,7 +29,7 @@ use darkfi::{
};
use darkfi_sdk::crypto::schnorr::Signature;
use crate::ExplorerDb;
use crate::ExplorerService;
#[derive(Debug, Clone)]
/// Structure representing a block record.
@@ -83,10 +83,10 @@ impl From<&BlockInfo> for BlockRecord {
}
}
impl ExplorerDb {
impl ExplorerService {
/// Resets blocks in the database by clearing all block related trees, returning an Ok result on success.
pub fn reset_blocks(&self) -> Result<()> {
let db = &self.blockchain.sled_db;
let db = &self.db.blockchain.sled_db;
// Initialize block related trees to reset
let trees_to_reset = [SLED_BLOCK_TREE, SLED_BLOCK_ORDER_TREE, SLED_BLOCK_DIFFICULTY_TREE];
@@ -108,7 +108,7 @@ impl ExplorerDb {
/// PoW rewards. After processing all transactions, the block is permanently persisted to
/// the explorer database.
pub async fn put_block(&self, block: &BlockInfo) -> Result<()> {
let blockchain_overlay = BlockchainOverlay::new(&self.blockchain)?;
let blockchain_overlay = BlockchainOverlay::new(&self.db.blockchain)?;
// Initialize collections to hold gas data and transactions that have gas data
let mut tx_gas_data = Vec::with_capacity(block.txs.len());
@@ -124,7 +124,7 @@ impl ExplorerDb {
// If the block contains transaction gas data, insert the gas metrics into the metrics store
if !tx_gas_data.is_empty() {
self.metrics_store.insert_gas_metrics(
self.db.metrics_store.insert_gas_metrics(
block.header.height,
&block.header.timestamp,
&txs_hashes_with_gas_data,
@@ -142,13 +142,13 @@ impl ExplorerDb {
/// Provides the total block count.
pub fn get_block_count(&self) -> usize {
self.blockchain.len()
self.db.blockchain.len()
}
/// Fetch all known blocks from the database.
pub fn get_blocks(&self) -> Result<Vec<BlockRecord>> {
// Fetch blocks and handle any errors encountered
let blocks = &self.blockchain.get_all().map_err(|e| {
let blocks = &self.db.blockchain.get_all().map_err(|e| {
Error::DatabaseError(format!("[get_blocks] Block retrieval failed: {e:?}"))
})?;
@@ -166,7 +166,7 @@ impl ExplorerDb {
.map_err(|_| Error::ParseFailed("[get_block_by_hash] Invalid header hash"))?;
// Fetch block by hash and handle encountered errors
match self.blockchain.get_blocks_by_hash(&[header_hash]) {
match self.db.blockchain.get_blocks_by_hash(&[header_hash]) {
Ok(blocks) => Ok(Some(BlockRecord::from(&blocks[0]))),
Err(Error::BlockNotFound(_)) => Ok(None),
Err(e) => Err(Error::DatabaseError(format!(
@@ -177,7 +177,7 @@ impl ExplorerDb {
/// Fetch the last block from the database.
pub fn last_block(&self) -> Result<Option<(u32, String)>> {
let block_store = &self.blockchain.blocks;
let block_store = &self.db.blockchain.blocks;
// Return None result when no blocks exist
if block_store.is_empty() {
@@ -196,7 +196,7 @@ impl ExplorerDb {
/// Fetch the last N blocks from the database.
pub fn get_last_n(&self, n: usize) -> Result<Vec<BlockRecord>> {
// Fetch the last n blocks and handle any errors encountered
let blocks_result = &self.blockchain.get_last_n(n).map_err(|e| {
let blocks_result = &self.db.blockchain.get_last_n(n).map_err(|e| {
Error::DatabaseError(format!("[get_last_n] Block retrieval failed: {e:?}"))
})?;
@@ -209,7 +209,7 @@ impl ExplorerDb {
/// Fetch blocks within a specified range from the database.
pub fn get_by_range(&self, start: u32, end: u32) -> Result<Vec<BlockRecord>> {
// Fetch blocks in the specified range and handle any errors encountered
let blocks_result = &self.blockchain.get_by_range(start, end).map_err(|e| {
let blocks_result = &self.db.blockchain.get_by_range(start, end).map_err(|e| {
Error::DatabaseError(format!("[get_by_range]: Block retrieval failed: {e:?}"))
})?;

View File

@@ -16,14 +16,12 @@
* along with this program. If not, see <https://www.gnu.org/licenses/>.
*/
use std::{
collections::{HashMap, HashSet},
sync::Arc,
};
use std::{collections::HashSet, sync::Arc};
use log::{debug, error, info};
use log::{error, info};
use rpc_blocks::subscribe_blocks;
use sled_overlay::sled;
use smol::{io::Cursor, lock::Mutex, stream::StreamExt};
use smol::{lock::Mutex, stream::StreamExt};
use structopt_toml::{serde::Deserialize, structopt::StructOpt, StructOptToml};
use url::Url;
@@ -31,30 +29,17 @@ use darkfi::{
async_daemonize,
blockchain::{Blockchain, BlockchainOverlay},
cli_desc,
error::TxVerifyFailed,
rpc::{
client::RpcClient,
server::{listen_and_serve, RequestHandler},
},
runtime::vm_runtime::Runtime,
system::{StoppableTask, StoppableTaskPtr},
tx::Transaction,
util::path::expand_path,
validator::{
fees::{circuit_gas_use, GasData, PALLAS_SCHNORR_SIGNATURE_FEE},
utils::deploy_native_contracts,
},
zk::VerifyingKey,
validator::utils::deploy_native_contracts,
Error, Result,
};
use darkfi_sdk::{
crypto::{ContractId, PublicKey},
deploy::DeployParamsV1,
pasta::pallas,
};
use darkfi_serial::{deserialize_async, serialize_async, AsyncDecodable, AsyncEncodable};
use crate::metrics_store::{GasMetrics, GasMetricsKey, MetricsStore};
use crate::metrics_store::MetricsStore;
/// Crate errors
mod error;
@@ -62,7 +47,6 @@ mod error;
/// JSON-RPC requests handler and methods
mod rpc;
mod rpc_blocks;
use rpc_blocks::subscribe_blocks;
mod rpc_statistics;
mod rpc_transactions;
@@ -117,227 +101,75 @@ struct Args {
verbose: u8,
}
/// Structure represents the explorer database backed by a sled DB connection.
/// Represents the service layer for the Explorer application, bridging the RPC layer and the database.
/// It encapsulates explorer business logic and provides a unified interface for core functionalities,
/// providing a clear separation of concerns between RPC handling and data management layers.
///
/// Core functionalities include:
///
/// - Data Transformation: Converting database data into structured responses suitable for RPC callers.
/// - Blocks: Synchronization, retrieval, counting, and management.
/// - Metrics: Providing metric-related data over the life of the chain.
/// - Transactions: Synchronization, calculating gas data, retrieval, counting, and related block information.
pub struct ExplorerService {
/// Explorer database instance
db: ExplorerDb,
}
impl ExplorerService {
/// Creates a new `ExplorerService` instance
///
/// The function sets up a new explorer database using the given [`String`] `db_path`, deploying
/// native contracts needed for calculating transaction gas data.
async fn new(db_path: String) -> Result<Self> {
// Initialize explorer database
let db = ExplorerDb::new(db_path)?;
// Deploy native contracts needed to calculated transaction gas data and commit changes
let overlay = BlockchainOverlay::new(&db.blockchain)?;
deploy_native_contracts(&overlay, 10).await?;
overlay.lock().unwrap().overlay.lock().unwrap().apply()?;
Ok(Self { db })
}
}
/// Represents the explorer database backed by a `sled` database connection, responsible for maintaining
/// persistent state required for blockchain exploration. It serves as the core data layer for the Explorer application,
/// storing and managing blockchain data, metrics, and contract-related information.
pub struct ExplorerDb {
/// Main pointer to the sled db connection
/// The main `sled` database connection used for data storage and retrieval
pub sled_db: sled::Db,
/// Explorer darkfid blockchain copy
/// Local copy of the Darkfi blockchain used for block synchronization and exploration
pub blockchain: Blockchain,
/// Metrics store instance
/// Store for tracking chain-related metrics
pub metrics_store: MetricsStore,
}
impl ExplorerDb {
/// Creates a new `BlockExplorerDb` instance
pub fn new(db_path: String) -> Result<ExplorerDb> {
/// Creates a new `ExplorerDb` instance
pub fn new(db_path: String) -> Result<Self> {
let db_path = expand_path(db_path.as_str())?;
let sled_db = sled::open(&db_path)?;
let blockchain = Blockchain::new(&sled_db)?;
let metrics_store = MetricsStore::new(&sled_db)?;
info!(target: "blockchain-explorer", "Initialized explorer database {}, block count: {}", db_path.display(), blockchain.len());
Ok(ExplorerDb { sled_db, blockchain, metrics_store })
}
/// Calculates the fee data for a given transaction, returning a [`GasData`] object detailing various aspects of the gas usage.
pub async fn calculate_tx_gas_data(
&self,
tx: &Transaction,
verify_fee: bool,
) -> Result<GasData> {
let tx_hash = tx.hash();
let overlay = BlockchainOverlay::new(&self.blockchain)?;
// Gas accumulators
let mut total_gas_used = 0;
let mut zk_circuit_gas_used = 0;
let mut wasm_gas_used = 0;
let mut deploy_gas_used = 0;
let mut gas_paid = 0;
// Table of public inputs used for ZK proof verification
let mut zkp_table = vec![];
// Table of public keys used for signature verification
let mut sig_table = vec![];
// Index of the Fee-paying call
let fee_call_idx = 0;
// Map of ZK proof verifying keys for the transaction
let mut verifying_keys: HashMap<[u8; 32], HashMap<String, VerifyingKey>> = HashMap::new();
for call in &tx.calls {
verifying_keys.insert(call.data.contract_id.to_bytes(), HashMap::new());
}
let block_target = self.blockchain.blocks.get_last()?.0 + 1;
// We'll also take note of all the circuits in a Vec so we can calculate their verification cost.
let mut circuits_to_verify = vec![];
// Iterate over all calls to get the metadata
for (idx, call) in tx.calls.iter().enumerate() {
// Transaction must not contain a Money::PoWReward(0x02) call
if call.data.is_money_pow_reward() {
error!(target: "block_explorer::calculate_tx_gas_data", "Reward transaction detected");
return Err(TxVerifyFailed::ErroneousTxs(vec![tx.clone()]).into())
}
// Write the actual payload data
let mut payload = vec![];
tx.calls.encode_async(&mut payload).await?;
let wasm = overlay.lock().unwrap().contracts.get(call.data.contract_id)?;
let mut runtime = Runtime::new(
&wasm,
overlay.clone(),
call.data.contract_id,
block_target,
block_target,
tx_hash,
idx as u8,
)?;
let metadata = runtime.metadata(&payload)?;
// Decode the metadata retrieved from the execution
let mut decoder = Cursor::new(&metadata);
// The tuple is (zkas_ns, public_inputs)
let zkp_pub: Vec<(String, Vec<pallas::Base>)> =
AsyncDecodable::decode_async(&mut decoder).await?;
let sig_pub: Vec<PublicKey> = AsyncDecodable::decode_async(&mut decoder).await?;
if decoder.position() != metadata.len() as u64 {
error!(
target: "block_explorer::calculate_tx_gas_data",
"[BLOCK_EXPLORER] Failed decoding entire metadata buffer for {}:{}", tx_hash, idx,
);
return Err(TxVerifyFailed::ErroneousTxs(vec![tx.clone()]).into())
}
// Here we'll look up verifying keys and insert them into the per-contract map.
for (zkas_ns, _) in &zkp_pub {
let inner_vk_map =
verifying_keys.get_mut(&call.data.contract_id.to_bytes()).unwrap();
// TODO: This will be a problem in case of ::deploy, unless we force a different
// namespace and disable updating existing circuit. Might be a smart idea to do
// so in order to have to care less about being able to verify historical txs.
if inner_vk_map.contains_key(zkas_ns.as_str()) {
continue
}
let (zkbin, vk) =
overlay.lock().unwrap().contracts.get_zkas(&call.data.contract_id, zkas_ns)?;
inner_vk_map.insert(zkas_ns.to_string(), vk);
circuits_to_verify.push(zkbin);
}
zkp_table.push(zkp_pub);
sig_table.push(sig_pub);
// Contracts are not included within blocks. They need to be deployed off-chain so that they can be accessed and utilized for fee data computation
if call.data.is_deployment()
/* DeployV1 */
{
// Deserialize the deployment parameters
let deploy_params: DeployParamsV1 = deserialize_async(&call.data.data[1..]).await?;
let deploy_cid = ContractId::derive_public(deploy_params.public_key);
// Instantiate the new deployment runtime
let mut deploy_runtime = Runtime::new(
&deploy_params.wasm_bincode,
overlay.clone(),
deploy_cid,
block_target,
block_target,
tx_hash,
idx as u8,
)?;
deploy_runtime.deploy(&deploy_params.ix)?;
deploy_gas_used = deploy_runtime.gas_used();
// Append the used deployment gas
total_gas_used += deploy_gas_used;
}
// At this point we're done with the call and move on to the next one.
// Accumulate the WASM gas used.
wasm_gas_used = runtime.gas_used();
// Append the used wasm gas
total_gas_used += wasm_gas_used;
}
// The signature fee is tx_size + fixed_sig_fee * n_signatures
let signature_gas_used = (PALLAS_SCHNORR_SIGNATURE_FEE * tx.signatures.len() as u64) +
serialize_async(tx).await.len() as u64;
// Append the used signature gas
total_gas_used += signature_gas_used;
// The ZK circuit fee is calculated using a function in validator/fees.rs
for zkbin in circuits_to_verify.iter() {
zk_circuit_gas_used = circuit_gas_use(zkbin);
// Append the used zk circuit gas
total_gas_used += zk_circuit_gas_used;
}
if verify_fee {
// Deserialize the fee call to find the paid fee
let fee: u64 = match deserialize_async(&tx.calls[fee_call_idx].data.data[1..9]).await {
Ok(v) => v,
Err(e) => {
error!(
target: "block_explorer::calculate_tx_gas_data",
"[VALIDATOR] Failed deserializing tx {} fee call: {}", tx_hash, e,
);
return Err(TxVerifyFailed::InvalidFee.into())
}
};
// TODO: This counts 1 gas as 1 token unit. Pricing should be better specified.
// Check that enough fee has been paid for the used gas in this transaction.
if total_gas_used > fee {
error!(
target: "block_explorer::calculate_tx_gas_data",
"[VALIDATOR] Transaction {} has insufficient fee. Required: {}, Paid: {}",
tx_hash, total_gas_used, fee,
);
return Err(TxVerifyFailed::InsufficientFee.into())
}
debug!(target: "block_explorer::calculate_tx_gas_data", "The gas paid for transaction {}: {}", tx_hash, gas_paid);
// Store paid fee
gas_paid = fee;
}
// Commit changes made to the overlay
overlay.lock().unwrap().overlay.lock().unwrap().apply()?;
let fee_data = GasData {
paid: gas_paid,
wasm: wasm_gas_used,
zk_circuits: zk_circuit_gas_used,
signatures: signature_gas_used,
deployments: deploy_gas_used,
};
debug!(target: "block_explorer::calculate_tx_gas_data", "The total gas usage for transaction {}: {:?}", tx_hash, fee_data);
Ok(fee_data)
Ok(Self { sled_db, blockchain, metrics_store })
}
}
/// Daemon structure
/// Defines a daemon structure responsible for handling incoming JSON-RPC requests and delegating them
/// to the backend layer for processing. It provides a JSON-RPC interface for managing operations related to
/// blocks, transactions, and metrics.
///
/// Upon startup, the daemon initializes a background task to handle incoming JSON-RPC requests.
/// This includes processing operations related to blocks, transactions, and metrics by
/// delegating them to the backend and returning appropriate RPC responses. Additionally, the daemon
/// synchronizes blocks from the `darkfid` daemon into the explorer database and subscribes
/// to new blocks, ensuring that the local database remains updated in real-time.
pub struct Explorerd {
/// Explorer database instance
pub db: ExplorerDb,
/// Explorer service instance
pub service: ExplorerService,
/// JSON-RPC connection tracker
pub rpc_connections: Mutex<HashSet<StoppableTaskPtr>>,
/// JSON-RPC client to execute requests to darkfid daemon
@@ -351,21 +183,10 @@ impl Explorerd {
let rpc_client = RpcClient::new(endpoint.clone(), ex).await?;
info!(target: "blockchain-explorer", "Created rpc client: {:?}", endpoint);
// Initialize explorer database
let explorer_db = ExplorerDb::new(db_path)?;
// Initialize explorer service
let service = ExplorerService::new(db_path).await?;
// Deploy native contracts need to calculated transaction gas data and commit changes
let overlay = BlockchainOverlay::new(&explorer_db.blockchain)?;
deploy_native_contracts(&overlay, 10).await?;
overlay.lock().unwrap().overlay.lock().unwrap().apply()?;
Ok(Self { rpc_connections: Mutex::new(HashSet::new()), rpc_client, db: explorer_db })
}
/// Fetches the most current metrics from the [`MetricsStore`], returning an `Option` containing
/// a pair of [`GasMetricsKey`] and [`GasMetrics`] upon success, or `None` if no metrics are found.
pub fn get_latest_metrics(&self) -> Result<Option<(GasMetricsKey, GasMetrics)>> {
self.db.metrics_store.get_last()
Ok(Self { rpc_connections: Mutex::new(HashSet::new()), rpc_client, service })
}
}

View File

@@ -58,7 +58,7 @@ impl Explorerd {
/// If reset flag is provided, all tables are reset, and start syncing from beginning.
pub async fn sync_blocks(&self, reset: bool) -> Result<()> {
// Grab last synced block height
let mut height = match self.db.last_block() {
let mut height = match self.service.last_block() {
Ok(Some((height, _))) => height,
Ok(None) => 0,
Err(e) => {
@@ -72,7 +72,7 @@ impl Explorerd {
// has been provided we reset, otherwise continue with
// the next block height
if height == 0 || reset {
self.db.reset_blocks()?;
self.service.reset_blocks()?;
height = 0;
} else {
height += 1;
@@ -101,7 +101,7 @@ impl Explorerd {
}
};
if let Err(e) = self.db.put_block(&block).await {
if let Err(e) = self.service.put_block(&block).await {
let error_message = format!("[sync_blocks] Put block failed: {:?}", e);
error!(target: "blockchain-explorer::rpc_blocks::sync_blocks", "{}", error_message);
return Err(Error::DatabaseError(error_message));
@@ -139,7 +139,7 @@ impl Explorerd {
};
// Fetch the blocks and handle potential errors
let blocks_result = match self.db.get_last_n(n) {
let blocks_result = match self.service.get_last_n(n) {
Ok(blocks) => blocks,
Err(e) => {
error!(target: "blockchain-explorer::rpc_blocks::blocks_get_last_n_blocks", "Failed fetching blocks: {}", e);
@@ -195,7 +195,7 @@ impl Explorerd {
}
// Fetch the blocks and handle potential errors
let blocks_result = match self.db.get_by_range(start, end) {
let blocks_result = match self.service.get_by_range(start, end) {
Ok(blocks) => blocks,
Err(e) => {
error!(target: "blockchain-explorer::rpc_blocks::blocks_get_blocks_in_height_range", "Failed fetching blocks: {}", e);
@@ -238,7 +238,7 @@ impl Explorerd {
};
// Fetch and transform block to json, handling any errors and returning the result
match self.db.get_block_by_hash(header_hash) {
match self.service.get_block_by_hash(header_hash) {
Ok(Some(block)) => JsonResponse::new(block.to_json_array(), id).into(),
Ok(None) => JsonResponse::new(JsonValue::Array(vec![]), id).into(),
Err(e) => {
@@ -272,7 +272,7 @@ pub async fn subscribe_blocks(
let (last_confirmed, _) = explorer.get_last_confirmed_block().await?;
// Grab last synced block
let last_synced = match explorer.db.last_block() {
let last_synced = match explorer.service.last_block() {
Ok(Some((height, _))) => height,
Ok(None) => 0,
Err(e) => {
@@ -361,7 +361,7 @@ pub async fn subscribe_blocks(
info!(target: "blockchain-explorer::rpc_blocks::subscribe_blocks", "=======================================");
info!(target: "blockchain-explorer::rpc_blocks::subscribe_blocks", "Deserialized successfully. Storing block...");
if let Err(e) = explorer.db.put_block(&block_data).await {
if let Err(e) = explorer.service.put_block(&block_data).await {
return Err(Error::DatabaseError(format!(
"[subscribe_blocks] Put block failed: {e:?}"
)))

View File

@@ -49,7 +49,7 @@ impl Explorerd {
}
// Fetch `BaseStatistics`, transform to `JsonResult`, and return results
match self.db.get_base_statistics() {
match self.service.get_base_statistics() {
Ok(Some(statistics)) => JsonResponse::new(statistics.to_json_array(), id).into(),
Ok(None) => JsonResponse::new(JsonValue::Array(vec![]), id).into(),
Err(e) => {
@@ -81,7 +81,7 @@ impl Explorerd {
return JsonError::new(InvalidParams, None, id).into()
}
// Fetch metric statistics and return results
let metrics = match self.db.get_metrics_statistics().await {
let metrics = match self.service.get_metrics_statistics().await {
Ok(v) => v,
Err(e) => {
error!(target: "blockchain-explorer::rpc_statistics::statistics_get_metric_statistics", "Failed fetching metric statistics: {}", e);
@@ -117,7 +117,7 @@ impl Explorerd {
return JsonError::new(InvalidParams, None, id).into()
}
// Fetch metric statistics and return results
let metrics = match self.db.get_latest_metrics_statistics().await {
let metrics = match self.service.get_latest_metrics_statistics().await {
Ok(v) => v,
Err(e) => {
error!(target: "blockchain-explorer::rpc_statistics::statistics_get_latest_metric_statistics", "Failed fetching metric statistics: {}", e);

View File

@@ -51,7 +51,7 @@ impl Explorerd {
}
let header_hash = params[0].get::<String>().unwrap();
let transactions = match self.db.get_transactions_by_header_hash(header_hash) {
let transactions = match self.service.get_transactions_by_header_hash(header_hash) {
Ok(v) => v,
Err(e) => {
error!(target: "blockchain-explorer::rpc_transactions::transactions_get_transaction_by_header_hash", "Failed fetching block transactions: {}", e);
@@ -96,7 +96,7 @@ impl Explorerd {
};
// Retrieve transaction by hash and return result
match self.db.get_transaction_by_hash(&tx_hash) {
match self.service.get_transaction_by_hash(&tx_hash) {
Ok(Some(transaction)) => JsonResponse::new(transaction.to_json_array(), id).into(),
Ok(None) => JsonResponse::new(JsonValue::Array(vec![]), id).into(),
Err(e) => {

View File

@@ -21,7 +21,7 @@ use tinyjson::JsonValue;
use darkfi::{Error, Result};
use darkfi_sdk::blockchain::block_epoch;
use crate::{metrics_store::GasMetrics, ExplorerDb};
use crate::{metrics_store::GasMetrics, ExplorerService};
#[derive(Debug, Clone)]
/// Structure representing basic statistic extracted from the database.
@@ -85,7 +85,7 @@ impl MetricStatistics {
])
}
}
impl ExplorerDb {
impl ExplorerService {
/// Fetches the latest [`BaseStatistics`] from the explorer database, or returns `None` if no block exists.
pub fn get_base_statistics(&self) -> Result<Option<BaseStatistics>> {
let last_block = self.last_block();
@@ -110,7 +110,7 @@ impl ExplorerDb {
/// [`MetricStatistics`] if found, or an empty Vec if no metrics exist.
pub async fn get_metrics_statistics(&self) -> Result<Vec<MetricStatistics>> {
// Fetch all metrics from the metrics store, handling any potential errors
let metrics = self.metrics_store.get_all_metrics().map_err(|e| {
let metrics = self.db.metrics_store.get_all_metrics().map_err(|e| {
Error::DatabaseError(format!(
"[get_metrics_statistics] Retrieving metrics failed: {:?}",
e
@@ -128,7 +128,7 @@ impl ExplorerDb {
/// or zero-initialized defaults when not.
pub async fn get_latest_metrics_statistics(&self) -> Result<MetricStatistics> {
// Fetch the latest metrics, handling any potential errors
match self.metrics_store.get_last().map_err(|e| {
match self.db.metrics_store.get_last().map_err(|e| {
Error::DatabaseError(format!(
"[get_metrics_statistics] Retrieving latest metrics failed: {:?}",
e

View File

@@ -16,22 +16,34 @@
* along with this program. If not, see <https://www.gnu.org/licenses/>.
*/
use log::info;
use std::collections::HashMap;
use log::{debug, error, info};
use smol::io::Cursor;
use tinyjson::JsonValue;
use darkfi::{
blockchain::{
BlockInfo, HeaderHash, SLED_PENDING_TX_ORDER_TREE, SLED_PENDING_TX_TREE,
BlockInfo, BlockchainOverlay, HeaderHash, SLED_PENDING_TX_ORDER_TREE, SLED_PENDING_TX_TREE,
SLED_TX_LOCATION_TREE, SLED_TX_TREE,
},
error::TxVerifyFailed,
runtime::vm_runtime::Runtime,
tx::Transaction,
util::time::Timestamp,
validator::fees::GasData,
validator::fees::{circuit_gas_use, GasData, PALLAS_SCHNORR_SIGNATURE_FEE},
zk::VerifyingKey,
Error, Result,
};
use darkfi_sdk::tx::TransactionHash;
use darkfi_sdk::{
crypto::{ContractId, PublicKey},
deploy::DeployParamsV1,
pasta::pallas,
tx::TransactionHash,
};
use darkfi_serial::{deserialize_async, serialize_async, AsyncDecodable, AsyncEncodable};
use crate::ExplorerDb;
use crate::ExplorerService;
#[derive(Debug, Clone)]
/// Structure representing a `TRANSACTIONS_TABLE` record.
@@ -74,7 +86,7 @@ impl TransactionRecord {
}
}
impl ExplorerDb {
impl ExplorerService {
/// Resets transactions in the database by clearing transaction-related trees, returning an Ok result on success.
pub fn reset_transactions(&self) -> Result<()> {
// Initialize transaction trees to reset
@@ -83,7 +95,7 @@ impl ExplorerDb {
// Iterate over each associated transaction tree and delete its contents
for tree_name in &trees_to_reset {
let tree = &self.blockchain.sled_db.open_tree(tree_name)?;
let tree = &self.db.blockchain.sled_db.open_tree(tree_name)?;
tree.clear()?;
let tree_name_str = std::str::from_utf8(tree_name)?;
info!(target: "blockchain-explorer::blocks", "Successfully reset transaction tree: {tree_name_str}");
@@ -94,7 +106,7 @@ impl ExplorerDb {
/// Provides the transaction count of all the transactions in the explorer database.
pub fn get_transaction_count(&self) -> usize {
self.blockchain.txs_len()
self.db.blockchain.txs_len()
}
/// Fetches all known transactions from the database.
@@ -104,7 +116,7 @@ impl ExplorerDb {
/// it returns an empty vector.
pub fn get_transactions(&self) -> Result<Vec<TransactionRecord>> {
// Retrieve all transactions and handle any errors encountered
let txs = self.blockchain.transactions.get_all().map_err(|e| {
let txs = self.db.blockchain.transactions.get_all().map_err(|e| {
Error::DatabaseError(format!("[get_transactions] Trxs retrieval: {e:?}"))
})?;
@@ -134,7 +146,7 @@ impl ExplorerDb {
.map_err(|_| Error::ParseFailed("[get_transactions_by_header_hash] Invalid hash"))?;
// Fetch block by hash and handle encountered errors
let block = match self.blockchain.get_blocks_by_hash(&[header_hash]) {
let block = match self.db.blockchain.get_blocks_by_hash(&[header_hash]) {
Ok(blocks) => blocks.first().cloned().unwrap(),
Err(Error::BlockNotFound(_)) => return Ok(vec![]),
Err(e) => {
@@ -161,7 +173,7 @@ impl ExplorerDb {
&self,
tx_hash: &TransactionHash,
) -> Result<Option<TransactionRecord>> {
let tx_store = &self.blockchain.transactions;
let tx_store = &self.db.blockchain.transactions;
// Attempt to retrieve the transaction using the provided hash handling any potential errors
let tx_opt = &tx_store.get(&[*tx_hash], false).map_err(|e| {
@@ -183,7 +195,7 @@ impl ExplorerDb {
fn get_tx_block_info(&self, tx_hash: &TransactionHash) -> Result<Option<BlockInfo>> {
// Retrieve the location of the transaction
let location =
self.blockchain.transactions.get_location(&[*tx_hash], false).map_err(|e| {
self.db.blockchain.transactions.get_location(&[*tx_hash], false).map_err(|e| {
Error::DatabaseError(format!(
"[get_tx_block_info] Location retrieval failed: {e:?}"
))
@@ -193,7 +205,7 @@ impl ExplorerDb {
let header_hash = match location {
None => return Ok(None),
Some((block_height, _)) => {
self.blockchain.blocks.get_order(&[block_height], false).map_err(|e| {
self.db.blockchain.blocks.get_order(&[block_height], false).map_err(|e| {
Error::DatabaseError(format!(
"[get_tx_block_info] Block retrieval failed: {e:?}"
))
@@ -218,7 +230,7 @@ impl ExplorerDb {
/// the specified [`HeaderHash`]. It returns the associated [`BlockInfo`] if found,
/// or `None` when not found.
fn get_block_info(&self, header_hash: HeaderHash) -> Result<Option<BlockInfo>> {
match self.blockchain.get_blocks_by_hash(&[header_hash]) {
match self.db.blockchain.get_blocks_by_hash(&[header_hash]) {
Err(Error::BlockNotFound(_)) => Ok(None),
Ok(block_info) => Ok(block_info.into_iter().next()),
Err(e) => Err(Error::DatabaseError(format!(
@@ -227,6 +239,202 @@ impl ExplorerDb {
}
}
/// Calculates the gas data for a given transaction, returning a [`GasData`] instance detailing
/// various aspects of the gas usage.
pub async fn calculate_tx_gas_data(
&self,
tx: &Transaction,
verify_fee: bool,
) -> Result<GasData> {
let tx_hash = tx.hash();
let overlay = BlockchainOverlay::new(&self.db.blockchain)?;
// Gas accumulators
let mut total_gas_used = 0;
let mut zk_circuit_gas_used = 0;
let mut wasm_gas_used = 0;
let mut deploy_gas_used = 0;
let mut gas_paid = 0;
// Table of public inputs used for ZK proof verification
let mut zkp_table = vec![];
// Table of public keys used for signature verification
let mut sig_table = vec![];
// Index of the Fee-paying call
let fee_call_idx = 0;
// Map of ZK proof verifying keys for the transaction
let mut verifying_keys: HashMap<[u8; 32], HashMap<String, VerifyingKey>> = HashMap::new();
for call in &tx.calls {
verifying_keys.insert(call.data.contract_id.to_bytes(), HashMap::new());
}
let block_target = self.db.blockchain.blocks.get_last()?.0 + 1;
// We'll also take note of all the circuits in a Vec so we can calculate their verification cost.
let mut circuits_to_verify = vec![];
// Iterate over all calls to get the metadata
for (idx, call) in tx.calls.iter().enumerate() {
// Transaction must not contain a Money::PoWReward(0x02) call
if call.data.is_money_pow_reward() {
error!(target: "block_explorer::calculate_tx_gas_data", "Reward transaction detected");
return Err(TxVerifyFailed::ErroneousTxs(vec![tx.clone()]).into())
}
// Write the actual payload data
let mut payload = vec![];
tx.calls.encode_async(&mut payload).await?;
let wasm = overlay.lock().unwrap().contracts.get(call.data.contract_id)?;
let mut runtime = Runtime::new(
&wasm,
overlay.clone(),
call.data.contract_id,
block_target,
block_target,
tx_hash,
idx as u8,
)?;
let metadata = runtime.metadata(&payload)?;
// Decode the metadata retrieved from the execution
let mut decoder = Cursor::new(&metadata);
// The tuple is (zkas_ns, public_inputs)
let zkp_pub: Vec<(String, Vec<pallas::Base>)> =
AsyncDecodable::decode_async(&mut decoder).await?;
let sig_pub: Vec<PublicKey> = AsyncDecodable::decode_async(&mut decoder).await?;
if decoder.position() != metadata.len() as u64 {
error!(
target: "block_explorer::calculate_tx_gas_data",
"[BLOCK_EXPLORER] Failed decoding entire metadata buffer for {}:{}", tx_hash, idx,
);
return Err(TxVerifyFailed::ErroneousTxs(vec![tx.clone()]).into())
}
// Here we'll look up verifying keys and insert them into the per-contract map.
for (zkas_ns, _) in &zkp_pub {
let inner_vk_map =
verifying_keys.get_mut(&call.data.contract_id.to_bytes()).unwrap();
// TODO: This will be a problem in case of ::deploy, unless we force a different
// namespace and disable updating existing circuit. Might be a smart idea to do
// so in order to have to care less about being able to verify historical txs.
if inner_vk_map.contains_key(zkas_ns.as_str()) {
continue
}
let (zkbin, vk) =
overlay.lock().unwrap().contracts.get_zkas(&call.data.contract_id, zkas_ns)?;
inner_vk_map.insert(zkas_ns.to_string(), vk);
circuits_to_verify.push(zkbin);
}
zkp_table.push(zkp_pub);
sig_table.push(sig_pub);
// Contracts are not included within blocks. They need to be deployed off-chain so that they can be accessed and utilized for fee data computation
if call.data.is_deployment()
/* DeployV1 */
{
// Deserialize the deployment parameters
let deploy_params: DeployParamsV1 = deserialize_async(&call.data.data[1..]).await?;
let deploy_cid = ContractId::derive_public(deploy_params.public_key);
// Instantiate the new deployment runtime
let mut deploy_runtime = Runtime::new(
&deploy_params.wasm_bincode,
overlay.clone(),
deploy_cid,
block_target,
block_target,
tx_hash,
idx as u8,
)?;
deploy_runtime.deploy(&deploy_params.ix)?;
deploy_gas_used = deploy_runtime.gas_used();
// Append the used deployment gas
total_gas_used += deploy_gas_used;
}
// At this point we're done with the call and move on to the next one.
// Accumulate the WASM gas used.
wasm_gas_used = runtime.gas_used();
// Append the used wasm gas
total_gas_used += wasm_gas_used;
}
// The signature fee is tx_size + fixed_sig_fee * n_signatures
let signature_gas_used = (PALLAS_SCHNORR_SIGNATURE_FEE * tx.signatures.len() as u64) +
serialize_async(tx).await.len() as u64;
// Append the used signature gas
total_gas_used += signature_gas_used;
// The ZK circuit fee is calculated using a function in validator/fees.rs
for zkbin in circuits_to_verify.iter() {
zk_circuit_gas_used = circuit_gas_use(zkbin);
// Append the used zk circuit gas
total_gas_used += zk_circuit_gas_used;
}
if verify_fee {
// Deserialize the fee call to find the paid fee
let fee: u64 = match deserialize_async(&tx.calls[fee_call_idx].data.data[1..9]).await {
Ok(v) => v,
Err(e) => {
error!(
target: "block_explorer::calculate_tx_gas_data",
"[VALIDATOR] Failed deserializing tx {} fee call: {}", tx_hash, e,
);
return Err(TxVerifyFailed::InvalidFee.into())
}
};
// TODO: This counts 1 gas as 1 token unit. Pricing should be better specified.
// Check that enough fee has been paid for the used gas in this transaction.
if total_gas_used > fee {
error!(
target: "block_explorer::calculate_tx_gas_data",
"[VALIDATOR] Transaction {} has insufficient fee. Required: {}, Paid: {}",
tx_hash, total_gas_used, fee,
);
return Err(TxVerifyFailed::InsufficientFee.into())
}
debug!(target: "block_explorer::calculate_tx_gas_data", "The gas paid for transaction {}: {}", tx_hash, gas_paid);
// Store paid fee
gas_paid = fee;
}
// Commit changes made to the overlay
overlay.lock().unwrap().overlay.lock().unwrap().apply()?;
let fee_data = GasData {
paid: gas_paid,
wasm: wasm_gas_used,
zk_circuits: zk_circuit_gas_used,
signatures: signature_gas_used,
deployments: deploy_gas_used,
};
debug!(target: "block_explorer::calculate_tx_gas_data", "The total gas usage for transaction {}: {:?}", tx_hash, fee_data);
Ok(fee_data)
}
/// Converts a [`Transaction`] and its associated block information into a [`TransactionRecord`].
///
/// This auxiliary function first retrieves the gas data associated with the provided transaction.
@@ -239,7 +447,7 @@ impl ExplorerDb {
tx: &Transaction,
) -> Result<TransactionRecord> {
// Fetch the gas data associated with the transaction
let gas_data_option = self.metrics_store.get_tx_gas_data(&tx.hash()).map_err(|e| {
let gas_data_option = self.db.metrics_store.get_tx_gas_data(&tx.hash()).map_err(|e| {
Error::DatabaseError(format!(
"[to_tx_record] Failed to fetch the gas data associated with transaction {}: {e:?}",
tx.hash()