From 3d7c12adcdf9550fdbea4d138a01e396bc868eba Mon Sep 17 00:00:00 2001 From: Luke Parker Date: Sun, 30 Oct 2022 10:08:35 -0400 Subject: [PATCH] Create a dedicated file for being a Tendermint authority --- Cargo.lock | 1 + substrate/node/Cargo.toml | 1 + substrate/node/src/service.rs | 54 +-- substrate/tendermint/client/src/authority.rs | 348 ++++++++++++++++++ .../tendermint/client/src/import_queue.rs | 19 +- substrate/tendermint/client/src/lib.rs | 55 +-- substrate/tendermint/client/src/tendermint.rs | 320 +--------------- substrate/tendermint/client/src/types.rs | 17 +- substrate/tendermint/client/src/validators.rs | 2 +- 9 files changed, 422 insertions(+), 395 deletions(-) create mode 100644 substrate/tendermint/client/src/authority.rs diff --git a/Cargo.lock b/Cargo.lock index 2b6ba170..917e0fd8 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7595,6 +7595,7 @@ dependencies = [ "pallet-tendermint", "pallet-transaction-payment", "pallet-transaction-payment-rpc", + "sc-basic-authorship", "sc-cli", "sc-client-api", "sc-consensus", diff --git a/substrate/node/Cargo.toml b/substrate/node/Cargo.toml index 4dfb3700..90c392f3 100644 --- a/substrate/node/Cargo.toml +++ b/substrate/node/Cargo.toml @@ -27,6 +27,7 @@ sp-block-builder = { git = "https://github.com/serai-dex/substrate" } sc-keystore = { git = "https://github.com/serai-dex/substrate" } sc-transaction-pool = { git = "https://github.com/serai-dex/substrate" } +sc-basic-authorship = { git = "https://github.com/serai-dex/substrate" } sc-transaction-pool-api = { git = "https://github.com/serai-dex/substrate" } sc-executor = { git = "https://github.com/serai-dex/substrate" } sc-service = { git = "https://github.com/serai-dex/substrate" } diff --git a/substrate/node/src/service.rs b/substrate/node/src/service.rs index 20579990..ac9921b0 100644 --- a/substrate/node/src/service.rs +++ b/substrate/node/src/service.rs @@ -1,15 +1,12 @@ -use std::sync::{Arc, RwLock}; - -use sp_core::H256; +use std::sync::Arc; use sc_executor::NativeElseWasmExecutor; use sc_service::{error::Error as ServiceError, Configuration, TaskManager}; -use sc_network::{NetworkService, NetworkBlock}; use sc_telemetry::{Telemetry, TelemetryWorker}; use serai_runtime::{self, opaque::Block, RuntimeApi}; pub(crate) use serai_consensus::{ - TendermintAuthority, ExecutorDispatch, Announce, FullClient, TendermintValidatorFirm, + TendermintImport, TendermintAuthority, ExecutorDispatch, FullClient, TendermintValidatorFirm, }; type FullBackend = sc_service::TFullBackend; @@ -24,30 +21,9 @@ type PartialComponents = sc_service::PartialComponents< Option, >; -#[derive(Clone)] -pub struct NetworkAnnounce(Arc>>>>); -impl NetworkAnnounce { - fn new() -> NetworkAnnounce { - NetworkAnnounce(Arc::new(RwLock::new(None))) - } -} -impl Announce for NetworkAnnounce { - fn announce(&self, hash: H256) { - if let Some(network) = self.0.read().unwrap().as_ref() { - network.announce_block(hash, None); - } - } -} - pub fn new_partial( config: &Configuration, -) -> Result< - ( - (NetworkAnnounce, TendermintAuthority>), - PartialComponents, - ), - ServiceError, -> { +) -> Result<(TendermintImport, PartialComponents), ServiceError> { if config.keystore_remote.is_some() { return Err(ServiceError::Other("Remote Keystores are not supported".to_string())); } @@ -91,19 +67,16 @@ pub fn new_partial( client.clone(), ); - let announce = NetworkAnnounce::new(); let (authority, import_queue) = serai_consensus::import_queue( - &task_manager, + &task_manager.spawn_essential_handle(), client.clone(), - announce.clone(), - transaction_pool.clone(), config.prometheus_registry(), ); let select_chain = serai_consensus::TendermintSelectChain::new(backend.clone()); Ok(( - (announce, authority), + authority, sc_service::PartialComponents { client, backend, @@ -119,7 +92,7 @@ pub fn new_partial( pub async fn new_full(config: Configuration) -> Result { let ( - (announce, authority), + authority, sc_service::PartialComponents { client, backend, @@ -142,7 +115,6 @@ pub async fn new_full(config: Configuration) -> Result Result Result { + // Block whose gossip is being tracked + number: Arc>, + // Outgoing message queue, placed here as the GossipEngine itself can't be + gossip_queue: Arc>>>, + + // Block producer + env: T::Environment, + announce: T::Network, +} + +pub struct TendermintAuthority { + import: TendermintImport, + active: Option>, +} + +impl TendermintAuthority { + pub fn new(import: TendermintImport) -> Self { + Self { import, active: None } + } + + fn get_last(&self) -> (::Hash, (BlockNumber, u64)) { + let info = self.import.client.info(); + + ( + info.best_hash, + ( + // Header::Number: TryInto doesn't implement Debug and can't be unwrapped + match info.best_number.try_into() { + Ok(best) => BlockNumber(best), + Err(_) => panic!("BlockNumber exceeded u64"), + }, + // Get the last time by grabbing the last block's justification and reading the time from + // that + Commit::>::decode( + &mut self + .import + .client + .justifications(&BlockId::Hash(info.best_hash)) + .unwrap() + .map(|justifications| justifications.get(CONSENSUS_ID).cloned().unwrap()) + .unwrap_or_default() + .as_ref(), + ) + .map(|commit| commit.end_time) + // TODO: Genesis start time + BLOCK_TIME + .unwrap_or_else(|_| SystemTime::now().duration_since(UNIX_EPOCH).unwrap().as_secs()), + ), + ) + } + + pub(crate) async fn get_proposal(&mut self, header: &::Header) -> T::Block { + let inherent_data = match self + .import + .providers + .read() + .await + .as_ref() + .unwrap() + .create_inherent_data_providers(header.hash(), ()) + .await + { + Ok(providers) => match providers.create_inherent_data() { + Ok(data) => Some(data), + Err(err) => { + warn!(target: "tendermint", "Failed to create inherent data: {}", err); + None + } + }, + Err(err) => { + warn!(target: "tendermint", "Failed to create inherent data providers: {}", err); + None + } + } + .unwrap_or_else(InherentData::new); + + let proposer = self + .active + .as_mut() + .unwrap() + .env + .init(header) + .await + .expect("Failed to create a proposer for the new block"); + // TODO: Production time, size limit + proposer + .propose(inherent_data, Digest::default(), Duration::from_secs(1), None) + .await + .expect("Failed to crate a new block proposal") + .block + } + + /// Act as a network authority, proposing and voting on blocks. This should be spawned on a task + /// as it will not return until the P2P stack shuts down. + pub async fn authority( + mut self, + providers: T::CIDP, + env: T::Environment, + network: T::Network, + registry: Option<&Registry>, + ) { + let (best_hash, last) = self.get_last(); + let mut last_number = last.0 .0 + 1; + + // Shared references between us and the Tendermint machine (and its actions via its Network + // trait) + let number = Arc::new(RwLock::new(last_number)); + let gossip_queue = Arc::new(RwLock::new(vec![])); + + // Create the gossip network + let mut gossip = GossipEngine::new( + network.clone(), + "tendermint", + Arc::new(TendermintGossip::new(number.clone(), self.import.validators.clone())), + registry, + ); + + // Create the Tendermint machine + let handle = { + // Set this struct as active + *self.import.providers.write().await = Some(providers); + self.active = Some(ActiveAuthority { + number: number.clone(), + gossip_queue: gossip_queue.clone(), + + env, + announce: network, + }); + + let proposal = self + .get_proposal(&self.import.client.header(BlockId::Hash(best_hash)).unwrap().unwrap()) + .await; + + TendermintMachine::new( + self, // We no longer need self, so let TendermintMachine become its owner + 0, // TODO: ValidatorId + last, proposal, + ) + }; + + // Start receiving messages about the Tendermint process for this block + let mut recv = gossip + .messages_for(TendermintGossip::>::topic::(last_number)); + + 'outer: loop { + // Send out any queued messages + let mut queue = gossip_queue.write().unwrap().drain(..).collect::>(); + for msg in queue.drain(..) { + gossip.gossip_message( + TendermintGossip::>::topic::(msg.number().0), + msg.encode(), + false, + ); + } + + // Handle any received messages + // This inner loop enables handling all pending messages before acquiring the out-queue lock + // again + 'inner: loop { + match recv.try_next() { + Ok(Some(msg)) => handle + .messages + .send(match SignedMessage::decode(&mut msg.message.as_ref()) { + Ok(msg) => msg, + Err(e) => { + warn!(target: "tendermint", "Couldn't decode valid message: {}", e); + continue; + } + }) + .await + .unwrap(), + + // Ok(None) IS NOT when there aren't messages available. It's when the channel is closed + // If we're no longer receiving messages from the network, it must no longer be running + // We should no longer be accordingly + Ok(None) => break 'outer, + + // No messages available + Err(_) => { + // Check if we the block updated and should be listening on a different topic + let curr = *number.read().unwrap(); + if last_number != curr { + last_number = curr; + // TODO: Will this return existing messages on the new height? Or will those have + // been ignored and are now gone? + recv = gossip.messages_for(TendermintGossip::>::topic::< + T::Block, + >(last_number)); + } + + // If there are no messages available, yield to not hog the thread, then return to the + // outer loop + yield_now().await; + break 'inner; + } + } + } + } + } +} + +#[async_trait] +impl Network for TendermintAuthority { + type ValidatorId = u16; + type SignatureScheme = TendermintValidators; + type Weights = TendermintValidators; + type Block = T::Block; + + const BLOCK_TIME: u32 = T::BLOCK_TIME_IN_SECONDS; + + fn signature_scheme(&self) -> Arc> { + self.import.validators.clone() + } + + fn weights(&self) -> Arc> { + self.import.validators.clone() + } + + async fn broadcast(&mut self, msg: SignedMessage) { + self.active.as_mut().unwrap().gossip_queue.write().unwrap().push(msg); + } + + async fn slash(&mut self, validator: u16) { + todo!() + } + + // The Tendermint machine will call add_block for any block which is committed to, regardless of + // validity. To determine validity, it expects a validate function, which Substrate doesn't + // directly offer, and an add function. In order to comply with Serai's modified view of inherent + // transactions, validate MUST check inherents, yet add_block must not. + // + // In order to acquire a validate function, any block proposed by a legitimate proposer is + // imported. This performs full validation and makes the block available as a tip. While this + // would be incredibly unsafe thanks to the unchecked inherents, it's defined as a tip with less + // work, despite being a child of some parent. This means it won't be moved to nor operated on by + // the node. + // + // When Tendermint completes, the block is finalized, setting it as the tip regardless of work. + async fn validate(&mut self, block: &T::Block) -> Result<(), BlockError> { + let hash = block.hash(); + let (header, body) = block.clone().deconstruct(); + let parent = *header.parent_hash(); + let number = *header.number(); + + let mut queue_write = self.import.queue.write().await; + *self.import.importing_block.write().unwrap() = Some(hash); + + queue_write.as_mut().unwrap().import_blocks( + // We do not want this block, which hasn't been confirmed, to be broadcast over the net + // Substrate will generate notifications unless it's Genesis, which this isn't, InitialSync, + // which changes telemtry behavior, or File, which is... close enough + BlockOrigin::File, + vec![IncomingBlock { + hash, + header: Some(header), + body: Some(body), + indexed_body: None, + justifications: None, + origin: None, + allow_missing_state: false, + skip_execution: false, + // TODO: Only set to true if block was rejected due to its inherents + import_existing: true, + state: None, + }], + ); + + if !ImportFuture::new(hash, queue_write.as_mut().unwrap()).await { + todo!() + } + + // Sanity checks that a child block can have less work than its parent + { + let info = self.import.client.info(); + assert_eq!(info.best_hash, parent); + assert_eq!(info.finalized_hash, parent); + assert_eq!(info.best_number, number - 1u8.into()); + assert_eq!(info.finalized_number, number - 1u8.into()); + } + + Ok(()) + } + + async fn add_block( + &mut self, + block: T::Block, + commit: Commit>, + ) -> T::Block { + let hash = block.hash(); + let justification = (CONSENSUS_ID, commit.encode()); + debug_assert!(self.import.verify_justification(hash, &justification).is_ok()); + + self + .import + .client + .finalize_block(BlockId::Hash(hash), Some(justification), true) + .map_err(|_| Error::InvalidJustification) + .unwrap(); + *self.active.as_mut().unwrap().number.write().unwrap() += 1; + self.active.as_ref().unwrap().announce.announce_block(hash, None); + + self.get_proposal(block.header()).await + } +} diff --git a/substrate/tendermint/client/src/import_queue.rs b/substrate/tendermint/client/src/import_queue.rs index 8e3fac01..5c6c58ba 100644 --- a/substrate/tendermint/client/src/import_queue.rs +++ b/substrate/tendermint/client/src/import_queue.rs @@ -14,10 +14,7 @@ use sc_service::ImportQueue; use substrate_prometheus_endpoint::Registry; -use crate::{ - types::TendermintValidator, - tendermint::{TendermintImport, TendermintAuthority}, -}; +use crate::{types::TendermintValidator, TendermintImport}; pub type TendermintImportQueue = BasicQueue; @@ -68,25 +65,21 @@ impl<'a, B: Block, T: Send> Future for ImportFuture<'a, B, T> { } pub fn import_queue( - client: Arc, - announce: T::Announce, - providers: Arc, - env: T::Environment, spawner: &impl sp_core::traits::SpawnEssentialNamed, + client: Arc, registry: Option<&Registry>, -) -> (TendermintAuthority, TendermintImportQueue) +) -> (TendermintImport, TendermintImportQueue) where Arc: BlockImport, as BlockImport>::Error: Into, { - let import = TendermintImport::::new(client, announce, providers, env); - let authority = TendermintAuthority(import.clone()); + let import = TendermintImport::::new(client); let boxed = Box::new(import.clone()); // Use None for the justification importer since justifications always come with blocks - // Therefore, they're never imported after the fact, mandating a importer + // Therefore, they're never imported after the fact, which is what mandates an importer let queue = || BasicQueue::new(import.clone(), boxed.clone(), None, spawner, registry); *futures::executor::block_on(import.queue.write()) = Some(queue()); - (authority, queue()) + (import.clone(), queue()) } diff --git a/substrate/tendermint/client/src/lib.rs b/substrate/tendermint/client/src/lib.rs index 36e3e349..a9196735 100644 --- a/substrate/tendermint/client/src/lib.rs +++ b/substrate/tendermint/client/src/lib.rs @@ -1,16 +1,14 @@ -use std::{marker::PhantomData, boxed::Box, sync::Arc, error::Error}; +use std::{boxed::Box, sync::Arc, error::Error}; use sp_runtime::traits::Block as BlockTrait; use sp_inherents::CreateInherentDataProviders; use sp_consensus::DisableProofRecording; -use sp_api::{TransactionFor, ProvideRuntimeApi}; +use sp_api::ProvideRuntimeApi; use sc_executor::{NativeVersion, NativeExecutionDispatch, NativeElseWasmExecutor}; use sc_transaction_pool::FullPool; use sc_network::NetworkService; -use sc_service::{TaskManager, TFullClient}; - -use substrate_prometheus_endpoint::Registry; +use sc_service::TFullClient; use serai_runtime::{self, opaque::Block, RuntimeApi}; @@ -19,14 +17,16 @@ use types::{TendermintClientMinimal, TendermintValidator}; mod validators; -mod tendermint; -pub use tendermint::TendermintAuthority; +pub(crate) mod tendermint; +pub use tendermint::TendermintImport; mod block_import; mod import_queue; -use import_queue::TendermintImportQueue; +pub use import_queue::{TendermintImportQueue, import_queue}; pub(crate) mod gossip; +pub(crate) mod authority; +pub use authority::TendermintAuthority; mod select_chain; pub use select_chain::TendermintSelectChain; @@ -51,10 +51,6 @@ impl NativeExecutionDispatch for ExecutorDispatch { pub type FullClient = TFullClient>; -pub trait Announce: Send + Sync + Clone + 'static { - fn announce(&self, hash: B::Hash); -} - pub struct Cidp; #[async_trait::async_trait] impl CreateInherentDataProviders for Cidp { @@ -68,15 +64,17 @@ impl CreateInherentDataProviders for Cidp { } } -pub struct TendermintValidatorFirm>(PhantomData); -impl> TendermintClientMinimal for TendermintValidatorFirm { +pub struct TendermintValidatorFirm; +impl TendermintClientMinimal for TendermintValidatorFirm { + const BLOCK_TIME_IN_SECONDS: u32 = { (serai_runtime::MILLISECS_PER_BLOCK / 1000) as u32 }; + type Block = Block; type Backend = sc_client_db::Backend; type Api = >::Api; type Client = FullClient; } -impl> TendermintValidator for TendermintValidatorFirm { +impl TendermintValidator for TendermintValidatorFirm { type CIDP = Cidp; type Environment = sc_basic_authorship::ProposerFactory< FullPool, @@ -86,33 +84,6 @@ impl> TendermintValidator for TendermintValidatorFirm { >; type Network = Arc::Hash>>; - type Announce = A; -} - -pub fn import_queue>( - task_manager: &TaskManager, - client: Arc, - announce: A, - pool: Arc>, - registry: Option<&Registry>, -) -> ( - TendermintAuthority>, - TendermintImportQueue>, -) { - import_queue::import_queue::>( - client.clone(), - announce, - Arc::new(Cidp), - sc_basic_authorship::ProposerFactory::new( - task_manager.spawn_handle(), - client, - pool, - registry, - None, - ), - &task_manager.spawn_essential_handle(), - registry, - ) } /* diff --git a/substrate/tendermint/client/src/tendermint.rs b/substrate/tendermint/client/src/tendermint.rs index 9ea02363..99e0f4ca 100644 --- a/substrate/tendermint/client/src/tendermint.rs +++ b/substrate/tendermint/client/src/tendermint.rs @@ -1,161 +1,40 @@ use std::{ marker::PhantomData, sync::{Arc, RwLock}, - time::{UNIX_EPOCH, SystemTime, Duration}, }; -use async_trait::async_trait; +use tokio::sync::RwLock as AsyncRwLock; -use log::warn; - -use tokio::{sync::RwLock as AsyncRwLock, task::yield_now}; - -use sp_core::{Encode, Decode, sr25519::Signature}; -use sp_inherents::{InherentData, InherentDataProvider, CreateInherentDataProviders}; +use sp_core::Decode; use sp_runtime::{ traits::{Header, Block}, - Digest, Justification, + Justification, }; use sp_blockchain::HeaderBackend; -use sp_api::BlockId; -use sp_consensus::{Error, BlockOrigin, Proposer, Environment}; -use sc_consensus::{ForkChoiceStrategy, BlockImportParams, import_queue::IncomingBlock}; +use sp_consensus::Error; +use sc_consensus::{ForkChoiceStrategy, BlockImportParams}; -use sc_service::ImportQueue; -use sc_client_api::{BlockBackend, Finalizer}; -use sc_network_gossip::GossipEngine; - -use substrate_prometheus_endpoint::Registry; - -use tendermint_machine::{ - ext::{BlockError, BlockNumber, Commit, Network}, - SignedMessage, TendermintMachine, -}; +use tendermint_machine::ext::{Commit, Network}; use crate::{ - CONSENSUS_ID, - types::TendermintValidator, - validators::TendermintValidators, - import_queue::{ImportFuture, TendermintImportQueue}, - gossip::TendermintGossip, - Announce, + CONSENSUS_ID, types::TendermintValidator, validators::TendermintValidators, + import_queue::TendermintImportQueue, authority::TendermintAuthority, }; -pub(crate) struct TendermintImport { +pub struct TendermintImport { _ta: PhantomData, - validators: Arc>, + pub(crate) validators: Arc>, - number: Arc>, - gossip_queue: Arc>>>, - importing_block: Arc::Hash>>>, + pub(crate) providers: Arc>>, + pub(crate) importing_block: Arc::Hash>>>, pub(crate) client: Arc, - announce: T::Announce, - providers: Arc, - - env: Arc>, pub(crate) queue: Arc>>>, } -pub struct TendermintAuthority(pub(crate) TendermintImport); -impl TendermintAuthority { - pub async fn validate(mut self, network: T::Network, registry: Option<&Registry>) { - let info = self.0.client.info(); - - // Header::Number: TryInto doesn't implement Debug and can't be unwrapped - let last_number = match info.best_number.try_into() { - Ok(best) => BlockNumber(best), - Err(_) => panic!("BlockNumber exceeded u64"), - }; - let last_time = Commit::>::decode( - &mut self - .0 - .client - .justifications(&BlockId::Hash(info.best_hash)) - .unwrap() - .map(|justifications| justifications.get(CONSENSUS_ID).cloned().unwrap()) - .unwrap_or_default() - .as_ref(), - ) - .map(|commit| commit.end_time) - // TODO: Genesis start time + BLOCK_TIME - .unwrap_or_else(|_| SystemTime::now().duration_since(UNIX_EPOCH).unwrap().as_secs()); - - let proposal = self - .0 - .get_proposal(&self.0.client.header(BlockId::Hash(info.best_hash)).unwrap().unwrap()) - .await; - - *self.0.number.write().unwrap() = last_number.0 + 1; - let mut gossip = GossipEngine::new( - network, - "tendermint", - Arc::new(TendermintGossip::new(self.0.number.clone(), self.0.validators.clone())), - registry, - ); - - let handle = TendermintMachine::new( - self.0.clone(), - // TODO - 0, // ValidatorId - (last_number, last_time), - proposal, - ); - - let mut last_number = last_number.0 + 1; - let mut recv = gossip - .messages_for(TendermintGossip::>::topic::(last_number)); - 'outer: loop { - // Send out any queued messages - let mut queue = self.0.gossip_queue.write().unwrap().drain(..).collect::>(); - for msg in queue.drain(..) { - gossip.gossip_message( - TendermintGossip::>::topic::(msg.number().0), - msg.encode(), - false, - ); - } - - // Handle any received messages - // Makes sure to handle all pending messages before acquiring the out-queue lock again - 'inner: loop { - match recv.try_next() { - Ok(Some(msg)) => handle - .messages - .send(match SignedMessage::decode(&mut msg.message.as_ref()) { - Ok(msg) => msg, - Err(e) => { - warn!("couldn't decode valid message: {}", e); - continue; - } - }) - .await - .unwrap(), - Ok(None) => break 'outer, - // No messages available - Err(_) => { - // Check if we the block updated and should be listening on a different topic - let curr = *self.0.number.read().unwrap(); - if last_number != curr { - last_number = curr; - // TODO: Will this return existing messages on the new height? Or will those have been - // ignored and are now gone? - recv = gossip.messages_for(TendermintGossip::>::topic::< - T::Block, - >(last_number)); - } - yield_now().await; - break 'inner; - } - } - } - } - } -} - impl Clone for TendermintImport { fn clone(&self) -> Self { TendermintImport { @@ -163,50 +42,31 @@ impl Clone for TendermintImport { validators: self.validators.clone(), - number: self.number.clone(), - gossip_queue: self.gossip_queue.clone(), + providers: self.providers.clone(), importing_block: self.importing_block.clone(), client: self.client.clone(), - announce: self.announce.clone(), - providers: self.providers.clone(), - - env: self.env.clone(), queue: self.queue.clone(), } } } impl TendermintImport { - pub(crate) fn new( - client: Arc, - announce: T::Announce, - providers: Arc, - env: T::Environment, - ) -> TendermintImport { + pub(crate) fn new(client: Arc) -> TendermintImport { TendermintImport { _ta: PhantomData, validators: Arc::new(TendermintValidators::new(client.clone())), - number: Arc::new(RwLock::new(0)), - gossip_queue: Arc::new(RwLock::new(vec![])), + providers: Arc::new(AsyncRwLock::new(None)), importing_block: Arc::new(RwLock::new(None)), client, - announce, - providers, - - env: Arc::new(AsyncRwLock::new(env)), queue: Arc::new(AsyncRwLock::new(None)), } } - async fn check_inherents( - &self, - block: T::Block, - providers: >::InherentDataProviders, - ) -> Result<(), Error> { + async fn check_inherents(&self, block: T::Block) -> Result<(), Error> { // TODO Ok(()) } @@ -250,7 +110,7 @@ impl TendermintImport { let commit: Commit> = Commit::decode(&mut justification.1.as_ref()).map_err(|_| Error::InvalidJustification)?; - if !self.verify_commit(hash, &commit) { + if !TendermintAuthority::new(self.clone()).verify_commit(hash, &commit) { Err(Error::InvalidJustification)?; } Ok(()) @@ -282,10 +142,10 @@ impl TendermintImport { block: &mut BlockImportParams, ) -> Result<(), Error> { if block.finalized { - if block.fork_choice.is_none() { + if block.fork_choice != Some(ForkChoiceStrategy::Custom(false)) { // Since we alw1ays set the fork choice, this means something else marked the block as // finalized, which shouldn't be possible. Ensuring nothing else is setting blocks as - // finalized ensures our security + // finalized helps ensure our security panic!("block was finalized despite not setting the fork choice"); } return Ok(()); @@ -301,12 +161,7 @@ impl TendermintImport { if !block.finalized { self.verify_origin(block.header.hash())?; if let Some(body) = block.body.clone() { - self - .check_inherents( - T::Block::new(block.header.clone(), body), - self.providers.create_inherent_data_providers(*block.header.parent_hash(), ()).await?, - ) - .await?; + self.check_inherents(T::Block::new(block.header.clone(), body)).await?; } } @@ -322,139 +177,4 @@ impl TendermintImport { Ok(()) } - - pub(crate) async fn get_proposal(&mut self, header: &::Header) -> T::Block { - let inherent_data = - match self.providers.create_inherent_data_providers(header.hash(), ()).await { - Ok(providers) => match providers.create_inherent_data() { - Ok(data) => Some(data), - Err(err) => { - warn!(target: "tendermint", "Failed to create inherent data: {}", err); - None - } - }, - Err(err) => { - warn!(target: "tendermint", "Failed to create inherent data providers: {}", err); - None - } - } - .unwrap_or_else(InherentData::new); - - let proposer = self - .env - .write() - .await - .init(header) - .await - .expect("Failed to create a proposer for the new block"); - // TODO: Production time, size limit - proposer - .propose(inherent_data, Digest::default(), Duration::from_secs(1), None) - .await - .expect("Failed to crate a new block proposal") - .block - } -} - -#[async_trait] -impl Network for TendermintImport { - type ValidatorId = u16; - type SignatureScheme = TendermintValidators; - type Weights = TendermintValidators; - type Block = T::Block; - - const BLOCK_TIME: u32 = { (serai_runtime::MILLISECS_PER_BLOCK / 1000) as u32 }; - - fn signature_scheme(&self) -> Arc> { - self.validators.clone() - } - - fn weights(&self) -> Arc> { - self.validators.clone() - } - - async fn broadcast(&mut self, msg: SignedMessage) { - self.gossip_queue.write().unwrap().push(msg); - } - - async fn slash(&mut self, validator: u16) { - todo!() - } - - // The Tendermint machine will call add_block for any block which is committed to, regardless of - // validity. To determine validity, it expects a validate function, which Substrate doesn't - // directly offer, and an add function. In order to comply with Serai's modified view of inherent - // transactions, validate MUST check inherents, yet add_block must not. - // - // In order to acquire a validate function, any block proposed by a legitimate proposer is - // imported. This performs full validation and makes the block available as a tip. While this - // would be incredibly unsafe thanks to the unchecked inherents, it's defined as a tip with less - // work, despite being a child of some parent. This means it won't be moved to nor operated on by - // the node. - // - // When Tendermint completes, the block is finalized, setting it as the tip regardless of work. - async fn validate(&mut self, block: &T::Block) -> Result<(), BlockError> { - let hash = block.hash(); - let (header, body) = block.clone().deconstruct(); - let parent = *header.parent_hash(); - let number = *header.number(); - - let mut queue_write = self.queue.write().await; - *self.importing_block.write().unwrap() = Some(hash); - - queue_write.as_mut().unwrap().import_blocks( - // We do not want this block, which hasn't been confirmed, to be broadcast over the net - // Substrate will generate notifications unless it's Genesis, which this isn't, InitialSync, - // which changes telemtry behavior, or File, which is... close enough - BlockOrigin::File, - vec![IncomingBlock { - hash, - header: Some(header), - body: Some(body), - indexed_body: None, - justifications: None, - origin: None, - allow_missing_state: false, - skip_execution: false, - // TODO: Only set to true if block was rejected due to its inherents - import_existing: true, - state: None, - }], - ); - - if !ImportFuture::new(hash, queue_write.as_mut().unwrap()).await { - todo!() - } - - // Sanity checks that a child block can have less work than its parent - { - let info = self.client.info(); - assert_eq!(info.best_hash, parent); - assert_eq!(info.finalized_hash, parent); - assert_eq!(info.best_number, number - 1u8.into()); - assert_eq!(info.finalized_number, number - 1u8.into()); - } - - Ok(()) - } - - async fn add_block( - &mut self, - block: T::Block, - commit: Commit>, - ) -> T::Block { - let hash = block.hash(); - let justification = (CONSENSUS_ID, commit.encode()); - debug_assert!(self.verify_justification(hash, &justification).is_ok()); - - self - .client - .finalize_block(BlockId::Hash(hash), Some(justification), true) - .map_err(|_| Error::InvalidJustification) - .unwrap(); - *self.number.write().unwrap() += 1; - self.announce.announce(hash); - - self.get_proposal(block.header()).await - } } diff --git a/substrate/tendermint/client/src/types.rs b/substrate/tendermint/client/src/types.rs index 5b2ce8ba..a1474ff5 100644 --- a/substrate/tendermint/client/src/types.rs +++ b/substrate/tendermint/client/src/types.rs @@ -8,14 +8,15 @@ use sp_consensus::Environment; use sc_consensus::BlockImport; use sc_client_api::{BlockBackend, Backend, Finalizer}; +use sc_network::NetworkBlock; use sc_network_gossip::Network; use sp_tendermint::TendermintApi; -use crate::Announce; - /// Trait consolidating all generics required by sc_tendermint for processing. pub trait TendermintClient: Send + Sync + 'static { + const BLOCK_TIME_IN_SECONDS: u32; + type Block: Block; type Backend: Backend + 'static; @@ -40,6 +41,8 @@ pub trait TendermintClient: Send + Sync + 'static { /// Trait implementable on firm types to automatically provide a full TendermintClient impl. pub trait TendermintClientMinimal: Send + Sync + 'static { + const BLOCK_TIME_IN_SECONDS: u32; + type Block: Block; type Backend: Backend + 'static; type Api: ApiExt + TendermintApi; @@ -58,6 +61,8 @@ where >::Api: TendermintApi, TransactionFor: Send + Sync + 'static, { + const BLOCK_TIME_IN_SECONDS: u32 = T::BLOCK_TIME_IN_SECONDS; + type Block = T::Block; type Backend = T::Backend; @@ -72,6 +77,10 @@ pub trait TendermintValidator: TendermintClient { type CIDP: CreateInherentDataProviders + 'static; type Environment: Send + Sync + Environment + 'static; - type Network: Clone + Send + Sync + Network + 'static; - type Announce: Announce; + type Network: Clone + + Send + + Sync + + Network + + NetworkBlock<::Hash, <::Header as Header>::Number> + + 'static; } diff --git a/substrate/tendermint/client/src/validators.rs b/substrate/tendermint/client/src/validators.rs index c21e3274..5398e127 100644 --- a/substrate/tendermint/client/src/validators.rs +++ b/substrate/tendermint/client/src/validators.rs @@ -79,7 +79,7 @@ impl Deref for Refresh { } } -pub(crate) struct TendermintValidators(Refresh); +pub struct TendermintValidators(Refresh); impl TendermintValidators { pub(crate) fn new(client: Arc) -> TendermintValidators {