diff --git a/substrate/consensus/src/block_import.rs b/substrate/consensus/src/block_import.rs new file mode 100644 index 00000000..9672f2ce --- /dev/null +++ b/substrate/consensus/src/block_import.rs @@ -0,0 +1,58 @@ +use std::collections::HashMap; + +use async_trait::async_trait; + +use sp_inherents::CreateInherentDataProviders; +use sp_runtime::traits::Block; +use sp_blockchain::HeaderBackend; +use sp_api::{TransactionFor, ProvideRuntimeApi}; + +use sp_consensus::{Error, CacheKeyId, Environment}; +use sc_consensus::{BlockCheckParams, BlockImportParams, ImportResult, BlockImport}; + +use sc_client_api::{Backend, Finalizer}; + +use crate::tendermint::TendermintImport; + +#[async_trait] +impl< + B: Block, + Be: Backend + 'static, + C: Send + Sync + HeaderBackend + Finalizer + ProvideRuntimeApi + 'static, + I: Send + Sync + BlockImport> + 'static, + CIDP: CreateInherentDataProviders + 'static, + E: Send + Sync + Environment + 'static, + > BlockImport for TendermintImport +where + I::Error: Into, + TransactionFor: Send + Sync + 'static, +{ + type Error = Error; + type Transaction = TransactionFor; + + // TODO: Is there a DoS where you send a block without justifications, causing it to error, + // yet adding it to the blacklist in the process preventing further syncing? + async fn check_block( + &mut self, + mut block: BlockCheckParams, + ) -> Result { + self.verify_order(block.parent_hash, block.number)?; + + // Does not verify origin here as origin only applies to unfinalized blocks + // We don't have context on if this block has justifications or not + + block.allow_missing_state = false; + block.allow_missing_parent = false; + + self.inner.write().await.check_block(block).await.map_err(Into::into) + } + + async fn import_block( + &mut self, + mut block: BlockImportParams>, + new_cache: HashMap>, + ) -> Result { + self.check(&mut block).await?; + self.inner.write().await.import_block(block, new_cache).await.map_err(Into::into) + } +} diff --git a/substrate/consensus/src/import_queue.rs b/substrate/consensus/src/import_queue.rs index 9b4f24eb..0a8d433a 100644 --- a/substrate/consensus/src/import_queue.rs +++ b/substrate/consensus/src/import_queue.rs @@ -1,386 +1,29 @@ -// 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. - use std::{ - marker::PhantomData, pin::Pin, sync::{Arc, RwLock}, task::{Poll, Context}, future::Future, - time::Duration, - collections::HashMap, }; -use async_trait::async_trait; +use tokio::runtime::Handle; -use log::warn; - -use tokio::{sync::RwLock as AsyncRwLock, runtime::Handle}; - -use sp_core::{Encode, Decode}; -use sp_application_crypto::sr25519::Signature; -use sp_inherents::{InherentData, InherentDataProvider, CreateInherentDataProviders}; -use sp_runtime::{ - traits::{Header, Block}, - Digest, Justification, -}; +use sp_inherents::CreateInherentDataProviders; +use sp_runtime::traits::{Header, Block}; use sp_blockchain::HeaderBackend; -use sp_api::{BlockId, TransactionFor, ProvideRuntimeApi}; +use sp_api::{TransactionFor, ProvideRuntimeApi}; -use sp_consensus::{Error, CacheKeyId, BlockOrigin, Proposer, Environment}; -#[rustfmt::skip] // rustfmt doesn't know how to handle this line -use sc_consensus::{ - ForkChoiceStrategy, - BlockCheckParams, - BlockImportParams, - Verifier, - ImportResult, - BlockImport, - JustificationImport, - import_queue::IncomingBlock, - BlockImportStatus, - BlockImportError, - Link, - BasicQueue, -}; +use sp_consensus::{Error, Environment}; +use sc_consensus::{BlockImport, BlockImportStatus, BlockImportError, Link, BasicQueue}; use sc_service::ImportQueue; use sc_client_api::{Backend, Finalizer}; use substrate_prometheus_endpoint::Registry; -use tendermint_machine::{ - ext::{BlockError, Commit, Network}, - SignedMessage, -}; - -use crate::{signature_scheme::TendermintSigner, weights::TendermintWeights}; - -const CONSENSUS_ID: [u8; 4] = *b"tend"; +use crate::tendermint::TendermintImport; pub type TendermintImportQueue = BasicQueue; -struct TendermintImport< - B: Block, - Be: Backend + 'static, - C: Send + Sync + HeaderBackend + Finalizer + ProvideRuntimeApi + 'static, - I: Send + Sync + BlockImport> + 'static, - CIDP: CreateInherentDataProviders + 'static, - E: Send + Sync + Environment + 'static, -> { - _block: PhantomData, - _backend: PhantomData, - - importing_block: Arc>>, - - client: Arc, - inner: Arc>, - providers: Arc, - - env: Arc>, - queue: Arc>>>>, -} - -impl< - B: Block, - Be: Backend + 'static, - C: Send + Sync + HeaderBackend + Finalizer + ProvideRuntimeApi + 'static, - I: Send + Sync + BlockImport> + 'static, - CIDP: CreateInherentDataProviders + 'static, - E: Send + Sync + Environment + 'static, - > Clone for TendermintImport -{ - fn clone(&self) -> Self { - TendermintImport { - _block: PhantomData, - _backend: PhantomData, - - importing_block: self.importing_block.clone(), - - client: self.client.clone(), - inner: self.inner.clone(), - providers: self.providers.clone(), - - env: self.env.clone(), - queue: self.queue.clone(), - } - } -} - -impl< - B: Block, - Be: Backend + 'static, - C: Send + Sync + HeaderBackend + Finalizer + ProvideRuntimeApi + 'static, - I: Send + Sync + BlockImport> + 'static, - CIDP: CreateInherentDataProviders + 'static, - E: Send + Sync + Environment + 'static, - > TendermintImport -where - TransactionFor: Send + Sync + 'static, -{ - async fn check_inherents( - &self, - block: B, - providers: CIDP::InherentDataProviders, - ) -> Result<(), Error> { - // TODO - Ok(()) - } - - // Ensure this is part of a sequential import - fn verify_order( - &self, - parent: B::Hash, - number: ::Number, - ) -> Result<(), Error> { - let info = self.client.info(); - if (info.best_hash != parent) || ((info.best_number + 1u16.into()) != number) { - Err(Error::Other("non-sequential import".into()))?; - } - Ok(()) - } - - // Do not allow blocks from the traditional network to be broadcast - // Only allow blocks from Tendermint - // Tendermint's propose message could be rewritten as a seal OR Tendermint could produce blocks - // which this checks the proposer slot for, and then tells the Tendermint machine - // While those would be more seamless with Substrate, there's no actual benefit to doing so - fn verify_origin(&self, hash: B::Hash) -> Result<(), Error> { - if let Some(tm_hash) = *self.importing_block.read().unwrap() { - if hash == tm_hash { - return Ok(()); - } - } - Err(Error::Other("block created outside of tendermint".into())) - } - - // Errors if the justification isn't valid - fn verify_justification( - &self, - hash: B::Hash, - justification: &Justification, - ) -> Result<(), Error> { - if justification.0 != CONSENSUS_ID { - Err(Error::InvalidJustification)?; - } - - let commit: Commit = - Commit::decode(&mut justification.1.as_ref()).map_err(|_| Error::InvalidJustification)?; - if !self.verify_commit(hash, &commit) { - Err(Error::InvalidJustification)?; - } - Ok(()) - } - - // Verifies the justifications aren't malformed, not that the block is justified - // Errors if justifications is neither empty nor a sinlge Tendermint justification - // If the block does have a justification, finalized will be set to true - fn verify_justifications(&self, block: &mut BlockImportParams) -> Result<(), Error> { - if !block.finalized { - if let Some(justifications) = &block.justifications { - let mut iter = justifications.iter(); - let next = iter.next(); - if next.is_none() || iter.next().is_some() { - Err(Error::InvalidJustification)?; - } - self.verify_justification(block.header.hash(), next.unwrap())?; - - block.finalized = true; // TODO: Is this setting valid? - } - } - Ok(()) - } - - async fn check(&self, block: &mut BlockImportParams) -> Result<(), Error> { - if block.finalized { - if block.fork_choice.is_none() { - // 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 - panic!("block was finalized despite not setting the fork choice"); - } - return Ok(()); - } - - // Set the block as a worse choice - block.fork_choice = Some(ForkChoiceStrategy::Custom(false)); - - self.verify_order(*block.header.parent_hash(), *block.header.number())?; - self.verify_justifications(block)?; - - // If the block wasn't finalized, verify the origin and validity of its inherents - if !block.finalized { - self.verify_origin(block.header.hash())?; - if let Some(body) = block.body.clone() { - self - .check_inherents( - B::new(block.header.clone(), body), - self.providers.create_inherent_data_providers(*block.header.parent_hash(), ()).await?, - ) - .await?; - } - } - - // Additionally check these fields are empty - // They *should* be unused, so requiring their emptiness prevents malleability and ensures - // nothing slips through - if !block.post_digests.is_empty() { - Err(Error::Other("post-digests included".into()))?; - } - if !block.auxiliary.is_empty() { - Err(Error::Other("auxiliary included".into()))?; - } - - Ok(()) - } - - async fn get_proposal(&mut self, block: &B) -> B { - let inherent_data = match self.providers.create_inherent_data_providers(block.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(block.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 - } - - fn import_justification_actual( - &mut self, - hash: B::Hash, - justification: Justification, - ) -> Result<(), Error> { - self.verify_justification(hash, &justification)?; - self - .client - .finalize_block(BlockId::Hash(hash), Some(justification), true) - .map_err(|_| Error::InvalidJustification) - } -} - -#[async_trait] -impl< - B: Block, - Be: Backend + 'static, - C: Send + Sync + HeaderBackend + Finalizer + ProvideRuntimeApi + 'static, - I: Send + Sync + BlockImport> + 'static, - CIDP: CreateInherentDataProviders + 'static, - E: Send + Sync + Environment + 'static, - > BlockImport for TendermintImport -where - I::Error: Into, - TransactionFor: Send + Sync + 'static, -{ - type Error = Error; - type Transaction = TransactionFor; - - // TODO: Is there a DoS where you send a block without justifications, causing it to error, - // yet adding it to the blacklist in the process preventing further syncing? - async fn check_block( - &mut self, - mut block: BlockCheckParams, - ) -> Result { - self.verify_order(block.parent_hash, block.number)?; - - // Does not verify origin here as origin only applies to unfinalized blocks - // We don't have context on if this block has justifications or not - - block.allow_missing_state = false; - block.allow_missing_parent = false; - - self.inner.write().await.check_block(block).await.map_err(Into::into) - } - - async fn import_block( - &mut self, - mut block: BlockImportParams>, - new_cache: HashMap>, - ) -> Result { - self.check(&mut block).await?; - self.inner.write().await.import_block(block, new_cache).await.map_err(Into::into) - } -} - -#[async_trait] -impl< - B: Block, - Be: Backend + 'static, - C: Send + Sync + HeaderBackend + Finalizer + ProvideRuntimeApi + 'static, - I: Send + Sync + BlockImport> + 'static, - CIDP: CreateInherentDataProviders + 'static, - E: Send + Sync + Environment + 'static, - > JustificationImport for TendermintImport -where - TransactionFor: Send + Sync + 'static, -{ - type Error = Error; - - async fn on_start(&mut self) -> Vec<(B::Hash, ::Number)> { - vec![] - } - - async fn import_justification( - &mut self, - hash: B::Hash, - _: ::Number, - justification: Justification, - ) -> Result<(), Error> { - self.import_justification_actual(hash, justification) - } -} - -#[async_trait] -impl< - B: Block, - Be: Backend + 'static, - C: Send + Sync + HeaderBackend + Finalizer + ProvideRuntimeApi + 'static, - I: Send + Sync + BlockImport> + 'static, - CIDP: CreateInherentDataProviders + 'static, - E: Send + Sync + Environment + 'static, - > Verifier for TendermintImport -where - TransactionFor: Send + Sync + 'static, -{ - async fn verify( - &mut self, - mut block: BlockImportParams, - ) -> Result<(BlockImportParams, Option)>>), String> { - self.check(&mut block).await.map_err(|e| format!("{}", e))?; - Ok((block, None)) - } -} - // Custom helpers for ImportQueue in order to obtain the result of a block's importing struct ValidateLink(Option<(B::Hash, bool)>); impl Link for ValidateLink { @@ -399,9 +42,15 @@ impl Link for ValidateLink { } } -struct ImportFuture<'a, B: Block, T: Send>(B::Hash, RwLock<&'a mut TendermintImportQueue>); +pub(crate) struct ImportFuture<'a, B: Block, T: Send>( + B::Hash, + RwLock<&'a mut TendermintImportQueue>, +); impl<'a, B: Block, T: Send> ImportFuture<'a, B, T> { - fn new(hash: B::Hash, queue: &'a mut TendermintImportQueue) -> ImportFuture { + pub(crate) fn new( + hash: B::Hash, + queue: &'a mut TendermintImportQueue, + ) -> ImportFuture { ImportFuture(hash, RwLock::new(queue)) } } @@ -421,75 +70,6 @@ impl<'a, B: Block, T: Send> Future for ImportFuture<'a, B, T> { } } -#[async_trait] -impl< - B: Block, - Be: Backend + 'static, - C: Send + Sync + HeaderBackend + Finalizer + ProvideRuntimeApi + 'static, - I: Send + Sync + BlockImport> + 'static, - CIDP: CreateInherentDataProviders + 'static, - E: Send + Sync + Environment + 'static, - > Network for TendermintImport -where - TransactionFor: Send + Sync + 'static, -{ - type ValidatorId = u16; - type SignatureScheme = TendermintSigner; - type Weights = TendermintWeights; - type Block = B; - - const BLOCK_TIME: u32 = { (serai_runtime::MILLISECS_PER_BLOCK / 1000) as u32 }; - - fn signature_scheme(&self) -> Arc { - Arc::new(TendermintSigner::new()) - } - - fn weights(&self) -> Arc { - Arc::new(TendermintWeights) - } - - async fn broadcast(&mut self, msg: SignedMessage) { - // TODO - } - - async fn slash(&mut self, validator: u16) { - todo!() - } - - async fn validate(&mut self, block: &B) -> Result<(), BlockError> { - let hash = block.hash(); - let (header, body) = block.clone().deconstruct(); - *self.importing_block.write().unwrap() = Some(hash); - self.queue.write().await.as_mut().unwrap().import_blocks( - BlockOrigin::NetworkBroadcast, - 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, self.queue.write().await.as_mut().unwrap()).await { - Ok(()) - } else { - todo!() - } - } - - async fn add_block(&mut self, block: B, commit: Commit) -> B { - self.import_justification_actual(block.hash(), (CONSENSUS_ID, commit.encode())).unwrap(); - self.get_proposal(&block).await - } -} - pub fn import_queue< B: Block, Be: Backend + 'static, @@ -509,19 +89,7 @@ where I::Error: Into, TransactionFor: Send + Sync + 'static, { - let import = TendermintImport { - _block: PhantomData, - _backend: PhantomData, - - importing_block: Arc::new(RwLock::new(None)), - - client, - inner: Arc::new(AsyncRwLock::new(inner)), - providers, - - env: Arc::new(AsyncRwLock::new(env)), - queue: Arc::new(AsyncRwLock::new(None)), - }; + let import = TendermintImport::new(client, inner, providers, env); let boxed = Box::new(import.clone()); let queue = diff --git a/substrate/consensus/src/justification_import.rs b/substrate/consensus/src/justification_import.rs new file mode 100644 index 00000000..fb5b5b53 --- /dev/null +++ b/substrate/consensus/src/justification_import.rs @@ -0,0 +1,46 @@ +use async_trait::async_trait; + +use sp_inherents::CreateInherentDataProviders; +use sp_runtime::{ + traits::{Header, Block}, + Justification, +}; +use sp_blockchain::HeaderBackend; +use sp_api::{TransactionFor, ProvideRuntimeApi}; + +use sp_consensus::{Error, Environment}; +use sc_consensus::{BlockImport, JustificationImport, BasicQueue}; + +use sc_client_api::{Backend, Finalizer}; + +use crate::tendermint::TendermintImport; + +pub type TendermintImportQueue = BasicQueue; + +#[async_trait] +impl< + B: Block, + Be: Backend + 'static, + C: Send + Sync + HeaderBackend + Finalizer + ProvideRuntimeApi + 'static, + I: Send + Sync + BlockImport> + 'static, + CIDP: CreateInherentDataProviders + 'static, + E: Send + Sync + Environment + 'static, + > JustificationImport for TendermintImport +where + TransactionFor: Send + Sync + 'static, +{ + type Error = Error; + + async fn on_start(&mut self) -> Vec<(B::Hash, ::Number)> { + vec![] + } + + async fn import_justification( + &mut self, + hash: B::Hash, + _: ::Number, + justification: Justification, + ) -> Result<(), Error> { + self.import_justification_actual(hash, justification) + } +} diff --git a/substrate/consensus/src/lib.rs b/substrate/consensus/src/lib.rs index a912d29e..4d7233c9 100644 --- a/substrate/consensus/src/lib.rs +++ b/substrate/consensus/src/lib.rs @@ -14,12 +14,19 @@ use serai_runtime::{self, opaque::Block, RuntimeApi}; mod signature_scheme; mod weights; +mod tendermint; +mod block_import; +mod justification_import; +mod verifier; + mod import_queue; use import_queue::TendermintImportQueue; mod select_chain; pub use select_chain::TendermintSelectChain; +const CONSENSUS_ID: [u8; 4] = *b"tend"; + pub struct ExecutorDispatch; impl NativeExecutionDispatch for ExecutorDispatch { #[cfg(feature = "runtime-benchmarks")] @@ -72,29 +79,6 @@ pub fn authority( } /* -// Produce a block every 6 seconds -async fn produce< - Block: sp_api::BlockT, - Algorithm: sc_pow::PowAlgorithm + Send + Sync + 'static, - C: sp_api::ProvideRuntimeApi + 'static, - Link: sc_consensus::JustificationSyncLink + 'static, - P: Send + 'static, ->( - worker: sc_pow::MiningHandle, -) where - sp_api::TransactionFor: Send + 'static, -{ - loop { - let worker_clone = worker.clone(); - std::thread::spawn(move || { - tokio::runtime::Runtime::new().unwrap().handle().block_on(async { - worker_clone.submit(vec![]).await; - }); - }); - tokio::time::sleep(Duration::from_secs(6)).await; - } -} - // If we're an authority, produce blocks pub fn authority + 'static>( task_manager: &TaskManager, diff --git a/substrate/consensus/src/tendermint.rs b/substrate/consensus/src/tendermint.rs new file mode 100644 index 00000000..bc1d1d57 --- /dev/null +++ b/substrate/consensus/src/tendermint.rs @@ -0,0 +1,360 @@ +use std::{ + marker::PhantomData, + sync::{Arc, RwLock}, + time::Duration, +}; + +use async_trait::async_trait; + +use log::warn; + +use tokio::sync::RwLock as AsyncRwLock; + +use sp_core::{Encode, Decode}; +use sp_application_crypto::sr25519::Signature; +use sp_inherents::{InherentData, InherentDataProvider, CreateInherentDataProviders}; +use sp_runtime::{ + traits::{Header, Block}, + Digest, Justification, +}; +use sp_blockchain::HeaderBackend; +use sp_api::{BlockId, TransactionFor, ProvideRuntimeApi}; + +use sp_consensus::{Error, BlockOrigin, Proposer, Environment}; +use sc_consensus::{ForkChoiceStrategy, BlockImportParams, BlockImport, import_queue::IncomingBlock}; + +use sc_service::ImportQueue; +use sc_client_api::{Backend, Finalizer}; + +use tendermint_machine::{ + ext::{BlockError, Commit, Network}, + SignedMessage, +}; + +use crate::{ + CONSENSUS_ID, + signature_scheme::TendermintSigner, + weights::TendermintWeights, + import_queue::{ImportFuture, TendermintImportQueue}, +}; + +pub(crate) struct TendermintImport< + B: Block, + Be: Backend + 'static, + C: Send + Sync + HeaderBackend + Finalizer + ProvideRuntimeApi + 'static, + I: Send + Sync + BlockImport> + 'static, + CIDP: CreateInherentDataProviders + 'static, + E: Send + Sync + Environment + 'static, +> { + _block: PhantomData, + _backend: PhantomData, + + importing_block: Arc>>, + + client: Arc, + pub(crate) inner: Arc>, + providers: Arc, + + env: Arc>, + pub(crate) queue: Arc>>>>, +} + +impl< + B: Block, + Be: Backend + 'static, + C: Send + Sync + HeaderBackend + Finalizer + ProvideRuntimeApi + 'static, + I: Send + Sync + BlockImport> + 'static, + CIDP: CreateInherentDataProviders + 'static, + E: Send + Sync + Environment + 'static, + > Clone for TendermintImport +{ + fn clone(&self) -> Self { + TendermintImport { + _block: PhantomData, + _backend: PhantomData, + + importing_block: self.importing_block.clone(), + + client: self.client.clone(), + inner: self.inner.clone(), + providers: self.providers.clone(), + + env: self.env.clone(), + queue: self.queue.clone(), + } + } +} + +impl< + B: Block, + Be: Backend + 'static, + C: Send + Sync + HeaderBackend + Finalizer + ProvideRuntimeApi + 'static, + I: Send + Sync + BlockImport> + 'static, + CIDP: CreateInherentDataProviders + 'static, + E: Send + Sync + Environment + 'static, + > TendermintImport +where + TransactionFor: Send + Sync + 'static, +{ + pub(crate) fn new( + client: Arc, + inner: I, + providers: Arc, + env: E, + ) -> TendermintImport { + TendermintImport { + _block: PhantomData, + _backend: PhantomData, + + importing_block: Arc::new(RwLock::new(None)), + + client, + inner: Arc::new(AsyncRwLock::new(inner)), + providers, + + env: Arc::new(AsyncRwLock::new(env)), + queue: Arc::new(AsyncRwLock::new(None)), + } + } + + async fn check_inherents( + &self, + block: B, + providers: CIDP::InherentDataProviders, + ) -> Result<(), Error> { + // TODO + Ok(()) + } + + // Ensure this is part of a sequential import + pub(crate) fn verify_order( + &self, + parent: B::Hash, + number: ::Number, + ) -> Result<(), Error> { + let info = self.client.info(); + if (info.best_hash != parent) || ((info.best_number + 1u16.into()) != number) { + Err(Error::Other("non-sequential import".into()))?; + } + Ok(()) + } + + // Do not allow blocks from the traditional network to be broadcast + // Only allow blocks from Tendermint + // Tendermint's propose message could be rewritten as a seal OR Tendermint could produce blocks + // which this checks the proposer slot for, and then tells the Tendermint machine + // While those would be more seamless with Substrate, there's no actual benefit to doing so + fn verify_origin(&self, hash: B::Hash) -> Result<(), Error> { + if let Some(tm_hash) = *self.importing_block.read().unwrap() { + if hash == tm_hash { + return Ok(()); + } + } + Err(Error::Other("block created outside of tendermint".into())) + } + + // Errors if the justification isn't valid + fn verify_justification( + &self, + hash: B::Hash, + justification: &Justification, + ) -> Result<(), Error> { + if justification.0 != CONSENSUS_ID { + Err(Error::InvalidJustification)?; + } + + let commit: Commit = + Commit::decode(&mut justification.1.as_ref()).map_err(|_| Error::InvalidJustification)?; + if !self.verify_commit(hash, &commit) { + Err(Error::InvalidJustification)?; + } + Ok(()) + } + + // Verifies the justifications aren't malformed, not that the block is justified + // Errors if justifications is neither empty nor a sinlge Tendermint justification + // If the block does have a justification, finalized will be set to true + fn verify_justifications(&self, block: &mut BlockImportParams) -> Result<(), Error> { + if !block.finalized { + if let Some(justifications) = &block.justifications { + let mut iter = justifications.iter(); + let next = iter.next(); + if next.is_none() || iter.next().is_some() { + Err(Error::InvalidJustification)?; + } + self.verify_justification(block.header.hash(), next.unwrap())?; + + block.finalized = true; // TODO: Is this setting valid? + } + } + Ok(()) + } + + pub(crate) async fn check(&self, block: &mut BlockImportParams) -> Result<(), Error> { + if block.finalized { + if block.fork_choice.is_none() { + // 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 + panic!("block was finalized despite not setting the fork choice"); + } + return Ok(()); + } + + // Set the block as a worse choice + block.fork_choice = Some(ForkChoiceStrategy::Custom(false)); + + self.verify_order(*block.header.parent_hash(), *block.header.number())?; + self.verify_justifications(block)?; + + // If the block wasn't finalized, verify the origin and validity of its inherents + if !block.finalized { + self.verify_origin(block.header.hash())?; + if let Some(body) = block.body.clone() { + self + .check_inherents( + B::new(block.header.clone(), body), + self.providers.create_inherent_data_providers(*block.header.parent_hash(), ()).await?, + ) + .await?; + } + } + + // Additionally check these fields are empty + // They *should* be unused, so requiring their emptiness prevents malleability and ensures + // nothing slips through + if !block.post_digests.is_empty() { + Err(Error::Other("post-digests included".into()))?; + } + if !block.auxiliary.is_empty() { + Err(Error::Other("auxiliary included".into()))?; + } + + Ok(()) + } + + async fn get_proposal(&mut self, block: &B) -> B { + let inherent_data = match self.providers.create_inherent_data_providers(block.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(block.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 + } + + pub(crate) fn import_justification_actual( + &mut self, + hash: B::Hash, + justification: Justification, + ) -> Result<(), Error> { + self.verify_justification(hash, &justification)?; + self + .client + .finalize_block(BlockId::Hash(hash), Some(justification), true) + .map_err(|_| Error::InvalidJustification) + } +} + +#[async_trait] +impl< + B: Block, + Be: Backend + 'static, + C: Send + Sync + HeaderBackend + Finalizer + ProvideRuntimeApi + 'static, + I: Send + Sync + BlockImport> + 'static, + CIDP: CreateInherentDataProviders + 'static, + E: Send + Sync + Environment + 'static, + > Network for TendermintImport +where + TransactionFor: Send + Sync + 'static, +{ + type ValidatorId = u16; + type SignatureScheme = TendermintSigner; + type Weights = TendermintWeights; + type Block = B; + + const BLOCK_TIME: u32 = { (serai_runtime::MILLISECS_PER_BLOCK / 1000) as u32 }; + + fn signature_scheme(&self) -> Arc { + Arc::new(TendermintSigner::new()) + } + + fn weights(&self) -> Arc { + Arc::new(TendermintWeights) + } + + async fn broadcast(&mut self, msg: SignedMessage) { + // TODO + } + + 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: &B) -> Result<(), BlockError> { + let hash = block.hash(); + let (header, body) = block.clone().deconstruct(); + *self.importing_block.write().unwrap() = Some(hash); + self.queue.write().await.as_mut().unwrap().import_blocks( + BlockOrigin::NetworkBroadcast, + 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, self.queue.write().await.as_mut().unwrap()).await { + Ok(()) + } else { + todo!() + } + } + + async fn add_block(&mut self, block: B, commit: Commit) -> B { + self.import_justification_actual(block.hash(), (CONSENSUS_ID, commit.encode())).unwrap(); + self.get_proposal(&block).await + } +} diff --git a/substrate/consensus/src/verifier.rs b/substrate/consensus/src/verifier.rs new file mode 100644 index 00000000..f5427c3f --- /dev/null +++ b/substrate/consensus/src/verifier.rs @@ -0,0 +1,34 @@ +use async_trait::async_trait; + +use sp_inherents::CreateInherentDataProviders; +use sp_runtime::traits::Block; +use sp_blockchain::HeaderBackend; +use sp_api::{TransactionFor, ProvideRuntimeApi}; + +use sp_consensus::{CacheKeyId, Environment}; +use sc_consensus::{BlockImportParams, Verifier, BlockImport}; + +use sc_client_api::{Backend, Finalizer}; + +use crate::tendermint::TendermintImport; + +#[async_trait] +impl< + B: Block, + Be: Backend + 'static, + C: Send + Sync + HeaderBackend + Finalizer + ProvideRuntimeApi + 'static, + I: Send + Sync + BlockImport> + 'static, + CIDP: CreateInherentDataProviders + 'static, + E: Send + Sync + Environment + 'static, + > Verifier for TendermintImport +where + TransactionFor: Send + Sync + 'static, +{ + async fn verify( + &mut self, + mut block: BlockImportParams, + ) -> Result<(BlockImportParams, Option)>>), String> { + self.check(&mut block).await.map_err(|e| format!("{}", e))?; + Ok((block, None)) + } +}