Skip to content

Commit

Permalink
chore: moved fee exhaust constant to consensus constants (#1166)
Browse files Browse the repository at this point in the history
Description
---

Fee exhaust divisor (mainly used to calculate leader fees) was a hard
coded constant and moved to the consensus constants to make it
configurable on a tari network basis (DAN constants - Tari Network
mapping).

Motivation and Context
---
Fee exhaust divisor (mainly used to calculate leader fees) was a simple
hard coded constant before and it should be network specific.

How Has This Been Tested?
---

Checking whether everything is still building and running.

What process can a PR reviewer use to test or verify this change?
---


Breaking Changes
---

- [x] None
- [ ] Requires data directory to be deleted
- [ ] Other - Please specify
  • Loading branch information
ksrichard authored Oct 7, 2024
1 parent 5eaebcb commit 686ced9
Show file tree
Hide file tree
Showing 21 changed files with 65 additions and 55 deletions.
2 changes: 2 additions & 0 deletions Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

1 change: 1 addition & 0 deletions applications/tari_dan_app_utilities/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ tari_common = { workspace = true }
tari_common_types = { workspace = true }
tari_core = { workspace = true, default-features = false, features = ["transactions"] }
tari_crypto = { workspace = true }
tari_consensus = { workspace = true }
tari_shutdown = { workspace = true }
tari_dan_common_types = { workspace = true }
tari_state_store_sqlite = { workspace = true }
Expand Down
6 changes: 2 additions & 4 deletions applications/tari_dan_app_utilities/src/base_layer_scanner.rs
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ use tari_base_node_client::{
BaseNodeClientError,
};
use tari_common_types::types::{Commitment, FixedHash, FixedHashSizeError, PublicKey};
use tari_consensus::consensus_constants::ConsensusConstants;
use tari_core::transactions::{
tari_amount::MicroMinotari,
transaction_components::{
Expand Down Expand Up @@ -61,10 +62,7 @@ use tari_state_store_sqlite::SqliteStateStore;
use tari_template_lib::models::{EncryptedData, TemplateAddress, UnclaimedConfidentialOutputAddress};
use tokio::{task, task::JoinHandle, time};

use crate::{
consensus_constants::ConsensusConstants,
template_manager::interface::{TemplateManagerError, TemplateManagerHandle, TemplateRegistration},
};
use crate::template_manager::interface::{TemplateManagerError, TemplateManagerHandle, TemplateRegistration};

const LOG_TARGET: &str = "tari::dan::base_layer_scanner";

Expand Down
1 change: 0 additions & 1 deletion applications/tari_dan_app_utilities/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@

pub mod base_layer_scanner;
pub mod configuration;
pub mod consensus_constants;
pub mod json_encoding;
pub mod keypair;
pub mod p2p_config;
Expand Down
1 change: 1 addition & 0 deletions applications/tari_indexer/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ tari_indexer_lib = { workspace = true }
tari_template_lib = { workspace = true }
tari_transaction = { workspace = true }
tari_dan_p2p = { workspace = true }
tari_consensus = { workspace = true }
tari_validator_node_rpc = { workspace = true }
tari_rpc_framework = { workspace = true }
tari_networking = { workspace = true }
Expand Down
2 changes: 1 addition & 1 deletion applications/tari_indexer/src/bootstrap.rs
Original file line number Diff line number Diff line change
Expand Up @@ -30,10 +30,10 @@ use tari_common::{
configuration::bootstrap::{grpc_default_port, ApplicationType},
exit_codes::{ExitCode, ExitError},
};
use tari_consensus::consensus_constants::ConsensusConstants;
use tari_crypto::tari_utilities::ByteArray;
use tari_dan_app_utilities::{
base_layer_scanner,
consensus_constants::ConsensusConstants,
keypair::RistrettoKeypair,
seed_peer::SeedPeer,
template_manager::{self, implementation::TemplateManager},
Expand Down
10 changes: 5 additions & 5 deletions applications/tari_indexer/src/event_scanner.rs
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,8 @@ use futures::StreamExt;
use log::*;
use tari_bor::decode;
use tari_common::configuration::Network;
use tari_consensus::consensus_constants::ConsensusConstants;
use tari_crypto::{ristretto::RistrettoPublicKey, tari_utilities::message_format::MessageFormat};
use tari_dan_app_utilities::consensus_constants::ConsensusConstants;
use tari_dan_common_types::{committee::Committee, Epoch, NumPreshards, PeerAddress, ShardGroup};
use tari_dan_p2p::proto::rpc::{GetTransactionResultRequest, PayloadResultStatus, SyncBlocksRequest};
use tari_dan_storage::consensus_models::{Block, BlockError, BlockId, Decision, TransactionRecord};
Expand Down Expand Up @@ -101,6 +101,7 @@ pub struct EventScanner {
client_factory: TariValidatorNodeRpcClientFactory,
substate_store: SqliteSubstateStore,
event_filters: Vec<EventFilter>,
consensus_constants: ConsensusConstants,
}

impl EventScanner {
Expand All @@ -111,6 +112,7 @@ impl EventScanner {
client_factory: TariValidatorNodeRpcClientFactory,
substate_store: SqliteSubstateStore,
event_filters: Vec<EventFilter>,
consensus_constants: ConsensusConstants,
) -> Self {
Self {
network,
Expand All @@ -119,6 +121,7 @@ impl EventScanner {
client_factory,
substate_store,
event_filters,
consensus_constants,
}
}

Expand Down Expand Up @@ -476,10 +479,7 @@ impl EventScanner {

let start_block_id = match start_block_id {
Some(block_id) => block_id,
None => {
let consensus_constants = ConsensusConstants::from(self.network);
self.build_genesis_block_id(consensus_constants.num_preshards)?
},
None => self.build_genesis_block_id(self.consensus_constants.num_preshards)?,
};

committee.shuffle();
Expand Down
9 changes: 4 additions & 5 deletions applications/tari_indexer/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -51,11 +51,8 @@ use tari_common::{
configuration::bootstrap::{grpc_default_port, ApplicationType},
exit_codes::{ExitCode, ExitError},
};
use tari_dan_app_utilities::{
consensus_constants::ConsensusConstants,
keypair::setup_keypair_prompt,
substate_file_cache::SubstateFileCache,
};
use tari_consensus::consensus_constants::ConsensusConstants;
use tari_dan_app_utilities::{keypair::setup_keypair_prompt, substate_file_cache::SubstateFileCache};
use tari_dan_storage::global::DbFactory;
use tari_dan_storage_sqlite::SqliteDbFactory;
use tari_epoch_manager::{EpochManagerEvent, EpochManagerReader};
Expand Down Expand Up @@ -179,13 +176,15 @@ pub async fn run_indexer(config: ApplicationConfig, mut shutdown_signal: Shutdow
.map(TryInto::try_into)
.collect::<Result<_, _>>()
.map_err(|e| ExitError::new(ExitCode::ConfigError, format!("Invalid event filters: {}", e)))?;
let consensus_constants = ConsensusConstants::from(config.network);
let event_scanner = Arc::new(EventScanner::new(
config.network,
config.indexer.sidechain_id,
Box::new(services.epoch_manager.clone()),
services.validator_node_client_factory.clone(),
services.substate_store.clone(),
event_filters,
consensus_constants,
));

// Run the GraphQL API
Expand Down
2 changes: 1 addition & 1 deletion applications/tari_validator_node/src/bootstrap.rs
Original file line number Diff line number Diff line change
Expand Up @@ -34,13 +34,13 @@ use tari_common::{
configuration::Network,
exit_codes::{ExitCode, ExitError},
};
use tari_consensus::consensus_constants::ConsensusConstants;
#[cfg(not(feature = "metrics"))]
use tari_consensus::traits::hooks::NoopHooks;
use tari_core::transactions::transaction_components::ValidatorNodeSignature;
use tari_crypto::{ristretto::RistrettoPublicKey, tari_utilities::ByteArray};
use tari_dan_app_utilities::{
base_layer_scanner,
consensus_constants::ConsensusConstants,
keypair::RistrettoKeypair,
seed_peer::SeedPeer,
substate_file_cache::SubstateFileCache,
Expand Down
7 changes: 2 additions & 5 deletions applications/tari_validator_node/src/consensus/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,6 @@ use tari_consensus::{
};
use tari_crypto::ristretto::RistrettoPublicKey;
use tari_dan_app_utilities::{
consensus_constants::ConsensusConstants,
template_manager::implementation::TemplateManager,
transaction_executor::TariDanTransactionProcessor,
};
Expand Down Expand Up @@ -52,6 +51,7 @@ mod spec;
pub use block_transaction_executor::*;
pub use handle::*;
pub use signature_service::*;
use tari_consensus::consensus_constants::ConsensusConstants;

use crate::{p2p::NopLogger, transaction_validators::WithContext};

Expand Down Expand Up @@ -84,10 +84,7 @@ pub async fn spawn(
let hs_config = HotstuffConfig {
network,
sidechain_id,
max_base_layer_blocks_behind: consensus_constants.max_base_layer_blocks_behind,
max_base_layer_blocks_ahead: consensus_constants.max_base_layer_blocks_ahead,
num_preshards: consensus_constants.num_preshards,
pacemaker_max_base_time: consensus_constants.pacemaker_max_base_time,
consensus_constants,
};

let hotstuff_worker = HotstuffWorker::<TariConsensusSpec>::new(
Expand Down
6 changes: 4 additions & 2 deletions applications/tari_validator_node/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,8 @@ use tari_common::{
configuration::bootstrap::{grpc_default_port, ApplicationType},
exit_codes::{ExitCode, ExitError},
};
use tari_dan_app_utilities::{consensus_constants::ConsensusConstants, keypair::setup_keypair_prompt};
use tari_consensus::consensus_constants::ConsensusConstants;
use tari_dan_app_utilities::keypair::setup_keypair_prompt;
use tari_dan_common_types::SubstateAddress;
use tari_dan_storage::global::DbFactory;
use tari_dan_storage_sqlite::SqliteDbFactory;
Expand Down Expand Up @@ -116,13 +117,14 @@ pub async fn run_validator_node(
#[cfg(feature = "metrics")]
let metrics_registry = create_metrics_registry(keypair.public_key());

let consensus_constants = ConsensusConstants::from(config.network);
let base_node_client = create_base_layer_client(config).await?;
let services = spawn_services(
config,
shutdown_signal.clone(),
keypair.clone(),
global_db,
ConsensusConstants::devnet(), // TODO: change this eventually
consensus_constants,
base_node_client.clone(),
#[cfg(feature = "metrics")]
&metrics_registry,
Expand Down
2 changes: 1 addition & 1 deletion dan_layer/consensus/src/block_validations.rs
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,7 @@ pub async fn check_base_layer_block_hash<TConsensusSpec: ConsensusSpec>(
// current: current_height,
// })?;
// }
if base_layer_block_height > current_height + config.max_base_layer_blocks_ahead {
if base_layer_block_height > current_height + config.consensus_constants.max_base_layer_blocks_ahead {
Err(ProposalValidationError::BlockHeightTooHigh {
proposed: base_layer_block_height,
current: current_height,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,8 @@ pub struct ConsensusConstants {
pub max_base_layer_blocks_behind: u64,
pub num_preshards: NumPreshards,
pub pacemaker_max_base_time: Duration,
/// The value that fees are divided by to determine the amount of fees to burn. 0 means no fees are burned.
pub fee_exhaust_divisor: u64,
}

impl ConsensusConstants {
Expand All @@ -44,6 +46,7 @@ impl ConsensusConstants {
max_base_layer_blocks_behind: 5,
num_preshards: NumPreshards::P256,
pacemaker_max_base_time: Duration::from_secs(10),
fee_exhaust_divisor: 20, // 5%
}
}
}
Expand Down
4 changes: 0 additions & 4 deletions dan_layer/consensus/src/hotstuff/common.rs
Original file line number Diff line number Diff line change
Expand Up @@ -46,10 +46,6 @@ use crate::{

const LOG_TARGET: &str = "tari::dan::consensus::hotstuff::common";

/// The value that fees are divided by to determine the amount of fees to burn. 0 means no fees are burned.
/// This is a placeholder for the fee exhaust consensus constant so that we know where it's used later.
pub const EXHAUST_DIVISOR: u64 = 20; // 5%

/// Calculates the dummy block required to reach the new height and returns the last dummy block (parent for next
/// proposal).
pub fn calculate_last_dummy_block<TAddr: NodeAddressable, TLeaderStrategy: LeaderStrategy<TAddr>>(
Expand Down
10 changes: 3 additions & 7 deletions dan_layer/consensus/src/hotstuff/config.rs
Original file line number Diff line number Diff line change
@@ -1,18 +1,14 @@
// Copyright 2023 The Tari Project
// SPDX-License-Identifier: BSD-3-Clause

use std::time::Duration;

use tari_common::configuration::Network;
use tari_crypto::ristretto::RistrettoPublicKey;
use tari_dan_common_types::NumPreshards;

use crate::consensus_constants::ConsensusConstants;

#[derive(Debug, Clone)]
pub struct HotstuffConfig {
pub network: Network,
pub max_base_layer_blocks_ahead: u64,
pub max_base_layer_blocks_behind: u64,
pub num_preshards: NumPreshards,
pub pacemaker_max_base_time: Duration,
pub sidechain_id: Option<RistrettoPublicKey>,
pub consensus_constants: ConsensusConstants,
}
12 changes: 8 additions & 4 deletions dan_layer/consensus/src/hotstuff/on_propose.rs
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,6 @@ use crate::{
TransactionLockConflicts,
},
HotstuffConfig,
EXHAUST_DIVISOR,
},
messages::{HotstuffMessage, ProposalMessage},
tracing::TraceTimer,
Expand Down Expand Up @@ -497,7 +496,11 @@ where TConsensusSpec: ConsensusSpec
);

// batch is empty for is_empty, is_epoch_end and is_epoch_start blocks
let mut substate_store = PendingSubstateStore::new(tx, *parent_block.block_id(), self.config.num_preshards);
let mut substate_store = PendingSubstateStore::new(
tx,
*parent_block.block_id(),
self.config.consensus_constants.num_preshards,
);
let mut executed_transactions = HashMap::new();
let timer = TraceTimer::info(LOG_TARGET, "Generating commands").with_iterations(batch.len());
let mut lock_conflicts = TransactionLockConflicts::new();
Expand Down Expand Up @@ -657,7 +660,8 @@ where TConsensusSpec: ConsensusSpec

if tx_rec.current_decision().is_commit() {
let involved = NonZeroU64::new(1).expect("1 > 0");
let leader_fee = tx_rec.calculate_leader_fee(involved, EXHAUST_DIVISOR);
let leader_fee =
tx_rec.calculate_leader_fee(involved, self.config.consensus_constants.fee_exhaust_divisor);
tx_rec.set_leader_fee(leader_fee);
let diff = execution.result().finalize.result.accept().ok_or_else(|| {
HotStuffError::InvariantError(format!(
Expand Down Expand Up @@ -863,7 +867,7 @@ where TConsensusSpec: ConsensusSpec
tx_rec.transaction_id(),
))
})?;
let leader_fee = tx_rec.calculate_leader_fee(involved, EXHAUST_DIVISOR);
let leader_fee = tx_rec.calculate_leader_fee(involved, self.config.consensus_constants.fee_exhaust_divisor);
tx_rec.set_leader_fee(leader_fee);
}
let atom = tx_rec.get_current_transaction_atom();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,6 @@ use crate::{
},
HotstuffConfig,
ProposalValidationError,
EXHAUST_DIVISOR,
},
tracing::TraceTimer,
traits::{ConsensusSpec, WriteableSubstateStore},
Expand Down Expand Up @@ -297,7 +296,8 @@ where TConsensusSpec: ConsensusSpec
) -> Result<(), HotStuffError> {
// Store used for transactions that have inputs without specific versions.
// It lives through the entire block so multiple transactions can be sequenced together in the same block
let mut substate_store = PendingSubstateStore::new(tx, *block.parent(), self.config.num_preshards);
let mut substate_store =
PendingSubstateStore::new(tx, *block.parent(), self.config.consensus_constants.num_preshards);
let mut total_leader_fee = 0;
let locked_block = LockedBlock::get(tx, block.epoch())?;

Expand Down Expand Up @@ -613,8 +613,10 @@ where TConsensusSpec: ConsensusSpec
return Ok(Some(NoVoteReason::NoLeaderFee));
}

let calculated_leader_fee =
tx_rec.calculate_leader_fee(NonZeroU64::new(1).expect("1 > 0"), EXHAUST_DIVISOR);
let calculated_leader_fee = tx_rec.calculate_leader_fee(
NonZeroU64::new(1).expect("1 > 0"),
self.config.consensus_constants.fee_exhaust_divisor,
);
if calculated_leader_fee != *atom.leader_fee.as_ref().expect("None already checked") {
warn!(
target: LOG_TARGET,
Expand Down Expand Up @@ -1226,7 +1228,8 @@ where TConsensusSpec: ConsensusSpec
let num_involved_shard_groups = tx_rec.evidence().num_shard_groups();
let involved = NonZeroU64::new(num_involved_shard_groups as u64)
.ok_or_else(|| HotStuffError::InvariantError("Number of involved shard groups is 0".to_string()))?;
let calculated_leader_fee = tx_rec.calculate_leader_fee(involved, EXHAUST_DIVISOR);
let calculated_leader_fee =
tx_rec.calculate_leader_fee(involved, self.config.consensus_constants.fee_exhaust_divisor);
if calculated_leader_fee != *leader_fee {
warn!(
target: LOG_TARGET,
Expand Down
10 changes: 6 additions & 4 deletions dan_layer/consensus/src/hotstuff/on_receive_local_proposal.rs
Original file line number Diff line number Diff line change
Expand Up @@ -314,7 +314,9 @@ impl<TConsensusSpec: ConsensusSpec> OnReceiveLocalProposalHandler<TConsensusSpec
if let Some(vn) = self.epoch_manager.get_our_validator_node(next_epoch).await.optional()? {
// TODO: Change VN db to include the shard group in the ValidatorNode struct.
let num_committees = self.epoch_manager.get_num_committees(next_epoch).await?;
let next_shard_group = vn.shard_key.to_shard_group(self.config.num_preshards, num_committees);
let next_shard_group = vn
.shard_key
.to_shard_group(self.config.consensus_constants.num_preshards, num_committees);
self.store.with_write_tx(|tx| {
// Generate checkpoint
create_epoch_checkpoint(tx, epoch, local_committee_info.shard_group())?;
Expand Down Expand Up @@ -412,7 +414,7 @@ impl<TConsensusSpec: ConsensusSpec> OnReceiveLocalProposalHandler<TConsensusSpec
self.leader_strategy.clone(),
self.outbound_messaging.clone(),
self.store.clone(),
self.config.num_preshards,
self.config.consensus_constants.num_preshards,
local_committee_info,
blocks,
));
Expand Down Expand Up @@ -611,7 +613,7 @@ impl<TConsensusSpec: ConsensusSpec> OnReceiveLocalProposalHandler<TConsensusSpec
block_description: candidate_block.to_string(),
justify_block: candidate_block.justify().as_leaf_block(),
}
.into());
.into());
};

if justify_block.height() != candidate_block.justify().block_height() {
Expand Down Expand Up @@ -660,7 +662,7 @@ impl<TConsensusSpec: ConsensusSpec> OnReceiveLocalProposalHandler<TConsensusSpec
justify_block_height: justify_block.height(),
candidate_block_height: candidate_block.height(),
}
.into());
.into());
};

if candidate_block.parent() != last_dummy.id() {
Expand Down
Loading

0 comments on commit 686ced9

Please sign in to comment.