From de14ad9c7f277ae510a2ad88eb90d3779fde6f4f Mon Sep 17 00:00:00 2001 From: Stan Bondi Date: Tue, 16 Jul 2024 17:50:14 +0400 Subject: [PATCH] feat!: state checkpoint sync, fixes, epoch change (#1067) Description --- feat: adds state sync fix: remove block sync feat: start new chain at each epoch fix: fixes bug that drops consensus messages (inbound messaging not cancel safe) fix: scope JMT to epoch/shard fix: only emit epoch changed event once scanning to tip has completed fix: swarm mines more initial blocks when registering many vns fix: bug that caused block signature to disappear after unparking the block feat: adds state checkpoint call test: fix epoch change and state sync cucumbers Motivation and Context --- State sync fetches state transitions and applies them to the local state db. Syncing historical blocks is not feasible across changing shards/epochs and this is removed in this PR. Closes #1070 Closes #1037 Outstanding items: - verify checkpoint on sync - construct a "tiered" state merkle root consisting of presharded JMT merkle roots applicable to the current shard space. - leader failure due to race condition when proposal for end of epoch is received before all validators have scanned to the base layer tip How Has This Been Tested? --- Manually: 1. 2 validators, mine to next epoch 2. 2 validators, send transactions, delete data of one vn and resync 3. 4 validators, delete data from one validator, leader failures, epoch change, resync validator 4. [TODO] new validator joining network 5. [TODO] multishard test What process can a PR reviewer use to test or verify this change? --- Swarm, mine to next epoch Breaking Changes --- - [ ] None - [x] Requires data directory to be deleted - [x] Other - Please specify --- .../src/base_layer_scanner.rs | 5 +- .../src/transaction_executor.rs | 4 +- .../sqlite_substate_store_factory.rs | 5 +- .../src/process_manager/instances/manager.rs | 4 +- .../src/process_manager/manager.rs | 23 +- .../processes/validator_node.rs | 8 +- .../src/webserver/rpc/validator_nodes.rs | 1 + .../tari_validator_node/src/bootstrap.rs | 13 +- .../src/consensus/handle.rs | 13 +- .../tari_validator_node/src/consensus/mod.rs | 9 +- .../src/json_rpc/handlers.rs | 4 + .../rpc/{sync_task.rs => block_sync_task.rs} | 124 +-- .../tari_validator_node/src/p2p/rpc/mod.rs | 5 +- .../src/p2p/rpc/service_impl.rs | 117 ++- .../src/p2p/rpc/state_sync_task.rs | 130 +++ .../src/p2p/services/mempool/executor.rs | 7 +- .../src/p2p/services/mempool/service.rs | 36 +- .../src/substate_resolver.rs | 3 + .../src/routes/Blocks/BlockDetails.tsx | 35 +- bindings/index.ts | 1 - bindings/src/types/Command.ts | 3 +- bindings/src/types/EpochEvent.ts | 3 - bindings/src/types/SubstateDestroyed.ts | 2 + bindings/src/types/SubstateRecord.ts | 2 + clients/validator_node_client/src/lib.rs | 7 + dan_layer/common_types/src/committee.rs | 4 + dan_layer/common_types/src/epoch.rs | 24 +- dan_layer/common_types/src/node_height.rs | 2 +- dan_layer/common_types/src/shard.rs | 4 + dan_layer/consensus/src/block_validations.rs | 49 +- .../src/hotstuff/block_change_set.rs | 2 + dan_layer/consensus/src/hotstuff/common.rs | 95 +- .../consensus/src/hotstuff/current_height.rs | 51 -- .../consensus/src/hotstuff/current_view.rs | 68 ++ dan_layer/consensus/src/hotstuff/error.rs | 12 +- dan_layer/consensus/src/hotstuff/event.rs | 10 +- dan_layer/consensus/src/hotstuff/mod.rs | 5 +- dan_layer/consensus/src/hotstuff/on_beat.rs | 6 + .../src/hotstuff/on_catch_up_sync.rs | 65 ++ .../consensus/src/hotstuff/on_force_beat.rs | 6 + .../src/hotstuff/on_inbound_message.rs | 427 ++------- .../src/hotstuff/on_leader_timeout.rs | 6 + .../src/hotstuff/on_message_validate.rs | 354 ++++++++ .../src/hotstuff/on_next_sync_view.rs | 13 +- .../consensus/src/hotstuff/on_propose.rs | 94 +- .../on_ready_to_vote_on_local_block.rs | 299 ++----- .../hotstuff/on_receive_foreign_proposal.rs | 4 +- .../src/hotstuff/on_receive_local_proposal.rs | 300 +++++-- .../src/hotstuff/on_receive_new_view.rs | 14 +- ...on_receive_request_missing_transactions.rs | 11 +- .../consensus/src/hotstuff/on_receive_vote.rs | 9 - .../consensus/src/hotstuff/on_sync_request.rs | 56 +- dan_layer/consensus/src/hotstuff/pacemaker.rs | 28 +- .../src/hotstuff/pacemaker_handle.rs | 62 +- dan_layer/consensus/src/hotstuff/proposer.rs | 2 +- .../src/hotstuff/state_machine/idle.rs | 5 +- .../src/hotstuff/state_machine/running.rs | 2 - .../src/hotstuff/state_machine/worker.rs | 2 + .../substate_store/chain_scoped_tree_store.rs | 66 ++ .../src/hotstuff/substate_store/mod.rs | 2 + .../hotstuff/substate_store/pending_store.rs | 35 +- .../consensus/src/hotstuff/vote_receiver.rs | 70 +- dan_layer/consensus/src/hotstuff/worker.rs | 342 +++---- dan_layer/consensus/src/messages/message.rs | 15 +- dan_layer/consensus/src/messages/proposal.rs | 8 + dan_layer/consensus/src/messages/vote.rs | 12 + dan_layer/consensus_tests/src/consensus.rs | 261 +++++- .../consensus_tests/src/substate_store.rs | 26 +- .../consensus_tests/src/support/address.rs | 4 + .../src/support/epoch_manager.rs | 5 + .../src/support/executions_store.rs | 3 +- .../consensus_tests/src/support/harness.rs | 69 +- .../consensus_tests/src/support/helpers.rs | 12 + .../consensus_tests/src/support/network.rs | 1 + .../src/support/signing_service.rs | 9 +- .../src/support/transaction.rs | 29 +- .../src/support/transaction_executor.rs | 2 +- .../src/support/validator/builder.rs | 12 +- .../base_layer/base_layer_epoch_manager.rs | 27 +- .../src/base_layer/epoch_manager_service.rs | 3 + .../epoch_manager/src/base_layer/handle.rs | 10 + .../epoch_manager/src/base_layer/types.rs | 3 + dan_layer/epoch_manager/src/traits.rs | 2 + dan_layer/p2p/proto/consensus.proto | 30 +- dan_layer/p2p/proto/rpc.proto | 48 +- dan_layer/p2p/src/conversions/consensus.rs | 127 ++- dan_layer/p2p/src/conversions/rpc.rs | 86 +- dan_layer/rpc_state_sync/src/error.rs | 8 +- dan_layer/rpc_state_sync/src/lib.rs | 1 + dan_layer/rpc_state_sync/src/manager.rs | 846 +++++------------- dan_layer/rpc_state_sync/src/manager_old.rs | 717 +++++++++++++++ .../up.sql | 49 +- .../src/chain_scoped_tree_store.rs | 104 +++ dan_layer/state_store_sqlite/src/lib.rs | 2 +- dan_layer/state_store_sqlite/src/reader.rs | 231 ++++- dan_layer/state_store_sqlite/src/schema.rs | 31 +- .../src/sql_models/bookkeeping.rs | 12 +- .../src/sql_models/leaf_block.rs | 4 +- .../state_store_sqlite/src/sql_models/mod.rs | 2 + .../src/sql_models/state_transition.rs | 91 ++ .../src/sql_models/substate.rs | 18 +- dan_layer/state_store_sqlite/src/writer.rs | 218 +++-- dan_layer/state_tree/src/jellyfish/types.rs | 10 +- dan_layer/state_tree/src/tree.rs | 14 +- dan_layer/state_tree/tests/support.rs | 2 +- .../storage/src/consensus_models/block.rs | 137 +-- .../storage/src/consensus_models/command.rs | 31 +- .../src/consensus_models/epoch_checkpoint.rs | 66 ++ .../consensus_models/executed_transaction.rs | 21 +- .../src/consensus_models/foreign_proposal.rs | 6 +- .../storage/src/consensus_models/high_qc.rs | 8 +- .../src/consensus_models/last_executed.rs | 3 +- .../src/consensus_models/last_proposed.rs | 9 +- .../src/consensus_models/last_voted.rs | 13 +- .../src/consensus_models/leaf_block.rs | 20 +- .../src/consensus_models/locked_block.rs | 7 +- dan_layer/storage/src/consensus_models/mod.rs | 4 + .../consensus_models/quorum_certificate.rs | 22 +- .../src/consensus_models/state_transition.rs | 133 +++ .../src/consensus_models/state_tree_diff.rs | 15 +- .../storage/src/consensus_models/substate.rs | 102 +-- .../src/consensus_models/transaction.rs | 9 +- .../consensus_models/transaction_execution.rs | 7 +- .../storage/src/global/backend_adapter.rs | 4 +- .../storage/src/global/validator_node_db.rs | 6 +- dan_layer/storage/src/state_store/mod.rs | 58 +- .../src/global/backend_adapter.rs | 21 +- dan_layer/storage_sqlite/tests/global_db.rs | 2 +- dan_layer/template_builtin/build.rs | 23 +- dan_layer/validator_node_rpc/src/client.rs | 4 + .../validator_node_rpc/src/rpc_service.rs | 12 + .../tests/features/epoch_change.feature | 17 +- ...{block_sync.feature => state_sync.feature} | 21 +- .../tests/steps/validator_node.rs | 81 +- networking/core/src/worker.rs | 4 +- networking/libp2p-messaging/src/handler.rs | 2 +- networking/rpc_framework/src/client/mod.rs | 10 +- networking/rpc_framework/src/server/mod.rs | 2 +- networking/rpc_framework/src/status.rs | 52 +- 139 files changed, 4714 insertions(+), 2552 deletions(-) rename applications/tari_validator_node/src/p2p/rpc/{sync_task.rs => block_sync_task.rs} (67%) create mode 100644 applications/tari_validator_node/src/p2p/rpc/state_sync_task.rs delete mode 100644 bindings/src/types/EpochEvent.ts delete mode 100644 dan_layer/consensus/src/hotstuff/current_height.rs create mode 100644 dan_layer/consensus/src/hotstuff/current_view.rs create mode 100644 dan_layer/consensus/src/hotstuff/on_catch_up_sync.rs create mode 100644 dan_layer/consensus/src/hotstuff/on_message_validate.rs create mode 100644 dan_layer/consensus/src/hotstuff/substate_store/chain_scoped_tree_store.rs create mode 100644 dan_layer/rpc_state_sync/src/manager_old.rs create mode 100644 dan_layer/state_store_sqlite/src/chain_scoped_tree_store.rs create mode 100644 dan_layer/state_store_sqlite/src/sql_models/state_transition.rs create mode 100644 dan_layer/storage/src/consensus_models/epoch_checkpoint.rs create mode 100644 dan_layer/storage/src/consensus_models/state_transition.rs rename integration_tests/tests/features/{block_sync.feature => state_sync.feature} (83%) diff --git a/applications/tari_dan_app_utilities/src/base_layer_scanner.rs b/applications/tari_dan_app_utilities/src/base_layer_scanner.rs index 375d1f74c..5d67a3807 100644 --- a/applications/tari_dan_app_utilities/src/base_layer_scanner.rs +++ b/applications/tari_dan_app_utilities/src/base_layer_scanner.rs @@ -44,7 +44,7 @@ use tari_crypto::{ ristretto::RistrettoPublicKey, tari_utilities::{hex::Hex, ByteArray}, }; -use tari_dan_common_types::{optional::Optional, Epoch, NodeAddressable, NodeHeight}; +use tari_dan_common_types::{optional::Optional, shard::Shard, Epoch, NodeAddressable, NodeHeight}; use tari_dan_storage::{ consensus_models::{Block, SubstateRecord}, global::{GlobalDb, MetadataKey}, @@ -451,7 +451,7 @@ impl BaseLayerScanner { }); self.state_store .with_write_tx(|tx| { - let genesis = Block::genesis(self.network); + let genesis = Block::genesis(self.network, Epoch(0), Shard::zero()); // TODO: This should be proposed in a block... SubstateRecord { @@ -463,6 +463,7 @@ impl BaseLayerScanner { created_justify: *genesis.justify().id(), created_block: *genesis.id(), created_height: NodeHeight::zero(), + created_by_shard: Shard::zero(), created_at_epoch: Epoch(0), destroyed: None, } diff --git a/applications/tari_dan_app_utilities/src/transaction_executor.rs b/applications/tari_dan_app_utilities/src/transaction_executor.rs index d29ec0b06..a2978b323 100644 --- a/applications/tari_dan_app_utilities/src/transaction_executor.rs +++ b/applications/tari_dan_app_utilities/src/transaction_executor.rs @@ -6,7 +6,7 @@ use std::{ time::{Duration, Instant}, }; -use indexmap::{IndexMap, IndexSet}; +use indexmap::IndexMap; use log::*; use tari_common::configuration::Network; use tari_common_types::types::PublicKey; @@ -53,7 +53,7 @@ impl ExecutionOutput { pub fn resolve_inputs( &self, inputs: IndexMap, - ) -> IndexSet { + ) -> Vec { if let Some(diff) = self.result.finalize.accept() { inputs .into_iter() diff --git a/applications/tari_indexer/src/substate_storage_sqlite/sqlite_substate_store_factory.rs b/applications/tari_indexer/src/substate_storage_sqlite/sqlite_substate_store_factory.rs index 8e1213712..d117502fa 100644 --- a/applications/tari_indexer/src/substate_storage_sqlite/sqlite_substate_store_factory.rs +++ b/applications/tari_indexer/src/substate_storage_sqlite/sqlite_substate_store_factory.rs @@ -38,6 +38,7 @@ use diesel::{ }; use diesel_migrations::{EmbeddedMigrations, MigrationHarness}; use log::*; +use tari_crypto::tari_utilities::hex::to_hex; use tari_dan_common_types::{shard::Shard, substate_type::SubstateType, Epoch}; use tari_dan_storage::{consensus_models::BlockId, StorageError}; use tari_dan_storage_sqlite::{error::SqliteStorageError, SqliteTransaction}; @@ -802,9 +803,9 @@ impl SubstateStoreWriteTransaction for SqliteSubstateStoreWriteTransaction<'_> { reason: format!("save_scanned_block_id error: {}", e), })?; - info!( + debug!( target: LOG_TARGET, - "Added new scanned block id {:?} for epoch {} and shard {:?}", new.last_block_id, new.epoch, new.shard + "Added new scanned block id {} for epoch {} and shard {:?}", to_hex(&new.last_block_id), new.epoch, new.shard ); Ok(()) diff --git a/applications/tari_swarm_daemon/src/process_manager/instances/manager.rs b/applications/tari_swarm_daemon/src/process_manager/instances/manager.rs index 5bf746274..78cde9cf9 100644 --- a/applications/tari_swarm_daemon/src/process_manager/instances/manager.rs +++ b/applications/tari_swarm_daemon/src/process_manager/instances/manager.rs @@ -362,11 +362,11 @@ fn forward_logs(path: PathBuf, reader: R, task::spawn(async move { let mut log_file = File::create(path).await.unwrap(); while let Some(output) = lines.next_line().await.unwrap() { - log::debug!("[{target}] {output}"); + log::debug!(target: "swarm", "[{target}] {output}"); log_file.write_all(output.as_bytes()).await.unwrap(); log_file.write_all(b"\n").await.unwrap(); log_file.flush().await.unwrap(); } - log::debug!("Process exited ({target})"); + log::debug!(target: "swarm", "Process exited ({target})"); }); } diff --git a/applications/tari_swarm_daemon/src/process_manager/manager.rs b/applications/tari_swarm_daemon/src/process_manager/manager.rs index ec2b3a810..f81463bbd 100644 --- a/applications/tari_swarm_daemon/src/process_manager/manager.rs +++ b/applications/tari_swarm_daemon/src/process_manager/manager.rs @@ -3,7 +3,7 @@ use std::{collections::HashMap, time::Duration}; -use anyhow::anyhow; +use anyhow::{anyhow, Context}; use log::info; use minotari_node_grpc_client::grpc; use tari_crypto::tari_utilities::ByteArray; @@ -56,10 +56,14 @@ impl ProcessManager { let num_vns = self.instance_manager.num_validator_nodes(); // Mine some initial funds, guessing 10 blocks to allow for coinbase maturity - self.mine(num_vns + 5).await?; - self.wait_for_wallet_funds(num_vns).await?; + self.mine(num_vns + 10).await.context("mining failed")?; + self.wait_for_wallet_funds(num_vns) + .await + .context("waiting for wallet funds")?; - self.register_all_validator_nodes().await?; + self.register_all_validator_nodes() + .await + .context("registering validator node via GRPC")?; loop { tokio::select! { @@ -180,6 +184,7 @@ impl ProcessManager { } async fn register_all_validator_nodes(&mut self) -> anyhow::Result<()> { + let mut skip = vec![]; for vn in self.instance_manager.validator_nodes_mut() { if !vn.instance_mut().check_running() { log::error!( @@ -187,7 +192,7 @@ impl ProcessManager { vn.instance().id(), vn.instance().name() ); - continue; + skip.push(vn.instance().id()); } } @@ -203,6 +208,9 @@ impl ProcessManager { })?; for vn in self.instance_manager.validator_nodes() { + if skip.contains(&vn.instance().id()) { + continue; + } info!("🟑 Registering validator node {}", vn.instance().name()); if let Err(err) = vn.wait_for_startup(Duration::from_secs(10)).await { log::error!( @@ -215,12 +223,12 @@ impl ProcessManager { let reg_info = vn.get_registration_info().await?; let tx_id = wallet.register_validator_node(reg_info).await?; - info!("🟒 Registered validator node with tx_id: {tx_id}"); + info!("🟒 Registered validator node {vn} with tx_id: {tx_id}"); // Just wait a bit :shrug: This could be a bug in the console wallet. If we submit too quickly it uses 0 // inputs for a transaction. sleep(Duration::from_secs(2)).await; } - self.mine(20).await?; + self.mine(10).await?; Ok(()) } @@ -257,7 +265,6 @@ impl ProcessManager { let reg_info = vn.get_registration_info().await?; wallet.register_validator_node(reg_info).await?; - self.mine(20).await?; Ok(()) } diff --git a/applications/tari_swarm_daemon/src/process_manager/processes/validator_node.rs b/applications/tari_swarm_daemon/src/process_manager/processes/validator_node.rs index e78fe75aa..2dc65a760 100644 --- a/applications/tari_swarm_daemon/src/process_manager/processes/validator_node.rs +++ b/applications/tari_swarm_daemon/src/process_manager/processes/validator_node.rs @@ -1,7 +1,7 @@ // Copyright 2024 The Tari Project // SPDX-License-Identifier: BSD-3-Clause -use std::time::Duration; +use std::{fmt::Display, time::Duration}; use anyhow::{anyhow, Context}; use serde::{Deserialize, Serialize}; @@ -89,6 +89,12 @@ impl ValidatorNodeProcess { } } +impl Display for ValidatorNodeProcess { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{} ({})", self.instance().name(), self.instance().id()) + } +} + #[derive(Serialize, Deserialize)] pub struct ValidatorRegistrationInfo { pub signature: ValidatorNodeSignature, diff --git a/applications/tari_swarm_daemon/src/webserver/rpc/validator_nodes.rs b/applications/tari_swarm_daemon/src/webserver/rpc/validator_nodes.rs index cc8f306c9..d3e9ef800 100644 --- a/applications/tari_swarm_daemon/src/webserver/rpc/validator_nodes.rs +++ b/applications/tari_swarm_daemon/src/webserver/rpc/validator_nodes.rs @@ -75,6 +75,7 @@ pub async fn create( if req.register { context.process_manager().register_validator_node(instance_id).await?; + context.process_manager().mine_blocks(10).await?; } Ok(ValidatorNodeCreateResponse { instance_id }) diff --git a/applications/tari_validator_node/src/bootstrap.rs b/applications/tari_validator_node/src/bootstrap.rs index f7bff68be..9a65a1c87 100644 --- a/applications/tari_validator_node/src/bootstrap.rs +++ b/applications/tari_validator_node/src/bootstrap.rs @@ -48,7 +48,7 @@ use tari_dan_app_utilities::{ template_manager::{implementation::TemplateManager, interface::TemplateManagerHandle}, transaction_executor::TariDanTransactionProcessor, }; -use tari_dan_common_types::{Epoch, NodeAddressable, NodeHeight, PeerAddress, SubstateAddress}; +use tari_dan_common_types::{shard::Shard, Epoch, NodeAddressable, NodeHeight, PeerAddress, SubstateAddress}; use tari_dan_engine::fees::FeeTable; use tari_dan_p2p::TariMessagingSpec; use tari_dan_storage::{ @@ -331,6 +331,7 @@ pub async fn spawn_services( spawn_p2p_rpc( config, &mut networking, + epoch_manager.clone(), state_store.clone(), mempool.clone(), virtual_substate_manager, @@ -422,6 +423,7 @@ impl Services { async fn spawn_p2p_rpc( config: &ApplicationConfig, networking: &mut NetworkingHandle, + epoch_manager: EpochManagerHandle, shard_store_store: SqliteStateStore, mempool: MempoolHandle, virtual_substate_manager: VirtualSubstateManager, EpochManagerHandle>, @@ -431,6 +433,7 @@ async fn spawn_p2p_rpc( .with_maximum_sessions_per_client(config.validator_node.rpc.max_sessions_per_client) .finish() .add_service(create_tari_validator_node_rpc_service( + epoch_manager, shard_store_store, mempool, virtual_substate_manager, @@ -451,7 +454,7 @@ where TTx::Target: StateStoreReadTransaction, TTx::Addr: NodeAddressable + Serialize, { - let genesis_block = Block::genesis(network); + let genesis_block = Block::genesis(network, Epoch(0), Shard::zero()); let substate_id = SubstateId::Resource(PUBLIC_IDENTITY_RESOURCE_ADDRESS); let substate_address = SubstateAddress::from_substate_id(&substate_id, 0); let mut metadata: Metadata = Default::default(); @@ -474,8 +477,9 @@ where state_hash: Default::default(), created_by_transaction: Default::default(), created_justify: *genesis_block.justify().id(), - created_block: BlockId::genesis(), + created_block: BlockId::zero(), created_height: NodeHeight(0), + created_by_shard: Shard::zero(), created_at_epoch: Epoch(0), destroyed: None, } @@ -503,9 +507,10 @@ where state_hash: Default::default(), created_by_transaction: Default::default(), created_justify: *genesis_block.justify().id(), - created_block: BlockId::genesis(), + created_block: BlockId::zero(), created_height: NodeHeight(0), created_at_epoch: Epoch(0), + created_by_shard: Shard::zero(), destroyed: None, } .create(tx)?; diff --git a/applications/tari_validator_node/src/consensus/handle.rs b/applications/tari_validator_node/src/consensus/handle.rs index 4e690c2e7..82db12b24 100644 --- a/applications/tari_validator_node/src/consensus/handle.rs +++ b/applications/tari_validator_node/src/consensus/handle.rs @@ -1,7 +1,7 @@ // Copyright 2023 The Tari Project // SPDX-License-Identifier: BSD-3-Clause -use tari_consensus::hotstuff::{ConsensusCurrentState, HotstuffEvent}; +use tari_consensus::hotstuff::{ConsensusCurrentState, CurrentView, HotstuffEvent}; use tokio::sync::{broadcast, watch}; use crate::event_subscription::EventSubscription; @@ -10,19 +10,26 @@ use crate::event_subscription::EventSubscription; pub struct ConsensusHandle { rx_current_state: watch::Receiver, events_subscription: EventSubscription, + current_view: CurrentView, } impl ConsensusHandle { pub(super) fn new( rx_current_state: watch::Receiver, events_subscription: EventSubscription, + current_view: CurrentView, ) -> Self { Self { rx_current_state, events_subscription, + current_view, } } + pub fn current_view(&self) -> &CurrentView { + &self.current_view + } + pub fn subscribe_to_hotstuff_events(&mut self) -> broadcast::Receiver { self.events_subscription.subscribe() } @@ -30,4 +37,8 @@ impl ConsensusHandle { pub fn get_current_state(&self) -> ConsensusCurrentState { *self.rx_current_state.borrow() } + + pub fn is_running(&self) -> bool { + self.get_current_state().is_running() + } } diff --git a/applications/tari_validator_node/src/consensus/mod.rs b/applications/tari_validator_node/src/consensus/mod.rs index eb0b1980b..fd16a42a7 100644 --- a/applications/tari_validator_node/src/consensus/mod.rs +++ b/applications/tari_validator_node/src/consensus/mod.rs @@ -96,12 +96,13 @@ pub async fn spawn( max_base_layer_blocks_ahead: consensus_constants.max_base_layer_blocks_ahead, }, ); + let current_view = hotstuff_worker.pacemaker().current_view().clone(); let (tx_current_state, rx_current_state) = watch::channel(Default::default()); let context = ConsensusWorkerContext { epoch_manager: epoch_manager.clone(), hotstuff: hotstuff_worker, - state_sync: RpcStateSyncManager::new(network, epoch_manager, store, leader_strategy, client_factory), + state_sync: RpcStateSyncManager::new(epoch_manager, store, client_factory), tx_current_state, }; @@ -109,7 +110,11 @@ pub async fn spawn( ( handle, - ConsensusHandle::new(rx_current_state, EventSubscription::new(tx_hotstuff_events)), + ConsensusHandle::new( + rx_current_state, + EventSubscription::new(tx_hotstuff_events), + current_view, + ), rx_mempool, ) } diff --git a/applications/tari_validator_node/src/json_rpc/handlers.rs b/applications/tari_validator_node/src/json_rpc/handlers.rs index f2b80d78b..f567290f3 100644 --- a/applications/tari_validator_node/src/json_rpc/handlers.rs +++ b/applications/tari_validator_node/src/json_rpc/handlers.rs @@ -563,6 +563,10 @@ impl JsonRpcHandlers { pub async fn get_epoch_manager_stats(&self, value: JsonRpcExtractor) -> JrpcResult { let answer_id = value.get_answer_id(); + self.epoch_manager + .wait_for_initial_scanning_to_complete() + .await + .map_err(internal_error(answer_id))?; let current_epoch = self.epoch_manager.current_epoch().await.map_err(|e| { JsonRpcResponse::error( answer_id, diff --git a/applications/tari_validator_node/src/p2p/rpc/sync_task.rs b/applications/tari_validator_node/src/p2p/rpc/block_sync_task.rs similarity index 67% rename from applications/tari_validator_node/src/p2p/rpc/sync_task.rs rename to applications/tari_validator_node/src/p2p/rpc/block_sync_task.rs index 7f8746613..b20ecb6d1 100644 --- a/applications/tari_validator_node/src/p2p/rpc/sync_task.rs +++ b/applications/tari_validator_node/src/p2p/rpc/block_sync_task.rs @@ -7,7 +7,7 @@ use log::*; use tari_dan_common_types::Epoch; use tari_dan_p2p::proto::rpc::{sync_blocks_response::SyncData, QuorumCertificates, SyncBlocksResponse, Transactions}; use tari_dan_storage::{ - consensus_models::{Block, BlockId, LeafBlock, LockedBlock, QuorumCertificate, SubstateUpdate, TransactionRecord}, + consensus_models::{Block, BlockId, QuorumCertificate, SubstateUpdate, TransactionRecord}, StateStore, StateStoreReadTransaction, StorageError, @@ -30,7 +30,7 @@ type BlockBuffer = Vec; pub struct BlockSyncTask { store: TStateStore, start_block: Block, - up_to_epoch: Option, + _up_to_epoch: Option, sender: mpsc::Sender>, } @@ -44,7 +44,7 @@ impl BlockSyncTask { Self { store, start_block, - up_to_epoch, + _up_to_epoch: up_to_epoch, sender, } } @@ -84,13 +84,13 @@ impl BlockSyncTask { } } - match self.fetch_last_blocks(&mut buffer, ¤t_block_id).await { - Ok(_) => (), - Err(err) => { - self.send(Err(RpcStatus::log_internal_error(LOG_TARGET)(err))).await?; - return Err(()); - }, - } + // match self.fetch_last_blocks(&mut buffer, ¤t_block_id).await { + // Ok(_) => (), + // Err(err) => { + // self.send(Err(RpcStatus::log_internal_error(LOG_TARGET)(err))).await?; + // return Err(()); + // }, + // } debug!( target: LOG_TARGET, @@ -140,58 +140,58 @@ impl BlockSyncTask { }) } - async fn fetch_last_blocks( - &self, - buffer: &mut BlockBuffer, - current_block_id: &BlockId, - ) -> Result<(), StorageError> { - // if let Some(up_to_epoch) = self.up_to_epoch { - // // Wait for the end of epoch block if the requested epoch has not yet completed - // // TODO: We should consider streaming blocks as they come in from consensus - // loop { - // let block = self.store.with_read_tx(|tx| LockedBlock::get(tx)?.get_block(tx))?; - // if block.is_epoch_end() && block.epoch() + Epoch(1) >= up_to_epoch { - // // If found the epoch end block, break. - // break; - // } - // tokio::time::sleep(Duration::from_secs(10)).await; - // } - // } - self.store.with_read_tx(|tx| { - // TODO: if there are any transactions in the block the syncing node will reject the block - - // If syncing to epoch, sync to the leaf block - let up_to_block = if self.up_to_epoch.is_none() { - let locked_block = LockedBlock::get(tx)?; - *locked_block.block_id() - } else { - let leaf_block = LeafBlock::get(tx)?; - *leaf_block.block_id() - }; - - let blocks = Block::get_all_blocks_between(tx, current_block_id, &up_to_block, false)?; - for block in blocks { - debug!( - target: LOG_TARGET, - "Fetching last blocks. Current block: {} to target {}", - block, - current_block_id - ); - let all_qcs = block - .commands() - .iter() - .filter(|cmd| cmd.transaction().is_some()) - .flat_map(|cmd| cmd.evidence().qc_ids_iter()) - .collect::>(); - let certificates = QuorumCertificate::get_all(tx, all_qcs)?; - - // No substate updates can occur for blocks after the last commit - buffer.push((block, certificates, vec![], vec![])); - } - - Ok::<_, StorageError>(()) - }) - } + // async fn fetch_last_blocks( + // &self, + // buffer: &mut BlockBuffer, + // current_block_id: &BlockId, + // ) -> Result<(), StorageError> { + // // if let Some(up_to_epoch) = self.up_to_epoch { + // // // Wait for the end of epoch block if the requested epoch has not yet completed + // // // TODO: We should consider streaming blocks as they come in from consensus + // // loop { + // // let block = self.store.with_read_tx(|tx| LockedBlock::get(tx)?.get_block(tx))?; + // // if block.is_epoch_end() && block.epoch() + Epoch(1) >= up_to_epoch { + // // // If found the epoch end block, break. + // // break; + // // } + // // tokio::time::sleep(Duration::from_secs(10)).await; + // // } + // // } + // self.store.with_read_tx(|tx| { + // // TODO: if there are any transactions in the block the syncing node will reject the block + // + // // If syncing to epoch, sync to the leaf block + // let up_to_block = if self.up_to_epoch.is_none() { + // let locked_block = LockedBlock::get(tx)?; + // *locked_block.block_id() + // } else { + // let leaf_block = LeafBlock::get(tx)?; + // *leaf_block.block_id() + // }; + // + // let blocks = Block::get_all_blocks_between(tx, current_block_id, &up_to_block, false)?; + // for block in blocks { + // debug!( + // target: LOG_TARGET, + // "Fetching last blocks. Current block: {} to target {}", + // block, + // current_block_id + // ); + // let all_qcs = block + // .commands() + // .iter() + // .filter(|cmd| cmd.transaction().is_some()) + // .flat_map(|cmd| cmd.evidence().qc_ids_iter()) + // .collect::>(); + // let certificates = QuorumCertificate::get_all(tx, all_qcs)?; + // + // // No substate updates can occur for blocks after the last commit + // buffer.push((block, certificates, vec![], vec![])); + // } + // + // Ok::<_, StorageError>(()) + // }) + // } async fn send(&mut self, result: Result) -> Result<(), ()> { if self.sender.send(result).await.is_err() { diff --git a/applications/tari_validator_node/src/p2p/rpc/mod.rs b/applications/tari_validator_node/src/p2p/rpc/mod.rs index c99f5581e..98a283a64 100644 --- a/applications/tari_validator_node/src/p2p/rpc/mod.rs +++ b/applications/tari_validator_node/src/p2p/rpc/mod.rs @@ -20,8 +20,9 @@ // WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE // USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +mod block_sync_task; mod service_impl; -mod sync_task; +mod state_sync_task; pub use service_impl::ValidatorNodeRpcServiceImpl; use tari_dan_common_types::PeerAddress; @@ -32,11 +33,13 @@ use tari_validator_node_rpc::rpc_service::ValidatorNodeRpcServer; use crate::{p2p::services::mempool::MempoolHandle, virtual_substate::VirtualSubstateManager}; pub fn create_tari_validator_node_rpc_service( + epoch_manager: EpochManagerHandle, shard_store_store: SqliteStateStore, mempool: MempoolHandle, virtual_substate_manager: VirtualSubstateManager, EpochManagerHandle>, ) -> ValidatorNodeRpcServer { ValidatorNodeRpcServer::new(ValidatorNodeRpcServiceImpl::new( + epoch_manager, shard_store_store, mempool, virtual_substate_manager, diff --git a/applications/tari_validator_node/src/p2p/rpc/service_impl.rs b/applications/tari_validator_node/src/p2p/rpc/service_impl.rs index e3e5e494a..928643d6a 100644 --- a/applications/tari_validator_node/src/p2p/rpc/service_impl.rs +++ b/applications/tari_validator_node/src/p2p/rpc/service_impl.rs @@ -24,10 +24,12 @@ use std::convert::{TryFrom, TryInto}; use log::*; use tari_bor::{decode_exact, encode}; -use tari_dan_common_types::{optional::Optional, PeerAddress, SubstateAddress}; +use tari_dan_common_types::{optional::Optional, shard::Shard, Epoch, PeerAddress, SubstateAddress}; use tari_dan_p2p::{ proto, proto::rpc::{ + GetCheckpointRequest, + GetCheckpointResponse, GetHighQcRequest, GetHighQcResponse, GetSubstateRequest, @@ -38,14 +40,26 @@ use tari_dan_p2p::{ SubstateStatus, SyncBlocksRequest, SyncBlocksResponse, + SyncStateRequest, + SyncStateResponse, }, }; use tari_dan_storage::{ - consensus_models::{Block, BlockId, HighQc, LockedBlock, QuorumCertificate, SubstateRecord, TransactionRecord}, + consensus_models::{ + Block, + BlockId, + EpochCheckpoint, + HighQc, + LockedBlock, + QuorumCertificate, + StateTransitionId, + SubstateRecord, + TransactionRecord, + }, StateStore, }; use tari_engine_types::virtual_substate::VirtualSubstateId; -use tari_epoch_manager::base_layer::EpochManagerHandle; +use tari_epoch_manager::{base_layer::EpochManagerHandle, EpochManagerReader}; use tari_rpc_framework::{Request, Response, RpcStatus, Streaming}; use tari_state_store_sqlite::SqliteStateStore; use tari_transaction::{Transaction, TransactionId}; @@ -53,13 +67,17 @@ use tari_validator_node_rpc::rpc_service::ValidatorNodeRpcService; use tokio::{sync::mpsc, task}; use crate::{ - p2p::{rpc::sync_task::BlockSyncTask, services::mempool::MempoolHandle}, + p2p::{ + rpc::{block_sync_task::BlockSyncTask, state_sync_task::StateSyncTask}, + services::mempool::MempoolHandle, + }, virtual_substate::VirtualSubstateManager, }; const LOG_TARGET: &str = "tari::dan::p2p::rpc"; pub struct ValidatorNodeRpcServiceImpl { + epoch_manager: EpochManagerHandle, shard_state_store: SqliteStateStore, mempool: MempoolHandle, virtual_substate_manager: VirtualSubstateManager, EpochManagerHandle>, @@ -67,6 +85,7 @@ pub struct ValidatorNodeRpcServiceImpl { impl ValidatorNodeRpcServiceImpl { pub fn new( + epoch_manager: EpochManagerHandle, shard_state_store: SqliteStateStore, mempool: MempoolHandle, virtual_substate_manager: VirtualSubstateManager< @@ -75,6 +94,7 @@ impl ValidatorNodeRpcServiceImpl { >, ) -> Self { Self { + epoch_manager, shard_state_store, mempool, virtual_substate_manager, @@ -93,14 +113,14 @@ impl ValidatorNodeRpcService for ValidatorNodeRpcServiceImpl { .transaction .ok_or_else(|| RpcStatus::bad_request("Missing transaction"))? .try_into() - .map_err(|e| RpcStatus::bad_request(&format!("Malformed transaction: {}", e)))?; + .map_err(|e| RpcStatus::bad_request(format!("Malformed transaction: {}", e)))?; let transaction_id = *transaction.id(); self.mempool .submit_transaction(transaction) .await - .map_err(|e| RpcStatus::bad_request(&format!("Invalid transaction: {}", e)))?; + .map_err(|e| RpcStatus::bad_request(format!("Invalid transaction: {}", e)))?; debug!(target: LOG_TARGET, "Accepted instruction into mempool"); @@ -113,7 +133,7 @@ impl ValidatorNodeRpcService for ValidatorNodeRpcServiceImpl { let req = req.into_message(); let address = SubstateAddress::from_bytes(&req.address) - .map_err(|e| RpcStatus::bad_request(&format!("Invalid encoded substate id: {}", e)))?; + .map_err(|e| RpcStatus::bad_request(format!("Invalid encoded substate id: {}", e)))?; let tx = self .shard_state_store @@ -177,7 +197,7 @@ impl ValidatorNodeRpcService for ValidatorNodeRpcServiceImpl { let req = req.into_message(); let address = decode_exact::(&req.address) - .map_err(|e| RpcStatus::bad_request(&format!("Invalid encoded substate id: {}", e)))?; + .map_err(|e| RpcStatus::bad_request(format!("Invalid encoded substate id: {}", e)))?; let substate = self .virtual_substate_manager @@ -186,8 +206,7 @@ impl ValidatorNodeRpcService for ValidatorNodeRpcServiceImpl { .map_err(RpcStatus::log_internal_error(LOG_TARGET))?; let resp = proto::rpc::GetVirtualSubstateResponse { - substate: encode(&substate) - .map_err(|e| RpcStatus::general(&format!("Unable to encode substate: {}", e)))?, + substate: encode(&substate).map_err(|e| RpcStatus::general(format!("Unable to encode substate: {}", e)))?, // TODO: evidence for the correctness of the substate quorum_certificates: vec![], }; @@ -254,12 +273,12 @@ impl ValidatorNodeRpcService for ValidatorNodeRpcServiceImpl { let (sender, receiver) = mpsc::channel(10); let start_block_id = BlockId::try_from(req.start_block_id) - .map_err(|e| RpcStatus::bad_request(&format!("Invalid encoded block id: {}", e)))?; + .map_err(|e| RpcStatus::bad_request(format!("Invalid encoded block id: {}", e)))?; // Check if we have the blocks let start_block = store .with_read_tx(|tx| Block::get(tx, &start_block_id).optional()) .map_err(RpcStatus::log_internal_error(LOG_TARGET))? - .ok_or_else(|| RpcStatus::not_found(&format!("start_block_id {start_block_id} not found")))?; + .ok_or_else(|| RpcStatus::not_found(format!("start_block_id {start_block_id} not found")))?; // Check that the start block let locked_block = store @@ -267,7 +286,7 @@ impl ValidatorNodeRpcService for ValidatorNodeRpcServiceImpl { .map_err(RpcStatus::log_internal_error(LOG_TARGET))? .ok_or_else(|| RpcStatus::not_found("No locked block"))?; if start_block.height() > locked_block.height() { - return Err(RpcStatus::not_found(&format!( + return Err(RpcStatus::not_found(format!( "start_block_id {} is after locked block {}", start_block_id, locked_block ))); @@ -302,4 +321,76 @@ impl ValidatorNodeRpcService for ValidatorNodeRpcServiceImpl { high_qc: Some((&high_qc).into()), })) } + + async fn get_checkpoint( + &self, + request: Request, + ) -> Result, RpcStatus> { + let msg = request.into_message(); + let current_epoch = self + .epoch_manager + .current_epoch() + .await + .map_err(RpcStatus::log_internal_error(LOG_TARGET))?; + if msg.current_epoch != current_epoch { + // This may occur if one of the nodes has not fully scanned the base layer + return Err(RpcStatus::bad_request(format!( + "Peer requested checkpoint with epoch {} but current epoch is {}", + msg.current_epoch, current_epoch + ))); + } + + let prev_epoch = current_epoch.saturating_sub(Epoch(1)); + if prev_epoch.is_zero() { + return Err(RpcStatus::not_found("Cannot generate checkpoint for genesis epoch")); + } + + let Some(local_committee_info) = self + .epoch_manager + .get_local_committee_info(prev_epoch) + .await + .optional() + .map_err(RpcStatus::log_internal_error(LOG_TARGET))? else + { + return Err(RpcStatus::bad_request(format!( + "This validator node is not registered for the previous epoch {prev_epoch}" + ))); + }; + + let checkpoint = self + .shard_state_store + .with_read_tx(|tx| EpochCheckpoint::generate(tx, prev_epoch, local_committee_info.shard())) + .optional() + .map_err(RpcStatus::log_internal_error(LOG_TARGET))?; + + Ok(Response::new(GetCheckpointResponse { + checkpoint: checkpoint.map(Into::into), + })) + } + + async fn sync_state(&self, request: Request) -> Result, RpcStatus> { + let req = request.into_message(); + + let (sender, receiver) = mpsc::channel(10); + + let last_state_transition_for_chain = + StateTransitionId::new(Epoch(req.start_epoch), Shard::from(req.start_shard), req.start_seq); + + // TODO: validate that we can provide the required sync data + let current_shard = Shard::from(req.current_shard); + let current_epoch = Epoch(req.current_epoch); + info!(target: LOG_TARGET, "🌍peer initiated sync with this node ({current_epoch}, {current_shard})"); + + task::spawn( + StateSyncTask::new( + self.shard_state_store.clone(), + sender, + last_state_transition_for_chain, + current_epoch, + ) + .run(), + ); + + Ok(Streaming::new(receiver)) + } } diff --git a/applications/tari_validator_node/src/p2p/rpc/state_sync_task.rs b/applications/tari_validator_node/src/p2p/rpc/state_sync_task.rs new file mode 100644 index 000000000..24322886a --- /dev/null +++ b/applications/tari_validator_node/src/p2p/rpc/state_sync_task.rs @@ -0,0 +1,130 @@ +// Copyright 2023 The Tari Project +// SPDX-License-Identifier: BSD-3-Clause + +use log::*; +use tari_dan_common_types::Epoch; +use tari_dan_p2p::proto::rpc::SyncStateResponse; +use tari_dan_storage::{ + consensus_models::{StateTransition, StateTransitionId}, + StateStore, + StorageError, +}; +use tari_rpc_framework::RpcStatus; +use tokio::sync::mpsc; + +const LOG_TARGET: &str = "tari::dan::rpc::sync_task"; + +const BATCH_SIZE: usize = 100; + +type UpdateBuffer = Vec; + +pub struct StateSyncTask { + store: TStateStore, + sender: mpsc::Sender>, + start_state_transition_id: StateTransitionId, + current_epoch: Epoch, +} + +impl StateSyncTask { + pub fn new( + store: TStateStore, + sender: mpsc::Sender>, + start_state_transition_id: StateTransitionId, + current_epoch: Epoch, + ) -> Self { + Self { + store, + sender, + start_state_transition_id, + current_epoch, + } + } + + pub async fn run(mut self) -> Result<(), ()> { + let mut buffer = Vec::with_capacity(BATCH_SIZE); + let mut current_state_transition_id = self.start_state_transition_id; + let mut counter = 0; + loop { + match self.fetch_next_batch(&mut buffer, current_state_transition_id) { + Ok(Some(last_state_transition_id)) => { + info!(target: LOG_TARGET, "🌍Fetched {} state transitions up to transition {}", buffer.len(), last_state_transition_id); + current_state_transition_id = last_state_transition_id; + }, + Ok(None) => { + // TODO: differentiate between not found and end of stream + // self.send(Err(RpcStatus::not_found(format!( + // "State transition not found with id={current_state_transition_id}" + // )))) + // .await?; + + // Finished + return Ok(()); + }, + Err(err) => { + self.send(Err(RpcStatus::log_internal_error(LOG_TARGET)(err))).await?; + return Err(()); + }, + } + + let num_items = buffer.len(); + debug!( + target: LOG_TARGET, + "Sending {num_items} state updates to peer. Current transition id: {current_state_transition_id}", + ); + + counter += buffer.len(); + self.send_state_transitions(buffer.drain(..)).await?; + + // If we didn't fill up the buffer, so we're done + if num_items < buffer.capacity() { + debug!( target: LOG_TARGET, "Sync to last commit complete. Streamed {} item(s)", counter); + break; + } + } + + Ok(()) + } + + fn fetch_next_batch( + &self, + buffer: &mut UpdateBuffer, + current_state_transition_id: StateTransitionId, + ) -> Result, StorageError> { + self.store.with_read_tx(|tx| { + let state_transitions = + StateTransition::get_n_after(tx, BATCH_SIZE, current_state_transition_id, self.current_epoch)?; + + let Some(last) = state_transitions.last() else { + return Ok(None); + }; + + let last_state_transition_id = last.id; + buffer.extend(state_transitions); + + Ok::<_, StorageError>(Some(last_state_transition_id)) + }) + } + + async fn send(&mut self, result: Result) -> Result<(), ()> { + if self.sender.send(result).await.is_err() { + debug!( + target: LOG_TARGET, + "Peer stream closed by client before completing. Aborting" + ); + return Err(()); + } + Ok(()) + } + + async fn send_state_transitions>( + &mut self, + state_transitions: I, + ) -> Result<(), ()> { + self.send(Ok(SyncStateResponse { + transitions: state_transitions.into_iter().map(Into::into).collect(), + })) + .await?; + + Ok(()) + } +} diff --git a/applications/tari_validator_node/src/p2p/services/mempool/executor.rs b/applications/tari_validator_node/src/p2p/services/mempool/executor.rs index d888c0ed0..946fcfaf2 100644 --- a/applications/tari_validator_node/src/p2p/services/mempool/executor.rs +++ b/applications/tari_validator_node/src/p2p/services/mempool/executor.rs @@ -1,7 +1,6 @@ // Copyright 2023 The Tari Project // SPDX-License-Identifier: BSD-3-Clause -use indexmap::IndexSet; use log::*; use tari_dan_app_utilities::transaction_executor::{TransactionExecutor, TransactionProcessorError}; use tari_dan_common_types::Epoch; @@ -74,7 +73,7 @@ where let versioned_inputs = local_substates .iter() .map(|(id, substate)| VersionedSubstateId::new(id.clone(), substate.version())) - .collect::>(); + .collect::>(); let state_db = new_state_db(); state_db.set_many(local_substates).expect("memory db is infallible"); @@ -94,7 +93,7 @@ where }; VersionedSubstateIdLockIntent::new(versioned_id, lock_flag) }) - .collect::>() + .collect() } else { versioned_inputs .into_iter() @@ -105,7 +104,7 @@ where // involved. VersionedSubstateIdLockIntent::new(versioned_id, SubstateLockFlag::Write) }) - .collect::>() + .collect() }; Ok(ExecutedTransaction::new( diff --git a/applications/tari_validator_node/src/p2p/services/mempool/service.rs b/applications/tari_validator_node/src/p2p/services/mempool/service.rs index c63bb2e31..1a583d8c9 100644 --- a/applications/tari_validator_node/src/p2p/services/mempool/service.rs +++ b/applications/tari_validator_node/src/p2p/services/mempool/service.rs @@ -260,7 +260,7 @@ where output_shards: unverified_output_shards, } = *msg; - if !self.consensus_handle.get_current_state().is_running() { + if !self.consensus_handle.is_running() { info!( target: LOG_TARGET, "🎱 Transaction {} received while not in running state. Ignoring", @@ -280,7 +280,7 @@ where transaction ); - let current_epoch = self.epoch_manager.current_epoch().await?; + let current_epoch = self.consensus_handle.current_view().get_epoch(); let num_committees = self.epoch_manager.get_num_committees(current_epoch).await?; let maybe_sender_shard = self .epoch_manager @@ -356,7 +356,7 @@ where warn!(target: LOG_TARGET, "⚠ No involved shards for payload"); } - let current_epoch = self.epoch_manager.current_epoch().await?; + let current_epoch = self.consensus_handle.current_view().get_epoch(); let tx_substate_address = SubstateAddress::for_transaction_receipt(transaction.id().into_receipt_address()); let local_committee_shard = self.epoch_manager.get_local_committee_info(current_epoch).await?; @@ -535,15 +535,13 @@ where async fn handle_deferred_execution(&mut self, transaction: Transaction) -> Result<(), MempoolError> { let transaction_id = *transaction.id(); - let is_consensus_running = self.consensus_handle.get_current_state().is_running(); - let pending_exec_size = self.pending_executions.len(); - if is_consensus_running && - // Notify consensus about the transaction - self.tx_executed_transactions - .send((transaction_id, pending_exec_size)) - .await - .is_err() + // Notify consensus about the transaction + if self + .tx_executed_transactions + .send((transaction_id, pending_exec_size)) + .await + .is_err() { debug!( target: LOG_TARGET, @@ -585,8 +583,6 @@ where return Ok(()); } - let is_consensus_running = self.consensus_handle.get_current_state().is_running(); - let executed = match exec_result { Ok(mut executed) => { info!( @@ -694,7 +690,7 @@ where }, }; - let current_epoch = self.epoch_manager.current_epoch().await?; + let current_epoch = self.consensus_handle.current_view().get_epoch(); let local_committee_shard = self.epoch_manager.get_local_committee_info(current_epoch).await?; let all_inputs_iter = executed.all_inputs_iter().map(|i| i.to_substate_address()); @@ -707,7 +703,7 @@ where let num_committees = self.epoch_manager.get_num_committees(current_epoch).await?; let input_shards = executed .resolved_inputs() - .into_iter() + .iter() .map(|s| s.versioned_substate_id().to_committee_shard(num_committees)) .collect::>(); let tx_substate_address = SubstateAddress::for_transaction_receipt(executed.id().into_receipt_address()); @@ -746,11 +742,11 @@ where // Notify consensus that a transaction is ready to go! let pending_exec_size = self.pending_executions.len(); - if is_consensus_running && - self.tx_executed_transactions - .send((*executed.id(), pending_exec_size)) - .await - .is_err() + if self + .tx_executed_transactions + .send((*executed.id(), pending_exec_size)) + .await + .is_err() { debug!( target: LOG_TARGET, diff --git a/applications/tari_validator_node/src/substate_resolver.rs b/applications/tari_validator_node/src/substate_resolver.rs index b1aefb3ea..856be9fb2 100644 --- a/applications/tari_validator_node/src/substate_resolver.rs +++ b/applications/tari_validator_node/src/substate_resolver.rs @@ -68,6 +68,9 @@ where let mut missing_substates = HashSet::with_capacity(missing_substate_ids.len()); for requested_input in transaction.all_inputs_iter() { if missing_substate_ids.contains(requested_input.substate_id()) { + // TODO/NOTE: This assumes that consensus is up to date (i.e. doesnt need to sync, or catch up). We need + // to check the if the substate is in our shard range. The best action then may be to + // let consensus handle it (deferred) which is what happens currently anyway. missing_substates.insert(requested_input); // Not a local substate, so we will need to fetch it remotely continue; diff --git a/applications/tari_validator_node_web_ui/src/routes/Blocks/BlockDetails.tsx b/applications/tari_validator_node_web_ui/src/routes/Blocks/BlockDetails.tsx index 46809bfa9..a59f89693 100644 --- a/applications/tari_validator_node_web_ui/src/routes/Blocks/BlockDetails.tsx +++ b/applications/tari_validator_node_web_ui/src/routes/Blocks/BlockDetails.tsx @@ -34,7 +34,7 @@ import Loading from "../../Components/Loading"; import { getBlock, getIdentity } from "../../utils/json_rpc"; import Transactions from "./Transactions"; import { primitiveDateTimeToDate, primitiveDateTimeToSecs } from "../../utils/helpers"; -import type { Block, EpochEvent, TransactionAtom } from "@tariproject/typescript-bindings"; +import type { Block, TransactionAtom } from "@tariproject/typescript-bindings"; import type { GetIdentityResponse } from "@tariproject/typescript-bindings/validator-node-client"; export default function BlockDetails() { @@ -47,7 +47,7 @@ export default function BlockDetails() { const [prepare, setPrepare] = useState([]); const [localPrepared, setLocalPrepared] = useState([]); const [accept, setAccept] = useState([]); - const [epochEvents, setEpochEvents] = useState([]); + const [epochEvents, setEpochEvents] = useState([]); const [identity, setIdentity] = useState(); const [blockTime, setBlockTime] = useState(0); @@ -72,21 +72,22 @@ export default function BlockDetails() { setAccept([]); setEpochEvents([]); for (let command of resp.block.commands) { - if ("LocalOnly" in command) { - let newLocalOnly = command.LocalOnly; - setLocalOnly((localOnly: TransactionAtom[]) => [...localOnly, newLocalOnly]); - } else if ("Prepare" in command) { - let newPrepare = command.Prepare; - setPrepare((prepare: TransactionAtom[]) => [...prepare, newPrepare]); - } else if ("LocalPrepared" in command) { - let newLocalPrepared = command.LocalPrepared; - setLocalPrepared((localPrepared: TransactionAtom[]) => [...localPrepared, newLocalPrepared]); - } else if ("Accept" in command) { - let newAccept = command.Accept; - setAccept((accept: TransactionAtom[]) => [...accept, newAccept]); - } else if ("EpochEvent" in command) { - const newEpochEvent = command.EpochEvent; - setEpochEvents((epochEvents: EpochEvent[]) => [...epochEvents, newEpochEvent]); + if (typeof command === "object") { + if ("LocalOnly" in command) { + let newLocalOnly = command.LocalOnly; + setLocalOnly((localOnly: TransactionAtom[]) => [...localOnly, newLocalOnly]); + } else if ("Prepare" in command) { + let newPrepare = command.Prepare; + setPrepare((prepare: TransactionAtom[]) => [...prepare, newPrepare]); + } else if ("LocalPrepared" in command) { + let newLocalPrepared = command.LocalPrepared; + setLocalPrepared((localPrepared: TransactionAtom[]) => [...localPrepared, newLocalPrepared]); + } else if ("Accept" in command) { + let newAccept = command.Accept; + setAccept((accept: TransactionAtom[]) => [...accept, newAccept]); + } + } else { + setEpochEvents((epochEvents: string[]) => [...epochEvents, command as string]); } } }) diff --git a/bindings/index.ts b/bindings/index.ts index bf83da3ac..dbc2f417a 100644 --- a/bindings/index.ts +++ b/bindings/index.ts @@ -29,7 +29,6 @@ export * from "./src/types/Decision"; export * from "./src/types/ElgamalVerifiableBalance"; export * from "./src/types/EntityId"; export * from "./src/types/Epoch"; -export * from "./src/types/EpochEvent"; export * from "./src/types/Event"; export * from "./src/types/Evidence"; export * from "./src/types/ExecuteResult"; diff --git a/bindings/src/types/Command.ts b/bindings/src/types/Command.ts index be5ac630f..10fc54196 100644 --- a/bindings/src/types/Command.ts +++ b/bindings/src/types/Command.ts @@ -1,5 +1,4 @@ // This file was generated by [ts-rs](https://github.com/Aleph-Alpha/ts-rs). Do not edit this file manually. -import type { EpochEvent } from "./EpochEvent"; import type { ForeignProposal } from "./ForeignProposal"; import type { TransactionAtom } from "./TransactionAtom"; @@ -9,4 +8,4 @@ export type Command = | { Accept: TransactionAtom } | { ForeignProposal: ForeignProposal } | { LocalOnly: TransactionAtom } - | { EpochEvent: EpochEvent }; + | "EndEpoch"; diff --git a/bindings/src/types/EpochEvent.ts b/bindings/src/types/EpochEvent.ts deleted file mode 100644 index 5efee345f..000000000 --- a/bindings/src/types/EpochEvent.ts +++ /dev/null @@ -1,3 +0,0 @@ -// This file was generated by [ts-rs](https://github.com/Aleph-Alpha/ts-rs). Do not edit this file manually. - -export type EpochEvent = "Start" | "End"; diff --git a/bindings/src/types/SubstateDestroyed.ts b/bindings/src/types/SubstateDestroyed.ts index f1918ae93..0d3304f73 100644 --- a/bindings/src/types/SubstateDestroyed.ts +++ b/bindings/src/types/SubstateDestroyed.ts @@ -1,9 +1,11 @@ // This file was generated by [ts-rs](https://github.com/Aleph-Alpha/ts-rs). Do not edit this file manually. import type { Epoch } from "./Epoch"; +import type { Shard } from "./Shard"; export interface SubstateDestroyed { by_transaction: string; justify: string; by_block: string; at_epoch: Epoch; + by_shard: Shard; } diff --git a/bindings/src/types/SubstateRecord.ts b/bindings/src/types/SubstateRecord.ts index 26bf2e917..f812fee52 100644 --- a/bindings/src/types/SubstateRecord.ts +++ b/bindings/src/types/SubstateRecord.ts @@ -1,6 +1,7 @@ // This file was generated by [ts-rs](https://github.com/Aleph-Alpha/ts-rs). Do not edit this file manually. import type { Epoch } from "./Epoch"; import type { NodeHeight } from "./NodeHeight"; +import type { Shard } from "./Shard"; import type { SubstateDestroyed } from "./SubstateDestroyed"; import type { SubstateId } from "./SubstateId"; import type { SubstateValue } from "./SubstateValue"; @@ -14,6 +15,7 @@ export interface SubstateRecord { created_justify: string; created_block: string; created_height: NodeHeight; + created_by_shard: Shard; created_at_epoch: Epoch; destroyed: SubstateDestroyed | null; } diff --git a/clients/validator_node_client/src/lib.rs b/clients/validator_node_client/src/lib.rs index 87cb0958c..5a1ca4b70 100644 --- a/clients/validator_node_client/src/lib.rs +++ b/clients/validator_node_client/src/lib.rs @@ -123,6 +123,13 @@ impl ValidatorNodeClient { self.send_request("list_blocks", request).await } + pub async fn list_blocks_paginated( + &mut self, + request: GetBlocksRequest, + ) -> Result { + self.send_request("get_blocks", request).await + } + pub async fn submit_transaction( &mut self, request: SubmitTransactionRequest, diff --git a/dan_layer/common_types/src/committee.rs b/dan_layer/common_types/src/committee.rs index 950056556..c97f61c44 100644 --- a/dan_layer/common_types/src/committee.rs +++ b/dan_layer/common_types/src/committee.rs @@ -203,6 +203,10 @@ impl CommitteeInfo { self.shard } + pub fn to_substate_address_range(&self) -> RangeInclusive { + self.shard.to_substate_address_range(self.num_committees) + } + pub fn includes_substate_address(&self, substate_address: &SubstateAddress) -> bool { let s = substate_address.to_shard(self.num_committees); self.shard == s diff --git a/dan_layer/common_types/src/epoch.rs b/dan_layer/common_types/src/epoch.rs index 0919eeada..ca504a6b1 100644 --- a/dan_layer/common_types/src/epoch.rs +++ b/dan_layer/common_types/src/epoch.rs @@ -32,7 +32,11 @@ use ts_rs::TS; pub struct Epoch(#[cfg_attr(feature = "ts", ts(type = "number"))] pub u64); impl Epoch { - pub fn as_u64(self) -> u64 { + pub const fn zero() -> Self { + Self(0) + } + + pub const fn as_u64(self) -> u64 { self.0 } @@ -44,8 +48,8 @@ impl Epoch { self.0.to_le_bytes() } - pub fn saturating_sub(&self, other: Epoch) -> Epoch { - Epoch(self.0.saturating_sub(other.0)) + pub fn saturating_sub>(&self, other: T) -> Epoch { + Epoch(self.0.saturating_sub(other.into().0)) } pub fn checked_sub(&self, other: Self) -> Option { @@ -59,9 +63,21 @@ impl From for Epoch { } } +impl PartialEq for Epoch { + fn eq(&self, other: &u64) -> bool { + self.0 == *other + } +} + +impl PartialEq for u64 { + fn eq(&self, other: &Epoch) -> bool { + *self == other.0 + } +} + impl Display for Epoch { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "{}", self.0) + write!(f, "Epoch({})", self.0) } } diff --git a/dan_layer/common_types/src/node_height.rs b/dan_layer/common_types/src/node_height.rs index 7f2ccb2c6..c44a59f24 100644 --- a/dan_layer/common_types/src/node_height.rs +++ b/dan_layer/common_types/src/node_height.rs @@ -10,7 +10,7 @@ use serde::{Deserialize, Serialize}; #[cfg(feature = "ts")] use ts_rs::TS; -#[derive(Copy, Clone, Debug, PartialEq, Eq, PartialOrd, Ord, Hash, Deserialize, Serialize)] +#[derive(Copy, Clone, Debug, Default, PartialEq, Eq, PartialOrd, Ord, Hash, Deserialize, Serialize)] #[cfg_attr(feature = "ts", derive(TS), ts(export, export_to = "../../bindings/src/types/"))] pub struct NodeHeight(#[cfg_attr(feature = "ts", ts(type = "number"))] pub u64); diff --git a/dan_layer/common_types/src/shard.rs b/dan_layer/common_types/src/shard.rs index 5cae5b1ad..c085d4593 100644 --- a/dan_layer/common_types/src/shard.rs +++ b/dan_layer/common_types/src/shard.rs @@ -15,6 +15,10 @@ use crate::{uint::U256, SubstateAddress}; pub struct Shard(#[cfg_attr(feature = "ts", ts(type = "number"))] u32); impl Shard { + pub const fn zero() -> Shard { + Shard(0) + } + pub fn as_u32(self) -> u32 { self.0 } diff --git a/dan_layer/consensus/src/block_validations.rs b/dan_layer/consensus/src/block_validations.rs index 77621c0d0..b315afa25 100644 --- a/dan_layer/consensus/src/block_validations.rs +++ b/dan_layer/consensus/src/block_validations.rs @@ -11,6 +11,29 @@ use crate::{ traits::{ConsensusSpec, LeaderStrategy, VoteSignatureService}, }; +pub async fn check_proposal( + block: &Block, + network: Network, + epoch_manager: &TConsensusSpec::EpochManager, + vote_signing_service: &TConsensusSpec::SignatureService, + leader_strategy: &TConsensusSpec::LeaderStrategy, + _config: &HotstuffConfig, +) -> Result<(), HotStuffError> { + // TODO: in order to do the base layer block has validation, we need to ensure that we have synced to the tip. + // If not, we need some strategy for "parking" the blocks until we are at least at the provided hash or the + // tip. Without this, the check has a race condition between the base layer scanner and consensus. + // check_base_layer_block_hash::(block, epoch_manager, config).await?; + check_network(block, network)?; + check_hash_and_height(block)?; + let committee_for_block = epoch_manager + .get_committee_by_validator_public_key(block.epoch(), block.proposed_by()) + .await?; + check_proposed_by_leader(leader_strategy, &committee_for_block, block)?; + check_signature(block)?; + check_quorum_certificate::(block, vote_signing_service, epoch_manager).await?; + Ok(()) +} + pub fn check_network(candidate_block: &Block, network: Network) -> Result<(), ProposalValidationError> { if candidate_block.network() != network { return Err(ProposalValidationError::InvalidNetwork { @@ -22,6 +45,8 @@ pub fn check_network(candidate_block: &Block, network: Network) -> Result<(), Pr Ok(()) } +// TODO: remove allow(dead_code) +#[allow(dead_code)] pub async fn check_base_layer_block_hash( block: &Block, epoch_manager: &TConsensusSpec::EpochManager, @@ -59,17 +84,17 @@ pub async fn check_base_layer_block_hash( current: current_height, })?; } - if block.is_epoch_end() && !epoch_manager.is_last_block_of_epoch(base_layer_block_height).await? { - Err(ProposalValidationError::NotLastBlockOfEpoch { - block_id: *block.id(), - base_layer_block_height, - })?; - } + // if block.is_epoch_end() && !epoch_manager.is_last_block_of_epoch(base_layer_block_height).await? { + // Err(ProposalValidationError::NotLastBlockOfEpoch { + // block_id: *block.id(), + // base_layer_block_height, + // })?; + // } Ok(()) } pub fn check_hash_and_height(candidate_block: &Block) -> Result<(), ProposalValidationError> { - if candidate_block.height().is_zero() || candidate_block.is_genesis() { + if candidate_block.is_genesis() { return Err(ProposalValidationError::ProposingGenesisBlock { proposed_by: candidate_block.proposed_by().to_string(), hash: *candidate_block.id(), @@ -114,7 +139,7 @@ pub fn check_signature(candidate_block: &Block) -> Result<(), ProposalValidation return Ok(()); } let validator_signature = candidate_block - .get_signature() + .signature() .ok_or(ProposalValidationError::MissingSignature { block_id: *candidate_block.id(), height: candidate_block.height(), @@ -134,8 +159,8 @@ pub async fn check_quorum_certificate( epoch_manager: &TConsensusSpec::EpochManager, ) -> Result<(), HotStuffError> { let qc = candidate_block.justify(); - if qc.is_genesis() { - // This is potentially dangerous. There should be a check + if qc.is_zero() { + // TODO: This is potentially dangerous. There should be a check // to make sure this is the start of the chain. return Ok(()); @@ -148,6 +173,10 @@ pub async fn check_quorum_certificate( .into()); } + if qc.signatures().is_empty() { + return Err(ProposalValidationError::QuorumWasNotReached { qc: qc.clone() }.into()); + } + let mut vns = vec![]; for signature in qc.signatures() { let vn = epoch_manager diff --git a/dan_layer/consensus/src/hotstuff/block_change_set.rs b/dan_layer/consensus/src/hotstuff/block_change_set.rs index b137fe352..8514d0856 100644 --- a/dan_layer/consensus/src/hotstuff/block_change_set.rs +++ b/dan_layer/consensus/src/hotstuff/block_change_set.rs @@ -4,6 +4,7 @@ use std::ops::Deref; use indexmap::IndexMap; +use tari_dan_common_types::Epoch; use tari_dan_storage::{ consensus_models::{ Block, @@ -33,6 +34,7 @@ pub struct BlockDecision { pub quorum_decision: Option, pub locked_blocks: Vec, pub finalized_transactions: Vec>, + pub end_of_epoch: Option, } #[derive(Debug, Clone)] diff --git a/dan_layer/consensus/src/hotstuff/common.rs b/dan_layer/consensus/src/hotstuff/common.rs index 1e9f00727..d62dbee0d 100644 --- a/dan_layer/consensus/src/hotstuff/common.rs +++ b/dan_layer/consensus/src/hotstuff/common.rs @@ -1,6 +1,8 @@ // Copyright 2023 The Tari Project // SPDX-License-Identifier: BSD-3-Clause +use std::ops::ControlFlow; + use log::*; use tari_common::configuration::Network; use tari_common_types::types::FixedHash; @@ -40,6 +42,80 @@ pub fn calculate_last_dummy_block Option { + let mut dummy = None; + with_dummy_blocks( + network, + epoch, + shard, + high_qc, + parent_merkle_root, + new_height, + leader_strategy, + local_committee, + parent_timestamp, + parent_base_layer_block_height, + parent_base_layer_block_hash, + |dummy_block| { + dummy = Some(dummy_block.as_leaf_block()); + ControlFlow::Continue(()) + }, + ); + + dummy +} + +/// Calculates the dummy block required to reach the new height +pub fn calculate_dummy_blocks>( + candidate_block: &Block, + justify_block: &Block, + leader_strategy: &TLeaderStrategy, + local_committee: &Committee, +) -> Vec { + let mut dummies = Vec::new(); + with_dummy_blocks( + candidate_block.network(), + justify_block.epoch(), + justify_block.shard(), + candidate_block.justify(), + *justify_block.merkle_root(), + candidate_block.height(), + leader_strategy, + local_committee, + justify_block.timestamp(), + justify_block.base_layer_block_height(), + *justify_block.base_layer_block_hash(), + |dummy_block| { + if dummy_block.id() == candidate_block.parent() { + dummies.push(dummy_block); + ControlFlow::Break(()) + } else { + dummies.push(dummy_block); + ControlFlow::Continue(()) + } + }, + ); + + dummies +} + +fn with_dummy_blocks( + network: Network, + epoch: Epoch, + shard: Shard, + high_qc: &QuorumCertificate, + parent_merkle_root: FixedHash, + new_height: NodeHeight, + leader_strategy: &TLeaderStrategy, + local_committee: &Committee, + parent_timestamp: u64, + parent_base_layer_block_height: u64, + parent_base_layer_block_hash: FixedHash, + mut callback: F, +) where + TAddr: NodeAddressable, + TLeaderStrategy: LeaderStrategy, + F: FnMut(Block) -> ControlFlow<()>, +{ let mut parent_block = high_qc.as_leaf_block(); let mut current_height = high_qc.block_height() + NodeHeight(1); if current_height > new_height { @@ -49,7 +125,7 @@ pub fn calculate_last_dummy_block impl Iterator + '_ { @@ -104,19 +182,20 @@ pub fn calculate_state_merkle_diff, I: IntoIterato tx: &TTx, current_version: Version, next_version: Version, - pending_tree_updates: Vec, + pending_tree_diffs: Vec, substate_changes: I, ) -> Result<(Hash, StateHashTreeDiff), StateTreeError> { debug!( target: LOG_TARGET, - "Calculating state merkle diff from version {} to {} with {} update(s)", + "Calculating state merkle diff from version {} to {} with {} pending diff(s)", current_version, next_version, - pending_tree_updates.len(), + pending_tree_diffs.len(), ); let mut store = StagedTreeStore::new(tx); - store.apply_ordered_diffs(pending_tree_updates.into_iter().map(|diff| diff.diff)); + store.apply_ordered_diffs(pending_tree_diffs.into_iter().map(|diff| diff.diff)); let mut state_tree = tari_state_tree::SpreadPrefixStateTree::new(&mut store); - let state_root = state_tree.put_substate_changes(current_version, next_version, substate_changes)?; + let state_root = + state_tree.put_substate_changes(Some(current_version).filter(|v| *v > 0), next_version, substate_changes)?; Ok((state_root, store.into_diff())) } diff --git a/dan_layer/consensus/src/hotstuff/current_height.rs b/dan_layer/consensus/src/hotstuff/current_height.rs deleted file mode 100644 index 4d489dcfa..000000000 --- a/dan_layer/consensus/src/hotstuff/current_height.rs +++ /dev/null @@ -1,51 +0,0 @@ -// Copyright 2023 The Tari Project -// SPDX-License-Identifier: BSD-3-Clause - -use std::{ - fmt::Display, - sync::{atomic, atomic::AtomicU64, Arc}, -}; - -use tari_dan_common_types::NodeHeight; - -#[derive(Debug, Clone)] -pub struct CurrentHeight { - height: Arc, -} - -impl CurrentHeight { - pub fn new() -> Self { - Self { - height: Arc::new(AtomicU64::new(0)), - } - } - - pub fn next_height(&self) -> NodeHeight { - self.height.fetch_add(1, atomic::Ordering::SeqCst).into() - } - - pub fn get(&self) -> NodeHeight { - self.height.load(atomic::Ordering::SeqCst).into() - } - - /// Updates the height if the new height is greater than the current height. - /// Returns true if the height was updated, otherwise false. - pub fn update(&self, height: NodeHeight) -> bool { - let current_height = self.get(); - if height > current_height { - self.set(height); - return true; - } - false - } - - pub fn set(&self, height: NodeHeight) { - self.height.store(height.as_u64(), atomic::Ordering::SeqCst); - } -} - -impl Display for CurrentHeight { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "{}", self.get()) - } -} diff --git a/dan_layer/consensus/src/hotstuff/current_view.rs b/dan_layer/consensus/src/hotstuff/current_view.rs new file mode 100644 index 000000000..e0d0efe0a --- /dev/null +++ b/dan_layer/consensus/src/hotstuff/current_view.rs @@ -0,0 +1,68 @@ +// Copyright 2023 The Tari Project +// SPDX-License-Identifier: BSD-3-Clause + +use std::{ + fmt::Display, + sync::{atomic, atomic::AtomicU64, Arc}, +}; + +use log::info; +use tari_dan_common_types::{Epoch, NodeHeight}; + +const LOG_TARGET: &str = "tari::dan::consensus::hotstuff::current_view"; + +#[derive(Debug, Clone, Default)] +pub struct CurrentView { + height: Arc, + epoch: Arc, +} + +impl CurrentView { + pub fn new() -> Self { + Self::default() + } + + pub(crate) fn set_next_height(&self) { + self.height.fetch_add(1, atomic::Ordering::SeqCst); + } + + pub fn get_epoch(&self) -> Epoch { + self.epoch.load(atomic::Ordering::SeqCst).into() + } + + pub fn get_height(&self) -> NodeHeight { + self.height.load(atomic::Ordering::SeqCst).into() + } + + /// Updates the height and epoch if they are greater than the current values. + pub(crate) fn update(&self, epoch: Epoch, height: NodeHeight) { + let current_epoch = self.get_epoch(); + let mut is_updated = false; + if epoch > current_epoch { + is_updated = true; + self.epoch.store(epoch.as_u64(), atomic::Ordering::SeqCst); + } + let current_height = self.get_height(); + if height > current_height { + is_updated = true; + self.height.store(height.as_u64(), atomic::Ordering::SeqCst); + } + + if is_updated { + info!(target: LOG_TARGET, "🧿 PACEMAKER: View updated to {self}"); + } + } + + /// Resets the height and epoch. Prefer update. + pub(crate) fn reset(&self, epoch: Epoch, height: NodeHeight) { + self.epoch.store(epoch.as_u64(), atomic::Ordering::SeqCst); + self.height.store(height.as_u64(), atomic::Ordering::SeqCst); + info!(target: LOG_TARGET, "🧿 PACEMAKER RESET: View updated to {epoch}/{height}"); + } +} + +impl Display for CurrentView { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}/{}", self.get_epoch(), self.get_height()) + } +} diff --git a/dan_layer/consensus/src/hotstuff/error.rs b/dan_layer/consensus/src/hotstuff/error.rs index de655312c..78586bb04 100644 --- a/dan_layer/consensus/src/hotstuff/error.rs +++ b/dan_layer/consensus/src/hotstuff/error.rs @@ -33,8 +33,8 @@ pub enum HotStuffError { EpochNotActive { epoch: Epoch, details: String }, #[error("Not registered for current epoch {epoch}")] NotRegisteredForCurrentEpoch { epoch: Epoch }, - #[error("Received message from non-committee member. Epoch: {epoch}, Sender: {sender}, {context}")] - ReceivedMessageFromNonCommitteeMember { + #[error("Received vote from non-committee member. Epoch: {epoch}, Sender: {sender}, {context}")] + ReceivedVoteFromNonCommitteeMember { epoch: Epoch, sender: String, context: String, @@ -53,8 +53,8 @@ pub enum HotStuffError { StateManagerError(anyhow::Error), #[error("Invalid vote signature from {signer_public_key} (unauthenticated)")] InvalidVoteSignature { signer_public_key: String }, - #[error("Vote sent from peer {address} did not match the expected signer public key {signer_public_key}")] - RejectingVoteNotSentBySigner { address: String, signer_public_key: String }, + #[error("Invalid vote {signer_public_key} (unauthenticated): {details}")] + InvalidVote { signer_public_key: String, details: String }, #[error("Transaction pool error: {0}")] TransactionPoolError(#[from] TransactionPoolError), #[error("Transaction {transaction_id} does not exist")] @@ -125,11 +125,11 @@ pub enum ProposalValidationError { }, #[error("Node proposed by {proposed_by} with hash {hash} is the genesis block")] ProposingGenesisBlock { proposed_by: String, hash: BlockId }, - #[error("Justification block {justify_block} for proposed block {block_description} by {proposed_by} not found")] + #[error("Justified block {justify_block} for proposed block {block_description} by {proposed_by} not found")] JustifyBlockNotFound { proposed_by: String, block_description: String, - justify_block: BlockId, + justify_block: LeafBlock, }, #[error("QC in block {block_id} that was proposed by {proposed_by} is invalid: {details}")] JustifyBlockInvalid { diff --git a/dan_layer/consensus/src/hotstuff/event.rs b/dan_layer/consensus/src/hotstuff/event.rs index 744e3a662..974cc83fe 100644 --- a/dan_layer/consensus/src/hotstuff/event.rs +++ b/dan_layer/consensus/src/hotstuff/event.rs @@ -1,13 +1,17 @@ // Copyright 2023 The Tari Project // SPDX-License-Identifier: BSD-3-Clause -use tari_dan_common_types::NodeHeight; +use tari_dan_common_types::{Epoch, NodeHeight}; use tari_dan_storage::consensus_models::{BlockId, LeafBlock}; #[derive(Debug, Clone, thiserror::Error)] pub enum HotstuffEvent { - #[error("Block {block_id} has been committed at height {height}")] - BlockCommitted { block_id: BlockId, height: NodeHeight }, + #[error("Block {block_id} has been committed for epoch {epoch} at height {height}")] + BlockCommitted { + epoch: Epoch, + block_id: BlockId, + height: NodeHeight, + }, #[error("Consensus failure: {message}")] Failure { message: String }, #[error("Leader timeout: new height {new_height}")] diff --git a/dan_layer/consensus/src/hotstuff/mod.rs b/dan_layer/consensus/src/hotstuff/mod.rs index da9e26792..5d08221fc 100644 --- a/dan_layer/consensus/src/hotstuff/mod.rs +++ b/dan_layer/consensus/src/hotstuff/mod.rs @@ -2,7 +2,7 @@ // SPDX-License-Identifier: BSD-3-Clause mod common; mod config; -mod current_height; +mod current_view; mod error; mod event; mod on_beat; @@ -21,6 +21,8 @@ mod on_receive_vote; mod on_sync_request; // mod on_sync_response; mod block_change_set; +mod on_catch_up_sync; +mod on_message_validate; mod pacemaker; mod pacemaker_handle; mod proposer; @@ -31,6 +33,7 @@ mod worker; pub use common::*; pub use config::HotstuffConfig; +pub use current_view::*; pub use error::*; pub use event::*; pub use state_machine::*; diff --git a/dan_layer/consensus/src/hotstuff/on_beat.rs b/dan_layer/consensus/src/hotstuff/on_beat.rs index a41de60bb..45637399a 100644 --- a/dan_layer/consensus/src/hotstuff/on_beat.rs +++ b/dan_layer/consensus/src/hotstuff/on_beat.rs @@ -28,3 +28,9 @@ impl OnBeat { self.sender.send(()).expect("receiver can never be dropped") } } + +impl Default for OnBeat { + fn default() -> Self { + Self::new() + } +} diff --git a/dan_layer/consensus/src/hotstuff/on_catch_up_sync.rs b/dan_layer/consensus/src/hotstuff/on_catch_up_sync.rs new file mode 100644 index 000000000..ddc96f943 --- /dev/null +++ b/dan_layer/consensus/src/hotstuff/on_catch_up_sync.rs @@ -0,0 +1,65 @@ +// Copyright 2024 The Tari Project +// SPDX-License-Identifier: BSD-3-Clause + +use log::{info, warn}; +use tari_dan_common_types::Epoch; +use tari_dan_storage::{consensus_models::HighQc, StateStore}; + +use crate::{ + hotstuff::{pacemaker_handle::PaceMakerHandle, HotStuffError}, + messages::{HotstuffMessage, SyncRequestMessage}, + traits::{ConsensusSpec, OutboundMessaging}, +}; + +const LOG_TARGET: &str = "tari::dan::consensus::hotstuff::on_catch_up_sync"; + +pub struct OnCatchUpSync { + store: TConsensusSpec::StateStore, + pacemaker: PaceMakerHandle, + outbound_messaging: TConsensusSpec::OutboundMessaging, +} + +impl OnCatchUpSync { + pub fn new( + store: TConsensusSpec::StateStore, + pacemaker: PaceMakerHandle, + outbound_messaging: TConsensusSpec::OutboundMessaging, + ) -> Self { + Self { + store, + pacemaker, + outbound_messaging, + } + } + + pub async fn request_sync(&mut self, epoch: Epoch, from: &TConsensusSpec::Addr) -> Result<(), HotStuffError> { + let high_qc = self.store.with_read_tx(|tx| HighQc::get(tx))?; + info!( + target: LOG_TARGET, + "⏰ Catch up required from block {} from {} (current view: {})", + high_qc, + from, + self.pacemaker.current_view() + ); + // Reset leader timeout since we're behind. TODO: This is hacky. + self.pacemaker + .reset_view(epoch, high_qc.block_height(), high_qc.block_height) + .await?; + + // Request a catch-up + if self + .outbound_messaging + .send( + from.clone(), + HotstuffMessage::CatchUpSyncRequest(SyncRequestMessage { epoch, high_qc }), + ) + .await + .is_err() + { + warn!(target: LOG_TARGET, "Leader channel closed while sending SyncRequest"); + return Ok(()); + } + + Ok(()) + } +} diff --git a/dan_layer/consensus/src/hotstuff/on_force_beat.rs b/dan_layer/consensus/src/hotstuff/on_force_beat.rs index 5ae8efbe5..be291289b 100644 --- a/dan_layer/consensus/src/hotstuff/on_force_beat.rs +++ b/dan_layer/consensus/src/hotstuff/on_force_beat.rs @@ -30,3 +30,9 @@ impl OnForceBeat { self.sender.send(parent_block).expect("receiver can never be dropped") } } + +impl Default for OnForceBeat { + fn default() -> Self { + Self::new() + } +} diff --git a/dan_layer/consensus/src/hotstuff/on_inbound_message.rs b/dan_layer/consensus/src/hotstuff/on_inbound_message.rs index 0edf829a9..41fef3bba 100644 --- a/dan_layer/consensus/src/hotstuff/on_inbound_message.rs +++ b/dan_layer/consensus/src/hotstuff/on_inbound_message.rs @@ -1,118 +1,54 @@ // Copyright 2023 The Tari Project // SPDX-License-Identifier: BSD-3-Clause -use std::collections::{BTreeMap, HashSet, VecDeque}; +use std::collections::{BTreeMap, VecDeque}; use log::*; -use tari_common::configuration::Network; -use tari_dan_common_types::{optional::Optional, NodeAddressable, NodeHeight}; -use tari_dan_storage::{ - consensus_models::{ - Block, - ExecutedTransaction, - TransactionAtom, - TransactionPool, - TransactionPoolRecord, - TransactionRecord, - }, - StateStore, - StateStoreWriteTransaction, -}; -use tari_epoch_manager::EpochManagerReader; -use tari_transaction::TransactionId; -use tokio::{ - sync::{broadcast, mpsc}, - time, -}; +use tari_dan_common_types::{Epoch, NodeAddressable, NodeHeight}; -use super::config::HotstuffConfig; use crate::{ - block_validations::{ - check_base_layer_block_hash, - check_hash_and_height, - check_network, - check_proposed_by_leader, - check_quorum_certificate, - check_signature, - }, - hotstuff::{error::HotStuffError, HotstuffEvent}, - messages::{HotstuffMessage, ProposalMessage, RequestMissingTransactionsMessage}, - traits::{ConsensusSpec, OutboundMessaging}, + hotstuff::error::HotStuffError, + messages::HotstuffMessage, + traits::{hooks::ConsensusHooks, ConsensusSpec, InboundMessaging}, }; const LOG_TARGET: &str = "tari::dan::consensus::hotstuff::inbound_messages"; -pub type IncomingMessageResult = Result, NeedsSync>; +type IncomingMessageResult = Result, HotStuffError>; pub struct OnInboundMessage { - network: Network, - config: HotstuffConfig, - store: TConsensusSpec::StateStore, - epoch_manager: TConsensusSpec::EpochManager, - leader_strategy: TConsensusSpec::LeaderStrategy, - vote_signing_service: TConsensusSpec::SignatureService, - outbound_messaging: TConsensusSpec::OutboundMessaging, - tx_msg_ready: mpsc::UnboundedSender<(TConsensusSpec::Addr, HotstuffMessage)>, - message_buffer: MessageBuffer, - transaction_pool: TransactionPool, - tx_events: broadcast::Sender, + message_buffer: MessageBuffer, + hooks: TConsensusSpec::Hooks, } -impl OnInboundMessage -where TConsensusSpec: ConsensusSpec -{ - pub fn new( - network: Network, - config: HotstuffConfig, - store: TConsensusSpec::StateStore, - epoch_manager: TConsensusSpec::EpochManager, - leader_strategy: TConsensusSpec::LeaderStrategy, - vote_signing_service: TConsensusSpec::SignatureService, - outbound_messaging: TConsensusSpec::OutboundMessaging, - transaction_pool: TransactionPool, - tx_events: broadcast::Sender, - ) -> Self { - let (tx_msg_ready, rx_msg_ready) = mpsc::unbounded_channel(); +impl OnInboundMessage { + pub fn new(inbound_messaging: TConsensusSpec::InboundMessaging, hooks: TConsensusSpec::Hooks) -> Self { Self { - network, - config, - store, - epoch_manager, - leader_strategy, - vote_signing_service, - outbound_messaging, - tx_msg_ready, - message_buffer: MessageBuffer::new(rx_msg_ready), - transaction_pool, - tx_events, + message_buffer: MessageBuffer::new(inbound_messaging), + hooks, } } - pub async fn handle( + /// Returns the next message that is ready for consensus. The future returned from this function is cancel safe, and + /// can be used with tokio::select! macro. + pub async fn next_message( &mut self, + current_epoch: Epoch, current_height: NodeHeight, - from: TConsensusSpec::Addr, - msg: HotstuffMessage, - ) -> Result<(), HotStuffError> { - match msg { - HotstuffMessage::Proposal(msg) => { - self.process_local_proposal(current_height, msg).await?; - }, - HotstuffMessage::ForeignProposal(ref proposal) => { - self.check_proposal(&proposal.block).await?; - self.report_message_ready(from, msg)?; + ) -> Option> { + // Then incoming messages for the current epoch/height + let result = self.message_buffer.next(current_epoch, current_height).await; + match result { + Ok(Some((from, msg))) => { + self.hooks.on_message_received(&msg); + Some(Ok((from, msg))) }, - msg => { - self.report_message_ready(from, msg)?; + Ok(None) => { + // Inbound messages terminated + None }, + Err(err) => Some(Err(err)), } - Ok(()) - } - - /// Returns the next message that is ready for consensus. The future returned from this function is cancel safe, and - /// can be used with tokio::select! macro. - pub async fn next_message(&mut self, current_height: NodeHeight) -> IncomingMessageResult { - self.message_buffer.next(current_height).await } /// Discards all buffered messages including ones queued up for processing and returns when complete. @@ -123,262 +59,53 @@ where TConsensusSpec: ConsensusSpec pub fn clear_buffer(&mut self) { self.message_buffer.clear_buffer(); } - - async fn check_proposal(&self, block: &Block) -> Result<(), HotStuffError> { - check_base_layer_block_hash::(block, &self.epoch_manager, &self.config).await?; - check_network(block, self.network)?; - check_hash_and_height(block)?; - let committee_for_block = self - .epoch_manager - .get_committee_by_validator_public_key(block.epoch(), block.proposed_by()) - .await?; - check_proposed_by_leader(&self.leader_strategy, &committee_for_block, block)?; - check_signature(block)?; - check_quorum_certificate::(block, &self.vote_signing_service, &self.epoch_manager).await?; - Ok(()) - } - - async fn process_local_proposal( - &mut self, - current_height: NodeHeight, - proposal: ProposalMessage, - ) -> Result<(), HotStuffError> { - let ProposalMessage { block } = proposal; - - info!( - target: LOG_TARGET, - "πŸ“œ new unvalidated PROPOSAL message {} from {} (current height = {})", - block, - block.proposed_by(), - current_height, - ); - - if block.height() < current_height { - info!( - target: LOG_TARGET, - "πŸ”₯ Block {} is lower than current height {}. Ignoring.", - block, - current_height - ); - return Ok(()); - } - - self.check_proposal(&block).await?; - let Some(ready_block) = self.handle_missing_transactions(block).await? else { - // Block not ready - return Ok(()); - }; - - let vn = self - .epoch_manager - .get_validator_node_by_public_key(ready_block.epoch(), ready_block.proposed_by()) - .await?; - - self.report_message_ready( - vn.address, - HotstuffMessage::Proposal(ProposalMessage { block: ready_block }), - )?; - - Ok(()) - } - - pub async fn update_parked_blocks( - &self, - current_height: NodeHeight, - transaction_id: &TransactionId, - ) -> Result<(), HotStuffError> { - let maybe_unparked_block = self - .store - .with_write_tx(|tx| tx.missing_transactions_remove(current_height, transaction_id))?; - - if let Some(unparked_block) = maybe_unparked_block { - info!(target: LOG_TARGET, "♻️ all transactions for block {unparked_block} are ready for consensus"); - - // todo(hacky): ensure that all transactions are in the pool. Race condition: because we have not yet - // received it yet in the select! loop. - self.store.with_write_tx(|tx| { - for tx_id in unparked_block.all_transaction_ids() { - if self.transaction_pool.exists(&**tx, tx_id)? { - continue; - } - - warn!( - target: LOG_TARGET, - "⚠️ Transaction {} is missing from the transaction pool. Attempting to recover.", - tx_id - ); - - let transaction = TransactionRecord::get(&**tx, tx_id)?; - // Did the mempool execute it? - if transaction.is_executed() { - // This should never fail - let executed = ExecutedTransaction::try_from(transaction)?; - self.transaction_pool.insert(tx, executed.to_atom())?; - } else { - // Deferred execution - self.transaction_pool - .insert(tx, TransactionAtom::deferred(*transaction.id()))?; - } - } - - Ok::<_, HotStuffError>(()) - })?; - - let vn = self - .epoch_manager - .get_validator_node_by_public_key(unparked_block.epoch(), unparked_block.proposed_by()) - .await?; - - let _ignore = self.tx_events.send(HotstuffEvent::ParkedBlockReady { - block: unparked_block.as_leaf_block(), - }); - - self.report_message_ready( - vn.address, - HotstuffMessage::Proposal(ProposalMessage { block: unparked_block }), - )?; - } - Ok(()) - } - - fn report_message_ready(&self, from: TConsensusSpec::Addr, msg: HotstuffMessage) -> Result<(), HotStuffError> { - self.tx_msg_ready - .send((from, msg)) - .map_err(|_| HotStuffError::InternalChannelClosed { - context: "tx_msg_ready in InboundMessageWorker::handle_hotstuff_message", - }) - } - - async fn handle_missing_transactions(&mut self, block: Block) -> Result, HotStuffError> { - let (missing_tx_ids, awaiting_execution) = self - .store - .with_write_tx(|tx| self.check_for_missing_transactions(tx, &block))?; - - if !missing_tx_ids.is_empty() || !awaiting_execution.is_empty() { - info!( - target: LOG_TARGET, - "πŸ”₯ Block {} has {} missing transactions and {} awaiting execution", block, missing_tx_ids.len(), awaiting_execution.len(), - ); - - let _ignore = self.tx_events.send(HotstuffEvent::ProposedBlockParked { - block: block.as_leaf_block(), - num_missing_txs: missing_tx_ids.len(), - num_awaiting_txs: awaiting_execution.len(), - }); - - if !missing_tx_ids.is_empty() { - let block_id = *block.id(); - let epoch = block.epoch(); - let block_proposed_by = block.proposed_by().clone(); - - let vn = self - .epoch_manager - .get_validator_node_by_public_key(epoch, &block_proposed_by) - .await?; - - self.outbound_messaging - .send( - vn.address, - HotstuffMessage::RequestMissingTransactions(RequestMissingTransactionsMessage { - block_id, - epoch, - transactions: missing_tx_ids, - }), - ) - .await?; - } - - return Ok(None); - } - - Ok(Some(block)) - } - - fn check_for_missing_transactions( - &self, - tx: &mut ::WriteTransaction<'_>, - block: &Block, - ) -> Result<(HashSet, HashSet), HotStuffError> { - if block.commands().is_empty() { - return Ok((HashSet::new(), HashSet::new())); - } - let (transactions, missing_tx_ids) = TransactionRecord::get_any(&**tx, block.all_transaction_ids())?; - let awaiting_execution_or_deferred = transactions - .into_iter() - .filter(|tx| tx.final_decision.is_some()) - .filter(|tx| tx.result.is_none()) - .map(|tx| *tx.transaction.id()) - .collect::>(); - - // TODO(hacky): improve this. We need to account for transactions that are deferred when determining which - // transactions are awaiting execution. - let mut awaiting_execution = HashSet::new(); - for id in &awaiting_execution_or_deferred { - if let Some(t) = TransactionPoolRecord::get(&**tx, id).optional()? { - if !t.is_deferred() { - awaiting_execution.insert(*id); - } - } - } - - if missing_tx_ids.is_empty() && awaiting_execution.is_empty() { - debug!( - target: LOG_TARGET, - "βœ… Block {} has no missing transactions", block - ); - return Ok((HashSet::new(), HashSet::new())); - } - - info!( - target: LOG_TARGET, - "πŸ”₯ Block {} has {} missing transactions and {} awaiting execution", block, missing_tx_ids.len(), awaiting_execution.len(), - ); - - tx.missing_transactions_insert(block, &missing_tx_ids, &awaiting_execution)?; - - Ok((missing_tx_ids, awaiting_execution)) - } } -pub struct MessageBuffer { - buffer: BTreeMap>, - rx_msg_ready: mpsc::UnboundedReceiver<(TAddr, HotstuffMessage)>, +type EpochAndHeight = (Epoch, NodeHeight); +pub struct MessageBuffer { + buffer: BTreeMap>, + inbound_messaging: TConsensusSpec::InboundMessaging, } -impl MessageBuffer { - pub fn new(rx_msg_ready: mpsc::UnboundedReceiver<(TAddr, HotstuffMessage)>) -> Self { +impl MessageBuffer { + pub fn new(inbound_messaging: TConsensusSpec::InboundMessaging) -> Self { Self { buffer: BTreeMap::new(), - rx_msg_ready, + inbound_messaging, } } - pub async fn next(&mut self, current_height: NodeHeight) -> IncomingMessageResult { - // Clear buffer with lower heights - self.buffer = self.buffer.split_off(¤t_height); + pub async fn next( + &mut self, + current_epoch: Epoch, + current_height: NodeHeight, + ) -> IncomingMessageResult { + // Clear buffer with lower (epoch, heights) + self.buffer = self.buffer.split_off(&(current_epoch, current_height)); // Check if message is in the buffer - if let Some(buffer) = self.buffer.get_mut(¤t_height) { + if let Some(buffer) = self.buffer.get_mut(&(current_epoch, current_height)) { if let Some(msg_tuple) = buffer.pop_front() { return Ok(Some(msg_tuple)); } } - while let Some((from, msg)) = self.next_message_or_sync(current_height).await? { - match msg_height(&msg) { + while let Some(result) = self.inbound_messaging.next_message().await { + let (from, msg) = result?; + match msg_epoch_and_height(&msg) { // Discard old message - Some(h) if h < current_height => { - debug!(target: LOG_TARGET, "Discard message {} is for previous height {}. Current height {}", msg, h, current_height); + Some((e, h)) if e < current_epoch || h < current_height => { + info!(target: LOG_TARGET, "Discard message {} is for previous view {}/{}. Current view {}/{}", msg, e, h, current_epoch,current_height); continue; }, - // Buffer message for future height - Some(h) if h > current_height => { + // Buffer message for future epoch/height + Some((epoch, height)) if epoch > current_epoch || height > current_height => { if msg.proposal().is_some() { - info!(target: LOG_TARGET, "Proposal {} is for future block {}. Current height {}", msg, h, current_height); + info!(target: LOG_TARGET, "🦴Proposal {msg} is for future view (Current view: {current_epoch}, {current_height})"); } else { - debug!(target: LOG_TARGET, "Message {} is for future height {}. Current height {}", msg, h, current_height); + info!(target: LOG_TARGET, "🦴Message {msg} is for future view (Current view: {current_epoch}, {current_height})"); } - self.push_to_buffer(h, from, msg); + self.push_to_buffer(epoch, height, from, msg); continue; }, // Height is irrelevant or current, return message @@ -386,51 +113,25 @@ impl MessageBuffer { } } + info!( + target: LOG_TARGET, + "Inbound messaging has terminated. Current view: {}/{}", current_epoch, current_height + ); + // Inbound messaging has terminated Ok(None) } pub async fn discard(&mut self) { self.clear_buffer(); - while self.rx_msg_ready.recv().await.is_some() {} + while self.inbound_messaging.next_message().await.is_some() {} } pub fn clear_buffer(&mut self) { self.buffer.clear(); } - async fn next_message_or_sync( - &mut self, - current_height: NodeHeight, - ) -> Result, NeedsSync> { - loop { - // Don't really like this but because we can receive proposals out of order, we need to wait a bit to see - // if we get a proposal at our height without switching to sync. - let timeout = time::sleep(time::Duration::from_secs(2)); - tokio::pin!(timeout); - tokio::select! { - msg = self.rx_msg_ready.recv() => return Ok(msg), - _ = timeout.as_mut() => { - // Check if we have any proposals - for queue in self.buffer.values() { - for (from, msg) in queue { - if let Some(proposal) = msg.proposal() { - if proposal.block.justify().block_height() > current_height { - return Err(NeedsSync { - from: from.clone(), - local_height: current_height, - qc_height: proposal.block.justify().block_height(), - }); - } - } - } - } - } - } - } - } - - fn push_to_buffer(&mut self, height: NodeHeight, from: TAddr, msg: HotstuffMessage) { - self.buffer.entry(height).or_default().push_back((from, msg)); + fn push_to_buffer(&mut self, epoch: Epoch, height: NodeHeight, from: TConsensusSpec::Addr, msg: HotstuffMessage) { + self.buffer.entry((epoch, height)).or_default().push_back((from, msg)); } } @@ -440,13 +141,15 @@ pub struct NeedsSync { pub from: TAddr, pub local_height: NodeHeight, pub qc_height: NodeHeight, + pub remote_epoch: Epoch, + pub local_epoch: Epoch, } -fn msg_height(msg: &HotstuffMessage) -> Option { +fn msg_epoch_and_height(msg: &HotstuffMessage) -> Option { match msg { - HotstuffMessage::Proposal(msg) => Some(msg.block.height()), + HotstuffMessage::Proposal(msg) => Some((msg.block.epoch(), msg.block.height())), // Votes for block 2, occur at current height 3 - HotstuffMessage::Vote(msg) => Some(msg.block_height.saturating_add(NodeHeight(1))), + HotstuffMessage::Vote(msg) => Some((msg.epoch, msg.block_height.saturating_add(NodeHeight(1)))), _ => None, } } diff --git a/dan_layer/consensus/src/hotstuff/on_leader_timeout.rs b/dan_layer/consensus/src/hotstuff/on_leader_timeout.rs index 513f91751..dfe5dd7f2 100644 --- a/dan_layer/consensus/src/hotstuff/on_leader_timeout.rs +++ b/dan_layer/consensus/src/hotstuff/on_leader_timeout.rs @@ -32,3 +32,9 @@ impl OnLeaderTimeout { self.sender.send(new_height).expect("receiver can never be dropped") } } + +impl Default for OnLeaderTimeout { + fn default() -> Self { + Self::new() + } +} diff --git a/dan_layer/consensus/src/hotstuff/on_message_validate.rs b/dan_layer/consensus/src/hotstuff/on_message_validate.rs new file mode 100644 index 000000000..4f04f94e8 --- /dev/null +++ b/dan_layer/consensus/src/hotstuff/on_message_validate.rs @@ -0,0 +1,354 @@ +// Copyright 2023 The Tari Project +// SPDX-License-Identifier: BSD-3-Clause + +use std::collections::HashSet; + +use log::*; +use tari_common::configuration::Network; +use tari_dan_common_types::{optional::Optional, NodeHeight}; +use tari_dan_storage::{ + consensus_models::{ + Block, + ExecutedTransaction, + TransactionAtom, + TransactionPool, + TransactionPoolRecord, + TransactionRecord, + }, + StateStore, + StateStoreWriteTransaction, +}; +use tari_epoch_manager::EpochManagerReader; +use tari_transaction::TransactionId; +use tokio::sync::broadcast; + +use super::config::HotstuffConfig; +use crate::{ + block_validations, + hotstuff::{error::HotStuffError, HotstuffEvent}, + messages::{HotstuffMessage, ProposalMessage, RequestMissingTransactionsMessage}, + traits::{ConsensusSpec, OutboundMessaging}, +}; + +const LOG_TARGET: &str = "tari::dan::consensus::hotstuff::on_message_validate"; + +pub struct OnMessageValidate { + local_validator_addr: TConsensusSpec::Addr, + network: Network, + config: HotstuffConfig, + store: TConsensusSpec::StateStore, + epoch_manager: TConsensusSpec::EpochManager, + leader_strategy: TConsensusSpec::LeaderStrategy, + vote_signing_service: TConsensusSpec::SignatureService, + outbound_messaging: TConsensusSpec::OutboundMessaging, + transaction_pool: TransactionPool, + tx_events: broadcast::Sender, +} + +impl OnMessageValidate { + pub fn new( + local_validator_addr: TConsensusSpec::Addr, + network: Network, + config: HotstuffConfig, + store: TConsensusSpec::StateStore, + epoch_manager: TConsensusSpec::EpochManager, + leader_strategy: TConsensusSpec::LeaderStrategy, + vote_signing_service: TConsensusSpec::SignatureService, + outbound_messaging: TConsensusSpec::OutboundMessaging, + transaction_pool: TransactionPool, + tx_events: broadcast::Sender, + ) -> Self { + Self { + local_validator_addr, + network, + config, + store, + epoch_manager, + leader_strategy, + vote_signing_service, + outbound_messaging, + transaction_pool, + tx_events, + } + } + + pub async fn handle( + &mut self, + current_height: NodeHeight, + from: TConsensusSpec::Addr, + msg: HotstuffMessage, + ) -> Result, HotStuffError> { + match msg { + HotstuffMessage::Proposal(msg) => self.process_local_proposal(current_height, from, msg).await, + HotstuffMessage::ForeignProposal(proposal) => { + if let Err(err) = self.check_proposal(&proposal.block).await { + return Ok(MessageValidationResult::Invalid { + from, + message: HotstuffMessage::Proposal(proposal), + err, + }); + } + Ok(MessageValidationResult::Ready { + from, + message: HotstuffMessage::ForeignProposal(proposal), + }) + }, + msg => Ok(MessageValidationResult::Ready { from, message: msg }), + } + } + + async fn process_local_proposal( + &mut self, + current_height: NodeHeight, + from: TConsensusSpec::Addr, + proposal: ProposalMessage, + ) -> Result, HotStuffError> { + let ProposalMessage { block } = proposal; + + info!( + target: LOG_TARGET, + "πŸ“œ [{}] new unvalidated PROPOSAL message {} from {} (current height = {})", + self.local_validator_addr, + block, + block.proposed_by(), + current_height, + ); + + if block.height() < current_height { + info!( + target: LOG_TARGET, + "πŸ”₯ Block {} is lower than current height {}. Ignoring.", + block, + current_height + ); + return Ok(MessageValidationResult::Discard); + } + + if let Err(err) = self.check_proposal(&block).await { + return Ok(MessageValidationResult::Invalid { + from, + message: HotstuffMessage::Proposal(ProposalMessage { block }), + err, + }); + } + + let Some(ready_block) = self.handle_missing_transactions(block).await? else { + // Block not ready -park it + return Ok(MessageValidationResult::NotReady); + }; + + // let vn = self + // .epoch_manager + // .get_validator_node_by_public_key(ready_block.epoch(), ready_block.proposed_by()) + // .await?; + + Ok(MessageValidationResult::Ready { + from, + message: HotstuffMessage::Proposal(ProposalMessage { block: ready_block }), + }) + } + + pub async fn update_parked_blocks( + &mut self, + current_height: NodeHeight, + transaction_id: &TransactionId, + ) -> Result, HotStuffError> { + let maybe_unparked_block = self + .store + .with_write_tx(|tx| tx.missing_transactions_remove(current_height, transaction_id))?; + + let Some(unparked_block) = maybe_unparked_block else { + return Ok(None); + }; + + info!(target: LOG_TARGET, "♻️ all transactions for block {unparked_block} are ready for consensus"); + + // todo(hacky): ensure that all transactions are in the pool. Race condition: because we have not yet + // received it yet in the select! loop. + self.store.with_write_tx(|tx| { + for tx_id in unparked_block.all_transaction_ids() { + if self.transaction_pool.exists(&**tx, tx_id)? { + continue; + } + + warn!( + target: LOG_TARGET, + "⚠️ Transaction {} is missing from the transaction pool. Attempting to recover.", + tx_id + ); + + let transaction = TransactionRecord::get(&**tx, tx_id)?; + // Did the mempool execute it? + if transaction.is_executed() { + // This should never fail + let executed = ExecutedTransaction::try_from(transaction)?; + self.transaction_pool.insert(tx, executed.to_atom())?; + } else { + // Deferred execution + self.transaction_pool + .insert(tx, TransactionAtom::deferred(*transaction.id()))?; + } + } + + Ok::<_, HotStuffError>(()) + })?; + + let vn = self + .epoch_manager + .get_validator_node_by_public_key(unparked_block.epoch(), unparked_block.proposed_by()) + .await?; + + let _ignore = self.tx_events.send(HotstuffEvent::ParkedBlockReady { + block: unparked_block.as_leaf_block(), + }); + + Ok(Some(( + vn.address, + HotstuffMessage::Proposal(ProposalMessage { block: unparked_block }), + ))) + } + + async fn check_proposal(&self, block: &Block) -> Result<(), HotStuffError> { + block_validations::check_proposal::( + block, + self.network, + &self.epoch_manager, + &self.vote_signing_service, + &self.leader_strategy, + &self.config, + ) + .await?; + Ok(()) + } + + async fn handle_missing_transactions(&mut self, block: Block) -> Result, HotStuffError> { + let (missing_tx_ids, awaiting_execution) = self + .store + .with_write_tx(|tx| self.check_for_missing_transactions(tx, &block))?; + + if missing_tx_ids.is_empty() && awaiting_execution.is_empty() { + return Ok(Some(block)); + } + + let _ignore = self.tx_events.send(HotstuffEvent::ProposedBlockParked { + block: block.as_leaf_block(), + num_missing_txs: missing_tx_ids.len(), + num_awaiting_txs: awaiting_execution.len(), + }); + + if !missing_tx_ids.is_empty() { + let block_id = *block.id(); + let epoch = block.epoch(); + let block_proposed_by = block.proposed_by().clone(); + + let vn = self + .epoch_manager + .get_validator_node_by_public_key(epoch, &block_proposed_by) + .await?; + + let mut request_from_address = vn.address; + + // (Yet another) Edge case: If we're catching up, we could be the proposer but we no longer have the + // transaction (we deleted our database) In this case, request from another random VN + // (TODO: not 100% reliable) + if request_from_address == self.local_validator_addr { + let mut local_committee = self.epoch_manager.get_local_committee(epoch).await?; + + local_committee.shuffle(); + match local_committee + .into_iter() + .find(|(addr, _)| *addr != self.local_validator_addr) + { + Some((addr, _)) => { + warn!(target: LOG_TARGET, "⚠️Requesting missing transactions from another validator {addr} because we are (presumably) catching up (local_peer_id = {})", self.local_validator_addr); + request_from_address = addr; + }, + None => { + warn!( + target: LOG_TARGET, + "❌NEVERHAPPEN: We're the only validator in the committee but we need to request missing transactions." + ); + return Ok(None); + }, + } + } + + self.outbound_messaging + .send( + request_from_address, + HotstuffMessage::RequestMissingTransactions(RequestMissingTransactionsMessage { + block_id, + epoch, + transactions: missing_tx_ids, + }), + ) + .await?; + } + + Ok(None) + } + + fn check_for_missing_transactions( + &self, + tx: &mut ::WriteTransaction<'_>, + block: &Block, + ) -> Result<(HashSet, HashSet), HotStuffError> { + if block.commands().is_empty() { + debug!( + target: LOG_TARGET, + "βœ… Block {} is empty (no missing transactions)", block + ); + return Ok((HashSet::new(), HashSet::new())); + } + let (transactions, missing_tx_ids) = TransactionRecord::get_any(&**tx, block.all_transaction_ids())?; + let awaiting_execution_or_deferred = transactions + .into_iter() + .filter(|tx| tx.final_decision.is_some()) + .filter(|tx| tx.result.is_none()) + .map(|tx| *tx.transaction.id()) + .collect::>(); + + // TODO(hacky): improve this. We need to account for transactions that are deferred when determining which + // transactions are awaiting execution. + let mut awaiting_execution = HashSet::new(); + for id in &awaiting_execution_or_deferred { + if let Some(t) = TransactionPoolRecord::get(&**tx, id).optional()? { + if !t.is_deferred() { + awaiting_execution.insert(*id); + } + } + } + + if missing_tx_ids.is_empty() && awaiting_execution.is_empty() { + debug!( + target: LOG_TARGET, + "βœ… Block {} has no missing transactions", block + ); + return Ok((HashSet::new(), HashSet::new())); + } + + info!( + target: LOG_TARGET, + "⏳ Block {} has {} missing transactions and {} awaiting execution", block, missing_tx_ids.len(), awaiting_execution.len(), + ); + + tx.missing_transactions_insert(block, &missing_tx_ids, &awaiting_execution)?; + + Ok((missing_tx_ids, awaiting_execution)) + } +} + +#[derive(Debug)] +pub enum MessageValidationResult { + Ready { + from: TAddr, + message: HotstuffMessage, + }, + NotReady, + Discard, + Invalid { + from: TAddr, + message: HotstuffMessage, + err: HotStuffError, + }, +} diff --git a/dan_layer/consensus/src/hotstuff/on_next_sync_view.rs b/dan_layer/consensus/src/hotstuff/on_next_sync_view.rs index 453249831..fa2d0c231 100644 --- a/dan_layer/consensus/src/hotstuff/on_next_sync_view.rs +++ b/dan_layer/consensus/src/hotstuff/on_next_sync_view.rs @@ -2,7 +2,7 @@ // SPDX-License-Identifier: BSD-3-Clause use log::*; -use tari_dan_common_types::{optional::Optional, NodeHeight}; +use tari_dan_common_types::{optional::Optional, Epoch, NodeHeight}; use tari_dan_storage::{ consensus_models::{HighQc, LastSentVote}, StateStore, @@ -39,11 +39,10 @@ impl OnNextSyncViewHandler { } } - pub async fn handle(&mut self, new_height: NodeHeight) -> Result<(), HotStuffError> { - let current_epoch = self.epoch_manager.current_epoch().await?; - info!(target: LOG_TARGET, "⚠️ Leader failure: NEXTSYNCVIEW for epoch {} and node height {}", current_epoch, new_height); + pub async fn handle(&mut self, epoch: Epoch, new_height: NodeHeight) -> Result<(), HotStuffError> { + info!(target: LOG_TARGET, "⚠️ Leader failure: NEXTSYNCVIEW for epoch {} and node height {}", epoch, new_height); // Is the VN registered? - if !self.epoch_manager.is_epoch_active(current_epoch).await? { + if !self.epoch_manager.is_epoch_active(epoch).await? { info!( target: LOG_TARGET, "[on_leader_timeout] Validator is not active within this epoch" @@ -59,7 +58,7 @@ impl OnNextSyncViewHandler { Ok::<_, HotStuffError>((high_qc, last_sent_vote)) })?; - let local_committee = self.epoch_manager.get_local_committee(current_epoch).await?; + let local_committee = self.epoch_manager.get_local_committee(epoch).await?; let next_leader = self .leader_strategy .get_leader_for_next_block(&local_committee, new_height); @@ -68,7 +67,7 @@ impl OnNextSyncViewHandler { let message = NewViewMessage { high_qc, new_height, - epoch: current_epoch, + epoch, last_vote: last_sent_vote.map(VoteMessage::from), }; diff --git a/dan_layer/consensus/src/hotstuff/on_propose.rs b/dan_layer/consensus/src/hotstuff/on_propose.rs index 9b54edae4..b55d58706 100644 --- a/dan_layer/consensus/src/hotstuff/on_propose.rs +++ b/dan_layer/consensus/src/hotstuff/on_propose.rs @@ -22,7 +22,6 @@ use tari_dan_storage::{ consensus_models::{ Block, Command, - EpochEvent, ExecutedTransaction, ForeignProposal, ForeignSendCounters, @@ -49,7 +48,7 @@ use crate::{ hotstuff::{ calculate_state_merkle_diff, error::HotStuffError, - substate_store::PendingSubstateStore, + substate_store::{ChainScopedTreeStore, PendingSubstateStore}, EXHAUST_DIVISOR, }, messages::{HotstuffMessage, ProposalMessage}, @@ -104,9 +103,10 @@ where TConsensusSpec: ConsensusSpec local_committee: &Committee, leaf_block: LeafBlock, is_newview_propose: bool, + propose_epoch_end: bool, ) -> Result<(), HotStuffError> { if let Some(last_proposed) = self.store.with_read_tx(|tx| LastProposed::get(tx)).optional()? { - if last_proposed.height > leaf_block.height { + if last_proposed.epoch == leaf_block.epoch && last_proposed.height > leaf_block.height { // is_newview_propose means that a NEWVIEW has reached quorum and nodes are expecting us to propose. // Re-broadcast the previous proposal if is_newview_propose { @@ -129,7 +129,7 @@ where TConsensusSpec: ConsensusSpec info!( target: LOG_TARGET, - "‡️ SKIPPING propose for leaf {} because we already proposed block {}", + "‡️ SKIPPING propose for {} because we already proposed block {}", leaf_block, last_proposed, ); @@ -142,54 +142,18 @@ where TConsensusSpec: ConsensusSpec let local_committee_info = self.epoch_manager.get_local_committee_info(epoch).await?; let (current_base_layer_block_height, current_base_layer_block_hash) = self.epoch_manager.current_base_layer_block_info().await?; - let (high_qc, qc_block, locked_block) = self.store.with_read_tx(|tx| { - let high_qc = HighQc::get(tx)?; - let qc_block = high_qc.get_block(tx)?; - let locked_block = LockedBlock::get(tx)?.get_block(tx)?; - Ok::<_, HotStuffError>((high_qc, qc_block, locked_block)) - })?; - - let parent_base_layer_block_hash = qc_block.base_layer_block_hash(); - - let base_layer_block_hash = if qc_block.base_layer_block_height() >= current_base_layer_block_height { - *parent_base_layer_block_hash - } else { - // We select our current base layer block hash as the base layer block hash for the next block if - // and only if we know that the parent block was smaller. - current_base_layer_block_hash - }; - // If epoch has changed, we should first end the epoch with an EpochEvent::End - let propose_epoch_end = - // If we didn't locked block with an EpochEvent::End - !locked_block.is_epoch_end() && - // The last block is from previous epoch or it is an EpochEnd block - (qc_block.epoch() < epoch || qc_block.is_epoch_end()) && - // If the previous epoch is the genesis epoch, we don't need to end it (there was no committee at epoch 0) - !qc_block.is_genesis(); - - // If the epoch is changed, we use the current epoch - let epoch = if propose_epoch_end { qc_block.epoch() } else { epoch }; - let base_layer_block_hash = if propose_epoch_end { - self.epoch_manager.get_last_block_of_current_epoch().await? - } else { - base_layer_block_hash - }; - let base_layer_block_height = self - .epoch_manager - .get_base_layer_block_height(base_layer_block_hash) - .await? - .unwrap(); - // The epoch is greater only when the EpochEnd event is locked. - let propose_epoch_start = qc_block.epoch() < epoch; + let base_layer_block_hash = current_base_layer_block_hash; + let base_layer_block_height = current_base_layer_block_height; let next_block = self.store.with_write_tx(|tx| { - let high_qc = high_qc.get_quorum_certificate(&**tx)?; + let high_qc = HighQc::get(&**tx)?; + let high_qc_cert = high_qc.get_quorum_certificate(&**tx)?; let (next_block, executed_transactions) = self.build_next_block( tx, epoch, &leaf_block, - high_qc, + high_qc_cert, validator.public_key, &local_committee_info, // TODO: This just avoids issues with proposed transactions causing leader failures. Not sure if this @@ -197,7 +161,6 @@ where TConsensusSpec: ConsensusSpec is_newview_propose, base_layer_block_height, base_layer_block_hash, - propose_epoch_start, propose_epoch_end, )?; @@ -230,7 +193,8 @@ where TConsensusSpec: ConsensusSpec info!( target: LOG_TARGET, - "🌿 PROPOSING new local block {} to {} validators. justify: {} ({}), parent: {}", + "🌿 [{}] PROPOSING new local block {} to {} validators. justify: {} ({}), parent: {}", + validator.address, next_block, local_committee.len(), next_block.justify().block_id(), @@ -386,7 +350,18 @@ where TConsensusSpec: ConsensusSpec tx_rec.transaction_id(), )) })?; - substate_store.put_diff(*tx_rec.transaction_id(), diff)?; + if let Err(err) = substate_store.put_diff(*tx_rec.transaction_id(), diff) { + warn!( + target: LOG_TARGET, + "πŸ”’ Transaction {} cannot be locked for LocalOnly: {}. Proposing to ABORT...", + tx_rec.transaction_id(), + err, + ); + // Only error if it is not related to lock errors + let _err = err.ok_or_fatal_error()?; + // If the transaction does not lock, we propose to abort it + return Ok(Some(Command::LocalOnly(tx_atom.abort()))); + } } return Ok(Some(Command::LocalOnly(tx_atom))); } @@ -483,12 +458,11 @@ where TConsensusSpec: ConsensusSpec empty_block: bool, base_layer_block_height: u64, base_layer_block_hash: FixedHash, - propose_epoch_start: bool, propose_epoch_end: bool, ) -> Result<(Block, HashMap), HotStuffError> { // TODO: Configure const TARGET_BLOCK_SIZE: usize = 1000; - let batch = if empty_block || propose_epoch_end || propose_epoch_start { + let batch = if empty_block || propose_epoch_end { vec![] } else { self.transaction_pool.get_batch_for_next_block(tx, TARGET_BLOCK_SIZE)? @@ -499,10 +473,8 @@ where TConsensusSpec: ConsensusSpec let mut total_leader_fee = 0; let locked_block = LockedBlock::get(tx)?; let pending_proposals = ForeignProposal::get_all_pending(tx, locked_block.block_id(), parent_block.block_id())?; - let mut commands = if propose_epoch_start { - BTreeSet::from_iter([Command::EpochEvent(EpochEvent::Start)]) - } else if propose_epoch_end { - BTreeSet::from_iter([Command::EpochEvent(EpochEvent::End)]) + let mut commands = if propose_epoch_end { + BTreeSet::from_iter([Command::EndEpoch]) } else { ForeignProposal::get_all_new(tx)? .into_iter() @@ -511,7 +483,7 @@ where TConsensusSpec: ConsensusSpec foreign_proposal.base_layer_block_height <= base_layer_block_height && // If the foreign proposal is already pending, don't propose it again !pending_proposals.iter().any(|pending_proposal| { - pending_proposal.bucket == foreign_proposal.bucket && + pending_proposal.shard == foreign_proposal.shard && pending_proposal.block_id == foreign_proposal.block_id }) }) @@ -523,7 +495,8 @@ where TConsensusSpec: ConsensusSpec }; // batch is empty for is_empty, is_epoch_end and is_epoch_start blocks - let mut substate_store = PendingSubstateStore::new(tx); + let tree_store = ChainScopedTreeStore::new(epoch, local_committee_info.shard(), tx); + let mut substate_store = PendingSubstateStore::new(tree_store); let mut executed_transactions = HashMap::new(); for transaction in batch { if let Some(command) = self.transaction_pool_record_to_command( @@ -548,13 +521,14 @@ where TConsensusSpec: ConsensusSpec commands.iter().map(|c| c.to_string()).collect::>().join(",") ); - let pending = PendingStateTreeDiff::get_all_up_to_commit_block(tx, high_qc.block_id())?; - + let pending_tree_diffs = + PendingStateTreeDiff::get_all_up_to_commit_block(tx, high_qc.epoch(), high_qc.shard(), high_qc.block_id())?; + let store = ChainScopedTreeStore::new(epoch, local_committee_info.shard(), tx); let (state_root, _) = calculate_state_merkle_diff( - tx, + &store, current_version, next_height.as_u64(), - pending, + pending_tree_diffs, substate_store.diff().iter().map(|ch| ch.into()), )?; diff --git a/dan_layer/consensus/src/hotstuff/on_ready_to_vote_on_local_block.rs b/dan_layer/consensus/src/hotstuff/on_ready_to_vote_on_local_block.rs index 612591f43..6448ddedc 100644 --- a/dan_layer/consensus/src/hotstuff/on_ready_to_vote_on_local_block.rs +++ b/dan_layer/consensus/src/hotstuff/on_ready_to_vote_on_local_block.rs @@ -5,11 +5,7 @@ use std::num::NonZeroU64; use log::*; -use tari_common::configuration::Network; -use tari_dan_common_types::{ - committee::{Committee, CommitteeInfo}, - optional::Optional, -}; +use tari_dan_common_types::{committee::CommitteeInfo, optional::Optional}; use tari_dan_storage::{ consensus_models::{ Block, @@ -17,11 +13,9 @@ use tari_dan_storage::{ BlockId, Command, Decision, - EpochEvent, ExecutedTransaction, ForeignProposal, LastExecuted, - LastSentVote, LastVoted, LockedBlock, PendingStateTreeDiff, @@ -38,99 +32,68 @@ use tari_dan_storage::{ StateStore, StateStoreWriteTransaction, }; -use tari_epoch_manager::EpochManagerReader; use tari_transaction::TransactionId; use tokio::sync::broadcast; -use super::proposer::Proposer; use crate::{ hotstuff::{ block_change_set::{BlockDecision, ProposedBlockChangeSet}, error::HotStuffError, event::HotstuffEvent, - substate_store::PendingSubstateStore, + substate_store::{ChainScopedTreeStore, PendingSubstateStore}, ProposalValidationError, EXHAUST_DIVISOR, }, - messages::{HotstuffMessage, VoteMessage}, - traits::{ - hooks::ConsensusHooks, - BlockTransactionExecutor, - ConsensusSpec, - LeaderStrategy, - OutboundMessaging, - VoteSignatureService, - WriteableSubstateStore, - }, + traits::{BlockTransactionExecutor, ConsensusSpec, WriteableSubstateStore}, }; -const LOG_TARGET: &str = "tari::dan::consensus::hotstuff::on_lock_block_ready"; +const LOG_TARGET: &str = "tari::dan::consensus::hotstuff::on_ready_to_vote_on_local_block"; pub struct OnReadyToVoteOnLocalBlock { local_validator_addr: TConsensusSpec::Addr, store: TConsensusSpec::StateStore, - epoch_manager: TConsensusSpec::EpochManager, - vote_signing_service: TConsensusSpec::SignatureService, - leader_strategy: TConsensusSpec::LeaderStrategy, transaction_pool: TransactionPool, - outbound_messaging: TConsensusSpec::OutboundMessaging, tx_events: broadcast::Sender, - proposer: Proposer, transaction_executor: TConsensusSpec::TransactionExecutor, - network: Network, - hooks: TConsensusSpec::Hooks, } impl OnReadyToVoteOnLocalBlock where TConsensusSpec: ConsensusSpec { - #[allow(clippy::too_many_arguments)] pub fn new( validator_addr: TConsensusSpec::Addr, store: TConsensusSpec::StateStore, - epoch_manager: TConsensusSpec::EpochManager, - vote_signing_service: TConsensusSpec::SignatureService, - leader_strategy: TConsensusSpec::LeaderStrategy, transaction_pool: TransactionPool, - outbound_messaging: TConsensusSpec::OutboundMessaging, tx_events: broadcast::Sender, - proposer: Proposer, transaction_executor: TConsensusSpec::TransactionExecutor, - network: Network, - hooks: TConsensusSpec::Hooks, ) -> Self { Self { local_validator_addr: validator_addr, store, - epoch_manager, - vote_signing_service, - leader_strategy, transaction_pool, - outbound_messaging, tx_events, - proposer, transaction_executor, - network, - hooks, } } - pub async fn handle(&mut self, valid_block: ValidBlock) -> Result<(), HotStuffError> { + pub fn handle( + &mut self, + valid_block: &ValidBlock, + local_committee_info: CommitteeInfo, + can_propose_epoch_end: bool, + ) -> Result { debug!( target: LOG_TARGET, "πŸ”₯ LOCAL PROPOSAL READY: {}", valid_block, ); - let local_committee_shard = self - .epoch_manager - .get_committee_info_by_validator_public_key(valid_block.epoch(), valid_block.proposed_by()) - .await?; - let block_decision = self.store.with_write_tx(|tx| { - let change_set = self.decide_on_block(&**tx, &local_committee_shard, &valid_block)?; + self.store.with_write_tx(|tx| { + let change_set = self.decide_on_block(&**tx, &local_committee_info, valid_block, can_propose_epoch_end)?; let mut locked_blocks = Vec::new(); let mut finalized_transactions = Vec::new(); + let mut end_of_epoch = None; if change_set.is_accept() { // Update nodes @@ -141,7 +104,10 @@ where TConsensusSpec: ConsensusSpec self.on_lock_block(tx, locked, block) }, |tx, last_exec, commit_block| { - let committed = self.on_commit(tx, last_exec, commit_block, &local_committee_shard)?; + let committed = self.on_commit(tx, last_exec, commit_block, &local_committee_info)?; + if commit_block.is_epoch_end() { + end_of_epoch = Some(commit_block.epoch()); + } if !committed.is_empty() { finalized_transactions.push(committed); } @@ -161,45 +127,9 @@ where TConsensusSpec: ConsensusSpec quorum_decision, locked_blocks, finalized_transactions, + end_of_epoch, }) - })?; - - self.hooks - .on_local_block_decide(&valid_block, block_decision.quorum_decision); - for t in block_decision.finalized_transactions.into_iter().flatten() { - self.hooks.on_transaction_finalized(&t); - } - self.propose_newly_locked_blocks(block_decision.locked_blocks).await?; - - if let Some(decision) = block_decision.quorum_decision { - let is_registered = self - .epoch_manager - .is_this_validator_registered_for_epoch(valid_block.epoch()) - .await?; - - if is_registered { - debug!( - target: LOG_TARGET, - "πŸ”₯ LOCAL PROPOSAL {} DECIDED {:?}", - valid_block, - decision, - ); - let local_committee = self.epoch_manager.get_local_committee(valid_block.epoch()).await?; - - let vote = self.generate_vote_message(valid_block.block(), decision).await?; - self.send_vote_to_leader(&local_committee, vote, valid_block.block()) - .await?; - } else { - info!( - target: LOG_TARGET, - "❓️ Local validator not registered for epoch {}. Not voting on block {}", - valid_block.epoch(), - valid_block, - ); - } - } - - Ok(()) + }) } fn decide_on_block( @@ -207,34 +137,19 @@ where TConsensusSpec: ConsensusSpec tx: &::ReadTransaction<'_>, local_committee_info: &CommitteeInfo, valid_block: &ValidBlock, + can_propose_epoch_end: bool, ) -> Result { - let qc_block = valid_block.block().justify().get_block(tx)?; - let locked_block = LockedBlock::get(tx)?.get_block(tx)?; + // let qc_block = valid_block.block().justify().get_block(tx)?; + // let locked_block = LockedBlock::get(tx)?.get_block(tx)?; // If the previous qc block was in different epoch, we have to have EpochEvent::Start - let epoch_should_start = qc_block.epoch() < valid_block.epoch(); - - let epoch_should_end = - // If the epoch has not changed yet - qc_block.epoch() == valid_block.epoch() && - // If the last justified block is an epoch end - qc_block.is_epoch_end() && - // if the locked block is an epoch end, then we do not expect this block to be an epoch end - !locked_block.is_epoch_end() && - // If the previous epoch is the genesis epoch, we don't need to end it (there was no committee at epoch 0) - !qc_block.is_genesis(); + // let epoch_should_start = qc_block.epoch() < valid_block.epoch(); if !self.should_vote(tx, valid_block.block())? { return Ok(ProposedBlockChangeSet::new(valid_block.block().as_leaf_block()).no_vote()); } - self.decide_what_to_vote( - tx, - valid_block.block(), - local_committee_info, - epoch_should_start, - epoch_should_end, - ) + self.decide_what_to_vote(tx, valid_block.block(), local_committee_info, can_propose_epoch_end) } /// if b_new .height > vheight && (b_new extends b_lock || b_new .justify.node.height > b_lock .height) @@ -264,68 +179,44 @@ where TConsensusSpec: ConsensusSpec Ok(true) } - async fn send_vote_to_leader( - &mut self, - local_committee: &Committee, - vote: VoteMessage, - block: &Block, - ) -> Result<(), HotStuffError> { - let leader = self - .leader_strategy - .get_leader_for_next_block(local_committee, block.height()); - info!( - target: LOG_TARGET, - "πŸ”₯ VOTE {:?} for block {} proposed by {} to next leader {:.4}", - vote.decision, - block, - block.proposed_by(), - leader, - ); - self.outbound_messaging - .send(leader.clone(), HotstuffMessage::Vote(vote.clone())) - .await?; - self.store.with_write_tx(|tx| { - let last_sent_vote = LastSentVote { - epoch: vote.epoch, - block_id: vote.block_id, - block_height: vote.block_height, - decision: vote.decision, - signature: vote.signature, - }; - last_sent_vote.set(tx) - })?; - Ok(()) - } - #[allow(clippy::too_many_lines)] fn decide_what_to_vote( &self, tx: &::ReadTransaction<'_>, block: &Block, local_committee_info: &CommitteeInfo, - epoch_should_start: bool, - epoch_should_end: bool, + can_propose_epoch_end: bool, ) -> Result { let mut total_leader_fee = 0; // 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); + let tree_store = ChainScopedTreeStore::new(block.epoch(), block.shard(), tx); + let mut substate_store = PendingSubstateStore::new(tree_store); let mut proposed_block_change_set = ProposedBlockChangeSet::new(block.as_leaf_block()); - if epoch_should_start && !block.is_epoch_start() { + // if epoch_should_start && !block.is_epoch_start() { + // warn!( + // target: LOG_TARGET, + // "❌ EpochEvent::Start command expected for block {} but not found", + // block.id() + // ); + // return Ok(proposed_block_change_set.no_vote()); + // } + // + // if epoch_should_end && !block.is_epoch_end() { + // warn!( + // target: LOG_TARGET, + // "❌ EpochEvent::End command expected for block {} but not found", + // block.id() + // ); + // return Ok(proposed_block_change_set.no_vote()); + // } + + if block.is_epoch_end() && block.commands().len() > 1 { warn!( target: LOG_TARGET, - "❌ EpochEvent::Start command expected for block {} but not found", - block.id() - ); - return Ok(proposed_block_change_set.no_vote()); - } - - if epoch_should_end && !block.is_epoch_end() { - warn!( - target: LOG_TARGET, - "❌ EpochEvent::End command expected for block {} but not found", + "❌ EpochEvent::End command in block {} but block contains other commands", block.id() ); return Ok(proposed_block_change_set.no_vote()); @@ -337,25 +228,20 @@ where TConsensusSpec: ConsensusSpec warn!( target: LOG_TARGET, "❌ Foreign proposal for block {block_id} from bucket {bucket} does not exist in the store", - block_id = foreign_proposal.block_id,bucket = foreign_proposal.bucket + block_id = foreign_proposal.block_id,bucket = foreign_proposal.shard ); return Ok(proposed_block_change_set.no_vote()); } continue; } - if let Command::EpochEvent(event) = cmd { - match event { - EpochEvent::Start => { - if !epoch_should_start { - warn!( - target: LOG_TARGET, - "❌ EpochEvent::Start command received for block {} but it is not the start of the epoch", - block.id() - ); - return Ok(proposed_block_change_set.no_vote()); - } - }, - EpochEvent::End => {}, + if cmd.is_epoch_end() { + if !can_propose_epoch_end { + warn!( + target: LOG_TARGET, + "❌ EpochEvent::End command received for block {} but it is not the next epoch", + block.id(), + ); + return Ok(proposed_block_change_set.no_vote()); } continue; } @@ -858,7 +744,7 @@ where TConsensusSpec: ConsensusSpec ); }, // This was already handled above - Command::EpochEvent(_) => {}, + Command::EndEpoch => {}, } } @@ -963,28 +849,6 @@ where TConsensusSpec: ConsensusSpec Ok(true) } - async fn generate_vote_message( - &self, - block: &Block, - decision: QuorumDecision, - ) -> Result { - let vn = self - .epoch_manager - .get_validator_node(block.epoch(), &self.local_validator_addr) - .await?; - let leaf_hash = vn.get_node_hash(self.network); - - let signature = self.vote_signing_service.sign_vote(&leaf_hash, block.id(), &decision); - - Ok(VoteMessage { - epoch: block.epoch(), - block_id: *block.id(), - block_height: block.height(), - decision, - signature, - }) - } - fn on_commit( &self, tx: &mut ::WriteTransaction<'_>, @@ -992,7 +856,7 @@ where TConsensusSpec: ConsensusSpec block: &Block, local_committee_info: &CommitteeInfo, ) -> Result, HotStuffError> { - let committed_transactions = self.execute(tx, block, local_committee_info)?; + let committed_transactions = self.finalize_block(tx, block, local_committee_info)?; debug!( target: LOG_TARGET, "βœ… COMMIT block {}, last executed height = {}", @@ -1000,6 +864,7 @@ where TConsensusSpec: ConsensusSpec last_executed.height ); self.publish_event(HotstuffEvent::BlockCommitted { + epoch: block.epoch(), block_id: *block.id(), height: block.height(), }); @@ -1033,57 +898,18 @@ where TConsensusSpec: ConsensusSpec Ok(()) } - async fn propose_newly_locked_blocks(&mut self, blocks: Vec) -> Result<(), HotStuffError> { - for block in blocks { - debug!(target:LOG_TARGET,"Broadcast new locked block: {block}"); - let local_committee = self - .epoch_manager - .get_committee_by_validator_public_key(block.epoch(), block.proposed_by()) - .await?; - let Some(our_addr) = self - .epoch_manager - .get_our_validator_node(block.epoch()) - .await - .optional()? - else { - info!( - target: LOG_TARGET, - "❌ Our validator node is not registered for epoch {}. Not proposing {block} to foreign committee", - block.epoch(), - ); - continue; - }; - let leader_index = self.leader_strategy.calculate_leader(&local_committee, block.height()); - let my_index = local_committee - .addresses() - .position(|addr| *addr == our_addr.address) - .ok_or_else(|| HotStuffError::InvariantError("Our address not found in local committee".to_string()))?; - // There are other ways to approach this. But for simplicty is better just to make sure at least one honest - // node will send it to the whole foreign committee. So we select the leader and f other nodes. It has to be - // deterministic so we select by index (leader, leader+1, ..., leader+f). FYI: The messages between - // committees and within committees are not different in terms of size, speed, etc. - let diff_from_leader = (my_index + local_committee.len() - leader_index as usize) % local_committee.len(); - // f+1 nodes (always including the leader) send the proposal to the foreign committee - // if diff_from_leader <= (local_committee.len() - 1) / 3 + 1 { - if diff_from_leader <= local_committee.len() / 3 { - self.proposer.broadcast_foreign_proposal_if_required(block).await?; - } - } - Ok(()) - } - fn publish_event(&self, event: HotstuffEvent) { let _ignore = self.tx_events.send(event); } - fn execute( + fn finalize_block( &self, tx: &mut ::WriteTransaction<'_>, block: &Block, local_committee_info: &CommitteeInfo, ) -> Result, HotStuffError> { - // Nothing to do here for empty dummy blocks if block.is_dummy() { + // Nothing to do here for empty dummy blocks. Just mark the block as committed. block.commit_diff(tx, BlockDiff::empty(*block.id()))?; return Ok(vec![]); } @@ -1114,7 +940,8 @@ where TConsensusSpec: ConsensusSpec tx.substate_locks_remove_many_for_transactions(block.all_accepted_transactions_ids())?; let pending = PendingStateTreeDiff::remove_by_block(tx, block.id())?; - let mut state_tree = tari_state_tree::SpreadPrefixStateTree::new(tx); + let mut store = ChainScopedTreeStore::new(block.epoch(), block.shard(), tx); + let mut state_tree = tari_state_tree::SpreadPrefixStateTree::new(&mut store); state_tree.commit_diff(pending.diff)?; let total_transaction_fee = block.total_transaction_fee(); diff --git a/dan_layer/consensus/src/hotstuff/on_receive_foreign_proposal.rs b/dan_layer/consensus/src/hotstuff/on_receive_foreign_proposal.rs index d7d707028..7347aabc7 100644 --- a/dan_layer/consensus/src/hotstuff/on_receive_foreign_proposal.rs +++ b/dan_layer/consensus/src/hotstuff/on_receive_foreign_proposal.rs @@ -52,7 +52,7 @@ where TConsensusSpec: ConsensusSpec pub async fn handle(&mut self, from: TConsensusSpec::Addr, message: ProposalMessage) -> Result<(), HotStuffError> { let ProposalMessage { block } = message; - debug!( + info!( target: LOG_TARGET, "πŸ”₯ Receive FOREIGN PROPOSAL for block {}, parent {}, height {} from {}", block.id(), @@ -232,7 +232,7 @@ where TConsensusSpec: ConsensusSpec // ), // }); // } - if candidate_block.height().is_zero() || candidate_block.is_genesis() { + if candidate_block.is_genesis() { return Err(ProposalValidationError::ProposingGenesisBlock { proposed_by: from.to_string(), hash: *candidate_block.id(), diff --git a/dan_layer/consensus/src/hotstuff/on_receive_local_proposal.rs b/dan_layer/consensus/src/hotstuff/on_receive_local_proposal.rs index 612eb142d..bd4a62e17 100644 --- a/dan_layer/consensus/src/hotstuff/on_receive_local_proposal.rs +++ b/dan_layer/consensus/src/hotstuff/on_receive_local_proposal.rs @@ -1,15 +1,12 @@ // Copyright 2023 The Tari Project // SPDX-License-Identifier: BSD-3-Clause -// (New, true) ----(cmd:Prepare) ---> (Prepared, true) -----cmd:LocalPrepared ---> (LocalPrepared, false) -// ----[foreign:LocalPrepared]--->(LocalPrepared, true) ----cmd:AllPrepare ---> (AllPrepared, true) ---cmd:Accept ---> -// Complete - use log::*; use tari_common::configuration::Network; use tari_dan_common_types::{ committee::{Committee, CommitteeInfo}, optional::Optional, + Epoch, NodeHeight, }; use tari_dan_storage::{ @@ -19,6 +16,8 @@ use tari_dan_storage::{ ExecutedTransaction, ForeignProposal, HighQc, + LastSentVote, + QuorumDecision, TransactionAtom, TransactionPool, TransactionPoolStage, @@ -33,19 +32,22 @@ use tokio::sync::broadcast; use super::proposer::Proposer; use crate::{ hotstuff::{ + calculate_dummy_blocks, + current_view::CurrentView, error::HotStuffError, on_ready_to_vote_on_local_block::OnReadyToVoteOnLocalBlock, pacemaker_handle::PaceMakerHandle, HotstuffEvent, ProposalValidationError, }, - messages::ProposalMessage, - traits::{hooks::ConsensusHooks, ConsensusSpec, LeaderStrategy}, + messages::{HotstuffMessage, ProposalMessage, VoteMessage}, + traits::{hooks::ConsensusHooks, ConsensusSpec, LeaderStrategy, OutboundMessaging, VoteSignatureService}, }; const LOG_TARGET: &str = "tari::dan::consensus::hotstuff::on_receive_local_proposal"; pub struct OnReceiveLocalProposalHandler { + local_validator_addr: TConsensusSpec::Addr, network: Network, store: TConsensusSpec::StateStore, epoch_manager: TConsensusSpec::EpochManager, @@ -53,13 +55,16 @@ pub struct OnReceiveLocalProposalHandler { pacemaker: PaceMakerHandle, transaction_pool: TransactionPool, on_ready_to_vote_on_local_block: OnReadyToVoteOnLocalBlock, + outbound_messaging: TConsensusSpec::OutboundMessaging, + vote_signing_service: TConsensusSpec::SignatureService, + proposer: Proposer, hooks: TConsensusSpec::Hooks, } impl OnReceiveLocalProposalHandler { #[allow(clippy::too_many_arguments)] pub fn new( - validator_addr: TConsensusSpec::Addr, + local_validator_addr: TConsensusSpec::Addr, store: TConsensusSpec::StateStore, epoch_manager: TConsensusSpec::EpochManager, leader_strategy: TConsensusSpec::LeaderStrategy, @@ -74,31 +79,28 @@ impl OnReceiveLocalProposalHandler Self { Self { + local_validator_addr: local_validator_addr.clone(), network, store: store.clone(), - epoch_manager: epoch_manager.clone(), - leader_strategy: leader_strategy.clone(), + epoch_manager, + leader_strategy, pacemaker, transaction_pool: transaction_pool.clone(), - hooks: hooks.clone(), + vote_signing_service, + outbound_messaging, + proposer, + hooks, on_ready_to_vote_on_local_block: OnReadyToVoteOnLocalBlock::new( - validator_addr, + local_validator_addr, store, - epoch_manager, - vote_signing_service, - leader_strategy, transaction_pool, - outbound_messaging, tx_events, - proposer, transaction_executor, - network, - hooks, ), } } - pub async fn handle(&mut self, message: ProposalMessage) -> Result<(), HotStuffError> { + pub async fn handle(&mut self, current_view: CurrentView, message: ProposalMessage) -> Result<(), HotStuffError> { let ProposalMessage { block } = message; debug!( @@ -108,7 +110,7 @@ impl OnReceiveLocalProposalHandler Ok(()), Err(err @ HotStuffError::ProposalValidationError(_)) => { self.hooks.on_block_validation_failed(&err); @@ -118,7 +120,8 @@ impl OnReceiveLocalProposalHandler Result<(), HotStuffError> { + #[allow(clippy::too_many_lines)] + async fn process_block(&mut self, current_view: CurrentView, block: Block) -> Result<(), HotStuffError> { if !self.epoch_manager.is_epoch_active(block.epoch()).await? { return Err(HotStuffError::EpochNotActive { epoch: block.epoch(), @@ -130,7 +133,7 @@ impl OnReceiveLocalProposalHandler OnReceiveLocalProposalHandler OnReceiveLocalProposalHandler current_view.get_epoch(); + + let block_decision = self.on_ready_to_vote_on_local_block.handle( + &valid_block, + local_committee_info, + can_propose_epoch_end, + )?; + + self.hooks + .on_local_block_decide(&valid_block, block_decision.quorum_decision); + for t in block_decision.finalized_transactions.into_iter().flatten() { + self.hooks.on_transaction_finalized(&t); + } + self.propose_newly_locked_blocks(block_decision.locked_blocks).await?; + + if let Some(decision) = block_decision.quorum_decision { + let is_registered = self + .epoch_manager + .is_this_validator_registered_for_epoch(valid_block.epoch()) + .await?; + + if is_registered { + debug!( + target: LOG_TARGET, + "πŸ”₯ LOCAL PROPOSAL {} DECIDED {:?}", + valid_block, + decision, + ); + let local_committee = self.epoch_manager.get_local_committee(valid_block.epoch()).await?; + + let vote = self.generate_vote_message(valid_block.block(), decision).await?; + self.send_vote_to_leader(&local_committee, vote, valid_block.block()) + .await?; + } else { + info!( + target: LOG_TARGET, + "❓️ Local validator not registered for epoch {}. Not voting on block {}", + valid_block.epoch(), + valid_block, + ); + } + } - self.on_ready_to_vote_on_local_block.handle(valid_block).await?; + match block_decision.end_of_epoch { + Some(epoch) => { + let next_epoch = epoch + Epoch(1); + + // If we're registered for the next epoch. Create a new genesis block. + if let Some(vn) = self.epoch_manager.get_our_validator_node(next_epoch).await.optional()? { + // TODO: Change VN db to include the shard in the ValidatorNode struct. + let num_committees = self.epoch_manager.get_num_committees(next_epoch).await?; + let next_shard = vn.shard_key.to_shard(num_committees); + self.store.with_write_tx(|tx| { + let genesis = Block::genesis(self.network, next_epoch, next_shard); + info!(target: LOG_TARGET, "⭐️ Creating new genesis block {genesis}"); + genesis.insert(tx)?; + // We'll propose using the new genesis as parent + genesis.as_locked_block().set(tx)?; + genesis.as_leaf_block().set(tx)?; + genesis.as_last_executed().set(tx)?; + genesis.as_last_voted().set(tx)?; + genesis.justify().as_high_qc().set(tx) + })?; + + // Set the pacemaker to next epoch + self.pacemaker.set_epoch(next_epoch).await?; + self.pacemaker.on_beat(); + } else { + info!( + target: LOG_TARGET, + "πŸ’€ Our validator node is not registered for epoch {next_epoch}.", + ) + } + }, + None => { + self.pacemaker + .update_view(valid_block.epoch(), valid_block.height(), high_qc.block_height()) + .await?; + }, + } } Ok(()) } + async fn send_vote_to_leader( + &mut self, + local_committee: &Committee, + vote: VoteMessage, + block: &Block, + ) -> Result<(), HotStuffError> { + let leader = self + .leader_strategy + .get_leader_for_next_block(local_committee, block.height()); + info!( + target: LOG_TARGET, + "πŸ”₯ VOTE {:?} for block {} proposed by {} to next leader {:.4}", + vote.decision, + block, + block.proposed_by(), + leader, + ); + self.outbound_messaging + .send(leader.clone(), HotstuffMessage::Vote(vote.clone())) + .await?; + self.store.with_write_tx(|tx| { + let last_sent_vote = LastSentVote { + epoch: vote.epoch, + block_id: vote.block_id, + block_height: vote.block_height, + decision: vote.decision, + signature: vote.signature, + }; + last_sent_vote.set(tx) + })?; + Ok(()) + } + + async fn propose_newly_locked_blocks(&mut self, blocks: Vec) -> Result<(), HotStuffError> { + for block in blocks { + debug!(target:LOG_TARGET,"Broadcast new locked block: {block}"); + let local_committee = self + .epoch_manager + .get_committee_by_validator_public_key(block.epoch(), block.proposed_by()) + .await?; + let Some(our_addr) = self + .epoch_manager + .get_our_validator_node(block.epoch()) + .await + .optional()? + else { + info!( + target: LOG_TARGET, + "❌ Our validator node is not registered for epoch {}. Not proposing {block} to foreign committee", + block.epoch(), + ); + continue; + }; + let leader_index = self.leader_strategy.calculate_leader(&local_committee, block.height()); + let my_index = local_committee + .addresses() + .position(|addr| *addr == our_addr.address) + .ok_or_else(|| HotStuffError::InvariantError("Our address not found in local committee".to_string()))?; + // There are other ways to approach this. But for simplicty is better just to make sure at least one honest + // node will send it to the whole foreign committee. So we select the leader and f other nodes. It has to be + // deterministic so we select by index (leader, leader+1, ..., leader+f). FYI: The messages between + // committees and within committees are not different in terms of size, speed, etc. + let diff_from_leader = (my_index + local_committee.len() - leader_index as usize) % local_committee.len(); + // f+1 nodes (always including the leader) send the proposal to the foreign committee + // if diff_from_leader <= (local_committee.len() - 1) / 3 + 1 { + if diff_from_leader <= local_committee.len() / 3 { + self.proposer.broadcast_foreign_proposal_if_required(block).await?; + } + } + Ok(()) + } + + async fn generate_vote_message( + &self, + block: &Block, + decision: QuorumDecision, + ) -> Result { + let vn = self + .epoch_manager + .get_validator_node(block.epoch(), &self.local_validator_addr) + .await?; + let leaf_hash = vn.get_node_hash(self.network); + + let signature = self.vote_signing_service.sign_vote(&leaf_hash, block.id(), &decision); + + Ok(VoteMessage { + epoch: block.epoch(), + block_id: *block.id(), + block_height: block.height(), + decision, + signature, + }) + } + fn save_block( &self, tx: &mut ::WriteTransaction<'_>, @@ -191,7 +364,7 @@ impl OnReceiveLocalProposalHandler OnReceiveLocalProposalHandler::ReadTransaction<'_>, candidate_block: Block, local_committee: &Committee, - local_committee_info: &CommitteeInfo, + _local_committee_info: &CommitteeInfo, ) -> Result { if Block::has_been_processed(tx, candidate_block.id())? { return Err(ProposalValidationError::BlockAlreadyProcessed { @@ -360,7 +533,7 @@ impl OnReceiveLocalProposalHandler OnReceiveLocalProposalHandler OnReceiveLocalProposalHandler OnReceiveLocalProposalHandler candidate_block.height() { - warn!(target: LOG_TARGET, "πŸ”₯ Bad proposal, dummy block height {} is greater than new height {}", last_dummy_block, candidate_block); - return Err(ProposalValidationError::CandidateBlockDoesNotExtendJustify { - justify_block_height, + if !candidate_block.justifies_parent() { + let dummy_blocks = + calculate_dummy_blocks(&candidate_block, &justify_block, &self.leader_strategy, local_committee); + + let Some(last_dummy) = dummy_blocks.last() else { + warn!(target: LOG_TARGET, "❌ Bad proposal, does not justify parent for candidate block {}", candidate_block); + return Err( + ProposalValidationError::CandidateBlockDoesNotExtendJustify { + justify_block_height: justify_block.height(), candidate_block_height: candidate_block.height(), - } - .into()); - } + }.into() + ); + }; - let next_height = last_dummy_block.height() + NodeHeight(1); - let leader = self.leader_strategy.get_leader_public_key(local_committee, next_height); - - // TODO: replace with actual leader's propose - dummy_blocks.push(Block::dummy_block( - self.network, - *last_dummy_block.id(), - leader.clone(), - next_height, - candidate_block.justify().clone(), - candidate_block.epoch(), - local_committee_info.shard(), - *candidate_block.merkle_root(), - timestamp, - base_layer_block_height, - base_layer_block_hash, - )); - last_dummy_block = dummy_blocks.last().unwrap(); - debug!(target: LOG_TARGET, "🍼 DUMMY BLOCK: {}. Leader: {}", last_dummy_block, leader); + if candidate_block.parent() != last_dummy.id() { + warn!(target: LOG_TARGET, "❌ Bad proposal, unable to find dummy blocks (last dummy: {}) for candidate block {}", last_dummy, candidate_block); + return Err(ProposalValidationError::CandidateBlockDoesNotExtendJustify { + justify_block_height: justify_block.height(), + candidate_block_height: candidate_block.height(), + } + .into()); } // The logic for not checking is_safe is as follows: diff --git a/dan_layer/consensus/src/hotstuff/on_receive_new_view.rs b/dan_layer/consensus/src/hotstuff/on_receive_new_view.rs index cc630481b..22a3f1e1e 100644 --- a/dan_layer/consensus/src/hotstuff/on_receive_new_view.rs +++ b/dan_layer/consensus/src/hotstuff/on_receive_new_view.rs @@ -5,7 +5,7 @@ use std::collections::{HashMap, HashSet}; use log::*; use tari_common::configuration::Network; -use tari_dan_common_types::NodeHeight; +use tari_dan_common_types::{optional::Optional, NodeHeight}; use tari_dan_storage::{ consensus_models::{Block, BlockId, LeafBlock, LockedBlock, QuorumCertificate}, StateStore, @@ -115,19 +115,19 @@ where TConsensusSpec: ConsensusSpec .store .with_read_tx(|tx| Block::record_exists(tx, high_qc.block_id()))?; if !exists { - let leaf = self + let local_height = self .store .with_read_tx(|tx| LeafBlock::get(tx)) - // We need something for the returned error even if this query fails - .unwrap_or_else(|_| LeafBlock::genesis()); + .optional()? + .map(|leaf| leaf.height()) + .unwrap_or_default(); return Err(HotStuffError::FallenBehind { - local_height: leaf.height(), + local_height, qc_height: high_qc.block_height(), }); } let local_committee = self.epoch_manager.get_local_committee(epoch).await?; - let local_committee_shard = self.epoch_manager.get_local_committee_info(epoch).await?; let leader = self .leader_strategy .get_leader_for_next_block(&local_committee, new_height); @@ -195,7 +195,7 @@ where TConsensusSpec: ConsensusSpec let last_dummy_block = calculate_last_dummy_block( self.network, epoch, - local_committee_shard.shard(), + high_qc.shard(), &high_qc, *high_qc_block.merkle_root(), new_height, diff --git a/dan_layer/consensus/src/hotstuff/on_receive_request_missing_transactions.rs b/dan_layer/consensus/src/hotstuff/on_receive_request_missing_transactions.rs index 6c0fb98a4..2f3fe4518 100644 --- a/dan_layer/consensus/src/hotstuff/on_receive_request_missing_transactions.rs +++ b/dan_layer/consensus/src/hotstuff/on_receive_request_missing_transactions.rs @@ -32,10 +32,17 @@ where TConsensusSpec: ConsensusSpec from: TConsensusSpec::Addr, msg: RequestMissingTransactionsMessage, ) -> Result<(), HotStuffError> { - debug!(target: LOG_TARGET, "{} is requesting {} missing transactions from block {}", from, msg.transactions.len(), msg.block_id); - let (txs, _) = self + info!(target: LOG_TARGET, "{} requested {} missing transaction(s) from block {}", from, msg.transactions.len(), msg.block_id); + let (txs, missing) = self .store .with_read_tx(|tx| TransactionRecord::get_any(tx, &msg.transactions))?; + if !missing.is_empty() { + warn!( + target: LOG_TARGET, + "Some requested transaction(s) not found: {}", missing.iter().map(|t| t.to_string()).collect::>().join(", ") + ) + } + self.outbound_messaging .send( from, diff --git a/dan_layer/consensus/src/hotstuff/on_receive_vote.rs b/dan_layer/consensus/src/hotstuff/on_receive_vote.rs index c0a6acdd2..668f06dd2 100644 --- a/dan_layer/consensus/src/hotstuff/on_receive_vote.rs +++ b/dan_layer/consensus/src/hotstuff/on_receive_vote.rs @@ -1,13 +1,9 @@ // Copyright 2023 The Tari Project // SPDX-License-Identifier: BSD-3-Clause -use log::*; - use super::vote_receiver::VoteReceiver; use crate::{hotstuff::error::HotStuffError, messages::VoteMessage, traits::ConsensusSpec}; -const LOG_TARGET: &str = "tari::dan::consensus::hotstuff::on_receive_vote"; - pub struct OnReceiveVoteHandler { vote_receiver: VoteReceiver, } @@ -20,11 +16,6 @@ where TConsensusSpec: ConsensusSpec } pub async fn handle(&self, from: TConsensusSpec::Addr, message: VoteMessage) -> Result<(), HotStuffError> { - debug!( - target: LOG_TARGET, - "πŸ”₯ Receive VOTE for node {} from {}", message.block_id, message.signature.public_key, - ); - self.vote_receiver.handle(from, message, true).await } } diff --git a/dan_layer/consensus/src/hotstuff/on_sync_request.rs b/dan_layer/consensus/src/hotstuff/on_sync_request.rs index ec67ff1bd..098d5a5f2 100644 --- a/dan_layer/consensus/src/hotstuff/on_sync_request.rs +++ b/dan_layer/consensus/src/hotstuff/on_sync_request.rs @@ -17,8 +17,6 @@ use crate::{ const LOG_TARGET: &str = "tari::dan::consensus::hotstuff::on_sync_request"; -pub(super) const MAX_BLOCKS_PER_SYNC: usize = 100; - #[derive(Debug)] pub struct OnSyncRequest { store: TConsensusSpec::StateStore, @@ -39,7 +37,7 @@ impl OnSyncRequest { task::spawn(async move { let result = store.with_read_tx(|tx| { - let leaf_block = LeafBlock::get(tx)?; + let leaf_block = LeafBlock::get(tx)?.get_block(tx)?; if leaf_block.height() < msg.high_qc.block_height() { return Err(HotStuffError::InvalidSyncRequest { @@ -58,28 +56,42 @@ impl OnSyncRequest { msg.high_qc, leaf_block ); - let blocks = Block::get_all_blocks_between(tx, msg.high_qc.block_id(), leaf_block.block_id(), false)?; - - debug!( - target: LOG_TARGET, - "🌐 Sending {} blocks to {}", - blocks.len(), - from - ); + // NOTE: We have to send dummy blocks, because the messaging will ignore heights > current_view + 1, + // until eventually the syncing node's pacemaker leader-fails a few times. + let blocks = Block::get_all_blocks_between( + tx, + leaf_block.epoch(), + leaf_block.shard(), + msg.high_qc.block_id(), + leaf_block.id(), + true, + )?; Ok::<_, HotStuffError>(blocks) }); let blocks = match result { - Ok(blocks) => blocks, + Ok(mut blocks) => { + blocks.retain(|b| !b.is_genesis()); + blocks + }, Err(err) => { warn!(target: LOG_TARGET, "Failed to fetch blocks for sync request: {}", err); return; }, }; - for block in blocks.into_iter().take(MAX_BLOCKS_PER_SYNC) { - debug!( + info!( + target: LOG_TARGET, + "🌐 Sending {} block(s) ({} to {}) to {}", + blocks.len(), + blocks.first().map(|b| b.height()).unwrap_or_default(), + blocks.last().map(|b| b.height()).unwrap_or_default(), + from + ); + + for block in blocks { + info!( target: LOG_TARGET, "🌐 Sending block {} to {}", block, @@ -94,11 +106,11 @@ impl OnSyncRequest { } } - // Send last vote. TODO: This isn't quite + // Send last vote. let maybe_last_vote = match store.with_read_tx(|tx| LastSentVote::get(tx)).optional() { Ok(last_vote) => last_vote, Err(err) => { - warn!(target: LOG_TARGET, "Failed to fetch last vote for sync request: {}", err); + warn!(target: LOG_TARGET, "Failed to fetch last vote for catch-up request: {}", err); return; }, }; @@ -107,19 +119,9 @@ impl OnSyncRequest { .send(from.clone(), HotstuffMessage::Vote(last_vote.into())) .await { - warn!(target: LOG_TARGET, "Leader channel closed while sending LastVote {err}"); + warn!(target: LOG_TARGET, "Failed to send LastVote {err}"); } } - - // let _ignore = outbound_messaging - // .send(( - // from, - // HotstuffMessage::SyncResponse(SyncResponseMessage { - // epoch: msg.epoch, - // blocks, - // }), - // )) - // .await; }); } } diff --git a/dan_layer/consensus/src/hotstuff/pacemaker.rs b/dan_layer/consensus/src/hotstuff/pacemaker.rs index 2fac95b4b..2c119c278 100644 --- a/dan_layer/consensus/src/hotstuff/pacemaker.rs +++ b/dan_layer/consensus/src/hotstuff/pacemaker.rs @@ -10,7 +10,7 @@ use tari_dan_common_types::NodeHeight; use tokio::sync::mpsc; use crate::hotstuff::{ - current_height::CurrentHeight, + current_view::CurrentView, on_beat::OnBeat, on_force_beat::OnForceBeat, on_leader_timeout::OnLeaderTimeout, @@ -25,7 +25,7 @@ const BLOCK_TIME: Duration = Duration::from_secs(10); pub struct PaceMaker { pace_maker_handle: PaceMakerHandle, handle_receiver: mpsc::Receiver, - current_height: CurrentHeight, + current_view: CurrentView, current_high_qc_height: NodeHeight, } @@ -36,7 +36,7 @@ impl PaceMaker { let on_beat = OnBeat::new(); let on_force_beat = OnForceBeat::new(); let on_leader_timeout = OnLeaderTimeout::new(); - let current_height = CurrentHeight::new(); + let current_height = CurrentView::new(); Self { handle_receiver: receiver, @@ -47,7 +47,7 @@ impl PaceMaker { on_leader_timeout, current_height.clone(), ), - current_height, + current_view: current_height, current_high_qc_height: NodeHeight(0), } } @@ -94,21 +94,23 @@ impl PaceMaker { continue; } - self.current_high_qc_height = high_qc_height; + if let Some(height) = high_qc_height { + self.current_high_qc_height = height; + } let delta = self.delta_time(); - info!(target: LOG_TARGET, "Reset! Current height: {}, Delta: {:.2?}", self.current_height, delta); + info!(target: LOG_TARGET, "Reset! Current height: {}, Delta: {:.2?}", self.current_view, delta); leader_timeout.as_mut().reset(tokio::time::Instant::now() + delta); // set a timer for when we must send a block... block_timer.as_mut().reset(tokio::time::Instant::now() + BLOCK_TIME); }, PacemakerRequest::Start { high_qc_height } => { - info!(target: LOG_TARGET, "πŸš€ Starting pacemaker at leaf height {} and high QC: {}", self.current_height, high_qc_height); + info!(target: LOG_TARGET, "πŸš€ Starting pacemaker at leaf height {} and high QC: {}", self.current_view, high_qc_height); if started { continue; } self.current_high_qc_height = high_qc_height; let delta = self.delta_time(); - info!(target: LOG_TARGET, "Reset! Current height: {}, Delta: {:.2?}", self.current_height, delta); + info!(target: LOG_TARGET, "Reset! Current height: {}, Delta: {:.2?}", self.current_view, delta); leader_timeout.as_mut().reset(tokio::time::Instant::now() + delta); block_timer.as_mut().reset(tokio::time::Instant::now() + BLOCK_TIME); on_beat.beat(); @@ -136,9 +138,9 @@ impl PaceMaker { let delta = self.delta_time(); leader_timeout.as_mut().reset(tokio::time::Instant::now() + delta); - info!(target: LOG_TARGET, "⚠️ Leader timeout! Current height: {}, Delta: {:.2?}", self.current_height, delta); - self.current_height.next_height(); - on_leader_timeout.leader_timed_out(self.current_height.get()); + info!(target: LOG_TARGET, "⚠️ Leader timeout! Current view: {}, Delta: {:.2?}", self.current_view, delta); + self.current_view.set_next_height(); + on_leader_timeout.leader_timed_out(self.current_view.get_height()); }, } @@ -151,8 +153,8 @@ impl PaceMaker { /// high QC height. This is always greater than the block time. /// Ensure that current_height and current_high_qc_height are set before calling this function. fn delta_time(&self) -> Duration { - let current_height = self.current_height.get(); - if current_height.is_zero() { + let current_height = self.current_view.get_height(); + if current_height.is_zero() || self.current_high_qc_height.is_zero() { // Allow extra time for the first block return BLOCK_TIME * 2; } diff --git a/dan_layer/consensus/src/hotstuff/pacemaker_handle.rs b/dan_layer/consensus/src/hotstuff/pacemaker_handle.rs index 06c02044a..0f74047ef 100644 --- a/dan_layer/consensus/src/hotstuff/pacemaker_handle.rs +++ b/dan_layer/consensus/src/hotstuff/pacemaker_handle.rs @@ -1,12 +1,12 @@ // Copyright 2022 The Tari Project // SPDX-License-Identifier: BSD-3-Clause -use tari_dan_common_types::NodeHeight; +use tari_dan_common_types::{Epoch, NodeHeight}; use tari_dan_storage::consensus_models::LeafBlock; use tokio::sync::mpsc; use crate::hotstuff::{ - current_height::CurrentHeight, + current_view::CurrentView, on_beat::OnBeat, on_force_beat::OnForceBeat, on_leader_timeout::OnLeaderTimeout, @@ -14,7 +14,7 @@ use crate::hotstuff::{ }; pub enum PacemakerRequest { - ResetLeaderTimeout { high_qc_height: NodeHeight }, + ResetLeaderTimeout { high_qc_height: Option }, Start { high_qc_height: NodeHeight }, Stop, } @@ -25,7 +25,7 @@ pub struct PaceMakerHandle { on_beat: OnBeat, on_force_beat: OnForceBeat, on_leader_timeout: OnLeaderTimeout, - current_height: CurrentHeight, + current_view: CurrentView, } impl PaceMakerHandle { @@ -34,20 +34,25 @@ impl PaceMakerHandle { on_beat: OnBeat, on_force_beat: OnForceBeat, on_leader_timeout: OnLeaderTimeout, - current_height: CurrentHeight, + current_view: CurrentView, ) -> Self { Self { sender, on_beat, on_force_beat, on_leader_timeout, - current_height, + current_view, } } /// Start the pacemaker if it hasn't already been started. If it has, this is a no-op - pub async fn start(&self, current_height: NodeHeight, high_qc_height: NodeHeight) -> Result<(), HotStuffError> { - self.current_height.update(current_height); + pub async fn start( + &self, + current_epoch: Epoch, + current_view: NodeHeight, + high_qc_height: NodeHeight, + ) -> Result<(), HotStuffError> { + self.current_view.update(current_epoch, current_view); self.sender .send(PacemakerRequest::Start { high_qc_height }) .await @@ -76,6 +81,10 @@ impl PaceMakerHandle { self.on_beat.clone() } + pub fn on_beat(&self) { + self.on_beat.beat() + } + pub fn get_on_force_beat(&self) -> OnForceBeat { self.on_force_beat.clone() } @@ -84,21 +93,44 @@ impl PaceMakerHandle { self.on_leader_timeout.clone() } + async fn reset_leader_timeout(&self, high_qc_height: Option) -> Result<(), HotStuffError> { + self.sender + .send(PacemakerRequest::ResetLeaderTimeout { high_qc_height }) + .await + .map_err(|e| HotStuffError::PacemakerChannelDropped { details: e.to_string() }) + } + /// Reset the leader timeout. This should be called when a valid leader proposal is received. pub async fn update_view( &self, + epoch: Epoch, last_seen_height: NodeHeight, high_qc_height: NodeHeight, ) -> Result<(), HotStuffError> { // Update current height here to prevent possibility of race conditions - self.current_height.update(last_seen_height); - self.sender - .send(PacemakerRequest::ResetLeaderTimeout { high_qc_height }) - .await - .map_err(|e| HotStuffError::PacemakerChannelDropped { details: e.to_string() }) + self.current_view.update(epoch, last_seen_height); + self.reset_leader_timeout(Some(high_qc_height)).await + } + + /// Reset the leader timeout. This should be called when a valid leader proposal is received. + pub async fn reset_view( + &self, + epoch: Epoch, + last_seen_height: NodeHeight, + high_qc_height: NodeHeight, + ) -> Result<(), HotStuffError> { + // Update current height here to prevent possibility of race conditions + self.current_view.reset(epoch, last_seen_height); + self.reset_leader_timeout(Some(high_qc_height)).await + } + + /// Reset the leader timeout. This should be called when an end of epoch proposal has been committed. + pub async fn set_epoch(&self, epoch: Epoch) -> Result<(), HotStuffError> { + self.current_view.reset(epoch, NodeHeight::zero()); + self.reset_leader_timeout(Some(NodeHeight::zero())).await } - pub fn current_height(&self) -> NodeHeight { - self.current_height.get() + pub fn current_view(&self) -> &CurrentView { + &self.current_view } } diff --git a/dan_layer/consensus/src/hotstuff/proposer.rs b/dan_layer/consensus/src/hotstuff/proposer.rs index b9c5a5bfc..1eb446ee1 100644 --- a/dan_layer/consensus/src/hotstuff/proposer.rs +++ b/dan_layer/consensus/src/hotstuff/proposer.rs @@ -73,7 +73,7 @@ where TConsensusSpec: ConsensusSpec .await?; info!( target: LOG_TARGET, - "🌿 Broadcasting new locked block {} to {} foreign committees.", + "🌿 FOREIGN PROPOSE: Broadcasting locked block {} to {} foreign committees.", block, non_local_committees.len(), ); diff --git a/dan_layer/consensus/src/hotstuff/state_machine/idle.rs b/dan_layer/consensus/src/hotstuff/state_machine/idle.rs index b32abe1f0..59a605b74 100644 --- a/dan_layer/consensus/src/hotstuff/state_machine/idle.rs +++ b/dan_layer/consensus/src/hotstuff/state_machine/idle.rs @@ -34,6 +34,7 @@ where TSpec: ConsensusSpec ) -> Result { // Subscribe before checking if we're registered to eliminate the chance that we miss the epoch event let mut epoch_events = context.epoch_manager.subscribe().await?; + context.epoch_manager.wait_for_initial_scanning_to_complete().await?; let current_epoch = context.epoch_manager.current_epoch().await?; if self.is_registered_for_epoch(context, current_epoch).await? { return Ok(ConsensusStateEvent::RegisteredForEpoch { epoch: current_epoch }); @@ -48,8 +49,8 @@ where TSpec: ConsensusSpec return Ok(event); } }, - Err(broadcast::error::RecvError::Lagged(_)) => { - debug!(target: LOG_TARGET, "Idle state lagged behind epoch manager event stream"); + Err(broadcast::error::RecvError::Lagged(n)) => { + debug!(target: LOG_TARGET, "Idle state lagged behind by {n} epoch manager events"); }, Err(broadcast::error::RecvError::Closed) => { break; diff --git a/dan_layer/consensus/src/hotstuff/state_machine/running.rs b/dan_layer/consensus/src/hotstuff/state_machine/running.rs index f8f4269ff..d979118d5 100644 --- a/dan_layer/consensus/src/hotstuff/state_machine/running.rs +++ b/dan_layer/consensus/src/hotstuff/state_machine/running.rs @@ -12,7 +12,6 @@ use crate::{ worker::ConsensusWorkerContext, }, HotStuffError, - ProposalValidationError, }, traits::ConsensusSpec, }; @@ -40,7 +39,6 @@ where TSpec: ConsensusSpec info!(target: LOG_TARGET, "Not registered for current epoch ({err})"); Ok(ConsensusStateEvent::NotRegisteredForEpoch { epoch }) }, - Err(err @ HotStuffError::ProposalValidationError(ProposalValidationError::JustifyBlockNotFound { .. })) | Err(err @ HotStuffError::FallenBehind { .. }) => { info!(target: LOG_TARGET, "Behind peers, starting sync ({err})"); Ok(ConsensusStateEvent::NeedSync) diff --git a/dan_layer/consensus/src/hotstuff/state_machine/worker.rs b/dan_layer/consensus/src/hotstuff/state_machine/worker.rs index 5065d9e63..4c9bfaa7b 100644 --- a/dan_layer/consensus/src/hotstuff/state_machine/worker.rs +++ b/dan_layer/consensus/src/hotstuff/state_machine/worker.rs @@ -113,6 +113,8 @@ where } pub async fn run(&mut self, mut context: ConsensusWorkerContext) { + // When starting up we will wait a bit. + // Context: in swarm, we start on epoch 2, then quickly go to epoch 3. let mut state = ConsensusState::Idle(Idle::new()); loop { let next_event = self.next_event(&mut context, &state).await; diff --git a/dan_layer/consensus/src/hotstuff/substate_store/chain_scoped_tree_store.rs b/dan_layer/consensus/src/hotstuff/substate_store/chain_scoped_tree_store.rs new file mode 100644 index 000000000..428965160 --- /dev/null +++ b/dan_layer/consensus/src/hotstuff/substate_store/chain_scoped_tree_store.rs @@ -0,0 +1,66 @@ +// Copyright 2024 The Tari Project +// SPDX-License-Identifier: BSD-3-Clause + +use std::ops::Deref; + +use tari_dan_common_types::{optional::Optional, shard::Shard, Epoch}; +use tari_dan_storage::{StateStoreReadTransaction, StateStoreWriteTransaction}; +use tari_state_tree::{Node, NodeKey, StaleTreeNode, TreeStoreReader, TreeStoreWriter, Version}; + +/// Tree store that is scoped to a specific chain (epoch and shard) +#[derive(Debug)] +pub struct ChainScopedTreeStore { + epoch: Epoch, + shard: Shard, + tx: TTx, +} + +impl ChainScopedTreeStore { + pub fn new(epoch: Epoch, shard: Shard, tx: TTx) -> Self { + Self { epoch, shard, tx } + } +} + +impl ChainScopedTreeStore { + pub fn transaction(&self) -> TTx { + self.tx.clone() + } +} + +impl<'a, TTx: StateStoreReadTransaction> TreeStoreReader for ChainScopedTreeStore<&'a TTx> { + fn get_node(&self, key: &NodeKey) -> Result, tari_state_tree::JmtStorageError> { + self.tx + .state_tree_nodes_get(self.epoch, self.shard, key) + .optional() + .map_err(|e| tari_state_tree::JmtStorageError::UnexpectedError(e.to_string()))? + .ok_or_else(|| tari_state_tree::JmtStorageError::NotFound(key.clone())) + } +} + +impl<'a, TTx> TreeStoreReader for ChainScopedTreeStore<&'a mut TTx> +where + TTx: StateStoreWriteTransaction + Deref, + TTx::Target: StateStoreReadTransaction, +{ + fn get_node(&self, key: &NodeKey) -> Result, tari_state_tree::JmtStorageError> { + self.tx + .state_tree_nodes_get(self.epoch, self.shard, key) + .optional() + .map_err(|e| tari_state_tree::JmtStorageError::UnexpectedError(e.to_string()))? + .ok_or_else(|| tari_state_tree::JmtStorageError::NotFound(key.clone())) + } +} + +impl<'a, TTx: StateStoreWriteTransaction> TreeStoreWriter for ChainScopedTreeStore<&'a mut TTx> { + fn insert_node(&mut self, key: NodeKey, node: Node) -> Result<(), tari_state_tree::JmtStorageError> { + self.tx + .state_tree_nodes_insert(self.epoch, self.shard, key, node) + .map_err(|e| tari_state_tree::JmtStorageError::UnexpectedError(e.to_string())) + } + + fn record_stale_tree_node(&mut self, node: StaleTreeNode) -> Result<(), tari_state_tree::JmtStorageError> { + self.tx + .state_tree_nodes_mark_stale_tree_node(self.epoch, self.shard, node) + .map_err(|e| tari_state_tree::JmtStorageError::UnexpectedError(e.to_string())) + } +} diff --git a/dan_layer/consensus/src/hotstuff/substate_store/mod.rs b/dan_layer/consensus/src/hotstuff/substate_store/mod.rs index 88ba7d19c..6db8650a1 100644 --- a/dan_layer/consensus/src/hotstuff/substate_store/mod.rs +++ b/dan_layer/consensus/src/hotstuff/substate_store/mod.rs @@ -1,8 +1,10 @@ // Copyright 2024 The Tari Project // SPDX-License-Identifier: BSD-3-Clause +mod chain_scoped_tree_store; mod error; mod pending_store; +pub use chain_scoped_tree_store::*; pub use error::*; pub use pending_store::*; diff --git a/dan_layer/consensus/src/hotstuff/substate_store/pending_store.rs b/dan_layer/consensus/src/hotstuff/substate_store/pending_store.rs index cf18bf16b..ecf97e815 100644 --- a/dan_layer/consensus/src/hotstuff/substate_store/pending_store.rs +++ b/dan_layer/consensus/src/hotstuff/substate_store/pending_store.rs @@ -26,14 +26,14 @@ use tari_transaction::{TransactionId, VersionedSubstateId}; use super::error::SubstateStoreError; use crate::{ - hotstuff::calculate_state_merkle_diff, + hotstuff::{calculate_state_merkle_diff, substate_store::chain_scoped_tree_store::ChainScopedTreeStore}, traits::{ReadableSubstateStore, WriteableSubstateStore}, }; const LOG_TARGET: &str = "tari::dan::hotstuff::substate_store::pending_store"; pub struct PendingSubstateStore<'a, 'tx, TStore: StateStore + 'a + 'tx> { - store: &'a TStore::ReadTransaction<'tx>, + store: ChainScopedTreeStore<&'a TStore::ReadTransaction<'tx>>, /// Map from substate address to the index in the diff list pending: HashMap, /// Append only list of changes ordered oldest to newest @@ -42,9 +42,9 @@ pub struct PendingSubstateStore<'a, 'tx, TStore: StateStore + 'a + 'tx> { } impl<'a, 'tx, TStore: StateStore + 'a> PendingSubstateStore<'a, 'tx, TStore> { - pub fn new(tx: &'a TStore::ReadTransaction<'tx>) -> Self { + pub fn new(store: ChainScopedTreeStore<&'a TStore::ReadTransaction<'tx>>) -> Self { Self { - store: tx, + store, pending: HashMap::new(), diff: Vec::new(), new_locks: IndexMap::new(), @@ -52,7 +52,7 @@ impl<'a, 'tx, TStore: StateStore + 'a> PendingSubstateStore<'a, 'tx, TStore> { } pub fn read_transaction(&self) -> &'a TStore::ReadTransaction<'tx> { - self.store + self.store.transaction() } } @@ -66,7 +66,7 @@ impl<'a, 'tx, TStore: StateStore + 'a + 'tx> ReadableSubstateStore for PendingSu }); } - let Some(substate) = SubstateRecord::get(self.store, key).optional()? else { + let Some(substate) = SubstateRecord::get(self.read_transaction(), key).optional()? else { return Err(SubstateStoreError::SubstateNotFound { address: *key }); }; Ok(substate.into_substate()) @@ -109,7 +109,7 @@ impl<'a, 'tx, TStore: StateStore + 'a + 'tx> PendingSubstateStore<'a, 'tx, TStor return Ok(substate.clone()); } - let substate = SubstateRecord::get_latest(self.store, id)?; + let substate = SubstateRecord::get_latest(self.read_transaction(), id)?; Ok(substate.into_substate()) } @@ -120,7 +120,12 @@ impl<'a, 'tx, TStore: StateStore + 'a + 'tx> PendingSubstateStore<'a, 'tx, TStor let current_version = block.justify().block_height().as_u64(); let next_version = block.height().as_u64(); - let pending = PendingStateTreeDiff::get_all_up_to_commit_block(self.store, block.justify().block_id())?; + let pending = PendingStateTreeDiff::get_all_up_to_commit_block( + self.read_transaction(), + block.epoch(), + block.shard(), + block.justify().block_id(), + )?; let changes = self.diff.iter().map(|ch| match ch { SubstateChange::Up { id, substate, .. } => SubstateTreeChange::Up { @@ -131,9 +136,8 @@ impl<'a, 'tx, TStore: StateStore + 'a + 'tx> PendingSubstateStore<'a, 'tx, TStor id: id.substate_id.clone(), }, }); - let (state_root, state_tree_diff) = - calculate_state_merkle_diff(self.store, current_version, next_version, pending, changes)?; + calculate_state_merkle_diff(&self.store, current_version, next_version, pending, changes)?; Ok((state_root, state_tree_diff)) } @@ -329,7 +333,10 @@ impl<'a, 'tx, TStore: StateStore + 'a + 'tx> PendingSubstateStore<'a, 'tx, TStor return Ok(Some(Cow::Borrowed(lock))); } - let maybe_lock = self.store.substate_locks_get_latest_for_substate(id).optional()?; + let maybe_lock = self + .read_transaction() + .substate_locks_get_latest_for_substate(id) + .optional()?; Ok(maybe_lock.map(Cow::Owned)) } @@ -346,7 +353,7 @@ impl<'a, 'tx, TStore: StateStore + 'a + 'tx> PendingSubstateStore<'a, 'tx, TStor return Ok(()); } - let is_up = SubstateRecord::substate_is_up(self.store, &address) + let is_up = SubstateRecord::substate_is_up(self.read_transaction(), &address) .optional()? .unwrap_or(false); if !is_up { @@ -365,7 +372,7 @@ impl<'a, 'tx, TStore: StateStore + 'a + 'tx> PendingSubstateStore<'a, 'tx, TStor return Ok(()); } - let Some(is_up) = SubstateRecord::substate_is_up(self.store, &address).optional()? else { + let Some(is_up) = SubstateRecord::substate_is_up(self.read_transaction(), &address).optional()? else { debug!(target: LOG_TARGET, "Expected substate {} to be DOWN but it does not exist", address); return Err(SubstateStoreError::SubstateNotFound { address }); }; @@ -385,7 +392,7 @@ impl<'a, 'tx, TStore: StateStore + 'a + 'tx> PendingSubstateStore<'a, 'tx, TStor return Ok(()); } - if SubstateRecord::exists(self.store, &address)? { + if SubstateRecord::exists(self.read_transaction(), &address)? { return Err(SubstateStoreError::ExpectedSubstateNotExist { id: id.clone() }); } diff --git a/dan_layer/consensus/src/hotstuff/vote_receiver.rs b/dan_layer/consensus/src/hotstuff/vote_receiver.rs index 084e752da..c89c3b474 100644 --- a/dan_layer/consensus/src/hotstuff/vote_receiver.rs +++ b/dan_layer/consensus/src/hotstuff/vote_receiver.rs @@ -78,44 +78,41 @@ where TConsensusSpec: ConsensusSpec message: VoteMessage, check_leadership: bool, ) -> Result { - // Is a committee member sending us this vote? - let committee = self.epoch_manager.get_local_committee(message.epoch).await?; - if !committee.contains(&from) { - return Err(HotStuffError::ReceivedMessageFromNonCommitteeMember { + debug!( + target: LOG_TARGET, + "πŸ“¬ Validating vote message from {from}: {message}" + ); + // Is a local committee member that signed this vote? + let sender_vn = self + .epoch_manager + .get_validator_node_by_public_key(message.epoch, &message.signature.public_key) + .await + .optional()?; + let Some(sender_vn) = sender_vn else { + return Err(HotStuffError::ReceivedVoteFromNonCommitteeMember { epoch: message.epoch, sender: from.to_string(), - context: "OnReceiveVote".to_string(), + context: "VoteReceiver::handle_vote (sender pk not from registered VN)".to_string(), }); - } - - // Are we the leader for the block being voted for? - let our_vn = self.epoch_manager.get_our_validator_node(message.epoch).await?; - - let local_committee_shard = self.epoch_manager.get_local_committee_info(message.epoch).await?; + }; // Get the sender shard, and check that they are in the local committee - let sender_vn = self.epoch_manager.get_validator_node(message.epoch, &from).await?; - if message.signature.public_key != sender_vn.public_key { - return Err(HotStuffError::RejectingVoteNotSentBySigner { - address: from.to_string(), - signer_public_key: message.signature.public_key.to_string(), - }); - } - - if !local_committee_shard.includes_substate_address(&sender_vn.shard_key) { - return Err(HotStuffError::ReceivedMessageFromNonCommitteeMember { + let our_vn = self.epoch_manager.get_our_validator_node(message.epoch).await?; + let committee = self + .epoch_manager + .get_committee_for_substate(message.epoch, our_vn.shard_key) + .await?; + if !committee.contains(&sender_vn.address) { + return Err(HotStuffError::ReceivedVoteFromNonCommitteeMember { epoch: message.epoch, - sender: message.signature.public_key.to_string(), - context: "OnReceiveVote".to_string(), + sender: sender_vn.address.to_string(), + context: "VoteReceiver::handle_vote (VN not in local committee)".to_string(), }); } let sender_leaf_hash = sender_vn.get_node_hash(self.network); - self.validate_vote_message(&message, &sender_leaf_hash)?; - let from = message.signature.public_key.clone(); - let count = self.store.with_write_tx(|tx| { Vote { epoch: message.epoch, @@ -130,12 +127,15 @@ where TConsensusSpec: ConsensusSpec Ok::<_, HotStuffError>(count) })?; + let local_committee_shard = self.epoch_manager.get_local_committee_info(message.epoch).await?; + // We only generate the next high qc once when we have a quorum of votes. Any subsequent votes are not included // in the QC. info!( target: LOG_TARGET, - "πŸ”₯ Received vote for block #{} {} from {} ({} of {})", + "πŸ”₯ Received vote for block {} {} {} from {} ({} of {})", + message.epoch, message.block_height, message.block_id, from, @@ -157,6 +157,7 @@ where TConsensusSpec: ConsensusSpec return Ok(false); }; + // Are we the leader for the block being voted for? if check_leadership && !self .leader_strategy @@ -228,7 +229,9 @@ where TConsensusSpec: ConsensusSpec info!(target: LOG_TARGET, "πŸ”₯ New QC {}", qc); let high_qc = self.store.with_write_tx(|tx| qc.update_high_qc(tx))?; - self.pacemaker.update_view(block_height, high_qc.block_height).await?; + self.pacemaker + .update_view(message.epoch, block_height, high_qc.block_height) + .await?; Ok(true) } @@ -255,6 +258,17 @@ where TConsensusSpec: ConsensusSpec } fn validate_vote_message(&self, message: &VoteMessage, sender_leaf_hash: &FixedHash) -> Result<(), HotStuffError> { + let current_epoch = self.pacemaker.current_view().get_epoch(); + if current_epoch != message.epoch { + return Err(HotStuffError::InvalidVote { + signer_public_key: message.signature.public_key.to_string(), + details: format!( + "Our current view is at epoch {} but the vote was for epoch {}", + current_epoch, message.epoch + ), + }); + } + if !self.vote_signature_service.verify( &message.signature, sender_leaf_hash, diff --git a/dan_layer/consensus/src/hotstuff/worker.rs b/dan_layer/consensus/src/hotstuff/worker.rs index e4c3a112d..38405af44 100644 --- a/dan_layer/consensus/src/hotstuff/worker.rs +++ b/dan_layer/consensus/src/hotstuff/worker.rs @@ -1,23 +1,18 @@ // Copyright 2023 The Tari Project // SPDX-License-Identifier: BSD-3-Clause -use std::{ - cmp, - fmt::{Debug, Formatter}, -}; +use std::fmt::{Debug, Formatter}; use log::*; use tari_common::configuration::Network; -use tari_dan_common_types::NodeHeight; +use tari_dan_common_types::{shard::Shard, Epoch, NodeHeight}; use tari_dan_storage::{ consensus_models::{ Block, BlockDiff, ExecutedTransaction, HighQc, - LastVoted, LeafBlock, - LockedBlock, TransactionAtom, TransactionPool, TransactionRecord, @@ -33,12 +28,15 @@ use super::{ config::HotstuffConfig, on_receive_requested_transactions::OnReceiveRequestedTransactions, proposer::Proposer, + ProposalValidationError, }; use crate::{ hotstuff::{ error::HotStuffError, event::HotstuffEvent, - on_inbound_message::{IncomingMessageResult, NeedsSync, OnInboundMessage}, + on_catch_up_sync::OnCatchUpSync, + on_inbound_message::OnInboundMessage, + on_message_validate::{MessageValidationResult, OnMessageValidate}, on_next_sync_view::OnNextSyncViewHandler, on_propose::OnPropose, on_receive_foreign_proposal::OnReceiveForeignProposalHandler, @@ -46,13 +44,13 @@ use crate::{ on_receive_new_view::OnReceiveNewViewHandler, on_receive_request_missing_transactions::OnReceiveRequestMissingTransactions, on_receive_vote::OnReceiveVoteHandler, - on_sync_request::{OnSyncRequest, MAX_BLOCKS_PER_SYNC}, + on_sync_request::OnSyncRequest, pacemaker::PaceMaker, pacemaker_handle::PaceMakerHandle, vote_receiver::VoteReceiver, }, - messages::{HotstuffMessage, SyncRequestMessage}, - traits::{hooks::ConsensusHooks, ConsensusSpec, InboundMessaging, LeaderStrategy, OutboundMessaging}, + messages::HotstuffMessage, + traits::{hooks::ConsensusHooks, ConsensusSpec, LeaderStrategy}, }; const LOG_TARGET: &str = "tari::dan::consensus::hotstuff::worker"; @@ -63,8 +61,6 @@ pub struct HotstuffWorker { hooks: TConsensusSpec::Hooks, tx_events: broadcast::Sender, - outbound_messaging: TConsensusSpec::OutboundMessaging, - inbound_messaging: TConsensusSpec::InboundMessaging, rx_new_transactions: mpsc::Receiver<(TransactionId, usize)>, on_inbound_message: OnInboundMessage, @@ -75,8 +71,10 @@ pub struct HotstuffWorker { on_receive_new_view: OnReceiveNewViewHandler, on_receive_request_missing_txs: OnReceiveRequestMissingTransactions, on_receive_requested_txs: OnReceiveRequestedTransactions, + on_message_validate: OnMessageValidate, on_propose: OnPropose, on_sync_request: OnSyncRequest, + on_catch_up_sync: OnCatchUpSync, state_store: TConsensusSpec::StateStore, leader_strategy: TConsensusSpec::LeaderStrategy, @@ -122,11 +120,11 @@ impl HotstuffWorker { validator_addr: validator_addr.clone(), network, tx_events: tx_events.clone(), - outbound_messaging: outbound_messaging.clone(), - inbound_messaging, rx_new_transactions, - on_inbound_message: OnInboundMessage::new( + on_inbound_message: OnInboundMessage::new(inbound_messaging, hooks.clone()), + on_message_validate: OnMessageValidate::new( + validator_addr.clone(), network, config, state_store.clone(), @@ -189,7 +187,8 @@ impl HotstuffWorker { outbound_messaging.clone(), ), - on_sync_request: OnSyncRequest::new(state_store.clone(), outbound_messaging), + on_sync_request: OnSyncRequest::new(state_store.clone(), outbound_messaging.clone()), + on_catch_up_sync: OnCatchUpSync::new(state_store.clone(), pacemaker.clone_handle(), outbound_messaging), state_store, leader_strategy, @@ -203,21 +202,38 @@ impl HotstuffWorker { } } + pub fn pacemaker(&self) -> &PaceMakerHandle { + &self.pacemaker + } + pub async fn start(&mut self) -> Result<(), HotStuffError> { - self.create_zero_block_if_required()?; - let (current_height, high_qc) = self.state_store.with_read_tx(|tx| { + let current_epoch = self.epoch_manager.current_epoch().await?; + let committee_info = self.epoch_manager.get_local_committee_info(current_epoch).await?; + + self.create_zero_block_if_required(current_epoch, committee_info.shard())?; + + // Resume pacemaker from the last epoch/height + let (current_epoch, current_height, high_qc) = self.state_store.with_read_tx(|tx| { let leaf = LeafBlock::get(tx)?; - let last_voted = LastVoted::get(tx)?; - Ok::<_, HotStuffError>((cmp::max(leaf.height(), last_voted.height()), HighQc::get(tx)?)) + let current_epoch = Some(leaf.epoch()).filter(|e| !e.is_zero()).unwrap_or(current_epoch); + let current_height = Some(leaf.height()) + .filter(|h| !h.is_zero()) + .unwrap_or_else(NodeHeight::zero); + + Ok::<_, HotStuffError>((current_epoch, current_height, HighQc::get(tx)?)) })?; + info!( target: LOG_TARGET, - "πŸš€ Pacemaker starting leaf_block: {}, high_qc: {}", + "πŸš€ Pacemaker starting for epoch {}, height: {}, high_qc: {}", + current_epoch, current_height, high_qc ); - self.pacemaker.start(current_height, high_qc.block_height()).await?; + self.pacemaker + .start(current_epoch, current_height, high_qc.block_height()) + .await?; self.run().await?; Ok(()) @@ -235,11 +251,13 @@ impl HotstuffWorker { let mut epoch_manager_events = self.epoch_manager.subscribe().await?; - self.request_initial_catch_up_sync().await?; + let mut prev_height = self.pacemaker.current_view().get_height(); + let current_epoch = self.pacemaker.current_view().get_epoch(); + self.request_initial_catch_up_sync(current_epoch).await?; - let mut prev_height = self.pacemaker.current_height(); loop { - let current_height = self.pacemaker.current_height() + NodeHeight(1); + let current_height = self.pacemaker.current_view().get_height() + NodeHeight(1); + let current_epoch = self.pacemaker.current_view().get_epoch(); if current_height != prev_height { self.hooks.on_pacemaker_height_changed(current_height); @@ -253,35 +271,26 @@ impl HotstuffWorker { ); tokio::select! { - Some(result) = self.inbound_messaging.next_message() => { - let (from, msg) = result?; - self.hooks.on_message_received(&msg); - if let Err(err) = self.on_inbound_message.handle(current_height, from, msg).await { + Some(result) = self.on_inbound_message.next_message(current_epoch, current_height) => { + if let Err(err) = self.on_unvalidated_message(current_height, result).await { self.hooks.on_error(&err); - error!(target: LOG_TARGET, "Error handling message: {}", err); - } - }, - - msg_or_sync = self.on_inbound_message.next_message(current_height) => { - if let Err(e) = self.dispatch_hotstuff_message(msg_or_sync).await { - self.on_failure("on_new_hs_message", &e).await; - return Err(e); + error!(target: LOG_TARGET, "🚨Error handling new message: {}", err); } }, Some((tx_id, pending)) = self.rx_new_transactions.recv() => { if let Err(err) = self.on_new_transaction(tx_id, pending, current_height).await { self.hooks.on_error(&err); - error!(target: LOG_TARGET, "Error handling new transaction: {}", err); + error!(target: LOG_TARGET, "🚨Error handling new transaction: {}", err); } }, Ok(event) = epoch_manager_events.recv() => { - self.handle_epoch_manager_event(event).await?; + self.on_epoch_manager_event(event).await?; }, _ = on_beat.wait() => { - if let Err(e) = self.on_beat().await { + if let Err(e) = self.on_beat(current_epoch).await { self.on_failure("on_beat", &e).await; return Err(e); } @@ -289,14 +298,14 @@ impl HotstuffWorker { maybe_leaf_block = on_force_beat.wait() => { self.hooks.on_beat(); - if let Err(e) = self.propose_if_leader(maybe_leaf_block).await { + if let Err(e) = self.propose_if_leader(current_epoch, maybe_leaf_block).await { self.on_failure("propose_if_leader", &e).await; return Err(e); } }, new_height = on_leader_timeout.wait() => { - if let Err(e) = self.on_leader_timeout(new_height).await { + if let Err(e) = self.on_leader_timeout(current_epoch, new_height).await { self.on_failure("on_leader_timeout", &e).await; return Err(e); } @@ -319,6 +328,27 @@ impl HotstuffWorker { Ok(()) } + async fn on_unvalidated_message( + &mut self, + current_height: NodeHeight, + result: Result<(TConsensusSpec::Addr, HotstuffMessage), HotStuffError>, + ) -> Result<(), HotStuffError> { + let (from, msg) = result?; + + match self.on_message_validate.handle(current_height, from, msg).await? { + MessageValidationResult::Ready { from, message: msg } => { + if let Err(e) = self.dispatch_hotstuff_message(from, msg).await { + self.on_failure("on_unvalidated_message -> dispatch_hotstuff_message", &e) + .await; + return Err(e); + } + Ok(()) + }, + MessageValidationResult::NotReady | MessageValidationResult::Discard => Ok(()), + MessageValidationResult::Invalid { err, .. } => Err(err), + } + } + async fn on_new_transaction( &mut self, tx_id: TransactionId, @@ -327,9 +357,15 @@ impl HotstuffWorker { ) -> Result<(), HotStuffError> { let exists = self.state_store.with_write_tx(|tx| { if self.transaction_pool.exists(&**tx, &tx_id)? { - return Ok(true); + return Ok(Some(true)); } let transaction = TransactionRecord::get(&**tx, &tx_id)?; + if transaction.is_finalized() { + warn!( + target: LOG_TARGET, "Transaction {} is already finalized. Consensus will ignore it.", transaction.id() + ); + return Ok(None); + } // Did the mempool execute it? if transaction.is_executed() { // This should never fail @@ -344,9 +380,13 @@ impl HotstuffWorker { self.transaction_pool .insert(tx, TransactionAtom::deferred(*transaction.id()))?; } - Ok::<_, HotStuffError>(false) + Ok::<_, HotStuffError>(Some(false)) })?; + let Some(exists) = exists else { + return Ok(()); + }; + debug!( target: LOG_TARGET, "πŸ”₯ new transaction ready for consensus: {} ({} pending, already exists = {})", @@ -359,14 +399,18 @@ impl HotstuffWorker { self.hooks.on_transaction_ready(&tx_id); } - if let Err(err) = self - .on_inbound_message + if let Some((from, msg)) = self + .on_message_validate .update_parked_blocks(current_height, &tx_id) - .await + .await? { - self.hooks.on_error(&err); - error!(target: LOG_TARGET, "Error checking parked blocks: {}", err); + if let Err(e) = self.dispatch_hotstuff_message(from, msg).await { + self.on_failure("on_new_transaction -> dispatch_hotstuff_message", &e) + .await; + return Err(e); + } } + // There are num_pending_txs transactions in the queue. If we have no pending transactions, we'll propose now if // able. if !exists && num_pending_txs == 0 { @@ -376,7 +420,7 @@ impl HotstuffWorker { Ok(()) } - async fn handle_epoch_manager_event(&mut self, event: EpochManagerEvent) -> Result<(), HotStuffError> { + async fn on_epoch_manager_event(&mut self, event: EpochManagerEvent) -> Result<(), HotStuffError> { match event { EpochManagerEvent::EpochChanged(epoch) => { if !self.epoch_manager.is_this_validator_registered_for_epoch(epoch).await? { @@ -387,23 +431,16 @@ impl HotstuffWorker { return Err(HotStuffError::NotRegisteredForCurrentEpoch { epoch }); } - // TODO: This is breaking my testing right now (division by zero, from time to time) - // Send the last vote to the leader at the next epoch so that they can justify the current tip. - // if let Some(last_voted) = self.state_store.with_read_tx(|tx| LastSentVote::get(tx)).optional()? { - // info!( - // target: LOG_TARGET, - // "πŸ’Œ Sending last vote to the leader at epoch {}: {}", - // epoch, - // last_voted - // ); - // let local_committee = self.epoch_manager.get_local_committee(epoch).await?; - // let leader = self - // .leader_strategy - // .get_leader_for_next_block(&local_committee, last_voted.block_height); - // self.outbound_messaging - // .send(leader.clone(), HotstuffMessage::Vote(last_voted.into())) - // .await?; + // Edge case: we have started a VN and have progressed a few epochs quickly and have no blocks in + // previous epochs to update the current view. This only really applies when mining is + // instant (localnet) + // let leaf_block = self.state_store.with_read_tx(|tx| LeafBlock::get(tx))?; + // if leaf_block.block_id.is_zero() { + // self.pacemaker.set_epoch(epoch).await?; // } + + // If we can propose a block end, let's not wait for the block time to do it + self.pacemaker.beat(); }, EpochManagerEvent::ThisValidatorIsRegistered { .. } => {}, } @@ -411,12 +448,11 @@ impl HotstuffWorker { Ok(()) } - async fn request_initial_catch_up_sync(&mut self) -> Result<(), HotStuffError> { - let current_epoch = self.epoch_manager.current_epoch().await?; + async fn request_initial_catch_up_sync(&mut self, current_epoch: Epoch) -> Result<(), HotStuffError> { let committee = self.epoch_manager.get_local_committee(current_epoch).await?; for member in committee.shuffled() { if *member != self.validator_addr { - self.on_catch_up_sync(member).await?; + self.on_catch_up_sync.request_sync(current_epoch, member).await?; break; } } @@ -445,49 +481,44 @@ impl HotstuffWorker { break; }, _ = self.on_inbound_message.discard() => {}, - _ = self.inbound_messaging.next_message() => {}, _ = self.rx_new_transactions.recv() => {} } } } - async fn on_leader_timeout(&mut self, new_height: NodeHeight) -> Result<(), HotStuffError> { + async fn on_leader_timeout(&mut self, current_epoch: Epoch, new_height: NodeHeight) -> Result<(), HotStuffError> { self.hooks.on_leader_timeout(new_height); - self.on_next_sync_view.handle(new_height).await?; + self.on_next_sync_view.handle(current_epoch, new_height).await?; self.publish_event(HotstuffEvent::LeaderTimeout { new_height }); Ok(()) } - async fn on_beat(&mut self) -> Result<(), HotStuffError> { + async fn on_beat(&mut self, epoch: Epoch) -> Result<(), HotStuffError> { self.hooks.on_beat(); if !self .state_store .with_read_tx(|tx| self.transaction_pool.has_uncommitted_transactions(tx))? { - debug!(target: LOG_TARGET, "[on_beat] No transactions to propose. Waiting for a timeout."); - return Ok(()); + let current_epoch = self.epoch_manager.current_epoch().await?; + // Propose quickly if we should end the epoch (i.e base layer epoch > pacemaker epoch) + if current_epoch == epoch { + debug!(target: LOG_TARGET, "[on_beat] No transactions to propose. Waiting for a timeout."); + return Ok(()); + } } - self.propose_if_leader(None).await?; + self.propose_if_leader(epoch, None).await?; Ok(()) } - async fn propose_if_leader(&mut self, leaf_block: Option) -> Result<(), HotStuffError> { + async fn propose_if_leader(&mut self, epoch: Epoch, leaf_block: Option) -> Result<(), HotStuffError> { let is_newview_propose = leaf_block.is_some(); let leaf_block = match leaf_block { Some(leaf_block) => leaf_block, None => self.state_store.with_read_tx(|tx| LeafBlock::get(tx))?, }; - let locked_block = self - .state_store - .with_read_tx(|tx| LockedBlock::get(tx)?.get_block(tx))?; - let current_epoch = self.epoch_manager.current_epoch().await?; - let epoch = if locked_block.is_epoch_end() || locked_block.is_genesis() { - current_epoch - } else { - locked_block.epoch() - }; + let local_committee = self.epoch_manager.get_local_committee(epoch).await?; let is_leader = @@ -504,61 +535,33 @@ impl HotstuffWorker { .len(), ); if is_leader { + let current_epoch = self.epoch_manager.current_epoch().await?; + let propose_epoch_end = current_epoch > epoch; + self.on_propose - .handle(current_epoch, &local_committee, leaf_block, is_newview_propose) + .handle( + epoch, + &local_committee, + leaf_block, + is_newview_propose, + propose_epoch_end, + ) .await?; - } else if is_newview_propose { - // We can make this a warm/error in future, but for now I want to be sure this never happens - panic!("propose_if_leader called with is_newview_propose=true but we're not the leader"); } else { - // Nothing to do + // We can make this a warm/error in future, but for now I want to be sure this never happens + debug_assert!( + !is_newview_propose, + "propose_if_leader called with is_newview_propose=true but we're not the leader" + ); } Ok(()) } async fn dispatch_hotstuff_message( &mut self, - result: IncomingMessageResult, + from: TConsensusSpec::Addr, + msg: HotstuffMessage, ) -> Result<(), HotStuffError> { - let (from, msg) = match result { - Ok(Some(msg)) => msg, - Ok(None) => return Ok(()), - Err(NeedsSync { - from, - local_height, - qc_height, - }) => { - self.hooks.on_needs_sync(local_height, qc_height); - if qc_height.as_u64() - local_height.as_u64() > MAX_BLOCKS_PER_SYNC as u64 { - warn!( - target: LOG_TARGET, - "⚠️ Node is too far behind to catch up from {} (local height: {}, qc height: {})", - from, - local_height, - qc_height - ); - return Err(HotStuffError::FallenBehind { - local_height, - qc_height, - }); - } - self.on_catch_up_sync(&from).await?; - return Ok(()); - }, - }; - - // if !self - // .epoch_manager - // .is_this_validator_registered_for_epoch(msg.epoch()) - // .await? - // { - // warn!( - // target: LOG_TARGET, - // "Received message for inactive epoch: {}", msg.epoch() - // ); - // return Ok(()); - // } - // TODO: check the message comes from a local committee member (except foreign proposals which must come from a // registered node) match msg { @@ -566,10 +569,29 @@ impl HotstuffWorker { "on_receive_new_view", self.on_receive_new_view.handle(from, message).await, ), - HotstuffMessage::Proposal(msg) => log_err( - "on_receive_local_proposal", - self.on_receive_local_proposal.handle(msg).await, - ), + HotstuffMessage::Proposal(msg) => { + let current_view = self.pacemaker.current_view().clone(); + let current_epoch = current_view.get_epoch(); + match log_err( + "on_receive_local_proposal", + self.on_receive_local_proposal.handle(current_view, msg).await, + ) { + Ok(_) => Ok(()), + Err( + err @ HotStuffError::ProposalValidationError(ProposalValidationError::JustifyBlockNotFound { + .. + }), + ) => { + warn!( + target: LOG_TARGET, + "⚠️This node has fallen behind due to a missing justified block: {err}" + ); + self.on_catch_up_sync.request_sync(current_epoch, &from).await?; + Ok(()) + }, + Err(err) => Err(err), + } + }, HotstuffMessage::ForeignProposal(msg) => log_err( "on_receive_foreign_proposal", self.on_receive_foreign_proposal.handle(from, msg).await, @@ -583,7 +605,7 @@ impl HotstuffWorker { "on_receive_requested_txs", self.on_receive_requested_txs.handle(from, msg).await, ), - HotstuffMessage::SyncRequest(msg) => { + HotstuffMessage::CatchUpSyncRequest(msg) => { self.on_sync_request.handle(from, msg); Ok(()) }, @@ -597,38 +619,7 @@ impl HotstuffWorker { } } - pub async fn on_catch_up_sync(&mut self, from: &TConsensusSpec::Addr) -> Result<(), HotStuffError> { - let high_qc = self.state_store.with_read_tx(|tx| HighQc::get(tx))?; - info!( - target: LOG_TARGET, - "⏰ Catch up required from block {} from {} (current height: {})", - high_qc, - from, - self.pacemaker.current_height() - ); - - let current_epoch = self.epoch_manager.current_epoch().await?; - // Send the request message - if self - .outbound_messaging - .send( - from.clone(), - HotstuffMessage::SyncRequest(SyncRequestMessage { - epoch: current_epoch, - high_qc, - }), - ) - .await - .is_err() - { - warn!(target: LOG_TARGET, "Leader channel closed while sending SyncRequest"); - return Ok(()); - } - - Ok(()) - } - - fn create_zero_block_if_required(&self) -> Result<(), HotStuffError> { + fn create_zero_block_if_required(&self, epoch: Epoch, shard: Shard) -> Result<(), HotStuffError> { self.state_store.with_write_tx(|tx| { // The parent for genesis blocks refer to this zero block let zero_block = Block::zero_block(self.network); @@ -644,6 +635,19 @@ impl HotstuffWorker { zero_block.commit_diff(tx, BlockDiff::empty(*zero_block.id()))?; } + let genesis = Block::genesis(self.network, epoch, shard); + if !genesis.exists(&**tx)? { + info!(target: LOG_TARGET, "✨Creating genesis block {genesis}"); + // No genesis.justify() insert because that is the zero block justify + genesis.insert(tx)?; + genesis.as_locked_block().set(tx)?; + genesis.as_leaf_block().set(tx)?; + genesis.as_last_executed().set(tx)?; + genesis.as_last_voted().set(tx)?; + genesis.justify().as_high_qc().set(tx)?; + genesis.commit_diff(tx, BlockDiff::empty(*genesis.id()))?; + } + Ok(()) }) } diff --git a/dan_layer/consensus/src/messages/message.rs b/dan_layer/consensus/src/messages/message.rs index 5d651edda..9b8c60b78 100644 --- a/dan_layer/consensus/src/messages/message.rs +++ b/dan_layer/consensus/src/messages/message.rs @@ -18,7 +18,8 @@ pub enum HotstuffMessage { Vote(VoteMessage), RequestMissingTransactions(RequestMissingTransactionsMessage), RequestedTransaction(RequestedTransactionMessage), - SyncRequest(SyncRequestMessage), + CatchUpSyncRequest(SyncRequestMessage), + // TODO: remove unused SyncResponse(SyncResponseMessage), } @@ -31,7 +32,7 @@ impl HotstuffMessage { HotstuffMessage::Vote(_) => "Vote", HotstuffMessage::RequestMissingTransactions(_) => "RequestMissingTransactions", HotstuffMessage::RequestedTransaction(_) => "RequestedTransaction", - HotstuffMessage::SyncRequest(_) => "SyncRequest", + HotstuffMessage::CatchUpSyncRequest(_) => "CatchUpSyncRequest", HotstuffMessage::SyncResponse(_) => "SyncResponse", } } @@ -44,7 +45,7 @@ impl HotstuffMessage { Self::Vote(msg) => msg.epoch, Self::RequestMissingTransactions(msg) => msg.epoch, Self::RequestedTransaction(msg) => msg.epoch, - Self::SyncRequest(msg) => msg.epoch, + Self::CatchUpSyncRequest(msg) => msg.epoch, Self::SyncResponse(msg) => msg.epoch, } } @@ -61,8 +62,10 @@ impl Display for HotstuffMessage { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { match self { HotstuffMessage::NewView(msg) => write!(f, "NewView({})", msg.new_height), - HotstuffMessage::Proposal(msg) => write!(f, "Proposal({})", msg.block.height()), - HotstuffMessage::ForeignProposal(msg) => write!(f, "ForeignProposal({})", msg.block.height()), + HotstuffMessage::Proposal(msg) => { + write!(f, "Proposal(Epoch={},Height={})", msg.block.epoch(), msg.block.height(),) + }, + HotstuffMessage::ForeignProposal(msg) => write!(f, "ForeignProposal({})", msg), HotstuffMessage::Vote(msg) => write!(f, "Vote({}, {}, {})", msg.block_height, msg.block_id, msg.decision), HotstuffMessage::RequestMissingTransactions(msg) => { write!( @@ -80,7 +83,7 @@ impl Display for HotstuffMessage { msg.block_id, msg.epoch ), - HotstuffMessage::SyncRequest(msg) => write!(f, "SyncRequest({})", msg.high_qc), + HotstuffMessage::CatchUpSyncRequest(msg) => write!(f, "SyncRequest({})", msg.high_qc), HotstuffMessage::SyncResponse(msg) => write!(f, "SyncResponse({} block(s))", msg.blocks.len()), } } diff --git a/dan_layer/consensus/src/messages/proposal.rs b/dan_layer/consensus/src/messages/proposal.rs index 3cbe86d7f..d9f759055 100644 --- a/dan_layer/consensus/src/messages/proposal.rs +++ b/dan_layer/consensus/src/messages/proposal.rs @@ -1,6 +1,8 @@ // Copyright 2023 The Tari Project // SPDX-License-Identifier: BSD-3-Clause +use std::fmt::{Display, Formatter}; + use serde::Serialize; use tari_dan_storage::consensus_models::Block; @@ -8,3 +10,9 @@ use tari_dan_storage::consensus_models::Block; pub struct ProposalMessage { pub block: Block, } + +impl Display for ProposalMessage { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "ProposalMessage({})", self.block) + } +} diff --git a/dan_layer/consensus/src/messages/vote.rs b/dan_layer/consensus/src/messages/vote.rs index 32a62130f..007fe018d 100644 --- a/dan_layer/consensus/src/messages/vote.rs +++ b/dan_layer/consensus/src/messages/vote.rs @@ -1,6 +1,8 @@ // Copyright 2023 The Tari Project // SPDX-License-Identifier: BSD-3-Clause +use std::fmt::Display; + use serde::Serialize; use tari_dan_common_types::{Epoch, NodeHeight}; use tari_dan_storage::consensus_models::{BlockId, LastSentVote, QuorumDecision, ValidatorSignature}; @@ -25,3 +27,13 @@ impl From for VoteMessage { } } } + +impl Display for VoteMessage { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!( + f, + "VoteMessage: {}, block_id: {}, {}, decision: {:?}, voter: {:?}", + self.epoch, self.block_id, self.block_height, self.decision, self.signature.public_key + ) + } +} diff --git a/dan_layer/consensus_tests/src/consensus.rs b/dan_layer/consensus_tests/src/consensus.rs index 5348b34f9..55649cd42 100644 --- a/dan_layer/consensus_tests/src/consensus.rs +++ b/dan_layer/consensus_tests/src/consensus.rs @@ -11,9 +11,9 @@ use std::time::Duration; use tari_consensus::hotstuff::HotStuffError; -use tari_dan_common_types::{optional::Optional, Epoch, NodeHeight}; +use tari_dan_common_types::{optional::Optional, shard::Shard, Epoch, NodeHeight}; use tari_dan_storage::{ - consensus_models::{Block, BlockId, Command, Decision}, + consensus_models::{BlockId, Command, Decision, TransactionRecord, VersionedSubstateIdLockIntent}, StateStore, StateStoreReadTransaction, }; @@ -39,7 +39,7 @@ async fn single_transaction() { let mut test = Test::builder().add_committee(0, vec!["1"]).start().await; // First get transaction in the mempool test.send_transaction_to_all(Decision::Commit, 1, 1).await; - test.start_epoch(Epoch(0)).await; + test.start_epoch(Epoch(1)).await; loop { test.on_block_committed().await; @@ -60,14 +60,14 @@ async fn single_transaction() { test.get_validator(&TestAddress::new("1")) .state_store .with_read_tx(|tx| { - let mut block = Some(Block::get_tip(tx)?); + let mut block = tx.blocks_get_tip(Epoch(1), Shard::from(0))?; loop { - block = block.as_ref().unwrap().get_parent(tx).optional()?; - let Some(b) = block.as_ref() else { + block = block.get_parent(tx)?; + if block.id().is_zero() { break; - }; + } - for cmd in b.commands() { + for cmd in block.commands() { assert!(matches!(cmd, Command::LocalOnly(_))); } } @@ -89,7 +89,7 @@ async fn propose_blocks_with_queued_up_transactions_until_all_committed() { for _ in 0..10 { test.send_transaction_to_all(Decision::Commit, 1, 5).await; } - test.start_epoch(Epoch(0)).await; + test.start_epoch(Epoch(1)).await; loop { test.on_block_committed().await; @@ -116,7 +116,7 @@ async fn propose_blocks_with_new_transactions_until_all_committed() { // .add_committee(0, vec!["1"]) .start().await; let mut remaining_txs = 10; - test.start_epoch(Epoch(0)).await; + test.start_epoch(Epoch(1)).await; loop { if remaining_txs > 0 { test.send_transaction_to_all(Decision::Commit, 1, 5).await; @@ -148,9 +148,9 @@ async fn node_requests_missing_transaction_from_local_leader() { test.send_transaction_to(&TestAddress::new("2"), Decision::Commit, 1, 5) .await; } - test.start_epoch(Epoch(0)).await; + test.start_epoch(Epoch(1)).await; loop { - let (_, _, committed_height) = test.on_block_committed().await; + let (_, _, _, committed_height) = test.on_block_committed().await; if test.is_transaction_pool_empty() { break; @@ -164,17 +164,16 @@ async fn node_requests_missing_transaction_from_local_leader() { test.get_validator(&TestAddress::new("2")) .state_store .with_read_tx(|tx| { - let mut block_id = BlockId::genesis(); + let mut block_id = BlockId::zero(); loop { let children = tx.blocks_get_all_by_parent(&block_id).unwrap(); - if children.is_empty() { + if block_id.is_zero() { break; } - if !block_id.is_genesis() { - assert_eq!(children.len(), 1); - } + + assert_eq!(children.len(), 1); for block in children { - if block.id().is_genesis() { + if block.is_genesis() { continue; } let missing = tx.blocks_get_pending_transactions(block.id()).unwrap(); @@ -195,14 +194,12 @@ async fn node_requests_missing_transaction_from_local_leader() { async fn multi_validator_propose_blocks_with_new_transactions_until_all_committed() { setup_logger(); let mut test = Test::builder() - // TODO: this timeout is just masking an issue. A node sometimes falls behind and then has to catch up, so we need more time for that. - // However, nodes should never fall behind in this test scenario. - .with_test_timeout(Duration::from_secs(60)) .add_committee(0, vec!["1", "2", "3", "4", "5"]) .start() .await; let mut remaining_txs = 10u32; - test.start_epoch(Epoch(0)).await; + + test.start_epoch(Epoch(1)).await; loop { if remaining_txs > 0 { test.send_transaction_to_all(Decision::Commit, 1, 5).await; @@ -237,10 +234,10 @@ async fn multi_shard_propose_blocks_with_new_transactions_until_all_committed() .start() .await; for _ in 0..20 { - test.send_transaction_to_all(Decision::Commit, 100, 5).await; + test.send_transaction_to_all(Decision::Commit, 100, 2).await; } - test.start_epoch(Epoch(0)).await; + test.start_epoch(Epoch(1)).await; loop { test.on_block_committed().await; @@ -285,7 +282,7 @@ async fn foreign_shard_decides_to_abort() { test.send_transaction_to_destination(TestNetworkDestination::Shard(1), tx2.clone()) .await; - test.start_epoch(Epoch(0)).await; + test.start_epoch(Epoch(1)).await; loop { test.on_block_committed().await; @@ -313,7 +310,7 @@ async fn foreign_shard_decides_to_abort() { test.assert_clean_shutdown().await; } -#[tokio::test(flavor = "multi_thread")] +#[tokio::test(flavor = "multi_thread", worker_threads = 4)] async fn output_conflict_abort() { setup_logger(); let mut test = Test::builder() @@ -329,7 +326,7 @@ async fn output_conflict_abort() { .await; let tx = Transaction::builder().sign(&Default::default()).build(); - let tx2 = build_transaction_from(tx, Decision::Commit, 1, resulting_outputs); + let tx2 = build_transaction_from(tx, Decision::Commit, 1, vec![], resulting_outputs); assert_ne!(tx1.id(), tx2.id()); // Transactions are sorted in the blocks, because we have a "first come first serve" policy for locking objects // the "first" will be Committed and the "last" Aborted @@ -339,7 +336,7 @@ async fn output_conflict_abort() { test.send_transaction_to_destination(TestNetworkDestination::All, tx2.clone()) .await; - test.start_epoch(Epoch(0)).await; + test.start_epoch(Epoch(1)).await; loop { test.on_block_committed().await; @@ -369,6 +366,200 @@ async fn output_conflict_abort() { test.assert_clean_shutdown().await; } +#[tokio::test(flavor = "multi_thread", worker_threads = 4)] +async fn inputs_depend_on_outputs_multishard() { + setup_logger(); + let mut test = Test::builder() + .with_test_timeout(Duration::from_secs(60)) + .add_committee(0, vec!["1", "2"]) + .add_committee(1, vec!["3", "4"]) + .start() + .await; + + let tx1 = build_transaction(Decision::Commit, 1, 5, 2); + let resulting_outputs = tx1.resulting_outputs().to_vec(); + test.send_transaction_to_destination(TestNetworkDestination::All, tx1.clone()) + .await; + + let tx = Transaction::builder() + .with_inputs(resulting_outputs.clone().into_iter().map(Into::into)) + .sign(&Default::default()) + .build(); + let tx2 = build_transaction_from( + tx, + Decision::Commit, + 1, + resulting_outputs + .into_iter() + .map(VersionedSubstateIdLockIntent::write) + .collect(), + vec![], + ); + assert_ne!(tx1.id(), tx2.id()); + + test.send_transaction_to_destination(TestNetworkDestination::All, tx2.clone()) + .await; + + test.start_epoch(Epoch(1)).await; + + loop { + test.on_block_committed().await; + + if test.is_transaction_pool_empty() { + break; + } + + let leaf1 = test.get_validator(&TestAddress::new("1")).get_leaf_block(); + let leaf2 = test.get_validator(&TestAddress::new("3")).get_leaf_block(); + if leaf1.height > NodeHeight(30) || leaf2.height > NodeHeight(30) { + panic!( + "Not all transaction committed after {}/{} blocks", + leaf1.height, leaf2.height, + ); + } + } + + test.assert_all_validators_at_same_height().await; + test.assert_all_validators_have_decision(tx1.id(), Decision::Commit) + .await; + test.assert_all_validators_have_decision(tx2.id(), Decision::Abort) + .await; + test.assert_all_validators_committed(); + + test.assert_clean_shutdown().await; + log::info!("total messages sent: {}", test.network().total_messages_sent()); +} + +#[tokio::test(flavor = "multi_thread", worker_threads = 4)] +async fn deferred_input_conflict() { + setup_logger(); + let mut test = Test::builder().add_committee(0, vec!["1", "2"]).start().await; + + let substate_id = test.create_substates_on_all_vns(1).pop().unwrap(); + + let tx1 = Transaction::builder() + .add_input(substate_id.clone()) + .sign(&Default::default()) + .build(); + let tx1 = TransactionRecord::new(tx1); + + let tx2 = Transaction::builder() + .add_input(substate_id.clone()) + .sign(&Default::default()) + .build(); + let tx2 = TransactionRecord::new(tx2); + + test.transaction_executions() + .insert(create_execution_result_for_transaction( + BlockId::zero(), + *tx1.id(), + Decision::Commit, + 0, + vec![VersionedSubstateIdLockIntent::read(substate_id.clone())], + vec![], + )) + .insert(create_execution_result_for_transaction( + BlockId::zero(), + *tx2.id(), + Decision::Commit, + 0, + vec![VersionedSubstateIdLockIntent::write(substate_id)], + vec![], + )); + // Transactions are sorted in the blocks, because we have a "first come first serve" policy for locking objects + // the "first" will be Committed and the "last" Aborted + let mut sorted_tx_ids = [tx1.id(), tx2.id()]; + sorted_tx_ids.sort(); + + test.send_transaction_to_destination(TestNetworkDestination::All, tx1.clone()) + .await; + test.send_transaction_to_destination(TestNetworkDestination::All, tx2.clone()) + .await; + + test.start_epoch(Epoch(1)).await; + + loop { + test.on_block_committed().await; + + if test.is_transaction_pool_empty() { + break; + } + + let leaf1 = test.get_validator(&TestAddress::new("1")).get_leaf_block(); + if leaf1.height > NodeHeight(30) { + panic!("Not all transaction committed after {} blocks", leaf1.height,); + } + } + + test.assert_all_validators_at_same_height().await; + test.assert_all_validators_committed(); + + test.assert_clean_shutdown().await; + log::info!("total messages sent: {}", test.network().total_messages_sent()); +} + +#[tokio::test(flavor = "multi_thread", worker_threads = 4)] +async fn epoch_change() { + setup_logger(); + let mut test = Test::builder() + .with_test_timeout(Duration::from_secs(60)) + .add_committee(0, vec!["1", "2"]) + .start() + .await; + + test.start_epoch(Epoch(1)).await; + let mut remaining_txs = 10; + + loop { + if remaining_txs > 0 { + test.send_transaction_to_all(Decision::Commit, 1, 5).await; + } + remaining_txs -= 1; + if remaining_txs == 5 { + test.start_epoch(Epoch(2)).await; + } + + if remaining_txs == 0 && test.is_transaction_pool_empty() { + break; + } + + let (_, _, epoch, height) = test.on_block_committed().await; + if height.as_u64() > 1 && epoch == 2u64 { + break; + } + + let leaf1 = test.get_validator(&TestAddress::new("1")).get_leaf_block(); + if leaf1.height > NodeHeight(30) { + panic!("Not all transaction committed after {} blocks", leaf1.height,); + } + } + + // Assert epoch changed + test.get_validator(&TestAddress::new("1")) + .state_store + .with_read_tx(|tx| { + let mut block = tx.blocks_get_tip(Epoch(1), Shard::from(0))?; + loop { + block = block.get_parent(tx)?; + if block.id().is_zero() { + break; + } + if block.is_epoch_end() { + return Ok::<_, HotStuffError>(()); + } + } + + panic!("No epoch end block found"); + }) + .unwrap(); + + test.assert_all_validators_at_same_height().await; + test.assert_all_validators_committed(); + + test.assert_clean_shutdown().await; + log::info!("total messages sent: {}", test.network().total_messages_sent()); +} + #[tokio::test(flavor = "multi_thread", worker_threads = 4)] async fn leader_failure_node_goes_down() { setup_logger(); @@ -383,10 +574,10 @@ async fn leader_failure_node_goes_down() { for _ in 0..10 { test.send_transaction_to_all(Decision::Commit, 1, 2).await; } - test.start_epoch(Epoch(0)).await; + test.start_epoch(Epoch(1)).await; loop { - let (_, _, committed_height) = test.on_block_committed().await; + let (_, _, _, committed_height) = test.on_block_committed().await; if committed_height == NodeHeight(1) { log::info!("😴 Node 2 goes offline"); @@ -446,7 +637,7 @@ async fn foreign_block_distribution() { } test.network().start(); - test.start_epoch(Epoch(0)).await; + test.start_epoch(Epoch(1)).await; loop { test.on_block_committed().await; @@ -489,17 +680,17 @@ async fn deferred_execution() { test.transaction_executions() .insert(create_execution_result_for_transaction( - BlockId::genesis(), + BlockId::zero(), *tx.id(), Decision::Commit, 0, - inputs, + inputs.into_iter().map(VersionedSubstateIdLockIntent::write).collect(), vec![], )); test.send_transaction_to_destination(TestNetworkDestination::All, tx) .await; - test.start_epoch(Epoch(0)).await; + test.start_epoch(Epoch(1)).await; loop { test.on_block_committed().await; @@ -520,7 +711,7 @@ async fn deferred_execution() { test.get_validator(&TestAddress::new("1")) .state_store .with_read_tx(|tx| { - let mut block = Some(Block::get_tip(tx)?); + let mut block = Some(tx.blocks_get_tip(Epoch(1), Shard::from(0))?); loop { block = block.as_ref().unwrap().get_parent(tx).optional()?; let Some(b) = block.as_ref() else { diff --git a/dan_layer/consensus_tests/src/substate_store.rs b/dan_layer/consensus_tests/src/substate_store.rs index 2462fa4d3..a5070ec17 100644 --- a/dan_layer/consensus_tests/src/substate_store.rs +++ b/dan_layer/consensus_tests/src/substate_store.rs @@ -2,10 +2,10 @@ // SPDX-License-Identifier: BSD-3-Clause use tari_consensus::{ - hotstuff::substate_store::{PendingSubstateStore, SubstateStoreError}, + hotstuff::substate_store::{ChainScopedTreeStore, PendingSubstateStore, SubstateStoreError}, traits::{ReadableSubstateStore, WriteableSubstateStore}, }; -use tari_dan_common_types::PeerAddress; +use tari_dan_common_types::{shard::Shard, Epoch, NodeAddressable, PeerAddress}; use tari_dan_storage::{ consensus_models::{ BlockId, @@ -37,7 +37,7 @@ fn it_allows_substate_up_for_v0() { let value = new_substate_value(0); let tx = store.create_read_tx().unwrap(); - let mut store = PendingSubstateStore::<'_, '_, TestStore>::new(&tx); + let mut store = create_pending_store(&tx); // Cannot put version 1 store .put(SubstateChange::Up { @@ -67,7 +67,7 @@ fn it_allows_down_then_up() { let id = add_substate(&store, 0, 0); let tx = store.create_read_tx().unwrap(); - let mut store = PendingSubstateStore::<'_, '_, TestStore>::new(&tx); + let mut store = create_pending_store(&tx); let s = store.get_latest(id.substate_id()).unwrap(); assert_substate_eq(s, new_substate(0, 0)); @@ -100,7 +100,7 @@ fn it_fails_if_previous_version_is_not_down() { let id = add_substate(&store, 0, 0); let tx = store.create_read_tx().unwrap(); - let mut store = PendingSubstateStore::<'_, '_, TestStore>::new(&tx); + let mut store = create_pending_store(&tx); let err = store .put(SubstateChange::Up { id: id.to_next_version(), @@ -119,7 +119,7 @@ fn it_disallows_more_than_one_write_lock_non_local_only() { let id = add_substate(&store, 0, 0); let tx = store.create_read_tx().unwrap(); - let mut store = PendingSubstateStore::<'_, '_, TestStore>::new(&tx); + let mut store = create_pending_store(&tx); store .try_lock( @@ -158,7 +158,7 @@ fn it_allows_locks_within_one_transaction() { let id = add_substate(&store, 0, 0); let tx = store.create_read_tx().unwrap(); - let mut store = PendingSubstateStore::<'_, '_, TestStore>::new(&tx); + let mut store = create_pending_store(&tx); store .try_lock( @@ -202,9 +202,10 @@ fn add_substate(store: &TestStore, seed: u8, version: u32) -> VersionedSubstateI substate_value: value, state_hash: [seed; 32].into(), created_by_transaction: Default::default(), - created_justify: QcId::genesis(), - created_block: BlockId::genesis(), + created_justify: QcId::zero(), + created_block: BlockId::zero(), created_height: 0.into(), + created_by_shard: Shard::zero(), created_at_epoch: 0.into(), destroyed: None, } @@ -219,6 +220,13 @@ fn create_store() -> TestStore { SqliteStateStore::connect(":memory:").unwrap() } +fn create_pending_store<'a, 'tx, TAddr: NodeAddressable>( + tx: &'a as StateStore>::ReadTransaction<'tx>, +) -> PendingSubstateStore<'a, 'tx, SqliteStateStore> { + let tree_store = ChainScopedTreeStore::new(Epoch::zero(), Shard::zero(), tx); + PendingSubstateStore::new(tree_store) +} + fn new_substate_id(seed: u8) -> SubstateId { ComponentAddress::from_array([seed; ObjectKey::LENGTH]).into() } diff --git a/dan_layer/consensus_tests/src/support/address.rs b/dan_layer/consensus_tests/src/support/address.rs index f2800e023..9a0b67ff1 100644 --- a/dan_layer/consensus_tests/src/support/address.rs +++ b/dan_layer/consensus_tests/src/support/address.rs @@ -14,6 +14,10 @@ impl TestAddress { pub fn new>(s: T) -> Self { TestAddress(s.into()) } + + pub fn as_bytes(&self) -> &[u8] { + self.0.as_bytes() + } } impl NodeAddressable for TestAddress { diff --git a/dan_layer/consensus_tests/src/support/epoch_manager.rs b/dan_layer/consensus_tests/src/support/epoch_manager.rs index ded0af4dc..d09273cfd 100644 --- a/dan_layer/consensus_tests/src/support/epoch_manager.rs +++ b/dan_layer/consensus_tests/src/support/epoch_manager.rs @@ -304,6 +304,11 @@ impl EpochManagerReader for TestEpochManager { async fn get_base_layer_block_height(&self, _hash: FixedHash) -> Result, EpochManagerError> { Ok(Some(self.inner.lock().await.current_block_info.0)) } + + async fn wait_for_initial_scanning_to_complete(&self) -> Result<(), EpochManagerError> { + // Scanning is not relevant to tests + Ok(()) + } } #[derive(Debug, Clone)] diff --git a/dan_layer/consensus_tests/src/support/executions_store.rs b/dan_layer/consensus_tests/src/support/executions_store.rs index 11fa8076d..843a27c33 100644 --- a/dan_layer/consensus_tests/src/support/executions_store.rs +++ b/dan_layer/consensus_tests/src/support/executions_store.rs @@ -21,11 +21,12 @@ impl TestTransactionExecutionsStore { } } - pub fn insert(&self, execution: TransactionExecution) { + pub fn insert(&self, execution: TransactionExecution) -> &Self { self.transactions .write() .unwrap() .insert(*execution.transaction_id(), execution); + self } pub fn get(&self, transaction_id: &TransactionId) -> Option { diff --git a/dan_layer/consensus_tests/src/support/harness.rs b/dan_layer/consensus_tests/src/support/harness.rs index cd7a06d37..38dcdac81 100644 --- a/dan_layer/consensus_tests/src/support/harness.rs +++ b/dan_layer/consensus_tests/src/support/harness.rs @@ -7,13 +7,12 @@ use std::{ }; use futures::{stream::FuturesUnordered, FutureExt, StreamExt}; -use tari_common_types::types::{PrivateKey, PublicKey}; use tari_consensus::hotstuff::HotstuffEvent; -use tari_crypto::keys::{PublicKey as _, SecretKey}; use tari_dan_common_types::{committee::Committee, shard::Shard, Epoch, NodeHeight}; use tari_dan_storage::{ - consensus_models::{Block, BlockId, Decision, QcId, SubstateRecord, TransactionRecord}, + consensus_models::{BlockId, Decision, QcId, SubstateRecord, TransactionRecord}, StateStore, + StateStoreReadTransaction, StorageError, }; use tari_engine_types::{ @@ -26,7 +25,7 @@ use tari_template_lib::models::ComponentAddress; use tari_transaction::{TransactionId, VersionedSubstateId}; use tokio::{sync::broadcast, task, time::sleep}; -use super::MessageFilter; +use super::{helpers, MessageFilter}; use crate::support::{ address::TestAddress, epoch_manager::TestEpochManager, @@ -105,11 +104,12 @@ impl Test { id, 0, value, + Shard::zero(), Epoch(0), NodeHeight(0), - BlockId::genesis(), + BlockId::zero(), TransactionId::default(), - QcId::genesis(), + QcId::zero(), ) }) .collect::>(); @@ -150,7 +150,7 @@ impl Test { .unwrap() } - pub async fn on_block_committed(&mut self) -> (TestAddress, BlockId, NodeHeight) { + pub async fn on_block_committed(&mut self) -> (TestAddress, BlockId, Epoch, NodeHeight) { loop { let (address, event) = if let Some(timeout) = self.timeout { tokio::time::timeout(timeout, self.on_hotstuff_event()) @@ -160,7 +160,11 @@ impl Test { self.on_hotstuff_event().await }; match event { - HotstuffEvent::BlockCommitted { block_id, height } => return (address, block_id, height), + HotstuffEvent::BlockCommitted { + block_id, + epoch, + height, + } => return (address, block_id, epoch, height), HotstuffEvent::Failure { message } => panic!("[{}] Consensus failure: {}", address, message), other => { log::info!("[{}] Ignoring event: {:?}", address, other); @@ -265,22 +269,26 @@ impl Test { } pub async fn assert_all_validators_at_same_height_except(&self, except: &[TestAddress]) { + let current_epoch = self.epoch_manager.current_epoch().await.unwrap(); let committees = self.epoch_manager.all_committees().await; let mut attempts = 0usize; 'outer: loop { - for committee in committees.values() { - let mut heights = self + for (shard, committee) in &committees { + let mut blocks = self .validators .values() .filter(|vn| committee.contains(&vn.address)) .filter(|vn| !except.contains(&vn.address)) .map(|v| { - let height = v.state_store.with_read_tx(|tx| Block::get_tip(tx)).unwrap().height(); - (v.address.clone(), height) + let block = v + .state_store + .with_read_tx(|tx| tx.blocks_get_tip(current_epoch, *shard)) + .unwrap(); + (v.address.clone(), block) }); - let (first_addr, first) = heights.next().unwrap(); - for (addr, height) in heights { - if first != height && attempts < 5 { + let (first_addr, first) = blocks.next().unwrap(); + for (addr, block) in blocks { + if (first.epoch() != block.epoch() || first.height() != block.height()) && attempts < 5 { attempts += 1; // Send this task to the back of the queue and try again after other tasks have executed // to allow validators to catch up @@ -288,9 +296,13 @@ impl Test { continue 'outer; } assert_eq!( - first, height, + first.id(), + block.id(), "Validator {} is at height {} but validator {} is at height {}", - first_addr, first, addr, height + first_addr, + first, + addr, + block ); } } @@ -349,9 +361,9 @@ impl Test { }); } - pub async fn assert_clean_shutdown(mut self) { + pub async fn assert_clean_shutdown(&mut self) { self.shutdown.trigger(); - for v in self.validators.into_values() { + for (_, v) in self.validators.drain() { v.handle.await.unwrap(); } } @@ -399,19 +411,16 @@ impl TestBuilder { self } - pub fn add_committee>(mut self, bucket: T, addresses: Vec<&'static str>) -> Self { + pub fn add_committee>(mut self, shard: T, addresses: Vec<&'static str>) -> Self { let entry = self .committees - .entry(bucket.into()) + .entry(shard.into()) .or_insert_with(|| Committee::new(vec![])); for addr in addresses { - let mut bytes = [0u8; 64]; - bytes[0..addr.as_bytes().len()].copy_from_slice(addr.as_bytes()); - let secret_key = PrivateKey::from_uniform_bytes(&bytes).unwrap(); - entry - .members - .push((TestAddress::new(addr), PublicKey::from_secret_key(&secret_key))); + let addr = TestAddress::new(addr); + let (_, pk) = helpers::derive_keypair_from_address(&addr); + entry.members.push((addr, pk)); } self } @@ -432,12 +441,14 @@ impl TestBuilder { .all_validators() .await .into_iter() - .map(|(address, bucket, shard, pk, _, _, _)| { + .map(|(address, bucket, shard, _, _, _, _)| { let sql_address = self.sql_address.replace("{}", &address.0); + let (sk, pk) = helpers::derive_keypair_from_address(&address); + let (channels, validator) = Validator::builder() .with_sql_url(sql_address) .with_transaction_executions(transaction_executions.clone()) - .with_address_and_public_key(address.clone(), pk.clone()) + .with_address_and_secret_key(address.clone(), sk) .with_shard(shard) .with_bucket(bucket) .with_epoch_manager(epoch_manager.clone_for(address.clone(), pk, shard)) diff --git a/dan_layer/consensus_tests/src/support/helpers.rs b/dan_layer/consensus_tests/src/support/helpers.rs index 6f0b9bc59..0326574c9 100644 --- a/dan_layer/consensus_tests/src/support/helpers.rs +++ b/dan_layer/consensus_tests/src/support/helpers.rs @@ -2,11 +2,15 @@ // SPDX-License-Identifier: BSD-3-Clause use rand::{rngs::OsRng, Rng}; +use tari_common_types::types::{PrivateKey, PublicKey}; +use tari_crypto::keys::{PublicKey as _, SecretKey}; use tari_dan_common_types::shard::Shard; use tari_engine_types::substate::SubstateId; use tari_template_lib::models::{ComponentAddress, ComponentKey, EntityId, ObjectKey}; use tari_transaction::VersionedSubstateId; +use crate::support::TestAddress; + pub(crate) fn random_substate_in_shard(shard: Shard, num_shards: u32) -> VersionedSubstateId { let range = shard.to_substate_address_range(num_shards); let size = range.end().to_u256() - range.start().to_u256(); @@ -23,3 +27,11 @@ fn copy_fixed(bytes: &[u8]) -> [u8; SZ] { out.copy_from_slice(bytes); out } + +pub fn derive_keypair_from_address(addr: &TestAddress) -> (PrivateKey, PublicKey) { + let mut bytes = [0u8; 64]; + bytes[0..addr.as_bytes().len()].copy_from_slice(addr.as_bytes()); + let secret_key = PrivateKey::from_uniform_bytes(&bytes).unwrap(); + let public_key = PublicKey::from_secret_key(&secret_key); + (secret_key, public_key) +} diff --git a/dan_layer/consensus_tests/src/support/network.rs b/dan_layer/consensus_tests/src/support/network.rs index 3efb50baa..d6f9ff7fe 100644 --- a/dan_layer/consensus_tests/src/support/network.rs +++ b/dan_layer/consensus_tests/src/support/network.rs @@ -343,6 +343,7 @@ impl TestNetworkWorker { } log::debug!("βœ‰οΈ Message {} from {} to {}", msg, from, to); if from != to && self.is_offline_destination(&from, u32::MAX.into()).await { + log::info!("πŸ›‘ Discarding message {msg}. Leader {from} is offline"); return; } self.on_message.send(Some(msg.clone())).unwrap(); diff --git a/dan_layer/consensus_tests/src/support/signing_service.rs b/dan_layer/consensus_tests/src/support/signing_service.rs index e5df6cca2..c9da2c213 100644 --- a/dan_layer/consensus_tests/src/support/signing_service.rs +++ b/dan_layer/consensus_tests/src/support/signing_service.rs @@ -4,10 +4,9 @@ use rand::rngs::OsRng; use tari_common_types::types::{FixedHash, PrivateKey, PublicKey}; use tari_consensus::traits::{ValidatorSignatureService, VoteSignatureService}; -use tari_crypto::keys::SecretKey; use tari_dan_storage::consensus_models::{BlockId, QuorumDecision, ValidatorSchnorrSignature, ValidatorSignature}; -use super::TestAddress; +use super::{helpers, TestAddress}; #[derive(Debug, Clone)] pub struct TestVoteSignatureService { @@ -17,10 +16,8 @@ pub struct TestVoteSignatureService { } impl TestVoteSignatureService { - pub fn new(public_key: PublicKey, addr: TestAddress) -> Self { - let mut bytes = [0u8; 64]; - bytes[0..addr.0.as_bytes().len()].copy_from_slice(addr.0.as_bytes()); - let secret_key = PrivateKey::from_uniform_bytes(&bytes).unwrap(); + pub fn new(addr: TestAddress) -> Self { + let (secret_key, public_key) = helpers::derive_keypair_from_address(&addr); Self { public_key, secret_key, diff --git a/dan_layer/consensus_tests/src/support/transaction.rs b/dan_layer/consensus_tests/src/support/transaction.rs index a2488af13..55121b7ae 100644 --- a/dan_layer/consensus_tests/src/support/transaction.rs +++ b/dan_layer/consensus_tests/src/support/transaction.rs @@ -9,7 +9,6 @@ use tari_dan_storage::consensus_models::{ BlockId, Decision, ExecutedTransaction, - SubstateLockFlag, TransactionExecution, TransactionRecord, VersionedSubstateIdLockIntent, @@ -28,29 +27,26 @@ pub fn build_transaction_from( tx: Transaction, decision: Decision, fee: u64, + resolved_inputs: Vec, resulting_outputs: Vec, ) -> TransactionRecord { let mut tx = TransactionRecord::new(tx); - // TODO: cleanup test api- this wont be expected - if !resulting_outputs.is_empty() { + if !decision.is_deferred() { let execution = create_execution_result_for_transaction( // We're just building the execution here for DRY purposes, so genesis block id isn't used - BlockId::genesis(), + BlockId::zero(), *tx.id(), decision, fee, - vec![], + resolved_inputs, resulting_outputs.clone(), ); tx.result = Some(execution.result); tx.resulting_outputs = execution.resulting_outputs; tx.execution_time = Some(execution.execution_time); - // If the transaction does not require any inputs, we immediately resolve them to an empty set - if tx.transaction.all_inputs_iter().next().is_none() { - tx.resolved_inputs = Some(execution.resolved_inputs); - } + tx.resolved_inputs = Some(execution.resolved_inputs); } tx } @@ -60,7 +56,7 @@ pub fn create_execution_result_for_transaction( tx_id: TransactionId, decision: Decision, fee: u64, - resolved_inputs: Vec, + resolved_inputs: Vec, resulting_outputs: Vec, ) -> TransactionExecution { let result = if decision.is_commit() { @@ -96,10 +92,7 @@ pub fn create_execution_result_for_transaction( cost_breakdown: vec![], }), }, - resolved_inputs - .into_iter() - .map(|v| VersionedSubstateIdLockIntent::new(v, SubstateLockFlag::Write)) - .collect(), + resolved_inputs, resulting_outputs, Duration::from_secs(0), ) @@ -123,7 +116,7 @@ pub fn build_transaction( }) .collect::>(); - build_transaction_from(tx, decision, fee, outputs) + build_transaction_from(tx, decision, fee, vec![], outputs) } pub fn build_transaction_with_inputs>( @@ -133,7 +126,7 @@ pub fn build_transaction_with_inputs ) -> TransactionRecord { let k = PrivateKey::default(); let tx = Transaction::builder().with_inputs(inputs).sign(&k).build(); - build_transaction_from(tx, decision, fee, vec![]) + build_transaction_from(tx, decision, fee, vec![], vec![]) } pub fn change_decision(tx: ExecutedTransaction, new_decision: Decision) -> TransactionRecord { @@ -144,6 +137,6 @@ pub fn change_decision(tx: ExecutedTransaction, new_decision: Decision) -> Trans .total_allocated_fee_payments() .as_u64_checked() .unwrap(); - let resulting_outputs = tx.resulting_outputs().to_vec(); - build_transaction_from(tx.into_transaction(), new_decision, total_fees_paid, resulting_outputs) + let (tx, _, resolved_inputs, resulting_outputs) = tx.dissolve(); + build_transaction_from(tx, new_decision, total_fees_paid, resolved_inputs, resulting_outputs) } diff --git a/dan_layer/consensus_tests/src/support/transaction_executor.rs b/dan_layer/consensus_tests/src/support/transaction_executor.rs index 7fcd54870..1eb12c6a8 100644 --- a/dan_layer/consensus_tests/src/support/transaction_executor.rs +++ b/dan_layer/consensus_tests/src/support/transaction_executor.rs @@ -32,7 +32,7 @@ impl BlockTransactionExecutor for TestBloc ) -> Result { if let Some(execution) = self.store.get(transaction.id()) { let mut rec = TransactionRecord::new(transaction); - rec.resolved_inputs = Some(execution.resolved_inputs().clone()); + rec.resolved_inputs = Some(execution.resolved_inputs().to_vec()); rec.result = Some(execution.result().clone()); rec.resulting_outputs.clone_from(execution.resulting_outputs()); rec.execution_time = Some(execution.execution_time()); diff --git a/dan_layer/consensus_tests/src/support/validator/builder.rs b/dan_layer/consensus_tests/src/support/validator/builder.rs index ca14f2f3d..368973188 100644 --- a/dan_layer/consensus_tests/src/support/validator/builder.rs +++ b/dan_layer/consensus_tests/src/support/validator/builder.rs @@ -2,11 +2,12 @@ // SPDX-License-Identifier: BSD-3-Clause use tari_common::configuration::Network; -use tari_common_types::types::PublicKey; +use tari_common_types::types::{PrivateKey, PublicKey}; use tari_consensus::{ hotstuff::{ConsensusCurrentState, ConsensusWorker, ConsensusWorkerContext, HotstuffConfig, HotstuffWorker}, traits::hooks::NoopHooks, }; +use tari_crypto::keys::PublicKey as _; use tari_dan_common_types::{shard::Shard, SubstateAddress}; use tari_dan_storage::consensus_models::TransactionPool; use tari_shutdown::ShutdownSignal; @@ -29,6 +30,7 @@ use crate::support::{ pub struct ValidatorBuilder { pub address: TestAddress, + pub secret_key: PrivateKey, pub public_key: PublicKey, pub shard: SubstateAddress, pub bucket: Shard, @@ -42,6 +44,7 @@ impl ValidatorBuilder { pub fn new() -> Self { Self { address: TestAddress::new("default"), + secret_key: PrivateKey::default(), public_key: PublicKey::default(), shard: SubstateAddress::zero(), bucket: Shard::from(0), @@ -52,9 +55,10 @@ impl ValidatorBuilder { } } - pub fn with_address_and_public_key(&mut self, address: TestAddress, public_key: PublicKey) -> &mut Self { + pub fn with_address_and_secret_key(&mut self, address: TestAddress, secret_key: PrivateKey) -> &mut Self { self.address = address; - self.public_key = public_key; + self.public_key = PublicKey::from_secret_key(&secret_key); + self.secret_key = secret_key; self } @@ -105,7 +109,7 @@ impl ValidatorBuilder { let inbound_messaging = TestInboundMessaging::new(self.address.clone(), rx_hs_message, rx_loopback); let store = SqliteStateStore::connect(&self.sql_url).unwrap(); - let signing_service = TestVoteSignatureService::new(self.public_key.clone(), self.address.clone()); + let signing_service = TestVoteSignatureService::new(self.address.clone()); let transaction_pool = TransactionPool::new(); let (tx_events, _) = broadcast::channel(100); diff --git a/dan_layer/epoch_manager/src/base_layer/base_layer_epoch_manager.rs b/dan_layer/epoch_manager/src/base_layer/base_layer_epoch_manager.rs index ba1ce400f..da00d0b4b 100644 --- a/dan_layer/epoch_manager/src/base_layer/base_layer_epoch_manager.rs +++ b/dan_layer/epoch_manager/src/base_layer/base_layer_epoch_manager.rs @@ -23,6 +23,7 @@ use std::{ cmp, collections::{HashMap, HashSet}, + mem, num::NonZeroU32, }; @@ -42,13 +43,12 @@ use tari_dan_common_types::{ use tari_dan_storage::global::{models::ValidatorNode, DbBaseLayerBlockInfo, DbEpoch, GlobalDb, MetadataKey}; use tari_dan_storage_sqlite::global::SqliteGlobalDbAdapter; use tari_utilities::{byte_array::ByteArray, hex::Hex}; -use tokio::sync::broadcast; +use tokio::sync::{broadcast, oneshot}; use crate::{base_layer::config::EpochManagerConfig, error::EpochManagerError, EpochManagerEvent}; const LOG_TARGET: &str = "tari::dan::epoch_manager::base_layer"; -#[derive(Clone)] pub struct BaseLayerEpochManager { global_db: GlobalDb, base_node_client: TBaseNodeClient, @@ -57,6 +57,7 @@ pub struct BaseLayerEpochManager { current_block_info: (u64, FixedHash), last_block_of_current_epoch: FixedHash, tx_events: broadcast::Sender, + waiting_for_scanning_complete: Vec>>, node_public_key: PublicKey, current_shard_key: Option, base_layer_consensus_constants: Option, @@ -80,6 +81,7 @@ impl current_epoch: Epoch(0), current_block_info: (0, Default::default()), last_block_of_current_epoch: Default::default(), + waiting_for_scanning_complete: Vec::new(), tx_events, node_public_key, current_shard_key: None, @@ -130,11 +132,6 @@ impl self.update_base_layer_consensus_constants(base_layer_constants)?; self.assign_validators_for_epoch(epoch)?; - // Only publish an epoch change event if we have synced the base layer (see on_scanning_complete) - if self.is_initial_base_layer_sync_complete { - self.publish_event(EpochManagerEvent::EpochChanged(epoch)); - } - Ok(()) } @@ -147,7 +144,7 @@ impl let num_committees = calculate_num_committees(vns.len() as u64, self.config.committee_size); for vn in &vns { - validator_nodes.set_committee_bucket( + validator_nodes.set_committee_shard( vn.shard_key, vn.shard_key.to_shard(num_committees), self.config.validator_node_sidechain_id.as_ref(), @@ -512,13 +509,25 @@ impl target: LOG_TARGET, "🌟 Initial base layer sync complete. Current epoch is {}", self.current_epoch ); - self.publish_event(EpochManagerEvent::EpochChanged(self.current_epoch)); self.is_initial_base_layer_sync_complete = true; + for reply in mem::take(&mut self.waiting_for_scanning_complete) { + let _ignore = reply.send(Ok(())); + } } + self.publish_event(EpochManagerEvent::EpochChanged(self.current_epoch)); + Ok(()) } + pub fn add_notify_on_scanning_complete(&mut self, reply: oneshot::Sender>) { + if self.is_initial_base_layer_sync_complete { + let _ignore = reply.send(Ok(())); + } else { + self.waiting_for_scanning_complete.push(reply); + } + } + pub async fn remaining_registration_epochs(&mut self) -> Result, EpochManagerError> { let last_registration_epoch = match self.last_registration_epoch()? { Some(epoch) => epoch, diff --git a/dan_layer/epoch_manager/src/base_layer/epoch_manager_service.rs b/dan_layer/epoch_manager/src/base_layer/epoch_manager_service.rs index 7ce719643..8f94a6651 100644 --- a/dan_layer/epoch_manager/src/base_layer/epoch_manager_service.rs +++ b/dan_layer/epoch_manager/src/base_layer/epoch_manager_service.rs @@ -211,6 +211,9 @@ impl EpochManagerRequest::NotifyScanningComplete { reply } => { handle(reply, self.inner.on_scanning_complete().await, context) }, + EpochManagerRequest::WaitForInitialScanningToComplete { reply } => { + self.inner.add_notify_on_scanning_complete(reply); + }, EpochManagerRequest::RemainingRegistrationEpochs { reply } => { handle(reply, self.inner.remaining_registration_epochs().await, context) }, diff --git a/dan_layer/epoch_manager/src/base_layer/handle.rs b/dan_layer/epoch_manager/src/base_layer/handle.rs index 0428b115b..13d345bbc 100644 --- a/dan_layer/epoch_manager/src/base_layer/handle.rs +++ b/dan_layer/epoch_manager/src/base_layer/handle.rs @@ -181,6 +181,16 @@ impl EpochManagerReader for EpochManagerHandle { rx.await.map_err(|_| EpochManagerError::ReceiveError)? } + async fn wait_for_initial_scanning_to_complete(&self) -> Result<(), EpochManagerError> { + let (tx, rx) = oneshot::channel(); + self.tx_request + .send(EpochManagerRequest::WaitForInitialScanningToComplete { reply: tx }) + .await + .map_err(|_| EpochManagerError::SendError)?; + + rx.await.map_err(|_| EpochManagerError::ReceiveError)? + } + async fn get_all_validator_nodes(&self, epoch: Epoch) -> Result>, EpochManagerError> { let (tx, rx) = oneshot::channel(); self.tx_request diff --git a/dan_layer/epoch_manager/src/base_layer/types.rs b/dan_layer/epoch_manager/src/base_layer/types.rs index 4a44145e9..22f18cf6a 100644 --- a/dan_layer/epoch_manager/src/base_layer/types.rs +++ b/dan_layer/epoch_manager/src/base_layer/types.rs @@ -99,6 +99,9 @@ pub enum EpochManagerRequest { NotifyScanningComplete { reply: Reply<()>, }, + WaitForInitialScanningToComplete { + reply: Reply<()>, + }, RemainingRegistrationEpochs { reply: Reply>, }, diff --git a/dan_layer/epoch_manager/src/traits.rs b/dan_layer/epoch_manager/src/traits.rs index da93a885d..e7e082579 100644 --- a/dan_layer/epoch_manager/src/traits.rs +++ b/dan_layer/epoch_manager/src/traits.rs @@ -42,6 +42,8 @@ pub trait EpochManagerReader: Send + Sync { async fn subscribe(&self) -> Result, EpochManagerError>; + async fn wait_for_initial_scanning_to_complete(&self) -> Result<(), EpochManagerError>; + async fn get_all_validator_nodes(&self, epoch: Epoch) -> Result>, EpochManagerError>; async fn get_committees(&self, epoch: Epoch) -> Result>, EpochManagerError>; diff --git a/dan_layer/p2p/proto/consensus.proto b/dan_layer/p2p/proto/consensus.proto index 5f308e4cf..4e3d4a777 100644 --- a/dan_layer/p2p/proto/consensus.proto +++ b/dan_layer/p2p/proto/consensus.proto @@ -56,6 +56,7 @@ message Block { uint64 timestamp = 13; uint64 base_layer_block_height = 14; bytes base_layer_block_hash = 15; + bool is_dummy = 16; } message LeaderFee { @@ -70,16 +71,10 @@ message Command { TransactionAtom accept = 3; TransactionAtom local_only = 4; ForeignProposal foreign_proposal = 5; - EpochEvent epoch_event = 6; + bool end_epoch = 6; } } -enum EpochEvent { - UNKNOWN_EVENT = 0; - START = 1; - END = 2; -} - enum ForeignProposalState { UNKNOWN_STATE = 0; NEW = 1; @@ -205,19 +200,21 @@ message Substate { bytes substate = 3; uint64 created_epoch = 4; - uint64 created_height = 5; - bytes created_block = 6; - bytes created_transaction = 7; - bytes created_justify = 8; + uint32 created_by_shard = 5; + uint64 created_height = 6; + bytes created_block = 7; + bytes created_transaction = 8; + bytes created_justify = 9; SubstateDestroyed destroyed = 10; } message SubstateDestroyed { - tari.dan.common.Epoch epoch = 9; - bytes block = 10; - bytes transaction = 11; - bytes justify = 12; + tari.dan.common.Epoch epoch = 1; + uint32 shard = 2; + uint64 block_height = 3; + bytes transaction = 4; + bytes justify = 5; } message SyncRequest { @@ -228,7 +225,8 @@ message SyncRequest { message HighQc { bytes block_id = 1; uint64 block_height = 2; - bytes qc_id = 3; + uint64 epoch = 3; + bytes qc_id = 4; } message SyncResponse { diff --git a/dan_layer/p2p/proto/rpc.proto b/dan_layer/p2p/proto/rpc.proto index f47fcda0b..3d21b72f4 100644 --- a/dan_layer/p2p/proto/rpc.proto +++ b/dan_layer/p2p/proto/rpc.proto @@ -161,15 +161,6 @@ message GetVirtualSubstateResponse { repeated tari.dan.consensus.QuorumCertificate quorum_certificates = 2; } -message SyncStateRequest { - bytes start_block_id = 1; - uint64 end_epoch = 2; -} - -message SyncStateResponse { - SubstateUpdate update = 1; -} - message SubstateCreatedProof { SubstateData substate = 1; tari.dan.consensus.QuorumCertificate created_justify = 2; @@ -225,3 +216,42 @@ message GetHighQcRequest {} message GetHighQcResponse { tari.dan.consensus.QuorumCertificate high_qc = 1; } + +message GetCheckpointRequest { + uint64 current_epoch = 1; +} + +message GetCheckpointResponse { + EpochCheckpoint checkpoint = 1; +} + +message EpochCheckpoint { + tari.dan.consensus.Block block = 1; + repeated tari.dan.consensus.QuorumCertificate qcs = 2; +} + +message SyncStateRequest { + uint64 start_epoch = 1; + uint32 start_shard = 2; + uint64 start_seq = 3; + // The shard in the current shard-epoch that is requested. + // This will limit the state transitions returned to those that fall within this shard-epoch. + uint64 current_epoch = 4; + uint32 current_shard = 5; +} + +message SyncStateResponse { + repeated StateTransition transitions = 1; +} + +message StateTransition { + StateTransitionId id = 1; + SubstateUpdate update = 2; + uint64 state_tree_version = 3; +} + +message StateTransitionId { + uint64 epoch = 1; + uint32 shard = 2; + uint64 seq = 3; +} \ No newline at end of file diff --git a/dan_layer/p2p/src/conversions/consensus.rs b/dan_layer/p2p/src/conversions/consensus.rs index 1e3d53f42..8883fce40 100644 --- a/dan_layer/p2p/src/conversions/consensus.rs +++ b/dan_layer/p2p/src/conversions/consensus.rs @@ -42,7 +42,6 @@ use tari_dan_storage::consensus_models::{ BlockId, Command, Decision, - EpochEvent, Evidence, ForeignProposal, ForeignProposalState, @@ -76,7 +75,9 @@ impl From<&HotstuffMessage> for proto::consensus::HotStuffMessage { HotstuffMessage::RequestedTransaction(msg) => { proto::consensus::hot_stuff_message::Message::RequestedTransaction(msg.into()) }, - HotstuffMessage::SyncRequest(msg) => proto::consensus::hot_stuff_message::Message::SyncRequest(msg.into()), + HotstuffMessage::CatchUpSyncRequest(msg) => { + proto::consensus::hot_stuff_message::Message::SyncRequest(msg.into()) + }, HotstuffMessage::SyncResponse(msg) => { proto::consensus::hot_stuff_message::Message::SyncResponse(msg.into()) }, @@ -104,7 +105,7 @@ impl TryFrom for HotstuffMessage { HotstuffMessage::RequestedTransaction(msg.try_into()?) }, proto::consensus::hot_stuff_message::Message::SyncRequest(msg) => { - HotstuffMessage::SyncRequest(msg.try_into()?) + HotstuffMessage::CatchUpSyncRequest(msg.try_into()?) }, proto::consensus::hot_stuff_message::Message::SyncResponse(msg) => { HotstuffMessage::SyncResponse(msg.try_into()?) @@ -263,10 +264,11 @@ impl From<&tari_dan_storage::consensus_models::Block> for proto::consensus::Bloc total_leader_fee: value.total_leader_fee(), commands: value.commands().iter().map(Into::into).collect(), foreign_indexes: encode(value.foreign_indexes()).unwrap(), - signature: value.get_signature().map(Into::into), + signature: value.signature().map(Into::into), timestamp: value.timestamp(), base_layer_block_height: value.base_layer_block_height(), base_layer_block_hash: value.base_layer_block_hash().as_bytes().to_vec(), + is_dummy: value.is_dummy(), } } } @@ -279,31 +281,51 @@ impl TryFrom for tari_dan_storage::consensus_models::Bl .map_err(|_| anyhow!("Block conversion: Invalid network byte {}", value.network))? .try_into()?; - Ok(Self::new( - network, - value.parent_id.try_into()?, - value - .justify - .ok_or_else(|| anyhow!("Block conversion: QC not provided"))? - .try_into()?, - NodeHeight(value.height), - Epoch(value.epoch), - Shard::from(value.shard), - PublicKey::from_canonical_bytes(&value.proposed_by) - .map_err(|_| anyhow!("Block conversion: Invalid proposed_by"))?, - value - .commands - .into_iter() - .map(TryInto::try_into) - .collect::>()?, - value.merkle_root.try_into()?, - value.total_leader_fee, - decode_exact(&value.foreign_indexes)?, - value.signature.map(TryInto::try_into).transpose()?, - value.timestamp, - value.base_layer_block_height, - value.base_layer_block_hash.try_into()?, - )) + if value.is_dummy { + Ok(Self::dummy_block( + network, + value.parent_id.try_into()?, + PublicKey::from_canonical_bytes(&value.proposed_by) + .map_err(|_| anyhow!("Block conversion: Invalid proposed_by"))?, + NodeHeight(value.height), + value + .justify + .ok_or_else(|| anyhow!("Block conversion: QC not provided"))? + .try_into()?, + Epoch(value.epoch), + Shard::from(value.shard), + value.merkle_root.try_into()?, + value.timestamp, + value.base_layer_block_height, + value.base_layer_block_hash.try_into()?, + )) + } else { + Ok(Self::new( + network, + value.parent_id.try_into()?, + value + .justify + .ok_or_else(|| anyhow!("Block conversion: QC not provided"))? + .try_into()?, + NodeHeight(value.height), + Epoch(value.epoch), + Shard::from(value.shard), + PublicKey::from_canonical_bytes(&value.proposed_by) + .map_err(|_| anyhow!("Block conversion: Invalid proposed_by"))?, + value + .commands + .into_iter() + .map(TryInto::try_into) + .collect::>()?, + value.merkle_root.try_into()?, + value.total_leader_fee, + decode_exact(&value.foreign_indexes)?, + value.signature.map(TryInto::try_into).transpose()?, + value.timestamp, + value.base_layer_block_height, + value.base_layer_block_hash.try_into()?, + )) + } } } @@ -319,9 +341,7 @@ impl From<&Command> for proto::consensus::Command { Command::ForeignProposal(foreign_proposal) => { proto::consensus::command::Command::ForeignProposal(foreign_proposal.into()) }, - Command::EpochEvent(event) => { - proto::consensus::command::Command::EpochEvent(proto::consensus::EpochEvent::from(event).into()) - }, + Command::EndEpoch => proto::consensus::command::Command::EndEpoch(true), }; Self { command: Some(command) } @@ -341,11 +361,7 @@ impl TryFrom for Command { proto::consensus::command::Command::ForeignProposal(foreign_proposal) => { Command::ForeignProposal(foreign_proposal.try_into()?) }, - proto::consensus::command::Command::EpochEvent(event) => Command::EpochEvent( - proto::consensus::EpochEvent::try_from(event) - .map_err(|_| anyhow!("Invalid epoch event value {}", event))? - .try_into()?, - ), + proto::consensus::command::Command::EndEpoch(_) => Command::EndEpoch, }) } } @@ -435,7 +451,7 @@ impl TryFrom for ForeignProposalState { impl From<&ForeignProposal> for proto::consensus::ForeignProposal { fn from(value: &ForeignProposal) -> Self { Self { - bucket: value.bucket.as_u32(), + bucket: value.shard.as_u32(), block_id: value.block_id.as_bytes().to_vec(), state: proto::consensus::ForeignProposalState::from(value.state).into(), mined_at: value.proposed_height.map(|a| a.0).unwrap_or(0), @@ -450,7 +466,7 @@ impl TryFrom for ForeignProposal { fn try_from(value: proto::consensus::ForeignProposal) -> Result { Ok(ForeignProposal { - bucket: Shard::from(value.bucket), + shard: Shard::from(value.bucket), block_id: BlockId::try_from(value.block_id)?, state: proto::consensus::ForeignProposalState::try_from(value.state) .map_err(|_| anyhow!("Invalid foreign proposal state value {}", value.state))? @@ -470,29 +486,6 @@ impl TryFrom for ForeignProposal { } } -// ------------------------------- EpochEvent ------------------------------- // - -impl From<&EpochEvent> for proto::consensus::EpochEvent { - fn from(value: &EpochEvent) -> Self { - match value { - EpochEvent::Start => proto::consensus::EpochEvent::Start, - EpochEvent::End => proto::consensus::EpochEvent::End, - } - } -} - -impl TryFrom for EpochEvent { - type Error = anyhow::Error; - - fn try_from(value: proto::consensus::EpochEvent) -> Result { - match value { - proto::consensus::EpochEvent::Start => Ok(EpochEvent::Start), - proto::consensus::EpochEvent::End => Ok(EpochEvent::End), - proto::consensus::EpochEvent::UnknownEvent => Err(anyhow!("Epoch event not provided")), - } - } -} - // -------------------------------- Decision -------------------------------- // impl From for proto::consensus::Decision { @@ -626,6 +619,7 @@ impl TryFrom for SubstateRecord { created_height: NodeHeight(value.created_height), destroyed: value.destroyed.map(TryInto::try_into).transpose()?, + created_by_shard: Shard::from(value.created_by_shard), }) } } @@ -642,6 +636,7 @@ impl From for proto::consensus::Substate { created_block: value.created_block.as_bytes().to_vec(), created_height: value.created_height.as_u64(), created_epoch: value.created_at_epoch.as_u64(), + created_by_shard: value.created_by_shard.as_u32(), destroyed: value.destroyed.map(Into::into), } @@ -656,11 +651,12 @@ impl TryFrom for SubstateDestroyed { Ok(Self { by_transaction: value.transaction.try_into()?, justify: value.justify.try_into()?, - by_block: value.block.try_into()?, + by_block: NodeHeight(value.block_height), at_epoch: value .epoch .map(Into::into) .ok_or_else(|| anyhow!("Epoch not provided"))?, + by_shard: Shard::from(value.shard), }) } } @@ -670,8 +666,9 @@ impl From for proto::consensus::SubstateDestroyed { Self { transaction: value.by_transaction.as_bytes().to_vec(), justify: value.justify.as_bytes().to_vec(), - block: value.by_block.as_bytes().to_vec(), + block_height: value.by_block.as_u64(), epoch: Some(value.at_epoch.into()), + shard: value.by_shard.as_u32(), } } } @@ -685,6 +682,7 @@ impl From<&SyncRequestMessage> for proto::consensus::SyncRequest { high_qc: Some(proto::consensus::HighQc { block_id: value.high_qc.block_id.as_bytes().to_vec(), block_height: value.high_qc.block_height.as_u64(), + epoch: value.epoch.as_u64(), qc_id: value.high_qc.qc_id.as_bytes().to_vec(), }), } @@ -703,6 +701,7 @@ impl TryFrom for SyncRequestMessage { Ok::<_, anyhow::Error>(HighQc { block_id: BlockId::try_from(value.block_id)?, block_height: NodeHeight(value.block_height), + epoch: Epoch(value.epoch), qc_id: QcId::try_from(value.qc_id)?, }) }) diff --git a/dan_layer/p2p/src/conversions/rpc.rs b/dan_layer/p2p/src/conversions/rpc.rs index b4babeee8..3d8297465 100644 --- a/dan_layer/p2p/src/conversions/rpc.rs +++ b/dan_layer/p2p/src/conversions/rpc.rs @@ -4,7 +4,16 @@ use std::convert::{TryFrom, TryInto}; use anyhow::anyhow; -use tari_dan_storage::consensus_models::{SubstateCreatedProof, SubstateData, SubstateDestroyedProof, SubstateUpdate}; +use tari_dan_common_types::{shard::Shard, Epoch}; +use tari_dan_storage::consensus_models::{ + EpochCheckpoint, + StateTransition, + StateTransitionId, + SubstateCreatedProof, + SubstateData, + SubstateDestroyedProof, + SubstateUpdate, +}; use tari_engine_types::substate::{SubstateId, SubstateValue}; use crate::proto; @@ -111,3 +120,78 @@ impl From for proto::rpc::SubstateData { } } } + +//---------------------------------- StateTransition --------------------------------------------// + +impl TryFrom for StateTransition { + type Error = anyhow::Error; + + fn try_from(value: proto::rpc::StateTransition) -> Result { + let id = value + .id + .map(StateTransitionId::try_from) + .transpose()? + .ok_or_else(|| anyhow::anyhow!("StateTransitionId is missing"))?; + let update = value + .update + .ok_or_else(|| anyhow::anyhow!("Missing state transition update"))?; + let update = SubstateUpdate::try_from(update)?; + Ok(Self { + id, + update, + state_tree_version: value.state_tree_version, + }) + } +} + +impl From for proto::rpc::StateTransition { + fn from(value: StateTransition) -> Self { + Self { + id: Some(value.id.into()), + update: Some(value.update.into()), + state_tree_version: value.state_tree_version, + } + } +} + +//---------------------------------- StateTransitionId --------------------------------------------// + +impl TryFrom for StateTransitionId { + type Error = anyhow::Error; + + fn try_from(value: proto::rpc::StateTransitionId) -> Result { + Ok(Self::new(Epoch(value.epoch), Shard::from(value.shard), value.seq)) + } +} + +impl From for proto::rpc::StateTransitionId { + fn from(value: StateTransitionId) -> Self { + Self { + epoch: value.epoch().as_u64(), + shard: value.shard().as_u32(), + seq: value.seq(), + } + } +} + +//---------------------------------- EpochCheckpoint --------------------------------------------// + +impl TryFrom for EpochCheckpoint { + type Error = anyhow::Error; + + fn try_from(value: proto::rpc::EpochCheckpoint) -> Result { + Ok(Self::new( + value.block.ok_or_else(|| anyhow!("block not provided"))?.try_into()?, + value.qcs.into_iter().map(TryInto::try_into).collect::>()?, + )) + } +} + +impl From for proto::rpc::EpochCheckpoint { + fn from(value: EpochCheckpoint) -> Self { + Self { + block: Some(value.block().into()), + qcs: value.qcs().iter().map(Into::into).collect(), + } + } +} diff --git a/dan_layer/rpc_state_sync/src/error.rs b/dan_layer/rpc_state_sync/src/error.rs index 1a2a5fdd9..3e297d828 100644 --- a/dan_layer/rpc_state_sync/src/error.rs +++ b/dan_layer/rpc_state_sync/src/error.rs @@ -7,7 +7,7 @@ use tari_dan_storage::{ StorageError, }; use tari_epoch_manager::EpochManagerError; -use tari_rpc_framework::RpcError; +use tari_rpc_framework::{RpcError, RpcStatus}; use tari_validator_node_rpc::ValidatorNodeRpcClientError; #[derive(Debug, thiserror::Error)] @@ -39,3 +39,9 @@ impl From for HotStuffError { HotStuffError::SyncError(value.into()) } } + +impl From for CommsRpcConsensusSyncError { + fn from(value: RpcStatus) -> Self { + Self::RpcError(value.into()) + } +} diff --git a/dan_layer/rpc_state_sync/src/lib.rs b/dan_layer/rpc_state_sync/src/lib.rs index b36f424fa..eab22b19d 100644 --- a/dan_layer/rpc_state_sync/src/lib.rs +++ b/dan_layer/rpc_state_sync/src/lib.rs @@ -5,6 +5,7 @@ mod error; mod manager; +// mod manager_old; pub use error::*; pub use manager::*; diff --git a/dan_layer/rpc_state_sync/src/manager.rs b/dan_layer/rpc_state_sync/src/manager.rs index 5788d06ce..0fa1c34ba 100644 --- a/dan_layer/rpc_state_sync/src/manager.rs +++ b/dan_layer/rpc_state_sync/src/manager.rs @@ -1,44 +1,32 @@ // Copyright 2023 The Tari Project // SPDX-License-Identifier: BSD-3-Clause -use std::{ - collections::{HashMap, HashSet}, - fmt::Display, - ops::Deref, -}; +use std::collections::{HashMap, HashSet}; +use anyhow::anyhow; use async_trait::async_trait; use futures::StreamExt; use log::*; -use tari_common::configuration::Network; -use tari_consensus::{ - hotstuff::{calculate_state_merkle_diff, ProposalValidationError}, - traits::{ConsensusSpec, LeaderStrategy, SyncManager, SyncStatus}, -}; +use tari_consensus::traits::{ConsensusSpec, SyncManager, SyncStatus}; use tari_dan_common_types::{committee::Committee, optional::Optional, shard::Shard, Epoch, NodeHeight, PeerAddress}; -use tari_dan_p2p::proto::rpc::{GetHighQcRequest, SyncBlocksRequest}; +use tari_dan_p2p::proto::rpc::{GetCheckpointRequest, GetCheckpointResponse, SyncStateRequest}; use tari_dan_storage::{ consensus_models::{ Block, - BlockDiff, - BlockId, - HighQc, + EpochCheckpoint, LeafBlock, - LockedBlock, - PendingStateTreeDiff, - QuorumCertificate, - SubstateChange, + StateTransition, + SubstateCreatedProof, + SubstateDestroyedProof, + SubstateRecord, SubstateUpdate, - TransactionPoolRecord, - TransactionRecord, }, StateStore, + StateStoreWriteTransaction, + StorageError, }; -use tari_engine_types::substate::hash_substate; use tari_epoch_manager::EpochManagerReader; -use tari_rpc_framework::RpcError; -use tari_state_tree::SubstateTreeChange; -use tari_transaction::{Transaction, VersionedSubstateId}; +use tari_transaction::VersionedSubstateId; use tari_validator_node_rpc::{ client::{TariValidatorNodeRpcClientFactory, ValidatorNodeClientFactory}, rpc_service::ValidatorNodeRpcClient, @@ -48,13 +36,9 @@ use crate::error::CommsRpcConsensusSyncError; const LOG_TARGET: &str = "tari::dan::comms_rpc_state_sync"; -const MAX_SUBSTATE_UPDATES: usize = 10000; - pub struct RpcStateSyncManager { - network: Network, epoch_manager: TConsensusSpec::EpochManager, state_store: TConsensusSpec::StateStore, - leader_strategy: TConsensusSpec::LeaderStrategy, client_factory: TariValidatorNodeRpcClientFactory, } @@ -62,509 +46,196 @@ impl RpcStateSyncManager where TConsensusSpec: ConsensusSpec { pub fn new( - network: Network, epoch_manager: TConsensusSpec::EpochManager, state_store: TConsensusSpec::StateStore, - leader_strategy: TConsensusSpec::LeaderStrategy, client_factory: TariValidatorNodeRpcClientFactory, ) -> Self { Self { - network, epoch_manager, state_store, - leader_strategy, client_factory, } } - async fn get_sync_peers(&self) -> Result, CommsRpcConsensusSyncError> { - let current_epoch = self.epoch_manager.current_epoch().await?; - let this_vn = self.epoch_manager.get_our_validator_node(current_epoch).await?; - let mut committee = self.epoch_manager.get_local_committee(current_epoch).await?; - committee.members.retain(|(addr, _)| *addr != this_vn.address); - committee.shuffle(); - Ok(committee) - } - - async fn sync_with_peer( - &mut self, - addr: &TConsensusSpec::Addr, - locked_block: &LockedBlock, - up_to_epoch: Option, - ) -> Result<(), CommsRpcConsensusSyncError> { - self.create_zero_block_if_required()?; + async fn establish_rpc_session( + &self, + addr: &PeerAddress, + ) -> Result { let mut rpc_client = self.client_factory.create_client(addr); - let mut client = rpc_client.client_connection().await?; - - info!(target: LOG_TARGET, "🌐 Syncing blocks from peer '{}' from Locked block {}", addr, locked_block); - self.sync_blocks(&mut client, locked_block, up_to_epoch).await?; - - Ok(()) + let client = rpc_client.client_connection().await?; + Ok(client) } - fn create_zero_block_if_required(&self) -> Result<(), CommsRpcConsensusSyncError> { - self.state_store.with_write_tx(|tx| { - let zero_block = Block::zero_block(self.network); - if !zero_block.exists(&**tx)? { - debug!(target: LOG_TARGET, "Creating zero block"); - zero_block.justify().insert(tx)?; - zero_block.insert(tx)?; - zero_block.as_locked_block().set(tx)?; - zero_block.as_leaf_block().set(tx)?; - zero_block.as_last_executed().set(tx)?; - zero_block.as_last_voted().set(tx)?; - zero_block.justify().as_high_qc().set(tx)?; - zero_block.commit_diff(tx, BlockDiff::empty(*zero_block.id()))?; - } - Ok::<_, CommsRpcConsensusSyncError>(()) - })?; - - Ok(()) - } - - #[allow(clippy::too_many_lines)] - async fn sync_blocks( - &mut self, + async fn fetch_epoch_checkpoint( + &self, client: &mut ValidatorNodeRpcClient, - locked_block: &LockedBlock, - up_to_epoch: Option, - ) -> Result<(), CommsRpcConsensusSyncError> { - let mut stream = client - .sync_blocks(SyncBlocksRequest { - start_block_id: locked_block.block_id.as_bytes().to_vec(), - up_to_epoch: up_to_epoch.map(|epoch| epoch.into()), + current_epoch: Epoch, + ) -> Result, CommsRpcConsensusSyncError> { + match client + .get_checkpoint(GetCheckpointRequest { + current_epoch: current_epoch.as_u64(), }) - .await?; - - let mut counter = 0usize; - - let mut expected_height = locked_block.height + NodeHeight(1); - // Stores the uncommitted state updates for each block. When a block reaches a 3-chain, the updates are removed - // and applied. - let mut pending_state_updates = HashMap::new(); - - while let Some(resp) = stream.next().await { - let msg = resp.map_err(RpcError::from)?; - let new_block = msg.into_block().ok_or_else(|| { - CommsRpcConsensusSyncError::InvalidResponse(anyhow::anyhow!("Expected peer to return a newblock",)) - })?; - - let block = Block::try_from(new_block).map_err(CommsRpcConsensusSyncError::InvalidResponse)?; - if block.justifies_parent() && block.height() != expected_height { - return Err(CommsRpcConsensusSyncError::InvalidResponse(anyhow::anyhow!( - "Peer returned block at height {} but expected {}", - block.height(), - expected_height, - ))); - } - - let Some(resp) = stream.next().await else { - return Err(CommsRpcConsensusSyncError::InvalidResponse(anyhow::anyhow!( - "Peer closed session before sending QC message" - ))); - }; - let msg = resp.map_err(RpcError::from)?; - let qcs = msg.into_quorum_certificates().ok_or_else(|| { - CommsRpcConsensusSyncError::InvalidResponse(anyhow::anyhow!("Expected peer to return QCs")) - })?; - - let qcs = qcs - .into_iter() - .map(QuorumCertificate::try_from) - .collect::, _>>() - .map_err(CommsRpcConsensusSyncError::InvalidResponse)?; - - // TODO: Validate - - let Some(resp) = stream.next().await else { - return Err(CommsRpcConsensusSyncError::InvalidResponse(anyhow::anyhow!( - "Peer closed session before sending substate update count message" - ))); - }; - let msg = resp.map_err(RpcError::from)?; - let num_substates = msg.substate_count().ok_or_else(|| { - CommsRpcConsensusSyncError::InvalidResponse(anyhow::anyhow!("Expected peer to return substate count",)) - })? as usize; - - if num_substates > MAX_SUBSTATE_UPDATES { - return Err(CommsRpcConsensusSyncError::InvalidResponse(anyhow::anyhow!( - "Peer returned {} substate updates, but the maximum is {}", - num_substates, - MAX_SUBSTATE_UPDATES, - ))); - } - - let mut updates = Vec::with_capacity(num_substates); - for _ in 0..num_substates { - let Some(resp) = stream.next().await else { - return Err(CommsRpcConsensusSyncError::InvalidResponse(anyhow::anyhow!( - "Peer closed session before sending substate updates message" - ))); - }; - let msg = resp.map_err(RpcError::from)?; - let update = msg.into_substate_update().ok_or_else(|| { - CommsRpcConsensusSyncError::InvalidResponse(anyhow::anyhow!( - "Expected peer to return substate updates", - )) - })?; - - let update = SubstateUpdate::try_from(update).map_err(CommsRpcConsensusSyncError::InvalidResponse)?; - updates.push(update); - } - - let Some(resp) = stream.next().await else { - return Err(CommsRpcConsensusSyncError::InvalidResponse(anyhow::anyhow!( - "Peer closed session before sending transactions message" - ))); - }; - let msg = resp.map_err(RpcError::from)?; - let transactions = msg.into_transactions().ok_or_else(|| { - CommsRpcConsensusSyncError::InvalidResponse(anyhow::anyhow!("Expected peer to return QCs")) - })?; - - debug!(target: LOG_TARGET, "🌐 Received block {}, {} transactions", block, transactions.len()); - - let transactions = transactions - .into_iter() - .map(Transaction::try_from) - .map(|r| r.map(TransactionRecord::new)) - .collect::, _>>() - .map_err(CommsRpcConsensusSyncError::InvalidResponse)?; - - // TODO: Validate - debug!( - target: LOG_TARGET, - "🌐 Received block {}, {} qcs and {} substate updates", - block, - qcs.len(), - updates.len(), - ); - counter += 1; - if counter % 100 == 0 { - info!(target: LOG_TARGET, "🌐 Syncing block {block}"); - } - if block.justifies_parent() { - expected_height += NodeHeight(1); - } else { - expected_height = block.height() + NodeHeight(1); - } - self.process_block(block, qcs, updates, transactions, &mut pending_state_updates) - .await?; - } - - info!(target: LOG_TARGET, "🌐 {counter} blocks synced to height {}", expected_height - NodeHeight(1)); - - Ok(()) - } - - async fn process_block( - &mut self, - mut block: Block, - qcs: Vec, - updates: Vec, - transactions: Vec, - pending_state_updates: &mut HashMap>, - ) -> Result<(), CommsRpcConsensusSyncError> { - info!(target: LOG_TARGET, "🌐 Processing block {}. {} substate update(s)", block, updates.len()); - // Note: this is only used for dummy block calculation, so we avoid the epoch manager call unless it is needed. - // Otherwise, the committee is empty. - let local_committee = if block.justifies_parent() { - Committee::new(vec![]) - } else { - self.epoch_manager - .get_committees_by_shards(block.epoch(), vec![block.shard()].into_iter().collect()) - .await? - .into_iter() - .next() - .map(|(_, committee)| committee) - .unwrap_or_else(Committee::empty) - }; - - // TODO: Validate before we save anything. - self.state_store.with_write_tx(|tx| { - for transaction in transactions { - transaction.save(tx)?; - } - - block.justify().save(tx)?; - - let justify_block = block.justify().get_block(&**tx)?; - - // Check if we need to calculate dummy blocks - // TODO: Validate before doing this. e.g. block.height() is maliciously larger then block.justify().block_height() - if !block.justifies_parent() { - let mut last_dummy_block = BlockIdAndHeight {id: *block.justify().block_id(), height: block.justify().block_height()}; - info!(target: LOG_TARGET, "🍼 START DUMMY BLOCK: {}. ", last_dummy_block, ); - // if the block parent is not the justify parent, then we have experienced a leader failure - // and should make dummy blocks to fill in the gaps. - while last_dummy_block.id != *block.parent() { - if last_dummy_block.height >= block.height() { - warn!(target: LOG_TARGET, "πŸ”₯ Bad proposal, no dummy block parent hash matches between block height {} and new block height {}.", last_dummy_block, block); - return Err( ProposalValidationError::CandidateBlockDoesNotExtendJustify { - justify_block_height: block.justify().block_height(), - candidate_block_height: block.height(), - }.into()); - } - - let next_height = last_dummy_block.height + NodeHeight(1); - let leader = self.leader_strategy.get_leader_public_key(&local_committee, next_height); - - let dummy_block = Block::dummy_block( - self.network, - last_dummy_block.id, - leader.clone(), - next_height, - block.justify().clone(), - block.epoch(), - block.shard(), - *block.merkle_root(), - justify_block.timestamp(), - justify_block.base_layer_block_height(), - *justify_block.base_layer_block_hash(), - ); - dummy_block.save(tx)?; - last_dummy_block = BlockIdAndHeight { id: *dummy_block.id(), height: next_height }; - info!(target: LOG_TARGET, "🍼 DUMMY BLOCK: {}. Leader: {}", last_dummy_block, leader); - } - } - - if !block.is_safe(&**tx)? { - return Err(CommsRpcConsensusSyncError::BlockNotSafe { block_id: *block.id() }); - } - - if !block.save(tx)? { - // We've already seen this block. This could happen because we're syncing from high qc and we receive a - // block that we already have - return Ok(()); - } - - for qc in qcs { - qc.save(tx)?; - } - - self.check_and_update_state_merkle_tree(tx, &block, &updates)?; - - if !updates.is_empty() { - pending_state_updates.insert(*block.id(), updates); - } - - block.update_nodes( - tx, - |_, _, _| Ok(()), - |tx, _last_executed, block| { - debug!(target: LOG_TARGET, "Sync is committing block {}", block); - Self::commit_block(tx, block, pending_state_updates)?; - block.as_last_executed().set(tx)?; - Ok::<_, CommsRpcConsensusSyncError>(()) - }, - )?; - - // Ensure we don't vote on or re-process a synced block - block.as_last_voted().set(tx)?; - block.set_as_processed(tx)?; - - Ok(()) - }) - } - - fn check_and_update_state_merkle_tree( - &self, - tx: &mut ::WriteTransaction<'_>, - block: &Block, - updates: &[SubstateUpdate], - ) -> Result<(), CommsRpcConsensusSyncError> { - let pending_tree_updates = PendingStateTreeDiff::get_all_up_to_commit_block(&**tx, block.id())?; - let current_version = block.justify().block_height().as_u64(); - let next_version = block.height().as_u64(); - - let changes = updates.iter().map(|update| match update { - SubstateUpdate::Create(create) => SubstateTreeChange::Up { - id: create.substate.substate_id.clone(), - value_hash: hash_substate(&create.substate.substate_value, create.substate.version), + .await + { + Ok(GetCheckpointResponse { + checkpoint: Some(checkpoint), + }) => match EpochCheckpoint::try_from(checkpoint) { + Ok(cp) => Ok(Some(cp)), + Err(err) => Err(CommsRpcConsensusSyncError::InvalidResponse(err)), }, - SubstateUpdate::Destroy(destroy) => SubstateTreeChange::Down { - id: destroy.substate_id.clone(), - }, - }); - - let (root_hash, tree_diff) = - calculate_state_merkle_diff(tx.deref(), current_version, next_version, pending_tree_updates, changes)?; - - if root_hash != *block.merkle_root() { - return Err(CommsRpcConsensusSyncError::InvalidResponse(anyhow::anyhow!( - "Merkle root in block {} does not match the merkle root of the state tree. Block MR: {}, Calculated \ - MR: {}", - block, - block.merkle_root(), - root_hash - ))); + Ok(GetCheckpointResponse { checkpoint: None }) => Ok(None), + Err(err) => Err(err.into()), } - - // Persist pending state tree diff - PendingStateTreeDiff::new(*block.id(), block.height(), tree_diff).save(tx)?; - - Ok(()) } - fn commit_block( - tx: &mut ::WriteTransaction<'_>, - block: &Block, - pending_state_updates: &mut HashMap>, + async fn start_state_sync( + &self, + client: &mut ValidatorNodeRpcClient, + checkpoint: EpochCheckpoint, ) -> Result<(), CommsRpcConsensusSyncError> { - let block_diff = BlockDiff::new( - *block.id(), - pending_state_updates - .drain() - .flat_map(|(_, v)| v) - .map(substate_update_to_change) - .collect(), - ); - - block.commit_diff(tx, block_diff)?; + let current_epoch = self.epoch_manager.current_epoch().await?; + let committee_info = self.epoch_manager.get_local_committee_info(current_epoch).await?; - if block.is_dummy() { + let last_state_transition_id = self.state_store.with_read_tx(|tx| StateTransition::get_last_id(tx))?; + if current_epoch == last_state_transition_id.epoch() { + info!(target: LOG_TARGET, "πŸ›œAlready up to date. No need to sync."); return Ok(()); } - // Finalize any ACCEPTED transactions - for tx_atom in block.commands().iter().filter_map(|cmd| cmd.accept()) { - if let Some(mut transaction) = tx_atom.get_transaction(&**tx).optional()? { - transaction.final_decision = Some(tx_atom.decision); - if tx_atom.decision.is_abort() { - transaction.abort_details = Some("Abort decision via sync".to_string()); - } - // TODO: execution result - we should execute or we should get the execution result and verify state via - // sync - transaction.update(tx)?; - } - } - - // Remove from pool including any pending updates - TransactionPoolRecord::remove_any( - tx, - block.commands().iter().filter_map(|cmd| cmd.accept()).map(|t| &t.id), - )?; - - let diff = PendingStateTreeDiff::remove_by_block(tx, block.id())?; - let mut tree = tari_state_tree::SpreadPrefixStateTree::new(tx); - tree.commit_diff(diff.diff)?; + info!( + target: LOG_TARGET, + "πŸ›œSyncing from state transition {last_state_transition_id}" + ); - debug!(target: LOG_TARGET, "βœ… COMMIT block {}", block); - Ok(()) - } + let mut state_stream = client + .sync_state(SyncStateRequest { + start_epoch: last_state_transition_id.epoch().as_u64(), + start_shard: last_state_transition_id.shard().as_u32(), + start_seq: last_state_transition_id.seq(), + current_epoch: current_epoch.as_u64(), + current_shard: committee_info.shard().as_u32(), + }) + .await?; - async fn check_sync_from_committee( - &self, - committee: Committee, - ) -> Result { - if committee.is_empty() { - warn!(target: LOG_TARGET, "No peers available for sync"); - return Ok(SyncStatus::UpToDate); - } - let mut highest_qc: Option = None; - let mut num_succeeded = 0; - let max_failures = committee.max_failures(); - let committee_size = committee.len(); - for addr in committee.addresses() { - let mut rpc_client = self.client_factory.create_client(addr); - let mut client = match rpc_client.client_connection().await { - Ok(client) => client, - Err(err) => { - warn!(target: LOG_TARGET, "Failed to connect to peer {}: {}", addr, err); - continue; + while let Some(result) = state_stream.next().await { + let msg = match result { + Ok(msg) => msg, + Err(err) if err.is_not_found() => { + return Ok(()); }, - }; - let result = client - .get_high_qc(GetHighQcRequest {}) - .await - .map_err(CommsRpcConsensusSyncError::RpcError) - .and_then(|resp| { - resp.high_qc - .map(QuorumCertificate::try_from) - .transpose() - .map_err(CommsRpcConsensusSyncError::InvalidResponse)? - .ok_or_else(|| { - CommsRpcConsensusSyncError::InvalidResponse(anyhow::anyhow!( - "Peer returned an empty high qc" - )) - }) - }); - let remote_high_qc = match result { - Ok(resp) => resp, Err(err) => { - warn!("Failed to get high qc from peer {}: {}", addr, err); - continue; + return Err(err.into()); }, }; - num_succeeded += 1; - if highest_qc - .as_ref() - .map(|qc| qc.block_height() < remote_high_qc.block_height()) - .unwrap_or(true) - { - // TODO: validate + info!(target: LOG_TARGET, "πŸ›œ Next state updates batch of size {}", msg.transitions.len()); + + self.state_store.with_write_tx(|tx| { + for transition in msg.transitions { + let transition = + StateTransition::try_from(transition).map_err(CommsRpcConsensusSyncError::InvalidResponse)?; + info!(target: LOG_TARGET, "πŸ›œ Applied state update {transition}"); + if transition.id.epoch() >= current_epoch { + return Err(CommsRpcConsensusSyncError::InvalidResponse(anyhow!( + "Received state transition for epoch {} which is at or ahead of our current epoch {}.", + transition.id.epoch(), + current_epoch + ))); + } - highest_qc = Some(remote_high_qc); - } + self.commit_update(tx, &checkpoint, transition)?; + } - if num_succeeded == max_failures { - break; - } + // let current_version = block.justify().block_height().as_u64(); + // let next_version = block.height().as_u64(); + // + // let changes = updates.iter().map(|update| match update { + // SubstateUpdate::Create(create) => SubstateTreeChange::Up { + // id: create.substate.substate_id.clone(), + // value_hash: hash_substate(&create.substate.substate_value, create.substate.version), + // }, + // SubstateUpdate::Destroy(destroy) => SubstateTreeChange::Down { + // id: destroy.substate_id.clone(), + // }, + // }); + // + // let mut store = ChainScopedTreeStore::new(epoch, shard, tx); + // let mut tree = tari_state_tree::SpreadPrefixStateTree::new(&mut store); + // let _state_root = tree.put_substate_changes(current_version, next_version, changes)?; + + Ok::<_, CommsRpcConsensusSyncError>(()) + })?; } - let Some(highest_qc) = highest_qc else { - return Err(CommsRpcConsensusSyncError::NoPeersAvailable { committee_size }); - }; + Ok(()) + } - let local_high_qc = self.state_store.with_read_tx(|tx| HighQc::get(tx).optional())?; - let local_height = local_high_qc - .as_ref() - .map(|qc| qc.block_height()) - .unwrap_or(NodeHeight(0)); - if highest_qc.block_height() > local_height { - info!( - target: LOG_TARGET, - "Highest QC from peers is at height {} and local high QC is at height {}", - highest_qc.block_height(), - local_height, - ); - return Ok(SyncStatus::Behind); + pub fn commit_update( + &self, + tx: &mut TTx, + checkpoint: &EpochCheckpoint, + transition: StateTransition, + ) -> Result<(), StorageError> { + match transition.update { + SubstateUpdate::Create(SubstateCreatedProof { substate, created_qc }) => { + SubstateRecord::new( + substate.substate_id, + substate.version, + substate.substate_value, + transition.id.shard(), + transition.id.epoch(), + NodeHeight(0), + *checkpoint.block().id(), + substate.created_by_transaction, + *created_qc.id(), + ) + .create(tx)?; + }, + SubstateUpdate::Destroy(SubstateDestroyedProof { + substate_id, + version, + justify, + destroyed_by_transaction, + }) => { + SubstateRecord::destroy( + tx, + VersionedSubstateId::new(substate_id, version), + transition.id.shard(), + transition.id.epoch(), + // TODO + checkpoint.block().height(), + justify.id(), + &destroyed_by_transaction, + )?; + }, } - - Ok(SyncStatus::UpToDate) + Ok(()) } - async fn sync_from_committee( - &mut self, - committee: Committee, - up_to_epoch: Option, - this_vn_address: PeerAddress, - ) -> Result, CommsRpcConsensusSyncError> { - let mut sync_error = None; - for member in committee.addresses() { - if *member == this_vn_address { - continue; - } - // Refresh the HighQC each time because a partial sync could have been achieved from a peer - let locked_block = self - .state_store - .with_read_tx(|tx| LockedBlock::get(tx).optional())? - .unwrap_or_else(|| Block::zero_block(self.network).as_locked_block()); - - match self.sync_with_peer(member, &locked_block, up_to_epoch).await { - Ok(()) => { - sync_error = None; - break; - }, - Err(err) => { - warn!(target: LOG_TARGET, "Failed to sync with peer {}: {}", member, err); - sync_error = Some(err); - continue; - }, - } - } - Ok(sync_error) + async fn get_sync_committees( + &self, + current_epoch: Epoch, + ) -> Result>, CommsRpcConsensusSyncError> { + // We are behind at least one epoch. + // We get the current substate range, and we asks committees from previous epoch in this range to give us + // data. + let local_shard = self.epoch_manager.get_local_committee_info(current_epoch).await?; + let range = local_shard.to_substate_address_range(); + let prev_epoch = current_epoch.saturating_sub(Epoch(1)); + info!(target: LOG_TARGET,"Previous epoch is {}", prev_epoch); + let prev_num_committee = self.epoch_manager.get_num_committees(prev_epoch).await?; + info!(target: LOG_TARGET,"Previous num committee {}", prev_num_committee); + let start = range.start().to_shard(prev_num_committee); + let end = range.end().to_shard(prev_num_committee); + info!(target: LOG_TARGET,"Start: {}, End: {}", start, end); + let committees = self + .epoch_manager + .get_committees_by_shards( + prev_epoch, + (start.as_u32()..=end.as_u32()).map(Shard::from).collect::>(), + ) + .await?; + Ok(committees) } } @@ -575,142 +246,87 @@ where TConsensusSpec: ConsensusSpec + Send + Sync + 'static type Error = CommsRpcConsensusSyncError; async fn check_sync(&self) -> Result { - let leaf_block = self.state_store.with_read_tx(|tx| LeafBlock::get(tx).optional())?; - let leaf_epoch = match leaf_block { - Some(leaf_block) => { - let block = self - .state_store - .with_read_tx(|tx| Block::get(tx, leaf_block.block_id()))?; - block.epoch() - }, - None => Epoch(0), - }; let current_epoch = self.epoch_manager.current_epoch().await?; + + let leaf_epoch = self.state_store.with_read_tx(|tx| { + let epoch = LeafBlock::get(tx) + .optional()? + .map(|leaf| Block::get(tx, leaf.block_id())) + .transpose()? + .map(|b| b.epoch()) + .unwrap_or(Epoch(0)); + Ok::<_, Self::Error>(epoch) + })?; + + // We only sync if we're behind by an epoch. The current epoch is replayed in consensus. if current_epoch > leaf_epoch { - info!(target: LOG_TARGET, "We are behind at least one epoch"); - // We are behind at least one epoch. - // We get the current substate range, and we asks committees from previous epoch in this range to give us - // data. - let local_shard = self.epoch_manager.get_local_committee_info(current_epoch).await?; - let current_num_committee = self.epoch_manager.get_num_committees(current_epoch).await?; - let range = local_shard.shard().to_substate_address_range(current_num_committee); - let prev_epoch = current_epoch.saturating_sub(Epoch(1)); - info!(target: LOG_TARGET,"Previous epoch is {}", prev_epoch); - let prev_num_committee = self.epoch_manager.get_num_committees(prev_epoch).await?; - info!(target: LOG_TARGET,"Previous num committee {}", prev_num_committee); - let start = range.start().to_shard(prev_num_committee); - let end = range.end().to_shard(prev_num_committee); - info!(target: LOG_TARGET,"Start: {}, End: {}", start, end); - let this_vn = self.epoch_manager.get_our_validator_node(current_epoch).await?; - let committees = self - .epoch_manager - .get_committees_by_shards( - prev_epoch, - (start.as_u32()..=end.as_u32()).map(Shard::from).collect::>(), - ) - .await?; - for (shard, mut committee) in committees { - info!(target: LOG_TARGET, "Syncing shard {} from previous epoch. Committee : {:?}", shard, committee); - committee.members.retain(|(addr, _)| *addr != this_vn.address); - committee.shuffle(); - if self.check_sync_from_committee(committee).await? == SyncStatus::Behind { - return Ok(SyncStatus::Behind); - } - } + info!(target: LOG_TARGET, "πŸ›œOur current leaf block is behind the current epoch. Syncing..."); + return Ok(SyncStatus::Behind); } - let committee: Committee = self.get_sync_peers().await?; - self.check_sync_from_committee(committee).await + Ok(SyncStatus::UpToDate) } async fn sync(&mut self) -> Result<(), Self::Error> { - info!(target: LOG_TARGET, "Syncing"); - let mut sync_error = None; - let leaf_block = self.state_store.with_read_tx(|tx| LeafBlock::get(tx).optional())?; - let leaf_epoch = match leaf_block { - Some(leaf_block) => { - let block = self - .state_store - .with_read_tx(|tx| Block::get(tx, leaf_block.block_id()))?; - block.epoch() - }, - None => Epoch(0), - }; let current_epoch = self.epoch_manager.current_epoch().await?; - let this_vn = self.epoch_manager.get_our_validator_node(current_epoch).await?; - if current_epoch > leaf_epoch { - info!(target: LOG_TARGET, "We are behind at least one epoch..sync"); - // We are behind at least one epoch. - // We get the current substate range, and we asks committees from previous epoch in this range to give us - // data. - let local_shard = self.epoch_manager.get_local_committee_info(current_epoch).await?; - let current_num_committee = self.epoch_manager.get_num_committees(current_epoch).await?; - let range = local_shard.shard().to_substate_address_range(current_num_committee); - let prev_epoch = current_epoch.saturating_sub(Epoch(1)); - info!(target: LOG_TARGET,"Previous epoch is {}", prev_epoch); - let prev_num_committee = self.epoch_manager.get_num_committees(prev_epoch).await?; - info!(target: LOG_TARGET,"Previous num committee {}", prev_num_committee); - let start = range.start().to_shard(prev_num_committee); - let end = range.end().to_shard(prev_num_committee); - info!(target: LOG_TARGET,"Start: {}, End: {}", start, end); - let committees = self - .epoch_manager - .get_committees_by_shards( - prev_epoch, - (start.as_u32()..=end.as_u32()).map(Shard::from).collect::>(), - ) - .await?; - for (_shard, committee) in committees { - info!(target:LOG_TARGET,"Syncing from committee {:?}",committee); - if let Some(error) = self - .sync_from_committee(committee, Some(current_epoch), this_vn.address) - .await? - { - sync_error = Some(error); + let prev_epoch_committees = self.get_sync_committees(current_epoch).await?; + let our_vn = self.epoch_manager.get_our_validator_node(current_epoch).await?; + + let mut last_error = None; + // Sync data from each committee in range of the committee we're joining. + // NOTE: we don't have to worry about substates in address range because shard boundaries are fixed. + for (shard, mut committee) in prev_epoch_committees { + info!(target: LOG_TARGET, "πŸ›œSyncing state for shard {shard} for epoch {}", current_epoch.saturating_sub(Epoch(1))); + committee.shuffle(); + for (addr, public_key) in committee { + if our_vn.public_key == public_key { + continue; } - } - } - - let committee = self.get_sync_peers().await?; - if committee.is_empty() { - warn!(target: LOG_TARGET, "No peers available for sync"); - return Ok(()); - } + let mut client = match self.establish_rpc_session(&addr).await { + Ok(c) => c, + Err(err) => { + warn!( + target: LOG_TARGET, + "Failed to establish RPC session with vn {addr}: {err}. Attempting another VN if available" + ); + last_error = Some(err); + continue; + }, + }; - if let Some(error) = self.sync_from_committee(committee, None, this_vn.address).await? { - sync_error = Some(error); - } + let checkpoint = match self.fetch_epoch_checkpoint(&mut client, current_epoch).await { + Ok(Some(cp)) => cp, + Ok(None) => { + // EDGE-CASE: This may occur because the previous epoch had not started consensus, typically in + // testing cases where transactions + warn!( + target: LOG_TARGET, + "❓No checkpoint for epoch {current_epoch}. This may mean that this is the first epoch in the network" + ); + return Ok(()); + }, + Err(err) => { + warn!( + target: LOG_TARGET, + "⚠️Failed to fetch checkpoint from {addr}: {err}. Attempting another peer if available" + ); + last_error = Some(err); + continue; + }, + }; + info!(target: LOG_TARGET, "πŸ›œ Checkpoint: {checkpoint}"); - if let Some(err) = sync_error { - return Err(err); + if let Err(err) = self.start_state_sync(&mut client, checkpoint).await { + warn!( + target: LOG_TARGET, + "⚠️Failed to sync state from {addr}: {err}. Attempting another peer if available" + ); + last_error = Some(err); + continue; + } + } } - Ok(()) - } -} - -struct BlockIdAndHeight { - id: BlockId, - height: NodeHeight, -} - -impl Display for BlockIdAndHeight { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "Block: {} (#{})", self.id, self.height) - } -} - -// TODO: these are similar structures. Clean this up. -fn substate_update_to_change(update: SubstateUpdate) -> SubstateChange { - match update { - SubstateUpdate::Create(create) => SubstateChange::Up { - id: VersionedSubstateId::new(create.substate.substate_id.clone(), create.substate.version), - transaction_id: create.substate.created_by_transaction, - substate: create.substate.into_substate(), - }, - SubstateUpdate::Destroy(destroy) => SubstateChange::Down { - id: VersionedSubstateId::new(destroy.substate_id, destroy.version), - transaction_id: destroy.destroyed_by_transaction, - }, + last_error.map(Err).unwrap_or(Ok(())) } } diff --git a/dan_layer/rpc_state_sync/src/manager_old.rs b/dan_layer/rpc_state_sync/src/manager_old.rs new file mode 100644 index 000000000..1a1c1cbc1 --- /dev/null +++ b/dan_layer/rpc_state_sync/src/manager_old.rs @@ -0,0 +1,717 @@ +// Copyright 2023 The Tari Project +// SPDX-License-Identifier: BSD-3-Clause + +use std::{ + collections::{HashMap, HashSet}, + fmt::Display, + ops::Deref, +}; + +use async_trait::async_trait; +use futures::StreamExt; +use log::*; +use tari_common::configuration::Network; +use tari_consensus::{ + hotstuff::{calculate_state_merkle_diff, ProposalValidationError}, + traits::{ConsensusSpec, LeaderStrategy, SyncManager, SyncStatus}, +}; +use tari_dan_common_types::{committee::Committee, optional::Optional, shard::Shard, Epoch, NodeHeight, PeerAddress}; +use tari_dan_p2p::proto::rpc::{GetHighQcRequest, SyncBlocksRequest}; +use tari_dan_storage::{ + consensus_models::{ + Block, + BlockDiff, + BlockId, + HighQc, + LeafBlock, + LockedBlock, + PendingStateTreeDiff, + QuorumCertificate, + SubstateChange, + SubstateUpdate, + TransactionPoolRecord, + TransactionRecord, + }, + StateStore, +}; +use tari_engine_types::substate::hash_substate; +use tari_epoch_manager::EpochManagerReader; +use tari_rpc_framework::RpcError; +use tari_state_tree::SubstateTreeChange; +use tari_transaction::{Transaction, VersionedSubstateId}; +use tari_validator_node_rpc::{ + client::{TariValidatorNodeRpcClientFactory, ValidatorNodeClientFactory}, + rpc_service::ValidatorNodeRpcClient, +}; + +use crate::error::CommsRpcConsensusSyncError; + +const LOG_TARGET: &str = "tari::dan::comms_rpc_state_sync"; + +const MAX_SUBSTATE_UPDATES: usize = 10000; + +pub struct RpcStateSyncManager { + network: Network, + epoch_manager: TConsensusSpec::EpochManager, + state_store: TConsensusSpec::StateStore, + leader_strategy: TConsensusSpec::LeaderStrategy, + client_factory: TariValidatorNodeRpcClientFactory, +} + +impl RpcStateSyncManager +where TConsensusSpec: ConsensusSpec +{ + pub fn new( + network: Network, + epoch_manager: TConsensusSpec::EpochManager, + state_store: TConsensusSpec::StateStore, + leader_strategy: TConsensusSpec::LeaderStrategy, + client_factory: TariValidatorNodeRpcClientFactory, + ) -> Self { + Self { + network, + epoch_manager, + state_store, + leader_strategy, + client_factory, + } + } + + async fn get_sync_peers(&self) -> Result, CommsRpcConsensusSyncError> { + let current_epoch = self.epoch_manager.current_epoch().await?; + let this_vn = self.epoch_manager.get_our_validator_node(current_epoch).await?; + let mut committee = self.epoch_manager.get_local_committee(current_epoch).await?; + committee.members.retain(|(addr, _)| *addr != this_vn.address); + committee.shuffle(); + Ok(committee) + } + + async fn sync_with_peer( + &mut self, + addr: &TConsensusSpec::Addr, + locked_block: &LockedBlock, + up_to_epoch: Option, + ) -> Result<(), CommsRpcConsensusSyncError> { + self.create_zero_block_if_required()?; + let mut rpc_client = self.client_factory.create_client(addr); + let mut client = rpc_client.client_connection().await?; + + info!(target: LOG_TARGET, "🌐 Syncing blocks from peer '{}' from Locked block {}", addr, locked_block); + self.sync_blocks(&mut client, locked_block, up_to_epoch).await?; + + Ok(()) + } + + fn create_zero_block_if_required(&self) -> Result<(), CommsRpcConsensusSyncError> { + self.state_store.with_write_tx(|tx| { + let zero_block = Block::zero_block(self.network); + if !zero_block.exists(&**tx)? { + debug!(target: LOG_TARGET, "Creating zero block"); + zero_block.justify().insert(tx)?; + zero_block.insert(tx)?; + zero_block.as_locked_block().set(tx)?; + zero_block.as_leaf_block().set(tx)?; + zero_block.as_last_executed().set(tx)?; + zero_block.as_last_voted().set(tx)?; + zero_block.justify().as_high_qc().set(tx)?; + zero_block.commit_diff(tx, BlockDiff::empty(*zero_block.id()))?; + } + Ok::<_, CommsRpcConsensusSyncError>(()) + })?; + + Ok(()) + } + + #[allow(clippy::too_many_lines)] + async fn sync_blocks( + &mut self, + client: &mut ValidatorNodeRpcClient, + locked_block: &LockedBlock, + up_to_epoch: Option, + ) -> Result<(), CommsRpcConsensusSyncError> { + let mut stream = client + .sync_blocks(SyncBlocksRequest { + start_block_id: locked_block.block_id.as_bytes().to_vec(), + up_to_epoch: up_to_epoch.map(|epoch| epoch.into()), + }) + .await?; + + let mut counter = 0usize; + + let mut expected_height = locked_block.height + NodeHeight(1); + // Stores the uncommitted state updates for each block. When a block reaches a 3-chain, the updates are removed + // and applied. + let mut pending_state_updates = HashMap::new(); + + while let Some(resp) = stream.next().await { + let msg = resp.map_err(RpcError::from)?; + let new_block = msg.into_block().ok_or_else(|| { + CommsRpcConsensusSyncError::InvalidResponse(anyhow::anyhow!("Expected peer to return a newblock",)) + })?; + + let block = Block::try_from(new_block).map_err(CommsRpcConsensusSyncError::InvalidResponse)?; + if block.justifies_parent() && block.height() != expected_height { + return Err(CommsRpcConsensusSyncError::InvalidResponse(anyhow::anyhow!( + "Peer returned block at height {} but expected {}", + block.height(), + expected_height, + ))); + } + + let Some(resp) = stream.next().await else { + return Err(CommsRpcConsensusSyncError::InvalidResponse(anyhow::anyhow!( + "Peer closed session before sending QC message" + ))); + }; + let msg = resp.map_err(RpcError::from)?; + let qcs = msg.into_quorum_certificates().ok_or_else(|| { + CommsRpcConsensusSyncError::InvalidResponse(anyhow::anyhow!("Expected peer to return QCs")) + })?; + + let qcs = qcs + .into_iter() + .map(QuorumCertificate::try_from) + .collect::, _>>() + .map_err(CommsRpcConsensusSyncError::InvalidResponse)?; + + // TODO: Validate + + let Some(resp) = stream.next().await else { + return Err(CommsRpcConsensusSyncError::InvalidResponse(anyhow::anyhow!( + "Peer closed session before sending substate update count message" + ))); + }; + let msg = resp.map_err(RpcError::from)?; + let num_substates = msg.substate_count().ok_or_else(|| { + CommsRpcConsensusSyncError::InvalidResponse(anyhow::anyhow!("Expected peer to return substate count",)) + })? as usize; + + if num_substates > MAX_SUBSTATE_UPDATES { + return Err(CommsRpcConsensusSyncError::InvalidResponse(anyhow::anyhow!( + "Peer returned {} substate updates, but the maximum is {}", + num_substates, + MAX_SUBSTATE_UPDATES, + ))); + } + + let mut updates = Vec::with_capacity(num_substates); + for _ in 0..num_substates { + let Some(resp) = stream.next().await else { + return Err(CommsRpcConsensusSyncError::InvalidResponse(anyhow::anyhow!( + "Peer closed session before sending substate updates message" + ))); + }; + let msg = resp.map_err(RpcError::from)?; + let update = msg.into_substate_update().ok_or_else(|| { + CommsRpcConsensusSyncError::InvalidResponse(anyhow::anyhow!( + "Expected peer to return substate updates", + )) + })?; + + let update = SubstateUpdate::try_from(update).map_err(CommsRpcConsensusSyncError::InvalidResponse)?; + updates.push(update); + } + + let Some(resp) = stream.next().await else { + return Err(CommsRpcConsensusSyncError::InvalidResponse(anyhow::anyhow!( + "Peer closed session before sending transactions message" + ))); + }; + let msg = resp.map_err(RpcError::from)?; + let transactions = msg.into_transactions().ok_or_else(|| { + CommsRpcConsensusSyncError::InvalidResponse(anyhow::anyhow!("Expected peer to return QCs")) + })?; + + debug!(target: LOG_TARGET, "🌐 Received block {}, {} transactions", block, transactions.len()); + + let transactions = transactions + .into_iter() + .map(Transaction::try_from) + .map(|r| r.map(TransactionRecord::new)) + .collect::, _>>() + .map_err(CommsRpcConsensusSyncError::InvalidResponse)?; + + // TODO: Validate + debug!( + target: LOG_TARGET, + "🌐 Received block {}, {} qcs and {} substate updates", + block, + qcs.len(), + updates.len(), + ); + counter += 1; + if counter % 100 == 0 { + info!(target: LOG_TARGET, "🌐 Syncing block {block}"); + } + if block.justifies_parent() { + expected_height += NodeHeight(1); + } else { + expected_height = block.height() + NodeHeight(1); + } + self.process_block(block, qcs, updates, transactions, &mut pending_state_updates) + .await?; + } + + info!(target: LOG_TARGET, "🌐 {counter} blocks synced to height {}", expected_height - NodeHeight(1)); + + Ok(()) + } + + async fn process_block( + &mut self, + mut block: Block, + qcs: Vec, + updates: Vec, + transactions: Vec, + pending_state_updates: &mut HashMap>, + ) -> Result<(), CommsRpcConsensusSyncError> { + info!(target: LOG_TARGET, "🌐 Processing block {}. {} substate update(s)", block, updates.len()); + // Note: this is only used for dummy block calculation, so we avoid the epoch manager call unless it is needed. + // Otherwise, the committee is empty. + let local_committee = if block.justifies_parent() { + Committee::new(vec![]) + } else { + self.epoch_manager + .get_committees_by_shards(block.epoch(), vec![block.shard()].into_iter().collect()) + .await? + .into_iter() + .next() + .map(|(_, committee)| committee) + .unwrap_or_else(Committee::empty) + }; + + // TODO: Validate before we save anything. + self.state_store.with_write_tx(|tx| { + for transaction in transactions { + transaction.save(tx)?; + } + + block.justify().save(tx)?; + + let justify_block = block.justify().get_block(&**tx)?; + + // Check if we need to calculate dummy blocks + // TODO: Validate before doing this. e.g. block.height() is maliciously larger then block.justify().block_height() + if !block.justifies_parent() { + let mut last_dummy_block = BlockIdAndHeight { id: *block.justify().block_id(), height: block.justify().block_height() }; + info!(target: LOG_TARGET, "🍼 START DUMMY BLOCK: {}. ", last_dummy_block, ); + // if the block parent is not the justify parent, then we have experienced a leader failure + // and should make dummy blocks to fill in the gaps. + while last_dummy_block.id != *block.parent() { + if last_dummy_block.height >= block.height() { + warn!(target: LOG_TARGET, "πŸ”₯ Bad proposal, no dummy block parent hash matches between block height {} and new block height {}.", last_dummy_block, block); + return Err( ProposalValidationError::CandidateBlockDoesNotExtendJustify { + justify_block_height: block.justify().block_height(), + candidate_block_height: block.height(), + }.into()); + } + + let next_height = last_dummy_block.height + NodeHeight(1); + let leader = self.leader_strategy.get_leader_public_key(&local_committee, next_height); + + let dummy_block = Block::dummy_block( + self.network, + last_dummy_block.id, + leader.clone(), + next_height, + block.justify().clone(), + block.epoch(), + block.shard(), + *block.merkle_root(), + justify_block.timestamp(), + justify_block.base_layer_block_height(), + *justify_block.base_layer_block_hash(), + ); + dummy_block.save(tx)?; + last_dummy_block = BlockIdAndHeight { id: *dummy_block.id(), height: next_height }; + info!(target: LOG_TARGET, "🍼 DUMMY BLOCK: {}. Leader: {}", last_dummy_block, leader); + } + } + + if !block.is_safe(&**tx)? { + return Err(CommsRpcConsensusSyncError::BlockNotSafe { block_id: *block.id() }); + } + + if !block.save(tx)? { + // We've already seen this block. This could happen because we're syncing from high qc and we receive a + // block that we already have + return Ok(()); + } + + for qc in qcs { + qc.save(tx)?; + } + + self.check_and_update_state_merkle_tree(tx, &block, &updates)?; + + if !updates.is_empty() { + pending_state_updates.insert(*block.id(), updates); + } + + block.update_nodes( + tx, + |_, _, _| Ok(()), + |tx, _last_executed, block| { + debug!(target: LOG_TARGET, "Sync is committing block {block}"); + Self::commit_block(tx, block, pending_state_updates)?; + block.as_last_executed().set(tx)?; + Ok::<_, CommsRpcConsensusSyncError>(()) + }, + )?; + + // Ensure we don't vote on or re-process a synced block + block.as_last_voted().set(tx)?; + block.set_as_processed(tx)?; + + Ok(()) + }) + } + + fn check_and_update_state_merkle_tree( + &self, + tx: &mut ::WriteTransaction<'_>, + block: &Block, + updates: &[SubstateUpdate], + ) -> Result<(), CommsRpcConsensusSyncError> { + let pending_tree_updates = + PendingStateTreeDiff::get_all_up_to_commit_block(&**tx, block.epoch(), block.shard(), block.id())?; + let current_version = block.justify().block_height().as_u64(); + let next_version = block.height().as_u64(); + + let changes = updates.iter().map(|update| match update { + SubstateUpdate::Create(create) => SubstateTreeChange::Up { + id: create.substate.substate_id.clone(), + value_hash: hash_substate(&create.substate.substate_value, create.substate.version), + }, + SubstateUpdate::Destroy(destroy) => SubstateTreeChange::Down { + id: destroy.substate_id.clone(), + }, + }); + + let (root_hash, tree_diff) = + calculate_state_merkle_diff(tx.deref(), current_version, next_version, pending_tree_updates, changes)?; + + if root_hash != *block.merkle_root() { + return Err(CommsRpcConsensusSyncError::InvalidResponse(anyhow::anyhow!( + "Merkle root in block {} does not match the merkle root of the state tree. Block MR: {}, Calculated \ + MR: {}", + block, + block.merkle_root(), + root_hash + ))); + } + + // Persist pending state tree diff + PendingStateTreeDiff::new(*block.id(), block.height(), tree_diff).save(tx)?; + + Ok(()) + } + + fn commit_block( + tx: &mut ::WriteTransaction<'_>, + block: &Block, + pending_state_updates: &mut HashMap>, + ) -> Result<(), CommsRpcConsensusSyncError> { + let block_diff = BlockDiff::new( + *block.id(), + pending_state_updates + .drain() + .flat_map(|(_, v)| v) + .map(substate_update_to_change) + .collect(), + ); + + block.commit_diff(tx, block_diff)?; + + if block.is_dummy() { + return Ok(()); + } + + // Finalize any ACCEPTED transactions + for tx_atom in block.commands().iter().filter_map(|cmd| cmd.accept()) { + if let Some(mut transaction) = tx_atom.get_transaction(&**tx).optional()? { + transaction.final_decision = Some(tx_atom.decision); + if tx_atom.decision.is_abort() { + transaction.abort_details = Some("Abort decision via sync".to_string()); + } + // TODO: execution result - we should execute or we should get the execution result and verify state via + // sync + transaction.update(tx)?; + } + } + + // Remove from pool including any pending updates + TransactionPoolRecord::remove_any( + tx, + block.commands().iter().filter_map(|cmd| cmd.accept()).map(|t| &t.id), + )?; + + let diff = PendingStateTreeDiff::remove_by_block(tx, block.id())?; + let mut tree = tari_state_tree::SpreadPrefixStateTree::new(tx); + tree.commit_diff(diff.diff)?; + + debug!(target: LOG_TARGET, "βœ… COMMIT block {}", block); + Ok(()) + } + + async fn check_sync_from_committee( + &self, + committee: Committee, + ) -> Result { + if committee.is_empty() { + warn!(target: LOG_TARGET, "No peers available for sync"); + return Ok(SyncStatus::UpToDate); + } + let mut highest_qc: Option = None; + let mut num_succeeded = 0; + let max_failures = committee.max_failures(); + let committee_size = committee.len(); + for addr in committee.addresses() { + let mut rpc_client = self.client_factory.create_client(addr); + let mut client = match rpc_client.client_connection().await { + Ok(client) => client, + Err(err) => { + warn!(target: LOG_TARGET, "Failed to connect to peer {}: {}", addr, err); + continue; + }, + }; + let result = client + .get_high_qc(GetHighQcRequest {}) + .await + .map_err(CommsRpcConsensusSyncError::RpcError) + .and_then(|resp| { + resp.high_qc + .map(QuorumCertificate::try_from) + .transpose() + .map_err(CommsRpcConsensusSyncError::InvalidResponse)? + .ok_or_else(|| { + CommsRpcConsensusSyncError::InvalidResponse(anyhow::anyhow!( + "Peer returned an empty high qc" + )) + }) + }); + let remote_high_qc = match result { + Ok(resp) => resp, + Err(err) => { + warn!("Failed to get high qc from peer {}: {}", addr, err); + continue; + }, + }; + + num_succeeded += 1; + if highest_qc + .as_ref() + .map(|qc| qc.block_height() < remote_high_qc.block_height()) + .unwrap_or(true) + { + // TODO: validate + + highest_qc = Some(remote_high_qc); + } + + if num_succeeded == max_failures { + break; + } + } + + let Some(highest_qc) = highest_qc else { + return Err(CommsRpcConsensusSyncError::NoPeersAvailable { committee_size }); + }; + + let local_high_qc = self.state_store.with_read_tx(|tx| HighQc::get(tx).optional())?; + let local_height = local_high_qc + .as_ref() + .map(|qc| qc.block_height()) + .unwrap_or(NodeHeight(0)); + if highest_qc.block_height() > local_height { + info!( + target: LOG_TARGET, + "Highest QC from peers is at height {} and local high QC is at height {}", + highest_qc.block_height(), + local_height, + ); + return Ok(SyncStatus::Behind); + } + + Ok(SyncStatus::UpToDate) + } + + async fn sync_from_committee( + &mut self, + committee: Committee, + up_to_epoch: Option, + this_vn_address: PeerAddress, + ) -> Result, CommsRpcConsensusSyncError> { + let mut sync_error = None; + for member in committee.addresses() { + if *member == this_vn_address { + continue; + } + // Refresh the HighQC each time because a partial sync could have been achieved from a peer + let locked_block = self + .state_store + .with_read_tx(|tx| LockedBlock::get(tx).optional())? + .unwrap_or_else(|| Block::zero_block(self.network).as_locked_block()); + + match self.sync_with_peer(member, &locked_block, up_to_epoch).await { + Ok(()) => { + sync_error = None; + break; + }, + Err(err) => { + warn!(target: LOG_TARGET, "Failed to sync with peer {}: {}", member, err); + sync_error = Some(err); + continue; + }, + } + } + Ok(sync_error) + } +} + +#[async_trait] +impl SyncManager for RpcStateSyncManager +where TConsensusSpec: ConsensusSpec + Send + Sync + 'static +{ + type Error = CommsRpcConsensusSyncError; + + async fn check_sync(&self) -> Result { + let leaf_block = self.state_store.with_read_tx(|tx| LeafBlock::get(tx).optional())?; + let leaf_epoch = match leaf_block { + Some(leaf_block) => { + let block = self + .state_store + .with_read_tx(|tx| Block::get(tx, leaf_block.block_id()))?; + block.epoch() + }, + None => Epoch(0), + }; + let current_epoch = self.epoch_manager.current_epoch().await?; + if current_epoch > leaf_epoch { + info!(target: LOG_TARGET, "We are behind at least one epoch"); + // We are behind at least one epoch. + // We get the current substate range, and we asks committees from previous epoch in this range to give us + // data. + let local_shard = self.epoch_manager.get_local_committee_info(current_epoch).await?; + let current_num_committee = self.epoch_manager.get_num_committees(current_epoch).await?; + let range = local_shard.shard().to_substate_address_range(current_num_committee); + let prev_epoch = current_epoch.saturating_sub(Epoch(1)); + info!(target: LOG_TARGET,"Previous epoch is {}", prev_epoch); + let prev_num_committee = self.epoch_manager.get_num_committees(prev_epoch).await?; + info!(target: LOG_TARGET,"Previous num committee {}", prev_num_committee); + let start = range.start().to_shard(prev_num_committee); + let end = range.end().to_shard(prev_num_committee); + info!(target: LOG_TARGET,"Start: {}, End: {}", start, end); + let this_vn = self.epoch_manager.get_our_validator_node(current_epoch).await?; + let committees = self + .epoch_manager + .get_committees_by_shards( + prev_epoch, + (start.as_u32()..=end.as_u32()).map(Shard::from).collect::>(), + ) + .await?; + for (shard, mut committee) in committees { + info!(target: LOG_TARGET, "Syncing shard {} from previous epoch. Committee : {:?}", shard, committee); + committee.members.retain(|(addr, _)| *addr != this_vn.address); + committee.shuffle(); + if self.check_sync_from_committee(committee).await? == SyncStatus::Behind { + return Ok(SyncStatus::Behind); + } + } + } + + let committee: Committee = self.get_sync_peers().await?; + self.check_sync_from_committee(committee).await + } + + async fn sync(&mut self) -> Result<(), Self::Error> { + info!(target: LOG_TARGET, "Syncing"); + let mut sync_error = None; + let leaf_block = self.state_store.with_read_tx(|tx| LeafBlock::get(tx).optional())?; + let leaf_epoch = match leaf_block { + Some(leaf_block) => { + let block = self + .state_store + .with_read_tx(|tx| Block::get(tx, leaf_block.block_id()))?; + block.epoch() + }, + None => Epoch(0), + }; + let current_epoch = self.epoch_manager.current_epoch().await?; + let this_vn = self.epoch_manager.get_our_validator_node(current_epoch).await?; + if current_epoch > leaf_epoch { + info!(target: LOG_TARGET, "We are behind at least one epoch..sync"); + // We are behind at least one epoch. + // We get the current substate range, and we asks committees from previous epoch in this range to give us + // data. + let local_shard = self.epoch_manager.get_local_committee_info(current_epoch).await?; + let current_num_committee = self.epoch_manager.get_num_committees(current_epoch).await?; + let range = local_shard.shard().to_substate_address_range(current_num_committee); + let prev_epoch = current_epoch.saturating_sub(Epoch(1)); + info!(target: LOG_TARGET,"Previous epoch is {}", prev_epoch); + let prev_num_committee = self.epoch_manager.get_num_committees(prev_epoch).await?; + info!(target: LOG_TARGET,"Previous num committee {}", prev_num_committee); + let start = range.start().to_shard(prev_num_committee); + let end = range.end().to_shard(prev_num_committee); + info!(target: LOG_TARGET,"Start: {}, End: {}", start, end); + let committees = self + .epoch_manager + .get_committees_by_shards( + prev_epoch, + (start.as_u32()..=end.as_u32()).map(Shard::from).collect::>(), + ) + .await?; + for (_shard, committee) in committees { + info!(target:LOG_TARGET,"Syncing from committee {:?}",committee); + if let Some(error) = self + .sync_from_committee(committee, Some(current_epoch), this_vn.address) + .await? + { + sync_error = Some(error); + } + } + } + + let committee = self.get_sync_peers().await?; + if committee.is_empty() { + warn!(target: LOG_TARGET, "No peers available for sync"); + return Ok(()); + } + + if let Some(error) = self.sync_from_committee(committee, None, this_vn.address).await? { + sync_error = Some(error); + } + + if let Some(err) = sync_error { + return Err(err); + } + + Ok(()) + } +} + +struct BlockIdAndHeight { + id: BlockId, + height: NodeHeight, +} + +impl Display for BlockIdAndHeight { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "Block: {} (#{})", self.id, self.height) + } +} + +// TODO: these are similar structures. Clean this up. +fn substate_update_to_change(update: SubstateUpdate) -> SubstateChange { + match update { + SubstateUpdate::Create(create) => SubstateChange::Up { + id: VersionedSubstateId::new(create.substate.substate_id.clone(), create.substate.version), + transaction_id: create.substate.created_by_transaction, + substate: create.substate.into_substate(), + }, + SubstateUpdate::Destroy(destroy) => SubstateChange::Down { + id: VersionedSubstateId::new(destroy.substate_id, destroy.version), + transaction_id: destroy.destroyed_by_transaction, + }, + } +} diff --git a/dan_layer/state_store_sqlite/migrations/2023-06-08-091819_create_state_store/up.sql b/dan_layer/state_store_sqlite/migrations/2023-06-08-091819_create_state_store/up.sql index 7e4048c9f..622ccdf03 100644 --- a/dan_layer/state_store_sqlite/migrations/2023-06-08-091819_create_state_store/up.sql +++ b/dan_layer/state_store_sqlite/migrations/2023-06-08-091819_create_state_store/up.sql @@ -14,7 +14,7 @@ create table blocks ( id integer not null primary key AUTOINCREMENT, block_id text not NULL, - parent_block_id text not NULL, + parent_block_id text not NULL REFERENCES blocks (block_id), merkle_root text not NULL, network text not NULL, height bigint not NULL, @@ -73,6 +73,7 @@ create table leaf_blocks id integer not null primary key AUTOINCREMENT, block_id text not NULL, block_height bigint not NULL, + epoch bigint not NULL, created_at timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP, FOREIGN KEY (block_id) REFERENCES blocks (block_id) ); @@ -83,13 +84,13 @@ create table block_diffs block_id text NOT NULL, transaction_id text NOT NULL, substate_id text NOT NULL, - version int NOT NULL, + version int NOT NULL, -- Up or Down change text NOT NULL, -- NULL for Down state text NULL, - created_at timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP, - FOREIGN KEY (transaction_id) REFERENCES transactions (transaction_id) + created_at timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP, + FOREIGN KEY (transaction_id) REFERENCES transactions (transaction_id), FOREIGN KEY (block_id) REFERENCES blocks (block_id) ); create index block_diffs_idx_block_id on block_diffs (block_id); @@ -106,11 +107,15 @@ create table substates created_justify text not NULL, created_block text not NULL, created_height bigint not NULL, + -- uniquely identifies the chain + created_at_epoch bigint not NULL, + created_by_shard int not NULL, destroyed_by_transaction text NULL, destroyed_justify text NULL, - destroyed_by_block text NULL, - created_at_epoch bigint not NULL, + destroyed_by_block bigint NULL, + -- uniquely identifies the chain destroyed_at_epoch bigint NULL, + destroyed_by_shard int NULL, created_at timestamp not NULL DEFAULT CURRENT_TIMESTAMP, destroyed_at timestamp NULL ); @@ -143,19 +148,19 @@ create table high_qcs id integer not null primary key autoincrement, block_id text not null, block_height bigint not null, + epoch bigint not null, qc_id text not null, created_at timestamp NOT NULL default current_timestamp, FOREIGN KEY (qc_id) REFERENCES quorum_certificates (qc_id), FOREIGN KEY (block_id) REFERENCES blocks (block_id) ); -create unique index high_qcs_uniq_idx_qc_id on high_qcs (qc_id); - create table last_voted ( id integer not null primary key autoincrement, block_id text not null, height bigint not null, + epoch bigint not null, created_at timestamp NOT NULL default current_timestamp ); @@ -176,6 +181,7 @@ create table last_executed id integer not null primary key autoincrement, block_id text not null, height bigint not null, + epoch bigint not null, created_at timestamp NOT NULL default current_timestamp, FOREIGN KEY (block_id) REFERENCES blocks (block_id) ); @@ -185,6 +191,7 @@ create table last_proposed id integer not null primary key autoincrement, block_id text not null, height bigint not null, + epoch bigint not null, created_at timestamp NOT NULL default current_timestamp ); @@ -193,6 +200,7 @@ create table locked_block id integer not null primary key autoincrement, block_id text not null, height bigint not null, + epoch bigint not null, created_at timestamp NOT NULL default current_timestamp, FOREIGN KEY (block_id) REFERENCES blocks (block_id) ); @@ -329,13 +337,17 @@ CREATE TABLE foreign_receive_counters CREATE TABLE state_tree ( id integer not NULL primary key AUTOINCREMENT, + epoch bigint not NULL, + shard int not NULL, key text not NULL, node text not NULL, is_stale boolean not null default '0' ); +-- Scoping by epoch,shard +CREATE INDEX state_tree_idx_epoch_shard_key on state_tree (epoch, shard); -- Duplicate keys are not allowed -CREATE UNIQUE INDEX state_tree_uniq_idx_key on state_tree (key); +CREATE UNIQUE INDEX state_tree_uniq_idx_key on state_tree (epoch, shard, key); -- filtering out or by is_stale is used in every query CREATE INDEX state_tree_idx_is_stale on state_tree (is_stale); @@ -351,6 +363,25 @@ CREATE TABLE pending_state_tree_diffs CREATE UNIQUE INDEX pending_state_tree_diffs_uniq_idx_block_id on pending_state_tree_diffs (block_id); +-- An append-only store of state transitions +CREATE TABLE state_transitions +( + id integer not NULL primary key AUTOINCREMENT, + -- tuple uniquely identifies the "chain" that created the state transition + epoch bigint not NULL, + shard int not NULL, + -- in conjunction with the , this uniquely identifies and orders the state transition + seq bigint not NULL, + substate_address text not NULL, + -- substate_id and version not required, just to make DB inspection easier + substate_id text not NULL, + version int not NULL, + transition text check (transition IN ('UP', 'DOWN')) not NULL, + state_hash text NULL, + state_version bigint not NULL, + created_at timestamp not NULL DEFAULT CURRENT_TIMESTAMP, + FOREIGN KEY (substate_address) REFERENCES substates (address) +); -- Debug Triggers CREATE TABLE transaction_pool_history diff --git a/dan_layer/state_store_sqlite/src/chain_scoped_tree_store.rs b/dan_layer/state_store_sqlite/src/chain_scoped_tree_store.rs new file mode 100644 index 000000000..c98884517 --- /dev/null +++ b/dan_layer/state_store_sqlite/src/chain_scoped_tree_store.rs @@ -0,0 +1,104 @@ +// Copyright 2024 The Tari Project +// SPDX-License-Identifier: BSD-3-Clause + +use diesel::{ExpressionMethods, OptionalExtension, QueryDsl, RunQueryDsl}; +use tari_dan_common_types::{shard::Shard, Epoch}; +use tari_state_tree::{Node, NodeKey, StaleTreeNode, TreeNode, TreeStoreReader, TreeStoreWriter, Version}; + +use crate::{reader::SqliteStateStoreReadTransaction, writer::SqliteStateStoreWriteTransaction}; + +pub struct SqliteChainScopedTreeStore { + epoch: Epoch, + shard: Shard, + db_tx: TTx, +} + +impl SqliteChainScopedTreeStore { + pub fn new(epoch: Epoch, shard: Shard, db_tx: TTx) -> Self { + Self { epoch, shard, db_tx } + } + + fn db_epoch(&self) -> i64 { + self.epoch.as_u64() as i64 + } + + fn db_shard(&self) -> i32 { + self.shard.as_u32() as i32 + } +} + +impl<'a, TAddr> TreeStoreReader + for SqliteChainScopedTreeStore<&'a SqliteStateStoreReadTransaction<'a, TAddr>> +{ + fn get_node(&self, key: &NodeKey) -> Result, tari_state_tree::JmtStorageError> { + use crate::schema::state_tree; + + let node = state_tree::table + .select(state_tree::node) + .filter(state_tree::epoch.eq(self.db_epoch())) + .filter(state_tree::shard.eq(self.db_shard())) + .filter(state_tree::key.eq(key.to_string())) + .filter(state_tree::is_stale.eq(false)) + .first::(self.connection()) + .optional() + .map_err(|e| tari_state_tree::JmtStorageError::UnexpectedError(e.to_string()))? + .ok_or_else(|| tari_state_tree::JmtStorageError::NotFound(key.clone()))?; + + let node = serde_json::from_str::(&node) + .map_err(|e| tari_state_tree::JmtStorageError::UnexpectedError(e.to_string()))?; + + Ok(node.into_node()) + } +} + +impl<'a, TAddr> TreeStoreReader + for SqliteChainScopedTreeStore<&'a mut SqliteStateStoreWriteTransaction<'a, TAddr>> +{ + fn get_node(&self, key: &NodeKey) -> Result, tari_state_tree::JmtStorageError> { + SqliteChainScopedTreeStore::new(self.epoch, self.shard, &**self.db_tx).get_node(key) + } +} + +impl<'a, TAddr> TreeStoreWriter + for SqliteChainScopedTreeStore<&'a mut SqliteStateStoreWriteTransaction<'a, TAddr>> +{ + fn insert_node(&mut self, key: NodeKey, node: Node) -> Result<(), tari_state_tree::JmtStorageError> { + use crate::schema::state_tree; + + let node = TreeNode::new_latest(node); + let node = serde_json::to_string(&node) + .map_err(|e| tari_state_tree::JmtStorageError::UnexpectedError(e.to_string()))?; + + let values = ( + state_tree::epoch.eq(self.db_epoch()), + state_tree::shard.eq(self.db_shard()), + state_tree::key.eq(key.to_string()), + state_tree::node.eq(node), + ); + diesel::insert_into(state_tree::table) + .values(&values) + .on_conflict((state_tree::epoch, state_tree::shard, state_tree::key)) + .do_update() + .set(values.clone()) + .execute(self.connection()) + .map_err(|e| tari_state_tree::JmtStorageError::UnexpectedError(e.to_string()))?; + + Ok(()) + } + + fn record_stale_tree_node(&mut self, node: StaleTreeNode) -> Result<(), tari_state_tree::JmtStorageError> { + use crate::schema::state_tree; + let key = node.as_node_key(); + diesel::update(state_tree::table) + .filter(state_tree::epoch.eq(self.db_epoch())) + .filter(state_tree::shard.eq(self.db_shard())) + .filter(state_tree::key.eq(key.to_string())) + .set(state_tree::is_stale.eq(true)) + .execute(self.connection()) + .optional() + .map_err(|e| tari_state_tree::JmtStorageError::UnexpectedError(e.to_string()))? + .ok_or_else(|| tari_state_tree::JmtStorageError::NotFound(key.clone()))?; + + Ok(()) + } +} diff --git a/dan_layer/state_store_sqlite/src/lib.rs b/dan_layer/state_store_sqlite/src/lib.rs index 8a8fadaa3..677be0292 100644 --- a/dan_layer/state_store_sqlite/src/lib.rs +++ b/dan_layer/state_store_sqlite/src/lib.rs @@ -8,7 +8,7 @@ mod serialization; mod sql_models; mod sqlite_transaction; mod store; -mod tree_store; +// mod tree_store; mod writer; pub use store::SqliteStateStore; diff --git a/dan_layer/state_store_sqlite/src/reader.rs b/dan_layer/state_store_sqlite/src/reader.rs index 8c2f479ee..cee2cf6cf 100644 --- a/dan_layer/state_store_sqlite/src/reader.rs +++ b/dan_layer/state_store_sqlite/src/reader.rs @@ -14,7 +14,7 @@ use diesel::{ dsl, query_builder::SqlQuery, sql_query, - sql_types::{BigInt, Text}, + sql_types::{BigInt, Integer, Text}, BoolExpressionMethods, ExpressionMethods, JoinOnDsl, @@ -29,7 +29,7 @@ use indexmap::IndexMap; use log::*; use serde::{de::DeserializeOwned, Serialize}; use tari_common_types::types::{FixedHash, PublicKey}; -use tari_dan_common_types::{Epoch, NodeAddressable, NodeHeight, SubstateAddress}; +use tari_dan_common_types::{shard::Shard, Epoch, NodeAddressable, NodeHeight, SubstateAddress}; use tari_dan_storage::{ consensus_models::{ Block, @@ -51,6 +51,8 @@ use tari_dan_storage::{ PendingStateTreeDiff, QcId, QuorumCertificate, + StateTransition, + StateTransitionId, SubstateRecord, TransactionExecution, TransactionPoolRecord, @@ -63,6 +65,7 @@ use tari_dan_storage::{ StorageError, }; use tari_engine_types::substate::SubstateId; +use tari_state_tree::{Node, NodeKey, TreeNode, Version}; use tari_transaction::{SubstateRequirement, TransactionId, VersionedSubstateId}; use tari_utilities::ByteArray; @@ -209,24 +212,32 @@ impl<'a, TAddr: NodeAddressable + Serialize + DeserializeOwned + 'a> SqliteState /// Returns the blocks from the start_block (inclusive) to the end_block (inclusive). fn get_block_ids_between( &self, + epoch: Epoch, + shard: Shard, start_block: &BlockId, end_block: &BlockId, ) -> Result, SqliteStorageError> { + debug!(target: LOG_TARGET, "get_block_ids_between: {epoch} {shard} start: {start_block}, end: {end_block}"); let block_ids = sql_query( r#" WITH RECURSIVE tree(bid, parent) AS ( - SELECT block_id, parent_block_id FROM blocks where block_id = ? + SELECT block_id, parent_block_id FROM blocks where block_id = ? AND epoch = ? AND shard = ? UNION ALL SELECT block_id, parent_block_id FROM blocks JOIN tree ON block_id = tree.parent AND tree.bid != ? + WHERE epoch = ? AND shard = ? LIMIT 1000 ) SELECT bid FROM tree"#, ) .bind::(serialize_hex(end_block)) + .bind::(epoch.as_u64() as i64) + .bind::(shard.as_u32() as i32) .bind::(serialize_hex(start_block)) + .bind::(epoch.as_u64() as i64) + .bind::(shard.as_u32() as i32) .load_iter::(self.connection()) .map_err(|e| SqliteStorageError::DieselError { operation: "get_block_ids_that_change_state_between", @@ -421,7 +432,7 @@ impl<'tx, TAddr: NodeAddressable + Serialize + DeserializeOwned + 'tx> StateStor use crate::schema::foreign_proposals; let foreign_proposals = foreign_proposals::table - .filter(foreign_proposals::bucket.eq(foreign_proposal.bucket.as_u32() as i32)) + .filter(foreign_proposals::bucket.eq(foreign_proposal.shard.as_u32() as i32)) .filter(foreign_proposals::block_id.eq(serialize_hex(foreign_proposal.block_id))) .filter(foreign_proposals::transactions.eq(serialize_json(&foreign_proposal.transactions)?)) .filter(foreign_proposals::base_layer_block_height.eq(foreign_proposal.base_layer_block_height as i64)) @@ -632,8 +643,8 @@ impl<'tx, TAddr: NodeAddressable + Serialize + DeserializeOwned + 'tx> StateStor ) -> Result { use crate::schema::transaction_executions; - // TODO: This get slower as the chain progresses. - let block_ids = self.get_block_ids_that_change_state_between(&BlockId::genesis(), from_block_id)?; + // TODO: This gets slower as the chain progresses. + let block_ids = self.get_block_ids_that_change_state_between(&BlockId::zero(), from_block_id)?; let execution = transaction_executions::table .filter(transaction_executions::transaction_id.eq(serialize_hex(tx_id))) @@ -672,12 +683,14 @@ impl<'tx, TAddr: NodeAddressable + Serialize + DeserializeOwned + 'tx> StateStor block.try_convert(qc) } - fn blocks_get_tip(&self) -> Result { + fn blocks_get_tip(&self, epoch: Epoch, shard: Shard) -> Result { use crate::schema::{blocks, quorum_certificates}; let (block, qc) = blocks::table .left_join(quorum_certificates::table.on(blocks::qc_id.eq(quorum_certificates::qc_id))) .select((blocks::all_columns, quorum_certificates::all_columns.nullable())) + .filter(blocks::epoch.eq(epoch.as_u64() as i64)) + .filter(blocks::shard.eq(shard.as_u32() as i32)) .order_by(blocks::height.desc()) .first::<(sql_models::Block, Option)>(self.connection()) .map_err(|e| SqliteStorageError::DieselError { @@ -696,22 +709,54 @@ impl<'tx, TAddr: NodeAddressable + Serialize + DeserializeOwned + 'tx> StateStor block.try_convert(qc) } + fn blocks_get_last_n_in_epoch(&self, n: usize, epoch: Epoch, shard: Shard) -> Result, StorageError> { + use crate::schema::{blocks, quorum_certificates}; + + let blocks = blocks::table + .left_join(quorum_certificates::table.on(blocks::qc_id.eq(quorum_certificates::qc_id))) + .select((blocks::all_columns, quorum_certificates::all_columns.nullable())) + .filter(blocks::epoch.eq(epoch.as_u64() as i64)) + .filter(blocks::shard.eq(shard.as_u32() as i32)) + .filter(blocks::is_committed.eq(true)) + .order_by(blocks::height.desc()) + .limit(n as i64) + .get_results::<(sql_models::Block, Option)>(self.connection()) + .map_err(|e| SqliteStorageError::DieselError { + operation: "blocks_get_last_n_in_epoch", + source: e, + })?; + + blocks + .into_iter() + // Order from lowest to highest height + .rev() + .map(|(b, qc)| { + qc.ok_or_else(|| StorageError::DataInconsistency { + details: format!( + "blocks_get_last_n_in_epoch: block {} references non-existent quorum certificate {}", + b.block_id, b.qc_id + ), + }) + .and_then(|qc| b.try_convert(qc)) + }) + .collect() + } + fn blocks_get_all_between( &self, + epoch: Epoch, + shard: Shard, start_block_id_exclusive: &BlockId, end_block_id_inclusive: &BlockId, include_dummy_blocks: bool, ) -> Result, StorageError> { use crate::schema::{blocks, quorum_certificates}; - let mut block_ids = self.get_block_ids_between(start_block_id_exclusive, end_block_id_inclusive)?; + let block_ids = self.get_block_ids_between(epoch, shard, start_block_id_exclusive, end_block_id_inclusive)?; if block_ids.is_empty() { return Ok(vec![]); } - // Exclude start block - block_ids.pop(); - let mut query = blocks::table .left_join(quorum_certificates::table.on(blocks::qc_id.eq(quorum_certificates::qc_id))) .select((blocks::all_columns, quorum_certificates::all_columns.nullable())) @@ -723,6 +768,8 @@ impl<'tx, TAddr: NodeAddressable + Serialize + DeserializeOwned + 'tx> StateStor } let results = query + .filter(blocks::epoch.eq(epoch.as_u64() as i64)) + .filter(blocks::shard.eq(shard.as_u32() as i32)) .order_by(blocks::height.asc()) .get_results::<(sql_models::Block, Option)>(self.connection()) .map_err(|e| SqliteStorageError::DieselError { @@ -971,24 +1018,28 @@ impl<'tx, TAddr: NodeAddressable + Serialize + DeserializeOwned + 'tx> StateStor Some(Ordering::Ascending) => match ordering_index { Some(0) => query.order_by(blocks::block_id.asc()), Some(1) => query.order_by(blocks::epoch.asc()), - Some(2) => query.order_by(blocks::height.asc()), + Some(2) => query.order_by(blocks::epoch.asc()).then_order_by(blocks::height.asc()), Some(4) => query.order_by(blocks::command_count.asc()), Some(5) => query.order_by(blocks::total_leader_fee.asc()), Some(6) => query.order_by(blocks::block_time.asc()), Some(7) => query.order_by(blocks::created_at.asc()), Some(8) => query.order_by(blocks::proposed_by.asc()), - _ => query.order_by(blocks::height.asc()), + _ => query.order_by(blocks::epoch.asc()).then_order_by(blocks::height.asc()), }, _ => match ordering_index { Some(0) => query.order_by(blocks::block_id.desc()), Some(1) => query.order_by(blocks::epoch.desc()), - Some(2) => query.order_by(blocks::height.desc()), + Some(2) => query + .order_by(blocks::epoch.desc()) + .then_order_by(blocks::height.desc()), Some(4) => query.order_by(blocks::command_count.desc()), Some(5) => query.order_by(blocks::total_leader_fee.desc()), Some(6) => query.order_by(blocks::block_time.desc()), Some(7) => query.order_by(blocks::created_at.desc()), Some(8) => query.order_by(blocks::proposed_by.desc()), - _ => query.order_by(blocks::height.desc()), + _ => query + .order_by(blocks::epoch.desc()) + .then_order_by(blocks::height.desc()), }, }; @@ -1720,6 +1771,31 @@ impl<'tx, TAddr: NodeAddressable + Serialize + DeserializeOwned + 'tx> StateStor Ok(count > 0) } + fn substates_get_n_after(&self, n: usize, after: &SubstateAddress) -> Result, StorageError> { + use crate::schema::substates; + + let start_id = substates::table + .select(substates::id) + .filter(substates::address.eq(after.to_string())) + .get_result::(self.connection()) + .map_err(|e| SqliteStorageError::DieselError { + operation: "substates_get_n_after", + source: e, + })?; + + let substates = substates::table + .filter(substates::id.gt(start_id)) + .limit(n as i64) + .order_by(substates::id.asc()) + .get_results::(self.connection()) + .map_err(|e| SqliteStorageError::DieselError { + operation: "substates_get_n_after", + source: e, + })?; + + substates.into_iter().map(TryInto::try_into).collect() + } + fn substates_get_many_within_range( &self, start: &SubstateAddress, @@ -1774,31 +1850,6 @@ impl<'tx, TAddr: NodeAddressable + Serialize + DeserializeOwned + 'tx> StateStor substates.into_iter().map(TryInto::try_into).collect() } - fn substates_get_all_for_block(&self, block_id: &BlockId) -> Result, StorageError> { - use crate::schema::substates; - - let block_id_hex = serialize_hex(block_id); - - let substates = substates::table - .filter( - substates::created_block - .eq(&block_id_hex) - .or(substates::destroyed_by_block.eq(Some(&block_id_hex))), - ) - .get_results::(self.connection()) - .map_err(|e| SqliteStorageError::DieselError { - operation: "substates_get_all_for_block", - source: e, - })?; - - let substates = substates - .into_iter() - .map(TryInto::try_into) - .collect::, _>>()?; - - Ok(substates) - } - fn substates_get_all_for_transaction( &self, transaction_id: &TransactionId, @@ -1840,7 +1891,7 @@ impl<'tx, TAddr: NodeAddressable + Serialize + DeserializeOwned + 'tx> StateStor // block, then we just have to exclude locks for the forked chain and load everything else. // For now, we just fetch all block ids in the chain relevant to the given block, which will be slower and more // memory intensive as the chain progresses. - let block_ids = self.get_block_ids_that_change_state_between(&BlockId::genesis(), &block_id)?; + let block_ids = self.get_block_ids_that_change_state_between(&BlockId::zero(), &block_id)?; let lock_recs = substate_locks::table .filter(substate_locks::block_id.eq_any(block_ids)) @@ -1899,6 +1950,8 @@ impl<'tx, TAddr: NodeAddressable + Serialize + DeserializeOwned + 'tx> StateStor fn pending_state_tree_diffs_get_all_up_to_commit_block( &self, + epoch: Epoch, + shard: Shard, block_id: &BlockId, ) -> Result, StorageError> { use crate::schema::pending_state_tree_diffs; @@ -1906,10 +1959,8 @@ impl<'tx, TAddr: NodeAddressable + Serialize + DeserializeOwned + 'tx> StateStor // Get the last committed block let committed_block_id = self.get_commit_block_id()?; - let mut block_ids = self.get_block_ids_between(&committed_block_id, block_id)?; + let block_ids = self.get_block_ids_between(epoch, shard, &committed_block_id, block_id)?; - // Exclude commit block - block_ids.pop(); if block_ids.is_empty() { return Ok(Vec::new()); } @@ -1925,6 +1976,98 @@ impl<'tx, TAddr: NodeAddressable + Serialize + DeserializeOwned + 'tx> StateStor diffs.into_iter().map(TryInto::try_into).collect() } + + fn state_transitions_get_n_after( + &self, + n: usize, + id: StateTransitionId, + end_epoch: Epoch, + ) -> Result, StorageError> { + use crate::schema::{state_transitions, substates}; + + let start_id = state_transitions::table + .select(state_transitions::id) + .filter(state_transitions::epoch.eq(id.epoch().as_u64() as i64)) + .filter(state_transitions::shard.eq(id.shard().as_u32() as i32)) + .filter(state_transitions::seq.eq(0i64)) + .order_by(state_transitions::id.asc()) + .first::(self.connection()) + .map_err(|e| SqliteStorageError::DieselError { + operation: "state_transitions_get_n_after", + source: e, + })?; + + let start_id = start_id + (id.seq() as i32); + + let transitions = state_transitions::table + .left_join(substates::table.on(state_transitions::substate_address.eq(substates::address))) + .select((state_transitions::all_columns, substates::all_columns.nullable())) + .filter(state_transitions::id.gt(start_id)) + .filter(state_transitions::epoch.lt(end_epoch.as_u64() as i64)) + .limit(n as i64) + .get_results::<(sql_models::StateTransition, Option)>(self.connection()) + .map_err(|e| SqliteStorageError::DieselError { + operation: "state_transitions_get_n_after", + source: e, + })?; + + transitions + .into_iter() + .map(|(t, s)| { + let s = s.ok_or_else(|| StorageError::DataInconsistency { + details: format!("substate entry does not exist for transition {}", t.id), + })?; + + t.try_convert(s) + }) + .collect() + } + + fn state_transitions_get_last_id(&self) -> Result { + use crate::schema::state_transitions; + + let (seq, epoch, shard) = state_transitions::table + .select(( + state_transitions::seq, + state_transitions::epoch, + state_transitions::shard, + )) + .order_by(state_transitions::epoch.desc()) + .then_order_by(state_transitions::seq.desc()) + .first::<(i64, i64, i32)>(self.connection()) + .map_err(|e| SqliteStorageError::DieselError { + operation: "state_transitions_get_last_id", + source: e, + })?; + + let epoch = Epoch(epoch as u64); + let shard = Shard::from(shard as u32); + let seq = seq as u64; + + Ok(StateTransitionId::new(epoch, shard, seq)) + } + + fn state_tree_nodes_get(&self, epoch: Epoch, shard: Shard, key: &NodeKey) -> Result, StorageError> { + use crate::schema::state_tree; + + let node = state_tree::table + .select(state_tree::node) + .filter(state_tree::epoch.eq(epoch.as_u64() as i64)) + .filter(state_tree::shard.eq(shard.as_u32() as i32)) + .filter(state_tree::key.eq(key.to_string())) + .filter(state_tree::is_stale.eq(false)) + .first::(self.connection()) + .map_err(|e| SqliteStorageError::DieselError { + operation: "state_tree_nodes_get", + source: e, + })?; + + let node = serde_json::from_str::(&node).map_err(|e| StorageError::DataInconsistency { + details: format!("Failed to deserialize state tree node: {}", e), + })?; + + Ok(node.into_node()) + } } #[derive(QueryableByName)] diff --git a/dan_layer/state_store_sqlite/src/schema.rs b/dan_layer/state_store_sqlite/src/schema.rs index b894ce82c..9a0cd2534 100644 --- a/dan_layer/state_store_sqlite/src/schema.rs +++ b/dan_layer/state_store_sqlite/src/schema.rs @@ -76,6 +76,7 @@ diesel::table! { id -> Integer, block_id -> Text, block_height -> BigInt, + epoch -> BigInt, qc_id -> Text, created_at -> Timestamp, } @@ -86,6 +87,7 @@ diesel::table! { id -> Integer, block_id -> Text, height -> BigInt, + epoch -> BigInt, created_at -> Timestamp, } } @@ -95,6 +97,7 @@ diesel::table! { id -> Integer, block_id -> Text, height -> BigInt, + epoch -> BigInt, created_at -> Timestamp, } } @@ -116,6 +119,7 @@ diesel::table! { id -> Integer, block_id -> Text, height -> BigInt, + epoch -> BigInt, created_at -> Timestamp, } } @@ -125,6 +129,7 @@ diesel::table! { id -> Integer, block_id -> Text, block_height -> BigInt, + epoch -> BigInt, created_at -> Timestamp, } } @@ -134,6 +139,7 @@ diesel::table! { id -> Integer, block_id -> Text, height -> BigInt, + epoch -> BigInt, created_at -> Timestamp, } } @@ -194,9 +200,27 @@ diesel::table! { } } +diesel::table! { + state_transitions (id) { + id -> Integer, + epoch -> BigInt, + shard -> Integer, + seq -> BigInt, + substate_address -> Text, + substate_id -> Text, + version -> Integer, + transition -> Text, + state_hash -> Nullable, + state_version -> BigInt, + created_at -> Timestamp, + } +} + diesel::table! { state_tree (id) { id -> Integer, + epoch -> BigInt, + shard -> Integer, key -> Text, node -> Text, is_stale -> Bool, @@ -228,11 +252,13 @@ diesel::table! { created_justify -> Text, created_block -> Text, created_height -> BigInt, + created_at_epoch -> BigInt, + created_by_shard -> Integer, destroyed_by_transaction -> Nullable, destroyed_justify -> Nullable, - destroyed_by_block -> Nullable, - created_at_epoch -> BigInt, + destroyed_by_block -> Nullable, destroyed_at_epoch -> Nullable, + destroyed_by_shard -> Nullable, created_at -> Timestamp, destroyed_at -> Nullable, } @@ -359,6 +385,7 @@ diesel::allow_tables_to_appear_in_same_query!( parked_blocks, pending_state_tree_diffs, quorum_certificates, + state_transitions, state_tree, substate_locks, substates, diff --git a/dan_layer/state_store_sqlite/src/sql_models/bookkeeping.rs b/dan_layer/state_store_sqlite/src/sql_models/bookkeeping.rs index 1bd62b6c7..b30bb24a2 100644 --- a/dan_layer/state_store_sqlite/src/sql_models/bookkeeping.rs +++ b/dan_layer/state_store_sqlite/src/sql_models/bookkeeping.rs @@ -19,6 +19,7 @@ pub struct HighQc { pub id: i32, pub block_id: String, pub block_height: i64, + pub epoch: i64, pub qc_id: String, pub created_at: PrimitiveDateTime, } @@ -30,6 +31,7 @@ impl TryFrom for consensus_models::HighQc { Ok(Self { block_id: deserialize_hex_try_from(&value.block_id)?, block_height: NodeHeight(value.block_height as u64), + epoch: Epoch(value.epoch as u64), qc_id: deserialize_hex_try_from(&value.qc_id)?, }) } @@ -52,7 +54,7 @@ impl TryFrom for consensus_models::ForeignProposal { fn try_from(value: ForeignProposal) -> Result { Ok(Self { - bucket: Shard::from(value.bucket as u32), + shard: Shard::from(value.bucket as u32), block_id: deserialize_hex_try_from(&value.block_id)?, state: parse_from_string(&value.state)?, proposed_height: value.mined_at.map(|mined_at| NodeHeight(mined_at as u64)), @@ -102,6 +104,7 @@ pub struct LockedBlock { pub id: i32, pub block_id: String, pub height: i64, + pub epoch: i64, pub created_at: PrimitiveDateTime, } @@ -112,6 +115,7 @@ impl TryFrom for consensus_models::LockedBlock { Ok(Self { block_id: deserialize_hex_try_from(&value.block_id)?, height: NodeHeight(value.height as u64), + epoch: Epoch(value.epoch as u64), }) } } @@ -121,6 +125,7 @@ pub struct LastExecuted { pub id: i32, pub block_id: String, pub height: i64, + pub epoch: i64, pub created_at: PrimitiveDateTime, } @@ -131,6 +136,7 @@ impl TryFrom for consensus_models::LastExecuted { Ok(Self { block_id: deserialize_hex_try_from(&value.block_id)?, height: NodeHeight(value.height as u64), + epoch: Epoch(value.epoch as u64), }) } } @@ -140,6 +146,7 @@ pub struct LastVoted { pub id: i32, pub block_id: String, pub height: i64, + pub epoch: i64, pub created_at: PrimitiveDateTime, } @@ -150,6 +157,7 @@ impl TryFrom for consensus_models::LastVoted { Ok(Self { block_id: deserialize_hex_try_from(&value.block_id)?, height: NodeHeight(value.height as u64), + epoch: Epoch(value.epoch as u64), }) } } @@ -193,6 +201,7 @@ pub struct LastProposed { pub id: i32, pub block_id: String, pub height: i64, + pub epoch: i64, pub created_at: PrimitiveDateTime, } @@ -203,6 +212,7 @@ impl TryFrom for consensus_models::LastProposed { Ok(Self { block_id: deserialize_hex_try_from(&value.block_id)?, height: NodeHeight(value.height as u64), + epoch: Epoch(value.epoch as u64), }) } } diff --git a/dan_layer/state_store_sqlite/src/sql_models/leaf_block.rs b/dan_layer/state_store_sqlite/src/sql_models/leaf_block.rs index 1fecbf9df..126d5bdcb 100644 --- a/dan_layer/state_store_sqlite/src/sql_models/leaf_block.rs +++ b/dan_layer/state_store_sqlite/src/sql_models/leaf_block.rs @@ -2,7 +2,7 @@ // SPDX-License-Identifier: BSD-3-Clause use diesel::Queryable; -use tari_dan_common_types::NodeHeight; +use tari_dan_common_types::{Epoch, NodeHeight}; use tari_dan_storage::{consensus_models, consensus_models::BlockId, StorageError}; use time::PrimitiveDateTime; @@ -13,6 +13,7 @@ pub struct LeafBlock { pub id: i32, pub block_id: String, pub block_height: i64, + pub epoch: i64, pub created_at: PrimitiveDateTime, } @@ -28,6 +29,7 @@ impl TryFrom for consensus_models::LeafBlock { } })?, height: NodeHeight(value.block_height as u64), + epoch: Epoch(value.epoch as u64), }) } } diff --git a/dan_layer/state_store_sqlite/src/sql_models/mod.rs b/dan_layer/state_store_sqlite/src/sql_models/mod.rs index 0a1e4250f..77778e603 100644 --- a/dan_layer/state_store_sqlite/src/sql_models/mod.rs +++ b/dan_layer/state_store_sqlite/src/sql_models/mod.rs @@ -7,6 +7,7 @@ mod bookkeeping; mod leaf_block; mod pending_state_tree_diff; mod quorum_certificate; +mod state_transition; mod substate; mod substate_lock; mod transaction; @@ -20,6 +21,7 @@ pub use bookkeeping::*; pub use leaf_block::*; pub use pending_state_tree_diff::*; pub use quorum_certificate::*; +pub use state_transition::*; pub use substate::*; pub use substate_lock::*; pub use transaction::*; diff --git a/dan_layer/state_store_sqlite/src/sql_models/state_transition.rs b/dan_layer/state_store_sqlite/src/sql_models/state_transition.rs new file mode 100644 index 000000000..c0daef42e --- /dev/null +++ b/dan_layer/state_store_sqlite/src/sql_models/state_transition.rs @@ -0,0 +1,91 @@ +// Copyright 2024 The Tari Project +// SPDX-License-Identifier: BSD-3-Clause + +use diesel::Queryable; +use tari_dan_common_types::{shard::Shard, Epoch}; +use tari_dan_storage::{ + consensus_models, + consensus_models::{ + QuorumCertificate, + StateTransitionId, + SubstateCreatedProof, + SubstateData, + SubstateDestroyedProof, + SubstateUpdate, + }, + StorageError, +}; +use time::PrimitiveDateTime; + +use crate::sql_models::SubstateRecord; + +#[derive(Debug, Clone, Queryable)] +pub struct StateTransition { + pub id: i32, + pub epoch: i64, + pub shard: i32, + pub seq: i64, + pub substate_address: String, + pub substate_id: String, + pub version: i32, + pub transition: String, + pub state_hash: Option, + pub state_version: i64, + pub created_at: PrimitiveDateTime, +} + +impl StateTransition { + pub fn try_convert(self, substate: SubstateRecord) -> Result { + let substate = consensus_models::SubstateRecord::try_from(substate)?; + let seq = self.seq as u64; + let epoch = Epoch(self.epoch as u64); + let shard = Shard::from(self.shard as u32); + + let update = match self.transition.as_str() { + "UP" => { + SubstateUpdate::Create(SubstateCreatedProof { + substate: SubstateData { + substate_id: substate.substate_id, + version: substate.version, + substate_value: substate.substate_value, + created_by_transaction: substate.created_by_transaction, + }, + // TODO + created_qc: QuorumCertificate::genesis(), + }) + }, + "DOWN" => { + if !substate.is_destroyed() { + return Err(StorageError::DataInconsistency { + details: format!( + "State transition for substate {}:{} is DOWN but the substate is not destroyed", + substate.substate_id, substate.version + ), + }); + } + + SubstateUpdate::Destroy(SubstateDestroyedProof { + destroyed_by_transaction: substate.destroyed().unwrap().by_transaction, + substate_id: substate.substate_id, + version: substate.version, + // TODO + justify: QuorumCertificate::genesis(), + }) + }, + _ => { + return Err(StorageError::DataInconsistency { + details: format!( + "StateTransition::try_convert: '{}' is not a valid transition", + self.transition + ), + }) + }, + }; + + Ok(consensus_models::StateTransition { + id: StateTransitionId::new(epoch, shard, seq), + update, + state_tree_version: self.state_version as u64, + }) + } +} diff --git a/dan_layer/state_store_sqlite/src/sql_models/substate.rs b/dan_layer/state_store_sqlite/src/sql_models/substate.rs index 240ca21e0..96fa95324 100644 --- a/dan_layer/state_store_sqlite/src/sql_models/substate.rs +++ b/dan_layer/state_store_sqlite/src/sql_models/substate.rs @@ -2,7 +2,7 @@ // SPDX-License-Identifier: BSD-3-Clause use diesel::Queryable; -use tari_dan_common_types::{Epoch, NodeHeight}; +use tari_dan_common_types::{shard::Shard, Epoch, NodeHeight}; use tari_dan_storage::{consensus_models, consensus_models::SubstateDestroyed, StorageError}; use time::PrimitiveDateTime; @@ -20,11 +20,13 @@ pub struct SubstateRecord { pub created_justify: String, pub created_block: String, pub created_height: i64, + pub created_at_epoch: i64, + pub created_by_shard: i32, pub destroyed_by_transaction: Option, pub destroyed_justify: Option, - pub destroyed_by_block: Option, - pub created_at_epoch: i64, + pub destroyed_by_block: Option, pub destroyed_at_epoch: Option, + pub destroyed_by_shard: Option, pub created_at: PrimitiveDateTime, pub destroyed_at: Option, } @@ -43,16 +45,21 @@ impl TryFrom for consensus_models::SubstateRecord { details: "destroyed_justify not provided".to_string(), } })?)?, - by_block: deserialize_hex_try_from(value.destroyed_by_block.as_deref().ok_or_else(|| { + by_block: value.destroyed_by_block.map(|v| NodeHeight(v as u64)).ok_or_else(|| { StorageError::DataInconsistency { details: "destroyed_by_block not provided".to_string(), } - })?)?, + })?, at_epoch: value.destroyed_at_epoch.map(|x| Epoch(x as u64)).ok_or_else(|| { StorageError::DataInconsistency { details: "destroyed_at_epoch not provided".to_string(), } })?, + by_shard: value.destroyed_by_shard.map(|x| Shard::from(x as u32)).ok_or_else(|| { + StorageError::DataInconsistency { + details: "destroyed_by_shard not provided".to_string(), + } + })?, }) }) .transpose()?; @@ -68,6 +75,7 @@ impl TryFrom for consensus_models::SubstateRecord { created_height: NodeHeight(value.created_height as u64), destroyed, created_at_epoch: Epoch(value.created_at_epoch as u64), + created_by_shard: Shard::from(value.created_by_shard as u32), }) } } diff --git a/dan_layer/state_store_sqlite/src/writer.rs b/dan_layer/state_store_sqlite/src/writer.rs index c5835eb85..03163daea 100644 --- a/dan_layer/state_store_sqlite/src/writer.rs +++ b/dan_layer/state_store_sqlite/src/writer.rs @@ -4,6 +4,7 @@ use std::ops::Deref; use diesel::{ + dsl, sql_types::Text, AsChangeset, ExpressionMethods, @@ -13,7 +14,7 @@ use diesel::{ SqliteConnection, }; use log::*; -use tari_dan_common_types::{optional::Optional, Epoch, NodeAddressable, NodeHeight, SubstateAddress}; +use tari_dan_common_types::{optional::Optional, shard::Shard, Epoch, NodeAddressable, NodeHeight}; use tari_dan_storage::{ consensus_models::{ Block, @@ -49,7 +50,8 @@ use tari_dan_storage::{ StorageError, }; use tari_engine_types::substate::SubstateId; -use tari_transaction::TransactionId; +use tari_state_tree::{Node, NodeKey, StaleTreeNode, TreeNode, Version}; +use tari_transaction::{TransactionId, VersionedSubstateId}; use tari_utilities::ByteArray; use time::{OffsetDateTime, PrimitiveDateTime}; @@ -158,6 +160,7 @@ impl<'a, TAddr: NodeAddressable> SqliteStateStoreWriteTransaction<'a, TAddr> { parked_blocks::justify.eq(serialize_json(block.justify())?), parked_blocks::foreign_indexes.eq(serialize_json(block.foreign_indexes())?), parked_blocks::block_time.eq(block.block_time().map(|v| v as i64)), + parked_blocks::signature.eq(block.signature().map(serialize_json).transpose()?), parked_blocks::timestamp.eq(block.timestamp() as i64), parked_blocks::base_layer_block_height.eq(block.base_layer_block_height() as i64), parked_blocks::base_layer_block_hash.eq(serialize_hex(block.base_layer_block_hash())), @@ -208,7 +211,7 @@ impl<'tx, TAddr: NodeAddressable + 'tx> StateStoreWriteTransaction for SqliteSta blocks::qc_id.eq(serialize_hex(block.justify().id())), blocks::is_dummy.eq(block.is_dummy()), blocks::is_processed.eq(block.is_processed()), - blocks::signature.eq(block.get_signature().map(serialize_json).transpose()?), + blocks::signature.eq(block.signature().map(serialize_json).transpose()?), blocks::foreign_indexes.eq(serialize_json(block.foreign_indexes())?), blocks::timestamp.eq(block.timestamp() as i64), blocks::base_layer_block_height.eq(block.base_layer_block_height() as i64), @@ -367,6 +370,7 @@ impl<'tx, TAddr: NodeAddressable + 'tx> StateStoreWriteTransaction for SqliteSta let insert = ( last_voted::block_id.eq(serialize_hex(last_voted.block_id)), last_voted::height.eq(last_voted.height.as_u64() as i64), + last_voted::epoch.eq(last_voted.epoch.as_u64() as i64), ); diesel::insert_into(last_voted::table) @@ -401,6 +405,7 @@ impl<'tx, TAddr: NodeAddressable + 'tx> StateStoreWriteTransaction for SqliteSta let insert = ( last_executed::block_id.eq(serialize_hex(last_exec.block_id)), last_executed::height.eq(last_exec.height.as_u64() as i64), + last_executed::epoch.eq(last_exec.epoch.as_u64() as i64), ); diesel::insert_into(last_executed::table) @@ -420,6 +425,7 @@ impl<'tx, TAddr: NodeAddressable + 'tx> StateStoreWriteTransaction for SqliteSta let insert = ( last_proposed::block_id.eq(serialize_hex(last_proposed.block_id)), last_proposed::height.eq(last_proposed.height.as_u64() as i64), + last_proposed::epoch.eq(last_proposed.epoch.as_u64() as i64), ); diesel::insert_into(last_proposed::table) @@ -454,6 +460,7 @@ impl<'tx, TAddr: NodeAddressable + 'tx> StateStoreWriteTransaction for SqliteSta let insert = ( leaf_blocks::block_id.eq(serialize_hex(leaf_node.block_id)), leaf_blocks::block_height.eq(leaf_node.height.as_u64() as i64), + leaf_blocks::epoch.eq(leaf_node.epoch.as_u64() as i64), ); diesel::insert_into(leaf_blocks::table) @@ -470,20 +477,10 @@ impl<'tx, TAddr: NodeAddressable + 'tx> StateStoreWriteTransaction for SqliteSta fn locked_block_set(&mut self, locked_block: &LockedBlock) -> Result<(), StorageError> { use crate::schema::locked_block; - if let Some(existing) = self.locked_block_get().optional()? { - if locked_block.height <= existing.height { - return Err(StorageError::QueryError { - reason: format!( - "Locked block height {} is not greater than existing height {}", - locked_block.height, existing.height - ), - }); - } - } - let insert = ( locked_block::block_id.eq(serialize_hex(locked_block.block_id)), locked_block::height.eq(locked_block.height.as_u64() as i64), + locked_block::epoch.eq(locked_block.epoch.as_u64() as i64), ); diesel::insert_into(locked_block::table) @@ -503,6 +500,7 @@ impl<'tx, TAddr: NodeAddressable + 'tx> StateStoreWriteTransaction for SqliteSta let insert = ( high_qcs::block_id.eq(serialize_hex(high_qc.block_id)), high_qcs::block_height.eq(high_qc.block_height().as_u64() as i64), + high_qcs::epoch.eq(high_qc.epoch().as_u64() as i64), high_qcs::qc_id.eq(serialize_hex(high_qc.qc_id)), ); @@ -521,7 +519,7 @@ impl<'tx, TAddr: NodeAddressable + 'tx> StateStoreWriteTransaction for SqliteSta use crate::schema::foreign_proposals; let values = ( - foreign_proposals::bucket.eq(foreign_proposal.bucket.as_u32() as i32), + foreign_proposals::bucket.eq(foreign_proposal.shard.as_u32() as i32), foreign_proposals::block_id.eq(serialize_hex(foreign_proposal.block_id)), foreign_proposals::state.eq(foreign_proposal.state.to_string()), foreign_proposals::proposed_height.eq(foreign_proposal.proposed_height.map(|h| h.as_u64() as i64)), @@ -546,7 +544,7 @@ impl<'tx, TAddr: NodeAddressable + 'tx> StateStoreWriteTransaction for SqliteSta use crate::schema::foreign_proposals; diesel::delete(foreign_proposals::table) - .filter(foreign_proposals::bucket.eq(foreign_proposal.bucket.as_u32() as i32)) + .filter(foreign_proposals::bucket.eq(foreign_proposal.shard.as_u32() as i32)) .filter(foreign_proposals::block_id.eq(serialize_hex(foreign_proposal.block_id))) .execute(self.connection()) .map_err(|e| SqliteStorageError::DieselError { @@ -805,6 +803,7 @@ impl<'tx, TAddr: NodeAddressable + 'tx> StateStoreWriteTransaction for SqliteSta diesel::insert_or_ignore_into(transaction_executions::table) .values(insert) + .on_conflict_do_nothing() .execute(self.connection()) .map_err(|e| SqliteStorageError::DieselError { operation: "transaction_executions_insert", @@ -1306,63 +1305,130 @@ impl<'tx, TAddr: NodeAddressable + 'tx> StateStoreWriteTransaction for SqliteSta Ok(()) } - fn substate_down_many>( + fn substates_create(&mut self, substate: SubstateRecord) -> Result<(), StorageError> { + use crate::schema::{state_transitions, substates}; + + if substate.is_destroyed() { + return Err(StorageError::QueryError { + reason: format!( + "calling substates_create with a destroyed SubstateRecord is not valid. substate_id = {}", + substate.substate_id + ), + }); + } + + let values = ( + substates::address.eq(serialize_hex(substate.to_substate_address())), + substates::substate_id.eq(substate.substate_id.to_string()), + substates::version.eq(substate.version as i32), + substates::data.eq(serialize_json(&substate.substate_value)?), + substates::state_hash.eq(serialize_hex(substate.state_hash)), + substates::created_by_transaction.eq(serialize_hex(substate.created_by_transaction)), + substates::created_justify.eq(serialize_hex(substate.created_justify)), + substates::created_block.eq(serialize_hex(substate.created_block)), + substates::created_height.eq(substate.created_height.as_u64() as i64), + substates::created_at_epoch.eq(substate.created_at_epoch.as_u64() as i64), + substates::created_by_shard.eq(substate.created_by_shard.as_u32() as i32), + ); + + diesel::insert_into(substates::table) + .values(values) + .execute(self.connection()) + .map_err(|e| SqliteStorageError::DieselError { + operation: "substates_create", + source: e, + })?; + + let seq = state_transitions::table + .select(dsl::max(state_transitions::seq)) + .filter(state_transitions::epoch.eq(substate.created_at_epoch.as_u64() as i64)) + .first::>(self.connection()) + .map_err(|e| SqliteStorageError::DieselError { + operation: "substates_create", + source: e, + })?; + let next_seq = seq.map(|s| s + 1).unwrap_or(0); + + let values = ( + state_transitions::seq.eq(next_seq), + state_transitions::epoch.eq(substate.created_at_epoch.as_u64() as i64), + state_transitions::shard.eq(substate.created_by_shard.as_u32() as i32), + state_transitions::substate_address.eq(serialize_hex(substate.to_substate_address())), + state_transitions::substate_id.eq(substate.substate_id.to_string()), + state_transitions::version.eq(substate.version as i32), + state_transitions::transition.eq("UP"), + state_transitions::state_hash.eq(serialize_hex(substate.state_hash)), + state_transitions::state_version.eq(substate.created_height.as_u64() as i64), + ); + + diesel::insert_into(state_transitions::table) + .values(values) + .execute(self.connection()) + .map_err(|e| SqliteStorageError::DieselError { + operation: "substates_create", + source: e, + })?; + + Ok(()) + } + + fn substates_down( &mut self, - addresses: I, + versioned_substate_id: VersionedSubstateId, + shard: Shard, epoch: Epoch, - destroyed_block_id: &BlockId, + destroyed_block_height: NodeHeight, destroyed_transaction_id: &TransactionId, destroyed_qc_id: &QcId, ) -> Result<(), StorageError> { - use crate::schema::substates; - - let addresses = addresses.into_iter().map(serialize_hex).collect::>(); + use crate::schema::{state_transitions, substates}; let changes = ( substates::destroyed_at.eq(diesel::dsl::now), substates::destroyed_by_transaction.eq(Some(serialize_hex(destroyed_transaction_id))), - substates::destroyed_by_block.eq(Some(serialize_hex(destroyed_block_id))), + substates::destroyed_by_block.eq(Some(destroyed_block_height.as_u64() as i64)), substates::destroyed_at_epoch.eq(Some(epoch.as_u64() as i64)), + substates::destroyed_by_shard.eq(Some(shard.as_u32() as i32)), substates::destroyed_justify.eq(Some(serialize_hex(destroyed_qc_id))), ); + let address = versioned_substate_id.to_substate_address(); + diesel::update(substates::table) - .filter(substates::address.eq_any(addresses)) + .filter(substates::address.eq(serialize_hex(address))) .set(changes) .execute(self.connection()) .map_err(|e| SqliteStorageError::DieselError { - operation: "substate_down", + operation: "substates_down", source: e, })?; - Ok(()) - } - - fn substates_create(&mut self, substate: SubstateRecord) -> Result<(), StorageError> { - use crate::schema::substates; + let seq = state_transitions::table + .select(dsl::max(state_transitions::seq)) + .filter(state_transitions::epoch.eq(epoch.as_u64() as i64)) + .first::>(self.connection()) + .map_err(|e| SqliteStorageError::DieselError { + operation: "substates_create", + source: e, + })?; + let next_seq = seq.map(|s| s + 1).unwrap_or(0); let values = ( - substates::address.eq(serialize_hex(substate.to_substate_address())), - substates::substate_id.eq(substate.substate_id.to_string()), - substates::version.eq(substate.version as i32), - substates::data.eq(serialize_json(&substate.substate_value)?), - substates::state_hash.eq(serialize_hex(substate.state_hash)), - substates::created_by_transaction.eq(serialize_hex(substate.created_by_transaction)), - substates::created_justify.eq(serialize_hex(substate.created_justify)), - substates::created_block.eq(serialize_hex(substate.created_block)), - substates::created_height.eq(substate.created_height.as_u64() as i64), - substates::created_at_epoch.eq(substate.created_at_epoch.as_u64() as i64), - substates::destroyed_by_transaction.eq(substate.destroyed().map(|d| serialize_hex(d.by_transaction))), - substates::destroyed_justify.eq(substate.destroyed().map(|d| serialize_hex(d.justify))), - substates::destroyed_by_block.eq(substate.destroyed().map(|d| serialize_hex(d.by_block))), - substates::destroyed_at_epoch.eq(substate.destroyed().map(|d| d.at_epoch.as_u64() as i64)), + state_transitions::seq.eq(next_seq), + state_transitions::epoch.eq(epoch.as_u64() as i64), + state_transitions::shard.eq(shard.as_u32() as i32), + state_transitions::substate_address.eq(serialize_hex(address)), + state_transitions::substate_id.eq(versioned_substate_id.substate_id.to_string()), + state_transitions::version.eq(versioned_substate_id.version as i32), + state_transitions::transition.eq("DOWN"), + state_transitions::state_version.eq(destroyed_block_height.as_u64() as i64), ); - diesel::insert_into(substates::table) + diesel::insert_into(state_transitions::table) .values(values) .execute(self.connection()) .map_err(|e| SqliteStorageError::DieselError { - operation: "substate_create", + operation: "substates_down", source: e, })?; @@ -1413,6 +1479,60 @@ impl<'tx, TAddr: NodeAddressable + 'tx> StateStoreWriteTransaction for SqliteSta Ok(()) } + + fn state_tree_nodes_insert( + &mut self, + epoch: Epoch, + shard: Shard, + key: NodeKey, + node: Node, + ) -> Result<(), StorageError> { + use crate::schema::state_tree; + + let node = TreeNode::new_latest(node); + let node = serde_json::to_string(&node).map_err(|e| StorageError::QueryError { + reason: format!("Failed to serialize node: {}", e), + })?; + + let values = ( + state_tree::epoch.eq(epoch.as_u64() as i64), + state_tree::shard.eq(shard.as_u32() as i32), + state_tree::key.eq(key.to_string()), + state_tree::node.eq(node), + ); + diesel::insert_into(state_tree::table) + .values(&values) + .execute(self.connection()) + .map_err(|e| SqliteStorageError::DieselError { + operation: "state_tree_nodes_insert", + source: e, + })?; + + Ok(()) + } + + fn state_tree_nodes_mark_stale_tree_node( + &mut self, + epoch: Epoch, + shard: Shard, + node: StaleTreeNode, + ) -> Result<(), StorageError> { + use crate::schema::state_tree; + + let key = node.as_node_key(); + diesel::update(state_tree::table) + .filter(state_tree::epoch.eq(epoch.as_u64() as i64)) + .filter(state_tree::shard.eq(shard.as_u32() as i32)) + .filter(state_tree::key.eq(key.to_string())) + .set(state_tree::is_stale.eq(true)) + .execute(self.connection()) + .map_err(|e| SqliteStorageError::DieselError { + operation: "state_tree_nodes_mark_stale_tree_node", + source: e, + })?; + + Ok(()) + } } impl<'a, TAddr> Deref for SqliteStateStoreWriteTransaction<'a, TAddr> { @@ -1423,12 +1543,6 @@ impl<'a, TAddr> Deref for SqliteStateStoreWriteTransaction<'a, TAddr> { } } -// impl<'a, TAddr> DerefMut for SqliteStateStoreWriteTransaction<'a, TAddr> { -// fn deref_mut(&mut self) -> &mut Self::Target { -// self.transaction.as_mut().unwrap() -// } -// } - impl Drop for SqliteStateStoreWriteTransaction<'_, TAddr> { fn drop(&mut self) { if self.transaction.is_some() { diff --git a/dan_layer/state_tree/src/jellyfish/types.rs b/dan_layer/state_tree/src/jellyfish/types.rs index 2ee68b633..d18ad6d59 100644 --- a/dan_layer/state_tree/src/jellyfish/types.rs +++ b/dan_layer/state_tree/src/jellyfish/types.rs @@ -1159,23 +1159,23 @@ impl From> for SparseMerkleLeafNode { /// The concrete node type of [`JellyfishMerkleTree`]. #[derive(Clone, Debug, Eq, PartialEq, Serialize, Deserialize)] -pub enum Node { +pub enum Node

{ /// A wrapper of [`InternalNode`]. Internal(InternalNode), /// A wrapper of [`LeafNode`]. - Leaf(LeafNode), + Leaf(LeafNode

), /// Represents empty tree only Null, } -impl From for Node { +impl

From for Node

{ fn from(node: InternalNode) -> Self { Node::Internal(node) } } -impl From> for Node { - fn from(node: LeafNode) -> Self { +impl From> for Node

{ + fn from(node: LeafNode

) -> Self { Node::Leaf(node) } } diff --git a/dan_layer/state_tree/src/tree.rs b/dan_layer/state_tree/src/tree.rs index 93f58cdf2..b1e4e9b92 100644 --- a/dan_layer/state_tree/src/tree.rs +++ b/dan_layer/state_tree/src/tree.rs @@ -56,16 +56,12 @@ impl<'a, S: TreeStore, M: DbKeyMapper> StateTree<'a, S, M> { /// Stores the substate changes in the state tree and returns the new root hash. pub fn put_substate_changes>( &mut self, - current_version: Version, + current_version: Option, next_version: Version, changes: I, ) -> Result { - let (root_hash, update_batch) = calculate_substate_changes::<_, M, _>( - self.store, - Some(current_version).filter(|v| *v > 0), - next_version, - changes, - )?; + let (root_hash, update_batch) = + calculate_substate_changes::<_, M, _>(self.store, current_version, next_version, changes)?; self.commit_diff(update_batch.into())?; Ok(root_hash) @@ -97,7 +93,7 @@ fn calculate_substate_changes< next_version: Version, changes: I, ) -> Result<(Hash, TreeUpdateBatch), StateTreeError> { - let smt = JellyfishMerkleTree::new(store); + let jmt = JellyfishMerkleTree::new(store); let changes = changes .into_iter() @@ -113,7 +109,7 @@ fn calculate_substate_changes< }) .collect::>(); - let (root_hash, update_result) = smt.batch_put_value_set( + let (root_hash, update_result) = jmt.batch_put_value_set( changes .iter() .map(|change| (&change.key, change.new_payload.as_ref())) diff --git a/dan_layer/state_tree/tests/support.rs b/dan_layer/state_tree/tests/support.rs index aabf1dc70..d6f33319d 100644 --- a/dan_layer/state_tree/tests/support.rs +++ b/dan_layer/state_tree/tests/support.rs @@ -59,7 +59,7 @@ impl> HashTreeTester { fn apply_database_updates(&mut self, changes: impl IntoIterator) -> Hash { let next_version = self.current_version.unwrap_or(0) + 1; - let current_version = self.current_version.replace(next_version).unwrap_or(0); + let current_version = self.current_version.replace(next_version); StateTree::<_, IdentityMapper>::new(&mut self.tree_store) .put_substate_changes(current_version, next_version, changes) .unwrap() diff --git a/dan_layer/storage/src/consensus_models/block.rs b/dan_layer/storage/src/consensus_models/block.rs index 545779fb9..a1da04691 100644 --- a/dan_layer/storage/src/consensus_models/block.rs +++ b/dan_layer/storage/src/consensus_models/block.rs @@ -5,7 +5,6 @@ use std::{ collections::{BTreeSet, HashSet}, fmt::{Debug, Display, Formatter}, hash::Hash, - iter, ops::{Deref, RangeInclusive}, }; @@ -130,7 +129,7 @@ impl Block { base_layer_block_hash: FixedHash, ) -> Self { let mut block = Self { - id: BlockId::genesis(), + id: BlockId::zero(), network, parent, justify, @@ -205,21 +204,22 @@ impl Block { } } - pub fn genesis(network: Network) -> Self { + pub fn genesis(network: Network, epoch: Epoch, shard: Shard) -> Self { Self::new( network, - BlockId::genesis(), + BlockId::zero(), QuorumCertificate::genesis(), NodeHeight(0), - Epoch(0), - Shard::from(0), + epoch, + shard, PublicKey::default(), Default::default(), + // TODO: the merkle hash should be initialized to something committing to the previous state. FixedHash::zero(), 0, IndexMap::new(), None, - EpochTime::now().as_u64(), + 0, 0, FixedHash::zero(), ) @@ -229,8 +229,8 @@ impl Block { pub fn zero_block(network: Network) -> Self { Self { network, - id: BlockId::genesis(), - parent: BlockId::genesis(), + id: BlockId::zero(), + parent: BlockId::zero(), justify: QuorumCertificate::genesis(), height: NodeHeight(0), epoch: Epoch(0), @@ -256,7 +256,7 @@ impl Block { network: Network, parent: BlockId, proposed_by: PublicKey, - node_height: NodeHeight, + height: NodeHeight, high_qc: QuorumCertificate, epoch: Epoch, shard: Shard, @@ -265,24 +265,30 @@ impl Block { parent_base_layer_block_height: u64, parent_base_layer_block_hash: FixedHash, ) -> Self { - let mut block = Self::new( + let mut block = Self { + id: BlockId::zero(), network, parent, - high_qc, - node_height, + justify: high_qc, + height, epoch, shard, proposed_by, - Default::default(), - parent_merkle_root, - 0, - IndexMap::new(), - None, - parent_timestamp, - parent_base_layer_block_height, - parent_base_layer_block_hash, - ); - block.is_dummy = true; + merkle_root: parent_merkle_root, + commands: BTreeSet::new(), + total_leader_fee: 0, + is_dummy: true, + is_processed: false, + is_committed: false, + foreign_indexes: IndexMap::new(), + stored_at: None, + signature: None, + block_time: None, + timestamp: parent_timestamp, + base_layer_block_height: parent_base_layer_block_height, + base_layer_block_hash: parent_base_layer_block_hash, + }; + block.id = block.calculate_hash().into(); block.is_processed = false; block } @@ -310,6 +316,7 @@ impl Block { .chain(&self.shard) .chain(&self.proposed_by) .chain(&self.merkle_root) + .chain(&self.is_dummy) .chain(&self.commands) .chain(&self.foreign_indexes) .chain(&self.timestamp) @@ -323,11 +330,7 @@ impl Block { impl Block { pub fn is_genesis(&self) -> bool { - self.id.is_genesis() - } - - pub fn is_epoch_start(&self) -> bool { - self.commands.iter().any(|c| c.is_epoch_start()) + self.height.is_zero() } pub fn is_epoch_end(&self) -> bool { @@ -357,6 +360,7 @@ impl Block { LockedBlock { height: self.height, block_id: self.id, + epoch: self.epoch, } } @@ -364,6 +368,7 @@ impl Block { LastExecuted { height: self.height, block_id: self.id, + epoch: self.epoch, } } @@ -371,6 +376,7 @@ impl Block { LastVoted { height: self.height, block_id: self.id, + epoch: self.epoch, } } @@ -378,6 +384,7 @@ impl Block { LeafBlock { height: self.height, block_id: self.id, + epoch: self.epoch, } } @@ -385,6 +392,7 @@ impl Block { LastProposed { height: self.height, block_id: self.id, + epoch: self.epoch, } } @@ -404,6 +412,10 @@ impl Block { &self.justify } + pub fn into_justify(self) -> QuorumCertificate { + self.justify + } + pub fn justifies_parent(&self) -> bool { *self.justify.block_id() == self.parent } @@ -476,7 +488,7 @@ impl Block { self.timestamp } - pub fn get_signature(&self) -> Option<&ValidatorSchnorrSignature> { + pub fn signature(&self) -> Option<&ValidatorSchnorrSignature> { self.signature.as_ref() } @@ -502,18 +514,22 @@ impl Block { tx.blocks_get(id) } - pub fn get_tip(tx: &TTx) -> Result { - tx.blocks_get_tip() - } - /// Returns all blocks from and excluding the start block (lower height) to the end block (inclusive) pub fn get_all_blocks_between( tx: &TTx, + epoch: Epoch, + shard: Shard, start_block_id_exclusive: &BlockId, end_block_id_inclusive: &BlockId, include_dummy_blocks: bool, ) -> Result, StorageError> { - tx.blocks_get_all_between(start_block_id_exclusive, end_block_id_inclusive, include_dummy_blocks) + tx.blocks_get_all_between( + epoch, + shard, + start_block_id_exclusive, + end_block_id_inclusive, + include_dummy_blocks, + ) } pub fn exists(&self, tx: &TTx) -> Result { @@ -603,6 +619,7 @@ impl Block { id.substate_id, id.version, substate.into_substate_value(), + self.shard(), self.epoch(), self.height(), *self.id(), @@ -612,11 +629,12 @@ impl Block { .create(tx)?; }, SubstateChange::Down { id, transaction_id } => { - SubstateRecord::destroy_many( + SubstateRecord::destroy( tx, - iter::once(id.to_substate_address()), + id, + self.shard(), self.epoch(), - self.id(), + self.height(), self.justify().id(), &transaction_id, )?; @@ -663,12 +681,13 @@ impl Block { } pub fn get_parent(&self, tx: &TTx) -> Result { - if self.id.is_genesis() { + if self.id.is_zero() && self.parent.is_zero() { return Err(StorageError::NotFound { - item: "Block".to_string(), - key: self.id.to_string(), + item: "Block parent".to_string(), + key: self.parent.to_string(), }); } + Block::get(tx, &self.parent) } @@ -731,7 +750,7 @@ impl Block { let committed = self .commands() .iter() - .filter_map(|c| c.local_only().or_else(|| c.accept())) + .filter_map(|c| c.committing()) .filter(|t| t.decision.is_commit()) .collect::>(); @@ -795,12 +814,14 @@ impl Block { return Ok(high_qc); }; - if !precommit_node.is_genesis() { - let locked = LockedBlock::get(&**tx)?; - if precommit_node.height() > locked.height { - on_locked_block_recurse(tx, &locked, &precommit_node, &mut on_lock_block)?; - precommit_node.as_locked_block().set(tx)?; - } + if precommit_node.is_genesis() { + return Ok(high_qc); + } + + let locked = LockedBlock::get(&**tx)?; + if precommit_node.height() > locked.height { + on_locked_block_recurse(tx, &locked, &precommit_node, &mut on_lock_block)?; + precommit_node.as_locked_block().set(tx)?; } // b <- b'.justify.node @@ -817,12 +838,13 @@ impl Block { ); // Commit prepare_node (b) - if !prepare_node.is_genesis() { - let prepare_node = Block::get(&**tx, prepare_node)?; - let last_executed = LastExecuted::get(&**tx)?; - on_commit_block_recurse(tx, &last_executed, &prepare_node, &mut on_commit)?; - prepare_node.as_last_executed().set(tx)?; + if prepare_node.is_zero() { + return Ok(high_qc); } + let prepare_node = Block::get(&**tx, prepare_node)?; + let last_executed = LastExecuted::get(&**tx)?; + on_commit_block_recurse(tx, &last_executed, &prepare_node, &mut on_commit)?; + prepare_node.as_last_executed().set(tx)?; } else { debug!( target: LOG_TARGET, @@ -910,8 +932,9 @@ impl Display for Block { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { write!( f, - "[{}, {}, {} command(s)]", + "[{}, {}, {}, {} command(s)]", self.height(), + self.epoch(), self.id(), self.commands().len() ) @@ -927,6 +950,10 @@ impl BlockId { Self(FixedHash::zero()) } + pub const fn zero() -> Self { + Self(FixedHash::zero()) + } + pub fn new>(hash: T) -> Self { Self(hash.into()) } @@ -939,7 +966,7 @@ impl BlockId { self.0.as_slice() } - pub fn is_genesis(&self) -> bool { + pub fn is_zero(&self) -> bool { self.0.iter().all(|b| *b == 0) } @@ -977,7 +1004,7 @@ impl TryFrom<&[u8]> for BlockId { } impl Display for BlockId { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { Display::fmt(&self.0, f) } } diff --git a/dan_layer/storage/src/consensus_models/command.rs b/dan_layer/storage/src/consensus_models/command.rs index f68d402d5..45554b8c2 100644 --- a/dan_layer/storage/src/consensus_models/command.rs +++ b/dan_layer/storage/src/consensus_models/command.rs @@ -43,7 +43,7 @@ impl Evidence { pub fn from_inputs_and_outputs( transaction_id: TransactionId, - resolved_inputs: &IndexSet, + resolved_inputs: &[VersionedSubstateIdLockIntent], resulting_outputs: &[VersionedSubstateId], ) -> Self { let mut deduped_evidence = HashMap::new(); @@ -230,21 +230,14 @@ pub enum Command { Accept(TransactionAtom), ForeignProposal(ForeignProposal), LocalOnly(TransactionAtom), - EpochEvent(EpochEvent), -} - -#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize, Ord, PartialOrd)] -#[cfg_attr(feature = "ts", derive(TS), ts(export, export_to = "../../bindings/src/types/"))] -pub enum EpochEvent { - Start, - End, + EndEpoch, } #[derive(PartialEq, Eq, Ord, PartialOrd)] pub enum CommandId { TransactionId(TransactionId), ForeignProposal(ForeignProposal), - EpochEvent(EpochEvent), + EndEpoch, } impl Display for CommandId { @@ -252,7 +245,7 @@ impl Display for CommandId { match self { CommandId::TransactionId(id) => write!(f, "Transaction({})", id), CommandId::ForeignProposal(fp) => write!(f, "ForeignProposal({})", fp.block_id), - CommandId::EpochEvent(event) => write!(f, "EpochEvent({:?})", event), + CommandId::EndEpoch => write!(f, "EndEpoch"), } } } @@ -265,7 +258,7 @@ impl Command { Command::Accept(tx) => Some(tx), Command::LocalOnly(tx) => Some(tx), Command::ForeignProposal(_) => None, - Command::EpochEvent(_) => None, + Command::EndEpoch => None, } } @@ -276,7 +269,7 @@ impl Command { Command::Accept(tx) => CommandId::TransactionId(tx.id), Command::LocalOnly(tx) => CommandId::TransactionId(tx.id), Command::ForeignProposal(foreign_proposal) => CommandId::ForeignProposal(foreign_proposal.clone()), - Command::EpochEvent(event) => CommandId::EpochEvent(event.clone()), + Command::EndEpoch => CommandId::EndEpoch, } } @@ -325,12 +318,8 @@ impl Command { committing.filter(|t| t.decision.is_commit()) } - pub fn is_epoch_start(&self) -> bool { - matches!(self, Command::EpochEvent(EpochEvent::Start)) - } - pub fn is_epoch_end(&self) -> bool { - matches!(self, Command::EpochEvent(EpochEvent::End)) + matches!(self, Command::EndEpoch) } pub fn involved_shards(&self) -> impl Iterator + '_ { @@ -340,7 +329,7 @@ impl Command { Command::Accept(tx) => tx.evidence.substate_addresses_iter(), Command::LocalOnly(tx) => tx.evidence.substate_addresses_iter(), Command::ForeignProposal(_) => panic!("ForeignProposal does not have involved shards"), - Command::EpochEvent(_) => panic!("EpochEvent does not have involved shards"), + Command::EndEpoch => panic!("EpochEvent does not have involved shards"), } } @@ -351,7 +340,7 @@ impl Command { Command::Accept(tx) => &tx.evidence, Command::LocalOnly(tx) => &tx.evidence, Command::ForeignProposal(_) => panic!("ForeignProposal does not have evidence"), - Command::EpochEvent(_) => panic!("EpochEvent does not have evidence"), + Command::EndEpoch => panic!("EpochEvent does not have evidence"), } } } @@ -376,7 +365,7 @@ impl Display for Command { Command::Accept(tx) => write!(f, "Accept({}, {})", tx.id, tx.decision), Command::LocalOnly(tx) => write!(f, "LocalOnly({}, {})", tx.id, tx.decision), Command::ForeignProposal(fp) => write!(f, "ForeignProposal {}", fp.block_id), - Command::EpochEvent(event) => write!(f, "EpochEvent {:?}", event), + Command::EndEpoch => write!(f, "EndEpoch"), } } } diff --git a/dan_layer/storage/src/consensus_models/epoch_checkpoint.rs b/dan_layer/storage/src/consensus_models/epoch_checkpoint.rs new file mode 100644 index 000000000..601f79483 --- /dev/null +++ b/dan_layer/storage/src/consensus_models/epoch_checkpoint.rs @@ -0,0 +1,66 @@ +// Copyright 2024 The Tari Project +// SPDX-License-Identifier: BSD-3-Clause + +use std::fmt::Display; + +use tari_dan_common_types::{shard::Shard, Epoch}; + +use crate::{ + consensus_models::{Block, QuorumCertificate}, + StateStoreReadTransaction, + StorageError, +}; + +#[derive(Debug, Clone)] +pub struct EpochCheckpoint { + block: Block, + qcs: Vec, +} + +impl EpochCheckpoint { + pub fn new(block: Block, qcs: Vec) -> Self { + Self { block, qcs } + } + + pub fn qcs(&self) -> &[QuorumCertificate] { + &self.qcs + } + + pub fn block(&self) -> &Block { + &self.block + } +} + +impl EpochCheckpoint { + pub fn generate( + tx: &TTx, + epoch: Epoch, + shard: Shard, + ) -> Result { + let mut blocks = tx.blocks_get_last_n_in_epoch(3, epoch, shard)?; + if blocks.is_empty() { + return Err(StorageError::NotFound { + item: format!("EpochCheckpoint: No blocks found for epoch {epoch}, shard {shard}"), + key: epoch.to_string(), + }); + } + + let commit_block = blocks.pop().unwrap(); + let qcs = blocks.into_iter().map(|b| b.into_justify()).collect(); + + Ok(Self { + block: commit_block, + qcs, + }) + } +} + +impl Display for EpochCheckpoint { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "EpochCheckpoint: block={}, qcs=", self.block)?; + for qc in self.qcs() { + write!(f, "{}, ", qc.id())?; + } + Ok(()) + } +} diff --git a/dan_layer/storage/src/consensus_models/executed_transaction.rs b/dan_layer/storage/src/consensus_models/executed_transaction.rs index 0c8821efe..47720e200 100644 --- a/dan_layer/storage/src/consensus_models/executed_transaction.rs +++ b/dan_layer/storage/src/consensus_models/executed_transaction.rs @@ -9,7 +9,6 @@ use std::{ time::Duration, }; -use indexmap::IndexSet; use serde::{Deserialize, Serialize}; use tari_dan_common_types::{optional::Optional, SubstateAddress}; use tari_engine_types::commit_result::ExecuteResult; @@ -40,7 +39,7 @@ pub struct ExecutedTransaction { transaction: Transaction, result: ExecuteResult, resulting_outputs: Vec, - resolved_inputs: IndexSet, + resolved_inputs: Vec, #[cfg_attr(feature = "ts", ts(type = "{secs: number, nanos: number}"))] execution_time: Duration, final_decision: Option, @@ -53,7 +52,7 @@ impl ExecutedTransaction { pub fn new( transaction: Transaction, result: ExecuteResult, - resolved_inputs: IndexSet, + resolved_inputs: Vec, resulting_outputs: Vec, execution_time: Duration, ) -> Self { @@ -144,7 +143,7 @@ impl ExecutedTransaction { &self.resulting_outputs } - pub fn resolved_inputs(&self) -> &IndexSet { + pub fn resolved_inputs(&self) -> &[VersionedSubstateIdLockIntent] { &self.resolved_inputs } @@ -153,7 +152,7 @@ impl ExecutedTransaction { ) -> ( Transaction, ExecuteResult, - IndexSet, + Vec, Vec, ) { ( @@ -405,6 +404,18 @@ impl VersionedSubstateIdLockIntent { } } + pub fn read(versioned_substate_id: VersionedSubstateId) -> Self { + Self::new(versioned_substate_id, SubstateLockFlag::Read) + } + + pub fn write(versioned_substate_id: VersionedSubstateId) -> Self { + Self::new(versioned_substate_id, SubstateLockFlag::Write) + } + + pub fn output(versioned_substate_id: VersionedSubstateId) -> Self { + Self::new(versioned_substate_id, SubstateLockFlag::Output) + } + pub fn to_substate_address(&self) -> SubstateAddress { self.versioned_substate_id.to_substate_address() } diff --git a/dan_layer/storage/src/consensus_models/foreign_proposal.rs b/dan_layer/storage/src/consensus_models/foreign_proposal.rs index 6ad0eb948..7c9b9c9ca 100644 --- a/dan_layer/storage/src/consensus_models/foreign_proposal.rs +++ b/dan_layer/storage/src/consensus_models/foreign_proposal.rs @@ -51,7 +51,7 @@ impl FromStr for ForeignProposalState { #[cfg_attr(feature = "ts", derive(TS), ts(export, export_to = "../../bindings/src/types/"))] pub struct ForeignProposal { #[cfg_attr(feature = "ts", ts(type = "number"))] - pub bucket: Shard, + pub shard: Shard, #[cfg_attr(feature = "ts", ts(type = "string"))] pub block_id: BlockId, pub state: ForeignProposalState, @@ -64,13 +64,13 @@ pub struct ForeignProposal { impl ForeignProposal { pub fn new( - bucket: Shard, + shard: Shard, block_id: BlockId, transactions: Vec, base_layer_block_height: u64, ) -> Self { Self { - bucket, + shard, block_id, state: ForeignProposalState::New, proposed_height: None, diff --git a/dan_layer/storage/src/consensus_models/high_qc.rs b/dan_layer/storage/src/consensus_models/high_qc.rs index fe4120251..dc7cdab40 100644 --- a/dan_layer/storage/src/consensus_models/high_qc.rs +++ b/dan_layer/storage/src/consensus_models/high_qc.rs @@ -23,7 +23,7 @@ use std::fmt::Display; use serde::Serialize; -use tari_dan_common_types::NodeHeight; +use tari_dan_common_types::{Epoch, NodeHeight}; use crate::{ consensus_models::{Block, BlockId, LeafBlock, QcId, QuorumCertificate}, @@ -36,6 +36,7 @@ use crate::{ pub struct HighQc { pub block_id: BlockId, pub block_height: NodeHeight, + pub epoch: Epoch, pub qc_id: QcId, } @@ -52,10 +53,15 @@ impl HighQc { &self.qc_id } + pub fn epoch(&self) -> Epoch { + self.epoch + } + pub fn as_leaf_block(&self) -> LeafBlock { LeafBlock { block_id: self.block_id, height: self.block_height, + epoch: self.epoch, } } } diff --git a/dan_layer/storage/src/consensus_models/last_executed.rs b/dan_layer/storage/src/consensus_models/last_executed.rs index aebad2d86..7e5d8f34f 100644 --- a/dan_layer/storage/src/consensus_models/last_executed.rs +++ b/dan_layer/storage/src/consensus_models/last_executed.rs @@ -1,13 +1,14 @@ // Copyright 2023 The Tari Project // SPDX-License-Identifier: BSD-3-Clause -use tari_dan_common_types::NodeHeight; +use tari_dan_common_types::{Epoch, NodeHeight}; use crate::{consensus_models::BlockId, StateStoreReadTransaction, StateStoreWriteTransaction, StorageError}; pub struct LastExecuted { pub height: NodeHeight, pub block_id: BlockId, + pub epoch: Epoch, } impl LastExecuted { diff --git a/dan_layer/storage/src/consensus_models/last_proposed.rs b/dan_layer/storage/src/consensus_models/last_proposed.rs index 6b6609bee..7278ccb58 100644 --- a/dan_layer/storage/src/consensus_models/last_proposed.rs +++ b/dan_layer/storage/src/consensus_models/last_proposed.rs @@ -1,7 +1,7 @@ // Copyright 2023 The Tari Project // SPDX-License-Identifier: BSD-3-Clause -use tari_dan_common_types::NodeHeight; +use tari_dan_common_types::{Epoch, NodeHeight}; use crate::{ consensus_models::{Block, BlockId}, @@ -13,6 +13,7 @@ use crate::{ pub struct LastProposed { pub height: NodeHeight, pub block_id: BlockId, + pub epoch: Epoch, } impl LastProposed { @@ -35,6 +36,10 @@ impl LastProposed { impl std::fmt::Display for LastProposed { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "(height: {}, block_id: {})", self.height, self.block_id) + write!( + f, + "LastProposed({}, BlockId({}), {})", + self.height, self.block_id, self.epoch + ) } } diff --git a/dan_layer/storage/src/consensus_models/last_voted.rs b/dan_layer/storage/src/consensus_models/last_voted.rs index 5fe38cdc7..9d08b094e 100644 --- a/dan_layer/storage/src/consensus_models/last_voted.rs +++ b/dan_layer/storage/src/consensus_models/last_voted.rs @@ -1,13 +1,14 @@ // Copyright 2023 The Tari Project // SPDX-License-Identifier: BSD-3-Clause -use tari_dan_common_types::NodeHeight; +use tari_dan_common_types::{Epoch, NodeHeight}; use crate::{consensus_models::BlockId, StateStoreReadTransaction, StateStoreWriteTransaction, StorageError}; pub struct LastVoted { pub block_id: BlockId, pub height: NodeHeight, + pub epoch: Epoch, } impl LastVoted { @@ -18,6 +19,10 @@ impl LastVoted { pub fn height(&self) -> NodeHeight { self.height } + + pub fn epoch(&self) -> Epoch { + self.epoch + } } impl LastVoted { @@ -36,6 +41,10 @@ impl LastVoted { impl std::fmt::Display for LastVoted { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "(block_id: {}, height: {})", self.block_id, self.height) + write!( + f, + "LastVoted(BlockId({}), {}, {})", + self.block_id, self.height, self.epoch + ) } } diff --git a/dan_layer/storage/src/consensus_models/leaf_block.rs b/dan_layer/storage/src/consensus_models/leaf_block.rs index c73c63513..81ad7c0ef 100644 --- a/dan_layer/storage/src/consensus_models/leaf_block.rs +++ b/dan_layer/storage/src/consensus_models/leaf_block.rs @@ -22,7 +22,7 @@ use std::fmt::Display; -use tari_dan_common_types::NodeHeight; +use tari_dan_common_types::{Epoch, NodeHeight}; use crate::{ consensus_models::{Block, BlockId}, @@ -35,20 +35,10 @@ use crate::{ pub struct LeafBlock { pub block_id: BlockId, pub height: NodeHeight, + pub epoch: Epoch, } impl LeafBlock { - pub fn is_genesis(&self) -> bool { - self.block_id.is_genesis() - } - - pub fn genesis() -> Self { - Self { - block_id: BlockId::genesis(), - height: NodeHeight(0), - } - } - pub fn height(&self) -> NodeHeight { self.height } @@ -56,6 +46,10 @@ impl LeafBlock { pub fn block_id(&self) -> &BlockId { &self.block_id } + + pub fn epoch(&self) -> Epoch { + self.epoch + } } impl LeafBlock { @@ -74,6 +68,6 @@ impl LeafBlock { impl Display for LeafBlock { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "{} #{}", self.block_id, self.height) + write!(f, "LeafBlock({} {} {})", self.height, self.block_id, self.epoch) } } diff --git a/dan_layer/storage/src/consensus_models/locked_block.rs b/dan_layer/storage/src/consensus_models/locked_block.rs index 57e95b960..f5bec74f6 100644 --- a/dan_layer/storage/src/consensus_models/locked_block.rs +++ b/dan_layer/storage/src/consensus_models/locked_block.rs @@ -3,7 +3,7 @@ use std::fmt::Display; -use tari_dan_common_types::NodeHeight; +use tari_dan_common_types::{Epoch, NodeHeight}; use crate::{ consensus_models::{Block, BlockId}, @@ -16,6 +16,7 @@ use crate::{ pub struct LockedBlock { pub height: NodeHeight, pub block_id: BlockId, + pub epoch: Epoch, } impl LockedBlock { @@ -26,6 +27,10 @@ impl LockedBlock { pub fn block_id(&self) -> &BlockId { &self.block_id } + + pub fn epoch(&self) -> Epoch { + self.epoch + } } impl LockedBlock { diff --git a/dan_layer/storage/src/consensus_models/mod.rs b/dan_layer/storage/src/consensus_models/mod.rs index b3167de66..162f9e282 100644 --- a/dan_layer/storage/src/consensus_models/mod.rs +++ b/dan_layer/storage/src/consensus_models/mod.rs @@ -4,6 +4,7 @@ mod block; mod block_diff; mod command; +mod epoch_checkpoint; mod executed_transaction; mod foreign_proposal; mod foreign_receive_counters; @@ -17,6 +18,7 @@ mod leaf_block; mod locked_block; mod quorum; mod quorum_certificate; +mod state_transition; mod state_tree_diff; mod substate; mod substate_change; @@ -33,6 +35,7 @@ mod vote_signature; pub use block::*; pub use block_diff::*; pub use command::*; +pub use epoch_checkpoint::*; pub use executed_transaction::*; pub use foreign_proposal::*; pub use foreign_receive_counters::*; @@ -46,6 +49,7 @@ pub use leaf_block::*; pub use locked_block::*; pub use quorum::*; pub use quorum_certificate::*; +pub use state_transition::*; pub use state_tree_diff::*; pub use substate::*; pub use substate_change::*; diff --git a/dan_layer/storage/src/consensus_models/quorum_certificate.rs b/dan_layer/storage/src/consensus_models/quorum_certificate.rs index 623ca0c8f..523aca80c 100644 --- a/dan_layer/storage/src/consensus_models/quorum_certificate.rs +++ b/dan_layer/storage/src/consensus_models/quorum_certificate.rs @@ -55,7 +55,7 @@ impl QuorumCertificate { ) -> Self { leaf_hashes.sort(); let mut qc = Self { - qc_id: QcId::genesis(), + qc_id: QcId::zero(), block_id: block, block_height, epoch, @@ -70,7 +70,7 @@ impl QuorumCertificate { pub fn genesis() -> Self { Self::new( - BlockId::genesis(), + BlockId::zero(), NodeHeight::zero(), Epoch(0), Shard::from(0), @@ -92,15 +92,11 @@ impl QuorumCertificate { .result() .into() } - - pub fn is_valid(&self) -> bool { - true - } } impl QuorumCertificate { - pub fn is_genesis(&self) -> bool { - self.block_id.is_genesis() + pub fn is_zero(&self) -> bool { + self.block_id.is_zero() } pub fn id(&self) -> &QcId { @@ -139,6 +135,7 @@ impl QuorumCertificate { HighQc { block_id: self.block_id, block_height: self.block_height, + epoch: self.epoch, qc_id: self.qc_id, } } @@ -147,6 +144,7 @@ impl QuorumCertificate { LeafBlock { block_id: self.block_id, height: self.block_height, + epoch: self.epoch, } } @@ -154,6 +152,7 @@ impl QuorumCertificate { LastVoted { block_id: self.block_id, height: self.block_height, + epoch: self.epoch, } } } @@ -197,7 +196,7 @@ impl QuorumCertificate { let mut high_qc = HighQc::get(&**tx)?; if high_qc.block_height() < self.block_height() { - debug!( + info!( target: LOG_TARGET, "πŸ”₯ UPDATE_HIGH_QC ({}, previous high QC: {} {})", self, @@ -247,7 +246,8 @@ impl Display for QuorumCertificate { pub struct QcId(#[serde(with = "serde_with::hex")] FixedHash); impl QcId { - pub const fn genesis() -> Self { + /// Represents a zero/null QC. This QC is used to represent the unsigned initial QC. + pub const fn zero() -> Self { Self(FixedHash::zero()) } @@ -263,7 +263,7 @@ impl QcId { self.0.as_slice() } - pub fn is_genesis(&self) -> bool { + pub fn is_zero(&self) -> bool { self.0.iter().all(|b| *b == 0) } } diff --git a/dan_layer/storage/src/consensus_models/state_transition.rs b/dan_layer/storage/src/consensus_models/state_transition.rs new file mode 100644 index 000000000..d5bba0600 --- /dev/null +++ b/dan_layer/storage/src/consensus_models/state_transition.rs @@ -0,0 +1,133 @@ +// Copyright 2024 The Tari Project +// SPDX-License-Identifier: BSD-3-Clause + +use std::{ + fmt::{Display, Formatter}, + io::{Read, Write}, + mem, +}; + +use tari_dan_common_types::{shard::Shard, Epoch}; + +use crate::{consensus_models::SubstateUpdate, StateStoreReadTransaction, StorageError}; + +#[derive(Debug, Clone)] +pub struct StateTransition { + pub id: StateTransitionId, + pub update: SubstateUpdate, + pub state_tree_version: u64, +} + +impl StateTransition { + pub fn get_n_after( + tx: &TTx, + n: usize, + after_id: StateTransitionId, + end_epoch: Epoch, + ) -> Result, StorageError> { + tx.state_transitions_get_n_after(n, after_id, end_epoch) + } + + pub fn get_last_id(tx: &TTx) -> Result { + tx.state_transitions_get_last_id() + } +} + +impl Display for StateTransition { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "{}: {}", self.id, self.update) + } +} + +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub struct StateTransitionId { + epoch: Epoch, + shard: Shard, + seq: u64, +} +impl StateTransitionId { + const BYTE_SIZE: usize = mem::size_of::(); + + pub fn new(epoch: Epoch, shard: Shard, seq: u64) -> Self { + Self { epoch, shard, seq } + } + + pub fn from_bytes(mut bytes: &[u8]) -> Option { + if bytes.len() < Self::BYTE_SIZE { + return None; + } + let bytes_mut = &mut bytes; + let epoch = Epoch(u64::from_le_bytes(copy_fixed(bytes_mut))); + let shard = Shard::from(u32::from_le_bytes(copy_fixed(bytes_mut))); + let seq = u64::from_le_bytes(copy_fixed(bytes_mut)); + Some(Self::new(epoch, shard, seq)) + } + + pub fn as_bytes(&self) -> [u8; Self::BYTE_SIZE] { + let mut buf = [0u8; Self::BYTE_SIZE]; + let buf_mut = &mut buf.as_mut_slice(); + write_fixed(self.epoch.to_le_bytes(), buf_mut); + write_fixed(self.shard.as_u32().to_le_bytes(), buf_mut); + write_fixed(self.seq.to_le_bytes(), buf_mut); + buf + } + + pub fn epoch(&self) -> Epoch { + self.epoch + } + + pub fn shard(&self) -> Shard { + self.shard + } + + pub fn seq(self) -> u64 { + self.seq + } +} + +impl Display for StateTransitionId { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!( + f, + "StateTransition(epoch = {}, shard = {}, seq = {})", + self.epoch(), + self.shard(), + self.seq() + ) + } +} + +/// Copies bytes into a fixed byte array. +/// +/// ## Panics +/// Caller must ensure that sufficient bytes remain on the mut ref to the input slice. +fn copy_fixed(bytes: &mut &[u8]) -> [u8; SZ] { + let mut buf = [0u8; SZ]; + bytes + .read_exact(&mut buf) + .expect("copy_fixed: Expected enough bytes to read"); + buf +} + +/// Writes fixed bytes into a buffer. +/// ## Panics +/// Caller must ensure that the buffer has sufficient space for the fixed bytes. +fn write_fixed(buf: [u8; SZ], out: &mut &mut [u8]) { + out.write_all(&buf) + .expect("write_fixed: Expected buffer to have sufficient space for fixed bytes"); +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn to_and_from_bytes() { + let id = StateTransitionId::new(Epoch(1), Shard::from(2), 3); + let bytes = id.as_bytes(); + let id2 = StateTransitionId::from_bytes(&bytes).unwrap(); + assert_eq!(id, id2); + + assert_eq!(StateTransitionId::from_bytes(&[1, 2, 3]), None); + } +} diff --git a/dan_layer/storage/src/consensus_models/state_tree_diff.rs b/dan_layer/storage/src/consensus_models/state_tree_diff.rs index 13491e09f..f3dd81fad 100644 --- a/dan_layer/storage/src/consensus_models/state_tree_diff.rs +++ b/dan_layer/storage/src/consensus_models/state_tree_diff.rs @@ -6,7 +6,7 @@ use std::ops::Deref; -use tari_dan_common_types::NodeHeight; +use tari_dan_common_types::{shard::Shard, Epoch, NodeHeight}; use crate::{consensus_models::BlockId, StateStoreReadTransaction, StateStoreWriteTransaction, StorageError}; @@ -29,9 +29,16 @@ impl PendingStateTreeDiff { impl PendingStateTreeDiff { /// Returns all pending state tree diffs from the last committed block (exclusive) to the given block (inclusive). - pub fn get_all_up_to_commit_block(tx: &TTx, block_id: &BlockId) -> Result, StorageError> - where TTx: StateStoreReadTransaction { - tx.pending_state_tree_diffs_get_all_up_to_commit_block(block_id) + pub fn get_all_up_to_commit_block( + tx: &TTx, + epoch: Epoch, + shard: Shard, + block_id: &BlockId, + ) -> Result, StorageError> + where + TTx: StateStoreReadTransaction, + { + tx.pending_state_tree_diffs_get_all_up_to_commit_block(epoch, shard, block_id) } pub fn remove_by_block(tx: &mut TTx, block_id: &BlockId) -> Result diff --git a/dan_layer/storage/src/consensus_models/substate.rs b/dan_layer/storage/src/consensus_models/substate.rs index 2366884ff..3837641e2 100644 --- a/dan_layer/storage/src/consensus_models/substate.rs +++ b/dan_layer/storage/src/consensus_models/substate.rs @@ -5,28 +5,26 @@ use std::{ borrow::Borrow, collections::HashSet, fmt, + fmt::Display, hash::Hash, iter, - ops::{Deref, RangeInclusive}, + ops::RangeInclusive, str::FromStr, }; -use log::*; use serde::{Deserialize, Serialize}; use tari_common_types::types::FixedHash; -use tari_dan_common_types::{optional::Optional, Epoch, NodeHeight, SubstateAddress}; +use tari_dan_common_types::{optional::Optional, shard::Shard, Epoch, NodeHeight, SubstateAddress}; use tari_engine_types::substate::{hash_substate, Substate, SubstateId, SubstateValue}; use tari_transaction::{SubstateRequirement, TransactionId, VersionedSubstateId}; use crate::{ - consensus_models::{Block, BlockId, LockedSubstate, QcId, QuorumCertificate}, + consensus_models::{BlockId, LockedSubstate, QcId, QuorumCertificate}, StateStoreReadTransaction, StateStoreWriteTransaction, StorageError, }; -const LOG_TARGET: &str = "tari::dan::storage::consensus_models::substate"; - #[derive(Debug, Clone, Serialize, Deserialize)] #[cfg_attr( feature = "ts", @@ -46,6 +44,7 @@ pub struct SubstateRecord { #[cfg_attr(feature = "ts", ts(type = "string"))] pub created_block: BlockId, pub created_height: NodeHeight, + pub created_by_shard: Shard, pub created_at_epoch: Epoch, pub destroyed: Option, } @@ -62,8 +61,9 @@ pub struct SubstateDestroyed { #[cfg_attr(feature = "ts", ts(type = "string"))] pub justify: QcId, #[cfg_attr(feature = "ts", ts(type = "string"))] - pub by_block: BlockId, + pub by_block: NodeHeight, pub at_epoch: Epoch, + pub by_shard: Shard, } impl SubstateRecord { @@ -71,6 +71,7 @@ impl SubstateRecord { substate_id: SubstateId, version: u32, substate_value: SubstateValue, + created_by_shard: Shard, created_at_epoch: Epoch, created_height: NodeHeight, created_block: BlockId, @@ -84,6 +85,7 @@ impl SubstateRecord { substate_value, created_height, created_justify, + created_by_shard, created_at_epoch, created_by_transaction, created_block, @@ -257,6 +259,14 @@ impl SubstateRecord { Ok(found) } + pub fn get_n_after( + tx: &TTx, + n: usize, + after: &SubstateAddress, + ) -> Result, StorageError> { + tx.substates_get_n_after(n, after) + } + pub fn get_many_within_range>>( tx: &TTx, bounds: B, @@ -295,16 +305,18 @@ impl SubstateRecord { .transpose() } - pub fn destroy_many>( + pub fn destroy( tx: &mut TTx, - substate_addresses: I, + versioned_substate_id: VersionedSubstateId, + shard: Shard, epoch: Epoch, - destroyed_by_block: &BlockId, + destroyed_by_block: NodeHeight, destroyed_justify: &QcId, destroyed_by_transaction: &TransactionId, ) -> Result<(), StorageError> { - tx.substate_down_many( - substate_addresses, + tx.substates_down( + versioned_substate_id, + shard, epoch, destroyed_by_block, destroyed_by_transaction, @@ -368,69 +380,21 @@ impl SubstateUpdate { } } -impl SubstateUpdate { - pub fn apply(self, tx: &mut TTx, block: &Block) -> Result<(), StorageError> - where - TTx: StateStoreWriteTransaction + Deref, - TTx::Target: StateStoreReadTransaction, - { - match self { - Self::Create(proof) => { - debug!( - target: LOG_TARGET, - "🌲 Applying substate CREATE for {} v{}", - proof.substate.substate_id, proof.substate.version - ); - proof.created_qc.save(tx)?; - SubstateRecord { - substate_id: proof.substate.substate_id, - version: proof.substate.version, - state_hash: hash_substate(&proof.substate.substate_value, proof.substate.version), - substate_value: proof.substate.substate_value, - created_by_transaction: proof.substate.created_by_transaction, - created_justify: *proof.created_qc.id(), - created_block: *block.id(), - created_height: block.height(), - created_at_epoch: block.epoch(), - destroyed: None, - } - .create(tx)?; - }, - Self::Destroy(SubstateDestroyedProof { - substate_id, - version, - justify: proof, - destroyed_by_transaction, - }) => { - debug!( - target: LOG_TARGET, - "πŸ”₯ Applying substate DESTROY for substate {}v{} (transaction {})", - substate_id, - version, - destroyed_by_transaction - ); - proof.save(tx)?; - SubstateRecord::destroy_many( - tx, - iter::once(SubstateAddress::from_substate_id(&substate_id, version)), - block.epoch(), - block.id(), - proof.id(), - &destroyed_by_transaction, - )?; - }, - } - - Ok(()) - } -} - impl From for SubstateUpdate { fn from(value: SubstateCreatedProof) -> Self { Self::Create(value) } } +impl Display for SubstateUpdate { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + Self::Create(proof) => write!(f, "Create: {}(v{})", proof.substate.substate_id, proof.substate.version), + Self::Destroy(proof) => write!(f, "Destroy: {}(v{})", proof.substate_id, proof.version), + } + } +} + /// Substate lock flags #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, Serialize, Deserialize)] #[cfg_attr( diff --git a/dan_layer/storage/src/consensus_models/transaction.rs b/dan_layer/storage/src/consensus_models/transaction.rs index 9f2777874..d24d10f7d 100644 --- a/dan_layer/storage/src/consensus_models/transaction.rs +++ b/dan_layer/storage/src/consensus_models/transaction.rs @@ -3,7 +3,6 @@ use std::{collections::HashSet, ops::Deref, time::Duration}; -use indexmap::IndexSet; use serde::Deserialize; use tari_engine_types::commit_result::{ExecuteResult, FinalizeResult, RejectReason}; use tari_transaction::{Transaction, TransactionId, VersionedSubstateId}; @@ -22,7 +21,7 @@ pub struct TransactionRecord { pub result: Option, pub execution_time: Option, pub resulting_outputs: Vec, - pub resolved_inputs: Option>, + pub resolved_inputs: Option>, pub final_decision: Option, pub finalized_time: Option, pub abort_details: Option, @@ -45,7 +44,7 @@ impl TransactionRecord { pub fn load( transaction: Transaction, result: Option, - resolved_inputs: Option>, + resolved_inputs: Option>, execution_time: Option, final_decision: Option, finalized_time: Option, @@ -92,8 +91,8 @@ impl TransactionRecord { &self.resulting_outputs } - pub fn resolved_inputs(&self) -> Option<&IndexSet> { - self.resolved_inputs.as_ref() + pub fn resolved_inputs(&self) -> Option<&[VersionedSubstateIdLockIntent]> { + self.resolved_inputs.as_deref() } pub fn final_decision(&self) -> Option { diff --git a/dan_layer/storage/src/consensus_models/transaction_execution.rs b/dan_layer/storage/src/consensus_models/transaction_execution.rs index bdac539e6..33baefaf1 100644 --- a/dan_layer/storage/src/consensus_models/transaction_execution.rs +++ b/dan_layer/storage/src/consensus_models/transaction_execution.rs @@ -3,7 +3,6 @@ use std::time::Duration; -use indexmap::IndexSet; use tari_engine_types::commit_result::ExecuteResult; use tari_transaction::{TransactionId, VersionedSubstateId}; @@ -19,7 +18,7 @@ pub struct TransactionExecution { pub block_id: BlockId, pub transaction_id: TransactionId, pub result: ExecuteResult, - pub resolved_inputs: IndexSet, + pub resolved_inputs: Vec, pub resulting_outputs: Vec, pub execution_time: Duration, } @@ -29,7 +28,7 @@ impl TransactionExecution { block_id: BlockId, transaction_id: TransactionId, result: ExecuteResult, - resolved_inputs: IndexSet, + resolved_inputs: Vec, resulting_outputs: Vec, execution_time: Duration, ) -> Self { @@ -59,7 +58,7 @@ impl TransactionExecution { &self.result } - pub fn resolved_inputs(&self) -> &IndexSet { + pub fn resolved_inputs(&self) -> &[VersionedSubstateIdLockIntent] { &self.resolved_inputs } diff --git a/dan_layer/storage/src/global/backend_adapter.rs b/dan_layer/storage/src/global/backend_adapter.rs index dba9e59cf..7f2f39084 100644 --- a/dan_layer/storage/src/global/backend_adapter.rs +++ b/dan_layer/storage/src/global/backend_adapter.rs @@ -127,11 +127,11 @@ pub trait GlobalDbAdapter: AtomicDb + Send + Sync + Clone { bucket: Shard, ) -> Result; - fn validator_nodes_set_committee_bucket( + fn validator_nodes_set_committee_shard( &self, tx: &mut Self::DbTransaction<'_>, shard_key: SubstateAddress, - bucket: Shard, + shard: Shard, sidechain_id: Option<&PublicKey>, epoch: Epoch, ) -> Result<(), Self::Error>; diff --git a/dan_layer/storage/src/global/validator_node_db.rs b/dan_layer/storage/src/global/validator_node_db.rs index 802e5a7aa..ec2b07345 100644 --- a/dan_layer/storage/src/global/validator_node_db.rs +++ b/dan_layer/storage/src/global/validator_node_db.rs @@ -143,15 +143,15 @@ impl<'a, 'tx, TGlobalDbAdapter: GlobalDbAdapter> ValidatorNodeDb<'a, 'tx, TGloba .map_err(TGlobalDbAdapter::Error::into) } - pub fn set_committee_bucket( + pub fn set_committee_shard( &mut self, substate_address: SubstateAddress, - committee_bucket: Shard, + shard: Shard, sidechain_id: Option<&PublicKey>, epoch: Epoch, ) -> Result<(), TGlobalDbAdapter::Error> { self.backend - .validator_nodes_set_committee_bucket(self.tx, substate_address, committee_bucket, sidechain_id, epoch) + .validator_nodes_set_committee_shard(self.tx, substate_address, shard, sidechain_id, epoch) .map_err(TGlobalDbAdapter::Error::into) } } diff --git a/dan_layer/storage/src/state_store/mod.rs b/dan_layer/storage/src/state_store/mod.rs index f2ea4e097..91e26b740 100644 --- a/dan_layer/storage/src/state_store/mod.rs +++ b/dan_layer/storage/src/state_store/mod.rs @@ -10,9 +10,9 @@ use std::{ use indexmap::IndexMap; use serde::{Deserialize, Serialize}; use tari_common_types::types::{FixedHash, PublicKey}; -use tari_dan_common_types::{Epoch, NodeAddressable, NodeHeight, SubstateAddress}; +use tari_dan_common_types::{shard::Shard, Epoch, NodeAddressable, NodeHeight, SubstateAddress}; use tari_engine_types::substate::SubstateId; -use tari_state_tree::{TreeStore, TreeStoreReader, Version}; +use tari_state_tree::{Node, NodeKey, StaleTreeNode, Version}; use tari_transaction::{SubstateRequirement, TransactionId, VersionedSubstateId}; #[cfg(feature = "ts")] use ts_rs::TS; @@ -38,6 +38,8 @@ use crate::{ PendingStateTreeDiff, QcId, QuorumCertificate, + StateTransition, + StateTransitionId, SubstateRecord, TransactionAtom, TransactionExecution, @@ -54,11 +56,9 @@ const LOG_TARGET: &str = "tari::dan::storage"; pub trait StateStore { type Addr: NodeAddressable; - type ReadTransaction<'a>: StateStoreReadTransaction + TreeStoreReader + type ReadTransaction<'a>: StateStoreReadTransaction where Self: 'a; - type WriteTransaction<'a>: StateStoreWriteTransaction - + TreeStore - + Deref> + type WriteTransaction<'a>: StateStoreWriteTransaction + Deref> where Self: 'a; fn create_read_tx(&self) -> Result, StorageError>; @@ -134,10 +134,13 @@ pub trait StateStoreReadTransaction: Sized { from_block_id: &BlockId, ) -> Result; fn blocks_get(&self, block_id: &BlockId) -> Result; - fn blocks_get_tip(&self) -> Result; + fn blocks_get_tip(&self, epoch: Epoch, shard: Shard) -> Result; + fn blocks_get_last_n_in_epoch(&self, n: usize, epoch: Epoch, shard: Shard) -> Result, StorageError>; /// Returns all blocks from and excluding the start block (lower height) to the end block (inclusive) fn blocks_get_all_between( &self, + epoch: Epoch, + shard: Shard, start_block_id_exclusive: &BlockId, end_block_id_inclusive: &BlockId, include_dummy_blocks: bool, @@ -235,6 +238,8 @@ pub trait StateStoreReadTransaction: Sized { fn substates_exists_for_transaction(&self, transaction_id: &TransactionId) -> Result; + fn substates_get_n_after(&self, n: usize, after: &SubstateAddress) -> Result, StorageError>; + fn substates_get_many_within_range( &self, start: &SubstateAddress, @@ -250,7 +255,6 @@ pub trait StateStoreReadTransaction: Sized { &self, tx_id: &TransactionId, ) -> Result, StorageError>; - fn substates_get_all_for_block(&self, block_id: &BlockId) -> Result, StorageError>; fn substates_get_all_for_transaction( &self, transaction_id: &TransactionId, @@ -266,8 +270,21 @@ pub trait StateStoreReadTransaction: Sized { fn pending_state_tree_diffs_exists_for_block(&self, block_id: &BlockId) -> Result; fn pending_state_tree_diffs_get_all_up_to_commit_block( &self, + epoch: Epoch, + shard: Shard, block_id: &BlockId, ) -> Result, StorageError>; + + fn state_transitions_get_n_after( + &self, + n: usize, + id: StateTransitionId, + end_epoch: Epoch, + ) -> Result, StorageError>; + + fn state_transitions_get_last_id(&self) -> Result; + + fn state_tree_nodes_get(&self, epoch: Epoch, shard: Shard, key: &NodeKey) -> Result, StorageError>; } pub trait StateStoreWriteTransaction { @@ -397,15 +414,16 @@ pub trait StateStoreWriteTransaction { transaction_ids: I, ) -> Result<(), StorageError>; - fn substate_down_many>( + fn substates_create(&mut self, substate: SubstateRecord) -> Result<(), StorageError>; + fn substates_down( &mut self, - substate_addresses: I, + versioned_substate_id: VersionedSubstateId, + shard: Shard, epoch: Epoch, - destroyed_block_id: &BlockId, + destroyed_block_height: NodeHeight, destroyed_transaction_id: &TransactionId, destroyed_qc_id: &QcId, ) -> Result<(), StorageError>; - fn substates_create(&mut self, substate: SubstateRecord) -> Result<(), StorageError>; // -------------------------------- Pending State Tree Diffs -------------------------------- // fn pending_state_tree_diffs_insert(&mut self, diff: &PendingStateTreeDiff) -> Result<(), StorageError>; @@ -413,6 +431,22 @@ pub trait StateStoreWriteTransaction { &mut self, block_id: &BlockId, ) -> Result; + + //---------------------------------- State tree --------------------------------------------// + fn state_tree_nodes_insert( + &mut self, + epoch: Epoch, + shard: Shard, + key: NodeKey, + node: Node, + ) -> Result<(), StorageError>; + + fn state_tree_nodes_mark_stale_tree_node( + &mut self, + epoch: Epoch, + shard: Shard, + node: StaleTreeNode, + ) -> Result<(), StorageError>; } #[derive(Debug, Clone, Copy, Serialize, Deserialize)] diff --git a/dan_layer/storage_sqlite/src/global/backend_adapter.rs b/dan_layer/storage_sqlite/src/global/backend_adapter.rs index f2fbe559d..4e89c381b 100644 --- a/dan_layer/storage_sqlite/src/global/backend_adapter.rs +++ b/dan_layer/storage_sqlite/src/global/backend_adapter.rs @@ -438,17 +438,6 @@ impl GlobalDbAdapter for SqliteGlobalDbAdapter { use crate::global::schema::validator_nodes; let vn = validator_nodes::table - .select(( - validator_nodes::id, - validator_nodes::public_key, - validator_nodes::shard_key, - validator_nodes::registered_at_base_height, - validator_nodes::start_epoch, - validator_nodes::end_epoch, - validator_nodes::fee_claim_public_key, - validator_nodes::address, - validator_nodes::sidechain_id, - )) .filter(validator_nodes::start_epoch.le(epoch.as_u64() as i64)) .filter(validator_nodes::end_epoch.gt(epoch.as_u64() as i64)) .filter(validator_nodes::public_key.eq(ByteArray::as_bytes(public_key))) @@ -538,12 +527,12 @@ impl GlobalDbAdapter for SqliteGlobalDbAdapter { })?; let mut committees = HashMap::new(); - for (bucket, address, public_key) in count { + for (shard, address, public_key) in count { let addr = DbValidatorNode::try_parse_address(&address)?; let pk = PublicKey::from_canonical_bytes(&public_key) .map_err(|_| SqliteStorageError::MalformedDbData("Invalid public key".to_string()))?; committees - .entry(Shard::from(bucket as u32)) + .entry(Shard::from(shard as u32)) .or_insert_with(Committee::empty) .members .push((addr, pk)); @@ -552,11 +541,11 @@ impl GlobalDbAdapter for SqliteGlobalDbAdapter { Ok(committees) } - fn validator_nodes_set_committee_bucket( + fn validator_nodes_set_committee_shard( &self, tx: &mut Self::DbTransaction<'_>, shard_key: SubstateAddress, - bucket: Shard, + shard: Shard, sidechain_id: Option<&PublicKey>, epoch: Epoch, ) -> Result<(), Self::Error> { @@ -580,7 +569,7 @@ impl GlobalDbAdapter for SqliteGlobalDbAdapter { .values(( committees::validator_node_id.eq(validator_id), committees::epoch.eq(epoch.as_u64() as i64), - committees::committee_bucket.eq(i64::from(bucket.as_u32())), + committees::committee_bucket.eq(i64::from(shard.as_u32())), )) .execute(tx.connection()) .map_err(|source| SqliteStorageError::DieselError { diff --git a/dan_layer/storage_sqlite/tests/global_db.rs b/dan_layer/storage_sqlite/tests/global_db.rs index 74b28f43b..736dc2049 100644 --- a/dan_layer/storage_sqlite/tests/global_db.rs +++ b/dan_layer/storage_sqlite/tests/global_db.rs @@ -72,7 +72,7 @@ fn update_committee_bucket( epoch: Epoch, ) { validator_nodes - .set_committee_bucket(derived_substate_address(public_key), committee_bucket, None, epoch) + .set_committee_shard(derived_substate_address(public_key), committee_bucket, None, epoch) .unwrap(); } diff --git a/dan_layer/template_builtin/build.rs b/dan_layer/template_builtin/build.rs index d3036c1f9..6dfd477c2 100644 --- a/dan_layer/template_builtin/build.rs +++ b/dan_layer/template_builtin/build.rs @@ -1,24 +1,5 @@ -// Copyright 2022, The Tari Project -// -// Redistribution and use in source and binary forms, with or without modification, are permitted provided that the -// following conditions are met: -// -// 1. Redistributions of source code must retain the above copyright notice, this list of conditions and the following -// disclaimer. -// -// 2. Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the -// following disclaimer in the documentation and/or other materials provided with the distribution. -// -// 3. Neither the name of the copyright holder nor the names of its contributors may be used to endorse or promote -// products derived from this software without specific prior written permission. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, -// INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE -// DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR -// SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, -// WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE -// USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +// Copyright 2024 The Tari Project +// SPDX-License-Identifier: BSD-3-Clause use std::{ env, diff --git a/dan_layer/validator_node_rpc/src/client.rs b/dan_layer/validator_node_rpc/src/client.rs index fba01ea41..036e48ee1 100644 --- a/dan_layer/validator_node_rpc/src/client.rs +++ b/dan_layer/validator_node_rpc/src/client.rs @@ -84,6 +84,10 @@ pub struct TariValidatorNodeRpcClient { } impl TariValidatorNodeRpcClient { + pub fn address(&self) -> &PeerAddress { + &self.address + } + pub async fn client_connection( &mut self, ) -> Result { diff --git a/dan_layer/validator_node_rpc/src/rpc_service.rs b/dan_layer/validator_node_rpc/src/rpc_service.rs index b599ebc79..7c93b4e97 100644 --- a/dan_layer/validator_node_rpc/src/rpc_service.rs +++ b/dan_layer/validator_node_rpc/src/rpc_service.rs @@ -42,4 +42,16 @@ pub trait ValidatorNodeRpcService: Send + Sync + 'static { &self, request: Request, ) -> Result, RpcStatus>; + + #[rpc(method = 7)] + async fn get_checkpoint( + &self, + request: Request, + ) -> Result, RpcStatus>; + + #[rpc(method = 8)] + async fn sync_state( + &self, + request: Request, + ) -> Result, RpcStatus>; } diff --git a/integration_tests/tests/features/epoch_change.feature b/integration_tests/tests/features/epoch_change.feature index 2b50f3725..5080b04ef 100644 --- a/integration_tests/tests/features/epoch_change.feature +++ b/integration_tests/tests/features/epoch_change.feature @@ -5,7 +5,7 @@ Feature: Epoch change @serial @current - Scenario: EpochEnd and EpochStart commands are used on epoch change + Scenario: EndEpoch command is used on epoch change Given fees are disabled # Initialize a base node, wallet, miner and VN Given a base node BASE @@ -20,20 +20,29 @@ Feature: Epoch change When miner MINER mines 6 new blocks When wallet WALLET has at least 20000 T + # Create a key for transactions + When I use an account key named K1 + # VN registration When validator node VAL sends a registration transaction to base wallet WALLET + When base wallet WALLET registers the template "faucet" # Mine them into registered epoch When miner MINER mines 16 new blocks Then VAL has scanned to height 19 Then the validator node VAL is listed as registered + Then the template "faucet" is listed as registered by the validator node VAL + + # Push a transaction through to get blocks + When I call function "mint" on template "faucet" on VAL with args "amount_10000" named "FAUCET" - When Block count on VN VAL is at least 5 - When miner MINER mines 1 new blocks + When Block count on VN VAL is at least 6 + When miner MINER mines 5 new blocks + Then VAL has scanned to height 24 Then the validator node VAL switches to epoch 2 # @serial -# Scenario: Commmittee is split into two during epoch change +# Scenario: Committee is split into two during epoch change # Given fees are disabled # # Initialize a base node, wallet, miner and VN # Given a base node BASE diff --git a/integration_tests/tests/features/block_sync.feature b/integration_tests/tests/features/state_sync.feature similarity index 83% rename from integration_tests/tests/features/block_sync.feature rename to integration_tests/tests/features/state_sync.feature index 66444d8bb..63a00187e 100644 --- a/integration_tests/tests/features/block_sync.feature +++ b/integration_tests/tests/features/state_sync.feature @@ -1,7 +1,7 @@ # Copyright 2022 The Tari Project # SPDX-License-Identifier: BSD-3-Clause -@block_sync -Feature: Block Sync +@state_sync +Feature: State Sync @serial @fixed Scenario: New validator node registers and syncs @@ -32,7 +32,7 @@ Feature: Block Sync When I create an account UNUSED2 via the wallet daemon WALLET_D When I create an account UNUSED3 via the wallet daemon WALLET_D - When I wait for validator VN has leaf block height of at least 15 + # When I wait for validator VN has leaf block height of at least 15 # Start a new VN that needs to sync Given a validator node VN2 connected to base node BASE and wallet daemon WALLET_D @@ -45,12 +45,11 @@ Feature: Block Sync Then VN2 has scanned to height 37 Then the validator node VN2 is listed as registered - When I wait for validator VN2 has leaf block height of at least 15 -# FIXME: This part fails because epoch change is not yet fully implemented. -# -# When I create an account UNUSED4 via the wallet daemon WALLET_D -# When I create an account UNUSED5 via the wallet daemon WALLET_D -# -# When I wait for validator VN has leaf block height of at least 18 -# When I wait for validator VN2 has leaf block height of at least 18 + When I wait for validator VN2 has leaf block height of at least 1 at epoch 3 + + When I create an account UNUSED4 via the wallet daemon WALLET_D + When I create an account UNUSED5 via the wallet daemon WALLET_D + + When I wait for validator VN has leaf block height of at least 5 at epoch 3 + When I wait for validator VN2 has leaf block height of at least 5 at epoch 3 diff --git a/integration_tests/tests/steps/validator_node.rs b/integration_tests/tests/steps/validator_node.rs index e896dced0..02bcd69eb 100644 --- a/integration_tests/tests/steps/validator_node.rs +++ b/integration_tests/tests/steps/validator_node.rs @@ -18,9 +18,9 @@ use libp2p::Multiaddr; use minotari_app_grpc::tari_rpc::{RegisterValidatorNodeRequest, Signature}; use tari_base_node_client::{grpc::GrpcBaseNodeClient, BaseNodeClient}; use tari_crypto::tari_utilities::ByteArray; -use tari_dan_common_types::{optional::Optional, Epoch, SubstateAddress}; +use tari_dan_common_types::{Epoch, SubstateAddress}; use tari_engine_types::substate::SubstateId; -use tari_validator_node_client::types::{AddPeerRequest, GetStateRequest, GetTemplateRequest, ListBlocksRequest}; +use tari_validator_node_client::types::{AddPeerRequest, GetBlocksRequest, GetStateRequest, GetTemplateRequest}; #[given(expr = "a validator node {word} connected to base node {word} and wallet daemon {word}")] async fn start_validator_node(world: &mut TariWorld, vn_name: String, bn_name: String, wallet_daemon_name: String) { @@ -405,21 +405,31 @@ async fn when_i_create_new_key_pair(world: &mut TariWorld, key_name: String) { create_key(world, key_name); } -#[when(expr = "I wait for validator {word} has leaf block height of at least {int}")] -async fn when_i_wait_for_validator_leaf_block_at_least(world: &mut TariWorld, name: String, height: u64) { +#[when(expr = "I wait for validator {word} has leaf block height of at least {int} at epoch {int}")] +async fn when_i_wait_for_validator_leaf_block_at_least(world: &mut TariWorld, name: String, height: u64, epoch: u64) { let vn = world.get_validator_node(&name); let mut client = vn.create_client(); - for _ in 0..20 { + for _ in 0..40 { let resp = client - .list_blocks(ListBlocksRequest { - from_id: None, + .list_blocks_paginated(GetBlocksRequest { limit: 1, + offset: 0, + ordering_index: None, + ordering: None, + filter_index: None, + filter: None, }) .await - .optional() .unwrap(); - if let Some(resp) = resp { - if resp.blocks.last().unwrap().height().as_u64() >= height { + + // for b in resp.blocks.iter() { + // eprintln!("----------> {b}"); + // } + // eprintln!("-----------"); + + if let Some(block) = resp.blocks.first() { + assert!(block.epoch().as_u64() <= epoch); + if block.epoch().as_u64() == epoch && block.height().as_u64() >= height { return; } } @@ -427,13 +437,17 @@ async fn when_i_wait_for_validator_leaf_block_at_least(world: &mut TariWorld, na } let resp = client - .list_blocks(ListBlocksRequest { - from_id: None, + .list_blocks_paginated(GetBlocksRequest { limit: 1, + offset: 0, + ordering_index: None, + ordering: None, + filter_index: None, + filter: None, }) .await .unwrap(); - let actual_height = resp.blocks.last().unwrap().height().as_u64(); + let actual_height = resp.blocks.first().unwrap().height().as_u64(); if actual_height < height { panic!( "Validator {} leaf block height {} is less than {}", @@ -461,33 +475,26 @@ async fn then_validator_node_switches_epoch(world: &mut TariWorld, vn_name: Stri let mut client = vn.create_client(); for _ in 0..200 { let list_block = client - .list_blocks(ListBlocksRequest { - from_id: None, - limit: 4, + .list_blocks_paginated(GetBlocksRequest { + limit: 10, + offset: 0, + ordering_index: None, + ordering: None, + filter_index: None, + filter: None, }) - .await; - let blocks = list_block.unwrap().blocks; - let newest = blocks.first().expect("Couldn't get blocks"); - if newest.epoch().as_u64() == epoch { - // The newest block as expected should be the EpochStart event. - assert!(newest - .commands() - .contains(&tari_dan_storage::consensus_models::Command::EpochEvent( - tari_dan_storage::consensus_models::EpochEvent::Start - ))); - // The 3 blocks before it should be all End events. 3 because we need to bury the epoch start to locked - // block. - for block in &blocks[1..] { - assert!(block - .commands() - .contains(&tari_dan_storage::consensus_models::Command::EpochEvent( - tari_dan_storage::consensus_models::EpochEvent::End - ))); - // All the epoch ends should be in previous epoch. - assert_eq!(block.epoch().as_u64() + 1, epoch); - } + .await + .unwrap(); + let blocks = list_block.blocks; + assert!( + blocks.iter().all(|b| b.epoch().as_u64() <= epoch), + "Epoch is greater than expected" + ); + if blocks.iter().any(|b| b.epoch().as_u64() == epoch) { + assert!(blocks.iter().any(|b| b.is_epoch_end()), "No end epoch block found"); return; } + tokio::time::sleep(Duration::from_secs(8)).await; } panic!("Validator node {vn_name} did not switch to epoch {epoch}"); diff --git a/networking/core/src/worker.rs b/networking/core/src/worker.rs index 2183534b2..2d3ad1408 100644 --- a/networking/core/src/worker.rs +++ b/networking/core/src/worker.rs @@ -565,7 +565,7 @@ where self.on_autonat_event(event)?; }, PeerSync(peersync::Event::LocalPeerRecordUpdated { record }) => { - info!(target: LOG_TARGET, "πŸ“ Local peer record updated: {:?} announce enabled = {}, has_sent_announce = {}",record, self.config.announce, self.has_sent_announce); + info!(target: LOG_TARGET, "πŸ§‘β€πŸ§‘β€πŸ§’β€πŸ§’ Local peer record updated: {:?} announce enabled = {}, has_sent_announce = {}",record, self.config.announce, self.has_sent_announce); if self.config.announce && !self.has_sent_announce && record.is_signed() { info!(target: LOG_TARGET, "πŸ“£ Sending local peer announce with {} address(es)", record.addresses().len()); self.swarm @@ -576,7 +576,7 @@ where } }, PeerSync(peersync::Event::PeerBatchReceived { new_peers, from_peer }) => { - info!(target: LOG_TARGET, "πŸ“ Peer batch received: from_peer={}, new_peers={}", from_peer, new_peers); + info!(target: LOG_TARGET, "πŸ§‘β€πŸ§‘β€πŸ§’β€πŸ§’ Peer batch received: from_peer={}, new_peers={}", from_peer, new_peers); }, PeerSync(event) => { info!(target: LOG_TARGET, "ℹ️ PeerSync event: {:?}", event); diff --git a/networking/libp2p-messaging/src/handler.rs b/networking/libp2p-messaging/src/handler.rs index 5e5f38a30..390401739 100644 --- a/networking/libp2p-messaging/src/handler.rs +++ b/networking/libp2p-messaging/src/handler.rs @@ -57,7 +57,7 @@ pub struct Handler { impl Handler { pub fn new(peer_id: PeerId, protocol: StreamProtocol, config: &Config) -> Self { - let (pending_events_sender, pending_events_receiver) = mpsc::channel(10); + let (pending_events_sender, pending_events_receiver) = mpsc::channel(20); Self { peer_id, protocol, diff --git a/networking/rpc_framework/src/client/mod.rs b/networking/rpc_framework/src/client/mod.rs index 8ce870ac5..9d241e12c 100644 --- a/networking/rpc_framework/src/client/mod.rs +++ b/networking/rpc_framework/src/client/mod.rs @@ -585,9 +585,9 @@ where TSubstream: AsyncRead + AsyncWrite + Unpin + Send let resp_flags = RpcMessageFlags::from_bits(u8::try_from(resp.flags).map_err(|_| { - RpcStatus::protocol_error(&format!("invalid message flag: must be less than {}", u8::MAX)) + RpcStatus::protocol_error(format!("invalid message flag: must be less than {}", u8::MAX)) })?) - .ok_or(RpcStatus::protocol_error(&format!( + .ok_or(RpcStatus::protocol_error(format!( "invalid message flag, does not match any flags ({})", resp.flags )))?; @@ -598,7 +598,7 @@ where TSubstream: AsyncRead + AsyncWrite + Unpin + Send self.peer_id, resp ); - let _result = reply.send(Err(RpcStatus::protocol_error(&format!( + let _result = reply.send(Err(RpcStatus::protocol_error(format!( "Received invalid ping response on protocol '{}'", self.protocol_name() )))); @@ -953,13 +953,13 @@ where TSubstream: AsyncRead + AsyncWrite + Unpin fn check_response(&self, resp: &proto::RpcResponse) -> Result<(), RpcError> { let resp_id = u16::try_from(resp.request_id) - .map_err(|_| RpcStatus::protocol_error(&format!("invalid request_id: must be less than {}", u16::MAX)))?; + .map_err(|_| RpcStatus::protocol_error(format!("invalid request_id: must be less than {}", u16::MAX)))?; if resp_id != self.request_id { return Err(RpcError::ResponseIdDidNotMatchRequest { expected: self.request_id, actual: u16::try_from(resp.request_id).map_err(|_| { - RpcStatus::protocol_error(&format!("invalid request_id: must be less than {}", u16::MAX)) + RpcStatus::protocol_error(format!("invalid request_id: must be less than {}", u16::MAX)) })?, }); } diff --git a/networking/rpc_framework/src/server/mod.rs b/networking/rpc_framework/src/server/mod.rs index 8ea7e5d5c..58aee9d5a 100644 --- a/networking/rpc_framework/src/server/mod.rs +++ b/networking/rpc_framework/src/server/mod.rs @@ -583,7 +583,7 @@ where TSvc: Service, Response = Response, Error = RpcStatus "({}) Client has an invalid deadline. {}", self.logging_context_string, decoded_msg ); // Let the client know that they have disobeyed the spec - let status = RpcStatus::bad_request(&format!( + let status = RpcStatus::bad_request(format!( "Invalid deadline ({:.0?}). The deadline MUST be greater than {:.0?}.", self.peer_id, deadline, )); diff --git a/networking/rpc_framework/src/status.rs b/networking/rpc_framework/src/status.rs index 664129715..6a02a98b9 100644 --- a/networking/rpc_framework/src/status.rs +++ b/networking/rpc_framework/src/status.rs @@ -25,65 +25,65 @@ impl RpcStatus { } } - pub fn unsupported_method(details: &T) -> Self { + pub fn unsupported_method>(details: T) -> Self { Self { code: RpcStatusCode::UnsupportedMethod, - details: details.to_string(), + details: details.into(), } } - pub fn not_implemented(details: &T) -> Self { + pub fn not_implemented>(details: T) -> Self { Self { code: RpcStatusCode::NotImplemented, - details: details.to_string(), + details: details.into(), } } - pub fn bad_request(details: &T) -> Self { + pub fn bad_request>(details: T) -> Self { Self { code: RpcStatusCode::BadRequest, - details: details.to_string(), + details: details.into(), } } /// Returns a general error. As with all other errors care should be taken not to leak sensitive data to remote /// peers through error messages. - pub fn general(details: &T) -> Self { + pub fn general>(details: T) -> Self { Self { code: RpcStatusCode::General, - details: details.to_string(), + details: details.into(), } } pub fn general_default() -> Self { - Self::general(&"General error") + Self::general("General error") } - pub fn timed_out(details: &T) -> Self { + pub fn timed_out>(details: T) -> Self { Self { code: RpcStatusCode::Timeout, - details: details.to_string(), + details: details.into(), } } - pub fn not_found(details: &T) -> Self { + pub fn not_found>(details: T) -> Self { Self { code: RpcStatusCode::NotFound, - details: details.to_string(), + details: details.into(), } } - pub fn forbidden(details: &T) -> Self { + pub fn forbidden>(details: T) -> Self { Self { code: RpcStatusCode::Forbidden, - details: details.to_string(), + details: details.into(), } } - pub fn conflict(details: &T) -> Self { + pub fn conflict>(details: T) -> Self { Self { code: RpcStatusCode::Conflict, - details: details.to_string(), + details: details.into(), } } @@ -96,10 +96,10 @@ impl RpcStatus { } } - pub(super) fn protocol_error(details: &T) -> Self { + pub(super) fn protocol_error>(details: T) -> Self { Self { code: RpcStatusCode::ProtocolError, - details: details.to_string(), + details: details.into(), } } @@ -141,7 +141,7 @@ impl From for RpcStatus { RpcError::RequestFailed(status) => status, err => { error!(target: LOG_TARGET, "Internal error: {}", err); - Self::general(&err.to_string()) + Self::general(err.to_string()) }, } } @@ -169,8 +169,8 @@ impl From for RpcStatus { pub trait RpcStatusResultExt { fn rpc_status_internal_error(self, target: &str) -> Result; - fn rpc_status_not_found(self, message: S) -> Result; - fn rpc_status_bad_request(self, message: S) -> Result; + fn rpc_status_not_found>(self, message: S) -> Result; + fn rpc_status_bad_request>(self, message: S) -> Result; } impl RpcStatusResultExt for Result { @@ -178,12 +178,12 @@ impl RpcStatusResultExt for Result { self.map_err(RpcStatus::log_internal_error(target)) } - fn rpc_status_not_found(self, message: S) -> Result { - self.map_err(|_| RpcStatus::not_found(&message)) + fn rpc_status_not_found>(self, message: S) -> Result { + self.map_err(|_| RpcStatus::not_found(message)) } - fn rpc_status_bad_request(self, message: S) -> Result { - self.map_err(|_| RpcStatus::bad_request(&message)) + fn rpc_status_bad_request>(self, message: S) -> Result { + self.map_err(|_| RpcStatus::bad_request(message)) } }