minerd: changed comms logic so it polls darkfid for new mining jobs

This commit is contained in:
skoupidi
2025-11-29 17:41:00 +02:00
parent ece4e4d871
commit 9b141f3fcb
46 changed files with 1519 additions and 1345 deletions

2
Cargo.lock generated
View File

@@ -4312,6 +4312,7 @@ dependencies = [
name = "minerd"
version = "0.5.0"
dependencies = [
"bs58",
"darkfi",
"darkfi-sdk",
"darkfi-serial",
@@ -4324,6 +4325,7 @@ dependencies = [
"structopt",
"structopt-toml",
"tinyjson",
"toml 0.9.8",
"tracing",
"tracing-appender",
"tracing-subscriber",

View File

@@ -17,22 +17,9 @@ database = "~/.local/share/darkfi/darkfid/testnet"
# Confirmation threshold, denominated by number of blocks
threshold = 6
# minerd JSON-RPC endpoint
#minerd_endpoint = "tcp://127.0.0.1:28467"
# PoW block production target, in seconds
pow_target = 120
# Wallet address to receive mining rewards.
#recipient = "YOUR_WALLET_ADDRESS_HERE"
# Optional contract spend hook to use in the mining reward
#spend_hook = "YOUR_SPEND_HOOK_HERE"
# Optional contract user data to use in the mining reward.
# This is not arbitrary data.
#user_data = "YOUR_USER_DATA_HERE"
# Skip syncing process and start node right away
skip_sync = false
@@ -156,22 +143,9 @@ database = "~/.local/share/darkfi/darkfid/mainnet"
# Confirmation threshold, denominated by number of blocks
threshold = 11
# minerd JSON-RPC endpoint
#minerd_endpoint = "tcp://127.0.0.1:28467"
# PoW block production target, in seconds
pow_target = 120
# Wallet address to receive mining rewards.
#recipient = "YOUR_WALLET_ADDRESS_HERE"
# Optional contract spend hook to use in the mining reward
#spend_hook = "YOUR_SPEND_HOOK_HERE"
# Optional contract user data to use in the mining reward.
# This is not arbitrary data.
#user_data = "YOUR_USER_DATA_HERE"
# Skip syncing process and start node right away
skip_sync = false
@@ -297,27 +271,12 @@ database = "~/.local/share/darkfi/darkfid/localnet"
# Confirmation threshold, denominated by number of blocks
threshold = 3
# minerd JSON-RPC endpoint
minerd_endpoint = "tcp://127.0.0.1:28467"
# PoW block production target, in seconds
pow_target = 10
# Optional fixed PoW difficulty, used for testing
pow_fixed_difficulty = 1
# Wallet address to receive mining rewards.
# This is a dummy one so the miner can start,
# replace with your own one.
recipient = "5ZHfYpt4mpJcwBNxfEyxLzeFJUEeoePs5NQ5jVEgHrMf"
# Optional contract spend hook to use in the mining reward
#spend_hook = "YOUR_SPEND_HOOK_HERE"
# Optional contract user data to use in the mining reward.
# This is not arbitrary data.
#user_data = "YOUR_USER_DATA_HERE"
# Skip syncing process and start node right away
skip_sync = true

View File

@@ -37,8 +37,16 @@ pub enum RpcError {
ContractStateNotFound = -32201,
ContractStateKeyNotFound = -32202,
// Misc errors
PingFailed = -32300,
// Miner errors
MinerMissingHeader = -32300,
MinerInvalidHeader = -32301,
MinerMissingRecipient = -32302,
MinerInvalidRecipient = -32303,
MinerInvalidSpendHook = -32304,
MinerInvalidUserData = -32305,
MinerMissingNonce = -32306,
MinerInvalidNonce = -32307,
MinerUnknownJob = -32308,
}
fn to_tuple(e: RpcError) -> (i32, String) {
@@ -55,8 +63,16 @@ fn to_tuple(e: RpcError) -> (i32, String) {
RpcError::ContractZkasDbNotFound => "zkas database not found for given contract",
RpcError::ContractStateNotFound => "Records not found for given contract state",
RpcError::ContractStateKeyNotFound => "Value not found for given contract state key",
// Misc errors
RpcError::PingFailed => "Miner daemon ping error",
// Miner errors
RpcError::MinerMissingHeader => "Request is missing the Header hash",
RpcError::MinerInvalidHeader => "Request Header hash is invalid",
RpcError::MinerMissingRecipient => "Request is missing the recipient wallet address",
RpcError::MinerInvalidRecipient => "Request recipient wallet address is invalid",
RpcError::MinerInvalidSpendHook => "Request spend hook is invalid",
RpcError::MinerInvalidUserData => "Request user data is invalid",
RpcError::MinerMissingNonce => "Request is missing the Header nonce",
RpcError::MinerInvalidNonce => "Request Header nonce is invalid",
RpcError::MinerUnknownJob => "Request job is unknown",
};
(e as i32, msg.to_string())

View File

@@ -22,8 +22,7 @@ use std::{
};
use smol::lock::Mutex;
use tracing::{debug, error, info, warn};
use url::Url;
use tracing::{debug, error, info};
use darkfi::{
blockchain::BlockInfo,
@@ -52,9 +51,11 @@ use error::{server_error, RpcError};
/// JSON-RPC requests handler and methods
mod rpc;
use rpc::{DefaultRpcHandler, MinerRpcClient, MmRpcHandler};
use rpc::{DefaultRpcHandler, MmRpcHandler};
mod rpc_blockchain;
mod rpc_miner;
mod rpc_tx;
use rpc_miner::BlockTemplate;
mod rpc_xmr;
/// Validator async tasks
@@ -78,14 +79,14 @@ pub struct DarkfiNode {
txs_batch_size: usize,
/// A map of various subscribers exporting live info from the blockchain
subscribers: HashMap<&'static str, JsonSubscriber>,
/// JSON-RPC connection tracker
rpc_connections: Mutex<HashSet<StoppableTaskPtr>>,
/// JSON-RPC client to execute requests to the miner daemon
rpc_client: Option<Mutex<MinerRpcClient>>,
/// HTTP JSON-RPC connection tracker
mm_rpc_connections: Mutex<HashSet<StoppableTaskPtr>>,
/// Native mining block templates
blocktemplates: Mutex<HashMap<Vec<u8>, BlockTemplate>>,
/// Merge mining block templates
mm_blocktemplates: Mutex<HashMap<Vec<u8>, (BlockInfo, f64, SecretKey)>>,
/// JSON-RPC connection tracker
rpc_connections: Mutex<HashSet<StoppableTaskPtr>>,
/// HTTP JSON-RPC connection tracker
mm_rpc_connections: Mutex<HashSet<StoppableTaskPtr>>,
/// PowRewardV1 ZK data
powrewardv1_zk: PowRewardV1Zk,
}
@@ -96,7 +97,6 @@ impl DarkfiNode {
validator: ValidatorPtr,
txs_batch_size: usize,
subscribers: HashMap<&'static str, JsonSubscriber>,
rpc_client: Option<Mutex<MinerRpcClient>>,
) -> Result<DarkfiNodePtr> {
let powrewardv1_zk = PowRewardV1Zk::new(validator.clone())?;
@@ -105,15 +105,15 @@ impl DarkfiNode {
validator,
txs_batch_size,
subscribers,
rpc_connections: Mutex::new(HashSet::new()),
rpc_client,
mm_rpc_connections: Mutex::new(HashSet::new()),
blocktemplates: Mutex::new(HashMap::new()),
mm_blocktemplates: Mutex::new(HashMap::new()),
rpc_connections: Mutex::new(HashSet::new()),
mm_rpc_connections: Mutex::new(HashSet::new()),
powrewardv1_zk,
}))
}
/// Grab best current fork
/// Auxiliary function to grab best current fork.
pub async fn best_current_fork(&self) -> Result<Fork> {
let forks = self.validator.consensus.forks.read().await;
let index = best_fork_index(&forks)?;
@@ -173,7 +173,6 @@ impl Darkfid {
sled_db: &sled_overlay::sled::Db,
config: &ValidatorConfig,
net_settings: &Settings,
minerd_endpoint: &Option<Url>,
txs_batch_size: &Option<usize>,
ex: &ExecutorPtr,
) -> Result<DarkfidPtr> {
@@ -203,17 +202,8 @@ impl Darkfid {
subscribers.insert("proposals", JsonSubscriber::new("blockchain.subscribe_proposals"));
subscribers.insert("dnet", JsonSubscriber::new("dnet.subscribe_events"));
// Initialize JSON-RPC client to perform requests to minerd
let rpc_client = match minerd_endpoint {
Some(endpoint) => {
Some(Mutex::new(MinerRpcClient::new(endpoint.clone(), ex.clone()).await))
}
None => None,
};
// Initialize node
let node = DarkfiNode::new(p2p_handler, validator, txs_batch_size, subscribers, rpc_client)
.await?;
let node = DarkfiNode::new(p2p_handler, validator, txs_batch_size, subscribers).await?;
// Generate the background tasks
let dnet_task = StoppableTask::new();
@@ -237,13 +227,6 @@ impl Darkfid {
) -> Result<()> {
info!(target: "darkfid::Darkfid::start", "Starting Darkfi daemon...");
// Pinging minerd daemon to verify it listens
if self.node.rpc_client.is_some() {
if let Err(e) = self.node.ping_miner_daemon().await {
warn!(target: "darkfid::Darkfid::start", "Failed to ping miner daemon: {e}");
}
}
// Start the `dnet` task
info!(target: "darkfid::Darkfid::start", "Starting dnet subs task");
let dnet_sub_ = self.node.subscribers.get("dnet").unwrap().clone();
@@ -366,12 +349,6 @@ impl Darkfid {
let flushed_bytes = self.node.validator.blockchain.sled_db.flush_async().await?;
info!(target: "darkfid::Darkfid::stop", "Flushed {flushed_bytes} bytes");
// Close the JSON-RPC client, if it was initialized
if let Some(ref rpc_client) = self.node.rpc_client {
info!(target: "darkfid::Darkfid::stop", "Stopping JSON-RPC client...");
rpc_client.lock().await.stop().await;
};
info!(target: "darkfid::Darkfid::stop", "Darkfi daemon terminated successfully!");
Ok(())
}

View File

@@ -21,7 +21,6 @@ use std::sync::Arc;
use smol::{fs::read_to_string, stream::StreamExt};
use structopt_toml::{serde::Deserialize, structopt::StructOpt, StructOptToml};
use tracing::{debug, error, info};
use url::Url;
use darkfi::{
async_daemonize,
@@ -99,14 +98,6 @@ pub struct BlockchainNetwork {
/// Confirmation threshold, denominated by number of blocks
threshold: usize,
#[structopt(long)]
/// minerd JSON-RPC endpoint
minerd_endpoint: Option<Url>,
#[structopt(skip)]
/// Optional JSON-RPC settings for p2pool merge mining requests
mm_rpc: Option<RpcSettingsOpt>,
#[structopt(long, default_value = "120")]
/// PoW block production target, in seconds
pow_target: u32,
@@ -115,19 +106,6 @@ pub struct BlockchainNetwork {
/// Optional fixed PoW difficulty, used for testing
pow_fixed_difficulty: Option<usize>,
#[structopt(long)]
/// Wallet address to receive mining rewards
recipient: Option<String>,
#[structopt(long)]
/// Optional contract spend hook to use in the mining reward
spend_hook: Option<String>,
#[structopt(long)]
/// Optional contract user data to use in the mining reward.
/// This is not arbitrary data.
user_data: Option<String>,
#[structopt(long)]
/// Skip syncing process and start node right away
skip_sync: bool,
@@ -159,6 +137,10 @@ pub struct BlockchainNetwork {
#[structopt(flatten)]
/// JSON-RPC settings
rpc: RpcSettingsOpt,
#[structopt(skip)]
/// Optional JSON-RPC settings for p2pool merge mining requests
mm_rpc: Option<RpcSettingsOpt>,
}
async_daemonize!(realmain);
@@ -255,7 +237,6 @@ async fn realmain(args: Args, ex: Arc<smol::Executor<'static>>) -> Result<()> {
&sled_db,
&config,
&blockchain_config.net.into(),
&blockchain_config.minerd_endpoint,
&blockchain_config.txs_batch_size,
&ex,
)
@@ -266,10 +247,6 @@ async fn realmain(args: Args, ex: Arc<smol::Executor<'static>>) -> Result<()> {
skip_sync: blockchain_config.skip_sync,
checkpoint_height: blockchain_config.checkpoint_height,
checkpoint: blockchain_config.checkpoint,
miner: blockchain_config.minerd_endpoint.is_some(),
recipient: blockchain_config.recipient,
spend_hook: blockchain_config.spend_hook,
user_data: blockchain_config.user_data,
bootstrap,
};
daemon

View File

@@ -16,65 +16,31 @@
* along with this program. If not, see <https://www.gnu.org/licenses/>.
*/
use std::{collections::HashSet, time::Instant};
use std::collections::HashSet;
use async_trait::async_trait;
use smol::lock::MutexGuard;
use tinyjson::JsonValue;
use tracing::{debug, error, info, warn};
use url::Url;
use tracing::debug;
use darkfi::{
net::P2pPtr,
rpc::{
client::RpcChadClient,
jsonrpc::{ErrorCode, JsonError, JsonRequest, JsonResponse, JsonResult},
p2p_method::HandlerP2p,
server::RequestHandler,
},
system::{sleep, ExecutorPtr, StoppableTaskPtr},
system::StoppableTaskPtr,
util::time::Timestamp,
Error, Result,
};
use crate::{
error::{server_error, RpcError},
DarkfiNode,
};
use crate::DarkfiNode;
/// Default JSON-RPC `RequestHandler` type
pub struct DefaultRpcHandler;
/// HTTP JSON-RPC `RequestHandler` type for p2pool
pub struct MmRpcHandler;
/// Structure to hold a JSON-RPC client and its config,
/// so we can recreate it in case of an error.
pub struct MinerRpcClient {
endpoint: Url,
ex: ExecutorPtr,
client: Option<RpcChadClient>,
}
impl MinerRpcClient {
pub async fn new(endpoint: Url, ex: ExecutorPtr) -> Self {
let client = match RpcChadClient::new(endpoint.clone(), ex.clone()).await {
Ok(c) => Some(c),
Err(_) => {
warn!(target: "darkfid::Darkfid::init", "Failed to initialize miner daemon rpc client, will try later");
None
}
};
Self { endpoint, ex, client }
}
/// Stop the client.
pub async fn stop(&self) {
if let Some(ref client) = self.client {
client.stop().await
}
}
}
#[async_trait]
#[rustfmt::skip]
impl RequestHandler<DefaultRpcHandler> for DarkfiNode {
@@ -87,7 +53,6 @@ impl RequestHandler<DefaultRpcHandler> for DarkfiNode {
// =====================
"ping" => <DarkfiNode as RequestHandler<DefaultRpcHandler>>::pong(self, req.id, req.params).await,
"clock" => self.clock(req.id, req.params).await,
"ping_miner" => self.ping_miner(req.id, req.params).await,
"dnet.switch" => self.dnet_switch(req.id, req.params).await,
"dnet.subscribe_events" => self.dnet_subscribe_events(req.id, req.params).await,
"p2p.get_info" => self.p2p_get_info(req.id, req.params).await,
@@ -116,6 +81,12 @@ impl RequestHandler<DefaultRpcHandler> for DarkfiNode {
"tx.clean_pending" => self.tx_clean_pending(req.id, req.params).await,
"tx.calculate_fee" => self.tx_calculate_fee(req.id, req.params).await,
// =============
// Miner methods
// =============
"miner.get_header" => self.miner_get_header(req.id, req.params).await,
"miner.submit_solution" => self.miner_submit_solution(req.id, req.params).await,
// ==============
// Invalid method
// ==============
@@ -204,83 +175,6 @@ impl DarkfiNode {
self.subscribers.get("dnet").unwrap().clone().into()
}
// RPCAPI:
// Pings configured miner daemon for liveness.
// Returns `true` on success.
//
// --> {"jsonrpc": "2.0", "method": "ping_miner", "params": [], "id": 1}
// <-- {"jsonrpc": "2.0", "result": "true", "id": 1}
async fn ping_miner(&self, id: u16, _params: JsonValue) -> JsonResult {
if let Err(e) = self.ping_miner_daemon().await {
error!(target: "darkfid::rpc::ping_miner", "Failed to ping miner daemon: {e}");
return server_error(RpcError::PingFailed, id, None)
}
JsonResponse::new(JsonValue::Boolean(true), id).into()
}
/// Ping configured miner daemon JSON-RPC endpoint.
pub async fn ping_miner_daemon(&self) -> Result<()> {
debug!(target: "darkfid::ping_miner_daemon", "Pinging miner daemon...");
self.miner_daemon_request("ping", &JsonValue::Array(vec![])).await?;
Ok(())
}
/// Auxiliary function to execute a request towards the configured miner daemon JSON-RPC endpoint.
pub async fn miner_daemon_request(
&self,
method: &str,
params: &JsonValue,
) -> Result<JsonValue> {
let Some(ref rpc_client) = self.rpc_client else { return Err(Error::RpcClientStopped) };
debug!(target: "darkfid::rpc::miner_daemon_request", "Executing request {method} with params: {params:?}");
let latency = Instant::now();
let req = JsonRequest::new(method, params.clone());
let lock = rpc_client.lock().await;
let Some(ref client) = lock.client else { return Err(Error::RpcClientStopped) };
let rep = client.request(req).await?;
drop(lock);
let latency = latency.elapsed();
debug!(target: "darkfid::rpc::miner_daemon_request", "Got reply: {rep:?}");
debug!(target: "darkfid::rpc::miner_daemon_request", "Latency: {latency:?}");
Ok(rep)
}
/// Auxiliary function to execute a request towards the configured miner daemon JSON-RPC endpoint,
/// but in case of failure, sleep and retry until connection is re-established.
pub async fn miner_daemon_request_with_retry(
&self,
method: &str,
params: &JsonValue,
) -> JsonValue {
loop {
// Try to execute the request using current client
match self.miner_daemon_request(method, params).await {
Ok(v) => return v,
Err(e) => {
error!(target: "darkfid::rpc::miner_daemon_request_with_retry", "Failed to execute miner daemon request: {e}");
}
}
loop {
// Sleep a bit before retrying
info!(target: "darkfid::rpc::miner_daemon_request_with_retry", "Sleeping so we can retry later");
sleep(10).await;
// Create a new client
let mut rpc_client = self.rpc_client.as_ref().unwrap().lock().await;
let Ok(client) =
RpcChadClient::new(rpc_client.endpoint.clone(), rpc_client.ex.clone()).await
else {
error!(target: "darkfid::rpc::miner_daemon_request_with_retry", "Failed to initialize miner daemon rpc client, check if minerd is running");
drop(rpc_client);
continue
};
info!(target: "darkfid::rpc::miner_daemon_request_with_retry", "Connection re-established!");
// Set the new client as the daemon one
rpc_client.client = Some(client);
break;
}
}
}
}
impl HandlerP2p for DarkfiNode {

View File

@@ -0,0 +1,540 @@
/* This file is part of DarkFi (https://dark.fi)
*
* Copyright (C) 2020-2025 Dyne.org foundation
*
* This program is free software: you can redistribute it and/or modify
* it under the terms of the GNU Affero 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 Affero General Public License for more details.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <https://www.gnu.org/licenses/>.
*/
use std::{collections::HashMap, str::FromStr};
use darkfi::{
blockchain::{BlockInfo, Header, HeaderHash},
rpc::jsonrpc::{ErrorCode, ErrorCode::InvalidParams, JsonError, JsonResponse, JsonResult},
tx::{ContractCallLeaf, Transaction, TransactionBuilder},
util::{encoding::base64, time::Timestamp},
validator::{
consensus::{Fork, Proposal},
pow::{RANDOMX_KEY_CHANGE_DELAY, RANDOMX_KEY_CHANGING_HEIGHT},
verification::apply_producer_transaction,
},
zk::ProvingKey,
zkas::ZkBinary,
Error, Result,
};
use darkfi_money_contract::{client::pow_reward_v1::PoWRewardCallBuilder, MoneyFunction};
use darkfi_sdk::{
crypto::{
pasta_prelude::PrimeField, FuncId, Keypair, MerkleTree, PublicKey, SecretKey,
MONEY_CONTRACT_ID,
},
pasta::pallas,
ContractCall,
};
use darkfi_serial::{serialize_async, Encodable};
use num_bigint::BigUint;
use rand::rngs::OsRng;
use tinyjson::JsonValue;
use tracing::{error, info};
use crate::{proto::ProposalMessage, server_error, DarkfiNode, RpcError};
/// Auxiliary structure representing node miner rewards recipient configuration.
pub struct MinerRewardsRecipientConfig {
/// Wallet mining address to receive mining rewards
pub recipient: PublicKey,
/// Optional contract spend hook to use in the mining reward
pub spend_hook: Option<FuncId>,
/// Optional contract user data to use in the mining reward.
/// This is not arbitrary data.
pub user_data: Option<pallas::Base>,
}
/// Auxiliary structure representing a block template for native mining.
pub struct BlockTemplate {
/// The block that is being mined
pub block: BlockInfo,
/// The base64 encoded RandomX key used
randomx_key: String,
/// The base64 encoded mining target used
target: String,
/// The signing secret for this block
secret: SecretKey,
}
impl DarkfiNode {
// RPCAPI:
// Queries the validator for the current best fork next header to
// mine.
// Returns the current RandomX key, the mining target and the next
// block header, all encoded as base64 strings.
//
// **Params:**
// * `header` : Mining job Header hash that is currently being polled (as string)
// * `recipient` : Wallet mining address to receive mining rewards (as string)
// * `spend_hook`: Optional contract spend hook to use in the mining reward (as string)
// * `user_data` : Optional contract user data (not arbitrary data) to use in the mining reward (as string)
//
// **Returns:**
// * `String`: Current best fork RandomX key (base64 encoded)
// * `String`: Current best fork mining target (base64 encoded)
// * `String`: Current best fork next block header (base64 encoded)
//
// --> {"jsonrpc": "2.0", "method": "miner.get_header", "params": {"header": "hash", "recipient": "address"}, "id": 1}
// <-- {"jsonrpc": "2.0", "result": ["randomx_key", "target", "header"], "id": 1}
pub async fn miner_get_header(&self, id: u16, params: JsonValue) -> JsonResult {
// Check if node is synced before responding to miner
if !*self.validator.synced.read().await {
return server_error(RpcError::NotSynced, id, None)
}
// Parse request params
let Some(params) = params.get::<HashMap<String, JsonValue>>() else {
return JsonError::new(InvalidParams, None, id).into()
};
if params.len() < 2 || params.len() > 4 {
return JsonError::new(InvalidParams, None, id).into()
}
// Parse header hash
let Some(header_hash) = params.get("header") else {
return server_error(RpcError::MinerMissingHeader, id, None)
};
let Some(header_hash) = header_hash.get::<String>() else {
return server_error(RpcError::MinerInvalidHeader, id, None)
};
let Ok(header_hash) = HeaderHash::from_str(header_hash) else {
return server_error(RpcError::MinerInvalidHeader, id, None)
};
// Parse recipient wallet address
let Some(recipient) = params.get("recipient") else {
return server_error(RpcError::MinerMissingRecipient, id, None)
};
let Some(recipient) = recipient.get::<String>() else {
return server_error(RpcError::MinerInvalidRecipient, id, None)
};
let Ok(recipient) = PublicKey::from_str(recipient) else {
return server_error(RpcError::MinerInvalidRecipient, id, None)
};
// Parse spend hook
let spend_hook = match params.get("spend_hook") {
Some(spend_hook) => {
let Some(spend_hook) = spend_hook.get::<String>() else {
return server_error(RpcError::MinerInvalidSpendHook, id, None)
};
let Ok(spend_hook) = FuncId::from_str(spend_hook) else {
return server_error(RpcError::MinerInvalidSpendHook, id, None)
};
Some(spend_hook)
}
None => None,
};
// Parse user data
let user_data: Option<pallas::Base> = match params.get("user_data") {
Some(user_data) => {
let Some(user_data) = user_data.get::<String>() else {
return server_error(RpcError::MinerInvalidUserData, id, None)
};
let Ok(bytes) = bs58::decode(&user_data).into_vec() else {
return server_error(RpcError::MinerInvalidUserData, id, None)
};
let bytes: [u8; 32] = match bytes.try_into() {
Ok(b) => b,
Err(_) => return server_error(RpcError::MinerInvalidUserData, id, None),
};
let Some(user_data) = pallas::Base::from_repr(bytes).into() else {
return server_error(RpcError::MinerInvalidUserData, id, None)
};
Some(user_data)
}
None => None,
};
// Now that method params format is correct, we can check if we
// already have a mining job for this wallet. If we already
// have it, we check if the fork it extends is still the best
// one. If both checks pass, we can just return an empty
// response if the request `aux_hash` matches the job one,
// otherwise return the job block template hash. In case the
// best fork has changed, we drop this job and generate a
// new one. If we don't know this wallet, we create a new job.
// We'll also obtain a lock here to avoid getting polled
// multiple times and potentially missing a job. The lock is
// released when this function exits.
let address_bytes = serialize_async(&(recipient, spend_hook, user_data)).await;
let mut blocktemplates = self.blocktemplates.lock().await;
let mut extended_fork = match self.best_current_fork().await {
Ok(f) => f,
Err(e) => {
error!(
target: "darkfid::rpc::miner_get_header",
"[RPC] Finding best fork index failed: {e}",
);
return JsonError::new(ErrorCode::InternalError, None, id).into()
}
};
if let Some(blocktemplate) = blocktemplates.get(&address_bytes) {
let last_proposal = match extended_fork.last_proposal() {
Ok(p) => p,
Err(e) => {
error!(
target: "darkfid::rpc::miner_get_header",
"[RPC] Retrieving best fork last proposal failed: {e}",
);
return JsonError::new(ErrorCode::InternalError, None, id).into()
}
};
if last_proposal.hash == blocktemplate.block.header.previous {
return if blocktemplate.block.header.hash() != header_hash {
JsonResponse::new(
JsonValue::Array(vec![
JsonValue::String(blocktemplate.randomx_key.clone()),
JsonValue::String(blocktemplate.target.clone()),
JsonValue::String(base64::encode(
&serialize_async(&blocktemplate.block.header).await,
)),
]),
id,
)
.into()
} else {
JsonResponse::new(JsonValue::Array(vec![]), id).into()
}
}
blocktemplates.remove(&address_bytes);
}
// At this point, we should query the Validator for a new blocktemplate.
// We first need to construct `MinerRewardsRecipientConfig` from the
// address configuration provided to us through the RPC.
let recipient_str = format!("{recipient}");
let spend_hook_str = match spend_hook {
Some(spend_hook) => format!("{spend_hook}"),
None => String::from("-"),
};
let user_data_str = match user_data {
Some(user_data) => bs58::encode(user_data.to_repr()).into_string(),
None => String::from("-"),
};
let recipient_config = MinerRewardsRecipientConfig { recipient, spend_hook, user_data };
// Now let's try to construct the blocktemplate.
let (target, block, secret) = match generate_next_block(
&mut extended_fork,
&recipient_config,
&self.powrewardv1_zk.zkbin,
&self.powrewardv1_zk.provingkey,
self.validator.consensus.module.read().await.target,
self.validator.verify_fees,
)
.await
{
Ok(v) => v,
Err(e) => {
error!(
target: "darkfid::rpc::miner_get_header",
"[RPC] Failed to generate next blocktemplate: {e}",
);
return JsonError::new(ErrorCode::InternalError, None, id).into()
}
};
// Grab the RandomX key to use.
// We only use the next key when the next block is the
// height changing one.
let randomx_key = if block.header.height > RANDOMX_KEY_CHANGING_HEIGHT &&
block.header.height % RANDOMX_KEY_CHANGING_HEIGHT == RANDOMX_KEY_CHANGE_DELAY
{
base64::encode(&serialize_async(&extended_fork.module.darkfi_rx_keys.1).await)
} else {
base64::encode(&serialize_async(&extended_fork.module.darkfi_rx_keys.0).await)
};
// Convert the target
let target = base64::encode(&target.to_bytes_le());
// Construct the block template
let blocktemplate = BlockTemplate {
block,
randomx_key: randomx_key.clone(),
target: target.clone(),
secret,
};
// Now we have the blocktemplate. We'll mark it down in memory,
// and then ship it to RPC.
let header_hash = blocktemplate.block.header.hash().to_string();
let header = base64::encode(&serialize_async(&blocktemplate.block.header).await);
blocktemplates.insert(address_bytes, blocktemplate);
info!(
target: "darkfid::rpc::miner_get_header",
"[RPC] Created new blocktemplate: address={recipient_str}, spend_hook={spend_hook_str}, user_data={user_data_str}, hash={header_hash}"
);
let response = JsonValue::Array(vec![
JsonValue::String(randomx_key),
JsonValue::String(target),
JsonValue::String(header),
]);
JsonResponse::new(response, id).into()
}
// RPCAPI:
// Submits a PoW solution header nonce for a block.
// Returns the block submittion status.
//
// **Params:**
// * `recipient` : Wallet mining address used (as string)
// * `spend_hook`: Optional contract spend hook used (as string)
// * `user_data` : Optional contract user data (not arbitrary data) used (as string)
// * `nonce` : The solution header nonce (as f64)
//
// **Returns:**
// * `String`: Block submit status
//
// --> {"jsonrpc": "2.0", "method": "miner.submit_solution", "params": {"recipient": "address", "nonce": 42}, "id": 1}
// <-- {"jsonrpc": "2.0", "result": "accepted", "id": 1}
pub async fn miner_submit_solution(&self, id: u16, params: JsonValue) -> JsonResult {
// Check if node is synced before responding to p2pool
if !*self.validator.synced.read().await {
return server_error(RpcError::NotSynced, id, None)
}
// Parse request params
let Some(params) = params.get::<HashMap<String, JsonValue>>() else {
return JsonError::new(InvalidParams, None, id).into()
};
if params.len() < 2 || params.len() > 4 {
return JsonError::new(InvalidParams, None, id).into()
}
// Parse recipient wallet address
let Some(recipient) = params.get("recipient") else {
return server_error(RpcError::MinerMissingRecipient, id, None)
};
let Some(recipient) = recipient.get::<String>() else {
return server_error(RpcError::MinerInvalidRecipient, id, None)
};
let Ok(recipient) = PublicKey::from_str(recipient) else {
return server_error(RpcError::MinerInvalidRecipient, id, None)
};
// Parse spend hook
let spend_hook = match params.get("spend_hook") {
Some(spend_hook) => {
let Some(spend_hook) = spend_hook.get::<String>() else {
return server_error(RpcError::MinerInvalidSpendHook, id, None)
};
let Ok(spend_hook) = FuncId::from_str(spend_hook) else {
return server_error(RpcError::MinerInvalidSpendHook, id, None)
};
Some(spend_hook)
}
None => None,
};
// Parse user data
let user_data: Option<pallas::Base> = match params.get("user_data") {
Some(user_data) => {
let Some(user_data) = user_data.get::<String>() else {
return server_error(RpcError::MinerInvalidUserData, id, None)
};
let Ok(bytes) = bs58::decode(&user_data).into_vec() else {
return server_error(RpcError::MinerInvalidUserData, id, None)
};
let bytes: [u8; 32] = match bytes.try_into() {
Ok(b) => b,
Err(_) => return server_error(RpcError::MinerInvalidUserData, id, None),
};
let Some(user_data) = pallas::Base::from_repr(bytes).into() else {
return server_error(RpcError::MinerInvalidUserData, id, None)
};
Some(user_data)
}
None => None,
};
// Parse nonce
let Some(nonce) = params.get("nonce") else {
return server_error(RpcError::MinerMissingNonce, id, None)
};
let Some(nonce) = nonce.get::<f64>() else {
return server_error(RpcError::MinerInvalidNonce, id, None)
};
// If we don't know about this job, we can just abort here.
let address_bytes = serialize_async(&(recipient, spend_hook, user_data)).await;
let mut blocktemplates = self.blocktemplates.lock().await;
let Some(blocktemplate) = blocktemplates.get(&address_bytes) else {
return server_error(RpcError::MinerUnknownJob, id, None)
};
info!(
target: "darkfid::rpc::miner_submit_solution",
"[RPC] Got solution submission for block template: {}", blocktemplate.block.header.hash(),
);
// Sign the DarkFi block
let mut block = blocktemplate.block.clone();
block.header.nonce = *nonce as u64;
block.sign(&blocktemplate.secret);
info!(
target: "darkfid::rpc::miner_submit_solution",
"[RPC] Mined block header hash: {}", blocktemplate.block.header.hash(),
);
// At this point we should be able to remove the submitted job.
// We still won't release the lock in hope of proposing the block
// first.
blocktemplates.remove(&address_bytes);
// Propose the new block
info!(
target: "darkfid::rpc::miner_submit_solution",
"[RPC] Proposing new block to network",
);
let proposal = Proposal::new(block);
if let Err(e) = self.validator.append_proposal(&proposal).await {
error!(
target: "darkfid::rpc::miner_submit_solution",
"[RPC] Error proposing new block: {e}",
);
return JsonResponse::new(JsonValue::String(String::from("rejected")), id).into()
}
let proposals_sub = self.subscribers.get("proposals").unwrap();
let enc_prop = JsonValue::String(base64::encode(&serialize_async(&proposal).await));
proposals_sub.notify(vec![enc_prop].into()).await;
info!(
target: "darkfid::rpc::miner_submit_solution",
"[RPC] Broadcasting new block to network",
);
let message = ProposalMessage(proposal);
self.p2p_handler.p2p.broadcast(&message).await;
JsonResponse::new(JsonValue::String(String::from("accepted")), id).into()
}
}
/// Auxiliary function to generate next block in an atomic manner.
pub async fn generate_next_block(
extended_fork: &mut Fork,
recipient_config: &MinerRewardsRecipientConfig,
zkbin: &ZkBinary,
pk: &ProvingKey,
block_target: u32,
verify_fees: bool,
) -> Result<(BigUint, BlockInfo, SecretKey)> {
// Grab forks' last block proposal(previous)
let last_proposal = extended_fork.last_proposal()?;
// Grab forks' next block height
let next_block_height = last_proposal.block.header.height + 1;
// Grab forks' unproposed transactions
let (mut txs, _, fees, overlay) = extended_fork
.unproposed_txs(&extended_fork.blockchain, next_block_height, block_target, verify_fees)
.await?;
// Create an ephemeral block signing keypair. Its secret key will
// be stored in the PowReward transaction's encrypted note for
// later retrieval. It is encrypted towards the recipient's public
// key.
let block_signing_keypair = Keypair::random(&mut OsRng);
// Generate reward transaction
let tx = generate_transaction(
next_block_height,
fees,
&block_signing_keypair,
recipient_config,
zkbin,
pk,
)?;
// Apply producer transaction in the overlay
let _ = apply_producer_transaction(
&overlay,
next_block_height,
block_target,
&tx,
&mut MerkleTree::new(1),
)
.await?;
txs.push(tx);
// Grab the updated contracts states root
overlay.lock().unwrap().contracts.update_state_monotree(&mut extended_fork.state_monotree)?;
let Some(state_root) = extended_fork.state_monotree.get_headroot()? else {
return Err(Error::ContractsStatesRootNotFoundError);
};
// Drop new trees opened by the unproposed transactions overlay
overlay.lock().unwrap().overlay.lock().unwrap().purge_new_trees()?;
// Generate the new header
let mut header =
Header::new(last_proposal.hash, next_block_height, Timestamp::current_time(), 0);
header.state_root = state_root;
// Generate the block
let mut next_block = BlockInfo::new_empty(header);
// Add transactions to the block
next_block.append_txs(txs);
// Grab the next mine target
let target = extended_fork.module.next_mine_target()?;
Ok((target, next_block, block_signing_keypair.secret))
}
/// Auxiliary function to generate a Money::PoWReward transaction.
fn generate_transaction(
block_height: u32,
fees: u64,
block_signing_keypair: &Keypair,
recipient_config: &MinerRewardsRecipientConfig,
zkbin: &ZkBinary,
pk: &ProvingKey,
) -> Result<Transaction> {
// Build the transaction debris
let debris = PoWRewardCallBuilder {
signature_keypair: *block_signing_keypair,
block_height,
fees,
recipient: Some(recipient_config.recipient),
spend_hook: recipient_config.spend_hook,
user_data: recipient_config.user_data,
mint_zkbin: zkbin.clone(),
mint_pk: pk.clone(),
}
.build()?;
// Generate and sign the actual transaction
let mut data = vec![MoneyFunction::PoWRewardV1 as u8];
debris.params.encode(&mut data)?;
let call = ContractCall { contract_id: *MONEY_CONTRACT_ID, data };
let mut tx_builder =
TransactionBuilder::new(ContractCallLeaf { call, proofs: debris.proofs }, vec![])?;
let mut tx = tx_builder.build()?;
let sigs = tx.create_sigs(&[block_signing_keypair.secret])?;
tx.signatures = vec![sigs];
Ok(tx)
}

View File

@@ -42,9 +42,8 @@ use tracing::{error, info};
use crate::{
proto::ProposalMessage,
server_error,
task::miner::{generate_next_block, MinerRewardsRecipientConfig},
DarkfiNode, RpcError,
rpc_miner::{generate_next_block, MinerRewardsRecipientConfig},
server_error, DarkfiNode, RpcError,
};
// https://github.com/SChernykh/p2pool/blob/master/docs/MERGE_MINING.MD

View File

@@ -25,28 +25,24 @@ use darkfi::{
util::{encoding::base64, time::Timestamp},
Error, Result,
};
use darkfi_sdk::{
crypto::{FuncId, PublicKey},
pasta::{group::ff::PrimeField, pallas},
};
use darkfi_serial::serialize_async;
use tracing::{error, info};
use crate::{
task::{garbage_collect_task, miner::MinerRewardsRecipientConfig, miner_task, sync_task},
task::{garbage_collect_task, sync_task},
DarkfiNodePtr,
};
/// Auxiliary structure representing node consensus init task configuration
/// Auxiliary structure representing node consensus init task configuration.
#[derive(Clone)]
pub struct ConsensusInitTaskConfig {
/// Skip syncing process and start node right away
pub skip_sync: bool,
/// Optional sync checkpoint height
pub checkpoint_height: Option<u32>,
/// Optional sync checkpoint hash
pub checkpoint: Option<String>,
pub miner: bool,
pub recipient: Option<String>,
pub spend_hook: Option<String>,
pub user_data: Option<String>,
/// Optional bootstrap timestamp
pub bootstrap: u64,
}
@@ -95,54 +91,9 @@ pub async fn consensus_init_task(
None
};
// Grab rewards recipient public key(address) if node is a miner,
// along with configured spend hook and user data.
let recipient_config = if config.miner {
if config.recipient.is_none() {
return Err(Error::ParseFailed("Recipient address missing"))
}
let recipient = match PublicKey::from_str(config.recipient.as_ref().unwrap()) {
Ok(address) => address,
Err(_) => return Err(Error::InvalidAddress),
};
let spend_hook = match &config.spend_hook {
Some(s) => match FuncId::from_str(s) {
Ok(s) => Some(s),
Err(_) => return Err(Error::ParseFailed("Invalid spend hook")),
},
None => None,
};
let user_data = match &config.user_data {
Some(u) => {
let bytes: [u8; 32] = match bs58::decode(&u).into_vec()?.try_into() {
Ok(b) => b,
Err(_) => return Err(Error::ParseFailed("Invalid user data")),
};
match pallas::Base::from_repr(bytes).into() {
Some(v) => Some(v),
None => return Err(Error::ParseFailed("Invalid user data")),
}
}
None => None,
};
Some(MinerRewardsRecipientConfig { recipient, spend_hook, user_data })
} else {
None
};
// Gracefully handle network disconnections
loop {
let result = if config.miner {
miner_task(&node, recipient_config.as_ref().unwrap(), config.skip_sync, &ex).await
} else {
replicator_task(&node, &ex).await
};
match result {
match listen_to_network(&node, &ex).await {
Ok(_) => return Ok(()),
Err(Error::NetworkNotConnected) => {
// Sync node again
@@ -160,7 +111,7 @@ pub async fn consensus_init_task(
}
/// Async task to start the consensus task, while monitoring for a network disconnections.
async fn replicator_task(node: &DarkfiNodePtr, ex: &ExecutorPtr) -> Result<()> {
async fn listen_to_network(node: &DarkfiNodePtr, ex: &ExecutorPtr) -> Result<()> {
// Grab proposals subscriber and subscribe to it
let proposals_sub = node.subscribers.get("proposals").unwrap();
let prop_subscription = proposals_sub.publisher.clone().subscribe().await;
@@ -225,8 +176,8 @@ async fn consensus_task(
continue
}
if let Err(e) = clean_mm_blocktemplates(node).await {
error!(target: "darkfid", "Failed cleaning merge mining block templates: {e}")
if let Err(e) = clean_blocktemplates(node).await {
error!(target: "darkfid", "Failed cleaning mining block templates: {e}")
}
let mut notif_blocks = Vec::with_capacity(confirmed.len());
@@ -253,14 +204,15 @@ async fn consensus_task(
}
}
/// Auxiliary function to drop merge mining block templates not
/// referencing active forks or last confirmed block.
pub async fn clean_mm_blocktemplates(node: &DarkfiNodePtr) -> Result<()> {
// Grab a lock over node merge mining templates
/// Auxiliary function to drop mining block templates not referencing
/// active forks or last confirmed block.
async fn clean_blocktemplates(node: &DarkfiNodePtr) -> Result<()> {
// Grab a lock over node mining templates
let mut blocktemplates = node.blocktemplates.lock().await;
let mut mm_blocktemplates = node.mm_blocktemplates.lock().await;
// Early return if no merge mining block templates exist
if mm_blocktemplates.is_empty() {
// Early return if no mining block templates exist
if blocktemplates.is_empty() && mm_blocktemplates.is_empty() {
return Ok(())
}
@@ -272,6 +224,34 @@ pub async fn clean_mm_blocktemplates(node: &DarkfiNodePtr) -> Result<()> {
// Loop through templates to find which can be dropped
let mut dropped_templates = vec![];
'outer: for (key, blocktemplate) in blocktemplates.iter() {
// Loop through all the forks
for fork in forks.iter() {
// Traverse fork proposals sequence in reverse
for p_hash in fork.proposals.iter().rev() {
// Check if job extends this fork
if &blocktemplate.block.header.previous == p_hash {
continue 'outer
}
}
}
// Check if it extends last confirmed block
if blocktemplate.block.header.previous == last_confirmed {
continue
}
// This job doesn't reference something so we drop it
dropped_templates.push(key.clone());
}
// Drop jobs not referencing active forks or last confirmed block
for key in dropped_templates {
blocktemplates.remove(&key);
}
// Loop through merge mining templates to find which can be dropped
let mut dropped_templates = vec![];
'outer: for (key, (block, _, _)) in mm_blocktemplates.iter() {
// Loop through all the forks
for fork in forks.iter() {

View File

@@ -1,428 +0,0 @@
/* This file is part of DarkFi (https://dark.fi)
*
* Copyright (C) 2020-2025 Dyne.org foundation
*
* This program is free software: you can redistribute it and/or modify
* it under the terms of the GNU Affero 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 Affero General Public License for more details.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <https://www.gnu.org/licenses/>.
*/
use darkfi::{
blockchain::{BlockInfo, Header, HeaderHash},
rpc::{jsonrpc::JsonNotification, util::JsonValue},
system::{ExecutorPtr, StoppableTask, Subscription},
tx::{ContractCallLeaf, Transaction, TransactionBuilder},
util::{encoding::base64, time::Timestamp},
validator::{
consensus::{Fork, Proposal},
pow::{RANDOMX_KEY_CHANGE_DELAY, RANDOMX_KEY_CHANGING_HEIGHT},
utils::best_fork_index,
verification::apply_producer_transaction,
},
zk::ProvingKey,
zkas::ZkBinary,
Error, Result,
};
use darkfi_money_contract::{client::pow_reward_v1::PoWRewardCallBuilder, MoneyFunction};
use darkfi_sdk::{
crypto::{FuncId, Keypair, MerkleTree, PublicKey, SecretKey, MONEY_CONTRACT_ID},
pasta::pallas,
ContractCall,
};
use darkfi_serial::{serialize_async, Encodable};
use num_bigint::BigUint;
use rand::rngs::OsRng;
use smol::channel::{Receiver, Sender};
use tracing::{error, info};
use crate::{
proto::ProposalMessage,
task::{consensus::clean_mm_blocktemplates, garbage_collect_task},
DarkfiNodePtr,
};
/// Auxiliary structure representing node miner rewards recipient configuration
pub struct MinerRewardsRecipientConfig {
pub recipient: PublicKey,
pub spend_hook: Option<FuncId>,
pub user_data: Option<pallas::Base>,
}
/// Async task used for participating in the PoW block production.
///
/// Miner initializes their setup and waits for next confirmation,
/// by listening for new proposals from the network, for optimal
/// conditions. After confirmation occurs, they start the actual
/// miner loop, where they first grab the best ranking fork to extend,
/// and start mining procedure for its next block. Additionally, they
/// listen to the network for new proposals, and check if these
/// proposals produce a new best ranking fork. If they do, the stop
/// mining. These two tasks run in parallel, and after one of them
/// finishes, node triggers confirmation check.
pub async fn miner_task(
node: &DarkfiNodePtr,
recipient_config: &MinerRewardsRecipientConfig,
skip_sync: bool,
ex: &ExecutorPtr,
) -> Result<()> {
// Initialize miner configuration
info!(target: "darkfid::task::miner_task", "Starting miner task...");
// Grab blocks subscriber
let block_sub = node.subscribers.get("blocks").unwrap();
// Grab proposals subscriber and subscribe to it
let proposals_sub = node.subscribers.get("proposals").unwrap();
let subscription = proposals_sub.publisher.clone().subscribe().await;
// Create channels so threads can signal each other
let (sender, stop_signal) = smol::channel::bounded(1);
// Create the garbage collection task using a dummy task
let gc_task = StoppableTask::new();
gc_task.clone().start(
async { Ok(()) },
|_| async { /* Do nothing */ },
Error::GarbageCollectionTaskStopped,
ex.clone(),
);
info!(target: "darkfid::task::miner_task", "Miner initialized successfully!");
// Start miner loop
loop {
// Grab best current fork
let extended_fork = match node.best_current_fork().await {
Ok(f) => f,
Err(e) => {
error!(
target: "darkfid::task::miner_task",
"Finding best fork index failed: {e}",
);
continue
}
};
// Grab extended fork last proposal hash
let last_proposal_hash = extended_fork.last_proposal()?.hash;
// Grab zkas proving keys and bin for PoWReward transaction
let zkbin = &node.powrewardv1_zk.zkbin;
let pk = &node.powrewardv1_zk.provingkey;
// Start listenning for network proposals and mining next block for best fork.
match smol::future::or(
listen_to_network(node, last_proposal_hash, &subscription, &sender),
mine(node, extended_fork, recipient_config, zkbin, pk, &stop_signal, skip_sync),
)
.await
{
Ok(_) => { /* Do nothing */ }
Err(Error::NetworkNotConnected) => {
error!(target: "darkfid::task::miner_task", "Node disconnected from the network");
subscription.unsubscribe().await;
return Err(Error::NetworkNotConnected)
}
Err(e) => {
error!(
target: "darkfid::task::miner_task",
"Error during listen_to_network() or mine(): {e}"
);
continue
}
}
// Check if we can confirm anything and broadcast them
let confirmed = match node.validator.confirmation().await {
Ok(f) => f,
Err(e) => {
error!(
target: "darkfid::task::miner_task",
"Confirmation failed: {e}"
);
continue
}
};
if confirmed.is_empty() {
continue
}
if let Err(e) = clean_mm_blocktemplates(node).await {
error!(target: "darkfid", "Failed cleaning merge mining block templates: {e}")
}
let mut notif_blocks = Vec::with_capacity(confirmed.len());
for block in confirmed {
notif_blocks.push(JsonValue::String(base64::encode(&serialize_async(&block).await)));
}
block_sub.notify(JsonValue::Array(notif_blocks)).await;
// Invoke the detached garbage collection task
gc_task.clone().stop().await;
gc_task.clone().start(
garbage_collect_task(node.clone()),
|res| async {
match res {
Ok(()) | Err(Error::GarbageCollectionTaskStopped) => { /* Do nothing */ }
Err(e) => {
error!(target: "darkfid", "Failed starting garbage collection task: {e}")
}
}
},
Error::GarbageCollectionTaskStopped,
ex.clone(),
);
}
}
/// Async task to listen for incoming proposals and check if the best fork has changed.
async fn listen_to_network(
node: &DarkfiNodePtr,
last_proposal_hash: HeaderHash,
subscription: &Subscription<JsonNotification>,
sender: &Sender<()>,
) -> Result<()> {
loop {
// Wait until a new proposal has been received
subscription.receive().await;
// Grab a lock over node forks
let forks = node.validator.consensus.forks.read().await;
// Grab best current fork index
let index = best_fork_index(&forks)?;
// Verify if proposals sequence has changed
if forks[index].last_proposal()?.hash != last_proposal_hash {
drop(forks);
break
}
drop(forks);
}
// Signal miner to abort mining
sender.send(()).await?;
if let Err(e) = node.miner_daemon_request("abort", &JsonValue::Array(vec![])).await {
error!(target: "darkfid::task::miner::listen_to_network", "Failed to execute miner daemon abort request: {e}");
}
Ok(())
}
/// Async task to generate and mine provided fork index next block,
/// while listening for a stop signal.
#[allow(clippy::too_many_arguments)]
async fn mine(
node: &DarkfiNodePtr,
extended_fork: Fork,
recipient_config: &MinerRewardsRecipientConfig,
zkbin: &ZkBinary,
pk: &ProvingKey,
stop_signal: &Receiver<()>,
skip_sync: bool,
) -> Result<()> {
smol::future::or(
wait_stop_signal(stop_signal),
mine_next_block(node, extended_fork, recipient_config, zkbin, pk, skip_sync),
)
.await
}
/// Async task to wait for listener's stop signal.
pub async fn wait_stop_signal(stop_signal: &Receiver<()>) -> Result<()> {
// Clean stop signal channel
if stop_signal.is_full() {
stop_signal.recv().await?;
}
// Wait for listener signal
stop_signal.recv().await?;
Ok(())
}
/// Async task to generate and mine provided fork index next block.
async fn mine_next_block(
node: &DarkfiNodePtr,
mut extended_fork: Fork,
recipient_config: &MinerRewardsRecipientConfig,
zkbin: &ZkBinary,
pk: &ProvingKey,
skip_sync: bool,
) -> Result<()> {
// Grab next target and block
let (next_target, mut next_block, block_signing_secret) = generate_next_block(
&mut extended_fork,
recipient_config,
zkbin,
pk,
node.validator.consensus.module.read().await.target,
node.validator.verify_fees,
)
.await?;
// Execute request to minerd and parse response
let target = JsonValue::String(base64::encode(&next_target.to_bytes_le()));
// Grab the RandomX key to use.
// We only use the next key when the next block is the
// height changing one.
let randomx_key = if next_block.header.height > RANDOMX_KEY_CHANGING_HEIGHT &&
next_block.header.height % RANDOMX_KEY_CHANGING_HEIGHT == RANDOMX_KEY_CHANGE_DELAY
{
JsonValue::String(base64::encode(
&serialize_async(&extended_fork.module.darkfi_rx_keys.1).await,
))
} else {
JsonValue::String(base64::encode(
&serialize_async(&extended_fork.module.darkfi_rx_keys.0).await,
))
};
let header = JsonValue::String(base64::encode(&serialize_async(&next_block.header).await));
let response = node
.miner_daemon_request_with_retry(
"mine",
&JsonValue::Array(vec![target, randomx_key, header]),
)
.await;
next_block.header.nonce = *response.get::<f64>().unwrap() as u64;
// Sign the mined block
next_block.sign(&block_signing_secret);
// Verify it
extended_fork.module.verify_current_block(&next_block.header)?;
// Check if we are connected to the network
if !skip_sync && !node.p2p_handler.p2p.is_connected() {
return Err(Error::NetworkNotConnected)
}
// Append the mined block as a proposal
let proposal = Proposal::new(next_block);
node.validator.append_proposal(&proposal).await?;
// Broadcast proposal to the network
let message = ProposalMessage(proposal);
node.p2p_handler.p2p.broadcast(&message).await;
Ok(())
}
/// Auxiliary function to generate next block in an atomic manner.
pub async fn generate_next_block(
extended_fork: &mut Fork,
recipient_config: &MinerRewardsRecipientConfig,
zkbin: &ZkBinary,
pk: &ProvingKey,
block_target: u32,
verify_fees: bool,
) -> Result<(BigUint, BlockInfo, SecretKey)> {
// Grab forks' last block proposal(previous)
let last_proposal = extended_fork.last_proposal()?;
// Grab forks' next block height
let next_block_height = last_proposal.block.header.height + 1;
// Grab forks' unproposed transactions
let (mut txs, _, fees, overlay) = extended_fork
.unproposed_txs(&extended_fork.blockchain, next_block_height, block_target, verify_fees)
.await?;
// Create an ephemeral block signing keypair. Its secret key will
// be stored in the PowReward transaction's encrypted note for
// later retrieval. It is encrypted towards the recipient's public
// key.
let block_signing_keypair = Keypair::random(&mut OsRng);
// Generate reward transaction
let tx = generate_transaction(
next_block_height,
fees,
&block_signing_keypair,
recipient_config,
zkbin,
pk,
)?;
// Apply producer transaction in the overlay
let _ = apply_producer_transaction(
&overlay,
next_block_height,
block_target,
&tx,
&mut MerkleTree::new(1),
)
.await?;
txs.push(tx);
// Grab the updated contracts states root
overlay.lock().unwrap().contracts.update_state_monotree(&mut extended_fork.state_monotree)?;
let Some(state_root) = extended_fork.state_monotree.get_headroot()? else {
return Err(Error::ContractsStatesRootNotFoundError);
};
// Drop new trees opened by the unproposed transactions overlay
overlay.lock().unwrap().overlay.lock().unwrap().purge_new_trees()?;
// Generate the new header
let mut header =
Header::new(last_proposal.hash, next_block_height, Timestamp::current_time(), 0);
header.state_root = state_root;
// Generate the block
let mut next_block = BlockInfo::new_empty(header);
// Add transactions to the block
next_block.append_txs(txs);
// Grab the next mine target
let target = extended_fork.module.next_mine_target()?;
Ok((target, next_block, block_signing_keypair.secret))
}
/// Auxiliary function to generate a Money::PoWReward transaction.
fn generate_transaction(
block_height: u32,
fees: u64,
block_signing_keypair: &Keypair,
recipient_config: &MinerRewardsRecipientConfig,
zkbin: &ZkBinary,
pk: &ProvingKey,
) -> Result<Transaction> {
// Build the transaction debris
let debris = PoWRewardCallBuilder {
signature_keypair: *block_signing_keypair,
block_height,
fees,
recipient: Some(recipient_config.recipient),
spend_hook: recipient_config.spend_hook,
user_data: recipient_config.user_data,
mint_zkbin: zkbin.clone(),
mint_pk: pk.clone(),
}
.build()?;
// Generate and sign the actual transaction
let mut data = vec![MoneyFunction::PoWRewardV1 as u8];
debris.params.encode(&mut data)?;
let call = ContractCall { contract_id: *MONEY_CONTRACT_ID, data };
let mut tx_builder =
TransactionBuilder::new(ContractCallLeaf { call, proofs: debris.proofs }, vec![])?;
let mut tx = tx_builder.build()?;
let sigs = tx.create_sigs(&[block_signing_keypair.secret])?;
tx.signatures = vec![sigs];
Ok(tx)
}

View File

@@ -19,9 +19,6 @@
pub mod consensus;
pub use consensus::consensus_init_task;
pub mod miner;
pub use miner::{generate_next_block, miner_task};
pub mod sync;
pub use sync::sync_task;

View File

@@ -289,8 +289,7 @@ pub async fn generate_node(
let p2p_handler = DarkfidP2pHandler::init(settings, ex).await?;
let node =
DarkfiNode::new(p2p_handler.clone(), validator.clone(), 50, subscribers.clone(), None)
.await?;
DarkfiNode::new(p2p_handler.clone(), validator.clone(), 50, subscribers.clone()).await?;
p2p_handler.clone().start(ex, &validator, &subscribers).await?;

View File

@@ -279,10 +279,6 @@ fn darkfid_programmatic_control() -> Result<()> {
skip_sync: true,
checkpoint_height: None,
checkpoint: None,
miner: false,
recipient: None,
spend_hook: None,
user_data: None,
bootstrap,
};
let rpc_settings = RpcSettings {
@@ -296,7 +292,6 @@ fn darkfid_programmatic_control() -> Result<()> {
&config,
&darkfi::net::Settings::default(),
&None,
&None,
&ex,
)
.await

View File

@@ -10,11 +10,12 @@ edition = "2021"
[dependencies]
# Darkfi
darkfi = {path = "../../", features = ["async-daemonize", "validator", "rpc"]}
darkfi = {path = "../../", features = ["async-daemonize", "validator", "rpc", "bs58"]}
darkfi-sdk = {path = "../../src/sdk"}
darkfi-serial = {version = "0.5.0", features = ["async"]}
# Misc
bs58 = "0.5.1"
tracing = "0.1.41"
num-bigint = "0.4.6"
@@ -34,6 +35,7 @@ smol = "2.0.2"
serde = {version = "1.0.228", features = ["derive"]}
structopt = "0.3.26"
structopt-toml = "0.5.1"
toml = "0.9.8"
[lints]
workspace = true

View File

@@ -9,10 +9,56 @@
# PoW miner number of threads to use
#threads = 4
# JSON-RPC settings
[rpc]
# JSON-RPC listen URL
rpc_listen = "tcp://127.0.0.1:28467"
# Polling rate to ask darkfid for mining jobs
#polling_rate = 2
# Disabled RPC methods
#rpc_disabled_methods = []
# Stop mining at given height (0 mines forever)
#stop_at_height = 0
# Blockchain network to use
network = "testnet"
# Localnet blockchain network configuration
[network_config."localnet"]
# Wallet mining address to receive mining rewards
recipient = "YOUR_WALLET_ADDRESS_HERE"
# Optional contract spend hook to use in the mining reward
#spend_hook = "YOUR_SPEND_HOOK_HERE"
# Optional contract user data to use in the mining reward.
# This is not arbitrary data.
#user_data = "YOUR_USER_DATA_HERE"
# darkfid JSON-RPC endpoint
endpoint = "tcp://127.0.0.1:8240"
# Testnet blockchain network configuration
[network_config."testnet"]
# Wallet mining address to receive mining rewards
recipient = "YOUR_WALLET_ADDRESS_HERE"
# Optional contract spend hook to use in the mining reward
#spend_hook = "YOUR_SPEND_HOOK_HERE"
# Optional contract user data to use in the mining reward.
# This is not arbitrary data.
#user_data = "YOUR_USER_DATA_HERE"
# darkfid JSON-RPC endpoint
endpoint = "tcp://127.0.0.1:8340"
# Mainnet blockchain network configuration
[network_config."mainnet"]
# Wallet mining address to receive mining rewards
recipient = "YOUR_WALLET_ADDRESS_HERE"
# Optional contract spend hook to use in the mining reward
#spend_hook = "YOUR_SPEND_HOOK_HERE"
# Optional contract user data to use in the mining reward.
# This is not arbitrary data.
#user_data = "YOUR_USER_DATA_HERE"
# darkfid JSON-RPC endpoint
endpoint = "tcp://127.0.0.1:8440"

View File

@@ -1,54 +0,0 @@
/* This file is part of DarkFi (https://dark.fi)
*
* Copyright (C) 2020-2025 Dyne.org foundation
*
* This program is free software: you can redistribute it and/or modify
* it under the terms of the GNU Affero 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 Affero General Public License for more details.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <https://www.gnu.org/licenses/>.
*/
use darkfi::rpc::jsonrpc::{ErrorCode::ServerError, JsonError, JsonResult};
/// Custom RPC errors available for minerd.
/// Please sort them sensefully.
pub enum RpcError {
// Parsing errors
TargetParseError = -32101,
BlockParseError = -32102,
// Miner errors
MiningFailed = -32201,
StopFailed = -32202,
}
fn to_tuple(e: RpcError) -> (i32, String) {
let msg = match e {
// Parsing errors
RpcError::TargetParseError => "Target parse error",
RpcError::BlockParseError => "Block parse error",
// Miner errors
RpcError::MiningFailed => "Mining block failed",
RpcError::StopFailed => "Failed to stop previous request",
};
(e as i32, msg.to_string())
}
pub fn server_error(e: RpcError, id: u16, msg: Option<&str>) -> JsonResult {
let (code, default_msg) = to_tuple(e);
if let Some(message) = msg {
return JsonError::new(ServerError(code), Some(message.to_string()), id).into()
}
JsonError::new(ServerError(code), Some(default_msg), id).into()
}

View File

@@ -16,60 +16,116 @@
* along with this program. If not, see <https://www.gnu.org/licenses/>.
*/
use std::{collections::HashSet, sync::Arc};
use std::{collections::HashMap, sync::Arc};
use smol::{
channel::{Receiver, Sender},
lock::Mutex,
lock::RwLock,
};
use tracing::{error, info};
use tracing::{debug, error, info};
use url::Url;
use darkfi::{
rpc::{
server::{listen_and_serve, RequestHandler},
settings::RpcSettings,
},
rpc::util::JsonValue,
system::{sleep, ExecutorPtr, StoppableTask, StoppableTaskPtr},
Error, Result,
Error,
};
use darkfi_sdk::crypto::Keypair;
/// Daemon error codes
mod error;
/// JSON-RPC server methods
/// darkfid JSON-RPC related methods
mod rpc;
use rpc::{polling_task, DarkfidRpcClient};
/// Auxiliary structure representing miner node configuration.
pub struct MinerNodeConfig {
/// PoW miner number of threads to use
threads: usize,
/// Polling rate to ask darkfid for mining jobs
polling_rate: u64,
/// Stop mining at this height (0 mines forever)
stop_at_height: u32,
/// Wallet mining configuration to receive mining rewards
wallet_config: HashMap<String, JsonValue>,
}
impl Default for MinerNodeConfig {
fn default() -> Self {
Self::new(
1,
5,
0,
HashMap::from([(
String::from("recipient"),
JsonValue::String(Keypair::default().public.to_string()),
)]),
)
}
}
impl MinerNodeConfig {
pub fn new(
threads: usize,
polling_rate: u64,
stop_at_height: u32,
wallet_config: HashMap<String, JsonValue>,
) -> Self {
Self { threads, polling_rate, stop_at_height, wallet_config }
}
}
/// Atomic pointer to the DarkFi mining node
pub type MinerNodePtr = Arc<MinerNode>;
/// Structure representing a DarkFi mining node
pub struct MinerNode {
/// PoW miner number of threads to use
threads: usize,
/// Stop mining at this height
stop_at_height: u32,
/// Node configuration
config: MinerNodeConfig,
/// Sender to stop miner threads
sender: Sender<()>,
/// Receiver to stop miner threads
stop_signal: Receiver<()>,
/// JSON-RPC connection tracker
rpc_connections: Mutex<HashSet<StoppableTaskPtr>>,
/// JSON-RPC client to execute requests to darkfid daemon
rpc_client: RwLock<DarkfidRpcClient>,
}
impl MinerNode {
pub fn new(
threads: usize,
stop_at_height: u32,
sender: Sender<()>,
stop_signal: Receiver<()>,
) -> MinerNodePtr {
Arc::new(Self {
threads,
stop_at_height,
sender,
stop_signal,
rpc_connections: Mutex::new(HashSet::new()),
})
pub async fn new(config: MinerNodeConfig, endpoint: Url, ex: &ExecutorPtr) -> MinerNodePtr {
// Initialize the smol channels to send signal between the threads
let (sender, stop_signal) = smol::channel::bounded(1);
// Initialize JSON-RPC client
let rpc_client = RwLock::new(DarkfidRpcClient::new(endpoint, ex.clone()).await);
Arc::new(Self { config, sender, stop_signal, rpc_client })
}
/// Auxiliary function to abort pending job.
pub async fn abort_pending(&self) {
// Check if a pending request is being processed
debug!(target: "minerd::abort_pending", "Checking if a pending job is being processed...");
if self.stop_signal.receiver_count() <= 1 {
debug!(target: "minerd::rpc", "No pending job!");
return
}
info!(target: "minerd::abort_pending", "Pending job is in progress, sending stop signal...");
// Send stop signal to worker
if let Err(e) = self.sender.try_send(()) {
error!(target: "minerd::abort_pending", "Failed to stop pending job: {e}");
return
}
// Wait for worker to terminate
info!(target: "minerd::abort_pending", "Waiting for job to terminate...");
while self.stop_signal.receiver_count() > 1 {
sleep(1).await;
}
info!(target: "minerd::abort_pending", "Pending job terminated!");
// Consume channel item so its empty again
if let Err(e) = self.stop_signal.try_recv() {
error!(target: "minerd::abort_pending", "Failed to cleanup stop signal channel: {e}");
}
}
}
@@ -80,77 +136,68 @@ pub type MinerdPtr = Arc<Minerd>;
pub struct Minerd {
/// Miner node instance conducting the mining operations
node: MinerNodePtr,
/// JSON-RPC background task
rpc_task: StoppableTaskPtr,
/// Miner darkfid polling background task
polling_task: StoppableTaskPtr,
}
impl Minerd {
/// Initialize a DarkFi mining daemon.
///
/// Corresponding communication channels are setup to generate a new `MinerNode`,
/// and a new task is generated to handle the JSON-RPC API.
pub fn init(threads: usize, stop_at_height: u32) -> MinerdPtr {
/// Generate a new `MinerNode` and a new task to handle the darkfid
/// polling.
pub async fn init(config: MinerNodeConfig, endpoint: Url, ex: &ExecutorPtr) -> MinerdPtr {
info!(target: "minerd::Minerd::init", "Initializing a new mining daemon...");
// Initialize the smol channels to send signal between the threads
let (sender, stop_signal) = smol::channel::bounded(1);
// Generate the node
let node = MinerNode::new(threads, stop_at_height, sender, stop_signal);
let node = MinerNode::new(config, endpoint, ex).await;
// Generate the JSON-RPC task
let rpc_task = StoppableTask::new();
// Generate the polling task
let polling_task = StoppableTask::new();
info!(target: "minerd::Minerd::init", "Mining daemon initialized successfully!");
Arc::new(Self { node, rpc_task })
Arc::new(Self { node, polling_task })
}
/// Start the DarkFi mining daemon in the given executor, using the provided JSON-RPC listen url.
pub fn start(&self, executor: &ExecutorPtr, rpc_settings: &RpcSettings) {
/// Start the DarkFi mining daemon in the given executor.
pub fn start(&self, ex: &ExecutorPtr) {
info!(target: "minerd::Minerd::start", "Starting mining daemon...");
// Start the JSON-RPC task
let node_ = self.node.clone();
self.rpc_task.clone().start(
listen_and_serve(rpc_settings.clone(), self.node.clone(), None, executor.clone()),
|res| async move {
// Start the polling task
self.polling_task.clone().start(
polling_task(self.node.clone(), ex.clone()),
|res| async {
match res {
Ok(()) | Err(Error::RpcServerStopped) => node_.stop_connections().await,
Err(e) => error!(target: "minerd::Minerd::start", "Failed starting JSON-RPC server: {e}"),
Ok(()) | Err(Error::DetachedTaskStopped) => { /* Do nothing */ }
Err(e) => {
error!(target: "minerd::Minerd::start", "Failed starting polling task: {e}")
}
}
},
Error::RpcServerStopped,
executor.clone(),
Error::DetachedTaskStopped,
ex.clone(),
);
info!(target: "minerd::Minerd::start", "Mining daemon started successfully!");
}
/// Stop the DarkFi mining daemon.
pub async fn stop(&self) -> Result<()> {
pub async fn stop(&self) {
info!(target: "minerd::Minerd::stop", "Terminating mining daemon...");
// Stop the polling task
info!(target: "minerd::Minerd::stop", "Stopping polling task...");
self.polling_task.stop().await;
// Stop the mining node
info!(target: "minerd::Minerd::stop", "Stopping miner threads...");
if self.node.stop_signal.is_empty() {
self.node.sender.send(()).await?;
}
while self.node.stop_signal.receiver_count() > 1 {
sleep(1).await;
}
self.node.abort_pending().await;
// Stop the JSON-RPC task
info!(target: "minerd::Minerd::stop", "Stopping JSON-RPC server...");
self.rpc_task.stop().await;
// Consume channel item so its empty again
if self.node.stop_signal.is_full() {
self.node.stop_signal.recv().await?;
}
// Close the JSON-RPC client
info!(target: "minerd::Minerd::stop", "Stopping JSON-RPC client...");
self.node.stop_rpc_client().await;
info!(target: "minerd::Minerd::stop", "Mining daemon terminated successfully!");
Ok(())
}
}
@@ -158,7 +205,6 @@ impl Minerd {
use {
darkfi::util::logger::{setup_test_logger, Level},
tracing::warn,
url::Url,
};
#[test]
@@ -166,7 +212,7 @@ use {
///
/// First we initialize a daemon, start it and then perform
/// couple of restarts to verify everything works as expected.
fn minerd_programmatic_control() -> Result<()> {
fn minerd_programmatic_control() {
// We check this error so we can execute same file tests in parallel,
// otherwise second one fails to init logger here.
if setup_test_logger(
@@ -182,74 +228,36 @@ fn minerd_programmatic_control() -> Result<()> {
warn!(target: "minerd_programmatic_control", "Logger already initialized");
}
// Daemon configuration
let threads = 4;
let rpc_settings =
RpcSettings { listen: Url::parse("tcp://127.0.0.1:28467")?, ..RpcSettings::default() };
// Create an executor and communication signals
let ex = Arc::new(smol::Executor::new());
let (signal, shutdown) = smol::channel::unbounded::<()>();
// Generate a dummy mining job
let target = darkfi::rpc::util::JsonValue::String(
num_bigint::BigUint::from_bytes_le(&[0xFF; 32]).to_string(),
);
let block = darkfi::rpc::util::JsonValue::String(darkfi::util::encoding::base64::encode(
&darkfi_serial::serialize(&darkfi::blockchain::BlockInfo::default()),
));
let mining_job = darkfi::rpc::jsonrpc::JsonRequest::new(
"mine",
darkfi::rpc::util::JsonValue::Array(vec![target, block]),
);
easy_parallel::Parallel::new()
.each(0..threads, |_| smol::block_on(ex.run(shutdown.recv())))
.finish(|| {
easy_parallel::Parallel::new().each(0..1, |_| smol::block_on(ex.run(shutdown.recv()))).finish(
|| {
smol::block_on(async {
// Initialize a daemon
let daemon = Minerd::init(threads, 0);
let daemon = Minerd::init(
MinerNodeConfig::default(),
Url::parse("tcp://127.0.0.1:12345").unwrap(),
&ex,
)
.await;
// Start it
daemon.start(&ex, &rpc_settings);
// Generate a JSON-RPC client to send mining jobs
let mut rpc_client =
darkfi::rpc::client::RpcClient::new(rpc_settings.listen.clone(), ex.clone())
.await;
while rpc_client.is_err() {
rpc_client = darkfi::rpc::client::RpcClient::new(
rpc_settings.listen.clone(),
ex.clone(),
)
.await;
}
let rpc_client = rpc_client.unwrap();
// Send a mining job but stop the daemon after it starts mining
smol::future::or(
async {
let _ = rpc_client.request(mining_job).await;
},
async {
// Wait node to start mining
darkfi::system::sleep(2).await;
daemon.stop().await.unwrap();
},
)
.await;
rpc_client.stop().await;
// Start it again
daemon.start(&ex, &rpc_settings);
daemon.start(&ex);
// Stop it
daemon.stop().await.unwrap();
daemon.stop().await;
// Start it again
daemon.start(&ex);
// Stop it
daemon.stop().await;
// Shutdown entirely
drop(signal);
})
});
Ok(())
},
);
}

View File

@@ -16,15 +16,23 @@
* along with this program. If not, see <https://www.gnu.org/licenses/>.
*/
use std::sync::Arc;
use std::{collections::HashMap, str::FromStr};
use smol::{stream::StreamExt, Executor};
use smol::{fs::read_to_string, stream::StreamExt};
use structopt_toml::{serde::Deserialize, structopt::StructOpt, StructOptToml};
use tracing::info;
use tracing::{debug, error, info};
use url::Url;
use darkfi::{async_daemonize, cli_desc, rpc::settings::RpcSettingsOpt, Error, Result};
use darkfi::{
async_daemonize, cli_desc, rpc::util::JsonValue, system::ExecutorPtr,
util::path::get_config_path, Error, Result,
};
use darkfi_sdk::{
crypto::{pasta_prelude::PrimeField, FuncId, PublicKey},
pasta::pallas,
};
use minerd::Minerd;
use minerd::{MinerNodeConfig, Minerd};
const CONFIG_FILE: &str = "minerd.toml";
const CONFIG_FILE_CONTENTS: &str = include_str!("../minerd.toml");
@@ -37,18 +45,22 @@ struct Args {
/// Configuration file to use
config: Option<String>,
#[structopt(flatten)]
/// JSON-RPC settings
rpc: RpcSettingsOpt,
#[structopt(short, long, default_value = "4")]
/// PoW miner number of threads to use
threads: usize,
#[structopt(short, long, default_value = "2")]
/// Polling rate to ask darkfid for mining jobs
polling_rate: u64,
#[structopt(long, default_value = "0")]
/// Refuse mining at given height (0 mines forever)
/// Stop mining at given height (0 mines forever)
stop_at_height: u32,
#[structopt(short, long, default_value = "testnet")]
/// Blockchain network to use
network: String,
#[structopt(short, long)]
/// Set log file to ouput into
log: Option<String>,
@@ -58,18 +70,131 @@ struct Args {
verbose: u8,
}
#[derive(Clone, Debug, Deserialize, StructOpt, StructOptToml)]
#[structopt()]
/// Defines a blockchain network configuration.
/// Default values correspond to a local network.
pub struct BlockchainNetwork {
#[structopt(short, long, default_value = "tcp://127.0.0.1:8240")]
/// darkfid JSON-RPC endpoint
endpoint: Url,
#[structopt(long, default_value = "")]
/// Wallet mining address to receive mining rewards
recipient: String,
#[structopt(long)]
/// Optional contract spend hook to use in the mining reward
spend_hook: Option<String>,
#[structopt(long)]
/// Optional contract user data to use in the mining reward.
/// This is not arbitrary data.
user_data: Option<String>,
}
/// Auxiliary function to parse minerd configuration file and extract
/// requested blockchain network config.
pub async fn parse_blockchain_config(
config: Option<String>,
network: &str,
) -> Result<BlockchainNetwork> {
// Grab config path
let config_path = get_config_path(config, CONFIG_FILE)?;
debug!(target: "minerd", "Parsing configuration file: {config_path:?}");
// Parse TOML file contents
let contents = read_to_string(&config_path).await?;
let contents: toml::Value = match toml::from_str(&contents) {
Ok(v) => v,
Err(e) => {
error!(target: "minerd", "Failed parsing TOML config: {e}");
return Err(Error::ParseFailed("Failed parsing TOML config"))
}
};
// Grab requested network config
let Some(table) = contents.as_table() else { return Err(Error::ParseFailed("TOML not a map")) };
let Some(network_configs) = table.get("network_config") else {
return Err(Error::ParseFailed("TOML does not contain network configurations"))
};
let Some(network_configs) = network_configs.as_table() else {
return Err(Error::ParseFailed("`network_config` not a map"))
};
let Some(network_config) = network_configs.get(network) else {
return Err(Error::ParseFailed("TOML does not contain requested network configuration"))
};
let network_config = toml::to_string(&network_config).unwrap();
let network_config =
match BlockchainNetwork::from_iter_with_toml::<Vec<String>>(&network_config, vec![]) {
Ok(v) => v,
Err(e) => {
error!(target: "minerd", "Failed parsing requested network configuration: {e}");
return Err(Error::ParseFailed("Failed parsing requested network configuration"))
}
};
debug!(target: "minerd", "Parsed network configuration: {network_config:?}");
Ok(network_config)
}
async_daemonize!(realmain);
async fn realmain(args: Args, ex: Arc<Executor<'static>>) -> Result<()> {
async fn realmain(args: Args, ex: ExecutorPtr) -> Result<()> {
info!(target: "minerd", "Starting DarkFi Mining Daemon...");
let daemon = Minerd::init(args.threads, args.stop_at_height);
daemon.start(&ex, &args.rpc.into());
// Grab blockchain network configuration
let blockchain_config = match args.network.as_str() {
"localnet" => parse_blockchain_config(args.config, "localnet").await?,
"testnet" => parse_blockchain_config(args.config, "testnet").await?,
"mainnet" => parse_blockchain_config(args.config, "mainnet").await?,
_ => {
error!(target: "minerd", "Unsupported chain `{}`", args.network);
return Err(Error::UnsupportedChain)
}
};
debug!(target: "minerd", "Blockchain config: {blockchain_config:?}");
// Parse the network wallet configuration
if PublicKey::from_str(&blockchain_config.recipient).is_err() {
return Err(Error::InvalidAddress)
}
let mut wallet_config = HashMap::from([(
String::from("recipient"),
JsonValue::String(blockchain_config.recipient),
)]);
if let Some(spend_hook) = &blockchain_config.spend_hook {
if FuncId::from_str(spend_hook).is_err() {
return Err(Error::ParseFailed("Invalid spend hook"))
}
wallet_config.insert(String::from("spend_hook"), JsonValue::String(spend_hook.to_string()));
}
if let Some(user_data_string) = &blockchain_config.user_data {
let bytes: [u8; 32] = match bs58::decode(&user_data_string).into_vec()?.try_into() {
Ok(b) => b,
Err(_) => return Err(Error::ParseFailed("Invalid user data")),
};
let user_data: Option<pallas::Base> = pallas::Base::from_repr(bytes).into();
if user_data.is_none() {
return Err(Error::ParseFailed("Invalid user data"))
}
wallet_config
.insert(String::from("user_data"), JsonValue::String(user_data_string.to_string()));
}
// Generate the daemon
let miner_config =
MinerNodeConfig::new(args.threads, args.polling_rate, args.stop_at_height, wallet_config);
let daemon = Minerd::init(miner_config, blockchain_config.endpoint, &ex).await;
daemon.start(&ex);
// Signal handling for graceful termination.
let (signals_handler, signals_task) = SignalHandler::new(ex)?;
signals_handler.wait_termination(signals_task).await?;
info!(target: "minerd", "Caught termination signal, cleaning up and exiting");
daemon.stop().await?;
daemon.stop().await;
info!(target: "minerd", "Shut down successfully");
Ok(())

View File

@@ -16,162 +16,266 @@
* along with this program. If not, see <https://www.gnu.org/licenses/>.
*/
use std::collections::HashSet;
use num_bigint::BigUint;
use smol::lock::MutexGuard;
use tracing::{debug, error, info};
use url::Url;
use darkfi::{
blockchain::header_store::{Header, HeaderHash},
rpc::{
jsonrpc::{ErrorCode, JsonError, JsonRequest, JsonResponse, JsonResult},
server::RequestHandler,
util::JsonValue,
},
system::{sleep, StoppableTaskPtr},
blockchain::{Header, HeaderHash},
rpc::{client::RpcClient, jsonrpc::JsonRequest, util::JsonValue},
system::{sleep, ExecutorPtr, StoppableTask},
util::encoding::base64,
validator::pow::mine_block,
Error, Result,
};
use darkfi_serial::{async_trait, deserialize_async};
use darkfi_serial::deserialize_async;
use crate::{
error::{server_error, RpcError},
MinerNode,
};
use crate::{MinerNode, MinerNodePtr};
#[async_trait]
impl RequestHandler<()> for MinerNode {
async fn handle_request(&self, req: JsonRequest) -> JsonResult {
debug!(target: "minerd::rpc", "--> {}", req.stringify().unwrap());
match req.method.as_str() {
"ping" => self.pong(req.id, req.params).await,
"abort" => self.abort(req.id, req.params).await,
"mine" => self.mine(req.id, req.params).await,
_ => JsonError::new(ErrorCode::MethodNotFound, None, req.id).into(),
}
/// Structure to hold a JSON-RPC client and its config,
/// so we can recreate it in case of an error.
pub struct DarkfidRpcClient {
endpoint: Url,
ex: ExecutorPtr,
client: Option<RpcClient>,
}
async fn connections_mut(&self) -> MutexGuard<'life0, HashSet<StoppableTaskPtr>> {
self.rpc_connections.lock().await
impl DarkfidRpcClient {
pub async fn new(endpoint: Url, ex: ExecutorPtr) -> Self {
let client = RpcClient::new(endpoint.clone(), ex.clone()).await.ok();
Self { endpoint, ex, client }
}
/// Stop the client.
pub async fn stop(&self) {
if let Some(ref client) = self.client {
client.stop().await
}
}
}
impl MinerNode {
// RPCAPI:
// Signals miner daemon to abort mining pending request.
// Returns `true` on success.
//
// --> {"jsonrpc": "2.0", "method": "abort", "params": [], "id": 42}
// <-- {"jsonrpc": "2.0", "result": "true", "id": 42}
async fn abort(&self, id: u16, _params: JsonValue) -> JsonResult {
if let Some(e) = self.abort_pending(id).await {
return e
};
JsonResponse::new(JsonValue::Boolean(true), id).into()
/// Auxiliary function to poll configured darkfid daemon for a new
/// mining job.
async fn poll(&self, header: &str) -> Result<(HeaderHash, BigUint, Header)> {
loop {
debug!(target: "minerd::rpc::poll", "Executing poll request to darkfid...");
let mut request_params = self.config.wallet_config.clone();
request_params.insert(String::from("header"), JsonValue::String(String::from(header)));
let params = match self
.darkfid_daemon_request("miner.get_header", &JsonValue::from(request_params))
.await
{
Ok(params) => params,
Err(e) => {
error!(target: "minerd::rpc::poll", "darkfid poll failed: {e}");
self.sleep().await?;
continue
}
};
debug!(target: "minerd::rpc::poll", "Got reply: {params:?}");
// RPCAPI:
// Mine provided block header for requested mine target, using
// provided RandomX VM key, and return the corresponding nonce
// value.
//
// --> {"jsonrpc": "2.0", "method": "mine", "params": ["target", "randomx_key", "header"], "id": 42}
// --> {"jsonrpc": "2.0", "result": "nonce", "id": 42}
async fn mine(&self, id: u16, params: JsonValue) -> JsonResult {
// Verify parameters
// Verify response parameters
if !params.is_array() {
return JsonError::new(ErrorCode::InvalidParams, None, id).into()
error!(target: "minerd::rpc::poll", "darkfid responded with invalid params: {params:?}");
self.sleep().await?;
continue
}
let params = params.get::<Vec<JsonValue>>().unwrap();
if params.is_empty() {
debug!(target: "minerd::rpc::poll", "darkfid response is empty");
self.sleep().await?;
continue
}
if params.len() != 3 ||
!params[0].is_string() ||
!params[1].is_string() ||
!params[2].is_string()
{
return JsonError::new(ErrorCode::InvalidParams, None, id).into()
error!(target: "minerd::rpc::poll", "darkfid responded with invalid params: {params:?}");
self.sleep().await?;
continue
}
// Parse parameters
let Some(target_bytes) = base64::decode(params[0].get::<String>().unwrap()) else {
error!(target: "minerd::rpc", "Failed to parse target bytes");
return server_error(RpcError::TargetParseError, id, None)
};
let target = BigUint::from_bytes_le(&target_bytes);
let Some(randomx_key_bytes) = base64::decode(params[1].get::<String>().unwrap()) else {
error!(target: "minerd::rpc", "Failed to parse RandomX key bytes");
return server_error(RpcError::BlockParseError, id, None)
let Some(randomx_key_bytes) = base64::decode(params[0].get::<String>().unwrap()) else {
error!(target: "minerd::rpc::poll", "Failed to parse RandomX key bytes");
self.sleep().await?;
continue
};
let Ok(randomx_key) = deserialize_async::<HeaderHash>(&randomx_key_bytes).await else {
error!(target: "minerd::rpc", "Failed to parse RandomX key");
return server_error(RpcError::BlockParseError, id, None)
error!(target: "minerd::rpc::poll", "Failed to parse RandomX key");
self.sleep().await?;
continue
};
let Some(target_bytes) = base64::decode(params[1].get::<String>().unwrap()) else {
error!(target: "minerd::rpc::poll", "Failed to parse target bytes");
self.sleep().await?;
continue
};
let target = BigUint::from_bytes_le(&target_bytes);
let Some(header_bytes) = base64::decode(params[2].get::<String>().unwrap()) else {
error!(target: "minerd::rpc", "Failed to parse header bytes");
return server_error(RpcError::BlockParseError, id, None)
error!(target: "minerd::rpc::poll", "Failed to parse header bytes");
self.sleep().await?;
continue
};
let Ok(mut header) = deserialize_async::<Header>(&header_bytes).await else {
error!(target: "minerd::rpc", "Failed to parse header");
return server_error(RpcError::BlockParseError, id, None)
let Ok(header) = deserialize_async::<Header>(&header_bytes).await else {
error!(target: "minerd::rpc::poll", "Failed to parse header");
self.sleep().await?;
continue
};
let header_hash = header.hash();
info!(target: "minerd::rpc", "Received request to mine block header {header_hash} with key {randomx_key} for target: {target}");
// If we have a requested mining height, we'll keep dropping here.
if self.stop_at_height > 0 && header.height >= self.stop_at_height {
info!(target: "minerd::rpc", "Reached requested mining height {}", self.stop_at_height);
return server_error(RpcError::MiningFailed, id, None)
return Ok((randomx_key, target, header))
}
}
// Check if another request is being processed
if let Some(e) = self.abort_pending(id).await {
return e
};
// Mine provided block header
info!(target: "minerd::rpc", "Mining block header {header_hash} with key {randomx_key} for target: {target}");
if let Err(e) =
mine_block(&target, &randomx_key, &mut header, self.threads, &self.stop_signal.clone())
/// Auxiliary function to submit a mining solution to configured
/// darkfid daemon.
async fn submit(&self, nonce: f64) -> String {
debug!(target: "minerd::rpc::submit", "Executing submit request to darkfid...");
let mut request_params = self.config.wallet_config.clone();
request_params.insert(String::from("nonce"), JsonValue::Number(nonce));
let result = match self
.darkfid_daemon_request("miner.submit_solution", &JsonValue::from(request_params))
.await
{
error!(target: "minerd::rpc", "Failed mining block header {header_hash} with error: {e}");
return server_error(RpcError::MiningFailed, id, None)
}
info!(target: "minerd::rpc", "Mined block header {header_hash} with nonce: {}", header.nonce);
Ok(result) => result,
Err(e) => return format!("darkfid submit failed: {e}"),
};
debug!(target: "minerd::rpc::submit", "Got reply: {result:?}");
// Return block header nonce
JsonResponse::new(JsonValue::Number(header.nonce as f64), id).into()
// Parse response
match result.get::<String>() {
Some(result) => result.clone(),
None => format!("darkfid responded with invalid params: {result:?}"),
}
}
/// Auxiliary function to abort pending request.
async fn abort_pending(&self, id: u16) -> Option<JsonResult> {
// Check if a pending request is being processed
info!(target: "minerd::rpc", "Checking if a pending request is being processed...");
if self.stop_signal.receiver_count() <= 1 {
info!(target: "minerd::rpc", "No pending requests!");
return None
/// Auxiliary function to execute a request towards the configured
/// darkfid daemon JSON-RPC endpoint.
async fn darkfid_daemon_request(&self, method: &str, params: &JsonValue) -> Result<JsonValue> {
let mut lock = self.rpc_client.write().await;
let req = JsonRequest::new(method, params.clone());
// Check the client is initialized
if let Some(ref client) = lock.client {
// Execute request
if let Ok(rep) = client.request(req.clone()).await {
drop(lock);
return Ok(rep);
}
}
info!(target: "minerd::rpc", "Pending request is in progress, sending stop signal...");
// Send stop signal to worker
if self.sender.send(()).await.is_err() {
error!(target: "minerd::rpc", "Failed to stop pending request");
return Some(server_error(RpcError::StopFailed, id, None))
// Reset the rpc client in case of an error and try again
let client = RpcClient::new(lock.endpoint.clone(), lock.ex.clone()).await?;
let rep = client.request(req).await?;
lock.client = Some(client);
drop(lock);
Ok(rep)
}
// Wait for worker to terminate
info!(target: "minerd::rpc", "Waiting for request to terminate...");
while self.stop_signal.receiver_count() > 1 {
sleep(1).await;
}
info!(target: "minerd::rpc", "Pending request terminated!");
// Consume channel item so its empty again
if self.stop_signal.recv().await.is_err() {
error!(target: "minerd::rpc", "Failed to cleanup stop signal channel");
return Some(server_error(RpcError::StopFailed, id, None))
/// Auxiliary function to stop current JSON-RPC client, if its
/// initialized.
pub async fn stop_rpc_client(&self) {
self.rpc_client.read().await.stop().await;
}
None
/// Auxiliary function to sleep for configured polling rate time.
async fn sleep(&self) -> Result<()> {
// Check if stop signal is received
if self.stop_signal.is_full() {
debug!(target: "minerd::rpc::sleep", "Stop signal received, exiting polling task");
return Err(Error::DetachedTaskStopped);
}
debug!(target: "minerd::rpc::sleep", "Sleeping for {} until next poll...", self.config.polling_rate);
sleep(self.config.polling_rate).await;
Ok(())
}
}
/// Async task to poll darkfid for new mining jobs. Once a new job is
/// received, spawns a mining task in the background.
pub async fn polling_task(miner: MinerNodePtr, ex: ExecutorPtr) -> Result<()> {
// Initialize a dummy Header to use on first poll
let mut current_job = Header::default().hash().to_string();
loop {
// Poll darkfid for a mining job
let (randomx_key, target, header) = miner.poll(&current_job).await?;
let header_hash = header.hash().to_string();
debug!(target: "minerd::rpc::polling_task", "Received job:");
debug!(target: "minerd::rpc::polling_task", "\tRandomX key - {randomx_key}");
debug!(target: "minerd::rpc::polling_task", "\tTarget - {target}");
debug!(target: "minerd::rpc::polling_task", "\tHeader - {header_hash}");
// Check if we are already processing this job
if header_hash == current_job {
debug!(target: "minerd::rpc::polling_task", "Already received job, skipping...");
miner.sleep().await?;
continue
}
// Check if we reached the stop height
if miner.config.stop_at_height > 0 && header.height > miner.config.stop_at_height {
info!(target: "minerd::rpc::polling_task", "Reached requested mining height: {}", miner.config.stop_at_height);
info!(target: "minerd::rpc::polling_task", "Daemon can be safely terminated now!");
break
}
info!(target: "minerd::rpc::polling_task", "Received new job to mine block header {header_hash} with key {randomx_key} for target: 0x{target:064x}");
// Abord pending job
miner.abort_pending().await;
// Detach mining task
StoppableTask::new().start(
mining_task(miner.clone(), randomx_key, target, header),
|res| async {
match res {
Ok(()) | Err(Error::DetachedTaskStopped) => { /* Do nothing */ }
Err(e) => error!(target: "minerd::rpc::polling_task", "Failed starting mining task: {e}"),
}
},
Error::DetachedTaskStopped,
ex.clone(),
);
// Update current job
current_job = header_hash;
// Sleep until next poll
miner.sleep().await?;
}
Ok(())
}
/// Async task to mine provided header and submit solution to darkfid.
async fn mining_task(
miner: MinerNodePtr,
randomx_key: HeaderHash,
target: BigUint,
mut header: Header,
) -> Result<()> {
// Mine provided block header
let header_hash = header.hash().to_string();
info!(target: "minerd::rpc::mining_task", "Mining block header {header_hash} with key {randomx_key} for target: 0x{target:064x}");
if let Err(e) = mine_block(
&randomx_key,
&target,
&mut header,
miner.config.threads,
&miner.stop_signal.clone(),
) {
error!(target: "minerd::rpc::mining_task", "Failed mining block header {header_hash} with error: {e}");
return Err(Error::DetachedTaskStopped)
}
info!(target: "minerd::rpc::mining_task", "Mined block header {header_hash} with nonce: {}", header.nonce);
info!(target: "minerd::rpc::mining_task", "Mined block header hash: {}", header.hash());
// Submit solution to darkfid
info!(target: "minerd::rpc::submit", "Submitting solution to darkfid...");
let result = miner.submit(header.nonce as f64).await;
info!(target: "minerd::rpc::submit", "Submition result: {result}");
Ok(())
}

View File

@@ -17,26 +17,11 @@ database = "darkfid0"
# Confirmation threshold, denominated by number of blocks
threshold = 6
# minerd JSON-RPC endpoint
minerd_endpoint = "tcp://127.0.0.1:48467"
# PoW block production target, in seconds
pow_target = 60
# Participate in block production
miner = true
# Wallet address to receive mining rewards.
# This is a dummy one so the miner can start,
# replace with your own one.
recipient = "9vw6WznKk7xEFQwwXhJWMMdjUPi3cXL8NrFKQpKifG1U"
# Optional contract spend hook to use in the mining reward
#spend_hook = "YOUR_SPEND_HOOK_HERE"
# Optional contract user data to use in the mining reward.
# This is not arbitrary data.
#user_data = "YOUR_USER_DATA_HERE"
# Optional fixed PoW difficulty, used for testing
#pow_fixed_difficulty = 1
# Skip syncing process and start node right away
skip_sync = true

View File

@@ -17,23 +17,11 @@ database = "darkfid1"
# Confirmation threshold, denominated by number of blocks
threshold = 6
# minerd JSON-RPC endpoint
minerd_endpoint = "tcp://127.0.0.1:48567"
# PoW block production target, in seconds
pow_target = 60
# Wallet address to receive mining rewards.
# This is a dummy one so the miner can start,
# replace with your own one.
recipient = "9vw6WznKk7xEFQwwXhJWMMdjUPi3cXL8NrFKQpKifG1U"
# Optional contract spend hook to use in the mining reward
#spend_hook = "YOUR_SPEND_HOOK_HERE"
# Optional contract user data to use in the mining reward.
# This is not arbitrary data.
#user_data = "YOUR_USER_DATA_HERE"
# Optional fixed PoW difficulty, used for testing
#pow_fixed_difficulty = 1
# Skip syncing process and start node right away
skip_sync = false

View File

@@ -17,23 +17,11 @@ database = "darkfid2"
# Confirmation threshold, denominated by number of blocks
threshold = 6
# minerd JSON-RPC endpoint
minerd_endpoint = "tcp://127.0.0.1:48667"
# PoW block production target, in seconds
pow_target = 60
# Wallet address to receive mining rewards.
# This is a dummy one so the miner can start,
# replace with your own one.
recipient = "9vw6WznKk7xEFQwwXhJWMMdjUPi3cXL8NrFKQpKifG1U"
# Optional contract spend hook to use in the mining reward
#spend_hook = "YOUR_SPEND_HOOK_HERE"
# Optional contract user data to use in the mining reward.
# This is not arbitrary data.
#user_data = "YOUR_USER_DATA_HERE"
# Optional fixed PoW difficulty, used for testing
#pow_fixed_difficulty = 1
# Skip syncing process and start node right away
skip_sync = false

View File

@@ -17,23 +17,11 @@ database = "darkfid3"
# Confirmation threshold, denominated by number of blocks
threshold = 6
# minerd JSON-RPC endpoint
minerd_endpoint = "tcp://127.0.0.1:48767"
# PoW block production target, in seconds
pow_target = 60
# Wallet address to receive mining rewards.
# This is a dummy one so the miner can start,
# replace with your own one.
recipient = "9vw6WznKk7xEFQwwXhJWMMdjUPi3cXL8NrFKQpKifG1U"
# Optional contract spend hook to use in the mining reward
#spend_hook = "YOUR_SPEND_HOOK_HERE"
# Optional contract user data to use in the mining reward.
# This is not arbitrary data.
#user_data = "YOUR_USER_DATA_HERE"
# Optional fixed PoW difficulty, used for testing
#pow_fixed_difficulty = 1
# Skip syncing process and start node right away
skip_sync = false

View File

@@ -17,23 +17,11 @@ database = "darkfid4"
# Confirmation threshold, denominated by number of blocks
threshold = 6
# minerd JSON-RPC endpoint
minerd_endpoint = "tcp://127.0.0.1:48867"
# PoW block production target, in seconds
pow_target = 60
# Wallet address to receive mining rewards.
# This is a dummy one so the miner can start,
# replace with your own one.
recipient = "9vw6WznKk7xEFQwwXhJWMMdjUPi3cXL8NrFKQpKifG1U"
# Optional contract spend hook to use in the mining reward
#spend_hook = "YOUR_SPEND_HOOK_HERE"
# Optional contract user data to use in the mining reward.
# This is not arbitrary data.
#user_data = "YOUR_USER_DATA_HERE"
# Optional fixed PoW difficulty, used for testing
#pow_fixed_difficulty = 1
# Skip syncing process and start node right away
skip_sync = false

View File

@@ -9,10 +9,28 @@
# PoW miner number of threads to use
threads = 1
# JSON-RPC settings
[rpc]
# JSON-RPC listen URL
rpc_listen = "tcp://127.0.0.1:48467"
# Polling rate to ask darkfid for mining jobs
#polling_rate = 2
# Disabled RPC methods
#rpc_disabled_methods = []
# Stop mining at given height (0 mines forever)
#stop_at_height = 0
# Blockchain network to use
network = "localnet"
# Localnet blockchain network configuration
[network_config."localnet"]
# Wallet mining address to receive mining rewards.
# This is a dummy one so the miner can start,
# replace with your own one.
recipient = "9vw6WznKk7xEFQwwXhJWMMdjUPi3cXL8NrFKQpKifG1U"
# Optional contract spend hook to use in the mining reward
#spend_hook = "YOUR_SPEND_HOOK_HERE"
# Optional contract user data to use in the mining reward.
# This is not arbitrary data.
#user_data = "YOUR_USER_DATA_HERE"
# darkfid JSON-RPC endpoint
endpoint = "tcp://127.0.0.1:48240"

View File

@@ -9,10 +9,28 @@
# PoW miner number of threads to use
threads = 1
# JSON-RPC settings
[rpc]
# JSON-RPC listen URL
rpc_listen = "tcp://127.0.0.1:48567"
# Polling rate to ask darkfid for mining jobs
#polling_rate = 2
# Disabled RPC methods
#rpc_disabled_methods = []
# Stop mining at given height (0 mines forever)
#stop_at_height = 0
# Blockchain network to use
network = "localnet"
# Localnet blockchain network configuration
[network_config."localnet"]
# Wallet mining address to receive mining rewards.
# This is a dummy one so the miner can start,
# replace with your own one.
recipient = "9vw6WznKk7xEFQwwXhJWMMdjUPi3cXL8NrFKQpKifG1U"
# Optional contract spend hook to use in the mining reward
#spend_hook = "YOUR_SPEND_HOOK_HERE"
# Optional contract user data to use in the mining reward.
# This is not arbitrary data.
#user_data = "YOUR_USER_DATA_HERE"
# darkfid JSON-RPC endpoint
endpoint = "tcp://127.0.0.1:48340"

View File

@@ -9,10 +9,28 @@
# PoW miner number of threads to use
threads = 1
# JSON-RPC settings
[rpc]
# JSON-RPC listen URL
rpc_listen = "tcp://127.0.0.1:48667"
# Polling rate to ask darkfid for mining jobs
#polling_rate = 2
# Disabled RPC methods
#rpc_disabled_methods = []
# Stop mining at given height (0 mines forever)
#stop_at_height = 0
# Blockchain network to use
network = "localnet"
# Localnet blockchain network configuration
[network_config."localnet"]
# Wallet mining address to receive mining rewards.
# This is a dummy one so the miner can start,
# replace with your own one.
recipient = "9vw6WznKk7xEFQwwXhJWMMdjUPi3cXL8NrFKQpKifG1U"
# Optional contract spend hook to use in the mining reward
#spend_hook = "YOUR_SPEND_HOOK_HERE"
# Optional contract user data to use in the mining reward.
# This is not arbitrary data.
#user_data = "YOUR_USER_DATA_HERE"
# darkfid JSON-RPC endpoint
endpoint = "tcp://127.0.0.1:48440"

View File

@@ -9,10 +9,28 @@
# PoW miner number of threads to use
threads = 1
# JSON-RPC settings
[rpc]
# JSON-RPC listen URL
rpc_listen = "tcp://127.0.0.1:48767"
# Polling rate to ask darkfid for mining jobs
#polling_rate = 2
# Disabled RPC methods
#rpc_disabled_methods = []
# Stop mining at given height (0 mines forever)
#stop_at_height = 0
# Blockchain network to use
network = "localnet"
# Localnet blockchain network configuration
[network_config."localnet"]
# Wallet mining address to receive mining rewards.
# This is a dummy one so the miner can start,
# replace with your own one.
recipient = "9vw6WznKk7xEFQwwXhJWMMdjUPi3cXL8NrFKQpKifG1U"
# Optional contract spend hook to use in the mining reward
#spend_hook = "YOUR_SPEND_HOOK_HERE"
# Optional contract user data to use in the mining reward.
# This is not arbitrary data.
#user_data = "YOUR_USER_DATA_HERE"
# darkfid JSON-RPC endpoint
endpoint = "tcp://127.0.0.1:48540"

View File

@@ -9,10 +9,28 @@
# PoW miner number of threads to use
threads = 1
# JSON-RPC settings
[rpc]
# JSON-RPC listen URL
rpc_listen = "tcp://127.0.0.1:48867"
# Polling rate to ask darkfid for mining jobs
#polling_rate = 2
# Disabled RPC methods
#rpc_disabled_methods = []
# Stop mining at given height (0 mines forever)
#stop_at_height = 0
# Blockchain network to use
network = "localnet"
# Localnet blockchain network configuration
[network_config."localnet"]
# Wallet mining address to receive mining rewards.
# This is a dummy one so the miner can start,
# replace with your own one.
recipient = "9vw6WznKk7xEFQwwXhJWMMdjUPi3cXL8NrFKQpKifG1U"
# Optional contract spend hook to use in the mining reward
#spend_hook = "YOUR_SPEND_HOOK_HERE"
# Optional contract user data to use in the mining reward.
# This is not arbitrary data.
#user_data = "YOUR_USER_DATA_HERE"
# darkfid JSON-RPC endpoint
endpoint = "tcp://127.0.0.1:48640"

View File

@@ -17,32 +17,32 @@ else
fi
tmux new-session -d -s $session -n "node0"
tmux send-keys -t $session "$MINERD $verbose -c minerd0.toml" Enter
tmux send-keys -t $session "$MINERD -c minerd0.toml $verbose" Enter
sleep 1
tmux split-window -t $session -v -l 90%
tmux send-keys -t $session "$DARKFID $verbose -c darkfid0.toml" Enter
tmux send-keys -t $session "$DARKFID -c darkfid0.toml $verbose" Enter
sleep 2
tmux new-window -t $session -n "node1"
tmux send-keys -t $session "$MINERD $verbose -c minerd1.toml" Enter
tmux send-keys -t $session "$MINERD -c minerd1.toml $verbose" Enter
sleep 1
tmux split-window -t $session -v -l 90%
tmux send-keys -t $session "$DARKFID $verbose -c darkfid1.toml" Enter
tmux send-keys -t $session "$DARKFID -c darkfid1.toml $verbose" Enter
sleep 2
tmux new-window -t $session -n "node2"
tmux send-keys -t $session "$MINERD $verbose -c minerd2.toml" Enter
tmux send-keys -t $session "$MINERD -c minerd2.toml $verbose" Enter
sleep 1
tmux split-window -t $session -v -l 90%
tmux send-keys -t $session "$DARKFID $verbose -c darkfid2.toml" Enter
tmux send-keys -t $session "$DARKFID -c darkfid2.toml $verbose" Enter
sleep 2
tmux new-window -t $session -n "node3"
tmux send-keys -t $session "$MINERD $verbose -c minerd3.toml" Enter
tmux send-keys -t $session "$MINERD -c minerd3.toml $verbose" Enter
sleep 1
tmux split-window -t $session -v -l 90%
tmux send-keys -t $session "$DARKFID $verbose -c darkfid3.toml" Enter
tmux send-keys -t $session "$DARKFID -c darkfid3.toml $verbose" Enter
sleep 2
tmux new-window -t $session -n "node4"
tmux send-keys -t $session "$MINERD $verbose -c minerd4.toml" Enter
tmux send-keys -t $session "$MINERD -c minerd4.toml $verbose" Enter
sleep 1
tmux split-window -t $session -v -l 90%
tmux send-keys -t $session "$DARKFID $verbose -c darkfid4.toml" Enter
tmux send-keys -t $session "$DARKFID -c darkfid4.toml $verbose" Enter
tmux attach -t $session

View File

@@ -9,12 +9,12 @@ a testing wallet and pass its address to the `darkfid`
config, so the wallet gets the block rewards the node
produces. We generate a wallet, set it as the default
and set its address as the `recipient` field in
`darkfid.toml`, using the porvided automated script:
`minerd.toml`, using the porvided automated script:
```
% ./init-wallet.sh
```
Then start `darkfid` and wait until its initialized:
Then start the daemones and wait until `darkfid` is initialized:
```
% ./tmux_sessions.sh
```

View File

@@ -1,3 +1,3 @@
#!/bin/sh
rm -rf darkfid drk
sed -i -e "s|recipient =.*|recipient = \"9vw6WznKk7xEFQwwXhJWMMdjUPi3cXL8NrFKQpKifG1U\"|g" darkfid.toml
sed -i -e "s|recipient =.*|recipient = \"9vw6WznKk7xEFQwwXhJWMMdjUPi3cXL8NrFKQpKifG1U\"|g" minerd.toml

View File

@@ -17,27 +17,12 @@ database = "darkfid"
# Confirmation threshold, denominated by number of blocks
threshold = 1
# minerd JSON-RPC endpoint
minerd_endpoint = "tcp://127.0.0.1:48467"
# PoW block production target, in seconds
pow_target = 10
# Optional fixed PoW difficulty, used for testing
pow_fixed_difficulty = 1
# Wallet address to receive mining rewards.
# This is a dummy one so the miner can start,
# replace with your own one.
recipient = "9vw6WznKk7xEFQwwXhJWMMdjUPi3cXL8NrFKQpKifG1U"
# Optional contract spend hook to use in the mining reward
#spend_hook = "6iW9nywZYvyhcM7P1iLwYkh92rvYtREDsC8hgqf2GLuT"
# Optional contract user data to use in the mining reward.
# This is not arbitrary data.
#user_data = "YOUR_USER_DATA_HERE"
# Skip syncing process and start node right away
skip_sync = true

View File

@@ -7,4 +7,4 @@ $DRK wallet initialize
$DRK wallet keygen
$DRK wallet default-address 1
wallet=$($DRK wallet address)
sed -i -e "s|9vw6WznKk7xEFQwwXhJWMMdjUPi3cXL8NrFKQpKifG1U|$wallet|g" darkfid.toml
sed -i -e "s|9vw6WznKk7xEFQwwXhJWMMdjUPi3cXL8NrFKQpKifG1U|$wallet|g" minerd.toml

View File

@@ -9,10 +9,30 @@
# PoW miner number of threads to use
#threads = 4
# JSON-RPC settings
[rpc]
# JSON-RPC listen URL
rpc_listen = "tcp://127.0.0.1:48467"
# Polling rate to ask darkfid for mining jobs
#polling_rate = 2
# Disabled RPC methods
#rpc_disabled_methods = []
# Stop mining at given height (0 mines forever)
#stop_at_height = 0
# Blockchain network to use
network = "localnet"
# Localnet blockchain network configuration
[network_config."localnet"]
# Wallet mining address to receive mining rewards.
# This is a dummy one so the miner can start,
# replace with your own one.
recipient = "9vw6WznKk7xEFQwwXhJWMMdjUPi3cXL8NrFKQpKifG1U"
# Optional contract spend hook to use in the mining reward.
# This is the DAO spend hook set for convinience,
# replace with your own one.
#spend_hook = "6iW9nywZYvyhcM7P1iLwYkh92rvYtREDsC8hgqf2GLuT"
# Optional contract user data to use in the mining reward.
# This is not arbitrary data.
#user_data = "YOUR_USER_DATA_HERE"
# darkfid JSON-RPC endpoint
endpoint = "tcp://127.0.0.1:48240"

View File

@@ -52,30 +52,30 @@ wait_token() {
}
mint_dao() {
$DRK dao create 20 10 10 0.67 MLDY > $OUTPUT_FOLDER/dao.toml
$DRK dao import MiladyMakerDAO < $OUTPUT_FOLDER/dao.toml
$DRK dao create 20 10 10 0.67 ANON > $OUTPUT_FOLDER/dao.toml
$DRK dao import AnonDAO < $OUTPUT_FOLDER/dao.toml
$DRK dao list
$DRK dao list MiladyMakerDAO
$DRK dao list AnonDAO
$DRK dao mint MiladyMakerDAO | tee $OUTPUT_FOLDER/dao-mint.tx | $DRK broadcast
$DRK dao mint AnonDAO | tee $OUTPUT_FOLDER/dao-mint.tx | $DRK broadcast
}
wait_dao_mint() {
while [ "$($DRK dao list MiladyMakerDAO | grep '^Transaction hash: ' | awk '{print $3}')" = None ]; do
while [ "$($DRK dao list AnonDAO | grep '^Transaction hash: ' | awk '{print $3}')" = None ]; do
sleep $SLEEP_TIME
sh ./sync-wallet.sh > /dev/null
done
}
fill_treasury() {
PUBKEY="$($DRK dao list MiladyMakerDAO | grep '^Notes Public key: ' | cut -d ' ' -f4)"
PUBKEY="$($DRK dao list AnonDAO | grep '^Notes Public key: ' | cut -d ' ' -f4)"
SPEND_HOOK="$($DRK dao spend-hook)"
BULLA="$($DRK dao list MiladyMakerDAO | grep '^Bulla: ' | cut -d' ' -f2)"
$DRK transfer 20 WCKD "$PUBKEY" "$SPEND_HOOK" "$BULLA" | tee $OUTPUT_FOLDER/xfer.tx | $DRK broadcast
BULLA="$($DRK dao list AnonDAO | grep '^Bulla: ' | cut -d' ' -f2)"
$DRK transfer 20 DAWN "$PUBKEY" "$SPEND_HOOK" "$BULLA" | tee $OUTPUT_FOLDER/xfer.tx | $DRK broadcast
}
dao_balance() {
BALANCE=$($DRK dao balance MiladyMakerDAO 2>/dev/null)
BALANCE=$($DRK dao balance AnonDAO 2>/dev/null)
# No tokens received at all yet
if echo "$BALANCE" | grep -q "No unspent balances found"; then
echo 0
@@ -94,7 +94,7 @@ dao_balance() {
}
wait_dao_treasury() {
while [ "$(dao_balance WCKD)" = 0 ]; do
while [ "$(dao_balance DAWN)" = 0 ]; do
sleep $SLEEP_TIME
sh ./sync-wallet.sh > /dev/null
done
@@ -102,12 +102,12 @@ wait_dao_treasury() {
propose() {
MY_ADDR=$($DRK wallet address)
PROPOSAL="$($DRK dao propose-transfer MiladyMakerDAO 1 5 WCKD "$MY_ADDR" | cut -d' ' -f3)"
PROPOSAL="$($DRK dao propose-transfer AnonDAO 1 5 DAWN "$MY_ADDR" | cut -d' ' -f3)"
$DRK dao proposal "$PROPOSAL" --mint-proposal | tee $OUTPUT_FOLDER/propose.tx | $DRK broadcast
}
wait_proposal() {
PROPOSAL="$($DRK dao proposals MiladyMakerDAO | cut -d' ' -f2)"
PROPOSAL="$($DRK dao proposals AnonDAO | cut -d' ' -f2)"
while [ "$($DRK dao proposal $PROPOSAL | grep '^Proposal transaction hash: ' | awk '{print $4}')" = None ]; do
sleep $SLEEP_TIME
sh ./sync-wallet.sh > /dev/null
@@ -115,12 +115,12 @@ wait_proposal() {
}
vote() {
PROPOSAL="$($DRK dao proposals MiladyMakerDAO | cut -d' ' -f2)"
PROPOSAL="$($DRK dao proposals AnonDAO | cut -d' ' -f2)"
$DRK dao vote "$PROPOSAL" 1 | tee $OUTPUT_FOLDER/dao-vote.tx | $DRK broadcast
}
wait_vote() {
PROPOSAL="$($DRK dao proposals MiladyMakerDAO | cut -d' ' -f2)"
PROPOSAL="$($DRK dao proposals AnonDAO | cut -d' ' -f2)"
while [ "$($DRK dao proposal $PROPOSAL | grep '^Current proposal outcome: ' | awk '{print $4}')" != "Approved" ]; do
sleep $SLEEP_TIME
sh ./sync-wallet.sh > /dev/null
@@ -128,12 +128,12 @@ wait_vote() {
}
do_exec() {
PROPOSAL="$($DRK dao proposals MiladyMakerDAO | cut -d' ' -f2)"
PROPOSAL="$($DRK dao proposals AnonDAO | cut -d' ' -f2)"
$DRK dao exec --early $PROPOSAL | tee $OUTPUT_FOLDER/dao-exec.tx | $DRK broadcast
}
wait_exec() {
PROPOSAL="$($DRK dao proposals MiladyMakerDAO | cut -d' ' -f2)"
PROPOSAL="$($DRK dao proposals AnonDAO | cut -d' ' -f2)"
while [ -z "$($DRK dao proposal $PROPOSAL | grep '^Proposal was executed on transaction: ')" ]; do
sleep $SLEEP_TIME
sh ./sync-wallet.sh > /dev/null
@@ -141,10 +141,10 @@ wait_exec() {
}
wait_token DRK
mint_token WCKD 42
wait_token WCKD
mint_token MLDY 20
wait_token MLDY
mint_token ANON 42
wait_token ANON
mint_token DAWN 20
wait_token DAWN
mint_dao
wait_dao_mint
fill_treasury

View File

@@ -3,6 +3,6 @@ darkfid localnet
This will start three `darkfid` node instances
in localnet mode. Two of the nodes are activelly
mining, while the other one is just a sync node.
producing blocks, while the other one is just a sync node.
We also start two `minerd` daemons to mine blocks,
one for each of the `darkfid` mining nodes.
one for each of the `darkfid` block producing nodes.

View File

@@ -17,23 +17,11 @@ database = "darkfid0"
# Confirmation threshold, denominated by number of blocks
threshold = 6
# minerd JSON-RPC endpoint
minerd_endpoint = "tcp://127.0.0.1:48467"
# PoW block production target, in seconds
pow_target = 20
# Wallet address to receive mining rewards.
# This is a dummy one so the miner can start,
# replace with your own one.
recipient = "9vw6WznKk7xEFQwwXhJWMMdjUPi3cXL8NrFKQpKifG1U"
# Optional contract spend hook to use in the mining reward
#spend_hook = "YOUR_SPEND_HOOK_HERE"
# Optional contract user data to use in the mining reward.
# This is not arbitrary data.
#user_data = "YOUR_USER_DATA_HERE"
# Optional fixed PoW difficulty, used for testing
#pow_fixed_difficulty = 1
# Skip syncing process and start node right away
skip_sync = true

View File

@@ -25,23 +25,11 @@ database = "darkfid1"
# Confirmation threshold, denominated by number of blocks
threshold = 6
# minerd JSON-RPC endpoint
minerd_endpoint = "tcp://127.0.0.1:48567"
# PoW block production target, in seconds
pow_target = 20
# Wallet address to receive mining rewards.
# This is a dummy one so the miner can start,
# replace with your own one.
recipient = "9vw6WznKk7xEFQwwXhJWMMdjUPi3cXL8NrFKQpKifG1U"
# Optional contract spend hook to use in the mining reward
#spend_hook = "YOUR_SPEND_HOOK_HERE"
# Optional contract user data to use in the mining reward.
# This is not arbitrary data.
#user_data = "YOUR_USER_DATA_HERE"
# Optional fixed PoW difficulty, used for testing
#pow_fixed_difficulty = 1
# Skip syncing process and start node right away
skip_sync = false

View File

@@ -17,12 +17,12 @@ database = "darkfid2"
# Confirmation threshold, denominated by number of blocks
threshold = 6
# minerd JSON-RPC endpoint
#minerd_endpoint = "tcp://127.0.0.1:28467"
# PoW block production target, in seconds
pow_target = 20
# Optional fixed PoW difficulty, used for testing
#pow_fixed_difficulty = 1
# Skip syncing process and start node right away
skip_sync = false

View File

@@ -9,10 +9,28 @@
# PoW miner number of threads to use
threads = 2
# JSON-RPC settings
[rpc]
# JSON-RPC listen URL
rpc_listen = "tcp://127.0.0.1:48467"
# Polling rate to ask darkfid for mining jobs
#polling_rate = 2
# Disabled RPC methods
#rpc_disabled_methods = []
# Stop mining at given height (0 mines forever)
#stop_at_height = 0
# Blockchain network to use
network = "localnet"
# Localnet blockchain network configuration
[network_config."localnet"]
# Wallet mining address to receive mining rewards.
# This is a dummy one so the miner can start,
# replace with your own one.
recipient = "9vw6WznKk7xEFQwwXhJWMMdjUPi3cXL8NrFKQpKifG1U"
# Optional contract spend hook to use in the mining reward
#spend_hook = "YOUR_SPEND_HOOK_HERE"
# Optional contract user data to use in the mining reward.
# This is not arbitrary data.
#user_data = "YOUR_USER_DATA_HERE"
# darkfid JSON-RPC endpoint
endpoint = "tcp://127.0.0.1:48240"

View File

@@ -9,10 +9,28 @@
# PoW miner number of threads to use
threads = 2
# JSON-RPC settings
[rpc]
# JSON-RPC listen URL
rpc_listen = "tcp://127.0.0.1:48567"
# Polling rate to ask darkfid for mining jobs
#polling_rate = 2
# Disabled RPC methods
#rpc_disabled_methods = []
# Stop mining at given height (0 mines forever)
#stop_at_height = 0
# Blockchain network to use
network = "localnet"
# Localnet blockchain network configuration
[network_config."localnet"]
# Wallet mining address to receive mining rewards.
# This is a dummy one so the miner can start,
# replace with your own one.
recipient = "9vw6WznKk7xEFQwwXhJWMMdjUPi3cXL8NrFKQpKifG1U"
# Optional contract spend hook to use in the mining reward
#spend_hook = "YOUR_SPEND_HOOK_HERE"
# Optional contract user data to use in the mining reward.
# This is not arbitrary data.
#user_data = "YOUR_USER_DATA_HERE"
# darkfid JSON-RPC endpoint
endpoint = "tcp://127.0.0.1:48340"

View File

@@ -18,17 +18,17 @@ else
fi
tmux new-session -d -s $session -n "node0"
tmux send-keys -t $session "$MINERD $verbose -c minerd0.toml" Enter
tmux send-keys -t $session "$MINERD -c minerd0.toml $verbose" Enter
sleep 1
tmux split-window -t $session -v -l 90%
tmux send-keys -t $session "$DARKFID $verbose -c darkfid0.toml" Enter
tmux send-keys -t $session "$DARKFID -c darkfid0.toml $verbose" Enter
sleep 2
tmux new-window -t $session -n "node1"
tmux send-keys -t $session "$MINERD $verbose -c minerd1.toml" Enter
tmux send-keys -t $session "$MINERD -c minerd1.toml $verbose" Enter
sleep 1
tmux split-window -t $session -v -l 90%
tmux send-keys -t $session "$DARKFID $verbose -c darkfid1.toml" Enter
tmux send-keys -t $session "$DARKFID -c darkfid1.toml $verbose" Enter
sleep 2
tmux new-window -t $session -n "node2"
tmux send-keys -t $session "$DARKFID $verbose -c darkfid2.toml" Enter
tmux send-keys -t $session "$DARKFID -c darkfid2.toml $verbose" Enter
tmux attach -t $session

View File

@@ -729,7 +729,7 @@ P2Pool wallet address to use:
{YOUR_DAO_P2POOL_WALLET_ADDRESS_CONFIGURATION}
```
Then configure a `darkfid` instance to mine for a DAO, by setting the
Then configure a `minerd` instance to mine for a DAO, by setting the
corresponding fields(uncomment if needed) as per retrieved
configuration:

View File

@@ -28,27 +28,27 @@ and stays connected to the p2p network.
* `drk` is a CLI wallet. It provides an interface to smart contracts such
as Money and DAO, manages our keys and coins, and scans the blockchain
to update our balances.
* `minerd` is the DarkFi mining daemon. `darkfid` connects to it over
RPC, and triggers commands to mine blocks.
* `minerd` is the DarkFi mining daemon. Connects to `darkfid` over RPC,
and requests new block headers to mine.
The config files for `darkfid` and `drk` are sectioned into three parts,
The config files for all three daemons are sectioned into three parts,
each marked `[network_config]`. The sections look like this:
* `[network_config."testnet"]`
* `[network_config."mainnet"]`
* `[network_config."localnet"]`
At the top of the `darkfid` and `drk` config file, we can modify the
network being used by changing the following line:
At the top of each daemon config file, we can modify the network being
used by changing the following line:
```toml
# Blockchain network to use
network = "testnet"
```
This enables us to configure `darkfid` and `drk` for different contexts,
namely mainnet, testnet and localnet. Mainnet is not active yet. Localnet
can be setup by following the instructions [here](#local-deployment). The
This enables us to configure the daemons for different contexts, namely
mainnet, testnet and localnet. Mainnet is not active yet. Localnet can
be setup by following the instructions [here](#local-deployment). The
rest of this tutorial assumes we are setting up a testnet node.
## Compiling
@@ -191,10 +191,8 @@ rest of the tutorial (`darkfid` and `drk` handle this), but if you want
to help secure the network, you can participate in the mining process
by running the native `minerd` mining daemon.
To mine on DarkFI we need to expose the `minerd` RPC to the `darkfid`
full node, which will initiate the mining process. We'll also need to
add a recipient to `darkfid` that specifies where the mining rewards
will be minted to.
To mine on DarkFI we need to add a recipient to `minerd` that specifies
where the mining rewards will be minted to.
First, compile it:
@@ -229,8 +227,32 @@ $ ./minerd
Config file created in "~/.config/darkfi/minerd_config.toml". Please review it and try again.
```
You now have to configure `minerd` to use your wallet address as the
rewards recipient, when it retrieves blocks from `darkfid` to mine.
Open your `minerd` config file with a text editor (the default path
is `~/.config/darkfi/minerd_config.toml`) and replace the
`YOUR_WALLET_ADDRESS_HERE` string with your `drk` wallet address:
```toml
# Put the address from `./drk wallet address` here
recipient = "YOUR_WALLET_ADDRESS_HERE"
```
You can retrieve your `drk` wallet address as follows:
```shell
$ ./drk wallet address
CbaqFqGTgn86Zh9AjUeMw3DJyVCshaPSPFtmj6Cyd5yU
```
Note: when modifying the `minerd` config file to use with the
testnet, be sure to change the values under the section marked
`[network_config."testnet"]` (not localnet or mainnet!).
Once that's in place, you can run it again and `minerd` will start,
waiting for requests to mine blocks.
polling `darkfid` for new block headers to mine.
```shell
$ ./minerd
@@ -240,41 +262,15 @@ $ ./minerd
14:20:06 [INFO] Mining daemon initialized successfully!
14:20:06 [INFO] Starting mining daemon...
14:20:06 [INFO] Mining daemon started successfully!
14:20:06 [INFO] Received new job to mine block header beb0...42aa with key 0edc...0679 for target: 0xffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff
14:20:06 [INFO] Mining block header beb0...42aa with key 0edc...0679 for target: 0xffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff
14:20:06 [INFO] Mined block header beb0...42aa with nonce: 1
14:20:06 [INFO] Mined block header hash: 36fe...753c
14:20:06 [INFO] Submitting solution to darkfid...
14:20:06 [INFO] Submition result: accepted
...
```
You now have to expose `minerd` RPC to `darkfid`, and configure it
to use your wallet address as the rewards recipient, when submitting
blocks to `minerd` to mine.
Open your `darkfid` config file with a text editor (the default path
is `~/.config/darkfi/darkfid_config.toml`). Find the `recipient` and
`minerd_endpoint` options under `[network_config."testnet"]`, and
uncomment them by removing the `#` character at the start of line,
like this:
```toml
# Put your `minerd` endpoint here (default for testnet is in this example)
minerd_endpoint = "tcp://127.0.0.1:28467"
# Put the address from `drk wallet address` here
recipient = "YOUR_WALLET_ADDRESS_HERE"
```
Now ensure that `minerd_endpoint` is set to the same value as the
`rpc_listen` address in your `minerd` config (the default path
is `~/.config/darkfi/minerd_config.toml`). Finally, replace the
`YOUR_WALLET_ADDRESS_HERE` string with your `drk` wallet address that
you can retrieve as follows:
```shell
$ ./drk wallet address
CbaqFqGTgn86Zh9AjUeMw3DJyVCshaPSPFtmj6Cyd5yU
```
Note: when modifying the `darkfid` config file to use with the
testnet, be sure to change the values under the section marked
`[network_config."testnet"]` (not localnet or mainnet!).
### Darkfid
Now that `darkfid` configuration is in place, you can run it again and
@@ -307,8 +303,7 @@ As its syncing, you'll see periodic messages like this:
This will give you an indication of the current progress. Keep it running,
and you should see a `Blockchain synced!` message after some time.
If you're running `minerd`, you should see a notification from the
`minerd` terminal like this:
If you're running `minerd`, you should see a notification like this:
```shell
...
@@ -321,36 +316,33 @@ This means that `darkfid` and `minerd` are connected over RPC and
```shell
...
14:23:56 [INFO] Mining block 4abc760a1f1c7198837e91c24d8e045e9fc9cb9fdf3a5fd45e184c25b03b0b51 for target:
115792089237316195423570985008687907853269984665640564039457584007913129639935
14:24:04 [INFO] Mined block 4abc760a1f1c7198837e91c24d8e045e9fc9cb9fdf3a5fd45e184c25b03b0b51 with nonce: 2
14:24:06 [INFO] Received request to mine block 17e7428ecb3d911477f8452170d0822c831c6912027abb120e4b4c4cf01d6020 for target:
115792089237316195423570985008687907853269984665640564039457584007913129639935
14:24:06 [INFO] Checking if a pending request is being processed...
14:23:56 [INFO] [RPC] Created new blocktemplate: address=9vw6...fG1U, spend_hook=-, user_data=-, hash=beb0...42aa
14:24:04 [INFO] [RPC] Got solution submission for block template: beb0...42aa
14:24:06 [INFO] [RPC] Mined block header hash: 36fe...753c
14:24:06 [INFO] [RPC] Proposing new block to network
...
```
When `darkfid` and `minerd` are correctly connected and you get an
error like this:
error on `minerd` like this:
```shell
...
[ERROR] minerd::rpc: Failed mining block f6b4a0f0c8f90905da271ec0add2e856939ef3b0d6cd5b28964d9c2b6d0a0fa9 with error:
Miner task stopped
[ERROR] Failed mining block header b757...5fb1 with error: Miner task stopped
...
```
That's expected behavior. It means your setup is correct and you are
mining blocks. `Failed mining block` happens when a new block was
received by `darkfid`, extending the current best fork, so it sends an
interuption message to `minerd` to stop mining the current block and
start mining the next height one.
mining blocks. `Failed mining block header` happens when a new block
was received by `darkfid`, extending the current best fork, so when
`minerd` polls it again it retrieves the new block header to mine,
interupting current mining workers to start mining the new one.
Otherwise, you'll see a notification like this:
```shell
...
[INFO] Mined block b6c7bd3545daa81d0e2e56ee780363beef6eb5b54579f54dca0cdd2a59989b76 with nonce: 266292
[INFO] Mined block header 36fe...753c with nonce: 266292
...
```
@@ -430,7 +422,7 @@ $ cd contrib/localnet/darkfid-single-node/
$ ./init-wallet.sh
```
Then start `darkfid` and wait until its initialized:
Then start the daemones and wait until `darkfid` is initialized:
```shell
$ ./tmux_sessions.sh

View File

@@ -396,9 +396,6 @@ impl PoWModule {
threads: usize,
stop_signal: &Receiver<()>,
) -> Result<()> {
// Grab the next mine target
let target = self.next_mine_target()?;
// Grab the RandomX key to use.
// We only use the next key when the next block is the
// height changing one.
@@ -410,7 +407,10 @@ impl PoWModule {
&self.darkfi_rx_keys.0
};
mine_block(&target, randomx_key, header, threads, stop_signal)
// Grab the next mine target
let target = self.next_mine_target()?;
mine_block(randomx_key, &target, header, threads, stop_signal)
}
}
@@ -435,8 +435,8 @@ fn get_mining_flags() -> RandomXFlags {
/// Auxiliary function to mine provided header using a single thread.
fn single_thread_mine(
target: &BigUint,
input: &HeaderHash,
target: &BigUint,
header: &mut Header,
stop_signal: &Receiver<()>,
) -> Result<()> {
@@ -486,8 +486,8 @@ fn single_thread_mine(
/// Auxiliary function to mine provided header using a multiple threads.
fn multi_thread_mine(
target: &BigUint,
input: &HeaderHash,
target: &BigUint,
header: &mut Header,
threads: usize,
stop_signal: &Receiver<()>,
@@ -614,8 +614,8 @@ fn multi_thread_mine(
/// Mine provided header, based on provided PoW module next mine target.
pub fn mine_block(
target: &BigUint,
input: &HeaderHash,
target: &BigUint,
header: &mut Header,
threads: usize,
stop_signal: &Receiver<()>,
@@ -634,8 +634,8 @@ pub fn mine_block(
error!(target: "validator::pow::mine_block", "[MINER] Can't use 0 threads!");
Err(Error::MinerTaskStopped)
}
1 => single_thread_mine(target, input, header, stop_signal),
_ => multi_thread_mine(target, input, header, threads, stop_signal),
1 => single_thread_mine(input, target, header, stop_signal),
_ => multi_thread_mine(input, target, header, threads, stop_signal),
}
}