mirror of
https://github.com/serai-dex/serai.git
synced 2025-01-24 11:36:18 +00:00
Create a dedicated file for being a Tendermint authority
This commit is contained in:
parent
066bc40a88
commit
3d7c12adcd
9 changed files with 422 additions and 395 deletions
1
Cargo.lock
generated
1
Cargo.lock
generated
|
@ -7595,6 +7595,7 @@ dependencies = [
|
||||||
"pallet-tendermint",
|
"pallet-tendermint",
|
||||||
"pallet-transaction-payment",
|
"pallet-transaction-payment",
|
||||||
"pallet-transaction-payment-rpc",
|
"pallet-transaction-payment-rpc",
|
||||||
|
"sc-basic-authorship",
|
||||||
"sc-cli",
|
"sc-cli",
|
||||||
"sc-client-api",
|
"sc-client-api",
|
||||||
"sc-consensus",
|
"sc-consensus",
|
||||||
|
|
|
@ -27,6 +27,7 @@ sp-block-builder = { git = "https://github.com/serai-dex/substrate" }
|
||||||
|
|
||||||
sc-keystore = { 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-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-transaction-pool-api = { git = "https://github.com/serai-dex/substrate" }
|
||||||
sc-executor = { 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" }
|
sc-service = { git = "https://github.com/serai-dex/substrate" }
|
||||||
|
|
|
@ -1,15 +1,12 @@
|
||||||
use std::sync::{Arc, RwLock};
|
use std::sync::Arc;
|
||||||
|
|
||||||
use sp_core::H256;
|
|
||||||
|
|
||||||
use sc_executor::NativeElseWasmExecutor;
|
use sc_executor::NativeElseWasmExecutor;
|
||||||
use sc_service::{error::Error as ServiceError, Configuration, TaskManager};
|
use sc_service::{error::Error as ServiceError, Configuration, TaskManager};
|
||||||
use sc_network::{NetworkService, NetworkBlock};
|
|
||||||
use sc_telemetry::{Telemetry, TelemetryWorker};
|
use sc_telemetry::{Telemetry, TelemetryWorker};
|
||||||
|
|
||||||
use serai_runtime::{self, opaque::Block, RuntimeApi};
|
use serai_runtime::{self, opaque::Block, RuntimeApi};
|
||||||
pub(crate) use serai_consensus::{
|
pub(crate) use serai_consensus::{
|
||||||
TendermintAuthority, ExecutorDispatch, Announce, FullClient, TendermintValidatorFirm,
|
TendermintImport, TendermintAuthority, ExecutorDispatch, FullClient, TendermintValidatorFirm,
|
||||||
};
|
};
|
||||||
|
|
||||||
type FullBackend = sc_service::TFullBackend<Block>;
|
type FullBackend = sc_service::TFullBackend<Block>;
|
||||||
|
@ -24,30 +21,9 @@ type PartialComponents = sc_service::PartialComponents<
|
||||||
Option<Telemetry>,
|
Option<Telemetry>,
|
||||||
>;
|
>;
|
||||||
|
|
||||||
#[derive(Clone)]
|
|
||||||
pub struct NetworkAnnounce(Arc<RwLock<Option<Arc<NetworkService<Block, H256>>>>>);
|
|
||||||
impl NetworkAnnounce {
|
|
||||||
fn new() -> NetworkAnnounce {
|
|
||||||
NetworkAnnounce(Arc::new(RwLock::new(None)))
|
|
||||||
}
|
|
||||||
}
|
|
||||||
impl Announce<Block> 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(
|
pub fn new_partial(
|
||||||
config: &Configuration,
|
config: &Configuration,
|
||||||
) -> Result<
|
) -> Result<(TendermintImport<TendermintValidatorFirm>, PartialComponents), ServiceError> {
|
||||||
(
|
|
||||||
(NetworkAnnounce, TendermintAuthority<TendermintValidatorFirm<NetworkAnnounce>>),
|
|
||||||
PartialComponents,
|
|
||||||
),
|
|
||||||
ServiceError,
|
|
||||||
> {
|
|
||||||
if config.keystore_remote.is_some() {
|
if config.keystore_remote.is_some() {
|
||||||
return Err(ServiceError::Other("Remote Keystores are not supported".to_string()));
|
return Err(ServiceError::Other("Remote Keystores are not supported".to_string()));
|
||||||
}
|
}
|
||||||
|
@ -91,19 +67,16 @@ pub fn new_partial(
|
||||||
client.clone(),
|
client.clone(),
|
||||||
);
|
);
|
||||||
|
|
||||||
let announce = NetworkAnnounce::new();
|
|
||||||
let (authority, import_queue) = serai_consensus::import_queue(
|
let (authority, import_queue) = serai_consensus::import_queue(
|
||||||
&task_manager,
|
&task_manager.spawn_essential_handle(),
|
||||||
client.clone(),
|
client.clone(),
|
||||||
announce.clone(),
|
|
||||||
transaction_pool.clone(),
|
|
||||||
config.prometheus_registry(),
|
config.prometheus_registry(),
|
||||||
);
|
);
|
||||||
|
|
||||||
let select_chain = serai_consensus::TendermintSelectChain::new(backend.clone());
|
let select_chain = serai_consensus::TendermintSelectChain::new(backend.clone());
|
||||||
|
|
||||||
Ok((
|
Ok((
|
||||||
(announce, authority),
|
authority,
|
||||||
sc_service::PartialComponents {
|
sc_service::PartialComponents {
|
||||||
client,
|
client,
|
||||||
backend,
|
backend,
|
||||||
|
@ -119,7 +92,7 @@ pub fn new_partial(
|
||||||
|
|
||||||
pub async fn new_full(config: Configuration) -> Result<TaskManager, ServiceError> {
|
pub async fn new_full(config: Configuration) -> Result<TaskManager, ServiceError> {
|
||||||
let (
|
let (
|
||||||
(announce, authority),
|
authority,
|
||||||
sc_service::PartialComponents {
|
sc_service::PartialComponents {
|
||||||
client,
|
client,
|
||||||
backend,
|
backend,
|
||||||
|
@ -142,7 +115,6 @@ pub async fn new_full(config: Configuration) -> Result<TaskManager, ServiceError
|
||||||
block_announce_validator_builder: None,
|
block_announce_validator_builder: None,
|
||||||
warp_sync: None,
|
warp_sync: None,
|
||||||
})?;
|
})?;
|
||||||
*announce.0.write().unwrap() = Some(network.clone());
|
|
||||||
|
|
||||||
if config.offchain_worker.enabled {
|
if config.offchain_worker.enabled {
|
||||||
sc_service::build_offchain_workers(
|
sc_service::build_offchain_workers(
|
||||||
|
@ -169,6 +141,7 @@ pub async fn new_full(config: Configuration) -> Result<TaskManager, ServiceError
|
||||||
|
|
||||||
let is_authority = config.role.is_authority();
|
let is_authority = config.role.is_authority();
|
||||||
|
|
||||||
|
let registry = config.prometheus_registry().cloned();
|
||||||
sc_service::spawn_tasks(sc_service::SpawnTasksParams {
|
sc_service::spawn_tasks(sc_service::SpawnTasksParams {
|
||||||
network: network.clone(),
|
network: network.clone(),
|
||||||
client: client.clone(),
|
client: client.clone(),
|
||||||
|
@ -187,7 +160,18 @@ pub async fn new_full(config: Configuration) -> Result<TaskManager, ServiceError
|
||||||
task_manager.spawn_essential_handle().spawn(
|
task_manager.spawn_essential_handle().spawn(
|
||||||
"tendermint",
|
"tendermint",
|
||||||
None,
|
None,
|
||||||
authority.validate(network, None),
|
TendermintAuthority::new(authority).authority(
|
||||||
|
serai_consensus::Cidp,
|
||||||
|
sc_basic_authorship::ProposerFactory::new(
|
||||||
|
task_manager.spawn_handle(),
|
||||||
|
client,
|
||||||
|
transaction_pool,
|
||||||
|
registry.as_ref(),
|
||||||
|
telemetry.map(|telemtry| telemtry.handle()),
|
||||||
|
),
|
||||||
|
network,
|
||||||
|
None,
|
||||||
|
),
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
348
substrate/tendermint/client/src/authority.rs
Normal file
348
substrate/tendermint/client/src/authority.rs
Normal file
|
@ -0,0 +1,348 @@
|
||||||
|
use std::{
|
||||||
|
sync::{Arc, RwLock},
|
||||||
|
time::{UNIX_EPOCH, SystemTime, Duration},
|
||||||
|
};
|
||||||
|
|
||||||
|
use async_trait::async_trait;
|
||||||
|
|
||||||
|
use log::warn;
|
||||||
|
|
||||||
|
use tokio::task::yield_now;
|
||||||
|
|
||||||
|
use sp_core::{Encode, Decode, sr25519::Signature};
|
||||||
|
use sp_inherents::{InherentData, InherentDataProvider, CreateInherentDataProviders};
|
||||||
|
use sp_runtime::{
|
||||||
|
traits::{Header, Block},
|
||||||
|
Digest,
|
||||||
|
};
|
||||||
|
use sp_blockchain::HeaderBackend;
|
||||||
|
use sp_api::BlockId;
|
||||||
|
|
||||||
|
use sp_consensus::{Error, BlockOrigin, Proposer, Environment};
|
||||||
|
use sc_consensus::import_queue::IncomingBlock;
|
||||||
|
|
||||||
|
use sc_service::ImportQueue;
|
||||||
|
use sc_client_api::{BlockBackend, Finalizer};
|
||||||
|
use sc_network::NetworkBlock;
|
||||||
|
use sc_network_gossip::GossipEngine;
|
||||||
|
|
||||||
|
use substrate_prometheus_endpoint::Registry;
|
||||||
|
|
||||||
|
use tendermint_machine::{
|
||||||
|
ext::{BlockError, BlockNumber, Commit, Network},
|
||||||
|
SignedMessage, TendermintMachine,
|
||||||
|
};
|
||||||
|
|
||||||
|
use crate::{
|
||||||
|
CONSENSUS_ID, types::TendermintValidator, validators::TendermintValidators,
|
||||||
|
import_queue::ImportFuture, tendermint::TendermintImport, gossip::TendermintGossip,
|
||||||
|
};
|
||||||
|
|
||||||
|
// Data for an active validator
|
||||||
|
// This is distinct as even when we aren't an authority, we still create stubbed Authority objects
|
||||||
|
// as it's only Authority which implements tendermint_machine::ext::Network. Network has
|
||||||
|
// verify_commit provided, and even non-authorities have to verify commits
|
||||||
|
struct ActiveAuthority<T: TendermintValidator> {
|
||||||
|
// Block whose gossip is being tracked
|
||||||
|
number: Arc<RwLock<u64>>,
|
||||||
|
// Outgoing message queue, placed here as the GossipEngine itself can't be
|
||||||
|
gossip_queue: Arc<RwLock<Vec<SignedMessage<u16, T::Block, Signature>>>>,
|
||||||
|
|
||||||
|
// Block producer
|
||||||
|
env: T::Environment,
|
||||||
|
announce: T::Network,
|
||||||
|
}
|
||||||
|
|
||||||
|
pub struct TendermintAuthority<T: TendermintValidator> {
|
||||||
|
import: TendermintImport<T>,
|
||||||
|
active: Option<ActiveAuthority<T>>,
|
||||||
|
}
|
||||||
|
|
||||||
|
impl<T: TendermintValidator> TendermintAuthority<T> {
|
||||||
|
pub fn new(import: TendermintImport<T>) -> Self {
|
||||||
|
Self { import, active: None }
|
||||||
|
}
|
||||||
|
|
||||||
|
fn get_last(&self) -> (<T::Block as Block>::Hash, (BlockNumber, u64)) {
|
||||||
|
let info = self.import.client.info();
|
||||||
|
|
||||||
|
(
|
||||||
|
info.best_hash,
|
||||||
|
(
|
||||||
|
// Header::Number: TryInto<u64> 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::<TendermintValidators<T>>::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: &<T::Block as Block>::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::<TendermintValidators<T>>::topic::<T::Block>(last_number));
|
||||||
|
|
||||||
|
'outer: loop {
|
||||||
|
// Send out any queued messages
|
||||||
|
let mut queue = gossip_queue.write().unwrap().drain(..).collect::<Vec<_>>();
|
||||||
|
for msg in queue.drain(..) {
|
||||||
|
gossip.gossip_message(
|
||||||
|
TendermintGossip::<TendermintValidators<T>>::topic::<T::Block>(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::<TendermintValidators<T>>::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<T: TendermintValidator> Network for TendermintAuthority<T> {
|
||||||
|
type ValidatorId = u16;
|
||||||
|
type SignatureScheme = TendermintValidators<T>;
|
||||||
|
type Weights = TendermintValidators<T>;
|
||||||
|
type Block = T::Block;
|
||||||
|
|
||||||
|
const BLOCK_TIME: u32 = T::BLOCK_TIME_IN_SECONDS;
|
||||||
|
|
||||||
|
fn signature_scheme(&self) -> Arc<TendermintValidators<T>> {
|
||||||
|
self.import.validators.clone()
|
||||||
|
}
|
||||||
|
|
||||||
|
fn weights(&self) -> Arc<TendermintValidators<T>> {
|
||||||
|
self.import.validators.clone()
|
||||||
|
}
|
||||||
|
|
||||||
|
async fn broadcast(&mut self, msg: SignedMessage<u16, Self::Block, Signature>) {
|
||||||
|
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<TendermintValidators<T>>,
|
||||||
|
) -> 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
|
||||||
|
}
|
||||||
|
}
|
|
@ -14,10 +14,7 @@ use sc_service::ImportQueue;
|
||||||
|
|
||||||
use substrate_prometheus_endpoint::Registry;
|
use substrate_prometheus_endpoint::Registry;
|
||||||
|
|
||||||
use crate::{
|
use crate::{types::TendermintValidator, TendermintImport};
|
||||||
types::TendermintValidator,
|
|
||||||
tendermint::{TendermintImport, TendermintAuthority},
|
|
||||||
};
|
|
||||||
|
|
||||||
pub type TendermintImportQueue<Block, Transaction> = BasicQueue<Block, Transaction>;
|
pub type TendermintImportQueue<Block, Transaction> = BasicQueue<Block, Transaction>;
|
||||||
|
|
||||||
|
@ -68,25 +65,21 @@ impl<'a, B: Block, T: Send> Future for ImportFuture<'a, B, T> {
|
||||||
}
|
}
|
||||||
|
|
||||||
pub fn import_queue<T: TendermintValidator>(
|
pub fn import_queue<T: TendermintValidator>(
|
||||||
client: Arc<T::Client>,
|
|
||||||
announce: T::Announce,
|
|
||||||
providers: Arc<T::CIDP>,
|
|
||||||
env: T::Environment,
|
|
||||||
spawner: &impl sp_core::traits::SpawnEssentialNamed,
|
spawner: &impl sp_core::traits::SpawnEssentialNamed,
|
||||||
|
client: Arc<T::Client>,
|
||||||
registry: Option<&Registry>,
|
registry: Option<&Registry>,
|
||||||
) -> (TendermintAuthority<T>, TendermintImportQueue<T::Block, T::BackendTransaction>)
|
) -> (TendermintImport<T>, TendermintImportQueue<T::Block, T::BackendTransaction>)
|
||||||
where
|
where
|
||||||
Arc<T::Client>: BlockImport<T::Block, Transaction = T::BackendTransaction>,
|
Arc<T::Client>: BlockImport<T::Block, Transaction = T::BackendTransaction>,
|
||||||
<Arc<T::Client> as BlockImport<T::Block>>::Error: Into<Error>,
|
<Arc<T::Client> as BlockImport<T::Block>>::Error: Into<Error>,
|
||||||
{
|
{
|
||||||
let import = TendermintImport::<T>::new(client, announce, providers, env);
|
let import = TendermintImport::<T>::new(client);
|
||||||
let authority = TendermintAuthority(import.clone());
|
|
||||||
|
|
||||||
let boxed = Box::new(import.clone());
|
let boxed = Box::new(import.clone());
|
||||||
// Use None for the justification importer since justifications always come with blocks
|
// 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);
|
let queue = || BasicQueue::new(import.clone(), boxed.clone(), None, spawner, registry);
|
||||||
|
|
||||||
*futures::executor::block_on(import.queue.write()) = Some(queue());
|
*futures::executor::block_on(import.queue.write()) = Some(queue());
|
||||||
(authority, queue())
|
(import.clone(), queue())
|
||||||
}
|
}
|
||||||
|
|
|
@ -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_runtime::traits::Block as BlockTrait;
|
||||||
use sp_inherents::CreateInherentDataProviders;
|
use sp_inherents::CreateInherentDataProviders;
|
||||||
use sp_consensus::DisableProofRecording;
|
use sp_consensus::DisableProofRecording;
|
||||||
use sp_api::{TransactionFor, ProvideRuntimeApi};
|
use sp_api::ProvideRuntimeApi;
|
||||||
|
|
||||||
use sc_executor::{NativeVersion, NativeExecutionDispatch, NativeElseWasmExecutor};
|
use sc_executor::{NativeVersion, NativeExecutionDispatch, NativeElseWasmExecutor};
|
||||||
use sc_transaction_pool::FullPool;
|
use sc_transaction_pool::FullPool;
|
||||||
use sc_network::NetworkService;
|
use sc_network::NetworkService;
|
||||||
use sc_service::{TaskManager, TFullClient};
|
use sc_service::TFullClient;
|
||||||
|
|
||||||
use substrate_prometheus_endpoint::Registry;
|
|
||||||
|
|
||||||
use serai_runtime::{self, opaque::Block, RuntimeApi};
|
use serai_runtime::{self, opaque::Block, RuntimeApi};
|
||||||
|
|
||||||
|
@ -19,14 +17,16 @@ use types::{TendermintClientMinimal, TendermintValidator};
|
||||||
|
|
||||||
mod validators;
|
mod validators;
|
||||||
|
|
||||||
mod tendermint;
|
pub(crate) mod tendermint;
|
||||||
pub use tendermint::TendermintAuthority;
|
pub use tendermint::TendermintImport;
|
||||||
mod block_import;
|
mod block_import;
|
||||||
|
|
||||||
mod import_queue;
|
mod import_queue;
|
||||||
use import_queue::TendermintImportQueue;
|
pub use import_queue::{TendermintImportQueue, import_queue};
|
||||||
|
|
||||||
pub(crate) mod gossip;
|
pub(crate) mod gossip;
|
||||||
|
pub(crate) mod authority;
|
||||||
|
pub use authority::TendermintAuthority;
|
||||||
|
|
||||||
mod select_chain;
|
mod select_chain;
|
||||||
pub use select_chain::TendermintSelectChain;
|
pub use select_chain::TendermintSelectChain;
|
||||||
|
@ -51,10 +51,6 @@ impl NativeExecutionDispatch for ExecutorDispatch {
|
||||||
|
|
||||||
pub type FullClient = TFullClient<Block, RuntimeApi, NativeElseWasmExecutor<ExecutorDispatch>>;
|
pub type FullClient = TFullClient<Block, RuntimeApi, NativeElseWasmExecutor<ExecutorDispatch>>;
|
||||||
|
|
||||||
pub trait Announce<B: BlockTrait>: Send + Sync + Clone + 'static {
|
|
||||||
fn announce(&self, hash: B::Hash);
|
|
||||||
}
|
|
||||||
|
|
||||||
pub struct Cidp;
|
pub struct Cidp;
|
||||||
#[async_trait::async_trait]
|
#[async_trait::async_trait]
|
||||||
impl CreateInherentDataProviders<Block, ()> for Cidp {
|
impl CreateInherentDataProviders<Block, ()> for Cidp {
|
||||||
|
@ -68,15 +64,17 @@ impl CreateInherentDataProviders<Block, ()> for Cidp {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
pub struct TendermintValidatorFirm<A: Announce<Block>>(PhantomData<A>);
|
pub struct TendermintValidatorFirm;
|
||||||
impl<A: Announce<Block>> TendermintClientMinimal for TendermintValidatorFirm<A> {
|
impl TendermintClientMinimal for TendermintValidatorFirm {
|
||||||
|
const BLOCK_TIME_IN_SECONDS: u32 = { (serai_runtime::MILLISECS_PER_BLOCK / 1000) as u32 };
|
||||||
|
|
||||||
type Block = Block;
|
type Block = Block;
|
||||||
type Backend = sc_client_db::Backend<Block>;
|
type Backend = sc_client_db::Backend<Block>;
|
||||||
type Api = <FullClient as ProvideRuntimeApi<Block>>::Api;
|
type Api = <FullClient as ProvideRuntimeApi<Block>>::Api;
|
||||||
type Client = FullClient;
|
type Client = FullClient;
|
||||||
}
|
}
|
||||||
|
|
||||||
impl<A: Announce<Block>> TendermintValidator for TendermintValidatorFirm<A> {
|
impl TendermintValidator for TendermintValidatorFirm {
|
||||||
type CIDP = Cidp;
|
type CIDP = Cidp;
|
||||||
type Environment = sc_basic_authorship::ProposerFactory<
|
type Environment = sc_basic_authorship::ProposerFactory<
|
||||||
FullPool<Block, FullClient>,
|
FullPool<Block, FullClient>,
|
||||||
|
@ -86,33 +84,6 @@ impl<A: Announce<Block>> TendermintValidator for TendermintValidatorFirm<A> {
|
||||||
>;
|
>;
|
||||||
|
|
||||||
type Network = Arc<NetworkService<Block, <Block as BlockTrait>::Hash>>;
|
type Network = Arc<NetworkService<Block, <Block as BlockTrait>::Hash>>;
|
||||||
type Announce = A;
|
|
||||||
}
|
|
||||||
|
|
||||||
pub fn import_queue<A: Announce<Block>>(
|
|
||||||
task_manager: &TaskManager,
|
|
||||||
client: Arc<FullClient>,
|
|
||||||
announce: A,
|
|
||||||
pool: Arc<FullPool<Block, FullClient>>,
|
|
||||||
registry: Option<&Registry>,
|
|
||||||
) -> (
|
|
||||||
TendermintAuthority<TendermintValidatorFirm<A>>,
|
|
||||||
TendermintImportQueue<Block, TransactionFor<FullClient, Block>>,
|
|
||||||
) {
|
|
||||||
import_queue::import_queue::<TendermintValidatorFirm<A>>(
|
|
||||||
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,
|
|
||||||
)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
|
|
|
@ -1,161 +1,40 @@
|
||||||
use std::{
|
use std::{
|
||||||
marker::PhantomData,
|
marker::PhantomData,
|
||||||
sync::{Arc, RwLock},
|
sync::{Arc, RwLock},
|
||||||
time::{UNIX_EPOCH, SystemTime, Duration},
|
|
||||||
};
|
};
|
||||||
|
|
||||||
use async_trait::async_trait;
|
use tokio::sync::RwLock as AsyncRwLock;
|
||||||
|
|
||||||
use log::warn;
|
use sp_core::Decode;
|
||||||
|
|
||||||
use tokio::{sync::RwLock as AsyncRwLock, task::yield_now};
|
|
||||||
|
|
||||||
use sp_core::{Encode, Decode, sr25519::Signature};
|
|
||||||
use sp_inherents::{InherentData, InherentDataProvider, CreateInherentDataProviders};
|
|
||||||
use sp_runtime::{
|
use sp_runtime::{
|
||||||
traits::{Header, Block},
|
traits::{Header, Block},
|
||||||
Digest, Justification,
|
Justification,
|
||||||
};
|
};
|
||||||
use sp_blockchain::HeaderBackend;
|
use sp_blockchain::HeaderBackend;
|
||||||
use sp_api::BlockId;
|
|
||||||
|
|
||||||
use sp_consensus::{Error, BlockOrigin, Proposer, Environment};
|
use sp_consensus::Error;
|
||||||
use sc_consensus::{ForkChoiceStrategy, BlockImportParams, import_queue::IncomingBlock};
|
use sc_consensus::{ForkChoiceStrategy, BlockImportParams};
|
||||||
|
|
||||||
use sc_service::ImportQueue;
|
use tendermint_machine::ext::{Commit, Network};
|
||||||
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 crate::{
|
use crate::{
|
||||||
CONSENSUS_ID,
|
CONSENSUS_ID, types::TendermintValidator, validators::TendermintValidators,
|
||||||
types::TendermintValidator,
|
import_queue::TendermintImportQueue, authority::TendermintAuthority,
|
||||||
validators::TendermintValidators,
|
|
||||||
import_queue::{ImportFuture, TendermintImportQueue},
|
|
||||||
gossip::TendermintGossip,
|
|
||||||
Announce,
|
|
||||||
};
|
};
|
||||||
|
|
||||||
pub(crate) struct TendermintImport<T: TendermintValidator> {
|
pub struct TendermintImport<T: TendermintValidator> {
|
||||||
_ta: PhantomData<T>,
|
_ta: PhantomData<T>,
|
||||||
|
|
||||||
validators: Arc<TendermintValidators<T>>,
|
pub(crate) validators: Arc<TendermintValidators<T>>,
|
||||||
|
|
||||||
number: Arc<RwLock<u64>>,
|
pub(crate) providers: Arc<AsyncRwLock<Option<T::CIDP>>>,
|
||||||
gossip_queue: Arc<RwLock<Vec<SignedMessage<u16, T::Block, Signature>>>>,
|
pub(crate) importing_block: Arc<RwLock<Option<<T::Block as Block>::Hash>>>,
|
||||||
importing_block: Arc<RwLock<Option<<T::Block as Block>::Hash>>>,
|
|
||||||
|
|
||||||
pub(crate) client: Arc<T::Client>,
|
pub(crate) client: Arc<T::Client>,
|
||||||
announce: T::Announce,
|
|
||||||
providers: Arc<T::CIDP>,
|
|
||||||
|
|
||||||
env: Arc<AsyncRwLock<T::Environment>>,
|
|
||||||
pub(crate) queue:
|
pub(crate) queue:
|
||||||
Arc<AsyncRwLock<Option<TendermintImportQueue<T::Block, T::BackendTransaction>>>>,
|
Arc<AsyncRwLock<Option<TendermintImportQueue<T::Block, T::BackendTransaction>>>>,
|
||||||
}
|
}
|
||||||
|
|
||||||
pub struct TendermintAuthority<T: TendermintValidator>(pub(crate) TendermintImport<T>);
|
|
||||||
impl<T: TendermintValidator> TendermintAuthority<T> {
|
|
||||||
pub async fn validate(mut self, network: T::Network, registry: Option<&Registry>) {
|
|
||||||
let info = self.0.client.info();
|
|
||||||
|
|
||||||
// Header::Number: TryInto<u64> 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::<TendermintValidators<T>>::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::<TendermintValidators<T>>::topic::<T::Block>(last_number));
|
|
||||||
'outer: loop {
|
|
||||||
// Send out any queued messages
|
|
||||||
let mut queue = self.0.gossip_queue.write().unwrap().drain(..).collect::<Vec<_>>();
|
|
||||||
for msg in queue.drain(..) {
|
|
||||||
gossip.gossip_message(
|
|
||||||
TendermintGossip::<TendermintValidators<T>>::topic::<T::Block>(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::<TendermintValidators<T>>::topic::<
|
|
||||||
T::Block,
|
|
||||||
>(last_number));
|
|
||||||
}
|
|
||||||
yield_now().await;
|
|
||||||
break 'inner;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
impl<T: TendermintValidator> Clone for TendermintImport<T> {
|
impl<T: TendermintValidator> Clone for TendermintImport<T> {
|
||||||
fn clone(&self) -> Self {
|
fn clone(&self) -> Self {
|
||||||
TendermintImport {
|
TendermintImport {
|
||||||
|
@ -163,50 +42,31 @@ impl<T: TendermintValidator> Clone for TendermintImport<T> {
|
||||||
|
|
||||||
validators: self.validators.clone(),
|
validators: self.validators.clone(),
|
||||||
|
|
||||||
number: self.number.clone(),
|
providers: self.providers.clone(),
|
||||||
gossip_queue: self.gossip_queue.clone(),
|
|
||||||
importing_block: self.importing_block.clone(),
|
importing_block: self.importing_block.clone(),
|
||||||
|
|
||||||
client: self.client.clone(),
|
client: self.client.clone(),
|
||||||
announce: self.announce.clone(),
|
|
||||||
providers: self.providers.clone(),
|
|
||||||
|
|
||||||
env: self.env.clone(),
|
|
||||||
queue: self.queue.clone(),
|
queue: self.queue.clone(),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
impl<T: TendermintValidator> TendermintImport<T> {
|
impl<T: TendermintValidator> TendermintImport<T> {
|
||||||
pub(crate) fn new(
|
pub(crate) fn new(client: Arc<T::Client>) -> TendermintImport<T> {
|
||||||
client: Arc<T::Client>,
|
|
||||||
announce: T::Announce,
|
|
||||||
providers: Arc<T::CIDP>,
|
|
||||||
env: T::Environment,
|
|
||||||
) -> TendermintImport<T> {
|
|
||||||
TendermintImport {
|
TendermintImport {
|
||||||
_ta: PhantomData,
|
_ta: PhantomData,
|
||||||
|
|
||||||
validators: Arc::new(TendermintValidators::new(client.clone())),
|
validators: Arc::new(TendermintValidators::new(client.clone())),
|
||||||
|
|
||||||
number: Arc::new(RwLock::new(0)),
|
providers: Arc::new(AsyncRwLock::new(None)),
|
||||||
gossip_queue: Arc::new(RwLock::new(vec![])),
|
|
||||||
importing_block: Arc::new(RwLock::new(None)),
|
importing_block: Arc::new(RwLock::new(None)),
|
||||||
|
|
||||||
client,
|
client,
|
||||||
announce,
|
|
||||||
providers,
|
|
||||||
|
|
||||||
env: Arc::new(AsyncRwLock::new(env)),
|
|
||||||
queue: Arc::new(AsyncRwLock::new(None)),
|
queue: Arc::new(AsyncRwLock::new(None)),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
async fn check_inherents(
|
async fn check_inherents(&self, block: T::Block) -> Result<(), Error> {
|
||||||
&self,
|
|
||||||
block: T::Block,
|
|
||||||
providers: <T::CIDP as CreateInherentDataProviders<T::Block, ()>>::InherentDataProviders,
|
|
||||||
) -> Result<(), Error> {
|
|
||||||
// TODO
|
// TODO
|
||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
|
@ -250,7 +110,7 @@ impl<T: TendermintValidator> TendermintImport<T> {
|
||||||
|
|
||||||
let commit: Commit<TendermintValidators<T>> =
|
let commit: Commit<TendermintValidators<T>> =
|
||||||
Commit::decode(&mut justification.1.as_ref()).map_err(|_| Error::InvalidJustification)?;
|
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)?;
|
Err(Error::InvalidJustification)?;
|
||||||
}
|
}
|
||||||
Ok(())
|
Ok(())
|
||||||
|
@ -282,10 +142,10 @@ impl<T: TendermintValidator> TendermintImport<T> {
|
||||||
block: &mut BlockImportParams<T::Block, BT>,
|
block: &mut BlockImportParams<T::Block, BT>,
|
||||||
) -> Result<(), Error> {
|
) -> Result<(), Error> {
|
||||||
if block.finalized {
|
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
|
// 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, 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");
|
panic!("block was finalized despite not setting the fork choice");
|
||||||
}
|
}
|
||||||
return Ok(());
|
return Ok(());
|
||||||
|
@ -301,12 +161,7 @@ impl<T: TendermintValidator> TendermintImport<T> {
|
||||||
if !block.finalized {
|
if !block.finalized {
|
||||||
self.verify_origin(block.header.hash())?;
|
self.verify_origin(block.header.hash())?;
|
||||||
if let Some(body) = block.body.clone() {
|
if let Some(body) = block.body.clone() {
|
||||||
self
|
self.check_inherents(T::Block::new(block.header.clone(), body)).await?;
|
||||||
.check_inherents(
|
|
||||||
T::Block::new(block.header.clone(), body),
|
|
||||||
self.providers.create_inherent_data_providers(*block.header.parent_hash(), ()).await?,
|
|
||||||
)
|
|
||||||
.await?;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -322,139 +177,4 @@ impl<T: TendermintValidator> TendermintImport<T> {
|
||||||
|
|
||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
|
|
||||||
pub(crate) async fn get_proposal(&mut self, header: &<T::Block as Block>::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<T: TendermintValidator> Network for TendermintImport<T> {
|
|
||||||
type ValidatorId = u16;
|
|
||||||
type SignatureScheme = TendermintValidators<T>;
|
|
||||||
type Weights = TendermintValidators<T>;
|
|
||||||
type Block = T::Block;
|
|
||||||
|
|
||||||
const BLOCK_TIME: u32 = { (serai_runtime::MILLISECS_PER_BLOCK / 1000) as u32 };
|
|
||||||
|
|
||||||
fn signature_scheme(&self) -> Arc<TendermintValidators<T>> {
|
|
||||||
self.validators.clone()
|
|
||||||
}
|
|
||||||
|
|
||||||
fn weights(&self) -> Arc<TendermintValidators<T>> {
|
|
||||||
self.validators.clone()
|
|
||||||
}
|
|
||||||
|
|
||||||
async fn broadcast(&mut self, msg: SignedMessage<u16, Self::Block, Signature>) {
|
|
||||||
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<TendermintValidators<T>>,
|
|
||||||
) -> 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
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -8,14 +8,15 @@ use sp_consensus::Environment;
|
||||||
use sc_consensus::BlockImport;
|
use sc_consensus::BlockImport;
|
||||||
|
|
||||||
use sc_client_api::{BlockBackend, Backend, Finalizer};
|
use sc_client_api::{BlockBackend, Backend, Finalizer};
|
||||||
|
use sc_network::NetworkBlock;
|
||||||
use sc_network_gossip::Network;
|
use sc_network_gossip::Network;
|
||||||
|
|
||||||
use sp_tendermint::TendermintApi;
|
use sp_tendermint::TendermintApi;
|
||||||
|
|
||||||
use crate::Announce;
|
|
||||||
|
|
||||||
/// Trait consolidating all generics required by sc_tendermint for processing.
|
/// Trait consolidating all generics required by sc_tendermint for processing.
|
||||||
pub trait TendermintClient: Send + Sync + 'static {
|
pub trait TendermintClient: Send + Sync + 'static {
|
||||||
|
const BLOCK_TIME_IN_SECONDS: u32;
|
||||||
|
|
||||||
type Block: Block;
|
type Block: Block;
|
||||||
type Backend: Backend<Self::Block> + 'static;
|
type Backend: Backend<Self::Block> + 'static;
|
||||||
|
|
||||||
|
@ -40,6 +41,8 @@ pub trait TendermintClient: Send + Sync + 'static {
|
||||||
|
|
||||||
/// Trait implementable on firm types to automatically provide a full TendermintClient impl.
|
/// Trait implementable on firm types to automatically provide a full TendermintClient impl.
|
||||||
pub trait TendermintClientMinimal: Send + Sync + 'static {
|
pub trait TendermintClientMinimal: Send + Sync + 'static {
|
||||||
|
const BLOCK_TIME_IN_SECONDS: u32;
|
||||||
|
|
||||||
type Block: Block;
|
type Block: Block;
|
||||||
type Backend: Backend<Self::Block> + 'static;
|
type Backend: Backend<Self::Block> + 'static;
|
||||||
type Api: ApiExt<Self::Block> + TendermintApi<Self::Block>;
|
type Api: ApiExt<Self::Block> + TendermintApi<Self::Block>;
|
||||||
|
@ -58,6 +61,8 @@ where
|
||||||
<T::Client as ProvideRuntimeApi<T::Block>>::Api: TendermintApi<T::Block>,
|
<T::Client as ProvideRuntimeApi<T::Block>>::Api: TendermintApi<T::Block>,
|
||||||
TransactionFor<T::Client, T::Block>: Send + Sync + 'static,
|
TransactionFor<T::Client, T::Block>: Send + Sync + 'static,
|
||||||
{
|
{
|
||||||
|
const BLOCK_TIME_IN_SECONDS: u32 = T::BLOCK_TIME_IN_SECONDS;
|
||||||
|
|
||||||
type Block = T::Block;
|
type Block = T::Block;
|
||||||
type Backend = T::Backend;
|
type Backend = T::Backend;
|
||||||
|
|
||||||
|
@ -72,6 +77,10 @@ pub trait TendermintValidator: TendermintClient {
|
||||||
type CIDP: CreateInherentDataProviders<Self::Block, ()> + 'static;
|
type CIDP: CreateInherentDataProviders<Self::Block, ()> + 'static;
|
||||||
type Environment: Send + Sync + Environment<Self::Block> + 'static;
|
type Environment: Send + Sync + Environment<Self::Block> + 'static;
|
||||||
|
|
||||||
type Network: Clone + Send + Sync + Network<Self::Block> + 'static;
|
type Network: Clone
|
||||||
type Announce: Announce<Self::Block>;
|
+ Send
|
||||||
|
+ Sync
|
||||||
|
+ Network<Self::Block>
|
||||||
|
+ NetworkBlock<<Self::Block as Block>::Hash, <<Self::Block as Block>::Header as Header>::Number>
|
||||||
|
+ 'static;
|
||||||
}
|
}
|
||||||
|
|
|
@ -79,7 +79,7 @@ impl<T: TendermintClient> Deref for Refresh<T> {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
pub(crate) struct TendermintValidators<T: TendermintClient>(Refresh<T>);
|
pub struct TendermintValidators<T: TendermintClient>(Refresh<T>);
|
||||||
|
|
||||||
impl<T: TendermintClient> TendermintValidators<T> {
|
impl<T: TendermintClient> TendermintValidators<T> {
|
||||||
pub(crate) fn new(client: Arc<T::Client>) -> TendermintValidators<T> {
|
pub(crate) fn new(client: Arc<T::Client>) -> TendermintValidators<T> {
|
||||||
|
|
Loading…
Reference in a new issue