diff --git a/chain/arweave/src/chain.rs b/chain/arweave/src/chain.rs index 8d40408a463..195abc09994 100644 --- a/chain/arweave/src/chain.rs +++ b/chain/arweave/src/chain.rs @@ -3,11 +3,11 @@ use graph::blockchain::client::ChainClient; use graph::blockchain::firehose_block_ingestor::FirehoseBlockIngestor; use graph::blockchain::{ BasicBlockchainBuilder, Block, BlockIngestor, BlockchainBuilder, BlockchainKind, - EmptyNodeCapabilities, NoopDecoderHook, NoopRuntimeAdapter, + EmptyNodeCapabilities, NoopDecoderHook, NoopRuntimeAdapter, TriggerFilterWrapper, }; use graph::cheap_clone::CheapClone; use graph::components::adapter::ChainId; -use graph::components::store::DeploymentCursorTracker; +use graph::components::store::{DeploymentCursorTracker, SourceableStore}; use graph::data::subgraph::UnifiedMappingApiVersion; use graph::env::EnvVars; use graph::firehose::FirehoseEndpoint; @@ -27,11 +27,13 @@ use graph::{ prelude::{async_trait, o, BlockNumber, ChainStore, Error, Logger, LoggerFactory}, }; use prost::Message; +use std::collections::HashSet; use std::sync::Arc; use crate::adapter::TriggerFilter; use crate::data_source::{DataSourceTemplate, UnresolvedDataSourceTemplate}; use crate::trigger::{self, ArweaveTrigger}; +use crate::Block as ArweaveBlock; use crate::{ codec, data_source::{DataSource, UnresolvedDataSource}, @@ -119,7 +121,8 @@ impl Blockchain for Chain { deployment: DeploymentLocator, store: impl DeploymentCursorTracker, start_blocks: Vec, - filter: Arc, + _source_subgraph_stores: Vec>, + filter: Arc>, unified_api_version: UnifiedMappingApiVersion, ) -> Result>, Error> { let adapter = self @@ -135,7 +138,10 @@ impl Blockchain for Chain { .subgraph_logger(&deployment) .new(o!("component" => "FirehoseBlockStream")); - let firehose_mapper = Arc::new(FirehoseMapper { adapter, filter }); + let firehose_mapper = Arc::new(FirehoseMapper { + adapter, + filter: filter.chain_filter.clone(), + }); Ok(Box::new(FirehoseBlockStream::new( deployment.hash, @@ -199,6 +205,10 @@ impl TriggersAdapterTrait for TriggersAdapter { panic!("Should never be called since not used by FirehoseBlockStream") } + async fn chain_head_ptr(&self) -> Result, Error> { + unimplemented!() + } + async fn triggers_in_block( &self, logger: &Logger, @@ -258,6 +268,14 @@ impl TriggersAdapterTrait for TriggersAdapter { number: block.number.saturating_sub(1), })) } + + async fn load_block_ptrs_by_numbers( + &self, + _logger: Logger, + _block_numbers: HashSet, + ) -> Result, Error> { + todo!() + } } pub struct FirehoseMapper { diff --git a/chain/cosmos/src/chain.rs b/chain/cosmos/src/chain.rs index 955aa7efc3c..0f07c3aaaf9 100644 --- a/chain/cosmos/src/chain.rs +++ b/chain/cosmos/src/chain.rs @@ -1,9 +1,10 @@ use graph::blockchain::firehose_block_ingestor::FirehoseBlockIngestor; -use graph::blockchain::{BlockIngestor, NoopDecoderHook}; +use graph::blockchain::{BlockIngestor, NoopDecoderHook, TriggerFilterWrapper}; use graph::components::adapter::ChainId; use graph::env::EnvVars; use graph::prelude::MetricsRegistry; use graph::substreams::Clock; +use std::collections::HashSet; use std::convert::TryFrom; use std::sync::Arc; @@ -11,7 +12,7 @@ use graph::blockchain::block_stream::{BlockStreamError, BlockStreamMapper, Fireh use graph::blockchain::client::ChainClient; use graph::blockchain::{BasicBlockchainBuilder, BlockchainBuilder, NoopRuntimeAdapter}; use graph::cheap_clone::CheapClone; -use graph::components::store::DeploymentCursorTracker; +use graph::components::store::{DeploymentCursorTracker, SourceableStore}; use graph::data::subgraph::UnifiedMappingApiVersion; use graph::{ blockchain::{ @@ -33,7 +34,7 @@ use crate::data_source::{ DataSource, DataSourceTemplate, EventOrigin, UnresolvedDataSource, UnresolvedDataSourceTemplate, }; use crate::trigger::CosmosTrigger; -use crate::{codec, TriggerFilter}; +use crate::{codec, Block, TriggerFilter}; pub struct Chain { logger_factory: LoggerFactory, @@ -113,7 +114,8 @@ impl Blockchain for Chain { deployment: DeploymentLocator, store: impl DeploymentCursorTracker, start_blocks: Vec, - filter: Arc, + _source_subgraph_stores: Vec>, + filter: Arc>, unified_api_version: UnifiedMappingApiVersion, ) -> Result>, Error> { let adapter = self @@ -129,7 +131,10 @@ impl Blockchain for Chain { .subgraph_logger(&deployment) .new(o!("component" => "FirehoseBlockStream")); - let firehose_mapper = Arc::new(FirehoseMapper { adapter, filter }); + let firehose_mapper = Arc::new(FirehoseMapper { + adapter, + filter: filter.chain_filter.clone(), + }); Ok(Box::new(FirehoseBlockStream::new( deployment.hash, @@ -192,6 +197,16 @@ impl TriggersAdapterTrait for TriggersAdapter { ) -> Result, Error> { panic!("Should never be called since not used by FirehoseBlockStream") } + async fn load_block_ptrs_by_numbers( + &self, + _logger: Logger, + _block_numbers: HashSet, + ) -> Result, Error> { + todo!() + } + async fn chain_head_ptr(&self) -> Result, Error> { + unimplemented!() + } async fn scan_triggers( &self, @@ -467,9 +482,12 @@ impl FirehoseMapperTrait for FirehoseMapper { #[cfg(test)] mod test { - use graph::prelude::{ - slog::{o, Discard, Logger}, - tokio, + use graph::{ + blockchain::Trigger, + prelude::{ + slog::{o, Discard, Logger}, + tokio, + }, }; use super::*; @@ -600,7 +618,10 @@ mod test { // they may not be in the same order for trigger in expected_triggers { assert!( - triggers.trigger_data.contains(&trigger), + triggers.trigger_data.iter().any(|t| match t { + Trigger::Chain(t) => t == &trigger, + _ => false, + }), "Expected trigger list to contain {:?}, but it only contains: {:?}", trigger, triggers.trigger_data diff --git a/chain/ethereum/src/adapter.rs b/chain/ethereum/src/adapter.rs index f78ff1b0bec..469e8932b5e 100644 --- a/chain/ethereum/src/adapter.rs +++ b/chain/ethereum/src/adapter.rs @@ -1,8 +1,9 @@ use anyhow::Error; -use ethabi::{Error as ABIError, Function, ParamType, Token}; +use ethabi::{Error as ABIError, ParamType, Token}; use graph::blockchain::ChainIdentifier; use graph::components::subgraph::MappingError; use graph::data::store::ethereum::call; +use graph::data_source::common::ContractCall; use graph::firehose::CallToFilter; use graph::firehose::CombinedFilter; use graph::firehose::LogFilter; @@ -93,16 +94,6 @@ impl EventSignatureWithTopics { } } -#[derive(Clone, Debug)] -pub struct ContractCall { - pub contract_name: String, - pub address: Address, - pub block_ptr: BlockPtr, - pub function: Function, - pub args: Vec, - pub gas: Option, -} - #[derive(Error, Debug)] pub enum EthereumRpcError { #[error("call error: {0}")] diff --git a/chain/ethereum/src/chain.rs b/chain/ethereum/src/chain.rs index 1def8c483cc..24486f9cf3a 100644 --- a/chain/ethereum/src/chain.rs +++ b/chain/ethereum/src/chain.rs @@ -3,18 +3,21 @@ use anyhow::{Context, Error}; use graph::blockchain::client::ChainClient; use graph::blockchain::firehose_block_ingestor::{FirehoseBlockIngestor, Transforms}; use graph::blockchain::{ - BlockIngestor, BlockTime, BlockchainKind, ChainIdentifier, TriggersAdapterSelector, + BlockIngestor, BlockTime, BlockchainKind, ChainIdentifier, ExtendedBlockPtr, + TriggerFilterWrapper, TriggersAdapterSelector, }; use graph::components::adapter::ChainId; -use graph::components::store::DeploymentCursorTracker; +use graph::components::store::{DeploymentCursorTracker, SourceableStore}; use graph::data::subgraph::UnifiedMappingApiVersion; use graph::firehose::{FirehoseEndpoint, ForkStep}; use graph::futures03::compat::Future01CompatExt; +use graph::futures03::TryStreamExt; use graph::prelude::{ BlockHash, ComponentLoggerConfig, ElasticComponentLoggerConfig, EthereumBlock, EthereumCallCache, LightEthereumBlock, LightEthereumBlockExt, MetricsRegistry, }; use graph::schema::InputSchema; +use graph::slog::{debug, error, trace}; use graph::substreams::Clock; use graph::{ blockchain::{ @@ -37,6 +40,7 @@ use graph::{ }; use prost::Message; use std::collections::HashSet; +use std::future::Future; use std::iter::FromIterator; use std::sync::Arc; use std::time::Duration; @@ -61,6 +65,7 @@ use crate::{BufferedCallCache, NodeCapabilities}; use crate::{EthereumAdapter, RuntimeAdapter}; use graph::blockchain::block_stream::{ BlockStream, BlockStreamBuilder, BlockStreamError, BlockStreamMapper, FirehoseCursor, + TriggersAdapterWrapper, }; /// Celo Mainnet: 42220, Testnet Alfajores: 44787, Testnet Baklava: 62320 @@ -121,24 +126,51 @@ impl BlockStreamBuilder for EthereumStreamBuilder { unimplemented!() } + async fn build_subgraph_block_stream( + &self, + chain: &Chain, + deployment: DeploymentLocator, + start_blocks: Vec, + source_subgraph_stores: Vec>, + subgraph_current_block: Option, + filter: Arc>, + unified_api_version: UnifiedMappingApiVersion, + ) -> Result>> { + self.build_polling( + chain, + deployment, + start_blocks, + source_subgraph_stores, + subgraph_current_block, + filter, + unified_api_version, + ) + .await + } + async fn build_polling( &self, chain: &Chain, deployment: DeploymentLocator, start_blocks: Vec, + source_subgraph_stores: Vec>, subgraph_current_block: Option, - filter: Arc<::TriggerFilter>, + filter: Arc>, unified_api_version: UnifiedMappingApiVersion, ) -> Result>> { - let requirements = filter.node_capabilities(); - let adapter = chain - .triggers_adapter(&deployment, &requirements, unified_api_version.clone()) - .unwrap_or_else(|_| { - panic!( - "no adapter for network {} with capabilities {}", - chain.name, requirements - ) - }); + let requirements = filter.chain_filter.node_capabilities(); + let is_using_subgraph_composition = !source_subgraph_stores.is_empty(); + let adapter = TriggersAdapterWrapper::new( + chain + .triggers_adapter(&deployment, &requirements, unified_api_version.clone()) + .unwrap_or_else(|_| { + panic!( + "no adapter for network {} with capabilities {}", + chain.name, requirements + ) + }), + source_subgraph_stores, + ); let logger = chain .logger_factory @@ -153,33 +185,45 @@ impl BlockStreamBuilder for EthereumStreamBuilder { // This is ok because Celo blocks are always final. And we _need_ to do this because // some events appear only in eth_getLogs but not in transaction receipts. // See also ca0edc58-0ec5-4c89-a7dd-2241797f5e50. - let chain_id = match chain.chain_client().as_ref() { + let reorg_threshold = match chain.chain_client().as_ref() { ChainClient::Rpc(adapter) => { - adapter + let chain_id = adapter .cheapest() .await .ok_or(anyhow!("unable to get eth adapter for chan_id call"))? .chain_id() - .await? + .await?; + + if CELO_CHAIN_IDS.contains(&chain_id) { + 0 + } else { + chain.reorg_threshold + } } - _ => panic!("expected rpc when using polling blockstream"), + _ if is_using_subgraph_composition => chain.reorg_threshold, + _ => panic!( + "expected rpc when using polling blockstream : {}", + is_using_subgraph_composition + ), }; - let reorg_threshold = match CELO_CHAIN_IDS.contains(&chain_id) { - false => chain.reorg_threshold, - true => 0, + + let max_block_range_size = if is_using_subgraph_composition { + ENV_VARS.max_block_range_size * 10 + } else { + ENV_VARS.max_block_range_size }; Ok(Box::new(PollingBlockStream::new( chain_store, chain_head_update_stream, - adapter, + Arc::new(adapter), chain.node_id.clone(), deployment.hash, filter, start_blocks, reorg_threshold, logger, - ENV_VARS.max_block_range_size, + max_block_range_size, ENV_VARS.target_triggers_per_block_range, unified_api_version, subgraph_current_block, @@ -201,7 +245,7 @@ impl BlockRefetcher for EthereumBlockRefetcher { logger: &Logger, cursor: FirehoseCursor, ) -> Result { - let endpoint = chain.chain_client().firehose_endpoint().await?; + let endpoint: Arc = chain.chain_client().firehose_endpoint().await?; let block = endpoint.get_block::(cursor, logger).await?; let ethereum_block: EthereumBlockWithCalls = (&block).try_into()?; Ok(BlockFinality::NonFinal(ethereum_block)) @@ -409,10 +453,27 @@ impl Blockchain for Chain { deployment: DeploymentLocator, store: impl DeploymentCursorTracker, start_blocks: Vec, - filter: Arc, + source_subgraph_stores: Vec>, + filter: Arc>, unified_api_version: UnifiedMappingApiVersion, ) -> Result>, Error> { let current_ptr = store.block_ptr(); + + if !filter.subgraph_filter.is_empty() { + return self + .block_stream_builder + .build_subgraph_block_stream( + self, + deployment, + start_blocks, + source_subgraph_stores, + current_ptr, + filter, + unified_api_version, + ) + .await; + } + match self.chain_client().as_ref() { ChainClient::Rpc(_) => { self.block_stream_builder @@ -420,6 +481,7 @@ impl Blockchain for Chain { self, deployment, start_blocks, + source_subgraph_stores, current_ptr, filter, unified_api_version, @@ -434,7 +496,7 @@ impl Blockchain for Chain { store.firehose_cursor(), start_blocks, current_ptr, - filter, + filter.chain_filter.clone(), unified_api_version, ) .await @@ -571,6 +633,8 @@ pub enum BlockFinality { // If a block may still be reorged, we need to work with more local data. NonFinal(EthereumBlockWithCalls), + + Ptr(Arc), } impl Default for BlockFinality { @@ -584,6 +648,7 @@ impl BlockFinality { match self { BlockFinality::Final(block) => block, BlockFinality::NonFinal(block) => &block.ethereum_block.block, + BlockFinality::Ptr(_) => unreachable!("light_block called on HeaderOnly"), } } } @@ -593,6 +658,7 @@ impl<'a> From<&'a BlockFinality> for BlockPtr { match block { BlockFinality::Final(b) => BlockPtr::from(&**b), BlockFinality::NonFinal(b) => BlockPtr::from(&b.ethereum_block), + BlockFinality::Ptr(b) => BlockPtr::new(b.hash.clone(), b.number), } } } @@ -602,6 +668,7 @@ impl Block for BlockFinality { match self { BlockFinality::Final(block) => block.block_ptr(), BlockFinality::NonFinal(block) => block.ethereum_block.block.block_ptr(), + BlockFinality::Ptr(block) => BlockPtr::new(block.hash.clone(), block.number), } } @@ -609,6 +676,9 @@ impl Block for BlockFinality { match self { BlockFinality::Final(block) => block.parent_ptr(), BlockFinality::NonFinal(block) => block.ethereum_block.block.parent_ptr(), + BlockFinality::Ptr(block) => { + Some(BlockPtr::new(block.parent_hash.clone(), block.number - 1)) + } } } @@ -641,16 +711,22 @@ impl Block for BlockFinality { json::to_value(eth_block) } BlockFinality::NonFinal(block) => json::to_value(&block.ethereum_block), + BlockFinality::Ptr(_) => Ok(json::Value::Null), } } fn timestamp(&self) -> BlockTime { - let ts = match self { - BlockFinality::Final(block) => block.timestamp, - BlockFinality::NonFinal(block) => block.ethereum_block.block.timestamp, - }; - let ts = i64::try_from(ts.as_u64()).unwrap(); - BlockTime::since_epoch(ts, 0) + match self { + BlockFinality::Final(block) => { + let ts = i64::try_from(block.timestamp.as_u64()).unwrap(); + BlockTime::since_epoch(ts, 0) + } + BlockFinality::NonFinal(block) => { + let ts = i64::try_from(block.ethereum_block.block.timestamp.as_u64()).unwrap(); + BlockTime::since_epoch(ts, 0) + } + BlockFinality::Ptr(block) => block.timestamp, + } } } @@ -665,6 +741,81 @@ pub struct TriggersAdapter { unified_api_version: UnifiedMappingApiVersion, } +/// Fetches blocks from the cache based on block numbers, excluding duplicates +/// (i.e., multiple blocks for the same number), and identifying missing blocks that +/// need to be fetched via RPC/Firehose. Returns a tuple of the found blocks and the missing block numbers. +async fn fetch_unique_blocks_from_cache( + logger: &Logger, + chain_store: Arc, + block_numbers: HashSet, +) -> (Vec>, Vec) { + // Load blocks from the cache + let blocks_map = chain_store + .cheap_clone() + .block_ptrs_by_numbers(block_numbers.iter().map(|&b| b.into()).collect::>()) + .await + .map_err(|e| { + error!(logger, "Error accessing block cache {}", e); + e + }) + .unwrap_or_default(); + + // Collect blocks and filter out ones with multiple entries + let blocks: Vec> = blocks_map + .into_iter() + .filter_map(|(_, values)| { + if values.len() == 1 { + Some(Arc::new(values[0].clone())) + } else { + None + } + }) + .collect(); + + // Identify missing blocks + let missing_blocks: Vec = block_numbers + .into_iter() + .filter(|&number| !blocks.iter().any(|block| block.block_number() == number)) + .collect(); + + if !missing_blocks.is_empty() { + debug!( + logger, + "Loading {} block(s) not in the block cache", + missing_blocks.len() + ); + debug!(logger, "Missing blocks {:?}", missing_blocks); + } + + (blocks, missing_blocks) +} + +/// Fetches blocks by their numbers, first attempting to load from cache. +/// Missing blocks are retrieved from an external source, with all blocks sorted and converted to `BlockFinality` format. +async fn load_blocks( + logger: &Logger, + chain_store: Arc, + block_numbers: HashSet, + fetch_missing: F, +) -> Result> +where + F: FnOnce(Vec) -> Fut, + Fut: Future>>>, +{ + // Fetch cached blocks and identify missing ones + let (mut cached_blocks, missing_block_numbers) = + fetch_unique_blocks_from_cache(logger, chain_store, block_numbers).await; + + // Fetch missing blocks if any + if !missing_block_numbers.is_empty() { + let missing_blocks = fetch_missing(missing_block_numbers).await?; + cached_blocks.extend(missing_blocks); + cached_blocks.sort_by_key(|block| block.number); + } + + Ok(cached_blocks.into_iter().map(BlockFinality::Ptr).collect()) +} + #[async_trait] impl TriggersAdapterTrait for TriggersAdapter { async fn scan_triggers( @@ -689,6 +840,82 @@ impl TriggersAdapterTrait for TriggersAdapter { .await } + async fn load_block_ptrs_by_numbers( + &self, + logger: Logger, + block_numbers: HashSet, + ) -> Result> { + match &*self.chain_client { + ChainClient::Firehose(endpoints) => { + trace!( + logger, + "Loading blocks from firehose"; + "block_numbers" => format!("{:?}", block_numbers) + ); + + let endpoint = endpoints.endpoint().await?; + let chain_store = self.chain_store.clone(); + let logger_clone = logger.clone(); + + load_blocks( + &logger, + chain_store, + block_numbers, + |missing_numbers| async move { + let blocks = endpoint + .load_blocks_by_numbers::( + missing_numbers.iter().map(|&n| n as u64).collect(), + &logger_clone, + ) + .await? + .into_iter() + .map(|block| { + Arc::new(ExtendedBlockPtr { + hash: block.hash(), + number: block.number(), + parent_hash: block.parent_hash().unwrap_or_default(), + timestamp: block.timestamp(), + }) + }) + .collect::>(); + Ok(blocks) + }, + ) + .await + } + + ChainClient::Rpc(client) => { + trace!( + logger, + "Loading blocks from RPC"; + "block_numbers" => format!("{:?}", block_numbers) + ); + + let adapter = client.cheapest_with(&self.capabilities).await?; + let chain_store = self.chain_store.clone(); + let logger_clone = logger.clone(); + + load_blocks( + &logger, + chain_store, + block_numbers, + |missing_numbers| async move { + adapter + .load_block_ptrs_by_numbers_rpc(logger_clone, missing_numbers) + .try_collect() + .await + }, + ) + .await + } + } + } + + async fn chain_head_ptr(&self) -> Result, Error> { + let chain_store = self.chain_store.clone(); + chain_store.chain_head_ptr().await + } + async fn triggers_in_block( &self, logger: &Logger, @@ -737,17 +964,30 @@ impl TriggersAdapterTrait for TriggersAdapter { triggers.append(&mut parse_block_triggers(&filter.block, full_block)); Ok(BlockWithTriggers::new(block, triggers, logger)) } + BlockFinality::Ptr(_) => unreachable!("triggers_in_block called on HeaderOnly"), } } async fn is_on_main_chain(&self, ptr: BlockPtr) -> Result { - self.chain_client - .rpc()? - .cheapest() - .await - .ok_or(anyhow!("unable to get adapter for is_on_main_chain"))? - .is_on_main_chain(&self.logger, ptr.clone()) - .await + match &*self.chain_client { + ChainClient::Firehose(endpoints) => { + let endpoint = endpoints.endpoint().await?; + let block = endpoint + .get_block_by_number::(ptr.number as u64, &self.logger) + .await + .map_err(|e| anyhow!("Failed to fetch block from firehose: {}", e))?; + + Ok(block.hash() == ptr.hash) + } + ChainClient::Rpc(adapter) => { + let adapter = adapter + .cheapest() + .await + .ok_or_else(|| anyhow!("unable to get adapter for is_on_main_chain"))?; + + adapter.is_on_main_chain(&self.logger, ptr).await + } + } } async fn ancestor_block( @@ -777,10 +1017,18 @@ impl TriggersAdapterTrait for TriggersAdapter { use graph::prelude::LightEthereumBlockExt; let block = match self.chain_client.as_ref() { - ChainClient::Firehose(_) => Some(BlockPtr { - hash: BlockHash::from(vec![0xff; 32]), - number: block.number.saturating_sub(1), - }), + ChainClient::Firehose(endpoints) => { + let endpoint = endpoints.endpoint().await?; + let block = endpoint + .get_block_by_ptr::(block, &self.logger) + .await + .context(format!( + "Failed to fetch block by ptr {} from firehose, backtrace: {}", + block, + std::backtrace::Backtrace::force_capture() + ))?; + block.parent_ptr() + } ChainClient::Rpc(adapters) => { let blocks = adapters .cheapest_with(&self.capabilities) @@ -945,3 +1193,138 @@ impl FirehoseMapperTrait for FirehoseMapper { .await } } + +#[cfg(test)] +mod tests { + use graph::blockchain::mock::MockChainStore; + use graph::{slog, tokio}; + + use super::*; + use std::collections::HashSet; + use std::sync::Arc; + + // Helper function to create test blocks + fn create_test_block(number: BlockNumber, hash: &str) -> ExtendedBlockPtr { + let hash = BlockHash(hash.as_bytes().to_vec().into_boxed_slice()); + let ptr = BlockPtr::new(hash.clone(), number); + ExtendedBlockPtr { + hash, + number, + parent_hash: BlockHash(vec![0; 32].into_boxed_slice()), + timestamp: BlockTime::for_test(&ptr), + } + } + + #[tokio::test] + async fn test_fetch_unique_blocks_single_block() { + let logger = Logger::root(slog::Discard, o!()); + let mut chain_store = MockChainStore::default(); + + // Add a single block + let block = create_test_block(1, "block1"); + chain_store.blocks.insert(1, vec![block.clone()]); + + let block_numbers: HashSet<_> = vec![1].into_iter().collect(); + + let (blocks, missing) = + fetch_unique_blocks_from_cache(&logger, Arc::new(chain_store), block_numbers).await; + + assert_eq!(blocks.len(), 1); + assert_eq!(blocks[0].number, 1); + assert!(missing.is_empty()); + } + + #[tokio::test] + async fn test_fetch_unique_blocks_duplicate_blocks() { + let logger = Logger::root(slog::Discard, o!()); + let mut chain_store = MockChainStore::default(); + + // Add multiple blocks for the same number + let block1 = create_test_block(1, "block1a"); + let block2 = create_test_block(1, "block1b"); + chain_store + .blocks + .insert(1, vec![block1.clone(), block2.clone()]); + + let block_numbers: HashSet<_> = vec![1].into_iter().collect(); + + let (blocks, missing) = + fetch_unique_blocks_from_cache(&logger, Arc::new(chain_store), block_numbers).await; + + // Should filter out the duplicate block + assert!(blocks.is_empty()); + assert_eq!(missing, vec![1]); + assert_eq!(missing[0], 1); + } + + #[tokio::test] + async fn test_fetch_unique_blocks_missing_blocks() { + let logger = Logger::root(slog::Discard, o!()); + let mut chain_store = MockChainStore::default(); + + // Add block number 1 but not 2 + let block = create_test_block(1, "block1"); + chain_store.blocks.insert(1, vec![block.clone()]); + + let block_numbers: HashSet<_> = vec![1, 2].into_iter().collect(); + + let (blocks, missing) = + fetch_unique_blocks_from_cache(&logger, Arc::new(chain_store), block_numbers).await; + + assert_eq!(blocks.len(), 1); + assert_eq!(blocks[0].number, 1); + assert_eq!(missing, vec![2]); + } + + #[tokio::test] + async fn test_fetch_unique_blocks_multiple_valid_blocks() { + let logger = Logger::root(slog::Discard, o!()); + let mut chain_store = MockChainStore::default(); + + // Add multiple valid blocks + let block1 = create_test_block(1, "block1"); + let block2 = create_test_block(2, "block2"); + chain_store.blocks.insert(1, vec![block1.clone()]); + chain_store.blocks.insert(2, vec![block2.clone()]); + + let block_numbers: HashSet<_> = vec![1, 2].into_iter().collect(); + + let (blocks, missing) = + fetch_unique_blocks_from_cache(&logger, Arc::new(chain_store), block_numbers).await; + + assert_eq!(blocks.len(), 2); + assert!(blocks.iter().any(|b| b.number == 1)); + assert!(blocks.iter().any(|b| b.number == 2)); + assert!(missing.is_empty()); + } + + #[tokio::test] + async fn test_fetch_unique_blocks_mixed_scenario() { + let logger = Logger::root(slog::Discard, o!()); + let mut chain_store = MockChainStore::default(); + + // Add a mix of scenarios: + // - Block 1: Single valid block + // - Block 2: Multiple blocks (duplicate) + // - Block 3: Missing + let block1 = create_test_block(1, "block1"); + let block2a = create_test_block(2, "block2a"); + let block2b = create_test_block(2, "block2b"); + + chain_store.blocks.insert(1, vec![block1.clone()]); + chain_store + .blocks + .insert(2, vec![block2a.clone(), block2b.clone()]); + + let block_numbers: HashSet<_> = vec![1, 2, 3].into_iter().collect(); + + let (blocks, missing) = + fetch_unique_blocks_from_cache(&logger, Arc::new(chain_store), block_numbers).await; + + assert_eq!(blocks.len(), 1); + assert_eq!(blocks[0].number, 1); + assert_eq!(missing.len(), 2); + assert!(missing.contains(&2)); + assert!(missing.contains(&3)); + } +} diff --git a/chain/ethereum/src/data_source.rs b/chain/ethereum/src/data_source.rs index c0253d2e60e..a2da3e6cb4e 100644 --- a/chain/ethereum/src/data_source.rs +++ b/chain/ethereum/src/data_source.rs @@ -5,20 +5,19 @@ use graph::components::metrics::subgraph::SubgraphInstanceMetrics; use graph::components::store::{EthereumCallCache, StoredDynamicDataSource}; use graph::components::subgraph::{HostMetrics, InstanceDSTemplateInfo, MappingError}; use graph::components::trigger_processor::RunnableTriggers; -use graph::data::value::Word; -use graph::data_source::CausalityRegion; +use graph::data_source::common::{ + CallDecls, DeclaredCall, FindMappingABI, MappingABI, UnresolvedMappingABI, +}; +use graph::data_source::{CausalityRegion, MappingTrigger as MappingTriggerType}; use graph::env::ENV_VARS; use graph::futures03::future::try_join; use graph::futures03::stream::FuturesOrdered; use graph::futures03::TryStreamExt; use graph::prelude::ethabi::ethereum_types::H160; -use graph::prelude::ethabi::{StateMutability, Token}; -use graph::prelude::lazy_static; -use graph::prelude::regex::Regex; +use graph::prelude::ethabi::StateMutability; use graph::prelude::{Link, SubgraphManifestValidationError}; use graph::slog::{debug, error, o, trace}; use itertools::Itertools; -use serde::de; use serde::de::Error as ErrorD; use serde::{Deserialize, Deserializer}; use std::collections::HashSet; @@ -30,10 +29,9 @@ use tiny_keccak::{keccak256, Keccak}; use graph::{ blockchain::{self, Blockchain}, - derive::CheapClone, prelude::{ async_trait, - ethabi::{Address, Contract, Event, Function, LogParam, ParamType, RawLog}, + ethabi::{Address, Event, Function, LogParam, ParamType, RawLog}, serde_json, warn, web3::types::{Log, Transaction, H256}, BlockNumber, CheapClone, EthereumCall, LightEthereumBlock, LightEthereumBlockExt, @@ -50,7 +48,7 @@ use crate::adapter::EthereumAdapter as _; use crate::chain::Chain; use crate::network::EthereumNetworkAdapters; use crate::trigger::{EthereumBlockTriggerType, EthereumTrigger, MappingTrigger}; -use crate::{ContractCall, NodeCapabilities}; +use crate::NodeCapabilities; // The recommended kind is `ethereum`, `ethereum/contract` is accepted for backwards compatibility. const ETHEREUM_KINDS: &[&str] = &["ethereum/contract", "ethereum"]; @@ -802,7 +800,12 @@ impl DataSource { "transaction" => format!("{}", &transaction.hash), }); let handler = event_handler.handler.clone(); - let calls = DeclaredCall::new(&self.mapping, &event_handler, &log, ¶ms)?; + let calls = DeclaredCall::from_log_trigger( + &self.mapping, + &event_handler.calls, + &log, + ¶ms, + )?; Ok(Some(TriggerWithHandler::::new_with_logging_extras( MappingTrigger::Log { block: block.cheap_clone(), @@ -933,73 +936,6 @@ impl DataSource { } } -#[derive(Clone, Debug, PartialEq)] -pub struct DeclaredCall { - /// The user-supplied label from the manifest - label: String, - contract_name: String, - address: Address, - function: Function, - args: Vec, -} - -impl DeclaredCall { - fn new( - mapping: &Mapping, - handler: &MappingEventHandler, - log: &Log, - params: &[LogParam], - ) -> Result, anyhow::Error> { - let mut calls = Vec::new(); - for decl in handler.calls.decls.iter() { - let contract_name = decl.expr.abi.to_string(); - let function_name = decl.expr.func.as_str(); - // Obtain the path to the contract ABI - let abi = mapping.find_abi(&contract_name)?; - // TODO: Handle overloaded functions - let function = { - // Behavior for apiVersion < 0.0.4: look up function by name; for overloaded - // functions this always picks the same overloaded variant, which is incorrect - // and may lead to encoding/decoding errors - abi.contract.function(function_name).with_context(|| { - format!( - "Unknown function \"{}::{}\" called from WASM runtime", - contract_name, function_name - ) - })? - }; - - let address = decl.address(log, params)?; - let args = decl.args(log, params)?; - - let call = DeclaredCall { - label: decl.label.clone(), - contract_name, - address, - function: function.clone(), - args, - }; - calls.push(call); - } - - Ok(calls) - } - - fn as_eth_call(self, block_ptr: BlockPtr, gas: Option) -> (ContractCall, String) { - ( - ContractCall { - contract_name: self.contract_name, - address: self.address, - block_ptr, - function: self.function, - args: self.args, - gas, - }, - self.label, - ) - } -} - pub struct DecoderHook { eth_adapters: Arc, call_cache: Arc, @@ -1098,6 +1034,115 @@ impl DecoderHook { .collect(); Ok(labels) } + + fn collect_declared_calls<'a>( + &self, + runnables: &Vec>, + ) -> Vec<(Arc, DeclaredCall)> { + // Extract all hosted triggers from runnables + let all_triggers = runnables + .iter() + .flat_map(|runnable| &runnable.hosted_triggers); + + // Collect calls from both onchain and subgraph triggers + let mut all_calls = Vec::new(); + + for trigger in all_triggers { + let host_metrics = trigger.host.host_metrics(); + + match &trigger.mapping_trigger.trigger { + MappingTriggerType::Onchain(t) => { + if let MappingTrigger::Log { calls, .. } = t { + for call in calls.clone() { + all_calls.push((host_metrics.cheap_clone(), call)); + } + } + } + MappingTriggerType::Subgraph(t) => { + for call in t.calls.clone() { + // Convert subgraph call to the expected DeclaredCall type if needed + // or handle differently based on the types + all_calls.push((host_metrics.cheap_clone(), call)); + } + } + MappingTriggerType::Offchain(_) => {} + } + } + + all_calls + } + + /// Deduplicate calls. Unfortunately, we can't get `DeclaredCall` to + /// implement `Hash` or `Ord` easily, so we can only deduplicate by + /// comparing the whole call not with a `HashSet` or `BTreeSet`. + /// Since that can be inefficient, we don't deduplicate if we have an + /// enormous amount of calls; in that case though, things will likely + /// blow up because of the amount of I/O that many calls cause. + /// Cutting off at 1000 is fairly arbitrary + fn deduplicate_calls( + &self, + calls: Vec<(Arc, DeclaredCall)>, + ) -> Vec<(Arc, DeclaredCall)> { + if calls.len() >= 1000 { + return calls; + } + + let mut uniq_calls = Vec::new(); + for (metrics, call) in calls { + if !uniq_calls.iter().any(|(_, c)| c == &call) { + uniq_calls.push((metrics, call)); + } + } + uniq_calls + } + + /// Log information about failed eth calls. 'Failure' here simply + /// means that the call was reverted; outright errors lead to a real + /// error. For reverted calls, `self.eth_calls` returns the label + /// from the manifest for that call. + /// + /// One reason why declared calls can fail is if they are attached + /// to the wrong handler, or if arguments are specified incorrectly. + /// Calls that revert every once in a while might be ok and what the + /// user intended, but we want to clearly log so that users can spot + /// mistakes in their manifest, which will lead to unnecessary eth + /// calls + fn log_declared_call_results( + logger: &Logger, + failures: &[String], + calls_count: usize, + trigger_count: usize, + elapsed: Duration, + ) { + let fail_count = failures.len(); + + if fail_count > 0 { + let mut counts: Vec<_> = failures.iter().counts().into_iter().collect(); + counts.sort_by_key(|(label, _)| *label); + + let failure_summary = counts + .into_iter() + .map(|(label, count)| { + let times = if count == 1 { "time" } else { "times" }; + format!("{label} ({count} {times})") + }) + .join(", "); + + error!(logger, "Declared calls failed"; + "triggers" => trigger_count, + "calls_count" => calls_count, + "fail_count" => fail_count, + "calls_ms" => elapsed.as_millis(), + "failures" => format!("[{}]", failure_summary) + ); + } else { + debug!(logger, "Declared calls"; + "triggers" => trigger_count, + "calls_count" => calls_count, + "calls_ms" => elapsed.as_millis() + ); + } + } } #[async_trait] @@ -1109,50 +1154,6 @@ impl blockchain::DecoderHook for DecoderHook { runnables: Vec>, metrics: &Arc, ) -> Result>, MappingError> { - /// Log information about failed eth calls. 'Failure' here simply - /// means that the call was reverted; outright errors lead to a real - /// error. For reverted calls, `self.eth_calls` returns the label - /// from the manifest for that call. - /// - /// One reason why declared calls can fail is if they are attached - /// to the wrong handler, or if arguments are specified incorrectly. - /// Calls that revert every once in a while might be ok and what the - /// user intended, but we want to clearly log so that users can spot - /// mistakes in their manifest, which will lead to unnecessary eth - /// calls - fn log_results( - logger: &Logger, - failures: &[String], - calls_count: usize, - trigger_count: usize, - elapsed: Duration, - ) { - let fail_count = failures.len(); - - if fail_count > 0 { - let mut counts: Vec<_> = failures.iter().counts().into_iter().collect(); - counts.sort_by_key(|(label, _)| *label); - let counts = counts - .into_iter() - .map(|(label, count)| { - let times = if count == 1 { "time" } else { "times" }; - format!("{label} ({count} {times})") - }) - .join(", "); - error!(logger, "Declared calls failed"; - "triggers" => trigger_count, - "calls_count" => calls_count, - "fail_count" => fail_count, - "calls_ms" => elapsed.as_millis(), - "failures" => format!("[{}]", counts)); - } else { - debug!(logger, "Declared calls"; - "triggers" => trigger_count, - "calls_count" => calls_count, - "calls_ms" => elapsed.as_millis()); - } - } - if ENV_VARS.mappings.disable_declared_calls { return Ok(runnables); } @@ -1160,51 +1161,17 @@ impl blockchain::DecoderHook for DecoderHook { let _section = metrics.stopwatch.start_section("declared_ethereum_call"); let start = Instant::now(); - let calls: Vec<_> = runnables - .iter() - .map(|r| &r.hosted_triggers) - .flatten() - .filter_map(|trigger| { - trigger - .mapping_trigger - .trigger - .as_onchain() - .map(|t| (trigger.host.host_metrics(), t)) - }) - .filter_map(|(metrics, trigger)| match trigger { - MappingTrigger::Log { calls, .. } => Some( - calls - .clone() - .into_iter() - .map(move |call| (metrics.cheap_clone(), call)), - ), - MappingTrigger::Block { .. } | MappingTrigger::Call { .. } => None, - }) - .flatten() - .collect(); + // Collect and process declared calls + let calls = self.collect_declared_calls(&runnables); + let deduplicated_calls = self.deduplicate_calls(calls); - // Deduplicate calls. Unfortunately, we can't get `DeclaredCall` to - // implement `Hash` or `Ord` easily, so we can only deduplicate by - // comparing the whole call not with a `HashSet` or `BTreeSet`. - // Since that can be inefficient, we don't deduplicate if we have an - // enormous amount of calls; in that case though, things will likely - // blow up because of the amount of I/O that many calls cause. - // Cutting off at 1000 is fairly arbitrary - let calls = if calls.len() < 1000 { - let mut uniq_calls = Vec::new(); - for (metrics, call) in calls { - if !uniq_calls.iter().any(|(_, c)| c == &call) { - uniq_calls.push((metrics, call)); - } - } - uniq_calls - } else { - calls - }; + // Execute calls and log results + let calls_count = deduplicated_calls.len(); + let results = self + .eth_calls(logger, block_ptr, deduplicated_calls) + .await?; - let calls_count = calls.len(); - let results = self.eth_calls(logger, block_ptr, calls).await?; - log_results( + Self::log_declared_call_results( logger, &results, calls_count, @@ -1372,8 +1339,10 @@ impl Mapping { .iter() .any(|handler| matches!(handler.filter, Some(BlockHandlerFilter::Call))) } +} - pub fn find_abi(&self, abi_name: &str) -> Result, Error> { +impl FindMappingABI for Mapping { + fn find_abi(&self, abi_name: &str) -> Result, Error> { Ok(self .abis .iter() @@ -1436,82 +1405,6 @@ impl UnresolvedMapping { } } -#[derive(Clone, Debug, Hash, Eq, PartialEq, Deserialize)] -pub struct UnresolvedMappingABI { - pub name: String, - pub file: Link, -} - -impl UnresolvedMappingABI { - pub async fn resolve( - self, - resolver: &Arc, - logger: &Logger, - ) -> Result { - let contract_bytes = resolver.cat(logger, &self.file).await.with_context(|| { - format!( - "failed to resolve ABI {} from {}", - self.name, self.file.link - ) - })?; - let contract = Contract::load(&*contract_bytes)?; - Ok(MappingABI { - name: self.name, - contract, - }) - } -} - -#[derive(Clone, Debug, PartialEq)] -pub struct MappingABI { - pub name: String, - pub contract: Contract, -} - -impl MappingABI { - pub fn function( - &self, - contract_name: &str, - name: &str, - signature: Option<&str>, - ) -> Result<&Function, Error> { - let contract = &self.contract; - let function = match signature { - // Behavior for apiVersion < 0.0.4: look up function by name; for overloaded - // functions this always picks the same overloaded variant, which is incorrect - // and may lead to encoding/decoding errors - None => contract.function(name).with_context(|| { - format!( - "Unknown function \"{}::{}\" called from WASM runtime", - contract_name, name - ) - })?, - - // Behavior for apiVersion >= 0.0.04: look up function by signature of - // the form `functionName(uint256,string) returns (bytes32,string)`; this - // correctly picks the correct variant of an overloaded function - Some(ref signature) => contract - .functions_by_name(name) - .with_context(|| { - format!( - "Unknown function \"{}::{}\" called from WASM runtime", - contract_name, name - ) - })? - .iter() - .find(|f| signature == &f.signature()) - .with_context(|| { - format!( - "Unknown function \"{}::{}\" with signature `{}` \ - called from WASM runtime", - contract_name, name, signature, - ) - })?, - }; - Ok(function) - } -} - #[derive(Clone, Debug, Hash, Eq, PartialEq, Deserialize)] pub struct MappingBlockHandler { pub handler: String, @@ -1644,225 +1537,3 @@ fn string_to_h256(s: &str) -> H256 { pub struct TemplateSource { pub abi: String, } - -/// Internal representation of declared calls. In the manifest that's -/// written as part of an event handler as -/// ```yaml -/// calls: -/// - myCall1: Contract[address].function(arg1, arg2, ...) -/// - .. -/// ``` -/// -/// The `address` and `arg` fields can be either `event.address` or -/// `event.params.`. Each entry under `calls` gets turned into a -/// `CallDcl` -#[derive(Clone, CheapClone, Debug, Default, Hash, Eq, PartialEq)] -pub struct CallDecls { - pub decls: Arc>, - readonly: (), -} - -/// A single call declaration, like `myCall1: -/// Contract[address].function(arg1, arg2, ...)` -#[derive(Clone, Debug, Hash, Eq, PartialEq)] -pub struct CallDecl { - /// A user-defined label - pub label: String, - /// The call expression - pub expr: CallExpr, - readonly: (), -} -impl CallDecl { - fn address(&self, log: &Log, params: &[LogParam]) -> Result { - let address = match &self.expr.address { - CallArg::Address => log.address, - CallArg::HexAddress(address) => *address, - CallArg::Param(name) => { - let value = params - .iter() - .find(|param| ¶m.name == name.as_str()) - .ok_or_else(|| anyhow!("unknown param {name}"))? - .value - .clone(); - value - .into_address() - .ok_or_else(|| anyhow!("param {name} is not an address"))? - } - }; - Ok(address) - } - - fn args(&self, log: &Log, params: &[LogParam]) -> Result, Error> { - self.expr - .args - .iter() - .map(|arg| match arg { - CallArg::Address => Ok(Token::Address(log.address)), - CallArg::HexAddress(address) => Ok(Token::Address(*address)), - CallArg::Param(name) => { - let value = params - .iter() - .find(|param| ¶m.name == name.as_str()) - .ok_or_else(|| anyhow!("unknown param {name}"))? - .value - .clone(); - Ok(value) - } - }) - .collect() - } -} - -impl<'de> de::Deserialize<'de> for CallDecls { - fn deserialize(deserializer: D) -> Result - where - D: de::Deserializer<'de>, - { - let decls: std::collections::HashMap = - de::Deserialize::deserialize(deserializer)?; - let decls = decls - .into_iter() - .map(|(name, expr)| { - expr.parse::().map(|expr| CallDecl { - label: name, - expr, - readonly: (), - }) - }) - .collect::>() - .map(|decls| Arc::new(decls)) - .map_err(de::Error::custom)?; - Ok(CallDecls { - decls, - readonly: (), - }) - } -} - -#[derive(Clone, Debug, Hash, Eq, PartialEq)] -pub struct CallExpr { - pub abi: Word, - pub address: CallArg, - pub func: Word, - pub args: Vec, - readonly: (), -} - -/// Parse expressions of the form `Contract[address].function(arg1, arg2, -/// ...)` where the `address` and the args are either `event.address` or -/// `event.params.`. -/// -/// The parser is pretty awful as it generates error messages that aren't -/// very helpful. We should replace all this with a real parser, most likely -/// `combine` which is what `graphql_parser` uses -impl FromStr for CallExpr { - type Err = anyhow::Error; - - fn from_str(s: &str) -> Result { - lazy_static! { - static ref RE: Regex = Regex::new( - r"(?x) - (?P[a-zA-Z0-9_]+)\[ - (?P
[^]]+)\] - \. - (?P[a-zA-Z0-9_]+)\( - (?P[^)]*) - \)" - ) - .unwrap(); - } - let x = RE - .captures(s) - .ok_or_else(|| anyhow!("invalid call expression `{s}`"))?; - let abi = Word::from(x.name("abi").unwrap().as_str()); - let address = x.name("address").unwrap().as_str().parse()?; - let func = Word::from(x.name("func").unwrap().as_str()); - let args: Vec = x - .name("args") - .unwrap() - .as_str() - .split(',') - .filter(|s| !s.is_empty()) - .map(|s| s.trim().parse::()) - .collect::>()?; - Ok(CallExpr { - abi, - address, - func, - args, - readonly: (), - }) - } -} - -#[derive(Clone, Debug, Hash, Eq, PartialEq)] -pub enum CallArg { - HexAddress(Address), - Address, - Param(Word), -} - -lazy_static! { - // Matches a 40-character hexadecimal string prefixed with '0x', typical for Ethereum addresses - static ref ADDR_RE: Regex = Regex::new(r"^0x[0-9a-fA-F]{40}$").unwrap(); -} - -impl FromStr for CallArg { - type Err = anyhow::Error; - - fn from_str(s: &str) -> Result { - if ADDR_RE.is_match(s) { - if let Ok(parsed_address) = Address::from_str(s) { - return Ok(CallArg::HexAddress(parsed_address)); - } - } - - let mut parts = s.split('.'); - match (parts.next(), parts.next(), parts.next()) { - (Some("event"), Some("address"), None) => Ok(CallArg::Address), - (Some("event"), Some("params"), Some(param)) => Ok(CallArg::Param(Word::from(param))), - _ => Err(anyhow!("invalid call argument `{}`", s)), - } - } -} - -#[test] -fn test_call_expr() { - let expr: CallExpr = "ERC20[event.address].balanceOf(event.params.token)" - .parse() - .unwrap(); - assert_eq!(expr.abi, "ERC20"); - assert_eq!(expr.address, CallArg::Address); - assert_eq!(expr.func, "balanceOf"); - assert_eq!(expr.args, vec![CallArg::Param("token".into())]); - - let expr: CallExpr = "Pool[event.params.pool].fees(event.params.token0, event.params.token1)" - .parse() - .unwrap(); - assert_eq!(expr.abi, "Pool"); - assert_eq!(expr.address, CallArg::Param("pool".into())); - assert_eq!(expr.func, "fees"); - assert_eq!( - expr.args, - vec![ - CallArg::Param("token0".into()), - CallArg::Param("token1".into()) - ] - ); - - let expr: CallExpr = "Pool[event.address].growth()".parse().unwrap(); - assert_eq!(expr.abi, "Pool"); - assert_eq!(expr.address, CallArg::Address); - assert_eq!(expr.func, "growth"); - assert_eq!(expr.args, vec![]); - - let expr: CallExpr = "Pool[0xDeaDbeefdEAdbeefdEadbEEFdeadbeEFdEaDbeeF].growth(0xDeaDbeefdEAdbeefdEadbEEFdeadbeEFdEaDbeeF)" - .parse() - .unwrap(); - let call_arg = - CallArg::HexAddress(H160::from_str("0xDeaDbeefdEAdbeefdEadbEEFdeadbeEFdEaDbeeF").unwrap()); - assert_eq!(expr.abi, "Pool"); - assert_eq!(expr.address, call_arg); - assert_eq!(expr.func, "growth"); - assert_eq!(expr.args, vec![call_arg]); -} diff --git a/chain/ethereum/src/env.rs b/chain/ethereum/src/env.rs index 75c313212b9..bc7223dbc07 100644 --- a/chain/ethereum/src/env.rs +++ b/chain/ethereum/src/env.rs @@ -33,6 +33,9 @@ pub struct EnvVars { /// Set by the environment variable `ETHEREUM_BLOCK_BATCH_SIZE`. The /// default value is 10 blocks. pub block_batch_size: usize, + /// Set by the environment variable `ETHEREUM_BLOCK_PTR_BATCH_SIZE`. The + /// default value is 10 blocks. + pub block_ptr_batch_size: usize, /// Maximum number of blocks to request in each chunk. /// /// Set by the environment variable `GRAPH_ETHEREUM_MAX_BLOCK_RANGE_SIZE`. @@ -116,6 +119,7 @@ impl From for EnvVars { trace_stream_step_size: x.trace_stream_step_size, max_event_only_range: x.max_event_only_range, block_batch_size: x.block_batch_size, + block_ptr_batch_size: x.block_ptr_batch_size, max_block_range_size: x.max_block_range_size, json_rpc_timeout: Duration::from_secs(x.json_rpc_timeout_in_secs), block_receipts_check_timeout: Duration::from_secs( @@ -160,6 +164,8 @@ struct Inner { max_event_only_range: BlockNumber, #[envconfig(from = "ETHEREUM_BLOCK_BATCH_SIZE", default = "10")] block_batch_size: usize, + #[envconfig(from = "ETHEREUM_BLOCK_PTR_BATCH_SIZE", default = "100")] + block_ptr_batch_size: usize, #[envconfig(from = "GRAPH_ETHEREUM_MAX_BLOCK_RANGE_SIZE", default = "2000")] max_block_range_size: BlockNumber, #[envconfig(from = "GRAPH_ETHEREUM_JSON_RPC_TIMEOUT", default = "180")] diff --git a/chain/ethereum/src/ethereum_adapter.rs b/chain/ethereum/src/ethereum_adapter.rs index c4ea6323c7d..78d0d084a85 100644 --- a/chain/ethereum/src/ethereum_adapter.rs +++ b/chain/ethereum/src/ethereum_adapter.rs @@ -2,11 +2,14 @@ use futures03::{future::BoxFuture, stream::FuturesUnordered}; use graph::blockchain::client::ChainClient; use graph::blockchain::BlockHash; use graph::blockchain::ChainIdentifier; +use graph::blockchain::ExtendedBlockPtr; + use graph::components::transaction_receipt::LightTransactionReceipt; use graph::data::store::ethereum::call; use graph::data::store::scalar; use graph::data::subgraph::UnifiedMappingApiVersion; use graph::data::subgraph::API_VERSION_0_0_7; +use graph::data_source::common::ContractCall; use graph::futures01::stream; use graph::futures01::Future; use graph::futures01::Stream; @@ -58,15 +61,16 @@ use crate::chain::BlockFinality; use crate::trigger::LogRef; use crate::Chain; use crate::NodeCapabilities; +use crate::TriggerFilter; use crate::{ adapter::{ - ContractCall, ContractCallError, EthGetLogsFilter, EthereumAdapter as EthereumAdapterTrait, + ContractCallError, EthGetLogsFilter, EthereumAdapter as EthereumAdapterTrait, EthereumBlockFilter, EthereumCallFilter, EthereumLogFilter, ProviderEthRpcMetrics, SubgraphEthRpcMetrics, }, transport::Transport, trigger::{EthereumBlockTriggerType, EthereumTrigger}, - TriggerFilter, ENV_VARS, + ENV_VARS, }; #[derive(Debug, Clone)] @@ -780,6 +784,64 @@ impl EthereumAdapter { .buffered(ENV_VARS.block_batch_size) } + /// Request blocks by number through JSON-RPC. + pub fn load_block_ptrs_by_numbers_rpc( + &self, + logger: Logger, + numbers: Vec, + ) -> impl futures03::Stream, Error>> + Send { + let web3 = self.web3.clone(); + + futures03::stream::iter(numbers.into_iter().map(move |number| { + let web3 = web3.clone(); + let logger = logger.clone(); + + async move { + retry(format!("load block {}", number), &logger) + .limit(ENV_VARS.request_retries) + .timeout_secs(ENV_VARS.json_rpc_timeout.as_secs()) + .run(move || { + let web3 = web3.clone(); + + async move { + let block_result = web3 + .eth() + .block(BlockId::Number(Web3BlockNumber::Number(number.into()))) + .await; + + match block_result { + Ok(Some(block)) => { + let ptr = ExtendedBlockPtr::try_from(( + block.hash, + block.number, + block.parent_hash, + block.timestamp, + )) + .map_err(|e| { + anyhow::anyhow!("Failed to convert block: {}", e) + })?; + Ok(Arc::new(ptr)) + } + Ok(None) => Err(anyhow::anyhow!( + "Ethereum node did not find block with number {:?}", + number + )), + Err(e) => Err(anyhow::anyhow!("Failed to fetch block: {}", e)), + } + } + }) + .await + .map_err(|e| match e { + TimeoutError::Elapsed => { + anyhow::anyhow!("Timeout while fetching block {}", number) + } + TimeoutError::Inner(e) => e, + }) + } + })) + .buffered(ENV_VARS.block_ptr_batch_size) + } + /// Request blocks ptrs for numbers through JSON-RPC. /// /// Reorg safety: If ids are numbers, they must be a final blocks. @@ -1923,6 +1985,9 @@ pub(crate) async fn get_calls( calls: Some(calls), })) } + BlockFinality::Ptr(_) => { + unreachable!("get_calls called with BlockFinality::Ptr") + } } } @@ -2077,8 +2142,8 @@ async fn filter_call_triggers_from_unsuccessful_transactions( let transaction_hashes: BTreeSet = block .trigger_data .iter() - .filter_map(|trigger| match trigger { - EthereumTrigger::Call(call_trigger) => Some(call_trigger.transaction_hash), + .filter_map(|trigger| match trigger.as_chain() { + Some(EthereumTrigger::Call(call_trigger)) => Some(call_trigger.transaction_hash), _ => None, }) .collect::>>() @@ -2104,6 +2169,11 @@ async fn filter_call_triggers_from_unsuccessful_transactions( "this function should not be called when dealing with non-final blocks" ) } + BlockFinality::Ptr(_block) => { + unreachable!( + "this function should not be called when dealing with header-only blocks" + ) + } } }; @@ -2169,7 +2239,7 @@ async fn filter_call_triggers_from_unsuccessful_transactions( // Filter call triggers from unsuccessful transactions block.trigger_data.retain(|trigger| { - if let EthereumTrigger::Call(call_trigger) = trigger { + if let Some(EthereumTrigger::Call(call_trigger)) = trigger.as_chain() { // Unwrap: We already checked that those values exist transaction_success[&call_trigger.transaction_hash.unwrap()] } else { diff --git a/chain/ethereum/src/lib.rs b/chain/ethereum/src/lib.rs index b83415146ac..8cf4e4cc669 100644 --- a/chain/ethereum/src/lib.rs +++ b/chain/ethereum/src/lib.rs @@ -19,7 +19,7 @@ pub use buffered_call_cache::BufferedCallCache; // ETHDEP: These concrete types should probably not be exposed. pub use data_source::{ - BlockHandlerFilter, DataSource, DataSourceTemplate, Mapping, MappingABI, TemplateSource, + BlockHandlerFilter, DataSource, DataSourceTemplate, Mapping, TemplateSource, }; pub mod chain; @@ -28,8 +28,8 @@ pub mod network; pub mod trigger; pub use crate::adapter::{ - ContractCall, ContractCallError, EthereumAdapter as EthereumAdapterTrait, - ProviderEthRpcMetrics, SubgraphEthRpcMetrics, TriggerFilter, + ContractCallError, EthereumAdapter as EthereumAdapterTrait, ProviderEthRpcMetrics, + SubgraphEthRpcMetrics, TriggerFilter, }; pub use crate::chain::Chain; pub use graph::blockchain::BlockIngestor; diff --git a/chain/ethereum/src/runtime/runtime_adapter.rs b/chain/ethereum/src/runtime/runtime_adapter.rs index 4147d61f5b0..01f148bdd4c 100644 --- a/chain/ethereum/src/runtime/runtime_adapter.rs +++ b/chain/ethereum/src/runtime/runtime_adapter.rs @@ -1,10 +1,9 @@ use std::{sync::Arc, time::Instant}; use crate::adapter::EthereumRpcError; -use crate::data_source::MappingABI; use crate::{ - capabilities::NodeCapabilities, network::EthereumNetworkAdapters, Chain, ContractCall, - ContractCallError, DataSource, EthereumAdapter, EthereumAdapterTrait, ENV_VARS, + capabilities::NodeCapabilities, network::EthereumNetworkAdapters, Chain, ContractCallError, + EthereumAdapter, EthereumAdapterTrait, ENV_VARS, }; use anyhow::{anyhow, Context, Error}; use blockchain::HostFn; @@ -13,6 +12,8 @@ use graph::components::subgraph::HostMetrics; use graph::data::store::ethereum::call; use graph::data::store::scalar::BigInt; use graph::data::subgraph::API_VERSION_0_0_9; +use graph::data_source; +use graph::data_source::common::{ContractCall, MappingABI}; use graph::futures03::compat::Future01CompatExt; use graph::prelude::web3::types::H160; use graph::runtime::gas::Gas; @@ -80,58 +81,93 @@ pub fn eth_call_gas(chain_identifier: &ChainIdentifier) -> Option { } impl blockchain::RuntimeAdapter for RuntimeAdapter { - fn host_fns(&self, ds: &DataSource) -> Result, Error> { - let abis = ds.mapping.abis.clone(); - let call_cache = self.call_cache.cheap_clone(); - let eth_adapters = self.eth_adapters.cheap_clone(); - let archive = ds.mapping.requires_archive()?; - let eth_call_gas = eth_call_gas(&self.chain_identifier); - - let ethereum_call = HostFn { - name: "ethereum.call", - func: Arc::new(move |ctx, wasm_ptr| { - // Ethereum calls should prioritise call-only adapters if one is available. - let eth_adapter = eth_adapters.call_or_cheapest(Some(&NodeCapabilities { - archive, - traces: false, - }))?; - ethereum_call( - ð_adapter, - call_cache.cheap_clone(), - ctx, - wasm_ptr, - &abis, - eth_call_gas, - ) - .map(|ptr| ptr.wasm_ptr()) - }), - }; - - let eth_adapters = self.eth_adapters.cheap_clone(); - let ethereum_get_balance = HostFn { - name: "ethereum.getBalance", - func: Arc::new(move |ctx, wasm_ptr| { - let eth_adapter = eth_adapters.unverified_cheapest_with(&NodeCapabilities { - archive, - traces: false, - })?; - eth_get_balance(ð_adapter, ctx, wasm_ptr).map(|ptr| ptr.wasm_ptr()) - }), - }; + fn host_fns(&self, ds: &data_source::DataSource) -> Result, Error> { + fn create_host_fns( + abis: Arc>>, // Use Arc to ensure `'static` lifetimes. + archive: bool, + call_cache: Arc, + eth_adapters: Arc, + eth_call_gas: Option, + ) -> Vec { + vec![ + HostFn { + name: "ethereum.call", + func: Arc::new({ + let eth_adapters = eth_adapters.clone(); + let call_cache = call_cache.clone(); + let abis = abis.clone(); + move |ctx, wasm_ptr| { + let eth_adapter = + eth_adapters.call_or_cheapest(Some(&NodeCapabilities { + archive, + traces: false, + }))?; + ethereum_call( + ð_adapter, + call_cache.clone(), + ctx, + wasm_ptr, + &abis, + eth_call_gas, + ) + .map(|ptr| ptr.wasm_ptr()) + } + }), + }, + HostFn { + name: "ethereum.getBalance", + func: Arc::new({ + let eth_adapters = eth_adapters.clone(); + move |ctx, wasm_ptr| { + let eth_adapter = + eth_adapters.unverified_cheapest_with(&NodeCapabilities { + archive, + traces: false, + })?; + eth_get_balance(ð_adapter, ctx, wasm_ptr).map(|ptr| ptr.wasm_ptr()) + } + }), + }, + HostFn { + name: "ethereum.hasCode", + func: Arc::new({ + let eth_adapters = eth_adapters.clone(); + move |ctx, wasm_ptr| { + let eth_adapter = + eth_adapters.unverified_cheapest_with(&NodeCapabilities { + archive, + traces: false, + })?; + eth_has_code(ð_adapter, ctx, wasm_ptr).map(|ptr| ptr.wasm_ptr()) + } + }), + }, + ] + } - let eth_adapters = self.eth_adapters.cheap_clone(); - let ethereum_get_code = HostFn { - name: "ethereum.hasCode", - func: Arc::new(move |ctx, wasm_ptr| { - let eth_adapter = eth_adapters.unverified_cheapest_with(&NodeCapabilities { - archive, - traces: false, - })?; - eth_has_code(ð_adapter, ctx, wasm_ptr).map(|ptr| ptr.wasm_ptr()) - }), + let host_fns = match ds { + data_source::DataSource::Onchain(onchain_ds) => { + let abis = Arc::new(onchain_ds.mapping.abis.clone()); + let archive = onchain_ds.mapping.requires_archive()?; + let call_cache = self.call_cache.cheap_clone(); + let eth_adapters = self.eth_adapters.cheap_clone(); + let eth_call_gas = eth_call_gas(&self.chain_identifier); + + create_host_fns(abis, archive, call_cache, eth_adapters, eth_call_gas) + } + data_source::DataSource::Subgraph(subgraph_ds) => { + let abis = Arc::new(subgraph_ds.mapping.abis.clone()); + let archive = subgraph_ds.mapping.requires_archive()?; + let call_cache = self.call_cache.cheap_clone(); + let eth_adapters = self.eth_adapters.cheap_clone(); + let eth_call_gas = eth_call_gas(&self.chain_identifier); + + create_host_fns(abis, archive, call_cache, eth_adapters, eth_call_gas) + } + data_source::DataSource::Offchain(_) => vec![], }; - Ok(vec![ethereum_call, ethereum_get_balance, ethereum_get_code]) + Ok(host_fns) } } diff --git a/chain/ethereum/src/tests.rs b/chain/ethereum/src/tests.rs index 455a7c07432..00873f8ea87 100644 --- a/chain/ethereum/src/tests.rs +++ b/chain/ethereum/src/tests.rs @@ -1,7 +1,7 @@ use std::sync::Arc; use graph::{ - blockchain::{block_stream::BlockWithTriggers, BlockPtr}, + blockchain::{block_stream::BlockWithTriggers, BlockPtr, Trigger}, prelude::{ web3::types::{Address, Bytes, Log, H160, H256, U64}, EthereumCall, LightEthereumBlock, @@ -107,10 +107,12 @@ fn test_trigger_ordering() { &logger, ); - assert_eq!( - block_with_triggers.trigger_data, - vec![log1, log2, call1, log3, call2, call4, call3, block2, block1] - ); + let expected = vec![log1, log2, call1, log3, call2, call4, call3, block2, block1] + .into_iter() + .map(|t| Trigger::Chain(t)) + .collect::>(); + + assert_eq!(block_with_triggers.trigger_data, expected); } #[test] @@ -203,8 +205,10 @@ fn test_trigger_dedup() { &logger, ); - assert_eq!( - block_with_triggers.trigger_data, - vec![log1, log2, call1, log3, call2, call3, block2, block1] - ); + let expected = vec![log1, log2, call1, log3, call2, call3, block2, block1] + .into_iter() + .map(|t| Trigger::Chain(t)) + .collect::>(); + + assert_eq!(block_with_triggers.trigger_data, expected); } diff --git a/chain/ethereum/src/trigger.rs b/chain/ethereum/src/trigger.rs index 128ed8d3e98..a5d83690b4b 100644 --- a/chain/ethereum/src/trigger.rs +++ b/chain/ethereum/src/trigger.rs @@ -3,6 +3,7 @@ use graph::blockchain::TriggerData; use graph::data::subgraph::API_VERSION_0_0_2; use graph::data::subgraph::API_VERSION_0_0_6; use graph::data::subgraph::API_VERSION_0_0_7; +use graph::data_source::common::DeclaredCall; use graph::prelude::ethabi::ethereum_types::H160; use graph::prelude::ethabi::ethereum_types::H256; use graph::prelude::ethabi::ethereum_types::U128; @@ -28,7 +29,6 @@ use graph_runtime_wasm::module::ToAscPtr; use std::ops::Deref; use std::{cmp::Ordering, sync::Arc}; -use crate::data_source::DeclaredCall; use crate::runtime::abi::AscEthereumBlock; use crate::runtime::abi::AscEthereumBlock_0_0_6; use crate::runtime::abi::AscEthereumCall; diff --git a/chain/near/src/chain.rs b/chain/near/src/chain.rs index 283552e7f33..3b9643cf7af 100644 --- a/chain/near/src/chain.rs +++ b/chain/near/src/chain.rs @@ -4,11 +4,11 @@ use graph::blockchain::firehose_block_ingestor::FirehoseBlockIngestor; use graph::blockchain::substreams_block_stream::SubstreamsBlockStream; use graph::blockchain::{ BasicBlockchainBuilder, BlockIngestor, BlockchainBuilder, BlockchainKind, NoopDecoderHook, - NoopRuntimeAdapter, + NoopRuntimeAdapter, Trigger, TriggerFilterWrapper, }; use graph::cheap_clone::CheapClone; use graph::components::adapter::ChainId; -use graph::components::store::DeploymentCursorTracker; +use graph::components::store::{DeploymentCursorTracker, SourceableStore}; use graph::data::subgraph::UnifiedMappingApiVersion; use graph::env::EnvVars; use graph::firehose::FirehoseEndpoint; @@ -32,10 +32,12 @@ use graph::{ prelude::{async_trait, o, BlockNumber, ChainStore, Error, Logger, LoggerFactory}, }; use prost::Message; +use std::collections::HashSet; use std::sync::Arc; use crate::adapter::TriggerFilter; use crate::codec::substreams_triggers::BlockAndReceipts; +use crate::codec::Block; use crate::data_source::{DataSourceTemplate, UnresolvedDataSourceTemplate}; use crate::trigger::{self, NearTrigger}; use crate::{ @@ -108,7 +110,6 @@ impl BlockStreamBuilder for NearStreamBuilder { chain.metrics_registry.clone(), ))) } - async fn build_firehose( &self, chain: &Chain, @@ -151,8 +152,9 @@ impl BlockStreamBuilder for NearStreamBuilder { _chain: &Chain, _deployment: DeploymentLocator, _start_blocks: Vec, + _source_subgraph_stores: Vec>, _subgraph_current_block: Option, - _filter: Arc<::TriggerFilter>, + _filter: Arc>, _unified_api_version: UnifiedMappingApiVersion, ) -> Result>> { todo!() @@ -230,7 +232,8 @@ impl Blockchain for Chain { deployment: DeploymentLocator, store: impl DeploymentCursorTracker, start_blocks: Vec, - filter: Arc, + _source_subgraph_stores: Vec>, + filter: Arc>, unified_api_version: UnifiedMappingApiVersion, ) -> Result>, Error> { if self.prefer_substreams { @@ -242,7 +245,7 @@ impl Blockchain for Chain { deployment, store.firehose_cursor(), store.block_ptr(), - filter, + filter.chain_filter.clone(), ) .await; } @@ -254,7 +257,7 @@ impl Blockchain for Chain { store.firehose_cursor(), start_blocks, store.block_ptr(), - filter, + filter.chain_filter.clone(), unified_api_version, ) .await @@ -322,6 +325,18 @@ impl TriggersAdapterTrait for TriggersAdapter { panic!("Should never be called since not used by FirehoseBlockStream") } + async fn load_block_ptrs_by_numbers( + &self, + _logger: Logger, + _block_numbers: HashSet, + ) -> Result> { + unimplemented!() + } + + async fn chain_head_ptr(&self) -> Result, Error> { + unimplemented!() + } + async fn triggers_in_block( &self, logger: &Logger, @@ -462,11 +477,13 @@ impl BlockStreamMapper for FirehoseMapper { .into_iter() .zip(receipt.into_iter()) .map(|(outcome, receipt)| { - NearTrigger::Receipt(Arc::new(trigger::ReceiptWithOutcome { - outcome, - receipt, - block: arc_block.clone(), - })) + Trigger::Chain(NearTrigger::Receipt(Arc::new( + trigger::ReceiptWithOutcome { + outcome, + receipt, + block: arc_block.clone(), + }, + ))) }) .collect(); @@ -973,8 +990,8 @@ mod test { .trigger_data .clone() .into_iter() - .filter_map(|x| match x { - crate::trigger::NearTrigger::Block(b) => b.header.clone().map(|x| x.height), + .filter_map(|x| match x.as_chain() { + Some(crate::trigger::NearTrigger::Block(b)) => b.header.clone().map(|x| x.height), _ => None, }) .collect() diff --git a/chain/starknet/src/chain.rs b/chain/starknet/src/chain.rs index cd10af5f965..d32f831d3a6 100644 --- a/chain/starknet/src/chain.rs +++ b/chain/starknet/src/chain.rs @@ -11,12 +11,12 @@ use graph::{ firehose_block_stream::FirehoseBlockStream, BasicBlockchainBuilder, Block, BlockIngestor, BlockPtr, Blockchain, BlockchainBuilder, BlockchainKind, EmptyNodeCapabilities, IngestorError, NoopDecoderHook, NoopRuntimeAdapter, - RuntimeAdapter as RuntimeAdapterTrait, + RuntimeAdapter as RuntimeAdapterTrait, TriggerFilterWrapper, }, cheap_clone::CheapClone, components::{ adapter::ChainId, - store::{DeploymentCursorTracker, DeploymentLocator}, + store::{DeploymentCursorTracker, DeploymentLocator, SourceableStore}, }, data::subgraph::UnifiedMappingApiVersion, env::EnvVars, @@ -30,7 +30,7 @@ use graph::{ slog::o, }; use prost::Message; -use std::sync::Arc; +use std::{collections::HashSet, sync::Arc}; use crate::{ adapter::TriggerFilter, @@ -39,6 +39,7 @@ use crate::{ DataSource, DataSourceTemplate, UnresolvedDataSource, UnresolvedDataSourceTemplate, }, trigger::{StarknetBlockTrigger, StarknetEventTrigger, StarknetTrigger}, + Block as StarknetBlock, }; pub struct Chain { @@ -115,7 +116,8 @@ impl Blockchain for Chain { deployment: DeploymentLocator, store: impl DeploymentCursorTracker, start_blocks: Vec, - filter: Arc, + _source_subgraph_stores: Vec>, + filter: Arc>, unified_api_version: UnifiedMappingApiVersion, ) -> Result>, Error> { self.block_stream_builder @@ -125,7 +127,7 @@ impl Blockchain for Chain { store.firehose_cursor(), start_blocks, store.block_ptr(), - filter, + filter.chain_filter.clone(), unified_api_version, ) .await @@ -238,8 +240,9 @@ impl BlockStreamBuilder for StarknetStreamBuilder { _chain: &Chain, _deployment: DeploymentLocator, _start_blocks: Vec, + _source_subgraph_stores: Vec>, _subgraph_current_block: Option, - _filter: Arc, + _filter: Arc>, _unified_api_version: UnifiedMappingApiVersion, ) -> Result>> { panic!("StarkNet does not support polling block stream") @@ -369,6 +372,18 @@ impl TriggersAdapterTrait for TriggersAdapter { panic!("Should never be called since FirehoseBlockStream cannot resolve it") } + async fn load_block_ptrs_by_numbers( + &self, + _logger: Logger, + _block_numbers: HashSet, + ) -> Result, Error> { + unimplemented!() + } + + async fn chain_head_ptr(&self) -> Result, Error> { + unimplemented!() + } + // Returns a sequence of blocks in increasing order of block number. // Each block will include all of its triggers that match the given `filter`. // The sequence may omit blocks that contain no triggers, @@ -379,7 +394,7 @@ impl TriggersAdapterTrait for TriggersAdapter { &self, _from: BlockNumber, _to: BlockNumber, - _filter: &crate::adapter::TriggerFilter, + _filter: &TriggerFilter, ) -> Result<(Vec>, BlockNumber), Error> { panic!("Should never be called since not used by FirehoseBlockStream") } diff --git a/chain/substreams/src/block_stream.rs b/chain/substreams/src/block_stream.rs index 8844df0610e..8008694f66b 100644 --- a/chain/substreams/src/block_stream.rs +++ b/chain/substreams/src/block_stream.rs @@ -7,9 +7,9 @@ use graph::{ BlockStream, BlockStreamBuilder as BlockStreamBuilderTrait, FirehoseCursor, }, substreams_block_stream::SubstreamsBlockStream, - Blockchain, + Blockchain, TriggerFilterWrapper, }, - components::store::DeploymentLocator, + components::store::{DeploymentLocator, SourceableStore}, data::subgraph::UnifiedMappingApiVersion, prelude::{async_trait, BlockNumber, BlockPtr}, schema::InputSchema, @@ -104,8 +104,9 @@ impl BlockStreamBuilderTrait for BlockStreamBuilder { _chain: &Chain, _deployment: DeploymentLocator, _start_blocks: Vec, + _source_subgraph_stores: Vec>, _subgraph_current_block: Option, - _filter: Arc, + _filter: Arc>, _unified_api_version: UnifiedMappingApiVersion, ) -> Result>> { unimplemented!("polling block stream is not support for substreams") diff --git a/chain/substreams/src/chain.rs b/chain/substreams/src/chain.rs index 28ef4bdc38b..1b2323a9bbd 100644 --- a/chain/substreams/src/chain.rs +++ b/chain/substreams/src/chain.rs @@ -4,10 +4,10 @@ use anyhow::Error; use graph::blockchain::client::ChainClient; use graph::blockchain::{ BasicBlockchainBuilder, BlockIngestor, BlockTime, EmptyNodeCapabilities, NoopDecoderHook, - NoopRuntimeAdapter, + NoopRuntimeAdapter, TriggerFilterWrapper, }; use graph::components::adapter::ChainId; -use graph::components::store::DeploymentCursorTracker; +use graph::components::store::{DeploymentCursorTracker, SourceableStore}; use graph::env::EnvVars; use graph::prelude::{BlockHash, CheapClone, Entity, LoggerFactory, MetricsRegistry}; use graph::schema::EntityKey; @@ -140,7 +140,8 @@ impl Blockchain for Chain { deployment: DeploymentLocator, store: impl DeploymentCursorTracker, _start_blocks: Vec, - filter: Arc, + _source_subgraph_stores: Vec>, + filter: Arc>, _unified_api_version: UnifiedMappingApiVersion, ) -> Result>, Error> { self.block_stream_builder @@ -150,7 +151,7 @@ impl Blockchain for Chain { deployment, store.firehose_cursor(), store.block_ptr(), - filter, + filter.chain_filter.clone(), ) .await } diff --git a/chain/substreams/src/trigger.rs b/chain/substreams/src/trigger.rs index 2b47e4e57b8..4dd8d5f74ac 100644 --- a/chain/substreams/src/trigger.rs +++ b/chain/substreams/src/trigger.rs @@ -16,7 +16,7 @@ use graph::{ }; use graph_runtime_wasm::module::ToAscPtr; use lazy_static::__Deref; -use std::sync::Arc; +use std::{collections::HashSet, sync::Arc}; use crate::{Block, Chain, NoopDataSourceTemplate, ParsedChanges}; @@ -136,6 +136,18 @@ impl blockchain::TriggersAdapter for TriggersAdapter { unimplemented!() } + async fn load_block_ptrs_by_numbers( + &self, + _logger: Logger, + _block_numbers: HashSet, + ) -> Result, Error> { + unimplemented!() + } + + async fn chain_head_ptr(&self) -> Result, Error> { + unimplemented!() + } + async fn scan_triggers( &self, _from: BlockNumber, @@ -225,7 +237,7 @@ where logger, ); - state.entity_cache.set(key, entity)?; + state.entity_cache.set(key, entity, block.number)?; } ParsedChanges::Delete(entity_key) => { let entity_type = entity_key.entity_type.cheap_clone(); diff --git a/core/src/subgraph/context/instance/hosts.rs b/core/src/subgraph/context/instance/hosts.rs index 73701fadb29..9c18e12ce1e 100644 --- a/core/src/subgraph/context/instance/hosts.rs +++ b/core/src/subgraph/context/instance/hosts.rs @@ -57,7 +57,7 @@ impl> OnchainHosts { } pub fn push(&mut self, host: Arc) { - assert!(host.data_source().as_onchain().is_some()); + assert!(host.data_source().is_chain_based()); self.hosts.push(host.cheap_clone()); let idx = self.hosts.len() - 1; @@ -194,7 +194,7 @@ impl> OffchainHosts { pub fn matches_by_address<'a>( &'a self, address: Option<&[u8]>, - ) -> Box + Send + 'a> { + ) -> Box + Send + 'a> { let Some(address) = address else { return Box::new(self.by_block.values().flatten().map(|host| host.as_ref())); }; diff --git a/core/src/subgraph/context/instance/mod.rs b/core/src/subgraph/context/instance/mod.rs index ed242836a28..86b64195493 100644 --- a/core/src/subgraph/context/instance/mod.rs +++ b/core/src/subgraph/context/instance/mod.rs @@ -22,13 +22,17 @@ pub(crate) struct SubgraphInstance> { pub(super) static_data_sources: Arc>>, host_metrics: Arc, - /// The hosts represent the data sources in the subgraph. There is one host per data source. + /// The hosts represent the onchain data sources in the subgraph. There is one host per data source. /// Data sources with no mappings (e.g. direct substreams) have no host. /// /// Onchain hosts must be created in increasing order of block number. `fn hosts_for_trigger` /// will return the onchain hosts in the same order as they were inserted. onchain_hosts: OnchainHosts, + /// `subgraph_hosts` represent subgraph data sources declared in the manifest. These are a special + /// kind of data source that depends on the data from another source subgraph. + subgraph_hosts: OnchainHosts, + offchain_hosts: OffchainHosts, /// Maps the hash of a module to a channel to the thread in which the module is instantiated. @@ -79,6 +83,7 @@ where network, static_data_sources: Arc::new(manifest.data_sources), onchain_hosts: OnchainHosts::new(), + subgraph_hosts: OnchainHosts::new(), offchain_hosts: OffchainHosts::new(), module_cache: HashMap::new(), templates, @@ -138,34 +143,44 @@ where ); } - let is_onchain = data_source.is_onchain(); let Some(host) = self.new_host(logger.clone(), data_source)? else { return Ok(None); }; // Check for duplicates and add the host. - if is_onchain { - // `onchain_hosts` will remain ordered by the creation block. - // See also 8f1bca33-d3b7-4035-affc-fd6161a12448. - ensure!( - self.onchain_hosts - .last() - .and_then(|h| h.creation_block_number()) - <= host.data_source().creation_block(), - ); + match host.data_source() { + DataSource::Onchain(_) => { + // `onchain_hosts` will remain ordered by the creation block. + // See also 8f1bca33-d3b7-4035-affc-fd6161a12448. + ensure!( + self.onchain_hosts + .last() + .and_then(|h| h.creation_block_number()) + <= host.data_source().creation_block(), + ); - if self.onchain_hosts.contains(&host) { - Ok(None) - } else { - self.onchain_hosts.push(host.cheap_clone()); - Ok(Some(host)) + if self.onchain_hosts.contains(&host) { + Ok(None) + } else { + self.onchain_hosts.push(host.cheap_clone()); + Ok(Some(host)) + } } - } else { - if self.offchain_hosts.contains(&host) { - Ok(None) - } else { - self.offchain_hosts.push(host.cheap_clone()); - Ok(Some(host)) + DataSource::Offchain(_) => { + if self.offchain_hosts.contains(&host) { + Ok(None) + } else { + self.offchain_hosts.push(host.cheap_clone()); + Ok(Some(host)) + } + } + DataSource::Subgraph(_) => { + if self.subgraph_hosts.contains(&host) { + Ok(None) + } else { + self.subgraph_hosts.push(host.cheap_clone()); + Ok(Some(host)) + } } } } @@ -226,6 +241,9 @@ where TriggerData::Offchain(trigger) => self .offchain_hosts .matches_by_address(trigger.source.address().as_ref().map(|a| a.as_slice())), + TriggerData::Subgraph(trigger) => self + .subgraph_hosts + .matches_by_address(Some(trigger.source.to_bytes().as_slice())), } } diff --git a/core/src/subgraph/context/mod.rs b/core/src/subgraph/context/mod.rs index 6ffc5a5aa12..7b7686a04fb 100644 --- a/core/src/subgraph/context/mod.rs +++ b/core/src/subgraph/context/mod.rs @@ -6,7 +6,7 @@ use crate::polling_monitor::{ use anyhow::{self, Error}; use bytes::Bytes; use graph::{ - blockchain::{BlockTime, Blockchain}, + blockchain::{BlockTime, Blockchain, TriggerFilterWrapper}, components::{ store::{DeploymentId, SubgraphFork}, subgraph::{HostMetrics, MappingError, RuntimeHost as _, SharedProofOfIndexing}, @@ -73,7 +73,7 @@ where pub(crate) instance: SubgraphInstance, pub instances: SubgraphKeepAlive, pub offchain_monitor: OffchainMonitor, - pub filter: Option, + pub filter: Option>, pub(crate) trigger_processor: Box>, pub(crate) decoder: Box>, } diff --git a/core/src/subgraph/inputs.rs b/core/src/subgraph/inputs.rs index b2e95c753f5..4fb080bb50b 100644 --- a/core/src/subgraph/inputs.rs +++ b/core/src/subgraph/inputs.rs @@ -1,7 +1,7 @@ use graph::{ - blockchain::{Blockchain, TriggersAdapter}, + blockchain::{block_stream::TriggersAdapterWrapper, Blockchain}, components::{ - store::{DeploymentLocator, SubgraphFork, WritableStore}, + store::{DeploymentLocator, SourceableStore, SubgraphFork, WritableStore}, subgraph::ProofOfIndexingVersion, }, data::subgraph::{SubgraphFeature, UnifiedMappingApiVersion}, @@ -16,10 +16,11 @@ pub struct IndexingInputs { pub features: BTreeSet, pub start_blocks: Vec, pub end_blocks: BTreeSet, + pub source_subgraph_stores: Vec>, pub stop_block: Option, pub store: Arc, pub debug_fork: Option>, - pub triggers_adapter: Arc>, + pub triggers_adapter: Arc>, pub chain: Arc, pub templates: Arc>>, pub unified_api_version: UnifiedMappingApiVersion, @@ -39,6 +40,7 @@ impl IndexingInputs { features, start_blocks, end_blocks, + source_subgraph_stores, stop_block, store: _, debug_fork, @@ -56,6 +58,7 @@ impl IndexingInputs { features: features.clone(), start_blocks: start_blocks.clone(), end_blocks: end_blocks.clone(), + source_subgraph_stores: source_subgraph_stores.clone(), stop_block: stop_block.clone(), store, debug_fork: debug_fork.clone(), diff --git a/core/src/subgraph/instance_manager.rs b/core/src/subgraph/instance_manager.rs index c98641539d9..2dca24cee9d 100644 --- a/core/src/subgraph/instance_manager.rs +++ b/core/src/subgraph/instance_manager.rs @@ -6,9 +6,10 @@ use crate::subgraph::Decoder; use std::collections::BTreeSet; use crate::subgraph::runner::SubgraphRunner; -use graph::blockchain::block_stream::BlockStreamMetrics; +use graph::blockchain::block_stream::{BlockStreamMetrics, TriggersAdapterWrapper}; use graph::blockchain::{Blockchain, BlockchainKind, DataSource, NodeCapabilities}; use graph::components::metrics::gas::GasMetrics; +use graph::components::store::SourceableStore; use graph::components::subgraph::ProofOfIndexingVersion; use graph::data::subgraph::{UnresolvedSubgraphManifest, SPEC_VERSION_0_0_6}; use graph::data::value::Word; @@ -202,6 +203,30 @@ impl SubgraphInstanceManager { } } + pub async fn get_sourceable_stores( + &self, + hashes: Vec, + is_runner_test: bool, + ) -> anyhow::Result>> { + if is_runner_test { + return Ok(Vec::new()); + } + + let mut sourceable_stores = Vec::new(); + let subgraph_store = self.subgraph_store.clone(); + + for hash in hashes { + let loc = subgraph_store + .active_locator(&hash)? + .ok_or_else(|| anyhow!("no active deployment for hash {}", hash))?; + + let sourceable_store = subgraph_store.clone().sourceable(loc.id.clone()).await?; + sourceable_stores.push(sourceable_store); + } + + Ok(sourceable_stores) + } + pub async fn build_subgraph_runner( &self, logger: Logger, @@ -211,6 +236,26 @@ impl SubgraphInstanceManager { stop_block: Option, tp: Box>>, ) -> anyhow::Result>> + where + C: Blockchain, + ::MappingTrigger: ToAscPtr, + { + self.build_subgraph_runner_inner( + logger, env_vars, deployment, manifest, stop_block, tp, false, + ) + .await + } + + pub async fn build_subgraph_runner_inner( + &self, + logger: Logger, + env_vars: Arc, + deployment: DeploymentLocator, + manifest: serde_yaml::Mapping, + stop_block: Option, + tp: Box>>, + is_runner_test: bool, + ) -> anyhow::Result>> where C: Blockchain, ::MappingTrigger: ToAscPtr, @@ -307,6 +352,16 @@ impl SubgraphInstanceManager { .filter_map(|d| d.as_onchain().cloned()) .collect::>(); + let subgraph_data_sources = data_sources + .iter() + .filter_map(|d| d.as_subgraph()) + .collect::>(); + + let subgraph_ds_source_deployments = subgraph_data_sources + .iter() + .map(|d| d.source.address()) + .collect::>(); + let required_capabilities = C::NodeCapabilities::from_data_sources(&onchain_data_sources); let network: Word = manifest.network_name().into(); @@ -318,7 +373,7 @@ impl SubgraphInstanceManager { let start_blocks: Vec = data_sources .iter() - .filter_map(|d| d.as_onchain().map(|d: &C::DataSource| d.start_block())) + .filter_map(|d| d.start_block()) .collect(); let end_blocks: BTreeSet = manifest @@ -413,11 +468,21 @@ impl SubgraphInstanceManager { let decoder = Box::new(Decoder::new(decoder_hook)); + let subgraph_data_source_stores = self + .get_sourceable_stores::(subgraph_ds_source_deployments, is_runner_test) + .await?; + + let triggers_adapter = Arc::new(TriggersAdapterWrapper::new( + triggers_adapter, + subgraph_data_source_stores.clone(), + )); + let inputs = IndexingInputs { deployment: deployment.clone(), features, start_blocks, end_blocks, + source_subgraph_stores: subgraph_data_source_stores, stop_block, store, debug_fork, diff --git a/core/src/subgraph/registrar.rs b/core/src/subgraph/registrar.rs index fe80d118457..e8ad83315e0 100644 --- a/core/src/subgraph/registrar.rs +++ b/core/src/subgraph/registrar.rs @@ -629,7 +629,6 @@ async fn create_subgraph_version( ) .map_err(SubgraphRegistrarError::ResolveError) .await?; - // Determine if the graft_base should be validated. // Validate the graft_base if there is a pending graft, ensuring its presence. // If the subgraph is new (indicated by DeploymentNotFound), the graft_base should be validated. diff --git a/core/src/subgraph/runner.rs b/core/src/subgraph/runner.rs index cd341ce2f99..daf1c4cf6da 100644 --- a/core/src/subgraph/runner.rs +++ b/core/src/subgraph/runner.rs @@ -7,7 +7,10 @@ use atomic_refcell::AtomicRefCell; use graph::blockchain::block_stream::{ BlockStreamError, BlockStreamEvent, BlockWithTriggers, FirehoseCursor, }; -use graph::blockchain::{Block, BlockTime, Blockchain, DataSource as _, TriggerFilter as _}; +use graph::blockchain::{ + Block, BlockTime, Blockchain, DataSource as _, SubgraphFilter, Trigger, TriggerFilter as _, + TriggerFilterWrapper, +}; use graph::components::store::{EmptyStore, GetScope, ReadStore, StoredDynamicDataSource}; use graph::components::subgraph::InstanceDSTemplate; use graph::components::{ @@ -30,6 +33,7 @@ use graph::schema::EntityKey; use graph::util::{backoff::ExponentialBackoff, lfu_cache::LfuCache}; use std::sync::Arc; use std::time::{Duration, Instant}; +use std::vec; const MINUTE: Duration = Duration::from_secs(60); @@ -116,7 +120,7 @@ where self.inputs.static_filters || self.ctx.hosts_len() > ENV_VARS.static_filters_threshold } - fn build_filter(&self) -> C::TriggerFilter { + fn build_filter(&self) -> TriggerFilterWrapper { let current_ptr = self.inputs.store.block_ptr(); let static_filters = self.is_static_filters_enabled(); @@ -128,10 +132,30 @@ where None => true, }; + let data_sources = self.ctx.static_data_sources(); + + let subgraph_filter = data_sources + .iter() + .filter_map(|ds| ds.as_subgraph()) + .map(|ds| SubgraphFilter { + subgraph: ds.source.address(), + start_block: ds.source.start_block, + entities: ds + .mapping + .handlers + .iter() + .map(|handler| handler.entity.clone()) + .collect(), + }) + .collect::>(); + // if static_filters is not enabled we just stick to the filter based on all the data sources. if !static_filters { - return C::TriggerFilter::from_data_sources( - self.ctx.onchain_data_sources().filter(end_block_filter), + return TriggerFilterWrapper::new( + C::TriggerFilter::from_data_sources( + self.ctx.onchain_data_sources().filter(end_block_filter), + ), + subgraph_filter, ); } @@ -158,11 +182,11 @@ where filter.extend_with_template(templates.iter().filter_map(|ds| ds.as_onchain()).cloned()); - filter + TriggerFilterWrapper::new(filter, subgraph_filter) } #[cfg(debug_assertions)] - pub fn build_filter_for_test(&self) -> C::TriggerFilter { + pub fn build_filter_for_test(&self) -> TriggerFilterWrapper { self.build_filter() } @@ -209,7 +233,7 @@ where let mut block_stream = new_block_stream( &self.inputs, - self.ctx.filter.as_ref().unwrap(), // Safe to unwrap as we just called `build_filter` in the previous line + self.ctx.filter.clone().unwrap(), // Safe to unwrap as we just called `build_filter` in the previous line &self.metrics.subgraph, ) .await? @@ -323,7 +347,10 @@ where .match_and_decode_many( &logger, &block, - triggers.into_iter().map(TriggerData::Onchain), + triggers.into_iter().map(|t| match t { + Trigger::Chain(t) => TriggerData::Onchain(t), + Trigger::Subgraph(t) => TriggerData::Subgraph(t), + }), hosts_filter, &self.metrics.subgraph, ) @@ -421,9 +448,12 @@ where let (data_sources, runtime_hosts) = self.create_dynamic_data_sources(block_state.drain_created_data_sources())?; - let filter = C::TriggerFilter::from_data_sources( - data_sources.iter().filter_map(DataSource::as_onchain), - ); + let filter = &Arc::new(TriggerFilterWrapper::new( + C::TriggerFilter::from_data_sources( + data_sources.iter().filter_map(DataSource::as_onchain), + ), + vec![], + )); let block: Arc = if self.inputs.chain.is_refetch_block_required() { let cur = firehose_cursor.clone(); @@ -452,7 +482,7 @@ where let block_with_triggers = self .inputs .triggers_adapter - .triggers_in_block(&logger, block.as_ref().clone(), &filter) + .triggers_in_block(&logger, block.as_ref().clone(), filter) .await?; let triggers = block_with_triggers.trigger_data; @@ -482,7 +512,10 @@ where .match_and_decode_many( &logger, &block, - triggers.into_iter().map(TriggerData::Onchain), + triggers.into_iter().map(|t| match t { + Trigger::Chain(t) => TriggerData::Onchain(t), + Trigger::Subgraph(_) => unreachable!(), // TODO(krishna): Re-evaulate this + }), |_| Box::new(runtime_hosts.iter().map(Arc::as_ref)), &self.metrics.subgraph, ) @@ -1570,6 +1603,7 @@ async fn update_proof_of_indexing( key: EntityKey, digest: Bytes, block_time: BlockTime, + block: BlockNumber, ) -> Result<(), Error> { let digest_name = entity_cache.schema.poi_digest(); let mut data = vec![ @@ -1584,11 +1618,12 @@ async fn update_proof_of_indexing( data.push((entity_cache.schema.poi_block_time(), block_time)); } let poi = entity_cache.make_entity(data)?; - entity_cache.set(key, poi) + entity_cache.set(key, poi, block) } let _section_guard = stopwatch.start_section("update_proof_of_indexing"); + let block_number = proof_of_indexing.get_block(); let mut proof_of_indexing = proof_of_indexing.take(); for (causality_region, stream) in proof_of_indexing.drain() { @@ -1624,6 +1659,7 @@ async fn update_proof_of_indexing( entity_key, updated_proof_of_indexing, block_time, + block_number, )?; } diff --git a/core/src/subgraph/stream.rs b/core/src/subgraph/stream.rs index c1d767e3fcf..5547543f13d 100644 --- a/core/src/subgraph/stream.rs +++ b/core/src/subgraph/stream.rs @@ -1,13 +1,13 @@ use crate::subgraph::inputs::IndexingInputs; use anyhow::bail; use graph::blockchain::block_stream::{BlockStream, BufferedBlockStream}; -use graph::blockchain::Blockchain; +use graph::blockchain::{Blockchain, TriggerFilterWrapper}; use graph::prelude::{CheapClone, Error, SubgraphInstanceMetrics}; use std::sync::Arc; pub async fn new_block_stream( inputs: &IndexingInputs, - filter: &C::TriggerFilter, + filter: TriggerFilterWrapper, metrics: &SubgraphInstanceMetrics, ) -> Result>, Error> { let is_firehose = inputs.chain.chain_client().is_firehose(); @@ -18,6 +18,7 @@ pub async fn new_block_stream( inputs.deployment.clone(), inputs.store.cheap_clone(), inputs.start_blocks.clone(), + inputs.source_subgraph_stores.clone(), Arc::new(filter.clone()), inputs.unified_api_version.clone(), ) diff --git a/graph/src/blockchain/block_stream.rs b/graph/src/blockchain/block_stream.rs index 25a923dd502..cfe36169810 100644 --- a/graph/src/blockchain/block_stream.rs +++ b/graph/src/blockchain/block_stream.rs @@ -1,3 +1,5 @@ +use crate::blockchain::SubgraphFilter; +use crate::data_source::{subgraph, CausalityRegion}; use crate::substreams::Clock; use crate::substreams_rpc::response::Message as SubstreamsMessage; use crate::substreams_rpc::BlockScopedData; @@ -5,6 +7,7 @@ use anyhow::Error; use async_stream::stream; use futures03::Stream; use prost_types::Any; +use std::collections::{BTreeMap, HashMap, HashSet}; use std::fmt; use std::sync::Arc; use std::time::Instant; @@ -12,13 +15,13 @@ use thiserror::Error; use tokio::sync::mpsc::{self, Receiver, Sender}; use super::substreams_block_stream::SubstreamsLogData; -use super::{Block, BlockPtr, BlockTime, Blockchain}; +use super::{Block, BlockPtr, BlockTime, Blockchain, Trigger, TriggerFilterWrapper}; use crate::anyhow::Result; -use crate::components::store::{BlockNumber, DeploymentLocator}; +use crate::components::store::{BlockNumber, DeploymentLocator, SourceableStore}; use crate::data::subgraph::UnifiedMappingApiVersion; use crate::firehose::{self, FirehoseEndpoint}; use crate::futures03::stream::StreamExt as _; -use crate::schema::InputSchema; +use crate::schema::{EntityType, InputSchema}; use crate::substreams_rpc::response::Message; use crate::{prelude::*, prometheus::labels}; @@ -144,10 +147,33 @@ pub trait BlockStreamBuilder: Send + Sync { chain: &C, deployment: DeploymentLocator, start_blocks: Vec, + source_subgraph_stores: Vec>, subgraph_current_block: Option, - filter: Arc, + filter: Arc>, unified_api_version: UnifiedMappingApiVersion, ) -> Result>>; + + async fn build_subgraph_block_stream( + &self, + chain: &C, + deployment: DeploymentLocator, + start_blocks: Vec, + source_subgraph_stores: Vec>, + subgraph_current_block: Option, + filter: Arc>, + unified_api_version: UnifiedMappingApiVersion, + ) -> Result>> { + self.build_polling( + chain, + deployment, + start_blocks, + source_subgraph_stores, + subgraph_current_block, + filter, + unified_api_version, + ) + .await + } } #[derive(Debug, Clone)] @@ -198,7 +224,7 @@ impl AsRef> for FirehoseCursor { #[derive(Debug)] pub struct BlockWithTriggers { pub block: C::Block, - pub trigger_data: Vec, + pub trigger_data: Vec>, } impl Clone for BlockWithTriggers @@ -216,7 +242,31 @@ where impl BlockWithTriggers { /// Creates a BlockWithTriggers structure, which holds /// the trigger data ordered and without any duplicates. - pub fn new(block: C::Block, mut trigger_data: Vec, logger: &Logger) -> Self { + pub fn new(block: C::Block, trigger_data: Vec, logger: &Logger) -> Self { + Self::new_with_triggers( + block, + trigger_data.into_iter().map(Trigger::Chain).collect(), + logger, + ) + } + + pub fn new_with_subgraph_triggers( + block: C::Block, + trigger_data: Vec, + logger: &Logger, + ) -> Self { + Self::new_with_triggers( + block, + trigger_data.into_iter().map(Trigger::Subgraph).collect(), + logger, + ) + } + + fn new_with_triggers( + block: C::Block, + mut trigger_data: Vec>, + logger: &Logger, + ) -> Self { // This is where triggers get sorted. trigger_data.sort(); @@ -256,6 +306,245 @@ impl BlockWithTriggers { pub fn parent_ptr(&self) -> Option { self.block.parent_ptr() } + + pub fn extend_triggers(&mut self, triggers: Vec>) { + self.trigger_data.extend(triggers); + self.trigger_data.sort(); + } +} + +/// The `TriggersAdapterWrapper` wraps the chain-specific `TriggersAdapter`, enabling chain-agnostic +/// handling of subgraph datasource triggers. Without this wrapper, we would have to duplicate the same +/// logic for each chain, increasing code repetition. +pub struct TriggersAdapterWrapper { + pub adapter: Arc>, + pub source_subgraph_stores: HashMap>, +} + +impl TriggersAdapterWrapper { + pub fn new( + adapter: Arc>, + source_subgraph_stores: Vec>, + ) -> Self { + let stores_map: HashMap<_, _> = source_subgraph_stores + .iter() + .map(|store| (store.input_schema().id().clone(), store.clone())) + .collect(); + Self { + adapter, + source_subgraph_stores: stores_map, + } + } + + pub async fn blocks_with_subgraph_triggers( + &self, + logger: &Logger, + subgraph_filter: &SubgraphFilter, + range: SubgraphTriggerScanRange, + ) -> Result>, Error> { + let store = self + .source_subgraph_stores + .get(&subgraph_filter.subgraph) + .ok_or_else(|| anyhow!("Store not found for subgraph: {}", subgraph_filter.subgraph))?; + + let schema = ::input_schema(store); + + let adapter = self.adapter.clone(); + + scan_subgraph_triggers::(logger, store, &adapter, &schema, &subgraph_filter, range).await + } +} + +fn create_subgraph_trigger_from_entities( + filter: &SubgraphFilter, + entities: Vec, +) -> Vec { + entities + .into_iter() + .map(|entity| subgraph::TriggerData { + source: filter.subgraph.clone(), + entity, + }) + .collect() +} + +async fn create_subgraph_triggers( + logger: Logger, + blocks: Vec, + filter: &SubgraphFilter, + mut entities: BTreeMap>, +) -> Result>, Error> { + let logger_clone = logger.cheap_clone(); + + let blocks: Vec> = blocks + .into_iter() + .map(|block| { + let block_number = block.number(); + let trigger_data = entities + .remove(&block_number) + .map(|e| create_subgraph_trigger_from_entities(filter, e)) + .unwrap_or_else(Vec::new); + + BlockWithTriggers::new_with_subgraph_triggers(block, trigger_data, &logger_clone) + }) + .collect(); + + Ok(blocks) +} + +pub enum SubgraphTriggerScanRange { + Single(C::Block), + Range(BlockNumber, BlockNumber), +} + +async fn scan_subgraph_triggers( + logger: &Logger, + store: &Arc, + adapter: &Arc>, + schema: &InputSchema, + filter: &SubgraphFilter, + range: SubgraphTriggerScanRange, +) -> Result>, Error> { + match range { + SubgraphTriggerScanRange::Single(block) => { + let entities = + get_entities_for_range(store, filter, schema, block.number(), block.number()) + .await?; + create_subgraph_triggers::(logger.clone(), vec![block], filter, entities).await + } + SubgraphTriggerScanRange::Range(from, to) => { + let entities = get_entities_for_range(store, filter, schema, from, to).await?; + let mut block_numbers: HashSet = entities.keys().cloned().collect(); + // Ensure the 'to' block is included in the block_numbers + block_numbers.insert(to); + + let blocks = adapter + .load_block_ptrs_by_numbers(logger.clone(), block_numbers) + .await?; + + create_subgraph_triggers::(logger.clone(), blocks, filter, entities).await + } + } +} + +#[derive(Debug, Clone, Eq, PartialEq)] +pub enum EntityOperationKind { + Create, + Modify, + Delete, +} + +#[derive(Debug, Clone, Eq, PartialEq)] +pub struct EntitySourceOperation { + pub entity_op: EntityOperationKind, + pub entity_type: EntityType, + pub entity: Entity, + pub vid: i64, +} + +async fn get_entities_for_range( + store: &Arc, + filter: &SubgraphFilter, + schema: &InputSchema, + from: BlockNumber, + to: BlockNumber, +) -> Result>, Error> { + let entity_types: Result> = filter + .entities + .iter() + .map(|name| schema.entity_type(name)) + .collect(); + Ok(store.get_range(entity_types?, CausalityRegion::ONCHAIN, from..to)?) +} + +impl TriggersAdapterWrapper { + pub async fn ancestor_block( + &self, + ptr: BlockPtr, + offset: BlockNumber, + root: Option, + ) -> Result, Error> { + self.adapter.ancestor_block(ptr, offset, root).await + } + + pub async fn scan_triggers( + &self, + logger: &Logger, + from: BlockNumber, + to: BlockNumber, + filter: &Arc>, + ) -> Result<(Vec>, BlockNumber), Error> { + if let Some(subgraph_filter) = filter.subgraph_filter.first() { + let blocks_with_triggers = self + .blocks_with_subgraph_triggers( + logger, + subgraph_filter, + SubgraphTriggerScanRange::Range(from, to), + ) + .await?; + + return Ok((blocks_with_triggers, to)); + } + + self.adapter + .scan_triggers(from, to, &filter.chain_filter) + .await + } + + pub async fn triggers_in_block( + &self, + logger: &Logger, + block: C::Block, + filter: &Arc>, + ) -> Result, Error> { + trace!( + logger, + "triggers_in_block"; + "block_number" => block.number(), + "block_hash" => block.hash().hash_hex(), + ); + + if let Some(subgraph_filter) = filter.subgraph_filter.first() { + let blocks_with_triggers = self + .blocks_with_subgraph_triggers( + logger, + subgraph_filter, + SubgraphTriggerScanRange::Single(block), + ) + .await?; + + return Ok(blocks_with_triggers.into_iter().next().unwrap()); + } + + self.adapter + .triggers_in_block(logger, block, &filter.chain_filter) + .await + } + + pub async fn is_on_main_chain(&self, ptr: BlockPtr) -> Result { + self.adapter.is_on_main_chain(ptr).await + } + + pub async fn parent_ptr(&self, block: &BlockPtr) -> Result, Error> { + self.adapter.parent_ptr(block).await + } + + pub async fn chain_head_ptr(&self) -> Result, Error> { + if self.source_subgraph_stores.is_empty() { + return self.adapter.chain_head_ptr().await; + } + + let ptrs = futures03::future::try_join_all( + self.source_subgraph_stores + .iter() + .map(|(_, store)| store.block_ptr()), + ) + .await?; + + let min_ptr = ptrs.into_iter().flatten().min_by_key(|ptr| ptr.number); + + Ok(min_ptr) + } } #[async_trait] @@ -298,6 +587,15 @@ pub trait TriggersAdapter: Send + Sync { /// Get pointer to parent of `block`. This is called when reverting `block`. async fn parent_ptr(&self, block: &BlockPtr) -> Result, Error>; + + /// Get pointer to parent of `block`. This is called when reverting `block`. + async fn chain_head_ptr(&self) -> Result, Error>; + + async fn load_block_ptrs_by_numbers( + &self, + logger: Logger, + block_numbers: HashSet, + ) -> Result>; } #[async_trait] diff --git a/graph/src/blockchain/client.rs b/graph/src/blockchain/client.rs index 8d83536b577..1ac1b4f892c 100644 --- a/graph/src/blockchain/client.rs +++ b/graph/src/blockchain/client.rs @@ -41,7 +41,7 @@ impl ChainClient { pub fn rpc(&self) -> anyhow::Result<&C::Client> { match self { Self::Rpc(rpc) => Ok(rpc), - _ => Err(anyhow!("rpc endpoint requested on firehose chain client")), + Self::Firehose(_) => Err(anyhow!("rpc endpoint requested on firehose chain client")), } } } diff --git a/graph/src/blockchain/mock.rs b/graph/src/blockchain/mock.rs index c89eca95727..ed602b7f745 100644 --- a/graph/src/blockchain/mock.rs +++ b/graph/src/blockchain/mock.rs @@ -2,22 +2,34 @@ use crate::{ bail, components::{ link_resolver::LinkResolver, - store::{BlockNumber, DeploymentCursorTracker, DeploymentLocator}, + store::{BlockNumber, DeploymentCursorTracker, DeploymentLocator, SourceableStore}, subgraph::InstanceDSTemplateInfo, }, data::subgraph::UnifiedMappingApiVersion, - prelude::{BlockHash, DataSourceTemplateInfo}, + data_source, + prelude::{ + transaction_receipt::LightTransactionReceipt, BlockHash, ChainStore, + DataSourceTemplateInfo, StoreError, + }, }; -use anyhow::Error; +use anyhow::{Error, Result}; use async_trait::async_trait; use serde::Deserialize; -use std::{collections::HashSet, convert::TryFrom, sync::Arc}; +use serde_json::Value; +use slog::Logger; +use std::{ + collections::{BTreeMap, HashMap, HashSet}, + convert::TryFrom, + sync::Arc, +}; +use web3::types::H256; use super::{ block_stream::{self, BlockStream, FirehoseCursor}, client::ChainClient, - BlockIngestor, BlockTime, EmptyNodeCapabilities, HostFn, IngestorError, MappingTriggerTrait, - NoopDecoderHook, TriggerWithHandler, + BlockIngestor, BlockTime, ChainIdentifier, EmptyNodeCapabilities, ExtendedBlockPtr, HostFn, + IngestorError, MappingTriggerTrait, NoopDecoderHook, Trigger, TriggerFilterWrapper, + TriggerWithHandler, }; use super::{ @@ -218,31 +230,49 @@ impl UnresolvedDataSourceTemplate for MockUnresolvedDataSource pub struct MockTriggersAdapter; #[async_trait] -impl TriggersAdapter for MockTriggersAdapter { +impl TriggersAdapter for MockTriggersAdapter { async fn ancestor_block( &self, _ptr: BlockPtr, _offset: BlockNumber, _root: Option, - ) -> Result, Error> { + ) -> Result, Error> { todo!() } + async fn load_block_ptrs_by_numbers( + &self, + _logger: Logger, + _block_numbers: HashSet, + ) -> Result> { + unimplemented!() + } + + async fn chain_head_ptr(&self) -> Result, Error> { + unimplemented!() + } + async fn scan_triggers( &self, - _from: crate::components::store::BlockNumber, - _to: crate::components::store::BlockNumber, - _filter: &C::TriggerFilter, - ) -> Result<(Vec>, BlockNumber), Error> { - todo!() + from: crate::components::store::BlockNumber, + to: crate::components::store::BlockNumber, + filter: &MockTriggerFilter, + ) -> Result< + ( + Vec>, + BlockNumber, + ), + Error, + > { + blocks_with_triggers(from, to, filter).await } async fn triggers_in_block( &self, _logger: &slog::Logger, - _block: C::Block, - _filter: &C::TriggerFilter, - ) -> Result, Error> { + _block: MockBlock, + _filter: &MockTriggerFilter, + ) -> Result, Error> { todo!() } @@ -255,6 +285,26 @@ impl TriggersAdapter for MockTriggersAdapter { } } +async fn blocks_with_triggers( + _from: crate::components::store::BlockNumber, + to: crate::components::store::BlockNumber, + _filter: &MockTriggerFilter, +) -> Result< + ( + Vec>, + BlockNumber, + ), + Error, +> { + Ok(( + vec![BlockWithTriggers { + block: MockBlock { number: 0 }, + trigger_data: vec![Trigger::Chain(MockTriggerData)], + }], + to, + )) +} + #[derive(Clone, Debug, Eq, PartialEq, Ord, PartialOrd)] pub struct MockTriggerData; @@ -303,7 +353,7 @@ impl TriggerFilter for MockTriggerFilter { pub struct MockRuntimeAdapter; impl RuntimeAdapter for MockRuntimeAdapter { - fn host_fns(&self, _ds: &C::DataSource) -> Result, Error> { + fn host_fns(&self, _ds: &data_source::DataSource) -> Result, Error> { todo!() } } @@ -347,7 +397,8 @@ impl Blockchain for MockBlockchain { _deployment: DeploymentLocator, _store: impl DeploymentCursorTracker, _start_blocks: Vec, - _filter: Arc, + _source_subgraph_stores: Vec>, + _filter: Arc>, _unified_api_version: UnifiedMappingApiVersion, ) -> Result>, Error> { todo!() @@ -391,3 +442,105 @@ impl Blockchain for MockBlockchain { todo!() } } + +// Mock implementation +#[derive(Default)] +pub struct MockChainStore { + pub blocks: BTreeMap>, +} + +#[async_trait] +impl ChainStore for MockChainStore { + async fn block_ptrs_by_numbers( + self: Arc, + numbers: Vec, + ) -> Result>, Error> { + let mut result = BTreeMap::new(); + for num in numbers { + if let Some(blocks) = self.blocks.get(&num) { + result.insert(num, blocks.clone()); + } + } + Ok(result) + } + + // Implement other required methods with minimal implementations + fn genesis_block_ptr(&self) -> Result { + unimplemented!() + } + async fn upsert_block(&self, _block: Arc) -> Result<(), Error> { + unimplemented!() + } + fn upsert_light_blocks(&self, _blocks: &[&dyn Block]) -> Result<(), Error> { + unimplemented!() + } + async fn attempt_chain_head_update( + self: Arc, + _ancestor_count: BlockNumber, + ) -> Result, Error> { + unimplemented!() + } + async fn chain_head_ptr(self: Arc) -> Result, Error> { + unimplemented!() + } + fn chain_head_cursor(&self) -> Result, Error> { + unimplemented!() + } + async fn set_chain_head( + self: Arc, + _block: Arc, + _cursor: String, + ) -> Result<(), Error> { + unimplemented!() + } + async fn blocks(self: Arc, _hashes: Vec) -> Result, Error> { + unimplemented!() + } + async fn ancestor_block( + self: Arc, + _block_ptr: BlockPtr, + _offset: BlockNumber, + _root: Option, + ) -> Result, Error> { + unimplemented!() + } + fn cleanup_cached_blocks( + &self, + _ancestor_count: BlockNumber, + ) -> Result, Error> { + unimplemented!() + } + fn block_hashes_by_block_number(&self, _number: BlockNumber) -> Result, Error> { + unimplemented!() + } + fn confirm_block_hash(&self, _number: BlockNumber, _hash: &BlockHash) -> Result { + unimplemented!() + } + async fn block_number( + &self, + _hash: &BlockHash, + ) -> Result, Option)>, StoreError> { + unimplemented!() + } + async fn block_numbers( + &self, + _hashes: Vec, + ) -> Result, StoreError> { + unimplemented!() + } + async fn transaction_receipts_in_block( + &self, + _block_ptr: &H256, + ) -> Result, StoreError> { + unimplemented!() + } + async fn clear_call_cache(&self, _from: BlockNumber, _to: BlockNumber) -> Result<(), Error> { + unimplemented!() + } + fn chain_identifier(&self) -> Result { + unimplemented!() + } + fn set_chain_identifier(&self, _ident: &ChainIdentifier) -> Result<(), Error> { + unimplemented!() + } +} diff --git a/graph/src/blockchain/mod.rs b/graph/src/blockchain/mod.rs index 73cac816728..c407fd4c4f1 100644 --- a/graph/src/blockchain/mod.rs +++ b/graph/src/blockchain/mod.rs @@ -20,13 +20,15 @@ use crate::{ components::{ adapter::ChainId, metrics::subgraph::SubgraphInstanceMetrics, - store::{DeploymentCursorTracker, DeploymentLocator, StoredDynamicDataSource}, + store::{ + DeploymentCursorTracker, DeploymentLocator, SourceableStore, StoredDynamicDataSource, + }, subgraph::{HostMetrics, InstanceDSTemplateInfo, MappingError}, trigger_processor::RunnableTriggers, }, data::subgraph::{UnifiedMappingApiVersion, MIN_SPEC_VERSION}, - data_source::{self, DataSourceTemplateInfo}, - prelude::DataSourceContext, + data_source::{self, subgraph, DataSourceTemplateInfo}, + prelude::{DataSourceContext, DeploymentHash}, runtime::{gas::GasCounter, AscHeap, HostExportError}, }; use crate::{ @@ -52,7 +54,7 @@ pub use block_stream::{ChainHeadUpdateListener, ChainHeadUpdateStream, TriggersA pub use builder::{BasicBlockchainBuilder, BlockchainBuilder}; pub use empty_node_capabilities::EmptyNodeCapabilities; pub use noop_runtime_adapter::NoopRuntimeAdapter; -pub use types::{BlockHash, BlockPtr, BlockTime, ChainIdentifier}; +pub use types::{BlockHash, BlockPtr, BlockTime, ChainIdentifier, ExtendedBlockPtr}; use self::{ block_stream::{BlockStream, FirehoseCursor}, @@ -189,7 +191,8 @@ pub trait Blockchain: Debug + Sized + Send + Sync + Unpin + 'static { deployment: DeploymentLocator, store: impl DeploymentCursorTracker, start_blocks: Vec, - filter: Arc, + source_subgraph_stores: Vec>, + filter: Arc>, unified_api_version: UnifiedMappingApiVersion, ) -> Result>, Error>; @@ -247,6 +250,42 @@ impl From for IngestorError { } } +/// The `TriggerFilterWrapper` is a higher-level wrapper around the chain-specific `TriggerFilter`, +/// enabling subgraph-based trigger filtering for subgraph datasources. This abstraction is necessary +/// because subgraph filtering operates at a higher level than chain-based filtering. By using this wrapper, +/// we reduce code duplication, allowing subgraph-based filtering to be implemented once, instead of +/// duplicating it across different chains. +#[derive(Debug)] +pub struct TriggerFilterWrapper { + pub chain_filter: Arc, + pub subgraph_filter: Vec, +} + +#[derive(Clone, Debug)] +pub struct SubgraphFilter { + pub subgraph: DeploymentHash, + pub start_block: BlockNumber, + pub entities: Vec, +} + +impl TriggerFilterWrapper { + pub fn new(filter: C::TriggerFilter, subgraph_filter: Vec) -> Self { + Self { + chain_filter: Arc::new(filter), + subgraph_filter, + } + } +} + +impl Clone for TriggerFilterWrapper { + fn clone(&self) -> Self { + Self { + chain_filter: self.chain_filter.cheap_clone(), + subgraph_filter: self.subgraph_filter.clone(), + } + } +} + pub trait TriggerFilter: Default + Clone + Send + Sync { fn from_data_sources<'a>( data_sources: impl Iterator + Clone, @@ -370,6 +409,75 @@ pub trait UnresolvedDataSource: ) -> Result; } +#[derive(Debug)] +pub enum Trigger { + Chain(C::TriggerData), + Subgraph(subgraph::TriggerData), +} + +impl Trigger { + pub fn as_chain(&self) -> Option<&C::TriggerData> { + match self { + Trigger::Chain(data) => Some(data), + _ => None, + } + } + + pub fn as_subgraph(&self) -> Option<&subgraph::TriggerData> { + match self { + Trigger::Subgraph(data) => Some(data), + _ => None, + } + } +} + +impl Eq for Trigger where C::TriggerData: Eq {} + +impl PartialEq for Trigger +where + C::TriggerData: PartialEq, +{ + fn eq(&self, other: &Self) -> bool { + match (self, other) { + (Trigger::Chain(data1), Trigger::Chain(data2)) => data1 == data2, + (Trigger::Subgraph(a), Trigger::Subgraph(b)) => a == b, + _ => false, + } + } +} + +impl Clone for Trigger +where + C::TriggerData: Clone, +{ + fn clone(&self) -> Self { + match self { + Trigger::Chain(data) => Trigger::Chain(data.clone()), + Trigger::Subgraph(data) => Trigger::Subgraph(data.clone()), + } + } +} + +impl Ord for Trigger +where + C::TriggerData: Ord, +{ + fn cmp(&self, other: &Self) -> std::cmp::Ordering { + match (self, other) { + (Trigger::Chain(data1), Trigger::Chain(data2)) => data1.cmp(data2), + (Trigger::Subgraph(_), Trigger::Chain(_)) => std::cmp::Ordering::Greater, + (Trigger::Chain(_), Trigger::Subgraph(_)) => std::cmp::Ordering::Less, + (Trigger::Subgraph(t1), Trigger::Subgraph(t2)) => t1.entity.vid.cmp(&t2.entity.vid), + } + } +} + +impl PartialOrd for Trigger { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + pub trait TriggerData { /// If there is an error when processing this trigger, this will called to add relevant context. /// For example an useful return is: `"block # (), transaction ". @@ -436,7 +544,7 @@ pub struct HostFn { } pub trait RuntimeAdapter: Send + Sync { - fn host_fns(&self, ds: &C::DataSource) -> Result, Error>; + fn host_fns(&self, ds: &data_source::DataSource) -> Result, Error>; } pub trait NodeCapabilities { @@ -489,6 +597,7 @@ impl FromStr for BlockchainKind { "cosmos" => Ok(BlockchainKind::Cosmos), "substreams" => Ok(BlockchainKind::Substreams), "starknet" => Ok(BlockchainKind::Starknet), + "subgraph" => Ok(BlockchainKind::Ethereum), // TODO(krishna): We should detect the blockchain kind from the source subgraph _ => Err(anyhow!("unknown blockchain kind {}", s)), } } diff --git a/graph/src/blockchain/noop_runtime_adapter.rs b/graph/src/blockchain/noop_runtime_adapter.rs index 2f30a30e608..0b8b9e0707c 100644 --- a/graph/src/blockchain/noop_runtime_adapter.rs +++ b/graph/src/blockchain/noop_runtime_adapter.rs @@ -1,5 +1,7 @@ use std::marker::PhantomData; +use crate::data_source; + use super::{Blockchain, HostFn, RuntimeAdapter}; /// A [`RuntimeAdapter`] that does not expose any host functions. @@ -16,7 +18,7 @@ impl RuntimeAdapter for NoopRuntimeAdapter where C: Blockchain, { - fn host_fns(&self, _ds: &C::DataSource) -> anyhow::Result> { + fn host_fns(&self, _ds: &data_source::DataSource) -> anyhow::Result> { Ok(vec![]) } } diff --git a/graph/src/blockchain/polling_block_stream.rs b/graph/src/blockchain/polling_block_stream.rs index ce3fdf2a4ef..fa774261227 100644 --- a/graph/src/blockchain/polling_block_stream.rs +++ b/graph/src/blockchain/polling_block_stream.rs @@ -9,9 +9,9 @@ use std::time::Duration; use super::block_stream::{ BlockStream, BlockStreamError, BlockStreamEvent, BlockWithTriggers, ChainHeadUpdateStream, - FirehoseCursor, TriggersAdapter, BUFFERED_BLOCK_STREAM_SIZE, + FirehoseCursor, TriggersAdapterWrapper, BUFFERED_BLOCK_STREAM_SIZE, }; -use super::{Block, BlockPtr, Blockchain}; +use super::{Block, BlockPtr, Blockchain, TriggerFilterWrapper}; use crate::components::store::BlockNumber; use crate::data::subgraph::UnifiedMappingApiVersion; @@ -79,13 +79,13 @@ where C: Blockchain, { chain_store: Arc, - adapter: Arc>, + adapter: Arc>, node_id: NodeId, subgraph_id: DeploymentHash, // This is not really a block number, but the (unsigned) difference // between two block numbers reorg_threshold: BlockNumber, - filter: Arc, + filter: Arc>, start_blocks: Vec, logger: Logger, previous_triggers_per_block: f64, @@ -146,10 +146,10 @@ where pub fn new( chain_store: Arc, chain_head_update_stream: ChainHeadUpdateStream, - adapter: Arc>, + adapter: Arc>, node_id: NodeId, subgraph_id: DeploymentHash, - filter: Arc, + filter: Arc>, start_blocks: Vec, reorg_threshold: BlockNumber, logger: Logger, @@ -218,7 +218,7 @@ where let max_block_range_size = self.max_block_range_size; // Get pointers from database for comparison - let head_ptr_opt = ctx.chain_store.chain_head_ptr().await?; + let head_ptr_opt = ctx.adapter.chain_head_ptr().await?; let subgraph_ptr = self.current_block.clone(); // If chain head ptr is not set yet @@ -379,7 +379,10 @@ where ); // Update with actually scanned range, to account for any skipped null blocks. - let (blocks, to) = self.adapter.scan_triggers(from, to, &self.filter).await?; + let (blocks, to) = self + .adapter + .scan_triggers(&self.logger, from, to, &self.filter) + .await?; let range_size = to - from + 1; // If the target block (`to`) is within the reorg threshold, indicating no non-null finalized blocks are diff --git a/graph/src/blockchain/types.rs b/graph/src/blockchain/types.rs index 931e52e2dd5..3708f33daa1 100644 --- a/graph/src/blockchain/types.rs +++ b/graph/src/blockchain/types.rs @@ -5,10 +5,11 @@ use diesel::serialize::{Output, ToSql}; use diesel::sql_types::Timestamptz; use diesel::sql_types::{Bytea, Nullable, Text}; use diesel_derives::{AsExpression, FromSqlRow}; +use serde::{Deserialize, Deserializer}; use std::convert::TryFrom; use std::time::Duration; use std::{fmt, str::FromStr}; -use web3::types::{Block, H256}; +use web3::types::{Block, H256, U256, U64}; use crate::cheap_clone::CheapClone; use crate::components::store::BlockNumber; @@ -31,6 +32,10 @@ impl BlockHash { &self.0 } + pub fn as_h256(&self) -> H256 { + H256::from_slice(self.as_slice()) + } + /// Encodes the block hash into a hexadecimal string **without** a "0x" /// prefix. Hashes are stored in the database in this format when the /// schema uses `text` columns, which is a legacy and such columns @@ -44,6 +49,16 @@ impl BlockHash { } } +impl<'de> Deserialize<'de> for BlockHash { + fn deserialize(deserializer: D) -> Result + where + D: Deserializer<'de>, + { + let s: String = Deserialize::deserialize(deserializer)?; + BlockHash::from_str(&s).map_err(serde::de::Error::custom) + } +} + impl CheapClone for BlockHash { fn cheap_clone(&self) -> Self { Self(self.0.clone()) @@ -326,6 +341,204 @@ impl From for BlockNumber { } } +fn deserialize_block_number<'de, D>(deserializer: D) -> Result +where + D: Deserializer<'de>, +{ + let s: String = Deserialize::deserialize(deserializer)?; + + if s.starts_with("0x") { + let s = s.trim_start_matches("0x"); + i32::from_str_radix(s, 16).map_err(serde::de::Error::custom) + } else { + i32::from_str(&s).map_err(serde::de::Error::custom) + } +} + +fn deserialize_block_time<'de, D>(deserializer: D) -> Result +where + D: Deserializer<'de>, +{ + let value = String::deserialize(deserializer)?; + + if value.starts_with("0x") { + let hex_value = value.trim_start_matches("0x"); + + i64::from_str_radix(hex_value, 16) + .map(|secs| BlockTime::since_epoch(secs, 0)) + .map_err(serde::de::Error::custom) + } else { + value + .parse::() + .map(|secs| BlockTime::since_epoch(secs, 0)) + .map_err(serde::de::Error::custom) + } +} +#[derive(Clone, PartialEq, Eq, Hash, Deserialize)] +#[serde(rename_all = "camelCase")] +pub struct ExtendedBlockPtr { + pub hash: BlockHash, + #[serde(deserialize_with = "deserialize_block_number")] + pub number: BlockNumber, + pub parent_hash: BlockHash, + #[serde(deserialize_with = "deserialize_block_time")] + pub timestamp: BlockTime, +} + +impl ExtendedBlockPtr { + pub fn new( + hash: BlockHash, + number: BlockNumber, + parent_hash: BlockHash, + timestamp: BlockTime, + ) -> Self { + Self { + hash, + number, + parent_hash, + timestamp, + } + } + + /// Encodes the block hash into a hexadecimal string **without** a "0x" prefix. + /// Hashes are stored in the database in this format. + pub fn hash_hex(&self) -> String { + self.hash.hash_hex() + } + + /// Encodes the parent block hash into a hexadecimal string **without** a "0x" prefix. + pub fn parent_hash_hex(&self) -> String { + self.parent_hash.hash_hex() + } + + /// Block number to be passed into the store. Panics if it does not fit in an i32. + pub fn block_number(&self) -> BlockNumber { + self.number + } + + pub fn hash_as_h256(&self) -> H256 { + H256::from_slice(&self.hash_slice()[..32]) + } + + pub fn parent_hash_as_h256(&self) -> H256 { + H256::from_slice(&self.parent_hash_slice()[..32]) + } + + pub fn hash_slice(&self) -> &[u8] { + self.hash.0.as_ref() + } + + pub fn parent_hash_slice(&self) -> &[u8] { + self.parent_hash.0.as_ref() + } +} + +impl fmt::Display for ExtendedBlockPtr { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!( + f, + "#{} ({}) [parent: {}]", + self.number, + self.hash_hex(), + self.parent_hash_hex() + ) + } +} + +impl fmt::Debug for ExtendedBlockPtr { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!( + f, + "#{} ({}) [parent: {}]", + self.number, + self.hash_hex(), + self.parent_hash_hex() + ) + } +} + +impl slog::Value for ExtendedBlockPtr { + fn serialize( + &self, + record: &slog::Record, + key: slog::Key, + serializer: &mut dyn slog::Serializer, + ) -> slog::Result { + slog::Value::serialize(&self.to_string(), record, key, serializer) + } +} + +impl IntoValue for ExtendedBlockPtr { + fn into_value(self) -> r::Value { + object! { + __typename: "Block", + hash: self.hash_hex(), + number: format!("{}", self.number), + parent_hash: self.parent_hash_hex(), + timestamp: format!("{}", self.timestamp), + } + } +} + +impl TryFrom<(Option, Option, H256, U256)> for ExtendedBlockPtr { + type Error = anyhow::Error; + + fn try_from(tuple: (Option, Option, H256, U256)) -> Result { + let (hash_opt, number_opt, parent_hash, timestamp_u256) = tuple; + + let hash = hash_opt.ok_or_else(|| anyhow!("Block hash is missing"))?; + let number = number_opt + .ok_or_else(|| anyhow!("Block number is missing"))? + .as_u64(); + + let block_number = + i32::try_from(number).map_err(|_| anyhow!("Block number out of range"))?; + + // Convert `U256` to `BlockTime` + let secs = + i64::try_from(timestamp_u256).map_err(|_| anyhow!("Timestamp out of range for i64"))?; + let block_time = BlockTime::since_epoch(secs, 0); + + Ok(ExtendedBlockPtr { + hash: hash.into(), + number: block_number, + parent_hash: parent_hash.into(), + timestamp: block_time, + }) + } +} + +impl TryFrom<(H256, i32, H256, U256)> for ExtendedBlockPtr { + type Error = anyhow::Error; + + fn try_from(tuple: (H256, i32, H256, U256)) -> Result { + let (hash, block_number, parent_hash, timestamp_u256) = tuple; + + // Convert `U256` to `BlockTime` + let secs = + i64::try_from(timestamp_u256).map_err(|_| anyhow!("Timestamp out of range for i64"))?; + let block_time = BlockTime::since_epoch(secs, 0); + + Ok(ExtendedBlockPtr { + hash: hash.into(), + number: block_number, + parent_hash: parent_hash.into(), + timestamp: block_time, + }) + } +} +impl From for H256 { + fn from(ptr: ExtendedBlockPtr) -> Self { + ptr.hash_as_h256() + } +} + +impl From for BlockNumber { + fn from(ptr: ExtendedBlockPtr) -> Self { + ptr.number + } +} + #[derive(Clone, Debug, PartialEq, Eq, Hash)] /// A collection of attributes that (kind of) uniquely identify a blockchain. pub struct ChainIdentifier { @@ -360,7 +573,9 @@ impl fmt::Display for ChainIdentifier { /// The timestamp associated with a block. This is used whenever a time /// needs to be connected to data within the block -#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, PartialOrd, Ord, FromSqlRow, AsExpression)] +#[derive( + Debug, Clone, Copy, PartialEq, Eq, Hash, PartialOrd, Ord, FromSqlRow, AsExpression, Deserialize, +)] #[diesel(sql_type = Timestamptz)] pub struct BlockTime(Timestamp); @@ -435,3 +650,80 @@ impl ToSql for BlockTime { >::to_sql(&self.0, out) } } + +impl fmt::Display for BlockTime { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "{}", self.0.as_microseconds_since_epoch()) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use serde_json; + + #[test] + fn test_blockhash_deserialization() { + let json_data = "\"0x8186da3ec5590631ae7b9415ce58548cb98c7f1dc68c5ea1c519a3f0f6a25aac\""; + + let block_hash: BlockHash = + serde_json::from_str(json_data).expect("Deserialization failed"); + + let expected_bytes = + hex::decode("8186da3ec5590631ae7b9415ce58548cb98c7f1dc68c5ea1c519a3f0f6a25aac") + .expect("Hex decoding failed"); + + assert_eq!( + *block_hash.0, expected_bytes, + "BlockHash does not match expected bytes" + ); + } + + #[test] + fn test_block_ptr_ext_deserialization() { + // JSON data with a hex string for BlockNumber + let json_data = r#" + { + "hash": "0x8186da3ec5590631ae7b9415ce58548cb98c7f1dc68c5ea1c519a3f0f6a25aac", + "number": "0x2A", + "parentHash": "0xd71699894d637632dea4d425396086edf033c1ff72b13753e8c4e67700e3eb8e", + "timestamp": "0x673b284f" + } + "#; + + // Deserialize the JSON string into a ExtendedBlockPtr + let block_ptr_ext: ExtendedBlockPtr = + serde_json::from_str(json_data).expect("Deserialization failed"); + + // Verify the deserialized values + assert_eq!(block_ptr_ext.number, 42); // 0x2A in hex is 42 in decimal + assert_eq!( + block_ptr_ext.hash_hex(), + "8186da3ec5590631ae7b9415ce58548cb98c7f1dc68c5ea1c519a3f0f6a25aac" + ); + assert_eq!( + block_ptr_ext.parent_hash_hex(), + "d71699894d637632dea4d425396086edf033c1ff72b13753e8c4e67700e3eb8e" + ); + assert_eq!(block_ptr_ext.timestamp.0.as_secs_since_epoch(), 1731930191); + } + + #[test] + fn test_invalid_block_number_deserialization() { + let invalid_json_data = r#" + { + "hash": "0x8186da3ec5590631ae7b9415ce58548cb98c7f1dc68c5ea1c519a3f0f6a25aac", + "number": "invalid_hex_string", + "parentHash": "0xd71699894d637632dea4d425396086edf033c1ff72b13753e8c4e67700e3eb8e", + "timestamp": "123456789012345678901234567890" + } + "#; + + let result: Result = serde_json::from_str(invalid_json_data); + + assert!( + result.is_err(), + "Deserialization should have failed for invalid block number" + ); + } +} diff --git a/graph/src/components/store/entity_cache.rs b/graph/src/components/store/entity_cache.rs index dfaae80f76a..f7a9a8fe88b 100644 --- a/graph/src/components/store/entity_cache.rs +++ b/graph/src/components/store/entity_cache.rs @@ -7,7 +7,7 @@ use std::sync::Arc; use crate::cheap_clone::CheapClone; use crate::components::store::write::EntityModification; use crate::components::store::{self as s, Entity, EntityOperation}; -use crate::data::store::{EntityValidationError, Id, IdType, IntoEntityIterator}; +use crate::data::store::{EntityV, EntityValidationError, Id, IdType, IntoEntityIterator}; use crate::prelude::ENV_VARS; use crate::schema::{EntityKey, InputSchema}; use crate::util::intern::Error as InternError; @@ -29,8 +29,8 @@ pub enum GetScope { #[derive(Debug, Clone)] enum EntityOp { Remove, - Update(Entity), - Overwrite(Entity), + Update(EntityV), + Overwrite(EntityV), } impl EntityOp { @@ -41,7 +41,7 @@ impl EntityOp { use EntityOp::*; match (self, entity) { (Remove, _) => Ok(None), - (Overwrite(new), _) | (Update(new), None) => Ok(Some(new)), + (Overwrite(new), _) | (Update(new), None) => Ok(Some(new.e)), (Update(updates), Some(entity)) => { let mut e = entity.borrow().clone(); e.merge_remove_null_fields(updates)?; @@ -65,7 +65,7 @@ impl EntityOp { match self { // This is how `Overwrite` is constructed, by accumulating `Update` onto `Remove`. Remove => *self = Overwrite(update), - Update(current) | Overwrite(current) => current.merge(update), + Update(current) | Overwrite(current) => current.e.merge(update.e), } } } @@ -105,6 +105,10 @@ pub struct EntityCache { /// generated IDs, the `EntityCache` needs to be newly instantiated for /// each block seq: u32, + + // Sequence number of the next VID value for this block. The value written + // in the database consist of a block number and this SEQ number. + pub vid_seq: u32, } impl Debug for EntityCache { @@ -132,6 +136,7 @@ impl EntityCache { schema: store.input_schema(), store, seq: 0, + vid_seq: 0, } } @@ -152,6 +157,7 @@ impl EntityCache { schema: store.input_schema(), store, seq: 0, + vid_seq: 0, } } @@ -278,9 +284,9 @@ impl EntityCache { ) -> Result, anyhow::Error> { match op { EntityOp::Update(entity) | EntityOp::Overwrite(entity) - if query.matches(key, entity) => + if query.matches(key, &entity.e) => { - Ok(Some(entity.clone())) + Ok(Some(entity.e.clone())) } EntityOp::Remove => Ok(None), _ => Ok(None), @@ -349,10 +355,20 @@ impl EntityCache { /// with existing data. The entity will be validated against the /// subgraph schema, and any errors will result in an `Err` being /// returned. - pub fn set(&mut self, key: EntityKey, entity: Entity) -> Result<(), anyhow::Error> { + pub fn set( + &mut self, + key: EntityKey, + entity: Entity, + block: BlockNumber, + ) -> Result<(), anyhow::Error> { // check the validate for derived fields let is_valid = entity.validate(&key).is_ok(); + // The next VID is based on a block number and a sequence within the block + let vid = ((block as i64) << 32) + self.vid_seq as i64; + self.vid_seq += 1; + let entity = EntityV::new(entity, vid); + self.entity_op(key.clone(), EntityOp::Update(entity)); // The updates we were given are not valid by themselves; force a @@ -458,19 +474,22 @@ impl EntityCache { // Entity was created (None, EntityOp::Update(mut updates)) | (None, EntityOp::Overwrite(mut updates)) => { - updates.remove_null_fields(); - let data = Arc::new(updates); + let vid = updates.vid; + updates.e.remove_null_fields(); + let data = Arc::new(updates.e.clone()); self.current.insert(key.clone(), Some(data.cheap_clone())); Some(Insert { key, data, block, end: None, + vid, }) } // Entity may have been changed (Some(current), EntityOp::Update(updates)) => { let mut data = current.as_ref().clone(); + let vid = updates.vid; data.merge_remove_null_fields(updates) .map_err(|e| key.unknown_attribute(e))?; let data = Arc::new(data); @@ -481,6 +500,7 @@ impl EntityCache { data, block, end: None, + vid, }) } else { None @@ -488,14 +508,16 @@ impl EntityCache { } // Entity was removed and then updated, so it will be overwritten (Some(current), EntityOp::Overwrite(data)) => { - let data = Arc::new(data); - self.current.insert(key.clone(), Some(data.clone())); + let vid = data.vid; + let data = Arc::new(data.e.clone()); + self.current.insert(key.clone(), Some(data.cheap_clone())); if current != data { Some(Overwrite { key, data, block, end: None, + vid, }) } else { None diff --git a/graph/src/components/store/mod.rs b/graph/src/components/store/mod.rs index 31b0e62cfae..9713b78c150 100644 --- a/graph/src/components/store/mod.rs +++ b/graph/src/components/store/mod.rs @@ -30,7 +30,7 @@ use crate::cheap_clone::CheapClone; use crate::components::store::write::EntityModification; use crate::constraint_violation; use crate::data::store::scalar::Bytes; -use crate::data::store::{Id, IdList, Value}; +use crate::data::store::{EntityV, Id, IdList, Value}; use crate::data::value::Word; use crate::data_source::CausalityRegion; use crate::derive::CheapClone; @@ -829,7 +829,7 @@ where pub enum EntityOperation { /// Locates the entity specified by `key` and sets its attributes according to the contents of /// `data`. If no entity exists with this key, creates a new entity. - Set { key: EntityKey, data: Entity }, + Set { key: EntityKey, data: EntityV }, /// Removes an entity with the specified key, if one exists. Remove { key: EntityKey }, diff --git a/graph/src/components/store/traits.rs b/graph/src/components/store/traits.rs index 69ed67c16b2..f157eeb78c0 100644 --- a/graph/src/components/store/traits.rs +++ b/graph/src/components/store/traits.rs @@ -1,12 +1,13 @@ use std::collections::HashMap; +use std::ops::Range; use anyhow::Error; use async_trait::async_trait; use web3::types::{Address, H256}; use super::*; -use crate::blockchain::block_stream::FirehoseCursor; -use crate::blockchain::{BlockTime, ChainIdentifier}; +use crate::blockchain::block_stream::{EntitySourceOperation, FirehoseCursor}; +use crate::blockchain::{BlockTime, ChainIdentifier, ExtendedBlockPtr}; use crate::components::metrics::stopwatch::StopwatchMetrics; use crate::components::server::index_node::VersionInfo; use crate::components::subgraph::SubgraphVersionSwitchingMode; @@ -185,6 +186,11 @@ pub trait SubgraphStore: Send + Sync + 'static { manifest_idx_and_name: Arc>, ) -> Result, StoreError>; + async fn sourceable( + self: Arc, + deployment: DeploymentId, + ) -> Result, StoreError>; + /// Initiate a graceful shutdown of the writable that a previous call to /// `writable` might have started async fn stop_subgraph(&self, deployment: &DeploymentLocator) -> Result<(), StoreError>; @@ -287,6 +293,44 @@ impl DeploymentCursorTracker for Arc { } } +#[async_trait] +pub trait SourceableStore: Sync + Send + 'static { + /// Returns all versions of entities of the given entity_type that were + /// changed in the given block_range. + fn get_range( + &self, + entity_types: Vec, + causality_region: CausalityRegion, + block_range: Range, + ) -> Result>, StoreError>; + + fn input_schema(&self) -> InputSchema; + + /// Get a pointer to the most recently processed block in the subgraph. + async fn block_ptr(&self) -> Result, StoreError>; +} + +// This silly impl is needed until https://github.com/rust-lang/rust/issues/65991 is stable. +#[async_trait] +impl SourceableStore for Arc { + fn get_range( + &self, + entity_types: Vec, + causality_region: CausalityRegion, + block_range: Range, + ) -> Result>, StoreError> { + (**self).get_range(entity_types, causality_region, block_range) + } + + fn input_schema(&self) -> InputSchema { + (**self).input_schema() + } + + async fn block_ptr(&self) -> Result, StoreError> { + (**self).block_ptr().await + } +} + /// A view of the store for indexing. All indexing-related operations need /// to go through this trait. Methods in this trait will never return a /// `StoreError::DatabaseUnavailable`. Instead, they will retry the @@ -478,6 +522,12 @@ pub trait ChainStore: Send + Sync + 'static { hashes: Vec, ) -> Result, Error>; + /// Returns the blocks present in the store for the given block numbers. + async fn block_ptrs_by_numbers( + self: Arc, + numbers: Vec, + ) -> Result>, Error>; + /// Get the `offset`th ancestor of `block_hash`, where offset=0 means the block matching /// `block_hash` and offset=1 means its parent. If `root` is passed, short-circuit upon finding /// a child of `root`. Returns None if unable to complete due to missing blocks in the chain diff --git a/graph/src/components/store/write.rs b/graph/src/components/store/write.rs index 721e3d80bc1..6dd2cda472b 100644 --- a/graph/src/components/store/write.rs +++ b/graph/src/components/store/write.rs @@ -45,6 +45,7 @@ pub enum EntityModification { data: Arc, block: BlockNumber, end: Option, + vid: i64, }, /// Update the entity by overwriting it Overwrite { @@ -52,6 +53,7 @@ pub enum EntityModification { data: Arc, block: BlockNumber, end: Option, + vid: i64, }, /// Remove the entity Remove { key: EntityKey, block: BlockNumber }, @@ -67,6 +69,7 @@ pub struct EntityWrite<'a> { // The end of the block range for which this write is valid. The value // of `end` itself is not included in the range pub end: Option, + pub vid: i64, } impl std::fmt::Display for EntityWrite<'_> { @@ -89,24 +92,28 @@ impl<'a> TryFrom<&'a EntityModification> for EntityWrite<'a> { data, block, end, + vid, } => Ok(EntityWrite { id: &key.entity_id, entity: data, causality_region: key.causality_region, block: *block, end: *end, + vid: *vid, }), EntityModification::Overwrite { key, data, block, end, + vid, } => Ok(EntityWrite { id: &key.entity_id, entity: &data, causality_region: key.causality_region, block: *block, end: *end, + vid: *vid, }), EntityModification::Remove { .. } => Err(()), @@ -213,11 +220,13 @@ impl EntityModification { data, block, end, + vid, } => Ok(Insert { key, data, block, end, + vid, }), Remove { key, .. } => { return Err(constraint_violation!( @@ -271,21 +280,23 @@ impl EntityModification { } impl EntityModification { - pub fn insert(key: EntityKey, data: Entity, block: BlockNumber) -> Self { + pub fn insert(key: EntityKey, data: Entity, block: BlockNumber, vid: i64) -> Self { EntityModification::Insert { key, data: Arc::new(data), block, end: None, + vid, } } - pub fn overwrite(key: EntityKey, data: Entity, block: BlockNumber) -> Self { + pub fn overwrite(key: EntityKey, data: Entity, block: BlockNumber, vid: i64) -> Self { EntityModification::Overwrite { key, data: Arc::new(data), block, end: None, + vid, } } @@ -1017,18 +1028,21 @@ mod test { let value = value.clone(); let key = THING_TYPE.parse_key("one").unwrap(); + let vid = 0; match value { Ins(block) => EntityModification::Insert { key, data: Arc::new(entity! { SCHEMA => id: "one", count: block }), block, end: None, + vid, }, Ovw(block) => EntityModification::Overwrite { key, data: Arc::new(entity! { SCHEMA => id: "one", count: block }), block, end: None, + vid, }, Rem(block) => EntityModification::Remove { key, block }, InsC(block, end) => EntityModification::Insert { @@ -1036,12 +1050,14 @@ mod test { data: Arc::new(entity! { SCHEMA => id: "one", count: block }), block, end: Some(end), + vid, }, OvwC(block, end) => EntityModification::Overwrite { key, data: Arc::new(entity! { SCHEMA => id: "one", count: block }), block, end: Some(end), + vid, }, } } diff --git a/graph/src/components/subgraph/instance.rs b/graph/src/components/subgraph/instance.rs index 470e50334d3..889690c3916 100644 --- a/graph/src/components/subgraph/instance.rs +++ b/graph/src/components/subgraph/instance.rs @@ -20,6 +20,7 @@ impl From<&DataSourceTemplate> for InstanceDSTemplate { match value { DataSourceTemplate::Onchain(ds) => Self::Onchain(ds.info()), DataSourceTemplate::Offchain(ds) => Self::Offchain(ds.clone()), + DataSourceTemplate::Subgraph(_) => todo!(), // TODO(krishna) } } } diff --git a/graph/src/components/subgraph/proof_of_indexing/online.rs b/graph/src/components/subgraph/proof_of_indexing/online.rs index caaa76f0a76..d47f08b0a8f 100644 --- a/graph/src/components/subgraph/proof_of_indexing/online.rs +++ b/graph/src/components/subgraph/proof_of_indexing/online.rs @@ -146,8 +146,8 @@ impl BlockEventStream { fn write(&mut self, event: &ProofOfIndexingEvent<'_>) { let children = &[ 1, // kvp -> v - 0, // PoICausalityRegion.blocks: Vec - self.block_index, // Vec -> [i] + 0, // PoICausalityRegion.blocks: Result> + self.block_index, // Result> -> [i] 0, // Block.events -> Vec self.vec_length, ]; @@ -242,6 +242,10 @@ impl ProofOfIndexing { pub fn take(self) -> HashMap { self.per_causality_region } + + pub fn get_block(&self) -> BlockNumber { + self.block_number + } } pub struct ProofOfIndexingFinisher { diff --git a/graph/src/data/store/mod.rs b/graph/src/data/store/mod.rs index 33d9286ceec..eb54d44511a 100644 --- a/graph/src/data/store/mod.rs +++ b/graph/src/data/store/mod.rs @@ -735,6 +735,9 @@ where lazy_static! { /// The name of the id attribute, `"id"` pub static ref ID: Word = Word::from("id"); + + /// The name of the vid attribute, `"vid"` + pub static ref VID: Word = Word::from("vid"); } /// An entity is represented as a map of attribute names to values. @@ -910,6 +913,25 @@ impl Entity { Id::try_from(self.get("id").unwrap().clone()).expect("the id is set to a valid value") } + /// Return the VID of this entity and if its missing or of a type different than + /// i64 it panics. + pub fn vid(&self) -> i64 { + self.get("vid") + .expect("the vid is set") + .as_int8() + .expect("the vid is set to a valid value") + } + + /// This version of the function returns 0 if the VID is not set. It should be + /// used only in the testing code for more lenient definition of entities. + #[cfg(debug_assertions)] + pub fn vid_or_default(&self) -> i64 { + self.get("vid") + .unwrap_or(&Value::Int8(0)) + .as_int8() + .expect("the vid is set to a valid value") + } + /// Merges an entity update `update` into this entity. /// /// If a key exists in both entities, the value from `update` is chosen. @@ -924,8 +946,8 @@ impl Entity { /// If a key exists in both entities, the value from `update` is chosen. /// If a key only exists on one entity, the value from that entity is chosen. /// If a key is set to `Value::Null` in `update`, the key/value pair is removed. - pub fn merge_remove_null_fields(&mut self, update: Entity) -> Result<(), InternError> { - for (key, value) in update.0.into_iter() { + pub fn merge_remove_null_fields(&mut self, update: EntityV) -> Result<(), InternError> { + for (key, value) in update.e.0.into_iter() { match value { Value::Null => self.0.remove(&key), _ => self.0.insert(&key, value)?, @@ -1076,6 +1098,19 @@ impl std::fmt::Debug for Entity { } } +/// An entity wrapper that has VID too. +#[derive(Debug, Clone, CacheWeight, PartialEq, Eq, Serialize)] +pub struct EntityV { + pub e: Entity, + pub vid: i64, +} + +impl EntityV { + pub fn new(e: Entity, vid: i64) -> Self { + Self { e, vid } + } +} + /// An object that is returned from a query. It's a an `r::Value` which /// carries the attributes of the object (`__typename`, `id` etc.) and /// possibly a pointer to its parent if the query that constructed it is one diff --git a/graph/src/data/subgraph/api_version.rs b/graph/src/data/subgraph/api_version.rs index e626e9f1dbc..dcfdb329476 100644 --- a/graph/src/data/subgraph/api_version.rs +++ b/graph/src/data/subgraph/api_version.rs @@ -54,8 +54,13 @@ pub const SPEC_VERSION_1_1_0: Version = Version::new(1, 1, 0); // Enables eth call declarations and indexed arguments(topics) filtering in manifest pub const SPEC_VERSION_1_2_0: Version = Version::new(1, 2, 0); +// Enables subgraphs as datasource +// Change the way the VID field is generated. It used to be autoincrement. Now its +// based on block number and the sequence of the entities in a block. +pub const SPEC_VERSION_1_3_0: Version = Version::new(1, 3, 0); + // The latest spec version available -pub const LATEST_VERSION: &Version = &SPEC_VERSION_1_2_0; +pub const LATEST_VERSION: &Version = &SPEC_VERSION_1_3_0; pub const MIN_SPEC_VERSION: Version = Version::new(0, 0, 2); diff --git a/graph/src/data/subgraph/mod.rs b/graph/src/data/subgraph/mod.rs index 52b0f4dfed1..941764b9a12 100644 --- a/graph/src/data/subgraph/mod.rs +++ b/graph/src/data/subgraph/mod.rs @@ -33,7 +33,7 @@ use web3::types::Address; use crate::{ bail, - blockchain::{BlockPtr, Blockchain, DataSource as _}, + blockchain::{BlockPtr, Blockchain}, components::{ link_resolver::LinkResolver, store::{StoreError, SubgraphStore}, @@ -140,6 +140,10 @@ impl DeploymentHash { link: format!("/ipfs/{}", self), } } + + pub fn to_bytes(&self) -> Vec { + self.0.as_bytes().to_vec() + } } impl Deref for DeploymentHash { @@ -719,7 +723,7 @@ impl UnvalidatedSubgraphManifest { .0 .data_sources .iter() - .filter_map(|d| Some(d.as_onchain()?.network()?.to_string())) + .filter_map(|d| Some(d.network()?.to_string())) .collect::>(); networks.sort(); networks.dedup(); @@ -765,11 +769,9 @@ impl SubgraphManifest { max_spec_version: semver::Version, ) -> Result { let unresolved = UnresolvedSubgraphManifest::parse(id, raw)?; - let resolved = unresolved .resolve(resolver, logger, max_spec_version) .await?; - Ok(resolved) } @@ -777,14 +779,14 @@ impl SubgraphManifest { // Assume the manifest has been validated, ensuring network names are homogenous self.data_sources .iter() - .find_map(|d| Some(d.as_onchain()?.network()?.to_string())) + .find_map(|d| Some(d.network()?.to_string())) .expect("Validated manifest does not have a network defined on any datasource") } pub fn start_blocks(&self) -> Vec { self.data_sources .iter() - .filter_map(|d| Some(d.as_onchain()?.start_block())) + .filter_map(|d| d.start_block()) .collect() } diff --git a/graph/src/data_source/common.rs b/graph/src/data_source/common.rs new file mode 100644 index 00000000000..a70f0ab8e17 --- /dev/null +++ b/graph/src/data_source/common.rs @@ -0,0 +1,750 @@ +use crate::blockchain::block_stream::EntitySourceOperation; +use crate::prelude::{BlockPtr, Value}; +use crate::{components::link_resolver::LinkResolver, data::value::Word, prelude::Link}; +use anyhow::{anyhow, Context, Error}; +use ethabi::{Address, Contract, Function, LogParam, ParamType, Token}; +use graph_derive::CheapClone; +use lazy_static::lazy_static; +use num_bigint::Sign; +use regex::Regex; +use serde::de; +use serde::Deserialize; +use slog::Logger; +use std::{str::FromStr, sync::Arc}; +use web3::types::{Log, H160}; + +#[derive(Clone, Debug, PartialEq)] +pub struct MappingABI { + pub name: String, + pub contract: Contract, +} + +impl MappingABI { + pub fn function( + &self, + contract_name: &str, + name: &str, + signature: Option<&str>, + ) -> Result<&Function, Error> { + let contract = &self.contract; + let function = match signature { + // Behavior for apiVersion < 0.0.4: look up function by name; for overloaded + // functions this always picks the same overloaded variant, which is incorrect + // and may lead to encoding/decoding errors + None => contract.function(name).with_context(|| { + format!( + "Unknown function \"{}::{}\" called from WASM runtime", + contract_name, name + ) + })?, + + // Behavior for apiVersion >= 0.0.04: look up function by signature of + // the form `functionName(uint256,string) returns (bytes32,string)`; this + // correctly picks the correct variant of an overloaded function + Some(ref signature) => contract + .functions_by_name(name) + .with_context(|| { + format!( + "Unknown function \"{}::{}\" called from WASM runtime", + contract_name, name + ) + })? + .iter() + .find(|f| signature == &f.signature()) + .with_context(|| { + format!( + "Unknown function \"{}::{}\" with signature `{}` \ + called from WASM runtime", + contract_name, name, signature, + ) + })?, + }; + Ok(function) + } +} + +#[derive(Clone, Debug, Hash, Eq, PartialEq, Deserialize)] +pub struct UnresolvedMappingABI { + pub name: String, + pub file: Link, +} + +impl UnresolvedMappingABI { + pub async fn resolve( + self, + resolver: &Arc, + logger: &Logger, + ) -> Result { + let contract_bytes = resolver.cat(logger, &self.file).await.with_context(|| { + format!( + "failed to resolve ABI {} from {}", + self.name, self.file.link + ) + })?; + let contract = Contract::load(&*contract_bytes)?; + Ok(MappingABI { + name: self.name, + contract, + }) + } +} + +/// Internal representation of declared calls. In the manifest that's +/// written as part of an event handler as +/// ```yaml +/// calls: +/// - myCall1: Contract[address].function(arg1, arg2, ...) +/// - .. +/// ``` +/// +/// The `address` and `arg` fields can be either `event.address` or +/// `event.params.`. Each entry under `calls` gets turned into a +/// `CallDcl` +#[derive(Clone, CheapClone, Debug, Default, Hash, Eq, PartialEq)] +pub struct CallDecls { + pub decls: Arc>, + readonly: (), +} + +/// A single call declaration, like `myCall1: +/// Contract[address].function(arg1, arg2, ...)` +#[derive(Clone, Debug, Hash, Eq, PartialEq)] +pub struct CallDecl { + /// A user-defined label + pub label: String, + /// The call expression + pub expr: CallExpr, + readonly: (), +} + +impl CallDecl { + pub fn validate_args(&self) -> Result<(), Error> { + self.expr.validate_args() + } + + pub fn address_for_log(&self, log: &Log, params: &[LogParam]) -> Result { + let address = match &self.expr.address { + CallArg::HexAddress(address) => *address, + CallArg::Ethereum(arg) => match arg { + EthereumArg::Address => log.address, + EthereumArg::Param(name) => { + let value = params + .iter() + .find(|param| ¶m.name == name.as_str()) + .ok_or_else(|| anyhow!("unknown param {name}"))? + .value + .clone(); + value + .into_address() + .ok_or_else(|| anyhow!("param {name} is not an address"))? + } + }, + CallArg::Subgraph(_) => { + return Err(anyhow!( + "Subgraph params are not supported for when declaring calls for event handlers" + )) + } + }; + Ok(address) + } + + pub fn args_for_log(&self, log: &Log, params: &[LogParam]) -> Result, Error> { + self.expr + .args + .iter() + .map(|arg| match arg { + CallArg::HexAddress(address) => Ok(Token::Address(*address)), + CallArg::Ethereum(arg) => match arg { + EthereumArg::Address => Ok(Token::Address(log.address)), + EthereumArg::Param(name) => { + let value = params + .iter() + .find(|param| ¶m.name == name.as_str()) + .ok_or_else(|| anyhow!("unknown param {name}"))? + .value + .clone(); + Ok(value) + } + }, + CallArg::Subgraph(_) => Err(anyhow!( + "Subgraph params are not supported for when declaring calls for event handlers" + )), + }) + .collect() + } + + pub fn get_function(&self, mapping: &dyn FindMappingABI) -> Result { + let contract_name = self.expr.abi.to_string(); + let function_name = self.expr.func.as_str(); + let abi = mapping.find_abi(&contract_name)?; + + // TODO: Handle overloaded functions + // Behavior for apiVersion < 0.0.4: look up function by name; for overloaded + // functions this always picks the same overloaded variant, which is incorrect + // and may lead to encoding/decoding errors + abi.contract + .function(function_name) + .cloned() + .with_context(|| { + format!( + "Unknown function \"{}::{}\" called from WASM runtime", + contract_name, function_name + ) + }) + } + + pub fn address_for_entity_handler( + &self, + entity: &EntitySourceOperation, + ) -> Result { + match &self.expr.address { + // Static hex address - just return it directly + CallArg::HexAddress(address) => Ok(*address), + + // Ethereum params not allowed here + CallArg::Ethereum(_) => Err(anyhow!( + "Ethereum params are not supported for entity handler calls" + )), + + // Look up address from entity parameter + CallArg::Subgraph(SubgraphArg::EntityParam(name)) => { + // Get the value for this parameter + let value = entity + .entity + .get(name.as_str()) + .ok_or_else(|| anyhow!("entity missing required param '{name}'"))?; + + // Make sure it's a bytes value and convert to address + match value { + Value::Bytes(bytes) => { + let address = H160::from_slice(bytes.as_slice()); + Ok(address) + } + _ => Err(anyhow!("param '{name}' must be an address")), + } + } + } + } + + /// Processes arguments for an entity handler, converting them to the expected token types. + /// Returns an error if argument count mismatches or if conversion fails. + pub fn args_for_entity_handler( + &self, + entity: &EntitySourceOperation, + param_types: Vec, + ) -> Result, Error> { + self.validate_entity_handler_args(¶m_types)?; + + self.expr + .args + .iter() + .zip(param_types.into_iter()) + .map(|(arg, expected_type)| { + self.process_entity_handler_arg(arg, &expected_type, entity) + }) + .collect() + } + + /// Validates that the number of provided arguments matches the expected parameter types. + fn validate_entity_handler_args(&self, param_types: &[ParamType]) -> Result<(), Error> { + if self.expr.args.len() != param_types.len() { + return Err(anyhow!( + "mismatched number of arguments: expected {}, got {}", + param_types.len(), + self.expr.args.len() + )); + } + Ok(()) + } + + /// Processes a single entity handler argument based on its type (HexAddress, Ethereum, or Subgraph). + /// Returns error for unsupported Ethereum params. + fn process_entity_handler_arg( + &self, + arg: &CallArg, + expected_type: &ParamType, + entity: &EntitySourceOperation, + ) -> Result { + match arg { + CallArg::HexAddress(address) => self.process_hex_address(*address, expected_type), + CallArg::Ethereum(_) => Err(anyhow!( + "Ethereum params are not supported for entity handler calls" + )), + CallArg::Subgraph(SubgraphArg::EntityParam(name)) => { + self.process_entity_param(name, expected_type, entity) + } + } + } + + /// Converts a hex address to a token, ensuring it matches the expected parameter type. + fn process_hex_address( + &self, + address: H160, + expected_type: &ParamType, + ) -> Result { + match expected_type { + ParamType::Address => Ok(Token::Address(address)), + _ => Err(anyhow!( + "type mismatch: hex address provided for non-address parameter" + )), + } + } + + /// Retrieves and processes an entity parameter, converting it to the expected token type. + fn process_entity_param( + &self, + name: &str, + expected_type: &ParamType, + entity: &EntitySourceOperation, + ) -> Result { + let value = entity + .entity + .get(name) + .ok_or_else(|| anyhow!("entity missing required param '{name}'"))?; + + self.convert_entity_value_to_token(value, expected_type, name) + } + + /// Converts a `Value` to the appropriate `Token` type based on the expected parameter type. + /// Handles various type conversions including primitives, bytes, and arrays. + fn convert_entity_value_to_token( + &self, + value: &Value, + expected_type: &ParamType, + param_name: &str, + ) -> Result { + match (expected_type, value) { + (ParamType::Address, Value::Bytes(b)) => { + Ok(Token::Address(H160::from_slice(b.as_slice()))) + } + (ParamType::Bytes, Value::Bytes(b)) => Ok(Token::Bytes(b.as_ref().to_vec())), + (ParamType::FixedBytes(size), Value::Bytes(b)) if b.len() == *size => { + Ok(Token::FixedBytes(b.as_ref().to_vec())) + } + (ParamType::String, Value::String(s)) => Ok(Token::String(s.to_string())), + (ParamType::Bool, Value::Bool(b)) => Ok(Token::Bool(*b)), + (ParamType::Int(_), Value::Int(i)) => Ok(Token::Int((*i).into())), + (ParamType::Int(_), Value::Int8(i)) => Ok(Token::Int((*i).into())), + (ParamType::Int(_), Value::BigInt(i)) => Ok(Token::Int(i.to_signed_u256())), + (ParamType::Uint(_), Value::Int(i)) if *i >= 0 => Ok(Token::Uint((*i).into())), + (ParamType::Uint(_), Value::BigInt(i)) if i.sign() == Sign::Plus => { + Ok(Token::Uint(i.to_unsigned_u256())) + } + (ParamType::Array(inner_type), Value::List(values)) => { + self.process_entity_array_values(values, inner_type.as_ref(), param_name) + } + _ => Err(anyhow!( + "type mismatch for param '{param_name}': cannot convert {:?} to {:?}", + value, + expected_type + )), + } + } + + fn process_entity_array_values( + &self, + values: &[Value], + inner_type: &ParamType, + param_name: &str, + ) -> Result { + let tokens: Result, Error> = values + .iter() + .enumerate() + .map(|(idx, v)| { + self.convert_entity_value_to_token(v, inner_type, &format!("{param_name}[{idx}]")) + }) + .collect(); + Ok(Token::Array(tokens?)) + } +} + +impl<'de> de::Deserialize<'de> for CallDecls { + fn deserialize(deserializer: D) -> Result + where + D: de::Deserializer<'de>, + { + let decls: std::collections::HashMap = + de::Deserialize::deserialize(deserializer)?; + let decls = decls + .into_iter() + .map(|(name, expr)| { + expr.parse::().map(|expr| CallDecl { + label: name, + expr, + readonly: (), + }) + }) + .collect::>() + .map(|decls| Arc::new(decls)) + .map_err(de::Error::custom)?; + Ok(CallDecls { + decls, + readonly: (), + }) + } +} + +#[derive(Clone, Debug, Hash, Eq, PartialEq)] +pub struct CallExpr { + pub abi: Word, + pub address: CallArg, + pub func: Word, + pub args: Vec, + readonly: (), +} + +impl CallExpr { + fn validate_args(&self) -> Result<(), anyhow::Error> { + // Consider address along with args for checking Ethereum/Subgraph mixing + let has_ethereum = matches!(self.address, CallArg::Ethereum(_)) + || self + .args + .iter() + .any(|arg| matches!(arg, CallArg::Ethereum(_))); + + let has_subgraph = matches!(self.address, CallArg::Subgraph(_)) + || self + .args + .iter() + .any(|arg| matches!(arg, CallArg::Subgraph(_))); + + if has_ethereum && has_subgraph { + return Err(anyhow!( + "Cannot mix Ethereum and Subgraph args in the same call expression" + )); + } + + Ok(()) + } +} +/// Parse expressions of the form `Contract[address].function(arg1, arg2, +/// ...)` where the `address` and the args are either `event.address` or +/// `event.params.`. +/// +/// The parser is pretty awful as it generates error messages that aren't +/// very helpful. We should replace all this with a real parser, most likely +/// `combine` which is what `graphql_parser` uses +impl FromStr for CallExpr { + type Err = anyhow::Error; + + fn from_str(s: &str) -> Result { + lazy_static! { + static ref RE: Regex = Regex::new( + r"(?x) + (?P[a-zA-Z0-9_]+)\[ + (?P
[^]]+)\] + \. + (?P[a-zA-Z0-9_]+)\( + (?P[^)]*) + \)" + ) + .unwrap(); + } + let x = RE + .captures(s) + .ok_or_else(|| anyhow!("invalid call expression `{s}`"))?; + let abi = Word::from(x.name("abi").unwrap().as_str()); + let address = x.name("address").unwrap().as_str().parse()?; + let func = Word::from(x.name("func").unwrap().as_str()); + let args: Vec = x + .name("args") + .unwrap() + .as_str() + .split(',') + .filter(|s| !s.is_empty()) + .map(|s| s.trim().parse::()) + .collect::>()?; + + let call_expr = CallExpr { + abi, + address, + func, + args, + readonly: (), + }; + + // Validate the arguments after constructing the CallExpr + call_expr.validate_args()?; + + Ok(call_expr) + } +} + +#[derive(Clone, Debug, Hash, Eq, PartialEq)] +pub enum CallArg { + // Hard-coded hex address + HexAddress(Address), + // Ethereum-specific variants + Ethereum(EthereumArg), + // Subgraph datasource specific variants + Subgraph(SubgraphArg), +} + +#[derive(Clone, Debug, Hash, Eq, PartialEq)] +pub enum EthereumArg { + Address, + Param(Word), +} + +#[derive(Clone, Debug, Hash, Eq, PartialEq)] +pub enum SubgraphArg { + EntityParam(Word), +} + +lazy_static! { + // Matches a 40-character hexadecimal string prefixed with '0x', typical for Ethereum addresses + static ref ADDR_RE: Regex = Regex::new(r"^0x[0-9a-fA-F]{40}$").unwrap(); +} + +impl FromStr for CallArg { + type Err = anyhow::Error; + + fn from_str(s: &str) -> Result { + if ADDR_RE.is_match(s) { + if let Ok(parsed_address) = Address::from_str(s) { + return Ok(CallArg::HexAddress(parsed_address)); + } + } + + let mut parts = s.split('.'); + match (parts.next(), parts.next(), parts.next()) { + (Some("event"), Some("address"), None) => Ok(CallArg::Ethereum(EthereumArg::Address)), + (Some("event"), Some("params"), Some(param)) => { + Ok(CallArg::Ethereum(EthereumArg::Param(Word::from(param)))) + } + (Some("entity"), Some(param), None) => Ok(CallArg::Subgraph(SubgraphArg::EntityParam( + Word::from(param), + ))), + _ => Err(anyhow!("invalid call argument `{}`", s)), + } + } +} + +pub trait FindMappingABI { + fn find_abi(&self, abi_name: &str) -> Result, Error>; +} + +#[derive(Clone, Debug, PartialEq)] +pub struct DeclaredCall { + /// The user-supplied label from the manifest + label: String, + contract_name: String, + address: Address, + function: Function, + args: Vec, +} + +impl DeclaredCall { + pub fn from_log_trigger( + mapping: &dyn FindMappingABI, + call_decls: &CallDecls, + log: &Log, + params: &[LogParam], + ) -> Result, anyhow::Error> { + Self::create_calls(mapping, call_decls, |decl, _| { + Ok(( + decl.address_for_log(log, params)?, + decl.args_for_log(log, params)?, + )) + }) + } + + pub fn from_entity_trigger( + mapping: &dyn FindMappingABI, + call_decls: &CallDecls, + entity: &EntitySourceOperation, + ) -> Result, anyhow::Error> { + Self::create_calls(mapping, call_decls, |decl, function| { + let param_types = function + .inputs + .iter() + .map(|param| param.kind.clone()) + .collect::>(); + + Ok(( + decl.address_for_entity_handler(entity)?, + decl.args_for_entity_handler(entity, param_types) + .context(format!( + "Failed to parse arguments for call to function \"{}\" of contract \"{}\"", + decl.expr.func.as_str(), + decl.expr.abi.to_string() + ))?, + )) + }) + } + + fn create_calls( + mapping: &dyn FindMappingABI, + call_decls: &CallDecls, + get_address_and_args: F, + ) -> Result, anyhow::Error> + where + F: Fn(&CallDecl, &Function) -> Result<(Address, Vec), anyhow::Error>, + { + let mut calls = Vec::new(); + for decl in call_decls.decls.iter() { + let contract_name = decl.expr.abi.to_string(); + let function = decl.get_function(mapping)?; + let (address, args) = get_address_and_args(decl, &function)?; + + calls.push(DeclaredCall { + label: decl.label.clone(), + contract_name, + address, + function: function.clone(), + args, + }); + } + Ok(calls) + } + + pub fn as_eth_call(self, block_ptr: BlockPtr, gas: Option) -> (ContractCall, String) { + ( + ContractCall { + contract_name: self.contract_name, + address: self.address, + block_ptr, + function: self.function, + args: self.args, + gas, + }, + self.label, + ) + } +} +#[derive(Clone, Debug)] +pub struct ContractCall { + pub contract_name: String, + pub address: Address, + pub block_ptr: BlockPtr, + pub function: Function, + pub args: Vec, + pub gas: Option, +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_ethereum_call_expr() { + let expr: CallExpr = "ERC20[event.address].balanceOf(event.params.token)" + .parse() + .unwrap(); + assert_eq!(expr.abi, "ERC20"); + assert_eq!(expr.address, CallArg::Ethereum(EthereumArg::Address)); + assert_eq!(expr.func, "balanceOf"); + assert_eq!( + expr.args, + vec![CallArg::Ethereum(EthereumArg::Param("token".into()))] + ); + + let expr: CallExpr = + "Pool[event.params.pool].fees(event.params.token0, event.params.token1)" + .parse() + .unwrap(); + assert_eq!(expr.abi, "Pool"); + assert_eq!( + expr.address, + CallArg::Ethereum(EthereumArg::Param("pool".into())) + ); + assert_eq!(expr.func, "fees"); + assert_eq!( + expr.args, + vec![ + CallArg::Ethereum(EthereumArg::Param("token0".into())), + CallArg::Ethereum(EthereumArg::Param("token1".into())) + ] + ); + } + + #[test] + fn test_subgraph_call_expr() { + let expr: CallExpr = "Token[entity.id].symbol()".parse().unwrap(); + assert_eq!(expr.abi, "Token"); + assert_eq!( + expr.address, + CallArg::Subgraph(SubgraphArg::EntityParam("id".into())) + ); + assert_eq!(expr.func, "symbol"); + assert_eq!(expr.args, vec![]); + + let expr: CallExpr = "Pair[entity.pair].getReserves(entity.token0)" + .parse() + .unwrap(); + assert_eq!(expr.abi, "Pair"); + assert_eq!( + expr.address, + CallArg::Subgraph(SubgraphArg::EntityParam("pair".into())) + ); + assert_eq!(expr.func, "getReserves"); + assert_eq!( + expr.args, + vec![CallArg::Subgraph(SubgraphArg::EntityParam("token0".into()))] + ); + } + + #[test] + fn test_hex_address_call_expr() { + let addr = "0xDeaDbeefdEAdbeefdEadbEEFdeadbeEFdEaDbeeF"; + let hex_address = CallArg::HexAddress(web3::types::H160::from_str(addr).unwrap()); + + // Test HexAddress in address position + let expr: CallExpr = format!("Pool[{}].growth()", addr).parse().unwrap(); + assert_eq!(expr.abi, "Pool"); + assert_eq!(expr.address, hex_address.clone()); + assert_eq!(expr.func, "growth"); + assert_eq!(expr.args, vec![]); + + // Test HexAddress in argument position + let expr: CallExpr = format!("Pool[event.address].approve({}, event.params.amount)", addr) + .parse() + .unwrap(); + assert_eq!(expr.abi, "Pool"); + assert_eq!(expr.address, CallArg::Ethereum(EthereumArg::Address)); + assert_eq!(expr.func, "approve"); + assert_eq!(expr.args.len(), 2); + assert_eq!(expr.args[0], hex_address); + } + + #[test] + fn test_invalid_call_args() { + // Invalid hex address + assert!("Pool[0xinvalid].test()".parse::().is_err()); + + // Invalid event path + assert!("Pool[event.invalid].test()".parse::().is_err()); + + // Invalid entity path + assert!("Pool[entity].test()".parse::().is_err()); + + // Empty address + assert!("Pool[].test()".parse::().is_err()); + + // Invalid parameter format + assert!("Pool[event.params].test()".parse::().is_err()); + } + + #[test] + fn test_from_str() { + // Test valid hex address + let addr = "0xDeaDbeefdEAdbeefdEadbEEFdeadbeEFdEaDbeeF"; + let arg = CallArg::from_str(addr).unwrap(); + assert!(matches!(arg, CallArg::HexAddress(_))); + + // Test Ethereum Address + let arg = CallArg::from_str("event.address").unwrap(); + assert!(matches!(arg, CallArg::Ethereum(EthereumArg::Address))); + + // Test Ethereum Param + let arg = CallArg::from_str("event.params.token").unwrap(); + assert!(matches!(arg, CallArg::Ethereum(EthereumArg::Param(_)))); + + // Test Subgraph EntityParam + let arg = CallArg::from_str("entity.token").unwrap(); + assert!(matches!( + arg, + CallArg::Subgraph(SubgraphArg::EntityParam(_)) + )); + } +} diff --git a/graph/src/data_source/mod.rs b/graph/src/data_source/mod.rs index a38148b25fe..751b71837e7 100644 --- a/graph/src/data_source/mod.rs +++ b/graph/src/data_source/mod.rs @@ -1,6 +1,9 @@ pub mod causality_region; +pub mod common; pub mod offchain; +pub mod subgraph; +pub use self::DataSource as DataSourceEnum; pub use causality_region::CausalityRegion; #[cfg(test)] @@ -16,7 +19,7 @@ use crate::{ link_resolver::LinkResolver, store::{BlockNumber, StoredDynamicDataSource}, }, - data_source::offchain::OFFCHAIN_KINDS, + data_source::{offchain::OFFCHAIN_KINDS, subgraph::SUBGRAPH_DS_KIND}, prelude::{CheapClone as _, DataSourceContext}, schema::{EntityType, InputSchema}, }; @@ -35,6 +38,7 @@ use thiserror::Error; pub enum DataSource { Onchain(C::DataSource), Offchain(offchain::DataSource), + Subgraph(subgraph::DataSource), } #[derive(Error, Debug)] @@ -89,6 +93,23 @@ impl DataSource { match self { Self::Onchain(ds) => Some(ds), Self::Offchain(_) => None, + Self::Subgraph(_) => None, + } + } + + pub fn as_subgraph(&self) -> Option<&subgraph::DataSource> { + match self { + Self::Onchain(_) => None, + Self::Offchain(_) => None, + Self::Subgraph(ds) => Some(ds), + } + } + + pub fn is_chain_based(&self) -> bool { + match self { + Self::Onchain(_) => true, + Self::Offchain(_) => false, + Self::Subgraph(_) => true, } } @@ -96,6 +117,23 @@ impl DataSource { match self { Self::Onchain(_) => None, Self::Offchain(ds) => Some(ds), + Self::Subgraph(_) => None, + } + } + + pub fn network(&self) -> Option<&str> { + match self { + DataSourceEnum::Onchain(ds) => ds.network(), + DataSourceEnum::Offchain(_) => None, + DataSourceEnum::Subgraph(ds) => ds.network(), + } + } + + pub fn start_block(&self) -> Option { + match self { + DataSourceEnum::Onchain(ds) => Some(ds.start_block()), + DataSourceEnum::Offchain(_) => None, + DataSourceEnum::Subgraph(ds) => Some(ds.source.start_block), } } @@ -111,6 +149,7 @@ impl DataSource { match self { Self::Onchain(ds) => ds.address().map(ToOwned::to_owned), Self::Offchain(ds) => ds.address(), + Self::Subgraph(ds) => ds.address(), } } @@ -118,6 +157,7 @@ impl DataSource { match self { Self::Onchain(ds) => ds.name(), Self::Offchain(ds) => &ds.name, + Self::Subgraph(ds) => &ds.name, } } @@ -125,6 +165,7 @@ impl DataSource { match self { Self::Onchain(ds) => ds.kind().to_owned(), Self::Offchain(ds) => ds.kind.to_string(), + Self::Subgraph(ds) => ds.kind.clone(), } } @@ -132,6 +173,7 @@ impl DataSource { match self { Self::Onchain(ds) => ds.min_spec_version(), Self::Offchain(ds) => ds.min_spec_version(), + Self::Subgraph(ds) => ds.min_spec_version(), } } @@ -139,6 +181,7 @@ impl DataSource { match self { Self::Onchain(ds) => ds.end_block(), Self::Offchain(_) => None, + Self::Subgraph(_) => None, } } @@ -146,6 +189,7 @@ impl DataSource { match self { Self::Onchain(ds) => ds.creation_block(), Self::Offchain(ds) => ds.creation_block, + Self::Subgraph(ds) => ds.creation_block, } } @@ -153,6 +197,7 @@ impl DataSource { match self { Self::Onchain(ds) => ds.context(), Self::Offchain(ds) => ds.context.clone(), + Self::Subgraph(ds) => ds.context.clone(), } } @@ -160,6 +205,7 @@ impl DataSource { match self { Self::Onchain(ds) => ds.api_version(), Self::Offchain(ds) => ds.mapping.api_version.clone(), + Self::Subgraph(ds) => ds.mapping.api_version.clone(), } } @@ -167,6 +213,7 @@ impl DataSource { match self { Self::Onchain(ds) => ds.runtime(), Self::Offchain(ds) => Some(ds.mapping.runtime.cheap_clone()), + Self::Subgraph(ds) => Some(ds.mapping.runtime.cheap_clone()), } } @@ -176,6 +223,7 @@ impl DataSource { // been enforced. Self::Onchain(_) => EntityTypeAccess::Any, Self::Offchain(ds) => EntityTypeAccess::Restriced(ds.mapping.entities.clone()), + Self::Subgraph(_) => EntityTypeAccess::Any, } } @@ -183,6 +231,7 @@ impl DataSource { match self { Self::Onchain(ds) => ds.handler_kinds(), Self::Offchain(ds) => vec![ds.handler_kind()].into_iter().collect(), + Self::Subgraph(ds) => vec![ds.handler_kind()].into_iter().collect(), } } @@ -190,6 +239,7 @@ impl DataSource { match self { Self::Onchain(ds) => ds.has_declared_calls(), Self::Offchain(_) => false, + Self::Subgraph(_) => false, } } @@ -207,8 +257,15 @@ impl DataSource { (Self::Offchain(ds), TriggerData::Offchain(trigger)) => { Ok(ds.match_and_decode(trigger)) } + (Self::Subgraph(ds), TriggerData::Subgraph(trigger)) => { + ds.match_and_decode(block, trigger) + } (Self::Onchain(_), TriggerData::Offchain(_)) - | (Self::Offchain(_), TriggerData::Onchain(_)) => Ok(None), + | (Self::Offchain(_), TriggerData::Onchain(_)) + | (Self::Onchain(_), TriggerData::Subgraph(_)) + | (Self::Offchain(_), TriggerData::Subgraph(_)) + | (Self::Subgraph(_), TriggerData::Onchain(_)) + | (Self::Subgraph(_), TriggerData::Offchain(_)) => Ok(None), } } @@ -224,6 +281,7 @@ impl DataSource { match self { Self::Onchain(ds) => ds.as_stored_dynamic_data_source(), Self::Offchain(ds) => ds.as_stored_dynamic_data_source(), + Self::Subgraph(_) => todo!(), // TODO(krishna) } } @@ -240,6 +298,7 @@ impl DataSource { offchain::DataSource::from_stored_dynamic_data_source(template, stored) .map(DataSource::Offchain) } + DataSourceTemplate::Subgraph(_) => todo!(), // TODO(krishna) } } @@ -247,6 +306,7 @@ impl DataSource { match self { Self::Onchain(ds) => ds.validate(spec_version), Self::Offchain(_) => vec![], + Self::Subgraph(_) => vec![], // TODO(krishna) } } @@ -254,6 +314,7 @@ impl DataSource { match self { Self::Onchain(_) => CausalityRegion::ONCHAIN, Self::Offchain(ds) => ds.causality_region, + Self::Subgraph(_) => CausalityRegion::ONCHAIN, } } } @@ -262,6 +323,7 @@ impl DataSource { pub enum UnresolvedDataSource { Onchain(C::UnresolvedDataSource), Offchain(offchain::UnresolvedDataSource), + Subgraph(subgraph::UnresolvedDataSource), } impl UnresolvedDataSource { @@ -276,6 +338,10 @@ impl UnresolvedDataSource { .resolve(resolver, logger, manifest_idx) .await .map(DataSource::Onchain), + Self::Subgraph(unresolved) => unresolved + .resolve(resolver, logger, manifest_idx) + .await + .map(DataSource::Subgraph), Self::Offchain(_unresolved) => { anyhow::bail!( "static file data sources are not yet supported, \\ @@ -299,6 +365,7 @@ pub struct DataSourceTemplateInfo { pub enum DataSourceTemplate { Onchain(C::DataSourceTemplate), Offchain(offchain::DataSourceTemplate), + Subgraph(subgraph::DataSourceTemplate), } impl DataSourceTemplate { @@ -306,6 +373,7 @@ impl DataSourceTemplate { match self { DataSourceTemplate::Onchain(template) => template.info(), DataSourceTemplate::Offchain(template) => template.clone().into(), + DataSourceTemplate::Subgraph(template) => template.clone().into(), } } @@ -313,6 +381,7 @@ impl DataSourceTemplate { match self { Self::Onchain(ds) => Some(ds), Self::Offchain(_) => None, + Self::Subgraph(_) => todo!(), // TODO(krishna) } } @@ -320,6 +389,7 @@ impl DataSourceTemplate { match self { Self::Onchain(_) => None, Self::Offchain(t) => Some(t), + Self::Subgraph(_) => todo!(), // TODO(krishna) } } @@ -327,6 +397,7 @@ impl DataSourceTemplate { match self { Self::Onchain(ds) => Some(ds), Self::Offchain(_) => None, + Self::Subgraph(_) => todo!(), // TODO(krishna) } } @@ -334,6 +405,7 @@ impl DataSourceTemplate { match self { Self::Onchain(ds) => &ds.name(), Self::Offchain(ds) => &ds.name, + Self::Subgraph(ds) => &ds.name, } } @@ -341,6 +413,7 @@ impl DataSourceTemplate { match self { Self::Onchain(ds) => ds.api_version(), Self::Offchain(ds) => ds.mapping.api_version.clone(), + Self::Subgraph(ds) => ds.mapping.api_version.clone(), } } @@ -348,6 +421,7 @@ impl DataSourceTemplate { match self { Self::Onchain(ds) => ds.runtime(), Self::Offchain(ds) => Some(ds.mapping.runtime.clone()), + Self::Subgraph(ds) => Some(ds.mapping.runtime.clone()), } } @@ -355,6 +429,7 @@ impl DataSourceTemplate { match self { Self::Onchain(ds) => ds.manifest_idx(), Self::Offchain(ds) => ds.manifest_idx, + Self::Subgraph(ds) => ds.manifest_idx, } } @@ -362,6 +437,7 @@ impl DataSourceTemplate { match self { Self::Onchain(ds) => ds.kind().to_string(), Self::Offchain(ds) => ds.kind.to_string(), + Self::Subgraph(ds) => ds.kind.clone(), } } } @@ -370,6 +446,7 @@ impl DataSourceTemplate { pub enum UnresolvedDataSourceTemplate { Onchain(C::UnresolvedDataSourceTemplate), Offchain(offchain::UnresolvedDataSourceTemplate), + Subgraph(subgraph::UnresolvedDataSourceTemplate), } impl Default for UnresolvedDataSourceTemplate { @@ -395,6 +472,10 @@ impl UnresolvedDataSourceTemplate { .resolve(resolver, logger, manifest_idx, schema) .await .map(DataSourceTemplate::Offchain), + Self::Subgraph(ds) => ds + .resolve(resolver, logger, manifest_idx) + .await + .map(DataSourceTemplate::Subgraph), } } } @@ -475,6 +556,7 @@ impl TriggerWithHandler { pub enum TriggerData { Onchain(C::TriggerData), Offchain(offchain::TriggerData), + Subgraph(subgraph::TriggerData), } impl TriggerData { @@ -482,6 +564,7 @@ impl TriggerData { match self { Self::Onchain(trigger) => trigger.error_context(), Self::Offchain(trigger) => format!("{:?}", trigger.source), + Self::Subgraph(trigger) => format!("{:?}", trigger.source), } } } @@ -490,6 +573,7 @@ impl TriggerData { pub enum MappingTrigger { Onchain(C::MappingTrigger), Offchain(offchain::TriggerData), + Subgraph(subgraph::MappingEntityTrigger), } impl MappingTrigger { @@ -497,6 +581,7 @@ impl MappingTrigger { match self { Self::Onchain(trigger) => Some(trigger.error_context()), Self::Offchain(_) => None, // TODO: Add error context for offchain triggers + Self::Subgraph(_) => None, // TODO(krishna) } } @@ -504,6 +589,7 @@ impl MappingTrigger { match self { Self::Onchain(trigger) => Some(trigger), Self::Offchain(_) => None, + Self::Subgraph(_) => None, // TODO(krishna) } } } @@ -515,6 +601,7 @@ macro_rules! clone_data_source { match self { Self::Onchain(ds) => Self::Onchain(ds.clone()), Self::Offchain(ds) => Self::Offchain(ds.clone()), + Self::Subgraph(ds) => Self::Subgraph(ds.clone()), } } } @@ -541,6 +628,10 @@ macro_rules! deserialize_data_source { offchain::$t::deserialize(map.into_deserializer()) .map_err(serde::de::Error::custom) .map($t::Offchain) + } else if SUBGRAPH_DS_KIND == kind { + subgraph::$t::deserialize(map.into_deserializer()) + .map_err(serde::de::Error::custom) + .map($t::Subgraph) } else if (&C::KIND.to_string() == kind) || C::ALIASES.contains(&kind) { C::$t::deserialize(map.into_deserializer()) .map_err(serde::de::Error::custom) diff --git a/graph/src/data_source/subgraph.rs b/graph/src/data_source/subgraph.rs new file mode 100644 index 00000000000..93f5d920825 --- /dev/null +++ b/graph/src/data_source/subgraph.rs @@ -0,0 +1,377 @@ +use crate::{ + blockchain::{block_stream::EntitySourceOperation, Block, Blockchain}, + components::{link_resolver::LinkResolver, store::BlockNumber}, + data::{ + subgraph::{calls_host_fn, SPEC_VERSION_1_3_0}, + value::Word, + }, + data_source::{self, common::DeclaredCall}, + ensure, + prelude::{CheapClone, DataSourceContext, DeploymentHash, Link}, +}; +use anyhow::{anyhow, Context, Error, Result}; +use futures03::{stream::FuturesOrdered, TryStreamExt}; +use serde::Deserialize; +use slog::{info, Logger}; +use std::{fmt, sync::Arc}; + +use super::{ + common::{CallDecls, FindMappingABI, MappingABI, UnresolvedMappingABI}, + DataSourceTemplateInfo, TriggerWithHandler, +}; + +pub const SUBGRAPH_DS_KIND: &str = "subgraph"; + +const ENTITY_HANDLER_KINDS: &str = "entity"; + +#[derive(Debug, Clone)] +pub struct DataSource { + pub kind: String, + pub name: String, + pub network: String, + pub manifest_idx: u32, + pub source: Source, + pub mapping: Mapping, + pub context: Arc>, + pub creation_block: Option, +} + +impl DataSource { + pub fn new( + kind: String, + name: String, + network: String, + manifest_idx: u32, + source: Source, + mapping: Mapping, + context: Arc>, + creation_block: Option, + ) -> Self { + Self { + kind, + name, + network, + manifest_idx, + source, + mapping, + context, + creation_block, + } + } + + pub fn min_spec_version(&self) -> semver::Version { + SPEC_VERSION_1_3_0 + } + + pub fn handler_kind(&self) -> &str { + ENTITY_HANDLER_KINDS + } + + pub fn network(&self) -> Option<&str> { + Some(&self.network) + } + + pub fn match_and_decode( + &self, + block: &Arc, + trigger: &TriggerData, + ) -> Result>>> { + if self.source.address != trigger.source { + return Ok(None); + } + + let mut matching_handlers: Vec<_> = self + .mapping + .handlers + .iter() + .filter(|handler| handler.entity == trigger.entity_type()) + .collect(); + + // Get the matching handler if any + let handler = match matching_handlers.pop() { + Some(handler) => handler, + None => return Ok(None), + }; + + ensure!( + matching_handlers.is_empty(), + format!( + "Multiple handlers defined for entity `{}`, only one is supported", + trigger.entity_type() + ) + ); + + let calls = + DeclaredCall::from_entity_trigger(&self.mapping, &handler.calls, &trigger.entity)?; + let mapping_trigger = MappingEntityTrigger { + data: trigger.clone(), + calls, + }; + + Ok(Some(TriggerWithHandler::new( + data_source::MappingTrigger::Subgraph(mapping_trigger), + handler.handler.clone(), + block.ptr(), + block.timestamp(), + ))) + } + + pub fn address(&self) -> Option> { + Some(self.source.address().to_bytes()) + } + + pub fn source_subgraph(&self) -> DeploymentHash { + self.source.address() + } +} + +pub type Base64 = Word; + +#[derive(Clone, Debug, Default, Hash, Eq, PartialEq, Deserialize)] +pub struct Source { + pub address: DeploymentHash, + #[serde(default)] + pub start_block: BlockNumber, +} + +impl Source { + /// The concept of an address may or not make sense for a subgraph data source, but graph node + /// will use this in a few places where some sort of not necessarily unique id is useful: + /// 1. This is used as the value to be returned to mappings from the `dataSource.address()` host + /// function, so changing this is a breaking change. + /// 2. This is used to match with triggers with hosts in `fn hosts_for_trigger`, so make sure + /// the `source` of the data source is equal the `source` of the `TriggerData`. + pub fn address(&self) -> DeploymentHash { + self.address.clone() + } +} + +#[derive(Clone, Debug)] +pub struct Mapping { + pub language: String, + pub api_version: semver::Version, + pub abis: Vec>, + pub entities: Vec, + pub handlers: Vec, + pub runtime: Arc>, + pub link: Link, +} + +impl Mapping { + pub fn requires_archive(&self) -> anyhow::Result { + calls_host_fn(&self.runtime, "ethereum.call") + } +} + +impl FindMappingABI for Mapping { + fn find_abi(&self, abi_name: &str) -> Result, Error> { + Ok(self + .abis + .iter() + .find(|abi| abi.name == abi_name) + .ok_or_else(|| anyhow!("No ABI entry with name `{}` found", abi_name))? + .cheap_clone()) + } +} + +#[derive(Clone, Debug, Hash, Eq, PartialEq, Deserialize)] +pub struct EntityHandler { + pub handler: String, + pub entity: String, + #[serde(default)] + pub calls: CallDecls, +} + +#[derive(Clone, Debug, Default, Eq, PartialEq, Deserialize)] +pub struct UnresolvedDataSource { + pub kind: String, + pub name: String, + pub network: String, + pub source: UnresolvedSource, + pub mapping: UnresolvedMapping, +} + +#[derive(Clone, Debug, Default, Hash, Eq, PartialEq, Deserialize)] +#[serde(rename_all = "camelCase")] +pub struct UnresolvedSource { + address: DeploymentHash, + #[serde(default)] + start_block: BlockNumber, +} + +#[derive(Clone, Debug, Default, Hash, Eq, PartialEq, Deserialize)] +#[serde(rename_all = "camelCase")] +pub struct UnresolvedMapping { + pub api_version: String, + pub language: String, + pub file: Link, + pub handlers: Vec, + pub abis: Option>, + pub entities: Vec, +} + +impl UnresolvedDataSource { + #[allow(dead_code)] + pub(super) async fn resolve( + self, + resolver: &Arc, + logger: &Logger, + manifest_idx: u32, + ) -> Result { + info!(logger, "Resolve subgraph data source"; + "name" => &self.name, + "kind" => &self.kind, + "source" => format_args!("{:?}", &self.source), + ); + + let kind = self.kind; + let source = Source { + address: self.source.address, + start_block: self.source.start_block, + }; + + Ok(DataSource { + manifest_idx, + kind, + name: self.name, + network: self.network, + source, + mapping: self.mapping.resolve(resolver, logger).await?, + context: Arc::new(None), + creation_block: None, + }) + } +} + +impl UnresolvedMapping { + pub async fn resolve( + self, + resolver: &Arc, + logger: &Logger, + ) -> Result { + info!(logger, "Resolve subgraph ds mapping"; "link" => &self.file.link); + + // Resolve each ABI and collect the results + let abis = match self.abis { + Some(abis) => { + abis.into_iter() + .map(|unresolved_abi| { + let resolver = Arc::clone(resolver); + let logger = logger.clone(); + async move { + let resolved_abi = unresolved_abi.resolve(&resolver, &logger).await?; + Ok::<_, Error>(Arc::new(resolved_abi)) + } + }) + .collect::>() + .try_collect::>() + .await? + } + None => Vec::new(), + }; + + Ok(Mapping { + language: self.language, + api_version: semver::Version::parse(&self.api_version)?, + entities: self.entities, + handlers: self.handlers, + abis, + runtime: Arc::new(resolver.cat(logger, &self.file).await?), + link: self.file, + }) + } +} + +#[derive(Clone, Debug, Deserialize)] +pub struct UnresolvedDataSourceTemplate { + pub kind: String, + pub network: Option, + pub name: String, + pub mapping: UnresolvedMapping, +} + +#[derive(Clone, Debug)] +pub struct DataSourceTemplate { + pub kind: String, + pub network: Option, + pub name: String, + pub manifest_idx: u32, + pub mapping: Mapping, +} + +impl Into for DataSourceTemplate { + fn into(self) -> DataSourceTemplateInfo { + let DataSourceTemplate { + kind, + network: _, + name, + manifest_idx, + mapping, + } = self; + + DataSourceTemplateInfo { + api_version: mapping.api_version.clone(), + runtime: Some(mapping.runtime), + name, + manifest_idx: Some(manifest_idx), + kind: kind.to_string(), + } + } +} + +impl UnresolvedDataSourceTemplate { + pub async fn resolve( + self, + resolver: &Arc, + logger: &Logger, + manifest_idx: u32, + ) -> Result { + let kind = self.kind; + + let mapping = self + .mapping + .resolve(resolver, logger) + .await + .with_context(|| format!("failed to resolve data source template {}", self.name))?; + + Ok(DataSourceTemplate { + kind, + network: self.network, + name: self.name, + manifest_idx, + mapping, + }) + } +} + +#[derive(Clone, PartialEq, Debug)] +pub struct MappingEntityTrigger { + pub data: TriggerData, + pub calls: Vec, +} + +#[derive(Clone, PartialEq, Eq)] +pub struct TriggerData { + pub source: DeploymentHash, + pub entity: EntitySourceOperation, +} + +impl TriggerData { + pub fn new(source: DeploymentHash, entity: EntitySourceOperation) -> Self { + Self { source, entity } + } + + pub fn entity_type(&self) -> &str { + self.entity.entity_type.as_str() + } +} + +impl fmt::Debug for TriggerData { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!( + f, + "TriggerData {{ source: {:?}, entity: {:?} }}", + self.source, self.entity, + ) + } +} diff --git a/graph/src/env/mod.rs b/graph/src/env/mod.rs index af53562528a..3b1ca5a2862 100644 --- a/graph/src/env/mod.rs +++ b/graph/src/env/mod.rs @@ -357,7 +357,7 @@ struct Inner { default = "false" )] allow_non_deterministic_fulltext_search: EnvVarBoolean, - #[envconfig(from = "GRAPH_MAX_SPEC_VERSION", default = "1.2.0")] + #[envconfig(from = "GRAPH_MAX_SPEC_VERSION", default = "1.3.0")] max_spec_version: Version, #[envconfig(from = "GRAPH_LOAD_WINDOW_SIZE", default = "300")] load_window_size_in_secs: u64, diff --git a/graph/src/firehose/endpoints.rs b/graph/src/firehose/endpoints.rs index ef00ec53c03..677f73edef4 100644 --- a/graph/src/firehose/endpoints.rs +++ b/graph/src/firehose/endpoints.rs @@ -25,7 +25,7 @@ use futures03::StreamExt; use http0::uri::{Scheme, Uri}; use itertools::Itertools; use prost::Message; -use slog::Logger; +use slog::{error, Logger}; use std::{ collections::HashMap, fmt::Display, marker::PhantomData, ops::ControlFlow, str::FromStr, sync::Arc, time::Duration, @@ -509,6 +509,110 @@ impl FirehoseEndpoint { } } + pub async fn get_block_by_ptr( + &self, + ptr: &BlockPtr, + logger: &Logger, + ) -> Result + where + M: prost::Message + BlockchainBlock + Default + 'static, + { + debug!( + logger, + "Connecting to firehose to retrieve block for ptr {}", ptr; + "provider" => self.provider.as_str(), + ); + + let req = firehose::SingleBlockRequest { + transforms: [].to_vec(), + reference: Some( + firehose::single_block_request::Reference::BlockHashAndNumber( + firehose::single_block_request::BlockHashAndNumber { + hash: ptr.hash.to_string(), + num: ptr.number as u64, + }, + ), + ), + }; + + let mut client = self.new_client(); + match client.block(req).await { + Ok(v) => Ok(M::decode( + v.get_ref().block.as_ref().unwrap().value.as_ref(), + )?), + Err(e) => return Err(anyhow::format_err!("firehose error {}", e)), + } + } + + pub async fn get_block_by_number( + &self, + number: u64, + logger: &Logger, + ) -> Result + where + M: prost::Message + BlockchainBlock + Default + 'static, + { + debug!( + logger, + "Connecting to firehose to retrieve block for number {}", number; + "provider" => self.provider.as_str(), + ); + + let req = firehose::SingleBlockRequest { + transforms: [].to_vec(), + reference: Some(firehose::single_block_request::Reference::BlockNumber( + firehose::single_block_request::BlockNumber { num: number }, + )), + }; + + let mut client = self.new_client(); + match client.block(req).await { + Ok(v) => Ok(M::decode( + v.get_ref().block.as_ref().unwrap().value.as_ref(), + )?), + Err(e) => return Err(anyhow::format_err!("firehose error {}", e)), + } + } + + pub async fn load_blocks_by_numbers( + &self, + numbers: Vec, + logger: &Logger, + ) -> Result, anyhow::Error> + where + M: prost::Message + BlockchainBlock + Default + 'static, + { + let mut blocks = Vec::with_capacity(numbers.len()); + + for number in numbers { + debug!( + logger, + "Loading block for block number {}", number; + "provider" => self.provider.as_str(), + ); + + match self.get_block_by_number::(number, logger).await { + Ok(block) => { + blocks.push(block); + } + Err(e) => { + error!( + logger, + "Failed to load block number {}: {}", number, e; + "provider" => self.provider.as_str(), + ); + return Err(anyhow::format_err!( + "failed to load block number {}: {}", + number, + e + )); + } + } + } + + Ok(blocks) + } + pub async fn genesis_block_ptr(&self, logger: &Logger) -> Result where M: prost::Message + BlockchainBlock + Default + 'static, diff --git a/graph/src/runtime/mod.rs b/graph/src/runtime/mod.rs index d20d1eccde3..f015e1e9563 100644 --- a/graph/src/runtime/mod.rs +++ b/graph/src/runtime/mod.rs @@ -368,6 +368,9 @@ pub enum IndexForAscTypeId { // ... // LastStarknetType = 4499, + // Subgraph Data Source types + AscEntityTrigger = 4500, + // Reserved discriminant space for a future blockchain type IDs: [4,500, 5,499] // // Generated with the following shell script: diff --git a/graph/src/schema/input/mod.rs b/graph/src/schema/input/mod.rs index 84897299785..440873f47d6 100644 --- a/graph/src/schema/input/mod.rs +++ b/graph/src/schema/input/mod.rs @@ -35,6 +35,7 @@ pub(crate) const POI_OBJECT: &str = "Poi$"; const POI_DIGEST: &str = "digest"; /// The name of the PoI attribute for storing the block time const POI_BLOCK_TIME: &str = "blockTime"; +const VID_FIELD: &str = "vid"; pub mod kw { pub const ENTITY: &str = "entity"; @@ -1597,6 +1598,8 @@ fn atom_pool(document: &s::Document) -> AtomPool { pool.intern(POI_DIGEST); pool.intern(POI_BLOCK_TIME); + pool.intern(VID_FIELD); + for definition in &document.definitions { match definition { s::Definition::TypeDefinition(typedef) => match typedef { diff --git a/graph/src/util/lfu_cache.rs b/graph/src/util/lfu_cache.rs index dbd66bee302..06ec6a475db 100644 --- a/graph/src/util/lfu_cache.rs +++ b/graph/src/util/lfu_cache.rs @@ -179,7 +179,7 @@ impl }) } - pub fn iter<'a>(&'a self) -> impl Iterator { + pub fn iter<'a>(&'a self) -> impl Iterator { self.queue .iter() .map(|entry| (&entry.0.key, &entry.0.value)) diff --git a/node/src/chain.rs b/node/src/chain.rs index 3e87ff8295b..dab3890f24e 100644 --- a/node/src/chain.rs +++ b/node/src/chain.rs @@ -529,6 +529,7 @@ pub async fn networks_as_chains( }; let client = Arc::new(cc); + let eth_adapters = Arc::new(eth_adapters); let adapter_selector = EthereumAdapterSelector::new( logger_factory.clone(), client.clone(), @@ -551,7 +552,7 @@ pub async fn networks_as_chains( Arc::new(EthereumBlockRefetcher {}), Arc::new(adapter_selector), Arc::new(EthereumRuntimeAdapterBuilder {}), - Arc::new(eth_adapters.clone()), + eth_adapters, ENV_VARS.reorg_threshold, polling_interval, true, diff --git a/runtime/test/src/common.rs b/runtime/test/src/common.rs index 46a17f54f22..0462b79d2a0 100644 --- a/runtime/test/src/common.rs +++ b/runtime/test/src/common.rs @@ -3,13 +3,12 @@ use graph::blockchain::BlockTime; use graph::components::store::DeploymentLocator; use graph::data::subgraph::*; use graph::data_source; +use graph::data_source::common::MappingABI; use graph::env::EnvVars; use graph::ipfs_client::IpfsClient; use graph::log; use graph::prelude::*; -use graph_chain_ethereum::{ - Chain, DataSource, DataSourceTemplate, Mapping, MappingABI, TemplateSource, -}; +use graph_chain_ethereum::{Chain, DataSource, DataSourceTemplate, Mapping, TemplateSource}; use graph_runtime_wasm::host_exports::DataSourceDetails; use graph_runtime_wasm::{HostExports, MappingContext}; use semver::Version; diff --git a/runtime/test/src/test.rs b/runtime/test/src/test.rs index 91895e07725..f9c04fb462e 100644 --- a/runtime/test/src/test.rs +++ b/runtime/test/src/test.rs @@ -476,17 +476,17 @@ async fn test_ipfs_block() { // The user_data value we use with calls to ipfs_map const USER_DATA: &str = "user_data"; -fn make_thing(id: &str, value: &str) -> (String, EntityModification) { +fn make_thing(id: &str, value: &str, vid: i64) -> (String, EntityModification) { const DOCUMENT: &str = " type Thing @entity { id: String!, value: String!, extra: String }"; lazy_static! { static ref SCHEMA: InputSchema = InputSchema::raw(DOCUMENT, "doesntmatter"); static ref THING_TYPE: EntityType = SCHEMA.entity_type("Thing").unwrap(); } - let data = entity! { SCHEMA => id: id, value: value, extra: USER_DATA }; + let data = entity! { SCHEMA => id: id, value: value, extra: USER_DATA}; let key = THING_TYPE.parse_key(id).unwrap(); ( format!("{{ \"id\": \"{}\", \"value\": \"{}\"}}", id, value), - EntityModification::insert(key, data, 0), + EntityModification::insert(key, data, 0, vid), ) } @@ -552,8 +552,8 @@ async fn test_ipfs_map(api_version: Version, json_error_msg: &str) { let subgraph_id = "ipfsMap"; // Try it with two valid objects - let (str1, thing1) = make_thing("one", "eins"); - let (str2, thing2) = make_thing("two", "zwei"); + let (str1, thing1) = make_thing("one", "eins", 0); + let (str2, thing2) = make_thing("two", "zwei", 0); let ops = run_ipfs_map( ipfs.clone(), subgraph_id, @@ -1022,8 +1022,8 @@ async fn test_entity_store(api_version: Version) { let schema = store.input_schema(&deployment.hash).unwrap(); - let alex = entity! { schema => id: "alex", name: "Alex" }; - let steve = entity! { schema => id: "steve", name: "Steve" }; + let alex = entity! { schema => id: "alex", name: "Alex", vid: 0i64 }; + let steve = entity! { schema => id: "steve", name: "Steve", vid: 1i64 }; let user_type = schema.entity_type("User").unwrap(); test_store::insert_entities( &deployment, diff --git a/runtime/wasm/src/host.rs b/runtime/wasm/src/host.rs index 3ecee7ba753..bc5610a63d0 100644 --- a/runtime/wasm/src/host.rs +++ b/runtime/wasm/src/host.rs @@ -142,11 +142,7 @@ where ens_lookup, )); - let host_fns = data_source - .as_onchain() - .map(|ds| runtime_adapter.host_fns(ds)) - .transpose()? - .unwrap_or_default(); + let host_fns = runtime_adapter.host_fns(&data_source).unwrap_or_default(); Ok(RuntimeHost { host_fns: Arc::new(host_fns), @@ -366,6 +362,7 @@ impl RuntimeHostTrait for RuntimeHost { match self.data_source() { DataSource::Onchain(_) => None, DataSource::Offchain(ds) => ds.done_at(), + DataSource::Subgraph(_) => None, } } @@ -373,6 +370,7 @@ impl RuntimeHostTrait for RuntimeHost { match self.data_source() { DataSource::Onchain(_) => {} DataSource::Offchain(ds) => ds.set_done_at(block), + DataSource::Subgraph(_) => {} } } diff --git a/runtime/wasm/src/host_exports.rs b/runtime/wasm/src/host_exports.rs index 4d050db23de..3de534b9069 100644 --- a/runtime/wasm/src/host_exports.rs +++ b/runtime/wasm/src/host_exports.rs @@ -350,7 +350,7 @@ impl HostExports { state.metrics.track_entity_write(&entity_type, &entity); - state.entity_cache.set(key, entity)?; + state.entity_cache.set(key, entity, block)?; Ok(()) } diff --git a/runtime/wasm/src/module/mod.rs b/runtime/wasm/src/module/mod.rs index ffe4f7aba8e..4b01b3a5fd8 100644 --- a/runtime/wasm/src/module/mod.rs +++ b/runtime/wasm/src/module/mod.rs @@ -4,6 +4,7 @@ use std::mem::MaybeUninit; use anyhow::anyhow; use anyhow::Error; use graph::blockchain::Blockchain; +use graph::data_source::subgraph; use graph::util::mem::init_slice; use semver::Version; use wasmtime::AsContext; @@ -69,6 +70,26 @@ impl ToAscPtr for offchain::TriggerData { } } +impl ToAscPtr for subgraph::TriggerData { + fn to_asc_ptr( + self, + heap: &mut H, + gas: &GasCounter, + ) -> Result, HostExportError> { + asc_new(heap, &self.entity, gas).map(|ptr| ptr.erase()) + } +} + +impl ToAscPtr for subgraph::MappingEntityTrigger { + fn to_asc_ptr( + self, + heap: &mut H, + gas: &GasCounter, + ) -> Result, HostExportError> { + asc_new(heap, &self.data.entity, gas).map(|ptr| ptr.erase()) + } +} + impl ToAscPtr for MappingTrigger where C::MappingTrigger: ToAscPtr, @@ -81,6 +102,7 @@ where match self { MappingTrigger::Onchain(trigger) => trigger.to_asc_ptr(heap, gas), MappingTrigger::Offchain(trigger) => trigger.to_asc_ptr(heap, gas), + MappingTrigger::Subgraph(trigger) => trigger.to_asc_ptr(heap, gas), } } } diff --git a/runtime/wasm/src/to_from/external.rs b/runtime/wasm/src/to_from/external.rs index f08eacee94f..30740e77696 100644 --- a/runtime/wasm/src/to_from/external.rs +++ b/runtime/wasm/src/to_from/external.rs @@ -1,15 +1,18 @@ use ethabi; +use graph::blockchain::block_stream::{EntityOperationKind, EntitySourceOperation}; use graph::data::store::scalar::Timestamp; use graph::data::value::Word; use graph::prelude::{BigDecimal, BigInt}; use graph::runtime::gas::GasCounter; use graph::runtime::{ - asc_get, asc_new, AscIndexId, AscPtr, AscType, AscValue, HostExportError, ToAscObj, + asc_get, asc_new, AscIndexId, AscPtr, AscType, AscValue, HostExportError, IndexForAscTypeId, + ToAscObj, }; use graph::{data::store, runtime::DeterministicHostError}; use graph::{prelude::serde_json, runtime::FromAscObj}; use graph::{prelude::web3::types as web3, runtime::AscHeap}; +use graph_runtime_derive::AscType; use crate::asc_abi::class::*; @@ -463,3 +466,43 @@ where }) } } + +#[derive(Debug, Clone, Eq, PartialEq, AscType)] +pub enum AscSubgraphEntityOp { + Create, + Modify, + Delete, +} + +#[derive(AscType)] +pub struct AscEntityTrigger { + pub entity_op: AscSubgraphEntityOp, + pub entity_type: AscPtr, + pub entity: AscPtr, + pub vid: i64, +} + +impl ToAscObj for EntitySourceOperation { + fn to_asc_obj( + &self, + heap: &mut H, + gas: &GasCounter, + ) -> Result { + let entity_op = match self.entity_op { + EntityOperationKind::Create => AscSubgraphEntityOp::Create, + EntityOperationKind::Modify => AscSubgraphEntityOp::Modify, + EntityOperationKind::Delete => AscSubgraphEntityOp::Delete, + }; + + Ok(AscEntityTrigger { + entity_op, + entity_type: asc_new(heap, &self.entity_type.as_str(), gas)?, + entity: asc_new(heap, &self.entity.sorted_ref(), gas)?, + vid: self.vid, + }) + } +} + +impl AscIndexId for AscEntityTrigger { + const INDEX_ASC_TYPE_ID: IndexForAscTypeId = IndexForAscTypeId::AscEntityTrigger; +} diff --git a/server/index-node/src/resolver.rs b/server/index-node/src/resolver.rs index fb3937afdc2..870d319dcf1 100644 --- a/server/index-node/src/resolver.rs +++ b/server/index-node/src/resolver.rs @@ -768,7 +768,7 @@ fn entity_changes_to_graphql(entity_changes: Vec) -> r::Value { .push(key.entity_id); } EntityOperation::Set { key, data } => { - updates.entry(key.entity_type).or_default().push(data); + updates.entry(key.entity_type).or_default().push(data.e); } } } diff --git a/store/postgres/src/block_range.rs b/store/postgres/src/block_range.rs index 1d81eac5e81..c7ffb9eba0c 100644 --- a/store/postgres/src/block_range.rs +++ b/store/postgres/src/block_range.rs @@ -50,7 +50,7 @@ lazy_static! { /// The range of blocks for which an entity is valid. We need this struct /// to bind ranges into Diesel queries. -#[derive(Clone, Debug)] +#[derive(Clone, Debug, Copy)] pub struct BlockRange(Bound, Bound); pub(crate) fn first_block_in_range( @@ -132,6 +132,87 @@ impl<'a> QueryFragment for BlockRangeUpperBoundClause<'a> { } } +#[derive(Debug, Clone, Copy)] +pub enum BoundSide { + Lower, + Upper, +} + +/// Helper for generating SQL fragments for selecting entities in a specific block range +#[derive(Debug, Clone, Copy)] +pub enum EntityBlockRange { + Mutable((BlockRange, BoundSide)), + Immutable(BlockRange), +} + +impl EntityBlockRange { + pub fn new( + immutable: bool, + block_range: std::ops::Range, + bound_side: BoundSide, + ) -> Self { + let start: Bound = Bound::Included(block_range.start); + let end: Bound = Bound::Excluded(block_range.end); + let block_range: BlockRange = BlockRange(start, end); + if immutable { + Self::Immutable(block_range) + } else { + Self::Mutable((block_range, bound_side)) + } + } + + /// Outputs SQL that matches only rows whose entities would trigger a change + /// event (Create, Modify, Delete) in a given interval of blocks. Otherwise said + /// a block_range border is contained in an interval of blocks. For instance + /// one of the following: + /// lower(block_range) >= $1 and lower(block_range) <= $2 + /// upper(block_range) >= $1 and upper(block_range) <= $2 + /// block$ >= $1 and block$ <= $2 + pub fn contains<'b>(&'b self, out: &mut AstPass<'_, 'b, Pg>) -> QueryResult<()> { + out.unsafe_to_cache_prepared(); + let block_range = match self { + EntityBlockRange::Mutable((br, _)) => br, + EntityBlockRange::Immutable(br) => br, + }; + let BlockRange(start, finish) = block_range; + + self.compare_column(out); + out.push_sql(">= "); + match start { + Bound::Included(block) => out.push_bind_param::(block)?, + Bound::Excluded(block) => { + out.push_bind_param::(block)?; + out.push_sql("+1"); + } + Bound::Unbounded => unimplemented!(), + }; + out.push_sql(" and"); + self.compare_column(out); + out.push_sql("<= "); + match finish { + Bound::Included(block) => { + out.push_bind_param::(block)?; + out.push_sql("+1"); + } + Bound::Excluded(block) => out.push_bind_param::(block)?, + Bound::Unbounded => unimplemented!(), + }; + Ok(()) + } + + pub fn compare_column(&self, out: &mut AstPass) { + match self { + EntityBlockRange::Mutable((_, BoundSide::Lower)) => { + out.push_sql(" lower(block_range) ") + } + EntityBlockRange::Mutable((_, BoundSide::Upper)) => { + out.push_sql(" upper(block_range) ") + } + EntityBlockRange::Immutable(_) => out.push_sql(" block$ "), + } + } +} + /// Helper for generating various SQL fragments for handling the block range /// of entity versions #[allow(unused)] diff --git a/store/postgres/src/chain_store.rs b/store/postgres/src/chain_store.rs index b399b15b788..097aa799eff 100644 --- a/store/postgres/src/chain_store.rs +++ b/store/postgres/src/chain_store.rs @@ -1,3 +1,4 @@ +use anyhow::anyhow; use diesel::pg::PgConnection; use diesel::prelude::*; use diesel::r2d2::{ConnectionManager, PooledConnection}; @@ -13,6 +14,7 @@ use graph::slog::Logger; use graph::stable_hash::crypto_stable_hash; use graph::util::herd_cache::HerdCache; +use std::collections::BTreeMap; use std::{ collections::HashMap, convert::{TryFrom, TryInto}, @@ -20,9 +22,9 @@ use std::{ sync::Arc, }; -use graph::blockchain::{Block, BlockHash, ChainIdentifier}; +use graph::blockchain::{Block, BlockHash, ChainIdentifier, ExtendedBlockPtr}; use graph::cheap_clone::CheapClone; -use graph::prelude::web3::types::H256; +use graph::prelude::web3::types::{H256, U256}; use graph::prelude::{ async_trait, serde_json as json, transaction_receipt::LightTransactionReceipt, BlockNumber, BlockPtr, CachedEthereumCall, CancelableError, ChainStore as ChainStoreTrait, Error, @@ -52,6 +54,14 @@ impl JsonBlock { data, } } + + fn timestamp(&self) -> Option { + self.data + .as_ref() + .and_then(|data| data.get("timestamp")) + .and_then(|ts| ts.as_str()) + .and_then(|ts| U256::from_dec_str(ts).ok()) + } } /// Tables in the 'public' database schema that store chain-specific data @@ -579,6 +589,50 @@ mod data { Ok(()) } + pub(super) fn block_ptrs_by_numbers( + &self, + conn: &mut PgConnection, + chain: &str, + numbers: &[BlockNumber], + ) -> Result, StoreError> { + let x = match self { + Storage::Shared => { + use public::ethereum_blocks as b; + + b::table + .select(( + b::hash, + b::number, + b::parent_hash, + sql::("coalesce(data -> 'block', data)"), + )) + .filter(b::network_name.eq(chain)) + .filter(b::number.eq_any(Vec::from_iter(numbers.iter().map(|&n| n as i64)))) + .load::<(BlockHash, i64, BlockHash, json::Value)>(conn) + } + Storage::Private(Schema { blocks, .. }) => blocks + .table() + .select(( + blocks.hash(), + blocks.number(), + blocks.parent_hash(), + sql::("coalesce(data -> 'block', data)"), + )) + .filter( + blocks + .number() + .eq_any(Vec::from_iter(numbers.iter().map(|&n| n as i64))), + ) + .load::<(BlockHash, i64, BlockHash, json::Value)>(conn), + }?; + + Ok(x.into_iter() + .map(|(hash, nr, parent, data)| { + JsonBlock::new(BlockPtr::new(hash, nr as i32), parent, Some(data)) + }) + .collect()) + } + pub(super) fn blocks( &self, conn: &mut PgConnection, @@ -1651,7 +1705,10 @@ impl ChainStoreMetrics { } #[derive(Clone, CheapClone)] -struct BlocksLookupResult(Arc, StoreError>>); +enum BlocksLookupResult { + ByHash(Arc, StoreError>>), + ByNumber(Arc>, StoreError>>), +} pub struct ChainStore { logger: Logger, @@ -1870,8 +1927,52 @@ impl ChainStore { .await?; Ok(values) } + + async fn blocks_from_store_by_numbers( + self: &Arc, + numbers: Vec, + ) -> Result>, StoreError> { + let store = self.cheap_clone(); + let pool = self.pool.clone(); + + let values = pool + .with_conn(move |conn, _| { + store + .storage + .block_ptrs_by_numbers(conn, &store.chain, &numbers) + .map_err(CancelableError::from) + }) + .await?; + + let mut block_map = BTreeMap::new(); + + for block in values { + let block_number = block.ptr.block_number(); + block_map + .entry(block_number) + .or_insert_with(Vec::new) + .push(block); + } + + Ok(block_map) + } } +fn json_block_to_block_ptr_ext(json_block: &JsonBlock) -> Result { + let hash = json_block.ptr.hash.clone(); + let number = json_block.ptr.number; + let parent_hash = json_block.parent_hash.clone(); + + let timestamp = json_block + .timestamp() + .ok_or_else(|| anyhow!("Timestamp is missing"))?; + + let ptr = + ExtendedBlockPtr::try_from((hash.as_h256(), number, parent_hash.as_h256(), timestamp)) + .map_err(|e| anyhow!("Failed to convert to ExtendedBlockPtr: {}", e))?; + + Ok(ptr) +} #[async_trait] impl ChainStoreTrait for ChainStore { fn genesis_block_ptr(&self) -> Result { @@ -2065,6 +2166,85 @@ impl ChainStoreTrait for ChainStore { Ok(()) } + async fn block_ptrs_by_numbers( + self: Arc, + numbers: Vec, + ) -> Result>, Error> { + let result = if ENV_VARS.store.disable_block_cache_for_lookup { + let values = self.blocks_from_store_by_numbers(numbers).await?; + + values + } else { + let cached = self.recent_blocks_cache.get_block_ptrs_by_numbers(&numbers); + + let stored = if cached.len() < numbers.len() { + let missing_numbers = numbers + .iter() + .filter(|num| !cached.iter().any(|(ptr, _)| ptr.block_number() == **num)) + .cloned() + .collect::>(); + + let hash = crypto_stable_hash(&missing_numbers); + let this = self.clone(); + let lookup_fut = async move { + let res = this.blocks_from_store_by_numbers(missing_numbers).await; + BlocksLookupResult::ByNumber(Arc::new(res)) + }; + let lookup_herd = self.lookup_herd.cheap_clone(); + let logger = self.logger.cheap_clone(); + let res = match lookup_herd.cached_query(hash, lookup_fut, &logger).await { + (BlocksLookupResult::ByNumber(res), _) => res, + _ => unreachable!(), + }; + let res = Arc::try_unwrap(res).unwrap_or_else(|arc| (*arc).clone()); + + match res { + Ok(blocks) => { + for (_, blocks_for_num) in &blocks { + if blocks.len() == 1 { + self.recent_blocks_cache + .insert_block(blocks_for_num[0].clone()); + } + } + blocks + } + Err(e) => { + return Err(e.into()); + } + } + } else { + BTreeMap::new() + }; + + let cached_map = cached + .into_iter() + .map(|(ptr, data)| (ptr.block_number(), vec![data])) + .collect::>(); + + let mut result = cached_map; + for (num, blocks) in stored { + if !result.contains_key(&num) { + result.insert(num, blocks); + } + } + + result + }; + + let ptrs = result + .into_iter() + .map(|(num, blocks)| { + let ptrs = blocks + .into_iter() + .filter_map(|block| json_block_to_block_ptr_ext(&block).ok()) + .collect(); + (num, ptrs) + }) + .collect(); + + Ok(ptrs) + } + async fn blocks(self: Arc, hashes: Vec) -> Result, Error> { if ENV_VARS.store.disable_block_cache_for_lookup { let values = self @@ -2094,12 +2274,22 @@ impl ChainStoreTrait for ChainStore { let this = self.clone(); let lookup_fut = async move { let res = this.blocks_from_store(hashes).await; - BlocksLookupResult(Arc::new(res)) + BlocksLookupResult::ByHash(Arc::new(res)) }; let lookup_herd = self.lookup_herd.cheap_clone(); let logger = self.logger.cheap_clone(); - let (BlocksLookupResult(res), _) = - lookup_herd.cached_query(hash, lookup_fut, &logger).await; + // This match can only return ByHash because lookup_fut explicitly constructs + // BlocksLookupResult::ByHash. The cache preserves the exact future result, + // so ByNumber variant is structurally impossible here. + let res = match lookup_herd.cached_query(hash, lookup_fut, &logger).await { + (BlocksLookupResult::ByHash(res), _) => res, + (BlocksLookupResult::ByNumber(_), _) => { + Arc::new(Err(StoreError::Unknown(anyhow::anyhow!( + "Unexpected BlocksLookupResult::ByNumber returned from cached block lookup by hash" + )))) + } + }; + // Try to avoid cloning a non-concurrent lookup; it's not // entirely clear whether that will actually avoid a clone // since it depends on a lot of the details of how the @@ -2361,6 +2551,10 @@ mod recent_blocks_cache { .and_then(|block| block.data.as_ref().map(|data| (&block.ptr, data))) } + fn get_block_by_number(&self, number: BlockNumber) -> Option<&JsonBlock> { + self.blocks.get(&number) + } + fn get_ancestor( &self, child_ptr: &BlockPtr, @@ -2483,6 +2677,28 @@ mod recent_blocks_cache { blocks } + pub fn get_block_ptrs_by_numbers( + &self, + numbers: &[BlockNumber], + ) -> Vec<(BlockPtr, JsonBlock)> { + let inner = self.inner.read(); + let mut blocks: Vec<(BlockPtr, JsonBlock)> = Vec::new(); + + for &number in numbers { + if let Some(block) = inner.get_block_by_number(number) { + blocks.push((block.ptr.clone(), block.clone())); + } + } + + inner.metrics.record_hit_and_miss( + &inner.network, + blocks.len(), + numbers.len() - blocks.len(), + ); + + blocks + } + /// Tentatively caches the `ancestor` of a [`BlockPtr`] (`child`), together with /// its associated `data`. Note that for this to work, `child` must be /// in the cache already. The first block in the cache should be diff --git a/store/postgres/src/deployment_store.rs b/store/postgres/src/deployment_store.rs index d8b04faac0b..ed34b2a4e42 100644 --- a/store/postgres/src/deployment_store.rs +++ b/store/postgres/src/deployment_store.rs @@ -4,7 +4,7 @@ use diesel::pg::PgConnection; use diesel::r2d2::{ConnectionManager, PooledConnection}; use diesel::{prelude::*, sql_query}; use graph::anyhow::Context; -use graph::blockchain::block_stream::FirehoseCursor; +use graph::blockchain::block_stream::{EntitySourceOperation, FirehoseCursor}; use graph::blockchain::BlockTime; use graph::components::store::write::RowGroup; use graph::components::store::{ @@ -29,8 +29,8 @@ use lru_time_cache::LruCache; use rand::{seq::SliceRandom, thread_rng}; use std::collections::{BTreeMap, HashMap}; use std::convert::Into; -use std::ops::Deref; use std::ops::{Bound, DerefMut}; +use std::ops::{Deref, Range}; use std::str::FromStr; use std::sync::{atomic::AtomicUsize, Arc, Mutex}; use std::time::{Duration, Instant}; @@ -1056,6 +1056,18 @@ impl DeploymentStore { layout.find_many(&mut conn, ids_for_type, block) } + pub(crate) fn get_range( + &self, + site: Arc, + entity_types: Vec, + causality_region: CausalityRegion, + block_range: Range, + ) -> Result>, StoreError> { + let mut conn = self.get_conn()?; + let layout = self.layout(&mut conn, site)?; + layout.find_range(&mut conn, entity_types, causality_region, block_range) + } + pub(crate) fn get_derived( &self, site: Arc, diff --git a/store/postgres/src/relational.rs b/store/postgres/src/relational.rs index 8ceb8d9c714..3fff8c8dae7 100644 --- a/store/postgres/src/relational.rs +++ b/store/postgres/src/relational.rs @@ -24,6 +24,7 @@ use diesel::serialize::{Output, ToSql}; use diesel::sql_types::Text; use diesel::{connection::SimpleConnection, Connection}; use diesel::{debug_query, sql_query, OptionalExtension, PgConnection, QueryResult, RunQueryDsl}; +use graph::blockchain::block_stream::{EntityOperationKind, EntitySourceOperation}; use graph::blockchain::BlockTime; use graph::cheap_clone::CheapClone; use graph::components::store::write::{RowGroup, WriteChunk}; @@ -46,30 +47,31 @@ use std::borrow::Borrow; use std::collections::{BTreeMap, BTreeSet, HashMap, HashSet}; use std::convert::{From, TryFrom}; use std::fmt::{self, Write}; +use std::ops::Range; use std::str::FromStr; use std::sync::{Arc, Mutex}; use std::time::{Duration, Instant}; use crate::relational_queries::{ - ConflictingEntitiesData, ConflictingEntitiesQuery, FindChangesQuery, FindDerivedQuery, - FindPossibleDeletionsQuery, ReturnedEntityData, + ConflictingEntitiesData, ConflictingEntitiesQuery, EntityDataExt, FindChangesQuery, + FindDerivedQuery, FindPossibleDeletionsQuery, ReturnedEntityData, }; use crate::{ primary::{Namespace, Site}, relational_queries::{ ClampRangeQuery, EntityData, EntityDeletion, FilterCollection, FilterQuery, FindManyQuery, - FindQuery, InsertQuery, RevertClampQuery, RevertRemoveQuery, + FindQuery, FindRangeQuery, InsertQuery, RevertClampQuery, RevertRemoveQuery, }, }; use graph::components::store::DerivedEntityQuery; -use graph::data::store::{Id, IdList, IdType, BYTES_SCALAR}; +use graph::data::store::{EntityV, Id, IdList, IdType, BYTES_SCALAR}; use graph::data::subgraph::schema::POI_TABLE; use graph::prelude::{ anyhow, info, BlockNumber, DeploymentHash, Entity, EntityChange, EntityOperation, Logger, QueryExecutionError, StoreError, StoreEvent, ValueType, BLOCK_NUMBER_MAX, }; -use crate::block_range::{BLOCK_COLUMN, BLOCK_RANGE_COLUMN}; +use crate::block_range::{BoundSide, BLOCK_COLUMN, BLOCK_RANGE_COLUMN}; pub use crate::catalog::Catalog; use crate::connection_pool::ForeignServer; use crate::{catalog, deployment}; @@ -514,6 +516,136 @@ impl Layout { Ok(entities) } + pub fn find_range( + &self, + conn: &mut PgConnection, + entity_types: Vec, + causality_region: CausalityRegion, + block_range: Range, + ) -> Result>, StoreError> { + let mut tables = vec![]; + for et in entity_types { + tables.push(self.table_for_entity(&et)?.as_ref()); + } + let mut entities: BTreeMap> = BTreeMap::new(); + + // Collect all entities that have their 'lower(block_range)' attribute in the + // interval of blocks defined by the variable block_range. For the immutable + // entities the respective attribute is 'block$'. + // Here are all entities that are created or modified in the block_range. + let lower_vec = FindRangeQuery::new( + &tables, + causality_region, + BoundSide::Lower, + block_range.clone(), + ) + .get_results::(conn) + .optional()? + .unwrap_or_default(); + // Collect all entities that have their 'upper(block_range)' attribute in the + // interval of blocks defined by the variable block_range. For the immutable + // entities no entries are returned. + // Here are all entities that are modified or deleted in the block_range, + // but will have the previous versions, i.e. in the case of an update, it's + // the version before the update, and lower_vec will have a corresponding + // entry with the new version. + let upper_vec = + FindRangeQuery::new(&tables, causality_region, BoundSide::Upper, block_range) + .get_results::(conn) + .optional()? + .unwrap_or_default(); + let mut lower_iter = lower_vec.iter().fuse().peekable(); + let mut upper_iter = upper_vec.iter().fuse().peekable(); + let mut lower_now = lower_iter.next(); + let mut upper_now = upper_iter.next(); + // A closure to convert the entity data from the database into entity operation. + let transform = |ede: &EntityDataExt, + entity_op: EntityOperationKind| + -> Result<(EntitySourceOperation, BlockNumber), StoreError> { + let e = EntityData::new(ede.entity.clone(), ede.data.clone()); + let block = ede.block_number; + let entity_type = e.entity_type(&self.input_schema); + let entity = e.deserialize_with_layout::(self, None)?; + let vid = ede.vid; + let ewt = EntitySourceOperation { + entity_op, + entity_type, + entity, + vid, + }; + Ok((ewt, block)) + }; + + // The algorithm is a similar to merge sort algorithm and it relays on the fact that both vectors + // are ordered by (block_number, entity_type, entity_id). It advances simultaneously entities from + // both lower_vec and upper_vec and tries to match entities that have entries in both vectors for + // a particular block. The match is successful if an entry in one array has the same values in the + // other one for the number of the block, entity type and the entity id. The comparison operation + // over the EntityDataExt implements that check. If there is a match it’s a modification operation, + // since both sides of a range are present for that block, entity type and id. If one side of the + // range exists and the other is missing it is a creation or deletion depending on which side is + // present. For immutable entities the entries in upper_vec are missing, hence they are considered + // having a lower bound at particular block and upper bound at infinity. + while lower_now.is_some() || upper_now.is_some() { + let (ewt, block) = match (lower_now, upper_now) { + (Some(lower), Some(upper)) => { + match lower.cmp(&upper) { + std::cmp::Ordering::Greater => { + // we have upper bound at this block, but no lower bounds at the same block so it's deletion + let (ewt, block) = transform(upper, EntityOperationKind::Delete)?; + // advance upper_vec pointer + upper_now = upper_iter.next(); + (ewt, block) + } + std::cmp::Ordering::Less => { + // we have lower bound at this block but no upper bound at the same block so its creation + let (ewt, block) = transform(lower, EntityOperationKind::Create)?; + // advance lower_vec pointer + lower_now = lower_iter.next(); + (ewt, block) + } + std::cmp::Ordering::Equal => { + let (ewt, block) = transform(lower, EntityOperationKind::Modify)?; + // advance both lower_vec and upper_vec pointers + lower_now = lower_iter.next(); + upper_now = upper_iter.next(); + (ewt, block) + } + } + } + (Some(lower), None) => { + // we have lower bound at this block but no upper bound at the same block so its creation + let (ewt, block) = transform(lower, EntityOperationKind::Create)?; + // advance lower_vec pointer + lower_now = lower_iter.next(); + (ewt, block) + } + (None, Some(upper)) => { + // we have upper bound at this block, but no lower bounds at all so it's deletion + let (ewt, block) = transform(upper, EntityOperationKind::Delete)?; + // advance upper_vec pointer + upper_now = upper_iter.next(); + (ewt, block) + } + _ => panic!("Imposible case to happen"), + }; + + match entities.get_mut(&block) { + Some(vec) => vec.push(ewt), + None => { + let _ = entities.insert(block, vec![ewt]); + } + }; + } + + // sort the elements in each blocks bucket by vid + for (_, vec) in &mut entities { + vec.sort_by(|a, b| a.vid.cmp(&b.vid)); + } + + Ok(entities) + } + pub fn find_derived( &self, conn: &mut PgConnection, @@ -565,9 +697,10 @@ impl Layout { let entity_id = data.id(); processed_entities.insert((entity_type.clone(), entity_id.clone())); + let vid = data.vid(); changes.push(EntityOperation::Set { key: entity_type.key_in(entity_id, CausalityRegion::from_entity(&data)), - data, + data: EntityV::new(data, vid), }); } diff --git a/store/postgres/src/relational/ddl.rs b/store/postgres/src/relational/ddl.rs index aa3aefd3561..38a7ac13ff7 100644 --- a/store/postgres/src/relational/ddl.rs +++ b/store/postgres/src/relational/ddl.rs @@ -116,12 +116,18 @@ impl Table { Ok(cols) } + let vid_type = if !self.object.is_object_type() { + "bigserial" + } else { + "bigint" + }; + if self.immutable { writeln!( out, " create table {qname} ( - {vid} bigserial primary key, + {vid} {vid_type} primary key, {block} int not null,\n\ {cols}, unique({id}) @@ -129,6 +135,7 @@ impl Table { qname = self.qualified_name, cols = columns_ddl(self)?, vid = VID_COLUMN, + vid_type = vid_type, block = BLOCK_COLUMN, id = self.primary_key().name ) @@ -137,13 +144,14 @@ impl Table { out, r#" create table {qname} ( - {vid} bigserial primary key, + {vid} {vid_type} primary key, {block_range} int4range not null, {cols} );"#, qname = self.qualified_name, cols = columns_ddl(self)?, vid = VID_COLUMN, + vid_type = vid_type, block_range = BLOCK_RANGE_COLUMN )?; diff --git a/store/postgres/src/relational/ddl_tests.rs b/store/postgres/src/relational/ddl_tests.rs index b7f9b44afac..86e9f232d49 100644 --- a/store/postgres/src/relational/ddl_tests.rs +++ b/store/postgres/src/relational/ddl_tests.rs @@ -384,7 +384,7 @@ create type sgd0815."size" as enum ('large', 'medium', 'small'); create table "sgd0815"."thing" ( - vid bigserial primary key, + vid bigint primary key, block_range int4range not null, "id" text not null, "big_thing" text not null @@ -405,7 +405,7 @@ create index attr_0_1_thing_big_thing create table "sgd0815"."scalar" ( - vid bigserial primary key, + vid bigint primary key, block_range int4range not null, "id" text not null, "bool" boolean, @@ -444,7 +444,7 @@ create index attr_1_7_scalar_color create table "sgd0815"."file_thing" ( - vid bigserial primary key, + vid bigint primary key, block_range int4range not null, causality_region int not null, "id" text not null @@ -469,7 +469,7 @@ create type sgd0815."size" as enum ('large', 'medium', 'small'); create table "sgd0815"."thing" ( - vid bigserial primary key, + vid bigint primary key, block_range int4range not null, "id" text not null, "big_thing" text not null @@ -490,7 +490,7 @@ create index attr_0_1_thing_big_thing create table "sgd0815"."scalar" ( - vid bigserial primary key, + vid bigint primary key, block_range int4range not null, "id" text not null, "bool" boolean, @@ -515,7 +515,7 @@ create index attr_1_0_scalar_id create table "sgd0815"."file_thing" ( - vid bigserial primary key, + vid bigint primary key, block_range int4range not null, causality_region int not null, "id" text not null @@ -575,7 +575,7 @@ type SongStat @entity { played: Int! }"#; const MUSIC_DDL: &str = r#"create table "sgd0815"."musician" ( - vid bigserial primary key, + vid bigint primary key, block_range int4range not null, "id" text not null, "name" text not null, @@ -598,7 +598,7 @@ create index attr_0_2_musician_main_band on "sgd0815"."musician" using gist("main_band", block_range); create table "sgd0815"."band" ( - vid bigserial primary key, + vid bigint primary key, block_range int4range not null, "id" text not null, "name" text not null, @@ -618,8 +618,8 @@ create index attr_1_1_band_name on "sgd0815"."band" using btree(left("name", 256)); create table "sgd0815"."song" ( - vid bigserial primary key, - block$ int not null, + vid bigint primary key, + block$ int not null, "id" text not null, "title" text not null, "written_by" text not null, @@ -634,7 +634,7 @@ create index attr_2_1_song_written_by on "sgd0815"."song" using btree("written_by", block$); create table "sgd0815"."song_stat" ( - vid bigserial primary key, + vid bigint primary key, block_range int4range not null, "id" text not null, "played" int4 not null @@ -676,7 +676,7 @@ type Habitat @entity { }"#; const FOREST_DDL: &str = r#"create table "sgd0815"."animal" ( - vid bigserial primary key, + vid bigint primary key, block_range int4range not null, "id" text not null, "forest" text @@ -695,8 +695,8 @@ create index attr_0_1_animal_forest on "sgd0815"."animal" using gist("forest", block_range); create table "sgd0815"."forest" ( - vid bigserial primary key, - block_range int4range not null, + vid bigint primary key, + block_range int4range not null, "id" text not null ); alter table "sgd0815"."forest" @@ -711,7 +711,7 @@ create index attr_1_0_forest_id on "sgd0815"."forest" using btree("id"); create table "sgd0815"."habitat" ( - vid bigserial primary key, + vid bigint primary key, block_range int4range not null, "id" text not null, "most_common" text not null, @@ -763,7 +763,7 @@ type Habitat @entity { }"#; const FULLTEXT_DDL: &str = r#"create table "sgd0815"."animal" ( - vid bigserial primary key, + vid bigint primary key, block_range int4range not null, "id" text not null, "name" text not null, @@ -791,7 +791,7 @@ create index attr_0_4_animal_search on "sgd0815"."animal" using gin("search"); create table "sgd0815"."forest" ( - vid bigserial primary key, + vid bigint primary key, block_range int4range not null, "id" text not null ); @@ -808,7 +808,7 @@ create index attr_1_0_forest_id on "sgd0815"."forest" using btree("id"); create table "sgd0815"."habitat" ( - vid bigserial primary key, + vid bigint primary key, block_range int4range not null, "id" text not null, "most_common" text not null, @@ -843,7 +843,7 @@ enum Orientation { const FORWARD_ENUM_SQL: &str = r#"create type sgd0815."orientation" as enum ('DOWN', 'UP'); create table "sgd0815"."thing" ( - vid bigserial primary key, + vid bigint primary key, block_range int4range not null, "id" text not null, "orientation" "sgd0815"."orientation" not null @@ -880,8 +880,8 @@ type Stats @aggregation(intervals: ["hour", "day"], source: "Data") { const TS_SQL: &str = r#" create table "sgd0815"."data" ( - vid bigserial primary key, - block$ int not null, + vid bigint primary key, + block$ int not null, "id" int8 not null, "timestamp" timestamptz not null, "amount" numeric not null, @@ -896,7 +896,7 @@ create index attr_0_1_data_amount create table "sgd0815"."stats_hour" ( vid bigserial primary key, - block$ int not null, + block$ int not null, "id" int8 not null, "timestamp" timestamptz not null, "volume" numeric not null, @@ -971,9 +971,9 @@ const LIFETIME_GQL: &str = r#" const LIFETIME_SQL: &str = r#" create table "sgd0815"."data" ( - vid bigserial primary key, - block$ int not null, -"id" int8 not null, + vid bigint primary key, + block$ int not null, + "id" int8 not null, "timestamp" timestamptz not null, "group_1" int4 not null, "group_2" int4 not null, @@ -993,8 +993,8 @@ on "sgd0815"."data" using btree("amount"); create table "sgd0815"."stats_1_hour" ( vid bigserial primary key, - block$ int not null, -"id" int8 not null, + block$ int not null, + "id" int8 not null, "timestamp" timestamptz not null, "volume" numeric not null, unique(id) @@ -1009,8 +1009,8 @@ on "sgd0815"."stats_1_hour" using btree("volume"); create table "sgd0815"."stats_1_day" ( vid bigserial primary key, - block$ int not null, -"id" int8 not null, + block$ int not null, + "id" int8 not null, "timestamp" timestamptz not null, "volume" numeric not null, unique(id) @@ -1025,8 +1025,8 @@ on "sgd0815"."stats_1_day" using btree("volume"); create table "sgd0815"."stats_2_hour" ( vid bigserial primary key, - block$ int not null, -"id" int8 not null, + block$ int not null, + "id" int8 not null, "timestamp" timestamptz not null, "group_1" int4 not null, "volume" numeric not null, @@ -1045,8 +1045,8 @@ on "sgd0815"."stats_2_hour"(group_1, timestamp); create table "sgd0815"."stats_2_day" ( vid bigserial primary key, - block$ int not null, -"id" int8 not null, + block$ int not null, + "id" int8 not null, "timestamp" timestamptz not null, "group_1" int4 not null, "volume" numeric not null, @@ -1065,8 +1065,8 @@ on "sgd0815"."stats_2_day"(group_1, timestamp); create table "sgd0815"."stats_3_hour" ( vid bigserial primary key, - block$ int not null, -"id" int8 not null, + block$ int not null, + "id" int8 not null, "timestamp" timestamptz not null, "group_2" int4 not null, "group_1" int4 not null, @@ -1088,8 +1088,8 @@ on "sgd0815"."stats_3_hour"(group_2, group_1, timestamp); create table "sgd0815"."stats_3_day" ( vid bigserial primary key, - block$ int not null, -"id" int8 not null, + block$ int not null, + "id" int8 not null, "timestamp" timestamptz not null, "group_2" int4 not null, "group_1" int4 not null, diff --git a/store/postgres/src/relational/prune.rs b/store/postgres/src/relational/prune.rs index 6b5fcdc6940..39337d2a485 100644 --- a/store/postgres/src/relational/prune.rs +++ b/store/postgres/src/relational/prune.rs @@ -17,12 +17,7 @@ use graph::{ }; use itertools::Itertools; -use crate::{ - catalog, - copy::AdaptiveBatchSize, - deployment, - relational::{Table, VID_COLUMN}, -}; +use crate::{catalog, copy::AdaptiveBatchSize, deployment, relational::Table}; use super::{Catalog, Layout, Namespace}; @@ -73,7 +68,6 @@ struct TablePair { // has the same name as `src` but is in a different namespace dst: Arc, src_nsp: Namespace, - dst_nsp: Namespace, } impl TablePair { @@ -100,12 +94,7 @@ impl TablePair { } conn.batch_execute(&query)?; - Ok(TablePair { - src, - dst, - src_nsp, - dst_nsp, - }) + Ok(TablePair { src, dst, src_nsp }) } /// Copy all entity versions visible between `earliest_block` and @@ -239,10 +228,6 @@ impl TablePair { let src_qname = &self.src.qualified_name; let dst_qname = &self.dst.qualified_name; let src_nsp = &self.src_nsp; - let dst_nsp = &self.dst_nsp; - - let vid_seq = format!("{}_{VID_COLUMN}_seq", self.src.name); - let mut query = String::new(); // What we are about to do would get blocked by autovacuum on our @@ -252,13 +237,6 @@ impl TablePair { "src" => src_nsp.as_str(), "error" => e.to_string()); } - // Make sure the vid sequence - // continues from where it was - writeln!( - query, - "select setval('{dst_nsp}.{vid_seq}', nextval('{src_nsp}.{vid_seq}'));" - )?; - writeln!(query, "drop table {src_qname};")?; writeln!(query, "alter table {dst_qname} set schema {src_nsp}")?; conn.transaction(|conn| conn.batch_execute(&query))?; diff --git a/store/postgres/src/relational_queries.rs b/store/postgres/src/relational_queries.rs index 4626ce0479e..681266f6a0e 100644 --- a/store/postgres/src/relational_queries.rs +++ b/store/postgres/src/relational_queries.rs @@ -26,13 +26,16 @@ use graph::schema::{EntityKey, EntityType, FulltextAlgorithm, FulltextConfig, In use graph::{components::store::AttributeNames, data::store::scalar}; use inflector::Inflector; use itertools::Itertools; +use std::cmp::Ordering; use std::collections::{BTreeMap, BTreeSet, HashSet}; use std::convert::TryFrom; use std::fmt::{self, Display}; use std::iter::FromIterator; +use std::ops::Range; use std::str::FromStr; use std::string::ToString; +use crate::block_range::{BoundSide, EntityBlockRange}; use crate::relational::{ Column, ColumnType, Layout, SqlName, Table, BYTE_ARRAY_PREFIX_SIZE, PRIMARY_KEY_COLUMN, STRING_PREFIX_SIZE, @@ -466,7 +469,7 @@ pub fn parse_id(id_type: IdType, json: serde_json::Value) -> Result EntityData { + EntityData { entity, data } + } + pub fn entity_type(&self, schema: &InputSchema) -> EntityType { schema.entity_type(&self.entity).unwrap() } @@ -551,6 +558,48 @@ impl EntityData { } } +#[derive(QueryableByName, Clone, Debug, Default, Eq)] +pub struct EntityDataExt { + #[diesel(sql_type = Text)] + pub entity: String, + #[diesel(sql_type = Jsonb)] + pub data: serde_json::Value, + #[diesel(sql_type = Integer)] + pub block_number: i32, + #[diesel(sql_type = Text)] + pub id: String, + #[diesel(sql_type = BigInt)] + pub vid: i64, +} + +impl Ord for EntityDataExt { + fn cmp(&self, other: &Self) -> Ordering { + let ord = self.block_number.cmp(&other.block_number); + if ord != Ordering::Equal { + ord + } else { + let ord = self.entity.cmp(&other.entity); + if ord != Ordering::Equal { + ord + } else { + self.id.cmp(&other.id) + } + } + } +} + +impl PartialOrd for EntityDataExt { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +impl PartialEq for EntityDataExt { + fn eq(&self, other: &Self) -> bool { + self.cmp(other) == Ordering::Equal + } +} + /// The equivalent of `graph::data::store::Value` but in a form that does /// not require further transformation during `walk_ast`. This form takes /// the idiosyncrasies of how we serialize values into account (e.g., that @@ -1971,6 +2020,104 @@ impl<'a> Query for FindQuery<'a> { impl<'a, Conn> RunQueryDsl for FindQuery<'a> {} +#[derive(Debug, Clone)] +pub struct FindRangeQuery<'a> { + tables: &'a Vec<&'a Table>, + causality_region: CausalityRegion, + bound_side: BoundSide, + imm_range: EntityBlockRange, + mut_range: EntityBlockRange, +} + +impl<'a> FindRangeQuery<'a> { + pub fn new( + tables: &'a Vec<&Table>, + causality_region: CausalityRegion, + bound_side: BoundSide, + block_range: Range, + ) -> Self { + let imm_range = EntityBlockRange::new(true, block_range.clone(), bound_side); + let mut_range = EntityBlockRange::new(false, block_range, bound_side); + Self { + tables, + causality_region, + bound_side, + imm_range, + mut_range, + } + } +} + +impl<'a> QueryFragment for FindRangeQuery<'a> { + fn walk_ast<'b>(&'b self, mut out: AstPass<'_, 'b, Pg>) -> QueryResult<()> { + out.unsafe_to_cache_prepared(); + let mut first = true; + + for table in self.tables.iter() { + // the immutable entities don't have upper range and also can't be modified or deleted + if matches!(self.bound_side, BoundSide::Lower) || !table.immutable { + if first { + first = false; + } else { + out.push_sql("\nunion all\n"); + } + + // Generate + // select '..' as entity, to_jsonb(e.*) as data, {BLOCK_STATEMENT} as block_number + // from schema.table e where ... + // Here the {BLOCK_STATEMENT} is 'block$' for immutable tables and either 'lower(block_range)' + // or 'upper(block_range)' depending on the bound_side variable. + out.push_sql("select "); + out.push_bind_param::(table.object.as_str())?; + out.push_sql(" as entity, to_jsonb(e.*) as data,"); + if table.immutable { + self.imm_range.compare_column(&mut out) + } else { + self.mut_range.compare_column(&mut out) + } + out.push_sql("as block_number, id, vid\n"); + out.push_sql(" from "); + out.push_sql(table.qualified_name.as_str()); + out.push_sql(" e\n where"); + // add casuality region to the query + if table.has_causality_region { + out.push_sql("causality_region = "); + out.push_bind_param::(&self.causality_region)?; + out.push_sql(" and "); + } + if table.immutable { + self.imm_range.contains(&mut out)?; + } else { + self.mut_range.contains(&mut out)?; + } + } + } + + if first { + // In case we have only immutable entities, the upper range will not create any + // select statement. So here we have to generate an SQL statement thet returns + // empty result. + out.push_sql("select 'dummy_entity' as entity, to_jsonb(1) as data, 1 as block_number, 1 as id, 1 as vid where false"); + } else { + out.push_sql("\norder by block_number, entity, id"); + } + + Ok(()) + } +} + +impl<'a> QueryId for FindRangeQuery<'a> { + type QueryId = (); + + const HAS_STATIC_QUERY_ID: bool = false; +} + +impl<'a> Query for FindRangeQuery<'a> { + type SqlType = Untyped; +} + +impl<'a, Conn> RunQueryDsl for FindRangeQuery<'a> {} + /// Builds a query over a given set of [`Table`]s in an attempt to find updated /// and/or newly inserted entities at a given block number; i.e. such that the /// block range's lower bound is equal to said block number. @@ -2266,6 +2413,7 @@ struct InsertRow<'a> { values: Vec>, br_value: BlockRangeValue, causality_region: CausalityRegion, + vid: i64, } impl<'a> InsertRow<'a> { @@ -2302,10 +2450,12 @@ impl<'a> InsertRow<'a> { } let br_value = BlockRangeValue::new(table, row.block, row.end); let causality_region = row.causality_region; + let vid = row.vid; Ok(Self { values, br_value, causality_region, + vid, }) } } @@ -2416,6 +2566,7 @@ impl<'a> QueryFragment for InsertQuery<'a> { out.push_sql(CAUSALITY_REGION_COLUMN); }; + out.push_sql(", vid"); out.push_sql(") values\n"); for (i, row) in self.rows.iter().enumerate() { @@ -2433,6 +2584,8 @@ impl<'a> QueryFragment for InsertQuery<'a> { out.push_sql(", "); out.push_bind_param::(&row.causality_region)?; }; + out.push_sql(", "); + out.push_bind_param::(&row.vid)?; out.push_sql(")"); } @@ -4950,6 +5103,7 @@ impl<'a> QueryFragment for CopyEntityBatchQuery<'a> { out.push_sql(", "); out.push_sql(CAUSALITY_REGION_COLUMN); }; + out.push_sql(", vid"); out.push_sql(")\nselect "); for column in &self.columns { @@ -5015,6 +5169,8 @@ impl<'a> QueryFragment for CopyEntityBatchQuery<'a> { )); } } + out.push_sql(", vid"); + out.push_sql(" from "); out.push_sql(self.src.qualified_name.as_str()); out.push_sql(" where vid >= "); diff --git a/store/postgres/src/subgraph_store.rs b/store/postgres/src/subgraph_store.rs index 41cbef15982..f6544a79e0d 100644 --- a/store/postgres/src/subgraph_store.rs +++ b/store/postgres/src/subgraph_store.rs @@ -44,7 +44,7 @@ use crate::{ index::{IndexList, Method}, Layout, }, - writable::WritableStore, + writable::{SourceableStore, WritableStore}, NotificationSender, }; use crate::{ @@ -268,6 +268,50 @@ impl SubgraphStore { pub fn for_site(&self, site: &Site) -> Result<&Arc, StoreError> { self.inner.for_site(site) } + + async fn get_or_create_writable_store( + self: Arc, + logger: Logger, + deployment: graph::components::store::DeploymentId, + manifest_idx_and_name: Arc>, + ) -> Result, StoreError> { + let deployment = deployment.into(); + // We cache writables to make sure calls to this method are + // idempotent and there is ever only one `WritableStore` for any + // deployment + if let Some(writable) = self.writables.lock().unwrap().get(&deployment) { + // A poisoned writable will not write anything anymore; we + // discard it and create a new one that is properly initialized + // according to the state in the database. + if !writable.poisoned() { + return Ok(writable.cheap_clone()); + } + } + + // Ideally the lower level functions would be asyncified. + let this = self.clone(); + let site = graph::spawn_blocking_allow_panic(move || -> Result<_, StoreError> { + this.find_site(deployment) + }) + .await + .unwrap()?; // Propagate panics, there shouldn't be any. + + let writable = Arc::new( + WritableStore::new( + self.as_ref().clone(), + logger, + site, + manifest_idx_and_name, + self.registry.clone(), + ) + .await?, + ); + self.writables + .lock() + .unwrap() + .insert(deployment, writable.cheap_clone()); + Ok(writable) + } } impl std::ops::Deref for SubgraphStore { @@ -1488,42 +1532,25 @@ impl SubgraphStoreTrait for SubgraphStore { deployment: graph::components::store::DeploymentId, manifest_idx_and_name: Arc>, ) -> Result, StoreError> { - let deployment = deployment.into(); - // We cache writables to make sure calls to this method are - // idempotent and there is ever only one `WritableStore` for any - // deployment - if let Some(writable) = self.writables.lock().unwrap().get(&deployment) { - // A poisoned writable will not write anything anymore; we - // discard it and create a new one that is properly initialized - // according to the state in the database. - if !writable.poisoned() { - return Ok(writable.cheap_clone()); - } - } + self.get_or_create_writable_store(logger, deployment, manifest_idx_and_name) + .await + .map(|store| store as Arc) + } - // Ideally the lower level functions would be asyncified. - let this = self.clone(); - let site = graph::spawn_blocking_allow_panic(move || -> Result<_, StoreError> { - this.find_site(deployment) - }) - .await - .unwrap()?; // Propagate panics, there shouldn't be any. + async fn sourceable( + self: Arc, + deployment: graph::components::store::DeploymentId, + ) -> Result, StoreError> { + let deployment = deployment.into(); + let site = self.find_site(deployment)?; + let store = self.for_site(&site)?; + let input_schema = self.input_schema(&site.deployment)?; - let writable = Arc::new( - WritableStore::new( - self.as_ref().clone(), - logger, - site, - manifest_idx_and_name, - self.registry.clone(), - ) - .await?, - ); - self.writables - .lock() - .unwrap() - .insert(deployment, writable.cheap_clone()); - Ok(writable) + Ok(Arc::new(SourceableStore::new( + site, + store.clone(), + input_schema, + ))) } async fn stop_subgraph(&self, loc: &DeploymentLocator) -> Result<(), StoreError> { diff --git a/store/postgres/src/writable.rs b/store/postgres/src/writable.rs index ee7a5e4754f..2efdcce399a 100644 --- a/store/postgres/src/writable.rs +++ b/store/postgres/src/writable.rs @@ -1,11 +1,12 @@ use std::collections::BTreeSet; -use std::ops::Deref; +use std::ops::{Deref, Range}; use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::{Mutex, RwLock, TryLockError as RwLockError}; use std::time::Instant; use std::{collections::BTreeMap, sync::Arc}; -use graph::blockchain::block_stream::FirehoseCursor; +use async_trait::async_trait; +use graph::blockchain::block_stream::{EntitySourceOperation, FirehoseCursor}; use graph::blockchain::BlockTime; use graph::components::store::{Batch, DeploymentCursorTracker, DerivedEntityQuery, ReadStore}; use graph::constraint_violation; @@ -1569,6 +1570,47 @@ impl ReadStore for WritableStore { } } +pub struct SourceableStore { + site: Arc, + store: Arc, + input_schema: InputSchema, +} + +impl SourceableStore { + pub fn new(site: Arc, store: Arc, input_schema: InputSchema) -> Self { + Self { + site, + store, + input_schema, + } + } +} + +#[async_trait] +impl store::SourceableStore for SourceableStore { + fn get_range( + &self, + entity_types: Vec, + causality_region: CausalityRegion, + block_range: Range, + ) -> Result>, StoreError> { + self.store.get_range( + self.site.clone(), + entity_types, + causality_region, + block_range, + ) + } + + fn input_schema(&self) -> InputSchema { + self.input_schema.cheap_clone() + } + + async fn block_ptr(&self) -> Result, StoreError> { + self.store.block_ptr(self.site.cheap_clone()).await + } +} + impl DeploymentCursorTracker for WritableStore { fn block_ptr(&self) -> Option { self.block_ptr.lock().unwrap().clone() diff --git a/store/test-store/src/store.rs b/store/test-store/src/store.rs index 2921d375286..30017db3d21 100644 --- a/store/test-store/src/store.rs +++ b/store/test-store/src/store.rs @@ -6,6 +6,7 @@ use graph::components::store::BlockStore; use graph::data::graphql::load_manager::LoadManager; use graph::data::query::QueryResults; use graph::data::query::QueryTarget; +use graph::data::store::EntityV; use graph::data::subgraph::schema::{DeploymentCreate, SubgraphError}; use graph::data::subgraph::SubgraphFeature; use graph::data_source::DataSource; @@ -422,12 +423,13 @@ pub async fn insert_entities( deployment: &DeploymentLocator, entities: Vec<(EntityType, Entity)>, ) -> Result<(), StoreError> { - let insert_ops = entities - .into_iter() - .map(|(entity_type, data)| EntityOperation::Set { + let insert_ops = entities.into_iter().map(|(entity_type, data)| { + let vid = data.vid_or_default(); + EntityOperation::Set { key: entity_type.key(data.id()), - data, - }); + data: EntityV::new(data, vid), + } + }); transact_entity_operations( &SUBGRAPH_STORE, diff --git a/store/test-store/tests/chain/ethereum/manifest.rs b/store/test-store/tests/chain/ethereum/manifest.rs index 9089ec4f572..23fdfae7c57 100644 --- a/store/test-store/tests/chain/ethereum/manifest.rs +++ b/store/test-store/tests/chain/ethereum/manifest.rs @@ -11,10 +11,10 @@ use graph::data::store::Value; use graph::data::subgraph::schema::SubgraphError; use graph::data::subgraph::{ Prune, LATEST_VERSION, SPEC_VERSION_0_0_4, SPEC_VERSION_0_0_7, SPEC_VERSION_0_0_8, - SPEC_VERSION_0_0_9, SPEC_VERSION_1_0_0, SPEC_VERSION_1_2_0, + SPEC_VERSION_0_0_9, SPEC_VERSION_1_0_0, SPEC_VERSION_1_2_0, SPEC_VERSION_1_3_0, }; use graph::data_source::offchain::OffchainDataSourceKind; -use graph::data_source::DataSourceTemplate; +use graph::data_source::{DataSourceEnum, DataSourceTemplate}; use graph::entity; use graph::env::ENV_VARS; use graph::prelude::web3::types::H256; @@ -166,10 +166,54 @@ specVersion: 0.0.7 let data_source = match &manifest.templates[0] { DataSourceTemplate::Offchain(ds) => ds, DataSourceTemplate::Onchain(_) => unreachable!(), + DataSourceTemplate::Subgraph(_) => unreachable!(), }; assert_eq!(data_source.kind, OffchainDataSourceKind::Ipfs); } +#[tokio::test] +async fn subgraph_ds_manifest() { + let yaml = " +schema: + file: + /: /ipfs/Qmschema +dataSources: + - name: SubgraphSource + kind: subgraph + entities: + - Gravatar + network: mainnet + source: + address: 'QmSWWT2yrTFDZSL8tRyoHEVrcEKAUsY2hj2TMQDfdDZU8h' + startBlock: 9562480 + mapping: + apiVersion: 0.0.6 + language: wasm/assemblyscript + entities: + - TestEntity + file: + /: /ipfs/Qmmapping + handlers: + - handler: handleEntity + entity: User +specVersion: 1.3.0 +"; + + let manifest = resolve_manifest(yaml, SPEC_VERSION_1_3_0).await; + + assert_eq!("Qmmanifest", manifest.id.as_str()); + assert_eq!(manifest.data_sources.len(), 1); + let data_source = &manifest.data_sources[0]; + match data_source { + DataSourceEnum::Subgraph(ds) => { + assert_eq!(ds.name, "SubgraphSource"); + assert_eq!(ds.kind, "subgraph"); + assert_eq!(ds.source.start_block, 9562480); + } + _ => panic!("Expected a subgraph data source"), + } +} + #[tokio::test] async fn graft_manifest() { const YAML: &str = " @@ -372,7 +416,7 @@ specVersion: 0.0.2 msg ); - let thing = entity! { schema => id: "datthing" }; + let thing = entity! { schema => id: "datthing", vid: 1i64 }; test_store::insert_entities( &deployment, vec![(schema.entity_type("Thing").unwrap(), thing)], @@ -1445,3 +1489,74 @@ dataSources: assert_eq!(4, decls.len()); }); } + +#[test] +fn parses_eth_call_decls_for_subgraph_datasource() { + const YAML: &str = " +specVersion: 1.3.0 +schema: + file: + /: /ipfs/Qmschema +features: + - ipfsOnEthereumContracts +dataSources: + - kind: subgraph + name: Factory + entities: + - Gravatar + network: mainnet + source: + address: 'QmSWWT2yrTFDZSL8tRyoHEVrcEKAUsY2hj2TMQDfdDZU8h' + startBlock: 9562480 + mapping: + apiVersion: 0.0.6 + language: wasm/assemblyscript + entities: + - TestEntity + file: + /: /ipfs/Qmmapping + abis: + - name: Factory + file: + /: /ipfs/Qmabi + handlers: + - handler: handleEntity + entity: User + calls: + fake1: Factory[entity.address].get(entity.user) + fake3: Factory[0xDeaDbeefdEAdbeefdEadbEEFdeadbeEFdEaDbeeF].get(entity.address) + fake4: Factory[0xDeaDbeefdEAdbeefdEadbEEFdeadbeEFdEaDbeeF].get(0xDeaDbeefdEAdbeefdEadbEEFdeadbeEFdEaDbeeF) +"; + + test_store::run_test_sequentially(|store| async move { + let store = store.subgraph_store(); + let unvalidated: UnvalidatedSubgraphManifest = { + let mut resolver = TextResolver::default(); + let id = DeploymentHash::new("Qmmanifest").unwrap(); + resolver.add(id.as_str(), &YAML); + resolver.add("/ipfs/Qmabi", &ABI); + resolver.add("/ipfs/Qmschema", &GQL_SCHEMA); + resolver.add("/ipfs/Qmmapping", &MAPPING_WITH_IPFS_FUNC_WASM); + + let resolver: Arc = Arc::new(resolver); + + let raw = serde_yaml::from_str(YAML).unwrap(); + UnvalidatedSubgraphManifest::resolve( + id, + raw, + &resolver, + &LOGGER, + SPEC_VERSION_1_3_0.clone(), + ) + .await + .expect("Parsing simple manifest works") + }; + + let manifest = unvalidated.validate(store.clone(), true).await.unwrap(); + let ds = &manifest.data_sources[0].as_subgraph().unwrap(); + // For more detailed tests of parsing CallDecls see the data_soure + // module in chain/ethereum + let decls = &ds.mapping.handlers[0].calls.decls; + assert_eq!(3, decls.len()); + }); +} diff --git a/store/test-store/tests/core/interfaces.rs b/store/test-store/tests/core/interfaces.rs index 78eb2fda390..3b8ca1724c2 100644 --- a/store/test-store/tests/core/interfaces.rs +++ b/store/test-store/tests/core/interfaces.rs @@ -201,9 +201,9 @@ async fn reference_interface_derived() { let query = "query { events { id transaction { id } } }"; - let buy = ("BuyEvent", entity! { schema => id: "buy" }); - let sell1 = ("SellEvent", entity! { schema => id: "sell1" }); - let sell2 = ("SellEvent", entity! { schema => id: "sell2" }); + let buy = ("BuyEvent", entity! { schema => id: "buy", vid: 0i64 }); + let sell1 = ("SellEvent", entity! { schema => id: "sell1", vid: 0i64 }); + let sell2 = ("SellEvent", entity! { schema => id: "sell2", vid: 1i64 }); let gift = ( "GiftEvent", entity! { schema => id: "gift", transaction: "txn" }, @@ -278,11 +278,11 @@ async fn follow_interface_reference() { let parent = ( "Animal", - entity! { schema => id: "parent", legs: 4, parent: Value::Null }, + entity! { schema => id: "parent", legs: 4, parent: Value::Null, vid: 0i64}, ); let child = ( "Animal", - entity! { schema => id: "child", legs: 3, parent: "parent" }, + entity! { schema => id: "child", legs: 3, parent: "parent" , vid: 1i64}, ); let res = insert_and_query(subgraph_id, document, vec![parent, child], query) @@ -459,16 +459,16 @@ async fn interface_inline_fragment_with_subquery() { "; let schema = InputSchema::raw(document, subgraph_id); - let mama_cow = ("Parent", entity! { schema => id: "mama_cow" }); + let mama_cow = ("Parent", entity! { schema => id: "mama_cow", vid: 0i64 }); let cow = ( "Animal", - entity! { schema => id: "1", name: "cow", legs: 4, parent: "mama_cow" }, + entity! { schema => id: "1", name: "cow", legs: 4, parent: "mama_cow", vid: 0i64 }, ); - let mama_bird = ("Parent", entity! { schema => id: "mama_bird" }); + let mama_bird = ("Parent", entity! { schema => id: "mama_bird", vid: 1i64 }); let bird = ( "Bird", - entity! { schema => id: "2", airspeed: 5, legs: 2, parent: "mama_bird" }, + entity! { schema => id: "2", airspeed: 5, legs: 2, parent: "mama_bird", vid: 1i64 }, ); let query = "query { leggeds(orderBy: legs) { legs ... on Bird { airspeed parent { id } } } }"; @@ -545,11 +545,11 @@ async fn alias() { let parent = ( "Animal", - entity! { schema => id: "parent", legs: 4, parent: Value::Null }, + entity! { schema => id: "parent", legs: 4, parent: Value::Null, vid: 0i64 }, ); let child = ( "Animal", - entity! { schema => id: "child", legs: 3, parent: "parent" }, + entity! { schema => id: "child", legs: 3, parent: "parent", vid: 1i64 }, ); let res = insert_and_query(subgraph_id, document, vec![parent, child], query) @@ -608,9 +608,15 @@ async fn fragments_dont_panic() { "; // The panic manifests if two parents exist. - let parent = ("Parent", entity! { schema => id: "p", child: "c" }); - let parent2 = ("Parent", entity! { schema => id: "p2", child: Value::Null }); - let child = ("Child", entity! { schema => id:"c" }); + let parent = ( + "Parent", + entity! { schema => id: "p", child: "c", vid: 0i64 }, + ); + let parent2 = ( + "Parent", + entity! { schema => id: "p2", child: Value::Null, vid: 1i64 }, + ); + let child = ("Child", entity! { schema => id:"c", vid: 2i64 }); let res = insert_and_query(subgraph_id, document, vec![parent, parent2, child], query) .await @@ -668,12 +674,15 @@ async fn fragments_dont_duplicate_data() { "; // This bug manifests if two parents exist. - let parent = ("Parent", entity! { schema => id: "p", children: vec!["c"] }); + let parent = ( + "Parent", + entity! { schema => id: "p", children: vec!["c"], vid: 0i64 }, + ); let parent2 = ( "Parent", - entity! { schema => id: "b", children: Vec::::new() }, + entity! { schema => id: "b", children: Vec::::new(), vid: 1i64 }, ); - let child = ("Child", entity! { schema => id:"c" }); + let child = ("Child", entity! { schema => id:"c", vid: 2i64 }); let res = insert_and_query(subgraph_id, document, vec![parent, parent2, child], query) .await @@ -721,11 +730,11 @@ async fn redundant_fields() { let parent = ( "Animal", - entity! { schema => id: "parent", parent: Value::Null }, + entity! { schema => id: "parent", parent: Value::Null, vid: 0i64 }, ); let child = ( "Animal", - entity! { schema => id: "child", parent: "parent" }, + entity! { schema => id: "child", parent: "parent", vid: 1i64 }, ); let res = insert_and_query(subgraph_id, document, vec![parent, child], query) @@ -783,8 +792,11 @@ async fn fragments_merge_selections() { } "; - let parent = ("Parent", entity! { schema => id: "p", children: vec!["c"] }); - let child = ("Child", entity! { schema => id: "c", foo: 1 }); + let parent = ( + "Parent", + entity! { schema => id: "p", children: vec!["c"], vid: 0i64 }, + ); + let child = ("Child", entity! { schema => id: "c", foo: 1, vid: 1i64 }); let res = insert_and_query(subgraph_id, document, vec![parent, child], query) .await @@ -1081,11 +1093,11 @@ async fn enums() { let entities = vec![ ( "Trajectory", - entity! { schema => id: "1", direction: "EAST", meters: 10 }, + entity! { schema => id: "1", direction: "EAST", meters: 10, vid: 0i64 }, ), ( "Trajectory", - entity! { schema => id: "2", direction: "NORTH", meters: 15 }, + entity! { schema => id: "2", direction: "NORTH", meters: 15, vid: 1i64 }, ), ]; let query = "query { trajectories { id, direction, meters } }"; @@ -1134,15 +1146,15 @@ async fn enum_list_filters() { let entities = vec![ ( "Trajectory", - entity! { schema => id: "1", direction: "EAST", meters: 10 }, + entity! { schema => id: "1", direction: "EAST", meters: 10, vid: 0i64 }, ), ( "Trajectory", - entity! { schema => id: "2", direction: "NORTH", meters: 15 }, + entity! { schema => id: "2", direction: "NORTH", meters: 15, vid: 1i64 }, ), ( "Trajectory", - entity! { schema => id: "3", direction: "WEST", meters: 20 }, + entity! { schema => id: "3", direction: "WEST", meters: 20, vid: 2i64 }, ), ]; @@ -1327,8 +1339,8 @@ async fn derived_interface_bytes() { let entities = vec![ ("Pool", entity! { schema => id: b("0xf001") }), - ("Sell", entity! { schema => id: b("0xc0"), pool: "0xf001"}), - ("Buy", entity! { schema => id: b("0xb0"), pool: "0xf001"}), + ("Sell", entity! { schema => id: b("0xc0"), pool: "0xf001" }), + ("Buy", entity! { schema => id: b("0xb0"), pool: "0xf001" }), ]; let res = insert_and_query(subgraph_id, document, entities, query) diff --git a/store/test-store/tests/graph/entity_cache.rs b/store/test-store/tests/graph/entity_cache.rs index d7ebb30785c..1df4b9adac9 100644 --- a/store/test-store/tests/graph/entity_cache.rs +++ b/store/test-store/tests/graph/entity_cache.rs @@ -4,7 +4,7 @@ use graph::components::store::{ DeploymentCursorTracker, DerivedEntityQuery, GetScope, LoadRelatedRequest, ReadStore, StoredDynamicDataSource, WritableStore, }; -use graph::data::store::Id; +use graph::data::store::{EntityV, Id}; use graph::data::subgraph::schema::{DeploymentCreate, SubgraphError, SubgraphHealth}; use graph::data_source::CausalityRegion; use graph::schema::{EntityKey, EntityType, InputSchema}; @@ -209,20 +209,22 @@ fn insert_modifications() { let mogwai_data = entity! { SCHEMA => id: "mogwai", name: "Mogwai" }; let mogwai_key = make_band_key("mogwai"); - cache.set(mogwai_key.clone(), mogwai_data.clone()).unwrap(); + cache + .set(mogwai_key.clone(), mogwai_data.clone(), 0) + .unwrap(); let sigurros_data = entity! { SCHEMA => id: "sigurros", name: "Sigur Ros" }; let sigurros_key = make_band_key("sigurros"); cache - .set(sigurros_key.clone(), sigurros_data.clone()) + .set(sigurros_key.clone(), sigurros_data.clone(), 0) .unwrap(); let result = cache.as_modifications(0); assert_eq!( sort_by_entity_key(result.unwrap().modifications), sort_by_entity_key(vec![ - EntityModification::insert(mogwai_key, mogwai_data, 0), - EntityModification::insert(sigurros_key, sigurros_data, 0) + EntityModification::insert(mogwai_key, mogwai_data, 0, 0), + EntityModification::insert(sigurros_key, sigurros_data, 0, 1) ]) ); } @@ -253,20 +255,22 @@ fn overwrite_modifications() { let mogwai_data = entity! { SCHEMA => id: "mogwai", name: "Mogwai", founded: 1995 }; let mogwai_key = make_band_key("mogwai"); - cache.set(mogwai_key.clone(), mogwai_data.clone()).unwrap(); + cache + .set(mogwai_key.clone(), mogwai_data.clone(), 0) + .unwrap(); let sigurros_data = entity! { SCHEMA => id: "sigurros", name: "Sigur Ros", founded: 1994 }; let sigurros_key = make_band_key("sigurros"); cache - .set(sigurros_key.clone(), sigurros_data.clone()) + .set(sigurros_key.clone(), sigurros_data.clone(), 0) .unwrap(); let result = cache.as_modifications(0); assert_eq!( sort_by_entity_key(result.unwrap().modifications), sort_by_entity_key(vec![ - EntityModification::overwrite(mogwai_key, mogwai_data, 0), - EntityModification::overwrite(sigurros_key, sigurros_data, 0) + EntityModification::overwrite(mogwai_key, mogwai_data, 0, 0), + EntityModification::overwrite(sigurros_key, sigurros_data, 0, 1) ]) ); } @@ -289,12 +293,12 @@ fn consecutive_modifications() { let update_data = entity! { SCHEMA => id: "mogwai", founded: 1995, label: "Rock Action Records" }; let update_key = make_band_key("mogwai"); - cache.set(update_key, update_data).unwrap(); + cache.set(update_key, update_data, 0).unwrap(); // Then, just reset the "label". let update_data = entity! { SCHEMA => id: "mogwai", label: Value::Null }; let update_key = make_band_key("mogwai"); - cache.set(update_key.clone(), update_data).unwrap(); + cache.set(update_key.clone(), update_data, 0).unwrap(); // We expect a single overwrite modification for the above that leaves "id" // and "name" untouched, sets "founded" and removes the "label" field. @@ -304,6 +308,7 @@ fn consecutive_modifications() { sort_by_entity_key(vec![EntityModification::overwrite( update_key, entity! { SCHEMA => id: "mogwai", name: "Mogwai", founded: 1995 }, + 0, 0 )]) ); @@ -428,17 +433,17 @@ async fn insert_test_data(store: Arc) -> DeploymentLocator .unwrap(); // 1 account 3 wallets - let test_entity_1 = create_account_entity("1", "Johnton", "tonofjohn@email.com", 67_i32); + let test_entity_1 = create_account_entity("1", "Johnton", "tonofjohn@email.com", 67_i32, 1); let id_one = WALLET_TYPE.parse_id("1").unwrap(); - let wallet_entity_1 = create_wallet_operation("1", &id_one, 67_i32); - let wallet_entity_2 = create_wallet_operation("2", &id_one, 92_i32); - let wallet_entity_3 = create_wallet_operation("3", &id_one, 192_i32); + let wallet_entity_1 = create_wallet_operation("1", &id_one, 67_i32, 1); + let wallet_entity_2 = create_wallet_operation("2", &id_one, 92_i32, 2); + let wallet_entity_3 = create_wallet_operation("3", &id_one, 192_i32, 3); // 1 account 1 wallet - let test_entity_2 = create_account_entity("2", "Cindini", "dinici@email.com", 42_i32); + let test_entity_2 = create_account_entity("2", "Cindini", "dinici@email.com", 42_i32, 2); let id_two = WALLET_TYPE.parse_id("2").unwrap(); - let wallet_entity_4 = create_wallet_operation("4", &id_two, 32_i32); + let wallet_entity_4 = create_wallet_operation("4", &id_two, 32_i32, 4); // 1 account 0 wallets - let test_entity_3 = create_account_entity("3", "Shaqueeena", "queensha@email.com", 28_i32); + let test_entity_3 = create_account_entity("3", "Shaqueeena", "queensha@email.com", 28_i32, 3); transact_entity_operations( &store, &deployment, @@ -458,13 +463,13 @@ async fn insert_test_data(store: Arc) -> DeploymentLocator deployment } -fn create_account_entity(id: &str, name: &str, email: &str, age: i32) -> EntityOperation { +fn create_account_entity(id: &str, name: &str, email: &str, age: i32, vid: i64) -> EntityOperation { let test_entity = entity! { LOAD_RELATED_SUBGRAPH => id: id, name: name, email: email, age: age }; EntityOperation::Set { key: ACCOUNT_TYPE.parse_key(id).unwrap(), - data: test_entity, + data: EntityV::new(test_entity, vid), } } @@ -472,8 +477,8 @@ fn create_wallet_entity(id: &str, account_id: &Id, balance: i32) -> Entity { let account_id = Value::from(account_id.clone()); entity! { LOAD_RELATED_SUBGRAPH => id: id, account: account_id, balance: balance } } -fn create_wallet_operation(id: &str, account_id: &Id, balance: i32) -> EntityOperation { - let test_wallet = create_wallet_entity(id, account_id, balance); +fn create_wallet_operation(id: &str, account_id: &Id, balance: i32, vid: i64) -> EntityOperation { + let test_wallet = EntityV::new(create_wallet_entity(id, account_id, balance), vid); EntityOperation::Set { key: WALLET_TYPE.parse_key(id).unwrap(), data: test_wallet, @@ -577,8 +582,8 @@ fn check_for_insert_async_store() { run_store_test(|mut cache, store, deployment, _writable| async move { let account_id = ACCOUNT_TYPE.parse_id("2").unwrap(); // insert a new wallet - let wallet_entity_5 = create_wallet_operation("5", &account_id, 79_i32); - let wallet_entity_6 = create_wallet_operation("6", &account_id, 200_i32); + let wallet_entity_5 = create_wallet_operation("5", &account_id, 79_i32, 12); + let wallet_entity_6 = create_wallet_operation("6", &account_id, 200_i32, 13); transact_entity_operations( &store, @@ -608,8 +613,8 @@ fn check_for_insert_async_not_related() { run_store_test(|mut cache, store, deployment, _writable| async move { let account_id = ACCOUNT_TYPE.parse_id("2").unwrap(); // insert a new wallet - let wallet_entity_5 = create_wallet_operation("5", &account_id, 79_i32); - let wallet_entity_6 = create_wallet_operation("6", &account_id, 200_i32); + let wallet_entity_5 = create_wallet_operation("5", &account_id, 79_i32, 5); + let wallet_entity_6 = create_wallet_operation("6", &account_id, 200_i32, 6); transact_entity_operations( &store, @@ -641,13 +646,13 @@ fn check_for_update_async_related() { run_store_test(|mut cache, store, deployment, writable| async move { let entity_key = WALLET_TYPE.parse_key("1").unwrap(); let account_id = entity_key.entity_id.clone(); - let wallet_entity_update = create_wallet_operation("1", &account_id, 79_i32); + let wallet_entity_update = create_wallet_operation("1", &account_id, 79_i32, 11); let new_data = match wallet_entity_update { EntityOperation::Set { ref data, .. } => data.clone(), _ => unreachable!(), }; - assert_ne!(writable.get(&entity_key).unwrap().unwrap(), new_data); + assert_ne!(writable.get(&entity_key).unwrap().unwrap(), new_data.e); // insert a new wallet transact_entity_operations( &store, @@ -667,7 +672,7 @@ fn check_for_update_async_related() { let result = cache.load_related(&request).unwrap(); let wallet_2 = create_wallet_entity("2", &account_id, 92_i32); let wallet_3 = create_wallet_entity("3", &account_id, 192_i32); - let expeted_vec = vec![new_data, wallet_2, wallet_3]; + let expeted_vec = vec![new_data.e, wallet_2, wallet_3]; assert_eq!(result, expeted_vec); }); @@ -702,7 +707,6 @@ fn check_for_delete_async_related() { assert_eq!(result, expeted_vec); }); } - #[test] fn scoped_get() { run_store_test(|mut cache, _store, _deployment, _writable| async move { @@ -715,7 +719,7 @@ fn scoped_get() { let account5 = ACCOUNT_TYPE.parse_id("5").unwrap(); let wallet5 = create_wallet_entity("5", &account5, 100); let key5 = WALLET_TYPE.parse_key("5").unwrap(); - cache.set(key5.clone(), wallet5.clone()).unwrap(); + cache.set(key5.clone(), wallet5.clone(), 0).unwrap(); // For the new entity, we can retrieve it with either scope let act5 = cache.get(&key5, GetScope::InBlock).unwrap(); @@ -736,7 +740,7 @@ fn scoped_get() { // But if it gets updated, it becomes visible with either scope let mut wallet1 = wallet1; wallet1.set("balance", 70).unwrap(); - cache.set(key1.clone(), wallet1.clone()).unwrap(); + cache.set(key1.clone(), wallet1.clone(), 0).unwrap(); let act1 = cache.get(&key1, GetScope::InBlock).unwrap(); assert_eq!(Some(&wallet1), act1.as_ref().map(|e| e.as_ref())); let act1 = cache.get(&key1, GetScope::Store).unwrap(); @@ -783,6 +787,6 @@ fn no_interface_mods() { let entity = entity! { LOAD_RELATED_SUBGRAPH => id: "1", balance: 100 }; - cache.set(key, entity).unwrap_err(); + cache.set(key, entity, 0).unwrap_err(); }) } diff --git a/store/test-store/tests/graphql/query.rs b/store/test-store/tests/graphql/query.rs index 08ad26ef9b9..5358249040d 100644 --- a/store/test-store/tests/graphql/query.rs +++ b/store/test-store/tests/graphql/query.rs @@ -1,6 +1,7 @@ use graph::blockchain::{Block, BlockTime}; use graph::data::query::Trace; use graph::data::store::scalar::Timestamp; +use graph::data::store::EntityV; use graph::data::subgraph::schema::DeploymentCreate; use graph::data::subgraph::LATEST_VERSION; use graph::entity; @@ -424,9 +425,12 @@ async fn insert_test_entities( .into_iter() .map(|(typename, entities)| { let entity_type = schema.entity_type(typename).unwrap(); - entities.into_iter().map(move |data| EntityOperation::Set { - key: entity_type.key(data.id()), - data, + entities.into_iter().map(move |data| { + let vid = data.vid_or_default(); + EntityOperation::Set { + key: entity_type.key(data.id()), + data: EntityV::new(data, vid), + } }) }) .flatten() @@ -468,115 +472,118 @@ async fn insert_test_entities( ( "Musician", vec![ - entity! { is => id: "m1", name: "John", mainBand: "b1", bands: vec!["b1", "b2"], favoriteCount: 10, birthDate: timestamp.clone() }, - entity! { is => id: "m2", name: "Lisa", mainBand: "b1", bands: vec!["b1"], favoriteCount: 100, birthDate: timestamp.clone() }, + entity! { is => id: "m1", name: "John", mainBand: "b1", bands: vec!["b1", "b2"], favoriteCount: 10, birthDate: timestamp.clone(), vid: 0i64 }, + entity! { is => id: "m2", name: "Lisa", mainBand: "b1", bands: vec!["b1"], favoriteCount: 100, birthDate: timestamp.clone(), vid: 1i64 }, ], ), - ("Publisher", vec![entity! { is => id: pub1 }]), + ("Publisher", vec![entity! { is => id: pub1, vid: 0i64 }]), ( "Band", vec![ - entity! { is => id: "b1", name: "The Musicians", originalSongs: vec![s[1], s[2]] }, - entity! { is => id: "b2", name: "The Amateurs", originalSongs: vec![s[1], s[3], s[4]] }, + entity! { is => id: "b1", name: "The Musicians", originalSongs: vec![s[1], s[2]], vid: 0i64 }, + entity! { is => id: "b2", name: "The Amateurs", originalSongs: vec![s[1], s[3], s[4]], vid: 1i64 }, ], ), ( "Song", vec![ - entity! { is => id: s[1], sid: "s1", title: "Cheesy Tune", publisher: pub1, writtenBy: "m1", media: vec![md[1], md[2]] }, - entity! { is => id: s[2], sid: "s2", title: "Rock Tune", publisher: pub1, writtenBy: "m2", media: vec![md[3], md[4]] }, - entity! { is => id: s[3], sid: "s3", title: "Pop Tune", publisher: pub1, writtenBy: "m1", media: vec![md[5]] }, - entity! { is => id: s[4], sid: "s4", title: "Folk Tune", publisher: pub1, writtenBy: "m3", media: vec![md[6]] }, + entity! { is => id: s[1], sid: "s1", title: "Cheesy Tune", publisher: pub1, writtenBy: "m1", media: vec![md[1], md[2]], vid: 0i64 }, + entity! { is => id: s[2], sid: "s2", title: "Rock Tune", publisher: pub1, writtenBy: "m2", media: vec![md[3], md[4]], vid: 1i64 }, + entity! { is => id: s[3], sid: "s3", title: "Pop Tune", publisher: pub1, writtenBy: "m1", media: vec![md[5]], vid: 2i64 }, + entity! { is => id: s[4], sid: "s4", title: "Folk Tune", publisher: pub1, writtenBy: "m3", media: vec![md[6]], vid: 3i64 }, ], ), ( "User", vec![ - entity! { is => id: "u1", name: "User 1", latestSongReview: "r3", latestBandReview: "r1", latestReview: "r3" }, + entity! { is => id: "u1", name: "User 1", latestSongReview: "r3", latestBandReview: "r1", latestReview: "r3", vid: 0i64 }, ], ), ( "SongStat", vec![ - entity! { is => id: s[1], played: 10 }, - entity! { is => id: s[2], played: 15 }, + entity! { is => id: s[1], played: 10, vid: 0i64 }, + entity! { is => id: s[2], played: 15, vid: 1i64 }, ], ), ( "BandReview", vec![ - entity! { is => id: "r1", body: "Bad musicians", band: "b1", author: "u1" }, - entity! { is => id: "r2", body: "Good amateurs", band: "b2", author: "u2" }, - entity! { is => id: "r5", body: "Very Bad musicians", band: "b1", author: "u3" }, + entity! { is => id: "r1", body: "Bad musicians", band: "b1", author: "u1", vid: 0i64 }, + entity! { is => id: "r2", body: "Good amateurs", band: "b2", author: "u2", vid: 1i64 }, + entity! { is => id: "r5", body: "Very Bad musicians", band: "b1", author: "u3", vid: 2i64 }, ], ), ( "SongReview", vec![ - entity! { is => id: "r3", body: "Bad", song: s[2], author: "u1" }, - entity! { is => id: "r4", body: "Good", song: s[3], author: "u2" }, - entity! { is => id: "r6", body: "Very Bad", song: s[2], author: "u3" }, + entity! { is => id: "r3", body: "Bad", song: s[2], author: "u1", vid: 0i64 }, + entity! { is => id: "r4", body: "Good", song: s[3], author: "u2", vid: 1i64 }, + entity! { is => id: "r6", body: "Very Bad", song: s[2], author: "u3", vid: 2i64 }, ], ), ( "User", vec![ - entity! { is => id: "u1", name: "Baden", latestSongReview: "r3", latestBandReview: "r1", latestReview: "r1" }, - entity! { is => id: "u2", name: "Goodwill", latestSongReview: "r4", latestBandReview: "r2", latestReview: "r2" }, + entity! { is => id: "u1", name: "Baden", latestSongReview: "r3", latestBandReview: "r1", latestReview: "r1", vid: 0i64 }, + entity! { is => id: "u2", name: "Goodwill", latestSongReview: "r4", latestBandReview: "r2", latestReview: "r2", vid: 1i64 }, ], ), ( "AnonymousUser", vec![ - entity! { is => id: "u3", name: "Anonymous 3", latestSongReview: "r6", latestBandReview: "r5", latestReview: "r5" }, + entity! { is => id: "u3", name: "Anonymous 3", latestSongReview: "r6", latestBandReview: "r5", latestReview: "r5", vid: 0i64 }, ], ), ( "Photo", vec![ - entity! { is => id: md[1], title: "Cheesy Tune Single Cover", author: "u1" }, - entity! { is => id: md[3], title: "Rock Tune Single Cover", author: "u1" }, - entity! { is => id: md[5], title: "Pop Tune Single Cover", author: "u1" }, + entity! { is => id: md[1], title: "Cheesy Tune Single Cover", author: "u1", vid: 0i64 }, + entity! { is => id: md[3], title: "Rock Tune Single Cover", author: "u1", vid: 1i64 }, + entity! { is => id: md[5], title: "Pop Tune Single Cover", author: "u1", vid: 2i64 }, ], ), ( "Video", vec![ - entity! { is => id: md[2], title: "Cheesy Tune Music Video", author: "u2" }, - entity! { is => id: md[4], title: "Rock Tune Music Video", author: "u2" }, - entity! { is => id: md[6], title: "Folk Tune Music Video", author: "u2" }, + entity! { is => id: md[2], title: "Cheesy Tune Music Video", author: "u2", vid: 0i64 }, + entity! { is => id: md[4], title: "Rock Tune Music Video", author: "u2", vid: 1i64 }, + entity! { is => id: md[6], title: "Folk Tune Music Video", author: "u2", vid: 2i64 }, ], ), ( "Album", - vec![entity! { is => id: "rl1", title: "Pop and Folk", songs: vec![s[3], s[4]] }], + vec![ + entity! { is => id: "rl1", title: "Pop and Folk", songs: vec![s[3], s[4]], vid: 0i64 }, + ], ), ( "Single", vec![ - entity! { is => id: "rl2", title: "Rock", songs: vec![s[2]] }, - entity! { is => id: "rl3", title: "Cheesy", songs: vec![s[1]] }, - entity! { is => id: "rl4", title: "Silence", songs: Vec::::new() }, + entity! { is => id: "rl2", title: "Rock", songs: vec![s[2]], vid: 0i64 }, + entity! { is => id: "rl3", title: "Cheesy", songs: vec![s[1]], vid: 1i64 }, + entity! { is => id: "rl4", title: "Silence", songs: Vec::::new(), vid: 2i64 }, ], ), ( "Plays", vec![ - entity! { is => id: 1i64, timestamp: ts0, song: s[1], user: "u1"}, - entity! { is => id: 2i64, timestamp: ts0, song: s[1], user: "u2"}, - entity! { is => id: 3i64, timestamp: ts0, song: s[2], user: "u1"}, - entity! { is => id: 4i64, timestamp: ts0, song: s[1], user: "u1"}, - entity! { is => id: 5i64, timestamp: ts0, song: s[1], user: "u1"}, + entity! { is => id: 1i64, timestamp: ts0, song: s[1], user: "u1", vid: 0i64 }, + entity! { is => id: 2i64, timestamp: ts0, song: s[1], user: "u2", vid: 1i64 }, + entity! { is => id: 3i64, timestamp: ts0, song: s[2], user: "u1", vid: 2i64 }, + entity! { is => id: 4i64, timestamp: ts0, song: s[1], user: "u1", vid: 3i64 }, + entity! { is => id: 5i64, timestamp: ts0, song: s[1], user: "u1", vid: 4i64 }, ], ), ]; + let entities0 = insert_ops(&manifest.schema, entities0); let entities1 = vec![( "Musician", vec![ - entity! { is => id: "m3", name: "Tom", mainBand: "b2", bands: vec!["b1", "b2"], favoriteCount: 5, birthDate: timestamp.clone() }, - entity! { is => id: "m4", name: "Valerie", bands: Vec::::new(), favoriteCount: 20, birthDate: timestamp.clone() }, + entity! { is => id: "m3", name: "Tom", mainBand: "b2", bands: vec!["b1", "b2"], favoriteCount: 5, birthDate: timestamp.clone(), vid: 2i64 }, + entity! { is => id: "m4", name: "Valerie", bands: Vec::::new(), favoriteCount: 20, birthDate: timestamp.clone(), vid: 3i64 }, ], )]; let entities1 = insert_ops(&manifest.schema, entities1); diff --git a/store/test-store/tests/postgres/aggregation.rs b/store/test-store/tests/postgres/aggregation.rs index 432bc685a62..de47b88d6a1 100644 --- a/store/test-store/tests/postgres/aggregation.rs +++ b/store/test-store/tests/postgres/aggregation.rs @@ -1,6 +1,7 @@ use std::fmt::Write; use std::{future::Future, sync::Arc}; +use graph::data::store::EntityV; use graph::{ blockchain::{block_stream::FirehoseCursor, BlockPtr, BlockTime}, components::{ @@ -82,7 +83,11 @@ pub async fn insert( .map(|data| { let data_type = schema.entity_type("Data").unwrap(); let key = data_type.key(data.id()); - EntityOperation::Set { data, key } + let vid = data.vid_or_default(); + EntityOperation::Set { + data: EntityV::new(data, vid), + key, + } }) .collect(); @@ -125,8 +130,8 @@ async fn insert_test_data(store: Arc, deployment: DeploymentL let ts64 = TIMES[0]; let entities = vec![ - entity! { schema => id: 1i64, timestamp: ts64, token: TOKEN1.clone(), price: bd(1), amount: bd(10) }, - entity! { schema => id: 2i64, timestamp: ts64, token: TOKEN2.clone(), price: bd(1), amount: bd(1) }, + entity! { schema => id: 1i64, timestamp: ts64, token: TOKEN1.clone(), price: bd(1), amount: bd(10), vid: 11i64 }, + entity! { schema => id: 2i64, timestamp: ts64, token: TOKEN2.clone(), price: bd(1), amount: bd(1), vid: 12i64 }, ]; insert(&store, &deployment, BLOCKS[0].clone(), TIMES[0], entities) @@ -135,8 +140,8 @@ async fn insert_test_data(store: Arc, deployment: DeploymentL let ts64 = TIMES[1]; let entities = vec![ - entity! { schema => id: 11i64, timestamp: ts64, token: TOKEN1.clone(), price: bd(2), amount: bd(2) }, - entity! { schema => id: 12i64, timestamp: ts64, token: TOKEN2.clone(), price: bd(2), amount: bd(20) }, + entity! { schema => id: 11i64, timestamp: ts64, token: TOKEN1.clone(), price: bd(2), amount: bd(2), vid: 21i64 }, + entity! { schema => id: 12i64, timestamp: ts64, token: TOKEN2.clone(), price: bd(2), amount: bd(20), vid: 22i64 }, ]; insert(&store, &deployment, BLOCKS[1].clone(), TIMES[1], entities) .await @@ -144,8 +149,8 @@ async fn insert_test_data(store: Arc, deployment: DeploymentL let ts64 = TIMES[2]; let entities = vec![ - entity! { schema => id: 21i64, timestamp: ts64, token: TOKEN1.clone(), price: bd(3), amount: bd(30) }, - entity! { schema => id: 22i64, timestamp: ts64, token: TOKEN2.clone(), price: bd(3), amount: bd(3) }, + entity! { schema => id: 21i64, timestamp: ts64, token: TOKEN1.clone(), price: bd(3), amount: bd(30), vid: 31i64 }, + entity! { schema => id: 22i64, timestamp: ts64, token: TOKEN2.clone(), price: bd(3), amount: bd(3), vid: 32i64 }, ]; insert(&store, &deployment, BLOCKS[2].clone(), TIMES[2], entities) .await @@ -153,8 +158,8 @@ async fn insert_test_data(store: Arc, deployment: DeploymentL let ts64 = TIMES[3]; let entities = vec![ - entity! { schema => id: 31i64, timestamp: ts64, token: TOKEN1.clone(), price: bd(4), amount: bd(4) }, - entity! { schema => id: 32i64, timestamp: ts64, token: TOKEN2.clone(), price: bd(4), amount: bd(40) }, + entity! { schema => id: 31i64, timestamp: ts64, token: TOKEN1.clone(), price: bd(4), amount: bd(4), vid: 41i64 }, + entity! { schema => id: 32i64, timestamp: ts64, token: TOKEN2.clone(), price: bd(4), amount: bd(40), vid: 42i64 }, ]; insert(&store, &deployment, BLOCKS[3].clone(), TIMES[3], entities) .await diff --git a/store/test-store/tests/postgres/graft.rs b/store/test-store/tests/postgres/graft.rs index 88f77c45b97..59cbe5e3f0a 100644 --- a/store/test-store/tests/postgres/graft.rs +++ b/store/test-store/tests/postgres/graft.rs @@ -9,7 +9,7 @@ use graph::components::store::{ DeploymentLocator, EntityOrder, EntityQuery, PruneReporter, PruneRequest, PruningStrategy, VersionStats, }; -use graph::data::store::{scalar, Id}; +use graph::data::store::{scalar, EntityV, Id}; use graph::data::subgraph::schema::*; use graph::data::subgraph::*; use graph::semver::Version; @@ -175,6 +175,7 @@ async fn insert_test_data(store: Arc) -> DeploymentLocator 184.4, false, None, + 0, ); transact_entity_operations(&store, &deployment, BLOCKS[0].clone(), vec![test_entity_1]) .await @@ -189,6 +190,7 @@ async fn insert_test_data(store: Arc) -> DeploymentLocator 159.1, true, Some("red"), + 1, ); let test_entity_3_1 = create_test_entity( "3", @@ -199,6 +201,7 @@ async fn insert_test_data(store: Arc) -> DeploymentLocator 111.7, false, Some("blue"), + 2, ); transact_entity_operations( &store, @@ -218,6 +221,7 @@ async fn insert_test_data(store: Arc) -> DeploymentLocator 111.7, false, None, + 3, ); transact_entity_operations( &store, @@ -241,6 +245,7 @@ fn create_test_entity( weight: f64, coffee: bool, favorite_color: Option<&str>, + vid: i64, ) -> EntityOperation { let bin_name = scalar::Bytes::from_str(&hex::encode(name)).unwrap(); let test_entity = entity! { TEST_SUBGRAPH_SCHEMA => @@ -258,7 +263,7 @@ fn create_test_entity( let entity_type = TEST_SUBGRAPH_SCHEMA.entity_type(entity_type).unwrap(); EntityOperation::Set { key: entity_type.parse_key(id).unwrap(), - data: test_entity, + data: EntityV::new(test_entity, vid), } } @@ -326,7 +331,7 @@ async fn check_graft( shaq.set("email", "shaq@gmail.com").unwrap(); let op = EntityOperation::Set { key: user_type.parse_key("3").unwrap(), - data: shaq, + data: EntityV::new(shaq, 3), }; transact_and_wait(&store, &deployment, BLOCKS[2].clone(), vec![op]) .await @@ -601,6 +606,7 @@ fn prune() { 157.1, true, Some("red"), + 4, ); transact_and_wait(&store, &src, BLOCKS[5].clone(), vec![user2]) .await diff --git a/store/test-store/tests/postgres/relational.rs b/store/test-store/tests/postgres/relational.rs index fe366b34509..8a2e5a2f3d9 100644 --- a/store/test-store/tests/postgres/relational.rs +++ b/store/test-store/tests/postgres/relational.rs @@ -318,6 +318,7 @@ fn insert_user_entity( drinks: Option>, visits: i64, block: BlockNumber, + vid: i64, ) { let user = make_user( &layout.input_schema, @@ -330,6 +331,7 @@ fn insert_user_entity( favorite_color, drinks, visits, + vid, ); insert_entity_at(conn, layout, entity_type, vec![user], block); @@ -346,6 +348,7 @@ fn make_user( favorite_color: Option<&str>, drinks: Option>, visits: i64, + vid: i64, ) -> Entity { let favorite_color = favorite_color .map(|s| Value::String(s.to_owned())) @@ -361,7 +364,8 @@ fn make_user( weight: BigDecimal::from(weight), coffee: coffee, favorite_color: favorite_color, - visits: visits + visits: visits, + vid: vid, }; if let Some(drinks) = drinks { user.insert("drinks", drinks.into()).unwrap(); @@ -384,6 +388,7 @@ fn insert_users(conn: &mut PgConnection, layout: &Layout) { None, 60, 0, + 0, ); insert_user_entity( conn, @@ -399,6 +404,7 @@ fn insert_users(conn: &mut PgConnection, layout: &Layout) { Some(vec!["beer", "wine"]), 50, 0, + 1, ); insert_user_entity( conn, @@ -414,6 +420,7 @@ fn insert_users(conn: &mut PgConnection, layout: &Layout) { Some(vec!["coffee", "tea"]), 22, 0, + 2, ); } @@ -431,6 +438,7 @@ fn update_user_entity( drinks: Option>, visits: i64, block: BlockNumber, + vid: i64, ) { let user = make_user( &layout.input_schema, @@ -443,6 +451,7 @@ fn update_user_entity( favorite_color, drinks, visits, + vid, ); update_entity_at(conn, layout, entity_type, vec![user], block); } @@ -454,17 +463,19 @@ fn insert_pet( id: &str, name: &str, block: BlockNumber, + vid: i64, ) { let pet = entity! { layout.input_schema => id: id, - name: name + name: name, + vid: vid, }; insert_entity_at(conn, layout, entity_type, vec![pet], block); } fn insert_pets(conn: &mut PgConnection, layout: &Layout) { - insert_pet(conn, layout, &*DOG_TYPE, "pluto", "Pluto", 0); - insert_pet(conn, layout, &*CAT_TYPE, "garfield", "Garfield", 0); + insert_pet(conn, layout, &*DOG_TYPE, "pluto", "Pluto", 0, 0); + insert_pet(conn, layout, &*CAT_TYPE, "garfield", "Garfield", 0, 1); } fn create_schema(conn: &mut PgConnection) -> Layout { @@ -484,6 +495,7 @@ fn create_schema(conn: &mut PgConnection) -> Layout { fn scrub(entity: &Entity) -> Entity { let mut scrubbed = entity.clone(); scrubbed.remove_null_fields(); + scrubbed.remove("vid"); scrubbed } @@ -597,6 +609,7 @@ fn update() { entity.set("string", "updated").unwrap(); entity.remove("strings"); entity.set("bool", Value::Null).unwrap(); + entity.set("vid", 1i64).unwrap(); let key = SCALAR_TYPE.key(entity.id()); let entity_type = layout.input_schema.entity_type("Scalar").unwrap(); @@ -624,8 +637,10 @@ fn update_many() { let mut one = SCALAR_ENTITY.clone(); let mut two = SCALAR_ENTITY.clone(); two.set("id", "two").unwrap(); + two.set("vid", 1i64).unwrap(); let mut three = SCALAR_ENTITY.clone(); three.set("id", "three").unwrap(); + three.set("vid", 2i64).unwrap(); insert_entity( conn, layout, @@ -647,6 +662,10 @@ fn update_many() { three.remove("strings"); three.set("color", "red").unwrap(); + one.set("vid", 3i64).unwrap(); + two.set("vid", 4i64).unwrap(); + three.set("vid", 5i64).unwrap(); + // generate keys let entity_type = layout.input_schema.entity_type("Scalar").unwrap(); let keys: Vec = ["one", "two", "three"] @@ -713,10 +732,13 @@ fn serialize_bigdecimal() { // Update with overwrite let mut entity = SCALAR_ENTITY.clone(); + let mut vid = 1i64; for d in &["50", "50.00", "5000", "0.5000", "0.050", "0.5", "0.05"] { let d = BigDecimal::from_str(d).unwrap(); entity.set("bigDecimal", d).unwrap(); + entity.set("vid", vid).unwrap(); + vid += 1; let key = SCALAR_TYPE.key(entity.id()); let entity_type = layout.input_schema.entity_type("Scalar").unwrap(); @@ -734,6 +756,7 @@ fn serialize_bigdecimal() { ) .expect("Failed to read Scalar[one]") .unwrap(); + entity.remove("vid"); assert_entity_eq!(entity, actual); } }); @@ -761,6 +784,7 @@ fn delete() { insert_entity(conn, layout, &*SCALAR_TYPE, vec![SCALAR_ENTITY.clone()]); let mut two = SCALAR_ENTITY.clone(); two.set("id", "two").unwrap(); + two.set("vid", 1i64).unwrap(); insert_entity(conn, layout, &*SCALAR_TYPE, vec![two]); // Delete where nothing is getting deleted @@ -795,8 +819,10 @@ fn insert_many_and_delete_many() { let one = SCALAR_ENTITY.clone(); let mut two = SCALAR_ENTITY.clone(); two.set("id", "two").unwrap(); + two.set("vid", 1i64).unwrap(); let mut three = SCALAR_ENTITY.clone(); three.set("id", "three").unwrap(); + three.set("vid", 2i64).unwrap(); insert_entity(conn, layout, &*SCALAR_TYPE, vec![one, two, three]); // confidence test: there should be 3 scalar entities in store right now @@ -877,6 +903,7 @@ fn conflicting_entity() { cat: &str, dog: &str, ferret: &str, + vid: i64, ) { let conflicting = |conn: &mut PgConnection, entity_type: &EntityType, types: Vec<&EntityType>| { @@ -891,6 +918,7 @@ fn conflicting_entity() { data: fred, block: 2, end: None, + vid: 0, }, 2, ) @@ -902,7 +930,7 @@ fn conflicting_entity() { let dog_type = layout.input_schema.entity_type(dog).unwrap(); let ferret_type = layout.input_schema.entity_type(ferret).unwrap(); - let fred = entity! { layout.input_schema => id: id.clone(), name: id.clone() }; + let fred = entity! { layout.input_schema => id: id.clone(), name: id.clone(), vid: vid }; insert_entity(conn, layout, &cat_type, vec![fred]); // If we wanted to create Fred the dog, which is forbidden, we'd run this: @@ -916,10 +944,10 @@ fn conflicting_entity() { run_test(|mut conn, layout| { let id = Value::String("fred".to_string()); - check(&mut conn, layout, id, "Cat", "Dog", "Ferret"); + check(&mut conn, layout, id, "Cat", "Dog", "Ferret", 0); let id = Value::Bytes(scalar::Bytes::from_str("0xf1ed").unwrap()); - check(&mut conn, layout, id, "ByteCat", "ByteDog", "ByteFerret"); + check(&mut conn, layout, id, "ByteCat", "ByteDog", "ByteFerret", 1); }) } @@ -931,7 +959,8 @@ fn revert_block() { let set_fred = |conn: &mut PgConnection, name, block| { let fred = entity! { layout.input_schema => id: id, - name: name + name: name, + vid: block as i64, }; if block == 0 { insert_entity_at(conn, layout, &*CAT_TYPE, vec![fred], block); @@ -971,6 +1000,7 @@ fn revert_block() { let marty = entity! { layout.input_schema => id: id, order: block, + vid: (block + 10) as i64 }; insert_entity_at(conn, layout, &*MINK_TYPE, vec![marty], block); } @@ -1049,6 +1079,7 @@ impl<'a> QueryChecker<'a> { None, 23, 0, + 3, ); insert_pets(conn, layout); @@ -1161,6 +1192,7 @@ fn check_block_finds() { None, 55, 1, + 4, ); checker @@ -1703,10 +1735,10 @@ struct FilterChecker<'a> { impl<'a> FilterChecker<'a> { fn new(conn: &'a mut PgConnection, layout: &'a Layout) -> Self { let (a1, a2, a2b, a3) = ferrets(); - insert_pet(conn, layout, &*FERRET_TYPE, "a1", &a1, 0); - insert_pet(conn, layout, &*FERRET_TYPE, "a2", &a2, 0); - insert_pet(conn, layout, &*FERRET_TYPE, "a2b", &a2b, 0); - insert_pet(conn, layout, &*FERRET_TYPE, "a3", &a3, 0); + insert_pet(conn, layout, &*FERRET_TYPE, "a1", &a1, 0, 0); + insert_pet(conn, layout, &*FERRET_TYPE, "a2", &a2, 0, 1); + insert_pet(conn, layout, &*FERRET_TYPE, "a2b", &a2b, 0, 2); + insert_pet(conn, layout, &*FERRET_TYPE, "a3", &a3, 0, 3); Self { conn, layout } } @@ -1850,7 +1882,8 @@ fn check_filters() { &*FERRET_TYPE, vec![entity! { layout.input_schema => id: "a1", - name: "Test" + name: "Test", + vid: 5i64 }], 1, ); diff --git a/store/test-store/tests/postgres/relational_bytes.rs b/store/test-store/tests/postgres/relational_bytes.rs index b7b8f36b7d7..6897dc9e914 100644 --- a/store/test-store/tests/postgres/relational_bytes.rs +++ b/store/test-store/tests/postgres/relational_bytes.rs @@ -57,6 +57,7 @@ lazy_static! { static ref BEEF_ENTITY: Entity = entity! { THINGS_SCHEMA => id: scalar::Bytes::from_str("deadbeef").unwrap(), name: "Beef", + vid: 0i64 }; static ref NAMESPACE: Namespace = Namespace::new("sgd0815".to_string()).unwrap(); static ref THING_TYPE: EntityType = THINGS_SCHEMA.entity_type("Thing").unwrap(); @@ -83,8 +84,9 @@ pub fn row_group_update( ) -> RowGroup { let mut group = RowGroup::new(entity_type.clone(), false); for (key, data) in data { + let vid = data.vid_or_default(); group - .push(EntityModification::overwrite(key, data, block), block) + .push(EntityModification::overwrite(key, data, block, vid), block) .unwrap(); } group @@ -97,8 +99,9 @@ pub fn row_group_insert( ) -> RowGroup { let mut group = RowGroup::new(entity_type.clone(), false); for (key, data) in data { + let vid = data.vid_or_default(); group - .push(EntityModification::insert(key, data, block), block) + .push(EntityModification::insert(key, data, block, vid), block) .unwrap(); } group @@ -128,14 +131,15 @@ fn insert_entity(conn: &mut PgConnection, layout: &Layout, entity_type: &str, en layout.insert(conn, &group, &MOCK_STOPWATCH).expect(&errmsg); } -fn insert_thing(conn: &mut PgConnection, layout: &Layout, id: &str, name: &str) { +fn insert_thing(conn: &mut PgConnection, layout: &Layout, id: &str, name: &str, vid: i64) { insert_entity( conn, layout, "Thing", entity! { layout.input_schema => id: id, - name: name + name: name, + vid: vid, }, ); } @@ -158,6 +162,7 @@ fn create_schema(conn: &mut PgConnection) -> Layout { fn scrub(entity: &Entity) -> Entity { let mut scrubbed = entity.clone(); scrubbed.remove_null_fields(); + scrubbed.remove("vid"); scrubbed } @@ -265,7 +270,7 @@ fn find() { const ID: &str = "deadbeef"; const NAME: &str = "Beef"; - insert_thing(&mut conn, layout, ID, NAME); + insert_thing(&mut conn, layout, ID, NAME, 0); // Happy path: find existing entity let entity = find_entity(conn, layout, ID).unwrap(); @@ -285,8 +290,8 @@ fn find_many() { const NAME: &str = "Beef"; const ID2: &str = "0xdeadbeef02"; const NAME2: &str = "Moo"; - insert_thing(&mut conn, layout, ID, NAME); - insert_thing(&mut conn, layout, ID2, NAME2); + insert_thing(&mut conn, layout, ID, NAME, 0); + insert_thing(&mut conn, layout, ID2, NAME2, 1); let mut id_map = BTreeMap::default(); let ids = IdList::try_from_iter( @@ -318,6 +323,7 @@ fn update() { // Update the entity let mut entity = BEEF_ENTITY.clone(); entity.set("name", "Moo").unwrap(); + entity.set("vid", 1i64).unwrap(); let key = THING_TYPE.key(entity.id()); let entity_id = entity.id(); @@ -333,7 +339,7 @@ fn update() { .expect("Failed to read Thing[deadbeef]") .unwrap(); - assert_entity_eq!(entity, actual); + assert_entity_eq!(scrub(&entity), actual); }); } @@ -345,6 +351,7 @@ fn delete() { insert_entity(&mut conn, layout, "Thing", BEEF_ENTITY.clone()); let mut two = BEEF_ENTITY.clone(); two.set("id", TWO_ID).unwrap(); + two.set("vid", 1i64).unwrap(); insert_entity(&mut conn, layout, "Thing", two); // Delete where nothing is getting deleted @@ -392,29 +399,34 @@ fn make_thing_tree(conn: &mut PgConnection, layout: &Layout) -> (Entity, Entity, let root = entity! { layout.input_schema => id: ROOT, name: "root", - children: vec!["babe01", "babe02"] + children: vec!["babe01", "babe02"], + vid: 0i64, }; let child1 = entity! { layout.input_schema => id: CHILD1, name: "child1", parent: "dead00", - children: vec![GRANDCHILD1] + children: vec![GRANDCHILD1], + vid: 1i64, }; let child2 = entity! { layout.input_schema => id: CHILD2, name: "child2", parent: "dead00", - children: vec![GRANDCHILD1] + children: vec![GRANDCHILD1], + vid: 2i64, }; let grand_child1 = entity! { layout.input_schema => id: GRANDCHILD1, name: "grandchild1", - parent: CHILD1 + parent: CHILD1, + vid: 3i64, }; let grand_child2 = entity! { layout.input_schema => id: GRANDCHILD2, name: "grandchild2", - parent: CHILD2 + parent: CHILD2, + vid: 4i64, }; insert_entity(conn, layout, "Thing", root.clone()); diff --git a/store/test-store/tests/postgres/store.rs b/store/test-store/tests/postgres/store.rs index aba953975a3..22e473e3810 100644 --- a/store/test-store/tests/postgres/store.rs +++ b/store/test-store/tests/postgres/store.rs @@ -2,11 +2,13 @@ use graph::blockchain::block_stream::FirehoseCursor; use graph::blockchain::BlockTime; use graph::data::graphql::ext::TypeDefinitionExt; use graph::data::query::QueryTarget; +use graph::data::store::EntityV; use graph::data::subgraph::schema::DeploymentCreate; +use graph::data_source::common::MappingABI; use graph::futures01::{future, Stream}; use graph::futures03::compat::Future01CompatExt; use graph::schema::{EntityType, InputSchema}; -use graph_chain_ethereum::{Mapping, MappingABI}; +use graph_chain_ethereum::Mapping; use hex_literal::hex; use lazy_static::lazy_static; use std::time::Duration; @@ -200,6 +202,7 @@ async fn insert_test_data(store: Arc) -> DeploymentLocator 184.4, false, None, + 0, ); transact_entity_operations( &store, @@ -219,6 +222,7 @@ async fn insert_test_data(store: Arc) -> DeploymentLocator 159.1, true, Some("red"), + 1, ); let test_entity_3_1 = create_test_entity( "3", @@ -229,6 +233,7 @@ async fn insert_test_data(store: Arc) -> DeploymentLocator 111.7, false, Some("blue"), + 2, ); transact_entity_operations( &store, @@ -248,6 +253,7 @@ async fn insert_test_data(store: Arc) -> DeploymentLocator 111.7, false, None, + 3, ); transact_and_wait( &store, @@ -271,6 +277,7 @@ fn create_test_entity( weight: f64, coffee: bool, favorite_color: Option<&str>, + vid: i64, ) -> EntityOperation { let bin_name = scalar::Bytes::from_str(&hex::encode(name)).unwrap(); let test_entity = entity! { TEST_SUBGRAPH_SCHEMA => @@ -287,7 +294,7 @@ fn create_test_entity( EntityOperation::Set { key: entity_type.parse_key(id).unwrap(), - data: test_entity, + data: EntityV::new(test_entity, vid), } } @@ -397,6 +404,7 @@ fn insert_entity() { 111.7, true, Some("green"), + 5, ); let count = get_entity_count(store.clone(), &deployment.hash); transact_and_wait( @@ -428,6 +436,7 @@ fn update_existing() { 111.7, true, Some("green"), + 6, ); let mut new_data = match op { EntityOperation::Set { ref data, .. } => data.clone(), @@ -435,7 +444,7 @@ fn update_existing() { }; // Verify that the entity before updating is different from what we expect afterwards - assert_ne!(writable.get(&entity_key).unwrap().unwrap(), new_data); + assert_ne!(writable.get(&entity_key).unwrap().unwrap(), new_data.e); // Set test entity; as the entity already exists an update should be performed let count = get_entity_count(store.clone(), &deployment.hash); @@ -450,13 +459,16 @@ fn update_existing() { assert_eq!(count, get_entity_count(store.clone(), &deployment.hash)); // Verify that the entity in the store has changed to what we have set. - let bin_name = match new_data.get("bin_name") { + let bin_name = match new_data.e.get("bin_name") { Some(Value::Bytes(bytes)) => bytes.clone(), _ => unreachable!(), }; - new_data.insert("bin_name", Value::Bytes(bin_name)).unwrap(); - assert_eq!(writable.get(&entity_key).unwrap(), Some(new_data)); + new_data + .e + .insert("bin_name", Value::Bytes(bin_name)) + .unwrap(); + assert_eq!(writable.get(&entity_key).unwrap(), Some(new_data.e)); }) } @@ -480,7 +492,7 @@ fn partially_update_existing() { TEST_BLOCK_3_PTR.clone(), vec![EntityOperation::Set { key: entity_key.clone(), - data: partial_entity.clone(), + data: EntityV::new(partial_entity.clone(), 11), }], ) .await @@ -1087,7 +1099,7 @@ fn revert_block_with_partial_update() { TEST_BLOCK_3_PTR.clone(), vec![EntityOperation::Set { key: entity_key.clone(), - data: partial_entity.clone(), + data: EntityV::new(partial_entity.clone(), 5), }], ) .await @@ -1182,7 +1194,7 @@ fn revert_block_with_dynamic_data_source_operations() { let ops = vec![EntityOperation::Set { key: user_key.clone(), - data: partial_entity.clone(), + data: EntityV::new(partial_entity.clone(), 5), }]; // Add user and dynamic data source to the store @@ -1290,9 +1302,12 @@ fn entity_changes_are_fired_and_forwarded_to_subscriptions() { let added_entities = vec![ ( "1".to_owned(), - entity! { schema => id: "1", name: "Johnny Boy" }, + entity! { schema => id: "1", name: "Johnny Boy", vid: 5i64 }, + ), + ( + "2".to_owned(), + entity! { schema => id: "2", name: "Tessa", vid: 6i64 }, ), - ("2".to_owned(), entity! { schema => id: "2", name: "Tessa" }), ]; transact_and_wait( &store.subgraph_store(), @@ -1302,7 +1317,7 @@ fn entity_changes_are_fired_and_forwarded_to_subscriptions() { .iter() .map(|(id, data)| EntityOperation::Set { key: USER_TYPE.parse_key(id.as_str()).unwrap(), - data: data.clone(), + data: EntityV::new(data.clone(), data.vid_or_default()), }) .collect(), ) @@ -1313,7 +1328,7 @@ fn entity_changes_are_fired_and_forwarded_to_subscriptions() { let updated_entity = entity! { schema => id: "1", name: "Johnny" }; let update_op = EntityOperation::Set { key: USER_TYPE.parse_key("1").unwrap(), - data: updated_entity.clone(), + data: EntityV::new(updated_entity.clone(), 7), }; // Delete an entity in the store @@ -1386,6 +1401,7 @@ fn throttle_subscription_delivers() { 120.7, false, None, + 7, ); transact_entity_operations( @@ -1431,6 +1447,7 @@ fn throttle_subscription_throttles() { 120.7, false, None, + 8, ); transact_entity_operations( @@ -1504,12 +1521,13 @@ fn handle_large_string_with_index() { name: &str, schema: &InputSchema, block: BlockNumber, + vid: i64, ) -> EntityModification { let data = entity! { schema => id: id, name: name }; let key = USER_TYPE.parse_key(id).unwrap(); - EntityModification::insert(key, data, block) + EntityModification::insert(key, data, block, vid) } run_test(|store, writable, deployment| async move { @@ -1538,8 +1556,8 @@ fn handle_large_string_with_index() { BlockTime::for_test(&*TEST_BLOCK_3_PTR), FirehoseCursor::None, vec![ - make_insert_op(ONE, &long_text, &schema, block), - make_insert_op(TWO, &other_text, &schema, block), + make_insert_op(ONE, &long_text, &schema, block, 11), + make_insert_op(TWO, &other_text, &schema, block, 12), ], &stopwatch_metrics, Vec::new(), @@ -1603,12 +1621,13 @@ fn handle_large_bytea_with_index() { name: &[u8], schema: &InputSchema, block: BlockNumber, + vid: i64, ) -> EntityModification { let data = entity! { schema => id: id, bin_name: scalar::Bytes::from(name) }; let key = USER_TYPE.parse_key(id).unwrap(); - EntityModification::insert(key, data, block) + EntityModification::insert(key, data, block, vid) } run_test(|store, writable, deployment| async move { @@ -1642,8 +1661,8 @@ fn handle_large_bytea_with_index() { BlockTime::for_test(&*TEST_BLOCK_3_PTR), FirehoseCursor::None, vec![ - make_insert_op(ONE, &long_bytea, &schema, block), - make_insert_op(TWO, &other_bytea, &schema, block), + make_insert_op(ONE, &long_bytea, &schema, block, 10), + make_insert_op(TWO, &other_bytea, &schema, block, 11), ], &stopwatch_metrics, Vec::new(), @@ -1811,34 +1830,35 @@ fn window() { id: &str, color: &str, age: i32, + vid: i64, ) -> EntityOperation { let entity = entity! { TEST_SUBGRAPH_SCHEMA => id: id, age: age, favorite_color: color }; EntityOperation::Set { key: entity_type.parse_key(id).unwrap(), - data: entity, + data: EntityV::new(entity, vid), } } - fn make_user(id: &str, color: &str, age: i32) -> EntityOperation { - make_color_and_age(&*USER_TYPE, id, color, age) + fn make_user(id: &str, color: &str, age: i32, vid: i64) -> EntityOperation { + make_color_and_age(&*USER_TYPE, id, color, age, vid) } - fn make_person(id: &str, color: &str, age: i32) -> EntityOperation { - make_color_and_age(&*PERSON_TYPE, id, color, age) + fn make_person(id: &str, color: &str, age: i32, vid: i64) -> EntityOperation { + make_color_and_age(&*PERSON_TYPE, id, color, age, vid) } let ops = vec![ - make_user("4", "green", 34), - make_user("5", "green", 17), - make_user("6", "green", 41), - make_user("7", "red", 25), - make_user("8", "red", 45), - make_user("9", "yellow", 37), - make_user("10", "blue", 27), - make_user("11", "blue", 19), - make_person("p1", "green", 12), - make_person("p2", "red", 15), + make_user("4", "green", 34, 11), + make_user("5", "green", 17, 12), + make_user("6", "green", 41, 13), + make_user("7", "red", 25, 14), + make_user("8", "red", 45, 15), + make_user("9", "yellow", 37, 16), + make_user("10", "blue", 27, 17), + make_user("11", "blue", 19, 18), + make_person("p1", "green", 12, 19), + make_person("p2", "red", 15, 20), ]; run_test(|store, _, deployment| async move { @@ -2076,6 +2096,7 @@ fn reorg_tracking() { deployment: &DeploymentLocator, age: i32, block: &BlockPtr, + vid: i64, ) { let test_entity_1 = create_test_entity( "1", @@ -2086,6 +2107,7 @@ fn reorg_tracking() { 184.4, false, None, + vid, ); transact_and_wait(store, deployment, block.clone(), vec![test_entity_1]) .await @@ -2136,15 +2158,15 @@ fn reorg_tracking() { check_state!(store, 2, 2, 2); // Forward to block 3 - update_john(&subgraph_store, &deployment, 70, &TEST_BLOCK_3_PTR).await; + update_john(&subgraph_store, &deployment, 70, &TEST_BLOCK_3_PTR, 5).await; check_state!(store, 2, 2, 3); // Forward to block 4 - update_john(&subgraph_store, &deployment, 71, &TEST_BLOCK_4_PTR).await; + update_john(&subgraph_store, &deployment, 71, &TEST_BLOCK_4_PTR, 6).await; check_state!(store, 2, 2, 4); // Forward to block 5 - update_john(&subgraph_store, &deployment, 72, &TEST_BLOCK_5_PTR).await; + update_john(&subgraph_store, &deployment, 72, &TEST_BLOCK_5_PTR, 7).await; check_state!(store, 2, 2, 5); // Revert all the way back to block 2 diff --git a/store/test-store/tests/postgres/writable.rs b/store/test-store/tests/postgres/writable.rs index 4a986e6f3fa..dbc66e5e401 100644 --- a/store/test-store/tests/postgres/writable.rs +++ b/store/test-store/tests/postgres/writable.rs @@ -1,14 +1,18 @@ -use graph::blockchain::block_stream::FirehoseCursor; +use graph::blockchain::block_stream::{EntitySourceOperation, FirehoseCursor}; +use graph::data::store::EntityV; use graph::data::subgraph::schema::DeploymentCreate; use graph::data::value::Word; use graph::data_source::CausalityRegion; use graph::schema::{EntityKey, EntityType, InputSchema}; use lazy_static::lazy_static; -use std::collections::BTreeSet; +use std::collections::{BTreeMap, BTreeSet}; use std::marker::PhantomData; +use std::ops::Range; use test_store::*; -use graph::components::store::{DeploymentLocator, DerivedEntityQuery, WritableStore}; +use graph::components::store::{ + DeploymentLocator, DerivedEntityQuery, SourceableStore, WritableStore, +}; use graph::data::subgraph::*; use graph::semver::Version; use graph::{entity, prelude::*}; @@ -21,9 +25,14 @@ const SCHEMA_GQL: &str = " id: ID!, count: Int!, } + type Counter2 @entity(immutable: true) { + id: ID!, + count: Int!, + } "; const COUNTER: &str = "Counter"; +const COUNTER2: &str = "Counter2"; lazy_static! { static ref TEST_SUBGRAPH_ID_STRING: String = String::from("writableSubgraph"); @@ -33,6 +42,7 @@ lazy_static! { InputSchema::parse_latest(SCHEMA_GQL, TEST_SUBGRAPH_ID.clone()) .expect("Failed to parse user schema"); static ref COUNTER_TYPE: EntityType = TEST_SUBGRAPH_SCHEMA.entity_type(COUNTER).unwrap(); + static ref COUNTER2_TYPE: EntityType = TEST_SUBGRAPH_SCHEMA.entity_type(COUNTER2).unwrap(); } /// Inserts test data into the store. @@ -80,7 +90,14 @@ fn remove_test_data(store: Arc) { /// Test harness for running database integration tests. fn run_test(test: F) where - F: FnOnce(Arc, Arc, DeploymentLocator) -> R + Send + 'static, + F: FnOnce( + Arc, + Arc, + Arc, + DeploymentLocator, + ) -> R + + Send + + 'static, R: std::future::Future + Send + 'static, { run_test_sequentially(|store| async move { @@ -95,10 +112,15 @@ where .writable(LOGGER.clone(), deployment.id, Arc::new(Vec::new())) .await .expect("we can get a writable store"); + let sourceable = store + .subgraph_store() + .sourceable(deployment.id) + .await + .expect("we can get a writable store"); // Run test and wait for the background writer to finish its work so // it won't conflict with the next test - test(store, writable, deployment).await; + test(store, writable, sourceable, deployment).await; }); } @@ -111,16 +133,45 @@ fn count_key(id: &str) -> EntityKey { COUNTER_TYPE.parse_key(id).unwrap() } -async fn insert_count(store: &Arc, deployment: &DeploymentLocator, count: u8) { +async fn insert_count( + store: &Arc, + deployment: &DeploymentLocator, + block: u8, + count: u8, + immutable_only: bool, +) { + let count_key_local = |counter_type: &EntityType, id: &str| counter_type.parse_key(id).unwrap(); let data = entity! { TEST_SUBGRAPH_SCHEMA => id: "1", count: count as i32 }; - let entity_op = EntityOperation::Set { - key: count_key(&data.get("id").unwrap().to_string()), - data, + let entity_op = if block != 3 && block != 5 && block != 7 { + EntityOperation::Set { + key: count_key_local(&COUNTER_TYPE, &data.get("id").unwrap().to_string()), + data: EntityV::new(data, block.into()), + } + } else { + EntityOperation::Remove { + key: count_key_local(&COUNTER_TYPE, &data.get("id").unwrap().to_string()), + } + }; + let mut ops = if immutable_only { + vec![] + } else { + vec![entity_op] }; - transact_entity_operations(store, deployment, block_pointer(count), vec![entity_op]) + if block < 6 { + let data = entity! { TEST_SUBGRAPH_SCHEMA => + id: &block.to_string(), + count :count as i32, + }; + let entity_op = EntityOperation::Set { + key: count_key_local(&COUNTER2_TYPE, &data.get("id").unwrap().to_string()), + data: EntityV::new(data, block.into()), + }; + ops.push(entity_op); + } + transact_entity_operations(store, deployment, block_pointer(block), ops) .await .unwrap(); } @@ -140,7 +191,7 @@ fn get_with_pending(batch: bool, read_count: F) where F: Send + Fn(&dyn WritableStore) -> i32 + Sync + 'static, { - run_test(move |store, writable, deployment| async move { + run_test(move |store, writable, _, deployment| async move { let subgraph_store = store.subgraph_store(); let read_count = || read_count(writable.as_ref()); @@ -150,13 +201,13 @@ where } for count in 1..4 { - insert_count(&subgraph_store, &deployment, count).await; + insert_count(&subgraph_store, &deployment, count, count, false).await; } // Test reading back with pending writes to the same entity pause_writer(&deployment).await; for count in 4..7 { - insert_count(&subgraph_store, &deployment, count).await; + insert_count(&subgraph_store, &deployment, count, count, false).await; } assert_eq!(6, read_count()); @@ -165,7 +216,7 @@ where // Test reading back with pending writes and a pending revert for count in 7..10 { - insert_count(&subgraph_store, &deployment, count).await; + insert_count(&subgraph_store, &deployment, count, count, false).await; } writable .revert_block_operations(block_pointer(2), FirehoseCursor::None) @@ -238,14 +289,14 @@ fn get_derived_nobatch() { #[test] fn restart() { - run_test(|store, writable, deployment| async move { + run_test(|store, writable, _, deployment| async move { let subgraph_store = store.subgraph_store(); let schema = subgraph_store.input_schema(&deployment.hash).unwrap(); // Cause an error by leaving out the non-nullable `count` attribute let entity_ops = vec![EntityOperation::Set { key: count_key("1"), - data: entity! { schema => id: "1" }, + data: EntityV::new(entity! { schema => id: "1"}, 0), }]; transact_entity_operations( &subgraph_store, @@ -269,7 +320,7 @@ fn restart() { // Retry our write with correct data let entity_ops = vec![EntityOperation::Set { key: count_key("1"), - data: entity! { schema => id: "1", count: 1 }, + data: EntityV::new(entity! { schema => id: "1", count: 1}, 0), }]; // `SubgraphStore` caches the correct writable so that this call // uses the restarted writable, and is equivalent to using @@ -286,3 +337,72 @@ fn restart() { writable.flush().await.unwrap(); }) } + +#[test] +fn read_range_test() { + run_test(|store, writable, sourceable, deployment| async move { + let result_entities = vec![ + r#"(1, [EntitySourceOperation { entity_op: Create, entity_type: EntityType(Counter), entity: Entity { count: Int(2), id: String("1") }, vid: 1 }, EntitySourceOperation { entity_op: Create, entity_type: EntityType(Counter2), entity: Entity { count: Int(2), id: String("1") }, vid: 1 }])"#, + r#"(2, [EntitySourceOperation { entity_op: Modify, entity_type: EntityType(Counter), entity: Entity { count: Int(4), id: String("1") }, vid: 2 }, EntitySourceOperation { entity_op: Create, entity_type: EntityType(Counter2), entity: Entity { count: Int(4), id: String("2") }, vid: 2 }])"#, + r#"(3, [EntitySourceOperation { entity_op: Delete, entity_type: EntityType(Counter), entity: Entity { count: Int(4), id: String("1") }, vid: 2 }, EntitySourceOperation { entity_op: Create, entity_type: EntityType(Counter2), entity: Entity { count: Int(6), id: String("3") }, vid: 3 }])"#, + r#"(4, [EntitySourceOperation { entity_op: Create, entity_type: EntityType(Counter), entity: Entity { count: Int(8), id: String("1") }, vid: 4 }, EntitySourceOperation { entity_op: Create, entity_type: EntityType(Counter2), entity: Entity { count: Int(8), id: String("4") }, vid: 4 }])"#, + r#"(5, [EntitySourceOperation { entity_op: Delete, entity_type: EntityType(Counter), entity: Entity { count: Int(8), id: String("1") }, vid: 4 }, EntitySourceOperation { entity_op: Create, entity_type: EntityType(Counter2), entity: Entity { count: Int(10), id: String("5") }, vid: 5 }])"#, + r#"(6, [EntitySourceOperation { entity_op: Create, entity_type: EntityType(Counter), entity: Entity { count: Int(12), id: String("1") }, vid: 6 }])"#, + r#"(7, [EntitySourceOperation { entity_op: Delete, entity_type: EntityType(Counter), entity: Entity { count: Int(12), id: String("1") }, vid: 6 }])"#, + ]; + let subgraph_store = store.subgraph_store(); + writable.deployment_synced().unwrap(); + + for count in 1..=5 { + insert_count(&subgraph_store, &deployment, count, 2 * count, false).await; + } + writable.flush().await.unwrap(); + writable.deployment_synced().unwrap(); + + let br: Range = 0..18; + let entity_types = vec![COUNTER_TYPE.clone(), COUNTER2_TYPE.clone()]; + let e: BTreeMap> = sourceable + .get_range(entity_types.clone(), CausalityRegion::ONCHAIN, br.clone()) + .unwrap(); + assert_eq!(e.len(), 5); + for en in &e { + let index = *en.0 - 1; + let a = result_entities[index as usize]; + assert_eq!(a, format!("{:?}", en)); + } + for count in 6..=7 { + insert_count(&subgraph_store, &deployment, count, 2 * count, false).await; + } + writable.flush().await.unwrap(); + writable.deployment_synced().unwrap(); + let e: BTreeMap> = sourceable + .get_range(entity_types, CausalityRegion::ONCHAIN, br) + .unwrap(); + assert_eq!(e.len(), 7); + for en in &e { + let index = *en.0 - 1; + let a = result_entities[index as usize]; + assert_eq!(a, format!("{:?}", en)); + } + }) +} + +#[test] +fn read_immutable_only_range_test() { + run_test(|store, writable, sourceable, deployment| async move { + let subgraph_store = store.subgraph_store(); + writable.deployment_synced().unwrap(); + + for count in 1..=4 { + insert_count(&subgraph_store, &deployment, count, 2 * count, true).await; + } + writable.flush().await.unwrap(); + writable.deployment_synced().unwrap(); + let br: Range = 0..18; + let entity_types = vec![COUNTER2_TYPE.clone()]; + let e: BTreeMap> = sourceable + .get_range(entity_types.clone(), CausalityRegion::ONCHAIN, br.clone()) + .unwrap(); + assert_eq!(e.len(), 4); + }) +} diff --git a/tests/contracts/out/LimitedContract.sol/LimitedContract.json b/tests/contracts/out/LimitedContract.sol/LimitedContract.json index 8dae4d1f7ce..f853978ad6c 100644 --- a/tests/contracts/out/LimitedContract.sol/LimitedContract.json +++ b/tests/contracts/out/LimitedContract.sol/LimitedContract.json @@ -1,450 +1 @@ -{ - "abi": [ - { "type": "constructor", "inputs": [], "stateMutability": "nonpayable" }, - { - "type": "function", - "name": "inc", - "inputs": [ - { "name": "value", "type": "uint256", "internalType": "uint256" } - ], - "outputs": [{ "name": "", "type": "uint256", "internalType": "uint256" }], - "stateMutability": "pure" - }, - { "type": "event", "name": "Trigger", "inputs": [], "anonymous": false } - ], - "bytecode": { - "object": "0x608060405234801561001057600080fd5b506040517f3d53a39550e04688065827f3bb86584cb007ab9ebca7ebd528e7301c9c31eb5d90600090a1610120806100496000396000f3fe6080604052348015600f57600080fd5b506004361060285760003560e01c8063812600df14602d575b600080fd5b603c603836600460b2565b604e565b60405190815260200160405180910390f35b6000600a821060a35760405162461bcd60e51b815260206004820152601b60248201527f63616e206f6e6c792068616e646c652076616c756573203c2031300000000000604482015260640160405180910390fd5b60ac82600160ca565b92915050565b60006020828403121560c357600080fd5b5035919050565b8082018082111560ac57634e487b7160e01b600052601160045260246000fdfea264697066735822122045679e894d199dcf13e7f3e6d9816bf08cd9cceab355500d502bbfada548205f64736f6c63430008130033", - "sourceMap": "57:257:0:-:0;;;110:45;;;;;;;;;-1:-1:-1;139:9:0;;;;;;;57:257;;;;;;", - "linkReferences": {} - }, - "deployedBytecode": { - "object": "0x6080604052348015600f57600080fd5b506004361060285760003560e01c8063812600df14602d575b600080fd5b603c603836600460b2565b604e565b60405190815260200160405180910390f35b6000600a821060a35760405162461bcd60e51b815260206004820152601b60248201527f63616e206f6e6c792068616e646c652076616c756573203c2031300000000000604482015260640160405180910390fd5b60ac82600160ca565b92915050565b60006020828403121560c357600080fd5b5035919050565b8082018082111560ac57634e487b7160e01b600052601160045260246000fdfea264697066735822122045679e894d199dcf13e7f3e6d9816bf08cd9cceab355500d502bbfada548205f64736f6c63430008130033", - "sourceMap": "57:257:0:-:0;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;161:151;;;;;;:::i;:::-;;:::i;:::-;;;345:25:4;;;333:2;318:18;161:151:0;;;;;;;;210:7;245:2;237:5;:10;229:50;;;;-1:-1:-1;;;229:50:0;;583:2:4;229:50:0;;;565:21:4;622:2;602:18;;;595:30;661:29;641:18;;;634:57;708:18;;229:50:0;;;;;;;;296:9;:5;304:1;296:9;:::i;:::-;289:16;161:151;-1:-1:-1;;161:151:0:o;14:180:4:-;73:6;126:2;114:9;105:7;101:23;97:32;94:52;;;142:1;139;132:12;94:52;-1:-1:-1;165:23:4;;14:180;-1:-1:-1;14:180:4:o;737:222::-;802:9;;;823:10;;;820:133;;;875:10;870:3;866:20;863:1;856:31;910:4;907:1;900:15;938:4;935:1;928:15", - "linkReferences": {} - }, - "methodIdentifiers": { "inc(uint256)": "812600df" }, - "rawMetadata": "{\"compiler\":{\"version\":\"0.8.19+commit.7dd6d404\"},\"language\":\"Solidity\",\"output\":{\"abi\":[{\"inputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"constructor\"},{\"anonymous\":false,\"inputs\":[],\"name\":\"Trigger\",\"type\":\"event\"},{\"inputs\":[{\"internalType\":\"uint256\",\"name\":\"value\",\"type\":\"uint256\"}],\"name\":\"inc\",\"outputs\":[{\"internalType\":\"uint256\",\"name\":\"\",\"type\":\"uint256\"}],\"stateMutability\":\"pure\",\"type\":\"function\"}],\"devdoc\":{\"kind\":\"dev\",\"methods\":{},\"version\":1},\"userdoc\":{\"kind\":\"user\",\"methods\":{},\"version\":1}},\"settings\":{\"compilationTarget\":{\"src/LimitedContract.sol\":\"LimitedContract\"},\"evmVersion\":\"paris\",\"libraries\":{},\"metadata\":{\"bytecodeHash\":\"ipfs\"},\"optimizer\":{\"enabled\":true,\"runs\":200},\"remappings\":[]},\"sources\":{\"src/LimitedContract.sol\":{\"keccak256\":\"0x7b291e6c8d7562ba65f036bd8b25c87587c57f5c35d5a6ea587a4eb6c7de4b02\",\"license\":\"MIT\",\"urls\":[\"bzz-raw://b7b7d9ad73d3f266dff610553eac7a1454f71e616036b0b50cee8610b999c2eb\",\"dweb:/ipfs/QmcdMqSxkNDwHJ8pMyh2jK2sA6Xrk4VSdm4nqZ86EK2Vut\"]}},\"version\":1}", - "metadata": { - "compiler": { "version": "0.8.19+commit.7dd6d404" }, - "language": "Solidity", - "output": { - "abi": [ - { - "inputs": [], - "stateMutability": "nonpayable", - "type": "constructor" - }, - { - "inputs": [], - "type": "event", - "name": "Trigger", - "anonymous": false - }, - { - "inputs": [ - { "internalType": "uint256", "name": "value", "type": "uint256" } - ], - "stateMutability": "pure", - "type": "function", - "name": "inc", - "outputs": [ - { "internalType": "uint256", "name": "", "type": "uint256" } - ] - } - ], - "devdoc": { "kind": "dev", "methods": {}, "version": 1 }, - "userdoc": { "kind": "user", "methods": {}, "version": 1 } - }, - "settings": { - "remappings": [], - "optimizer": { "enabled": true, "runs": 200 }, - "metadata": { "bytecodeHash": "ipfs" }, - "compilationTarget": { "src/LimitedContract.sol": "LimitedContract" }, - "evmVersion": "paris", - "libraries": {} - }, - "sources": { - "src/LimitedContract.sol": { - "keccak256": "0x7b291e6c8d7562ba65f036bd8b25c87587c57f5c35d5a6ea587a4eb6c7de4b02", - "urls": [ - "bzz-raw://b7b7d9ad73d3f266dff610553eac7a1454f71e616036b0b50cee8610b999c2eb", - "dweb:/ipfs/QmcdMqSxkNDwHJ8pMyh2jK2sA6Xrk4VSdm4nqZ86EK2Vut" - ], - "license": "MIT" - } - }, - "version": 1 - }, - "ast": { - "absolutePath": "src/LimitedContract.sol", - "id": 31, - "exportedSymbols": { "LimitedContract": [30] }, - "nodeType": "SourceUnit", - "src": "32:283:0", - "nodes": [ - { - "id": 1, - "nodeType": "PragmaDirective", - "src": "32:23:0", - "nodes": [], - "literals": ["solidity", "^", "0.8", ".0"] - }, - { - "id": 30, - "nodeType": "ContractDefinition", - "src": "57:257:0", - "nodes": [ - { - "id": 3, - "nodeType": "EventDefinition", - "src": "88:16:0", - "nodes": [], - "anonymous": false, - "eventSelector": "3d53a39550e04688065827f3bb86584cb007ab9ebca7ebd528e7301c9c31eb5d", - "name": "Trigger", - "nameLocation": "94:7:0", - "parameters": { - "id": 2, - "nodeType": "ParameterList", - "parameters": [], - "src": "101:2:0" - } - }, - { - "id": 10, - "nodeType": "FunctionDefinition", - "src": "110:45:0", - "nodes": [], - "body": { - "id": 9, - "nodeType": "Block", - "src": "124:31:0", - "nodes": [], - "statements": [ - { - "eventCall": { - "arguments": [], - "expression": { - "argumentTypes": [], - "id": 6, - "name": "Trigger", - "nodeType": "Identifier", - "overloadedDeclarations": [], - "referencedDeclaration": 3, - "src": "139:7:0", - "typeDescriptions": { - "typeIdentifier": "t_function_event_nonpayable$__$returns$__$", - "typeString": "function ()" - } - }, - "id": 7, - "isConstant": false, - "isLValue": false, - "isPure": false, - "kind": "functionCall", - "lValueRequested": false, - "nameLocations": [], - "names": [], - "nodeType": "FunctionCall", - "src": "139:9:0", - "tryCall": false, - "typeDescriptions": { - "typeIdentifier": "t_tuple$__$", - "typeString": "tuple()" - } - }, - "id": 8, - "nodeType": "EmitStatement", - "src": "134:14:0" - } - ] - }, - "implemented": true, - "kind": "constructor", - "modifiers": [], - "name": "", - "nameLocation": "-1:-1:-1", - "parameters": { - "id": 4, - "nodeType": "ParameterList", - "parameters": [], - "src": "121:2:0" - }, - "returnParameters": { - "id": 5, - "nodeType": "ParameterList", - "parameters": [], - "src": "124:0:0" - }, - "scope": 30, - "stateMutability": "nonpayable", - "virtual": false, - "visibility": "public" - }, - { - "id": 29, - "nodeType": "FunctionDefinition", - "src": "161:151:0", - "nodes": [], - "body": { - "id": 28, - "nodeType": "Block", - "src": "219:93:0", - "nodes": [], - "statements": [ - { - "expression": { - "arguments": [ - { - "commonType": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - }, - "id": 20, - "isConstant": false, - "isLValue": false, - "isPure": false, - "lValueRequested": false, - "leftExpression": { - "id": 18, - "name": "value", - "nodeType": "Identifier", - "overloadedDeclarations": [], - "referencedDeclaration": 12, - "src": "237:5:0", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - } - }, - "nodeType": "BinaryOperation", - "operator": "<", - "rightExpression": { - "hexValue": "3130", - "id": 19, - "isConstant": false, - "isLValue": false, - "isPure": true, - "kind": "number", - "lValueRequested": false, - "nodeType": "Literal", - "src": "245:2:0", - "typeDescriptions": { - "typeIdentifier": "t_rational_10_by_1", - "typeString": "int_const 10" - }, - "value": "10" - }, - "src": "237:10:0", - "typeDescriptions": { - "typeIdentifier": "t_bool", - "typeString": "bool" - } - }, - { - "hexValue": "63616e206f6e6c792068616e646c652076616c756573203c203130", - "id": 21, - "isConstant": false, - "isLValue": false, - "isPure": true, - "kind": "string", - "lValueRequested": false, - "nodeType": "Literal", - "src": "249:29:0", - "typeDescriptions": { - "typeIdentifier": "t_stringliteral_578cd1fc098748633f5d7d46bba428bb3129c1e63324f2b7151699cae5146449", - "typeString": "literal_string \"can only handle values < 10\"" - }, - "value": "can only handle values < 10" - } - ], - "expression": { - "argumentTypes": [ - { "typeIdentifier": "t_bool", "typeString": "bool" }, - { - "typeIdentifier": "t_stringliteral_578cd1fc098748633f5d7d46bba428bb3129c1e63324f2b7151699cae5146449", - "typeString": "literal_string \"can only handle values < 10\"" - } - ], - "id": 17, - "name": "require", - "nodeType": "Identifier", - "overloadedDeclarations": [-18, -18], - "referencedDeclaration": -18, - "src": "229:7:0", - "typeDescriptions": { - "typeIdentifier": "t_function_require_pure$_t_bool_$_t_string_memory_ptr_$returns$__$", - "typeString": "function (bool,string memory) pure" - } - }, - "id": 22, - "isConstant": false, - "isLValue": false, - "isPure": false, - "kind": "functionCall", - "lValueRequested": false, - "nameLocations": [], - "names": [], - "nodeType": "FunctionCall", - "src": "229:50:0", - "tryCall": false, - "typeDescriptions": { - "typeIdentifier": "t_tuple$__$", - "typeString": "tuple()" - } - }, - "id": 23, - "nodeType": "ExpressionStatement", - "src": "229:50:0" - }, - { - "expression": { - "commonType": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - }, - "id": 26, - "isConstant": false, - "isLValue": false, - "isPure": false, - "lValueRequested": false, - "leftExpression": { - "id": 24, - "name": "value", - "nodeType": "Identifier", - "overloadedDeclarations": [], - "referencedDeclaration": 12, - "src": "296:5:0", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - } - }, - "nodeType": "BinaryOperation", - "operator": "+", - "rightExpression": { - "hexValue": "31", - "id": 25, - "isConstant": false, - "isLValue": false, - "isPure": true, - "kind": "number", - "lValueRequested": false, - "nodeType": "Literal", - "src": "304:1:0", - "typeDescriptions": { - "typeIdentifier": "t_rational_1_by_1", - "typeString": "int_const 1" - }, - "value": "1" - }, - "src": "296:9:0", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - } - }, - "functionReturnParameters": 16, - "id": 27, - "nodeType": "Return", - "src": "289:16:0" - } - ] - }, - "functionSelector": "812600df", - "implemented": true, - "kind": "function", - "modifiers": [], - "name": "inc", - "nameLocation": "170:3:0", - "parameters": { - "id": 13, - "nodeType": "ParameterList", - "parameters": [ - { - "constant": false, - "id": 12, - "mutability": "mutable", - "name": "value", - "nameLocation": "182:5:0", - "nodeType": "VariableDeclaration", - "scope": 29, - "src": "174:13:0", - "stateVariable": false, - "storageLocation": "default", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - }, - "typeName": { - "id": 11, - "name": "uint256", - "nodeType": "ElementaryTypeName", - "src": "174:7:0", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - } - }, - "visibility": "internal" - } - ], - "src": "173:15:0" - }, - "returnParameters": { - "id": 16, - "nodeType": "ParameterList", - "parameters": [ - { - "constant": false, - "id": 15, - "mutability": "mutable", - "name": "", - "nameLocation": "-1:-1:-1", - "nodeType": "VariableDeclaration", - "scope": 29, - "src": "210:7:0", - "stateVariable": false, - "storageLocation": "default", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - }, - "typeName": { - "id": 14, - "name": "uint256", - "nodeType": "ElementaryTypeName", - "src": "210:7:0", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - } - }, - "visibility": "internal" - } - ], - "src": "209:9:0" - }, - "scope": 30, - "stateMutability": "pure", - "virtual": false, - "visibility": "public" - } - ], - "abstract": false, - "baseContracts": [], - "canonicalName": "LimitedContract", - "contractDependencies": [], - "contractKind": "contract", - "fullyImplemented": true, - "linearizedBaseContracts": [30], - "name": "LimitedContract", - "nameLocation": "66:15:0", - "scope": 31, - "usedErrors": [] - } - ], - "license": "MIT" - }, - "id": 0 -} +{"abi":[{"type":"constructor","inputs":[],"stateMutability":"nonpayable"},{"type":"function","name":"inc","inputs":[{"name":"value","type":"uint256","internalType":"uint256"}],"outputs":[{"name":"","type":"uint256","internalType":"uint256"}],"stateMutability":"pure"},{"type":"event","name":"Trigger","inputs":[],"anonymous":false}],"bytecode":{"object":"0x608060405234801561001057600080fd5b506040517f3d53a39550e04688065827f3bb86584cb007ab9ebca7ebd528e7301c9c31eb5d90600090a1610120806100496000396000f3fe6080604052348015600f57600080fd5b506004361060285760003560e01c8063812600df14602d575b600080fd5b603c603836600460b2565b604e565b60405190815260200160405180910390f35b6000600a821060a35760405162461bcd60e51b815260206004820152601b60248201527f63616e206f6e6c792068616e646c652076616c756573203c2031300000000000604482015260640160405180910390fd5b60ac82600160ca565b92915050565b60006020828403121560c357600080fd5b5035919050565b8082018082111560ac57634e487b7160e01b600052601160045260246000fdfea264697066735822122045679e894d199dcf13e7f3e6d9816bf08cd9cceab355500d502bbfada548205f64736f6c63430008130033","sourceMap":"57:257:0:-:0;;;110:45;;;;;;;;;-1:-1:-1;139:9:0;;;;;;;57:257;;;;;;","linkReferences":{}},"deployedBytecode":{"object":"0x6080604052348015600f57600080fd5b506004361060285760003560e01c8063812600df14602d575b600080fd5b603c603836600460b2565b604e565b60405190815260200160405180910390f35b6000600a821060a35760405162461bcd60e51b815260206004820152601b60248201527f63616e206f6e6c792068616e646c652076616c756573203c2031300000000000604482015260640160405180910390fd5b60ac82600160ca565b92915050565b60006020828403121560c357600080fd5b5035919050565b8082018082111560ac57634e487b7160e01b600052601160045260246000fdfea264697066735822122045679e894d199dcf13e7f3e6d9816bf08cd9cceab355500d502bbfada548205f64736f6c63430008130033","sourceMap":"57:257:0:-:0;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;161:151;;;;;;:::i;:::-;;:::i;:::-;;;345:25:4;;;333:2;318:18;161:151:0;;;;;;;;210:7;245:2;237:5;:10;229:50;;;;-1:-1:-1;;;229:50:0;;583:2:4;229:50:0;;;565:21:4;622:2;602:18;;;595:30;661:29;641:18;;;634:57;708:18;;229:50:0;;;;;;;;296:9;:5;304:1;296:9;:::i;:::-;289:16;161:151;-1:-1:-1;;161:151:0:o;14:180:4:-;73:6;126:2;114:9;105:7;101:23;97:32;94:52;;;142:1;139;132:12;94:52;-1:-1:-1;165:23:4;;14:180;-1:-1:-1;14:180:4:o;737:222::-;802:9;;;823:10;;;820:133;;;875:10;870:3;866:20;863:1;856:31;910:4;907:1;900:15;938:4;935:1;928:15","linkReferences":{}},"methodIdentifiers":{"inc(uint256)":"812600df"},"rawMetadata":"{\"compiler\":{\"version\":\"0.8.19+commit.7dd6d404\"},\"language\":\"Solidity\",\"output\":{\"abi\":[{\"inputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"constructor\"},{\"anonymous\":false,\"inputs\":[],\"name\":\"Trigger\",\"type\":\"event\"},{\"inputs\":[{\"internalType\":\"uint256\",\"name\":\"value\",\"type\":\"uint256\"}],\"name\":\"inc\",\"outputs\":[{\"internalType\":\"uint256\",\"name\":\"\",\"type\":\"uint256\"}],\"stateMutability\":\"pure\",\"type\":\"function\"}],\"devdoc\":{\"kind\":\"dev\",\"methods\":{},\"version\":1},\"userdoc\":{\"kind\":\"user\",\"methods\":{},\"version\":1}},\"settings\":{\"compilationTarget\":{\"src/LimitedContract.sol\":\"LimitedContract\"},\"evmVersion\":\"paris\",\"libraries\":{},\"metadata\":{\"bytecodeHash\":\"ipfs\"},\"optimizer\":{\"enabled\":true,\"runs\":200},\"remappings\":[]},\"sources\":{\"src/LimitedContract.sol\":{\"keccak256\":\"0x7b291e6c8d7562ba65f036bd8b25c87587c57f5c35d5a6ea587a4eb6c7de4b02\",\"license\":\"MIT\",\"urls\":[\"bzz-raw://b7b7d9ad73d3f266dff610553eac7a1454f71e616036b0b50cee8610b999c2eb\",\"dweb:/ipfs/QmcdMqSxkNDwHJ8pMyh2jK2sA6Xrk4VSdm4nqZ86EK2Vut\"]}},\"version\":1}","metadata":{"compiler":{"version":"0.8.19+commit.7dd6d404"},"language":"Solidity","output":{"abi":[{"inputs":[],"stateMutability":"nonpayable","type":"constructor"},{"inputs":[],"type":"event","name":"Trigger","anonymous":false},{"inputs":[{"internalType":"uint256","name":"value","type":"uint256"}],"stateMutability":"pure","type":"function","name":"inc","outputs":[{"internalType":"uint256","name":"","type":"uint256"}]}],"devdoc":{"kind":"dev","methods":{},"version":1},"userdoc":{"kind":"user","methods":{},"version":1}},"settings":{"remappings":[],"optimizer":{"enabled":true,"runs":200},"metadata":{"bytecodeHash":"ipfs"},"compilationTarget":{"src/LimitedContract.sol":"LimitedContract"},"evmVersion":"paris","libraries":{}},"sources":{"src/LimitedContract.sol":{"keccak256":"0x7b291e6c8d7562ba65f036bd8b25c87587c57f5c35d5a6ea587a4eb6c7de4b02","urls":["bzz-raw://b7b7d9ad73d3f266dff610553eac7a1454f71e616036b0b50cee8610b999c2eb","dweb:/ipfs/QmcdMqSxkNDwHJ8pMyh2jK2sA6Xrk4VSdm4nqZ86EK2Vut"],"license":"MIT"}},"version":1},"ast":{"absolutePath":"src/LimitedContract.sol","id":31,"exportedSymbols":{"LimitedContract":[30]},"nodeType":"SourceUnit","src":"32:283:0","nodes":[{"id":1,"nodeType":"PragmaDirective","src":"32:23:0","nodes":[],"literals":["solidity","^","0.8",".0"]},{"id":30,"nodeType":"ContractDefinition","src":"57:257:0","nodes":[{"id":3,"nodeType":"EventDefinition","src":"88:16:0","nodes":[],"anonymous":false,"eventSelector":"3d53a39550e04688065827f3bb86584cb007ab9ebca7ebd528e7301c9c31eb5d","name":"Trigger","nameLocation":"94:7:0","parameters":{"id":2,"nodeType":"ParameterList","parameters":[],"src":"101:2:0"}},{"id":10,"nodeType":"FunctionDefinition","src":"110:45:0","nodes":[],"body":{"id":9,"nodeType":"Block","src":"124:31:0","nodes":[],"statements":[{"eventCall":{"arguments":[],"expression":{"argumentTypes":[],"id":6,"name":"Trigger","nodeType":"Identifier","overloadedDeclarations":[],"referencedDeclaration":3,"src":"139:7:0","typeDescriptions":{"typeIdentifier":"t_function_event_nonpayable$__$returns$__$","typeString":"function ()"}},"id":7,"isConstant":false,"isLValue":false,"isPure":false,"kind":"functionCall","lValueRequested":false,"nameLocations":[],"names":[],"nodeType":"FunctionCall","src":"139:9:0","tryCall":false,"typeDescriptions":{"typeIdentifier":"t_tuple$__$","typeString":"tuple()"}},"id":8,"nodeType":"EmitStatement","src":"134:14:0"}]},"implemented":true,"kind":"constructor","modifiers":[],"name":"","nameLocation":"-1:-1:-1","parameters":{"id":4,"nodeType":"ParameterList","parameters":[],"src":"121:2:0"},"returnParameters":{"id":5,"nodeType":"ParameterList","parameters":[],"src":"124:0:0"},"scope":30,"stateMutability":"nonpayable","virtual":false,"visibility":"public"},{"id":29,"nodeType":"FunctionDefinition","src":"161:151:0","nodes":[],"body":{"id":28,"nodeType":"Block","src":"219:93:0","nodes":[],"statements":[{"expression":{"arguments":[{"commonType":{"typeIdentifier":"t_uint256","typeString":"uint256"},"id":20,"isConstant":false,"isLValue":false,"isPure":false,"lValueRequested":false,"leftExpression":{"id":18,"name":"value","nodeType":"Identifier","overloadedDeclarations":[],"referencedDeclaration":12,"src":"237:5:0","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"}},"nodeType":"BinaryOperation","operator":"<","rightExpression":{"hexValue":"3130","id":19,"isConstant":false,"isLValue":false,"isPure":true,"kind":"number","lValueRequested":false,"nodeType":"Literal","src":"245:2:0","typeDescriptions":{"typeIdentifier":"t_rational_10_by_1","typeString":"int_const 10"},"value":"10"},"src":"237:10:0","typeDescriptions":{"typeIdentifier":"t_bool","typeString":"bool"}},{"hexValue":"63616e206f6e6c792068616e646c652076616c756573203c203130","id":21,"isConstant":false,"isLValue":false,"isPure":true,"kind":"string","lValueRequested":false,"nodeType":"Literal","src":"249:29:0","typeDescriptions":{"typeIdentifier":"t_stringliteral_578cd1fc098748633f5d7d46bba428bb3129c1e63324f2b7151699cae5146449","typeString":"literal_string \"can only handle values < 10\""},"value":"can only handle values < 10"}],"expression":{"argumentTypes":[{"typeIdentifier":"t_bool","typeString":"bool"},{"typeIdentifier":"t_stringliteral_578cd1fc098748633f5d7d46bba428bb3129c1e63324f2b7151699cae5146449","typeString":"literal_string \"can only handle values < 10\""}],"id":17,"name":"require","nodeType":"Identifier","overloadedDeclarations":[-18,-18],"referencedDeclaration":-18,"src":"229:7:0","typeDescriptions":{"typeIdentifier":"t_function_require_pure$_t_bool_$_t_string_memory_ptr_$returns$__$","typeString":"function (bool,string memory) pure"}},"id":22,"isConstant":false,"isLValue":false,"isPure":false,"kind":"functionCall","lValueRequested":false,"nameLocations":[],"names":[],"nodeType":"FunctionCall","src":"229:50:0","tryCall":false,"typeDescriptions":{"typeIdentifier":"t_tuple$__$","typeString":"tuple()"}},"id":23,"nodeType":"ExpressionStatement","src":"229:50:0"},{"expression":{"commonType":{"typeIdentifier":"t_uint256","typeString":"uint256"},"id":26,"isConstant":false,"isLValue":false,"isPure":false,"lValueRequested":false,"leftExpression":{"id":24,"name":"value","nodeType":"Identifier","overloadedDeclarations":[],"referencedDeclaration":12,"src":"296:5:0","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"}},"nodeType":"BinaryOperation","operator":"+","rightExpression":{"hexValue":"31","id":25,"isConstant":false,"isLValue":false,"isPure":true,"kind":"number","lValueRequested":false,"nodeType":"Literal","src":"304:1:0","typeDescriptions":{"typeIdentifier":"t_rational_1_by_1","typeString":"int_const 1"},"value":"1"},"src":"296:9:0","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"}},"functionReturnParameters":16,"id":27,"nodeType":"Return","src":"289:16:0"}]},"functionSelector":"812600df","implemented":true,"kind":"function","modifiers":[],"name":"inc","nameLocation":"170:3:0","parameters":{"id":13,"nodeType":"ParameterList","parameters":[{"constant":false,"id":12,"mutability":"mutable","name":"value","nameLocation":"182:5:0","nodeType":"VariableDeclaration","scope":29,"src":"174:13:0","stateVariable":false,"storageLocation":"default","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"},"typeName":{"id":11,"name":"uint256","nodeType":"ElementaryTypeName","src":"174:7:0","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"}},"visibility":"internal"}],"src":"173:15:0"},"returnParameters":{"id":16,"nodeType":"ParameterList","parameters":[{"constant":false,"id":15,"mutability":"mutable","name":"","nameLocation":"-1:-1:-1","nodeType":"VariableDeclaration","scope":29,"src":"210:7:0","stateVariable":false,"storageLocation":"default","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"},"typeName":{"id":14,"name":"uint256","nodeType":"ElementaryTypeName","src":"210:7:0","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"}},"visibility":"internal"}],"src":"209:9:0"},"scope":30,"stateMutability":"pure","virtual":false,"visibility":"public"}],"abstract":false,"baseContracts":[],"canonicalName":"LimitedContract","contractDependencies":[],"contractKind":"contract","fullyImplemented":true,"linearizedBaseContracts":[30],"name":"LimitedContract","nameLocation":"66:15:0","scope":31,"usedErrors":[]}],"license":"MIT"},"id":0} \ No newline at end of file diff --git a/tests/contracts/out/OverloadedContract.sol/OverloadedContract.json b/tests/contracts/out/OverloadedContract.sol/OverloadedContract.json index 6d14e1951d4..5c4fc74d7cf 100644 --- a/tests/contracts/out/OverloadedContract.sol/OverloadedContract.json +++ b/tests/contracts/out/OverloadedContract.sol/OverloadedContract.json @@ -1,583 +1 @@ -{ - "abi": [ - { "type": "constructor", "inputs": [], "stateMutability": "nonpayable" }, - { - "type": "function", - "name": "exampleFunction", - "inputs": [{ "name": "", "type": "bytes32", "internalType": "bytes32" }], - "outputs": [{ "name": "", "type": "uint256", "internalType": "uint256" }], - "stateMutability": "pure" - }, - { - "type": "function", - "name": "exampleFunction", - "inputs": [{ "name": "", "type": "uint256", "internalType": "uint256" }], - "outputs": [{ "name": "", "type": "string", "internalType": "string" }], - "stateMutability": "pure" - }, - { - "type": "function", - "name": "exampleFunction", - "inputs": [{ "name": "", "type": "string", "internalType": "string" }], - "outputs": [{ "name": "", "type": "string", "internalType": "string" }], - "stateMutability": "pure" - }, - { "type": "event", "name": "Trigger", "inputs": [], "anonymous": false } - ], - "bytecode": { - "object": "0x608060405234801561001057600080fd5b506040517f3d53a39550e04688065827f3bb86584cb007ab9ebca7ebd528e7301c9c31eb5d90600090a1610252806100496000396000f3fe608060405234801561001057600080fd5b50600436106100415760003560e01c806331870cbc14610046578063934bc29d1461006e578063bc2d73ba146100b5575b600080fd5b61005b6100543660046100ee565b5061010090565b6040519081526020015b60405180910390f35b6100a861007c3660046100ee565b5060408051808201909152601181527075696e74323536202d3e20737472696e6760781b602082015290565b6040516100659190610107565b6100a86100c336600461016b565b5060408051808201909152601081526f737472696e67202d3e20737472696e6760801b602082015290565b60006020828403121561010057600080fd5b5035919050565b600060208083528351808285015260005b8181101561013457858101830151858201604001528201610118565b506000604082860101526040601f19601f8301168501019250505092915050565b634e487b7160e01b600052604160045260246000fd5b60006020828403121561017d57600080fd5b813567ffffffffffffffff8082111561019557600080fd5b818401915084601f8301126101a957600080fd5b8135818111156101bb576101bb610155565b604051601f8201601f19908116603f011681019083821181831017156101e3576101e3610155565b816040528281528760208487010111156101fc57600080fd5b82602086016020830137600092810160200192909252509594505050505056fea2646970667358221220d7abec9e326f4c25cc8f45f8ee265c92b595b8cf7f1d5a1d863735dee11ed7d064736f6c63430008130033", - "sourceMap": "57:457:1:-:0;;;113:45;;;;;;;;;-1:-1:-1;142:9:1;;;;;;;57:457;;;;;;", - "linkReferences": {} - }, - "deployedBytecode": { - "object": "0x608060405234801561001057600080fd5b50600436106100415760003560e01c806331870cbc14610046578063934bc29d1461006e578063bc2d73ba146100b5575b600080fd5b61005b6100543660046100ee565b5061010090565b6040519081526020015b60405180910390f35b6100a861007c3660046100ee565b5060408051808201909152601181527075696e74323536202d3e20737472696e6760781b602082015290565b6040516100659190610107565b6100a86100c336600461016b565b5060408051808201909152601081526f737472696e67202d3e20737472696e6760801b602082015290565b60006020828403121561010057600080fd5b5035919050565b600060208083528351808285015260005b8181101561013457858101830151858201604001528201610118565b506000604082860101526040601f19601f8301168501019250505092915050565b634e487b7160e01b600052604160045260246000fd5b60006020828403121561017d57600080fd5b813567ffffffffffffffff8082111561019557600080fd5b818401915084601f8301126101a957600080fd5b8135818111156101bb576101bb610155565b604051601f8201601f19908116603f011681019083821181831017156101e3576101e3610155565b816040528281528760208487010111156101fc57600080fd5b82602086016020830137600092810160200192909252509594505050505056fea2646970667358221220d7abec9e326f4c25cc8f45f8ee265c92b595b8cf7f1d5a1d863735dee11ed7d064736f6c63430008130033", - "sourceMap": "57:457:1:-:0;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;421:91;;;;;;:::i;:::-;-1:-1:-1;502:3:1;;421:91;;;;345:25:4;;;333:2;318:18;421:91:1;;;;;;;;302:113;;;;;;:::i;:::-;-1:-1:-1;382:26:1;;;;;;;;;;;;-1:-1:-1;;;382:26:1;;;;;302:113;;;;;;;;:::i;164:132::-;;;;;;:::i;:::-;-1:-1:-1;264:25:1;;;;;;;;;;;;-1:-1:-1;;;264:25:1;;;;;164:132;14:180:4;73:6;126:2;114:9;105:7;101:23;97:32;94:52;;;142:1;139;132:12;94:52;-1:-1:-1;165:23:4;;14:180;-1:-1:-1;14:180:4:o;566:548::-;678:4;707:2;736;725:9;718:21;768:6;762:13;811:6;806:2;795:9;791:18;784:34;836:1;846:140;860:6;857:1;854:13;846:140;;;955:14;;;951:23;;945:30;921:17;;;940:2;917:26;910:66;875:10;;846:140;;;850:3;1035:1;1030:2;1021:6;1010:9;1006:22;1002:31;995:42;1105:2;1098;1094:7;1089:2;1081:6;1077:15;1073:29;1062:9;1058:45;1054:54;1046:62;;;;566:548;;;;:::o;1119:127::-;1180:10;1175:3;1171:20;1168:1;1161:31;1211:4;1208:1;1201:15;1235:4;1232:1;1225:15;1251:922;1320:6;1373:2;1361:9;1352:7;1348:23;1344:32;1341:52;;;1389:1;1386;1379:12;1341:52;1429:9;1416:23;1458:18;1499:2;1491:6;1488:14;1485:34;;;1515:1;1512;1505:12;1485:34;1553:6;1542:9;1538:22;1528:32;;1598:7;1591:4;1587:2;1583:13;1579:27;1569:55;;1620:1;1617;1610:12;1569:55;1656:2;1643:16;1678:2;1674;1671:10;1668:36;;;1684:18;;:::i;:::-;1759:2;1753:9;1727:2;1813:13;;-1:-1:-1;;1809:22:4;;;1833:2;1805:31;1801:40;1789:53;;;1857:18;;;1877:22;;;1854:46;1851:72;;;1903:18;;:::i;:::-;1943:10;1939:2;1932:22;1978:2;1970:6;1963:18;2018:7;2013:2;2008;2004;2000:11;1996:20;1993:33;1990:53;;;2039:1;2036;2029:12;1990:53;2095:2;2090;2086;2082:11;2077:2;2069:6;2065:15;2052:46;2140:1;2118:15;;;2135:2;2114:24;2107:35;;;;-1:-1:-1;2122:6:4;1251:922;-1:-1:-1;;;;;1251:922:4:o", - "linkReferences": {} - }, - "methodIdentifiers": { - "exampleFunction(bytes32)": "31870cbc", - "exampleFunction(string)": "bc2d73ba", - "exampleFunction(uint256)": "934bc29d" - }, - "rawMetadata": "{\"compiler\":{\"version\":\"0.8.19+commit.7dd6d404\"},\"language\":\"Solidity\",\"output\":{\"abi\":[{\"inputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"constructor\"},{\"anonymous\":false,\"inputs\":[],\"name\":\"Trigger\",\"type\":\"event\"},{\"inputs\":[{\"internalType\":\"bytes32\",\"name\":\"\",\"type\":\"bytes32\"}],\"name\":\"exampleFunction\",\"outputs\":[{\"internalType\":\"uint256\",\"name\":\"\",\"type\":\"uint256\"}],\"stateMutability\":\"pure\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"uint256\",\"name\":\"\",\"type\":\"uint256\"}],\"name\":\"exampleFunction\",\"outputs\":[{\"internalType\":\"string\",\"name\":\"\",\"type\":\"string\"}],\"stateMutability\":\"pure\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"string\",\"name\":\"\",\"type\":\"string\"}],\"name\":\"exampleFunction\",\"outputs\":[{\"internalType\":\"string\",\"name\":\"\",\"type\":\"string\"}],\"stateMutability\":\"pure\",\"type\":\"function\"}],\"devdoc\":{\"kind\":\"dev\",\"methods\":{},\"version\":1},\"userdoc\":{\"kind\":\"user\",\"methods\":{},\"version\":1}},\"settings\":{\"compilationTarget\":{\"src/OverloadedContract.sol\":\"OverloadedContract\"},\"evmVersion\":\"paris\",\"libraries\":{},\"metadata\":{\"bytecodeHash\":\"ipfs\"},\"optimizer\":{\"enabled\":true,\"runs\":200},\"remappings\":[]},\"sources\":{\"src/OverloadedContract.sol\":{\"keccak256\":\"0xc6734859398f3be8468d6e6c7fd8b03a52243223799ce17d5e4ab9d9aca1fc45\",\"license\":\"MIT\",\"urls\":[\"bzz-raw://2c860b9cd7d0a2086e164ce38a2aa24a5b7f681bb575a5a656f732d3742761be\",\"dweb:/ipfs/QmPwazDSTPrNpVrRY2vunso7VXunWp5dn1641TzxK9eZfe\"]}},\"version\":1}", - "metadata": { - "compiler": { "version": "0.8.19+commit.7dd6d404" }, - "language": "Solidity", - "output": { - "abi": [ - { - "inputs": [], - "stateMutability": "nonpayable", - "type": "constructor" - }, - { - "inputs": [], - "type": "event", - "name": "Trigger", - "anonymous": false - }, - { - "inputs": [ - { "internalType": "bytes32", "name": "", "type": "bytes32" } - ], - "stateMutability": "pure", - "type": "function", - "name": "exampleFunction", - "outputs": [ - { "internalType": "uint256", "name": "", "type": "uint256" } - ] - }, - { - "inputs": [ - { "internalType": "uint256", "name": "", "type": "uint256" } - ], - "stateMutability": "pure", - "type": "function", - "name": "exampleFunction", - "outputs": [ - { "internalType": "string", "name": "", "type": "string" } - ] - }, - { - "inputs": [ - { "internalType": "string", "name": "", "type": "string" } - ], - "stateMutability": "pure", - "type": "function", - "name": "exampleFunction", - "outputs": [ - { "internalType": "string", "name": "", "type": "string" } - ] - } - ], - "devdoc": { "kind": "dev", "methods": {}, "version": 1 }, - "userdoc": { "kind": "user", "methods": {}, "version": 1 } - }, - "settings": { - "remappings": [], - "optimizer": { "enabled": true, "runs": 200 }, - "metadata": { "bytecodeHash": "ipfs" }, - "compilationTarget": { - "src/OverloadedContract.sol": "OverloadedContract" - }, - "evmVersion": "paris", - "libraries": {} - }, - "sources": { - "src/OverloadedContract.sol": { - "keccak256": "0xc6734859398f3be8468d6e6c7fd8b03a52243223799ce17d5e4ab9d9aca1fc45", - "urls": [ - "bzz-raw://2c860b9cd7d0a2086e164ce38a2aa24a5b7f681bb575a5a656f732d3742761be", - "dweb:/ipfs/QmPwazDSTPrNpVrRY2vunso7VXunWp5dn1641TzxK9eZfe" - ], - "license": "MIT" - } - }, - "version": 1 - }, - "ast": { - "absolutePath": "src/OverloadedContract.sol", - "id": 73, - "exportedSymbols": { "OverloadedContract": [72] }, - "nodeType": "SourceUnit", - "src": "32:483:1", - "nodes": [ - { - "id": 32, - "nodeType": "PragmaDirective", - "src": "32:23:1", - "nodes": [], - "literals": ["solidity", "^", "0.8", ".0"] - }, - { - "id": 72, - "nodeType": "ContractDefinition", - "src": "57:457:1", - "nodes": [ - { - "id": 34, - "nodeType": "EventDefinition", - "src": "91:16:1", - "nodes": [], - "anonymous": false, - "eventSelector": "3d53a39550e04688065827f3bb86584cb007ab9ebca7ebd528e7301c9c31eb5d", - "name": "Trigger", - "nameLocation": "97:7:1", - "parameters": { - "id": 33, - "nodeType": "ParameterList", - "parameters": [], - "src": "104:2:1" - } - }, - { - "id": 41, - "nodeType": "FunctionDefinition", - "src": "113:45:1", - "nodes": [], - "body": { - "id": 40, - "nodeType": "Block", - "src": "127:31:1", - "nodes": [], - "statements": [ - { - "eventCall": { - "arguments": [], - "expression": { - "argumentTypes": [], - "id": 37, - "name": "Trigger", - "nodeType": "Identifier", - "overloadedDeclarations": [], - "referencedDeclaration": 34, - "src": "142:7:1", - "typeDescriptions": { - "typeIdentifier": "t_function_event_nonpayable$__$returns$__$", - "typeString": "function ()" - } - }, - "id": 38, - "isConstant": false, - "isLValue": false, - "isPure": false, - "kind": "functionCall", - "lValueRequested": false, - "nameLocations": [], - "names": [], - "nodeType": "FunctionCall", - "src": "142:9:1", - "tryCall": false, - "typeDescriptions": { - "typeIdentifier": "t_tuple$__$", - "typeString": "tuple()" - } - }, - "id": 39, - "nodeType": "EmitStatement", - "src": "137:14:1" - } - ] - }, - "implemented": true, - "kind": "constructor", - "modifiers": [], - "name": "", - "nameLocation": "-1:-1:-1", - "parameters": { - "id": 35, - "nodeType": "ParameterList", - "parameters": [], - "src": "124:2:1" - }, - "returnParameters": { - "id": 36, - "nodeType": "ParameterList", - "parameters": [], - "src": "127:0:1" - }, - "scope": 72, - "stateMutability": "nonpayable", - "virtual": false, - "visibility": "public" - }, - { - "id": 51, - "nodeType": "FunctionDefinition", - "src": "164:132:1", - "nodes": [], - "body": { - "id": 50, - "nodeType": "Block", - "src": "254:42:1", - "nodes": [], - "statements": [ - { - "expression": { - "hexValue": "737472696e67202d3e20737472696e67", - "id": 48, - "isConstant": false, - "isLValue": false, - "isPure": true, - "kind": "string", - "lValueRequested": false, - "nodeType": "Literal", - "src": "271:18:1", - "typeDescriptions": { - "typeIdentifier": "t_stringliteral_a675d5271e48bf44b2d3a2abcbe5392d4a4159912e3d2d332a49139a8b50d538", - "typeString": "literal_string \"string -> string\"" - }, - "value": "string -> string" - }, - "functionReturnParameters": 47, - "id": 49, - "nodeType": "Return", - "src": "264:25:1" - } - ] - }, - "functionSelector": "bc2d73ba", - "implemented": true, - "kind": "function", - "modifiers": [], - "name": "exampleFunction", - "nameLocation": "173:15:1", - "parameters": { - "id": 44, - "nodeType": "ParameterList", - "parameters": [ - { - "constant": false, - "id": 43, - "mutability": "mutable", - "name": "", - "nameLocation": "-1:-1:-1", - "nodeType": "VariableDeclaration", - "scope": 51, - "src": "198:13:1", - "stateVariable": false, - "storageLocation": "memory", - "typeDescriptions": { - "typeIdentifier": "t_string_memory_ptr", - "typeString": "string" - }, - "typeName": { - "id": 42, - "name": "string", - "nodeType": "ElementaryTypeName", - "src": "198:6:1", - "typeDescriptions": { - "typeIdentifier": "t_string_storage_ptr", - "typeString": "string" - } - }, - "visibility": "internal" - } - ], - "src": "188:29:1" - }, - "returnParameters": { - "id": 47, - "nodeType": "ParameterList", - "parameters": [ - { - "constant": false, - "id": 46, - "mutability": "mutable", - "name": "", - "nameLocation": "-1:-1:-1", - "nodeType": "VariableDeclaration", - "scope": 51, - "src": "239:13:1", - "stateVariable": false, - "storageLocation": "memory", - "typeDescriptions": { - "typeIdentifier": "t_string_memory_ptr", - "typeString": "string" - }, - "typeName": { - "id": 45, - "name": "string", - "nodeType": "ElementaryTypeName", - "src": "239:6:1", - "typeDescriptions": { - "typeIdentifier": "t_string_storage_ptr", - "typeString": "string" - } - }, - "visibility": "internal" - } - ], - "src": "238:15:1" - }, - "scope": 72, - "stateMutability": "pure", - "virtual": false, - "visibility": "public" - }, - { - "id": 61, - "nodeType": "FunctionDefinition", - "src": "302:113:1", - "nodes": [], - "body": { - "id": 60, - "nodeType": "Block", - "src": "372:43:1", - "nodes": [], - "statements": [ - { - "expression": { - "hexValue": "75696e74323536202d3e20737472696e67", - "id": 58, - "isConstant": false, - "isLValue": false, - "isPure": true, - "kind": "string", - "lValueRequested": false, - "nodeType": "Literal", - "src": "389:19:1", - "typeDescriptions": { - "typeIdentifier": "t_stringliteral_56541f37aba8911ed7b3fc4c5c74297515444b42d7c1b74ff1c1abc66e2d65cd", - "typeString": "literal_string \"uint256 -> string\"" - }, - "value": "uint256 -> string" - }, - "functionReturnParameters": 57, - "id": 59, - "nodeType": "Return", - "src": "382:26:1" - } - ] - }, - "functionSelector": "934bc29d", - "implemented": true, - "kind": "function", - "modifiers": [], - "name": "exampleFunction", - "nameLocation": "311:15:1", - "parameters": { - "id": 54, - "nodeType": "ParameterList", - "parameters": [ - { - "constant": false, - "id": 53, - "mutability": "mutable", - "name": "", - "nameLocation": "-1:-1:-1", - "nodeType": "VariableDeclaration", - "scope": 61, - "src": "327:7:1", - "stateVariable": false, - "storageLocation": "default", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - }, - "typeName": { - "id": 52, - "name": "uint256", - "nodeType": "ElementaryTypeName", - "src": "327:7:1", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - } - }, - "visibility": "internal" - } - ], - "src": "326:9:1" - }, - "returnParameters": { - "id": 57, - "nodeType": "ParameterList", - "parameters": [ - { - "constant": false, - "id": 56, - "mutability": "mutable", - "name": "", - "nameLocation": "-1:-1:-1", - "nodeType": "VariableDeclaration", - "scope": 61, - "src": "357:13:1", - "stateVariable": false, - "storageLocation": "memory", - "typeDescriptions": { - "typeIdentifier": "t_string_memory_ptr", - "typeString": "string" - }, - "typeName": { - "id": 55, - "name": "string", - "nodeType": "ElementaryTypeName", - "src": "357:6:1", - "typeDescriptions": { - "typeIdentifier": "t_string_storage_ptr", - "typeString": "string" - } - }, - "visibility": "internal" - } - ], - "src": "356:15:1" - }, - "scope": 72, - "stateMutability": "pure", - "virtual": false, - "visibility": "public" - }, - { - "id": 71, - "nodeType": "FunctionDefinition", - "src": "421:91:1", - "nodes": [], - "body": { - "id": 70, - "nodeType": "Block", - "src": "485:27:1", - "nodes": [], - "statements": [ - { - "expression": { - "hexValue": "323536", - "id": 68, - "isConstant": false, - "isLValue": false, - "isPure": true, - "kind": "number", - "lValueRequested": false, - "nodeType": "Literal", - "src": "502:3:1", - "typeDescriptions": { - "typeIdentifier": "t_rational_256_by_1", - "typeString": "int_const 256" - }, - "value": "256" - }, - "functionReturnParameters": 67, - "id": 69, - "nodeType": "Return", - "src": "495:10:1" - } - ] - }, - "functionSelector": "31870cbc", - "implemented": true, - "kind": "function", - "modifiers": [], - "name": "exampleFunction", - "nameLocation": "430:15:1", - "parameters": { - "id": 64, - "nodeType": "ParameterList", - "parameters": [ - { - "constant": false, - "id": 63, - "mutability": "mutable", - "name": "", - "nameLocation": "-1:-1:-1", - "nodeType": "VariableDeclaration", - "scope": 71, - "src": "446:7:1", - "stateVariable": false, - "storageLocation": "default", - "typeDescriptions": { - "typeIdentifier": "t_bytes32", - "typeString": "bytes32" - }, - "typeName": { - "id": 62, - "name": "bytes32", - "nodeType": "ElementaryTypeName", - "src": "446:7:1", - "typeDescriptions": { - "typeIdentifier": "t_bytes32", - "typeString": "bytes32" - } - }, - "visibility": "internal" - } - ], - "src": "445:9:1" - }, - "returnParameters": { - "id": 67, - "nodeType": "ParameterList", - "parameters": [ - { - "constant": false, - "id": 66, - "mutability": "mutable", - "name": "", - "nameLocation": "-1:-1:-1", - "nodeType": "VariableDeclaration", - "scope": 71, - "src": "476:7:1", - "stateVariable": false, - "storageLocation": "default", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - }, - "typeName": { - "id": 65, - "name": "uint256", - "nodeType": "ElementaryTypeName", - "src": "476:7:1", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - } - }, - "visibility": "internal" - } - ], - "src": "475:9:1" - }, - "scope": 72, - "stateMutability": "pure", - "virtual": false, - "visibility": "public" - } - ], - "abstract": false, - "baseContracts": [], - "canonicalName": "OverloadedContract", - "contractDependencies": [], - "contractKind": "contract", - "fullyImplemented": true, - "linearizedBaseContracts": [72], - "name": "OverloadedContract", - "nameLocation": "66:18:1", - "scope": 73, - "usedErrors": [] - } - ], - "license": "MIT" - }, - "id": 1 -} +{"abi":[{"type":"constructor","inputs":[],"stateMutability":"nonpayable"},{"type":"function","name":"exampleFunction","inputs":[{"name":"","type":"bytes32","internalType":"bytes32"}],"outputs":[{"name":"","type":"uint256","internalType":"uint256"}],"stateMutability":"pure"},{"type":"function","name":"exampleFunction","inputs":[{"name":"","type":"uint256","internalType":"uint256"}],"outputs":[{"name":"","type":"string","internalType":"string"}],"stateMutability":"pure"},{"type":"function","name":"exampleFunction","inputs":[{"name":"","type":"string","internalType":"string"}],"outputs":[{"name":"","type":"string","internalType":"string"}],"stateMutability":"pure"},{"type":"event","name":"Trigger","inputs":[],"anonymous":false}],"bytecode":{"object":"0x608060405234801561001057600080fd5b506040517f3d53a39550e04688065827f3bb86584cb007ab9ebca7ebd528e7301c9c31eb5d90600090a1610252806100496000396000f3fe608060405234801561001057600080fd5b50600436106100415760003560e01c806331870cbc14610046578063934bc29d1461006e578063bc2d73ba146100b5575b600080fd5b61005b6100543660046100ee565b5061010090565b6040519081526020015b60405180910390f35b6100a861007c3660046100ee565b5060408051808201909152601181527075696e74323536202d3e20737472696e6760781b602082015290565b6040516100659190610107565b6100a86100c336600461016b565b5060408051808201909152601081526f737472696e67202d3e20737472696e6760801b602082015290565b60006020828403121561010057600080fd5b5035919050565b600060208083528351808285015260005b8181101561013457858101830151858201604001528201610118565b506000604082860101526040601f19601f8301168501019250505092915050565b634e487b7160e01b600052604160045260246000fd5b60006020828403121561017d57600080fd5b813567ffffffffffffffff8082111561019557600080fd5b818401915084601f8301126101a957600080fd5b8135818111156101bb576101bb610155565b604051601f8201601f19908116603f011681019083821181831017156101e3576101e3610155565b816040528281528760208487010111156101fc57600080fd5b82602086016020830137600092810160200192909252509594505050505056fea2646970667358221220d7abec9e326f4c25cc8f45f8ee265c92b595b8cf7f1d5a1d863735dee11ed7d064736f6c63430008130033","sourceMap":"57:457:1:-:0;;;113:45;;;;;;;;;-1:-1:-1;142:9:1;;;;;;;57:457;;;;;;","linkReferences":{}},"deployedBytecode":{"object":"0x608060405234801561001057600080fd5b50600436106100415760003560e01c806331870cbc14610046578063934bc29d1461006e578063bc2d73ba146100b5575b600080fd5b61005b6100543660046100ee565b5061010090565b6040519081526020015b60405180910390f35b6100a861007c3660046100ee565b5060408051808201909152601181527075696e74323536202d3e20737472696e6760781b602082015290565b6040516100659190610107565b6100a86100c336600461016b565b5060408051808201909152601081526f737472696e67202d3e20737472696e6760801b602082015290565b60006020828403121561010057600080fd5b5035919050565b600060208083528351808285015260005b8181101561013457858101830151858201604001528201610118565b506000604082860101526040601f19601f8301168501019250505092915050565b634e487b7160e01b600052604160045260246000fd5b60006020828403121561017d57600080fd5b813567ffffffffffffffff8082111561019557600080fd5b818401915084601f8301126101a957600080fd5b8135818111156101bb576101bb610155565b604051601f8201601f19908116603f011681019083821181831017156101e3576101e3610155565b816040528281528760208487010111156101fc57600080fd5b82602086016020830137600092810160200192909252509594505050505056fea2646970667358221220d7abec9e326f4c25cc8f45f8ee265c92b595b8cf7f1d5a1d863735dee11ed7d064736f6c63430008130033","sourceMap":"57:457:1:-:0;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;421:91;;;;;;:::i;:::-;-1:-1:-1;502:3:1;;421:91;;;;345:25:4;;;333:2;318:18;421:91:1;;;;;;;;302:113;;;;;;:::i;:::-;-1:-1:-1;382:26:1;;;;;;;;;;;;-1:-1:-1;;;382:26:1;;;;;302:113;;;;;;;;:::i;164:132::-;;;;;;:::i;:::-;-1:-1:-1;264:25:1;;;;;;;;;;;;-1:-1:-1;;;264:25:1;;;;;164:132;14:180:4;73:6;126:2;114:9;105:7;101:23;97:32;94:52;;;142:1;139;132:12;94:52;-1:-1:-1;165:23:4;;14:180;-1:-1:-1;14:180:4:o;566:548::-;678:4;707:2;736;725:9;718:21;768:6;762:13;811:6;806:2;795:9;791:18;784:34;836:1;846:140;860:6;857:1;854:13;846:140;;;955:14;;;951:23;;945:30;921:17;;;940:2;917:26;910:66;875:10;;846:140;;;850:3;1035:1;1030:2;1021:6;1010:9;1006:22;1002:31;995:42;1105:2;1098;1094:7;1089:2;1081:6;1077:15;1073:29;1062:9;1058:45;1054:54;1046:62;;;;566:548;;;;:::o;1119:127::-;1180:10;1175:3;1171:20;1168:1;1161:31;1211:4;1208:1;1201:15;1235:4;1232:1;1225:15;1251:922;1320:6;1373:2;1361:9;1352:7;1348:23;1344:32;1341:52;;;1389:1;1386;1379:12;1341:52;1429:9;1416:23;1458:18;1499:2;1491:6;1488:14;1485:34;;;1515:1;1512;1505:12;1485:34;1553:6;1542:9;1538:22;1528:32;;1598:7;1591:4;1587:2;1583:13;1579:27;1569:55;;1620:1;1617;1610:12;1569:55;1656:2;1643:16;1678:2;1674;1671:10;1668:36;;;1684:18;;:::i;:::-;1759:2;1753:9;1727:2;1813:13;;-1:-1:-1;;1809:22:4;;;1833:2;1805:31;1801:40;1789:53;;;1857:18;;;1877:22;;;1854:46;1851:72;;;1903:18;;:::i;:::-;1943:10;1939:2;1932:22;1978:2;1970:6;1963:18;2018:7;2013:2;2008;2004;2000:11;1996:20;1993:33;1990:53;;;2039:1;2036;2029:12;1990:53;2095:2;2090;2086;2082:11;2077:2;2069:6;2065:15;2052:46;2140:1;2118:15;;;2135:2;2114:24;2107:35;;;;-1:-1:-1;2122:6:4;1251:922;-1:-1:-1;;;;;1251:922:4:o","linkReferences":{}},"methodIdentifiers":{"exampleFunction(bytes32)":"31870cbc","exampleFunction(string)":"bc2d73ba","exampleFunction(uint256)":"934bc29d"},"rawMetadata":"{\"compiler\":{\"version\":\"0.8.19+commit.7dd6d404\"},\"language\":\"Solidity\",\"output\":{\"abi\":[{\"inputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"constructor\"},{\"anonymous\":false,\"inputs\":[],\"name\":\"Trigger\",\"type\":\"event\"},{\"inputs\":[{\"internalType\":\"bytes32\",\"name\":\"\",\"type\":\"bytes32\"}],\"name\":\"exampleFunction\",\"outputs\":[{\"internalType\":\"uint256\",\"name\":\"\",\"type\":\"uint256\"}],\"stateMutability\":\"pure\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"uint256\",\"name\":\"\",\"type\":\"uint256\"}],\"name\":\"exampleFunction\",\"outputs\":[{\"internalType\":\"string\",\"name\":\"\",\"type\":\"string\"}],\"stateMutability\":\"pure\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"string\",\"name\":\"\",\"type\":\"string\"}],\"name\":\"exampleFunction\",\"outputs\":[{\"internalType\":\"string\",\"name\":\"\",\"type\":\"string\"}],\"stateMutability\":\"pure\",\"type\":\"function\"}],\"devdoc\":{\"kind\":\"dev\",\"methods\":{},\"version\":1},\"userdoc\":{\"kind\":\"user\",\"methods\":{},\"version\":1}},\"settings\":{\"compilationTarget\":{\"src/OverloadedContract.sol\":\"OverloadedContract\"},\"evmVersion\":\"paris\",\"libraries\":{},\"metadata\":{\"bytecodeHash\":\"ipfs\"},\"optimizer\":{\"enabled\":true,\"runs\":200},\"remappings\":[]},\"sources\":{\"src/OverloadedContract.sol\":{\"keccak256\":\"0xc6734859398f3be8468d6e6c7fd8b03a52243223799ce17d5e4ab9d9aca1fc45\",\"license\":\"MIT\",\"urls\":[\"bzz-raw://2c860b9cd7d0a2086e164ce38a2aa24a5b7f681bb575a5a656f732d3742761be\",\"dweb:/ipfs/QmPwazDSTPrNpVrRY2vunso7VXunWp5dn1641TzxK9eZfe\"]}},\"version\":1}","metadata":{"compiler":{"version":"0.8.19+commit.7dd6d404"},"language":"Solidity","output":{"abi":[{"inputs":[],"stateMutability":"nonpayable","type":"constructor"},{"inputs":[],"type":"event","name":"Trigger","anonymous":false},{"inputs":[{"internalType":"bytes32","name":"","type":"bytes32"}],"stateMutability":"pure","type":"function","name":"exampleFunction","outputs":[{"internalType":"uint256","name":"","type":"uint256"}]},{"inputs":[{"internalType":"uint256","name":"","type":"uint256"}],"stateMutability":"pure","type":"function","name":"exampleFunction","outputs":[{"internalType":"string","name":"","type":"string"}]},{"inputs":[{"internalType":"string","name":"","type":"string"}],"stateMutability":"pure","type":"function","name":"exampleFunction","outputs":[{"internalType":"string","name":"","type":"string"}]}],"devdoc":{"kind":"dev","methods":{},"version":1},"userdoc":{"kind":"user","methods":{},"version":1}},"settings":{"remappings":[],"optimizer":{"enabled":true,"runs":200},"metadata":{"bytecodeHash":"ipfs"},"compilationTarget":{"src/OverloadedContract.sol":"OverloadedContract"},"evmVersion":"paris","libraries":{}},"sources":{"src/OverloadedContract.sol":{"keccak256":"0xc6734859398f3be8468d6e6c7fd8b03a52243223799ce17d5e4ab9d9aca1fc45","urls":["bzz-raw://2c860b9cd7d0a2086e164ce38a2aa24a5b7f681bb575a5a656f732d3742761be","dweb:/ipfs/QmPwazDSTPrNpVrRY2vunso7VXunWp5dn1641TzxK9eZfe"],"license":"MIT"}},"version":1},"ast":{"absolutePath":"src/OverloadedContract.sol","id":73,"exportedSymbols":{"OverloadedContract":[72]},"nodeType":"SourceUnit","src":"32:483:1","nodes":[{"id":32,"nodeType":"PragmaDirective","src":"32:23:1","nodes":[],"literals":["solidity","^","0.8",".0"]},{"id":72,"nodeType":"ContractDefinition","src":"57:457:1","nodes":[{"id":34,"nodeType":"EventDefinition","src":"91:16:1","nodes":[],"anonymous":false,"eventSelector":"3d53a39550e04688065827f3bb86584cb007ab9ebca7ebd528e7301c9c31eb5d","name":"Trigger","nameLocation":"97:7:1","parameters":{"id":33,"nodeType":"ParameterList","parameters":[],"src":"104:2:1"}},{"id":41,"nodeType":"FunctionDefinition","src":"113:45:1","nodes":[],"body":{"id":40,"nodeType":"Block","src":"127:31:1","nodes":[],"statements":[{"eventCall":{"arguments":[],"expression":{"argumentTypes":[],"id":37,"name":"Trigger","nodeType":"Identifier","overloadedDeclarations":[],"referencedDeclaration":34,"src":"142:7:1","typeDescriptions":{"typeIdentifier":"t_function_event_nonpayable$__$returns$__$","typeString":"function ()"}},"id":38,"isConstant":false,"isLValue":false,"isPure":false,"kind":"functionCall","lValueRequested":false,"nameLocations":[],"names":[],"nodeType":"FunctionCall","src":"142:9:1","tryCall":false,"typeDescriptions":{"typeIdentifier":"t_tuple$__$","typeString":"tuple()"}},"id":39,"nodeType":"EmitStatement","src":"137:14:1"}]},"implemented":true,"kind":"constructor","modifiers":[],"name":"","nameLocation":"-1:-1:-1","parameters":{"id":35,"nodeType":"ParameterList","parameters":[],"src":"124:2:1"},"returnParameters":{"id":36,"nodeType":"ParameterList","parameters":[],"src":"127:0:1"},"scope":72,"stateMutability":"nonpayable","virtual":false,"visibility":"public"},{"id":51,"nodeType":"FunctionDefinition","src":"164:132:1","nodes":[],"body":{"id":50,"nodeType":"Block","src":"254:42:1","nodes":[],"statements":[{"expression":{"hexValue":"737472696e67202d3e20737472696e67","id":48,"isConstant":false,"isLValue":false,"isPure":true,"kind":"string","lValueRequested":false,"nodeType":"Literal","src":"271:18:1","typeDescriptions":{"typeIdentifier":"t_stringliteral_a675d5271e48bf44b2d3a2abcbe5392d4a4159912e3d2d332a49139a8b50d538","typeString":"literal_string \"string -> string\""},"value":"string -> string"},"functionReturnParameters":47,"id":49,"nodeType":"Return","src":"264:25:1"}]},"functionSelector":"bc2d73ba","implemented":true,"kind":"function","modifiers":[],"name":"exampleFunction","nameLocation":"173:15:1","parameters":{"id":44,"nodeType":"ParameterList","parameters":[{"constant":false,"id":43,"mutability":"mutable","name":"","nameLocation":"-1:-1:-1","nodeType":"VariableDeclaration","scope":51,"src":"198:13:1","stateVariable":false,"storageLocation":"memory","typeDescriptions":{"typeIdentifier":"t_string_memory_ptr","typeString":"string"},"typeName":{"id":42,"name":"string","nodeType":"ElementaryTypeName","src":"198:6:1","typeDescriptions":{"typeIdentifier":"t_string_storage_ptr","typeString":"string"}},"visibility":"internal"}],"src":"188:29:1"},"returnParameters":{"id":47,"nodeType":"ParameterList","parameters":[{"constant":false,"id":46,"mutability":"mutable","name":"","nameLocation":"-1:-1:-1","nodeType":"VariableDeclaration","scope":51,"src":"239:13:1","stateVariable":false,"storageLocation":"memory","typeDescriptions":{"typeIdentifier":"t_string_memory_ptr","typeString":"string"},"typeName":{"id":45,"name":"string","nodeType":"ElementaryTypeName","src":"239:6:1","typeDescriptions":{"typeIdentifier":"t_string_storage_ptr","typeString":"string"}},"visibility":"internal"}],"src":"238:15:1"},"scope":72,"stateMutability":"pure","virtual":false,"visibility":"public"},{"id":61,"nodeType":"FunctionDefinition","src":"302:113:1","nodes":[],"body":{"id":60,"nodeType":"Block","src":"372:43:1","nodes":[],"statements":[{"expression":{"hexValue":"75696e74323536202d3e20737472696e67","id":58,"isConstant":false,"isLValue":false,"isPure":true,"kind":"string","lValueRequested":false,"nodeType":"Literal","src":"389:19:1","typeDescriptions":{"typeIdentifier":"t_stringliteral_56541f37aba8911ed7b3fc4c5c74297515444b42d7c1b74ff1c1abc66e2d65cd","typeString":"literal_string \"uint256 -> string\""},"value":"uint256 -> string"},"functionReturnParameters":57,"id":59,"nodeType":"Return","src":"382:26:1"}]},"functionSelector":"934bc29d","implemented":true,"kind":"function","modifiers":[],"name":"exampleFunction","nameLocation":"311:15:1","parameters":{"id":54,"nodeType":"ParameterList","parameters":[{"constant":false,"id":53,"mutability":"mutable","name":"","nameLocation":"-1:-1:-1","nodeType":"VariableDeclaration","scope":61,"src":"327:7:1","stateVariable":false,"storageLocation":"default","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"},"typeName":{"id":52,"name":"uint256","nodeType":"ElementaryTypeName","src":"327:7:1","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"}},"visibility":"internal"}],"src":"326:9:1"},"returnParameters":{"id":57,"nodeType":"ParameterList","parameters":[{"constant":false,"id":56,"mutability":"mutable","name":"","nameLocation":"-1:-1:-1","nodeType":"VariableDeclaration","scope":61,"src":"357:13:1","stateVariable":false,"storageLocation":"memory","typeDescriptions":{"typeIdentifier":"t_string_memory_ptr","typeString":"string"},"typeName":{"id":55,"name":"string","nodeType":"ElementaryTypeName","src":"357:6:1","typeDescriptions":{"typeIdentifier":"t_string_storage_ptr","typeString":"string"}},"visibility":"internal"}],"src":"356:15:1"},"scope":72,"stateMutability":"pure","virtual":false,"visibility":"public"},{"id":71,"nodeType":"FunctionDefinition","src":"421:91:1","nodes":[],"body":{"id":70,"nodeType":"Block","src":"485:27:1","nodes":[],"statements":[{"expression":{"hexValue":"323536","id":68,"isConstant":false,"isLValue":false,"isPure":true,"kind":"number","lValueRequested":false,"nodeType":"Literal","src":"502:3:1","typeDescriptions":{"typeIdentifier":"t_rational_256_by_1","typeString":"int_const 256"},"value":"256"},"functionReturnParameters":67,"id":69,"nodeType":"Return","src":"495:10:1"}]},"functionSelector":"31870cbc","implemented":true,"kind":"function","modifiers":[],"name":"exampleFunction","nameLocation":"430:15:1","parameters":{"id":64,"nodeType":"ParameterList","parameters":[{"constant":false,"id":63,"mutability":"mutable","name":"","nameLocation":"-1:-1:-1","nodeType":"VariableDeclaration","scope":71,"src":"446:7:1","stateVariable":false,"storageLocation":"default","typeDescriptions":{"typeIdentifier":"t_bytes32","typeString":"bytes32"},"typeName":{"id":62,"name":"bytes32","nodeType":"ElementaryTypeName","src":"446:7:1","typeDescriptions":{"typeIdentifier":"t_bytes32","typeString":"bytes32"}},"visibility":"internal"}],"src":"445:9:1"},"returnParameters":{"id":67,"nodeType":"ParameterList","parameters":[{"constant":false,"id":66,"mutability":"mutable","name":"","nameLocation":"-1:-1:-1","nodeType":"VariableDeclaration","scope":71,"src":"476:7:1","stateVariable":false,"storageLocation":"default","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"},"typeName":{"id":65,"name":"uint256","nodeType":"ElementaryTypeName","src":"476:7:1","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"}},"visibility":"internal"}],"src":"475:9:1"},"scope":72,"stateMutability":"pure","virtual":false,"visibility":"public"}],"abstract":false,"baseContracts":[],"canonicalName":"OverloadedContract","contractDependencies":[],"contractKind":"contract","fullyImplemented":true,"linearizedBaseContracts":[72],"name":"OverloadedContract","nameLocation":"66:18:1","scope":73,"usedErrors":[]}],"license":"MIT"},"id":1} \ No newline at end of file diff --git a/tests/contracts/out/RevertingContract.sol/RevertingContract.json b/tests/contracts/out/RevertingContract.sol/RevertingContract.json index e925485a006..4c447a28729 100644 --- a/tests/contracts/out/RevertingContract.sol/RevertingContract.json +++ b/tests/contracts/out/RevertingContract.sol/RevertingContract.json @@ -1,450 +1 @@ -{ - "abi": [ - { "type": "constructor", "inputs": [], "stateMutability": "nonpayable" }, - { - "type": "function", - "name": "inc", - "inputs": [ - { "name": "value", "type": "uint256", "internalType": "uint256" } - ], - "outputs": [{ "name": "", "type": "uint256", "internalType": "uint256" }], - "stateMutability": "pure" - }, - { "type": "event", "name": "Trigger", "inputs": [], "anonymous": false } - ], - "bytecode": { - "object": "0x608060405234801561001057600080fd5b506040517f3d53a39550e04688065827f3bb86584cb007ab9ebca7ebd528e7301c9c31eb5d90600090a1610120806100496000396000f3fe6080604052348015600f57600080fd5b506004361060285760003560e01c8063812600df14602d575b600080fd5b603c603836600460b2565b604e565b60405190815260200160405180910390f35b6000600a821060a35760405162461bcd60e51b815260206004820152601b60248201527f63616e206f6e6c792068616e646c652076616c756573203c2031300000000000604482015260640160405180910390fd5b60ac82600160ca565b92915050565b60006020828403121560c357600080fd5b5035919050565b8082018082111560ac57634e487b7160e01b600052601160045260246000fdfea2646970667358221220ad875f460a402063be4ff63412a90d65fa24398c907d52e2a0926375442cb6f064736f6c63430008130033", - "sourceMap": "57:259:2:-:0;;;112:45;;;;;;;;;-1:-1:-1;141:9:2;;;;;;;57:259;;;;;;", - "linkReferences": {} - }, - "deployedBytecode": { - "object": "0x6080604052348015600f57600080fd5b506004361060285760003560e01c8063812600df14602d575b600080fd5b603c603836600460b2565b604e565b60405190815260200160405180910390f35b6000600a821060a35760405162461bcd60e51b815260206004820152601b60248201527f63616e206f6e6c792068616e646c652076616c756573203c2031300000000000604482015260640160405180910390fd5b60ac82600160ca565b92915050565b60006020828403121560c357600080fd5b5035919050565b8082018082111560ac57634e487b7160e01b600052601160045260246000fdfea2646970667358221220ad875f460a402063be4ff63412a90d65fa24398c907d52e2a0926375442cb6f064736f6c63430008130033", - "sourceMap": "57:259:2:-:0;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;163:151;;;;;;:::i;:::-;;:::i;:::-;;;345:25:4;;;333:2;318:18;163:151:2;;;;;;;;212:7;247:2;239:5;:10;231:50;;;;-1:-1:-1;;;231:50:2;;583:2:4;231:50:2;;;565:21:4;622:2;602:18;;;595:30;661:29;641:18;;;634:57;708:18;;231:50:2;;;;;;;;298:9;:5;306:1;298:9;:::i;:::-;291:16;163:151;-1:-1:-1;;163:151:2:o;14:180:4:-;73:6;126:2;114:9;105:7;101:23;97:32;94:52;;;142:1;139;132:12;94:52;-1:-1:-1;165:23:4;;14:180;-1:-1:-1;14:180:4:o;737:222::-;802:9;;;823:10;;;820:133;;;875:10;870:3;866:20;863:1;856:31;910:4;907:1;900:15;938:4;935:1;928:15", - "linkReferences": {} - }, - "methodIdentifiers": { "inc(uint256)": "812600df" }, - "rawMetadata": "{\"compiler\":{\"version\":\"0.8.19+commit.7dd6d404\"},\"language\":\"Solidity\",\"output\":{\"abi\":[{\"inputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"constructor\"},{\"anonymous\":false,\"inputs\":[],\"name\":\"Trigger\",\"type\":\"event\"},{\"inputs\":[{\"internalType\":\"uint256\",\"name\":\"value\",\"type\":\"uint256\"}],\"name\":\"inc\",\"outputs\":[{\"internalType\":\"uint256\",\"name\":\"\",\"type\":\"uint256\"}],\"stateMutability\":\"pure\",\"type\":\"function\"}],\"devdoc\":{\"kind\":\"dev\",\"methods\":{},\"version\":1},\"userdoc\":{\"kind\":\"user\",\"methods\":{},\"version\":1}},\"settings\":{\"compilationTarget\":{\"src/RevertingContract.sol\":\"RevertingContract\"},\"evmVersion\":\"paris\",\"libraries\":{},\"metadata\":{\"bytecodeHash\":\"ipfs\"},\"optimizer\":{\"enabled\":true,\"runs\":200},\"remappings\":[]},\"sources\":{\"src/RevertingContract.sol\":{\"keccak256\":\"0xb0ccab460539f08d5f40044fee3e45c26590431d6d08734acde070ca01d84e23\",\"license\":\"MIT\",\"urls\":[\"bzz-raw://3cece4cf2b0d867fb8ef474375f8907df5412056773e20e804e12061d98d057b\",\"dweb:/ipfs/QmeLfvzWjkpA6mCt1FJyNvgKeugzJJTRSBdyDUSBCovyrb\"]}},\"version\":1}", - "metadata": { - "compiler": { "version": "0.8.19+commit.7dd6d404" }, - "language": "Solidity", - "output": { - "abi": [ - { - "inputs": [], - "stateMutability": "nonpayable", - "type": "constructor" - }, - { - "inputs": [], - "type": "event", - "name": "Trigger", - "anonymous": false - }, - { - "inputs": [ - { "internalType": "uint256", "name": "value", "type": "uint256" } - ], - "stateMutability": "pure", - "type": "function", - "name": "inc", - "outputs": [ - { "internalType": "uint256", "name": "", "type": "uint256" } - ] - } - ], - "devdoc": { "kind": "dev", "methods": {}, "version": 1 }, - "userdoc": { "kind": "user", "methods": {}, "version": 1 } - }, - "settings": { - "remappings": [], - "optimizer": { "enabled": true, "runs": 200 }, - "metadata": { "bytecodeHash": "ipfs" }, - "compilationTarget": { "src/RevertingContract.sol": "RevertingContract" }, - "evmVersion": "paris", - "libraries": {} - }, - "sources": { - "src/RevertingContract.sol": { - "keccak256": "0xb0ccab460539f08d5f40044fee3e45c26590431d6d08734acde070ca01d84e23", - "urls": [ - "bzz-raw://3cece4cf2b0d867fb8ef474375f8907df5412056773e20e804e12061d98d057b", - "dweb:/ipfs/QmeLfvzWjkpA6mCt1FJyNvgKeugzJJTRSBdyDUSBCovyrb" - ], - "license": "MIT" - } - }, - "version": 1 - }, - "ast": { - "absolutePath": "src/RevertingContract.sol", - "id": 104, - "exportedSymbols": { "RevertingContract": [103] }, - "nodeType": "SourceUnit", - "src": "32:285:2", - "nodes": [ - { - "id": 74, - "nodeType": "PragmaDirective", - "src": "32:23:2", - "nodes": [], - "literals": ["solidity", "^", "0.8", ".0"] - }, - { - "id": 103, - "nodeType": "ContractDefinition", - "src": "57:259:2", - "nodes": [ - { - "id": 76, - "nodeType": "EventDefinition", - "src": "90:16:2", - "nodes": [], - "anonymous": false, - "eventSelector": "3d53a39550e04688065827f3bb86584cb007ab9ebca7ebd528e7301c9c31eb5d", - "name": "Trigger", - "nameLocation": "96:7:2", - "parameters": { - "id": 75, - "nodeType": "ParameterList", - "parameters": [], - "src": "103:2:2" - } - }, - { - "id": 83, - "nodeType": "FunctionDefinition", - "src": "112:45:2", - "nodes": [], - "body": { - "id": 82, - "nodeType": "Block", - "src": "126:31:2", - "nodes": [], - "statements": [ - { - "eventCall": { - "arguments": [], - "expression": { - "argumentTypes": [], - "id": 79, - "name": "Trigger", - "nodeType": "Identifier", - "overloadedDeclarations": [], - "referencedDeclaration": 76, - "src": "141:7:2", - "typeDescriptions": { - "typeIdentifier": "t_function_event_nonpayable$__$returns$__$", - "typeString": "function ()" - } - }, - "id": 80, - "isConstant": false, - "isLValue": false, - "isPure": false, - "kind": "functionCall", - "lValueRequested": false, - "nameLocations": [], - "names": [], - "nodeType": "FunctionCall", - "src": "141:9:2", - "tryCall": false, - "typeDescriptions": { - "typeIdentifier": "t_tuple$__$", - "typeString": "tuple()" - } - }, - "id": 81, - "nodeType": "EmitStatement", - "src": "136:14:2" - } - ] - }, - "implemented": true, - "kind": "constructor", - "modifiers": [], - "name": "", - "nameLocation": "-1:-1:-1", - "parameters": { - "id": 77, - "nodeType": "ParameterList", - "parameters": [], - "src": "123:2:2" - }, - "returnParameters": { - "id": 78, - "nodeType": "ParameterList", - "parameters": [], - "src": "126:0:2" - }, - "scope": 103, - "stateMutability": "nonpayable", - "virtual": false, - "visibility": "public" - }, - { - "id": 102, - "nodeType": "FunctionDefinition", - "src": "163:151:2", - "nodes": [], - "body": { - "id": 101, - "nodeType": "Block", - "src": "221:93:2", - "nodes": [], - "statements": [ - { - "expression": { - "arguments": [ - { - "commonType": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - }, - "id": 93, - "isConstant": false, - "isLValue": false, - "isPure": false, - "lValueRequested": false, - "leftExpression": { - "id": 91, - "name": "value", - "nodeType": "Identifier", - "overloadedDeclarations": [], - "referencedDeclaration": 85, - "src": "239:5:2", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - } - }, - "nodeType": "BinaryOperation", - "operator": "<", - "rightExpression": { - "hexValue": "3130", - "id": 92, - "isConstant": false, - "isLValue": false, - "isPure": true, - "kind": "number", - "lValueRequested": false, - "nodeType": "Literal", - "src": "247:2:2", - "typeDescriptions": { - "typeIdentifier": "t_rational_10_by_1", - "typeString": "int_const 10" - }, - "value": "10" - }, - "src": "239:10:2", - "typeDescriptions": { - "typeIdentifier": "t_bool", - "typeString": "bool" - } - }, - { - "hexValue": "63616e206f6e6c792068616e646c652076616c756573203c203130", - "id": 94, - "isConstant": false, - "isLValue": false, - "isPure": true, - "kind": "string", - "lValueRequested": false, - "nodeType": "Literal", - "src": "251:29:2", - "typeDescriptions": { - "typeIdentifier": "t_stringliteral_578cd1fc098748633f5d7d46bba428bb3129c1e63324f2b7151699cae5146449", - "typeString": "literal_string \"can only handle values < 10\"" - }, - "value": "can only handle values < 10" - } - ], - "expression": { - "argumentTypes": [ - { "typeIdentifier": "t_bool", "typeString": "bool" }, - { - "typeIdentifier": "t_stringliteral_578cd1fc098748633f5d7d46bba428bb3129c1e63324f2b7151699cae5146449", - "typeString": "literal_string \"can only handle values < 10\"" - } - ], - "id": 90, - "name": "require", - "nodeType": "Identifier", - "overloadedDeclarations": [-18, -18], - "referencedDeclaration": -18, - "src": "231:7:2", - "typeDescriptions": { - "typeIdentifier": "t_function_require_pure$_t_bool_$_t_string_memory_ptr_$returns$__$", - "typeString": "function (bool,string memory) pure" - } - }, - "id": 95, - "isConstant": false, - "isLValue": false, - "isPure": false, - "kind": "functionCall", - "lValueRequested": false, - "nameLocations": [], - "names": [], - "nodeType": "FunctionCall", - "src": "231:50:2", - "tryCall": false, - "typeDescriptions": { - "typeIdentifier": "t_tuple$__$", - "typeString": "tuple()" - } - }, - "id": 96, - "nodeType": "ExpressionStatement", - "src": "231:50:2" - }, - { - "expression": { - "commonType": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - }, - "id": 99, - "isConstant": false, - "isLValue": false, - "isPure": false, - "lValueRequested": false, - "leftExpression": { - "id": 97, - "name": "value", - "nodeType": "Identifier", - "overloadedDeclarations": [], - "referencedDeclaration": 85, - "src": "298:5:2", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - } - }, - "nodeType": "BinaryOperation", - "operator": "+", - "rightExpression": { - "hexValue": "31", - "id": 98, - "isConstant": false, - "isLValue": false, - "isPure": true, - "kind": "number", - "lValueRequested": false, - "nodeType": "Literal", - "src": "306:1:2", - "typeDescriptions": { - "typeIdentifier": "t_rational_1_by_1", - "typeString": "int_const 1" - }, - "value": "1" - }, - "src": "298:9:2", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - } - }, - "functionReturnParameters": 89, - "id": 100, - "nodeType": "Return", - "src": "291:16:2" - } - ] - }, - "functionSelector": "812600df", - "implemented": true, - "kind": "function", - "modifiers": [], - "name": "inc", - "nameLocation": "172:3:2", - "parameters": { - "id": 86, - "nodeType": "ParameterList", - "parameters": [ - { - "constant": false, - "id": 85, - "mutability": "mutable", - "name": "value", - "nameLocation": "184:5:2", - "nodeType": "VariableDeclaration", - "scope": 102, - "src": "176:13:2", - "stateVariable": false, - "storageLocation": "default", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - }, - "typeName": { - "id": 84, - "name": "uint256", - "nodeType": "ElementaryTypeName", - "src": "176:7:2", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - } - }, - "visibility": "internal" - } - ], - "src": "175:15:2" - }, - "returnParameters": { - "id": 89, - "nodeType": "ParameterList", - "parameters": [ - { - "constant": false, - "id": 88, - "mutability": "mutable", - "name": "", - "nameLocation": "-1:-1:-1", - "nodeType": "VariableDeclaration", - "scope": 102, - "src": "212:7:2", - "stateVariable": false, - "storageLocation": "default", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - }, - "typeName": { - "id": 87, - "name": "uint256", - "nodeType": "ElementaryTypeName", - "src": "212:7:2", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - } - }, - "visibility": "internal" - } - ], - "src": "211:9:2" - }, - "scope": 103, - "stateMutability": "pure", - "virtual": false, - "visibility": "public" - } - ], - "abstract": false, - "baseContracts": [], - "canonicalName": "RevertingContract", - "contractDependencies": [], - "contractKind": "contract", - "fullyImplemented": true, - "linearizedBaseContracts": [103], - "name": "RevertingContract", - "nameLocation": "66:17:2", - "scope": 104, - "usedErrors": [] - } - ], - "license": "MIT" - }, - "id": 2 -} +{"abi":[{"type":"constructor","inputs":[],"stateMutability":"nonpayable"},{"type":"function","name":"inc","inputs":[{"name":"value","type":"uint256","internalType":"uint256"}],"outputs":[{"name":"","type":"uint256","internalType":"uint256"}],"stateMutability":"pure"},{"type":"event","name":"Trigger","inputs":[],"anonymous":false}],"bytecode":{"object":"0x608060405234801561001057600080fd5b506040517f3d53a39550e04688065827f3bb86584cb007ab9ebca7ebd528e7301c9c31eb5d90600090a1610120806100496000396000f3fe6080604052348015600f57600080fd5b506004361060285760003560e01c8063812600df14602d575b600080fd5b603c603836600460b2565b604e565b60405190815260200160405180910390f35b6000600a821060a35760405162461bcd60e51b815260206004820152601b60248201527f63616e206f6e6c792068616e646c652076616c756573203c2031300000000000604482015260640160405180910390fd5b60ac82600160ca565b92915050565b60006020828403121560c357600080fd5b5035919050565b8082018082111560ac57634e487b7160e01b600052601160045260246000fdfea2646970667358221220ad875f460a402063be4ff63412a90d65fa24398c907d52e2a0926375442cb6f064736f6c63430008130033","sourceMap":"57:259:2:-:0;;;112:45;;;;;;;;;-1:-1:-1;141:9:2;;;;;;;57:259;;;;;;","linkReferences":{}},"deployedBytecode":{"object":"0x6080604052348015600f57600080fd5b506004361060285760003560e01c8063812600df14602d575b600080fd5b603c603836600460b2565b604e565b60405190815260200160405180910390f35b6000600a821060a35760405162461bcd60e51b815260206004820152601b60248201527f63616e206f6e6c792068616e646c652076616c756573203c2031300000000000604482015260640160405180910390fd5b60ac82600160ca565b92915050565b60006020828403121560c357600080fd5b5035919050565b8082018082111560ac57634e487b7160e01b600052601160045260246000fdfea2646970667358221220ad875f460a402063be4ff63412a90d65fa24398c907d52e2a0926375442cb6f064736f6c63430008130033","sourceMap":"57:259:2:-:0;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;163:151;;;;;;:::i;:::-;;:::i;:::-;;;345:25:4;;;333:2;318:18;163:151:2;;;;;;;;212:7;247:2;239:5;:10;231:50;;;;-1:-1:-1;;;231:50:2;;583:2:4;231:50:2;;;565:21:4;622:2;602:18;;;595:30;661:29;641:18;;;634:57;708:18;;231:50:2;;;;;;;;298:9;:5;306:1;298:9;:::i;:::-;291:16;163:151;-1:-1:-1;;163:151:2:o;14:180:4:-;73:6;126:2;114:9;105:7;101:23;97:32;94:52;;;142:1;139;132:12;94:52;-1:-1:-1;165:23:4;;14:180;-1:-1:-1;14:180:4:o;737:222::-;802:9;;;823:10;;;820:133;;;875:10;870:3;866:20;863:1;856:31;910:4;907:1;900:15;938:4;935:1;928:15","linkReferences":{}},"methodIdentifiers":{"inc(uint256)":"812600df"},"rawMetadata":"{\"compiler\":{\"version\":\"0.8.19+commit.7dd6d404\"},\"language\":\"Solidity\",\"output\":{\"abi\":[{\"inputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"constructor\"},{\"anonymous\":false,\"inputs\":[],\"name\":\"Trigger\",\"type\":\"event\"},{\"inputs\":[{\"internalType\":\"uint256\",\"name\":\"value\",\"type\":\"uint256\"}],\"name\":\"inc\",\"outputs\":[{\"internalType\":\"uint256\",\"name\":\"\",\"type\":\"uint256\"}],\"stateMutability\":\"pure\",\"type\":\"function\"}],\"devdoc\":{\"kind\":\"dev\",\"methods\":{},\"version\":1},\"userdoc\":{\"kind\":\"user\",\"methods\":{},\"version\":1}},\"settings\":{\"compilationTarget\":{\"src/RevertingContract.sol\":\"RevertingContract\"},\"evmVersion\":\"paris\",\"libraries\":{},\"metadata\":{\"bytecodeHash\":\"ipfs\"},\"optimizer\":{\"enabled\":true,\"runs\":200},\"remappings\":[]},\"sources\":{\"src/RevertingContract.sol\":{\"keccak256\":\"0xb0ccab460539f08d5f40044fee3e45c26590431d6d08734acde070ca01d84e23\",\"license\":\"MIT\",\"urls\":[\"bzz-raw://3cece4cf2b0d867fb8ef474375f8907df5412056773e20e804e12061d98d057b\",\"dweb:/ipfs/QmeLfvzWjkpA6mCt1FJyNvgKeugzJJTRSBdyDUSBCovyrb\"]}},\"version\":1}","metadata":{"compiler":{"version":"0.8.19+commit.7dd6d404"},"language":"Solidity","output":{"abi":[{"inputs":[],"stateMutability":"nonpayable","type":"constructor"},{"inputs":[],"type":"event","name":"Trigger","anonymous":false},{"inputs":[{"internalType":"uint256","name":"value","type":"uint256"}],"stateMutability":"pure","type":"function","name":"inc","outputs":[{"internalType":"uint256","name":"","type":"uint256"}]}],"devdoc":{"kind":"dev","methods":{},"version":1},"userdoc":{"kind":"user","methods":{},"version":1}},"settings":{"remappings":[],"optimizer":{"enabled":true,"runs":200},"metadata":{"bytecodeHash":"ipfs"},"compilationTarget":{"src/RevertingContract.sol":"RevertingContract"},"evmVersion":"paris","libraries":{}},"sources":{"src/RevertingContract.sol":{"keccak256":"0xb0ccab460539f08d5f40044fee3e45c26590431d6d08734acde070ca01d84e23","urls":["bzz-raw://3cece4cf2b0d867fb8ef474375f8907df5412056773e20e804e12061d98d057b","dweb:/ipfs/QmeLfvzWjkpA6mCt1FJyNvgKeugzJJTRSBdyDUSBCovyrb"],"license":"MIT"}},"version":1},"ast":{"absolutePath":"src/RevertingContract.sol","id":104,"exportedSymbols":{"RevertingContract":[103]},"nodeType":"SourceUnit","src":"32:285:2","nodes":[{"id":74,"nodeType":"PragmaDirective","src":"32:23:2","nodes":[],"literals":["solidity","^","0.8",".0"]},{"id":103,"nodeType":"ContractDefinition","src":"57:259:2","nodes":[{"id":76,"nodeType":"EventDefinition","src":"90:16:2","nodes":[],"anonymous":false,"eventSelector":"3d53a39550e04688065827f3bb86584cb007ab9ebca7ebd528e7301c9c31eb5d","name":"Trigger","nameLocation":"96:7:2","parameters":{"id":75,"nodeType":"ParameterList","parameters":[],"src":"103:2:2"}},{"id":83,"nodeType":"FunctionDefinition","src":"112:45:2","nodes":[],"body":{"id":82,"nodeType":"Block","src":"126:31:2","nodes":[],"statements":[{"eventCall":{"arguments":[],"expression":{"argumentTypes":[],"id":79,"name":"Trigger","nodeType":"Identifier","overloadedDeclarations":[],"referencedDeclaration":76,"src":"141:7:2","typeDescriptions":{"typeIdentifier":"t_function_event_nonpayable$__$returns$__$","typeString":"function ()"}},"id":80,"isConstant":false,"isLValue":false,"isPure":false,"kind":"functionCall","lValueRequested":false,"nameLocations":[],"names":[],"nodeType":"FunctionCall","src":"141:9:2","tryCall":false,"typeDescriptions":{"typeIdentifier":"t_tuple$__$","typeString":"tuple()"}},"id":81,"nodeType":"EmitStatement","src":"136:14:2"}]},"implemented":true,"kind":"constructor","modifiers":[],"name":"","nameLocation":"-1:-1:-1","parameters":{"id":77,"nodeType":"ParameterList","parameters":[],"src":"123:2:2"},"returnParameters":{"id":78,"nodeType":"ParameterList","parameters":[],"src":"126:0:2"},"scope":103,"stateMutability":"nonpayable","virtual":false,"visibility":"public"},{"id":102,"nodeType":"FunctionDefinition","src":"163:151:2","nodes":[],"body":{"id":101,"nodeType":"Block","src":"221:93:2","nodes":[],"statements":[{"expression":{"arguments":[{"commonType":{"typeIdentifier":"t_uint256","typeString":"uint256"},"id":93,"isConstant":false,"isLValue":false,"isPure":false,"lValueRequested":false,"leftExpression":{"id":91,"name":"value","nodeType":"Identifier","overloadedDeclarations":[],"referencedDeclaration":85,"src":"239:5:2","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"}},"nodeType":"BinaryOperation","operator":"<","rightExpression":{"hexValue":"3130","id":92,"isConstant":false,"isLValue":false,"isPure":true,"kind":"number","lValueRequested":false,"nodeType":"Literal","src":"247:2:2","typeDescriptions":{"typeIdentifier":"t_rational_10_by_1","typeString":"int_const 10"},"value":"10"},"src":"239:10:2","typeDescriptions":{"typeIdentifier":"t_bool","typeString":"bool"}},{"hexValue":"63616e206f6e6c792068616e646c652076616c756573203c203130","id":94,"isConstant":false,"isLValue":false,"isPure":true,"kind":"string","lValueRequested":false,"nodeType":"Literal","src":"251:29:2","typeDescriptions":{"typeIdentifier":"t_stringliteral_578cd1fc098748633f5d7d46bba428bb3129c1e63324f2b7151699cae5146449","typeString":"literal_string \"can only handle values < 10\""},"value":"can only handle values < 10"}],"expression":{"argumentTypes":[{"typeIdentifier":"t_bool","typeString":"bool"},{"typeIdentifier":"t_stringliteral_578cd1fc098748633f5d7d46bba428bb3129c1e63324f2b7151699cae5146449","typeString":"literal_string \"can only handle values < 10\""}],"id":90,"name":"require","nodeType":"Identifier","overloadedDeclarations":[-18,-18],"referencedDeclaration":-18,"src":"231:7:2","typeDescriptions":{"typeIdentifier":"t_function_require_pure$_t_bool_$_t_string_memory_ptr_$returns$__$","typeString":"function (bool,string memory) pure"}},"id":95,"isConstant":false,"isLValue":false,"isPure":false,"kind":"functionCall","lValueRequested":false,"nameLocations":[],"names":[],"nodeType":"FunctionCall","src":"231:50:2","tryCall":false,"typeDescriptions":{"typeIdentifier":"t_tuple$__$","typeString":"tuple()"}},"id":96,"nodeType":"ExpressionStatement","src":"231:50:2"},{"expression":{"commonType":{"typeIdentifier":"t_uint256","typeString":"uint256"},"id":99,"isConstant":false,"isLValue":false,"isPure":false,"lValueRequested":false,"leftExpression":{"id":97,"name":"value","nodeType":"Identifier","overloadedDeclarations":[],"referencedDeclaration":85,"src":"298:5:2","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"}},"nodeType":"BinaryOperation","operator":"+","rightExpression":{"hexValue":"31","id":98,"isConstant":false,"isLValue":false,"isPure":true,"kind":"number","lValueRequested":false,"nodeType":"Literal","src":"306:1:2","typeDescriptions":{"typeIdentifier":"t_rational_1_by_1","typeString":"int_const 1"},"value":"1"},"src":"298:9:2","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"}},"functionReturnParameters":89,"id":100,"nodeType":"Return","src":"291:16:2"}]},"functionSelector":"812600df","implemented":true,"kind":"function","modifiers":[],"name":"inc","nameLocation":"172:3:2","parameters":{"id":86,"nodeType":"ParameterList","parameters":[{"constant":false,"id":85,"mutability":"mutable","name":"value","nameLocation":"184:5:2","nodeType":"VariableDeclaration","scope":102,"src":"176:13:2","stateVariable":false,"storageLocation":"default","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"},"typeName":{"id":84,"name":"uint256","nodeType":"ElementaryTypeName","src":"176:7:2","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"}},"visibility":"internal"}],"src":"175:15:2"},"returnParameters":{"id":89,"nodeType":"ParameterList","parameters":[{"constant":false,"id":88,"mutability":"mutable","name":"","nameLocation":"-1:-1:-1","nodeType":"VariableDeclaration","scope":102,"src":"212:7:2","stateVariable":false,"storageLocation":"default","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"},"typeName":{"id":87,"name":"uint256","nodeType":"ElementaryTypeName","src":"212:7:2","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"}},"visibility":"internal"}],"src":"211:9:2"},"scope":103,"stateMutability":"pure","virtual":false,"visibility":"public"}],"abstract":false,"baseContracts":[],"canonicalName":"RevertingContract","contractDependencies":[],"contractKind":"contract","fullyImplemented":true,"linearizedBaseContracts":[103],"name":"RevertingContract","nameLocation":"66:17:2","scope":104,"usedErrors":[]}],"license":"MIT"},"id":2} \ No newline at end of file diff --git a/tests/contracts/out/SimpleContract.sol/SimpleContract.json b/tests/contracts/out/SimpleContract.sol/SimpleContract.json index 4839740968c..57eb93d7eee 100644 --- a/tests/contracts/out/SimpleContract.sol/SimpleContract.json +++ b/tests/contracts/out/SimpleContract.sol/SimpleContract.json @@ -1,845 +1 @@ -{ - "abi": [ - { "type": "constructor", "inputs": [], "stateMutability": "nonpayable" }, - { - "type": "function", - "name": "emitAnotherTrigger", - "inputs": [ - { "name": "a", "type": "uint256", "internalType": "uint256" }, - { "name": "b", "type": "uint256", "internalType": "uint256" }, - { "name": "c", "type": "uint256", "internalType": "uint256" }, - { "name": "data", "type": "string", "internalType": "string" } - ], - "outputs": [], - "stateMutability": "nonpayable" - }, - { - "type": "function", - "name": "emitTrigger", - "inputs": [{ "name": "x", "type": "uint16", "internalType": "uint16" }], - "outputs": [], - "stateMutability": "nonpayable" - }, - { - "type": "event", - "name": "AnotherTrigger", - "inputs": [ - { - "name": "a", - "type": "uint256", - "indexed": true, - "internalType": "uint256" - }, - { - "name": "b", - "type": "uint256", - "indexed": true, - "internalType": "uint256" - }, - { - "name": "c", - "type": "uint256", - "indexed": true, - "internalType": "uint256" - }, - { - "name": "data", - "type": "string", - "indexed": false, - "internalType": "string" - } - ], - "anonymous": false - }, - { - "type": "event", - "name": "Trigger", - "inputs": [ - { - "name": "x", - "type": "uint16", - "indexed": false, - "internalType": "uint16" - } - ], - "anonymous": false - } - ], - "bytecode": { - "object": "0x608060405234801561001057600080fd5b50604051600081527f166a7d625edff952ff346d1bca4edef10254353f72916b7fb072d55d0f97b5449060200160405180910390a1610270806100546000396000f3fe608060405234801561001057600080fd5b50600436106100365760003560e01c806316d04e0d1461003b578063931919ea14610050575b600080fd5b61004e6100493660046100dd565b610063565b005b61004e61005e36600461011e565b61009d565b60405161ffff821681527f166a7d625edff952ff346d1bca4edef10254353f72916b7fb072d55d0f97b5449060200160405180910390a150565b8183857f2cb351db58390c313534745d80b5f0abff9230502a6374a97b9caa76b31c5d8a846040516100cf91906101ec565b60405180910390a450505050565b6000602082840312156100ef57600080fd5b813561ffff8116811461010157600080fd5b9392505050565b634e487b7160e01b600052604160045260246000fd5b6000806000806080858703121561013457600080fd5b843593506020850135925060408501359150606085013567ffffffffffffffff8082111561016157600080fd5b818701915087601f83011261017557600080fd5b81358181111561018757610187610108565b604051601f8201601f19908116603f011681019083821181831017156101af576101af610108565b816040528281528a60208487010111156101c857600080fd5b82602086016020830137600060208483010152809550505050505092959194509250565b600060208083528351808285015260005b81811015610219578581018301518582016040015282016101fd565b506000604082860101526040601f19601f830116850101925050509291505056fea264697066735822122051969b527a63ab67686e528eb2de0bd24f1a84835193586c0318cfb81b2cb0ac64736f6c63430008130033", - "sourceMap": "57:596:0:-:0;;;308:46;;;;;;;;;-1:-1:-1;337:10:0;;345:1;167:38:1;;337:10:0;;155:2:1;140:18;337:10:0;;;;;;;57:596;;;;;;", - "linkReferences": {} - }, - "deployedBytecode": { - "object": "0x608060405234801561001057600080fd5b50600436106100365760003560e01c806316d04e0d1461003b578063931919ea14610050575b600080fd5b61004e6100493660046100dd565b610063565b005b61004e61005e36600461011e565b61009d565b60405161ffff821681527f166a7d625edff952ff346d1bca4edef10254353f72916b7fb072d55d0f97b5449060200160405180910390a150565b8183857f2cb351db58390c313534745d80b5f0abff9230502a6374a97b9caa76b31c5d8a846040516100cf91906101ec565b60405180910390a450505050565b6000602082840312156100ef57600080fd5b813561ffff8116811461010157600080fd5b9392505050565b634e487b7160e01b600052604160045260246000fd5b6000806000806080858703121561013457600080fd5b843593506020850135925060408501359150606085013567ffffffffffffffff8082111561016157600080fd5b818701915087601f83011261017557600080fd5b81358181111561018757610187610108565b604051601f8201601f19908116603f011681019083821181831017156101af576101af610108565b816040528281528a60208487010111156101c857600080fd5b82602086016020830137600060208483010152809550505050505092959194509250565b600060208083528351808285015260005b81811015610219578581018301518582016040015282016101fd565b506000604082860101526040601f19601f830116850101925050509291505056fea264697066735822122051969b527a63ab67686e528eb2de0bd24f1a84835193586c0318cfb81b2cb0ac64736f6c63430008130033", - "sourceMap": "57:596:0:-:0;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;360:70;;;;;;:::i;:::-;;:::i;:::-;;474:177;;;;;;:::i;:::-;;:::i;360:70::-;413:10;;1729:6:1;1717:19;;1699:38;;413:10:0;;1687:2:1;1672:18;413:10:0;;;;;;;360:70;:::o;474:177::-;636:1;633;630;615:29;639:4;615:29;;;;;;:::i;:::-;;;;;;;;474:177;;;;:::o;14:272:1:-;72:6;125:2;113:9;104:7;100:23;96:32;93:52;;;141:1;138;131:12;93:52;180:9;167:23;230:6;223:5;219:18;212:5;209:29;199:57;;252:1;249;242:12;199:57;275:5;14:272;-1:-1:-1;;;14:272:1:o;291:127::-;352:10;347:3;343:20;340:1;333:31;383:4;380:1;373:15;407:4;404:1;397:15;423:1127;519:6;527;535;543;596:3;584:9;575:7;571:23;567:33;564:53;;;613:1;610;603:12;564:53;649:9;636:23;626:33;;706:2;695:9;691:18;678:32;668:42;;757:2;746:9;742:18;729:32;719:42;;812:2;801:9;797:18;784:32;835:18;876:2;868:6;865:14;862:34;;;892:1;889;882:12;862:34;930:6;919:9;915:22;905:32;;975:7;968:4;964:2;960:13;956:27;946:55;;997:1;994;987:12;946:55;1033:2;1020:16;1055:2;1051;1048:10;1045:36;;;1061:18;;:::i;:::-;1136:2;1130:9;1104:2;1190:13;;-1:-1:-1;;1186:22:1;;;1210:2;1182:31;1178:40;1166:53;;;1234:18;;;1254:22;;;1231:46;1228:72;;;1280:18;;:::i;:::-;1320:10;1316:2;1309:22;1355:2;1347:6;1340:18;1395:7;1390:2;1385;1381;1377:11;1373:20;1370:33;1367:53;;;1416:1;1413;1406:12;1367:53;1472:2;1467;1463;1459:11;1454:2;1446:6;1442:15;1429:46;1517:1;1512:2;1507;1499:6;1495:15;1491:24;1484:35;1538:6;1528:16;;;;;;;423:1127;;;;;;;:::o;1748:548::-;1860:4;1889:2;1918;1907:9;1900:21;1950:6;1944:13;1993:6;1988:2;1977:9;1973:18;1966:34;2018:1;2028:140;2042:6;2039:1;2036:13;2028:140;;;2137:14;;;2133:23;;2127:30;2103:17;;;2122:2;2099:26;2092:66;2057:10;;2028:140;;;2032:3;2217:1;2212:2;2203:6;2192:9;2188:22;2184:31;2177:42;2287:2;2280;2276:7;2271:2;2263:6;2259:15;2255:29;2244:9;2240:45;2236:54;2228:62;;;;1748:548;;;;:::o", - "linkReferences": {} - }, - "methodIdentifiers": { - "emitAnotherTrigger(uint256,uint256,uint256,string)": "931919ea", - "emitTrigger(uint16)": "16d04e0d" - }, - "rawMetadata": "{\"compiler\":{\"version\":\"0.8.19+commit.7dd6d404\"},\"language\":\"Solidity\",\"output\":{\"abi\":[{\"inputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"constructor\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":true,\"internalType\":\"uint256\",\"name\":\"a\",\"type\":\"uint256\"},{\"indexed\":true,\"internalType\":\"uint256\",\"name\":\"b\",\"type\":\"uint256\"},{\"indexed\":true,\"internalType\":\"uint256\",\"name\":\"c\",\"type\":\"uint256\"},{\"indexed\":false,\"internalType\":\"string\",\"name\":\"data\",\"type\":\"string\"}],\"name\":\"AnotherTrigger\",\"type\":\"event\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":false,\"internalType\":\"uint16\",\"name\":\"x\",\"type\":\"uint16\"}],\"name\":\"Trigger\",\"type\":\"event\"},{\"inputs\":[{\"internalType\":\"uint256\",\"name\":\"a\",\"type\":\"uint256\"},{\"internalType\":\"uint256\",\"name\":\"b\",\"type\":\"uint256\"},{\"internalType\":\"uint256\",\"name\":\"c\",\"type\":\"uint256\"},{\"internalType\":\"string\",\"name\":\"data\",\"type\":\"string\"}],\"name\":\"emitAnotherTrigger\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"uint16\",\"name\":\"x\",\"type\":\"uint16\"}],\"name\":\"emitTrigger\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"}],\"devdoc\":{\"kind\":\"dev\",\"methods\":{},\"version\":1},\"userdoc\":{\"kind\":\"user\",\"methods\":{},\"version\":1}},\"settings\":{\"compilationTarget\":{\"src/SimpleContract.sol\":\"SimpleContract\"},\"evmVersion\":\"paris\",\"libraries\":{},\"metadata\":{\"bytecodeHash\":\"ipfs\"},\"optimizer\":{\"enabled\":true,\"runs\":200},\"remappings\":[]},\"sources\":{\"src/SimpleContract.sol\":{\"keccak256\":\"0xda954fc2eb36f5f3658f71e59fdb487c6f8947efa45e5e3fb7038c7faff99de0\",\"license\":\"MIT\",\"urls\":[\"bzz-raw://e8253c13afee68eee23965caf364c3812ca6065eac5655faf9c20d9f231b9b1d\",\"dweb:/ipfs/QmXPdwfDAMniiwJHPt2WBvaT5gK1LUK3aM81Jq5m3n8UPF\"]}},\"version\":1}", - "metadata": { - "compiler": { "version": "0.8.19+commit.7dd6d404" }, - "language": "Solidity", - "output": { - "abi": [ - { - "inputs": [], - "stateMutability": "nonpayable", - "type": "constructor" - }, - { - "inputs": [ - { - "internalType": "uint256", - "name": "a", - "type": "uint256", - "indexed": true - }, - { - "internalType": "uint256", - "name": "b", - "type": "uint256", - "indexed": true - }, - { - "internalType": "uint256", - "name": "c", - "type": "uint256", - "indexed": true - }, - { - "internalType": "string", - "name": "data", - "type": "string", - "indexed": false - } - ], - "type": "event", - "name": "AnotherTrigger", - "anonymous": false - }, - { - "inputs": [ - { - "internalType": "uint16", - "name": "x", - "type": "uint16", - "indexed": false - } - ], - "type": "event", - "name": "Trigger", - "anonymous": false - }, - { - "inputs": [ - { "internalType": "uint256", "name": "a", "type": "uint256" }, - { "internalType": "uint256", "name": "b", "type": "uint256" }, - { "internalType": "uint256", "name": "c", "type": "uint256" }, - { "internalType": "string", "name": "data", "type": "string" } - ], - "stateMutability": "nonpayable", - "type": "function", - "name": "emitAnotherTrigger" - }, - { - "inputs": [ - { "internalType": "uint16", "name": "x", "type": "uint16" } - ], - "stateMutability": "nonpayable", - "type": "function", - "name": "emitTrigger" - } - ], - "devdoc": { "kind": "dev", "methods": {}, "version": 1 }, - "userdoc": { "kind": "user", "methods": {}, "version": 1 } - }, - "settings": { - "remappings": [], - "optimizer": { "enabled": true, "runs": 200 }, - "metadata": { "bytecodeHash": "ipfs" }, - "compilationTarget": { "src/SimpleContract.sol": "SimpleContract" }, - "evmVersion": "paris", - "libraries": {} - }, - "sources": { - "src/SimpleContract.sol": { - "keccak256": "0xda954fc2eb36f5f3658f71e59fdb487c6f8947efa45e5e3fb7038c7faff99de0", - "urls": [ - "bzz-raw://e8253c13afee68eee23965caf364c3812ca6065eac5655faf9c20d9f231b9b1d", - "dweb:/ipfs/QmXPdwfDAMniiwJHPt2WBvaT5gK1LUK3aM81Jq5m3n8UPF" - ], - "license": "MIT" - } - }, - "version": 1 - }, - "ast": { - "absolutePath": "src/SimpleContract.sol", - "id": 54, - "exportedSymbols": { "SimpleContract": [53] }, - "nodeType": "SourceUnit", - "src": "32:622:0", - "nodes": [ - { - "id": 1, - "nodeType": "PragmaDirective", - "src": "32:23:0", - "nodes": [], - "literals": ["solidity", "^", "0.8", ".0"] - }, - { - "id": 53, - "nodeType": "ContractDefinition", - "src": "57:596:0", - "nodes": [ - { - "id": 5, - "nodeType": "EventDefinition", - "src": "87:24:0", - "nodes": [], - "anonymous": false, - "eventSelector": "166a7d625edff952ff346d1bca4edef10254353f72916b7fb072d55d0f97b544", - "name": "Trigger", - "nameLocation": "93:7:0", - "parameters": { - "id": 4, - "nodeType": "ParameterList", - "parameters": [ - { - "constant": false, - "id": 3, - "indexed": false, - "mutability": "mutable", - "name": "x", - "nameLocation": "108:1:0", - "nodeType": "VariableDeclaration", - "scope": 5, - "src": "101:8:0", - "stateVariable": false, - "storageLocation": "default", - "typeDescriptions": { - "typeIdentifier": "t_uint16", - "typeString": "uint16" - }, - "typeName": { - "id": 2, - "name": "uint16", - "nodeType": "ElementaryTypeName", - "src": "101:6:0", - "typeDescriptions": { - "typeIdentifier": "t_uint16", - "typeString": "uint16" - } - }, - "visibility": "internal" - } - ], - "src": "100:10:0" - } - }, - { - "id": 15, - "nodeType": "EventDefinition", - "src": "173:129:0", - "nodes": [], - "anonymous": false, - "eventSelector": "2cb351db58390c313534745d80b5f0abff9230502a6374a97b9caa76b31c5d8a", - "name": "AnotherTrigger", - "nameLocation": "179:14:0", - "parameters": { - "id": 14, - "nodeType": "ParameterList", - "parameters": [ - { - "constant": false, - "id": 7, - "indexed": true, - "mutability": "mutable", - "name": "a", - "nameLocation": "219:1:0", - "nodeType": "VariableDeclaration", - "scope": 15, - "src": "203:17:0", - "stateVariable": false, - "storageLocation": "default", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - }, - "typeName": { - "id": 6, - "name": "uint256", - "nodeType": "ElementaryTypeName", - "src": "203:7:0", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - } - }, - "visibility": "internal" - }, - { - "constant": false, - "id": 9, - "indexed": true, - "mutability": "mutable", - "name": "b", - "nameLocation": "246:1:0", - "nodeType": "VariableDeclaration", - "scope": 15, - "src": "230:17:0", - "stateVariable": false, - "storageLocation": "default", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - }, - "typeName": { - "id": 8, - "name": "uint256", - "nodeType": "ElementaryTypeName", - "src": "230:7:0", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - } - }, - "visibility": "internal" - }, - { - "constant": false, - "id": 11, - "indexed": true, - "mutability": "mutable", - "name": "c", - "nameLocation": "273:1:0", - "nodeType": "VariableDeclaration", - "scope": 15, - "src": "257:17:0", - "stateVariable": false, - "storageLocation": "default", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - }, - "typeName": { - "id": 10, - "name": "uint256", - "nodeType": "ElementaryTypeName", - "src": "257:7:0", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - } - }, - "visibility": "internal" - }, - { - "constant": false, - "id": 13, - "indexed": false, - "mutability": "mutable", - "name": "data", - "nameLocation": "291:4:0", - "nodeType": "VariableDeclaration", - "scope": 15, - "src": "284:11:0", - "stateVariable": false, - "storageLocation": "default", - "typeDescriptions": { - "typeIdentifier": "t_string_memory_ptr", - "typeString": "string" - }, - "typeName": { - "id": 12, - "name": "string", - "nodeType": "ElementaryTypeName", - "src": "284:6:0", - "typeDescriptions": { - "typeIdentifier": "t_string_storage_ptr", - "typeString": "string" - } - }, - "visibility": "internal" - } - ], - "src": "193:108:0" - } - }, - { - "id": 23, - "nodeType": "FunctionDefinition", - "src": "308:46:0", - "nodes": [], - "body": { - "id": 22, - "nodeType": "Block", - "src": "322:32:0", - "nodes": [], - "statements": [ - { - "eventCall": { - "arguments": [ - { - "hexValue": "30", - "id": 19, - "isConstant": false, - "isLValue": false, - "isPure": true, - "kind": "number", - "lValueRequested": false, - "nodeType": "Literal", - "src": "345:1:0", - "typeDescriptions": { - "typeIdentifier": "t_rational_0_by_1", - "typeString": "int_const 0" - }, - "value": "0" - } - ], - "expression": { - "argumentTypes": [ - { - "typeIdentifier": "t_rational_0_by_1", - "typeString": "int_const 0" - } - ], - "id": 18, - "name": "Trigger", - "nodeType": "Identifier", - "overloadedDeclarations": [], - "referencedDeclaration": 5, - "src": "337:7:0", - "typeDescriptions": { - "typeIdentifier": "t_function_event_nonpayable$_t_uint16_$returns$__$", - "typeString": "function (uint16)" - } - }, - "id": 20, - "isConstant": false, - "isLValue": false, - "isPure": false, - "kind": "functionCall", - "lValueRequested": false, - "nameLocations": [], - "names": [], - "nodeType": "FunctionCall", - "src": "337:10:0", - "tryCall": false, - "typeDescriptions": { - "typeIdentifier": "t_tuple$__$", - "typeString": "tuple()" - } - }, - "id": 21, - "nodeType": "EmitStatement", - "src": "332:15:0" - } - ] - }, - "implemented": true, - "kind": "constructor", - "modifiers": [], - "name": "", - "nameLocation": "-1:-1:-1", - "parameters": { - "id": 16, - "nodeType": "ParameterList", - "parameters": [], - "src": "319:2:0" - }, - "returnParameters": { - "id": 17, - "nodeType": "ParameterList", - "parameters": [], - "src": "322:0:0" - }, - "scope": 53, - "stateMutability": "nonpayable", - "virtual": false, - "visibility": "public" - }, - { - "id": 33, - "nodeType": "FunctionDefinition", - "src": "360:70:0", - "nodes": [], - "body": { - "id": 32, - "nodeType": "Block", - "src": "398:32:0", - "nodes": [], - "statements": [ - { - "eventCall": { - "arguments": [ - { - "id": 29, - "name": "x", - "nodeType": "Identifier", - "overloadedDeclarations": [], - "referencedDeclaration": 25, - "src": "421:1:0", - "typeDescriptions": { - "typeIdentifier": "t_uint16", - "typeString": "uint16" - } - } - ], - "expression": { - "argumentTypes": [ - { "typeIdentifier": "t_uint16", "typeString": "uint16" } - ], - "id": 28, - "name": "Trigger", - "nodeType": "Identifier", - "overloadedDeclarations": [], - "referencedDeclaration": 5, - "src": "413:7:0", - "typeDescriptions": { - "typeIdentifier": "t_function_event_nonpayable$_t_uint16_$returns$__$", - "typeString": "function (uint16)" - } - }, - "id": 30, - "isConstant": false, - "isLValue": false, - "isPure": false, - "kind": "functionCall", - "lValueRequested": false, - "nameLocations": [], - "names": [], - "nodeType": "FunctionCall", - "src": "413:10:0", - "tryCall": false, - "typeDescriptions": { - "typeIdentifier": "t_tuple$__$", - "typeString": "tuple()" - } - }, - "id": 31, - "nodeType": "EmitStatement", - "src": "408:15:0" - } - ] - }, - "functionSelector": "16d04e0d", - "implemented": true, - "kind": "function", - "modifiers": [], - "name": "emitTrigger", - "nameLocation": "369:11:0", - "parameters": { - "id": 26, - "nodeType": "ParameterList", - "parameters": [ - { - "constant": false, - "id": 25, - "mutability": "mutable", - "name": "x", - "nameLocation": "388:1:0", - "nodeType": "VariableDeclaration", - "scope": 33, - "src": "381:8:0", - "stateVariable": false, - "storageLocation": "default", - "typeDescriptions": { - "typeIdentifier": "t_uint16", - "typeString": "uint16" - }, - "typeName": { - "id": 24, - "name": "uint16", - "nodeType": "ElementaryTypeName", - "src": "381:6:0", - "typeDescriptions": { - "typeIdentifier": "t_uint16", - "typeString": "uint16" - } - }, - "visibility": "internal" - } - ], - "src": "380:10:0" - }, - "returnParameters": { - "id": 27, - "nodeType": "ParameterList", - "parameters": [], - "src": "398:0:0" - }, - "scope": 53, - "stateMutability": "nonpayable", - "virtual": false, - "visibility": "public" - }, - { - "id": 52, - "nodeType": "FunctionDefinition", - "src": "474:177:0", - "nodes": [], - "body": { - "id": 51, - "nodeType": "Block", - "src": "600:51:0", - "nodes": [], - "statements": [ - { - "eventCall": { - "arguments": [ - { - "id": 45, - "name": "a", - "nodeType": "Identifier", - "overloadedDeclarations": [], - "referencedDeclaration": 35, - "src": "630:1:0", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - } - }, - { - "id": 46, - "name": "b", - "nodeType": "Identifier", - "overloadedDeclarations": [], - "referencedDeclaration": 37, - "src": "633:1:0", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - } - }, - { - "id": 47, - "name": "c", - "nodeType": "Identifier", - "overloadedDeclarations": [], - "referencedDeclaration": 39, - "src": "636:1:0", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - } - }, - { - "id": 48, - "name": "data", - "nodeType": "Identifier", - "overloadedDeclarations": [], - "referencedDeclaration": 41, - "src": "639:4:0", - "typeDescriptions": { - "typeIdentifier": "t_string_memory_ptr", - "typeString": "string memory" - } - } - ], - "expression": { - "argumentTypes": [ - { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - }, - { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - }, - { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - }, - { - "typeIdentifier": "t_string_memory_ptr", - "typeString": "string memory" - } - ], - "id": 44, - "name": "AnotherTrigger", - "nodeType": "Identifier", - "overloadedDeclarations": [], - "referencedDeclaration": 15, - "src": "615:14:0", - "typeDescriptions": { - "typeIdentifier": "t_function_event_nonpayable$_t_uint256_$_t_uint256_$_t_uint256_$_t_string_memory_ptr_$returns$__$", - "typeString": "function (uint256,uint256,uint256,string memory)" - } - }, - "id": 49, - "isConstant": false, - "isLValue": false, - "isPure": false, - "kind": "functionCall", - "lValueRequested": false, - "nameLocations": [], - "names": [], - "nodeType": "FunctionCall", - "src": "615:29:0", - "tryCall": false, - "typeDescriptions": { - "typeIdentifier": "t_tuple$__$", - "typeString": "tuple()" - } - }, - "id": 50, - "nodeType": "EmitStatement", - "src": "610:34:0" - } - ] - }, - "functionSelector": "931919ea", - "implemented": true, - "kind": "function", - "modifiers": [], - "name": "emitAnotherTrigger", - "nameLocation": "483:18:0", - "parameters": { - "id": 42, - "nodeType": "ParameterList", - "parameters": [ - { - "constant": false, - "id": 35, - "mutability": "mutable", - "name": "a", - "nameLocation": "519:1:0", - "nodeType": "VariableDeclaration", - "scope": 52, - "src": "511:9:0", - "stateVariable": false, - "storageLocation": "default", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - }, - "typeName": { - "id": 34, - "name": "uint256", - "nodeType": "ElementaryTypeName", - "src": "511:7:0", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - } - }, - "visibility": "internal" - }, - { - "constant": false, - "id": 37, - "mutability": "mutable", - "name": "b", - "nameLocation": "538:1:0", - "nodeType": "VariableDeclaration", - "scope": 52, - "src": "530:9:0", - "stateVariable": false, - "storageLocation": "default", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - }, - "typeName": { - "id": 36, - "name": "uint256", - "nodeType": "ElementaryTypeName", - "src": "530:7:0", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - } - }, - "visibility": "internal" - }, - { - "constant": false, - "id": 39, - "mutability": "mutable", - "name": "c", - "nameLocation": "557:1:0", - "nodeType": "VariableDeclaration", - "scope": 52, - "src": "549:9:0", - "stateVariable": false, - "storageLocation": "default", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - }, - "typeName": { - "id": 38, - "name": "uint256", - "nodeType": "ElementaryTypeName", - "src": "549:7:0", - "typeDescriptions": { - "typeIdentifier": "t_uint256", - "typeString": "uint256" - } - }, - "visibility": "internal" - }, - { - "constant": false, - "id": 41, - "mutability": "mutable", - "name": "data", - "nameLocation": "582:4:0", - "nodeType": "VariableDeclaration", - "scope": 52, - "src": "568:18:0", - "stateVariable": false, - "storageLocation": "memory", - "typeDescriptions": { - "typeIdentifier": "t_string_memory_ptr", - "typeString": "string" - }, - "typeName": { - "id": 40, - "name": "string", - "nodeType": "ElementaryTypeName", - "src": "568:6:0", - "typeDescriptions": { - "typeIdentifier": "t_string_storage_ptr", - "typeString": "string" - } - }, - "visibility": "internal" - } - ], - "src": "501:91:0" - }, - "returnParameters": { - "id": 43, - "nodeType": "ParameterList", - "parameters": [], - "src": "600:0:0" - }, - "scope": 53, - "stateMutability": "nonpayable", - "virtual": false, - "visibility": "public" - } - ], - "abstract": false, - "baseContracts": [], - "canonicalName": "SimpleContract", - "contractDependencies": [], - "contractKind": "contract", - "fullyImplemented": true, - "linearizedBaseContracts": [53], - "name": "SimpleContract", - "nameLocation": "66:14:0", - "scope": 54, - "usedErrors": [] - } - ], - "license": "MIT" - }, - "id": 0 -} +{"abi":[{"type":"constructor","inputs":[],"stateMutability":"nonpayable"},{"type":"function","name":"emitAnotherTrigger","inputs":[{"name":"a","type":"uint256","internalType":"uint256"},{"name":"b","type":"uint256","internalType":"uint256"},{"name":"c","type":"uint256","internalType":"uint256"},{"name":"data","type":"string","internalType":"string"}],"outputs":[],"stateMutability":"nonpayable"},{"type":"function","name":"emitTrigger","inputs":[{"name":"x","type":"uint16","internalType":"uint16"}],"outputs":[],"stateMutability":"nonpayable"},{"type":"event","name":"AnotherTrigger","inputs":[{"name":"a","type":"uint256","indexed":true,"internalType":"uint256"},{"name":"b","type":"uint256","indexed":true,"internalType":"uint256"},{"name":"c","type":"uint256","indexed":true,"internalType":"uint256"},{"name":"data","type":"string","indexed":false,"internalType":"string"}],"anonymous":false},{"type":"event","name":"Trigger","inputs":[{"name":"x","type":"uint16","indexed":false,"internalType":"uint16"}],"anonymous":false}],"bytecode":{"object":"0x608060405234801561001057600080fd5b50604051600081527f166a7d625edff952ff346d1bca4edef10254353f72916b7fb072d55d0f97b5449060200160405180910390a1610270806100546000396000f3fe608060405234801561001057600080fd5b50600436106100365760003560e01c806316d04e0d1461003b578063931919ea14610050575b600080fd5b61004e6100493660046100dd565b610063565b005b61004e61005e36600461011e565b61009d565b60405161ffff821681527f166a7d625edff952ff346d1bca4edef10254353f72916b7fb072d55d0f97b5449060200160405180910390a150565b8183857f2cb351db58390c313534745d80b5f0abff9230502a6374a97b9caa76b31c5d8a846040516100cf91906101ec565b60405180910390a450505050565b6000602082840312156100ef57600080fd5b813561ffff8116811461010157600080fd5b9392505050565b634e487b7160e01b600052604160045260246000fd5b6000806000806080858703121561013457600080fd5b843593506020850135925060408501359150606085013567ffffffffffffffff8082111561016157600080fd5b818701915087601f83011261017557600080fd5b81358181111561018757610187610108565b604051601f8201601f19908116603f011681019083821181831017156101af576101af610108565b816040528281528a60208487010111156101c857600080fd5b82602086016020830137600060208483010152809550505050505092959194509250565b600060208083528351808285015260005b81811015610219578581018301518582016040015282016101fd565b506000604082860101526040601f19601f830116850101925050509291505056fea264697066735822122051969b527a63ab67686e528eb2de0bd24f1a84835193586c0318cfb81b2cb0ac64736f6c63430008130033","sourceMap":"57:596:3:-:0;;;308:46;;;;;;;;;-1:-1:-1;337:10:3;;345:1;167:38:4;;337:10:3;;155:2:4;140:18;337:10:3;;;;;;;57:596;;;;;;","linkReferences":{}},"deployedBytecode":{"object":"0x608060405234801561001057600080fd5b50600436106100365760003560e01c806316d04e0d1461003b578063931919ea14610050575b600080fd5b61004e6100493660046100dd565b610063565b005b61004e61005e36600461011e565b61009d565b60405161ffff821681527f166a7d625edff952ff346d1bca4edef10254353f72916b7fb072d55d0f97b5449060200160405180910390a150565b8183857f2cb351db58390c313534745d80b5f0abff9230502a6374a97b9caa76b31c5d8a846040516100cf91906101ec565b60405180910390a450505050565b6000602082840312156100ef57600080fd5b813561ffff8116811461010157600080fd5b9392505050565b634e487b7160e01b600052604160045260246000fd5b6000806000806080858703121561013457600080fd5b843593506020850135925060408501359150606085013567ffffffffffffffff8082111561016157600080fd5b818701915087601f83011261017557600080fd5b81358181111561018757610187610108565b604051601f8201601f19908116603f011681019083821181831017156101af576101af610108565b816040528281528a60208487010111156101c857600080fd5b82602086016020830137600060208483010152809550505050505092959194509250565b600060208083528351808285015260005b81811015610219578581018301518582016040015282016101fd565b506000604082860101526040601f19601f830116850101925050509291505056fea264697066735822122051969b527a63ab67686e528eb2de0bd24f1a84835193586c0318cfb81b2cb0ac64736f6c63430008130033","sourceMap":"57:596:3:-:0;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;360:70;;;;;;:::i;:::-;;:::i;:::-;;474:177;;;;;;:::i;:::-;;:::i;360:70::-;413:10;;1729:6:4;1717:19;;1699:38;;413:10:3;;1687:2:4;1672:18;413:10:3;;;;;;;360:70;:::o;474:177::-;636:1;633;630;615:29;639:4;615:29;;;;;;:::i;:::-;;;;;;;;474:177;;;;:::o;14:272:4:-;72:6;125:2;113:9;104:7;100:23;96:32;93:52;;;141:1;138;131:12;93:52;180:9;167:23;230:6;223:5;219:18;212:5;209:29;199:57;;252:1;249;242:12;199:57;275:5;14:272;-1:-1:-1;;;14:272:4:o;291:127::-;352:10;347:3;343:20;340:1;333:31;383:4;380:1;373:15;407:4;404:1;397:15;423:1127;519:6;527;535;543;596:3;584:9;575:7;571:23;567:33;564:53;;;613:1;610;603:12;564:53;649:9;636:23;626:33;;706:2;695:9;691:18;678:32;668:42;;757:2;746:9;742:18;729:32;719:42;;812:2;801:9;797:18;784:32;835:18;876:2;868:6;865:14;862:34;;;892:1;889;882:12;862:34;930:6;919:9;915:22;905:32;;975:7;968:4;964:2;960:13;956:27;946:55;;997:1;994;987:12;946:55;1033:2;1020:16;1055:2;1051;1048:10;1045:36;;;1061:18;;:::i;:::-;1136:2;1130:9;1104:2;1190:13;;-1:-1:-1;;1186:22:4;;;1210:2;1182:31;1178:40;1166:53;;;1234:18;;;1254:22;;;1231:46;1228:72;;;1280:18;;:::i;:::-;1320:10;1316:2;1309:22;1355:2;1347:6;1340:18;1395:7;1390:2;1385;1381;1377:11;1373:20;1370:33;1367:53;;;1416:1;1413;1406:12;1367:53;1472:2;1467;1463;1459:11;1454:2;1446:6;1442:15;1429:46;1517:1;1512:2;1507;1499:6;1495:15;1491:24;1484:35;1538:6;1528:16;;;;;;;423:1127;;;;;;;:::o;1748:548::-;1860:4;1889:2;1918;1907:9;1900:21;1950:6;1944:13;1993:6;1988:2;1977:9;1973:18;1966:34;2018:1;2028:140;2042:6;2039:1;2036:13;2028:140;;;2137:14;;;2133:23;;2127:30;2103:17;;;2122:2;2099:26;2092:66;2057:10;;2028:140;;;2032:3;2217:1;2212:2;2203:6;2192:9;2188:22;2184:31;2177:42;2287:2;2280;2276:7;2271:2;2263:6;2259:15;2255:29;2244:9;2240:45;2236:54;2228:62;;;;1748:548;;;;:::o","linkReferences":{}},"methodIdentifiers":{"emitAnotherTrigger(uint256,uint256,uint256,string)":"931919ea","emitTrigger(uint16)":"16d04e0d"},"rawMetadata":"{\"compiler\":{\"version\":\"0.8.19+commit.7dd6d404\"},\"language\":\"Solidity\",\"output\":{\"abi\":[{\"inputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"constructor\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":true,\"internalType\":\"uint256\",\"name\":\"a\",\"type\":\"uint256\"},{\"indexed\":true,\"internalType\":\"uint256\",\"name\":\"b\",\"type\":\"uint256\"},{\"indexed\":true,\"internalType\":\"uint256\",\"name\":\"c\",\"type\":\"uint256\"},{\"indexed\":false,\"internalType\":\"string\",\"name\":\"data\",\"type\":\"string\"}],\"name\":\"AnotherTrigger\",\"type\":\"event\"},{\"anonymous\":false,\"inputs\":[{\"indexed\":false,\"internalType\":\"uint16\",\"name\":\"x\",\"type\":\"uint16\"}],\"name\":\"Trigger\",\"type\":\"event\"},{\"inputs\":[{\"internalType\":\"uint256\",\"name\":\"a\",\"type\":\"uint256\"},{\"internalType\":\"uint256\",\"name\":\"b\",\"type\":\"uint256\"},{\"internalType\":\"uint256\",\"name\":\"c\",\"type\":\"uint256\"},{\"internalType\":\"string\",\"name\":\"data\",\"type\":\"string\"}],\"name\":\"emitAnotherTrigger\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"},{\"inputs\":[{\"internalType\":\"uint16\",\"name\":\"x\",\"type\":\"uint16\"}],\"name\":\"emitTrigger\",\"outputs\":[],\"stateMutability\":\"nonpayable\",\"type\":\"function\"}],\"devdoc\":{\"kind\":\"dev\",\"methods\":{},\"version\":1},\"userdoc\":{\"kind\":\"user\",\"methods\":{},\"version\":1}},\"settings\":{\"compilationTarget\":{\"src/SimpleContract.sol\":\"SimpleContract\"},\"evmVersion\":\"paris\",\"libraries\":{},\"metadata\":{\"bytecodeHash\":\"ipfs\"},\"optimizer\":{\"enabled\":true,\"runs\":200},\"remappings\":[]},\"sources\":{\"src/SimpleContract.sol\":{\"keccak256\":\"0xda954fc2eb36f5f3658f71e59fdb487c6f8947efa45e5e3fb7038c7faff99de0\",\"license\":\"MIT\",\"urls\":[\"bzz-raw://e8253c13afee68eee23965caf364c3812ca6065eac5655faf9c20d9f231b9b1d\",\"dweb:/ipfs/QmXPdwfDAMniiwJHPt2WBvaT5gK1LUK3aM81Jq5m3n8UPF\"]}},\"version\":1}","metadata":{"compiler":{"version":"0.8.19+commit.7dd6d404"},"language":"Solidity","output":{"abi":[{"inputs":[],"stateMutability":"nonpayable","type":"constructor"},{"inputs":[{"internalType":"uint256","name":"a","type":"uint256","indexed":true},{"internalType":"uint256","name":"b","type":"uint256","indexed":true},{"internalType":"uint256","name":"c","type":"uint256","indexed":true},{"internalType":"string","name":"data","type":"string","indexed":false}],"type":"event","name":"AnotherTrigger","anonymous":false},{"inputs":[{"internalType":"uint16","name":"x","type":"uint16","indexed":false}],"type":"event","name":"Trigger","anonymous":false},{"inputs":[{"internalType":"uint256","name":"a","type":"uint256"},{"internalType":"uint256","name":"b","type":"uint256"},{"internalType":"uint256","name":"c","type":"uint256"},{"internalType":"string","name":"data","type":"string"}],"stateMutability":"nonpayable","type":"function","name":"emitAnotherTrigger"},{"inputs":[{"internalType":"uint16","name":"x","type":"uint16"}],"stateMutability":"nonpayable","type":"function","name":"emitTrigger"}],"devdoc":{"kind":"dev","methods":{},"version":1},"userdoc":{"kind":"user","methods":{},"version":1}},"settings":{"remappings":[],"optimizer":{"enabled":true,"runs":200},"metadata":{"bytecodeHash":"ipfs"},"compilationTarget":{"src/SimpleContract.sol":"SimpleContract"},"evmVersion":"paris","libraries":{}},"sources":{"src/SimpleContract.sol":{"keccak256":"0xda954fc2eb36f5f3658f71e59fdb487c6f8947efa45e5e3fb7038c7faff99de0","urls":["bzz-raw://e8253c13afee68eee23965caf364c3812ca6065eac5655faf9c20d9f231b9b1d","dweb:/ipfs/QmXPdwfDAMniiwJHPt2WBvaT5gK1LUK3aM81Jq5m3n8UPF"],"license":"MIT"}},"version":1},"ast":{"absolutePath":"src/SimpleContract.sol","id":158,"exportedSymbols":{"SimpleContract":[157]},"nodeType":"SourceUnit","src":"32:622:3","nodes":[{"id":105,"nodeType":"PragmaDirective","src":"32:23:3","nodes":[],"literals":["solidity","^","0.8",".0"]},{"id":157,"nodeType":"ContractDefinition","src":"57:596:3","nodes":[{"id":109,"nodeType":"EventDefinition","src":"87:24:3","nodes":[],"anonymous":false,"eventSelector":"166a7d625edff952ff346d1bca4edef10254353f72916b7fb072d55d0f97b544","name":"Trigger","nameLocation":"93:7:3","parameters":{"id":108,"nodeType":"ParameterList","parameters":[{"constant":false,"id":107,"indexed":false,"mutability":"mutable","name":"x","nameLocation":"108:1:3","nodeType":"VariableDeclaration","scope":109,"src":"101:8:3","stateVariable":false,"storageLocation":"default","typeDescriptions":{"typeIdentifier":"t_uint16","typeString":"uint16"},"typeName":{"id":106,"name":"uint16","nodeType":"ElementaryTypeName","src":"101:6:3","typeDescriptions":{"typeIdentifier":"t_uint16","typeString":"uint16"}},"visibility":"internal"}],"src":"100:10:3"}},{"id":119,"nodeType":"EventDefinition","src":"173:129:3","nodes":[],"anonymous":false,"eventSelector":"2cb351db58390c313534745d80b5f0abff9230502a6374a97b9caa76b31c5d8a","name":"AnotherTrigger","nameLocation":"179:14:3","parameters":{"id":118,"nodeType":"ParameterList","parameters":[{"constant":false,"id":111,"indexed":true,"mutability":"mutable","name":"a","nameLocation":"219:1:3","nodeType":"VariableDeclaration","scope":119,"src":"203:17:3","stateVariable":false,"storageLocation":"default","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"},"typeName":{"id":110,"name":"uint256","nodeType":"ElementaryTypeName","src":"203:7:3","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"}},"visibility":"internal"},{"constant":false,"id":113,"indexed":true,"mutability":"mutable","name":"b","nameLocation":"246:1:3","nodeType":"VariableDeclaration","scope":119,"src":"230:17:3","stateVariable":false,"storageLocation":"default","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"},"typeName":{"id":112,"name":"uint256","nodeType":"ElementaryTypeName","src":"230:7:3","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"}},"visibility":"internal"},{"constant":false,"id":115,"indexed":true,"mutability":"mutable","name":"c","nameLocation":"273:1:3","nodeType":"VariableDeclaration","scope":119,"src":"257:17:3","stateVariable":false,"storageLocation":"default","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"},"typeName":{"id":114,"name":"uint256","nodeType":"ElementaryTypeName","src":"257:7:3","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"}},"visibility":"internal"},{"constant":false,"id":117,"indexed":false,"mutability":"mutable","name":"data","nameLocation":"291:4:3","nodeType":"VariableDeclaration","scope":119,"src":"284:11:3","stateVariable":false,"storageLocation":"default","typeDescriptions":{"typeIdentifier":"t_string_memory_ptr","typeString":"string"},"typeName":{"id":116,"name":"string","nodeType":"ElementaryTypeName","src":"284:6:3","typeDescriptions":{"typeIdentifier":"t_string_storage_ptr","typeString":"string"}},"visibility":"internal"}],"src":"193:108:3"}},{"id":127,"nodeType":"FunctionDefinition","src":"308:46:3","nodes":[],"body":{"id":126,"nodeType":"Block","src":"322:32:3","nodes":[],"statements":[{"eventCall":{"arguments":[{"hexValue":"30","id":123,"isConstant":false,"isLValue":false,"isPure":true,"kind":"number","lValueRequested":false,"nodeType":"Literal","src":"345:1:3","typeDescriptions":{"typeIdentifier":"t_rational_0_by_1","typeString":"int_const 0"},"value":"0"}],"expression":{"argumentTypes":[{"typeIdentifier":"t_rational_0_by_1","typeString":"int_const 0"}],"id":122,"name":"Trigger","nodeType":"Identifier","overloadedDeclarations":[],"referencedDeclaration":109,"src":"337:7:3","typeDescriptions":{"typeIdentifier":"t_function_event_nonpayable$_t_uint16_$returns$__$","typeString":"function (uint16)"}},"id":124,"isConstant":false,"isLValue":false,"isPure":false,"kind":"functionCall","lValueRequested":false,"nameLocations":[],"names":[],"nodeType":"FunctionCall","src":"337:10:3","tryCall":false,"typeDescriptions":{"typeIdentifier":"t_tuple$__$","typeString":"tuple()"}},"id":125,"nodeType":"EmitStatement","src":"332:15:3"}]},"implemented":true,"kind":"constructor","modifiers":[],"name":"","nameLocation":"-1:-1:-1","parameters":{"id":120,"nodeType":"ParameterList","parameters":[],"src":"319:2:3"},"returnParameters":{"id":121,"nodeType":"ParameterList","parameters":[],"src":"322:0:3"},"scope":157,"stateMutability":"nonpayable","virtual":false,"visibility":"public"},{"id":137,"nodeType":"FunctionDefinition","src":"360:70:3","nodes":[],"body":{"id":136,"nodeType":"Block","src":"398:32:3","nodes":[],"statements":[{"eventCall":{"arguments":[{"id":133,"name":"x","nodeType":"Identifier","overloadedDeclarations":[],"referencedDeclaration":129,"src":"421:1:3","typeDescriptions":{"typeIdentifier":"t_uint16","typeString":"uint16"}}],"expression":{"argumentTypes":[{"typeIdentifier":"t_uint16","typeString":"uint16"}],"id":132,"name":"Trigger","nodeType":"Identifier","overloadedDeclarations":[],"referencedDeclaration":109,"src":"413:7:3","typeDescriptions":{"typeIdentifier":"t_function_event_nonpayable$_t_uint16_$returns$__$","typeString":"function (uint16)"}},"id":134,"isConstant":false,"isLValue":false,"isPure":false,"kind":"functionCall","lValueRequested":false,"nameLocations":[],"names":[],"nodeType":"FunctionCall","src":"413:10:3","tryCall":false,"typeDescriptions":{"typeIdentifier":"t_tuple$__$","typeString":"tuple()"}},"id":135,"nodeType":"EmitStatement","src":"408:15:3"}]},"functionSelector":"16d04e0d","implemented":true,"kind":"function","modifiers":[],"name":"emitTrigger","nameLocation":"369:11:3","parameters":{"id":130,"nodeType":"ParameterList","parameters":[{"constant":false,"id":129,"mutability":"mutable","name":"x","nameLocation":"388:1:3","nodeType":"VariableDeclaration","scope":137,"src":"381:8:3","stateVariable":false,"storageLocation":"default","typeDescriptions":{"typeIdentifier":"t_uint16","typeString":"uint16"},"typeName":{"id":128,"name":"uint16","nodeType":"ElementaryTypeName","src":"381:6:3","typeDescriptions":{"typeIdentifier":"t_uint16","typeString":"uint16"}},"visibility":"internal"}],"src":"380:10:3"},"returnParameters":{"id":131,"nodeType":"ParameterList","parameters":[],"src":"398:0:3"},"scope":157,"stateMutability":"nonpayable","virtual":false,"visibility":"public"},{"id":156,"nodeType":"FunctionDefinition","src":"474:177:3","nodes":[],"body":{"id":155,"nodeType":"Block","src":"600:51:3","nodes":[],"statements":[{"eventCall":{"arguments":[{"id":149,"name":"a","nodeType":"Identifier","overloadedDeclarations":[],"referencedDeclaration":139,"src":"630:1:3","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"}},{"id":150,"name":"b","nodeType":"Identifier","overloadedDeclarations":[],"referencedDeclaration":141,"src":"633:1:3","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"}},{"id":151,"name":"c","nodeType":"Identifier","overloadedDeclarations":[],"referencedDeclaration":143,"src":"636:1:3","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"}},{"id":152,"name":"data","nodeType":"Identifier","overloadedDeclarations":[],"referencedDeclaration":145,"src":"639:4:3","typeDescriptions":{"typeIdentifier":"t_string_memory_ptr","typeString":"string memory"}}],"expression":{"argumentTypes":[{"typeIdentifier":"t_uint256","typeString":"uint256"},{"typeIdentifier":"t_uint256","typeString":"uint256"},{"typeIdentifier":"t_uint256","typeString":"uint256"},{"typeIdentifier":"t_string_memory_ptr","typeString":"string memory"}],"id":148,"name":"AnotherTrigger","nodeType":"Identifier","overloadedDeclarations":[],"referencedDeclaration":119,"src":"615:14:3","typeDescriptions":{"typeIdentifier":"t_function_event_nonpayable$_t_uint256_$_t_uint256_$_t_uint256_$_t_string_memory_ptr_$returns$__$","typeString":"function (uint256,uint256,uint256,string memory)"}},"id":153,"isConstant":false,"isLValue":false,"isPure":false,"kind":"functionCall","lValueRequested":false,"nameLocations":[],"names":[],"nodeType":"FunctionCall","src":"615:29:3","tryCall":false,"typeDescriptions":{"typeIdentifier":"t_tuple$__$","typeString":"tuple()"}},"id":154,"nodeType":"EmitStatement","src":"610:34:3"}]},"functionSelector":"931919ea","implemented":true,"kind":"function","modifiers":[],"name":"emitAnotherTrigger","nameLocation":"483:18:3","parameters":{"id":146,"nodeType":"ParameterList","parameters":[{"constant":false,"id":139,"mutability":"mutable","name":"a","nameLocation":"519:1:3","nodeType":"VariableDeclaration","scope":156,"src":"511:9:3","stateVariable":false,"storageLocation":"default","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"},"typeName":{"id":138,"name":"uint256","nodeType":"ElementaryTypeName","src":"511:7:3","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"}},"visibility":"internal"},{"constant":false,"id":141,"mutability":"mutable","name":"b","nameLocation":"538:1:3","nodeType":"VariableDeclaration","scope":156,"src":"530:9:3","stateVariable":false,"storageLocation":"default","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"},"typeName":{"id":140,"name":"uint256","nodeType":"ElementaryTypeName","src":"530:7:3","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"}},"visibility":"internal"},{"constant":false,"id":143,"mutability":"mutable","name":"c","nameLocation":"557:1:3","nodeType":"VariableDeclaration","scope":156,"src":"549:9:3","stateVariable":false,"storageLocation":"default","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"},"typeName":{"id":142,"name":"uint256","nodeType":"ElementaryTypeName","src":"549:7:3","typeDescriptions":{"typeIdentifier":"t_uint256","typeString":"uint256"}},"visibility":"internal"},{"constant":false,"id":145,"mutability":"mutable","name":"data","nameLocation":"582:4:3","nodeType":"VariableDeclaration","scope":156,"src":"568:18:3","stateVariable":false,"storageLocation":"memory","typeDescriptions":{"typeIdentifier":"t_string_memory_ptr","typeString":"string"},"typeName":{"id":144,"name":"string","nodeType":"ElementaryTypeName","src":"568:6:3","typeDescriptions":{"typeIdentifier":"t_string_storage_ptr","typeString":"string"}},"visibility":"internal"}],"src":"501:91:3"},"returnParameters":{"id":147,"nodeType":"ParameterList","parameters":[],"src":"600:0:3"},"scope":157,"stateMutability":"nonpayable","virtual":false,"visibility":"public"}],"abstract":false,"baseContracts":[],"canonicalName":"SimpleContract","contractDependencies":[],"contractKind":"contract","fullyImplemented":true,"linearizedBaseContracts":[157],"name":"SimpleContract","nameLocation":"66:14:3","scope":158,"usedErrors":[]}],"license":"MIT"},"id":3} \ No newline at end of file diff --git a/tests/integration-tests/source-subgraph/abis/Contract.abi b/tests/integration-tests/source-subgraph/abis/Contract.abi new file mode 100644 index 00000000000..02da1a9e7f3 --- /dev/null +++ b/tests/integration-tests/source-subgraph/abis/Contract.abi @@ -0,0 +1,33 @@ +[ + { + "inputs": [], + "stateMutability": "nonpayable", + "type": "constructor" + }, + { + "anonymous": false, + "inputs": [ + { + "indexed": false, + "internalType": "uint16", + "name": "x", + "type": "uint16" + } + ], + "name": "Trigger", + "type": "event" + }, + { + "inputs": [ + { + "internalType": "uint16", + "name": "x", + "type": "uint16" + } + ], + "name": "emitTrigger", + "outputs": [], + "stateMutability": "nonpayable", + "type": "function" + } +] diff --git a/tests/integration-tests/source-subgraph/package.json b/tests/integration-tests/source-subgraph/package.json new file mode 100644 index 00000000000..6ca576d414e --- /dev/null +++ b/tests/integration-tests/source-subgraph/package.json @@ -0,0 +1,25 @@ +{ + "name": "source-subgraph", + "version": "0.1.0", + "scripts": { + "build-contracts": "../../common/build-contracts.sh", + "codegen": "graph codegen --skip-migrations", + "test": "yarn build-contracts && truffle test --compile-none --network test", + "create:test": "graph create test/source-subgraph --node $GRAPH_NODE_ADMIN_URI", + "deploy:test": "graph deploy test/source-subgraph --version-label v0.0.1 --ipfs $IPFS_URI --node $GRAPH_NODE_ADMIN_URI" + }, + "devDependencies": { + "@graphprotocol/graph-cli": "0.69.0", + "@graphprotocol/graph-ts": "0.34.0", + "solc": "^0.8.2" + }, + "dependencies": { + "@truffle/contract": "^4.3", + "@truffle/hdwallet-provider": "^1.2", + "apollo-fetch": "^0.7.0", + "babel-polyfill": "^6.26.0", + "babel-register": "^6.26.0", + "gluegun": "^4.6.1", + "truffle": "^5.2" + } +} diff --git a/tests/integration-tests/source-subgraph/schema.graphql b/tests/integration-tests/source-subgraph/schema.graphql new file mode 100644 index 00000000000..15bb2a33921 --- /dev/null +++ b/tests/integration-tests/source-subgraph/schema.graphql @@ -0,0 +1,13 @@ +type Block @entity { + id: ID! + number: BigInt! + hash: Bytes! + testMessage: String +} + +type Block2 @entity { + id: ID! + number: BigInt! + hash: Bytes! + testMessage: String +} diff --git a/tests/integration-tests/source-subgraph/src/mapping.ts b/tests/integration-tests/source-subgraph/src/mapping.ts new file mode 100644 index 00000000000..ad27c43c2a3 --- /dev/null +++ b/tests/integration-tests/source-subgraph/src/mapping.ts @@ -0,0 +1,57 @@ +import { ethereum, log, store } from '@graphprotocol/graph-ts'; +import { Block, Block2 } from '../generated/schema'; +import { BigInt } from '@graphprotocol/graph-ts'; + +export function handleBlock(block: ethereum.Block): void { + log.info('handleBlock {}', [block.number.toString()]); + + let id = block.number.toString().concat('-v1'); + let blockEntity = new Block(id); + blockEntity.number = block.number; + blockEntity.hash = block.hash; + blockEntity.save(); + + let id2 = block.number.toString().concat('-v2'); + let blockEntity2 = new Block(id2); + blockEntity2.number = block.number; + blockEntity2.hash = block.hash; + blockEntity2.save(); + + let id3 = block.number.toString().concat('-v3'); + let blockEntity3 = new Block2(id3); + blockEntity3.number = block.number; + blockEntity3.hash = block.hash; + blockEntity3.save(); + + if (block.number.equals(BigInt.fromI32(1))) { + let id = 'TEST'; + let entity = new Block(id); + entity.number = block.number; + entity.hash = block.hash; + entity.testMessage = 'Created at block 1'; + log.info('Created entity at block 1', []); + entity.save(); + } + + if (block.number.equals(BigInt.fromI32(2))) { + let id = 'TEST'; + let blockEntity1 = Block.load(id); + if (blockEntity1) { + // Update the block entity + blockEntity1.testMessage = 'Updated at block 2'; + log.info('Updated entity at block 2', []); + blockEntity1.save(); + } + } + + if (block.number.equals(BigInt.fromI32(3))) { + let id = 'TEST'; + let blockEntity1 = Block.load(id); + if (blockEntity1) { + blockEntity1.testMessage = 'Deleted at block 3'; + log.info('Deleted entity at block 3', []); + blockEntity1.save(); + store.remove('Block', id); + } + } +} diff --git a/tests/integration-tests/source-subgraph/subgraph.yaml b/tests/integration-tests/source-subgraph/subgraph.yaml new file mode 100644 index 00000000000..c531c44cb6c --- /dev/null +++ b/tests/integration-tests/source-subgraph/subgraph.yaml @@ -0,0 +1,23 @@ +specVersion: 0.0.8 +schema: + file: ./schema.graphql +dataSources: + - kind: ethereum/contract + name: BlockHandlerTest + network: test + source: + address: "@SimpleContract@" + abi: Contract + startBlock: 1 + mapping: + kind: ethereum/events + apiVersion: 0.0.7 + language: wasm/assemblyscript + abis: + - name: Contract + file: ./abis/Contract.abi + entities: + - Call + blockHandlers: + - handler: handleBlock + file: ./src/mapping.ts \ No newline at end of file diff --git a/tests/integration-tests/subgraph-data-sources/abis/Contract.abi b/tests/integration-tests/subgraph-data-sources/abis/Contract.abi new file mode 100644 index 00000000000..9d9f56b9263 --- /dev/null +++ b/tests/integration-tests/subgraph-data-sources/abis/Contract.abi @@ -0,0 +1,15 @@ +[ + { + "anonymous": false, + "inputs": [ + { + "indexed": false, + "internalType": "string", + "name": "testCommand", + "type": "string" + } + ], + "name": "TestEvent", + "type": "event" + } +] diff --git a/tests/integration-tests/subgraph-data-sources/package.json b/tests/integration-tests/subgraph-data-sources/package.json new file mode 100644 index 00000000000..87537290ad2 --- /dev/null +++ b/tests/integration-tests/subgraph-data-sources/package.json @@ -0,0 +1,13 @@ +{ + "name": "subgraph-data-sources", + "version": "0.1.0", + "scripts": { + "codegen": "graph codegen --skip-migrations", + "create:test": "graph create test/subgraph-data-sources --node $GRAPH_NODE_ADMIN_URI", + "deploy:test": "graph deploy test/subgraph-data-sources --version-label v0.0.1 --ipfs $IPFS_URI --node $GRAPH_NODE_ADMIN_URI" + }, + "devDependencies": { + "@graphprotocol/graph-cli": "0.79.0-alpha-20240711124603-49edf22", + "@graphprotocol/graph-ts": "0.31.0" + } +} diff --git a/tests/integration-tests/subgraph-data-sources/schema.graphql b/tests/integration-tests/subgraph-data-sources/schema.graphql new file mode 100644 index 00000000000..18c8153f8fd --- /dev/null +++ b/tests/integration-tests/subgraph-data-sources/schema.graphql @@ -0,0 +1,6 @@ +type MirrorBlock @entity { + id: String! + number: BigInt! + hash: Bytes! + testMessage: String +} diff --git a/tests/integration-tests/subgraph-data-sources/src/mapping.ts b/tests/integration-tests/subgraph-data-sources/src/mapping.ts new file mode 100644 index 00000000000..dc5743040f9 --- /dev/null +++ b/tests/integration-tests/subgraph-data-sources/src/mapping.ts @@ -0,0 +1,46 @@ +import { Entity, log, store } from '@graphprotocol/graph-ts'; +import { MirrorBlock } from '../generated/schema'; + +export class EntityTrigger { + constructor( + public entityOp: u32, + public entityType: string, + public entity: Entity, + public vid: i64, + ) {} +} + +export function handleEntity(trigger: EntityTrigger): void { + let blockEntity = trigger.entity; + let blockNumber = blockEntity.getBigInt('number'); + let blockHash = blockEntity.getBytes('hash'); + let testMessage = blockEntity.get('testMessage'); + let id = blockEntity.getString('id'); + + log.info('Block number: {}', [blockNumber.toString()]); + + if (trigger.entityOp == 2) { + log.info('Removing block entity with id: {}', [id]); + store.remove('MirrorBlock', id); + return; + } + + let block = loadOrCreateMirrorBlock(id); + block.number = blockNumber; + block.hash = blockHash; + if (testMessage) { + block.testMessage = testMessage.toString(); + } + + block.save(); +} + +export function loadOrCreateMirrorBlock(id: string): MirrorBlock { + let block = MirrorBlock.load(id); + if (!block) { + log.info('Creating new block entity with id: {}', [id]); + block = new MirrorBlock(id); + } + + return block; +} diff --git a/tests/integration-tests/subgraph-data-sources/subgraph.yaml b/tests/integration-tests/subgraph-data-sources/subgraph.yaml new file mode 100644 index 00000000000..cdcbcbabec7 --- /dev/null +++ b/tests/integration-tests/subgraph-data-sources/subgraph.yaml @@ -0,0 +1,21 @@ +specVersion: 1.3.0 +schema: + file: ./schema.graphql +dataSources: + - kind: subgraph + name: Contract + network: test + source: + address: 'Qmaqf8cRxfxbduZppSHKG9DMuX5JZPMoGuwGb2DQuo48sq' + startBlock: 0 + mapping: + apiVersion: 0.0.7 + language: wasm/assemblyscript + entities: + - Gravatar + handlers: + - handler: handleEntity + entity: Block + - handler: handleEntity + entity: Block2 + file: ./src/mapping.ts diff --git a/tests/integration-tests/yarn.lock b/tests/integration-tests/yarn.lock index f81274832bf..e3115d31258 100644 --- a/tests/integration-tests/yarn.lock +++ b/tests/integration-tests/yarn.lock @@ -738,6 +738,47 @@ which "2.0.2" yaml "1.10.2" +"@graphprotocol/graph-cli@0.79.0-alpha-20240711124603-49edf22": + version "0.79.0-alpha-20240711124603-49edf22" + resolved "https://registry.yarnpkg.com/@graphprotocol/graph-cli/-/graph-cli-0.79.0-alpha-20240711124603-49edf22.tgz#4e3f6201932a0b68ce64d6badd8432cf2bead3c2" + integrity sha512-fZrdPiFbbbBVMnvsjfKA+j48WzzquaHQIpozBqnUKRPCV1n1NenIaq2nH16mlMwovRIS7AAIVCpa0QYQuPzw7Q== + dependencies: + "@float-capital/float-subgraph-uncrashable" "^0.0.0-alpha.4" + "@oclif/core" "2.8.6" + "@oclif/plugin-autocomplete" "^2.3.6" + "@oclif/plugin-not-found" "^2.4.0" + "@whatwg-node/fetch" "^0.8.4" + assemblyscript "0.19.23" + binary-install-raw "0.0.13" + chalk "3.0.0" + chokidar "3.5.3" + debug "4.3.4" + docker-compose "0.23.19" + dockerode "2.5.8" + fs-extra "9.1.0" + glob "9.3.5" + gluegun "5.1.6" + graphql "15.5.0" + immutable "4.2.1" + ipfs-http-client "55.0.0" + jayson "4.0.0" + js-yaml "3.14.1" + open "8.4.2" + prettier "3.0.3" + semver "7.4.0" + sync-request "6.1.0" + tmp-promise "3.0.3" + web3-eth-abi "1.7.0" + which "2.0.2" + yaml "1.10.2" + +"@graphprotocol/graph-ts@0.31.0": + version "0.31.0" + resolved "https://registry.yarnpkg.com/@graphprotocol/graph-ts/-/graph-ts-0.31.0.tgz#730668c0369828b31bef81e8d9bc66b9b48e3480" + integrity sha512-xreRVM6ho2BtolyOh2flDkNoGZximybnzUnF53zJVp0+Ed0KnAlO1/KOCUYw06euVI9tk0c9nA2Z/D5SIQV2Rg== + dependencies: + assemblyscript "0.19.10" + "@graphprotocol/graph-ts@0.34.0": version "0.34.0" resolved "https://registry.yarnpkg.com/@graphprotocol/graph-ts/-/graph-ts-0.34.0.tgz#ca47398295b114f25b412faa364b98af31fa2bb7" @@ -3544,6 +3585,11 @@ define-data-property@^1.1.2: es-errors "^1.3.0" gopd "^1.0.1" +define-lazy-prop@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/define-lazy-prop/-/define-lazy-prop-2.0.0.tgz#3f7ae421129bcaaac9bc74905c98a0009ec9ee7f" + integrity sha512-Ds09qNh8yw3khSjiJjiUInaGX9xlqZDY7JVryGxdxV7NPeuqQfplOpQ66yJFZut3jLa5zOwkXw1g9EI2uKh4Og== + delay@^5.0.0: version "5.0.0" resolved "https://registry.yarnpkg.com/delay/-/delay-5.0.0.tgz#137045ef1b96e5071060dd5be60bf9334436bd1d" @@ -5429,7 +5475,7 @@ is-core-module@^2.13.0: dependencies: hasown "^2.0.0" -is-docker@^2.0.0: +is-docker@^2.0.0, is-docker@^2.1.1: version "2.2.1" resolved "https://registry.yarnpkg.com/is-docker/-/is-docker-2.2.1.tgz#33eeabe23cfe86f14bde4408a02c0cfb853acdaa" integrity sha512-F+i2BKsFrH66iaUFc0woD8sLy8getkwTwtOBjvs56Cx4CgJDeKQeqfz8wAYiSb8JOprWhHH5p77PbmYCvvUuXQ== @@ -6883,6 +6929,15 @@ onetime@^5.1.0, onetime@^5.1.2: dependencies: mimic-fn "^2.1.0" +open@8.4.2: + version "8.4.2" + resolved "https://registry.yarnpkg.com/open/-/open-8.4.2.tgz#5b5ffe2a8f793dcd2aad73e550cb87b59cb084f9" + integrity sha512-7x81NCL719oNbsq/3mh+hVrAWmFuEYUqrq/Iw3kUzH8ReypT9QQ0BLoJS7/G9k6N81XjW4qHWtjWwe/9eLy1EQ== + dependencies: + define-lazy-prop "^2.0.0" + is-docker "^2.1.1" + is-wsl "^2.2.0" + ora@4.0.2: version "4.0.2" resolved "https://registry.yarnpkg.com/ora/-/ora-4.0.2.tgz#0e1e68fd45b135d28648b27cf08081fa6e8a297d" diff --git a/tests/runner-tests/subgraph-data-sources/abis/Contract.abi b/tests/runner-tests/subgraph-data-sources/abis/Contract.abi new file mode 100644 index 00000000000..9d9f56b9263 --- /dev/null +++ b/tests/runner-tests/subgraph-data-sources/abis/Contract.abi @@ -0,0 +1,15 @@ +[ + { + "anonymous": false, + "inputs": [ + { + "indexed": false, + "internalType": "string", + "name": "testCommand", + "type": "string" + } + ], + "name": "TestEvent", + "type": "event" + } +] diff --git a/tests/runner-tests/subgraph-data-sources/package.json b/tests/runner-tests/subgraph-data-sources/package.json new file mode 100644 index 00000000000..87537290ad2 --- /dev/null +++ b/tests/runner-tests/subgraph-data-sources/package.json @@ -0,0 +1,13 @@ +{ + "name": "subgraph-data-sources", + "version": "0.1.0", + "scripts": { + "codegen": "graph codegen --skip-migrations", + "create:test": "graph create test/subgraph-data-sources --node $GRAPH_NODE_ADMIN_URI", + "deploy:test": "graph deploy test/subgraph-data-sources --version-label v0.0.1 --ipfs $IPFS_URI --node $GRAPH_NODE_ADMIN_URI" + }, + "devDependencies": { + "@graphprotocol/graph-cli": "0.79.0-alpha-20240711124603-49edf22", + "@graphprotocol/graph-ts": "0.31.0" + } +} diff --git a/tests/runner-tests/subgraph-data-sources/schema.graphql b/tests/runner-tests/subgraph-data-sources/schema.graphql new file mode 100644 index 00000000000..6f97fa65c43 --- /dev/null +++ b/tests/runner-tests/subgraph-data-sources/schema.graphql @@ -0,0 +1,6 @@ +type Data @entity { + id: ID! + foo: String + bar: Int + isTest: Boolean +} diff --git a/tests/runner-tests/subgraph-data-sources/src/mapping.ts b/tests/runner-tests/subgraph-data-sources/src/mapping.ts new file mode 100644 index 00000000000..cd5c1d4dcd1 --- /dev/null +++ b/tests/runner-tests/subgraph-data-sources/src/mapping.ts @@ -0,0 +1,35 @@ +import { Entity, log } from '@graphprotocol/graph-ts'; + +export const SubgraphEntityOpCreate: u32 = 0; +export const SubgraphEntityOpModify: u32 = 1; +export const SubgraphEntityOpDelete: u32 = 2; + +export class EntityTrigger { + constructor( + public entityOp: u32, + public entityType: string, + public entity: Entity, + public vid: i64, + ) {} +} + +export function handleBlock(content: EntityTrigger): void { + let stringContent = content.entity.getString('val'); + log.info('Content: {}', [stringContent]); + log.info('EntityOp: {}', [content.entityOp.toString()]); + + switch (content.entityOp) { + case SubgraphEntityOpCreate: { + log.info('Entity created: {}', [content.entityType]); + break + } + case SubgraphEntityOpModify: { + log.info('Entity modified: {}', [content.entityType]); + break; + } + case SubgraphEntityOpDelete: { + log.info('Entity deleted: {}', [content.entityType]); + break; + } + } +} diff --git a/tests/runner-tests/subgraph-data-sources/subgraph.yaml b/tests/runner-tests/subgraph-data-sources/subgraph.yaml new file mode 100644 index 00000000000..01f719d069f --- /dev/null +++ b/tests/runner-tests/subgraph-data-sources/subgraph.yaml @@ -0,0 +1,19 @@ +specVersion: 1.3.0 +schema: + file: ./schema.graphql +dataSources: + - kind: subgraph + name: Contract + network: test + source: + address: 'QmRFXhvyvbm4z5Lo7z2mN9Ckmo623uuB2jJYbRmAXgYKXJ' + startBlock: 0 + mapping: + apiVersion: 0.0.7 + language: wasm/assemblyscript + entities: + - Gravatar + handlers: + - handler: handleBlock + entity: User + file: ./src/mapping.ts diff --git a/tests/runner-tests/yarn.lock b/tests/runner-tests/yarn.lock index 50e0c2b471f..9f3bdae834d 100644 --- a/tests/runner-tests/yarn.lock +++ b/tests/runner-tests/yarn.lock @@ -349,6 +349,40 @@ which "2.0.2" yaml "1.10.2" +"@graphprotocol/graph-cli@0.79.0-alpha-20240711124603-49edf22": + version "0.79.0-alpha-20240711124603-49edf22" + resolved "https://registry.yarnpkg.com/@graphprotocol/graph-cli/-/graph-cli-0.79.0-alpha-20240711124603-49edf22.tgz#4e3f6201932a0b68ce64d6badd8432cf2bead3c2" + integrity sha512-fZrdPiFbbbBVMnvsjfKA+j48WzzquaHQIpozBqnUKRPCV1n1NenIaq2nH16mlMwovRIS7AAIVCpa0QYQuPzw7Q== + dependencies: + "@float-capital/float-subgraph-uncrashable" "^0.0.0-alpha.4" + "@oclif/core" "2.8.6" + "@oclif/plugin-autocomplete" "^2.3.6" + "@oclif/plugin-not-found" "^2.4.0" + "@whatwg-node/fetch" "^0.8.4" + assemblyscript "0.19.23" + binary-install-raw "0.0.13" + chalk "3.0.0" + chokidar "3.5.3" + debug "4.3.4" + docker-compose "0.23.19" + dockerode "2.5.8" + fs-extra "9.1.0" + glob "9.3.5" + gluegun "5.1.6" + graphql "15.5.0" + immutable "4.2.1" + ipfs-http-client "55.0.0" + jayson "4.0.0" + js-yaml "3.14.1" + open "8.4.2" + prettier "3.0.3" + semver "7.4.0" + sync-request "6.1.0" + tmp-promise "3.0.3" + web3-eth-abi "1.7.0" + which "2.0.2" + yaml "1.10.2" + "@graphprotocol/graph-ts@0.30.0": version "0.30.0" resolved "https://registry.npmjs.org/@graphprotocol/graph-ts/-/graph-ts-0.30.0.tgz" @@ -1473,6 +1507,11 @@ defaults@^1.0.3: dependencies: clone "^1.0.2" +define-lazy-prop@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/define-lazy-prop/-/define-lazy-prop-2.0.0.tgz#3f7ae421129bcaaac9bc74905c98a0009ec9ee7f" + integrity sha512-Ds09qNh8yw3khSjiJjiUInaGX9xlqZDY7JVryGxdxV7NPeuqQfplOpQ66yJFZut3jLa5zOwkXw1g9EI2uKh4Og== + delay@^5.0.0: version "5.0.0" resolved "https://registry.npmjs.org/delay/-/delay-5.0.0.tgz" @@ -1545,6 +1584,13 @@ ejs@3.1.6: dependencies: jake "^10.6.1" +ejs@3.1.8: + version "3.1.8" + resolved "https://registry.yarnpkg.com/ejs/-/ejs-3.1.8.tgz#758d32910c78047585c7ef1f92f9ee041c1c190b" + integrity sha512-/sXZeMlhS0ArkfX2Aw780gJzXSMPnKjtspYZv+f3NiKLlubezAHDU5+9xz6gd3/NhG3txQCo6xlglmTS+oTGEQ== + dependencies: + jake "^10.8.5" + ejs@^3.1.8: version "3.1.9" resolved "https://registry.npmjs.org/ejs/-/ejs-3.1.9.tgz" @@ -1996,6 +2042,42 @@ gluegun@5.1.2: which "2.0.2" yargs-parser "^21.0.0" +gluegun@5.1.6: + version "5.1.6" + resolved "https://registry.yarnpkg.com/gluegun/-/gluegun-5.1.6.tgz#74ec13193913dc610f5c1a4039972c70c96a7bad" + integrity sha512-9zbi4EQWIVvSOftJWquWzr9gLX2kaDgPkNR5dYWbM53eVvCI3iKuxLlnKoHC0v4uPoq+Kr/+F569tjoFbA4DSA== + dependencies: + apisauce "^2.1.5" + app-module-path "^2.2.0" + cli-table3 "0.6.0" + colors "1.4.0" + cosmiconfig "7.0.1" + cross-spawn "7.0.3" + ejs "3.1.8" + enquirer "2.3.6" + execa "5.1.1" + fs-jetpack "4.3.1" + lodash.camelcase "^4.3.0" + lodash.kebabcase "^4.1.1" + lodash.lowercase "^4.3.0" + lodash.lowerfirst "^4.3.1" + lodash.pad "^4.5.1" + lodash.padend "^4.6.1" + lodash.padstart "^4.6.1" + lodash.repeat "^4.1.0" + lodash.snakecase "^4.1.1" + lodash.startcase "^4.4.0" + lodash.trim "^4.5.1" + lodash.trimend "^4.5.1" + lodash.trimstart "^4.5.1" + lodash.uppercase "^4.3.0" + lodash.upperfirst "^4.3.1" + ora "4.0.2" + pluralize "^8.0.0" + semver "7.3.5" + which "2.0.2" + yargs-parser "^21.0.0" + graceful-fs@^4.1.6, graceful-fs@^4.2.0: version "4.2.11" resolved "https://registry.yarnpkg.com/graceful-fs/-/graceful-fs-4.2.11.tgz#4183e4e8bf08bb6e05bbb2f7d2e0c8f712ca40e3" @@ -2282,7 +2364,7 @@ is-binary-path@~2.1.0: dependencies: binary-extensions "^2.0.0" -is-docker@^2.0.0: +is-docker@^2.0.0, is-docker@^2.1.1: version "2.2.1" resolved "https://registry.npmjs.org/is-docker/-/is-docker-2.2.1.tgz" integrity sha512-F+i2BKsFrH66iaUFc0woD8sLy8getkwTwtOBjvs56Cx4CgJDeKQeqfz8wAYiSb8JOprWhHH5p77PbmYCvvUuXQ== @@ -2922,6 +3004,15 @@ onetime@^5.1.0, onetime@^5.1.2: dependencies: mimic-fn "^2.1.0" +open@8.4.2: + version "8.4.2" + resolved "https://registry.yarnpkg.com/open/-/open-8.4.2.tgz#5b5ffe2a8f793dcd2aad73e550cb87b59cb084f9" + integrity sha512-7x81NCL719oNbsq/3mh+hVrAWmFuEYUqrq/Iw3kUzH8ReypT9QQ0BLoJS7/G9k6N81XjW4qHWtjWwe/9eLy1EQ== + dependencies: + define-lazy-prop "^2.0.0" + is-docker "^2.1.1" + is-wsl "^2.2.0" + ora@4.0.2: version "4.0.2" resolved "https://registry.npmjs.org/ora/-/ora-4.0.2.tgz" @@ -3042,6 +3133,11 @@ prettier@1.19.1: resolved "https://registry.npmjs.org/prettier/-/prettier-1.19.1.tgz" integrity sha512-s7PoyDv/II1ObgQunCbB9PdLmUcBZcnWOcxDh7O0N/UwDEsHyqkW+Qh28jW+mVuCdx7gLB0BotYI1Y6uI9iyew== +prettier@3.0.3: + version "3.0.3" + resolved "https://registry.yarnpkg.com/prettier/-/prettier-3.0.3.tgz#432a51f7ba422d1469096c0fdc28e235db8f9643" + integrity sha512-L/4pUDMxcNa8R/EthV08Zt42WBO4h1rarVtK0K+QJG0X187OLo7l699jWw0GKuwzkPQ//jMFA/8Xm6Fh3J/DAg== + process-nextick-args@~2.0.0: version "2.0.1" resolved "https://registry.npmjs.org/process-nextick-args/-/process-nextick-args-2.0.1.tgz" diff --git a/tests/src/fixture/ethereum.rs b/tests/src/fixture/ethereum.rs index b20672ce563..2ff94744f8e 100644 --- a/tests/src/fixture/ethereum.rs +++ b/tests/src/fixture/ethereum.rs @@ -6,12 +6,15 @@ use super::{ test_ptr, CommonChainConfig, MutexBlockStreamBuilder, NoopAdapterSelector, NoopRuntimeAdapterBuilder, StaticBlockRefetcher, StaticStreamBuilder, Stores, TestChain, }; +use graph::blockchain::block_stream::{EntityOperationKind, EntitySourceOperation}; use graph::blockchain::client::ChainClient; -use graph::blockchain::{BlockPtr, TriggersAdapterSelector}; +use graph::blockchain::{BlockPtr, Trigger, TriggersAdapterSelector}; use graph::cheap_clone::CheapClone; +use graph::data_source::subgraph; use graph::prelude::ethabi::ethereum_types::H256; use graph::prelude::web3::types::{Address, Log, Transaction, H160}; -use graph::prelude::{ethabi, tiny_keccak, LightEthereumBlock, ENV_VARS}; +use graph::prelude::{ethabi, tiny_keccak, DeploymentHash, Entity, LightEthereumBlock, ENV_VARS}; +use graph::schema::EntityType; use graph::{blockchain::block_stream::BlockWithTriggers, prelude::ethabi::ethereum_types::U64}; use graph_chain_ethereum::network::EthereumNetworkAdapters; use graph_chain_ethereum::trigger::LogRef; @@ -81,7 +84,10 @@ pub fn genesis() -> BlockWithTriggers { number: Some(U64::from(ptr.number)), ..Default::default() })), - trigger_data: vec![EthereumTrigger::Block(ptr, EthereumBlockTriggerType::End)], + trigger_data: vec![Trigger::Chain(EthereumTrigger::Block( + ptr, + EthereumBlockTriggerType::End, + ))], } } @@ -128,7 +134,10 @@ pub fn empty_block(parent_ptr: BlockPtr, ptr: BlockPtr) -> BlockWithTriggers, payload: impl Into, + source: DeploymentHash, + entity: Entity, + entity_type: EntityType, + entity_op: EntityOperationKind, + vid: i64, +) { + let entity = EntitySourceOperation { + entity: entity, + entity_type: entity_type, + entity_op: entity_op, + vid, + }; + + block + .trigger_data + .push(Trigger::Subgraph(subgraph::TriggerData { source, entity })); } pub fn push_test_command( @@ -175,12 +206,16 @@ pub fn push_test_command( }); block .trigger_data - .push(EthereumTrigger::Log(LogRef::FullLog(log, None))) + .push(Trigger::Chain(EthereumTrigger::Log(LogRef::FullLog( + log, None, + )))) } pub fn push_test_polling_trigger(block: &mut BlockWithTriggers) { - block.trigger_data.push(EthereumTrigger::Block( - block.ptr(), - EthereumBlockTriggerType::End, - )) + block + .trigger_data + .push(Trigger::Chain(EthereumTrigger::Block( + block.ptr(), + EthereumBlockTriggerType::End, + ))) } diff --git a/tests/src/fixture/mod.rs b/tests/src/fixture/mod.rs index ebed1d3a115..5c272e8d440 100644 --- a/tests/src/fixture/mod.rs +++ b/tests/src/fixture/mod.rs @@ -1,7 +1,7 @@ pub mod ethereum; pub mod substreams; -use std::collections::HashMap; +use std::collections::{HashMap, HashSet}; use std::marker::PhantomData; use std::sync::Mutex; use std::time::{Duration, Instant}; @@ -14,17 +14,18 @@ use graph::blockchain::block_stream::{ }; use graph::blockchain::{ Block, BlockHash, BlockPtr, Blockchain, BlockchainMap, ChainIdentifier, RuntimeAdapter, - TriggersAdapter, TriggersAdapterSelector, + TriggerFilterWrapper, TriggersAdapter, TriggersAdapterSelector, }; use graph::cheap_clone::CheapClone; use graph::components::adapter::ChainId; use graph::components::link_resolver::{ArweaveClient, ArweaveResolver, FileSizeLimit}; use graph::components::metrics::MetricsRegistry; -use graph::components::store::{BlockStore, DeploymentLocator, EthereumCallCache}; +use graph::components::store::{BlockStore, DeploymentLocator, EthereumCallCache, SourceableStore}; use graph::components::subgraph::Settings; use graph::data::graphql::load_manager::LoadManager; use graph::data::query::{Query, QueryTarget}; use graph::data::subgraph::schema::{SubgraphError, SubgraphHealth}; +use graph::data_source::DataSource; use graph::endpoint::EndpointMetrics; use graph::env::EnvVars; use graph::firehose::{FirehoseEndpoint, FirehoseEndpoints, NoopGenesisDecoder, SubgraphLimit}; @@ -211,13 +212,14 @@ impl TestContext { let tp: Box> = Box::new(SubgraphTriggerProcessor {}); self.instance_manager - .build_subgraph_runner( + .build_subgraph_runner_inner( logger, self.env_vars.cheap_clone(), deployment, raw, Some(stop_block.block_number()), tp, + true, ) .await .unwrap() @@ -236,13 +238,14 @@ impl TestContext { ); self.instance_manager - .build_subgraph_runner( + .build_subgraph_runner_inner( logger, self.env_vars.cheap_clone(), deployment, raw, Some(stop_block.block_number()), tp, + true, ) .await .unwrap() @@ -718,14 +721,27 @@ impl BlockStreamBuilder for MutexBlockStreamBuilder { async fn build_polling( &self, - _chain: &C, - _deployment: DeploymentLocator, - _start_blocks: Vec, - _subgraph_current_block: Option, - _filter: Arc<::TriggerFilter>, - _unified_api_version: graph::data::subgraph::UnifiedMappingApiVersion, + chain: &C, + deployment: DeploymentLocator, + start_blocks: Vec, + source_subgraph_stores: Vec>, + subgraph_current_block: Option, + filter: Arc>, + unified_api_version: graph::data::subgraph::UnifiedMappingApiVersion, ) -> anyhow::Result>> { - unimplemented!("only firehose mode should be used for tests") + let builder = self.0.lock().unwrap().clone(); + + builder + .build_polling( + chain, + deployment, + start_blocks, + source_subgraph_stores, + subgraph_current_block, + filter, + unified_api_version, + ) + .await } } @@ -783,11 +799,22 @@ where _chain: &C, _deployment: DeploymentLocator, _start_blocks: Vec, - _subgraph_current_block: Option, - _filter: Arc, + _source_subgraph_stores: Vec>, + subgraph_current_block: Option, + _filter: Arc>, _unified_api_version: graph::data::subgraph::UnifiedMappingApiVersion, ) -> anyhow::Result>> { - unimplemented!("only firehose mode should be used for tests") + let current_idx = subgraph_current_block.map(|current_block| { + self.chain + .iter() + .enumerate() + .find(|(_, b)| b.ptr() == current_block) + .unwrap() + .0 + }); + Ok(Box::new(StaticStream { + stream: Box::pin(stream_events(self.chain.clone(), current_idx)), + })) } } @@ -866,10 +893,7 @@ struct NoopRuntimeAdapter { } impl RuntimeAdapter for NoopRuntimeAdapter { - fn host_fns( - &self, - _ds: &::DataSource, - ) -> Result, Error> { + fn host_fns(&self, _ds: &DataSource) -> Result, Error> { Ok(vec![]) } } @@ -952,11 +976,23 @@ impl TriggersAdapter for MockTriggersAdapter { todo!() } + async fn load_block_ptrs_by_numbers( + &self, + _logger: Logger, + _block_numbers: HashSet, + ) -> Result, Error> { + unimplemented!() + } + + async fn chain_head_ptr(&self) -> Result, Error> { + todo!() + } + async fn scan_triggers( &self, _from: BlockNumber, _to: BlockNumber, - _filter: &::TriggerFilter, + _filter: &C::TriggerFilter, ) -> Result<(Vec>, BlockNumber), Error> { todo!() } diff --git a/tests/tests/integration_tests.rs b/tests/tests/integration_tests.rs index f2ff40f9ad2..e317050edc7 100644 --- a/tests/tests/integration_tests.rs +++ b/tests/tests/integration_tests.rs @@ -13,7 +13,7 @@ use std::future::Future; use std::pin::Pin; use std::time::{Duration, Instant}; -use anyhow::{anyhow, bail, Context}; +use anyhow::{anyhow, bail, Context, Result}; use graph::futures03::StreamExt; use graph::prelude::serde_json::{json, Value}; use graph::prelude::web3::types::U256; @@ -95,6 +95,7 @@ impl TestResult { struct TestCase { name: String, test: TestFn, + source_subgraph: Option, } impl TestCase { @@ -112,10 +113,87 @@ impl TestCase { Self { name: name.to_string(), test: force_boxed(test), + source_subgraph: None, } } + fn new_with_source_subgraph( + name: &str, + test: fn(TestContext) -> T, + source_subgraph: &str, + ) -> Self + where + T: Future> + Send + 'static, + { + fn force_boxed(f: fn(TestContext) -> T) -> TestFn + where + T: Future> + Send + 'static, + { + Box::new(move |ctx| Box::pin(f(ctx))) + } + + Self { + name: name.to_string(), + test: force_boxed(test), + source_subgraph: Some(source_subgraph.to_string()), + } + } + + async fn deploy_and_wait( + &self, + subgraph_name: &str, + contracts: &[Contract], + ) -> Result { + status!(&self.name, "Deploying subgraph"); + let subgraph_name = match Subgraph::deploy(&subgraph_name, contracts).await { + Ok(name) => name, + Err(e) => { + error!(&self.name, "Deploy failed"); + return Err(anyhow!(e.context("Deploy failed"))); + } + }; + + status!(&self.name, "Waiting for subgraph to become ready"); + let subgraph = match Subgraph::wait_ready(&subgraph_name).await { + Ok(subgraph) => subgraph, + Err(e) => { + error!(&self.name, "Subgraph never synced or failed"); + return Err(anyhow!(e.context("Subgraph never synced or failed"))); + } + }; + + if subgraph.healthy { + status!(&self.name, "Subgraph ({}) is synced", subgraph.deployment); + } else { + status!(&self.name, "Subgraph ({}) has failed", subgraph.deployment); + } + + Ok(subgraph) + } + async fn run(self, contracts: &[Contract]) -> TestResult { + // If a subgraph has a subgraph datasource, then deploy the source subgraph first + if let Some(source_subgraph) = &self.source_subgraph { + let subgraph = self.deploy_and_wait(source_subgraph, contracts).await; + match subgraph { + Ok(subgraph) => { + status!( + source_subgraph, + "source subgraph deployed with hash {}", + subgraph.deployment + ); + } + Err(e) => { + error!(source_subgraph, "source subgraph deployment failed"); + return TestResult { + name: self.name.clone(), + subgraph: None, + status: TestStatus::Err(e), + }; + } + } + } + status!(&self.name, "Deploying subgraph"); let subgraph_name = match Subgraph::deploy(&self.name, contracts).await { Ok(name) => name, @@ -439,6 +517,91 @@ async fn test_eth_api(ctx: TestContext) -> anyhow::Result<()> { Ok(()) } +async fn subgraph_data_sources(ctx: TestContext) -> anyhow::Result<()> { + let subgraph = ctx.subgraph; + assert!(subgraph.healthy); + let expected_response = json!({ + "mirrorBlocks": [ + { "id": "1-v1", "number": "1" }, + { "id": "1-v2", "number": "1" }, + { "id": "1-v3", "number": "1" }, + { "id": "2-v1", "number": "2" }, + { "id": "2-v2", "number": "2" }, + { "id": "2-v3", "number": "2" }, + { "id": "3-v1", "number": "3" }, + { "id": "3-v2", "number": "3" }, + { "id": "3-v3", "number": "3" }, + { "id": "4-v1", "number": "4" }, + { "id": "4-v2", "number": "4" }, + { "id": "4-v3", "number": "4" }, + { "id": "5-v1", "number": "5" }, + { "id": "5-v2", "number": "5" }, + { "id": "5-v3", "number": "5" }, + { "id": "6-v1", "number": "6" }, + { "id": "6-v2", "number": "6" }, + { "id": "6-v3", "number": "6" }, + { "id": "7-v1", "number": "7" }, + { "id": "7-v2", "number": "7" }, + { "id": "7-v3", "number": "7" }, + { "id": "8-v1", "number": "8" }, + { "id": "8-v2", "number": "8" }, + { "id": "8-v3", "number": "8" }, + { "id": "9-v1", "number": "9" }, + { "id": "9-v2", "number": "9" }, + { "id": "9-v3", "number": "9" }, + { "id": "10-v1", "number": "10" }, + { "id": "10-v2", "number": "10" }, + { "id": "10-v3", "number": "10" }, + ] + }); + + query_succeeds( + "Blocks should be right", + &subgraph, + "{ mirrorBlocks(where: {number_lte: 10}, orderBy: number) { id, number } }", + expected_response, + ) + .await?; + + let expected_response = json!({ + "mirrorBlock": { "id": "TEST", "number": "1", "testMessage": "Created at block 1" }, + }); + + query_succeeds( + "Blocks should be right", + &subgraph, + "{ mirrorBlock(id: \"TEST\", block: {number: 1}) { id, number, testMessage } }", + expected_response, + ) + .await?; + + let expected_response = json!({ + "mirrorBlock": { "id": "TEST", "number": "1", "testMessage": "Updated at block 2" }, + }); + + query_succeeds( + "Blocks should be right", + &subgraph, + "{ mirrorBlock(id: \"TEST\", block: {number: 2}) { id, number, testMessage } }", + expected_response, + ) + .await?; + + let expected_response = json!({ + "mirrorBlock": null, + }); + + query_succeeds( + "Blocks should be right", + &subgraph, + "{ mirrorBlock(id: \"TEST\", block: {number: 3}) { id, number, testMessage } }", + expected_response, + ) + .await?; + + Ok(()) +} + async fn test_topic_filters(ctx: TestContext) -> anyhow::Result<()> { let subgraph = ctx.subgraph; assert!(subgraph.healthy); @@ -790,6 +953,11 @@ async fn integration_tests() -> anyhow::Result<()> { TestCase::new("timestamp", test_timestamp), TestCase::new("ethereum-api-tests", test_eth_api), TestCase::new("topic-filter", test_topic_filters), + TestCase::new_with_source_subgraph( + "subgraph-data-sources", + subgraph_data_sources, + "source-subgraph", + ), ]; // Filter the test cases if a specific test name is provided diff --git a/tests/tests/runner_tests.rs b/tests/tests/runner_tests.rs index 7da707ac7cd..a6eb3200829 100644 --- a/tests/tests/runner_tests.rs +++ b/tests/tests/runner_tests.rs @@ -6,7 +6,7 @@ use std::sync::Arc; use std::time::Duration; use assert_json_diff::assert_json_eq; -use graph::blockchain::block_stream::BlockWithTriggers; +use graph::blockchain::block_stream::{BlockWithTriggers, EntityOperationKind}; use graph::blockchain::{Block, BlockPtr, Blockchain}; use graph::data::store::scalar::Bytes; use graph::data::subgraph::schema::{SubgraphError, SubgraphHealth}; @@ -18,11 +18,12 @@ use graph::object; use graph::prelude::ethabi::ethereum_types::H256; use graph::prelude::web3::types::Address; use graph::prelude::{ - hex, CheapClone, DeploymentHash, SubgraphAssignmentProvider, SubgraphName, SubgraphStore, + hex, CheapClone, DeploymentHash, SubgraphAssignmentProvider, SubgraphName, SubgraphStore, Value, }; +use graph::schema::InputSchema; use graph_tests::fixture::ethereum::{ chain, empty_block, generate_empty_blocks_for_range, genesis, push_test_command, push_test_log, - push_test_polling_trigger, + push_test_polling_trigger, push_test_subgraph_trigger, }; use graph_tests::fixture::substreams::chain as substreams_chain; @@ -500,10 +501,19 @@ async fn substreams_trigger_filter_construction() -> anyhow::Result<()> { let runner = ctx.runner_substreams(test_ptr(0)).await; let filter = runner.build_filter_for_test(); - assert_eq!(filter.module_name(), "graph_out"); - assert_eq!(filter.modules().as_ref().unwrap().modules.len(), 2); - assert_eq!(filter.start_block().unwrap(), 0); - assert_eq!(filter.data_sources_len(), 1); + assert_eq!(filter.chain_filter.module_name(), "graph_out"); + assert_eq!( + filter + .chain_filter + .modules() + .as_ref() + .unwrap() + .modules + .len(), + 2 + ); + assert_eq!(filter.chain_filter.start_block().unwrap(), 0); + assert_eq!(filter.chain_filter.data_sources_len(), 1); Ok(()) } @@ -525,7 +535,11 @@ async fn end_block() -> anyhow::Result<()> { let runner = ctx.runner(block_ptr.clone()).await; let runner = runner.run_for_test(false).await.unwrap(); let filter = runner.context().filter.as_ref().unwrap(); - let addresses = filter.log().contract_addresses().collect::>(); + let addresses = filter + .chain_filter + .log() + .contract_addresses() + .collect::>(); if should_contain_addr { assert!(addresses.contains(&addr)); @@ -1077,6 +1091,54 @@ async fn parse_data_source_context() { ); } +#[tokio::test] +async fn subgraph_data_sources() { + let RunnerTestRecipe { stores, test_info } = + RunnerTestRecipe::new("subgraph-data-sources", "subgraph-data-sources").await; + + let schema = InputSchema::parse_latest( + "type User @entity { id: String!, val: String! }", + DeploymentHash::new("test").unwrap(), + ) + .unwrap(); + + let entity = schema + .make_entity(vec![ + ("id".into(), Value::String("id".to_owned())), + ("val".into(), Value::String("DATA".to_owned())), + ]) + .unwrap(); + + let entity_type = schema.entity_type("User").unwrap(); + + let blocks = { + let block_0 = genesis(); + let mut block_1 = empty_block(block_0.ptr(), test_ptr(1)); + + push_test_subgraph_trigger( + &mut block_1, + DeploymentHash::new("QmRFXhvyvbm4z5Lo7z2mN9Ckmo623uuB2jJYbRmAXgYKXJ").unwrap(), + entity, + entity_type, + EntityOperationKind::Create, + 1, + ); + + let block_2 = empty_block(block_1.ptr(), test_ptr(2)); + vec![block_0, block_1, block_2] + }; + let stop_block = blocks.last().unwrap().block.ptr(); + let chain = chain(&test_info.test_name, blocks, &stores, None).await; + + let ctx = fixture::setup(&test_info, &stores, &chain, None, None).await; + let _ = ctx + .runner(stop_block) + .await + .run_for_test(true) + .await + .unwrap(); +} + #[tokio::test] async fn retry_create_ds() { let RunnerTestRecipe { stores, test_info } =