feat: support `IWeightedMultisigIsm` in relayer (#4360)

### Description

<!--
What's included in this PR?
-->

### Drive-by changes

<!--
Are there any minor or drive-by changes also included?
-->

### Related issues

- fixes https://github.com/hyperlane-xyz/issues/issues/1345

### Backward compatibility

<!--
Are these changes backward compatible? Are there any infrastructure
implications, e.g. changes that would prohibit deploying older commits
using this infra tooling?

Yes/No
-->

### Testing

<!--
What kind of testing have these changes undergone?

None/Manual/Unit Tests
-->
pull/4399/head
Kunal Arora 3 months ago committed by GitHub
parent 707db4a279
commit 24c8188e55
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
  1. 21
      rust/agents/relayer/src/msg/metadata/base.rs
  2. 37
      rust/agents/relayer/src/msg/metadata/multisig/base.rs
  3. 34
      rust/agents/relayer/src/msg/metadata/multisig/merkle_root_multisig.rs
  4. 36
      rust/agents/relayer/src/msg/metadata/multisig/message_id_multisig.rs
  5. 5
      rust/agents/relayer/src/msg/metadata/multisig/mod.rs
  6. 108
      rust/agents/relayer/src/msg/metadata/multisig/weighted.rs
  7. 75
      rust/chains/hyperlane-ethereum/abis/IStaticWeightedMultisigIsm.abi.json
  8. 3
      rust/chains/hyperlane-ethereum/src/ism/mod.rs
  9. 123
      rust/chains/hyperlane-ethereum/src/ism/weighted_multisig_ism.rs
  10. 23
      rust/hyperlane-base/src/settings/chains.rs
  11. 78
      rust/hyperlane-base/src/types/multisig.rs
  12. 6
      rust/hyperlane-core/src/traits/interchain_security_module.rs
  13. 12
      rust/hyperlane-core/src/traits/multisig_ism.rs
  14. 10
      typescript/infra/config/environments/test/aggregationIsm.ts

@ -27,12 +27,16 @@ use hyperlane_base::{
use hyperlane_core::{
accumulator::merkle::Proof, AggregationIsm, CcipReadIsm, Checkpoint, HyperlaneDomain,
HyperlaneMessage, InterchainSecurityModule, Mailbox, ModuleType, MultisigIsm, RoutingIsm,
ValidatorAnnounce, H160, H256,
ValidatorAnnounce, WeightedMultisigIsm, H160, H256,
};
use tokio::sync::RwLock;
use tracing::{debug, info, instrument, warn};
use super::multisig::{
WeightedMerkleRootMultisigMetadataBuilder, WeightedMessageIdMultisigMetadataBuilder,
};
#[derive(Debug, thiserror::Error)]
pub enum MetadataBuilderError {
#[error("Unknown or invalid module type ({0})")]
@ -253,6 +257,12 @@ impl MessageMetadataBuilder {
ModuleType::Aggregation => Box::new(AggregationIsmMetadataBuilder::new(cloned)),
ModuleType::Null => Box::new(NullMetadataBuilder::new()),
ModuleType::CcipRead => Box::new(CcipReadIsmMetadataBuilder::new(cloned)),
ModuleType::WeightedMerkleRootMultisig => {
Box::new(WeightedMerkleRootMultisigMetadataBuilder::new(cloned))
}
ModuleType::WeightedMessageIdMultisig => {
Box::new(WeightedMessageIdMultisigMetadataBuilder::new(cloned))
}
_ => return Err(MetadataBuilderError::UnsupportedModuleType(module_type).into()),
};
let meta = metadata_builder
@ -362,6 +372,15 @@ impl BaseMetadataBuilder {
.await
}
pub async fn build_weighted_multisig_ism(
&self,
address: H256,
) -> Result<Box<dyn WeightedMultisigIsm>> {
self.destination_chain_setup
.build_weighted_multisig_ism(address, &self.metrics)
.await
}
pub async fn build_checkpoint_syncer(
&self,
validators: &[H256],

@ -6,7 +6,7 @@ use derive_new::new;
use ethers::abi::Token;
use eyre::{Context, Result};
use hyperlane_base::MultisigCheckpointSyncer;
use hyperlane_base::{MultisigCheckpointSyncer, ValidatorWithWeight, Weight};
use hyperlane_core::accumulator::merkle::Proof;
use hyperlane_core::{HyperlaneMessage, MultisigSignedCheckpoint, H256};
use strum::Display;
@ -40,8 +40,8 @@ pub enum MetadataToken {
pub trait MultisigIsmMetadataBuilder: AsRef<MessageMetadataBuilder> + Send + Sync {
async fn fetch_metadata(
&self,
validators: &[H256],
threshold: u8,
validators: &[ValidatorWithWeight],
threshold: Weight,
message: &HyperlaneMessage,
checkpoint_syncer: &MultisigCheckpointSyncer,
) -> Result<Option<MultisigMetadata>>;
@ -89,6 +89,12 @@ pub trait MultisigIsmMetadataBuilder: AsRef<MessageMetadataBuilder> + Send + Syn
let metas: Result<Vec<Vec<u8>>> = self.token_layout().iter().map(build_token).collect();
Ok(metas?.into_iter().flatten().collect())
}
async fn ism_validator_requirements(
&self,
ism_address: H256,
message: &HyperlaneMessage,
) -> Result<(Vec<ValidatorWithWeight>, Weight)>;
}
#[async_trait]
@ -99,22 +105,18 @@ impl<T: MultisigIsmMetadataBuilder> MetadataBuilder for T {
message: &HyperlaneMessage,
) -> Result<Option<Vec<u8>>> {
const CTX: &str = "When fetching MultisigIsm metadata";
let multisig_ism = self
.as_ref()
.build_multisig_ism(ism_address)
.await
.context(CTX)?;
let (validators, threshold) = multisig_ism
.validators_and_threshold(message)
.await
.context(CTX)?;
let (weighted_validators, threshold_weight) = self
.ism_validator_requirements(ism_address, message)
.await?;
if validators.is_empty() {
if weighted_validators.is_empty() {
info!("Could not fetch metadata: No validator set found for ISM");
return Ok(None);
}
let validators: Vec<H256> = weighted_validators.iter().map(|vw| vw.validator).collect();
let checkpoint_syncer = self
.as_ref()
.build_checkpoint_syncer(&validators, self.as_ref().app_context.clone())
@ -122,7 +124,12 @@ impl<T: MultisigIsmMetadataBuilder> MetadataBuilder for T {
.context(CTX)?;
if let Some(metadata) = self
.fetch_metadata(&validators, threshold, message, &checkpoint_syncer)
.fetch_metadata(
&weighted_validators,
threshold_weight,
message,
&checkpoint_syncer,
)
.await
.context(CTX)?
{
@ -130,7 +137,7 @@ impl<T: MultisigIsmMetadataBuilder> MetadataBuilder for T {
Ok(Some(self.format_metadata(metadata)?))
} else {
info!(
?message, ?validators, threshold, ism=%multisig_ism.address(),
?message, ?weighted_validators, threshold_weight, ism=%ism_address,
"Could not fetch metadata: Unable to reach quorum"
);
Ok(None)

@ -5,7 +5,7 @@ use derive_more::{AsRef, Deref};
use derive_new::new;
use eyre::{Context, Result};
use hyperlane_base::MultisigCheckpointSyncer;
use hyperlane_base::{MultisigCheckpointSyncer, ValidatorWithWeight, Weight};
use hyperlane_core::{unwrap_or_none_result, HyperlaneMessage, H256};
use tracing::debug;
@ -30,8 +30,8 @@ impl MultisigIsmMetadataBuilder for MerkleRootMultisigMetadataBuilder {
async fn fetch_metadata(
&self,
validators: &[H256],
threshold: u8,
validators: &[ValidatorWithWeight],
threshold_weight: Weight,
message: &HyperlaneMessage,
checkpoint_syncer: &MultisigCheckpointSyncer,
) -> Result<Option<MultisigMetadata>> {
@ -53,7 +53,7 @@ impl MultisigIsmMetadataBuilder for MerkleRootMultisigMetadataBuilder {
checkpoint_syncer
.fetch_checkpoint_in_range(
validators,
threshold as usize,
threshold_weight,
leaf_index,
highest_leaf_index,
self.origin_domain(),
@ -76,4 +76,30 @@ impl MultisigIsmMetadataBuilder for MerkleRootMultisigMetadataBuilder {
Some(proof),
)))
}
// fetches the validators and threshold for the unit variant - each validator has a weight of 1
async fn ism_validator_requirements(
&self,
ism_address: H256,
message: &HyperlaneMessage,
) -> Result<(Vec<ValidatorWithWeight>, Weight)> {
const CTX: &str = "When fetching MultisigIsm metadata";
let multisig_ism = self
.as_ref()
.build_multisig_ism(ism_address)
.await
.context(CTX)?;
let (validators, threshold) = multisig_ism
.validators_and_threshold(message)
.await
.context(CTX)?;
let unit_validators: Vec<ValidatorWithWeight> = validators
.into_iter()
.map(|v| ValidatorWithWeight::new(v, 1))
.collect();
Ok((unit_validators, threshold as Weight))
}
}

@ -5,7 +5,7 @@ use derive_more::{AsRef, Deref};
use derive_new::new;
use eyre::{Context, Result};
use hyperlane_base::MultisigCheckpointSyncer;
use hyperlane_base::{MultisigCheckpointSyncer, ValidatorWithWeight, Weight};
use hyperlane_core::{unwrap_or_none_result, HyperlaneMessage, H256};
use tracing::{debug, warn};
@ -29,8 +29,8 @@ impl MultisigIsmMetadataBuilder for MessageIdMultisigMetadataBuilder {
async fn fetch_metadata(
&self,
validators: &[H256],
threshold: u8,
validators: &[ValidatorWithWeight],
threshold_weight: Weight,
message: &HyperlaneMessage,
checkpoint_syncer: &MultisigCheckpointSyncer,
) -> Result<Option<MultisigMetadata>> {
@ -50,7 +50,7 @@ impl MultisigIsmMetadataBuilder for MessageIdMultisigMetadataBuilder {
// Update the validator latest checkpoint metrics.
let _ = checkpoint_syncer
.get_validator_latest_checkpoints_and_update_metrics(
validators,
&validators.iter().map(|vw| vw.validator).collect::<Vec<_>>(),
self.origin_domain(),
self.destination_domain(),
)
@ -58,7 +58,7 @@ impl MultisigIsmMetadataBuilder for MessageIdMultisigMetadataBuilder {
let quorum_checkpoint = unwrap_or_none_result!(
checkpoint_syncer
.fetch_checkpoint(validators, threshold as usize, leaf_index)
.fetch_checkpoint(validators, threshold_weight, leaf_index)
.await
.context(CTX)?,
debug!("No quorum checkpoint found")
@ -84,4 +84,30 @@ impl MultisigIsmMetadataBuilder for MessageIdMultisigMetadataBuilder {
None,
)))
}
// fetches the validators and threshold for the unit variant - each validator has a weight of 1
async fn ism_validator_requirements(
&self,
ism_address: H256,
message: &HyperlaneMessage,
) -> Result<(Vec<ValidatorWithWeight>, Weight)> {
const CTX: &str = "When fetching MultisigIsm metadata";
let multisig_ism = self
.as_ref()
.build_multisig_ism(ism_address)
.await
.context(CTX)?;
let (validators, threshold) = multisig_ism
.validators_and_threshold(message)
.await
.context(CTX)?;
let unit_validators: Vec<ValidatorWithWeight> = validators
.into_iter()
.map(|v| ValidatorWithWeight::new(v, 1))
.collect();
Ok((unit_validators, threshold as Weight))
}
}

@ -1,8 +1,13 @@
mod base;
mod merkle_root_multisig;
mod message_id_multisig;
mod weighted;
pub use base::{MetadataToken, MultisigIsmMetadataBuilder, MultisigMetadata};
pub use merkle_root_multisig::MerkleRootMultisigMetadataBuilder;
pub use message_id_multisig::MessageIdMultisigMetadataBuilder;
pub use weighted::{
WeightedMerkleRootMultisigMetadataBuilder, WeightedMessageIdMultisigMetadataBuilder,
};

@ -0,0 +1,108 @@
use super::base::MultisigIsmMetadataBuilder;
use crate::msg::metadata::{
multisig::{
MerkleRootMultisigMetadataBuilder, MessageIdMultisigMetadataBuilder, MetadataToken,
MultisigMetadata,
},
MessageMetadataBuilder,
};
use async_trait::async_trait;
use derive_more::{AsRef, Deref};
use derive_new::new;
use eyre::{Context, Result};
use hyperlane_base::{MultisigCheckpointSyncer, ValidatorWithWeight, Weight};
use hyperlane_core::{HyperlaneMessage, H256};
#[derive(Debug, Clone, Deref, new, AsRef)]
pub struct WeightedMerkleRootMultisigMetadataBuilder(MessageMetadataBuilder);
#[async_trait]
impl MultisigIsmMetadataBuilder for WeightedMerkleRootMultisigMetadataBuilder {
async fn ism_validator_requirements(
&self,
ism_address: H256,
message: &HyperlaneMessage,
) -> Result<(Vec<ValidatorWithWeight>, Weight)> {
const CTX: &str = "When fetching WeightedMultisigIsm metadata";
let weighted_multisig_ism = self
.as_ref()
.build_weighted_multisig_ism(ism_address)
.await
.context(CTX)?;
let (validators, threshold) = weighted_multisig_ism
.validators_and_threshold_weight(message)
.await
.context(CTX)?;
let validators: Vec<ValidatorWithWeight> = validators
.into_iter()
.map(|(validator, weight)| ValidatorWithWeight::new(validator, weight))
.collect();
Ok((validators, threshold))
}
fn token_layout(&self) -> Vec<MetadataToken> {
MerkleRootMultisigMetadataBuilder::new(self.0.clone()).token_layout()
}
async fn fetch_metadata(
&self,
validators: &[ValidatorWithWeight],
threshold_weight: Weight,
message: &HyperlaneMessage,
checkpoint_syncer: &MultisigCheckpointSyncer,
) -> Result<Option<MultisigMetadata>> {
MerkleRootMultisigMetadataBuilder::new(self.0.clone())
.fetch_metadata(validators, threshold_weight, message, checkpoint_syncer)
.await
}
}
#[derive(Debug, Clone, Deref, new, AsRef)]
pub struct WeightedMessageIdMultisigMetadataBuilder(MessageMetadataBuilder);
#[async_trait]
impl MultisigIsmMetadataBuilder for WeightedMessageIdMultisigMetadataBuilder {
async fn ism_validator_requirements(
&self,
ism_address: H256,
message: &HyperlaneMessage,
) -> Result<(Vec<ValidatorWithWeight>, u64)> {
const CTX: &str = "When fetching WeightedMultisigIsm metadata";
let weighted_multisig_ism = self
.as_ref()
.build_weighted_multisig_ism(ism_address)
.await
.context(CTX)?;
let (validators, threshold) = weighted_multisig_ism
.validators_and_threshold_weight(message)
.await
.context(CTX)?;
let validators: Vec<ValidatorWithWeight> = validators
.into_iter()
.map(|(validator, weight)| ValidatorWithWeight::new(validator, weight))
.collect();
Ok((validators, threshold))
}
fn token_layout(&self) -> Vec<MetadataToken> {
MessageIdMultisigMetadataBuilder::new(self.0.clone()).token_layout()
}
async fn fetch_metadata(
&self,
validators: &[ValidatorWithWeight],
threshold_weight: Weight,
message: &HyperlaneMessage,
checkpoint_syncer: &MultisigCheckpointSyncer,
) -> Result<Option<MultisigMetadata>> {
MessageIdMultisigMetadataBuilder::new(self.0.clone())
.fetch_metadata(validators, threshold_weight, message, checkpoint_syncer)
.await
}
}

@ -0,0 +1,75 @@
[
{
"type": "function",
"name": "moduleType",
"inputs": [],
"outputs": [
{
"name": "",
"type": "uint8",
"internalType": "uint8"
}
],
"stateMutability": "view"
},
{
"type": "function",
"name": "validatorsAndThresholdWeight",
"inputs": [
{
"name": "_message",
"type": "bytes",
"internalType": "bytes"
}
],
"outputs": [
{
"name": "validators",
"type": "tuple[]",
"internalType": "struct IStaticWeightedMultisigIsm.ValidatorInfo[]",
"components": [
{
"name": "signingAddress",
"type": "address",
"internalType": "address"
},
{
"name": "weight",
"type": "uint96",
"internalType": "uint96"
}
]
},
{
"name": "thresholdWeight",
"type": "uint96",
"internalType": "uint96"
}
],
"stateMutability": "view"
},
{
"type": "function",
"name": "verify",
"inputs": [
{
"name": "_metadata",
"type": "bytes",
"internalType": "bytes"
},
{
"name": "_message",
"type": "bytes",
"internalType": "bytes"
}
],
"outputs": [
{
"name": "",
"type": "bool",
"internalType": "bool"
}
],
"stateMutability": "nonpayable"
}
]

@ -1,6 +1,6 @@
pub use {
aggregation_ism::*, ccip_read_ism::*, interchain_security_module::*, multisig_ism::*,
routing_ism::*,
routing_ism::*, weighted_multisig_ism::*,
};
mod aggregation_ism;
@ -8,3 +8,4 @@ mod ccip_read_ism;
mod interchain_security_module;
mod multisig_ism;
mod routing_ism;
mod weighted_multisig_ism;

@ -0,0 +1,123 @@
use async_trait::async_trait;
use ethers::providers::Middleware;
use hyperlane_core::{
ChainResult, ContractLocator, HyperlaneAbi, HyperlaneChain, HyperlaneContract, HyperlaneDomain,
HyperlaneMessage, HyperlaneProvider, RawHyperlaneMessage, WeightedMultisigIsm, H256,
};
use std::collections::HashMap;
use std::sync::Arc;
use tracing::instrument;
use crate::{BuildableWithProvider, ConnectionConf, EthereumProvider};
use crate::interfaces::i_static_weighted_multisig_ism::{
IStaticWeightedMultisigIsm as EthereumWeightedMultisigIsmInternal,
ISTATICWEIGHTEDMULTISIGISM_ABI,
};
/// Builder for WeightedMultisigIsm contracts
pub struct WeighedMultisigIsmBuilder {}
#[async_trait]
impl BuildableWithProvider for WeighedMultisigIsmBuilder {
type Output = Box<dyn WeightedMultisigIsm>;
const NEEDS_SIGNER: bool = false;
async fn build_with_provider<M: Middleware + 'static>(
&self,
provider: M,
_conn: &ConnectionConf,
locator: &ContractLocator,
) -> Self::Output {
Box::new(EthereumWeightedMultisigIsm::new(
Arc::new(provider),
locator,
))
}
}
/// A reference to an WeightedMultisigIsm contract on some Ethereum chain
#[derive(Debug)]
pub struct EthereumWeightedMultisigIsm<M>
where
M: Middleware,
{
contract: Arc<EthereumWeightedMultisigIsmInternal<M>>,
domain: HyperlaneDomain,
}
impl<M> EthereumWeightedMultisigIsm<M>
where
M: Middleware + 'static,
{
/// Create a reference to a mailbox at a specific Ethereum address on some
/// chain
pub fn new(provider: Arc<M>, locator: &ContractLocator) -> Self {
Self {
contract: Arc::new(EthereumWeightedMultisigIsmInternal::new(
locator.address,
provider,
)),
domain: locator.domain.clone(),
}
}
}
impl<M> HyperlaneChain for EthereumWeightedMultisigIsm<M>
where
M: Middleware + 'static,
{
fn domain(&self) -> &HyperlaneDomain {
&self.domain
}
fn provider(&self) -> Box<dyn HyperlaneProvider> {
Box::new(EthereumProvider::new(
self.contract.client(),
self.domain.clone(),
))
}
}
impl<M> HyperlaneContract for EthereumWeightedMultisigIsm<M>
where
M: Middleware + 'static,
{
fn address(&self) -> H256 {
self.contract.address().into()
}
}
#[async_trait]
impl<M> WeightedMultisigIsm for EthereumWeightedMultisigIsm<M>
where
M: Middleware + 'static,
{
#[instrument(err)]
async fn validators_and_threshold_weight(
&self,
message: &HyperlaneMessage,
) -> ChainResult<(Vec<(H256, u64)>, u64)> {
let (validator_addresses, threshold) = self
.contract
.validators_and_threshold_weight(RawHyperlaneMessage::from(message).to_vec().into())
.call()
.await?;
let validators: Vec<(H256, u64)> = validator_addresses
.iter()
.map(|x| (H256::from(x.signing_address), x.weight as u64))
.collect();
Ok((validators, threshold as u64))
}
}
/// ABI for WeightedMultisigIsm contracts
pub struct EthereumWeightedMultisigIsmAbi;
impl HyperlaneAbi for EthereumWeightedMultisigIsmAbi {
const SELECTOR_SIZE_BYTES: usize = 4;
fn fn_map() -> HashMap<Vec<u8>, &'static str> {
crate::extract_fn_map(&ISTATICWEIGHTEDMULTISIGISM_ABI)
}
}

@ -11,7 +11,7 @@ use hyperlane_core::{
HyperlaneDomain, HyperlaneDomainProtocol, HyperlaneMessage, HyperlaneProvider, IndexMode,
InterchainGasPaymaster, InterchainGasPayment, InterchainSecurityModule, Mailbox,
MerkleTreeHook, MerkleTreeInsertion, MultisigIsm, RoutingIsm, SequenceAwareIndexer,
ValidatorAnnounce, H256,
ValidatorAnnounce, WeightedMultisigIsm, H256,
};
use hyperlane_cosmos as h_cosmos;
use hyperlane_ethereum::{
@ -663,6 +663,27 @@ impl ChainConf {
.context(ctx)
}
/// Try to convert the chain setting into a Multisig Ism contract
pub async fn build_weighted_multisig_ism(
&self,
address: H256,
metrics: &CoreMetrics,
) -> Result<Box<dyn WeightedMultisigIsm>> {
let ctx = "Building weighed multisig ISM";
let locator = self.locator(address);
match &self.connection {
ChainConnectionConf::Ethereum(conf) => {
self.build_ethereum(conf, &locator, metrics, h_eth::WeighedMultisigIsmBuilder {})
.await
}
ChainConnectionConf::Fuel(_) => todo!(),
ChainConnectionConf::Sealevel(_) => todo!(),
ChainConnectionConf::Cosmos(_) => todo!(),
}
.context(ctx)
}
async fn signer<S: BuildableWithSignerConf>(&self) -> Result<Option<S>> {
if let Some(conf) = &self.signer {
Ok(Some(conf.build::<S>().await?))

@ -1,3 +1,4 @@
use itertools::Itertools;
use std::collections::HashMap;
use std::sync::Arc;
@ -10,6 +11,18 @@ use hyperlane_core::{
};
use crate::{CheckpointSyncer, CoreMetrics};
/// Weights are scaled by 1e10 as 100%
pub type Weight = u64;
/// Struct for representing both weighted and unweighted types
/// for unweighted, we have (validator, 1), threshold_weight = threshold
/// for weighted, we have (validator, weight)
#[derive(Debug, Clone, Copy, PartialEq, Eq, new)]
pub struct ValidatorWithWeight {
/// The validator's address
pub validator: H256,
/// The validator's weight
pub weight: Weight,
}
/// For a particular validator set, fetches signed checkpoints from multiple
/// validators to create MultisigSignedCheckpoints.
@ -90,15 +103,16 @@ impl MultisigCheckpointSyncer {
#[instrument(err, skip(self))]
pub async fn fetch_checkpoint_in_range(
&self,
validators: &[H256],
threshold: usize,
weighted_validators: &[ValidatorWithWeight],
threshold_weight: Weight,
minimum_index: u32,
maximum_index: u32,
origin: &HyperlaneDomain,
destination: &HyperlaneDomain,
) -> Result<Option<MultisigSignedCheckpoint>> {
let validators: Vec<H256> = weighted_validators.iter().map(|vw| vw.validator).collect();
let mut latest_indices = self
.get_validator_latest_checkpoints_and_update_metrics(validators, origin, destination)
.get_validator_latest_checkpoints_and_update_metrics(&validators, origin, destination)
.await;
debug!(
@ -114,7 +128,19 @@ impl MultisigCheckpointSyncer {
// Sort in descending order. The n'th index will represent
// the highest index for which we (supposedly) have (n+1) signed checkpoints
latest_indices.sort_by(|a, b| b.cmp(a));
if let Some(&highest_quorum_index) = latest_indices.get(threshold - 1) {
// Find the highest index that meets the threshold weight
let mut cumulative_weight = 0;
let mut highest_quorum_index = None;
for (i, &index) in latest_indices.iter().enumerate() {
cumulative_weight += weighted_validators[i].weight; // Add the weight of this validator
if cumulative_weight >= threshold_weight {
highest_quorum_index = Some(index);
break;
}
}
if let Some(highest_quorum_index) = highest_quorum_index {
// The highest viable checkpoint index is the minimum of the highest index
// we (supposedly) have a quorum for, and the maximum index for which we can
// generate a proof.
@ -124,8 +150,9 @@ impl MultisigCheckpointSyncer {
return Ok(None);
}
for index in (minimum_index..=start_index).rev() {
if let Ok(Some(checkpoint)) =
self.fetch_checkpoint(validators, threshold, index).await
if let Ok(Some(checkpoint)) = self
.fetch_checkpoint(weighted_validators, threshold_weight, index)
.await
{
return Ok(Some(checkpoint));
}
@ -141,8 +168,8 @@ impl MultisigCheckpointSyncer {
#[instrument(err, skip(self))]
pub async fn fetch_checkpoint(
&self,
validators: &[H256],
threshold: usize,
weighted_validators: &[ValidatorWithWeight],
threshold_weight: Weight,
index: u32,
) -> Result<Option<MultisigSignedCheckpoint>> {
// Keeps track of signed validator checkpoints for a particular root.
@ -151,8 +178,15 @@ impl MultisigCheckpointSyncer {
let mut signed_checkpoints_per_root: HashMap<H256, Vec<SignedCheckpointWithMessageId>> =
HashMap::new();
for validator in validators.iter() {
let addr = H160::from(*validator);
let sorted_validators: Vec<(usize, &ValidatorWithWeight)> = weighted_validators
.iter()
.enumerate()
.sorted_by_key(|(_, vw)| std::cmp::Reverse(vw.weight))
.collect();
let mut cumulative_weight: u64 = 0;
for (_, vw) in sorted_validators {
let addr = H160::from(vw.validator);
if let Some(checkpoint_syncer) = self.checkpoint_syncers.get(&addr) {
// Gracefully ignore an error fetching the checkpoint from a validator's
// checkpoint syncer, which can happen if the validator has not
@ -162,7 +196,7 @@ impl MultisigCheckpointSyncer {
// If the signed checkpoint is for a different index, ignore it
if signed_checkpoint.value.index != index {
debug!(
validator = format!("{:#x}", validator),
validator = format!("{:#x}", vw.validator),
index = index,
checkpoint_index = signed_checkpoint.value.index,
"Checkpoint index mismatch"
@ -173,9 +207,9 @@ impl MultisigCheckpointSyncer {
// Ensure that the signature is actually by the validator
let signer = signed_checkpoint.recover()?;
if H256::from(signer) != *validator {
if H256::from(signer) != vw.validator {
debug!(
validator = format!("{:#x}", validator),
validator = format!("{:#x}", vw.validator),
index = index,
"Checkpoint signature mismatch"
);
@ -186,32 +220,40 @@ impl MultisigCheckpointSyncer {
let root = signed_checkpoint.value.root;
let signed_checkpoints = signed_checkpoints_per_root.entry(root).or_default();
signed_checkpoints.push(signed_checkpoint);
cumulative_weight += vw.weight;
// Count the number of signatures for this signed checkpoint
let signature_count = signed_checkpoints.len();
debug!(
validator = format!("{:#x}", validator),
validator = format!("{:#x}", vw.validator),
index = index,
root = format!("{:#x}", root),
signature_count = signature_count,
"Found signed checkpoint"
);
// If we've hit a quorum, create a MultisigSignedCheckpoint
if signature_count >= threshold {
// If we've hit a quorum in weight, create a MultisigSignedCheckpoint
if cumulative_weight >= threshold_weight {
// to conform to the onchain ordering of the set by address
signed_checkpoints.sort_by_key(|sc| {
weighted_validators
.iter()
.position(|vw| vw.validator == H256::from(sc.recover().unwrap()))
.unwrap()
});
let checkpoint: MultisigSignedCheckpoint = signed_checkpoints.try_into()?;
debug!(checkpoint=?checkpoint, "Fetched multisig checkpoint");
return Ok(Some(checkpoint));
}
} else {
debug!(
validator = format!("{:#x}", validator),
validator = format!("{:#x}", vw.validator),
index = index,
"Unable to find signed checkpoint"
);
}
} else {
debug!(%validator, "Unable to find checkpoint syncer");
debug!(%vw.validator, "Unable to find checkpoint syncer");
continue;
}
}

@ -41,6 +41,12 @@ pub enum ModuleType {
Null,
/// Ccip Read ISM (accepts offchain signature information)
CcipRead,
/// Arbitrum L2 to L1 ISM - unimplemented
ArbL2ToL1,
/// Weighted Merkle Proof ISM (similar to Merkle Proof ISM but with each validator having a specific weight)
WeightedMerkleRootMultisig,
/// Weighted Message ID ISM (similar to Message ID ISM but with each validator having a specific weight)
WeightedMessageIdMultisig,
}
/// Interface for the InterchainSecurityModule chain contract. Allows abstraction over

@ -16,3 +16,15 @@ pub trait MultisigIsm: HyperlaneContract + Send + Sync + Debug {
message: &HyperlaneMessage,
) -> ChainResult<(Vec<H256>, u8)>;
}
/// Interface for the WeightedMultisigIsm chain contract. Allows abstraction over
/// different chains
#[async_trait]
#[auto_impl(&, Box, Arc)]
pub trait WeightedMultisigIsm: HyperlaneContract + Send + Sync + Debug {
/// Returns the validator and threshold needed to verify message
async fn validators_and_threshold_weight(
&self,
message: &HyperlaneMessage,
) -> ChainResult<(Vec<(H256, u64)>, u64)>;
}

@ -1,6 +1,10 @@
import { AggregationIsmConfig, IsmType } from '@hyperlane-xyz/sdk';
import { merkleRootMultisig, messageIdMultisig } from './multisigIsm.js';
import {
merkleRootMultisig,
messageIdMultisig,
uniformlyWeightedMultisigIsm,
} from './multisigIsm.js';
export const aggregationIsm = (validatorKey: string): AggregationIsmConfig => {
return {
@ -8,7 +12,9 @@ export const aggregationIsm = (validatorKey: string): AggregationIsmConfig => {
modules: [
merkleRootMultisig(validatorKey),
messageIdMultisig(validatorKey),
uniformlyWeightedMultisigIsm(merkleRootMultisig(validatorKey)),
uniformlyWeightedMultisigIsm(messageIdMultisig(validatorKey)),
],
threshold: 2,
threshold: 4,
};
};

Loading…
Cancel
Save