mirror of
https://github.com/serai-dex/serai.git
synced 2025-12-08 12:19:24 +00:00
Validator DHT (#494)
* Route validators for any active set through sc-authority-discovery Additionally adds an RPC route to retrieve their P2P addresses. * Have the coordinator get peers from substrate * Have the RPC return one address, not up to 3 Prevents the coordinator from believing it has 3 peers when it has one. * Add missing feature to serai-client * Correct network argument in serai-client for p2p_validators call * Add a test in serai-client to check DHT population with a much quicker failure than the coordinator tests * Update to latest Substrate Removes distinguishing BABE/AuthorityDiscovery keys which causes sc_authority_discovery to populate as desired. * Update to a properly tagged substrate commit * Add all dialed to peers to GossipSub * cargo fmt * Reduce common code in serai-coordinator-tests with amore involved new_test * Use a recursive async function to spawn `n` DockerTests with the necessary networking configuration * Merge UNIQUE_ID and ONE_AT_A_TIME * Tidy up the new recursive code in tests/coordinator * Use a Mutex in CONTEXT to let it be set multiple times * Make complimentary edits to full-stack tests * Augment coordinator P2p connection logs * Drop lock acquisitions before recursing * Better scope lock acquisitions in full-stack, preventing a deadlock * Ensure OUTER_OPS is reset across the test boundary * Add cargo deny allowance for dockertest fork
This commit is contained in:
@@ -27,6 +27,7 @@ serde_json = { version = "1", optional = true }
|
||||
|
||||
serai-abi = { path = "../abi", version = "0.1" }
|
||||
|
||||
multiaddr = { version = "0.18", optional = true }
|
||||
sp-core = { git = "https://github.com/serai-dex/substrate", optional = true }
|
||||
sp-runtime = { git = "https://github.com/serai-dex/substrate", optional = true }
|
||||
frame-system = { git = "https://github.com/serai-dex/substrate", optional = true }
|
||||
@@ -56,7 +57,7 @@ dockertest = "0.4"
|
||||
serai-docker-tests = { path = "../../tests/docker" }
|
||||
|
||||
[features]
|
||||
serai = ["thiserror", "serde", "serde_json", "sp-core", "sp-runtime", "frame-system", "simple-request"]
|
||||
serai = ["thiserror", "serde", "serde_json", "serai-abi/serde", "multiaddr", "sp-core", "sp-runtime", "frame-system", "simple-request"]
|
||||
borsh = ["serai-abi/borsh"]
|
||||
|
||||
networks = []
|
||||
|
||||
@@ -16,7 +16,7 @@ pub use abi::{primitives, Transaction};
|
||||
use abi::*;
|
||||
|
||||
pub use primitives::{SeraiAddress, Signature, Amount};
|
||||
use primitives::Header;
|
||||
use primitives::{Header, NetworkId};
|
||||
|
||||
pub mod coins;
|
||||
pub use coins::SeraiCoins;
|
||||
@@ -306,6 +306,14 @@ impl Serai {
|
||||
pub fn as_of(&self, block: [u8; 32]) -> TemporalSerai {
|
||||
TemporalSerai { serai: self, block, events: RwLock::new(None) }
|
||||
}
|
||||
|
||||
/// Return the P2P Multiaddrs for the validators of the specified network.
|
||||
pub async fn p2p_validators(
|
||||
&self,
|
||||
network: NetworkId,
|
||||
) -> Result<Vec<multiaddr::Multiaddr>, SeraiError> {
|
||||
self.call("p2p_validators", network).await
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a> TemporalSerai<'a> {
|
||||
|
||||
59
substrate/client/tests/dht.rs
Normal file
59
substrate/client/tests/dht.rs
Normal file
@@ -0,0 +1,59 @@
|
||||
use serai_client::{primitives::NetworkId, Serai};
|
||||
|
||||
#[tokio::test]
|
||||
async fn dht() {
|
||||
use dockertest::{
|
||||
PullPolicy, StartPolicy, LogOptions, LogAction, LogPolicy, LogSource, Image,
|
||||
TestBodySpecification, DockerTest,
|
||||
};
|
||||
|
||||
serai_docker_tests::build("serai".to_string());
|
||||
|
||||
let handle = |name| format!("serai_client-serai_node-{name}");
|
||||
let composition = |name| {
|
||||
TestBodySpecification::with_image(
|
||||
Image::with_repository("serai-dev-serai").pull_policy(PullPolicy::Never),
|
||||
)
|
||||
.replace_cmd(vec![
|
||||
"serai-node".to_string(),
|
||||
"--unsafe-rpc-external".to_string(),
|
||||
"--rpc-cors".to_string(),
|
||||
"all".to_string(),
|
||||
"--chain".to_string(),
|
||||
"local".to_string(),
|
||||
format!("--{name}"),
|
||||
])
|
||||
.set_publish_all_ports(true)
|
||||
.set_handle(handle(name))
|
||||
.set_start_policy(StartPolicy::Strict)
|
||||
.set_log_options(Some(LogOptions {
|
||||
action: LogAction::Forward,
|
||||
policy: LogPolicy::Always,
|
||||
source: LogSource::Both,
|
||||
}))
|
||||
};
|
||||
|
||||
let mut test = DockerTest::new().with_network(dockertest::Network::Isolated);
|
||||
test.provide_container(composition("alice"));
|
||||
test.provide_container(composition("bob"));
|
||||
test.provide_container(composition("charlie"));
|
||||
test.provide_container(composition("dave"));
|
||||
test
|
||||
.run_async(|ops| async move {
|
||||
// Sleep until the Substrate RPC starts
|
||||
let alice = handle("alice");
|
||||
let serai_rpc = ops.handle(&alice).host_port(9944).unwrap();
|
||||
let serai_rpc = format!("http://{}:{}", serai_rpc.0, serai_rpc.1);
|
||||
// Sleep for a minute
|
||||
tokio::time::sleep(core::time::Duration::from_secs(60)).await;
|
||||
// Check the DHT has been populated
|
||||
assert!(!Serai::new(serai_rpc.clone())
|
||||
.await
|
||||
.unwrap()
|
||||
.p2p_validators(NetworkId::Bitcoin)
|
||||
.await
|
||||
.unwrap()
|
||||
.is_empty());
|
||||
})
|
||||
.await;
|
||||
}
|
||||
@@ -145,7 +145,7 @@ pub mod pallet {
|
||||
fn increase_balance_internal(to: Public, balance: Balance) -> Result<(), Error<T, I>> {
|
||||
let coin = &balance.coin;
|
||||
|
||||
// sub amount from account
|
||||
// add amount to account
|
||||
let new_amount = Self::balances(to, coin)
|
||||
.checked_add(balance.amount.0)
|
||||
.ok_or(Error::<T, I>::AmountOverflowed)?;
|
||||
|
||||
@@ -20,11 +20,6 @@ workspace = true
|
||||
name = "serai-node"
|
||||
|
||||
[dependencies]
|
||||
clap = { version = "4", features = ["derive"] }
|
||||
|
||||
futures-util = "0.3"
|
||||
jsonrpsee = { version = "0.16", features = ["server"] }
|
||||
|
||||
sp-core = { git = "https://github.com/serai-dex/substrate" }
|
||||
sp-timestamp = { git = "https://github.com/serai-dex/substrate" }
|
||||
sp-io = { git = "https://github.com/serai-dex/substrate" }
|
||||
@@ -37,6 +32,12 @@ frame-benchmarking = { git = "https://github.com/serai-dex/substrate" }
|
||||
|
||||
serai-runtime = { path = "../runtime", features = ["std"] }
|
||||
|
||||
clap = { version = "4", features = ["derive"] }
|
||||
|
||||
futures-util = "0.3"
|
||||
tokio = { version = "1", features = ["sync", "rt-multi-thread"] }
|
||||
jsonrpsee = { version = "0.16", features = ["server"] }
|
||||
|
||||
sc-offchain = { git = "https://github.com/serai-dex/substrate" }
|
||||
sc-transaction-pool = { git = "https://github.com/serai-dex/substrate" }
|
||||
sc-transaction-pool-api = { git = "https://github.com/serai-dex/substrate" }
|
||||
|
||||
@@ -1,16 +1,18 @@
|
||||
use std::sync::Arc;
|
||||
|
||||
use jsonrpsee::RpcModule;
|
||||
use std::{sync::Arc, collections::HashSet};
|
||||
|
||||
use sp_blockchain::{Error as BlockchainError, HeaderBackend, HeaderMetadata};
|
||||
use sp_block_builder::BlockBuilder;
|
||||
use sp_api::ProvideRuntimeApi;
|
||||
|
||||
use serai_runtime::{
|
||||
primitives::{SubstrateAmount, PublicKey},
|
||||
Nonce, Block,
|
||||
primitives::{NetworkId, SubstrateAmount, PublicKey},
|
||||
Nonce, Block, SeraiRuntimeApi,
|
||||
};
|
||||
|
||||
use tokio::sync::RwLock;
|
||||
|
||||
use jsonrpsee::RpcModule;
|
||||
|
||||
pub use sc_rpc_api::DenyUnsafe;
|
||||
use sc_transaction_pool_api::TransactionPool;
|
||||
|
||||
@@ -18,6 +20,7 @@ pub struct FullDeps<C, P> {
|
||||
pub client: Arc<C>,
|
||||
pub pool: Arc<P>,
|
||||
pub deny_unsafe: DenyUnsafe,
|
||||
pub authority_discovery: Option<sc_authority_discovery::Service>,
|
||||
}
|
||||
|
||||
pub fn create_full<
|
||||
@@ -34,16 +37,56 @@ pub fn create_full<
|
||||
where
|
||||
C::Api: substrate_frame_rpc_system::AccountNonceApi<Block, PublicKey, Nonce>
|
||||
+ pallet_transaction_payment_rpc::TransactionPaymentRuntimeApi<Block, SubstrateAmount>
|
||||
+ SeraiRuntimeApi<Block>
|
||||
+ BlockBuilder<Block>,
|
||||
{
|
||||
use substrate_frame_rpc_system::{System, SystemApiServer};
|
||||
use pallet_transaction_payment_rpc::{TransactionPayment, TransactionPaymentApiServer};
|
||||
|
||||
let mut module = RpcModule::new(());
|
||||
let FullDeps { client, pool, deny_unsafe } = deps;
|
||||
let FullDeps { client, pool, deny_unsafe, authority_discovery } = deps;
|
||||
|
||||
module.merge(System::new(client.clone(), pool, deny_unsafe).into_rpc())?;
|
||||
module.merge(TransactionPayment::new(client).into_rpc())?;
|
||||
module.merge(TransactionPayment::new(client.clone()).into_rpc())?;
|
||||
|
||||
if let Some(authority_discovery) = authority_discovery {
|
||||
let mut authority_discovery_module = RpcModule::new((client, RwLock::new(authority_discovery)));
|
||||
authority_discovery_module.register_async_method(
|
||||
"p2p_validators",
|
||||
|params, context| async move {
|
||||
let network: NetworkId = params.parse()?;
|
||||
let (client, authority_discovery) = &*context;
|
||||
let latest_block = client.info().best_hash;
|
||||
|
||||
let validators = client.runtime_api().validators(latest_block, network).map_err(|_| {
|
||||
jsonrpsee::core::Error::to_call_error(std::io::Error::other(format!(
|
||||
"couldn't get validators from the latest block, which is likely a fatal bug. {}",
|
||||
"please report this at https://github.com/serai-dex/serai",
|
||||
)))
|
||||
})?;
|
||||
let mut all_p2p_addresses = vec![];
|
||||
for validator in validators {
|
||||
let mut returned_addresses = authority_discovery
|
||||
.write()
|
||||
.await
|
||||
.get_addresses_by_authority_id(validator.into())
|
||||
.await
|
||||
.unwrap_or_else(HashSet::new)
|
||||
.into_iter();
|
||||
// Only take a single address
|
||||
// There should be one, there may be two if their IP address changed, and more should only
|
||||
// occur if they have multiple proxies/an IP address changing frequently/some issue
|
||||
// preventing consistent self-identification
|
||||
// It isn't beneficial to use multiple addresses for a single peer here
|
||||
if let Some(address) = returned_addresses.next() {
|
||||
all_p2p_addresses.push(address);
|
||||
}
|
||||
}
|
||||
Ok(all_p2p_addresses)
|
||||
},
|
||||
)?;
|
||||
module.merge(authority_discovery_module)?;
|
||||
}
|
||||
|
||||
Ok(module)
|
||||
}
|
||||
|
||||
@@ -206,6 +206,42 @@ pub fn new_full(config: Configuration) -> Result<TaskManager, ServiceError> {
|
||||
);
|
||||
}
|
||||
|
||||
let role = config.role.clone();
|
||||
let keystore = keystore_container.keystore();
|
||||
let prometheus_registry = config.prometheus_registry().cloned();
|
||||
|
||||
// TODO: Ensure we're considered as an authority is a validator of an external network
|
||||
let authority_discovery = if role.is_authority() {
|
||||
let (worker, service) = sc_authority_discovery::new_worker_and_service_with_config(
|
||||
#[allow(clippy::field_reassign_with_default)]
|
||||
{
|
||||
let mut worker = sc_authority_discovery::WorkerConfig::default();
|
||||
worker.publish_non_global_ips = publish_non_global_ips;
|
||||
worker.strict_record_validation = true;
|
||||
worker
|
||||
},
|
||||
client.clone(),
|
||||
network.clone(),
|
||||
Box::pin(network.event_stream("authority-discovery").filter_map(|e| async move {
|
||||
match e {
|
||||
Event::Dht(e) => Some(e),
|
||||
_ => None,
|
||||
}
|
||||
})),
|
||||
sc_authority_discovery::Role::PublishAndDiscover(keystore.clone()),
|
||||
prometheus_registry.clone(),
|
||||
);
|
||||
task_manager.spawn_handle().spawn(
|
||||
"authority-discovery-worker",
|
||||
Some("networking"),
|
||||
worker.run(),
|
||||
);
|
||||
|
||||
Some(service)
|
||||
} else {
|
||||
None
|
||||
};
|
||||
|
||||
let rpc_builder = {
|
||||
let client = client.clone();
|
||||
let pool = transaction_pool.clone();
|
||||
@@ -215,18 +251,15 @@ pub fn new_full(config: Configuration) -> Result<TaskManager, ServiceError> {
|
||||
client: client.clone(),
|
||||
pool: pool.clone(),
|
||||
deny_unsafe,
|
||||
authority_discovery: authority_discovery.clone(),
|
||||
})
|
||||
.map_err(Into::into)
|
||||
})
|
||||
};
|
||||
|
||||
let enable_grandpa = !config.disable_grandpa;
|
||||
let role = config.role.clone();
|
||||
let force_authoring = config.force_authoring;
|
||||
let name = config.network.node_name.clone();
|
||||
let prometheus_registry = config.prometheus_registry().cloned();
|
||||
|
||||
let keystore = keystore_container.keystore();
|
||||
|
||||
sc_service::spawn_tasks(sc_service::SpawnTasksParams {
|
||||
config,
|
||||
@@ -251,7 +284,7 @@ pub fn new_full(config: Configuration) -> Result<TaskManager, ServiceError> {
|
||||
select_chain,
|
||||
env: sc_basic_authorship::ProposerFactory::new(
|
||||
task_manager.spawn_handle(),
|
||||
client.clone(),
|
||||
client,
|
||||
transaction_pool.clone(),
|
||||
prometheus_registry.as_ref(),
|
||||
telemetry.as_ref().map(Telemetry::handle),
|
||||
@@ -277,33 +310,6 @@ pub fn new_full(config: Configuration) -> Result<TaskManager, ServiceError> {
|
||||
);
|
||||
}
|
||||
|
||||
if role.is_authority() {
|
||||
task_manager.spawn_handle().spawn(
|
||||
"authority-discovery-worker",
|
||||
Some("networking"),
|
||||
sc_authority_discovery::new_worker_and_service_with_config(
|
||||
#[allow(clippy::field_reassign_with_default)]
|
||||
{
|
||||
let mut worker = sc_authority_discovery::WorkerConfig::default();
|
||||
worker.publish_non_global_ips = publish_non_global_ips;
|
||||
worker
|
||||
},
|
||||
client,
|
||||
network.clone(),
|
||||
Box::pin(network.event_stream("authority-discovery").filter_map(|e| async move {
|
||||
match e {
|
||||
Event::Dht(e) => Some(e),
|
||||
_ => None,
|
||||
}
|
||||
})),
|
||||
sc_authority_discovery::Role::PublishAndDiscover(keystore.clone()),
|
||||
prometheus_registry.clone(),
|
||||
)
|
||||
.0
|
||||
.run(),
|
||||
);
|
||||
}
|
||||
|
||||
if enable_grandpa {
|
||||
task_manager.spawn_essential_handle().spawn_blocking(
|
||||
"grandpa-voter",
|
||||
|
||||
@@ -19,6 +19,8 @@ ignored = ["scale", "scale-info"]
|
||||
workspace = true
|
||||
|
||||
[dependencies]
|
||||
hashbrown = { version = "0.14", default-features = false, features = ["ahash", "inline-more"] }
|
||||
|
||||
scale = { package = "parity-scale-codec", version = "3", default-features = false, features = ["derive"] }
|
||||
scale-info = { version = "2", default-features = false, features = ["derive"] }
|
||||
|
||||
|
||||
@@ -11,6 +11,7 @@ use core::marker::PhantomData;
|
||||
// Re-export all components
|
||||
pub use serai_primitives as primitives;
|
||||
pub use primitives::{BlockNumber, Header};
|
||||
use primitives::{NetworkId, NETWORKS};
|
||||
|
||||
pub use frame_system as system;
|
||||
pub use frame_support as support;
|
||||
@@ -43,7 +44,7 @@ use sp_runtime::{
|
||||
create_runtime_str, generic, impl_opaque_keys, KeyTypeId,
|
||||
traits::{Convert, BlakeTwo256, Block as BlockT},
|
||||
transaction_validity::{TransactionSource, TransactionValidity},
|
||||
Perbill, ApplyExtrinsicResult,
|
||||
BoundedVec, Perbill, ApplyExtrinsicResult,
|
||||
};
|
||||
|
||||
use primitives::{PublicKey, AccountLookup, SubstrateAmount};
|
||||
@@ -377,6 +378,13 @@ mod benches {
|
||||
);
|
||||
}
|
||||
|
||||
sp_api::decl_runtime_apis! {
|
||||
#[api_version(1)]
|
||||
pub trait SeraiRuntimeApi {
|
||||
fn validators(network_id: NetworkId) -> Vec<PublicKey>;
|
||||
}
|
||||
}
|
||||
|
||||
sp_api::impl_runtime_apis! {
|
||||
impl sp_api::Core<Block> for Runtime {
|
||||
fn version() -> RuntimeVersion {
|
||||
@@ -561,10 +569,41 @@ sp_api::impl_runtime_apis! {
|
||||
|
||||
impl sp_authority_discovery::AuthorityDiscoveryApi<Block> for Runtime {
|
||||
fn authorities() -> Vec<AuthorityDiscoveryId> {
|
||||
Babe::authorities()
|
||||
// Converts to `[u8; 32]` so it can be hashed
|
||||
let serai_validators = Babe::authorities()
|
||||
.into_iter()
|
||||
.map(|(id, _)| AuthorityDiscoveryId::from(id.into_inner()))
|
||||
.collect()
|
||||
.map(|(id, _)| id.into_inner().0)
|
||||
.collect::<hashbrown::HashSet<_>>();
|
||||
let mut all = serai_validators;
|
||||
for network in NETWORKS {
|
||||
if network == NetworkId::Serai {
|
||||
continue;
|
||||
}
|
||||
let participants =
|
||||
ValidatorSets::participants_for_latest_decided_set(network)
|
||||
.map_or(vec![], BoundedVec::into_inner);
|
||||
for (participant, _) in participants {
|
||||
all.insert(participant.0);
|
||||
}
|
||||
}
|
||||
all.into_iter().map(|id| AuthorityDiscoveryId::from(PublicKey::from_raw(id))).collect()
|
||||
}
|
||||
}
|
||||
|
||||
impl crate::SeraiRuntimeApi<Block> for Runtime {
|
||||
fn validators(network_id: NetworkId) -> Vec<PublicKey> {
|
||||
if network_id == NetworkId::Serai {
|
||||
Babe::authorities()
|
||||
.into_iter()
|
||||
.map(|(id, _)| id.into_inner())
|
||||
.collect()
|
||||
} else {
|
||||
ValidatorSets::participants_for_latest_decided_set(network_id)
|
||||
.map_or(
|
||||
vec![],
|
||||
|vec| vec.into_inner().into_iter().map(|(validator, _)| validator).collect()
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -124,8 +124,9 @@ pub mod pallet {
|
||||
#[pallet::getter(fn allocation_per_key_share)]
|
||||
pub type AllocationPerKeyShare<T: Config> =
|
||||
StorageMap<_, Identity, NetworkId, Amount, OptionQuery>;
|
||||
/// The validators selected to be in-set who haven't been removed.
|
||||
/// The validators selected to be in-set.
|
||||
#[pallet::storage]
|
||||
#[pallet::getter(fn participants_for_latest_decided_set)]
|
||||
pub(crate) type Participants<T: Config> = StorageMap<
|
||||
_,
|
||||
Identity,
|
||||
|
||||
Reference in New Issue
Block a user