mirror of
https://github.com/serai-dex/serai.git
synced 2025-12-08 20:29:23 +00:00
Redo Tendermint folder structure
This commit is contained in:
48
substrate/tendermint/client/Cargo.toml
Normal file
48
substrate/tendermint/client/Cargo.toml
Normal file
@@ -0,0 +1,48 @@
|
||||
[package]
|
||||
name = "serai-consensus"
|
||||
version = "0.1.0"
|
||||
description = "Serai consensus module"
|
||||
license = "AGPL-3.0-only"
|
||||
repository = "https://github.com/serai-dex/serai/tree/develop/substrate/tendermint/client"
|
||||
authors = ["Luke Parker <lukeparker5132@gmail.com>"]
|
||||
edition = "2021"
|
||||
publish = false
|
||||
|
||||
[package.metadata.docs.rs]
|
||||
all-features = true
|
||||
rustdoc-args = ["--cfg", "docsrs"]
|
||||
|
||||
[dependencies]
|
||||
async-trait = "0.1"
|
||||
|
||||
log = "0.4"
|
||||
|
||||
futures = "0.3"
|
||||
tokio = { version = "1", features = ["sync", "rt"] }
|
||||
|
||||
sp-core = { git = "https://github.com/serai-dex/substrate" }
|
||||
sp-application-crypto = { git = "https://github.com/serai-dex/substrate" }
|
||||
sp-inherents = { git = "https://github.com/serai-dex/substrate" }
|
||||
sp-timestamp = { git = "https://github.com/serai-dex/substrate" }
|
||||
sp-staking = { git = "https://github.com/serai-dex/substrate" }
|
||||
sp-blockchain = { git = "https://github.com/serai-dex/substrate" }
|
||||
sp-runtime = { git = "https://github.com/serai-dex/substrate" }
|
||||
sp-api = { git = "https://github.com/serai-dex/substrate" }
|
||||
sp-consensus = { 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-executor = { git = "https://github.com/serai-dex/substrate" }
|
||||
sc-network = { git = "https://github.com/serai-dex/substrate" }
|
||||
sc-service = { git = "https://github.com/serai-dex/substrate" }
|
||||
sc-client-api = { git = "https://github.com/serai-dex/substrate" }
|
||||
sc-consensus = { git = "https://github.com/serai-dex/substrate" }
|
||||
|
||||
frame-support = { git = "https://github.com/serai-dex/substrate" }
|
||||
pallet-session = { git = "https://github.com/serai-dex/substrate" }
|
||||
|
||||
substrate-prometheus-endpoint = { git = "https://github.com/serai-dex/substrate" }
|
||||
|
||||
tendermint-machine = { path = "../tendermint", features = ["substrate"] }
|
||||
|
||||
serai-runtime = { path = "../runtime" }
|
||||
15
substrate/tendermint/client/LICENSE
Normal file
15
substrate/tendermint/client/LICENSE
Normal file
@@ -0,0 +1,15 @@
|
||||
AGPL-3.0-only license
|
||||
|
||||
Copyright (c) 2022 Luke Parker
|
||||
|
||||
This program is free software: you can redistribute it and/or modify
|
||||
it under the terms of the GNU Affero General Public License Version 3 as
|
||||
published by the Free Software Foundation.
|
||||
|
||||
This program is distributed in the hope that it will be useful,
|
||||
but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
GNU Affero General Public License for more details.
|
||||
|
||||
You should have received a copy of the GNU Affero General Public License
|
||||
along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
68
substrate/tendermint/client/src/block_import.rs
Normal file
68
substrate/tendermint/client/src/block_import.rs
Normal file
@@ -0,0 +1,68 @@
|
||||
use std::{sync::Arc, collections::HashMap};
|
||||
|
||||
use async_trait::async_trait;
|
||||
|
||||
use sp_core::sr25519::Public;
|
||||
use sp_inherents::CreateInherentDataProviders;
|
||||
use sp_runtime::traits::Block;
|
||||
use sp_api::TransactionFor;
|
||||
|
||||
use sp_consensus::{Error, CacheKeyId, Environment};
|
||||
use sc_consensus::{BlockCheckParams, BlockImportParams, ImportResult, BlockImport};
|
||||
|
||||
use sc_client_api::Backend;
|
||||
|
||||
use frame_support::traits::ValidatorSet;
|
||||
|
||||
use crate::{
|
||||
tendermint::{TendermintClient, TendermintImport},
|
||||
Announce,
|
||||
};
|
||||
|
||||
#[async_trait]
|
||||
impl<
|
||||
B: Block,
|
||||
Be: Backend<B> + 'static,
|
||||
C: TendermintClient<B, Be>,
|
||||
CIDP: CreateInherentDataProviders<B, ()> + 'static,
|
||||
E: Send + Sync + Environment<B> + 'static,
|
||||
A: Announce<B>,
|
||||
> BlockImport<B> for TendermintImport<B, Be, C, CIDP, E, A>
|
||||
where
|
||||
TransactionFor<C, B>: Send + Sync + 'static,
|
||||
Arc<C>: BlockImport<B, Transaction = TransactionFor<C, B>>,
|
||||
<Arc<C> as BlockImport<B>>::Error: Into<Error>,
|
||||
C::Api: ValidatorSet<Public>,
|
||||
{
|
||||
type Error = Error;
|
||||
type Transaction = TransactionFor<C, B>;
|
||||
|
||||
// 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<B>,
|
||||
) -> Result<ImportResult, Self::Error> {
|
||||
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.client.check_block(block).await.map_err(Into::into)
|
||||
}
|
||||
|
||||
async fn import_block(
|
||||
&mut self,
|
||||
mut block: BlockImportParams<B, TransactionFor<C, B>>,
|
||||
new_cache: HashMap<CacheKeyId, Vec<u8>>,
|
||||
) -> Result<ImportResult, Self::Error> {
|
||||
self.check(&mut block).await?;
|
||||
self.client.import_block(block, new_cache).await.map_err(Into::into)
|
||||
|
||||
// TODO: If we're a validator who just successfully synced a block, recreate the tendermint
|
||||
// machine with the new height
|
||||
}
|
||||
}
|
||||
149
substrate/tendermint/client/src/import_queue.rs
Normal file
149
substrate/tendermint/client/src/import_queue.rs
Normal file
@@ -0,0 +1,149 @@
|
||||
use std::{
|
||||
pin::Pin,
|
||||
sync::{Arc, RwLock},
|
||||
task::{Poll, Context},
|
||||
future::Future,
|
||||
time::{UNIX_EPOCH, SystemTime},
|
||||
};
|
||||
|
||||
use sp_core::{Decode, sr25519::Public};
|
||||
use sp_inherents::CreateInherentDataProviders;
|
||||
use sp_runtime::traits::{Header, Block};
|
||||
use sp_api::{BlockId, TransactionFor};
|
||||
|
||||
use sp_consensus::{Error, Environment};
|
||||
use sc_consensus::{BlockImportStatus, BlockImportError, BlockImport, Link, BasicQueue};
|
||||
|
||||
use sc_service::ImportQueue;
|
||||
use sc_client_api::Backend;
|
||||
|
||||
use substrate_prometheus_endpoint::Registry;
|
||||
|
||||
use frame_support::traits::ValidatorSet;
|
||||
|
||||
use tendermint_machine::{
|
||||
ext::{BlockNumber, Commit},
|
||||
TendermintMachine,
|
||||
};
|
||||
|
||||
use crate::{
|
||||
CONSENSUS_ID,
|
||||
validators::TendermintValidators,
|
||||
tendermint::{TendermintClient, TendermintImport},
|
||||
Announce,
|
||||
};
|
||||
|
||||
pub type TendermintImportQueue<Block, Transaction> = BasicQueue<Block, Transaction>;
|
||||
|
||||
// Custom helpers for ImportQueue in order to obtain the result of a block's importing
|
||||
struct ValidateLink<B: Block>(Option<(B::Hash, bool)>);
|
||||
impl<B: Block> Link<B> for ValidateLink<B> {
|
||||
fn blocks_processed(
|
||||
&mut self,
|
||||
imported: usize,
|
||||
count: usize,
|
||||
results: Vec<(
|
||||
Result<BlockImportStatus<<B::Header as Header>::Number>, BlockImportError>,
|
||||
B::Hash,
|
||||
)>,
|
||||
) {
|
||||
assert_eq!(imported, 1);
|
||||
assert_eq!(count, 1);
|
||||
self.0 = Some((results[0].1, results[0].0.is_ok()));
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) struct ImportFuture<'a, B: Block, T: Send>(
|
||||
B::Hash,
|
||||
RwLock<&'a mut TendermintImportQueue<B, T>>,
|
||||
);
|
||||
impl<'a, B: Block, T: Send> ImportFuture<'a, B, T> {
|
||||
pub(crate) fn new(
|
||||
hash: B::Hash,
|
||||
queue: &'a mut TendermintImportQueue<B, T>,
|
||||
) -> ImportFuture<B, T> {
|
||||
ImportFuture(hash, RwLock::new(queue))
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a, B: Block, T: Send> Future for ImportFuture<'a, B, T> {
|
||||
type Output = bool;
|
||||
|
||||
fn poll(self: Pin<&mut Self>, ctx: &mut Context<'_>) -> Poll<Self::Output> {
|
||||
let mut link = ValidateLink(None);
|
||||
self.1.write().unwrap().poll_actions(ctx, &mut link);
|
||||
if let Some(res) = link.0 {
|
||||
assert_eq!(res.0, self.0);
|
||||
Poll::Ready(res.1)
|
||||
} else {
|
||||
Poll::Pending
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub fn import_queue<
|
||||
B: Block,
|
||||
Be: Backend<B> + 'static,
|
||||
C: TendermintClient<B, Be>,
|
||||
CIDP: CreateInherentDataProviders<B, ()> + 'static,
|
||||
E: Send + Sync + Environment<B> + 'static,
|
||||
A: Announce<B>,
|
||||
>(
|
||||
client: Arc<C>,
|
||||
announce: A,
|
||||
providers: Arc<CIDP>,
|
||||
env: E,
|
||||
spawner: &impl sp_core::traits::SpawnEssentialNamed,
|
||||
registry: Option<&Registry>,
|
||||
) -> (impl Future<Output = ()>, TendermintImportQueue<B, TransactionFor<C, B>>)
|
||||
where
|
||||
TransactionFor<C, B>: Send + Sync + 'static,
|
||||
Arc<C>: BlockImport<B, Transaction = TransactionFor<C, B>>,
|
||||
<Arc<C> as BlockImport<B>>::Error: Into<Error>,
|
||||
C::Api: ValidatorSet<Public>,
|
||||
{
|
||||
let import = TendermintImport::new(client, announce, providers, env);
|
||||
|
||||
let authority = {
|
||||
let machine_clone = import.machine.clone();
|
||||
let mut import_clone = import.clone();
|
||||
let best = import.client.info().best_number;
|
||||
async move {
|
||||
*machine_clone.write().unwrap() = Some(TendermintMachine::new(
|
||||
import_clone.clone(),
|
||||
// TODO
|
||||
0,
|
||||
(
|
||||
// Header::Number: TryInto<u64> doesn't implement Debug and can't be unwrapped
|
||||
match best.try_into() {
|
||||
Ok(best) => BlockNumber(best),
|
||||
Err(_) => panic!("BlockNumber exceeded u64"),
|
||||
},
|
||||
Commit::<TendermintValidators<B, C>>::decode(
|
||||
&mut import_clone
|
||||
.client
|
||||
.justifications(&BlockId::Number(best))
|
||||
.unwrap()
|
||||
.map(|justifications| justifications.get(CONSENSUS_ID).cloned().unwrap())
|
||||
.unwrap_or_default()
|
||||
.as_ref(),
|
||||
)
|
||||
.map(|commit| commit.end_time)
|
||||
// TODO: Genesis start time
|
||||
.unwrap_or_else(|_| SystemTime::now().duration_since(UNIX_EPOCH).unwrap().as_secs()),
|
||||
),
|
||||
import_clone
|
||||
.get_proposal(&import_clone.client.header(BlockId::Number(0u8.into())).unwrap().unwrap())
|
||||
.await,
|
||||
));
|
||||
}
|
||||
};
|
||||
|
||||
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
|
||||
let queue = || BasicQueue::new(import.clone(), boxed.clone(), None, spawner, registry);
|
||||
|
||||
*futures::executor::block_on(import.queue.write()) = Some(queue());
|
||||
(authority, queue())
|
||||
}
|
||||
118
substrate/tendermint/client/src/lib.rs
Normal file
118
substrate/tendermint/client/src/lib.rs
Normal file
@@ -0,0 +1,118 @@
|
||||
use std::{sync::Arc, future::Future};
|
||||
|
||||
use sp_runtime::traits::Block as BlockTrait;
|
||||
use sp_api::TransactionFor;
|
||||
|
||||
use sc_executor::{NativeVersion, NativeExecutionDispatch, NativeElseWasmExecutor};
|
||||
use sc_transaction_pool::FullPool;
|
||||
use sc_service::{TaskManager, TFullClient};
|
||||
|
||||
use substrate_prometheus_endpoint::Registry;
|
||||
|
||||
use serai_runtime::{self, opaque::Block, RuntimeApi};
|
||||
|
||||
mod validators;
|
||||
|
||||
mod tendermint;
|
||||
mod block_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")]
|
||||
type ExtendHostFunctions = frame_benchmarking::benchmarking::HostFunctions;
|
||||
#[cfg(not(feature = "runtime-benchmarks"))]
|
||||
type ExtendHostFunctions = ();
|
||||
|
||||
fn dispatch(method: &str, data: &[u8]) -> Option<Vec<u8>> {
|
||||
serai_runtime::api::dispatch(method, data)
|
||||
}
|
||||
|
||||
fn native_version() -> NativeVersion {
|
||||
serai_runtime::native_version()
|
||||
}
|
||||
}
|
||||
|
||||
pub type FullClient = TFullClient<Block, RuntimeApi, NativeElseWasmExecutor<ExecutorDispatch>>;
|
||||
|
||||
pub trait Announce<B: BlockTrait>: Send + Sync + Clone + 'static {
|
||||
fn announce(&self, hash: B::Hash);
|
||||
}
|
||||
|
||||
pub fn import_queue<A: Announce<Block>>(
|
||||
task_manager: &TaskManager,
|
||||
client: Arc<FullClient>,
|
||||
announce: A,
|
||||
pool: Arc<FullPool<Block, FullClient>>,
|
||||
registry: Option<&Registry>,
|
||||
) -> (impl Future<Output = ()>, TendermintImportQueue<Block, TransactionFor<FullClient, Block>>) {
|
||||
import_queue::import_queue(
|
||||
client.clone(),
|
||||
announce,
|
||||
Arc::new(|_, _| async { Ok(sp_timestamp::InherentDataProvider::from_system_time()) }),
|
||||
sc_basic_authorship::ProposerFactory::new(
|
||||
task_manager.spawn_handle(),
|
||||
client,
|
||||
pool,
|
||||
registry,
|
||||
None,
|
||||
),
|
||||
&task_manager.spawn_essential_handle(),
|
||||
registry,
|
||||
)
|
||||
}
|
||||
|
||||
/*
|
||||
// If we're an authority, produce blocks
|
||||
pub fn authority<S: sp_consensus::SelectChain<Block> + 'static>(
|
||||
task_manager: &TaskManager,
|
||||
client: Arc<FullClient>,
|
||||
network: Arc<sc_network::NetworkService<Block, <Block as sp_runtime::traits::Block>::Hash>>,
|
||||
pool: Arc<sc_transaction_pool::FullPool<Block, FullClient>>,
|
||||
select_chain: S,
|
||||
registry: Option<&Registry>,
|
||||
) {
|
||||
let proposer = sc_basic_authorship::ProposerFactory::new(
|
||||
task_manager.spawn_handle(),
|
||||
client.clone(),
|
||||
pool,
|
||||
registry,
|
||||
None,
|
||||
);
|
||||
|
||||
let pow_block_import = Box::new(sc_pow::PowBlockImport::new(
|
||||
client.clone(),
|
||||
client.clone(),
|
||||
algorithm::AcceptAny,
|
||||
0, // Block to start checking inherents at
|
||||
select_chain.clone(),
|
||||
move |_, _| async { Ok(sp_timestamp::InherentDataProvider::from_system_time()) },
|
||||
));
|
||||
|
||||
let (worker, worker_task) = sc_pow::start_mining_worker(
|
||||
pow_block_import,
|
||||
client,
|
||||
select_chain,
|
||||
algorithm::AcceptAny,
|
||||
proposer,
|
||||
network.clone(),
|
||||
network,
|
||||
None,
|
||||
move |_, _| async { Ok(sp_timestamp::InherentDataProvider::from_system_time()) },
|
||||
Duration::from_secs(6),
|
||||
Duration::from_secs(2),
|
||||
);
|
||||
|
||||
task_manager.spawn_essential_handle().spawn_blocking("pow", None, worker_task);
|
||||
|
||||
task_manager.spawn_essential_handle().spawn("producer", None, produce(worker));
|
||||
}
|
||||
*/
|
||||
59
substrate/tendermint/client/src/select_chain.rs
Normal file
59
substrate/tendermint/client/src/select_chain.rs
Normal file
@@ -0,0 +1,59 @@
|
||||
// SelectChain, while provided by Substrate and part of PartialComponents, isn't used by Substrate
|
||||
// It's common between various block-production/finality crates, yet Substrate as a system doesn't
|
||||
// rely on it, which is good, because its definition is explicitly incompatible with Tendermint
|
||||
//
|
||||
// leaves is supposed to return all leaves of the blockchain. While Tendermint maintains that view,
|
||||
// an honest node will only build on the most recently finalized block, so it is a 'leaf' despite
|
||||
// having descendants
|
||||
//
|
||||
// best_chain will always be this finalized block, yet Substrate explicitly defines it as one of
|
||||
// the above leaves, which this finalized block is explicitly not included in. Accordingly, we
|
||||
// can never provide a compatible decision
|
||||
//
|
||||
// Since PartialComponents expects it, an implementation which does its best is provided. It panics
|
||||
// if leaves is called, yet returns the finalized chain tip for best_chain, as that's intended to
|
||||
// be the header to build upon
|
||||
|
||||
use std::{marker::PhantomData, sync::Arc};
|
||||
|
||||
use async_trait::async_trait;
|
||||
|
||||
use sp_api::BlockId;
|
||||
use sp_runtime::traits::Block;
|
||||
use sp_blockchain::{HeaderBackend, Backend as BlockchainBackend};
|
||||
use sc_client_api::Backend;
|
||||
use sp_consensus::{Error, SelectChain};
|
||||
|
||||
pub struct TendermintSelectChain<B: Block, Be: Backend<B>>(Arc<Be>, PhantomData<B>);
|
||||
|
||||
impl<B: Block, Be: Backend<B>> Clone for TendermintSelectChain<B, Be> {
|
||||
fn clone(&self) -> Self {
|
||||
TendermintSelectChain(self.0.clone(), PhantomData)
|
||||
}
|
||||
}
|
||||
|
||||
impl<B: Block, Be: Backend<B>> TendermintSelectChain<B, Be> {
|
||||
pub fn new(backend: Arc<Be>) -> TendermintSelectChain<B, Be> {
|
||||
TendermintSelectChain(backend, PhantomData)
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
impl<B: Block, Be: Backend<B>> SelectChain<B> for TendermintSelectChain<B, Be> {
|
||||
async fn leaves(&self) -> Result<Vec<B::Hash>, Error> {
|
||||
panic!("Substrate definition of leaves is incompatible with Tendermint")
|
||||
}
|
||||
|
||||
async fn best_chain(&self) -> Result<B::Header, Error> {
|
||||
Ok(
|
||||
self
|
||||
.0
|
||||
.blockchain()
|
||||
// There should always be a finalized block
|
||||
.header(BlockId::Hash(self.0.blockchain().last_finalized().unwrap()))
|
||||
// There should not be an error in retrieving it and since it's finalized, it should exist
|
||||
.unwrap()
|
||||
.unwrap(),
|
||||
)
|
||||
}
|
||||
}
|
||||
427
substrate/tendermint/client/src/tendermint.rs
Normal file
427
substrate/tendermint/client/src/tendermint.rs
Normal file
@@ -0,0 +1,427 @@
|
||||
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,
|
||||
sr25519::{Public, 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::{BlockBackend, Backend, Finalizer};
|
||||
|
||||
use frame_support::traits::ValidatorSet;
|
||||
|
||||
use tendermint_machine::{
|
||||
ext::{BlockError, Commit, Network},
|
||||
SignedMessage, TendermintHandle,
|
||||
};
|
||||
|
||||
use crate::{
|
||||
CONSENSUS_ID,
|
||||
validators::TendermintValidators,
|
||||
import_queue::{ImportFuture, TendermintImportQueue},
|
||||
Announce,
|
||||
};
|
||||
|
||||
pub trait TendermintClient<B: Block, Be: Backend<B> + 'static>:
|
||||
Send
|
||||
+ Sync
|
||||
+ HeaderBackend<B>
|
||||
+ BlockBackend<B>
|
||||
+ BlockImport<B, Transaction = TransactionFor<Self, B>>
|
||||
+ Finalizer<B, Be>
|
||||
+ ProvideRuntimeApi<B>
|
||||
+ 'static
|
||||
where
|
||||
TransactionFor<Self, B>: Send + Sync + 'static,
|
||||
Self::Api: ValidatorSet<Public>,
|
||||
{
|
||||
}
|
||||
impl<
|
||||
B: Send + Sync + Block + 'static,
|
||||
Be: Send + Sync + Backend<B> + 'static,
|
||||
C: Send
|
||||
+ Sync
|
||||
+ HeaderBackend<B>
|
||||
+ BlockBackend<B>
|
||||
+ BlockImport<B, Transaction = TransactionFor<C, B>>
|
||||
+ Finalizer<B, Be>
|
||||
+ ProvideRuntimeApi<B>
|
||||
+ 'static,
|
||||
> TendermintClient<B, Be> for C
|
||||
where
|
||||
TransactionFor<C, B>: Send + Sync + 'static,
|
||||
C::Api: ValidatorSet<Public>,
|
||||
{
|
||||
}
|
||||
|
||||
pub(crate) struct TendermintImport<
|
||||
B: Block,
|
||||
Be: Backend<B> + 'static,
|
||||
C: TendermintClient<B, Be>,
|
||||
CIDP: CreateInherentDataProviders<B, ()> + 'static,
|
||||
E: Send + Sync + Environment<B> + 'static,
|
||||
A: Announce<B>,
|
||||
> where
|
||||
TransactionFor<C, B>: Send + Sync + 'static,
|
||||
C::Api: ValidatorSet<Public>,
|
||||
{
|
||||
_block: PhantomData<B>,
|
||||
_backend: PhantomData<Be>,
|
||||
|
||||
validators: Arc<TendermintValidators<B, C>>,
|
||||
|
||||
importing_block: Arc<RwLock<Option<B::Hash>>>,
|
||||
pub(crate) machine: Arc<RwLock<Option<TendermintHandle<Self>>>>,
|
||||
|
||||
pub(crate) client: Arc<C>,
|
||||
announce: A,
|
||||
providers: Arc<CIDP>,
|
||||
|
||||
env: Arc<AsyncRwLock<E>>,
|
||||
pub(crate) queue: Arc<AsyncRwLock<Option<TendermintImportQueue<B, TransactionFor<C, B>>>>>,
|
||||
}
|
||||
|
||||
impl<
|
||||
B: Block,
|
||||
Be: Backend<B> + 'static,
|
||||
C: TendermintClient<B, Be>,
|
||||
CIDP: CreateInherentDataProviders<B, ()> + 'static,
|
||||
E: Send + Sync + Environment<B> + 'static,
|
||||
A: Announce<B>,
|
||||
> Clone for TendermintImport<B, Be, C, CIDP, E, A>
|
||||
where
|
||||
TransactionFor<C, B>: Send + Sync + 'static,
|
||||
C::Api: ValidatorSet<Public>,
|
||||
{
|
||||
fn clone(&self) -> Self {
|
||||
TendermintImport {
|
||||
_block: PhantomData,
|
||||
_backend: PhantomData,
|
||||
|
||||
validators: self.validators.clone(),
|
||||
|
||||
importing_block: self.importing_block.clone(),
|
||||
machine: self.machine.clone(),
|
||||
|
||||
client: self.client.clone(),
|
||||
announce: self.announce.clone(),
|
||||
providers: self.providers.clone(),
|
||||
|
||||
env: self.env.clone(),
|
||||
queue: self.queue.clone(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<
|
||||
B: Block,
|
||||
Be: Backend<B> + 'static,
|
||||
C: TendermintClient<B, Be>,
|
||||
CIDP: CreateInherentDataProviders<B, ()> + 'static,
|
||||
E: Send + Sync + Environment<B> + 'static,
|
||||
A: Announce<B>,
|
||||
> TendermintImport<B, Be, C, CIDP, E, A>
|
||||
where
|
||||
TransactionFor<C, B>: Send + Sync + 'static,
|
||||
C::Api: ValidatorSet<Public>,
|
||||
{
|
||||
pub(crate) fn new(
|
||||
client: Arc<C>,
|
||||
announce: A,
|
||||
providers: Arc<CIDP>,
|
||||
env: E,
|
||||
) -> TendermintImport<B, Be, C, CIDP, E, A> {
|
||||
TendermintImport {
|
||||
_block: PhantomData,
|
||||
_backend: PhantomData,
|
||||
|
||||
validators: TendermintValidators::new(client),
|
||||
|
||||
importing_block: Arc::new(RwLock::new(None)),
|
||||
machine: 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: 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: <B::Header as Header>::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
|
||||
pub(crate) fn verify_justification(
|
||||
&self,
|
||||
hash: B::Hash,
|
||||
justification: &Justification,
|
||||
) -> Result<(), Error> {
|
||||
if justification.0 != CONSENSUS_ID {
|
||||
Err(Error::InvalidJustification)?;
|
||||
}
|
||||
|
||||
let commit: Commit<TendermintValidators<B, C>> =
|
||||
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<T>(&self, block: &mut BlockImportParams<B, T>) -> 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;
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub(crate) async fn check<T>(&self, block: &mut BlockImportParams<B, T>) -> 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(())
|
||||
}
|
||||
|
||||
pub(crate) async fn get_proposal(&mut self, header: &B::Header) -> B {
|
||||
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<
|
||||
B: Block,
|
||||
Be: Backend<B> + 'static,
|
||||
C: TendermintClient<B, Be>,
|
||||
CIDP: CreateInherentDataProviders<B, ()> + 'static,
|
||||
E: Send + Sync + Environment<B> + 'static,
|
||||
A: Announce<B>,
|
||||
> Network for TendermintImport<B, Be, C, CIDP, E, A>
|
||||
where
|
||||
TransactionFor<C, B>: Send + Sync + 'static,
|
||||
C::Api: ValidatorSet<Public>,
|
||||
{
|
||||
type ValidatorId = u16;
|
||||
type SignatureScheme = TendermintValidators<B, C>;
|
||||
type Weights = TendermintValidators<B, C>;
|
||||
type Block = B;
|
||||
|
||||
const BLOCK_TIME: u32 = { (serai_runtime::MILLISECS_PER_BLOCK / 1000) as u32 };
|
||||
|
||||
fn signature_scheme(&self) -> Arc<TendermintValidators<B, C>> {
|
||||
self.validators.clone()
|
||||
}
|
||||
|
||||
fn weights(&self) -> Arc<TendermintValidators<B, C>> {
|
||||
self.validators.clone()
|
||||
}
|
||||
|
||||
async fn broadcast(&mut self, msg: SignedMessage<u16, Self::Block, Signature>) {
|
||||
// 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();
|
||||
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: B, commit: Commit<TendermintValidators<B, C>>) -> B {
|
||||
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.announce.announce(hash);
|
||||
|
||||
self.get_proposal(block.header()).await
|
||||
}
|
||||
}
|
||||
147
substrate/tendermint/client/src/validators.rs
Normal file
147
substrate/tendermint/client/src/validators.rs
Normal file
@@ -0,0 +1,147 @@
|
||||
use core::{marker::PhantomData, ops::Deref};
|
||||
use std::sync::{Arc, RwLock};
|
||||
|
||||
use sp_application_crypto::{
|
||||
RuntimePublic as PublicTrait, Pair as PairTrait,
|
||||
sr25519::{Public, Pair, Signature},
|
||||
};
|
||||
|
||||
use sp_runtime::traits::Block;
|
||||
use sp_staking::SessionIndex;
|
||||
use sp_api::ProvideRuntimeApi;
|
||||
|
||||
use frame_support::traits::ValidatorSet;
|
||||
|
||||
use tendermint_machine::ext::{BlockNumber, Round, Weights, SignatureScheme};
|
||||
|
||||
struct TendermintValidatorsStruct {
|
||||
session: SessionIndex,
|
||||
|
||||
total_weight: u64,
|
||||
weights: Vec<u64>,
|
||||
|
||||
keys: Pair, // TODO: sp_keystore
|
||||
lookup: Vec<Public>, // TODO: sessions
|
||||
}
|
||||
|
||||
impl TendermintValidatorsStruct {
|
||||
fn from_module<B: Block, C: Send + Sync + ProvideRuntimeApi<B>>(
|
||||
client: C,
|
||||
) -> TendermintValidatorsStruct
|
||||
where
|
||||
C::Api: ValidatorSet<Public>,
|
||||
{
|
||||
let validators = client.runtime_api().validators();
|
||||
assert_eq!(validators.len(), 1);
|
||||
let keys = Pair::from_string("//Alice", None).unwrap();
|
||||
TendermintValidatorsStruct {
|
||||
session: client.runtime_api().session_index(),
|
||||
|
||||
// TODO
|
||||
total_weight: validators.len().try_into().unwrap(),
|
||||
weights: vec![1; validators.len()],
|
||||
|
||||
lookup: vec![keys.public()],
|
||||
keys,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Wrap every access of the validators struct in something which forces calling refresh
|
||||
struct Refresh<B: Block, C: Send + Sync + ProvideRuntimeApi<B>> {
|
||||
_block: PhantomData<B>,
|
||||
client: C,
|
||||
_refresh: Arc<RwLock<TendermintValidatorsStruct>>,
|
||||
}
|
||||
impl<B: Block, C: Send + Sync + ProvideRuntimeApi<B>> Refresh<B, C>
|
||||
where
|
||||
C::Api: ValidatorSet<Public>,
|
||||
{
|
||||
// If the session has changed, re-create the struct with the data on it
|
||||
fn refresh(&self) {
|
||||
let session = self._refresh.read().unwrap().session;
|
||||
if session != self.client.runtime_api().session_index() {
|
||||
*self._refresh.write().unwrap() = TendermintValidatorsStruct::from_module(self.client);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<B: Block, C: Send + Sync + ProvideRuntimeApi<B>> Deref for Refresh<B, C>
|
||||
where
|
||||
C::Api: ValidatorSet<Public>,
|
||||
{
|
||||
type Target = RwLock<TendermintValidatorsStruct>;
|
||||
fn deref(&self) -> &RwLock<TendermintValidatorsStruct> {
|
||||
self.refresh();
|
||||
&self._refresh
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) struct TendermintValidators<B: Block, C: Send + Sync + ProvideRuntimeApi<B>>(
|
||||
Refresh<B, C>,
|
||||
);
|
||||
impl<B: Block, C: Send + Sync + ProvideRuntimeApi<B>> TendermintValidators<B, C>
|
||||
where
|
||||
C::Api: ValidatorSet<Public>,
|
||||
{
|
||||
pub(crate) fn new(client: C) -> TendermintValidators<B, C> {
|
||||
TendermintValidators(Refresh {
|
||||
_block: PhantomData,
|
||||
client,
|
||||
_refresh: Arc::new(RwLock::new(TendermintValidatorsStruct::from_module())),
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
impl<B: Block, C: Send + Sync + ProvideRuntimeApi<B>> SignatureScheme for TendermintValidators<B, C>
|
||||
where
|
||||
C::Api: ValidatorSet<Public>,
|
||||
{
|
||||
type ValidatorId = u16;
|
||||
type Signature = Signature;
|
||||
type AggregateSignature = Vec<Signature>;
|
||||
|
||||
fn sign(&self, msg: &[u8]) -> Signature {
|
||||
self.0.read().unwrap().keys.sign(msg)
|
||||
}
|
||||
|
||||
fn verify(&self, validator: u16, msg: &[u8], sig: &Signature) -> bool {
|
||||
self.0.read().unwrap().lookup[usize::try_from(validator).unwrap()].verify(&msg, sig)
|
||||
}
|
||||
|
||||
fn aggregate(sigs: &[Signature]) -> Vec<Signature> {
|
||||
sigs.to_vec()
|
||||
}
|
||||
|
||||
fn verify_aggregate(&self, validators: &[u16], msg: &[u8], sigs: &Vec<Signature>) -> bool {
|
||||
if validators.len() != sigs.len() {
|
||||
return false;
|
||||
}
|
||||
for (v, sig) in validators.iter().zip(sigs.iter()) {
|
||||
if !self.verify(*v, msg, sig) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
true
|
||||
}
|
||||
}
|
||||
|
||||
impl<B: Block, C: Send + Sync + ProvideRuntimeApi<B>> Weights for TendermintValidators<B, C>
|
||||
where
|
||||
C::Api: ValidatorSet<Public>,
|
||||
{
|
||||
type ValidatorId = u16;
|
||||
|
||||
fn total_weight(&self) -> u64 {
|
||||
self.0.read().unwrap().total_weight
|
||||
}
|
||||
|
||||
fn weight(&self, id: u16) -> u64 {
|
||||
self.0.read().unwrap().weights[usize::try_from(id).unwrap()]
|
||||
}
|
||||
|
||||
// TODO
|
||||
fn proposer(&self, number: BlockNumber, round: Round) -> u16 {
|
||||
u16::try_from(number.0 + u64::from(round.0)).unwrap()
|
||||
}
|
||||
}
|
||||
44
substrate/tendermint/client/src/verifier.rs
Normal file
44
substrate/tendermint/client/src/verifier.rs
Normal file
@@ -0,0 +1,44 @@
|
||||
use std::sync::Arc;
|
||||
|
||||
use async_trait::async_trait;
|
||||
|
||||
use sp_core::sr25519::Public;
|
||||
use sp_inherents::CreateInherentDataProviders;
|
||||
use sp_runtime::traits::Block;
|
||||
use sp_api::TransactionFor;
|
||||
|
||||
use sp_consensus::{Error, CacheKeyId, Environment};
|
||||
use sc_consensus::{BlockImportParams, BlockImport, Verifier};
|
||||
|
||||
use sc_client_api::Backend;
|
||||
|
||||
use frame_support::traits::ValidatorSet;
|
||||
|
||||
use crate::{
|
||||
tendermint::{TendermintClient, TendermintImport},
|
||||
Announce,
|
||||
};
|
||||
|
||||
#[async_trait]
|
||||
impl<
|
||||
B: Block,
|
||||
Be: Backend<B> + 'static,
|
||||
C: TendermintClient<B, Be>,
|
||||
CIDP: CreateInherentDataProviders<B, ()> + 'static,
|
||||
E: Send + Sync + Environment<B> + 'static,
|
||||
A: Announce<B>,
|
||||
> Verifier<B> for TendermintImport<B, Be, C, CIDP, E, A>
|
||||
where
|
||||
TransactionFor<C, B>: Send + Sync + 'static,
|
||||
Arc<C>: BlockImport<B, Transaction = TransactionFor<C, B>>,
|
||||
<Arc<C> as BlockImport<B>>::Error: Into<Error>,
|
||||
C::Api: ValidatorSet<Public>,
|
||||
{
|
||||
async fn verify(
|
||||
&mut self,
|
||||
mut block: BlockImportParams<B, ()>,
|
||||
) -> Result<(BlockImportParams<B, ()>, Option<Vec<(CacheKeyId, Vec<u8>)>>), String> {
|
||||
self.check(&mut block).await.map_err(|e| format!("{}", e))?;
|
||||
Ok((block, None))
|
||||
}
|
||||
}
|
||||
@@ -3,7 +3,7 @@ name = "tendermint-machine"
|
||||
version = "0.1.0"
|
||||
description = "An implementation of the Tendermint state machine in Rust"
|
||||
license = "MIT"
|
||||
repository = "https://github.com/serai-dex/serai/tree/develop/substrate/tendermint"
|
||||
repository = "https://github.com/serai-dex/serai/tree/develop/substrate/tendermint/machine"
|
||||
authors = ["Luke Parker <lukeparker5132@gmail.com>"]
|
||||
edition = "2021"
|
||||
|
||||
36
substrate/tendermint/pallet/Cargo.toml
Normal file
36
substrate/tendermint/pallet/Cargo.toml
Normal file
@@ -0,0 +1,36 @@
|
||||
[package]
|
||||
name = "pallet-tendermint"
|
||||
version = "0.1.0"
|
||||
description = "Tendermint pallet for Substrate"
|
||||
license = "AGPL-3.0-only"
|
||||
repository = "https://github.com/serai-dex/serai/tree/develop/substrate/tendermint/pallet"
|
||||
authors = ["Luke Parker <lukeparker5132@gmail.com>"]
|
||||
edition = "2021"
|
||||
|
||||
[package.metadata.docs.rs]
|
||||
all-features = true
|
||||
rustdoc-args = ["--cfg", "docsrs"]
|
||||
|
||||
[dependencies]
|
||||
parity-scale-codec = { version = "3", default-features = false, features = ["derive"] }
|
||||
scale-info = { version = "2", default-features = false, features = ["derive"] }
|
||||
|
||||
sp-application-crypto = { git = "https://github.com/serai-dex/substrate", default-features = false }
|
||||
|
||||
frame-system = { git = "https://github.com/serai-dex/substrate", default-features = false }
|
||||
frame-support = { git = "https://github.com/serai-dex/substrate", default-features = false }
|
||||
|
||||
[features]
|
||||
std = [
|
||||
"sp-application-crypto/std",
|
||||
|
||||
"frame-system/std",
|
||||
"frame-support/std",
|
||||
]
|
||||
|
||||
runtime-benchmarks = [
|
||||
"frame-system/runtime-benchmarks",
|
||||
"frame-support/runtime-benchmarks",
|
||||
]
|
||||
|
||||
default = ["std"]
|
||||
15
substrate/tendermint/pallet/LICENSE
Normal file
15
substrate/tendermint/pallet/LICENSE
Normal file
@@ -0,0 +1,15 @@
|
||||
AGPL-3.0-only license
|
||||
|
||||
Copyright (c) 2022 Luke Parker
|
||||
|
||||
This program is free software: you can redistribute it and/or modify
|
||||
it under the terms of the GNU Affero General Public License Version 3 as
|
||||
published by the Free Software Foundation.
|
||||
|
||||
This program is distributed in the hope that it will be useful,
|
||||
but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
GNU Affero General Public License for more details.
|
||||
|
||||
You should have received a copy of the GNU Affero General Public License
|
||||
along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
60
substrate/tendermint/pallet/src/lib.rs
Normal file
60
substrate/tendermint/pallet/src/lib.rs
Normal file
@@ -0,0 +1,60 @@
|
||||
#![cfg_attr(not(feature = "std"), no_std)]
|
||||
|
||||
use frame_support::traits::OneSessionHandler;
|
||||
|
||||
#[frame_support::pallet]
|
||||
pub mod pallet {
|
||||
use frame_support::pallet_prelude::*;
|
||||
|
||||
#[pallet::config]
|
||||
pub trait Config: frame_system::Config {}
|
||||
|
||||
#[pallet::pallet]
|
||||
pub struct Pallet<T>(PhantomData<T>);
|
||||
}
|
||||
|
||||
pub use pallet::*;
|
||||
|
||||
pub mod crypto {
|
||||
use sp_application_crypto::{KeyTypeId, app_crypto, sr25519};
|
||||
app_crypto!(sr25519, KeyTypeId(*b"tend"));
|
||||
|
||||
impl<C> sp_application_crypto::BoundToRuntimeAppPublic for crate::Pallet<C> {
|
||||
type Public = Public;
|
||||
}
|
||||
|
||||
sp_application_crypto::with_pair! {
|
||||
pub type AuthorityPair = Pair;
|
||||
}
|
||||
pub type AuthoritySignature = Signature;
|
||||
pub type AuthorityId = Public;
|
||||
}
|
||||
|
||||
impl<C, V> OneSessionHandler<V> for Pallet<C> {
|
||||
type Key = crypto::Public;
|
||||
|
||||
fn on_genesis_session<'a, I: 'a>(_validators: I)
|
||||
where
|
||||
I: Iterator<Item = (&'a V, Self::Key)>,
|
||||
V: 'a,
|
||||
{
|
||||
}
|
||||
|
||||
fn on_new_session<'a, I: 'a>(_changed: bool, _validators: I, _queued: I)
|
||||
where
|
||||
I: Iterator<Item = (&'a V, Self::Key)>,
|
||||
V: 'a,
|
||||
{
|
||||
/*
|
||||
if !changed {
|
||||
return;
|
||||
}
|
||||
|
||||
for validator in validators {
|
||||
...
|
||||
}
|
||||
*/
|
||||
}
|
||||
|
||||
fn on_disabled(_validator_index: u32) {}
|
||||
}
|
||||
12
substrate/tendermint/primitives/Cargo.toml
Normal file
12
substrate/tendermint/primitives/Cargo.toml
Normal file
@@ -0,0 +1,12 @@
|
||||
[package]
|
||||
name = "sp-tendermint"
|
||||
version = "0.1.0"
|
||||
description = "Substrate primitives for Tendermint"
|
||||
license = "AGPL-3.0-only"
|
||||
repository = "https://github.com/serai-dex/serai/tree/develop/substrate/tendermint/primitives"
|
||||
authors = ["Luke Parker <lukeparker5132@gmail.com>"]
|
||||
edition = "2021"
|
||||
|
||||
[package.metadata.docs.rs]
|
||||
all-features = true
|
||||
rustdoc-args = ["--cfg", "docsrs"]
|
||||
15
substrate/tendermint/primitives/LICENSE
Normal file
15
substrate/tendermint/primitives/LICENSE
Normal file
@@ -0,0 +1,15 @@
|
||||
AGPL-3.0-only license
|
||||
|
||||
Copyright (c) 2022 Luke Parker
|
||||
|
||||
This program is free software: you can redistribute it and/or modify
|
||||
it under the terms of the GNU Affero General Public License Version 3 as
|
||||
published by the Free Software Foundation.
|
||||
|
||||
This program is distributed in the hope that it will be useful,
|
||||
but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
GNU Affero General Public License for more details.
|
||||
|
||||
You should have received a copy of the GNU Affero General Public License
|
||||
along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
7
substrate/tendermint/primitives/src/lib.rs
Normal file
7
substrate/tendermint/primitives/src/lib.rs
Normal file
@@ -0,0 +1,7 @@
|
||||
#![cfg_attr(not(feature = "std"), no_std)]
|
||||
|
||||
use sp_core::sr25519::Public;
|
||||
|
||||
trait TendermintApi {
|
||||
fn validators() -> Vec<Public>;
|
||||
}
|
||||
Reference in New Issue
Block a user