Skip to content

Commit

Permalink
feat: Make HyperlaneProvider to request block by height, not by hash (#…
Browse files Browse the repository at this point in the history
…4727)

### Description

Make HyperlaneProvider to request block by height, not by hash.

Since it is hard to find a block by hash for Solana, we switch to use
block height. We have to do it for all type of chains since we need to
change signature of chain-agnostic method

### Drive-by changes

* Small method rename

### Related issues

- Contributes into
#4272

### Backward compatibility

Yes

### Testing

Manual run of Scraper for Ethereum and Neutron chains.

---------

Co-authored-by: Danil Nemirovsky <4614623+ameten@users.noreply.github.com>
  • Loading branch information
ameten and ameten authored Oct 24, 2024
1 parent a64af8b commit ffbe1dd
Show file tree
Hide file tree
Showing 12 changed files with 181 additions and 85 deletions.
32 changes: 21 additions & 11 deletions rust/main/agents/scraper/src/chain_scraper/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@ use async_trait::async_trait;
use eyre::Result;
use hyperlane_base::settings::IndexSettings;
use hyperlane_core::{
unwrap_or_none_result, BlockInfo, Delivery, HyperlaneDomain, HyperlaneLogStore,
unwrap_or_none_result, BlockId, BlockInfo, Delivery, HyperlaneDomain, HyperlaneLogStore,
HyperlaneMessage, HyperlaneProvider, HyperlaneSequenceAwareIndexerStoreReader,
HyperlaneWatermarkedLogStore, Indexed, InterchainGasPayment, LogMeta, H256,
};
Expand Down Expand Up @@ -78,30 +78,30 @@ impl HyperlaneSqlDb {
&self,
log_meta: impl Iterator<Item = &LogMeta>,
) -> Result<impl Iterator<Item = TxnWithId>> {
let block_hash_by_txn_hash: HashMap<H256, H256> = log_meta
let block_id_by_txn_hash: HashMap<H256, BlockId> = log_meta
.map(|meta| {
(
meta.transaction_id
.try_into()
.expect("256-bit transaction ids are the maximum supported at this time"),
meta.block_hash,
BlockId::new(meta.block_hash, meta.block_number),
)
})
.collect();

// all blocks we care about
// hash of block maps to the block id and timestamp
let blocks: HashMap<_, _> = self
.ensure_blocks(block_hash_by_txn_hash.values().copied())
.ensure_blocks(block_id_by_txn_hash.values().copied())
.await?
.map(|block| (block.hash, block))
.collect();
trace!(?blocks, "Ensured blocks");

// We ensure transactions only from blocks which are inserted into database
let txn_hash_with_block_ids = block_hash_by_txn_hash
let txn_hash_with_block_ids = block_id_by_txn_hash
.into_iter()
.filter_map(move |(txn, block)| blocks.get(&block).map(|b| (txn, b.id)))
.filter_map(move |(txn, block)| blocks.get(&block.hash).map(|b| (txn, b.id)))
.map(|(txn_hash, block_id)| TxnWithBlockId { txn_hash, block_id });
let txns_with_ids = self.ensure_txns(txn_hash_with_block_ids).await?;

Expand Down Expand Up @@ -195,11 +195,17 @@ impl HyperlaneSqlDb {
/// this method.
async fn ensure_blocks(
&self,
block_hashes: impl Iterator<Item = H256>,
block_ids: impl Iterator<Item = BlockId>,
) -> Result<impl Iterator<Item = BasicBlock>> {
// Mapping from block hash to block ids (hash and height)
let block_hash_to_block_id_map: HashMap<H256, BlockId> =
block_ids.map(|b| (b.hash, b)).collect();

// Mapping of block hash to `BasicBlock` which contains database block id and block hash.
let mut blocks: HashMap<H256, Option<BasicBlock>> =
block_hashes.map(|b| (b, None)).collect();
let mut blocks: HashMap<H256, Option<BasicBlock>> = block_hash_to_block_id_map
.keys()
.map(|hash| (*hash, None))
.collect();

let db_blocks: Vec<BasicBlock> = if !blocks.is_empty() {
// check database to see which blocks we already know and fetch their IDs
Expand Down Expand Up @@ -230,10 +236,14 @@ impl HyperlaneSqlDb {
for chunk in as_chunks(blocks_to_fetch, CHUNK_SIZE) {
debug_assert!(!chunk.is_empty());
for (hash, block_info) in chunk {
let info = match self.provider.get_block_by_hash(hash).await {
// We should have block_id in this map for every hashes
let block_id = block_hash_to_block_id_map[hash];
let block_height = block_id.height;

let info = match self.provider.get_block_by_height(block_height).await {
Ok(info) => info,
Err(e) => {
warn!(?hash, ?e, "error fetching and parsing block");
warn!(block_hash = ?hash, ?block_height, ?e, "error fetching and parsing block");
continue;
}
};
Expand Down
33 changes: 13 additions & 20 deletions rust/main/chains/hyperlane-cosmos/src/providers/cosmos/provider.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,8 @@ use tracing::{error, warn};
use crypto::decompress_public_key;
use hyperlane_core::{
AccountAddressType, BlockInfo, ChainCommunicationError, ChainInfo, ChainResult,
ContractLocator, HyperlaneChain, HyperlaneDomain, HyperlaneProvider, TxnInfo, TxnReceiptInfo,
H256, U256,
ContractLocator, HyperlaneChain, HyperlaneDomain, HyperlaneProvider, HyperlaneProviderError,
TxnInfo, TxnReceiptInfo, H256, U256,
};

use crate::grpc::{WasmGrpcProvider, WasmProvider};
Expand Down Expand Up @@ -367,33 +367,26 @@ impl HyperlaneChain for CosmosProvider {

#[async_trait]
impl HyperlaneProvider for CosmosProvider {
async fn get_block_by_hash(&self, hash: &H256) -> ChainResult<BlockInfo> {
let tendermint_hash = Hash::from_bytes(Algorithm::Sha256, hash.as_bytes())
.expect("block hash should be of correct size");

let response = self.rpc_client.get_block_by_hash(tendermint_hash).await?;
async fn get_block_by_height(&self, height: u64) -> ChainResult<BlockInfo> {
let response = self.rpc_client.get_block(height as u32).await?;

let received_hash = H256::from_slice(response.block_id.hash.as_bytes());
let block = response.block;
let block_height = block.header.height.value();

if &received_hash != hash {
return Err(ChainCommunicationError::from_other_str(
&format!("received incorrect block, expected hash: {hash:?}, received hash: {received_hash:?}")
));
if block_height != height {
Err(HyperlaneProviderError::IncorrectBlockByHeight(
height,
block_height,
))?
}

let block = response.block.ok_or_else(|| {
ChainCommunicationError::from_other_str(&format!(
"empty block info for block: {:?}",
hash
))
})?;

let hash = H256::from_slice(response.block_id.hash.as_bytes());
let time: OffsetDateTime = block.header.time.into();

let block_info = BlockInfo {
hash: hash.to_owned(),
timestamp: time.unix_timestamp() as u64,
number: block.header.height.value(),
number: block_height,
};

Ok(block_info)
Expand Down
65 changes: 45 additions & 20 deletions rust/main/chains/hyperlane-ethereum/src/rpc_clients/provider.rs
Original file line number Diff line number Diff line change
Expand Up @@ -49,26 +49,49 @@ where
{
#[instrument(err, skip(self))]
#[allow(clippy::blocks_in_conditions)] // TODO: `rustc` 1.80.1 clippy issue
async fn get_block_by_hash(&self, hash: &H256) -> ChainResult<BlockInfo> {
let block = get_with_retry_on_none(hash, |h| {
let eth_h256: ethers_core_types::H256 = h.into();
self.provider.get_block(eth_h256)
})
async fn get_block_by_height(&self, height: u64) -> ChainResult<BlockInfo> {
let block = get_with_retry_on_none(
&height,
|h| self.provider.get_block(*h),
|h| HyperlaneProviderError::CouldNotFindBlockByHeight(*h),
)
.await?;
Ok(BlockInfo {
hash: *hash,

let block_height = block
.number
.ok_or(HyperlaneProviderError::CouldNotFindBlockByHeight(height))?
.as_u64();

if block_height != height {
Err(HyperlaneProviderError::IncorrectBlockByHeight(
height,
block_height,
))?;
}

let block_hash = block
.hash
.ok_or(HyperlaneProviderError::BlockWithoutHash(height))?;

let block_info = BlockInfo {
hash: block_hash.into(),
timestamp: block.timestamp.as_u64(),
number: block
.number
.ok_or(HyperlaneProviderError::BlockIsNotPartOfChainYet(*hash))?
.as_u64(),
})
number: block_height,
};

Ok(block_info)
}

#[instrument(err, skip(self))]
#[allow(clippy::blocks_in_conditions)] // TODO: `rustc` 1.80.1 clippy issue
async fn get_txn_by_hash(&self, hash: &H256) -> ChainResult<TxnInfo> {
let txn = get_with_retry_on_none(hash, |h| self.provider.get_transaction(*h)).await?;
let txn = get_with_retry_on_none(
hash,
|h| self.provider.get_transaction(*h),
|h| HyperlaneProviderError::CouldNotFindTransactionByHash(*h),
)
.await?;

let receipt = self
.provider
.get_transaction_receipt(*hash)
Expand Down Expand Up @@ -193,22 +216,24 @@ impl BuildableWithProvider for HyperlaneProviderBuilder {
/// Call a get function that returns a Result<Option<T>> and retry if the inner
/// option is None. This can happen because the provider has not discovered the
/// object we are looking for yet.
async fn get_with_retry_on_none<T, F, O, E>(hash: &H256, get: F) -> ChainResult<T>
async fn get_with_retry_on_none<T, F, O, E, I, N>(
id: &I,
get: F,
not_found_error: N,
) -> ChainResult<T>
where
F: Fn(&H256) -> O,
F: Fn(&I) -> O,
O: Future<Output = Result<Option<T>, E>>,
E: std::error::Error + Send + Sync + 'static,
N: Fn(&I) -> HyperlaneProviderError,
{
for _ in 0..3 {
if let Some(t) = get(hash)
.await
.map_err(ChainCommunicationError::from_other)?
{
if let Some(t) = get(id).await.map_err(ChainCommunicationError::from_other)? {
return Ok(t);
} else {
sleep(Duration::from_secs(5)).await;
continue;
};
}
Err(HyperlaneProviderError::CouldNotFindObjectByHash(*hash).into())
Err(not_found_error(id).into())
}
35 changes: 23 additions & 12 deletions rust/main/chains/hyperlane-fuel/src/provider.rs
Original file line number Diff line number Diff line change
@@ -1,7 +1,6 @@
use std::{collections::HashMap, ops::Deref};

use async_trait::async_trait;

use fuels::{
client::{FuelClient, PageDirection, PaginationRequest},
prelude::Provider,
Expand All @@ -13,13 +12,14 @@ use fuels::{
transaction::{Transaction, TransactionType},
transaction_response::TransactionResponse,
tx_status::TxStatus,
Address, Bytes32, ContractId,
Address, BlockHeight, Bytes32, ContractId,
},
};
use futures::future::join_all;
use hyperlane_core::{
BlockInfo, ChainCommunicationError, ChainInfo, ChainResult, HyperlaneChain, HyperlaneDomain,
HyperlaneMessage, HyperlaneProvider, Indexed, LogMeta, TxnInfo, H256, H512, U256,
HyperlaneMessage, HyperlaneProvider, HyperlaneProviderError, Indexed, LogMeta, TxnInfo, H256,
H512, U256,
};

use crate::{make_client, make_provider, prelude::FuelIntoH256, ConnectionConf};
Expand Down Expand Up @@ -285,19 +285,30 @@ impl HyperlaneChain for FuelProvider {
impl HyperlaneProvider for FuelProvider {
/// Used by scraper
#[allow(clippy::clone_on_copy)] // TODO: `rustc` 1.80.1 clippy issue
async fn get_block_by_hash(&self, hash: &H256) -> ChainResult<BlockInfo> {
let block_res = self.provider.block(&hash.0.into()).await.map_err(|e| {
ChainCommunicationError::CustomError(format!("Failed to get block: {}", e))
})?;
async fn get_block_by_height(&self, height: u64) -> ChainResult<BlockInfo> {
let block_res = self
.provider
.block_by_height(BlockHeight::new(height as u32))
.await
.map_err(|e| HyperlaneProviderError::CouldNotFindBlockByHeight(height))?;

match block_res {
Some(block) => Ok(BlockInfo {
let block_info = match block_res {
Some(block) => BlockInfo {
hash: H256::from_slice(block.id.as_slice()),
number: block.header.height.into(),
timestamp: block.header.time.map_or(0, |t| t.timestamp() as u64),
}),
None => Err(ChainCommunicationError::BlockNotFound(hash.clone())),
number: block.header.height.into(),
},
None => Err(HyperlaneProviderError::CouldNotFindBlockByHeight(height))?,
};

if block_info.number != height {
Err(HyperlaneProviderError::IncorrectBlockByHeight(
height,
block_info.number,
))?;
}

Ok(block_info)
}

/// Used by scraper
Expand Down
3 changes: 3 additions & 0 deletions rust/main/chains/hyperlane-sealevel/src/error.rs
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,9 @@ pub enum HyperlaneSealevelError {
/// ClientError error
#[error("{0}")]
ClientError(#[from] ClientError),
/// Decoding error
#[error("{0}")]
Decoding(#[from] solana_sdk::bs58::decode::Error),
}

impl From<HyperlaneSealevelError> for ChainCommunicationError {
Expand Down
2 changes: 1 addition & 1 deletion rust/main/chains/hyperlane-sealevel/src/mailbox.rs
Original file line number Diff line number Diff line change
Expand Up @@ -430,7 +430,7 @@ impl Mailbox for SealevelMailbox {

let account = self
.rpc()
.get_possible_account_with_finalized_commitment(&processed_message_account_key)
.get_account_option_with_finalized_commitment(&processed_message_account_key)
.await?;

Ok(account.is_some())
Expand Down
27 changes: 22 additions & 5 deletions rust/main/chains/hyperlane-sealevel/src/provider.rs
Original file line number Diff line number Diff line change
@@ -1,11 +1,11 @@
use std::{str::FromStr, sync::Arc};

use async_trait::async_trait;

use hyperlane_core::{
BlockInfo, ChainInfo, ChainResult, HyperlaneChain, HyperlaneDomain, HyperlaneProvider, TxnInfo,
H256, U256,
BlockInfo, ChainInfo, ChainResult, HyperlaneChain, HyperlaneDomain, HyperlaneProvider,
HyperlaneProviderError, TxnInfo, H256, U256,
};
use solana_sdk::bs58;
use solana_sdk::pubkey::Pubkey;

use crate::{error::HyperlaneSealevelError, ConnectionConf, SealevelRpcClient};
Expand Down Expand Up @@ -47,8 +47,25 @@ impl HyperlaneChain for SealevelProvider {

#[async_trait]
impl HyperlaneProvider for SealevelProvider {
async fn get_block_by_hash(&self, _hash: &H256) -> ChainResult<BlockInfo> {
todo!() // FIXME
async fn get_block_by_height(&self, slot: u64) -> ChainResult<BlockInfo> {
let confirmed_block = self.rpc_client.get_block(slot).await?;

let hash_binary = bs58::decode(confirmed_block.blockhash)
.into_vec()
.map_err(HyperlaneSealevelError::Decoding)?;
let block_hash = H256::from_slice(&hash_binary);

let block_time = confirmed_block
.block_time
.ok_or(HyperlaneProviderError::CouldNotFindBlockByHeight(slot))?;

let block_info = BlockInfo {
hash: block_hash,
timestamp: block_time as u64,
number: slot,
};

Ok(block_info)
}

async fn get_txn_by_hash(&self, _hash: &H256) -> ChainResult<TxnInfo> {
Expand Down
Loading

0 comments on commit ffbe1dd

Please sign in to comment.