Skip to content

Commit

Permalink
get rid of ChainWithMessages::WeightInfo, because we can't have exact…
Browse files Browse the repository at this point in the history
… weights for "external chains" (#1899)
  • Loading branch information
svyatonik committed Feb 22, 2023
1 parent 8ccaa02 commit 3c15c36
Show file tree
Hide file tree
Showing 9 changed files with 121 additions and 89 deletions.
1 change: 1 addition & 0 deletions modules/messages/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@ pub use weights::WeightInfo;
pub use weights_ext::{
ensure_able_to_receive_confirmation, ensure_able_to_receive_message,
ensure_weights_are_correct, WeightInfoExt, EXPECTED_DEFAULT_MESSAGE_LENGTH,
EXTRA_STORAGE_PROOF_SIZE,
};

use crate::{
Expand Down
3 changes: 0 additions & 3 deletions relays/client-bridge-hub-rococo/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -132,9 +132,6 @@ impl ChainWithMessages for BridgeHubRococo {
bp_bridge_hub_rococo::MAX_UNREWARDED_RELAYERS_IN_CONFIRMATION_TX;
const MAX_UNCONFIRMED_MESSAGES_IN_CONFIRMATION_TX: MessageNonce =
bp_bridge_hub_rococo::MAX_UNCONFIRMED_MESSAGES_IN_CONFIRMATION_TX;

// TODO: fix (https://github.com/paritytech/parity-bridges-common/issues/1640)
type WeightInfo = ();
}

#[cfg(test)]
Expand Down
3 changes: 0 additions & 3 deletions relays/client-bridge-hub-wococo/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -132,9 +132,6 @@ impl ChainWithMessages for BridgeHubWococo {
bp_bridge_hub_wococo::MAX_UNREWARDED_RELAYERS_IN_CONFIRMATION_TX;
const MAX_UNCONFIRMED_MESSAGES_IN_CONFIRMATION_TX: MessageNonce =
bp_bridge_hub_wococo::MAX_UNCONFIRMED_MESSAGES_IN_CONFIRMATION_TX;

// TODO: fix (https://github.com/paritytech/parity-bridges-common/issues/1640)
type WeightInfo = ();
}

#[cfg(test)]
Expand Down
1 change: 0 additions & 1 deletion relays/client-millau/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,6 @@ impl ChainWithMessages for Millau {
bp_millau::MAX_UNREWARDED_RELAYERS_IN_CONFIRMATION_TX;
const MAX_UNCONFIRMED_MESSAGES_IN_CONFIRMATION_TX: MessageNonce =
bp_millau::MAX_UNCONFIRMED_MESSAGES_IN_CONFIRMATION_TX;
type WeightInfo = ();
}

impl Chain for Millau {
Expand Down
1 change: 0 additions & 1 deletion relays/client-rialto-parachain/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,6 @@ impl ChainWithMessages for RialtoParachain {
bp_rialto_parachain::MAX_UNREWARDED_RELAYERS_IN_CONFIRMATION_TX;
const MAX_UNCONFIRMED_MESSAGES_IN_CONFIRMATION_TX: MessageNonce =
bp_rialto_parachain::MAX_UNCONFIRMED_MESSAGES_IN_CONFIRMATION_TX;
type WeightInfo = ();
}

impl ChainWithTransactions for RialtoParachain {
Expand Down
1 change: 0 additions & 1 deletion relays/client-rialto/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,6 @@ impl ChainWithMessages for Rialto {
bp_rialto::MAX_UNREWARDED_RELAYERS_IN_CONFIRMATION_TX;
const MAX_UNCONFIRMED_MESSAGES_IN_CONFIRMATION_TX: MessageNonce =
bp_rialto::MAX_UNCONFIRMED_MESSAGES_IN_CONFIRMATION_TX;
type WeightInfo = ();
}

impl ChainWithBalances for Rialto {
Expand Down
3 changes: 0 additions & 3 deletions relays/client-substrate/src/chain.rs
Original file line number Diff line number Diff line change
Expand Up @@ -136,9 +136,6 @@ pub trait ChainWithMessages: Chain {
/// Maximal number of unconfirmed messages in a single confirmation transaction at this
/// `ChainWithMessages`.
const MAX_UNCONFIRMED_MESSAGES_IN_CONFIRMATION_TX: MessageNonce;

/// Weights of message pallet calls.
type WeightInfo: pallet_bridge_messages::WeightInfoExt;
}

/// Call type used by the chain.
Expand Down
50 changes: 21 additions & 29 deletions relays/client-substrate/src/client.rs
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ use crate::{
rpc::{
SubstrateAuthorClient, SubstrateChainClient, SubstrateFinalityClient,
SubstrateFrameSystemClient, SubstrateStateClient, SubstrateSystemClient,
SubstrateTransactionPaymentClient,
},
transaction_stall_timeout, AccountKeyPairOf, ConnectionParams, Error, HashOf, HeaderIdOf,
Result, SignParam, TransactionTracker, UnsignedTransaction,
Expand All @@ -31,15 +30,16 @@ use async_std::sync::{Arc, Mutex};
use async_trait::async_trait;
use bp_runtime::{HeaderIdProvider, StorageDoubleMapKeyProvider, StorageMapKeyProvider};
use codec::{Decode, Encode};
use frame_support::weights::Weight;
use frame_system::AccountInfo;
use futures::{SinkExt, StreamExt};
use jsonrpsee::{
core::DeserializeOwned,
ws_client::{WsClient as RpcClient, WsClientBuilder as RpcClientBuilder},
};
use num_traits::{Bounded, Saturating, Zero};
use num_traits::{Saturating, Zero};
use pallet_balances::AccountData;
use pallet_transaction_payment::InclusionFee;
use pallet_transaction_payment::RuntimeDispatchInfo;
use relay_utils::{relay_loop::RECONNECT_DELAY, STALL_TIMEOUT};
use sp_core::{
storage::{StorageData, StorageKey},
Expand All @@ -51,10 +51,11 @@ use sp_runtime::{
};
use sp_trie::StorageProof;
use sp_version::RuntimeVersion;
use std::{convert::TryFrom, future::Future};
use std::future::Future;

const SUB_API_GRANDPA_AUTHORITIES: &str = "GrandpaApi_grandpa_authorities";
const SUB_API_TXPOOL_VALIDATE_TRANSACTION: &str = "TaggedTransactionQueue_validate_transaction";
const SUB_API_TX_PAYMENT_QUERY_INFO: &str = "TransactionPaymentApi_query_info";
const MAX_SUBSCRIPTION_CAPACITY: usize = 4096;

/// The difference between best block number and number of its ancestor, that is enough
Expand Down Expand Up @@ -591,33 +592,24 @@ impl<C: Chain> Client<C> {
.await
}

/// Estimate fee that will be spent on given extrinsic.
pub async fn estimate_extrinsic_fee(
/// Returns weight of the given transaction.
pub async fn extimate_extrinsic_weight<SignedTransaction: Encode + Send + 'static>(
&self,
transaction: Bytes,
) -> Result<InclusionFee<C::Balance>> {
transaction: SignedTransaction,
) -> Result<Weight> {
self.jsonrpsee_execute(move |client| async move {
let fee_details =
SubstrateTransactionPaymentClient::<C>::fee_details(&*client, transaction, None)
.await?;
let inclusion_fee = fee_details
.inclusion_fee
.map(|inclusion_fee| InclusionFee {
base_fee: C::Balance::try_from(inclusion_fee.base_fee.into_u256())
.unwrap_or_else(|_| C::Balance::max_value()),
len_fee: C::Balance::try_from(inclusion_fee.len_fee.into_u256())
.unwrap_or_else(|_| C::Balance::max_value()),
adjusted_weight_fee: C::Balance::try_from(
inclusion_fee.adjusted_weight_fee.into_u256(),
)
.unwrap_or_else(|_| C::Balance::max_value()),
})
.unwrap_or_else(|| InclusionFee {
base_fee: Zero::zero(),
len_fee: Zero::zero(),
adjusted_weight_fee: Zero::zero(),
});
Ok(inclusion_fee)
let transaction_len = transaction.encoded_size() as u32;

let call = SUB_API_TX_PAYMENT_QUERY_INFO.to_string();
let data = Bytes((transaction, transaction_len).encode());

let encoded_response =
SubstrateStateClient::<C>::call(&*client, call, data, None).await?;
let dispatch_info =
RuntimeDispatchInfo::<C::Balance>::decode(&mut &encoded_response.0[..])
.map_err(Error::ResponseParseFailed)?;

Ok(dispatch_info.weight)
})
.await
}
Expand Down
147 changes: 99 additions & 48 deletions relays/lib-substrate-relay/src/messages_lane.rs
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,9 @@ use crate::{

use async_std::sync::Arc;
use bp_messages::{LaneId, MessageNonce};
use bp_runtime::{AccountIdOf, Chain as _, HeaderIdOf, WeightExtraOps};
use bp_runtime::{
AccountIdOf, Chain as _, EncodedOrDecodedCall, HeaderIdOf, TransactionEra, WeightExtraOps,
};
use bridge_runtime_common::messages::{
source::FromBridgedChainMessagesDeliveryProof, target::FromBridgedChainMessagesProof,
};
Expand All @@ -35,13 +37,15 @@ use messages_relay::{message_lane::MessageLane, message_lane_loop::BatchTransact
use pallet_bridge_messages::{Call as BridgeMessagesCall, Config as BridgeMessagesConfig};
use relay_substrate_client::{
transaction_stall_timeout, AccountKeyPairOf, BalanceOf, BlockNumberOf, CallOf, Chain,
ChainWithMessages, ChainWithTransactions, Client, Error as SubstrateError, HashOf,
ChainWithMessages, ChainWithTransactions, Client, Error as SubstrateError, HashOf, SignParam,
UnsignedTransaction,
};
use relay_utils::{
metrics::{GlobalMetrics, MetricsParams, StandaloneMetric},
STALL_TIMEOUT,
};
use sp_core::Pair;
use sp_runtime::traits::Zero;
use std::{convert::TryFrom, fmt::Debug, marker::PhantomData};

/// Substrate -> Substrate messages synchronization pipeline.
Expand Down Expand Up @@ -159,25 +163,25 @@ where
AccountIdOf<P::TargetChain>: From<<AccountKeyPairOf<P::TargetChain> as Pair>::Public>,
BalanceOf<P::SourceChain>: TryFrom<BalanceOf<P::TargetChain>>,
{
let source_client = params.source_client;
let target_client = params.target_client;
let relayer_id_at_source: AccountIdOf<P::SourceChain> =
params.source_transaction_params.signer.public().into();

// 2/3 is reserved for proofs and tx overhead
let max_messages_size_in_single_batch = P::TargetChain::max_extrinsic_size() / 3;
// we don't know exact weights of the Polkadot runtime. So to guess weights we'll be using
// weights from Rialto and then simply dividing it by x2.
let (max_messages_in_single_batch, max_messages_weight_in_single_batch) =
crate::messages_lane::select_delivery_transaction_limits::<
<P::TargetChain as ChainWithMessages>::WeightInfo,
>(
select_delivery_transaction_limits_rpc::<P>(
&params,
P::TargetChain::max_extrinsic_weight(),
P::SourceChain::MAX_UNREWARDED_RELAYERS_IN_CONFIRMATION_TX,
);
)
.await?;
let (max_messages_in_single_batch, max_messages_weight_in_single_batch) =
(max_messages_in_single_batch / 2, max_messages_weight_in_single_batch / 2);

let source_client = params.source_client;
let target_client = params.target_client;
let relayer_id_at_source: AccountIdOf<P::SourceChain> =
params.source_transaction_params.signer.public().into();

log::info!(
target: "bridge",
"Starting {} -> {} messages relay.\n\t\
Expand Down Expand Up @@ -437,12 +441,15 @@ macro_rules! generate_receive_message_delivery_proof_call_builder {
};
}

/// Returns maximal number of messages and their maximal cumulative dispatch weight, based
/// on given chain parameters.
pub fn select_delivery_transaction_limits<W: pallet_bridge_messages::WeightInfoExt>(
/// Returns maximal number of messages and their maximal cumulative dispatch weight.
async fn select_delivery_transaction_limits_rpc<P: SubstrateMessageLane>(
params: &MessagesRelayParams<P>,
max_extrinsic_weight: Weight,
max_unconfirmed_messages_at_inbound_lane: MessageNonce,
) -> (MessageNonce, Weight) {
) -> anyhow::Result<(MessageNonce, Weight)>
where
AccountIdOf<P::SourceChain>: From<<AccountKeyPairOf<P::SourceChain> as Pair>::Public>,
{
// We may try to guess accurate value, based on maximal number of messages and per-message
// weight overhead, but the relay loop isn't using this info in a super-accurate way anyway.
// So just a rough guess: let's say 1/3 of max tx weight is for tx itself and the rest is
Expand All @@ -455,13 +462,35 @@ pub fn select_delivery_transaction_limits<W: pallet_bridge_messages::WeightInfoE
let weight_for_delivery_tx = max_extrinsic_weight / 3;
let weight_for_messages_dispatch = max_extrinsic_weight - weight_for_delivery_tx;

let delivery_tx_base_weight = W::receive_messages_proof_overhead() +
W::receive_messages_proof_outbound_lane_state_overhead();
let delivery_tx_weight_rest = weight_for_delivery_tx - delivery_tx_base_weight;
// weight of empty message delivery with outbound lane state
let delivery_tx_with_zero_messages = dummy_messages_delivery_transaction::<P>(params, 0)?;
let delivery_tx_with_zero_messages_weight = params
.target_client
.extimate_extrinsic_weight(delivery_tx_with_zero_messages)
.await
.map_err(|e| {
anyhow::format_err!("Failed to estimate delivery extrinsic weight: {:?}", e)
})?;

// weight of single message delivery with outbound lane state
let delivery_tx_with_one_message = dummy_messages_delivery_transaction::<P>(params, 1)?;
let delivery_tx_with_one_message_weight = params
.target_client
.extimate_extrinsic_weight(delivery_tx_with_one_message)
.await
.map_err(|e| {
anyhow::format_err!("Failed to estimate delivery extrinsic weight: {:?}", e)
})?;

// message overhead is roughly `delivery_tx_with_one_message_weight -
// delivery_tx_with_zero_messages_weight`
let delivery_tx_weight_rest = weight_for_delivery_tx - delivery_tx_with_zero_messages_weight;
let delivery_tx_message_overhead =
delivery_tx_with_one_message_weight.saturating_sub(delivery_tx_with_zero_messages_weight);

let max_number_of_messages = std::cmp::min(
delivery_tx_weight_rest
.min_components_checked_div(W::receive_messages_proof_messages_overhead(1))
.min_components_checked_div(delivery_tx_message_overhead)
.unwrap_or(u64::MAX),
max_unconfirmed_messages_at_inbound_lane,
);
Expand All @@ -475,36 +504,58 @@ pub fn select_delivery_transaction_limits<W: pallet_bridge_messages::WeightInfoE
"Relay shall be able to deliver messages with dispatch weight = max_extrinsic_weight / 2",
);

(max_number_of_messages, weight_for_messages_dispatch)
Ok((max_number_of_messages, weight_for_messages_dispatch))
}

#[cfg(test)]
mod tests {
use super::*;
use bp_runtime::Chain;

type RialtoToMillauMessagesWeights =
pallet_bridge_messages::weights::BridgeWeight<rialto_runtime::Runtime>;

#[test]
fn select_delivery_transaction_limits_is_sane() {
// we want to check the `proof_size` component here too. But for Rialto and Millau
// it is set to `u64::MAX` (as for Polkadot and other relay/standalone chains).
// So let's use RialtoParachain limits here - it has `proof_size` limit as all
// Cumulus-based parachains do.
let (max_count, max_weight) =
select_delivery_transaction_limits::<RialtoToMillauMessagesWeights>(
bp_rialto_parachain::RialtoParachain::max_extrinsic_weight(),
bp_rialto::MAX_UNREWARDED_RELAYERS_IN_CONFIRMATION_TX,
);
assert_eq!(
(max_count, max_weight),
// We don't actually care about these values, so feel free to update them whenever test
// fails. The only thing to do before that is to ensure that new values looks sane:
// i.e. weight reserved for messages dispatch allows dispatch of non-trivial messages.
//
// Any significant change in this values should attract additional attention.
(1024, Weight::from_parts(866_600_106_667, 2_271_915)),
/// Returns dummy message delivery transaction with zero messages and `1kb` proof.
fn dummy_messages_delivery_transaction<P: SubstrateMessageLane>(
params: &MessagesRelayParams<P>,
messages: u32,
) -> anyhow::Result<<P::TargetChain as ChainWithTransactions>::SignedTransaction>
where
AccountIdOf<P::SourceChain>: From<<AccountKeyPairOf<P::SourceChain> as Pair>::Public>,
{
// we don't care about any call values here, because all that the estimation RPC does
// is calls `GetDispatchInfo::get_dispatch_info` for the wrapped call. So we only are
// interested in values that affect call weight - e.g. number of messages and the
// storage proof size

let dummy_messages_delivery_call =
P::ReceiveMessagesProofCallBuilder::build_receive_messages_proof_call(
params.source_transaction_params.signer.public().into(),
(
Weight::zero(),
FromBridgedChainMessagesProof {
bridged_header_hash: Default::default(),
// we may use per-chain `EXTRA_STORAGE_PROOF_SIZE`, but since we don't need
// exact values, this global estimation is fine
storage_proof: vec![vec![
42u8;
pallet_bridge_messages::EXTRA_STORAGE_PROOF_SIZE
as usize
]],
lane: Default::default(),
nonces_start: 1,
nonces_end: messages as u64,
},
),
messages,
Weight::zero(),
false,
);
}
P::TargetChain::sign_transaction(
SignParam {
spec_version: 0,
transaction_version: 0,
genesis_hash: Default::default(),
signer: params.target_transaction_params.signer.clone(),
},
UnsignedTransaction {
call: EncodedOrDecodedCall::Decoded(dummy_messages_delivery_call),
nonce: Zero::zero(),
tip: Zero::zero(),
era: TransactionEra::Immortal,
},
)
.map_err(Into::into)
}

0 comments on commit 3c15c36

Please sign in to comment.