Skip to content

Commit

Permalink
Fix multiple parachain headers submission for single message delivery (
Browse files Browse the repository at this point in the history
…paritytech#1916)

* switch off parachains relay when we don't need to relay parachains (temp solution)

* parachains relay now only works with single parachain

* fix usages of parachains relay

* revert hacky fix

* fixes

* fixed Westmint parachain ID

* fixed metrics

* fixed compilation

* fmt

* clippy

* call -> typed_state_call
  • Loading branch information
svyatonik authored and serban300 committed Apr 9, 2024
1 parent c3d879f commit f1d7091
Show file tree
Hide file tree
Showing 17 changed files with 418 additions and 801 deletions.
2 changes: 1 addition & 1 deletion bridges/primitives/chain-westend/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -101,7 +101,7 @@ pub const WITH_WESTEND_BRIDGE_PARAS_PALLET_NAME: &str = "BridgeWestendParachains
pub const MAX_NESTED_PARACHAIN_HEAD_DATA_SIZE: u32 = 128;

/// Identifier of Westmint parachain at the Westend relay chain.
pub const WESTMINT_PARACHAIN_ID: u32 = 2000;
pub const WESTMINT_PARACHAIN_ID: u32 = 1000;

decl_bridge_finality_runtime_apis!(westend);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@
//! Rialto-to-Millau parachains sync entrypoint.
use crate::cli::bridge::{CliBridgeBase, MessagesCliBridge, ParachainToRelayHeadersCliBridge};
use parachains_relay::ParachainsPipeline;
use relay_millau_client::Millau;
use relay_rialto_client::Rialto;
use relay_rialto_parachain_client::RialtoParachain;
Expand All @@ -29,11 +28,6 @@ use substrate_relay_helper::parachains::{
#[derive(Clone, Debug)]
pub struct RialtoParachainsToMillau;

impl ParachainsPipeline for RialtoParachainsToMillau {
type SourceChain = Rialto;
type TargetChain = Millau;
}

impl SubstrateParachainsPipeline for RialtoParachainsToMillau {
type SourceParachain = RialtoParachain;
type SourceRelayChain = Rialto;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
use crate::cli::bridge::{CliBridgeBase, MessagesCliBridge, ParachainToRelayHeadersCliBridge};
use bp_polkadot_core::parachains::{ParaHash, ParaHeadsProof, ParaId};
use parachains_relay::ParachainsPipeline;
use relay_substrate_client::{CallOf, HeaderIdOf};
use substrate_relay_helper::parachains::{
SubmitParachainHeadsCallBuilder, SubstrateParachainsPipeline,
Expand All @@ -28,11 +27,6 @@ use substrate_relay_helper::parachains::{
#[derive(Clone, Debug)]
pub struct BridgeHubRococoToBridgeHubWococo;

impl ParachainsPipeline for BridgeHubRococoToBridgeHubWococo {
type SourceChain = relay_rococo_client::Rococo;
type TargetChain = relay_bridge_hub_wococo_client::BridgeHubWococo;
}

impl SubstrateParachainsPipeline for BridgeHubRococoToBridgeHubWococo {
type SourceParachain = relay_bridge_hub_rococo_client::BridgeHubRococo;
type SourceRelayChain = relay_rococo_client::Rococo;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@
//! Westend-to-Millau parachains sync entrypoint.
use crate::cli::bridge::{CliBridgeBase, ParachainToRelayHeadersCliBridge};
use parachains_relay::ParachainsPipeline;
use relay_millau_client::Millau;
use relay_westend_client::{Westend, Westmint};
use substrate_relay_helper::parachains::{
Expand All @@ -28,11 +27,6 @@ use substrate_relay_helper::parachains::{
#[derive(Clone, Debug)]
pub struct WestendParachainsToMillau;

impl ParachainsPipeline for WestendParachainsToMillau {
type SourceChain = Westend;
type TargetChain = Millau;
}

impl SubstrateParachainsPipeline for WestendParachainsToMillau {
type SourceParachain = Westmint;
type SourceRelayChain = Westend;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
use crate::cli::bridge::{CliBridgeBase, MessagesCliBridge, ParachainToRelayHeadersCliBridge};
use bp_polkadot_core::parachains::{ParaHash, ParaHeadsProof, ParaId};
use parachains_relay::ParachainsPipeline;
use relay_substrate_client::{CallOf, HeaderIdOf};
use substrate_relay_helper::parachains::{
SubmitParachainHeadsCallBuilder, SubstrateParachainsPipeline,
Expand All @@ -28,11 +27,6 @@ use substrate_relay_helper::parachains::{
#[derive(Clone, Debug)]
pub struct BridgeHubWococoToBridgeHubRococo;

impl ParachainsPipeline for BridgeHubWococoToBridgeHubRococo {
type SourceChain = relay_wococo_client::Wococo;
type TargetChain = relay_bridge_hub_rococo_client::BridgeHubRococo;
}

impl SubstrateParachainsPipeline for BridgeHubWococoToBridgeHubRococo {
type SourceParachain = relay_bridge_hub_wococo_client::BridgeHubWococo;
type SourceRelayChain = relay_wococo_client::Wococo;
Expand Down
9 changes: 4 additions & 5 deletions bridges/relays/bin-substrate/src/cli/bridge.rs
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@

use crate::cli::CliChain;
use pallet_bridge_parachains::{RelayBlockHash, RelayBlockHasher, RelayBlockNumber};
use parachains_relay::ParachainsPipeline;
use relay_substrate_client::{Chain, ChainWithTransactions, Parachain, RelayChain};
use strum::{EnumString, EnumVariantNames};
use substrate_relay_helper::{
Expand Down Expand Up @@ -87,10 +86,10 @@ where
+ RelayChain;
/// Finality proofs synchronization pipeline (source parachain -> target).
type ParachainFinality: SubstrateParachainsPipeline<
SourceRelayChain = Self::SourceRelay,
SourceParachain = Self::Source,
TargetChain = Self::Target,
> + ParachainsPipeline<SourceChain = Self::SourceRelay, TargetChain = Self::Target>;
SourceRelayChain = Self::SourceRelay,
SourceParachain = Self::Source,
TargetChain = Self::Target,
>;
/// Finality proofs synchronization pipeline (source relay chain -> target).
type RelayFinality: SubstrateFinalitySyncPipeline<
SourceChain = Self::SourceRelay,
Expand Down
19 changes: 3 additions & 16 deletions bridges/relays/bin-substrate/src/cli/relay_parachains.rs
Original file line number Diff line number Diff line change
Expand Up @@ -22,20 +22,14 @@ use crate::chains::{
};
use async_std::sync::Mutex;
use async_trait::async_trait;
use bp_polkadot_core::parachains::ParaId;
use parachains_relay::parachains_loop::{
AvailableHeader, ParachainSyncParams, SourceClient, TargetClient,
};
use relay_substrate_client::{Parachain, ParachainBase};
use parachains_relay::parachains_loop::{AvailableHeader, SourceClient, TargetClient};
use relay_substrate_client::Parachain;
use relay_utils::metrics::{GlobalMetrics, StandaloneMetric};
use std::sync::Arc;
use structopt::StructOpt;
use strum::{EnumString, EnumVariantNames, VariantNames};
use substrate_relay_helper::{
parachains::{
source::ParachainsSource, target::ParachainsTarget, ParachainsPipelineAdapter,
SubstrateParachainsPipeline,
},
parachains::{source::ParachainsSource, target::ParachainsTarget, ParachainsPipelineAdapter},
TransactionParams,
};

Expand Down Expand Up @@ -105,13 +99,6 @@ where
parachains_relay::parachains_loop::run(
source_client,
target_client,
ParachainSyncParams {
parachains: vec![
ParaId(<Self::ParachainFinality as SubstrateParachainsPipeline>::SourceParachain::PARACHAIN_ID)
],
stall_timeout: std::time::Duration::from_secs(60),
strategy: parachains_relay::parachains_loop::ParachainSyncStrategy::Any,
},
metrics_params,
futures::future::pending(),
)
Expand Down
2 changes: 1 addition & 1 deletion bridges/relays/client-substrate/src/chain.rs
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ pub trait RelayChain: Chain {
/// Name of the bridge parachains pallet (used in `construct_runtime` macro call) that is
/// deployed at the **bridged** chain.
///
/// We assume that all chains that are bridging with this `ChainWithGrandpa` are using
/// We assume that all chains that are bridging with this `RelayChain` are using
/// the same name.
const PARACHAINS_FINALITY_PALLET_NAME: &'static str;
}
Expand Down
48 changes: 48 additions & 0 deletions bridges/relays/client-substrate/src/test_chain.rs
Original file line number Diff line number Diff line change
Expand Up @@ -68,3 +68,51 @@ impl ChainWithBalances for TestChain {
unreachable!()
}
}

/// Primitives-level parachain that may be used in tests.
#[derive(Clone, Debug, PartialEq, Eq)]
pub struct TestParachainBase;

impl bp_runtime::Chain for TestParachainBase {
type BlockNumber = u32;
type Hash = sp_core::H256;
type Hasher = sp_runtime::traits::BlakeTwo256;
type Header = sp_runtime::generic::Header<u32, sp_runtime::traits::BlakeTwo256>;

type AccountId = u32;
type Balance = u32;
type Index = u32;
type Signature = sp_runtime::testing::TestSignature;

fn max_extrinsic_size() -> u32 {
unreachable!()
}

fn max_extrinsic_weight() -> Weight {
unreachable!()
}
}

impl bp_runtime::Parachain for TestParachainBase {
const PARACHAIN_ID: u32 = 1000;
}

/// Parachain that may be used in tests.
#[derive(Clone, Debug, PartialEq, Eq)]
pub struct TestParachain;

impl bp_runtime::UnderlyingChainProvider for TestParachain {
type Chain = TestParachainBase;
}

impl Chain for TestParachain {
const NAME: &'static str = "TestParachain";
const TOKEN_ID: Option<&'static str> = None;
const BEST_FINALIZED_HEADER_ID_METHOD: &'static str = "TestParachainMethod";
const AVERAGE_BLOCK_INTERVAL: Duration = Duration::from_millis(0);

type SignedBlock = sp_runtime::generic::SignedBlock<
sp_runtime::generic::Block<Self::Header, sp_runtime::OpaqueExtrinsic>,
>;
type Call = ();
}
56 changes: 24 additions & 32 deletions bridges/relays/lib-substrate-relay/src/on_demand/parachains.rs
Original file line number Diff line number Diff line change
Expand Up @@ -36,9 +36,7 @@ use bp_runtime::HeaderIdProvider;
use futures::{select, FutureExt};
use num_traits::Zero;
use pallet_bridge_parachains::{RelayBlockHash, RelayBlockHasher, RelayBlockNumber};
use parachains_relay::parachains_loop::{
AvailableHeader, ParachainSyncParams, SourceClient, TargetClient,
};
use parachains_relay::parachains_loop::{AvailableHeader, SourceClient, TargetClient};
use relay_substrate_client::{
is_ancient_block, AccountIdOf, AccountKeyPairOf, BlockNumberOf, CallOf, Chain, Client,
Error as SubstrateError, HashOf, HeaderIdOf, ParachainBase,
Expand Down Expand Up @@ -183,7 +181,7 @@ where
let mut proved_parachain_block = selected_parachain_block;
if proved_relay_block != selected_relay_block {
proved_parachain_block = parachains_source
.on_chain_para_head_id(proved_relay_block, para_id)
.on_chain_para_head_id(proved_relay_block)
.await?
// this could happen e.g. if parachain has been offboarded?
.ok_or_else(|| {
Expand All @@ -209,11 +207,11 @@ where
}

// and finally - prove parachain head
let (para_proof, para_hashes) =
parachains_source.prove_parachain_heads(proved_relay_block, &[para_id]).await?;
let (para_proof, para_hash) =
parachains_source.prove_parachain_head(proved_relay_block).await?;
calls.push(P::SubmitParachainHeadsCallBuilder::build_submit_parachain_heads_call(
proved_relay_block,
para_hashes.into_iter().map(|h| (para_id, h)).collect(),
vec![(para_id, para_hash)],
para_proof,
));

Expand Down Expand Up @@ -241,16 +239,14 @@ async fn background_task<P: SubstrateParachainsPipeline>(

let mut relay_state = RelayState::Idle;
let mut required_parachain_header_number = Zero::zero();
let required_para_header_number_ref = Arc::new(Mutex::new(AvailableHeader::Unavailable));
let required_para_header_ref = Arc::new(Mutex::new(AvailableHeader::Unavailable));

let mut restart_relay = true;
let parachains_relay_task = futures::future::Fuse::terminated();
futures::pin_mut!(parachains_relay_task);

let mut parachains_source = ParachainsSource::<P>::new(
source_relay_client.clone(),
required_para_header_number_ref.clone(),
);
let mut parachains_source =
ParachainsSource::<P>::new(source_relay_client.clone(), required_para_header_ref.clone());
let mut parachains_target =
ParachainsTarget::<P>::new(target_client.clone(), target_transaction_params.clone());

Expand All @@ -271,13 +267,20 @@ async fn background_task<P: SubstrateParachainsPipeline>(
},
};

// keep in mind that we are not updating `required_para_header_number_ref` here, because
// keep in mind that we are not updating `required_para_header_ref` here, because
// then we'll be submitting all previous headers as well (while required relay headers are
// delivered) and we want to avoid that (to reduce cost)
required_parachain_header_number = std::cmp::max(
required_parachain_header_number,
new_required_parachain_header_number,
);
if new_required_parachain_header_number > required_parachain_header_number {
log::trace!(
target: "bridge",
"[{}] More {} headers required. Going to sync up to the {}",
relay_task_name,
P::SourceParachain::NAME,
new_required_parachain_header_number,
);

required_parachain_header_number = new_required_parachain_header_number;
}
},
_ = async_std::task::sleep(P::TargetChain::AVERAGE_BLOCK_INTERVAL).fuse() => {},
_ = parachains_relay_task => {
Expand Down Expand Up @@ -351,7 +354,7 @@ async fn background_task<P: SubstrateParachainsPipeline>(
.await;
},
RelayState::RelayingParaHeader(required_para_header) => {
*required_para_header_number_ref.lock().await =
*required_para_header_ref.lock().await =
AvailableHeader::Available(required_para_header);
},
}
Expand Down Expand Up @@ -379,11 +382,6 @@ async fn background_task<P: SubstrateParachainsPipeline>(
parachains_relay::parachains_loop::run(
parachains_source.clone(),
parachains_target.clone(),
ParachainSyncParams {
parachains: vec![P::SourceParachain::PARACHAIN_ID.into()],
stall_timeout: std::time::Duration::from_secs(60),
strategy: parachains_relay::parachains_loop::ParachainSyncStrategy::Any,
},
MetricsParams::disabled(),
futures::future::pending(),
)
Expand Down Expand Up @@ -489,10 +487,7 @@ where
source.client().best_finalized_header().await.map_err(map_source_err)?;
let best_finalized_relay_block_id = best_finalized_relay_header.id();
let para_header_at_source = source
.on_chain_para_head_id(
best_finalized_relay_block_id,
P::SourceParachain::PARACHAIN_ID.into(),
)
.on_chain_para_head_id(best_finalized_relay_block_id)
.await
.map_err(map_source_err)?;

Expand All @@ -515,10 +510,7 @@ where
let para_header_at_relay_header_at_target =
if let Some(available_relay_header_at_target) = available_relay_header_at_target {
source
.on_chain_para_head_id(
available_relay_header_at_target,
P::SourceParachain::PARACHAIN_ID.into(),
)
.on_chain_para_head_id(available_relay_header_at_target)
.await
.map_err(map_source_err)?
} else {
Expand Down Expand Up @@ -669,7 +661,7 @@ impl<'a, P: SubstrateParachainsPipeline>
&self,
at_relay_block: HeaderIdOf<P::SourceRelayChain>,
) -> Result<Option<HeaderIdOf<P::SourceParachain>>, SubstrateError> {
self.1.on_chain_para_head_id(at_relay_block, self.parachain_id()).await
self.1.on_chain_para_head_id(at_relay_block).await
}
}

Expand Down
3 changes: 2 additions & 1 deletion bridges/relays/lib-substrate-relay/src/parachains/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,8 @@ pub struct ParachainsPipelineAdapter<P: SubstrateParachainsPipeline> {
}

impl<P: SubstrateParachainsPipeline> ParachainsPipeline for ParachainsPipelineAdapter<P> {
type SourceChain = P::SourceRelayChain;
type SourceParachain = P::SourceParachain;
type SourceRelayChain = P::SourceRelayChain;
type TargetChain = P::TargetChain;
}

Expand Down
Loading

0 comments on commit f1d7091

Please sign in to comment.