From a6b33f5d8925810625543dce8b015096e87ae59a Mon Sep 17 00:00:00 2001
From: Theo Butler <theodusbutler@gmail.com>
Date: Wed, 27 Jul 2022 22:07:24 -0400
Subject: [PATCH 01/33] Refactor manifest data sources

---
 chain/ethereum/src/chain.rs           |   1 +
 chain/ethereum/tests/manifest.rs      |  20 +++-
 core/src/subgraph/instance.rs         |  17 ++-
 core/src/subgraph/instance_manager.rs |   9 +-
 core/src/subgraph/loader.rs           |   9 +-
 core/src/subgraph/runner.rs           |   5 +-
 graph/src/blockchain/mod.rs           |   1 +
 graph/src/components/subgraph/host.rs |   3 +-
 graph/src/data/subgraph/mod.rs        | 106 +++++++------------
 graph/src/data_source.rs              | 146 ++++++++++++++++++++++++++
 graph/src/lib.rs                      |   2 +
 graph/src/offchain/mod.rs             |  15 ++-
 graphql/tests/query.rs                |   1 -
 runtime/test/src/common.rs            |   3 +-
 runtime/wasm/src/host.rs              |  24 +++--
 runtime/wasm/src/host_exports.rs      |   4 +-
 store/postgres/tests/graft.rs         |   1 -
 store/postgres/tests/store.rs         |   2 -
 store/postgres/tests/subgraph.rs      |   1 -
 store/postgres/tests/writable.rs      |   1 -
 store/test-store/src/store.rs         |   1 -
 21 files changed, 262 insertions(+), 110 deletions(-)
 create mode 100644 graph/src/data_source.rs

diff --git a/chain/ethereum/src/chain.rs b/chain/ethereum/src/chain.rs
index 7be41a145cd..dd998db7b75 100644
--- a/chain/ethereum/src/chain.rs
+++ b/chain/ethereum/src/chain.rs
@@ -245,6 +245,7 @@ impl Chain {
 #[async_trait]
 impl Blockchain for Chain {
     const KIND: BlockchainKind = BlockchainKind::Ethereum;
+    const ALIASES: &'static [&'static str] = &["ethereum/contract"];
 
     type Block = BlockFinality;
 
diff --git a/chain/ethereum/tests/manifest.rs b/chain/ethereum/tests/manifest.rs
index 66dc82836e9..2f3782fcbf9 100644
--- a/chain/ethereum/tests/manifest.rs
+++ b/chain/ethereum/tests/manifest.rs
@@ -4,6 +4,8 @@ use std::time::Duration;
 
 use graph::data::subgraph::schema::SubgraphError;
 use graph::data::subgraph::{SPEC_VERSION_0_0_4, SPEC_VERSION_0_0_7};
+use graph::data_source::DataSource;
+use graph::offchain;
 use graph::prelude::{
     anyhow, async_trait, serde_yaml, tokio, DeploymentHash, Entity, Link, Logger, SubgraphManifest,
     SubgraphManifestValidationError, UnvalidatedSubgraphManifest,
@@ -152,8 +154,15 @@ specVersion: 0.0.7
     let manifest = resolve_manifest(YAML, SPEC_VERSION_0_0_7).await;
 
     assert_eq!("Qmmanifest", manifest.id.as_str());
-    assert_eq!(manifest.offchain_data_sources.len(), 1);
-    assert_eq!(manifest.data_sources.len(), 0);
+    assert_eq!(manifest.data_sources.len(), 1);
+    let data_source = match &manifest.data_sources[0] {
+        DataSource::Offchain(ds) => ds,
+        DataSource::Onchain(_) => unreachable!(),
+    };
+    assert_eq!(
+        data_source.source,
+        Some(offchain::Source::Ipfs(Link::from("/ipfs/Qmfile")))
+    );
 }
 
 #[tokio::test]
@@ -392,7 +401,12 @@ specVersion: 0.0.2
 ";
 
     let manifest = resolve_manifest(YAML, SPEC_VERSION_0_0_4).await;
-    let required_capabilities = NodeCapabilities::from_data_sources(&manifest.data_sources);
+    let onchain_data_sources = manifest
+        .data_sources
+        .iter()
+        .filter_map(|ds| ds.as_onchain().cloned())
+        .collect::<Vec<_>>();
+    let required_capabilities = NodeCapabilities::from_data_sources(&onchain_data_sources);
 
     assert_eq!("Qmmanifest", manifest.id.as_str());
     assert_eq!(true, required_capabilities.traces);
diff --git a/core/src/subgraph/instance.rs b/core/src/subgraph/instance.rs
index f14a96606d5..5301f3c48a1 100644
--- a/core/src/subgraph/instance.rs
+++ b/core/src/subgraph/instance.rs
@@ -1,18 +1,15 @@
 use futures01::sync::mpsc::Sender;
-use graph::components::subgraph::ProofOfIndexingVersion;
-use graph::data::subgraph::SPEC_VERSION_0_0_6;
-
-use std::collections::HashMap;
-
 use graph::{
     blockchain::Blockchain,
     components::{
         store::SubgraphFork,
-        subgraph::{MappingError, SharedProofOfIndexing},
+        subgraph::{MappingError, ProofOfIndexingVersion, SharedProofOfIndexing},
     },
-    prelude::ENV_VARS,
+    data::subgraph::SPEC_VERSION_0_0_6,
+    data_source::DataSource,
+    prelude::*,
 };
-use graph::{blockchain::DataSource, prelude::*};
+use std::collections::HashMap;
 
 pub struct SubgraphInstance<C: Blockchain, T: RuntimeHostBuilder<C>> {
     subgraph_id: DeploymentHash,
@@ -92,7 +89,7 @@ where
     fn new_host(
         &mut self,
         logger: Logger,
-        data_source: C::DataSource,
+        data_source: DataSource<C>,
         module_bytes: &Arc<Vec<u8>>,
         templates: Arc<Vec<C::DataSourceTemplate>>,
         host_metrics: Arc<HostMetrics>,
@@ -151,7 +148,7 @@ where
     pub(crate) fn add_dynamic_data_source(
         &mut self,
         logger: &Logger,
-        data_source: C::DataSource,
+        data_source: DataSource<C>,
         templates: Arc<Vec<C::DataSourceTemplate>>,
         metrics: Arc<HostMetrics>,
     ) -> Result<Option<Arc<T::Host>>, Error> {
diff --git a/core/src/subgraph/instance_manager.rs b/core/src/subgraph/instance_manager.rs
index 5a066a22162..7eb80cd8f5c 100644
--- a/core/src/subgraph/instance_manager.rs
+++ b/core/src/subgraph/instance_manager.rs
@@ -220,7 +220,12 @@ impl<S: SubgraphStore> SubgraphInstanceManager<S> {
             (manifest, manifest_idx_and_name)
         };
 
-        let required_capabilities = C::NodeCapabilities::from_data_sources(&manifest.data_sources);
+        let onchain_data_sources = manifest
+            .data_sources
+            .iter()
+            .filter_map(|d| d.as_onchain().cloned())
+            .collect::<Vec<_>>();
+        let required_capabilities = C::NodeCapabilities::from_data_sources(&onchain_data_sources);
         let network = manifest.network_name();
 
         let chain = self
@@ -230,7 +235,7 @@ impl<S: SubgraphStore> SubgraphInstanceManager<S> {
             .clone();
 
         // Obtain filters from the manifest
-        let mut filter = C::TriggerFilter::from_data_sources(manifest.data_sources.iter());
+        let mut filter = C::TriggerFilter::from_data_sources(onchain_data_sources.iter());
 
         if self.static_filters {
             filter.extend_with_template(manifest.templates.clone().into_iter());
diff --git a/core/src/subgraph/loader.rs b/core/src/subgraph/loader.rs
index 442cc4d15f2..8d73827c3b1 100644
--- a/core/src/subgraph/loader.rs
+++ b/core/src/subgraph/loader.rs
@@ -1,7 +1,8 @@
 use std::time::Instant;
 
-use graph::blockchain::{Blockchain, DataSource, DataSourceTemplate as _};
+use graph::blockchain::{Blockchain, DataSource as _, DataSourceTemplate as _};
 use graph::components::store::WritableStore;
+use graph::data_source::DataSource;
 use graph::prelude::*;
 
 pub async fn load_dynamic_data_sources<C: Blockchain>(
@@ -9,10 +10,10 @@ pub async fn load_dynamic_data_sources<C: Blockchain>(
     logger: Logger,
     manifest: &SubgraphManifest<C>,
     manifest_idx_and_name: Vec<(u32, String)>,
-) -> Result<Vec<C::DataSource>, Error> {
+) -> Result<Vec<DataSource<C>>, Error> {
     let start_time = Instant::now();
 
-    let mut data_sources: Vec<C::DataSource> = vec![];
+    let mut data_sources: Vec<DataSource<C>> = vec![];
 
     for stored in store
         .load_dynamic_data_sources(manifest_idx_and_name)
@@ -33,7 +34,7 @@ pub async fn load_dynamic_data_sources<C: Blockchain>(
             "Assertion failure: new data source has lower creation block than existing ones"
         );
 
-        data_sources.push(ds);
+        data_sources.push(DataSource::Onchain(ds));
     }
 
     trace!(
diff --git a/core/src/subgraph/runner.rs b/core/src/subgraph/runner.rs
index 8abf122daba..3830973bed3 100644
--- a/core/src/subgraph/runner.rs
+++ b/core/src/subgraph/runner.rs
@@ -5,7 +5,7 @@ use crate::subgraph::state::IndexingState;
 use crate::subgraph::stream::new_block_stream;
 use atomic_refcell::AtomicRefCell;
 use graph::blockchain::block_stream::{BlockStreamEvent, BlockWithTriggers, FirehoseCursor};
-use graph::blockchain::{Block, Blockchain, DataSource, TriggerFilter as _};
+use graph::blockchain::{Block, Blockchain, DataSource as _, TriggerFilter as _};
 use graph::components::store::EntityKey;
 use graph::components::{
     store::ModificationsAndCache,
@@ -16,6 +16,7 @@ use graph::data::subgraph::{
     schema::{SubgraphError, SubgraphHealth, POI_OBJECT},
     SubgraphFeature,
 };
+use graph::data_source::DataSource;
 use graph::prelude::*;
 use graph::util::{backoff::ExponentialBackoff, lfu_cache::LfuCache};
 use std::convert::TryFrom;
@@ -465,7 +466,7 @@ where
             // Try to create a runtime host for the data source
             let host = self.ctx.instance.add_dynamic_data_source(
                 &self.logger,
-                data_source.clone(),
+                DataSource::Onchain(data_source.clone()),
                 self.inputs.templates.clone(),
                 self.metrics.host.clone(),
             )?;
diff --git a/graph/src/blockchain/mod.rs b/graph/src/blockchain/mod.rs
index 8b5cbd7c12a..2b5b417d481 100644
--- a/graph/src/blockchain/mod.rs
+++ b/graph/src/blockchain/mod.rs
@@ -81,6 +81,7 @@ pub trait Block: Send + Sync {
 // This is only `Debug` because some tests require that
 pub trait Blockchain: Debug + Sized + Send + Sync + Unpin + 'static {
     const KIND: BlockchainKind;
+    const ALIASES: &'static [&'static str] = &[];
 
     // The `Clone` bound is used when reprocessing a block, because `triggers_in_block` requires an
     // owned `Block`. It would be good to come up with a way to remove this bound.
diff --git a/graph/src/components/subgraph/host.rs b/graph/src/components/subgraph/host.rs
index a3371323187..a824aef79ab 100644
--- a/graph/src/components/subgraph/host.rs
+++ b/graph/src/components/subgraph/host.rs
@@ -8,6 +8,7 @@ use futures::sync::mpsc;
 
 use crate::blockchain::TriggerWithHandler;
 use crate::components::store::SubgraphFork;
+use crate::data_source::DataSource;
 use crate::prelude::*;
 use crate::{blockchain::Blockchain, components::subgraph::SharedProofOfIndexing};
 use crate::{components::metrics::HistogramVec, runtime::DeterministicHostError};
@@ -151,7 +152,7 @@ pub trait RuntimeHostBuilder<C: Blockchain>: Clone + Send + Sync + 'static {
         &self,
         network_name: String,
         subgraph_id: DeploymentHash,
-        data_source: C::DataSource,
+        data_source: DataSource<C>,
         top_level_templates: Arc<Vec<C::DataSourceTemplate>>,
         mapping_request_sender: mpsc::Sender<Self::Req>,
         metrics: Arc<HostMetrics>,
diff --git a/graph/src/data/subgraph/mod.rs b/graph/src/data/subgraph/mod.rs
index ae1d634e053..39b1244dcc4 100644
--- a/graph/src/data/subgraph/mod.rs
+++ b/graph/src/data/subgraph/mod.rs
@@ -12,39 +12,37 @@ pub use features::{SubgraphFeature, SubgraphFeatureValidationError};
 
 use anyhow::ensure;
 use anyhow::{anyhow, Error};
-use futures03::{
-    future::try_join4,
-    stream::{FuturesOrdered, FuturesUnordered},
-    TryStreamExt as _,
-};
+use futures03::{future::try_join3, stream::FuturesOrdered, TryStreamExt as _};
 use semver::Version;
-use serde::de;
-use serde::ser;
+use serde::{de, ser};
 use serde_yaml;
 use slog::{debug, info, Logger};
 use stable_hash::{FieldAddress, StableHash};
 use stable_hash_legacy::SequenceNumber;
-use std::{collections::BTreeSet, marker::PhantomData, mem::take};
+use std::{collections::BTreeSet, marker::PhantomData};
 use thiserror::Error;
 use wasmparser;
 use web3::types::Address;
 
 use crate::blockchain::BlockPtr;
+use crate::data::graphql::TryFromValue;
 use crate::data::store::Entity;
 use crate::data::{
     schema::{Schema, SchemaImportError, SchemaValidationError},
     subgraph::features::validate_subgraph_features,
 };
-use crate::offchain;
+use crate::offchain::OFFCHAIN_KINDS;
 use crate::prelude::{r, CheapClone, ENV_VARS};
-use crate::{blockchain::DataSource, data::graphql::TryFromValue};
-use crate::{blockchain::DataSourceTemplate as _, data::query::QueryExecutionError};
 use crate::{
-    blockchain::{Blockchain, UnresolvedDataSource as _, UnresolvedDataSourceTemplate as _},
+    blockchain::{
+        Blockchain, DataSource as _, DataSourceTemplate as _, UnresolvedDataSourceTemplate as _,
+    },
     components::{
         link_resolver::LinkResolver,
         store::{DeploymentLocator, StoreError, SubgraphStore},
     },
+    data::query::QueryExecutionError,
+    data_source::{DataSource, UnresolvedDataSource},
 };
 
 use crate::prelude::{impl_slog_value, BlockNumber, Deserialize, Serialize};
@@ -396,9 +394,11 @@ pub struct Link {
     pub link: String,
 }
 
-impl From<String> for Link {
-    fn from(s: String) -> Self {
-        Self { link: s }
+impl<S: ToString> From<S> for Link {
+    fn from(s: S) -> Self {
+        Self {
+            link: s.to_string(),
+        }
     }
 }
 
@@ -504,7 +504,7 @@ impl Graft {
 
 #[derive(Debug, Deserialize)]
 #[serde(rename_all = "camelCase")]
-pub struct BaseSubgraphManifest<C, S, D, T, O> {
+pub struct BaseSubgraphManifest<C, S, D, T> {
     pub id: DeploymentHash,
     pub spec_version: Version,
     #[serde(default)]
@@ -516,8 +516,6 @@ pub struct BaseSubgraphManifest<C, S, D, T, O> {
     pub graft: Option<Graft>,
     #[serde(default)]
     pub templates: Vec<T>,
-    #[serde(default)]
-    pub offchain_data_sources: Vec<O>,
     #[serde(skip_serializing, default)]
     pub chain: PhantomData<C>,
 }
@@ -526,19 +524,13 @@ pub struct BaseSubgraphManifest<C, S, D, T, O> {
 type UnresolvedSubgraphManifest<C> = BaseSubgraphManifest<
     C,
     UnresolvedSchema,
-    <C as Blockchain>::UnresolvedDataSource,
+    UnresolvedDataSource<C>,
     <C as Blockchain>::UnresolvedDataSourceTemplate,
-    offchain::UnresolvedDataSource,
 >;
 
 /// SubgraphManifest validated with IPFS links resolved
-pub type SubgraphManifest<C> = BaseSubgraphManifest<
-    C,
-    Schema,
-    <C as Blockchain>::DataSource,
-    <C as Blockchain>::DataSourceTemplate,
-    offchain::DataSource,
->;
+pub type SubgraphManifest<C> =
+    BaseSubgraphManifest<C, Schema, DataSource<C>, <C as Blockchain>::DataSourceTemplate>;
 
 /// Unvalidated SubgraphManifest
 pub struct UnvalidatedSubgraphManifest<C: Blockchain>(SubgraphManifest<C>);
@@ -591,7 +583,7 @@ impl<C: Blockchain> UnvalidatedSubgraphManifest<C> {
             .0
             .data_sources
             .iter()
-            .filter_map(|d| d.network().map(|n| n.to_string()))
+            .filter_map(|d| Some(d.as_onchain()?.network()?.to_string()))
             .collect::<Vec<String>>();
         networks.sort();
         networks.dedup();
@@ -655,56 +647,43 @@ impl<C: Blockchain> SubgraphManifest<C> {
         // Inject the IPFS hash as the ID of the subgraph into the definition.
         raw.insert("id".into(), id.to_string().into());
 
-        let spec_version = raw
-            .get(&"specVersion".into())
-            .and_then(|v| v.as_str()?.parse::<Version>().ok());
-
-        if spec_version >= Some(SPEC_VERSION_0_0_7) {
-            // Separate offchain data sources (where `kind` starts with "file").
-            let data_sources: Vec<serde_yaml::Value> = raw
-                .remove(&serde_yaml::Value::from("dataSources"))
-                .and_then(|mut v| v.as_sequence_mut().map(take))
-                .unwrap_or_default();
-            let mut onchain_data_sources = Vec::new();
-            let mut offchain_data_sources = Vec::new();
-            for data_source in data_sources {
-                let kind = data_source
-                    .get(&serde_yaml::Value::from("kind"))
-                    .and_then(|v| v.as_str())
-                    .unwrap_or("");
-                if kind.starts_with("file") {
-                    offchain_data_sources.push(data_source);
-                } else {
-                    onchain_data_sources.push(data_source);
-                }
-            }
-            raw.insert("dataSources".into(), onchain_data_sources.into());
-            raw.insert("offchainDataSources".into(), offchain_data_sources.into());
-        }
-
         // Parse the YAML data into an UnresolvedSubgraphManifest
         let unresolved: UnresolvedSubgraphManifest<C> = serde_yaml::from_value(raw.into())?;
 
         debug!(logger, "Features {:?}", unresolved.features);
 
-        unresolved
+        let resolved = unresolved
             .resolve(resolver, logger, max_spec_version)
             .await
-            .map_err(SubgraphManifestResolveError::ResolveError)
+            .map_err(SubgraphManifestResolveError::ResolveError)?;
+
+        if (resolved.spec_version < SPEC_VERSION_0_0_7)
+            && resolved
+                .data_sources
+                .iter()
+                .any(|ds| OFFCHAIN_KINDS.contains(&ds.kind()))
+        {
+            return Err(SubgraphManifestResolveError::ResolveError(anyhow!(
+                "Offchain data sources not supported prior to {}",
+                SPEC_VERSION_0_0_7
+            )));
+        }
+
+        Ok(resolved)
     }
 
     pub fn network_name(&self) -> String {
         // Assume the manifest has been validated, ensuring network names are homogenous
         self.data_sources
             .iter()
-            .find_map(|d| d.network().map(|n| n.to_string()))
+            .find_map(|d| Some(d.as_onchain()?.network()?.to_string()))
             .expect("Validated manifest does not have a network defined on any datasource")
     }
 
     pub fn start_blocks(&self) -> Vec<BlockNumber> {
         self.data_sources
             .iter()
-            .map(|data_source| data_source.start_block())
+            .filter_map(|d| Some(d.as_onchain()?.start_block()))
             .collect()
     }
 
@@ -750,7 +729,6 @@ impl<C: Blockchain> UnresolvedSubgraphManifest<C> {
             data_sources,
             graft,
             templates,
-            offchain_data_sources,
             chain,
         } = self;
 
@@ -771,7 +749,7 @@ impl<C: Blockchain> UnresolvedSubgraphManifest<C> {
             ));
         }
 
-        let (schema, data_sources, templates, offchain_data_sources) = try_join4(
+        let (schema, data_sources, templates) = try_join3(
             schema.resolve(id.clone(), &resolver, logger),
             data_sources
                 .into_iter()
@@ -787,11 +765,6 @@ impl<C: Blockchain> UnresolvedSubgraphManifest<C> {
                 })
                 .collect::<FuturesOrdered<_>>()
                 .try_collect::<Vec<_>>(),
-            offchain_data_sources
-                .into_iter()
-                .map(|ds| ds.resolve(&resolver, logger))
-                .collect::<FuturesUnordered<_>>()
-                .try_collect::<Vec<_>>(),
         )
         .await?;
 
@@ -816,7 +789,6 @@ impl<C: Blockchain> UnresolvedSubgraphManifest<C> {
             data_sources,
             graft,
             templates,
-            offchain_data_sources,
             chain,
         })
     }
diff --git a/graph/src/data_source.rs b/graph/src/data_source.rs
new file mode 100644
index 00000000000..c868478b72b
--- /dev/null
+++ b/graph/src/data_source.rs
@@ -0,0 +1,146 @@
+use crate::{
+    blockchain::{Blockchain, DataSource as _, UnresolvedDataSource as _},
+    components::{link_resolver::LinkResolver, store::BlockNumber},
+    offchain::{self, OFFCHAIN_KINDS},
+    prelude::{CheapClone as _, DataSourceContext},
+};
+use anyhow::Error;
+use semver::Version;
+use serde::{de::IntoDeserializer as _, Deserialize, Deserializer};
+use slog::Logger;
+use std::{collections::BTreeMap, sync::Arc};
+
+#[derive(Debug)]
+pub enum DataSource<C: Blockchain> {
+    Onchain(C::DataSource),
+    Offchain(offchain::DataSource),
+}
+
+impl<C: Blockchain> DataSource<C> {
+    pub fn as_onchain(&self) -> Option<&C::DataSource> {
+        match self {
+            Self::Onchain(ds) => Some(&ds),
+            Self::Offchain(_) => None,
+        }
+    }
+
+    pub fn address(&self) -> Option<&[u8]> {
+        match self {
+            Self::Onchain(ds) => ds.address(),
+            Self::Offchain(_) => None,
+        }
+    }
+
+    pub fn name(&self) -> &str {
+        match self {
+            Self::Onchain(ds) => ds.name(),
+            Self::Offchain(ds) => &ds.name,
+        }
+    }
+
+    pub fn kind(&self) -> &str {
+        match self {
+            Self::Onchain(ds) => ds.kind(),
+            Self::Offchain(ds) => &ds.kind,
+        }
+    }
+
+    pub fn creation_block(&self) -> Option<BlockNumber> {
+        match self {
+            Self::Onchain(ds) => ds.creation_block(),
+            Self::Offchain(ds) => ds.creation_block,
+        }
+    }
+
+    pub fn context(&self) -> Arc<Option<DataSourceContext>> {
+        match self {
+            Self::Onchain(ds) => ds.context(),
+            Self::Offchain(ds) => ds.context.clone(),
+        }
+    }
+
+    pub fn api_version(&self) -> Version {
+        match self {
+            Self::Onchain(ds) => ds.api_version(),
+            Self::Offchain(ds) => ds.mapping.api_version.clone(),
+        }
+    }
+
+    pub fn runtime(&self) -> Option<Arc<Vec<u8>>> {
+        match self {
+            Self::Onchain(ds) => ds.runtime(),
+            Self::Offchain(ds) => Some(ds.mapping.runtime.cheap_clone()),
+        }
+    }
+
+    pub fn is_duplicate_of(&self, other: &Self) -> bool {
+        match (self, other) {
+            (Self::Onchain(a), Self::Onchain(b)) => a.is_duplicate_of(b),
+            (Self::Offchain(a), Self::Offchain(b)) => {
+                a.kind == b.kind && a.name == b.name && a.source == b.source
+            }
+            _ => false,
+        }
+    }
+
+    pub fn validate(&self) -> Vec<Error> {
+        match self {
+            Self::Onchain(ds) => ds.validate(),
+            Self::Offchain(_) => vec![],
+        }
+    }
+}
+
+#[derive(Debug)]
+pub enum UnresolvedDataSource<C: Blockchain> {
+    Onchain(C::UnresolvedDataSource),
+    Offchain(offchain::UnresolvedDataSource),
+}
+
+impl<'de, C: Blockchain> Deserialize<'de> for UnresolvedDataSource<C> {
+    fn deserialize<D>(deserializer: D) -> Result<Self, D::Error>
+    where
+        D: Deserializer<'de>,
+    {
+        let map: BTreeMap<String, serde_json::Value> = BTreeMap::deserialize(deserializer)?;
+        let kind = map
+            .get("kind")
+            .ok_or(serde::de::Error::missing_field("kind"))?
+            .as_str()
+            .unwrap_or("?");
+        if OFFCHAIN_KINDS.contains(&kind) {
+            offchain::UnresolvedDataSource::deserialize(map.into_deserializer())
+                .map_err(serde::de::Error::custom)
+                .map(UnresolvedDataSource::Offchain)
+        } else if (&C::KIND.to_string() == kind) || C::ALIASES.contains(&kind) {
+            C::UnresolvedDataSource::deserialize(map.into_deserializer())
+                .map_err(serde::de::Error::custom)
+                .map(UnresolvedDataSource::Onchain)
+        } else {
+            Err(serde::de::Error::custom(format!(
+                "data source has invalid `kind`; expected {}, file/ipfs",
+                C::KIND,
+            )))
+        }
+    }
+}
+
+impl<C: Blockchain> UnresolvedDataSource<C> {
+    pub async fn resolve(
+        self,
+        resolver: &Arc<dyn LinkResolver>,
+        logger: &Logger,
+        manifest_idx: u32,
+    ) -> Result<DataSource<C>, anyhow::Error> {
+        match self {
+            Self::Onchain(unresolved) => unresolved
+                .resolve(resolver, logger, manifest_idx)
+                .await
+                .map(DataSource::Onchain),
+            Self::Offchain(unresolved) => unresolved
+                .resolve(resolver, logger)
+                .await
+                .map(DataSource::Offchain),
+        }
+    }
+}
diff --git a/graph/src/lib.rs b/graph/src/lib.rs
index e1d02bd3aae..d238926a385 100644
--- a/graph/src/lib.rs
+++ b/graph/src/lib.rs
@@ -18,6 +18,8 @@ pub mod cheap_clone;
 
 pub mod ipfs_client;
 
+pub mod data_source;
+
 pub mod blockchain;
 
 pub mod offchain;
diff --git a/graph/src/offchain/mod.rs b/graph/src/offchain/mod.rs
index fe5e08d7b8c..57ded430e5f 100644
--- a/graph/src/offchain/mod.rs
+++ b/graph/src/offchain/mod.rs
@@ -1,18 +1,26 @@
-use crate::{components::link_resolver::LinkResolver, prelude::Link};
+use crate::{
+    components::{link_resolver::LinkResolver, store::BlockNumber},
+    prelude::{DataSourceContext, Link},
+};
 
 use anyhow::anyhow;
 use serde::Deserialize;
 use slog::{info, Logger};
 use std::sync::Arc;
 
+pub const OFFCHAIN_KINDS: &'static [&'static str] = &["file/ipfs"];
+
 #[derive(Debug)]
 pub struct DataSource {
+    pub kind: String,
     pub name: String,
     pub source: Option<Source>,
     pub mapping: Mapping,
+    pub context: Arc<Option<DataSourceContext>>,
+    pub creation_block: Option<BlockNumber>,
 }
 
-#[derive(Debug)]
+#[derive(Debug, Eq, PartialEq)]
 pub enum Source {
     Ipfs(Link),
 }
@@ -71,9 +79,12 @@ impl UnresolvedDataSource {
             }
         };
         Ok(DataSource {
+            kind: self.kind,
             name: self.name,
             source,
             mapping: self.mapping.resolve(&*resolver, logger).await?,
+            context: Arc::new(None),
+            creation_block: None,
         })
     }
 }
diff --git a/graphql/tests/query.rs b/graphql/tests/query.rs
index 598f55f2f36..d9720a7b881 100644
--- a/graphql/tests/query.rs
+++ b/graphql/tests/query.rs
@@ -121,7 +121,6 @@ async fn setup(
             data_sources: vec![],
             graft: None,
             templates: vec![],
-            offchain_data_sources: vec![],
             chain: PhantomData,
         };
 
diff --git a/runtime/test/src/common.rs b/runtime/test/src/common.rs
index e548a41ebc6..9fa1f228b89 100644
--- a/runtime/test/src/common.rs
+++ b/runtime/test/src/common.rs
@@ -1,6 +1,7 @@
 use ethabi::Contract;
 use graph::components::store::DeploymentLocator;
 use graph::data::subgraph::*;
+use graph::data_source;
 use graph::env::EnvVars;
 use graph::ipfs_client::IpfsClient;
 use graph::log;
@@ -55,7 +56,7 @@ fn mock_host_exports(
     let ens_lookup = store.ens_lookup();
     HostExports::new(
         subgraph_id,
-        &data_source,
+        &data_source::DataSource::Onchain(data_source),
         network,
         Arc::new(templates),
         Arc::new(graph_core::LinkResolver::new(
diff --git a/runtime/wasm/src/host.rs b/runtime/wasm/src/host.rs
index 8e4943e179d..5a670804f99 100644
--- a/runtime/wasm/src/host.rs
+++ b/runtime/wasm/src/host.rs
@@ -5,11 +5,10 @@ use async_trait::async_trait;
 use futures::sync::mpsc::Sender;
 use futures03::channel::oneshot::channel;
 
-use graph::blockchain::RuntimeAdapter;
-use graph::blockchain::{Blockchain, DataSource};
-use graph::blockchain::{HostFn, TriggerWithHandler};
+use graph::blockchain::{Blockchain, DataSource as _, HostFn, RuntimeAdapter, TriggerWithHandler};
 use graph::components::store::{EnsLookup, SubgraphFork};
 use graph::components::subgraph::{MappingError, SharedProofOfIndexing};
+use graph::data_source::DataSource;
 use graph::prelude::{
     RuntimeHost as RuntimeHostTrait, RuntimeHostBuilder as RuntimeHostBuilderTrait, *,
 };
@@ -76,7 +75,7 @@ impl<C: Blockchain> RuntimeHostBuilderTrait<C> for RuntimeHostBuilder<C> {
         &self,
         network_name: String,
         subgraph_id: DeploymentHash,
-        data_source: C::DataSource,
+        data_source: DataSource<C>,
         templates: Arc<Vec<C::DataSourceTemplate>>,
         mapping_request_sender: Sender<MappingRequest<C>>,
         metrics: Arc<HostMetrics>,
@@ -97,7 +96,7 @@ impl<C: Blockchain> RuntimeHostBuilderTrait<C> for RuntimeHostBuilder<C> {
 
 pub struct RuntimeHost<C: Blockchain> {
     host_fns: Arc<Vec<HostFn>>,
-    data_source: C::DataSource,
+    data_source: DataSource<C>,
     mapping_request_sender: Sender<MappingRequest<C>>,
     host_exports: Arc<HostExports<C>>,
     metrics: Arc<HostMetrics>,
@@ -112,7 +111,7 @@ where
         link_resolver: Arc<dyn LinkResolver>,
         network_name: String,
         subgraph_id: DeploymentHash,
-        data_source: C::DataSource,
+        data_source: DataSource<C>,
         templates: Arc<Vec<C::DataSourceTemplate>>,
         mapping_request_sender: Sender<MappingRequest<C>>,
         metrics: Arc<HostMetrics>,
@@ -129,10 +128,14 @@ where
             ens_lookup,
         ));
 
-        let host_fns = Arc::new(runtime_adapter.host_fns(&data_source)?);
+        let host_fns = data_source
+            .as_onchain()
+            .map(|ds| runtime_adapter.host_fns(ds))
+            .transpose()?
+            .unwrap_or_default();
 
         Ok(RuntimeHost {
-            host_fns,
+            host_fns: Arc::new(host_fns),
             data_source,
             mapping_request_sender,
             host_exports,
@@ -215,7 +218,10 @@ impl<C: Blockchain> RuntimeHostTrait<C> for RuntimeHost<C> {
         block: &Arc<C::Block>,
         logger: &Logger,
     ) -> Result<Option<TriggerWithHandler<C>>, Error> {
-        self.data_source.match_and_decode(trigger, block, logger)
+        self.data_source
+            .as_onchain()
+            .map(|ds| ds.match_and_decode(trigger, block, logger))
+            .unwrap_or(Ok(None))
     }
 
     async fn process_mapping_trigger(
diff --git a/runtime/wasm/src/host_exports.rs b/runtime/wasm/src/host_exports.rs
index 1acce38d7e6..c04a4a342f7 100644
--- a/runtime/wasm/src/host_exports.rs
+++ b/runtime/wasm/src/host_exports.rs
@@ -8,12 +8,12 @@ use semver::Version;
 use wasmtime::Trap;
 use web3::types::H160;
 
-use graph::blockchain::DataSource;
 use graph::blockchain::{Blockchain, DataSourceTemplate as _};
 use graph::components::store::EnsLookup;
 use graph::components::store::{EntityKey, EntityType};
 use graph::components::subgraph::{CausalityRegion, ProofOfIndexingEvent, SharedProofOfIndexing};
 use graph::data::store;
+use graph::data_source::DataSource;
 use graph::ensure;
 use graph::prelude::ethabi::param_type::Reader;
 use graph::prelude::ethabi::{decode, encode, Token};
@@ -74,7 +74,7 @@ pub struct HostExports<C: Blockchain> {
 impl<C: Blockchain> HostExports<C> {
     pub fn new(
         subgraph_id: DeploymentHash,
-        data_source: &impl DataSource<C>,
+        data_source: &DataSource<C>,
         data_source_network: String,
         templates: Arc<Vec<C::DataSourceTemplate>>,
         link_resolver: Arc<dyn LinkResolver>,
diff --git a/store/postgres/tests/graft.rs b/store/postgres/tests/graft.rs
index df9d9245009..b7d51b3dfda 100644
--- a/store/postgres/tests/graft.rs
+++ b/store/postgres/tests/graft.rs
@@ -146,7 +146,6 @@ async fn insert_test_data(store: Arc<DieselSubgraphStore>) -> DeploymentLocator
         data_sources: vec![],
         graft: None,
         templates: vec![],
-        offchain_data_sources: vec![],
         chain: PhantomData,
     };
 
diff --git a/store/postgres/tests/store.rs b/store/postgres/tests/store.rs
index ba27bf24fd8..2abf2ce494e 100644
--- a/store/postgres/tests/store.rs
+++ b/store/postgres/tests/store.rs
@@ -166,7 +166,6 @@ async fn insert_test_data(store: Arc<DieselSubgraphStore>) -> DeploymentLocator
         data_sources: vec![],
         graft: None,
         templates: vec![],
-        offchain_data_sources: vec![],
         chain: PhantomData,
     };
 
@@ -1294,7 +1293,6 @@ fn entity_changes_are_fired_and_forwarded_to_subscriptions() {
             data_sources: vec![],
             graft: None,
             templates: vec![],
-            offchain_data_sources: vec![],
             chain: PhantomData,
         };
 
diff --git a/store/postgres/tests/subgraph.rs b/store/postgres/tests/subgraph.rs
index da75a69ef3c..12d6e2c9be7 100644
--- a/store/postgres/tests/subgraph.rs
+++ b/store/postgres/tests/subgraph.rs
@@ -146,7 +146,6 @@ fn create_subgraph() {
             data_sources: vec![],
             graft: None,
             templates: vec![],
-            offchain_data_sources: vec![],
             chain: PhantomData,
         };
         let deployment = DeploymentCreate::new(&manifest, None);
diff --git a/store/postgres/tests/writable.rs b/store/postgres/tests/writable.rs
index dbe765c9834..25e9711facc 100644
--- a/store/postgres/tests/writable.rs
+++ b/store/postgres/tests/writable.rs
@@ -43,7 +43,6 @@ async fn insert_test_data(store: Arc<DieselSubgraphStore>) -> DeploymentLocator
         data_sources: vec![],
         graft: None,
         templates: vec![],
-        offchain_data_sources: vec![],
         chain: PhantomData,
     };
 
diff --git a/store/test-store/src/store.rs b/store/test-store/src/store.rs
index aa49f91920e..bfb968c919d 100644
--- a/store/test-store/src/store.rs
+++ b/store/test-store/src/store.rs
@@ -164,7 +164,6 @@ pub async fn create_subgraph(
         data_sources: vec![],
         graft: None,
         templates: vec![],
-        offchain_data_sources: vec![],
         chain: PhantomData,
     };
 

From 48ccdd3513323c30bf203d93a4d7911a54d851c2 Mon Sep 17 00:00:00 2001
From: Theo Butler <theodusbutler@gmail.com>
Date: Thu, 28 Jul 2022 21:19:21 -0400
Subject: [PATCH 02/33] Refactor manifest data source templates

---
 chain/ethereum/src/data_source.rs         |   3 +
 core/src/subgraph/inputs.rs               |   9 +-
 core/src/subgraph/instance.rs             |   6 +-
 core/src/subgraph/instance_manager.rs     |  12 +-
 core/src/subgraph/loader.rs               |   6 +-
 core/src/subgraph/runner.rs               |  18 +-
 graph/src/blockchain/mod.rs               |   2 +-
 graph/src/components/store/cache.rs       |   4 +-
 graph/src/components/subgraph/host.rs     |   4 +-
 graph/src/components/subgraph/instance.rs |  15 +-
 graph/src/data/subgraph/mod.rs            |  31 ++--
 graph/src/data_source.rs                  | 212 ++++++++++++++++++----
 graph/src/offchain/mod.rs                 | 123 ++++++++++++-
 runtime/test/src/common.rs                |  44 ++---
 runtime/test/src/test.rs                  |   2 +-
 runtime/wasm/src/host.rs                  |   6 +-
 runtime/wasm/src/host_exports.rs          |   8 +-
 17 files changed, 389 insertions(+), 116 deletions(-)

diff --git a/chain/ethereum/src/data_source.rs b/chain/ethereum/src/data_source.rs
index 246fd468096..a2d100341cc 100644
--- a/chain/ethereum/src/data_source.rs
+++ b/chain/ethereum/src/data_source.rs
@@ -757,6 +757,9 @@ impl TryFrom<DataSourceTemplateInfo<Chain>> for DataSource {
             context,
             creation_block,
         } = info;
+        let template = template.into_onchain().ok_or(anyhow!(
+            "Cannot create onchain data source from offchain template"
+        ))?;
 
         // Obtain the address from the parameters
         let string = params
diff --git a/core/src/subgraph/inputs.rs b/core/src/subgraph/inputs.rs
index b507e2bf6ad..f17a0ba46eb 100644
--- a/core/src/subgraph/inputs.rs
+++ b/core/src/subgraph/inputs.rs
@@ -1,9 +1,8 @@
-use graph::blockchain::TriggersAdapter;
-use graph::components::store::WritableStore;
 use graph::{
-    blockchain::Blockchain,
-    components::store::{DeploymentLocator, SubgraphFork},
+    blockchain::{Blockchain, TriggersAdapter},
+    components::store::{DeploymentLocator, SubgraphFork, WritableStore},
     data::subgraph::{SubgraphFeature, UnifiedMappingApiVersion},
+    data_source::DataSourceTemplate,
     prelude::BlockNumber,
 };
 use std::collections::BTreeSet;
@@ -18,7 +17,7 @@ pub struct IndexingInputs<C: Blockchain> {
     pub debug_fork: Option<Arc<dyn SubgraphFork>>,
     pub triggers_adapter: Arc<dyn TriggersAdapter<C>>,
     pub chain: Arc<C>,
-    pub templates: Arc<Vec<C::DataSourceTemplate>>,
+    pub templates: Arc<Vec<DataSourceTemplate<C>>>,
     pub unified_api_version: UnifiedMappingApiVersion,
     pub static_filters: bool,
 
diff --git a/core/src/subgraph/instance.rs b/core/src/subgraph/instance.rs
index 5301f3c48a1..a277544a7ed 100644
--- a/core/src/subgraph/instance.rs
+++ b/core/src/subgraph/instance.rs
@@ -6,7 +6,7 @@ use graph::{
         subgraph::{MappingError, ProofOfIndexingVersion, SharedProofOfIndexing},
     },
     data::subgraph::SPEC_VERSION_0_0_6,
-    data_source::DataSource,
+    data_source::{DataSource, DataSourceTemplate},
     prelude::*,
 };
 use std::collections::HashMap;
@@ -91,7 +91,7 @@ where
         logger: Logger,
         data_source: DataSource<C>,
         module_bytes: &Arc<Vec<u8>>,
-        templates: Arc<Vec<C::DataSourceTemplate>>,
+        templates: Arc<Vec<DataSourceTemplate<C>>>,
         host_metrics: Arc<HostMetrics>,
     ) -> Result<T::Host, Error> {
         let mapping_request_sender = {
@@ -149,7 +149,7 @@ where
         &mut self,
         logger: &Logger,
         data_source: DataSource<C>,
-        templates: Arc<Vec<C::DataSourceTemplate>>,
+        templates: Arc<Vec<DataSourceTemplate<C>>>,
         metrics: Arc<HostMetrics>,
     ) -> Result<Option<Arc<T::Host>>, Error> {
         // Protect against creating more than the allowed maximum number of data sources
diff --git a/core/src/subgraph/instance_manager.rs b/core/src/subgraph/instance_manager.rs
index 7eb80cd8f5c..f621cddf8a9 100644
--- a/core/src/subgraph/instance_manager.rs
+++ b/core/src/subgraph/instance_manager.rs
@@ -4,8 +4,8 @@ use crate::subgraph::loader::load_dynamic_data_sources;
 use crate::subgraph::runner::SubgraphRunner;
 use crate::subgraph::SubgraphInstance;
 use graph::blockchain::block_stream::BlockStreamMetrics;
+use graph::blockchain::Blockchain;
 use graph::blockchain::NodeCapabilities;
-use graph::blockchain::{Blockchain, DataSourceTemplate};
 use graph::blockchain::{BlockchainKind, TriggerFilter};
 use graph::prelude::{SubgraphInstanceManager as SubgraphInstanceManagerTrait, *};
 use graph::{blockchain::BlockchainMap, components::store::DeploymentLocator};
@@ -192,7 +192,7 @@ impl<S: SubgraphStore> SubgraphInstanceManager<S> {
             let manifest_idx_and_name: Vec<(u32, String)> = manifest
                 .templates
                 .iter()
-                .map(|t: &C::DataSourceTemplate| t.name().to_owned())
+                .map(|t| t.name().to_owned())
                 .enumerate()
                 .map(|(idx, name)| (ds_len + idx as u32, name))
                 .collect();
@@ -238,7 +238,13 @@ impl<S: SubgraphStore> SubgraphInstanceManager<S> {
         let mut filter = C::TriggerFilter::from_data_sources(onchain_data_sources.iter());
 
         if self.static_filters {
-            filter.extend_with_template(manifest.templates.clone().into_iter());
+            filter.extend_with_template(
+                manifest
+                    .templates
+                    .iter()
+                    .filter_map(|ds| ds.as_onchain())
+                    .cloned(),
+            );
         }
 
         let start_blocks = manifest.start_blocks();
diff --git a/core/src/subgraph/loader.rs b/core/src/subgraph/loader.rs
index 8d73827c3b1..3e5931984db 100644
--- a/core/src/subgraph/loader.rs
+++ b/core/src/subgraph/loader.rs
@@ -1,6 +1,6 @@
 use std::time::Instant;
 
-use graph::blockchain::{Blockchain, DataSource as _, DataSourceTemplate as _};
+use graph::blockchain::Blockchain;
 use graph::components::store::WritableStore;
 use graph::data_source::DataSource;
 use graph::prelude::*;
@@ -25,7 +25,7 @@ pub async fn load_dynamic_data_sources<C: Blockchain>(
             .find(|template| template.manifest_idx() == stored.manifest_idx)
             .ok_or_else(|| anyhow!("no template with idx `{}` was found", stored.manifest_idx))?;
 
-        let ds = C::DataSource::from_stored_dynamic_data_source(&template, stored)?;
+        let ds = DataSource::from_stored_dynamic_data_source(&template, stored)?;
 
         // The data sources are ordered by the creation block.
         // See also 8f1bca33-d3b7-4035-affc-fd6161a12448.
@@ -34,7 +34,7 @@ pub async fn load_dynamic_data_sources<C: Blockchain>(
             "Assertion failure: new data source has lower creation block than existing ones"
         );
 
-        data_sources.push(DataSource::Onchain(ds));
+        data_sources.push(ds);
     }
 
     trace!(
diff --git a/core/src/subgraph/runner.rs b/core/src/subgraph/runner.rs
index 3830973bed3..27b195ecd2e 100644
--- a/core/src/subgraph/runner.rs
+++ b/core/src/subgraph/runner.rs
@@ -5,7 +5,7 @@ use crate::subgraph::state::IndexingState;
 use crate::subgraph::stream::new_block_stream;
 use atomic_refcell::AtomicRefCell;
 use graph::blockchain::block_stream::{BlockStreamEvent, BlockWithTriggers, FirehoseCursor};
-use graph::blockchain::{Block, Blockchain, DataSource as _, TriggerFilter as _};
+use graph::blockchain::{Block, Blockchain, TriggerFilter as _};
 use graph::components::store::EntityKey;
 use graph::components::{
     store::ModificationsAndCache,
@@ -221,7 +221,9 @@ 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());
+            let filter = C::TriggerFilter::from_data_sources(
+                data_sources.iter().filter_map(|ds| ds.as_onchain()),
+            );
 
             // Reprocess the triggers from this block that match the new data sources
             let block_with_triggers = self
@@ -455,18 +457,18 @@ where
     fn create_dynamic_data_sources(
         &mut self,
         created_data_sources: Vec<DataSourceTemplateInfo<C>>,
-    ) -> Result<(Vec<C::DataSource>, Vec<Arc<T::Host>>), Error> {
+    ) -> Result<(Vec<DataSource<C>>, Vec<Arc<T::Host>>), Error> {
         let mut data_sources = vec![];
         let mut runtime_hosts = vec![];
 
         for info in created_data_sources {
             // Try to instantiate a data source from the template
-            let data_source = C::DataSource::try_from(info)?;
+            let data_source = DataSource::try_from(info)?;
 
             // Try to create a runtime host for the data source
             let host = self.ctx.instance.add_dynamic_data_source(
                 &self.logger,
-                DataSource::Onchain(data_source.clone()),
+                data_source.clone(),
                 self.inputs.templates.clone(),
                 self.metrics.host.clone(),
             )?;
@@ -496,7 +498,7 @@ where
     fn persist_dynamic_data_sources(
         &mut self,
         entity_cache: &mut EntityCache,
-        data_sources: Vec<C::DataSource>,
+        data_sources: Vec<DataSource<C>>,
     ) {
         if !data_sources.is_empty() {
             debug!(
@@ -519,7 +521,9 @@ where
         }
 
         // Merge filters from data sources into the block stream builder
-        self.ctx.filter.extend(data_sources.iter());
+        self.ctx
+            .filter
+            .extend(data_sources.iter().filter_map(|ds| ds.as_onchain()));
     }
 }
 
diff --git a/graph/src/blockchain/mod.rs b/graph/src/blockchain/mod.rs
index 2b5b417d481..06ab4458da3 100644
--- a/graph/src/blockchain/mod.rs
+++ b/graph/src/blockchain/mod.rs
@@ -90,7 +90,7 @@ pub trait Blockchain: Debug + Sized + Send + Sync + Unpin + 'static {
     type UnresolvedDataSource: UnresolvedDataSource<Self>;
 
     type DataSourceTemplate: DataSourceTemplate<Self> + Clone;
-    type UnresolvedDataSourceTemplate: UnresolvedDataSourceTemplate<Self>;
+    type UnresolvedDataSourceTemplate: UnresolvedDataSourceTemplate<Self> + Clone;
 
     /// Trigger data as parsed from the triggers adapter.
     type TriggerData: TriggerData + Ord + Send + Sync + Debug;
diff --git a/graph/src/components/store/cache.rs b/graph/src/components/store/cache.rs
index 90f40e3c72c..46f7b28a8c7 100644
--- a/graph/src/components/store/cache.rs
+++ b/graph/src/components/store/cache.rs
@@ -4,10 +4,10 @@ use std::fmt::{self, Debug};
 use std::sync::Arc;
 
 use crate::blockchain::BlockPtr;
-use crate::blockchain::DataSource;
 use crate::components::store::{
     self as s, Entity, EntityKey, EntityOp, EntityOperation, EntityType,
 };
+use crate::data_source::DataSource;
 use crate::prelude::ENV_VARS;
 use crate::util::lfu_cache::LfuCache;
 
@@ -189,7 +189,7 @@ impl EntityCache {
     }
 
     /// Add a dynamic data source
-    pub fn add_data_source<C: s::Blockchain>(&mut self, data_source: &impl DataSource<C>) {
+    pub fn add_data_source<C: s::Blockchain>(&mut self, data_source: &DataSource<C>) {
         self.data_sources
             .push(data_source.as_stored_dynamic_data_source());
     }
diff --git a/graph/src/components/subgraph/host.rs b/graph/src/components/subgraph/host.rs
index a824aef79ab..5ba795b4673 100644
--- a/graph/src/components/subgraph/host.rs
+++ b/graph/src/components/subgraph/host.rs
@@ -8,7 +8,7 @@ use futures::sync::mpsc;
 
 use crate::blockchain::TriggerWithHandler;
 use crate::components::store::SubgraphFork;
-use crate::data_source::DataSource;
+use crate::data_source::{DataSource, DataSourceTemplate};
 use crate::prelude::*;
 use crate::{blockchain::Blockchain, components::subgraph::SharedProofOfIndexing};
 use crate::{components::metrics::HistogramVec, runtime::DeterministicHostError};
@@ -153,7 +153,7 @@ pub trait RuntimeHostBuilder<C: Blockchain>: Clone + Send + Sync + 'static {
         network_name: String,
         subgraph_id: DeploymentHash,
         data_source: DataSource<C>,
-        top_level_templates: Arc<Vec<C::DataSourceTemplate>>,
+        top_level_templates: Arc<Vec<DataSourceTemplate<C>>>,
         mapping_request_sender: mpsc::Sender<Self::Req>,
         metrics: Arc<HostMetrics>,
     ) -> Result<Self::Host, Error>;
diff --git a/graph/src/components/subgraph/instance.rs b/graph/src/components/subgraph/instance.rs
index 915f3f99a32..f81744df9e8 100644
--- a/graph/src/components/subgraph/instance.rs
+++ b/graph/src/components/subgraph/instance.rs
@@ -1,12 +1,15 @@
-use crate::blockchain::Blockchain;
-use crate::components::store::EntityKey;
-use crate::prelude::*;
-use crate::util::lfu_cache::LfuCache;
-use crate::{components::store::WritableStore, data::subgraph::schema::SubgraphError};
+use crate::{
+    blockchain::Blockchain,
+    components::store::{EntityKey, WritableStore},
+    data::subgraph::schema::SubgraphError,
+    data_source::DataSourceTemplate,
+    prelude::*,
+    util::lfu_cache::LfuCache,
+};
 
 #[derive(Clone, Debug)]
 pub struct DataSourceTemplateInfo<C: Blockchain> {
-    pub template: C::DataSourceTemplate,
+    pub template: DataSourceTemplate<C>,
     pub params: Vec<String>,
     pub context: Option<DataSourceContext>,
     pub creation_block: BlockNumber,
diff --git a/graph/src/data/subgraph/mod.rs b/graph/src/data/subgraph/mod.rs
index 39b1244dcc4..d6c8b9a89ae 100644
--- a/graph/src/data/subgraph/mod.rs
+++ b/graph/src/data/subgraph/mod.rs
@@ -24,25 +24,24 @@ use thiserror::Error;
 use wasmparser;
 use web3::types::Address;
 
-use crate::blockchain::BlockPtr;
-use crate::data::graphql::TryFromValue;
-use crate::data::store::Entity;
-use crate::data::{
-    schema::{Schema, SchemaImportError, SchemaValidationError},
-    subgraph::features::validate_subgraph_features,
-};
-use crate::offchain::OFFCHAIN_KINDS;
-use crate::prelude::{r, CheapClone, ENV_VARS};
 use crate::{
-    blockchain::{
-        Blockchain, DataSource as _, DataSourceTemplate as _, UnresolvedDataSourceTemplate as _,
-    },
+    blockchain::{BlockPtr, Blockchain, DataSource as _},
     components::{
         link_resolver::LinkResolver,
         store::{DeploymentLocator, StoreError, SubgraphStore},
     },
-    data::query::QueryExecutionError,
-    data_source::{DataSource, UnresolvedDataSource},
+    data::{
+        graphql::TryFromValue,
+        query::QueryExecutionError,
+        schema::{Schema, SchemaImportError, SchemaValidationError},
+        store::Entity,
+        subgraph::features::validate_subgraph_features,
+    },
+    data_source::{
+        DataSource, DataSourceTemplate, UnresolvedDataSource, UnresolvedDataSourceTemplate,
+    },
+    offchain::OFFCHAIN_KINDS,
+    prelude::{r, CheapClone, ENV_VARS},
 };
 
 use crate::prelude::{impl_slog_value, BlockNumber, Deserialize, Serialize};
@@ -525,12 +524,12 @@ type UnresolvedSubgraphManifest<C> = BaseSubgraphManifest<
     C,
     UnresolvedSchema,
     UnresolvedDataSource<C>,
-    <C as Blockchain>::UnresolvedDataSourceTemplate,
+    UnresolvedDataSourceTemplate<C>,
 >;
 
 /// SubgraphManifest validated with IPFS links resolved
 pub type SubgraphManifest<C> =
-    BaseSubgraphManifest<C, Schema, DataSource<C>, <C as Blockchain>::DataSourceTemplate>;
+    BaseSubgraphManifest<C, Schema, DataSource<C>, DataSourceTemplate<C>>;
 
 /// Unvalidated SubgraphManifest
 pub struct UnvalidatedSubgraphManifest<C: Blockchain>(SubgraphManifest<C>);
diff --git a/graph/src/data_source.rs b/graph/src/data_source.rs
index c868478b72b..603c23cf7db 100644
--- a/graph/src/data_source.rs
+++ b/graph/src/data_source.rs
@@ -1,6 +1,13 @@
 use crate::{
-    blockchain::{Blockchain, DataSource as _, UnresolvedDataSource as _},
-    components::{link_resolver::LinkResolver, store::BlockNumber},
+    blockchain::{
+        Blockchain, DataSource as _, DataSourceTemplate as _, UnresolvedDataSource as _,
+        UnresolvedDataSourceTemplate as _,
+    },
+    components::{
+        link_resolver::LinkResolver,
+        store::{BlockNumber, StoredDynamicDataSource},
+        subgraph::DataSourceTemplateInfo,
+    },
     offchain::{self, OFFCHAIN_KINDS},
     prelude::{CheapClone as _, DataSourceContext},
 };
@@ -16,6 +23,21 @@ pub enum DataSource<C: Blockchain> {
     Offchain(offchain::DataSource),
 }
 
+impl<C: Blockchain> TryFrom<DataSourceTemplateInfo<C>> for DataSource<C> {
+    type Error = Error;
+
+    fn try_from(info: DataSourceTemplateInfo<C>) -> Result<Self, Self::Error> {
+        match &info.template {
+            DataSourceTemplate::Onchain(_) => {
+                C::DataSource::try_from(info).map(DataSource::Onchain)
+            }
+            DataSourceTemplate::Offchain(_) => {
+                offchain::DataSource::try_from(info).map(DataSource::Offchain)
+            }
+        }
+    }
+}
+
 impl<C: Blockchain> DataSource<C> {
     pub fn as_onchain(&self) -> Option<&C::DataSource> {
         match self {
@@ -83,6 +105,29 @@ impl<C: Blockchain> DataSource<C> {
         }
     }
 
+    pub fn as_stored_dynamic_data_source(&self) -> StoredDynamicDataSource {
+        match self {
+            Self::Onchain(ds) => ds.as_stored_dynamic_data_source(),
+            Self::Offchain(ds) => ds.as_stored_dynamic_data_source(),
+        }
+    }
+
+    pub fn from_stored_dynamic_data_source(
+        template: &DataSourceTemplate<C>,
+        stored: StoredDynamicDataSource,
+    ) -> Result<Self, Error> {
+        match template {
+            DataSourceTemplate::Onchain(template) => {
+                C::DataSource::from_stored_dynamic_data_source(template, stored)
+                    .map(DataSource::Onchain)
+            }
+            DataSourceTemplate::Offchain(template) => {
+                offchain::DataSource::from_stored_dynamic_data_source(template, stored)
+                    .map(DataSource::Offchain)
+            }
+        }
+    }
+
     pub fn validate(&self) -> Vec<Error> {
         match self {
             Self::Onchain(ds) => ds.validate(),
@@ -97,34 +142,6 @@ pub enum UnresolvedDataSource<C: Blockchain> {
     Offchain(offchain::UnresolvedDataSource),
 }
 
-impl<'de, C: Blockchain> Deserialize<'de> for UnresolvedDataSource<C> {
-    fn deserialize<D>(deserializer: D) -> Result<Self, D::Error>
-    where
-        D: Deserializer<'de>,
-    {
-        let map: BTreeMap<String, serde_json::Value> = BTreeMap::deserialize(deserializer)?;
-        let kind = map
-            .get("kind")
-            .ok_or(serde::de::Error::missing_field("kind"))?
-            .as_str()
-            .unwrap_or("?");
-        if OFFCHAIN_KINDS.contains(&kind) {
-            offchain::UnresolvedDataSource::deserialize(map.into_deserializer())
-                .map_err(serde::de::Error::custom)
-                .map(UnresolvedDataSource::Offchain)
-        } else if (&C::KIND.to_string() == kind) || C::ALIASES.contains(&kind) {
-            C::UnresolvedDataSource::deserialize(map.into_deserializer())
-                .map_err(serde::de::Error::custom)
-                .map(UnresolvedDataSource::Onchain)
-        } else {
-            Err(serde::de::Error::custom(format!(
-                "data source has invalid `kind`; expected {}, file/ipfs",
-                C::KIND,
-            )))
-        }
-    }
-}
-
 impl<C: Blockchain> UnresolvedDataSource<C> {
     pub async fn resolve(
         self,
@@ -138,9 +155,142 @@ impl<C: Blockchain> UnresolvedDataSource<C> {
                 .await
                 .map(DataSource::Onchain),
             Self::Offchain(unresolved) => unresolved
-                .resolve(resolver, logger)
+                .resolve(resolver, logger, manifest_idx)
                 .await
                 .map(DataSource::Offchain),
         }
     }
 }
+
+#[derive(Debug)]
+pub enum DataSourceTemplate<C: Blockchain> {
+    Onchain(C::DataSourceTemplate),
+    Offchain(offchain::DataSourceTemplate),
+}
+
+impl<C: Blockchain> DataSourceTemplate<C> {
+    pub fn as_onchain(&self) -> Option<&C::DataSourceTemplate> {
+        match self {
+            Self::Onchain(ds) => Some(ds),
+            Self::Offchain(_) => None,
+        }
+    }
+
+    pub fn into_onchain(self) -> Option<C::DataSourceTemplate> {
+        match self {
+            Self::Onchain(ds) => Some(ds),
+            Self::Offchain(_) => None,
+        }
+    }
+
+    pub fn name(&self) -> &str {
+        match self {
+            Self::Onchain(ds) => ds.name(),
+            Self::Offchain(ds) => &ds.name,
+        }
+    }
+
+    pub fn api_version(&self) -> semver::Version {
+        match self {
+            Self::Onchain(ds) => ds.api_version(),
+            Self::Offchain(ds) => ds.mapping.api_version.clone(),
+        }
+    }
+
+    pub fn runtime(&self) -> Option<Arc<Vec<u8>>> {
+        match self {
+            Self::Onchain(ds) => ds.runtime(),
+            Self::Offchain(ds) => Some(ds.mapping.runtime.clone()),
+        }
+    }
+
+    pub fn manifest_idx(&self) -> u32 {
+        match self {
+            Self::Onchain(ds) => ds.manifest_idx(),
+            Self::Offchain(ds) => ds.manifest_idx,
+        }
+    }
+}
+
+#[derive(Clone, Debug)]
+pub enum UnresolvedDataSourceTemplate<C: Blockchain> {
+    Onchain(C::UnresolvedDataSourceTemplate),
+    Offchain(offchain::UnresolvedDataSourceTemplate),
+}
+
+impl<C: Blockchain> Default for UnresolvedDataSourceTemplate<C> {
+    fn default() -> Self {
+        Self::Onchain(C::UnresolvedDataSourceTemplate::default())
+    }
+}
+
+impl<C: Blockchain> UnresolvedDataSourceTemplate<C> {
+    pub async fn resolve(
+        self,
+        resolver: &Arc<dyn LinkResolver>,
+        logger: &Logger,
+        manifest_idx: u32,
+    ) -> Result<DataSourceTemplate<C>, Error> {
+        match self {
+            Self::Onchain(ds) => ds
+                .resolve(resolver, logger, manifest_idx)
+                .await
+                .map(DataSourceTemplate::Onchain),
+            Self::Offchain(ds) => ds
+                .resolve(resolver, logger, manifest_idx)
+                .await
+                .map(DataSourceTemplate::Offchain),
+        }
+    }
+}
+
+macro_rules! clone_data_source {
+    ($t:ident) => {
+        impl<C: Blockchain> Clone for $t<C> {
+            fn clone(&self) -> Self {
+                match self {
+                    Self::Onchain(ds) => Self::Onchain(ds.clone()),
+                    Self::Offchain(ds) => Self::Offchain(ds.clone()),
+                }
+            }
+        }
+    };
+}
+
+clone_data_source!(DataSource);
+clone_data_source!(DataSourceTemplate);
+
+macro_rules! deserialize_data_source {
+    ($t:ident) => {
+        impl<'de, C: Blockchain> Deserialize<'de> for $t<C> {
+            fn deserialize<D>(deserializer: D) -> Result<Self, D::Error>
+            where
+                D: Deserializer<'de>,
+            {
+                let map: BTreeMap<String, serde_json::Value> = BTreeMap::deserialize(deserializer)?;
+                let kind = map
+                    .get("kind")
+                    .ok_or(serde::de::Error::missing_field("kind"))?
+                    .as_str()
+                    .unwrap_or("?");
+                if OFFCHAIN_KINDS.contains(&kind) {
+                    offchain::$t::deserialize(map.into_deserializer())
+                        .map_err(serde::de::Error::custom)
+                        .map($t::Offchain)
+                } else if (&C::KIND.to_string() == kind) || C::ALIASES.contains(&kind) {
+                    C::$t::deserialize(map.into_deserializer())
+                        .map_err(serde::de::Error::custom)
+                        .map($t::Onchain)
+                } else {
+                    Err(serde::de::Error::custom(format!(
+                        "data source has invalid `kind`; expected {}, file/ipfs",
+                        C::KIND,
+                    )))
+                }
+            }
+        }
+    };
+}
+
+deserialize_data_source!(UnresolvedDataSource);
+deserialize_data_source!(UnresolvedDataSourceTemplate);
diff --git a/graph/src/offchain/mod.rs b/graph/src/offchain/mod.rs
index 57ded430e5f..426536f098a 100644
--- a/graph/src/offchain/mod.rs
+++ b/graph/src/offchain/mod.rs
@@ -1,26 +1,100 @@
 use crate::{
-    components::{link_resolver::LinkResolver, store::BlockNumber},
+    blockchain::Blockchain,
+    components::{
+        link_resolver::LinkResolver,
+        store::{BlockNumber, StoredDynamicDataSource},
+        subgraph::DataSourceTemplateInfo,
+    },
+    data::store::scalar::Bytes,
+    data_source,
     prelude::{DataSourceContext, Link},
 };
 
-use anyhow::anyhow;
+use anyhow::{self, Error};
 use serde::Deserialize;
 use slog::{info, Logger};
 use std::sync::Arc;
 
 pub const OFFCHAIN_KINDS: &'static [&'static str] = &["file/ipfs"];
 
-#[derive(Debug)]
+#[derive(Clone, Debug)]
 pub struct DataSource {
     pub kind: String,
     pub name: String,
+    pub manifest_idx: u32,
     pub source: Option<Source>,
     pub mapping: Mapping,
     pub context: Arc<Option<DataSourceContext>>,
     pub creation_block: Option<BlockNumber>,
 }
 
-#[derive(Debug, Eq, PartialEq)]
+impl<C: Blockchain> TryFrom<DataSourceTemplateInfo<C>> for DataSource {
+    type Error = Error;
+
+    fn try_from(info: DataSourceTemplateInfo<C>) -> Result<Self, Self::Error> {
+        let template = match info.template {
+            data_source::DataSourceTemplate::Offchain(template) => template,
+            data_source::DataSourceTemplate::Onchain(_) => {
+                anyhow::bail!("Cannot create offchain data source from onchain template")
+            }
+        };
+        let source = info.params.get(0).ok_or(anyhow::anyhow!(
+            "Failed to create data source from template `{}`: source parameter is missing",
+            template.name
+        ))?;
+        Ok(Self {
+            kind: template.kind.clone(),
+            name: template.name.clone(),
+            manifest_idx: template.manifest_idx,
+            source: Some(Source::Ipfs(Link::from(source))),
+            mapping: template.mapping.clone(),
+            context: Arc::new(info.context),
+            creation_block: Some(info.creation_block),
+        })
+    }
+}
+
+impl DataSource {
+    pub fn as_stored_dynamic_data_source(&self) -> StoredDynamicDataSource {
+        let param = self.source.as_ref().map(|source| match source {
+            Source::Ipfs(link) => Bytes::from(link.link.as_bytes()),
+        });
+        let context = self
+            .context
+            .as_ref()
+            .as_ref()
+            .map(|ctx| serde_json::to_value(&ctx).unwrap());
+        StoredDynamicDataSource {
+            manifest_idx: self.manifest_idx,
+            param,
+            context,
+            creation_block: self.creation_block,
+        }
+    }
+
+    pub fn from_stored_dynamic_data_source(
+        template: &DataSourceTemplate,
+        stored: StoredDynamicDataSource,
+    ) -> Result<Self, Error> {
+        let source = stored.param.and_then(|bytes| {
+            String::from_utf8(bytes.as_slice().to_vec())
+                .ok()
+                .map(|link| Source::Ipfs(Link::from(link)))
+        });
+        let context = Arc::new(stored.context.map(serde_json::from_value).transpose()?);
+        Ok(Self {
+            kind: template.kind.clone(),
+            name: template.name.clone(),
+            manifest_idx: stored.manifest_idx,
+            source,
+            mapping: template.mapping.clone(),
+            context,
+            creation_block: stored.creation_block,
+        })
+    }
+}
+
+#[derive(Clone, Debug, Eq, PartialEq)]
 pub enum Source {
     Ipfs(Link),
 }
@@ -63,7 +137,8 @@ impl UnresolvedDataSource {
         self,
         resolver: &Arc<dyn LinkResolver>,
         logger: &Logger,
-    ) -> Result<DataSource, anyhow::Error> {
+        manifest_idx: u32,
+    ) -> Result<DataSource, Error> {
         info!(logger, "Resolve offchain data source";
             "name" => &self.name,
             "kind" => &self.kind,
@@ -72,13 +147,14 @@ impl UnresolvedDataSource {
         let source = match self.kind.as_str() {
             "file/ipfs" => self.source.map(|src| Source::Ipfs(src.file)),
             _ => {
-                return Err(anyhow!(
+                anyhow::bail!(
                     "offchain data source has invalid `kind`, expected `file/ipfs` but found {}",
                     self.kind
-                ))
+                );
             }
         };
         Ok(DataSource {
+            manifest_idx,
             kind: self.kind,
             name: self.name,
             source,
@@ -94,7 +170,7 @@ impl UnresolvedMapping {
         self,
         resolver: &Arc<dyn LinkResolver>,
         logger: &Logger,
-    ) -> Result<Mapping, anyhow::Error> {
+    ) -> Result<Mapping, Error> {
         info!(logger, "Resolve offchain mapping"; "link" => &self.file.link);
         Ok(Mapping {
             language: self.language,
@@ -106,3 +182,34 @@ impl UnresolvedMapping {
         })
     }
 }
+
+#[derive(Clone, Debug, Hash, Eq, PartialEq, Deserialize)]
+pub struct BaseDataSourceTemplate<M> {
+    pub kind: String,
+    pub network: Option<String>,
+    pub name: String,
+    pub manifest_idx: u32,
+    pub mapping: M,
+}
+
+pub type UnresolvedDataSourceTemplate = BaseDataSourceTemplate<UnresolvedMapping>;
+pub type DataSourceTemplate = BaseDataSourceTemplate<Mapping>;
+
+impl UnresolvedDataSourceTemplate {
+    pub async fn resolve(
+        self,
+        resolver: &Arc<dyn LinkResolver>,
+        logger: &Logger,
+        manifest_idx: u32,
+    ) -> Result<DataSourceTemplate, Error> {
+        info!(logger, "Resolve data source template"; "name" => &self.name);
+
+        Ok(DataSourceTemplate {
+            kind: self.kind,
+            network: self.network,
+            name: self.name,
+            manifest_idx,
+            mapping: self.mapping.resolve(resolver, logger).await?,
+        })
+    }
+}
diff --git a/runtime/test/src/common.rs b/runtime/test/src/common.rs
index 9fa1f228b89..8b0d77b4aa5 100644
--- a/runtime/test/src/common.rs
+++ b/runtime/test/src/common.rs
@@ -28,29 +28,31 @@ fn mock_host_exports(
     store: Arc<impl SubgraphStore>,
     api_version: Version,
 ) -> HostExports<Chain> {
-    let templates = vec![DataSourceTemplate {
-        kind: String::from("ethereum/contract"),
-        name: String::from("example template"),
-        manifest_idx: 0,
-        network: Some(String::from("mainnet")),
-        source: TemplateSource {
-            abi: String::from("foo"),
-        },
-        mapping: Mapping {
-            kind: String::from("ethereum/events"),
-            api_version,
-            language: String::from("wasm/assemblyscript"),
-            entities: vec![],
-            abis: vec![],
-            event_handlers: vec![],
-            call_handlers: vec![],
-            block_handlers: vec![],
-            link: Link {
-                link: "link".to_owned(),
+    let templates = vec![data_source::DataSourceTemplate::Onchain(
+        DataSourceTemplate {
+            kind: String::from("ethereum/contract"),
+            name: String::from("example template"),
+            manifest_idx: 0,
+            network: Some(String::from("mainnet")),
+            source: TemplateSource {
+                abi: String::from("foo"),
+            },
+            mapping: Mapping {
+                kind: String::from("ethereum/events"),
+                api_version,
+                language: String::from("wasm/assemblyscript"),
+                entities: vec![],
+                abis: vec![],
+                event_handlers: vec![],
+                call_handlers: vec![],
+                block_handlers: vec![],
+                link: Link {
+                    link: "link".to_owned(),
+                },
+                runtime: Arc::new(vec![]),
             },
-            runtime: Arc::new(vec![]),
         },
-    }];
+    )];
 
     let network = data_source.network.clone().unwrap();
     let ens_lookup = store.ens_lookup();
diff --git a/runtime/test/src/test.rs b/runtime/test/src/test.rs
index b384bace4dd..6b17c29327d 100644
--- a/runtime/test/src/test.rs
+++ b/runtime/test/src/test.rs
@@ -833,7 +833,7 @@ async fn test_data_source_create(api_version: Version, gas_used: u64) {
     .await
     .expect("unexpected error returned from dataSourceCreate");
     assert_eq!(result[0].params, params.clone());
-    assert_eq!(result[0].template.name, template);
+    assert_eq!(result[0].template.name(), template);
 
     // Test with a template that doesn't exist
     let template = String::from("nonexistent template");
diff --git a/runtime/wasm/src/host.rs b/runtime/wasm/src/host.rs
index 5a670804f99..c540b6cc4e2 100644
--- a/runtime/wasm/src/host.rs
+++ b/runtime/wasm/src/host.rs
@@ -8,7 +8,7 @@ use futures03::channel::oneshot::channel;
 use graph::blockchain::{Blockchain, DataSource as _, HostFn, RuntimeAdapter, TriggerWithHandler};
 use graph::components::store::{EnsLookup, SubgraphFork};
 use graph::components::subgraph::{MappingError, SharedProofOfIndexing};
-use graph::data_source::DataSource;
+use graph::data_source::{DataSource, DataSourceTemplate};
 use graph::prelude::{
     RuntimeHost as RuntimeHostTrait, RuntimeHostBuilder as RuntimeHostBuilderTrait, *,
 };
@@ -76,7 +76,7 @@ impl<C: Blockchain> RuntimeHostBuilderTrait<C> for RuntimeHostBuilder<C> {
         network_name: String,
         subgraph_id: DeploymentHash,
         data_source: DataSource<C>,
-        templates: Arc<Vec<C::DataSourceTemplate>>,
+        templates: Arc<Vec<DataSourceTemplate<C>>>,
         mapping_request_sender: Sender<MappingRequest<C>>,
         metrics: Arc<HostMetrics>,
     ) -> Result<Self::Host, Error> {
@@ -112,7 +112,7 @@ where
         network_name: String,
         subgraph_id: DeploymentHash,
         data_source: DataSource<C>,
-        templates: Arc<Vec<C::DataSourceTemplate>>,
+        templates: Arc<Vec<DataSourceTemplate<C>>>,
         mapping_request_sender: Sender<MappingRequest<C>>,
         metrics: Arc<HostMetrics>,
         ens_lookup: Arc<dyn EnsLookup>,
diff --git a/runtime/wasm/src/host_exports.rs b/runtime/wasm/src/host_exports.rs
index c04a4a342f7..cbe1efbe9b0 100644
--- a/runtime/wasm/src/host_exports.rs
+++ b/runtime/wasm/src/host_exports.rs
@@ -8,12 +8,12 @@ use semver::Version;
 use wasmtime::Trap;
 use web3::types::H160;
 
-use graph::blockchain::{Blockchain, DataSourceTemplate as _};
+use graph::blockchain::Blockchain;
 use graph::components::store::EnsLookup;
 use graph::components::store::{EntityKey, EntityType};
 use graph::components::subgraph::{CausalityRegion, ProofOfIndexingEvent, SharedProofOfIndexing};
 use graph::data::store;
-use graph::data_source::DataSource;
+use graph::data_source::{DataSource, DataSourceTemplate};
 use graph::ensure;
 use graph::prelude::ethabi::param_type::Reader;
 use graph::prelude::ethabi::{decode, encode, Token};
@@ -66,7 +66,7 @@ pub struct HostExports<C: Blockchain> {
     /// and merge the results later. Right now, this is just the ethereum
     /// networks but will be expanded for ipfs and the availability chain.
     causality_region: String,
-    templates: Arc<Vec<C::DataSourceTemplate>>,
+    templates: Arc<Vec<DataSourceTemplate<C>>>,
     pub(crate) link_resolver: Arc<dyn LinkResolver>,
     ens_lookup: Arc<dyn EnsLookup>,
 }
@@ -76,7 +76,7 @@ impl<C: Blockchain> HostExports<C> {
         subgraph_id: DeploymentHash,
         data_source: &DataSource<C>,
         data_source_network: String,
-        templates: Arc<Vec<C::DataSourceTemplate>>,
+        templates: Arc<Vec<DataSourceTemplate<C>>>,
         link_resolver: Arc<dyn LinkResolver>,
         ens_lookup: Arc<dyn EnsLookup>,
     ) -> Self {

From 55f8847720b7cfa7d84d9680cd9afde0b68bf163 Mon Sep 17 00:00:00 2001
From: Theo Butler <theodusbutler@gmail.com>
Date: Fri, 29 Jul 2022 15:39:41 -0400
Subject: [PATCH 03/33] Start offchain monitors for static sources

---
 Cargo.lock                                    |  1 +
 chain/ethereum/tests/manifest.rs              | 19 ++---
 core/src/polling_monitor/mod.rs               |  2 +-
 core/src/subgraph/context.rs                  | 72 ++++++++++++++++++-
 core/src/subgraph/instance.rs                 | 12 +++-
 core/src/subgraph/instance_manager.rs         | 15 ++++
 graph/Cargo.toml                              |  3 +-
 graph/src/data/subgraph/mod.rs                |  4 +-
 .../{data_source.rs => data_source/mod.rs}    |  4 +-
 .../mod.rs => data_source/offchain.rs}        | 17 +++--
 graph/src/lib.rs                              |  2 -
 node/src/main.rs                              |  2 +
 node/src/manager/commands/run.rs              |  2 +
 tests/src/fixture.rs                          |  6 +-
 14 files changed, 134 insertions(+), 27 deletions(-)
 rename graph/src/{data_source.rs => data_source/mod.rs} (99%)
 rename graph/src/{offchain/mod.rs => data_source/offchain.rs} (94%)

diff --git a/Cargo.lock b/Cargo.lock
index 20211d35b5a..4f2a232d01d 100644
--- a/Cargo.lock
+++ b/Cargo.lock
@@ -1463,6 +1463,7 @@ dependencies = [
  "bigdecimal",
  "bytes",
  "chrono",
+ "cid",
  "diesel",
  "diesel_derives",
  "envconfig",
diff --git a/chain/ethereum/tests/manifest.rs b/chain/ethereum/tests/manifest.rs
index 2f3782fcbf9..5fee9aad492 100644
--- a/chain/ethereum/tests/manifest.rs
+++ b/chain/ethereum/tests/manifest.rs
@@ -4,8 +4,7 @@ use std::time::Duration;
 
 use graph::data::subgraph::schema::SubgraphError;
 use graph::data::subgraph::{SPEC_VERSION_0_0_4, SPEC_VERSION_0_0_7};
-use graph::data_source::DataSource;
-use graph::offchain;
+use graph::data_source::{offchain, DataSource};
 use graph::prelude::{
     anyhow, async_trait, serde_yaml, tokio, DeploymentHash, Entity, Link, Logger, SubgraphManifest,
     SubgraphManifestValidationError, UnvalidatedSubgraphManifest,
@@ -28,6 +27,7 @@ const GQL_SCHEMA_FULLTEXT: &str = include_str!("full-text.graphql");
 const MAPPING_WITH_IPFS_FUNC_WASM: &[u8] = include_bytes!("ipfs-on-ethereum-contracts.wasm");
 const ABI: &str = "[{\"type\":\"function\", \"inputs\": [{\"name\": \"i\",\"type\": \"uint256\"}],\"name\":\"get\",\"outputs\": [{\"type\": \"address\",\"name\": \"o\"}]}]";
 const FILE: &str = "{}";
+const FILE_CID: &str = "bafkreigkhuldxkyfkoaye4rgcqcwr45667vkygd45plwq6hawy7j4rbdky";
 
 #[derive(Default, Debug, Clone)]
 struct TextResolver {
@@ -84,7 +84,7 @@ async fn resolve_manifest(
     resolver.add("/ipfs/Qmschema", &GQL_SCHEMA);
     resolver.add("/ipfs/Qmabi", &ABI);
     resolver.add("/ipfs/Qmmapping", &MAPPING_WITH_IPFS_FUNC_WASM);
-    resolver.add("/ipfs/Qmfile", &FILE);
+    resolver.add(FILE_CID, &FILE);
 
     let resolver: Arc<dyn LinkResolverTrait> = Arc::new(resolver);
 
@@ -130,7 +130,8 @@ specVersion: 0.0.2
 
 #[tokio::test]
 async fn ipfs_manifest() {
-    const YAML: &str = "
+    let yaml = format!(
+        "
 schema:
   file:
     /: /ipfs/Qmschema
@@ -139,7 +140,7 @@ dataSources:
     kind: file/ipfs
     source:
       file:
-        /: /ipfs/Qmfile
+        /: {}
     mapping:
       apiVersion: 0.0.6
       language: wasm/assemblyscript
@@ -149,9 +150,11 @@ dataSources:
         /: /ipfs/Qmmapping
       handler: handleFile
 specVersion: 0.0.7
-";
+",
+        FILE_CID
+    );
 
-    let manifest = resolve_manifest(YAML, SPEC_VERSION_0_0_7).await;
+    let manifest = resolve_manifest(&yaml, SPEC_VERSION_0_0_7).await;
 
     assert_eq!("Qmmanifest", manifest.id.as_str());
     assert_eq!(manifest.data_sources.len(), 1);
@@ -161,7 +164,7 @@ specVersion: 0.0.7
     };
     assert_eq!(
         data_source.source,
-        Some(offchain::Source::Ipfs(Link::from("/ipfs/Qmfile")))
+        Some(offchain::Source::Ipfs(FILE_CID.parse().unwrap()))
     );
 }
 
diff --git a/core/src/polling_monitor/mod.rs b/core/src/polling_monitor/mod.rs
index 1ffe87c9482..cb8dbf4c340 100644
--- a/core/src/polling_monitor/mod.rs
+++ b/core/src/polling_monitor/mod.rs
@@ -14,7 +14,7 @@ use graph::util::monitored::MonitoredVecDeque as VecDeque;
 use tokio::sync::{mpsc, watch};
 use tower::{Service, ServiceExt};
 
-use self::metrics::PollingMonitorMetrics;
+pub use self::metrics::PollingMonitorMetrics;
 
 /// Spawn a monitor that actively polls a service. Whenever the service has capacity, the monitor
 /// pulls object ids from the queue and polls the service. If the object is not present or in case
diff --git a/core/src/subgraph/context.rs b/core/src/subgraph/context.rs
index a98e3579999..c30047018e3 100644
--- a/core/src/subgraph/context.rs
+++ b/core/src/subgraph/context.rs
@@ -1,10 +1,21 @@
-use crate::subgraph::SubgraphInstance;
+use crate::{
+    polling_monitor::{
+        ipfs_service::IpfsService, spawn_monitor, PollingMonitor, PollingMonitorMetrics,
+    },
+    subgraph::SubgraphInstance,
+};
+use bytes::Bytes;
+use cid::Cid;
 use graph::{
     blockchain::Blockchain,
     components::store::DeploymentId,
-    prelude::{CancelGuard, RuntimeHostBuilder},
+    data_source::offchain,
+    env::ENV_VARS,
+    ipfs_client::IpfsClient,
+    prelude::{CancelGuard, DeploymentHash, MetricsRegistry, RuntimeHostBuilder},
+    slog::Logger,
+    tokio::{select, spawn, sync::mpsc},
 };
-
 use std::collections::HashMap;
 use std::sync::{Arc, RwLock};
 
@@ -18,4 +29,59 @@ where
     pub instance: SubgraphInstance<C, T>,
     pub instances: SharedInstanceKeepAliveMap,
     pub filter: C::TriggerFilter,
+    pub offchain_monitor: OffchainMonitor,
+    pub offchain_monitor_rx: mpsc::Receiver<(offchain::Source, Bytes)>,
+}
+
+pub struct OffchainMonitor {
+    ipfs_monitor: PollingMonitor<Cid>,
+}
+
+impl OffchainMonitor {
+    pub fn new(
+        logger: Logger,
+        registry: Arc<dyn MetricsRegistry>,
+        subgraph_hash: &DeploymentHash,
+        client: IpfsClient,
+    ) -> (Self, mpsc::Receiver<(offchain::Source, Bytes)>) {
+        let (ipfs_monitor_tx, mut ipfs_monitor_rx) = mpsc::channel(10);
+        let ipfs_service = IpfsService::new(
+            client,
+            ENV_VARS.mappings.max_ipfs_file_bytes.unwrap_or(1 << 20) as u64,
+            ENV_VARS.mappings.ipfs_timeout,
+            10,
+        );
+        let ipfs_monitor = spawn_monitor(
+            ipfs_service,
+            ipfs_monitor_tx,
+            logger,
+            PollingMonitorMetrics::new(registry, subgraph_hash),
+        );
+        let (tx, rx) = mpsc::channel(10);
+        spawn(async move {
+            loop {
+                select! {
+                    msg = ipfs_monitor_rx.recv() => {
+                        match msg {
+                            Some((cid, bytes)) => {
+                                if let Err(_) = tx.send((offchain::Source::Ipfs(cid), bytes)).await {
+                                    break;
+                                }
+                            },
+                            None => break,
+                        }
+                    }
+                }
+            }
+        });
+        (Self { ipfs_monitor }, rx)
+    }
+
+    pub fn monitor(&self, source: offchain::Source) {
+        match source {
+            offchain::Source::Ipfs(cid) => {
+                self.ipfs_monitor.monitor(cid);
+            }
+        }
+    }
 }
diff --git a/core/src/subgraph/instance.rs b/core/src/subgraph/instance.rs
index a277544a7ed..c09fc78f544 100644
--- a/core/src/subgraph/instance.rs
+++ b/core/src/subgraph/instance.rs
@@ -11,6 +11,8 @@ use graph::{
 };
 use std::collections::HashMap;
 
+use super::context::OffchainMonitor;
+
 pub struct SubgraphInstance<C: Blockchain, T: RuntimeHostBuilder<C>> {
     subgraph_id: DeploymentHash,
     network: String,
@@ -40,6 +42,7 @@ where
         host_builder: T,
         trigger_processor: Box<dyn TriggerProcessor<C, T>>,
         host_metrics: Arc<HostMetrics>,
+        offchain_monitor: &OffchainMonitor,
     ) -> Result<Self, Error> {
         let subgraph_id = manifest.id.clone();
         let network = manifest.network_name();
@@ -71,6 +74,13 @@ where
                 Some(ref module_bytes) => module_bytes,
             };
 
+            // Create services for static offchain data sources
+            if let DataSource::Offchain(ds) = &ds {
+                if let Some(source) = &ds.source {
+                    offchain_monitor.monitor(source.clone());
+                }
+            }
+
             let host = this.new_host(
                 logger.cheap_clone(),
                 ds,
@@ -78,7 +88,7 @@ where
                 templates.cheap_clone(),
                 host_metrics.cheap_clone(),
             )?;
-            this.hosts.push(Arc::new(host))
+            this.hosts.push(Arc::new(host));
         }
 
         Ok(this)
diff --git a/core/src/subgraph/instance_manager.rs b/core/src/subgraph/instance_manager.rs
index f621cddf8a9..8ae0c22e3e2 100644
--- a/core/src/subgraph/instance_manager.rs
+++ b/core/src/subgraph/instance_manager.rs
@@ -7,11 +7,13 @@ use graph::blockchain::block_stream::BlockStreamMetrics;
 use graph::blockchain::Blockchain;
 use graph::blockchain::NodeCapabilities;
 use graph::blockchain::{BlockchainKind, TriggerFilter};
+use graph::ipfs_client::IpfsClient;
 use graph::prelude::{SubgraphInstanceManager as SubgraphInstanceManagerTrait, *};
 use graph::{blockchain::BlockchainMap, components::store::DeploymentLocator};
 use graph_runtime_wasm::RuntimeHostBuilder;
 use tokio::task;
 
+use super::context::OffchainMonitor;
 use super::SubgraphTriggerProcessor;
 
 pub struct SubgraphInstanceManager<S: SubgraphStore> {
@@ -22,6 +24,7 @@ pub struct SubgraphInstanceManager<S: SubgraphStore> {
     manager_metrics: SubgraphInstanceManagerMetrics,
     instances: SharedInstanceKeepAliveMap,
     link_resolver: Arc<dyn LinkResolver>,
+    ipfs_client: IpfsClient,
     static_filters: bool,
 }
 
@@ -133,6 +136,7 @@ impl<S: SubgraphStore> SubgraphInstanceManager<S> {
         chains: Arc<BlockchainMap>,
         metrics_registry: Arc<dyn MetricsRegistry>,
         link_resolver: Arc<dyn LinkResolver>,
+        ipfs_client: IpfsClient,
         static_filters: bool,
     ) -> Self {
         let logger = logger_factory.component_logger("SubgraphInstanceManager", None);
@@ -146,6 +150,7 @@ impl<S: SubgraphStore> SubgraphInstanceManager<S> {
             metrics_registry,
             instances: SharedInstanceKeepAliveMap::default(),
             link_resolver,
+            ipfs_client,
             static_filters,
         }
     }
@@ -290,6 +295,13 @@ impl<S: SubgraphStore> SubgraphInstanceManager<S> {
             stopwatch_metrics,
         ));
 
+        let (offchain_monitor, offchain_monitor_rx) = OffchainMonitor::new(
+            logger.cheap_clone(),
+            registry.cheap_clone(),
+            &manifest.id,
+            self.ipfs_client.cheap_clone(),
+        );
+
         // Initialize deployment_head with current deployment head. Any sort of trouble in
         // getting the deployment head ptr leads to initializing with 0
         let deployment_head = store.block_ptr().map(|ptr| ptr.number).unwrap_or(0) as f64;
@@ -309,6 +321,7 @@ impl<S: SubgraphStore> SubgraphInstanceManager<S> {
             host_builder,
             tp,
             host_metrics.clone(),
+            &offchain_monitor,
         )?;
 
         let inputs = IndexingInputs {
@@ -331,6 +344,8 @@ impl<S: SubgraphStore> SubgraphInstanceManager<S> {
             instance,
             instances: self.instances.cheap_clone(),
             filter,
+            offchain_monitor,
+            offchain_monitor_rx,
         };
 
         let metrics = RunnerMetrics {
diff --git a/graph/Cargo.toml b/graph/Cargo.toml
index 2af199a1d71..6338d38c197 100644
--- a/graph/Cargo.toml
+++ b/graph/Cargo.toml
@@ -10,6 +10,7 @@ async-stream = "0.3"
 atomic_refcell = "0.1.8"
 bigdecimal = { version = "0.1.0", features = ["serde"] }
 bytes = "1.0.1"
+cid = "0.8.3"
 diesel = { version = "1.4.8", features = ["postgres", "serde_json", "numeric", "r2d2", "chrono"] }
 diesel_derives = "1.4"
 chrono = "0.4.22"
@@ -69,4 +70,4 @@ maplit = "1.0.2"
 structopt = { version = "0.3" }
 
 [build-dependencies]
-tonic-build = { version = "0.7.2", features = ["prost"] }
\ No newline at end of file
+tonic-build = { version = "0.7.2", features = ["prost"] }
diff --git a/graph/src/data/subgraph/mod.rs b/graph/src/data/subgraph/mod.rs
index d6c8b9a89ae..939c5bd3f65 100644
--- a/graph/src/data/subgraph/mod.rs
+++ b/graph/src/data/subgraph/mod.rs
@@ -38,9 +38,9 @@ use crate::{
         subgraph::features::validate_subgraph_features,
     },
     data_source::{
-        DataSource, DataSourceTemplate, UnresolvedDataSource, UnresolvedDataSourceTemplate,
+        offchain::OFFCHAIN_KINDS, DataSource, DataSourceTemplate, UnresolvedDataSource,
+        UnresolvedDataSourceTemplate,
     },
-    offchain::OFFCHAIN_KINDS,
     prelude::{r, CheapClone, ENV_VARS},
 };
 
diff --git a/graph/src/data_source.rs b/graph/src/data_source/mod.rs
similarity index 99%
rename from graph/src/data_source.rs
rename to graph/src/data_source/mod.rs
index 603c23cf7db..56231b5060d 100644
--- a/graph/src/data_source.rs
+++ b/graph/src/data_source/mod.rs
@@ -1,3 +1,5 @@
+pub mod offchain;
+
 use crate::{
     blockchain::{
         Blockchain, DataSource as _, DataSourceTemplate as _, UnresolvedDataSource as _,
@@ -8,7 +10,7 @@ use crate::{
         store::{BlockNumber, StoredDynamicDataSource},
         subgraph::DataSourceTemplateInfo,
     },
-    offchain::{self, OFFCHAIN_KINDS},
+    data_source::offchain::OFFCHAIN_KINDS,
     prelude::{CheapClone as _, DataSourceContext},
 };
 use anyhow::Error;
diff --git a/graph/src/offchain/mod.rs b/graph/src/data_source/offchain.rs
similarity index 94%
rename from graph/src/offchain/mod.rs
rename to graph/src/data_source/offchain.rs
index 426536f098a..d2d6fb79e61 100644
--- a/graph/src/offchain/mod.rs
+++ b/graph/src/data_source/offchain.rs
@@ -9,8 +9,8 @@ use crate::{
     data_source,
     prelude::{DataSourceContext, Link},
 };
-
 use anyhow::{self, Error};
+use cid::Cid;
 use serde::Deserialize;
 use slog::{info, Logger};
 use std::sync::Arc;
@@ -46,7 +46,7 @@ impl<C: Blockchain> TryFrom<DataSourceTemplateInfo<C>> for DataSource {
             kind: template.kind.clone(),
             name: template.name.clone(),
             manifest_idx: template.manifest_idx,
-            source: Some(Source::Ipfs(Link::from(source))),
+            source: Some(Source::Ipfs(source.parse()?)),
             mapping: template.mapping.clone(),
             context: Arc::new(info.context),
             creation_block: Some(info.creation_block),
@@ -57,7 +57,7 @@ impl<C: Blockchain> TryFrom<DataSourceTemplateInfo<C>> for DataSource {
 impl DataSource {
     pub fn as_stored_dynamic_data_source(&self) -> StoredDynamicDataSource {
         let param = self.source.as_ref().map(|source| match source {
-            Source::Ipfs(link) => Bytes::from(link.link.as_bytes()),
+            Source::Ipfs(link) => Bytes::from(link.to_bytes()),
         });
         let context = self
             .context
@@ -77,9 +77,9 @@ impl DataSource {
         stored: StoredDynamicDataSource,
     ) -> Result<Self, Error> {
         let source = stored.param.and_then(|bytes| {
-            String::from_utf8(bytes.as_slice().to_vec())
+            Cid::try_from(bytes.as_slice().to_vec())
                 .ok()
-                .map(|link| Source::Ipfs(Link::from(link)))
+                .map(Source::Ipfs)
         });
         let context = Arc::new(stored.context.map(serde_json::from_value).transpose()?);
         Ok(Self {
@@ -96,7 +96,7 @@ impl DataSource {
 
 #[derive(Clone, Debug, Eq, PartialEq)]
 pub enum Source {
-    Ipfs(Link),
+    Ipfs(Cid),
 }
 
 #[derive(Clone, Debug)]
@@ -145,7 +145,10 @@ impl UnresolvedDataSource {
             "source" => format_args!("{:?}", &self.source),
         );
         let source = match self.kind.as_str() {
-            "file/ipfs" => self.source.map(|src| Source::Ipfs(src.file)),
+            "file/ipfs" => self
+                .source
+                .map(|src| src.file.link.parse().map(Source::Ipfs))
+                .transpose()?,
             _ => {
                 anyhow::bail!(
                     "offchain data source has invalid `kind`, expected `file/ipfs` but found {}",
diff --git a/graph/src/lib.rs b/graph/src/lib.rs
index d238926a385..8d8073fc013 100644
--- a/graph/src/lib.rs
+++ b/graph/src/lib.rs
@@ -22,8 +22,6 @@ pub mod data_source;
 
 pub mod blockchain;
 
-pub mod offchain;
-
 pub mod runtime;
 
 pub mod firehose;
diff --git a/node/src/main.rs b/node/src/main.rs
index 6ed5f63672b..acf68a63b0b 100644
--- a/node/src/main.rs
+++ b/node/src/main.rs
@@ -195,6 +195,7 @@ async fn main() {
 
     // Try to create IPFS clients for each URL specified in `--ipfs`
     let ipfs_clients: Vec<_> = create_ipfs_clients(&logger, &opt.ipfs);
+    let ipfs_client = ipfs_clients.first().cloned().expect("Missing IPFS client");
 
     // Convert the clients into a link resolver. Since we want to get past
     // possible temporary DNS failures, make the resolver retry
@@ -433,6 +434,7 @@ async fn main() {
             blockchain_map.cheap_clone(),
             metrics_registry.clone(),
             link_resolver.clone(),
+            ipfs_client,
             static_filters,
         );
 
diff --git a/node/src/manager/commands/run.rs b/node/src/manager/commands/run.rs
index aafacb9f086..7e6887819ba 100644
--- a/node/src/manager/commands/run.rs
+++ b/node/src/manager/commands/run.rs
@@ -62,6 +62,7 @@ pub async fn run(
 
     // FIXME: Hard-coded IPFS config, take it from config file instead?
     let ipfs_clients: Vec<_> = create_ipfs_clients(&logger, &ipfs_url);
+    let ipfs_client = ipfs_clients.first().cloned().expect("Missing IPFS client");
 
     // Convert the clients into a link resolver. Since we want to get past
     // possible temporary DNS failures, make the resolver retry
@@ -153,6 +154,7 @@ pub async fn run(
         blockchain_map.clone(),
         metrics_registry.clone(),
         link_resolver.cheap_clone(),
+        ipfs_client,
         static_filters,
     );
 
diff --git a/tests/src/fixture.rs b/tests/src/fixture.rs
index 97a25cfd267..c0a54b299db 100644
--- a/tests/src/fixture.rs
+++ b/tests/src/fixture.rs
@@ -199,7 +199,10 @@ pub async fn setup<C: Blockchain>(
     let static_filters = ENV_VARS.experimental_static_filters;
 
     let ipfs = IpfsClient::localhost();
-    let link_resolver = Arc::new(LinkResolver::new(vec![ipfs], Default::default()));
+    let link_resolver = Arc::new(LinkResolver::new(
+        vec![ipfs.cheap_clone()],
+        Default::default(),
+    ));
 
     let blockchain_map = Arc::new(blockchain_map);
     let subgraph_instance_manager = SubgraphInstanceManager::new(
@@ -208,6 +211,7 @@ pub async fn setup<C: Blockchain>(
         blockchain_map.clone(),
         mock_registry.clone(),
         link_resolver.cheap_clone(),
+        ipfs,
         static_filters,
     );
 

From ca94702d845a8f28e61adffc387d44f7383b43da Mon Sep 17 00:00:00 2001
From: Theo Butler <theodusbutler@gmail.com>
Date: Sat, 30 Jul 2022 13:44:04 -0400
Subject: [PATCH 04/33] Run offchain handlers

---
 Cargo.lock                                |   1 +
 chain/arweave/src/data_source.rs          |   2 +-
 chain/arweave/src/trigger.rs              |   4 +-
 chain/cosmos/src/data_source.rs           |   2 +-
 chain/cosmos/src/trigger.rs               |   5 +-
 chain/ethereum/src/chain.rs               |   6 ++
 chain/ethereum/src/data_source.rs         |   6 +-
 chain/ethereum/src/trigger.rs             |   4 +-
 chain/near/src/data_source.rs             |   2 +-
 chain/near/src/trigger.rs                 |  10 +-
 chain/substreams/Cargo.toml               |   1 +
 chain/substreams/src/trigger.rs           |   6 +-
 core/src/subgraph/context.rs              |  47 ++++-----
 core/src/subgraph/instance.rs             |  10 +-
 core/src/subgraph/instance_manager.rs     |  11 ++-
 core/src/subgraph/runner.rs               | 104 ++++++++++++++++++--
 core/src/subgraph/trigger_processor.rs    |   6 +-
 graph/src/blockchain/mock.rs              |  17 +---
 graph/src/blockchain/mod.rs               |  72 +-------------
 graph/src/components/subgraph/host.rs     |  11 ++-
 graph/src/components/trigger_processor.rs |   4 +-
 graph/src/data_source/mod.rs              | 113 +++++++++++++++++++++-
 graph/src/data_source/offchain.rs         |  24 ++++-
 runtime/wasm/src/host.rs                  |  25 ++---
 runtime/wasm/src/mapping.rs               |  19 ++--
 runtime/wasm/src/module/mod.rs            |  56 ++++++++++-
 26 files changed, 388 insertions(+), 180 deletions(-)

diff --git a/Cargo.lock b/Cargo.lock
index 4f2a232d01d..d8f88615003 100644
--- a/Cargo.lock
+++ b/Cargo.lock
@@ -1611,6 +1611,7 @@ dependencies = [
  "futures 0.1.31",
  "graph",
  "graph-core",
+ "graph-runtime-wasm",
  "hex",
  "http",
  "itertools",
diff --git a/chain/arweave/src/data_source.rs b/chain/arweave/src/data_source.rs
index 56be161858d..dcf74e8e760 100644
--- a/chain/arweave/src/data_source.rs
+++ b/chain/arweave/src/data_source.rs
@@ -65,7 +65,7 @@ impl blockchain::DataSource<Chain> for DataSource {
             },
         };
 
-        Ok(Some(TriggerWithHandler::new(
+        Ok(Some(TriggerWithHandler::<Chain>::new(
             trigger.cheap_clone(),
             handler.to_owned(),
         )))
diff --git a/chain/arweave/src/trigger.rs b/chain/arweave/src/trigger.rs
index 3963795b666..9d2f7ad3a4d 100644
--- a/chain/arweave/src/trigger.rs
+++ b/chain/arweave/src/trigger.rs
@@ -1,4 +1,3 @@
-use graph::blockchain;
 use graph::blockchain::Block;
 use graph::blockchain::TriggerData;
 use graph::cheap_clone::CheapClone;
@@ -9,6 +8,7 @@ use graph::runtime::gas::GasCounter;
 use graph::runtime::AscHeap;
 use graph::runtime::AscPtr;
 use graph::runtime::DeterministicHostError;
+use graph_runtime_wasm::module::ToAscPtr;
 use std::{cmp::Ordering, sync::Arc};
 
 use crate::codec;
@@ -33,7 +33,7 @@ impl std::fmt::Debug for ArweaveTrigger {
     }
 }
 
-impl blockchain::MappingTrigger for ArweaveTrigger {
+impl ToAscPtr for ArweaveTrigger {
     fn to_asc_ptr<H: AscHeap>(
         self,
         heap: &mut H,
diff --git a/chain/cosmos/src/data_source.rs b/chain/cosmos/src/data_source.rs
index 768ba272391..79e5e43f485 100644
--- a/chain/cosmos/src/data_source.rs
+++ b/chain/cosmos/src/data_source.rs
@@ -74,7 +74,7 @@ impl blockchain::DataSource<Chain> for DataSource {
             },
         };
 
-        Ok(Some(TriggerWithHandler::new(
+        Ok(Some(TriggerWithHandler::<Chain>::new(
             trigger.cheap_clone(),
             handler,
         )))
diff --git a/chain/cosmos/src/trigger.rs b/chain/cosmos/src/trigger.rs
index 5fa5d1d3c73..35880c52e62 100644
--- a/chain/cosmos/src/trigger.rs
+++ b/chain/cosmos/src/trigger.rs
@@ -1,9 +1,10 @@
 use std::{cmp::Ordering, sync::Arc};
 
-use graph::blockchain::{Block, BlockHash, MappingTrigger, TriggerData};
+use graph::blockchain::{Block, BlockHash, TriggerData};
 use graph::cheap_clone::CheapClone;
 use graph::prelude::{BlockNumber, Error};
 use graph::runtime::{asc_new, gas::GasCounter, AscHeap, AscPtr, DeterministicHostError};
+use graph_runtime_wasm::module::ToAscPtr;
 
 use crate::codec;
 use crate::data_source::EventOrigin;
@@ -34,7 +35,7 @@ impl std::fmt::Debug for CosmosTrigger {
     }
 }
 
-impl MappingTrigger for CosmosTrigger {
+impl ToAscPtr for CosmosTrigger {
     fn to_asc_ptr<H: AscHeap>(
         self,
         heap: &mut H,
diff --git a/chain/ethereum/src/chain.rs b/chain/ethereum/src/chain.rs
index dd998db7b75..c793cd4e445 100644
--- a/chain/ethereum/src/chain.rs
+++ b/chain/ethereum/src/chain.rs
@@ -390,6 +390,12 @@ pub enum BlockFinality {
     NonFinal(EthereumBlockWithCalls),
 }
 
+impl Default for BlockFinality {
+    fn default() -> Self {
+        Self::Final(Arc::default())
+    }
+}
+
 impl BlockFinality {
     pub(crate) fn light_block(&self) -> &Arc<LightEthereumBlock> {
         match self {
diff --git a/chain/ethereum/src/data_source.rs b/chain/ethereum/src/data_source.rs
index a2d100341cc..428f71b8c76 100644
--- a/chain/ethereum/src/data_source.rs
+++ b/chain/ethereum/src/data_source.rs
@@ -485,7 +485,7 @@ impl DataSource {
                     Some(handler) => handler,
                     None => return Ok(None),
                 };
-                Ok(Some(TriggerWithHandler::new(
+                Ok(Some(TriggerWithHandler::<Chain>::new(
                     MappingTrigger::Block {
                         block: block.cheap_clone(),
                     },
@@ -587,7 +587,7 @@ impl DataSource {
                     "address" => format!("{}", &log.address),
                     "transaction" => format!("{}", &transaction.hash),
                 });
-                Ok(Some(TriggerWithHandler::new_with_logging_extras(
+                Ok(Some(TriggerWithHandler::<Chain>::new_with_logging_extras(
                     MappingTrigger::Log {
                         block: block.cheap_clone(),
                         transaction: Arc::new(transaction),
@@ -696,7 +696,7 @@ impl DataSource {
                     "to" => format!("{}", &call.to),
                     "transaction" => format!("{}", &transaction.hash),
                 });
-                Ok(Some(TriggerWithHandler::new_with_logging_extras(
+                Ok(Some(TriggerWithHandler::<Chain>::new_with_logging_extras(
                     MappingTrigger::Call {
                         block: block.cheap_clone(),
                         transaction,
diff --git a/chain/ethereum/src/trigger.rs b/chain/ethereum/src/trigger.rs
index c06f251cfc0..6aaae1a65b3 100644
--- a/chain/ethereum/src/trigger.rs
+++ b/chain/ethereum/src/trigger.rs
@@ -1,4 +1,3 @@
-use graph::blockchain;
 use graph::blockchain::TriggerData;
 use graph::data::subgraph::API_VERSION_0_0_2;
 use graph::data::subgraph::API_VERSION_0_0_6;
@@ -24,6 +23,7 @@ use graph::runtime::AscHeap;
 use graph::runtime::AscPtr;
 use graph::runtime::DeterministicHostError;
 use graph::semver::Version;
+use graph_runtime_wasm::module::ToAscPtr;
 use std::convert::TryFrom;
 use std::ops::Deref;
 use std::{cmp::Ordering, sync::Arc};
@@ -111,7 +111,7 @@ impl std::fmt::Debug for MappingTrigger {
     }
 }
 
-impl blockchain::MappingTrigger for MappingTrigger {
+impl ToAscPtr for MappingTrigger {
     fn to_asc_ptr<H: AscHeap>(
         self,
         heap: &mut H,
diff --git a/chain/near/src/data_source.rs b/chain/near/src/data_source.rs
index 15d9ca84eb4..95b2ce099d3 100644
--- a/chain/near/src/data_source.rs
+++ b/chain/near/src/data_source.rs
@@ -102,7 +102,7 @@ impl blockchain::DataSource<Chain> for DataSource {
             }
         };
 
-        Ok(Some(TriggerWithHandler::new(
+        Ok(Some(TriggerWithHandler::<Chain>::new(
             trigger.cheap_clone(),
             handler.to_owned(),
         )))
diff --git a/chain/near/src/trigger.rs b/chain/near/src/trigger.rs
index ecdc8a3d2f7..049cce52bc8 100644
--- a/chain/near/src/trigger.rs
+++ b/chain/near/src/trigger.rs
@@ -1,4 +1,3 @@
-use graph::blockchain;
 use graph::blockchain::Block;
 use graph::blockchain::TriggerData;
 use graph::cheap_clone::CheapClone;
@@ -6,6 +5,7 @@ use graph::prelude::hex;
 use graph::prelude::web3::types::H256;
 use graph::prelude::BlockNumber;
 use graph::runtime::{asc_new, gas::GasCounter, AscHeap, AscPtr, DeterministicHostError};
+use graph_runtime_wasm::module::ToAscPtr;
 use std::{cmp::Ordering, sync::Arc};
 
 use crate::codec;
@@ -35,7 +35,7 @@ impl std::fmt::Debug for NearTrigger {
     }
 }
 
-impl blockchain::MappingTrigger for NearTrigger {
+impl ToAscPtr for NearTrigger {
     fn to_asc_ptr<H: AscHeap>(
         self,
         heap: &mut H,
@@ -159,8 +159,7 @@ mod tests {
         let mut heap = BytesHeap::new(API_VERSION_0_0_5);
         let trigger = NearTrigger::Block(Arc::new(block()));
 
-        let result =
-            blockchain::MappingTrigger::to_asc_ptr(trigger, &mut heap, &GasCounter::default());
+        let result = trigger.to_asc_ptr(&mut heap, &GasCounter::default());
         assert!(result.is_ok());
     }
 
@@ -173,8 +172,7 @@ mod tests {
             receipt: receipt().unwrap(),
         }));
 
-        let result =
-            blockchain::MappingTrigger::to_asc_ptr(trigger, &mut heap, &GasCounter::default());
+        let result = trigger.to_asc_ptr(&mut heap, &GasCounter::default());
         assert!(result.is_ok());
     }
 
diff --git a/chain/substreams/Cargo.toml b/chain/substreams/Cargo.toml
index 6829877cbea..bc9bdbca51a 100644
--- a/chain/substreams/Cargo.toml
+++ b/chain/substreams/Cargo.toml
@@ -13,6 +13,7 @@ futures = "0.1.21"
 http = "0.2.4"
 jsonrpc-core = "18.0.0"
 graph = { path = "../../graph" }
+graph-runtime-wasm = { path = "../../runtime/wasm" }
 lazy_static = "1.2.0"
 serde = "1.0"
 prost = "0.10.4"
diff --git a/chain/substreams/src/trigger.rs b/chain/substreams/src/trigger.rs
index 0253422da46..b2286c82fd7 100644
--- a/chain/substreams/src/trigger.rs
+++ b/chain/substreams/src/trigger.rs
@@ -8,6 +8,7 @@ use graph::{
         subgraph::{MappingError, ProofOfIndexingEvent, SharedProofOfIndexing},
     },
     data::store::scalar::Bytes,
+    data_source,
     prelude::{
         anyhow, async_trait, BigDecimal, BigInt, BlockHash, BlockNumber, BlockState, Entity,
         RuntimeHostBuilder, Value,
@@ -15,6 +16,7 @@ use graph::{
     slog::Logger,
     substreams::Modules,
 };
+use graph_runtime_wasm::module::ToAscPtr;
 use lazy_static::__Deref;
 
 use crate::{
@@ -32,7 +34,7 @@ impl blockchain::TriggerData for TriggerData {
     }
 }
 
-impl blockchain::MappingTrigger for TriggerData {
+impl ToAscPtr for TriggerData {
     // substreams doesn't rely on wasm on the graph-node so this is not needed.
     fn to_asc_ptr<H: graph::runtime::AscHeap>(
         self,
@@ -166,7 +168,7 @@ where
         logger: &Logger,
         _hosts: &[Arc<T::Host>],
         block: &Arc<Block>,
-        _trigger: &TriggerData,
+        _trigger: &data_source::TriggerData<Chain>,
         mut state: BlockState<Chain>,
         proof_of_indexing: &SharedProofOfIndexing,
         causality_region: &str,
diff --git a/core/src/subgraph/context.rs b/core/src/subgraph/context.rs
index c30047018e3..7677ef2b8b4 100644
--- a/core/src/subgraph/context.rs
+++ b/core/src/subgraph/context.rs
@@ -4,6 +4,7 @@ use crate::{
     },
     subgraph::SubgraphInstance,
 };
+use anyhow::{self, Error};
 use bytes::Bytes;
 use cid::Cid;
 use graph::{
@@ -14,7 +15,7 @@ use graph::{
     ipfs_client::IpfsClient,
     prelude::{CancelGuard, DeploymentHash, MetricsRegistry, RuntimeHostBuilder},
     slog::Logger,
-    tokio::{select, spawn, sync::mpsc},
+    tokio::sync::mpsc,
 };
 use std::collections::HashMap;
 use std::sync::{Arc, RwLock};
@@ -30,11 +31,12 @@ where
     pub instances: SharedInstanceKeepAliveMap,
     pub filter: C::TriggerFilter,
     pub offchain_monitor: OffchainMonitor,
-    pub offchain_monitor_rx: mpsc::Receiver<(offchain::Source, Bytes)>,
 }
 
 pub struct OffchainMonitor {
     ipfs_monitor: PollingMonitor<Cid>,
+    pub ipfs_monitor_rx: mpsc::Receiver<(Cid, Bytes)>,
+    pub data_sources: Vec<offchain::DataSource>,
 }
 
 impl OffchainMonitor {
@@ -43,8 +45,8 @@ impl OffchainMonitor {
         registry: Arc<dyn MetricsRegistry>,
         subgraph_hash: &DeploymentHash,
         client: IpfsClient,
-    ) -> (Self, mpsc::Receiver<(offchain::Source, Bytes)>) {
-        let (ipfs_monitor_tx, mut ipfs_monitor_rx) = mpsc::channel(10);
+    ) -> Self {
+        let (ipfs_monitor_tx, ipfs_monitor_rx) = mpsc::channel(10);
         let ipfs_service = IpfsService::new(
             client,
             ENV_VARS.mappings.max_ipfs_file_bytes.unwrap_or(1 << 20) as u64,
@@ -57,31 +59,22 @@ impl OffchainMonitor {
             logger,
             PollingMonitorMetrics::new(registry, subgraph_hash),
         );
-        let (tx, rx) = mpsc::channel(10);
-        spawn(async move {
-            loop {
-                select! {
-                    msg = ipfs_monitor_rx.recv() => {
-                        match msg {
-                            Some((cid, bytes)) => {
-                                if let Err(_) = tx.send((offchain::Source::Ipfs(cid), bytes)).await {
-                                    break;
-                                }
-                            },
-                            None => break,
-                        }
-                    }
-                }
-            }
-        });
-        (Self { ipfs_monitor }, rx)
+        Self {
+            ipfs_monitor,
+            ipfs_monitor_rx,
+            data_sources: Vec::new(),
+        }
     }
 
-    pub fn monitor(&self, source: offchain::Source) {
+    pub fn add_data_source(&mut self, ds: offchain::DataSource) -> Result<(), Error> {
+        let source = match &ds.source {
+            Some(source) => source,
+            None => anyhow::bail!("Failed to add offchain data source (missing source)"),
+        };
         match source {
-            offchain::Source::Ipfs(cid) => {
-                self.ipfs_monitor.monitor(cid);
-            }
-        }
+            offchain::Source::Ipfs(cid) => self.ipfs_monitor.monitor(cid.clone()),
+        };
+        self.data_sources.push(ds);
+        Ok(())
     }
 }
diff --git a/core/src/subgraph/instance.rs b/core/src/subgraph/instance.rs
index c09fc78f544..8d43e794194 100644
--- a/core/src/subgraph/instance.rs
+++ b/core/src/subgraph/instance.rs
@@ -6,7 +6,7 @@ use graph::{
         subgraph::{MappingError, ProofOfIndexingVersion, SharedProofOfIndexing},
     },
     data::subgraph::SPEC_VERSION_0_0_6,
-    data_source::{DataSource, DataSourceTemplate},
+    data_source::{DataSource, DataSourceTemplate, TriggerData},
     prelude::*,
 };
 use std::collections::HashMap;
@@ -42,7 +42,7 @@ where
         host_builder: T,
         trigger_processor: Box<dyn TriggerProcessor<C, T>>,
         host_metrics: Arc<HostMetrics>,
-        offchain_monitor: &OffchainMonitor,
+        offchain_monitor: &mut OffchainMonitor,
     ) -> Result<Self, Error> {
         let subgraph_id = manifest.id.clone();
         let network = manifest.network_name();
@@ -76,8 +76,8 @@ where
 
             // Create services for static offchain data sources
             if let DataSource::Offchain(ds) = &ds {
-                if let Some(source) = &ds.source {
-                    offchain_monitor.monitor(source.clone());
+                if ds.source.is_some() {
+                    offchain_monitor.add_data_source(ds.clone())?;
                 }
             }
 
@@ -133,7 +133,7 @@ where
         &self,
         logger: &Logger,
         block: &Arc<C::Block>,
-        trigger: &C::TriggerData,
+        trigger: &TriggerData<C>,
         state: BlockState<C>,
         proof_of_indexing: &SharedProofOfIndexing,
         causality_region: &str,
diff --git a/core/src/subgraph/instance_manager.rs b/core/src/subgraph/instance_manager.rs
index 8ae0c22e3e2..c031ed4d3a9 100644
--- a/core/src/subgraph/instance_manager.rs
+++ b/core/src/subgraph/instance_manager.rs
@@ -10,6 +10,7 @@ use graph::blockchain::{BlockchainKind, TriggerFilter};
 use graph::ipfs_client::IpfsClient;
 use graph::prelude::{SubgraphInstanceManager as SubgraphInstanceManagerTrait, *};
 use graph::{blockchain::BlockchainMap, components::store::DeploymentLocator};
+use graph_runtime_wasm::module::ToAscPtr;
 use graph_runtime_wasm::RuntimeHostBuilder;
 use tokio::task;
 
@@ -162,7 +163,10 @@ impl<S: SubgraphStore> SubgraphInstanceManager<S> {
         manifest: serde_yaml::Mapping,
         stop_block: Option<BlockNumber>,
         tp: Box<dyn TriggerProcessor<C, RuntimeHostBuilder<C>>>,
-    ) -> Result<(), Error> {
+    ) -> Result<(), Error>
+    where
+        <C as Blockchain>::MappingTrigger: ToAscPtr,
+    {
         let subgraph_store = self.subgraph_store.cheap_clone();
         let registry = self.metrics_registry.cheap_clone();
         let store = self
@@ -295,7 +299,7 @@ impl<S: SubgraphStore> SubgraphInstanceManager<S> {
             stopwatch_metrics,
         ));
 
-        let (offchain_monitor, offchain_monitor_rx) = OffchainMonitor::new(
+        let mut offchain_monitor = OffchainMonitor::new(
             logger.cheap_clone(),
             registry.cheap_clone(),
             &manifest.id,
@@ -321,7 +325,7 @@ impl<S: SubgraphStore> SubgraphInstanceManager<S> {
             host_builder,
             tp,
             host_metrics.clone(),
-            &offchain_monitor,
+            &mut offchain_monitor,
         )?;
 
         let inputs = IndexingInputs {
@@ -345,7 +349,6 @@ impl<S: SubgraphStore> SubgraphInstanceManager<S> {
             instances: self.instances.cheap_clone(),
             filter,
             offchain_monitor,
-            offchain_monitor_rx,
         };
 
         let metrics = RunnerMetrics {
diff --git a/core/src/subgraph/runner.rs b/core/src/subgraph/runner.rs
index 27b195ecd2e..1434d9b74c3 100644
--- a/core/src/subgraph/runner.rs
+++ b/core/src/subgraph/runner.rs
@@ -16,7 +16,7 @@ use graph::data::subgraph::{
     schema::{SubgraphError, SubgraphHealth, POI_OBJECT},
     SubgraphFeature,
 };
-use graph::data_source::DataSource;
+use graph::data_source::{self, offchain, DataSource, TriggerData};
 use graph::prelude::*;
 use graph::util::{backoff::ExponentialBackoff, lfu_cache::LfuCache};
 use std::convert::TryFrom;
@@ -113,6 +113,8 @@ where
 
             // Process events from the stream as long as no restart is needed
             loop {
+                self.handle_offchain_queue().await?;
+
                 let event = {
                     let _section = self.metrics.stream.stopwatch.start_section("scan_blocks");
 
@@ -221,9 +223,18 @@ 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(|ds| ds.as_onchain()),
-            );
+            let mut onchain_data_sources = Vec::new();
+            for ds in data_sources {
+                match ds {
+                    DataSource::Onchain(ds) => onchain_data_sources.push(ds),
+                    DataSource::Offchain(ds) => {
+                        self.ctx.offchain_monitor.add_data_source(ds.clone())?
+                    }
+                }
+            }
+            let data_sources = onchain_data_sources;
+
+            let filter = C::TriggerFilter::from_data_sources(data_sources.iter());
 
             // Reprocess the triggers from this block that match the new data sources
             let block_with_triggers = self
@@ -249,7 +260,10 @@ where
 
             // Add entity operations for the new data sources to the block state
             // and add runtimes for the data sources to the subgraph instance.
-            self.persist_dynamic_data_sources(&mut block_state.entity_cache, data_sources);
+            self.persist_dynamic_data_sources(
+                &mut block_state.entity_cache,
+                data_sources.into_iter().map(DataSource::Onchain).collect(),
+            );
 
             // Process the triggers in each host in the same order the
             // corresponding data sources have been created.
@@ -262,7 +276,7 @@ where
                         &logger,
                         &runtime_hosts,
                         &block,
-                        &trigger,
+                        &TriggerData::Onchain(trigger),
                         block_state,
                         &proof_of_indexing,
                         &causality_region,
@@ -426,9 +440,8 @@ where
             std::mem::take(&mut self.state.entity_lfu_cache),
         );
 
-        use graph::blockchain::TriggerData;
-
         for trigger in triggers {
+            let trigger = TriggerData::Onchain(trigger);
             block_state = self
                 .ctx
                 .instance
@@ -555,6 +568,81 @@ where
 
         Ok(action)
     }
+
+    async fn handle_offchain_queue(&mut self) -> Result<(), Error> {
+        loop {
+            if let Ok((cid, data)) = self.ctx.offchain_monitor.ipfs_monitor_rx.try_recv() {
+                self.handle_offchain_update(offchain::Source::Ipfs(cid), data)
+                    .await?;
+                continue;
+            }
+            break;
+        }
+        Ok(())
+    }
+
+    async fn handle_offchain_update(
+        &mut self,
+        source: offchain::Source,
+        data: bytes::Bytes,
+    ) -> Result<(), Error> {
+        let matching_data_sources = self
+            .ctx
+            .offchain_monitor
+            .data_sources
+            .iter()
+            .filter(|ds| ds.source.as_ref() == Some(&source));
+
+        let mut block_state =
+            BlockState::<C>::new(self.inputs.store.cheap_clone(), LfuCache::new());
+        let entity_cache = std::mem::take(&mut self.state.entity_lfu_cache);
+
+        self.persist_dynamic_data_sources(
+            &mut block_state.entity_cache,
+            matching_data_sources
+                .cloned()
+                .map(data_source::DataSource::Offchain)
+                .collect(),
+        );
+
+        let causality_region = match &source {
+            offchain::Source::Ipfs(cid) => format!("ipfs/{}", cid.to_string()),
+        };
+        let trigger = TriggerData::Offchain(offchain::TriggerData {
+            source: source.clone(),
+            data: Arc::new(data),
+        });
+
+        let block_state = self
+            .ctx
+            .instance
+            .process_trigger(
+                &self.logger,
+                &Arc::default(),
+                &trigger,
+                block_state,
+                &None,
+                &causality_region,
+                &self.inputs.debug_fork,
+                &self.metrics.subgraph,
+            )
+            .await
+            .map_err(move |err| {
+                let err = match err {
+                    MappingError::PossibleReorg(_) => unreachable!(),
+                    MappingError::Unknown(err) => err,
+                };
+                err.context("failed to process trigger".to_string())
+            })?;
+
+        anyhow::ensure!(
+            !block_state.has_created_data_sources(),
+            "Attempted to create data source in offchain data source handler. This is not yet supported.",
+        );
+
+        self.state.entity_lfu_cache = entity_cache;
+        Ok(())
+    }
 }
 
 #[derive(Debug)]
diff --git a/core/src/subgraph/trigger_processor.rs b/core/src/subgraph/trigger_processor.rs
index ebab54afc8e..7358a45402b 100644
--- a/core/src/subgraph/trigger_processor.rs
+++ b/core/src/subgraph/trigger_processor.rs
@@ -1,15 +1,15 @@
-use std::sync::Arc;
-
 use async_trait::async_trait;
 use graph::blockchain::{Block, Blockchain};
 use graph::cheap_clone::CheapClone;
 use graph::components::store::SubgraphFork;
 use graph::components::subgraph::{MappingError, SharedProofOfIndexing};
+use graph::data_source::TriggerData;
 use graph::prelude::tokio::time::Instant;
 use graph::prelude::{
     BlockState, RuntimeHost, RuntimeHostBuilder, SubgraphInstanceMetrics, TriggerProcessor,
 };
 use graph::slog::Logger;
+use std::sync::Arc;
 
 pub struct SubgraphTriggerProcessor {}
 
@@ -24,7 +24,7 @@ where
         logger: &Logger,
         hosts: &[Arc<T::Host>],
         block: &Arc<C::Block>,
-        trigger: &C::TriggerData,
+        trigger: &TriggerData<C>,
         mut state: BlockState<C>,
         proof_of_indexing: &SharedProofOfIndexing,
         causality_region: &str,
diff --git a/graph/src/blockchain/mock.rs b/graph/src/blockchain/mock.rs
index d519a29078d..ba638d82e71 100644
--- a/graph/src/blockchain/mock.rs
+++ b/graph/src/blockchain/mock.rs
@@ -1,7 +1,6 @@
 use crate::{
     components::{link_resolver::LinkResolver, store::BlockNumber},
     prelude::DataSourceTemplateInfo,
-    runtime::gas::GasCounter,
 };
 use anyhow::Error;
 use async_trait::async_trait;
@@ -16,14 +15,14 @@ use super::{
 
 use super::{
     block_stream::BlockWithTriggers, Block, BlockPtr, Blockchain, BlockchainKind, DataSource,
-    DataSourceTemplate, MappingTrigger, NodeCapabilities, RuntimeAdapter, TriggerData,
-    TriggerFilter, TriggersAdapter, UnresolvedDataSource, UnresolvedDataSourceTemplate,
+    DataSourceTemplate, NodeCapabilities, RuntimeAdapter, TriggerData, TriggerFilter,
+    TriggersAdapter, UnresolvedDataSource, UnresolvedDataSourceTemplate,
 };
 
 #[derive(Debug)]
 pub struct MockBlockchain;
 
-#[derive(Clone, Hash, Eq, PartialEq, Debug)]
+#[derive(Clone, Hash, Eq, PartialEq, Debug, Default)]
 pub struct MockBlock {
     pub number: u64,
 }
@@ -222,16 +221,6 @@ impl TriggerData for MockTriggerData {
 #[derive(Debug)]
 pub struct MockMappingTrigger {}
 
-impl MappingTrigger for MockMappingTrigger {
-    fn to_asc_ptr<H: crate::runtime::AscHeap>(
-        self,
-        _heap: &mut H,
-        _gas: &GasCounter,
-    ) -> Result<crate::runtime::AscPtr<()>, crate::runtime::DeterministicHostError> {
-        todo!()
-    }
-}
-
 #[derive(Clone, Default)]
 pub struct MockTriggerFilter;
 
diff --git a/graph/src/blockchain/mod.rs b/graph/src/blockchain/mod.rs
index 06ab4458da3..88df6170843 100644
--- a/graph/src/blockchain/mod.rs
+++ b/graph/src/blockchain/mod.rs
@@ -15,8 +15,9 @@ use crate::{
     cheap_clone::CheapClone,
     components::store::{DeploymentLocator, StoredDynamicDataSource},
     data::subgraph::UnifiedMappingApiVersion,
+    data_source,
     prelude::DataSourceContext,
-    runtime::{gas::GasCounter, AscHeap, AscPtr, DeterministicHostError, HostExportError},
+    runtime::{gas::GasCounter, AscHeap, HostExportError},
 };
 use crate::{
     components::{
@@ -30,7 +31,6 @@ use async_trait::async_trait;
 use serde::de::DeserializeOwned;
 use serde::{Deserialize, Serialize};
 use slog::Logger;
-use slog::{self, SendSyncRefUnwindSafeKV};
 use std::{
     any::Any,
     collections::HashMap,
@@ -85,7 +85,7 @@ pub trait Blockchain: Debug + Sized + Send + Sync + Unpin + 'static {
 
     // The `Clone` bound is used when reprocessing a block, because `triggers_in_block` requires an
     // owned `Block`. It would be good to come up with a way to remove this bound.
-    type Block: Block + Clone + Debug;
+    type Block: Block + Clone + Debug + Default;
     type DataSource: DataSource<Self>;
     type UnresolvedDataSource: UnresolvedDataSource<Self>;
 
@@ -97,7 +97,7 @@ pub trait Blockchain: Debug + Sized + Send + Sync + Unpin + 'static {
 
     /// Decoded trigger ready to be processed by the mapping.
     /// New implementations should have this be the same as `TriggerData`.
-    type MappingTrigger: MappingTrigger + Debug;
+    type MappingTrigger: Send + Sync + Debug;
 
     /// Trigger filter used as input to the triggers adapter.
     type TriggerFilter: TriggerFilter<Self>;
@@ -271,16 +271,6 @@ pub trait TriggerData {
     fn error_context(&self) -> String;
 }
 
-pub trait MappingTrigger: Send + Sync {
-    /// A flexible interface for writing a type to AS memory, any pointer can be returned.
-    /// Use `AscPtr::erased` to convert `AscPtr<T>` into `AscPtr<()>`.
-    fn to_asc_ptr<H: AscHeap>(
-        self,
-        heap: &mut H,
-        gas: &GasCounter,
-    ) -> Result<AscPtr<()>, DeterministicHostError>;
-}
-
 pub struct HostFnCtx<'a> {
     pub logger: Logger,
     pub block_ptr: BlockPtr,
@@ -403,56 +393,4 @@ impl BlockchainMap {
     }
 }
 
-pub struct TriggerWithHandler<C: Blockchain> {
-    trigger: C::MappingTrigger,
-    handler: String,
-    logging_extras: Arc<dyn SendSyncRefUnwindSafeKV>,
-}
-
-impl<C: Blockchain> fmt::Debug for TriggerWithHandler<C> {
-    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
-        let mut builder = f.debug_struct("TriggerWithHandler");
-        builder.field("trigger", &self.trigger);
-        builder.field("handler", &self.handler);
-        builder.finish()
-    }
-}
-
-impl<C: Blockchain> TriggerWithHandler<C> {
-    pub fn new(trigger: C::MappingTrigger, handler: String) -> Self {
-        TriggerWithHandler {
-            trigger,
-            handler,
-            logging_extras: Arc::new(slog::o! {}),
-        }
-    }
-
-    pub fn new_with_logging_extras(
-        trigger: C::MappingTrigger,
-        handler: String,
-        logging_extras: Arc<dyn SendSyncRefUnwindSafeKV>,
-    ) -> Self {
-        TriggerWithHandler {
-            trigger,
-            handler,
-            logging_extras,
-        }
-    }
-
-    /// Additional key-value pairs to be logged with the "Done processing trigger" message.
-    pub fn logging_extras(&self) -> Arc<dyn SendSyncRefUnwindSafeKV> {
-        self.logging_extras.cheap_clone()
-    }
-
-    pub fn handler_name(&self) -> &str {
-        &self.handler
-    }
-
-    pub fn to_asc_ptr<H: AscHeap>(
-        self,
-        heap: &mut H,
-        gas: &GasCounter,
-    ) -> Result<AscPtr<()>, DeterministicHostError> {
-        self.trigger.to_asc_ptr(heap, gas)
-    }
-}
+pub type TriggerWithHandler<C> = data_source::TriggerWithHandler<<C as Blockchain>::MappingTrigger>;
diff --git a/graph/src/components/subgraph/host.rs b/graph/src/components/subgraph/host.rs
index 5ba795b4673..fed8d99efbe 100644
--- a/graph/src/components/subgraph/host.rs
+++ b/graph/src/components/subgraph/host.rs
@@ -6,9 +6,10 @@ use anyhow::Error;
 use async_trait::async_trait;
 use futures::sync::mpsc;
 
-use crate::blockchain::TriggerWithHandler;
 use crate::components::store::SubgraphFork;
-use crate::data_source::{DataSource, DataSourceTemplate};
+use crate::data_source::{
+    DataSource, DataSourceTemplate, MappingTrigger, TriggerData, TriggerWithHandler,
+};
 use crate::prelude::*;
 use crate::{blockchain::Blockchain, components::subgraph::SharedProofOfIndexing};
 use crate::{components::metrics::HistogramVec, runtime::DeterministicHostError};
@@ -47,16 +48,16 @@ impl MappingError {
 pub trait RuntimeHost<C: Blockchain>: Send + Sync + 'static {
     fn match_and_decode(
         &self,
-        trigger: &C::TriggerData,
+        trigger: &TriggerData<C>,
         block: &Arc<C::Block>,
         logger: &Logger,
-    ) -> Result<Option<TriggerWithHandler<C>>, Error>;
+    ) -> Result<Option<TriggerWithHandler<MappingTrigger<C>>>, Error>;
 
     async fn process_mapping_trigger(
         &self,
         logger: &Logger,
         block_ptr: BlockPtr,
-        trigger: TriggerWithHandler<C>,
+        trigger: TriggerWithHandler<MappingTrigger<C>>,
         state: BlockState<C>,
         proof_of_indexing: SharedProofOfIndexing,
         debug_fork: &Option<Arc<dyn SubgraphFork>>,
diff --git a/graph/src/components/trigger_processor.rs b/graph/src/components/trigger_processor.rs
index 55997c6fa1e..ce02a212a6c 100644
--- a/graph/src/components/trigger_processor.rs
+++ b/graph/src/components/trigger_processor.rs
@@ -3,7 +3,7 @@ use std::sync::Arc;
 use async_trait::async_trait;
 use slog::Logger;
 
-use crate::{blockchain::Blockchain, prelude::SubgraphInstanceMetrics};
+use crate::{blockchain::Blockchain, data_source::TriggerData, prelude::SubgraphInstanceMetrics};
 
 use super::{
     store::SubgraphFork,
@@ -21,7 +21,7 @@ where
         logger: &Logger,
         hosts: &[Arc<T::Host>],
         block: &Arc<C::Block>,
-        trigger: &C::TriggerData,
+        trigger: &TriggerData<C>,
         mut state: BlockState<C>,
         proof_of_indexing: &SharedProofOfIndexing,
         causality_region: &str,
diff --git a/graph/src/data_source/mod.rs b/graph/src/data_source/mod.rs
index 56231b5060d..9a62d3b34a5 100644
--- a/graph/src/data_source/mod.rs
+++ b/graph/src/data_source/mod.rs
@@ -2,8 +2,8 @@ pub mod offchain;
 
 use crate::{
     blockchain::{
-        Blockchain, DataSource as _, DataSourceTemplate as _, UnresolvedDataSource as _,
-        UnresolvedDataSourceTemplate as _,
+        Blockchain, DataSource as _, DataSourceTemplate as _, TriggerData as _,
+        UnresolvedDataSource as _, UnresolvedDataSourceTemplate as _,
     },
     components::{
         link_resolver::LinkResolver,
@@ -16,8 +16,8 @@ use crate::{
 use anyhow::Error;
 use semver::Version;
 use serde::{de::IntoDeserializer as _, Deserialize, Deserializer};
-use slog::Logger;
-use std::{collections::BTreeMap, sync::Arc};
+use slog::{Logger, SendSyncRefUnwindSafeKV};
+use std::{collections::BTreeMap, fmt, sync::Arc};
 
 #[derive(Debug)]
 pub enum DataSource<C: Blockchain> {
@@ -97,6 +97,23 @@ impl<C: Blockchain> DataSource<C> {
         }
     }
 
+    pub fn match_and_decode(
+        &self,
+        trigger: &TriggerData<C>,
+        block: &Arc<C::Block>,
+        logger: &Logger,
+    ) -> Result<Option<TriggerWithHandler<MappingTrigger<C>>>, Error> {
+        match self {
+            Self::Onchain(ds) => ds
+                .match_and_decode(trigger.as_onchain().unwrap().clone(), block, logger)
+                .map(|t| t.map(|t| t.map(MappingTrigger::Onchain))),
+            Self::Offchain(ds) => Ok(Some(TriggerWithHandler::new(
+                MappingTrigger::Offchain(trigger.as_offchain().unwrap().clone()),
+                ds.mapping.handler.clone(),
+            ))),
+        }
+    }
+
     pub fn is_duplicate_of(&self, other: &Self) -> bool {
         match (self, other) {
             (Self::Onchain(a), Self::Onchain(b)) => a.is_duplicate_of(b),
@@ -246,6 +263,94 @@ impl<C: Blockchain> UnresolvedDataSourceTemplate<C> {
     }
 }
 
+pub struct TriggerWithHandler<T> {
+    pub trigger: T,
+    handler: String,
+    logging_extras: Arc<dyn SendSyncRefUnwindSafeKV>,
+}
+
+impl<T: fmt::Debug> fmt::Debug for TriggerWithHandler<T> {
+    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
+        let mut builder = f.debug_struct("TriggerWithHandler");
+        builder.field("trigger", &self.trigger);
+        builder.field("handler", &self.handler);
+        builder.finish()
+    }
+}
+
+impl<T> TriggerWithHandler<T> {
+    pub fn new(trigger: T, handler: String) -> Self {
+        Self {
+            trigger,
+            handler,
+            logging_extras: Arc::new(slog::o! {}),
+        }
+    }
+
+    pub fn new_with_logging_extras(
+        trigger: T,
+        handler: String,
+        logging_extras: Arc<dyn SendSyncRefUnwindSafeKV>,
+    ) -> Self {
+        TriggerWithHandler {
+            trigger,
+            handler,
+            logging_extras,
+        }
+    }
+
+    /// Additional key-value pairs to be logged with the "Done processing trigger" message.
+    pub fn logging_extras(&self) -> Arc<dyn SendSyncRefUnwindSafeKV> {
+        self.logging_extras.cheap_clone()
+    }
+
+    pub fn handler_name(&self) -> &str {
+        &self.handler
+    }
+
+    fn map<T_>(self, f: impl FnOnce(T) -> T_) -> TriggerWithHandler<T_> {
+        TriggerWithHandler {
+            trigger: f(self.trigger),
+            handler: self.handler,
+            logging_extras: self.logging_extras,
+        }
+    }
+}
+
+pub enum TriggerData<C: Blockchain> {
+    Onchain(C::TriggerData),
+    Offchain(offchain::TriggerData),
+}
+
+impl<C: Blockchain> TriggerData<C> {
+    fn as_onchain(&self) -> Option<&C::TriggerData> {
+        match self {
+            Self::Onchain(trigger) => Some(trigger),
+            Self::Offchain(_) => None,
+        }
+    }
+
+    fn as_offchain(&self) -> Option<&offchain::TriggerData> {
+        match self {
+            Self::Onchain(_) => None,
+            Self::Offchain(trigger) => Some(trigger),
+        }
+    }
+
+    pub fn error_context(&self) -> String {
+        match self {
+            Self::Onchain(trigger) => trigger.error_context(),
+            Self::Offchain(trigger) => format!("{:?}", trigger.source),
+        }
+    }
+}
+
+#[derive(Debug)]
+pub enum MappingTrigger<C: Blockchain> {
+    Onchain(C::MappingTrigger),
+    Offchain(offchain::TriggerData),
+}
+
 macro_rules! clone_data_source {
     ($t:ident) => {
         impl<C: Blockchain> Clone for $t<C> {
diff --git a/graph/src/data_source/offchain.rs b/graph/src/data_source/offchain.rs
index d2d6fb79e61..ef2a814eb27 100644
--- a/graph/src/data_source/offchain.rs
+++ b/graph/src/data_source/offchain.rs
@@ -13,7 +13,7 @@ use anyhow::{self, Error};
 use cid::Cid;
 use serde::Deserialize;
 use slog::{info, Logger};
-use std::sync::Arc;
+use std::{fmt, sync::Arc};
 
 pub const OFFCHAIN_KINDS: &'static [&'static str] = &["file/ipfs"];
 
@@ -216,3 +216,25 @@ impl UnresolvedDataSourceTemplate {
         })
     }
 }
+
+#[derive(Clone)]
+pub struct TriggerData {
+    pub source: Source,
+    pub data: Arc<bytes::Bytes>,
+}
+
+impl fmt::Debug for TriggerData {
+    fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
+        #[derive(Debug)]
+        struct TriggerDataWithoutData<'a> {
+            _source: &'a Source,
+        }
+        write!(
+            f,
+            "{:?}",
+            TriggerDataWithoutData {
+                _source: &self.source
+            }
+        )
+    }
+}
diff --git a/runtime/wasm/src/host.rs b/runtime/wasm/src/host.rs
index c540b6cc4e2..8e26f98e6db 100644
--- a/runtime/wasm/src/host.rs
+++ b/runtime/wasm/src/host.rs
@@ -5,15 +5,18 @@ use async_trait::async_trait;
 use futures::sync::mpsc::Sender;
 use futures03::channel::oneshot::channel;
 
-use graph::blockchain::{Blockchain, DataSource as _, HostFn, RuntimeAdapter, TriggerWithHandler};
+use graph::blockchain::{Blockchain, HostFn, RuntimeAdapter};
 use graph::components::store::{EnsLookup, SubgraphFork};
 use graph::components::subgraph::{MappingError, SharedProofOfIndexing};
-use graph::data_source::{DataSource, DataSourceTemplate};
+use graph::data_source::{
+    DataSource, DataSourceTemplate, MappingTrigger, TriggerData, TriggerWithHandler,
+};
 use graph::prelude::{
     RuntimeHost as RuntimeHostTrait, RuntimeHostBuilder as RuntimeHostBuilderTrait, *,
 };
 
 use crate::mapping::{MappingContext, MappingRequest};
+use crate::module::ToAscPtr;
 use crate::{host_exports::HostExports, module::ExperimentalFeatures};
 use graph::runtime::gas::Gas;
 
@@ -47,7 +50,10 @@ impl<C: Blockchain> RuntimeHostBuilder<C> {
     }
 }
 
-impl<C: Blockchain> RuntimeHostBuilderTrait<C> for RuntimeHostBuilder<C> {
+impl<C: Blockchain> RuntimeHostBuilderTrait<C> for RuntimeHostBuilder<C>
+where
+    <C as Blockchain>::MappingTrigger: ToAscPtr,
+{
     type Host = RuntimeHost<C>;
     type Req = MappingRequest<C>;
 
@@ -149,7 +155,7 @@ where
         &self,
         logger: &Logger,
         state: BlockState<C>,
-        trigger: TriggerWithHandler<C>,
+        trigger: TriggerWithHandler<MappingTrigger<C>>,
         block_ptr: BlockPtr,
         proof_of_indexing: SharedProofOfIndexing,
         debug_fork: &Option<Arc<dyn SubgraphFork>>,
@@ -214,21 +220,18 @@ where
 impl<C: Blockchain> RuntimeHostTrait<C> for RuntimeHost<C> {
     fn match_and_decode(
         &self,
-        trigger: &C::TriggerData,
+        trigger: &TriggerData<C>,
         block: &Arc<C::Block>,
         logger: &Logger,
-    ) -> Result<Option<TriggerWithHandler<C>>, Error> {
-        self.data_source
-            .as_onchain()
-            .map(|ds| ds.match_and_decode(trigger, block, logger))
-            .unwrap_or(Ok(None))
+    ) -> Result<Option<TriggerWithHandler<MappingTrigger<C>>>, Error> {
+        self.data_source.match_and_decode(trigger, block, logger)
     }
 
     async fn process_mapping_trigger(
         &self,
         logger: &Logger,
         block_ptr: BlockPtr,
-        trigger: TriggerWithHandler<C>,
+        trigger: TriggerWithHandler<MappingTrigger<C>>,
         state: BlockState<C>,
         proof_of_indexing: SharedProofOfIndexing,
         debug_fork: &Option<Arc<dyn SubgraphFork>>,
diff --git a/runtime/wasm/src/mapping.rs b/runtime/wasm/src/mapping.rs
index 580fb48d35e..cd1198bbbdf 100644
--- a/runtime/wasm/src/mapping.rs
+++ b/runtime/wasm/src/mapping.rs
@@ -1,10 +1,11 @@
 use crate::gas_rules::GasRules;
-use crate::module::{ExperimentalFeatures, WasmInstance};
+use crate::module::{ExperimentalFeatures, ToAscPtr, WasmInstance};
 use futures::sync::mpsc;
 use futures03::channel::oneshot::Sender;
-use graph::blockchain::{Blockchain, HostFn, TriggerWithHandler};
+use graph::blockchain::{Blockchain, HostFn};
 use graph::components::store::SubgraphFork;
 use graph::components::subgraph::{MappingError, SharedProofOfIndexing};
+use graph::data_source::{MappingTrigger, TriggerWithHandler};
 use graph::prelude::*;
 use graph::runtime::gas::Gas;
 use std::collections::BTreeMap;
@@ -20,7 +21,10 @@ pub fn spawn_module<C: Blockchain>(
     runtime: tokio::runtime::Handle,
     timeout: Option<Duration>,
     experimental_features: ExperimentalFeatures,
-) -> Result<mpsc::Sender<MappingRequest<C>>, anyhow::Error> {
+) -> Result<mpsc::Sender<MappingRequest<C>>, anyhow::Error>
+where
+    <C as Blockchain>::MappingTrigger: ToAscPtr,
+{
     let valid_module = Arc::new(ValidModule::new(&logger, raw_module)?);
 
     // Create channel for event handling requests
@@ -77,11 +81,14 @@ pub fn spawn_module<C: Blockchain>(
 fn instantiate_module_and_handle_trigger<C: Blockchain>(
     valid_module: Arc<ValidModule>,
     ctx: MappingContext<C>,
-    trigger: TriggerWithHandler<C>,
+    trigger: TriggerWithHandler<MappingTrigger<C>>,
     host_metrics: Arc<HostMetrics>,
     timeout: Option<Duration>,
     experimental_features: ExperimentalFeatures,
-) -> Result<(BlockState<C>, Gas), MappingError> {
+) -> Result<(BlockState<C>, Gas), MappingError>
+where
+    <C as Blockchain>::MappingTrigger: ToAscPtr,
+{
     let logger = ctx.logger.cheap_clone();
 
     // Start the WASM module runtime.
@@ -104,7 +111,7 @@ fn instantiate_module_and_handle_trigger<C: Blockchain>(
 
 pub struct MappingRequest<C: Blockchain> {
     pub(crate) ctx: MappingContext<C>,
-    pub(crate) trigger: TriggerWithHandler<C>,
+    pub(crate) trigger: TriggerWithHandler<MappingTrigger<C>>,
     pub(crate) result_sender: Sender<Result<(BlockState<C>, Gas), MappingError>>,
 }
 
diff --git a/runtime/wasm/src/module/mod.rs b/runtime/wasm/src/module/mod.rs
index 46a3ef11855..327b00e5e85 100644
--- a/runtime/wasm/src/module/mod.rs
+++ b/runtime/wasm/src/module/mod.rs
@@ -13,9 +13,10 @@ use never::Never;
 use semver::Version;
 use wasmtime::{Memory, Trap};
 
-use graph::blockchain::{Blockchain, HostFnCtx, TriggerWithHandler};
+use graph::blockchain::{Blockchain, HostFnCtx};
 use graph::data::store;
 use graph::data::subgraph::schema::SubgraphError;
+use graph::data_source::{offchain, MappingTrigger, TriggerWithHandler};
 use graph::prelude::*;
 use graph::runtime::{
     asc_get, asc_new,
@@ -46,6 +47,52 @@ pub trait IntoTrap {
     fn into_trap(self) -> Trap;
 }
 
+/// A flexible interface for writing a type to AS memory, any pointer can be returned.
+/// Use `AscPtr::erased` to convert `AscPtr<T>` into `AscPtr<()>`.
+pub trait ToAscPtr {
+    fn to_asc_ptr<H: AscHeap>(
+        self,
+        heap: &mut H,
+        gas: &GasCounter,
+    ) -> Result<AscPtr<()>, DeterministicHostError>;
+}
+
+impl ToAscPtr for offchain::TriggerData {
+    fn to_asc_ptr<H: AscHeap>(
+        self,
+        heap: &mut H,
+        gas: &GasCounter,
+    ) -> Result<AscPtr<()>, DeterministicHostError> {
+        asc_new(heap, self.data.as_ref() as &[u8], gas).map(|ptr| ptr.erase())
+    }
+}
+
+impl<C: Blockchain> ToAscPtr for MappingTrigger<C>
+where
+    C::MappingTrigger: ToAscPtr,
+{
+    fn to_asc_ptr<H: AscHeap>(
+        self,
+        heap: &mut H,
+        gas: &GasCounter,
+    ) -> Result<AscPtr<()>, DeterministicHostError> {
+        match self {
+            MappingTrigger::Onchain(trigger) => trigger.to_asc_ptr(heap, gas),
+            MappingTrigger::Offchain(trigger) => trigger.to_asc_ptr(heap, gas),
+        }
+    }
+}
+
+impl<T: ToAscPtr> ToAscPtr for TriggerWithHandler<T> {
+    fn to_asc_ptr<H: AscHeap>(
+        self,
+        heap: &mut H,
+        gas: &GasCounter,
+    ) -> Result<AscPtr<()>, DeterministicHostError> {
+        self.trigger.to_asc_ptr(heap, gas)
+    }
+}
+
 /// Handle to a WASM instance, which is terminated if and only if this is dropped.
 pub struct WasmInstance<C: Blockchain> {
     pub instance: wasmtime::Instance,
@@ -117,8 +164,11 @@ impl<C: Blockchain> WasmInstance<C> {
 
     pub(crate) fn handle_trigger(
         mut self,
-        trigger: TriggerWithHandler<C>,
-    ) -> Result<(BlockState<C>, Gas), MappingError> {
+        trigger: TriggerWithHandler<MappingTrigger<C>>,
+    ) -> Result<(BlockState<C>, Gas), MappingError>
+    where
+        <C as Blockchain>::MappingTrigger: ToAscPtr,
+    {
         let handler_name = trigger.handler_name().to_owned();
         let gas = self.gas.clone();
         let asc_trigger = trigger.to_asc_ptr(self.instance_ctx_mut().deref_mut(), &gas)?;

From c6338c6086c8285b925cad650f159cbd0a28b061 Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Wed, 10 Aug 2022 18:01:49 +0100
Subject: [PATCH 05/33] offchain: dont expect `manifest_idx` in the manifest

---
 graph/src/data_source/offchain.rs | 18 ++++++++++++------
 1 file changed, 12 insertions(+), 6 deletions(-)

diff --git a/graph/src/data_source/offchain.rs b/graph/src/data_source/offchain.rs
index ef2a814eb27..4d1c4e07167 100644
--- a/graph/src/data_source/offchain.rs
+++ b/graph/src/data_source/offchain.rs
@@ -186,17 +186,23 @@ impl UnresolvedMapping {
     }
 }
 
-#[derive(Clone, Debug, Hash, Eq, PartialEq, Deserialize)]
-pub struct BaseDataSourceTemplate<M> {
+#[derive(Clone, Debug, Deserialize)]
+pub struct UnresolvedDataSourceTemplate {
     pub kind: String,
     pub network: Option<String>,
     pub name: String,
-    pub manifest_idx: u32,
-    pub mapping: M,
+    pub mapping: UnresolvedMapping,
 }
 
-pub type UnresolvedDataSourceTemplate = BaseDataSourceTemplate<UnresolvedMapping>;
-pub type DataSourceTemplate = BaseDataSourceTemplate<Mapping>;
+
+#[derive(Clone, Debug)]
+pub struct DataSourceTemplate {
+    pub kind: String,
+    pub network: Option<String>,
+    pub name: String,
+    pub manifest_idx: u32,
+    pub mapping: Mapping,
+}
 
 impl UnresolvedDataSourceTemplate {
     pub async fn resolve(

From 3f51dc01e023aff9603889d4b04dff1b02d15b5b Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Wed, 10 Aug 2022 18:49:09 +0100
Subject: [PATCH 06/33] offchain: add `match_and_decode`, persist normally,
 require source

---
 core/src/subgraph/context.rs      |  6 +---
 core/src/subgraph/instance.rs     |  4 +--
 core/src/subgraph/runner.rs       | 50 ++++++++++---------------------
 graph/src/data_source/mod.rs      | 29 +++++-------------
 graph/src/data_source/offchain.rs | 43 +++++++++++++++-----------
 5 files changed, 51 insertions(+), 81 deletions(-)

diff --git a/core/src/subgraph/context.rs b/core/src/subgraph/context.rs
index 7677ef2b8b4..f9fceb57b3c 100644
--- a/core/src/subgraph/context.rs
+++ b/core/src/subgraph/context.rs
@@ -67,11 +67,7 @@ impl OffchainMonitor {
     }
 
     pub fn add_data_source(&mut self, ds: offchain::DataSource) -> Result<(), Error> {
-        let source = match &ds.source {
-            Some(source) => source,
-            None => anyhow::bail!("Failed to add offchain data source (missing source)"),
-        };
-        match source {
+        match ds.source {
             offchain::Source::Ipfs(cid) => self.ipfs_monitor.monitor(cid.clone()),
         };
         self.data_sources.push(ds);
diff --git a/core/src/subgraph/instance.rs b/core/src/subgraph/instance.rs
index 8d43e794194..ec40d977006 100644
--- a/core/src/subgraph/instance.rs
+++ b/core/src/subgraph/instance.rs
@@ -76,9 +76,7 @@ where
 
             // Create services for static offchain data sources
             if let DataSource::Offchain(ds) = &ds {
-                if ds.source.is_some() {
-                    offchain_monitor.add_data_source(ds.clone())?;
-                }
+                offchain_monitor.add_data_source(ds.clone())?;
             }
 
             let host = this.new_host(
diff --git a/core/src/subgraph/runner.rs b/core/src/subgraph/runner.rs
index 1434d9b74c3..aad671422cd 100644
--- a/core/src/subgraph/runner.rs
+++ b/core/src/subgraph/runner.rs
@@ -16,7 +16,7 @@ use graph::data::subgraph::{
     schema::{SubgraphError, SubgraphHealth, POI_OBJECT},
     SubgraphFeature,
 };
-use graph::data_source::{self, offchain, DataSource, TriggerData};
+use graph::data_source::{offchain, DataSource, TriggerData};
 use graph::prelude::*;
 use graph::util::{backoff::ExponentialBackoff, lfu_cache::LfuCache};
 use std::convert::TryFrom;
@@ -223,18 +223,15 @@ where
             let (data_sources, runtime_hosts) =
                 self.create_dynamic_data_sources(block_state.drain_created_data_sources())?;
 
-            let mut onchain_data_sources = Vec::new();
-            for ds in data_sources {
-                match ds {
-                    DataSource::Onchain(ds) => onchain_data_sources.push(ds),
-                    DataSource::Offchain(ds) => {
-                        self.ctx.offchain_monitor.add_data_source(ds.clone())?
-                    }
+            for ds in &data_sources {
+                if let DataSource::Offchain(ds) = ds {
+                    self.ctx.offchain_monitor.add_data_source(ds.clone())?
                 }
             }
-            let data_sources = onchain_data_sources;
 
-            let filter = C::TriggerFilter::from_data_sources(data_sources.iter());
+            let filter = C::TriggerFilter::from_data_sources(
+                data_sources.iter().filter_map(DataSource::as_onchain),
+            );
 
             // Reprocess the triggers from this block that match the new data sources
             let block_with_triggers = self
@@ -260,10 +257,7 @@ where
 
             // Add entity operations for the new data sources to the block state
             // and add runtimes for the data sources to the subgraph instance.
-            self.persist_dynamic_data_sources(
-                &mut block_state.entity_cache,
-                data_sources.into_iter().map(DataSource::Onchain).collect(),
-            );
+            self.persist_dynamic_data_sources(&mut block_state.entity_cache, data_sources);
 
             // Process the triggers in each host in the same order the
             // corresponding data sources have been created.
@@ -586,30 +580,16 @@ where
         source: offchain::Source,
         data: bytes::Bytes,
     ) -> Result<(), Error> {
-        let matching_data_sources = self
-            .ctx
-            .offchain_monitor
-            .data_sources
-            .iter()
-            .filter(|ds| ds.source.as_ref() == Some(&source));
-
-        let mut block_state =
-            BlockState::<C>::new(self.inputs.store.cheap_clone(), LfuCache::new());
+        let block_state = BlockState::<C>::new(self.inputs.store.cheap_clone(), LfuCache::new());
         let entity_cache = std::mem::take(&mut self.state.entity_lfu_cache);
 
-        self.persist_dynamic_data_sources(
-            &mut block_state.entity_cache,
-            matching_data_sources
-                .cloned()
-                .map(data_source::DataSource::Offchain)
-                .collect(),
-        );
+        // TODO: Set per-file causality region
+        // let causality_region = match &source {
+        //     offchain::Source::Ipfs(cid) => format!("ipfs/{}", cid.to_string()),
+        // };
 
-        let causality_region = match &source {
-            offchain::Source::Ipfs(cid) => format!("ipfs/{}", cid.to_string()),
-        };
         let trigger = TriggerData::Offchain(offchain::TriggerData {
-            source: source.clone(),
+            source,
             data: Arc::new(data),
         });
 
@@ -622,7 +602,7 @@ where
                 &trigger,
                 block_state,
                 &None,
-                &causality_region,
+                "IPFS TODO",
                 &self.inputs.debug_fork,
                 &self.metrics.subgraph,
             )
diff --git a/graph/src/data_source/mod.rs b/graph/src/data_source/mod.rs
index 9a62d3b34a5..f72c1c9ed99 100644
--- a/graph/src/data_source/mod.rs
+++ b/graph/src/data_source/mod.rs
@@ -103,14 +103,15 @@ impl<C: Blockchain> DataSource<C> {
         block: &Arc<C::Block>,
         logger: &Logger,
     ) -> Result<Option<TriggerWithHandler<MappingTrigger<C>>>, Error> {
-        match self {
-            Self::Onchain(ds) => ds
-                .match_and_decode(trigger.as_onchain().unwrap().clone(), block, logger)
+        match (self, trigger) {
+            (Self::Onchain(ds), TriggerData::Onchain(trigger)) => ds
+                .match_and_decode(trigger, block, logger)
                 .map(|t| t.map(|t| t.map(MappingTrigger::Onchain))),
-            Self::Offchain(ds) => Ok(Some(TriggerWithHandler::new(
-                MappingTrigger::Offchain(trigger.as_offchain().unwrap().clone()),
-                ds.mapping.handler.clone(),
-            ))),
+            (Self::Offchain(ds), TriggerData::Offchain(trigger)) => {
+                Ok(ds.match_and_decode(trigger))
+            }
+            (Self::Onchain(_), TriggerData::Offchain(_))
+            | (Self::Offchain(_), TriggerData::Onchain(_)) => Ok(None),
         }
     }
 
@@ -323,20 +324,6 @@ pub enum TriggerData<C: Blockchain> {
 }
 
 impl<C: Blockchain> TriggerData<C> {
-    fn as_onchain(&self) -> Option<&C::TriggerData> {
-        match self {
-            Self::Onchain(trigger) => Some(trigger),
-            Self::Offchain(_) => None,
-        }
-    }
-
-    fn as_offchain(&self) -> Option<&offchain::TriggerData> {
-        match self {
-            Self::Onchain(_) => None,
-            Self::Offchain(trigger) => Some(trigger),
-        }
-    }
-
     pub fn error_context(&self) -> String {
         match self {
             Self::Onchain(trigger) => trigger.error_context(),
diff --git a/graph/src/data_source/offchain.rs b/graph/src/data_source/offchain.rs
index 4d1c4e07167..87be4a3f0c2 100644
--- a/graph/src/data_source/offchain.rs
+++ b/graph/src/data_source/offchain.rs
@@ -9,12 +9,14 @@ use crate::{
     data_source,
     prelude::{DataSourceContext, Link},
 };
-use anyhow::{self, Error};
+use anyhow::{self, Context, Error};
 use cid::Cid;
 use serde::Deserialize;
 use slog::{info, Logger};
 use std::{fmt, sync::Arc};
 
+use super::TriggerWithHandler;
+
 pub const OFFCHAIN_KINDS: &'static [&'static str] = &["file/ipfs"];
 
 #[derive(Clone, Debug)]
@@ -22,7 +24,7 @@ pub struct DataSource {
     pub kind: String,
     pub name: String,
     pub manifest_idx: u32,
-    pub source: Option<Source>,
+    pub source: Source,
     pub mapping: Mapping,
     pub context: Arc<Option<DataSourceContext>>,
     pub creation_block: Option<BlockNumber>,
@@ -46,7 +48,7 @@ impl<C: Blockchain> TryFrom<DataSourceTemplateInfo<C>> for DataSource {
             kind: template.kind.clone(),
             name: template.name.clone(),
             manifest_idx: template.manifest_idx,
-            source: Some(Source::Ipfs(source.parse()?)),
+            source: Source::Ipfs(source.parse()?),
             mapping: template.mapping.clone(),
             context: Arc::new(info.context),
             creation_block: Some(info.creation_block),
@@ -55,10 +57,24 @@ impl<C: Blockchain> TryFrom<DataSourceTemplateInfo<C>> for DataSource {
 }
 
 impl DataSource {
+    pub fn match_and_decode<C: Blockchain>(
+        &self,
+        trigger: &TriggerData,
+    ) -> Option<TriggerWithHandler<super::MappingTrigger<C>>> {
+        if self.source != trigger.source {
+            return None;
+        }
+
+        Some(TriggerWithHandler::new(
+            data_source::MappingTrigger::Offchain(trigger.clone()),
+            self.mapping.handler.clone(),
+        ))
+    }
+
     pub fn as_stored_dynamic_data_source(&self) -> StoredDynamicDataSource {
-        let param = self.source.as_ref().map(|source| match source {
+        let param = match self.source {
             Source::Ipfs(link) => Bytes::from(link.to_bytes()),
-        });
+        };
         let context = self
             .context
             .as_ref()
@@ -66,7 +82,7 @@ impl DataSource {
             .map(|ctx| serde_json::to_value(&ctx).unwrap());
         StoredDynamicDataSource {
             manifest_idx: self.manifest_idx,
-            param,
+            param: Some(param),
             context,
             creation_block: self.creation_block,
         }
@@ -76,11 +92,8 @@ impl DataSource {
         template: &DataSourceTemplate,
         stored: StoredDynamicDataSource,
     ) -> Result<Self, Error> {
-        let source = stored.param.and_then(|bytes| {
-            Cid::try_from(bytes.as_slice().to_vec())
-                .ok()
-                .map(Source::Ipfs)
-        });
+        let param = stored.param.context("no param on stored data source")?;
+        let source = Source::Ipfs(Cid::try_from(param.as_slice().to_vec())?);
         let context = Arc::new(stored.context.map(serde_json::from_value).transpose()?);
         Ok(Self {
             kind: template.kind.clone(),
@@ -113,7 +126,7 @@ pub struct Mapping {
 pub struct UnresolvedDataSource {
     pub kind: String,
     pub name: String,
-    pub source: Option<UnresolvedSource>,
+    pub source: UnresolvedSource,
     pub mapping: UnresolvedMapping,
 }
 
@@ -145,10 +158,7 @@ impl UnresolvedDataSource {
             "source" => format_args!("{:?}", &self.source),
         );
         let source = match self.kind.as_str() {
-            "file/ipfs" => self
-                .source
-                .map(|src| src.file.link.parse().map(Source::Ipfs))
-                .transpose()?,
+            "file/ipfs" => Source::Ipfs(self.source.file.link.parse()?),
             _ => {
                 anyhow::bail!(
                     "offchain data source has invalid `kind`, expected `file/ipfs` but found {}",
@@ -194,7 +204,6 @@ pub struct UnresolvedDataSourceTemplate {
     pub mapping: UnresolvedMapping,
 }
 
-
 #[derive(Clone, Debug)]
 pub struct DataSourceTemplate {
     pub kind: String,

From 4def210b4932f77044a5a97b10166398a76f34dc Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Wed, 10 Aug 2022 20:30:15 +0100
Subject: [PATCH 07/33] trigger processor: take block ptr from the trigger

---
 chain/arweave/src/data_source.rs       |  1 +
 chain/cosmos/src/data_source.rs        |  1 +
 chain/ethereum/src/data_source.rs      |  3 +++
 chain/near/src/data_source.rs          |  1 +
 core/src/subgraph/trigger_processor.rs |  4 ++--
 graph/src/data_source/mod.rs           | 13 +++++++++++--
 graph/src/data_source/offchain.rs      |  3 ++-
 7 files changed, 21 insertions(+), 5 deletions(-)

diff --git a/chain/arweave/src/data_source.rs b/chain/arweave/src/data_source.rs
index dcf74e8e760..d378174edf8 100644
--- a/chain/arweave/src/data_source.rs
+++ b/chain/arweave/src/data_source.rs
@@ -68,6 +68,7 @@ impl blockchain::DataSource<Chain> for DataSource {
         Ok(Some(TriggerWithHandler::<Chain>::new(
             trigger.cheap_clone(),
             handler.to_owned(),
+            block.ptr(),
         )))
     }
 
diff --git a/chain/cosmos/src/data_source.rs b/chain/cosmos/src/data_source.rs
index 79e5e43f485..7535a50668e 100644
--- a/chain/cosmos/src/data_source.rs
+++ b/chain/cosmos/src/data_source.rs
@@ -77,6 +77,7 @@ impl blockchain::DataSource<Chain> for DataSource {
         Ok(Some(TriggerWithHandler::<Chain>::new(
             trigger.cheap_clone(),
             handler,
+            block.ptr(),
         )))
     }
 
diff --git a/chain/ethereum/src/data_source.rs b/chain/ethereum/src/data_source.rs
index 428f71b8c76..0eae1fd428d 100644
--- a/chain/ethereum/src/data_source.rs
+++ b/chain/ethereum/src/data_source.rs
@@ -490,6 +490,7 @@ impl DataSource {
                         block: block.cheap_clone(),
                     },
                     handler.handler,
+                    block.block_ptr(),
                 )))
             }
             EthereumTrigger::Log(log, receipt) => {
@@ -596,6 +597,7 @@ impl DataSource {
                         receipt: receipt.clone(),
                     },
                     event_handler.handler,
+                    block.block_ptr(),
                     logging_extras,
                 )))
             }
@@ -705,6 +707,7 @@ impl DataSource {
                         outputs,
                     },
                     handler.handler,
+                    block.block_ptr(),
                     logging_extras,
                 )))
             }
diff --git a/chain/near/src/data_source.rs b/chain/near/src/data_source.rs
index 95b2ce099d3..c0fa5c6371e 100644
--- a/chain/near/src/data_source.rs
+++ b/chain/near/src/data_source.rs
@@ -105,6 +105,7 @@ impl blockchain::DataSource<Chain> for DataSource {
         Ok(Some(TriggerWithHandler::<Chain>::new(
             trigger.cheap_clone(),
             handler.to_owned(),
+            block.ptr(),
         )))
     }
 
diff --git a/core/src/subgraph/trigger_processor.rs b/core/src/subgraph/trigger_processor.rs
index 7358a45402b..0c2a6cfee5f 100644
--- a/core/src/subgraph/trigger_processor.rs
+++ b/core/src/subgraph/trigger_processor.rs
@@ -1,5 +1,5 @@
 use async_trait::async_trait;
-use graph::blockchain::{Block, Blockchain};
+use graph::blockchain::Blockchain;
 use graph::cheap_clone::CheapClone;
 use graph::components::store::SubgraphFork;
 use graph::components::subgraph::{MappingError, SharedProofOfIndexing};
@@ -52,7 +52,7 @@ where
             state = host
                 .process_mapping_trigger(
                     logger,
-                    block.ptr(),
+                    mapping_trigger.block_ptr(),
                     mapping_trigger,
                     state,
                     proof_of_indexing.cheap_clone(),
diff --git a/graph/src/data_source/mod.rs b/graph/src/data_source/mod.rs
index f72c1c9ed99..697eb8725b2 100644
--- a/graph/src/data_source/mod.rs
+++ b/graph/src/data_source/mod.rs
@@ -2,7 +2,7 @@ pub mod offchain;
 
 use crate::{
     blockchain::{
-        Blockchain, DataSource as _, DataSourceTemplate as _, TriggerData as _,
+        BlockPtr, Blockchain, DataSource as _, DataSourceTemplate as _, TriggerData as _,
         UnresolvedDataSource as _, UnresolvedDataSourceTemplate as _,
     },
     components::{
@@ -267,6 +267,7 @@ impl<C: Blockchain> UnresolvedDataSourceTemplate<C> {
 pub struct TriggerWithHandler<T> {
     pub trigger: T,
     handler: String,
+    block_ptr: BlockPtr,
     logging_extras: Arc<dyn SendSyncRefUnwindSafeKV>,
 }
 
@@ -280,10 +281,11 @@ impl<T: fmt::Debug> fmt::Debug for TriggerWithHandler<T> {
 }
 
 impl<T> TriggerWithHandler<T> {
-    pub fn new(trigger: T, handler: String) -> Self {
+    pub fn new(trigger: T, handler: String, block_ptr: BlockPtr) -> Self {
         Self {
             trigger,
             handler,
+            block_ptr,
             logging_extras: Arc::new(slog::o! {}),
         }
     }
@@ -291,11 +293,13 @@ impl<T> TriggerWithHandler<T> {
     pub fn new_with_logging_extras(
         trigger: T,
         handler: String,
+        block_ptr: BlockPtr,
         logging_extras: Arc<dyn SendSyncRefUnwindSafeKV>,
     ) -> Self {
         TriggerWithHandler {
             trigger,
             handler,
+            block_ptr,
             logging_extras,
         }
     }
@@ -313,9 +317,14 @@ impl<T> TriggerWithHandler<T> {
         TriggerWithHandler {
             trigger: f(self.trigger),
             handler: self.handler,
+            block_ptr: self.block_ptr,
             logging_extras: self.logging_extras,
         }
     }
+
+    pub fn block_ptr(&self) -> BlockPtr {
+        self.block_ptr.clone()
+    }
 }
 
 pub enum TriggerData<C: Blockchain> {
diff --git a/graph/src/data_source/offchain.rs b/graph/src/data_source/offchain.rs
index 87be4a3f0c2..d0a984c2bb4 100644
--- a/graph/src/data_source/offchain.rs
+++ b/graph/src/data_source/offchain.rs
@@ -1,5 +1,5 @@
 use crate::{
-    blockchain::Blockchain,
+    blockchain::{BlockPtr, Blockchain},
     components::{
         link_resolver::LinkResolver,
         store::{BlockNumber, StoredDynamicDataSource},
@@ -68,6 +68,7 @@ impl DataSource {
         Some(TriggerWithHandler::new(
             data_source::MappingTrigger::Offchain(trigger.clone()),
             self.mapping.handler.clone(),
+            BlockPtr::new(Default::default(), self.creation_block.unwrap_or(0)),
         ))
     }
 

From f237fea350c0df9b11887cd2385e840b444f7949 Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Wed, 10 Aug 2022 20:58:03 +0100
Subject: [PATCH 08/33] offchain: Return cid to dataSource.address host fn

---
 graph/src/data_source/mod.rs      | 6 +++---
 graph/src/data_source/offchain.rs | 8 ++++++++
 runtime/wasm/src/host_exports.rs  | 2 +-
 3 files changed, 12 insertions(+), 4 deletions(-)

diff --git a/graph/src/data_source/mod.rs b/graph/src/data_source/mod.rs
index 697eb8725b2..e166d5f08af 100644
--- a/graph/src/data_source/mod.rs
+++ b/graph/src/data_source/mod.rs
@@ -48,10 +48,10 @@ impl<C: Blockchain> DataSource<C> {
         }
     }
 
-    pub fn address(&self) -> Option<&[u8]> {
+    pub fn address(&self) -> Option<Vec<u8>> {
         match self {
-            Self::Onchain(ds) => ds.address(),
-            Self::Offchain(_) => None,
+            Self::Onchain(ds) => ds.address().map(ToOwned::to_owned),
+            Self::Offchain(ds) => ds.address(),
         }
     }
 
diff --git a/graph/src/data_source/offchain.rs b/graph/src/data_source/offchain.rs
index d0a984c2bb4..51929f9b883 100644
--- a/graph/src/data_source/offchain.rs
+++ b/graph/src/data_source/offchain.rs
@@ -106,6 +106,14 @@ impl DataSource {
             creation_block: stored.creation_block,
         })
     }
+
+    /// The concept of an address may or not make sense for an offchain data source, but this is
+    /// used as the value to be returned to mappings from the `dataSource.address()` host function.
+    pub fn address(&self) -> Option<Vec<u8>> {
+        match self.source {
+            Source::Ipfs(cid) => Some(cid.to_bytes()),
+        }
+    }
 }
 
 #[derive(Clone, Debug, Eq, PartialEq)]
diff --git a/runtime/wasm/src/host_exports.rs b/runtime/wasm/src/host_exports.rs
index cbe1efbe9b0..28ef3b8192c 100644
--- a/runtime/wasm/src/host_exports.rs
+++ b/runtime/wasm/src/host_exports.rs
@@ -84,7 +84,7 @@ impl<C: Blockchain> HostExports<C> {
             subgraph_id,
             api_version: data_source.api_version(),
             data_source_name: data_source.name().to_owned(),
-            data_source_address: data_source.address().unwrap_or_default().to_owned(),
+            data_source_address: data_source.address().unwrap_or_default(),
             data_source_context: data_source.context().cheap_clone(),
             causality_region: CausalityRegion::from_network(&data_source_network),
             data_source_network,

From d43a2fad79a5ccd39f97ba1507b20cce32e433fe Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Wed, 10 Aug 2022 22:51:33 +0100
Subject: [PATCH 09/33] runner: transact modifications of offchain events

---
 core/src/subgraph/runner.rs | 116 ++++++++++++++++++++----------------
 1 file changed, 64 insertions(+), 52 deletions(-)

diff --git a/core/src/subgraph/runner.rs b/core/src/subgraph/runner.rs
index aad671422cd..73668bfe6d8 100644
--- a/core/src/subgraph/runner.rs
+++ b/core/src/subgraph/runner.rs
@@ -113,8 +113,6 @@ where
 
             // Process events from the stream as long as no restart is needed
             loop {
-                self.handle_offchain_queue().await?;
-
                 let event = {
                     let _section = self.metrics.stream.stopwatch.start_section("scan_blocks");
 
@@ -181,7 +179,12 @@ where
         // Process events one after the other, passing in entity operations
         // collected previously to every new event being processed
         let mut block_state = match self
-            .process_triggers(&proof_of_indexing, &block, triggers, &causality_region)
+            .process_triggers(
+                &proof_of_indexing,
+                &block,
+                triggers.into_iter().map(TriggerData::Onchain),
+                &causality_region,
+            )
             .await
         {
             // Triggers processed with no errors or with only deterministic errors.
@@ -330,6 +333,13 @@ where
             .map_err(|e| BlockProcessingError::Unknown(e.into()))?;
         section.end();
 
+        // Check for offchain events and process them, including their entity modifications.
+        {
+            let offchain_events = self.queued_offchain_events().await?;
+            let offchain_mods = self.handle_offchain_triggers(offchain_events).await?;
+            mods.extend(offchain_mods);
+        }
+
         // Put the cache back in the state, asserting that the placeholder cache was not used.
         assert!(self.state.entity_lfu_cache.is_empty());
         self.state.entity_lfu_cache = cache;
@@ -426,7 +436,7 @@ where
         &mut self,
         proof_of_indexing: &SharedProofOfIndexing,
         block: &Arc<C::Block>,
-        triggers: Vec<C::TriggerData>,
+        triggers: impl Iterator<Item = TriggerData<C>>,
         causality_region: &str,
     ) -> Result<BlockState<C>, MappingError> {
         let mut block_state = BlockState::new(
@@ -435,7 +445,6 @@ where
         );
 
         for trigger in triggers {
-            let trigger = TriggerData::Onchain(trigger);
             block_state = self
                 .ctx
                 .instance
@@ -563,65 +572,68 @@ where
         Ok(action)
     }
 
-    async fn handle_offchain_queue(&mut self) -> Result<(), Error> {
+    async fn queued_offchain_events(&mut self) -> Result<Vec<offchain::TriggerData>, Error> {
+        use graph::tokio::sync::mpsc::error::TryRecvError;
+
+        let mut triggers = vec![];
         loop {
-            if let Ok((cid, data)) = self.ctx.offchain_monitor.ipfs_monitor_rx.try_recv() {
-                self.handle_offchain_update(offchain::Source::Ipfs(cid), data)
-                    .await?;
-                continue;
+            match self.ctx.offchain_monitor.ipfs_monitor_rx.try_recv() {
+                Ok((cid, data)) => triggers.push(offchain::TriggerData {
+                    source: offchain::Source::Ipfs(cid),
+                    data: Arc::new(data),
+                }),
+                Err(TryRecvError::Disconnected) => {
+                    anyhow::bail!("ipfs monitor unexpectedly terminated")
+                }
+                Err(TryRecvError::Empty) => break,
             }
-            break;
         }
-        Ok(())
+        Ok(triggers)
     }
 
-    async fn handle_offchain_update(
+    async fn handle_offchain_triggers(
         &mut self,
-        source: offchain::Source,
-        data: bytes::Bytes,
-    ) -> Result<(), Error> {
-        let block_state = BlockState::<C>::new(self.inputs.store.cheap_clone(), LfuCache::new());
-        let entity_cache = std::mem::take(&mut self.state.entity_lfu_cache);
-
-        // TODO: Set per-file causality region
-        // let causality_region = match &source {
-        //     offchain::Source::Ipfs(cid) => format!("ipfs/{}", cid.to_string()),
-        // };
-
-        let trigger = TriggerData::Offchain(offchain::TriggerData {
-            source,
-            data: Arc::new(data),
-        });
-
-        let block_state = self
-            .ctx
-            .instance
-            .process_trigger(
-                &self.logger,
-                &Arc::default(),
-                &trigger,
-                block_state,
-                &None,
-                "IPFS TODO",
-                &self.inputs.debug_fork,
-                &self.metrics.subgraph,
-            )
-            .await
-            .map_err(move |err| {
-                let err = match err {
-                    MappingError::PossibleReorg(_) => unreachable!(),
-                    MappingError::Unknown(err) => err,
-                };
-                err.context("failed to process trigger".to_string())
-            })?;
+        triggers: Vec<offchain::TriggerData>,
+    ) -> Result<Vec<EntityModification>, Error> {
+        // TODO: Dont expose store with onchain entites
+        let mut block_state =
+            BlockState::<C>::new(self.inputs.store.cheap_clone(), LfuCache::new());
+
+        for trigger in triggers {
+            let causality_region = match &trigger.source {
+                offchain::Source::Ipfs(cid) => format!("ipfs/{}", cid.to_string()),
+            };
+
+            block_state = self
+                .ctx
+                .instance
+                .process_trigger(
+                    &self.logger,
+                    &Arc::default(),
+                    &TriggerData::Offchain(trigger),
+                    block_state,
+                    &None,
+                    &causality_region,
+                    &self.inputs.debug_fork,
+                    &self.metrics.subgraph,
+                )
+                .await
+                .map_err(move |err| {
+                    let err = match err {
+                        // Ignoring `PossibleReorg` isn't so bad since the subgraph will retry
+                        // non-deterministic errors.
+                        MappingError::PossibleReorg(e) | MappingError::Unknown(e) => e,
+                    };
+                    err.context("failed to process trigger".to_string())
+                })?;
+        }
 
         anyhow::ensure!(
             !block_state.has_created_data_sources(),
             "Attempted to create data source in offchain data source handler. This is not yet supported.",
         );
 
-        self.state.entity_lfu_cache = entity_cache;
-        Ok(())
+        Ok(block_state.entity_cache.as_modifications()?.modifications)
     }
 }
 

From b1b224aae8b37f3edd711a014bff91e0613481f2 Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Thu, 11 Aug 2022 10:52:03 +0100
Subject: [PATCH 10/33] ethereum: fix test build

---
 Cargo.lock                       | 2 --
 chain/ethereum/Cargo.toml        | 1 -
 chain/ethereum/tests/manifest.rs | 2 +-
 chain/near/src/chain.rs          | 1 +
 store/test-store/Cargo.toml      | 3 +--
 store/test-store/src/store.rs    | 2 +-
 6 files changed, 4 insertions(+), 7 deletions(-)

diff --git a/Cargo.lock b/Cargo.lock
index d8f88615003..37e43bb33c5 100644
--- a/Cargo.lock
+++ b/Cargo.lock
@@ -1568,7 +1568,6 @@ dependencies = [
  "envconfig",
  "futures 0.1.31",
  "graph",
- "graph-core",
  "graph-runtime-derive",
  "graph-runtime-wasm",
  "hex",
@@ -4168,7 +4167,6 @@ version = "0.27.0"
 dependencies = [
  "diesel",
  "graph",
- "graph-chain-ethereum",
  "graph-graphql",
  "graph-mock",
  "graph-node",
diff --git a/chain/ethereum/Cargo.toml b/chain/ethereum/Cargo.toml
index 315112d6835..b3143cd1f89 100644
--- a/chain/ethereum/Cargo.toml
+++ b/chain/ethereum/Cargo.toml
@@ -25,7 +25,6 @@ graph-runtime-wasm = { path = "../../runtime/wasm" }
 graph-runtime-derive = { path = "../../runtime/derive" }
 
 [dev-dependencies]
-graph-core = { path = "../../core" }
 test-store = { path = "../../store/test-store" }
 base64 = "0.13.0"
 
diff --git a/chain/ethereum/tests/manifest.rs b/chain/ethereum/tests/manifest.rs
index 5fee9aad492..cf9fdb8f626 100644
--- a/chain/ethereum/tests/manifest.rs
+++ b/chain/ethereum/tests/manifest.rs
@@ -164,7 +164,7 @@ specVersion: 0.0.7
     };
     assert_eq!(
         data_source.source,
-        Some(offchain::Source::Ipfs(FILE_CID.parse().unwrap()))
+        offchain::Source::Ipfs(FILE_CID.parse().unwrap())
     );
 }
 
diff --git a/chain/near/src/chain.rs b/chain/near/src/chain.rs
index 7082d05f83d..226a915f4c3 100644
--- a/chain/near/src/chain.rs
+++ b/chain/near/src/chain.rs
@@ -821,6 +821,7 @@ mod test {
         codec::Block {
             header: Some(BlockHeader {
                 height,
+                hash: Some(codec::CryptoHash { bytes: vec![0; 32] }),
                 ..Default::default()
             }),
             shards: vec![IndexerShard {
diff --git a/store/test-store/Cargo.toml b/store/test-store/Cargo.toml
index 05da5781ae8..de9dbe691cb 100644
--- a/store/test-store/Cargo.toml
+++ b/store/test-store/Cargo.toml
@@ -15,5 +15,4 @@ graph-store-postgres = { path = "../postgres" }
 lazy_static = "1.1"
 hex-literal = "0.3"
 diesel = { version = "1.4.8", features = ["postgres", "serde_json", "numeric", "r2d2"] }
-graph-chain-ethereum = { path = "../../chain/ethereum" }
-serde = "1.0"
\ No newline at end of file
+serde = "1.0"
diff --git a/store/test-store/src/store.rs b/store/test-store/src/store.rs
index bfb968c919d..929b571332f 100644
--- a/store/test-store/src/store.rs
+++ b/store/test-store/src/store.rs
@@ -154,7 +154,7 @@ pub async fn create_subgraph(
 ) -> Result<DeploymentLocator, StoreError> {
     let schema = Schema::parse(schema, subgraph_id.clone()).unwrap();
 
-    let manifest = SubgraphManifest::<graph_chain_ethereum::Chain> {
+    let manifest = SubgraphManifest::<graph::blockchain::mock::MockBlockchain> {
         id: subgraph_id.clone(),
         spec_version: Version::new(1, 0, 0),
         features: BTreeSet::new(),

From d5a51743ddd6efa5a6db6bc6d63b6c33632cd375 Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Thu, 11 Aug 2022 12:32:21 +0100
Subject: [PATCH 11/33] ipfs: Set a default maximum file size

---
 core/src/link_resolver.rs                | 34 +++++++++++-------------
 core/src/polling_monitor/ipfs_service.rs |  1 -
 core/src/subgraph/context.rs             |  2 +-
 docs/environment-variables.md            |  2 +-
 graph/src/env/mappings.rs                | 11 +++-----
 5 files changed, 22 insertions(+), 28 deletions(-)

diff --git a/core/src/link_resolver.rs b/core/src/link_resolver.rs
index b9d9ed5b8fa..9b2be6f107c 100644
--- a/core/src/link_resolver.rs
+++ b/core/src/link_resolver.rs
@@ -94,16 +94,14 @@ async fn select_fastest_client_with_stat(
 }
 
 // Returns an error if the stat is bigger than `max_file_bytes`
-fn restrict_file_size(path: &str, size: u64, max_file_bytes: &Option<u64>) -> Result<(), Error> {
-    if let Some(max_file_bytes) = max_file_bytes {
-        if size > *max_file_bytes {
-            return Err(anyhow!(
-                "IPFS file {} is too large. It can be at most {} bytes but is {} bytes",
-                path,
-                max_file_bytes,
-                size
-            ));
-        }
+fn restrict_file_size(path: &str, size: u64, max_file_bytes: usize) -> Result<(), Error> {
+    if size > max_file_bytes as u64 {
+        return Err(anyhow!(
+            "IPFS file {} is too large. It can be at most {} bytes but is {} bytes",
+            path,
+            max_file_bytes,
+            size
+        ));
     }
     Ok(())
 }
@@ -183,8 +181,8 @@ impl LinkResolverTrait for LinkResolver {
         .await?;
 
         let max_cache_file_size = self.env_vars.mappings.max_ipfs_cache_file_size;
-        let max_file_size = self.env_vars.mappings.max_ipfs_file_bytes.map(|n| n as u64);
-        restrict_file_size(&path, size, &max_file_size)?;
+        let max_file_size = self.env_vars.mappings.max_ipfs_file_bytes;
+        restrict_file_size(&path, size, max_file_size)?;
 
         let req_path = path.clone();
         let timeout = self.timeout;
@@ -197,7 +195,7 @@ impl LinkResolverTrait for LinkResolver {
             .await?;
 
         // The size reported by `files/stat` is not guaranteed to be exact, so check the limit again.
-        restrict_file_size(&path, data.len() as u64, &max_file_size)?;
+        restrict_file_size(&path, data.len() as u64, max_file_size)?;
 
         // Only cache files if they are not too large
         if data.len() <= max_cache_file_size {
@@ -227,8 +225,8 @@ impl LinkResolverTrait for LinkResolver {
         )
         .await?;
 
-        let max_file_size = self.env_vars.mappings.max_ipfs_file_bytes.map(|n| n as u64);
-        restrict_file_size(&link.link, size, &max_file_size)?;
+        let max_file_size = self.env_vars.mappings.max_ipfs_file_bytes;
+        restrict_file_size(&link.link, size, max_file_size)?;
 
         let link = link.link.clone();
         let data = retry_policy(self.retry, "ipfs.getBlock", &logger)
@@ -259,8 +257,8 @@ impl LinkResolverTrait for LinkResolver {
         )
         .await?;
 
-        let max_file_size = Some(self.env_vars.mappings.max_ipfs_map_file_size as u64);
-        restrict_file_size(path, size, &max_file_size)?;
+        let max_file_size = self.env_vars.mappings.max_ipfs_map_file_size;
+        restrict_file_size(path, size, max_file_size)?;
 
         let mut stream = client.cat(path, None).await?.fuse().boxed().compat();
 
@@ -334,7 +332,7 @@ mod tests {
     #[tokio::test]
     async fn max_file_size() {
         let mut env_vars = EnvVars::default();
-        env_vars.mappings.max_ipfs_file_bytes = Some(200);
+        env_vars.mappings.max_ipfs_file_bytes = 200;
 
         let file: &[u8] = &[0u8; 201];
         let client = IpfsClient::localhost();
diff --git a/core/src/polling_monitor/ipfs_service.rs b/core/src/polling_monitor/ipfs_service.rs
index dae0e44429b..81bcbaae938 100644
--- a/core/src/polling_monitor/ipfs_service.rs
+++ b/core/src/polling_monitor/ipfs_service.rs
@@ -33,7 +33,6 @@ impl CheapClone for IpfsService {
 }
 
 impl IpfsService {
-    #[allow(dead_code)]
     pub fn new(
         client: IpfsClient,
         max_file_size: u64,
diff --git a/core/src/subgraph/context.rs b/core/src/subgraph/context.rs
index f9fceb57b3c..698f6d9ab40 100644
--- a/core/src/subgraph/context.rs
+++ b/core/src/subgraph/context.rs
@@ -49,7 +49,7 @@ impl OffchainMonitor {
         let (ipfs_monitor_tx, ipfs_monitor_rx) = mpsc::channel(10);
         let ipfs_service = IpfsService::new(
             client,
-            ENV_VARS.mappings.max_ipfs_file_bytes.unwrap_or(1 << 20) as u64,
+            ENV_VARS.mappings.max_ipfs_file_bytes as u64,
             ENV_VARS.mappings.ipfs_timeout,
             10,
         );
diff --git a/docs/environment-variables.md b/docs/environment-variables.md
index 3a87514bc77..520b9b23619 100644
--- a/docs/environment-variables.md
+++ b/docs/environment-variables.md
@@ -61,7 +61,7 @@ those.
 - `GRAPH_IPFS_TIMEOUT`: timeout for IPFS, which includes requests for manifest files
   and from mappings using `ipfs.cat` or `ipfs.map` (in seconds, default is 30).
 - `GRAPH_MAX_IPFS_FILE_BYTES`: maximum size for a file that can be retrieved
-  with `ipfs.cat` (in bytes, default is unlimited)
+  with `ipfs.cat` (in bytes, default is 256 MiB)
 - `GRAPH_MAX_IPFS_MAP_FILE_SIZE`: maximum size of files that can be processed
   with `ipfs.map`. When a file is processed through `ipfs.map`, the entities
   generated from that are kept in memory until the entire file is done
diff --git a/graph/src/env/mappings.rs b/graph/src/env/mappings.rs
index 57502f9ddcb..a9b47f84da7 100644
--- a/graph/src/env/mappings.rs
+++ b/graph/src/env/mappings.rs
@@ -46,11 +46,8 @@ pub struct EnvVarsMapping {
     /// Sets the `ipfs.cat` file size limit.
     ///
     /// Set by the environment variable `GRAPH_MAX_IPFS_FILE_BYTES` (expressed in
-    /// bytes). No default value is provided.
-    ///
-    /// FIXME: Having an env variable here is a problem for consensus.
-    /// Index Nodes should not disagree on whether the file should be read.
-    pub max_ipfs_file_bytes: Option<usize>,
+    /// bytes). Defaults to 256 MiB.
+    pub max_ipfs_file_bytes: usize,
     /// Set by the flag `GRAPH_ALLOW_NON_DETERMINISTIC_IPFS`. Off by
     /// default.
     pub allow_non_deterministic_ipfs: bool,
@@ -77,7 +74,7 @@ impl From<InnerMappingHandlers> for EnvVarsMapping {
             max_ipfs_cache_size: x.max_ipfs_cache_size,
             ipfs_timeout: Duration::from_secs(x.ipfs_timeout_in_secs),
             max_ipfs_map_file_size: x.max_ipfs_map_file_size.0,
-            max_ipfs_file_bytes: x.max_ipfs_file_bytes,
+            max_ipfs_file_bytes: x.max_ipfs_file_bytes.0,
             allow_non_deterministic_ipfs: x.allow_non_deterministic_ipfs.0,
         }
     }
@@ -106,7 +103,7 @@ pub struct InnerMappingHandlers {
     #[envconfig(from = "GRAPH_MAX_IPFS_MAP_FILE_SIZE", default = "")]
     max_ipfs_map_file_size: WithDefaultUsize<usize, { 256 * 1024 * 1024 }>,
     #[envconfig(from = "GRAPH_MAX_IPFS_FILE_BYTES")]
-    max_ipfs_file_bytes: Option<usize>,
+    max_ipfs_file_bytes: WithDefaultUsize<usize, { 256 * 1024 * 1024 }>,
     #[envconfig(from = "GRAPH_ALLOW_NON_DETERMINISTIC_IPFS", default = "false")]
     allow_non_deterministic_ipfs: EnvVarBoolean,
 }

From 7347307a3fd4c80ce2e9364422aa4a48303dfd45 Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Thu, 11 Aug 2022 13:36:31 +0100
Subject: [PATCH 12/33] ipfs: Add env var for max concurrent requests

---
 core/src/subgraph/context.rs  |  2 +-
 docs/environment-variables.md | 27 ++++++++++++++-------------
 graph/src/env/mappings.rs     |  6 +++++-
 3 files changed, 20 insertions(+), 15 deletions(-)

diff --git a/core/src/subgraph/context.rs b/core/src/subgraph/context.rs
index 698f6d9ab40..e710abe0fe7 100644
--- a/core/src/subgraph/context.rs
+++ b/core/src/subgraph/context.rs
@@ -51,7 +51,7 @@ impl OffchainMonitor {
             client,
             ENV_VARS.mappings.max_ipfs_file_bytes as u64,
             ENV_VARS.mappings.ipfs_timeout,
-            10,
+            ENV_VARS.mappings.max_ipfs_concurrent_requests,
         );
         let ipfs_monitor = spawn_monitor(
             ipfs_service,
diff --git a/docs/environment-variables.md b/docs/environment-variables.md
index 520b9b23619..da827579dfc 100644
--- a/docs/environment-variables.md
+++ b/docs/environment-variables.md
@@ -58,19 +58,6 @@ those.
 
 - `GRAPH_MAPPING_HANDLER_TIMEOUT`: amount of time a mapping handler is allowed to
   take (in seconds, default is unlimited)
-- `GRAPH_IPFS_TIMEOUT`: timeout for IPFS, which includes requests for manifest files
-  and from mappings using `ipfs.cat` or `ipfs.map` (in seconds, default is 30).
-- `GRAPH_MAX_IPFS_FILE_BYTES`: maximum size for a file that can be retrieved
-  with `ipfs.cat` (in bytes, default is 256 MiB)
-- `GRAPH_MAX_IPFS_MAP_FILE_SIZE`: maximum size of files that can be processed
-  with `ipfs.map`. When a file is processed through `ipfs.map`, the entities
-  generated from that are kept in memory until the entire file is done
-  processing. This setting therefore limits how much memory a call to `ipfs.map`
-  may use. (in bytes, defaults to 256MB)
-- `GRAPH_MAX_IPFS_CACHE_SIZE`: maximum number of files cached in the the
-  `ipfs.cat` cache (defaults to 50).
-- `GRAPH_MAX_IPFS_CACHE_FILE_SIZE`: maximum size of files that are cached in the
-  `ipfs.cat` cache (defaults to 1MiB)
 - `GRAPH_ENTITY_CACHE_SIZE`: Size of the entity cache, in kilobytes. Defaults to 10000 which is 10MB.
 - `GRAPH_MAX_API_VERSION`: Maximum `apiVersion` supported, if a developer tries to create a subgraph
   with a higher `apiVersion` than this in their mappings, they'll receive an error. Defaults to `0.0.7`.
@@ -79,6 +66,20 @@ those.
 - `GRAPH_RUNTIME_MAX_STACK_SIZE`: Maximum stack size for the WASM runtime, if exceeded the execution
   stops and an error is thrown. Defaults to 512KiB.
 
+## IPFS
+
+- `GRAPH_IPFS_TIMEOUT`: timeout for IPFS, which includes requests for manifest files
+  and from mappings (in seconds, default is 30).
+- `GRAPH_MAX_IPFS_FILE_BYTES`: maximum size for a file that can be retrieved (in bytes, default is 256 MiB).
+- `GRAPH_MAX_IPFS_MAP_FILE_SIZE`: maximum size of files that can be processed
+  with `ipfs.map`. When a file is processed through `ipfs.map`, the entities
+  generated from that are kept in memory until the entire file is done
+  processing. This setting therefore limits how much memory a call to `ipfs.map`
+  may use (in bytes, defaults to 256MB).
+- `GRAPH_MAX_IPFS_CACHE_SIZE`: maximum number of files cached (defaults to 50).
+- `GRAPH_MAX_IPFS_CACHE_FILE_SIZE`: maximum size of each cached file (in bytes, defaults to 1MiB).
+- `GRAPH_MAX_IPFS_CONCURRENT_REQUESTS`: maximum concurrent requests to IPFS from file data sources (defaults to 10).
+
 ## GraphQL
 
 - `GRAPH_GRAPHQL_QUERY_TIMEOUT`: maximum execution time for a graphql query, in
diff --git a/graph/src/env/mappings.rs b/graph/src/env/mappings.rs
index a9b47f84da7..f14086679df 100644
--- a/graph/src/env/mappings.rs
+++ b/graph/src/env/mappings.rs
@@ -48,6 +48,7 @@ pub struct EnvVarsMapping {
     /// Set by the environment variable `GRAPH_MAX_IPFS_FILE_BYTES` (expressed in
     /// bytes). Defaults to 256 MiB.
     pub max_ipfs_file_bytes: usize,
+    pub max_ipfs_concurrent_requests: u16,
     /// Set by the flag `GRAPH_ALLOW_NON_DETERMINISTIC_IPFS`. Off by
     /// default.
     pub allow_non_deterministic_ipfs: bool,
@@ -75,6 +76,7 @@ impl From<InnerMappingHandlers> for EnvVarsMapping {
             ipfs_timeout: Duration::from_secs(x.ipfs_timeout_in_secs),
             max_ipfs_map_file_size: x.max_ipfs_map_file_size.0,
             max_ipfs_file_bytes: x.max_ipfs_file_bytes.0,
+            max_ipfs_concurrent_requests: x.max_ipfs_concurrent_requests,
             allow_non_deterministic_ipfs: x.allow_non_deterministic_ipfs.0,
         }
     }
@@ -102,8 +104,10 @@ pub struct InnerMappingHandlers {
     ipfs_timeout_in_secs: u64,
     #[envconfig(from = "GRAPH_MAX_IPFS_MAP_FILE_SIZE", default = "")]
     max_ipfs_map_file_size: WithDefaultUsize<usize, { 256 * 1024 * 1024 }>,
-    #[envconfig(from = "GRAPH_MAX_IPFS_FILE_BYTES")]
+    #[envconfig(from = "GRAPH_MAX_IPFS_FILE_BYTES", default = "")]
     max_ipfs_file_bytes: WithDefaultUsize<usize, { 256 * 1024 * 1024 }>,
+    #[envconfig(from = "GRAPH_MAX_IPFS_CONCURRENT_REQUESTS", default = "10")]
+    max_ipfs_concurrent_requests: u16,
     #[envconfig(from = "GRAPH_ALLOW_NON_DETERMINISTIC_IPFS", default = "false")]
     allow_non_deterministic_ipfs: EnvVarBoolean,
 }

From e17fd104f4ee547a70bd85f77131f2647dc24a4e Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Thu, 11 Aug 2022 15:06:31 +0100
Subject: [PATCH 13/33] ipfs: Share ipfs service across subgraphs

---
 core/src/polling_monitor/ipfs_service.rs |  1 +
 core/src/subgraph/context.rs             | 10 +---------
 core/src/subgraph/instance_manager.rs    | 10 +++++-----
 node/src/main.rs                         |  9 ++++++++-
 node/src/manager/commands/run.rs         |  9 ++++++++-
 tests/src/fixture.rs                     |  9 ++++++++-
 6 files changed, 31 insertions(+), 17 deletions(-)

diff --git a/core/src/polling_monitor/ipfs_service.rs b/core/src/polling_monitor/ipfs_service.rs
index 81bcbaae938..05d904f17fc 100644
--- a/core/src/polling_monitor/ipfs_service.rs
+++ b/core/src/polling_monitor/ipfs_service.rs
@@ -13,6 +13,7 @@ use tower::Service;
 const CLOUDFLARE_TIMEOUT: u16 = 524;
 const GATEWAY_TIMEOUT: u16 = 504;
 
+/// Reference type, clones will refer to the same service.
 #[derive(Clone)]
 pub struct IpfsService {
     client: IpfsClient,
diff --git a/core/src/subgraph/context.rs b/core/src/subgraph/context.rs
index e710abe0fe7..5fc95302c20 100644
--- a/core/src/subgraph/context.rs
+++ b/core/src/subgraph/context.rs
@@ -11,8 +11,6 @@ use graph::{
     blockchain::Blockchain,
     components::store::DeploymentId,
     data_source::offchain,
-    env::ENV_VARS,
-    ipfs_client::IpfsClient,
     prelude::{CancelGuard, DeploymentHash, MetricsRegistry, RuntimeHostBuilder},
     slog::Logger,
     tokio::sync::mpsc,
@@ -44,15 +42,9 @@ impl OffchainMonitor {
         logger: Logger,
         registry: Arc<dyn MetricsRegistry>,
         subgraph_hash: &DeploymentHash,
-        client: IpfsClient,
+        ipfs_service: IpfsService,
     ) -> Self {
         let (ipfs_monitor_tx, ipfs_monitor_rx) = mpsc::channel(10);
-        let ipfs_service = IpfsService::new(
-            client,
-            ENV_VARS.mappings.max_ipfs_file_bytes as u64,
-            ENV_VARS.mappings.ipfs_timeout,
-            ENV_VARS.mappings.max_ipfs_concurrent_requests,
-        );
         let ipfs_monitor = spawn_monitor(
             ipfs_service,
             ipfs_monitor_tx,
diff --git a/core/src/subgraph/instance_manager.rs b/core/src/subgraph/instance_manager.rs
index c031ed4d3a9..14feb8d8812 100644
--- a/core/src/subgraph/instance_manager.rs
+++ b/core/src/subgraph/instance_manager.rs
@@ -1,3 +1,4 @@
+use crate::polling_monitor::ipfs_service::IpfsService;
 use crate::subgraph::context::{IndexingContext, SharedInstanceKeepAliveMap};
 use crate::subgraph::inputs::IndexingInputs;
 use crate::subgraph::loader::load_dynamic_data_sources;
@@ -7,7 +8,6 @@ use graph::blockchain::block_stream::BlockStreamMetrics;
 use graph::blockchain::Blockchain;
 use graph::blockchain::NodeCapabilities;
 use graph::blockchain::{BlockchainKind, TriggerFilter};
-use graph::ipfs_client::IpfsClient;
 use graph::prelude::{SubgraphInstanceManager as SubgraphInstanceManagerTrait, *};
 use graph::{blockchain::BlockchainMap, components::store::DeploymentLocator};
 use graph_runtime_wasm::module::ToAscPtr;
@@ -25,7 +25,7 @@ pub struct SubgraphInstanceManager<S: SubgraphStore> {
     manager_metrics: SubgraphInstanceManagerMetrics,
     instances: SharedInstanceKeepAliveMap,
     link_resolver: Arc<dyn LinkResolver>,
-    ipfs_client: IpfsClient,
+    ipfs_service: IpfsService,
     static_filters: bool,
 }
 
@@ -137,7 +137,7 @@ impl<S: SubgraphStore> SubgraphInstanceManager<S> {
         chains: Arc<BlockchainMap>,
         metrics_registry: Arc<dyn MetricsRegistry>,
         link_resolver: Arc<dyn LinkResolver>,
-        ipfs_client: IpfsClient,
+        ipfs_service: IpfsService,
         static_filters: bool,
     ) -> Self {
         let logger = logger_factory.component_logger("SubgraphInstanceManager", None);
@@ -151,7 +151,7 @@ impl<S: SubgraphStore> SubgraphInstanceManager<S> {
             metrics_registry,
             instances: SharedInstanceKeepAliveMap::default(),
             link_resolver,
-            ipfs_client,
+            ipfs_service,
             static_filters,
         }
     }
@@ -303,7 +303,7 @@ impl<S: SubgraphStore> SubgraphInstanceManager<S> {
             logger.cheap_clone(),
             registry.cheap_clone(),
             &manifest.id,
-            self.ipfs_client.cheap_clone(),
+            self.ipfs_service.cheap_clone(),
         );
 
         // Initialize deployment_head with current deployment head. Any sort of trouble in
diff --git a/node/src/main.rs b/node/src/main.rs
index acf68a63b0b..8e2218ebc14 100644
--- a/node/src/main.rs
+++ b/node/src/main.rs
@@ -20,6 +20,7 @@ use graph_chain_cosmos::{self as cosmos, Block as CosmosFirehoseBlock};
 use graph_chain_ethereum as ethereum;
 use graph_chain_near::{self as near, HeaderOnlyBlock as NearFirehoseHeaderOnlyBlock};
 use graph_chain_substreams as substreams;
+use graph_core::polling_monitor::ipfs_service::IpfsService;
 use graph_core::{
     LinkResolver, MetricsRegistry, SubgraphAssignmentProvider as IpfsSubgraphAssignmentProvider,
     SubgraphInstanceManager, SubgraphRegistrar as IpfsSubgraphRegistrar,
@@ -196,6 +197,12 @@ async fn main() {
     // Try to create IPFS clients for each URL specified in `--ipfs`
     let ipfs_clients: Vec<_> = create_ipfs_clients(&logger, &opt.ipfs);
     let ipfs_client = ipfs_clients.first().cloned().expect("Missing IPFS client");
+    let ipfs_service = IpfsService::new(
+        ipfs_client,
+        ENV_VARS.mappings.max_ipfs_file_bytes as u64,
+        ENV_VARS.mappings.ipfs_timeout,
+        ENV_VARS.mappings.max_ipfs_concurrent_requests,
+    );
 
     // Convert the clients into a link resolver. Since we want to get past
     // possible temporary DNS failures, make the resolver retry
@@ -434,7 +441,7 @@ async fn main() {
             blockchain_map.cheap_clone(),
             metrics_registry.clone(),
             link_resolver.clone(),
-            ipfs_client,
+            ipfs_service,
             static_filters,
         );
 
diff --git a/node/src/manager/commands/run.rs b/node/src/manager/commands/run.rs
index 7e6887819ba..df0db21325d 100644
--- a/node/src/manager/commands/run.rs
+++ b/node/src/manager/commands/run.rs
@@ -26,6 +26,7 @@ use graph::prelude::{
 use graph::slog::{debug, error, info, o, Logger};
 use graph::util::security::SafeDisplay;
 use graph_chain_ethereum::{self as ethereum, EthereumAdapterTrait, Transport};
+use graph_core::polling_monitor::ipfs_service::IpfsService;
 use graph_core::{
     LinkResolver, SubgraphAssignmentProvider as IpfsSubgraphAssignmentProvider,
     SubgraphInstanceManager, SubgraphRegistrar as IpfsSubgraphRegistrar,
@@ -63,6 +64,12 @@ pub async fn run(
     // FIXME: Hard-coded IPFS config, take it from config file instead?
     let ipfs_clients: Vec<_> = create_ipfs_clients(&logger, &ipfs_url);
     let ipfs_client = ipfs_clients.first().cloned().expect("Missing IPFS client");
+    let ipfs_service = IpfsService::new(
+        ipfs_client,
+        ENV_VARS.mappings.max_ipfs_file_bytes as u64,
+        ENV_VARS.mappings.ipfs_timeout,
+        ENV_VARS.mappings.max_ipfs_concurrent_requests,
+    );
 
     // Convert the clients into a link resolver. Since we want to get past
     // possible temporary DNS failures, make the resolver retry
@@ -154,7 +161,7 @@ pub async fn run(
         blockchain_map.clone(),
         metrics_registry.clone(),
         link_resolver.cheap_clone(),
-        ipfs_client,
+        ipfs_service,
         static_filters,
     );
 
diff --git a/tests/src/fixture.rs b/tests/src/fixture.rs
index c0a54b299db..1b491975e51 100644
--- a/tests/src/fixture.rs
+++ b/tests/src/fixture.rs
@@ -26,6 +26,7 @@ use graph::prelude::{
     SubgraphAssignmentProvider, SubgraphName, SubgraphRegistrar, SubgraphStore as _,
     SubgraphVersionSwitchingMode,
 };
+use graph_core::polling_monitor::ipfs_service::IpfsService;
 use graph_core::{
     LinkResolver, SubgraphAssignmentProvider as IpfsSubgraphAssignmentProvider,
     SubgraphInstanceManager, SubgraphRegistrar as IpfsSubgraphRegistrar,
@@ -203,6 +204,12 @@ pub async fn setup<C: Blockchain>(
         vec![ipfs.cheap_clone()],
         Default::default(),
     ));
+    let ipfs_service = IpfsService::new(
+        ipfs,
+        ENV_VARS.mappings.max_ipfs_file_bytes as u64,
+        ENV_VARS.mappings.ipfs_timeout,
+        ENV_VARS.mappings.max_ipfs_concurrent_requests,
+    );
 
     let blockchain_map = Arc::new(blockchain_map);
     let subgraph_instance_manager = SubgraphInstanceManager::new(
@@ -211,7 +218,7 @@ pub async fn setup<C: Blockchain>(
         blockchain_map.clone(),
         mock_registry.clone(),
         link_resolver.cheap_clone(),
-        ipfs,
+        ipfs_service,
         static_filters,
     );
 

From 516e10423f60f6a7ac5f51968d5683d124ad453d Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Thu, 11 Aug 2022 15:57:30 +0100
Subject: [PATCH 14/33] offchain: move `ready_offchain_events` to
 `OffchainMonitor`

---
 core/src/subgraph/context.rs | 27 +++++++++++++++++++++++----
 core/src/subgraph/runner.rs  | 25 ++++---------------------
 2 files changed, 27 insertions(+), 25 deletions(-)

diff --git a/core/src/subgraph/context.rs b/core/src/subgraph/context.rs
index 5fc95302c20..d5ca178a95c 100644
--- a/core/src/subgraph/context.rs
+++ b/core/src/subgraph/context.rs
@@ -28,13 +28,13 @@ where
     pub instance: SubgraphInstance<C, T>,
     pub instances: SharedInstanceKeepAliveMap,
     pub filter: C::TriggerFilter,
-    pub offchain_monitor: OffchainMonitor,
+    pub(crate) offchain_monitor: OffchainMonitor,
 }
 
-pub struct OffchainMonitor {
+pub(crate) struct OffchainMonitor {
     ipfs_monitor: PollingMonitor<Cid>,
-    pub ipfs_monitor_rx: mpsc::Receiver<(Cid, Bytes)>,
-    pub data_sources: Vec<offchain::DataSource>,
+    ipfs_monitor_rx: mpsc::Receiver<(Cid, Bytes)>,
+    data_sources: Vec<offchain::DataSource>,
 }
 
 impl OffchainMonitor {
@@ -65,4 +65,23 @@ impl OffchainMonitor {
         self.data_sources.push(ds);
         Ok(())
     }
+
+    pub fn ready_offchain_events(&mut self) -> Result<Vec<offchain::TriggerData>, Error> {
+        use graph::tokio::sync::mpsc::error::TryRecvError;
+
+        let mut triggers = vec![];
+        loop {
+            match self.ipfs_monitor_rx.try_recv() {
+                Ok((cid, data)) => triggers.push(offchain::TriggerData {
+                    source: offchain::Source::Ipfs(cid),
+                    data: Arc::new(data),
+                }),
+                Err(TryRecvError::Disconnected) => {
+                    anyhow::bail!("ipfs monitor unexpectedly terminated")
+                }
+                Err(TryRecvError::Empty) => break,
+            }
+        }
+        Ok(triggers)
+    }
 }
diff --git a/core/src/subgraph/runner.rs b/core/src/subgraph/runner.rs
index 73668bfe6d8..c6b37b33a33 100644
--- a/core/src/subgraph/runner.rs
+++ b/core/src/subgraph/runner.rs
@@ -335,7 +335,7 @@ where
 
         // Check for offchain events and process them, including their entity modifications.
         {
-            let offchain_events = self.queued_offchain_events().await?;
+            let offchain_events = self.ctx.offchain_monitor.ready_offchain_events()?;
             let offchain_mods = self.handle_offchain_triggers(offchain_events).await?;
             mods.extend(offchain_mods);
         }
@@ -572,25 +572,6 @@ where
         Ok(action)
     }
 
-    async fn queued_offchain_events(&mut self) -> Result<Vec<offchain::TriggerData>, Error> {
-        use graph::tokio::sync::mpsc::error::TryRecvError;
-
-        let mut triggers = vec![];
-        loop {
-            match self.ctx.offchain_monitor.ipfs_monitor_rx.try_recv() {
-                Ok((cid, data)) => triggers.push(offchain::TriggerData {
-                    source: offchain::Source::Ipfs(cid),
-                    data: Arc::new(data),
-                }),
-                Err(TryRecvError::Disconnected) => {
-                    anyhow::bail!("ipfs monitor unexpectedly terminated")
-                }
-                Err(TryRecvError::Empty) => break,
-            }
-        }
-        Ok(triggers)
-    }
-
     async fn handle_offchain_triggers(
         &mut self,
         triggers: Vec<offchain::TriggerData>,
@@ -604,12 +585,14 @@ where
                 offchain::Source::Ipfs(cid) => format!("ipfs/{}", cid.to_string()),
             };
 
+            // We'll eventually need to do better here, but using an empty block works for now.
+            let block = Arc::default();
             block_state = self
                 .ctx
                 .instance
                 .process_trigger(
                     &self.logger,
-                    &Arc::default(),
+                    &block,
                     &TriggerData::Offchain(trigger),
                     block_state,
                     &None,

From 3f940c0247e55b5309ed87a051d3138b98ddaa63 Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Fri, 12 Aug 2022 15:48:14 +0100
Subject: [PATCH 15/33] runner: Clarify comments

---
 core/src/subgraph/runner.rs | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)

diff --git a/core/src/subgraph/runner.rs b/core/src/subgraph/runner.rs
index c6b37b33a33..e17479ba90a 100644
--- a/core/src/subgraph/runner.rs
+++ b/core/src/subgraph/runner.rs
@@ -173,7 +173,7 @@ where
             None
         };
 
-        // There are currently no other causality regions since offchain data is not supported.
+        // Causality region for onchain triggers.
         let causality_region = CausalityRegion::from_network(self.ctx.instance.network());
 
         // Process events one after the other, passing in entity operations
@@ -201,7 +201,7 @@ where
                 // In case of a possible reorg, we want this function to do nothing and restart the
                 // block stream so it has a chance to detect the reorg.
                 //
-                // The `state` is unchanged at this point, except for having cleared the entity cache.
+                // The state is unchanged at this point, except for having cleared the entity cache.
                 // Losing the cache is a bit annoying but not an issue for correctness.
                 //
                 // See also b21fa73b-6453-4340-99fb-1a78ec62efb1.
@@ -333,7 +333,8 @@ where
             .map_err(|e| BlockProcessingError::Unknown(e.into()))?;
         section.end();
 
-        // Check for offchain events and process them, including their entity modifications.
+        // Check for offchain events and process them, including their entity modifications in the
+        // set to be transacted.
         {
             let offchain_events = self.ctx.offchain_monitor.ready_offchain_events()?;
             let offchain_mods = self.handle_offchain_triggers(offchain_events).await?;

From c3a0855e18604b9ddf2a1cd0dfaadd42a69a193a Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Fri, 12 Aug 2022 16:17:36 +0100
Subject: [PATCH 16/33] core: Remove unecessary params from
 `add_dynamic_data_source`

---
 core/src/subgraph/instance.rs | 30 +++++++++---------------------
 core/src/subgraph/runner.rs   | 10 ++++------
 2 files changed, 13 insertions(+), 27 deletions(-)

diff --git a/core/src/subgraph/instance.rs b/core/src/subgraph/instance.rs
index ec40d977006..e7b749db8a9 100644
--- a/core/src/subgraph/instance.rs
+++ b/core/src/subgraph/instance.rs
@@ -19,6 +19,8 @@ pub struct SubgraphInstance<C: Blockchain, T: RuntimeHostBuilder<C>> {
     pub poi_version: ProofOfIndexingVersion,
     host_builder: T,
     pub(crate) trigger_processor: Box<dyn TriggerProcessor<C, T>>,
+    templates: Arc<Vec<DataSourceTemplate<C>>>,
+    host_metrics: Arc<HostMetrics>,
 
     /// Runtime hosts, one for each data source mapping.
     ///
@@ -62,6 +64,8 @@ where
             module_cache: HashMap::new(),
             poi_version,
             trigger_processor,
+            templates,
+            host_metrics,
         };
 
         // Create a new runtime host for each data source in the subgraph manifest;
@@ -79,13 +83,7 @@ where
                 offchain_monitor.add_data_source(ds.clone())?;
             }
 
-            let host = this.new_host(
-                logger.cheap_clone(),
-                ds,
-                module_bytes,
-                templates.cheap_clone(),
-                host_metrics.cheap_clone(),
-            )?;
+            let host = this.new_host(logger.cheap_clone(), ds, module_bytes)?;
             this.hosts.push(Arc::new(host));
         }
 
@@ -99,8 +97,6 @@ where
         logger: Logger,
         data_source: DataSource<C>,
         module_bytes: &Arc<Vec<u8>>,
-        templates: Arc<Vec<DataSourceTemplate<C>>>,
-        host_metrics: Arc<HostMetrics>,
     ) -> Result<T::Host, Error> {
         let mapping_request_sender = {
             let module_hash = tiny_keccak::keccak256(module_bytes.as_ref());
@@ -111,7 +107,7 @@ where
                     module_bytes.as_ref(),
                     logger,
                     self.subgraph_id.clone(),
-                    host_metrics.clone(),
+                    self.host_metrics.cheap_clone(),
                 )?;
                 self.module_cache.insert(module_hash, sender.clone());
                 sender
@@ -121,9 +117,9 @@ where
             self.network.clone(),
             self.subgraph_id.clone(),
             data_source,
-            templates,
+            self.templates.cheap_clone(),
             mapping_request_sender,
-            host_metrics,
+            self.host_metrics.cheap_clone(),
         )
     }
 
@@ -157,8 +153,6 @@ where
         &mut self,
         logger: &Logger,
         data_source: DataSource<C>,
-        templates: Arc<Vec<DataSourceTemplate<C>>>,
-        metrics: Arc<HostMetrics>,
     ) -> Result<Option<Arc<T::Host>>, Error> {
         // Protect against creating more than the allowed maximum number of data sources
         if let Some(max_data_sources) = ENV_VARS.subgraph_max_data_sources {
@@ -182,13 +176,7 @@ where
             Some(ref module_bytes) => module_bytes.cheap_clone(),
         };
 
-        let host = Arc::new(self.new_host(
-            logger.clone(),
-            data_source,
-            &module_bytes,
-            templates,
-            metrics,
-        )?);
+        let host = Arc::new(self.new_host(logger.clone(), data_source, &module_bytes)?);
 
         Ok(if self.hosts.contains(&host) {
             None
diff --git a/core/src/subgraph/runner.rs b/core/src/subgraph/runner.rs
index e17479ba90a..19125bedf88 100644
--- a/core/src/subgraph/runner.rs
+++ b/core/src/subgraph/runner.rs
@@ -483,12 +483,10 @@ where
             let data_source = DataSource::try_from(info)?;
 
             // Try to create a runtime host for the data source
-            let host = self.ctx.instance.add_dynamic_data_source(
-                &self.logger,
-                data_source.clone(),
-                self.inputs.templates.clone(),
-                self.metrics.host.clone(),
-            )?;
+            let host = self
+                .ctx
+                .instance
+                .add_dynamic_data_source(&self.logger, data_source.clone())?;
 
             match host {
                 Some(host) => {

From 69363764cda8e36d0d32999a6c3ac3a75235faf3 Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Fri, 12 Aug 2022 16:35:01 +0100
Subject: [PATCH 17/33] core: Move poi_version out of the instance

---
 core/src/subgraph/inputs.rs           |  7 ++++++-
 core/src/subgraph/instance.rs         | 15 +--------------
 core/src/subgraph/instance_manager.rs | 10 ++++++++++
 core/src/subgraph/runner.rs           |  4 ++--
 4 files changed, 19 insertions(+), 17 deletions(-)

diff --git a/core/src/subgraph/inputs.rs b/core/src/subgraph/inputs.rs
index f17a0ba46eb..191dc69cbf4 100644
--- a/core/src/subgraph/inputs.rs
+++ b/core/src/subgraph/inputs.rs
@@ -1,6 +1,9 @@
 use graph::{
     blockchain::{Blockchain, TriggersAdapter},
-    components::store::{DeploymentLocator, SubgraphFork, WritableStore},
+    components::{
+        store::{DeploymentLocator, SubgraphFork, WritableStore},
+        subgraph::ProofOfIndexingVersion,
+    },
     data::subgraph::{SubgraphFeature, UnifiedMappingApiVersion},
     data_source::DataSourceTemplate,
     prelude::BlockNumber,
@@ -20,6 +23,8 @@ pub struct IndexingInputs<C: Blockchain> {
     pub templates: Arc<Vec<DataSourceTemplate<C>>>,
     pub unified_api_version: UnifiedMappingApiVersion,
     pub static_filters: bool,
+    pub poi_version: ProofOfIndexingVersion,
+    pub network: String,
 
     // Correspondence between data source or template position in the manifest and name.
     pub manifest_idx_and_name: Vec<(u32, String)>,
diff --git a/core/src/subgraph/instance.rs b/core/src/subgraph/instance.rs
index e7b749db8a9..05f14698431 100644
--- a/core/src/subgraph/instance.rs
+++ b/core/src/subgraph/instance.rs
@@ -3,9 +3,8 @@ use graph::{
     blockchain::Blockchain,
     components::{
         store::SubgraphFork,
-        subgraph::{MappingError, ProofOfIndexingVersion, SharedProofOfIndexing},
+        subgraph::{MappingError, SharedProofOfIndexing},
     },
-    data::subgraph::SPEC_VERSION_0_0_6,
     data_source::{DataSource, DataSourceTemplate, TriggerData},
     prelude::*,
 };
@@ -16,7 +15,6 @@ use super::context::OffchainMonitor;
 pub struct SubgraphInstance<C: Blockchain, T: RuntimeHostBuilder<C>> {
     subgraph_id: DeploymentHash,
     network: String,
-    pub poi_version: ProofOfIndexingVersion,
     host_builder: T,
     pub(crate) trigger_processor: Box<dyn TriggerProcessor<C, T>>,
     templates: Arc<Vec<DataSourceTemplate<C>>>,
@@ -50,19 +48,12 @@ where
         let network = manifest.network_name();
         let templates = Arc::new(manifest.templates);
 
-        let poi_version = if manifest.spec_version.ge(&SPEC_VERSION_0_0_6) {
-            ProofOfIndexingVersion::Fast
-        } else {
-            ProofOfIndexingVersion::Legacy
-        };
-
         let mut this = SubgraphInstance {
             host_builder,
             subgraph_id,
             network,
             hosts: Vec::new(),
             module_cache: HashMap::new(),
-            poi_version,
             trigger_processor,
             templates,
             host_metrics,
@@ -198,8 +189,4 @@ where
             self.hosts.pop();
         }
     }
-
-    pub(crate) fn network(&self) -> &str {
-        &self.network
-    }
 }
diff --git a/core/src/subgraph/instance_manager.rs b/core/src/subgraph/instance_manager.rs
index 14feb8d8812..1ddae1abd07 100644
--- a/core/src/subgraph/instance_manager.rs
+++ b/core/src/subgraph/instance_manager.rs
@@ -8,6 +8,8 @@ use graph::blockchain::block_stream::BlockStreamMetrics;
 use graph::blockchain::Blockchain;
 use graph::blockchain::NodeCapabilities;
 use graph::blockchain::{BlockchainKind, TriggerFilter};
+use graph::components::subgraph::ProofOfIndexingVersion;
+use graph::data::subgraph::SPEC_VERSION_0_0_6;
 use graph::prelude::{SubgraphInstanceManager as SubgraphInstanceManagerTrait, *};
 use graph::{blockchain::BlockchainMap, components::store::DeploymentLocator};
 use graph_runtime_wasm::module::ToAscPtr;
@@ -319,6 +321,12 @@ impl<S: SubgraphStore> SubgraphInstanceManager<S> {
 
         let features = manifest.features.clone();
         let unified_api_version = manifest.unified_mapping_api_version()?;
+        let poi_version = if manifest.spec_version.ge(&SPEC_VERSION_0_0_6) {
+            ProofOfIndexingVersion::Fast
+        } else {
+            ProofOfIndexingVersion::Legacy
+        };
+
         let instance = SubgraphInstance::from_manifest(
             &logger,
             manifest,
@@ -341,6 +349,8 @@ impl<S: SubgraphStore> SubgraphInstanceManager<S> {
             unified_api_version,
             static_filters: self.static_filters,
             manifest_idx_and_name,
+            poi_version,
+            network,
         };
 
         // The subgraph state tracks the state of the subgraph instance over time
diff --git a/core/src/subgraph/runner.rs b/core/src/subgraph/runner.rs
index 19125bedf88..5e26db71c6f 100644
--- a/core/src/subgraph/runner.rs
+++ b/core/src/subgraph/runner.rs
@@ -167,14 +167,14 @@ where
         let proof_of_indexing = if self.inputs.store.supports_proof_of_indexing().await? {
             Some(Arc::new(AtomicRefCell::new(ProofOfIndexing::new(
                 block_ptr.number,
-                self.ctx.instance.poi_version.clone(),
+                self.inputs.poi_version,
             ))))
         } else {
             None
         };
 
         // Causality region for onchain triggers.
-        let causality_region = CausalityRegion::from_network(self.ctx.instance.network());
+        let causality_region = CausalityRegion::from_network(&self.inputs.network);
 
         // Process events one after the other, passing in entity operations
         // collected previously to every new event being processed

From 801eb36daefcd163072e2bc2d3e98c98760527c1 Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Mon, 15 Aug 2022 11:47:05 +0100
Subject: [PATCH 18/33] core: Move `mod instance` under `mod context`

This should clarify the SubgraphInstance responsiblity of keeping
track of the hosts.
---
 core/src/subgraph/context.rs                | 111 ++++++++++++++++++--
 core/src/subgraph/{ => context}/instance.rs |  49 ++-------
 core/src/subgraph/instance_manager.rs       |  11 +-
 core/src/subgraph/mod.rs                    |   2 -
 core/src/subgraph/runner.rs                 |  11 +-
 5 files changed, 119 insertions(+), 65 deletions(-)
 rename core/src/subgraph/{ => context}/instance.rs (78%)

diff --git a/core/src/subgraph/context.rs b/core/src/subgraph/context.rs
index d5ca178a95c..4ce55ead15b 100644
--- a/core/src/subgraph/context.rs
+++ b/core/src/subgraph/context.rs
@@ -1,34 +1,125 @@
-use crate::{
-    polling_monitor::{
-        ipfs_service::IpfsService, spawn_monitor, PollingMonitor, PollingMonitorMetrics,
-    },
-    subgraph::SubgraphInstance,
+pub mod instance;
+
+use crate::polling_monitor::{
+    ipfs_service::IpfsService, spawn_monitor, PollingMonitor, PollingMonitorMetrics,
 };
 use anyhow::{self, Error};
 use bytes::Bytes;
 use cid::Cid;
 use graph::{
     blockchain::Blockchain,
-    components::store::DeploymentId,
-    data_source::offchain,
-    prelude::{CancelGuard, DeploymentHash, MetricsRegistry, RuntimeHostBuilder},
+    components::{
+        store::{DeploymentId, SubgraphFork},
+        subgraph::{MappingError, SharedProofOfIndexing},
+    },
+    data_source::{offchain, DataSource, TriggerData},
+    prelude::{
+        BlockNumber, BlockState, CancelGuard, DeploymentHash, MetricsRegistry, RuntimeHostBuilder,
+        SubgraphInstanceMetrics, TriggerProcessor,
+    },
     slog::Logger,
     tokio::sync::mpsc,
 };
 use std::collections::HashMap;
 use std::sync::{Arc, RwLock};
 
+use self::instance::SubgraphInstance;
+
 pub type SharedInstanceKeepAliveMap = Arc<RwLock<HashMap<DeploymentId, CancelGuard>>>;
 
-pub struct IndexingContext<C, T>
+pub(crate) struct IndexingContext<C, T>
 where
     T: RuntimeHostBuilder<C>,
     C: Blockchain,
 {
-    pub instance: SubgraphInstance<C, T>,
+    instance: SubgraphInstance<C, T>,
     pub instances: SharedInstanceKeepAliveMap,
     pub filter: C::TriggerFilter,
     pub(crate) offchain_monitor: OffchainMonitor,
+    trigger_processor: Box<dyn TriggerProcessor<C, T>>,
+}
+
+impl<C: Blockchain, T: RuntimeHostBuilder<C>> IndexingContext<C, T> {
+    pub fn new(
+        instance: SubgraphInstance<C, T>,
+        instances: SharedInstanceKeepAliveMap,
+        filter: C::TriggerFilter,
+        offchain_monitor: OffchainMonitor,
+        trigger_processor: Box<dyn TriggerProcessor<C, T>>,
+    ) -> Self {
+        Self {
+            instance,
+            instances,
+            filter,
+            offchain_monitor,
+            trigger_processor,
+        }
+    }
+
+    pub async fn process_trigger(
+        &self,
+        logger: &Logger,
+        block: &Arc<C::Block>,
+        trigger: &TriggerData<C>,
+        state: BlockState<C>,
+        proof_of_indexing: &SharedProofOfIndexing,
+        causality_region: &str,
+        debug_fork: &Option<Arc<dyn SubgraphFork>>,
+        subgraph_metrics: &Arc<SubgraphInstanceMetrics>,
+    ) -> Result<BlockState<C>, MappingError> {
+        self.trigger_processor
+            .process_trigger(
+                logger,
+                &self.instance.hosts(),
+                block,
+                trigger,
+                state,
+                proof_of_indexing,
+                causality_region,
+                debug_fork,
+                subgraph_metrics,
+            )
+            .await
+    }
+
+    pub async fn process_trigger_in_hosts(
+        &self,
+        logger: &Logger,
+        hosts: &[Arc<T::Host>],
+        block: &Arc<C::Block>,
+        trigger: &TriggerData<C>,
+        state: BlockState<C>,
+        proof_of_indexing: &SharedProofOfIndexing,
+        causality_region: &str,
+        debug_fork: &Option<Arc<dyn SubgraphFork>>,
+        subgraph_metrics: &Arc<SubgraphInstanceMetrics>,
+    ) -> Result<BlockState<C>, MappingError> {
+        self.trigger_processor
+            .process_trigger(
+                logger,
+                hosts,
+                block,
+                trigger,
+                state,
+                proof_of_indexing,
+                causality_region,
+                debug_fork,
+                subgraph_metrics,
+            )
+            .await
+    }
+
+    pub fn revert_data_sources(&mut self, reverted_block: BlockNumber) {
+        self.instance.revert_data_sources(reverted_block)
+    }
+
+    pub fn add_dynamic_data_source(
+        &mut self,
+        logger: &Logger,
+        data_source: DataSource<C>,
+    ) -> Result<Option<Arc<T::Host>>, Error> {
+        self.instance.add_dynamic_data_source(logger, data_source)
+    }
 }
 
 pub(crate) struct OffchainMonitor {
diff --git a/core/src/subgraph/instance.rs b/core/src/subgraph/context/instance.rs
similarity index 78%
rename from core/src/subgraph/instance.rs
rename to core/src/subgraph/context/instance.rs
index 05f14698431..83fb55913dd 100644
--- a/core/src/subgraph/instance.rs
+++ b/core/src/subgraph/context/instance.rs
@@ -1,22 +1,17 @@
 use futures01::sync::mpsc::Sender;
 use graph::{
     blockchain::Blockchain,
-    components::{
-        store::SubgraphFork,
-        subgraph::{MappingError, SharedProofOfIndexing},
-    },
-    data_source::{DataSource, DataSourceTemplate, TriggerData},
+    data_source::{DataSource, DataSourceTemplate},
     prelude::*,
 };
 use std::collections::HashMap;
 
-use super::context::OffchainMonitor;
+use super::OffchainMonitor;
 
-pub struct SubgraphInstance<C: Blockchain, T: RuntimeHostBuilder<C>> {
+pub(crate) struct SubgraphInstance<C: Blockchain, T: RuntimeHostBuilder<C>> {
     subgraph_id: DeploymentHash,
     network: String,
     host_builder: T,
-    pub(crate) trigger_processor: Box<dyn TriggerProcessor<C, T>>,
     templates: Arc<Vec<DataSourceTemplate<C>>>,
     host_metrics: Arc<HostMetrics>,
 
@@ -36,11 +31,10 @@ where
     C: Blockchain,
     T: RuntimeHostBuilder<C>,
 {
-    pub(crate) fn from_manifest(
+    pub fn from_manifest(
         logger: &Logger,
         manifest: SubgraphManifest<C>,
         host_builder: T,
-        trigger_processor: Box<dyn TriggerProcessor<C, T>>,
         host_metrics: Arc<HostMetrics>,
         offchain_monitor: &mut OffchainMonitor,
     ) -> Result<Self, Error> {
@@ -54,7 +48,6 @@ where
             network,
             hosts: Vec::new(),
             module_cache: HashMap::new(),
-            trigger_processor,
             templates,
             host_metrics,
         };
@@ -114,33 +107,7 @@ where
         )
     }
 
-    pub(crate) async fn process_trigger(
-        &self,
-        logger: &Logger,
-        block: &Arc<C::Block>,
-        trigger: &TriggerData<C>,
-        state: BlockState<C>,
-        proof_of_indexing: &SharedProofOfIndexing,
-        causality_region: &str,
-        debug_fork: &Option<Arc<dyn SubgraphFork>>,
-        subgraph_metrics: &Arc<SubgraphInstanceMetrics>,
-    ) -> Result<BlockState<C>, MappingError> {
-        self.trigger_processor
-            .process_trigger(
-                logger,
-                &self.hosts,
-                block,
-                trigger,
-                state,
-                proof_of_indexing,
-                causality_region,
-                debug_fork,
-                subgraph_metrics,
-            )
-            .await
-    }
-
-    pub(crate) fn add_dynamic_data_source(
+    pub(super) fn add_dynamic_data_source(
         &mut self,
         logger: &Logger,
         data_source: DataSource<C>,
@@ -177,7 +144,7 @@ where
         })
     }
 
-    pub(crate) fn revert_data_sources(&mut self, reverted_block: BlockNumber) {
+    pub(super) fn revert_data_sources(&mut self, reverted_block: BlockNumber) {
         // `hosts` is ordered by the creation block.
         // See also 8f1bca33-d3b7-4035-affc-fd6161a12448.
         while self
@@ -189,4 +156,8 @@ where
             self.hosts.pop();
         }
     }
+
+    pub(super) fn hosts(&self) -> &[Arc<T::Host>] {
+        &self.hosts
+    }
 }
diff --git a/core/src/subgraph/instance_manager.rs b/core/src/subgraph/instance_manager.rs
index 1ddae1abd07..6085c4841e7 100644
--- a/core/src/subgraph/instance_manager.rs
+++ b/core/src/subgraph/instance_manager.rs
@@ -3,7 +3,6 @@ use crate::subgraph::context::{IndexingContext, SharedInstanceKeepAliveMap};
 use crate::subgraph::inputs::IndexingInputs;
 use crate::subgraph::loader::load_dynamic_data_sources;
 use crate::subgraph::runner::SubgraphRunner;
-use crate::subgraph::SubgraphInstance;
 use graph::blockchain::block_stream::BlockStreamMetrics;
 use graph::blockchain::Blockchain;
 use graph::blockchain::NodeCapabilities;
@@ -327,11 +326,10 @@ impl<S: SubgraphStore> SubgraphInstanceManager<S> {
             ProofOfIndexingVersion::Legacy
         };
 
-        let instance = SubgraphInstance::from_manifest(
+        let instance = super::context::instance::SubgraphInstance::from_manifest(
             &logger,
             manifest,
             host_builder,
-            tp,
             host_metrics.clone(),
             &mut offchain_monitor,
         )?;
@@ -354,12 +352,13 @@ impl<S: SubgraphStore> SubgraphInstanceManager<S> {
         };
 
         // The subgraph state tracks the state of the subgraph instance over time
-        let ctx = IndexingContext {
+        let ctx = IndexingContext::new(
             instance,
-            instances: self.instances.cheap_clone(),
+            self.instances.cheap_clone(),
             filter,
             offchain_monitor,
-        };
+            tp,
+        );
 
         let metrics = RunnerMetrics {
             subgraph: subgraph_metrics,
diff --git a/core/src/subgraph/mod.rs b/core/src/subgraph/mod.rs
index ce0978d13da..490c45f791c 100644
--- a/core/src/subgraph/mod.rs
+++ b/core/src/subgraph/mod.rs
@@ -1,7 +1,6 @@
 mod context;
 mod error;
 mod inputs;
-mod instance;
 mod instance_manager;
 mod loader;
 mod provider;
@@ -11,7 +10,6 @@ mod state;
 mod stream;
 mod trigger_processor;
 
-pub use self::instance::SubgraphInstance;
 pub use self::instance_manager::SubgraphInstanceManager;
 pub use self::provider::SubgraphAssignmentProvider;
 pub use self::registrar::SubgraphRegistrar;
diff --git a/core/src/subgraph/runner.rs b/core/src/subgraph/runner.rs
index 5e26db71c6f..db638c5c794 100644
--- a/core/src/subgraph/runner.rs
+++ b/core/src/subgraph/runner.rs
@@ -27,7 +27,7 @@ const MINUTE: Duration = Duration::from_secs(60);
 
 const SKIP_PTR_UPDATES_THRESHOLD: Duration = Duration::from_secs(60 * 5);
 
-pub struct SubgraphRunner<C, T>
+pub(crate) struct SubgraphRunner<C, T>
 where
     C: Blockchain,
     T: RuntimeHostBuilder<C>,
@@ -267,9 +267,7 @@ where
             for trigger in triggers {
                 block_state = self
                     .ctx
-                    .instance
-                    .trigger_processor
-                    .process_trigger(
+                    .process_trigger_in_hosts(
                         &logger,
                         &runtime_hosts,
                         &block,
@@ -448,7 +446,6 @@ where
         for trigger in triggers {
             block_state = self
                 .ctx
-                .instance
                 .process_trigger(
                     &self.logger,
                     block,
@@ -485,7 +482,6 @@ where
             // Try to create a runtime host for the data source
             let host = self
                 .ctx
-                .instance
                 .add_dynamic_data_source(&self.logger, data_source.clone())?;
 
             match host {
@@ -588,7 +584,6 @@ where
             let block = Arc::default();
             block_state = self
                 .ctx
-                .instance
                 .process_trigger(
                     &self.logger,
                     &block,
@@ -896,7 +891,7 @@ where
         // will be broader than necessary. This is not ideal for performance, but is not
         // incorrect since we will discard triggers that match the filters but do not
         // match any data sources.
-        self.ctx.instance.revert_data_sources(subgraph_ptr.number);
+        self.ctx.revert_data_sources(subgraph_ptr.number);
         self.state.entity_lfu_cache = LfuCache::new();
 
         Ok(Action::Continue)

From 8930c5ca11ad05672cfefde163ca0074ee41913a Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Mon, 15 Aug 2022 13:29:50 +0100
Subject: [PATCH 19/33] core: Refactor OffchainMonitor::add_data_source

---
 core/src/subgraph/context.rs          | 24 ++++++++++++++++++------
 core/src/subgraph/context/instance.rs |  7 +++++--
 core/src/subgraph/runner.rs           | 13 +------------
 3 files changed, 24 insertions(+), 20 deletions(-)

diff --git a/core/src/subgraph/context.rs b/core/src/subgraph/context.rs
index 4ce55ead15b..3cbd254d0b4 100644
--- a/core/src/subgraph/context.rs
+++ b/core/src/subgraph/context.rs
@@ -27,6 +27,11 @@ use self::instance::SubgraphInstance;
 
 pub type SharedInstanceKeepAliveMap = Arc<RwLock<HashMap<DeploymentId, CancelGuard>>>;
 
+// The context keeps track of mutable in-memory state that is retained across blocks.
+//
+// Currently most of the changes are applied in `runner.rs`, but ideally more of that would be
+// refactored into the context so it wouldn't need `pub` fields. The entity cache should probaby
+// also be moved here.
 pub(crate) struct IndexingContext<C, T>
 where
     T: RuntimeHostBuilder<C>,
@@ -35,7 +40,7 @@ where
     instance: SubgraphInstance<C, T>,
     pub instances: SharedInstanceKeepAliveMap,
     pub filter: C::TriggerFilter,
-    pub(crate) offchain_monitor: OffchainMonitor,
+    pub offchain_monitor: OffchainMonitor,
     trigger_processor: Box<dyn TriggerProcessor<C, T>>,
 }
 
@@ -109,6 +114,13 @@ impl<C: Blockchain, T: RuntimeHostBuilder<C>> IndexingContext<C, T> {
             .await
     }
 
+    // Removes data sources hosts with a creation block greater or equal to `reverted_block`, so
+    // that they are no longer candidates for `process_trigger`.
+    //
+    // This does not currently affect the `offchain_monitor` or the `filter`, so they will continue
+    // to include data sources that have been reverted. This is not ideal for performance, but it
+    // does not affect correctness since triggers that have no matching host will be ignored by
+    // `process_trigger`.
     pub fn revert_data_sources(&mut self, reverted_block: BlockNumber) {
         self.instance.revert_data_sources(reverted_block)
     }
@@ -118,6 +130,9 @@ impl<C: Blockchain, T: RuntimeHostBuilder<C>> IndexingContext<C, T> {
         logger: &Logger,
         data_source: DataSource<C>,
     ) -> Result<Option<Arc<T::Host>>, Error> {
+        if let DataSource::Offchain(ds) = &data_source {
+            self.offchain_monitor.add_source(&ds.source)?;
+        }
         self.instance.add_dynamic_data_source(logger, data_source)
     }
 }
@@ -125,7 +140,6 @@ impl<C: Blockchain, T: RuntimeHostBuilder<C>> IndexingContext<C, T> {
 pub(crate) struct OffchainMonitor {
     ipfs_monitor: PollingMonitor<Cid>,
     ipfs_monitor_rx: mpsc::Receiver<(Cid, Bytes)>,
-    data_sources: Vec<offchain::DataSource>,
 }
 
 impl OffchainMonitor {
@@ -145,15 +159,13 @@ impl OffchainMonitor {
         Self {
             ipfs_monitor,
             ipfs_monitor_rx,
-            data_sources: Vec::new(),
         }
     }
 
-    pub fn add_data_source(&mut self, ds: offchain::DataSource) -> Result<(), Error> {
-        match ds.source {
+    fn add_source(&mut self, source: &offchain::Source) -> Result<(), Error> {
+        match source {
             offchain::Source::Ipfs(cid) => self.ipfs_monitor.monitor(cid.clone()),
         };
-        self.data_sources.push(ds);
         Ok(())
     }
 
diff --git a/core/src/subgraph/context/instance.rs b/core/src/subgraph/context/instance.rs
index 83fb55913dd..a9e3ade7778 100644
--- a/core/src/subgraph/context/instance.rs
+++ b/core/src/subgraph/context/instance.rs
@@ -56,15 +56,18 @@ where
         // we use the same order here as in the subgraph manifest to make the
         // event processing behavior predictable
         for ds in manifest.data_sources {
+            // TODO: This is duplicating code from `IndexingContext::add_dynamic_data_source` and
+            // `SubgraphInstance::add_dynamic_data_source`. Ideally this should be refactored into
+            // `IndexingContext`.
+
             let runtime = ds.runtime();
             let module_bytes = match runtime {
                 None => continue,
                 Some(ref module_bytes) => module_bytes,
             };
 
-            // Create services for static offchain data sources
             if let DataSource::Offchain(ds) = &ds {
-                offchain_monitor.add_data_source(ds.clone())?;
+                offchain_monitor.add_source(&ds.source)?;
             }
 
             let host = this.new_host(logger.cheap_clone(), ds, module_bytes)?;
diff --git a/core/src/subgraph/runner.rs b/core/src/subgraph/runner.rs
index db638c5c794..9de12722ca1 100644
--- a/core/src/subgraph/runner.rs
+++ b/core/src/subgraph/runner.rs
@@ -226,12 +226,6 @@ where
             let (data_sources, runtime_hosts) =
                 self.create_dynamic_data_sources(block_state.drain_created_data_sources())?;
 
-            for ds in &data_sources {
-                if let DataSource::Offchain(ds) = ds {
-                    self.ctx.offchain_monitor.add_data_source(ds.clone())?
-                }
-            }
-
             let filter = C::TriggerFilter::from_data_sources(
                 data_sources.iter().filter_map(DataSource::as_onchain),
             );
@@ -884,13 +878,8 @@ where
             .set(subgraph_ptr.number as f64);
 
         // Revert the in-memory state:
-        // - Remove hosts for reverted dynamic data sources.
+        // - Revert any dynamic data sources.
         // - Clear the entity cache.
-        //
-        // Note that we do not currently revert the filters, which means the filters
-        // will be broader than necessary. This is not ideal for performance, but is not
-        // incorrect since we will discard triggers that match the filters but do not
-        // match any data sources.
         self.ctx.revert_data_sources(subgraph_ptr.number);
         self.state.entity_lfu_cache = LfuCache::new();
 

From 9338787a4f8e212c92fe56c901b835437a37fc8d Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Mon, 15 Aug 2022 14:33:37 +0100
Subject: [PATCH 20/33] offchain: Better handling of duplicates

---
 core/src/subgraph/context.rs | 12 +++++++++---
 graph/src/data_source/mod.rs | 12 +++++++++++-
 2 files changed, 20 insertions(+), 4 deletions(-)

diff --git a/core/src/subgraph/context.rs b/core/src/subgraph/context.rs
index 3cbd254d0b4..e3a3354b25d 100644
--- a/core/src/subgraph/context.rs
+++ b/core/src/subgraph/context.rs
@@ -130,10 +130,16 @@ impl<C: Blockchain, T: RuntimeHostBuilder<C>> IndexingContext<C, T> {
         logger: &Logger,
         data_source: DataSource<C>,
     ) -> Result<Option<Arc<T::Host>>, Error> {
-        if let DataSource::Offchain(ds) = &data_source {
-            self.offchain_monitor.add_source(&ds.source)?;
+        let source = data_source.as_offchain().map(|ds| ds.source.clone());
+        let host = self.instance.add_dynamic_data_source(logger, data_source)?;
+
+        if host.is_some() {
+            if let Some(source) = source {
+                self.offchain_monitor.add_source(&source)?;
+            }
         }
-        self.instance.add_dynamic_data_source(logger, data_source)
+
+        Ok(host)
     }
 }
 
diff --git a/graph/src/data_source/mod.rs b/graph/src/data_source/mod.rs
index e166d5f08af..cbd6b56c1b7 100644
--- a/graph/src/data_source/mod.rs
+++ b/graph/src/data_source/mod.rs
@@ -48,6 +48,13 @@ impl<C: Blockchain> DataSource<C> {
         }
     }
 
+    pub fn as_offchain(&self) -> Option<&offchain::DataSource> {
+        match self {
+            Self::Onchain(_) => None,
+            Self::Offchain(ds) => Some(&ds),
+        }
+    }
+
     pub fn address(&self) -> Option<Vec<u8>> {
         match self {
             Self::Onchain(ds) => ds.address().map(ToOwned::to_owned),
@@ -119,7 +126,10 @@ impl<C: Blockchain> DataSource<C> {
         match (self, other) {
             (Self::Onchain(a), Self::Onchain(b)) => a.is_duplicate_of(b),
             (Self::Offchain(a), Self::Offchain(b)) => {
-                a.kind == b.kind && a.name == b.name && a.source == b.source
+                a.kind == b.kind
+                    && a.name == b.name
+                    && a.source == b.source
+                    && a.context == b.context
             }
             _ => false,
         }

From acdc5a70ddb2bea06d91904cb8703c50b252f45e Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Mon, 15 Aug 2022 14:42:21 +0100
Subject: [PATCH 21/33] offchain: Bump max ipfs concurrent requests to 100

---
 docs/environment-variables.md | 2 +-
 graph/src/env/mappings.rs     | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/docs/environment-variables.md b/docs/environment-variables.md
index da827579dfc..a969ed0c024 100644
--- a/docs/environment-variables.md
+++ b/docs/environment-variables.md
@@ -78,7 +78,7 @@ those.
   may use (in bytes, defaults to 256MB).
 - `GRAPH_MAX_IPFS_CACHE_SIZE`: maximum number of files cached (defaults to 50).
 - `GRAPH_MAX_IPFS_CACHE_FILE_SIZE`: maximum size of each cached file (in bytes, defaults to 1MiB).
-- `GRAPH_MAX_IPFS_CONCURRENT_REQUESTS`: maximum concurrent requests to IPFS from file data sources (defaults to 10).
+- `GRAPH_MAX_IPFS_CONCURRENT_REQUESTS`: maximum concurrent requests to IPFS from file data sources (defaults to 100).
 
 ## GraphQL
 
diff --git a/graph/src/env/mappings.rs b/graph/src/env/mappings.rs
index f14086679df..82507da569b 100644
--- a/graph/src/env/mappings.rs
+++ b/graph/src/env/mappings.rs
@@ -106,7 +106,7 @@ pub struct InnerMappingHandlers {
     max_ipfs_map_file_size: WithDefaultUsize<usize, { 256 * 1024 * 1024 }>,
     #[envconfig(from = "GRAPH_MAX_IPFS_FILE_BYTES", default = "")]
     max_ipfs_file_bytes: WithDefaultUsize<usize, { 256 * 1024 * 1024 }>,
-    #[envconfig(from = "GRAPH_MAX_IPFS_CONCURRENT_REQUESTS", default = "10")]
+    #[envconfig(from = "GRAPH_MAX_IPFS_CONCURRENT_REQUESTS", default = "100")]
     max_ipfs_concurrent_requests: u16,
     #[envconfig(from = "GRAPH_ALLOW_NON_DETERMINISTIC_IPFS", default = "false")]
     allow_non_deterministic_ipfs: EnvVarBoolean,

From ea0311e5ae87d4d4c86aa18da54921772ba4a454 Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Mon, 15 Aug 2022 11:47:05 +0100
Subject: [PATCH 22/33] refactor: Expose RuntimeHost data source

---
 core/src/subgraph/context.rs          | 25 ++++++++++++-------------
 graph/src/components/subgraph/host.rs |  2 ++
 runtime/wasm/src/host.rs              |  4 ++++
 3 files changed, 18 insertions(+), 13 deletions(-)

diff --git a/core/src/subgraph/context.rs b/core/src/subgraph/context.rs
index e3a3354b25d..653d6c8875f 100644
--- a/core/src/subgraph/context.rs
+++ b/core/src/subgraph/context.rs
@@ -72,19 +72,18 @@ impl<C: Blockchain, T: RuntimeHostBuilder<C>> IndexingContext<C, T> {
         debug_fork: &Option<Arc<dyn SubgraphFork>>,
         subgraph_metrics: &Arc<SubgraphInstanceMetrics>,
     ) -> Result<BlockState<C>, MappingError> {
-        self.trigger_processor
-            .process_trigger(
-                logger,
-                &self.instance.hosts(),
-                block,
-                trigger,
-                state,
-                proof_of_indexing,
-                causality_region,
-                debug_fork,
-                subgraph_metrics,
-            )
-            .await
+        self.process_trigger_in_hosts(
+            logger,
+            &self.instance.hosts(),
+            block,
+            trigger,
+            state,
+            proof_of_indexing,
+            causality_region,
+            debug_fork,
+            subgraph_metrics,
+        )
+        .await
     }
 
     pub async fn process_trigger_in_hosts(
diff --git a/graph/src/components/subgraph/host.rs b/graph/src/components/subgraph/host.rs
index fed8d99efbe..27d7d4ca56e 100644
--- a/graph/src/components/subgraph/host.rs
+++ b/graph/src/components/subgraph/host.rs
@@ -46,6 +46,8 @@ impl MappingError {
 /// Common trait for runtime host implementations.
 #[async_trait]
 pub trait RuntimeHost<C: Blockchain>: Send + Sync + 'static {
+    fn data_source(&self) -> &DataSource<C>;
+
     fn match_and_decode(
         &self,
         trigger: &TriggerData<C>,
diff --git a/runtime/wasm/src/host.rs b/runtime/wasm/src/host.rs
index 8e26f98e6db..e6423a572bb 100644
--- a/runtime/wasm/src/host.rs
+++ b/runtime/wasm/src/host.rs
@@ -218,6 +218,10 @@ where
 
 #[async_trait]
 impl<C: Blockchain> RuntimeHostTrait<C> for RuntimeHost<C> {
+    fn data_source(&self) -> &DataSource<C> {
+        &self.data_source
+    }
+
     fn match_and_decode(
         &self,
         trigger: &TriggerData<C>,

From a95454fa398401cf36179efaaeaaa76099612bb8 Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Mon, 15 Aug 2022 11:47:05 +0100
Subject: [PATCH 23/33] offchain: Remove dses that have been processed

---
 core/src/subgraph/runner.rs               | 17 ++++-----
 core/src/subgraph/trigger_processor.rs    |  7 ++++
 graph/src/components/store/traits.rs      |  1 +
 graph/src/components/subgraph/instance.rs |  8 ++++-
 graph/src/data_source/mod.rs              |  6 ++--
 graph/tests/entity_cache.rs               |  1 +
 graphql/tests/query.rs                    |  1 -
 store/postgres/src/deployment_store.rs    |  3 ++
 store/postgres/src/dynds/mod.rs           | 18 ++++++++++
 store/postgres/src/dynds/private.rs       | 44 +++++++++++++++++++++++
 store/postgres/src/writable.rs            | 15 ++++++++
 store/postgres/tests/store.rs             |  2 ++
 store/test-store/src/store.rs             |  2 ++
 13 files changed, 111 insertions(+), 14 deletions(-)

diff --git a/core/src/subgraph/runner.rs b/core/src/subgraph/runner.rs
index 9de12722ca1..888cf9872a6 100644
--- a/core/src/subgraph/runner.rs
+++ b/core/src/subgraph/runner.rs
@@ -6,7 +6,7 @@ use crate::subgraph::stream::new_block_stream;
 use atomic_refcell::AtomicRefCell;
 use graph::blockchain::block_stream::{BlockStreamEvent, BlockWithTriggers, FirehoseCursor};
 use graph::blockchain::{Block, Blockchain, TriggerFilter as _};
-use graph::components::store::EntityKey;
+use graph::components::store::{EntityKey, StoredDynamicDataSource};
 use graph::components::{
     store::ModificationsAndCache,
     subgraph::{CausalityRegion, MappingError, ProofOfIndexing, SharedProofOfIndexing},
@@ -327,11 +327,10 @@ where
 
         // Check for offchain events and process them, including their entity modifications in the
         // set to be transacted.
-        {
-            let offchain_events = self.ctx.offchain_monitor.ready_offchain_events()?;
-            let offchain_mods = self.handle_offchain_triggers(offchain_events).await?;
-            mods.extend(offchain_mods);
-        }
+        let offchain_events = self.ctx.offchain_monitor.ready_offchain_events()?;
+        let (offchain_mods, offchain_to_remove) =
+            self.handle_offchain_triggers(offchain_events).await?;
+        mods.extend(offchain_mods);
 
         // Put the cache back in the state, asserting that the placeholder cache was not used.
         assert!(self.state.entity_lfu_cache.is_empty());
@@ -385,6 +384,7 @@ where
                 data_sources,
                 deterministic_errors,
                 self.inputs.manifest_idx_and_name.clone(),
+                offchain_to_remove,
             )
             .await
             .context("Failed to transact block operations")?;
@@ -564,7 +564,7 @@ where
     async fn handle_offchain_triggers(
         &mut self,
         triggers: Vec<offchain::TriggerData>,
-    ) -> Result<Vec<EntityModification>, Error> {
+    ) -> Result<(Vec<EntityModification>, Vec<StoredDynamicDataSource>), Error> {
         // TODO: Dont expose store with onchain entites
         let mut block_state =
             BlockState::<C>::new(self.inputs.store.cheap_clone(), LfuCache::new());
@@ -604,7 +604,8 @@ where
             "Attempted to create data source in offchain data source handler. This is not yet supported.",
         );
 
-        Ok(block_state.entity_cache.as_modifications()?.modifications)
+        let mods = block_state.entity_cache.as_modifications()?.modifications;
+        Ok((mods, block_state.offchain_to_remove))
     }
 }
 
diff --git a/core/src/subgraph/trigger_processor.rs b/core/src/subgraph/trigger_processor.rs
index 0c2a6cfee5f..f8d3e78122b 100644
--- a/core/src/subgraph/trigger_processor.rs
+++ b/core/src/subgraph/trigger_processor.rs
@@ -61,6 +61,13 @@ where
                 .await?;
             let elapsed = start.elapsed().as_secs_f64();
             subgraph_metrics.observe_trigger_processing_duration(elapsed);
+
+            if host.data_source().as_offchain().is_some() {
+                // Remove this offchain data source since it has just been processed.
+                state
+                    .offchain_to_remove
+                    .push(host.data_source().as_stored_dynamic_data_source());
+            }
         }
 
         if let Some(proof_of_indexing) = proof_of_indexing {
diff --git a/graph/src/components/store/traits.rs b/graph/src/components/store/traits.rs
index 99cebdd45a2..bc532e94ce0 100644
--- a/graph/src/components/store/traits.rs
+++ b/graph/src/components/store/traits.rs
@@ -211,6 +211,7 @@ pub trait WritableStore: Send + Sync + 'static {
         data_sources: Vec<StoredDynamicDataSource>,
         deterministic_errors: Vec<SubgraphError>,
         manifest_idx_and_name: Vec<(u32, String)>,
+        offchain_to_remove: Vec<StoredDynamicDataSource>,
     ) -> Result<(), StoreError>;
 
     /// Look up multiple entities as of the latest block. Returns a map of
diff --git a/graph/src/components/subgraph/instance.rs b/graph/src/components/subgraph/instance.rs
index f81744df9e8..c3686ef2d0e 100644
--- a/graph/src/components/subgraph/instance.rs
+++ b/graph/src/components/subgraph/instance.rs
@@ -1,6 +1,6 @@
 use crate::{
     blockchain::Blockchain,
-    components::store::{EntityKey, WritableStore},
+    components::store::{EntityKey, StoredDynamicDataSource, WritableStore},
     data::subgraph::schema::SubgraphError,
     data_source::DataSourceTemplate,
     prelude::*,
@@ -24,6 +24,9 @@ pub struct BlockState<C: Blockchain> {
     // Data sources created in the current handler.
     handler_created_data_sources: Vec<DataSourceTemplateInfo<C>>,
 
+    // offchain data sources to be removed because they've been processed.
+    pub offchain_to_remove: Vec<StoredDynamicDataSource>,
+
     // Marks whether a handler is currently executing.
     in_handler: bool,
 }
@@ -38,6 +41,7 @@ impl<C: Blockchain> BlockState<C> {
             deterministic_errors: Vec::new(),
             created_data_sources: Vec::new(),
             handler_created_data_sources: Vec::new(),
+            offchain_to_remove: Vec::new(),
             in_handler: false,
         }
     }
@@ -50,6 +54,7 @@ impl<C: Blockchain> BlockState<C> {
             deterministic_errors,
             created_data_sources,
             handler_created_data_sources,
+            offchain_to_remove,
             in_handler,
         } = self;
 
@@ -59,6 +64,7 @@ impl<C: Blockchain> BlockState<C> {
         }
         deterministic_errors.extend(other.deterministic_errors);
         entity_cache.extend(other.entity_cache);
+        offchain_to_remove.extend(other.offchain_to_remove);
     }
 
     pub fn has_errors(&self) -> bool {
diff --git a/graph/src/data_source/mod.rs b/graph/src/data_source/mod.rs
index cbd6b56c1b7..2d92a27a80c 100644
--- a/graph/src/data_source/mod.rs
+++ b/graph/src/data_source/mod.rs
@@ -126,10 +126,8 @@ impl<C: Blockchain> DataSource<C> {
         match (self, other) {
             (Self::Onchain(a), Self::Onchain(b)) => a.is_duplicate_of(b),
             (Self::Offchain(a), Self::Offchain(b)) => {
-                a.kind == b.kind
-                    && a.name == b.name
-                    && a.source == b.source
-                    && a.context == b.context
+                // See also: data-source-is-duplicate-of
+                a.manifest_idx == b.manifest_idx && a.source == b.source && a.context == b.context
             }
             _ => false,
         }
diff --git a/graph/tests/entity_cache.rs b/graph/tests/entity_cache.rs
index 1f0cf594e7b..7dd5b0c873c 100644
--- a/graph/tests/entity_cache.rs
+++ b/graph/tests/entity_cache.rs
@@ -108,6 +108,7 @@ impl WritableStore for MockStore {
         _: Vec<StoredDynamicDataSource>,
         _: Vec<SubgraphError>,
         _: Vec<(u32, String)>,
+        _: Vec<StoredDynamicDataSource>,
     ) -> Result<(), StoreError> {
         unimplemented!()
     }
diff --git a/graphql/tests/query.rs b/graphql/tests/query.rs
index d9720a7b881..854f8782dd6 100644
--- a/graphql/tests/query.rs
+++ b/graphql/tests/query.rs
@@ -741,7 +741,6 @@ fn mixed_parent_child_id() {
         };
         let data = extract_data!(result).unwrap();
         assert_eq!(data, exp);
-        dbg!(&data);
     })
 }
 
diff --git a/store/postgres/src/deployment_store.rs b/store/postgres/src/deployment_store.rs
index 0eac541e62a..d87f4d74194 100644
--- a/store/postgres/src/deployment_store.rs
+++ b/store/postgres/src/deployment_store.rs
@@ -987,6 +987,7 @@ impl DeploymentStore {
         data_sources: &[StoredDynamicDataSource],
         deterministic_errors: &[SubgraphError],
         manifest_idx_and_name: &[(u32, String)],
+        offchain_to_remove: &[StoredDynamicDataSource],
     ) -> Result<StoreEvent, StoreError> {
         let conn = {
             let _section = stopwatch.start_section("transact_blocks_get_conn");
@@ -1020,6 +1021,8 @@ impl DeploymentStore {
                 manifest_idx_and_name,
             )?;
 
+            dynds::remove_offchain(&conn, &site, offchain_to_remove)?;
+
             if !deterministic_errors.is_empty() {
                 deployment::insert_subgraph_errors(
                     &conn,
diff --git a/store/postgres/src/dynds/mod.rs b/store/postgres/src/dynds/mod.rs
index dbd4ebb5e7a..9a352705405 100644
--- a/store/postgres/src/dynds/mod.rs
+++ b/store/postgres/src/dynds/mod.rs
@@ -8,6 +8,7 @@ use diesel::PgConnection;
 use graph::{
     blockchain::BlockPtr,
     components::store::StoredDynamicDataSource,
+    constraint_violation,
     prelude::{BlockNumber, StoreError},
 };
 
@@ -56,3 +57,20 @@ pub(crate) fn revert(
         false => shared::revert(conn, &site.deployment, block),
     }
 }
+
+pub(crate) fn remove_offchain(
+    conn: &PgConnection,
+    site: &Site,
+    data_sources: &[StoredDynamicDataSource],
+) -> Result<(), StoreError> {
+    if data_sources.len() == 0 {
+        return Ok(());
+    }
+
+    match site.schema_version.private_data_sources() {
+        true => DataSourcesTable::new(site.namespace.clone()).remove_offchain(conn, data_sources),
+        false => Err(constraint_violation!(
+            "shared schema does not support data source offchain_found",
+        )),
+    }
+}
diff --git a/store/postgres/src/dynds/private.rs b/store/postgres/src/dynds/private.rs
index 6fefd34d4f3..981818d03ff 100644
--- a/store/postgres/src/dynds/private.rs
+++ b/store/postgres/src/dynds/private.rs
@@ -220,4 +220,48 @@ impl DataSourcesTable {
 
         Ok(count)
     }
+
+    // Remove offchain data sources by checking for equality. Their range will be set to the empty range.
+    pub(super) fn remove_offchain(
+        &self,
+        conn: &PgConnection,
+        data_sources: &[StoredDynamicDataSource],
+    ) -> Result<(), StoreError> {
+        for ds in data_sources {
+            let StoredDynamicDataSource {
+                manifest_idx,
+                param,
+                context,
+                creation_block,
+            } = ds;
+
+            let query = format!(
+                "update {} set block_range = 'empty'::int4range \
+                 where manifest_idx = $1
+                    and param is not distinct from $2
+                    and context is not distinct from $3
+                    and lower(block_range) is not distinct from $4",
+                self.qname
+            );
+
+            let count = sql_query(query)
+                .bind::<Integer, _>(*manifest_idx as i32)
+                .bind::<Nullable<Binary>, _>(param.as_ref().map(|p| &**p))
+                .bind::<Nullable<Jsonb>, _>(context)
+                .bind::<Nullable<Integer>, _>(creation_block)
+                .execute(conn)?;
+
+            if count > 1 {
+                // Data source deduplication enforces this invariant.
+                // See also: data-source-is-duplicate-of
+                return Err(constraint_violation!(
+                    "expected to remove at most one offchain data source but removed {}, ds: {:?}",
+                    count,
+                    ds
+                ));
+            }
+        }
+
+        Ok(())
+    }
 }
diff --git a/store/postgres/src/writable.rs b/store/postgres/src/writable.rs
index 2c615d79a87..327710375c9 100644
--- a/store/postgres/src/writable.rs
+++ b/store/postgres/src/writable.rs
@@ -254,6 +254,7 @@ impl SyncStore {
         data_sources: &[StoredDynamicDataSource],
         deterministic_errors: &[SubgraphError],
         manifest_idx_and_name: &[(u32, String)],
+        offchain_to_remove: &[StoredDynamicDataSource],
     ) -> Result<(), StoreError> {
         self.retry("transact_block_operations", move || {
             let event = self.writable.transact_block_operations(
@@ -265,6 +266,7 @@ impl SyncStore {
                 data_sources,
                 deterministic_errors,
                 manifest_idx_and_name,
+                offchain_to_remove,
             )?;
 
             let _section = stopwatch.start_section("send_store_event");
@@ -421,6 +423,7 @@ enum Request {
         data_sources: Vec<StoredDynamicDataSource>,
         deterministic_errors: Vec<SubgraphError>,
         manifest_idx_and_name: Vec<(u32, String)>,
+        offchain_to_remove: Vec<StoredDynamicDataSource>,
     },
     RevertTo {
         store: Arc<SyncStore>,
@@ -442,6 +445,7 @@ impl Request {
                 data_sources,
                 deterministic_errors,
                 manifest_idx_and_name,
+                offchain_to_remove,
             } => store.transact_block_operations(
                 block_ptr_to,
                 firehose_cursor,
@@ -450,6 +454,7 @@ impl Request {
                 data_sources,
                 deterministic_errors,
                 manifest_idx_and_name,
+                offchain_to_remove,
             ),
             Request::RevertTo {
                 store,
@@ -756,10 +761,15 @@ impl Queue {
                 Request::Write {
                     block_ptr,
                     data_sources,
+                    offchain_to_remove,
                     ..
                 } => {
                     if tracker.visible(block_ptr) {
                         dds.extend(data_sources.clone());
+                        dds = dds
+                            .into_iter()
+                            .filter(|dds| !offchain_to_remove.contains(dds))
+                            .collect();
                     }
                 }
                 Request::RevertTo { .. } => { /* nothing to do */ }
@@ -812,6 +822,7 @@ impl Writer {
         data_sources: Vec<StoredDynamicDataSource>,
         deterministic_errors: Vec<SubgraphError>,
         manifest_idx_and_name: Vec<(u32, String)>,
+        offchain_to_remove: Vec<StoredDynamicDataSource>,
     ) -> Result<(), StoreError> {
         match self {
             Writer::Sync(store) => store.transact_block_operations(
@@ -822,6 +833,7 @@ impl Writer {
                 &data_sources,
                 &deterministic_errors,
                 &manifest_idx_and_name,
+                &offchain_to_remove,
             ),
             Writer::Async(queue) => {
                 let req = Request::Write {
@@ -833,6 +845,7 @@ impl Writer {
                     data_sources,
                     deterministic_errors,
                     manifest_idx_and_name,
+                    offchain_to_remove,
                 };
                 queue.push(req).await
             }
@@ -1012,6 +1025,7 @@ impl WritableStoreTrait for WritableStore {
         data_sources: Vec<StoredDynamicDataSource>,
         deterministic_errors: Vec<SubgraphError>,
         manifest_idx_and_name: Vec<(u32, String)>,
+        offchain_to_remove: Vec<StoredDynamicDataSource>,
     ) -> Result<(), StoreError> {
         self.writer
             .write(
@@ -1022,6 +1036,7 @@ impl WritableStoreTrait for WritableStore {
                 data_sources,
                 deterministic_errors,
                 manifest_idx_and_name,
+                offchain_to_remove,
             )
             .await?;
 
diff --git a/store/postgres/tests/store.rs b/store/postgres/tests/store.rs
index 2abf2ce494e..fd7ad578332 100644
--- a/store/postgres/tests/store.rs
+++ b/store/postgres/tests/store.rs
@@ -1579,6 +1579,7 @@ fn handle_large_string_with_index() {
                 Vec::new(),
                 Vec::new(),
                 Vec::new(),
+                Vec::new(),
             )
             .await
             .expect("Failed to insert large text");
@@ -1677,6 +1678,7 @@ fn handle_large_bytea_with_index() {
                 Vec::new(),
                 Vec::new(),
                 Vec::new(),
+                Vec::new(),
             )
             .await
             .expect("Failed to insert large text");
diff --git a/store/test-store/src/store.rs b/store/test-store/src/store.rs
index 929b571332f..ef8856e4d6c 100644
--- a/store/test-store/src/store.rs
+++ b/store/test-store/src/store.rs
@@ -233,6 +233,7 @@ pub async fn transact_errors(
             Vec::new(),
             errs,
             Vec::new(),
+            Vec::new(),
         )
         .await?;
     flush(deployment).await
@@ -307,6 +308,7 @@ pub async fn transact_entities_and_dynamic_data_sources(
             data_sources,
             Vec::new(),
             manifest_idx_and_name,
+            Vec::new(),
         )
         .await
 }

From f30b210cb19ca1b2d25a97f3e8e92b1f4d58d09e Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Thu, 18 Aug 2022 14:32:05 +0100
Subject: [PATCH 24/33] refactor: Extract ReadStore out of WritableStore

---
 graph/src/components/store/cache.rs       | 23 +++++-----
 graph/src/components/store/traits.rs      | 46 +++++++++++++------
 graph/src/components/subgraph/instance.rs |  9 ++--
 graph/tests/entity_cache.rs               | 54 ++++++++++++-----------
 runtime/test/src/test.rs                  |  2 +-
 store/postgres/src/writable.rs            | 33 +++++++-------
 store/test-store/src/store.rs             |  2 +-
 7 files changed, 98 insertions(+), 71 deletions(-)

diff --git a/graph/src/components/store/cache.rs b/graph/src/components/store/cache.rs
index 46f7b28a8c7..e5e990343e4 100644
--- a/graph/src/components/store/cache.rs
+++ b/graph/src/components/store/cache.rs
@@ -8,7 +8,7 @@ use crate::components::store::{
     self as s, Entity, EntityKey, EntityOp, EntityOperation, EntityType,
 };
 use crate::data_source::DataSource;
-use crate::prelude::ENV_VARS;
+use crate::prelude::{Schema, ENV_VARS};
 use crate::util::lfu_cache::LfuCache;
 
 /// A cache for entities from the store that provides the basic functionality
@@ -35,7 +35,9 @@ pub struct EntityCache {
     data_sources: Vec<s::StoredDynamicDataSource>,
 
     /// The store is only used to read entities.
-    pub store: Arc<dyn s::WritableStore>,
+    pub store: Arc<dyn s::ReadStore>,
+
+    schema: Arc<Schema>,
 }
 
 impl Debug for EntityCache {
@@ -54,19 +56,20 @@ pub struct ModificationsAndCache {
 }
 
 impl EntityCache {
-    pub fn new(store: Arc<dyn s::WritableStore>) -> Self {
+    pub fn new(store: Arc<dyn s::ReadStore>) -> Self {
         Self {
             current: LfuCache::new(),
             updates: HashMap::new(),
             handler_updates: HashMap::new(),
             in_handler: false,
             data_sources: vec![],
+            schema: store.input_schema(),
             store,
         }
     }
 
     pub fn with_current(
-        store: Arc<dyn s::WritableStore>,
+        store: Arc<dyn s::ReadStore>,
         current: LfuCache<EntityKey, Option<Entity>>,
     ) -> EntityCache {
         EntityCache {
@@ -75,6 +78,7 @@ impl EntityCache {
             handler_updates: HashMap::new(),
             in_handler: false,
             data_sources: vec![],
+            schema: store.input_schema(),
             store,
         }
     }
@@ -147,12 +151,12 @@ impl EntityCache {
                 // The validation will catch the type mismatch
             }
             None => {
-                let value = self.store.input_schema().id_value(&key)?;
+                let value = self.schema.id_value(&key)?;
                 entity.set("id", value);
             }
         }
 
-        let is_valid = entity.validate(&self.store.input_schema(), &key).is_ok();
+        let is_valid = entity.validate(&self.schema, &key).is_ok();
 
         self.entity_op(key.clone(), EntityOp::Update(entity));
 
@@ -167,7 +171,7 @@ impl EntityCache {
                     key.entity_id
                 )
             })?;
-            entity.validate(&self.store.input_schema(), &key)?;
+            entity.validate(&self.schema, &key)?;
         }
 
         Ok(())
@@ -240,8 +244,7 @@ impl EntityCache {
         // is wrong and the store already has a version of the entity from a
         // previous block, the attempt to insert will trigger a constraint
         // violation in the database, ensuring correctness
-        let missing =
-            missing.filter(|key| !self.store.input_schema().is_immutable(&key.entity_type));
+        let missing = missing.filter(|key| !self.schema.is_immutable(&key.entity_type));
 
         let mut missing_by_type: BTreeMap<&EntityType, Vec<&str>> = BTreeMap::new();
         for key in missing {
@@ -321,7 +324,7 @@ impl LfuCache<EntityKey, Option<Entity>> {
     // Helper for cached lookup of an entity.
     fn get_entity(
         &mut self,
-        store: &(impl s::WritableStore + ?Sized),
+        store: &(impl s::ReadStore + ?Sized),
         key: &EntityKey,
     ) -> Result<Option<Entity>, s::QueryExecutionError> {
         match self.get(key) {
diff --git a/graph/src/components/store/traits.rs b/graph/src/components/store/traits.rs
index bc532e94ce0..702f4f35a38 100644
--- a/graph/src/components/store/traits.rs
+++ b/graph/src/components/store/traits.rs
@@ -149,12 +149,44 @@ pub trait SubgraphStore: Send + Sync + 'static {
     fn locators(&self, hash: &str) -> Result<Vec<DeploymentLocator>, StoreError>;
 }
 
+pub trait ReadStore: Send + Sync + 'static {
+    /// Looks up an entity using the given store key at the latest block.
+    fn get(&self, key: &EntityKey) -> Result<Option<Entity>, StoreError>;
+
+    /// Look up multiple entities as of the latest block. Returns a map of
+    /// entities by type.
+    fn get_many(
+        &self,
+        ids_for_type: BTreeMap<&EntityType, Vec<&str>>,
+    ) -> Result<BTreeMap<EntityType, Vec<Entity>>, StoreError>;
+
+    fn input_schema(&self) -> Arc<Schema>;
+}
+
+// This silly impl is needed until https://github.com/rust-lang/rust/issues/65991 is stable.
+impl<T: ?Sized + ReadStore> ReadStore for Arc<T> {
+    fn get(&self, key: &EntityKey) -> Result<Option<Entity>, StoreError> {
+        (**self).get(key)
+    }
+
+    fn get_many(
+        &self,
+        ids_for_type: BTreeMap<&EntityType, Vec<&str>>,
+    ) -> Result<BTreeMap<EntityType, Vec<Entity>>, StoreError> {
+        (**self).get_many(ids_for_type)
+    }
+
+    fn input_schema(&self) -> Arc<Schema> {
+        (**self).input_schema()
+    }
+}
+
 /// 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
 /// operation indefinitely until it succeeds.
 #[async_trait]
-pub trait WritableStore: Send + Sync + 'static {
+pub trait WritableStore: ReadStore {
     /// Get a pointer to the most recently processed block in the subgraph.
     fn block_ptr(&self) -> Option<BlockPtr>;
 
@@ -195,9 +227,6 @@ pub trait WritableStore: Send + Sync + 'static {
 
     async fn supports_proof_of_indexing(&self) -> Result<bool, StoreError>;
 
-    /// Looks up an entity using the given store key at the latest block.
-    fn get(&self, key: &EntityKey) -> Result<Option<Entity>, StoreError>;
-
     /// Transact the entity changes from a single block atomically into the store, and update the
     /// subgraph block pointer to `block_ptr_to`, and update the firehose cursor to `firehose_cursor`
     ///
@@ -214,13 +243,6 @@ pub trait WritableStore: Send + Sync + 'static {
         offchain_to_remove: Vec<StoredDynamicDataSource>,
     ) -> Result<(), StoreError>;
 
-    /// Look up multiple entities as of the latest block. Returns a map of
-    /// entities by type.
-    fn get_many(
-        &self,
-        ids_for_type: BTreeMap<&EntityType, Vec<&str>>,
-    ) -> Result<BTreeMap<EntityType, Vec<Entity>>, StoreError>;
-
     /// The deployment `id` finished syncing, mark it as synced in the database
     /// and promote it to the current version in the subgraphs where it was the
     /// pending version so far
@@ -244,8 +266,6 @@ pub trait WritableStore: Send + Sync + 'static {
 
     async fn health(&self) -> Result<SubgraphHealth, StoreError>;
 
-    fn input_schema(&self) -> Arc<Schema>;
-
     /// Wait for the background writer to finish processing its queue
     async fn flush(&self) -> Result<(), StoreError>;
 }
diff --git a/graph/src/components/subgraph/instance.rs b/graph/src/components/subgraph/instance.rs
index c3686ef2d0e..ec35c829f94 100644
--- a/graph/src/components/subgraph/instance.rs
+++ b/graph/src/components/subgraph/instance.rs
@@ -1,6 +1,6 @@
 use crate::{
     blockchain::Blockchain,
-    components::store::{EntityKey, StoredDynamicDataSource, WritableStore},
+    components::store::{EntityKey, ReadStore, StoredDynamicDataSource},
     data::subgraph::schema::SubgraphError,
     data_source::DataSourceTemplate,
     prelude::*,
@@ -32,12 +32,9 @@ pub struct BlockState<C: Blockchain> {
 }
 
 impl<C: Blockchain> BlockState<C> {
-    pub fn new(
-        store: Arc<dyn WritableStore>,
-        lfu_cache: LfuCache<EntityKey, Option<Entity>>,
-    ) -> Self {
+    pub fn new(store: impl ReadStore, lfu_cache: LfuCache<EntityKey, Option<Entity>>) -> Self {
         BlockState {
-            entity_cache: EntityCache::with_current(store, lfu_cache),
+            entity_cache: EntityCache::with_current(Arc::new(store), lfu_cache),
             deterministic_errors: Vec::new(),
             created_data_sources: Vec::new(),
             handler_created_data_sources: Vec::new(),
diff --git a/graph/tests/entity_cache.rs b/graph/tests/entity_cache.rs
index 7dd5b0c873c..10935fd7ddc 100644
--- a/graph/tests/entity_cache.rs
+++ b/graph/tests/entity_cache.rs
@@ -8,7 +8,9 @@ use slog::Logger;
 use std::collections::BTreeMap;
 use std::sync::Arc;
 
-use graph::components::store::{EntityKey, EntityType, StoredDynamicDataSource, WritableStore};
+use graph::components::store::{
+    EntityKey, EntityType, ReadStore, StoredDynamicDataSource, WritableStore,
+};
 use graph::{
     components::store::{DeploymentId, DeploymentLocator},
     prelude::{anyhow, DeploymentHash, Entity, EntityCache, EntityModification, Value},
@@ -44,6 +46,32 @@ impl MockStore {
     }
 }
 
+impl ReadStore for MockStore {
+    fn get(&self, key: &EntityKey) -> Result<Option<Entity>, StoreError> {
+        match self.get_many_res.get(&key.entity_type) {
+            Some(entities) => Ok(entities
+                .iter()
+                .find(|entity| entity.id().ok().as_deref() == Some(key.entity_id.as_str()))
+                .cloned()),
+            None => Err(StoreError::Unknown(anyhow!(
+                "nothing for type {}",
+                key.entity_type
+            ))),
+        }
+    }
+
+    fn get_many(
+        &self,
+        _ids_for_type: BTreeMap<&EntityType, Vec<&str>>,
+    ) -> Result<BTreeMap<EntityType, Vec<Entity>>, StoreError> {
+        Ok(self.get_many_res.clone())
+    }
+
+    fn input_schema(&self) -> Arc<Schema> {
+        SCHEMA.clone()
+    }
+}
+
 #[async_trait]
 impl WritableStore for MockStore {
     fn block_ptr(&self) -> Option<BlockPtr> {
@@ -86,19 +114,6 @@ impl WritableStore for MockStore {
         unimplemented!()
     }
 
-    fn get(&self, key: &EntityKey) -> Result<Option<Entity>, StoreError> {
-        match self.get_many_res.get(&key.entity_type) {
-            Some(entities) => Ok(entities
-                .iter()
-                .find(|entity| entity.id().ok().as_deref() == Some(key.entity_id.as_str()))
-                .cloned()),
-            None => Err(StoreError::Unknown(anyhow!(
-                "nothing for type {}",
-                key.entity_type
-            ))),
-        }
-    }
-
     async fn transact_block_operations(
         &self,
         _: BlockPtr,
@@ -113,13 +128,6 @@ impl WritableStore for MockStore {
         unimplemented!()
     }
 
-    fn get_many(
-        &self,
-        _ids_for_type: BTreeMap<&EntityType, Vec<&str>>,
-    ) -> Result<BTreeMap<EntityType, Vec<Entity>>, StoreError> {
-        Ok(self.get_many_res.clone())
-    }
-
     async fn is_deployment_synced(&self) -> Result<bool, StoreError> {
         unimplemented!()
     }
@@ -147,10 +155,6 @@ impl WritableStore for MockStore {
         unimplemented!()
     }
 
-    fn input_schema(&self) -> Arc<Schema> {
-        SCHEMA.clone()
-    }
-
     async fn flush(&self) -> Result<(), StoreError> {
         unimplemented!()
     }
diff --git a/runtime/test/src/test.rs b/runtime/test/src/test.rs
index 6b17c29327d..cad61b5c897 100644
--- a/runtime/test/src/test.rs
+++ b/runtime/test/src/test.rs
@@ -973,7 +973,7 @@ async fn test_entity_store(api_version: Version) {
     let writable = store.writable(LOGGER.clone(), deployment.id).await.unwrap();
     let cache = std::mem::replace(
         &mut module.instance_ctx_mut().ctx.state.entity_cache,
-        EntityCache::new(writable.clone()),
+        EntityCache::new(Arc::new(writable.clone())),
     );
     let mut mods = cache.as_modifications().unwrap().modifications;
     assert_eq!(1, mods.len());
diff --git a/store/postgres/src/writable.rs b/store/postgres/src/writable.rs
index 327710375c9..800a0ced080 100644
--- a/store/postgres/src/writable.rs
+++ b/store/postgres/src/writable.rs
@@ -5,6 +5,7 @@ use std::{collections::BTreeMap, sync::Arc};
 
 use graph::blockchain::block_stream::FirehoseCursor;
 use graph::components::store::EntityKey;
+use graph::components::store::ReadStore;
 use graph::data::subgraph::schema;
 use graph::env::env_var;
 use graph::prelude::{
@@ -942,6 +943,23 @@ impl WritableStore {
     }
 }
 
+impl ReadStore for WritableStore {
+    fn get(&self, key: &EntityKey) -> Result<Option<Entity>, StoreError> {
+        self.writer.get(key)
+    }
+
+    fn get_many(
+        &self,
+        ids_for_type: BTreeMap<&EntityType, Vec<&str>>,
+    ) -> Result<BTreeMap<EntityType, Vec<Entity>>, StoreError> {
+        self.writer.get_many(ids_for_type)
+    }
+
+    fn input_schema(&self) -> Arc<Schema> {
+        self.store.input_schema()
+    }
+}
+
 #[async_trait::async_trait]
 impl WritableStoreTrait for WritableStore {
     fn block_ptr(&self) -> Option<BlockPtr> {
@@ -1012,10 +1030,6 @@ impl WritableStoreTrait for WritableStore {
         self.store.supports_proof_of_indexing().await
     }
 
-    fn get(&self, key: &EntityKey) -> Result<Option<Entity>, StoreError> {
-        self.writer.get(key)
-    }
-
     async fn transact_block_operations(
         &self,
         block_ptr_to: BlockPtr,
@@ -1046,13 +1060,6 @@ impl WritableStoreTrait for WritableStore {
         Ok(())
     }
 
-    fn get_many(
-        &self,
-        ids_for_type: BTreeMap<&EntityType, Vec<&str>>,
-    ) -> Result<BTreeMap<EntityType, Vec<Entity>>, StoreError> {
-        self.writer.get_many(ids_for_type)
-    }
-
     fn deployment_synced(&self) -> Result<(), StoreError> {
         self.store.deployment_synced()
     }
@@ -1082,10 +1089,6 @@ impl WritableStoreTrait for WritableStore {
         self.store.health().await
     }
 
-    fn input_schema(&self) -> Arc<Schema> {
-        self.store.input_schema()
-    }
-
     async fn flush(&self) -> Result<(), StoreError> {
         self.writer.flush().await
     }
diff --git a/store/test-store/src/store.rs b/store/test-store/src/store.rs
index ef8856e4d6c..c1158da0b2a 100644
--- a/store/test-store/src/store.rs
+++ b/store/test-store/src/store.rs
@@ -286,7 +286,7 @@ pub async fn transact_entities_and_dynamic_data_sources(
 ) -> Result<(), StoreError> {
     let store =
         futures03::executor::block_on(store.cheap_clone().writable(LOGGER.clone(), deployment.id))?;
-    let mut entity_cache = EntityCache::new(store.clone());
+    let mut entity_cache = EntityCache::new(Arc::new(store.clone()));
     entity_cache.append(ops);
     let mods = entity_cache
         .as_modifications()

From ad6264d008182f01d6759f106b685085f01c9663 Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Fri, 19 Aug 2022 13:54:52 +0100
Subject: [PATCH 25/33] test: Add graphql queries to end-to-end tests

---
 Cargo.lock            |  2 ++
 tests/Cargo.toml      |  2 ++
 tests/src/fixture.rs  | 41 ++++++++++++++++++++++++++++++++++++++---
 tests/tests/runner.rs | 14 ++++++++++++++
 4 files changed, 56 insertions(+), 3 deletions(-)

diff --git a/Cargo.lock b/Cargo.lock
index 37e43bb33c5..c95634aa04e 100644
--- a/Cargo.lock
+++ b/Cargo.lock
@@ -1902,9 +1902,11 @@ dependencies = [
  "graph-chain-ethereum",
  "graph-chain-near",
  "graph-core",
+ "graph-graphql",
  "graph-mock",
  "graph-node",
  "graph-store-postgres",
+ "graphql-parser",
  "lazy_static",
  "port_check",
  "serde_yaml",
diff --git a/tests/Cargo.toml b/tests/Cargo.toml
index 1cb36b00ca0..c1c695fe5bb 100644
--- a/tests/Cargo.toml
+++ b/tests/Cargo.toml
@@ -14,8 +14,10 @@ async-stream = "0.3.3"
 graph-node = { path = "../node" }
 graph-core = { path = "../core" }
 graph-mock = { path = "../mock" }
+graph-graphql = { path = "../graphql" }
 graph-store-postgres = { path = "../store/postgres" }
 slog = { version = "2.7.0", features = ["release_max_level_trace", "max_level_trace"] }
+graphql-parser = "0.4.0"
 
 [dev-dependencies]
 bollard = "0.10"
diff --git a/tests/src/fixture.rs b/tests/src/fixture.rs
index 1b491975e51..3e7dd01ac8c 100644
--- a/tests/src/fixture.rs
+++ b/tests/src/fixture.rs
@@ -18,19 +18,22 @@ use graph::blockchain::{
 };
 use graph::cheap_clone::CheapClone;
 use graph::components::store::{BlockStore, DeploymentLocator};
+use graph::data::graphql::effort::LoadManager;
+use graph::data::query::{Query, QueryTarget};
 use graph::env::ENV_VARS;
 use graph::ipfs_client::IpfsClient;
 use graph::prelude::ethabi::ethereum_types::H256;
 use graph::prelude::{
-    async_trait, BlockNumber, DeploymentHash, LoggerFactory, MetricsRegistry, NodeId,
-    SubgraphAssignmentProvider, SubgraphName, SubgraphRegistrar, SubgraphStore as _,
-    SubgraphVersionSwitchingMode,
+    async_trait, r, BlockNumber, DeploymentHash, GraphQlRunner as _, LoggerFactory,
+    MetricsRegistry, NodeId, QueryError, SubgraphAssignmentProvider, SubgraphName,
+    SubgraphRegistrar, SubgraphStore as _, SubgraphVersionSwitchingMode,
 };
 use graph_core::polling_monitor::ipfs_service::IpfsService;
 use graph_core::{
     LinkResolver, SubgraphAssignmentProvider as IpfsSubgraphAssignmentProvider,
     SubgraphInstanceManager, SubgraphRegistrar as IpfsSubgraphRegistrar,
 };
+use graph_graphql::prelude::GraphQlRunner;
 use graph_mock::MockMetricsRegistry;
 use graph_node::manager::PanicSubscriptionManager;
 use graph_node::{config::Config, store_builder::StoreBuilder};
@@ -104,6 +107,7 @@ pub struct TestContext {
     >,
     pub store: Arc<SubgraphStore>,
     pub deployment: DeploymentLocator,
+    graphql_runner: Arc<GraphQlRunner<Store, PanicSubscriptionManager>>,
 }
 
 impl TestContext {
@@ -117,6 +121,25 @@ impl TestContext {
             .await
             .unwrap();
     }
+
+    pub async fn query(&self, query: &str) -> Result<Option<r::Value>, Vec<QueryError>> {
+        let target = QueryTarget::Deployment(self.deployment.hash.clone());
+
+        self.graphql_runner
+            .clone()
+            .run_query(
+                Query::new(
+                    graphql_parser::parse_query(query).unwrap().into_static(),
+                    None,
+                ),
+                target,
+            )
+            .await
+            .first()
+            .unwrap()
+            .duplicate()
+            .to_result()
+    }
 }
 
 pub struct Stores {
@@ -222,6 +245,17 @@ pub async fn setup<C: Blockchain>(
         static_filters,
     );
 
+    // Graphql runner
+    let subscription_manager = Arc::new(PanicSubscriptionManager {});
+    let load_manager = LoadManager::new(&logger, Vec::new(), mock_registry.clone());
+    let graphql_runner = Arc::new(GraphQlRunner::new(
+        &logger,
+        stores.network_store.clone(),
+        subscription_manager.clone(),
+        Arc::new(load_manager),
+        mock_registry.clone(),
+    ));
+
     // Create IPFS-based subgraph provider
     let subgraph_provider = Arc::new(IpfsSubgraphAssignmentProvider::new(
         &logger_factory,
@@ -263,6 +297,7 @@ pub async fn setup<C: Blockchain>(
         provider: subgraph_provider,
         store: subgraph_store,
         deployment,
+        graphql_runner,
     }
 }
 
diff --git a/tests/tests/runner.rs b/tests/tests/runner.rs
index 421ecd85302..52fa3d6821f 100644
--- a/tests/tests/runner.rs
+++ b/tests/tests/runner.rs
@@ -1,6 +1,7 @@
 use std::sync::Arc;
 
 use graph::blockchain::{Block, BlockPtr};
+use graph::object;
 use graph::prelude::ethabi::ethereum_types::H256;
 use graph::prelude::{SubgraphAssignmentProvider, SubgraphName};
 use graph_tests::fixture::ethereum::{chain, empty_block, genesis};
@@ -53,6 +54,19 @@ async fn data_source_revert() -> anyhow::Result<()> {
     let stop_block = test_ptr(4);
     ctx.start_and_sync_to(stop_block).await;
 
+    let query_res = ctx
+        .query(r#"{ dataSourceCount(id: "4") { id, count } }"#)
+        .await
+        .unwrap();
+
+    // TODO: The semantically correct value for `count` would be 5. But because the test fixture
+    // uses a `NoopTriggersAdapter` the data sources are not reprocessed in the block in which they
+    // are created.
+    assert_eq!(
+        query_res,
+        Some(object! { dataSourceCount: object!{ id: "4", count: 4 } })
+    );
+
     fixture::cleanup(&ctx.store, &subgraph_name, &hash);
 
     Ok(())

From f59d9c92a3bd7dcf4717618394b17b21a0d486b7 Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Sun, 21 Aug 2022 13:37:06 +0100
Subject: [PATCH 26/33] feat(file ds): Bump max spec version to 0.0.7

---
 graph/src/env/mod.rs | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/graph/src/env/mod.rs b/graph/src/env/mod.rs
index e5ddffafe31..b7e1fc71071 100644
--- a/graph/src/env/mod.rs
+++ b/graph/src/env/mod.rs
@@ -110,7 +110,7 @@ pub struct EnvVars {
     /// are enabled.
     pub allow_non_deterministic_fulltext_search: bool,
     /// Set by the environment variable `GRAPH_MAX_SPEC_VERSION`. The default
-    /// value is `0.0.6`.
+    /// value is `0.0.7`.
     pub max_spec_version: Version,
     /// Set by the flag `GRAPH_DISABLE_GRAFTS`.
     pub disable_grafts: bool,
@@ -310,7 +310,7 @@ struct Inner {
         default = "false"
     )]
     allow_non_deterministic_fulltext_search: EnvVarBoolean,
-    #[envconfig(from = "GRAPH_MAX_SPEC_VERSION", default = "0.0.6")]
+    #[envconfig(from = "GRAPH_MAX_SPEC_VERSION", default = "0.0.7")]
     max_spec_version: Version,
     #[envconfig(from = "GRAPH_DISABLE_GRAFTS", default = "false")]
     disable_grafts: EnvVarBoolean,

From 7e9913522747cf0c62a037f8778f275d4119d02f Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Sun, 21 Aug 2022 13:45:04 +0100
Subject: [PATCH 27/33] test: Add basic file data sources e2e test

---
 Cargo.lock                                    |  2 +
 tests/Cargo.toml                              |  2 +
 .../file-data-sources/abis/Contract.abi       |  1 +
 .../file-data-sources/package.json            | 13 +++++
 .../file-data-sources/schema.graphql          |  4 ++
 .../file-data-sources/src/mapping.ts          | 16 ++++++
 .../file-data-sources/subgraph.yaml           | 36 +++++++++++++
 tests/src/fixture.rs                          | 27 +++++++---
 tests/tests/runner.rs                         | 50 ++++++++++++++++---
 9 files changed, 139 insertions(+), 12 deletions(-)
 create mode 100644 tests/integration-tests/file-data-sources/abis/Contract.abi
 create mode 100644 tests/integration-tests/file-data-sources/package.json
 create mode 100644 tests/integration-tests/file-data-sources/schema.graphql
 create mode 100644 tests/integration-tests/file-data-sources/src/mapping.ts
 create mode 100644 tests/integration-tests/file-data-sources/subgraph.yaml

diff --git a/Cargo.lock b/Cargo.lock
index c95634aa04e..2cadf90c32f 100644
--- a/Cargo.lock
+++ b/Cargo.lock
@@ -1897,6 +1897,7 @@ dependencies = [
  "anyhow",
  "async-stream",
  "bollard",
+ "cid",
  "futures 0.3.16",
  "graph",
  "graph-chain-ethereum",
@@ -1907,6 +1908,7 @@ dependencies = [
  "graph-node",
  "graph-store-postgres",
  "graphql-parser",
+ "hex",
  "lazy_static",
  "port_check",
  "serde_yaml",
diff --git a/tests/Cargo.toml b/tests/Cargo.toml
index c1c695fe5bb..cc67dd925a1 100644
--- a/tests/Cargo.toml
+++ b/tests/Cargo.toml
@@ -18,6 +18,7 @@ graph-graphql = { path = "../graphql" }
 graph-store-postgres = { path = "../store/postgres" }
 slog = { version = "2.7.0", features = ["release_max_level_trace", "max_level_trace"] }
 graphql-parser = "0.4.0"
+hex = "0.4.3"
 
 [dev-dependencies]
 bollard = "0.10"
@@ -25,4 +26,5 @@ anyhow = "1.0.62"
 lazy_static = "1.4.0"
 tokio-stream = "0.1"
 serde_yaml = "0.8"
+cid = "0.8.6"
 graph-chain-near = { path = "../chain/near" }
diff --git a/tests/integration-tests/file-data-sources/abis/Contract.abi b/tests/integration-tests/file-data-sources/abis/Contract.abi
new file mode 100644
index 00000000000..1e3ec7217af
--- /dev/null
+++ b/tests/integration-tests/file-data-sources/abis/Contract.abi
@@ -0,0 +1 @@
+[ ]
diff --git a/tests/integration-tests/file-data-sources/package.json b/tests/integration-tests/file-data-sources/package.json
new file mode 100644
index 00000000000..7c68f0411a7
--- /dev/null
+++ b/tests/integration-tests/file-data-sources/package.json
@@ -0,0 +1,13 @@
+{
+  "name": "file-data-sources",
+  "version": "0.1.0",
+  "scripts": {
+    "codegen": "graph codegen",
+    "create:test": "graph create test/file-data-sources --node $GRAPH_NODE_ADMIN_URI",
+    "deploy:test": "graph deploy test/file-data-sources --version-label v0.0.1 --ipfs $IPFS_URI --node $GRAPH_NODE_ADMIN_URI"
+  },
+  "devDependencies": {
+    "@graphprotocol/graph-cli": "https://github.com/graphprotocol/graph-cli#main",
+    "@graphprotocol/graph-ts": "https://github.com/graphprotocol/graph-ts#main"
+  }
+}
\ No newline at end of file
diff --git a/tests/integration-tests/file-data-sources/schema.graphql b/tests/integration-tests/file-data-sources/schema.graphql
new file mode 100644
index 00000000000..476bce7976d
--- /dev/null
+++ b/tests/integration-tests/file-data-sources/schema.graphql
@@ -0,0 +1,4 @@
+type IpfsFile @entity {
+  id: ID!
+  content: String!
+}
diff --git a/tests/integration-tests/file-data-sources/src/mapping.ts b/tests/integration-tests/file-data-sources/src/mapping.ts
new file mode 100644
index 00000000000..3431f17bb1a
--- /dev/null
+++ b/tests/integration-tests/file-data-sources/src/mapping.ts
@@ -0,0 +1,16 @@
+import { ethereum, dataSource, BigInt, Bytes } from '@graphprotocol/graph-ts'
+import { IpfsFile } from '../generated/schema'
+
+export function handleBlock(block: ethereum.Block): void {
+  if (block.number == BigInt.fromI32(0)) {
+    // CID QmVkvoPGi9jvvuxsHDVJDgzPEzagBaWSZRYoRDzU244HjZ is the file
+    // `file-data-sources/abis/Contract.abi` after being processed by graph-cli.
+    dataSource.create("File", ["QmVkvoPGi9jvvuxsHDVJDgzPEzagBaWSZRYoRDzU244HjZ"])
+  }
+}
+
+export function handleFile(data: Bytes): void {
+  let entity = new IpfsFile(dataSource.address().toHexString())
+  entity.content = data.toString()
+  entity.save()
+}
diff --git a/tests/integration-tests/file-data-sources/subgraph.yaml b/tests/integration-tests/file-data-sources/subgraph.yaml
new file mode 100644
index 00000000000..d13fefe163c
--- /dev/null
+++ b/tests/integration-tests/file-data-sources/subgraph.yaml
@@ -0,0 +1,36 @@
+specVersion: 0.0.7
+schema:
+  file: ./schema.graphql
+dataSources:
+  - kind: ethereum/contract
+    name: Contract
+    network: test
+    source:
+      address: "0xCfEB869F69431e42cdB54A4F4f105C19C080A601"
+      abi: Contract
+    mapping:
+      kind: ethereum/events
+      apiVersion: 0.0.7
+      language: wasm/assemblyscript
+      entities:
+        - Gravatar
+      abis:
+        - name: Contract
+          file: ./abis/Contract.abi
+      blockHandlers:
+        - handler: handleBlock
+      file: ./src/mapping.ts
+templates:
+  - kind: file/ipfs
+    name: File
+    mapping:
+      kind: ethereum/events
+      apiVersion: 0.0.7
+      language: wasm/assemblyscript
+      entities:
+        - Gravatar
+      abis:
+        - name: Contract
+          file: ./abis/Contract.abi
+      handler: handleFile
+      file: ./src/mapping.ts
diff --git a/tests/src/fixture.rs b/tests/src/fixture.rs
index 3e7dd01ac8c..ed6792a2ae2 100644
--- a/tests/src/fixture.rs
+++ b/tests/src/fixture.rs
@@ -38,7 +38,7 @@ use graph_mock::MockMetricsRegistry;
 use graph_node::manager::PanicSubscriptionManager;
 use graph_node::{config::Config, store_builder::StoreBuilder};
 use graph_store_postgres::{ChainHeadUpdateListener, ChainStore, Store, SubgraphStore};
-use slog::{info, Logger};
+use slog::{crit, info, Logger};
 use std::env::VarError;
 use std::pin::Pin;
 use std::sync::Arc;
@@ -107,6 +107,7 @@ pub struct TestContext {
     >,
     pub store: Arc<SubgraphStore>,
     pub deployment: DeploymentLocator,
+    pub subgraph_name: SubgraphName,
     graphql_runner: Arc<GraphQlRunner<Store, PanicSubscriptionManager>>,
 }
 
@@ -142,6 +143,14 @@ impl TestContext {
     }
 }
 
+impl Drop for TestContext {
+    fn drop(&mut self) {
+        if let Err(e) = cleanup(&self.store, &self.subgraph_name, &self.deployment.hash) {
+            crit!(self.logger, "error cleaning up test subgraph"; "error" => e.to_string());
+        }
+    }
+}
+
 pub struct Stores {
     network_name: String,
     chain_head_listener: Arc<ChainHeadUpdateListener>,
@@ -215,7 +224,7 @@ pub async fn setup<C: Blockchain>(
 
     // Make sure we're starting from a clean state.
     let subgraph_store = stores.network_store.subgraph_store();
-    cleanup(&subgraph_store, &subgraph_name, hash);
+    cleanup(&subgraph_store, &subgraph_name, hash).unwrap();
 
     let mut blockchain_map = BlockchainMap::new();
     blockchain_map.insert(stores.network_name.clone(), chain);
@@ -297,16 +306,22 @@ pub async fn setup<C: Blockchain>(
         provider: subgraph_provider,
         store: subgraph_store,
         deployment,
+        subgraph_name,
         graphql_runner,
     }
 }
 
-pub fn cleanup(subgraph_store: &SubgraphStore, name: &SubgraphName, hash: &DeploymentHash) {
-    let locators = subgraph_store.locators(hash).unwrap();
-    subgraph_store.remove_subgraph(name.clone()).unwrap();
+pub fn cleanup(
+    subgraph_store: &SubgraphStore,
+    name: &SubgraphName,
+    hash: &DeploymentHash,
+) -> Result<(), Error> {
+    let locators = subgraph_store.locators(hash)?;
+    subgraph_store.remove_subgraph(name.clone())?;
     for locator in locators {
-        subgraph_store.remove_deployment(locator.id.into()).unwrap();
+        subgraph_store.remove_deployment(locator.id.into())?;
     }
+    Ok(())
 }
 
 pub async fn wait_for_sync(
diff --git a/tests/tests/runner.rs b/tests/tests/runner.rs
index 52fa3d6821f..0b0281e4455 100644
--- a/tests/tests/runner.rs
+++ b/tests/tests/runner.rs
@@ -1,5 +1,6 @@
 use std::sync::Arc;
 
+use cid::Cid;
 use graph::blockchain::{Block, BlockPtr};
 use graph::object;
 use graph::prelude::ethabi::ethereum_types::H256;
@@ -67,15 +68,12 @@ async fn data_source_revert() -> anyhow::Result<()> {
         Some(object! { dataSourceCount: object!{ id: "4", count: 4 } })
     );
 
-    fixture::cleanup(&ctx.store, &subgraph_name, &hash);
-
     Ok(())
 }
 
 #[tokio::test]
 async fn typename() -> anyhow::Result<()> {
-    let subgraph_name = SubgraphName::new("typename")
-        .expect("Subgraph name must contain only a-z, A-Z, 0-9, '-' and '_'");
+    let subgraph_name = SubgraphName::new("typename").unwrap();
 
     let hash = {
         let test_dir = format!("./integration-tests/{}", subgraph_name);
@@ -103,7 +101,47 @@ async fn typename() -> anyhow::Result<()> {
 
     ctx.start_and_sync_to(stop_block).await;
 
-    fixture::cleanup(&ctx.store, &subgraph_name, &hash);
-
     Ok(())
 }
+
+#[tokio::test]
+async fn file_data_sources() {
+    let stores = stores("./integration-tests/config.simple.toml").await;
+
+    let subgraph_name = SubgraphName::new("file-data-sources").unwrap();
+    let hash = {
+        let test_dir = format!("./integration-tests/{}", subgraph_name);
+        fixture::build_subgraph(&test_dir).await
+    };
+
+    let blocks = {
+        let block_0 = genesis();
+        let block_1 = empty_block(block_0.ptr(), test_ptr(1));
+        vec![block_0, block_1]
+    };
+    let stop_block = blocks.last().unwrap().block.ptr();
+    let chain = Arc::new(chain(blocks, &stores).await);
+    let ctx = fixture::setup(subgraph_name.clone(), &hash, &stores, chain, None).await;
+    ctx.start_and_sync_to(stop_block).await;
+
+    // CID QmVkvoPGi9jvvuxsHDVJDgzPEzagBaWSZRYoRDzU244HjZ is the file
+    // `file-data-sources/abis/Contract.abi` after being processed by graph-cli.
+    let id = format!(
+        "0x{}",
+        hex::encode(
+            Cid::try_from("QmVkvoPGi9jvvuxsHDVJDgzPEzagBaWSZRYoRDzU244HjZ")
+                .unwrap()
+                .to_bytes(),
+        )
+    );
+
+    let query_res = ctx
+        .query(&format!(r#"{{ ipfsFile(id: "{id}") {{ id, content }} }}"#,))
+        .await
+        .unwrap();
+
+    assert_eq!(
+        query_res,
+        Some(object! { ipfsFile: object!{ id: id , content: "[]" } })
+    );
+}

From d22b51f8251c7b5b9a034a08631588177c5921d2 Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Mon, 22 Aug 2022 14:10:39 +0100
Subject: [PATCH 28/33] runner: Isolate offchain data sources

---
 core/src/subgraph/runner.rs       | 36 ++++++++++++++++++-------------
 graph/src/components/store/mod.rs | 28 ++++++++++++++++++++++++
 tests/src/fixture.rs              |  4 ++--
 3 files changed, 51 insertions(+), 17 deletions(-)

diff --git a/core/src/subgraph/runner.rs b/core/src/subgraph/runner.rs
index 888cf9872a6..9ae886ce805 100644
--- a/core/src/subgraph/runner.rs
+++ b/core/src/subgraph/runner.rs
@@ -6,7 +6,7 @@ use crate::subgraph::stream::new_block_stream;
 use atomic_refcell::AtomicRefCell;
 use graph::blockchain::block_stream::{BlockStreamEvent, BlockWithTriggers, FirehoseCursor};
 use graph::blockchain::{Block, Blockchain, TriggerFilter as _};
-use graph::components::store::{EntityKey, StoredDynamicDataSource};
+use graph::components::store::{EmptyStore, EntityKey, StoredDynamicDataSource};
 use graph::components::{
     store::ModificationsAndCache,
     subgraph::{CausalityRegion, MappingError, ProofOfIndexing, SharedProofOfIndexing},
@@ -565,14 +565,18 @@ where
         &mut self,
         triggers: Vec<offchain::TriggerData>,
     ) -> Result<(Vec<EntityModification>, Vec<StoredDynamicDataSource>), Error> {
-        // TODO: Dont expose store with onchain entites
-        let mut block_state =
-            BlockState::<C>::new(self.inputs.store.cheap_clone(), LfuCache::new());
+        let mut mods = vec![];
+        let mut offchain_to_remove = vec![];
 
         for trigger in triggers {
-            let causality_region = match &trigger.source {
-                offchain::Source::Ipfs(cid) => format!("ipfs/{}", cid.to_string()),
-            };
+            // Using an `EmptyStore` and clearing the cache for each trigger is a makeshift way to
+            // get causality region isolation.
+            let schema = self.inputs.store.input_schema();
+            let mut block_state = BlockState::<C>::new(EmptyStore::new(schema), LfuCache::new());
+
+            // PoI ignores offchain events.
+            let proof_of_indexing = None;
+            let causality_region = "";
 
             // We'll eventually need to do better here, but using an empty block works for now.
             let block = Arc::default();
@@ -583,7 +587,7 @@ where
                     &block,
                     &TriggerData::Offchain(trigger),
                     block_state,
-                    &None,
+                    &proof_of_indexing,
                     &causality_region,
                     &self.inputs.debug_fork,
                     &self.metrics.subgraph,
@@ -597,15 +601,17 @@ where
                     };
                     err.context("failed to process trigger".to_string())
                 })?;
-        }
 
-        anyhow::ensure!(
-            !block_state.has_created_data_sources(),
-            "Attempted to create data source in offchain data source handler. This is not yet supported.",
-        );
+            anyhow::ensure!(
+                !block_state.has_created_data_sources(),
+                "Attempted to create data source in offchain data source handler. This is not yet supported.",
+            );
+
+            mods.extend(block_state.entity_cache.as_modifications()?.modifications);
+            offchain_to_remove.extend(block_state.offchain_to_remove);
+        }
 
-        let mods = block_state.entity_cache.as_modifications()?.modifications;
-        Ok((mods, block_state.offchain_to_remove))
+        Ok((mods, offchain_to_remove))
     }
 }
 
diff --git a/graph/src/components/store/mod.rs b/graph/src/components/store/mod.rs
index 009913d6d28..bfe5040dcaa 100644
--- a/graph/src/components/store/mod.rs
+++ b/graph/src/components/store/mod.rs
@@ -1047,3 +1047,31 @@ impl fmt::Display for DeploymentSchemaVersion {
         fmt::Display::fmt(&(*self as i32), f)
     }
 }
+
+/// A `ReadStore` that is always empty.
+pub struct EmptyStore {
+    schema: Arc<Schema>,
+}
+
+impl EmptyStore {
+    pub fn new(schema: Arc<Schema>) -> Self {
+        EmptyStore { schema }
+    }
+}
+
+impl ReadStore for EmptyStore {
+    fn get(&self, _key: &EntityKey) -> Result<Option<Entity>, StoreError> {
+        Ok(None)
+    }
+
+    fn get_many(
+        &self,
+        _ids_for_type: BTreeMap<&EntityType, Vec<&str>>,
+    ) -> Result<BTreeMap<EntityType, Vec<Entity>>, StoreError> {
+        Ok(BTreeMap::new())
+    }
+
+    fn input_schema(&self) -> Arc<Schema> {
+        self.schema.cheap_clone()
+    }
+}
diff --git a/tests/src/fixture.rs b/tests/src/fixture.rs
index ed6792a2ae2..2a5f2b7562f 100644
--- a/tests/src/fixture.rs
+++ b/tests/src/fixture.rs
@@ -24,7 +24,7 @@ use graph::env::ENV_VARS;
 use graph::ipfs_client::IpfsClient;
 use graph::prelude::ethabi::ethereum_types::H256;
 use graph::prelude::{
-    async_trait, r, BlockNumber, DeploymentHash, GraphQlRunner as _, LoggerFactory,
+    async_trait, r, ApiVersion, BlockNumber, DeploymentHash, GraphQlRunner as _, LoggerFactory,
     MetricsRegistry, NodeId, QueryError, SubgraphAssignmentProvider, SubgraphName,
     SubgraphRegistrar, SubgraphStore as _, SubgraphVersionSwitchingMode,
 };
@@ -124,7 +124,7 @@ impl TestContext {
     }
 
     pub async fn query(&self, query: &str) -> Result<Option<r::Value>, Vec<QueryError>> {
-        let target = QueryTarget::Deployment(self.deployment.hash.clone());
+        let target = QueryTarget::Deployment(self.deployment.hash.clone(), ApiVersion::default());
 
         self.graphql_runner
             .clone()

From 824957ba09e97bc53ad2762c2cd7364cd51955f4 Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Mon, 22 Aug 2022 15:10:56 +0100
Subject: [PATCH 29/33] offchain: Forbid static file data sources

---
 chain/ethereum/tests/manifest.rs  | 28 ++++++++++------------------
 graph/src/data_source/offchain.rs |  7 +++++++
 2 files changed, 17 insertions(+), 18 deletions(-)

diff --git a/chain/ethereum/tests/manifest.rs b/chain/ethereum/tests/manifest.rs
index cf9fdb8f626..2c82f1ff657 100644
--- a/chain/ethereum/tests/manifest.rs
+++ b/chain/ethereum/tests/manifest.rs
@@ -4,7 +4,7 @@ use std::time::Duration;
 
 use graph::data::subgraph::schema::SubgraphError;
 use graph::data::subgraph::{SPEC_VERSION_0_0_4, SPEC_VERSION_0_0_7};
-use graph::data_source::{offchain, DataSource};
+use graph::data_source::DataSourceTemplate;
 use graph::prelude::{
     anyhow, async_trait, serde_yaml, tokio, DeploymentHash, Entity, Link, Logger, SubgraphManifest,
     SubgraphManifestValidationError, UnvalidatedSubgraphManifest,
@@ -130,17 +130,14 @@ specVersion: 0.0.2
 
 #[tokio::test]
 async fn ipfs_manifest() {
-    let yaml = format!(
-        "
+    let yaml = "
 schema:
   file:
     /: /ipfs/Qmschema
-dataSources:
+dataSources: []
+templates:
   - name: IpfsSource
     kind: file/ipfs
-    source:
-      file:
-        /: {}
     mapping:
       apiVersion: 0.0.6
       language: wasm/assemblyscript
@@ -150,22 +147,17 @@ dataSources:
         /: /ipfs/Qmmapping
       handler: handleFile
 specVersion: 0.0.7
-",
-        FILE_CID
-    );
+";
 
     let manifest = resolve_manifest(&yaml, SPEC_VERSION_0_0_7).await;
 
     assert_eq!("Qmmanifest", manifest.id.as_str());
-    assert_eq!(manifest.data_sources.len(), 1);
-    let data_source = match &manifest.data_sources[0] {
-        DataSource::Offchain(ds) => ds,
-        DataSource::Onchain(_) => unreachable!(),
+    assert_eq!(manifest.data_sources.len(), 0);
+    let data_source = match &manifest.templates[0] {
+        DataSourceTemplate::Offchain(ds) => ds,
+        DataSourceTemplate::Onchain(_) => unreachable!(),
     };
-    assert_eq!(
-        data_source.source,
-        offchain::Source::Ipfs(FILE_CID.parse().unwrap())
-    );
+    assert_eq!(data_source.kind, "file/ipfs");
 }
 
 #[tokio::test]
diff --git a/graph/src/data_source/offchain.rs b/graph/src/data_source/offchain.rs
index 51929f9b883..f0d8ced9562 100644
--- a/graph/src/data_source/offchain.rs
+++ b/graph/src/data_source/offchain.rs
@@ -155,12 +155,19 @@ pub struct UnresolvedMapping {
 }
 
 impl UnresolvedDataSource {
+    #[allow(unreachable_code)]
+    #[allow(unused_variables)]
     pub async fn resolve(
         self,
         resolver: &Arc<dyn LinkResolver>,
         logger: &Logger,
         manifest_idx: u32,
     ) -> Result<DataSource, Error> {
+        anyhow::bail!(
+            "static file data sources are not yet supported, \\
+             for details see https://github.com/graphprotocol/graph-node/issues/3864"
+        );
+
         info!(logger, "Resolve offchain data source";
             "name" => &self.name,
             "kind" => &self.kind,

From 717b1d0c9d100cfbd422de9dc26daec861814761 Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Mon, 22 Aug 2022 15:46:19 +0100
Subject: [PATCH 30/33] store: Assign separate causality region for offchain
 dses

---
 chain/ethereum/src/data_source.rs   |  7 +++
 graph/src/components/store/mod.rs   |  1 +
 graph/src/data_source/offchain.rs   |  1 +
 store/postgres/src/dynds/private.rs | 85 +++++++++++++++++++----------
 store/postgres/src/dynds/shared.rs  | 12 ++++
 tests/tests/runner.rs               | 10 +++-
 6 files changed, 85 insertions(+), 31 deletions(-)

diff --git a/chain/ethereum/src/data_source.rs b/chain/ethereum/src/data_source.rs
index 0eae1fd428d..9bf89494cfc 100644
--- a/chain/ethereum/src/data_source.rs
+++ b/chain/ethereum/src/data_source.rs
@@ -131,6 +131,7 @@ impl blockchain::DataSource<Chain> for DataSource {
                 .as_ref()
                 .map(|ctx| serde_json::to_value(&ctx).unwrap()),
             creation_block: self.creation_block,
+            is_offchain: false,
         }
     }
 
@@ -143,8 +144,14 @@ impl blockchain::DataSource<Chain> for DataSource {
             param,
             context,
             creation_block,
+            is_offchain,
         } = stored;
 
+        ensure!(
+            !is_offchain,
+            "attempted to convert offchain data source to ethereum data source"
+        );
+
         let context = context.map(serde_json::from_value).transpose()?;
 
         let contract_abi = template.mapping.find_abi(&template.source.abi)?;
diff --git a/graph/src/components/store/mod.rs b/graph/src/components/store/mod.rs
index bfe5040dcaa..eba7f9fab82 100644
--- a/graph/src/components/store/mod.rs
+++ b/graph/src/components/store/mod.rs
@@ -799,6 +799,7 @@ pub struct StoredDynamicDataSource {
     pub param: Option<Bytes>,
     pub context: Option<serde_json::Value>,
     pub creation_block: Option<BlockNumber>,
+    pub is_offchain: bool,
 }
 
 /// An internal identifer for the specific instance of a deployment. The
diff --git a/graph/src/data_source/offchain.rs b/graph/src/data_source/offchain.rs
index f0d8ced9562..a36ca1f3fa1 100644
--- a/graph/src/data_source/offchain.rs
+++ b/graph/src/data_source/offchain.rs
@@ -86,6 +86,7 @@ impl DataSource {
             param: Some(param),
             context,
             creation_block: self.creation_block,
+            is_offchain: true,
         }
     }
 
diff --git a/store/postgres/src/dynds/private.rs b/store/postgres/src/dynds/private.rs
index 981818d03ff..476846534f8 100644
--- a/store/postgres/src/dynds/private.rs
+++ b/store/postgres/src/dynds/private.rs
@@ -25,7 +25,7 @@ pub(crate) struct DataSourcesTable {
     table: DynTable,
     vid: DynColumn<Integer>,
     block_range: DynColumn<sql_types::Range<Integer>>,
-    _causality_region: DynColumn<Integer>,
+    causality_region: DynColumn<Integer>,
     manifest_idx: DynColumn<Integer>,
     param: DynColumn<Nullable<Binary>>,
     context: DynColumn<Nullable<Jsonb>>,
@@ -43,7 +43,7 @@ impl DataSourcesTable {
             namespace,
             vid: table.column("vid"),
             block_range: table.column("block_range"),
-            _causality_region: table.column("causality_region"),
+            causality_region: table.column("causality_region"),
             manifest_idx: table.column("manifest_idx"),
             param: table.column("param"),
             context: table.column("context"),
@@ -85,6 +85,7 @@ impl DataSourcesTable {
             i32,
             Option<Vec<u8>>,
             Option<serde_json::Value>,
+            i32,
         );
         let tuples = self
             .table
@@ -95,26 +96,34 @@ impl DataSourcesTable {
                 &self.manifest_idx,
                 &self.param,
                 &self.context,
+                &self.causality_region,
             ))
             .order_by(&self.vid)
             .load::<Tuple>(conn)?;
 
         let mut dses: Vec<_> = tuples
             .into_iter()
-            .map(|(block_range, manifest_idx, param, context)| {
-                let creation_block = match block_range.0 {
-                    Bound::Included(block) => Some(block),
-
-                    // Should never happen.
-                    Bound::Excluded(_) | Bound::Unbounded => unreachable!("dds with open creation"),
-                };
-                StoredDynamicDataSource {
-                    manifest_idx: manifest_idx as u32,
-                    param: param.map(|p| p.into()),
-                    context,
-                    creation_block,
-                }
-            })
+            .map(
+                |(block_range, manifest_idx, param, context, causality_region)| {
+                    let creation_block = match block_range.0 {
+                        Bound::Included(block) => Some(block),
+
+                        // Should never happen.
+                        Bound::Excluded(_) | Bound::Unbounded => {
+                            unreachable!("dds with open creation")
+                        }
+                    };
+
+                    let is_offchain = causality_region > 0;
+                    StoredDynamicDataSource {
+                        manifest_idx: manifest_idx as u32,
+                        param: param.map(|p| p.into()),
+                        context,
+                        creation_block,
+                        is_offchain,
+                    }
+                },
+            )
             .collect();
 
         // This sort is stable and `tuples` was ordered by vid, so `dses` will be ordered by `(creation_block, vid)`.
@@ -129,9 +138,6 @@ impl DataSourcesTable {
         data_sources: &[StoredDynamicDataSource],
         block: BlockNumber,
     ) -> Result<usize, StoreError> {
-        // Currently all data sources share the same causality region.
-        let causality_region = 0;
-
         let mut inserted_total = 0;
 
         for ds in data_sources {
@@ -140,6 +146,7 @@ impl DataSourcesTable {
                 param,
                 context,
                 creation_block,
+                is_offchain,
             } = ds;
 
             if creation_block != &Some(block) {
@@ -150,19 +157,32 @@ impl DataSourcesTable {
                 ));
             }
 
-            let query = format!(
-                "insert into {}(block_range, manifest_idx, causality_region, param, context) \
-                 values (int4range($1, null), $2, $3, $4, $5)",
-                self.qname
-            );
-
-            inserted_total += sql_query(query)
+            // Onchain data sources have the causality region explicitly set to 0, while offchain
+            // data sources have an unique causality region assigned from the sequence.
+            let query = match is_offchain {
+                false => format!(
+                    "insert into {}(block_range, manifest_idx, param, context, causality_region) \
+                            values (int4range($1, null), $2, $3, $4, $5)",
+                    self.qname
+                ),
+
+                true => format!(
+                    "insert into {}(block_range, manifest_idx, param, context) \
+                            values (int4range($1, null), $2, $3, $4)",
+                    self.qname
+                ),
+            };
+
+            let query = sql_query(query)
                 .bind::<Nullable<Integer>, _>(creation_block)
                 .bind::<Integer, _>(*manifest_idx as i32)
-                .bind::<Integer, _>(causality_region)
                 .bind::<Nullable<Binary>, _>(param.as_ref().map(|p| &**p))
-                .bind::<Nullable<Jsonb>, _>(context)
-                .execute(conn)?;
+                .bind::<Nullable<Jsonb>, _>(context);
+
+            inserted_total += match is_offchain {
+                false => query.bind::<Integer, _>(0).execute(conn)?,
+                true => query.execute(conn)?,
+            };
         }
 
         Ok(inserted_total)
@@ -233,8 +253,15 @@ impl DataSourcesTable {
                 param,
                 context,
                 creation_block,
+                is_offchain,
             } = ds;
 
+            if !is_offchain {
+                return Err(constraint_violation!(
+                    "called remove_offchain with onchain data sources"
+                ));
+            }
+
             let query = format!(
                 "update {} set block_range = 'empty'::int4range \
                  where manifest_idx = $1
diff --git a/store/postgres/src/dynds/shared.rs b/store/postgres/src/dynds/shared.rs
index 5fb124ae59e..a151a779cbe 100644
--- a/store/postgres/src/dynds/shared.rs
+++ b/store/postgres/src/dynds/shared.rs
@@ -81,6 +81,10 @@ pub(super) fn load(
             param: Some(address.into()),
             context: context.map(|ctx| serde_json::from_str(&ctx)).transpose()?,
             creation_block,
+
+            // The shared schema is only used for legacy deployments, and therefore not used for
+            // subgraphs that use file data sources.
+            is_offchain: false,
         };
 
         if data_sources.last().and_then(|d| d.creation_block) > data_source.creation_block {
@@ -116,7 +120,15 @@ pub(super) fn insert(
                 param,
                 context,
                 creation_block: _,
+                is_offchain,
             } = ds;
+
+            if *is_offchain {
+                return Err(constraint_violation!(
+                    "using shared data source schema with file data sources"
+                ));
+            }
+
             let address = match param {
                 Some(param) => param,
                 None => {
diff --git a/tests/tests/runner.rs b/tests/tests/runner.rs
index 0b0281e4455..133be063034 100644
--- a/tests/tests/runner.rs
+++ b/tests/tests/runner.rs
@@ -117,9 +117,10 @@ async fn file_data_sources() {
     let blocks = {
         let block_0 = genesis();
         let block_1 = empty_block(block_0.ptr(), test_ptr(1));
-        vec![block_0, block_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 stop_block = test_ptr(1);
     let chain = Arc::new(chain(blocks, &stores).await);
     let ctx = fixture::setup(subgraph_name.clone(), &hash, &stores, chain, None).await;
     ctx.start_and_sync_to(stop_block).await;
@@ -144,4 +145,9 @@ async fn file_data_sources() {
         query_res,
         Some(object! { ipfsFile: object!{ id: id , content: "[]" } })
     );
+
+    // Test loading offchain data sources from DB.
+    ctx.provider.stop(ctx.deployment.clone()).await.unwrap();
+    let stop_block = test_ptr(2);
+    ctx.start_and_sync_to(stop_block).await;
 }

From ae8e2d7fc65ea21b835c081bd9a1b6aed9658566 Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Tue, 23 Aug 2022 11:07:49 +0100
Subject: [PATCH 31/33] graph: Fix release build

---
 .github/workflows/ci.yml       | 2 +-
 graph/src/data/query/result.rs | 1 -
 2 files changed, 1 insertion(+), 2 deletions(-)

diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml
index 03d41432c39..984ca9964ec 100644
--- a/.github/workflows/ci.yml
+++ b/.github/workflows/ci.yml
@@ -120,7 +120,7 @@ jobs:
           TESTS_GANACHE_HARD_WAIT_SECONDS: "30"
         with:
           command: test
-          args: --verbose --package graph-tests -- --nocapture --skip parallel_integration_tests
+          args: --verbose --package graph-tests -- --skip parallel_integration_tests
 
   integration-tests:
     name: Run integration tests
diff --git a/graph/src/data/query/result.rs b/graph/src/data/query/result.rs
index 1afbe509994..6b8bebd8766 100644
--- a/graph/src/data/query/result.rs
+++ b/graph/src/data/query/result.rs
@@ -222,7 +222,6 @@ impl QueryResult {
     /// This is really `clone`, but we do not want to implement `Clone`;
     /// this is only meant for test purposes and should not be used in production
     /// code since cloning query results can be very expensive
-    #[cfg(debug_assertions)]
     pub fn duplicate(&self) -> Self {
         Self {
             data: self.data.clone(),

From 11633ef6037e6c4dc586c2b1ce34aa6ae70b0231 Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Tue, 23 Aug 2022 17:13:27 +0100
Subject: [PATCH 32/33] tests: yarn upgrade, add file ds to the workspace

---
 .../data-source-revert/grafted.yaml           |   6 +-
 .../data-source-revert/subgraph.yaml          |   4 +-
 tests/integration-tests/package.json          |   5 +-
 tests/integration-tests/yarn.lock             | 952 +++++++++++-------
 4 files changed, 621 insertions(+), 346 deletions(-)

diff --git a/tests/integration-tests/data-source-revert/grafted.yaml b/tests/integration-tests/data-source-revert/grafted.yaml
index 3cb184beeec..64e9180a1fc 100644
--- a/tests/integration-tests/data-source-revert/grafted.yaml
+++ b/tests/integration-tests/data-source-revert/grafted.yaml
@@ -5,7 +5,7 @@ schema:
   file: ./schema.graphql
 graft:
   # Must match the id from building `subgraph.yaml`
-  base: QmRhW72iAE6AEY6fiL9nPt5ZVffzbq9XswKDbH9LC3JPUh
+  base: QmW6kCVyaV3d37Pm9ZZKZAWU9G6oSooJiarjBKrqLE9FJy
   block: 3
 dataSources:
   - kind: ethereum/contract
@@ -16,7 +16,7 @@ dataSources:
       abi: Contract
     mapping:
       kind: ethereum/events
-      apiVersion: 0.0.5
+      apiVersion: 0.0.6
       language: wasm/assemblyscript
       entities:
         - Gravatar
@@ -34,7 +34,7 @@ templates:
       abi: Contract
     mapping:
       kind: ethereum/events
-      apiVersion: 0.0.5
+      apiVersion: 0.0.6
       language: wasm/assemblyscript
       entities:
         - Gravatar
diff --git a/tests/integration-tests/data-source-revert/subgraph.yaml b/tests/integration-tests/data-source-revert/subgraph.yaml
index 87f85563093..1c6d33908a0 100644
--- a/tests/integration-tests/data-source-revert/subgraph.yaml
+++ b/tests/integration-tests/data-source-revert/subgraph.yaml
@@ -10,7 +10,7 @@ dataSources:
       abi: Contract
     mapping:
       kind: ethereum/events
-      apiVersion: 0.0.5
+      apiVersion: 0.0.6
       language: wasm/assemblyscript
       entities:
         - Gravatar
@@ -28,7 +28,7 @@ templates:
       abi: Contract
     mapping:
       kind: ethereum/events
-      apiVersion: 0.0.5
+      apiVersion: 0.0.6
       language: wasm/assemblyscript
       entities:
         - Gravatar
diff --git a/tests/integration-tests/package.json b/tests/integration-tests/package.json
index 15771107466..845bd090977 100644
--- a/tests/integration-tests/package.json
+++ b/tests/integration-tests/package.json
@@ -11,6 +11,7 @@
     "poi-for-failed-subgraph",
     "remove-then-update",
     "typename",
-    "value-roundtrip"
+    "value-roundtrip",
+    "file-data-sources"
   ]
-}
+}
\ No newline at end of file
diff --git a/tests/integration-tests/yarn.lock b/tests/integration-tests/yarn.lock
index 33892455546..f1b8e0c67d9 100644
--- a/tests/integration-tests/yarn.lock
+++ b/tests/integration-tests/yarn.lock
@@ -72,7 +72,14 @@
   dependencies:
     tslib "~2.0.1"
 
-"@babel/code-frame@^7.0.0", "@babel/code-frame@^7.12.13":
+"@babel/code-frame@^7.0.0":
+  version "7.18.6"
+  resolved "https://registry.yarnpkg.com/@babel/code-frame/-/code-frame-7.18.6.tgz#3b25d38c89600baa2dcc219edfa88a74eb2c427a"
+  integrity sha512-TDCmlK5eOvH+eH7cdAFlNXeVJqWIQ7gW9tY1GJIpUtFb6CmjVyq2VM3u71bOyR8CRihcCgMUYoDNyLXao3+70Q==
+  dependencies:
+    "@babel/highlight" "^7.18.6"
+
+"@babel/code-frame@^7.12.13":
   version "7.12.13"
   resolved "https://registry.yarnpkg.com/@babel/code-frame/-/code-frame-7.12.13.tgz#dcfc826beef65e75c50e21d3837d7d95798dd658"
   integrity sha512-HV1Cm0Q3ZrpCR93tkWOYiuYIgLxZXZFVG2VgK+MBWjUqZTundupbfx2aXarXuw5Ko5aMcjtJgbSs4vUGBS5v6g==
@@ -245,10 +252,10 @@
   dependencies:
     "@babel/types" "^7.12.13"
 
-"@babel/helper-validator-identifier@^7.12.11":
-  version "7.12.11"
-  resolved "https://registry.yarnpkg.com/@babel/helper-validator-identifier/-/helper-validator-identifier-7.12.11.tgz#c9a1f021917dcb5ccf0d4e453e399022981fc9ed"
-  integrity sha512-np/lG3uARFybkoHokJUmf1QfEvRVCPbmQeUQpKow5cQ3xWrV9i3rUHodKDJPQfTVX61qKi+UdYk8kik84n7XOw==
+"@babel/helper-validator-identifier@^7.12.11", "@babel/helper-validator-identifier@^7.18.6":
+  version "7.18.6"
+  resolved "https://registry.yarnpkg.com/@babel/helper-validator-identifier/-/helper-validator-identifier-7.18.6.tgz#9c97e30d31b2b8c72a1d08984f2ca9b574d7a076"
+  integrity sha512-MmetCkz9ej86nJQV+sFCxoGGrUbU3q02kgLciwkrt9QqEB7cP39oKEY0PakknEO0Gu20SskMRi+AYZ3b1TpN9g==
 
 "@babel/helper-validator-option@^7.12.17":
   version "7.12.17"
@@ -264,12 +271,12 @@
     "@babel/traverse" "^7.13.0"
     "@babel/types" "^7.13.0"
 
-"@babel/highlight@^7.12.13":
-  version "7.13.10"
-  resolved "https://registry.yarnpkg.com/@babel/highlight/-/highlight-7.13.10.tgz#a8b2a66148f5b27d666b15d81774347a731d52d1"
-  integrity sha512-5aPpe5XQPzflQrFwL1/QoeHkP2MsA4JCntcXHRhEsdsfPVkvPi2w7Qix4iV7t5S/oC9OodGrggd8aco1g3SZFg==
+"@babel/highlight@^7.12.13", "@babel/highlight@^7.18.6":
+  version "7.18.6"
+  resolved "https://registry.yarnpkg.com/@babel/highlight/-/highlight-7.18.6.tgz#81158601e93e2563795adcbfbdf5d64be3f2ecdf"
+  integrity sha512-u7stbOuYjaPezCuLj29hNW1v64M2Md2qupEKP1fHc7WdOA3DgLh37suiSrZYY7haUB7iBeQZ9P1uiRF359do3g==
   dependencies:
-    "@babel/helper-validator-identifier" "^7.12.11"
+    "@babel/helper-validator-identifier" "^7.18.6"
     chalk "^2.0.0"
     js-tokens "^4.0.0"
 
@@ -507,9 +514,9 @@
     regenerator-runtime "^0.13.4"
 
 "@babel/runtime@^7.9.2":
-  version "7.18.3"
-  resolved "https://registry.yarnpkg.com/@babel/runtime/-/runtime-7.18.3.tgz#c7b654b57f6f63cf7f8b418ac9ca04408c4579f4"
-  integrity sha512-38Y8f7YUhce/K7RMwTp7m0uCumpv9hZkitCbBClqQIow1qSbCvGkcegKOXpEWCQLfWmevgRiWokZ1GkpfhbZug==
+  version "7.18.9"
+  resolved "https://registry.yarnpkg.com/@babel/runtime/-/runtime-7.18.9.tgz#b4fcfce55db3d2e5e080d2490f608a3b9f407f4a"
+  integrity sha512-lkqXDcvlFT5rvEjiu6+QYO+1GXrEHRo2LOtS7E4GtX5ESIZOgepqsZBVIj6Pv+a6zqsya9VCgiK1KAK4BvJDAw==
   dependencies:
     regenerator-runtime "^0.13.4"
 
@@ -600,31 +607,31 @@
     "@ethersproject/properties" "^5.0.3"
     "@ethersproject/strings" "^5.0.4"
 
-"@ethersproject/abstract-provider@^5.0.8":
-  version "5.0.10"
-  resolved "https://registry.yarnpkg.com/@ethersproject/abstract-provider/-/abstract-provider-5.0.10.tgz#a533aed39a5f27312745c8c4c40fa25fc884831c"
-  integrity sha512-OSReY5iz94iIaPlRvLiJP8YVIvQLx4aUvMMnHWSaA/vTU8QHZmgNlt4OBdYV1+aFY8Xl+VRYiWBHq72ZDKXXCQ==
-  dependencies:
-    "@ethersproject/bignumber" "^5.0.13"
-    "@ethersproject/bytes" "^5.0.9"
-    "@ethersproject/logger" "^5.0.8"
-    "@ethersproject/networks" "^5.0.7"
-    "@ethersproject/properties" "^5.0.7"
-    "@ethersproject/transactions" "^5.0.9"
-    "@ethersproject/web" "^5.0.12"
-
-"@ethersproject/abstract-signer@^5.0.10":
-  version "5.0.14"
-  resolved "https://registry.yarnpkg.com/@ethersproject/abstract-signer/-/abstract-signer-5.0.14.tgz#30ef912b0f86599d90fdffc65c110452e7b55cf1"
-  integrity sha512-JztBwVO7o5OHLh2vyjordlS4/1EjRyaECtc8vPdXTF1i4dXN+J0coeRoPN6ZFbBvi/YbaB6br2fvqhst1VQD/g==
-  dependencies:
-    "@ethersproject/abstract-provider" "^5.0.8"
-    "@ethersproject/bignumber" "^5.0.13"
-    "@ethersproject/bytes" "^5.0.9"
-    "@ethersproject/logger" "^5.0.8"
-    "@ethersproject/properties" "^5.0.7"
-
-"@ethersproject/address@>=5.0.0-beta.128", "@ethersproject/address@^5.0.4", "@ethersproject/address@^5.0.9":
+"@ethersproject/abstract-provider@^5.7.0":
+  version "5.7.0"
+  resolved "https://registry.yarnpkg.com/@ethersproject/abstract-provider/-/abstract-provider-5.7.0.tgz#b0a8550f88b6bf9d51f90e4795d48294630cb9ef"
+  integrity sha512-R41c9UkchKCpAqStMYUpdunjo3pkEvZC3FAwZn5S5MGbXoMQOHIdHItezTETxAO5bevtMApSyEhn9+CHcDsWBw==
+  dependencies:
+    "@ethersproject/bignumber" "^5.7.0"
+    "@ethersproject/bytes" "^5.7.0"
+    "@ethersproject/logger" "^5.7.0"
+    "@ethersproject/networks" "^5.7.0"
+    "@ethersproject/properties" "^5.7.0"
+    "@ethersproject/transactions" "^5.7.0"
+    "@ethersproject/web" "^5.7.0"
+
+"@ethersproject/abstract-signer@^5.0.10", "@ethersproject/abstract-signer@^5.7.0":
+  version "5.7.0"
+  resolved "https://registry.yarnpkg.com/@ethersproject/abstract-signer/-/abstract-signer-5.7.0.tgz#13f4f32117868452191a4649723cb086d2b596b2"
+  integrity sha512-a16V8bq1/Cz+TGCkE2OPMTOUDLS3grCpdjoJCYNnVBbdYEMSgKrU0+B90s8b6H+ByYTBZN7a3g76jdIJi7UfKQ==
+  dependencies:
+    "@ethersproject/abstract-provider" "^5.7.0"
+    "@ethersproject/bignumber" "^5.7.0"
+    "@ethersproject/bytes" "^5.7.0"
+    "@ethersproject/logger" "^5.7.0"
+    "@ethersproject/properties" "^5.7.0"
+
+"@ethersproject/address@>=5.0.0-beta.128":
   version "5.0.11"
   resolved "https://registry.yarnpkg.com/@ethersproject/address/-/address-5.0.11.tgz#12022e8c590c33939beb5ab18b401ecf585eac59"
   integrity sha512-Et4GBdD8/tsBGjCEOKee9upN29qjL5kbRcmJifb4Penmiuh9GARXL2/xpXvEp5EW+EIW/rfCHFJrkYBgoQFQBw==
@@ -635,14 +642,25 @@
     "@ethersproject/logger" "^5.0.8"
     "@ethersproject/rlp" "^5.0.7"
 
-"@ethersproject/base64@^5.0.7":
-  version "5.0.9"
-  resolved "https://registry.yarnpkg.com/@ethersproject/base64/-/base64-5.0.9.tgz#bb1f35d3dba92082a574d5e2418f9202a0a1a7e6"
-  integrity sha512-37RBz5LEZ9SlTNGiWCYFttnIN9J7qVs9Xo2EbqGqDH5LfW9EIji66S+YDMpXVo1zWDax1FkEldAoatxHK2gfgA==
+"@ethersproject/address@^5.0.4", "@ethersproject/address@^5.0.9", "@ethersproject/address@^5.7.0":
+  version "5.7.0"
+  resolved "https://registry.yarnpkg.com/@ethersproject/address/-/address-5.7.0.tgz#19b56c4d74a3b0a46bfdbb6cfcc0a153fc697f37"
+  integrity sha512-9wYhYt7aghVGo758POM5nqcOMaE168Q6aRLJZwUmiqSrAungkG74gSSeKEIR7ukixesdRZGPgVqme6vmxs1fkA==
   dependencies:
-    "@ethersproject/bytes" "^5.0.9"
+    "@ethersproject/bignumber" "^5.7.0"
+    "@ethersproject/bytes" "^5.7.0"
+    "@ethersproject/keccak256" "^5.7.0"
+    "@ethersproject/logger" "^5.7.0"
+    "@ethersproject/rlp" "^5.7.0"
+
+"@ethersproject/base64@^5.7.0":
+  version "5.7.0"
+  resolved "https://registry.yarnpkg.com/@ethersproject/base64/-/base64-5.7.0.tgz#ac4ee92aa36c1628173e221d0d01f53692059e1c"
+  integrity sha512-Dr8tcHt2mEbsZr/mwTPIQAf3Ai0Bks/7gTw9dSqk1mQvhW3XvRlmDJr/4n+wg1JmCl16NZue17CDh8xb/vZ0sQ==
+  dependencies:
+    "@ethersproject/bytes" "^5.7.0"
 
-"@ethersproject/bignumber@>=5.0.0-beta.130", "@ethersproject/bignumber@^5.0.13", "@ethersproject/bignumber@^5.0.7":
+"@ethersproject/bignumber@>=5.0.0-beta.130":
   version "5.0.15"
   resolved "https://registry.yarnpkg.com/@ethersproject/bignumber/-/bignumber-5.0.15.tgz#b089b3f1e0381338d764ac1c10512f0c93b184ed"
   integrity sha512-MTADqnyacvdRwtKh7o9ujwNDSM1SDJjYDMYAzjIgjoi9rh6TY4suMbhCa3i2vh3SUXiXSICyTI8ui+NPdrZ9Lw==
@@ -651,21 +669,44 @@
     "@ethersproject/logger" "^5.0.8"
     bn.js "^4.4.0"
 
-"@ethersproject/bytes@>=5.0.0-beta.129", "@ethersproject/bytes@^5.0.4", "@ethersproject/bytes@^5.0.9":
+"@ethersproject/bignumber@^5.0.13", "@ethersproject/bignumber@^5.0.7", "@ethersproject/bignumber@^5.7.0":
+  version "5.7.0"
+  resolved "https://registry.yarnpkg.com/@ethersproject/bignumber/-/bignumber-5.7.0.tgz#e2f03837f268ba655ffba03a57853e18a18dc9c2"
+  integrity sha512-n1CAdIHRWjSucQO3MC1zPSVgV/6dy/fjL9pMrPP9peL+QxEg9wOsVqwD4+818B6LUEtaXzVHQiuivzRoxPxUGw==
+  dependencies:
+    "@ethersproject/bytes" "^5.7.0"
+    "@ethersproject/logger" "^5.7.0"
+    bn.js "^5.2.1"
+
+"@ethersproject/bytes@>=5.0.0-beta.129":
   version "5.0.11"
   resolved "https://registry.yarnpkg.com/@ethersproject/bytes/-/bytes-5.0.11.tgz#21118e75b1d00db068984c15530e316021101276"
   integrity sha512-D51plLYY5qF05AsoVQwIZVLqlBkaTPVHVP/1WmmBIWyHB0cRW0C9kh0kx5Exo51rB63Hk8PfHxc7SmpoaQFEyg==
   dependencies:
     "@ethersproject/logger" "^5.0.8"
 
-"@ethersproject/constants@>=5.0.0-beta.128", "@ethersproject/constants@^5.0.4", "@ethersproject/constants@^5.0.8":
+"@ethersproject/bytes@^5.0.4", "@ethersproject/bytes@^5.0.9", "@ethersproject/bytes@^5.7.0":
+  version "5.7.0"
+  resolved "https://registry.yarnpkg.com/@ethersproject/bytes/-/bytes-5.7.0.tgz#a00f6ea8d7e7534d6d87f47188af1148d71f155d"
+  integrity sha512-nsbxwgFXWh9NyYWo+U8atvmMsSdKJprTcICAkvbBffT75qDocbuggBU0SJiVK2MuTrp0q+xvLkTnGMPK1+uA9A==
+  dependencies:
+    "@ethersproject/logger" "^5.7.0"
+
+"@ethersproject/constants@>=5.0.0-beta.128":
   version "5.0.10"
   resolved "https://registry.yarnpkg.com/@ethersproject/constants/-/constants-5.0.10.tgz#eb0c604fbc44c53ba9641eed31a1d0c9e1ebcadc"
   integrity sha512-OSo8jxkHLDXieCy8bgOFR7lMfgPxEzKvSDdP+WAWHCDM8+orwch0B6wzkTmiQFgryAtIctrBt5glAdJikZ3hGw==
   dependencies:
     "@ethersproject/bignumber" "^5.0.13"
 
-"@ethersproject/hash@>=5.0.0-beta.128", "@ethersproject/hash@^5.0.4":
+"@ethersproject/constants@^5.0.4", "@ethersproject/constants@^5.0.8", "@ethersproject/constants@^5.7.0":
+  version "5.7.0"
+  resolved "https://registry.yarnpkg.com/@ethersproject/constants/-/constants-5.7.0.tgz#df80a9705a7e08984161f09014ea012d1c75295e"
+  integrity sha512-DHI+y5dBNvkpYUMiRQyxRBYBefZkJfo70VUkUAsRjcPs47muV9evftfZ0PJVCXYbAiCgght0DtcF9srFQmIgWA==
+  dependencies:
+    "@ethersproject/bignumber" "^5.7.0"
+
+"@ethersproject/hash@>=5.0.0-beta.128":
   version "5.0.12"
   resolved "https://registry.yarnpkg.com/@ethersproject/hash/-/hash-5.0.12.tgz#1074599f7509e2ca2bb7a3d4f4e39ab3a796da42"
   integrity sha512-kn4QN+fhNFbUgX3XZTZUaQixi0oyfIEY+hfW+KtkHu+rq7dV76oAIvaLEEynu1/4npOL38E4X4YI42gGZk+C0Q==
@@ -679,7 +720,22 @@
     "@ethersproject/properties" "^5.0.7"
     "@ethersproject/strings" "^5.0.8"
 
-"@ethersproject/keccak256@>=5.0.0-beta.127", "@ethersproject/keccak256@^5.0.3", "@ethersproject/keccak256@^5.0.7":
+"@ethersproject/hash@^5.0.4":
+  version "5.7.0"
+  resolved "https://registry.yarnpkg.com/@ethersproject/hash/-/hash-5.7.0.tgz#eb7aca84a588508369562e16e514b539ba5240a7"
+  integrity sha512-qX5WrQfnah1EFnO5zJv1v46a8HW0+E5xuBBDTwMFZLuVTx0tbU2kkx15NqdjxecrLGatQN9FGQKpb1FKdHCt+g==
+  dependencies:
+    "@ethersproject/abstract-signer" "^5.7.0"
+    "@ethersproject/address" "^5.7.0"
+    "@ethersproject/base64" "^5.7.0"
+    "@ethersproject/bignumber" "^5.7.0"
+    "@ethersproject/bytes" "^5.7.0"
+    "@ethersproject/keccak256" "^5.7.0"
+    "@ethersproject/logger" "^5.7.0"
+    "@ethersproject/properties" "^5.7.0"
+    "@ethersproject/strings" "^5.7.0"
+
+"@ethersproject/keccak256@>=5.0.0-beta.127":
   version "5.0.9"
   resolved "https://registry.yarnpkg.com/@ethersproject/keccak256/-/keccak256-5.0.9.tgz#ca0d86e4af56c13b1ef25e533bde3e96d28f647d"
   integrity sha512-zhdUTj6RGtCJSgU+bDrWF6cGbvW453LoIC1DSNWrTlXzC7WuH4a+EiPrgc7/kNoRxerKuA/cxYlI8GwNtVtDlw==
@@ -687,44 +743,66 @@
     "@ethersproject/bytes" "^5.0.9"
     js-sha3 "0.5.7"
 
-"@ethersproject/logger@>=5.0.0-beta.129", "@ethersproject/logger@^5.0.5", "@ethersproject/logger@^5.0.8":
+"@ethersproject/keccak256@^5.0.3", "@ethersproject/keccak256@^5.0.7", "@ethersproject/keccak256@^5.7.0":
+  version "5.7.0"
+  resolved "https://registry.yarnpkg.com/@ethersproject/keccak256/-/keccak256-5.7.0.tgz#3186350c6e1cd6aba7940384ec7d6d9db01f335a"
+  integrity sha512-2UcPboeL/iW+pSg6vZ6ydF8tCnv3Iu/8tUmLLzWWGzxWKFFqOBQFLo6uLUv6BDrLgCDfN28RJ/wtByx+jZ4KBg==
+  dependencies:
+    "@ethersproject/bytes" "^5.7.0"
+    js-sha3 "0.8.0"
+
+"@ethersproject/logger@>=5.0.0-beta.129":
   version "5.0.10"
   resolved "https://registry.yarnpkg.com/@ethersproject/logger/-/logger-5.0.10.tgz#fd884688b3143253e0356ef92d5f22d109d2e026"
   integrity sha512-0y2T2NqykDrbPM3Zw9RSbPkDOxwChAL8detXaom76CfYoGxsOnRP/zTX8OUAV+x9LdwzgbWvWmeXrc0M7SuDZw==
 
-"@ethersproject/networks@^5.0.7":
-  version "5.0.9"
-  resolved "https://registry.yarnpkg.com/@ethersproject/networks/-/networks-5.0.9.tgz#ec5da11e4d4bfd69bec4eaebc9ace33eb9569279"
-  integrity sha512-L8+VCQwArBLGkxZb/5Ns/OH/OxP38AcaveXIxhUTq+VWpXYjrObG3E7RDQIKkUx1S1IcQl/UWTz5w4DK0UitJg==
+"@ethersproject/logger@^5.0.5", "@ethersproject/logger@^5.0.8", "@ethersproject/logger@^5.7.0":
+  version "5.7.0"
+  resolved "https://registry.yarnpkg.com/@ethersproject/logger/-/logger-5.7.0.tgz#6ce9ae168e74fecf287be17062b590852c311892"
+  integrity sha512-0odtFdXu/XHtjQXJYA3u9G0G8btm0ND5Cu8M7i5vhEcE8/HmF4Lbdqanwyv4uQTr2tx6b7fQRmgLrsnpQlmnig==
+
+"@ethersproject/networks@^5.7.0":
+  version "5.7.0"
+  resolved "https://registry.yarnpkg.com/@ethersproject/networks/-/networks-5.7.0.tgz#df72a392f1a63a57f87210515695a31a245845ad"
+  integrity sha512-MG6oHSQHd4ebvJrleEQQ4HhVu8Ichr0RDYEfHzsVAVjHNM+w36x9wp9r+hf1JstMXtseXDtkiVoARAG6M959AA==
   dependencies:
-    "@ethersproject/logger" "^5.0.8"
+    "@ethersproject/logger" "^5.7.0"
 
-"@ethersproject/properties@>=5.0.0-beta.131", "@ethersproject/properties@^5.0.3", "@ethersproject/properties@^5.0.7":
+"@ethersproject/properties@>=5.0.0-beta.131":
   version "5.0.9"
   resolved "https://registry.yarnpkg.com/@ethersproject/properties/-/properties-5.0.9.tgz#d7aae634680760136ea522e25c3ef043ec15b5c2"
   integrity sha512-ZCjzbHYTw+rF1Pn8FDCEmx3gQttwIHcm/6Xee8g/M3Ga3SfW4tccNMbs5zqnBH0E4RoOPaeNgyg1O68TaF0tlg==
   dependencies:
     "@ethersproject/logger" "^5.0.8"
 
-"@ethersproject/rlp@^5.0.7":
-  version "5.0.9"
-  resolved "https://registry.yarnpkg.com/@ethersproject/rlp/-/rlp-5.0.9.tgz#da205bf8a34d3c3409eb73ddd237130a4b376aff"
-  integrity sha512-ns1U7ZMVeruUW6JXc4om+1w3w4ynHN/0fpwmeNTsAjwGKoF8SAUgue6ylKpHKWSti2idx7jDxbn8hNNFHk67CA==
+"@ethersproject/properties@^5.0.3", "@ethersproject/properties@^5.0.7", "@ethersproject/properties@^5.7.0":
+  version "5.7.0"
+  resolved "https://registry.yarnpkg.com/@ethersproject/properties/-/properties-5.7.0.tgz#a6e12cb0439b878aaf470f1902a176033067ed30"
+  integrity sha512-J87jy8suntrAkIZtecpxEPxY//szqr1mlBaYlQ0r4RCaiD2hjheqF9s1LVE8vVuJCXisjIP+JgtK/Do54ej4Sw==
   dependencies:
-    "@ethersproject/bytes" "^5.0.9"
-    "@ethersproject/logger" "^5.0.8"
+    "@ethersproject/logger" "^5.7.0"
 
-"@ethersproject/signing-key@^5.0.8":
-  version "5.0.11"
-  resolved "https://registry.yarnpkg.com/@ethersproject/signing-key/-/signing-key-5.0.11.tgz#19fc5c4597e18ad0a5efc6417ba5b74069fdd2af"
-  integrity sha512-Jfcru/BGwdkXhLxT+8WCZtFy7LL0TPFZw05FAb5asxB/MyVsEfNdNxGDtjVE9zXfmRSPe/EusXYY4K7wcygOyQ==
+"@ethersproject/rlp@^5.0.7", "@ethersproject/rlp@^5.7.0":
+  version "5.7.0"
+  resolved "https://registry.yarnpkg.com/@ethersproject/rlp/-/rlp-5.7.0.tgz#de39e4d5918b9d74d46de93af80b7685a9c21304"
+  integrity sha512-rBxzX2vK8mVF7b0Tol44t5Tb8gomOHkj5guL+HhzQ1yBh/ydjGnpw6at+X6Iw0Kp3OzzzkcKp8N9r0W4kYSs9w==
   dependencies:
-    "@ethersproject/bytes" "^5.0.9"
-    "@ethersproject/logger" "^5.0.8"
-    "@ethersproject/properties" "^5.0.7"
+    "@ethersproject/bytes" "^5.7.0"
+    "@ethersproject/logger" "^5.7.0"
+
+"@ethersproject/signing-key@^5.0.8", "@ethersproject/signing-key@^5.7.0":
+  version "5.7.0"
+  resolved "https://registry.yarnpkg.com/@ethersproject/signing-key/-/signing-key-5.7.0.tgz#06b2df39411b00bc57c7c09b01d1e41cf1b16ab3"
+  integrity sha512-MZdy2nL3wO0u7gkB4nA/pEf8lu1TlFswPNmy8AiYkfKTdO6eXBJyUdmHO/ehm/htHw9K/qF8ujnTyUAD+Ry54Q==
+  dependencies:
+    "@ethersproject/bytes" "^5.7.0"
+    "@ethersproject/logger" "^5.7.0"
+    "@ethersproject/properties" "^5.7.0"
+    bn.js "^5.2.1"
     elliptic "6.5.4"
+    hash.js "1.1.7"
 
-"@ethersproject/strings@>=5.0.0-beta.130", "@ethersproject/strings@^5.0.4", "@ethersproject/strings@^5.0.8":
+"@ethersproject/strings@>=5.0.0-beta.130":
   version "5.0.10"
   resolved "https://registry.yarnpkg.com/@ethersproject/strings/-/strings-5.0.10.tgz#ddce1e9724f4ac4f3f67e0cac0b48748e964bfdb"
   integrity sha512-KAeoS1tZ9/5ECXiIZA6S6hywbD0so2VmuW+Wfyo5EDXeyZ6Na1nxTPhTnW7voQmjbeYJffCrOc0qLFJeylyg7w==
@@ -733,7 +811,16 @@
     "@ethersproject/constants" "^5.0.8"
     "@ethersproject/logger" "^5.0.8"
 
-"@ethersproject/transactions@^5.0.0-beta.135", "@ethersproject/transactions@^5.0.9":
+"@ethersproject/strings@^5.0.4", "@ethersproject/strings@^5.0.8", "@ethersproject/strings@^5.7.0":
+  version "5.7.0"
+  resolved "https://registry.yarnpkg.com/@ethersproject/strings/-/strings-5.7.0.tgz#54c9d2a7c57ae8f1205c88a9d3a56471e14d5ed2"
+  integrity sha512-/9nu+lj0YswRNSH0NXYqrh8775XNyEdUQAuf3f+SmOrnVewcJ5SBNAjF7lpgehKi4abvNNXyf+HX86czCdJ8Mg==
+  dependencies:
+    "@ethersproject/bytes" "^5.7.0"
+    "@ethersproject/constants" "^5.7.0"
+    "@ethersproject/logger" "^5.7.0"
+
+"@ethersproject/transactions@^5.0.0-beta.135":
   version "5.0.11"
   resolved "https://registry.yarnpkg.com/@ethersproject/transactions/-/transactions-5.0.11.tgz#b31df5292f47937136a45885d6ee6112477c13df"
   integrity sha512-ftsRvR9+gQp7L63F6+XmstvsZ4w8GtWvQB08e/zB+oB86Fnhq8+i/tkgpJplSHC8I/qgiCisva+M3u2GVhDFPA==
@@ -748,20 +835,35 @@
     "@ethersproject/rlp" "^5.0.7"
     "@ethersproject/signing-key" "^5.0.8"
 
-"@ethersproject/web@^5.0.12":
-  version "5.0.14"
-  resolved "https://registry.yarnpkg.com/@ethersproject/web/-/web-5.0.14.tgz#6e7bebdd9fb967cb25ee60f44d9218dc0803bac4"
-  integrity sha512-QpTgplslwZ0Sp9oKNLoRuS6TKxnkwfaEk3gr7zd7XLF8XBsYejsrQO/03fNfnMx/TAT/RR6WEw/mbOwpRSeVRA==
-  dependencies:
-    "@ethersproject/base64" "^5.0.7"
-    "@ethersproject/bytes" "^5.0.9"
-    "@ethersproject/logger" "^5.0.8"
-    "@ethersproject/properties" "^5.0.7"
-    "@ethersproject/strings" "^5.0.8"
+"@ethersproject/transactions@^5.7.0":
+  version "5.7.0"
+  resolved "https://registry.yarnpkg.com/@ethersproject/transactions/-/transactions-5.7.0.tgz#91318fc24063e057885a6af13fdb703e1f993d3b"
+  integrity sha512-kmcNicCp1lp8qanMTC3RIikGgoJ80ztTyvtsFvCYpSCfkjhD0jZ2LOrnbcuxuToLIUYYf+4XwD1rP+B/erDIhQ==
+  dependencies:
+    "@ethersproject/address" "^5.7.0"
+    "@ethersproject/bignumber" "^5.7.0"
+    "@ethersproject/bytes" "^5.7.0"
+    "@ethersproject/constants" "^5.7.0"
+    "@ethersproject/keccak256" "^5.7.0"
+    "@ethersproject/logger" "^5.7.0"
+    "@ethersproject/properties" "^5.7.0"
+    "@ethersproject/rlp" "^5.7.0"
+    "@ethersproject/signing-key" "^5.7.0"
+
+"@ethersproject/web@^5.7.0":
+  version "5.7.0"
+  resolved "https://registry.yarnpkg.com/@ethersproject/web/-/web-5.7.0.tgz#40850c05260edad8b54827923bbad23d96aac0bc"
+  integrity sha512-ApHcbbj+muRASVDSCl/tgxaH2LBkRMEYfLOLVa0COipx0+nlu0QKet7U2lEg0vdkh8XRSLf2nd1f1Uk9SrVSGA==
+  dependencies:
+    "@ethersproject/base64" "^5.7.0"
+    "@ethersproject/bytes" "^5.7.0"
+    "@ethersproject/logger" "^5.7.0"
+    "@ethersproject/properties" "^5.7.0"
+    "@ethersproject/strings" "^5.7.0"
 
 "@graphprotocol/graph-cli@https://github.com/graphprotocol/graph-cli#main":
-  version "0.30.4"
-  resolved "https://github.com/graphprotocol/graph-cli#75f53a164b2aab9772a920efbe262ca87bb661ad"
+  version "0.33.0"
+  resolved "https://github.com/graphprotocol/graph-cli#47e075a9701680580e0e8e09c5444963224dbf5c"
   dependencies:
     assemblyscript "0.19.10"
     binary-install-raw "0.0.13"
@@ -1610,20 +1712,13 @@
   dependencies:
     "@types/node" "*"
 
-"@types/connect@*":
+"@types/connect@*", "@types/connect@^3.4.33":
   version "3.4.35"
   resolved "https://registry.yarnpkg.com/@types/connect/-/connect-3.4.35.tgz#5fcf6ae445e4021d1fc2219a4873cc73a3bb2ad1"
   integrity sha512-cdeYyv4KWoEgpBISTxWvqYsVy444DOqehiF3fM3ne10AmJ62RSyNkUnxMJXHQWRQQX2eR94m5y1IZyDwBjV9FQ==
   dependencies:
     "@types/node" "*"
 
-"@types/connect@^3.4.33":
-  version "3.4.34"
-  resolved "https://registry.yarnpkg.com/@types/connect/-/connect-3.4.34.tgz#170a40223a6d666006d93ca128af2beb1d9b1901"
-  integrity sha512-ePPA/JuI+X0vb+gSWlPKOY0NdNAie/rPUqX2GUPpbZwiKTkSPhjXWuee47E4MtE54QVzGCQMQkAL6JhV2E1+cQ==
-  dependencies:
-    "@types/node" "*"
-
 "@types/content-disposition@*":
   version "0.5.4"
   resolved "https://registry.yarnpkg.com/@types/content-disposition/-/content-disposition-0.5.4.tgz#de48cf01c79c9f1560bcfd8ae43217ab028657f8"
@@ -1654,9 +1749,9 @@
     "@types/range-parser" "*"
 
 "@types/express-serve-static-core@^4.17.9":
-  version "4.17.18"
-  resolved "https://registry.yarnpkg.com/@types/express-serve-static-core/-/express-serve-static-core-4.17.18.tgz#8371e260f40e0e1ca0c116a9afcd9426fa094c40"
-  integrity sha512-m4JTwx5RUBNZvky/JJ8swEJPKFd8si08pPF2PfizYjGZOKr/svUWPcoUmLow6MmPzhasphB7gSTINY67xn3JNA==
+  version "4.17.30"
+  resolved "https://registry.yarnpkg.com/@types/express-serve-static-core/-/express-serve-static-core-4.17.30.tgz#0f2f99617fa8f9696170c46152ccf7500b34ac04"
+  integrity sha512-gstzbTWro2/nFed1WXtf+TtrpwxH7Ggs4RLYTLbeVgIkUQOI3WG/JKjgeOU1zXDvezllupjrf8OPIdvTbIaVOQ==
   dependencies:
     "@types/node" "*"
     "@types/qs" "*"
@@ -1723,9 +1818,9 @@
     "@types/node" "*"
 
 "@types/lodash@^4.14.159":
-  version "4.14.168"
-  resolved "https://registry.yarnpkg.com/@types/lodash/-/lodash-4.14.168.tgz#fe24632e79b7ade3f132891afff86caa5e5ce008"
-  integrity sha512-oVfRvqHV/V6D1yifJbVRU3TMp8OT6o6BG+U9MkwuJ3U8/CsDHvalRpsxBqivn71ztOFZBTfJMvETbqHiaNSj7Q==
+  version "4.14.184"
+  resolved "https://registry.yarnpkg.com/@types/lodash/-/lodash-4.14.184.tgz#23f96cd2a21a28e106dc24d825d4aa966de7a9fe"
+  integrity sha512-RoZphVtHbxPZizt4IcILciSWiC6dcn+eZ8oX9IWEYfDMcocdd42f7NPI6fQj+6zI8y4E0L7gu2pcZKLGTRaV9Q==
 
 "@types/long@^4.0.0":
   version "4.0.1"
@@ -1738,9 +1833,9 @@
   integrity sha512-YATxVxgRqNH6nHEIsvg6k2Boc1JHI9ZbH5iWFFv/MTkchz3b1ieGDa5T0a9RznNdI0KhVbdbWSN+KWWrQZRxTw==
 
 "@types/node@*":
-  version "16.11.6"
-  resolved "https://registry.yarnpkg.com/@types/node/-/node-16.11.6.tgz#6bef7a2a0ad684cf6e90fcfe31cecabd9ce0a3ae"
-  integrity sha512-ua7PgUoeQFjmWPcoo9khiPum3Pd60k4/2ZGXt18sm2Slk0W0xZTqt5Y0Ny1NyBiN1EVQ/+FaF9NcY4Qe6rwk5w==
+  version "18.7.11"
+  resolved "https://registry.yarnpkg.com/@types/node/-/node-18.7.11.tgz#486e72cfccde88da24e1f23ff1b7d8bfb64e6250"
+  integrity sha512-KZhFpSLlmK/sdocfSAjqPETTMd0ug6HIMIAwkwUpU79olnZdQtMxpQP+G1wDzCH7na+FltSIhbaZuKdwZ8RDrw==
 
 "@types/node@^10.0.3", "@types/node@^10.1.0":
   version "10.17.60"
@@ -1752,7 +1847,12 @@
   resolved "https://registry.yarnpkg.com/@types/node/-/node-10.17.55.tgz#a147f282edec679b894d4694edb5abeb595fecbd"
   integrity sha512-koZJ89uLZufDvToeWO5BrC4CR4OUfHnUz2qoPs/daQH6qq3IN62QFxCTZ+bKaCE0xaoCAJYE4AXre8AbghCrhg==
 
-"@types/node@^12.12.54", "@types/node@^12.12.6", "@types/node@^12.6.1":
+"@types/node@^12.12.54":
+  version "12.20.55"
+  resolved "https://registry.yarnpkg.com/@types/node/-/node-12.20.55.tgz#c329cbd434c42164f846b909bd6f85b5537f6240"
+  integrity sha512-J8xLz7q2OFulZ2cyGTLE1TbbZcjpno7FaN6zdJNrgAdrJ+DZzh/uFR6YrTb4C+nXakvud8Q4+rbhoIWlYQbUFQ==
+
+"@types/node@^12.12.6", "@types/node@^12.6.1":
   version "12.20.4"
   resolved "https://registry.yarnpkg.com/@types/node/-/node-12.20.4.tgz#73687043dd00fcb6962c60fbf499553a24d6bdf2"
   integrity sha512-xRCgeE0Q4pT5UZ189TJ3SpYuX/QGl6QIAOAIeDSbAVAd2gX1NxSZup4jNVK7cxIeP8KDSbJgcckun495isP1jQ==
@@ -1785,9 +1885,9 @@
   integrity sha512-EEhsLsD6UsDM1yFhAvy0Cjr6VwmpMWqFBCb9w07wVugF7w9nfajxLuVmngTIpgS6svCnm6Vaw+MZhoDCKnOfsw==
 
 "@types/secp256k1@^4.0.1":
-  version "4.0.1"
-  resolved "https://registry.yarnpkg.com/@types/secp256k1/-/secp256k1-4.0.1.tgz#fb3aa61a1848ad97d7425ff9dcba784549fca5a4"
-  integrity sha512-+ZjSA8ELlOp8SlKi0YLB2tz9d5iPNEmOBd+8Rz21wTMdaXQIa9b6TEnD6l5qKOCypE7FSyPyck12qZJxSDNoog==
+  version "4.0.3"
+  resolved "https://registry.yarnpkg.com/@types/secp256k1/-/secp256k1-4.0.3.tgz#1b8e55d8e00f08ee7220b4d59a6abe89c37a901c"
+  integrity sha512-Da66lEIFeIz9ltsdMZcpQvmrmmoqrfju8pm1BH8WbYjZSwUgCwXLb9C+9XYogwBITnbsSaMdVPb2ekf7TV+03w==
   dependencies:
     "@types/node" "*"
 
@@ -1859,7 +1959,7 @@
 JSONStream@1.3.2:
   version "1.3.2"
   resolved "https://registry.yarnpkg.com/JSONStream/-/JSONStream-1.3.2.tgz#c102371b6ec3a7cf3b847ca00c20bb0fce4c6dea"
-  integrity sha1-wQI3G27Dp887hHygDCC7D85Mbeo=
+  integrity sha512-mn0KSip7N4e0UDPZHnqDsHECo5uGQrixQKnAskOM1BIB8hd7QKbd6il8IPRPudPHOeHiECoCFqhyMaRO9+nWyA==
   dependencies:
     jsonparse "^1.2.0"
     through ">=2.2.7 <3"
@@ -2005,19 +2105,19 @@ ansi-regex@^2.0.0:
   integrity sha1-w7M6te42DYbg5ijwRorn7yfWVN8=
 
 ansi-regex@^3.0.0:
-  version "3.0.0"
-  resolved "https://registry.yarnpkg.com/ansi-regex/-/ansi-regex-3.0.0.tgz#ed0317c322064f79466c02966bddb605ab37d998"
-  integrity sha1-7QMXwyIGT3lGbAKWa922Bas32Zg=
+  version "3.0.1"
+  resolved "https://registry.yarnpkg.com/ansi-regex/-/ansi-regex-3.0.1.tgz#123d6479e92ad45ad897d4054e3c7ca7db4944e1"
+  integrity sha512-+O9Jct8wf++lXxxFc4hc8LsjaSq0HFzzL7cVsw8pRDIPdjKD2mT4ytDZlLuSBZ4cLKZFXIrMGO7DbQCtMJJMKw==
 
 ansi-regex@^4.1.0:
   version "4.1.0"
   resolved "https://registry.yarnpkg.com/ansi-regex/-/ansi-regex-4.1.0.tgz#8b9f8f08cf1acb843756a839ca8c7e3168c51997"
   integrity sha512-1apePfXM1UOSqw0o9IiFAovVz9M5S1Dg+4TrDwfMewQ6p/rmMueb7tWZjQ1rx4Loy1ArBggoqGpfqqdI4rondg==
 
-ansi-regex@^5.0.0:
-  version "5.0.0"
-  resolved "https://registry.yarnpkg.com/ansi-regex/-/ansi-regex-5.0.0.tgz#388539f55179bf39339c81af30a654d69f87cb75"
-  integrity sha512-bY6fj56OUQ0hU1KjFNDQuJFezqKdrAyFdIevADiqrWHwSlbmBNMHp5ak2f40Pm8JTFyM2mqxkG6ngkHO11f/lg==
+ansi-regex@^5.0.1:
+  version "5.0.1"
+  resolved "https://registry.yarnpkg.com/ansi-regex/-/ansi-regex-5.0.1.tgz#082cb2c89c9fe8659a311a53bd6a4dc5301db304"
+  integrity sha512-quJQXlTSUGL2LH9SUXo8VwsY4soanhgo6LNSm84E1LBcE8s3O0wpdiRzyR9z/ZZJMlMWv37qOOb9pdJlMUEKFQ==
 
 ansi-styles@^2.2.1:
   version "2.2.1"
@@ -2044,9 +2144,9 @@ any-promise@^1.3.0:
   integrity sha1-q8av7tzqUugJzcA3au0845Y10X8=
 
 anymatch@~3.1.1:
-  version "3.1.1"
-  resolved "https://registry.yarnpkg.com/anymatch/-/anymatch-3.1.1.tgz#c55ecf02185e2469259399310c173ce31233b142"
-  integrity sha512-mM8522psRCqzV+6LhomX5wgp25YVibjh8Wj23I5RPkPppSVSjyKD2A2mBJmWGa+KN7f2D6LNh9jkBCeyLktzjg==
+  version "3.1.2"
+  resolved "https://registry.yarnpkg.com/anymatch/-/anymatch-3.1.2.tgz#c0557c096af32f106198f4f4e2a383537e378716"
+  integrity sha512-P43ePfOAIupkguHUycrc4qJ9kz8ZiuOUijaETwX7THt0Y/GNK7v0aa8rY816xWjZ7rJdA5XdMcpVFTKMq+RvWg==
   dependencies:
     normalize-path "^3.0.0"
     picomatch "^2.0.4"
@@ -2248,7 +2348,7 @@ apollo-utilities@^1.0.1, apollo-utilities@^1.3.0:
 app-module-path@^2.2.0:
   version "2.2.0"
   resolved "https://registry.yarnpkg.com/app-module-path/-/app-module-path-2.2.0.tgz#641aa55dfb7d6a6f0a8141c4b9c0aa50b6c24dd5"
-  integrity sha1-ZBqlXft9am8KgUHEucCqULbCTdU=
+  integrity sha512-gkco+qxENJV+8vFcDiiFhuoSvRXb2a/QPqpSoWhVz829VNJfOTnELbBmPmNKFxf3xdNnw4DWCkzkDaavcX/1YQ==
 
 aproba@^1.0.3:
   version "1.2.0"
@@ -2344,9 +2444,9 @@ asn1.js@^5.0.1, asn1.js@^5.2.0:
     safer-buffer "^2.1.0"
 
 asn1@~0.2.3:
-  version "0.2.4"
-  resolved "https://registry.yarnpkg.com/asn1/-/asn1-0.2.4.tgz#8d2475dfab553bb33e77b54e59e880bb8ce23136"
-  integrity sha512-jxwzQpLQjSmWXgwaCZE9Nz+glAG01yF1QnWgbhGwHI5A6FRIEY6IVqtHhIepHqI7/kyEyQEagBC5mBEFlIYvdg==
+  version "0.2.6"
+  resolved "https://registry.yarnpkg.com/asn1/-/asn1-0.2.6.tgz#0d3a7bb6e64e02a90c0303b31f292868ea09a08d"
+  integrity sha512-ix/FxPn0MDjeyJ7i/yoHGFt/EX6LyNbxSEhPPXODPL+KB0VPk86UYfL0lMdy+KCnv+fmvIzySwaK5COwqVbWTQ==
   dependencies:
     safer-buffer "~2.1.0"
 
@@ -2383,7 +2483,7 @@ assemblyscript@0.19.10:
 assert-plus@1.0.0, assert-plus@^1.0.0:
   version "1.0.0"
   resolved "https://registry.yarnpkg.com/assert-plus/-/assert-plus-1.0.0.tgz#f12e0f3c5d77b0b1cdd9146942e4e96c1e4dd525"
-  integrity sha1-8S4PPF13sLHN2RRpQuTpbB5N1SU=
+  integrity sha512-NfJ4UzBCcQGLDlQq7nHxH+tv3kyZ0hHQqF5BO6J7tNJeP5do1llPr8dZ8zHonfhAu0PHAdMkSo+8o0wxg9lZWw==
 
 async-eventemitter@^0.2.2:
   version "0.2.4"
@@ -2409,17 +2509,24 @@ async@^1.4.2:
   resolved "https://registry.yarnpkg.com/async/-/async-1.5.2.tgz#ec6a61ae56480c0c3cb241c95618e20892f9672a"
   integrity sha1-7GphrlZIDAw8skHJVhjiCJL5Zyo=
 
-async@^2.0.1, async@^2.1.2, async@^2.4.0, async@^2.5.0, async@^2.6.1, async@^2.6.2, async@^2.6.3:
+async@^2.0.1, async@^2.1.2, async@^2.4.0, async@^2.5.0:
   version "2.6.3"
   resolved "https://registry.yarnpkg.com/async/-/async-2.6.3.tgz#d72625e2344a3656e3a3ad4fa749fa83299d82ff"
   integrity sha512-zflvls11DCy+dQWzTW2dzuilv8Z5X/pjfmZOWba6TNIVDm+2UDaJmXSOXlasHKfNBs8oo3M0aT50fDEWfKZjXg==
   dependencies:
     lodash "^4.17.14"
 
+async@^2.6.1, async@^2.6.2, async@^2.6.3:
+  version "2.6.4"
+  resolved "https://registry.yarnpkg.com/async/-/async-2.6.4.tgz#706b7ff6084664cd7eae713f6f965433b5504221"
+  integrity sha512-mzo5dfJYwAn29PeiJ0zvwTo04zj8HDJj0Mn8TD7sno7q12prdbnasKJHhkm2c1LgrhlJ0teaea8860oxi51mGA==
+  dependencies:
+    lodash "^4.17.14"
+
 asynckit@^0.4.0:
   version "0.4.0"
   resolved "https://registry.yarnpkg.com/asynckit/-/asynckit-0.4.0.tgz#c79ed97f7f34cb8f2ba1bc9790bcc366474b4b79"
-  integrity sha1-x57Zf380y48robyXkLzDZkdLS3k=
+  integrity sha512-Oei9OH4tRh0YqU3GxhX79dM/mwVgvbZJaSNaRk+bshkj0S5cfHcgYakreBjrHwatXKbz+IoIdYLxrKim2MjW0Q==
 
 at-least-node@^1.0.0:
   version "1.0.0"
@@ -2446,7 +2553,7 @@ await-semaphore@^0.1.3:
 aws-sign2@~0.7.0:
   version "0.7.0"
   resolved "https://registry.yarnpkg.com/aws-sign2/-/aws-sign2-0.7.0.tgz#b46e890934a9591f2d2f6f86d7e6a9f1b3fe76a8"
-  integrity sha1-tG6JCTSpWR8tL2+G1+ap8bP+dqg=
+  integrity sha512-08kcGqnYf/YmjoRhfxyu+CLxBjUtHLXLXX/vUfx9l2LYzG3c1m61nrpyFUZI6zeS+Li/wWMMidD9KgrqtGq3mA==
 
 aws4@^1.8.0:
   version "1.11.0"
@@ -2676,14 +2783,14 @@ backoff@^2.5.0:
     precond "0.2"
 
 balanced-match@^1.0.0:
-  version "1.0.0"
-  resolved "https://registry.yarnpkg.com/balanced-match/-/balanced-match-1.0.0.tgz#89b4d199ab2bee49de164ea02b89ce462d71b767"
-  integrity sha1-ibTRmasr7kneFk6gK4nORi1xt2c=
+  version "1.0.2"
+  resolved "https://registry.yarnpkg.com/balanced-match/-/balanced-match-1.0.2.tgz#e83e3a7e3f300b34cb9d87f615fa0cbf357690ee"
+  integrity sha512-3oSeUO0TMV67hN1AmbXsK4yaqU7tjiHlbxRDZOpH0KW9+CeX4bRAaX0Anxt0tx2MrpRpWwQaPwIlISEJhYU5Pw==
 
 base-x@^3.0.2, base-x@^3.0.8:
-  version "3.0.8"
-  resolved "https://registry.yarnpkg.com/base-x/-/base-x-3.0.8.tgz#1e1106c2537f0162e8b52474a557ebb09000018d"
-  integrity sha512-Rl/1AWP4J/zRrk54hhlxH4drNxPJXYUaKffODVI53/dAsV4t9fBxyxYKAVPU1XBHxYwOWP9h9H0hM2MVw4YfJA==
+  version "3.0.9"
+  resolved "https://registry.yarnpkg.com/base-x/-/base-x-3.0.9.tgz#6349aaabb58526332de9f60995e548a53fe21320"
+  integrity sha512-H7JU6iBHTal1gp56aKoaa//YUxEaAOUiydvrV/pILqIHXTtqxSkATOnDA2u+jZ/61sD+L/412+7kzXRtWukhpQ==
   dependencies:
     safe-buffer "^5.0.1"
 
@@ -2695,7 +2802,7 @@ base64-js@^1.3.1:
 bcrypt-pbkdf@^1.0.0:
   version "1.0.2"
   resolved "https://registry.yarnpkg.com/bcrypt-pbkdf/-/bcrypt-pbkdf-1.0.2.tgz#a4301d389b6a43f9b67ff3ca11a3f6637e360e9e"
-  integrity sha1-pDAdOJtqQ/m2f/PKEaP2Y342Dp4=
+  integrity sha512-qeFIXtP4MSoi6NLqO12WfqARWWuCKi2Rn/9hJLEmtB5yTNr9DqFWkJRCf2qShWzPeAMRnOgCrq0sg/KLv5ES9w==
   dependencies:
     tweetnacl "^0.14.3"
 
@@ -2709,7 +2816,12 @@ bignumber.js@^7.2.1:
   resolved "https://registry.yarnpkg.com/bignumber.js/-/bignumber.js-7.2.1.tgz#80c048759d826800807c4bfd521e50edbba57a5f"
   integrity sha512-S4XzBk5sMB+Rcb/LNcpzXr57VRTxgAvaAEDAl1AwRx27j00hT84O6OkteE7u8UB3NuaaygCRrEpqox4uDOrbdQ==
 
-bignumber.js@^9.0.0, bignumber.js@^9.0.1:
+bignumber.js@^9.0.0:
+  version "9.1.0"
+  resolved "https://registry.yarnpkg.com/bignumber.js/-/bignumber.js-9.1.0.tgz#8d340146107fe3a6cb8d40699643c302e8773b62"
+  integrity sha512-4LwHK4nfDOraBCtst+wOWIHbu1vhvAPJK8g8nROd4iuc3PSEjWif/qwbkh8jwCJz6yDBvtU4KPynETgrfh7y3A==
+
+bignumber.js@^9.0.1:
   version "9.0.1"
   resolved "https://registry.yarnpkg.com/bignumber.js/-/bignumber.js-9.0.1.tgz#8d7ba124c882bfd8e43260c67475518d0689e4e5"
   integrity sha512-IdZR9mh6ahOBv/hYGiXyVuyCetmGJhtYkqLBpTStdhEGjegpPlUawydyaF3pbIOFynJTpllEs+NP+CS9jKFLjA==
@@ -2748,7 +2860,7 @@ bindings@^1.5.0:
 bip66@^1.1.5:
   version "1.1.5"
   resolved "https://registry.yarnpkg.com/bip66/-/bip66-1.1.5.tgz#01fa8748785ca70955d5011217d1b3139969ca22"
-  integrity sha1-AfqHSHhcpwlV1QESF9GzE5lpyiI=
+  integrity sha512-nemMHz95EmS38a26XbbdxIYj5csHd3RMP3H5bwQknX0WYHF01qhpufP42mLOwVICuH2JmhIhXiWs89MfUGL7Xw==
   dependencies:
     safe-buffer "^5.0.1"
 
@@ -2777,9 +2889,9 @@ bl@^4.0.3:
     readable-stream "^3.4.0"
 
 blakejs@^1.1.0:
-  version "1.1.0"
-  resolved "https://registry.yarnpkg.com/blakejs/-/blakejs-1.1.0.tgz#69df92ef953aa88ca51a32df6ab1c54a155fc7a5"
-  integrity sha1-ad+S75U6qIylGjLfarHFShVfx6U=
+  version "1.2.1"
+  resolved "https://registry.yarnpkg.com/blakejs/-/blakejs-1.2.1.tgz#5057e4206eadb4a97f7c0b6e197a505042fc3814"
+  integrity sha512-QXUSXI3QVc/gJME0dBpXrag1kbzOqCjCX8/b54ntNyW6sjtoqxqRk3LTmXzaJoh71zMsDCjM+47jS7XiwN/+fQ==
 
 bluebird@^3.4.7, bluebird@^3.5.0:
   version "3.7.2"
@@ -2789,7 +2901,7 @@ bluebird@^3.4.7, bluebird@^3.5.0:
 bn.js@4.11.6:
   version "4.11.6"
   resolved "https://registry.yarnpkg.com/bn.js/-/bn.js-4.11.6.tgz#53344adb14617a13f6e8dd2ce28905d1c0ba3215"
-  integrity sha1-UzRK2xRhehP26N0s4okF0cC6MhU=
+  integrity sha512-XWwnNNFCuuSQ0m3r3C4LE3EiORltHd9M05pq6FOlVeiophzRbMo50Sbz1ehl8K3Z+jw9+vmgnXefY1hz8X+2wA==
 
 bn.js@4.11.8:
   version "4.11.8"
@@ -2801,11 +2913,16 @@ bn.js@^4.0.0, bn.js@^4.1.0, bn.js@^4.11.0, bn.js@^4.11.1, bn.js@^4.11.6, bn.js@^
   resolved "https://registry.yarnpkg.com/bn.js/-/bn.js-4.12.0.tgz#775b3f278efbb9718eec7361f483fb36fbbfea88"
   integrity sha512-c98Bf3tPniI+scsdk237ku1Dc3ujXQTSgyiPUDEOe7tRkhrqridvh8klBv0HCEso1OLOYcHuCv/cS6DNxKH+ZA==
 
-bn.js@^5.0.0, bn.js@^5.1.1, bn.js@^5.1.2, bn.js@^5.1.3:
+bn.js@^5.0.0, bn.js@^5.1.1, bn.js@^5.1.3:
   version "5.2.0"
   resolved "https://registry.yarnpkg.com/bn.js/-/bn.js-5.2.0.tgz#358860674396c6997771a9d051fcc1b57d4ae002"
   integrity sha512-D7iWRBvnZE8ecXiLj/9wbxH7Tk79fAh8IHaTNq1RWRixsS02W+5qS+iE9yq6RYl0asXx5tw0bLhmT5pIfbSquw==
 
+bn.js@^5.1.2, bn.js@^5.2.0, bn.js@^5.2.1:
+  version "5.2.1"
+  resolved "https://registry.yarnpkg.com/bn.js/-/bn.js-5.2.1.tgz#0bc527a6a0d18d0aa8d5b0538ce4a77dccfa7b70"
+  integrity sha512-eXRvHzWyYPBuB4NBy0cmYQjGitUrtqwbvlzP3G6VFnNRbsZQIxQ10PbKKHt8gZ/HW/D/747aDl+QkDqg3KQLMQ==
+
 body-parser@1.19.0, body-parser@^1.16.0, body-parser@^1.18.3:
   version "1.19.0"
   resolved "https://registry.yarnpkg.com/body-parser/-/body-parser-1.19.0.tgz#96b2709e57c9c4e09a6fd66a8fd979844f69f08a"
@@ -2942,7 +3059,7 @@ browserslist@^4.14.5, browserslist@^4.16.3:
 bs58@^4.0.0, bs58@^4.0.1:
   version "4.0.1"
   resolved "https://registry.yarnpkg.com/bs58/-/bs58-4.0.1.tgz#be161e76c354f6f788ae4071f63f34e8c4f0a42a"
-  integrity sha1-vhYedsNU9veIrkBx9j806MTwpCo=
+  integrity sha512-Ok3Wdf5vOIlBrgCvTq96gBkJw+JUEzdBgyaza5HLtPm7yTHkjRy8+JzNyHF7BHa0bNWOQIp3m5YF0nnFcOIKLw==
   dependencies:
     base-x "^3.0.2"
 
@@ -2983,18 +3100,23 @@ buffer-alloc@^1.2.0:
 buffer-fill@^1.0.0:
   version "1.0.0"
   resolved "https://registry.yarnpkg.com/buffer-fill/-/buffer-fill-1.0.0.tgz#f8f78b76789888ef39f205cd637f68e702122b2c"
-  integrity sha1-+PeLdniYiO858gXNY39o5wISKyw=
+  integrity sha512-T7zexNBwiiaCOGDg9xNX9PBmjrubblRkENuptryuI64URkXDFum9il/JGL8Lm8wYfAXpredVXXZz7eMHilimiQ==
 
 buffer-from@1.1.0:
   version "1.1.0"
   resolved "https://registry.yarnpkg.com/buffer-from/-/buffer-from-1.1.0.tgz#87fcaa3a298358e0ade6e442cfce840740d1ad04"
   integrity sha512-c5mRlguI/Pe2dSZmpER62rSCu0ryKmWddzRYsuXc50U2/g8jMOulc31VZMa4mYx31U5xsmSOpDCgH88Vl9cDGQ==
 
-buffer-from@1.1.1, buffer-from@^1.0.0:
+buffer-from@1.1.1:
   version "1.1.1"
   resolved "https://registry.yarnpkg.com/buffer-from/-/buffer-from-1.1.1.tgz#32713bc028f75c02fdb710d7c7bcec1f2c6070ef"
   integrity sha512-MQcXEUbCKtEo7bhqEs6560Hyd4XaovZlO/k9V3hjVUF/zwW7KBVdSK4gIt/bzwS9MbR5qob+F5jusZsb0YQK2A==
 
+buffer-from@^1.0.0:
+  version "1.1.2"
+  resolved "https://registry.yarnpkg.com/buffer-from/-/buffer-from-1.1.2.tgz#2b146a6fd72e80b4f55d255f35ed59a3a9a41bd5"
+  integrity sha512-E+XQCRwSbaaiChtv6k6Dwgc+bx+Bs6vuKJHHl5kox/BaKbhiXzqQOwK4cO22yElGp2OCmjwVhT3HmxgyPGnJfQ==
+
 buffer-to-arraybuffer@^0.0.5:
   version "0.0.5"
   resolved "https://registry.yarnpkg.com/buffer-to-arraybuffer/-/buffer-to-arraybuffer-0.0.5.tgz#6064a40fa76eb43c723aba9ef8f6e1216d10511a"
@@ -3003,7 +3125,7 @@ buffer-to-arraybuffer@^0.0.5:
 buffer-xor@^1.0.3:
   version "1.0.3"
   resolved "https://registry.yarnpkg.com/buffer-xor/-/buffer-xor-1.0.3.tgz#26e61ed1422fb70dd42e6e36729ed51d855fe8d9"
-  integrity sha1-JuYe0UIvtw3ULm42cp7VHYVf6Nk=
+  integrity sha512-571s0T7nZWK6vB67HI5dyUF7wXiNcfaPPPTl6zYCNApANjIvYJTg7hlud/+cJpdAhS7dVzqMLmfhfHR3rAcOjQ==
 
 buffer@^5.0.5, buffer@^5.2.1, buffer@^5.4.2, buffer@^5.4.3, buffer@^5.5.0, buffer@^5.6.0, buffer@^5.7.0:
   version "5.7.1"
@@ -3013,6 +3135,14 @@ buffer@^5.0.5, buffer@^5.2.1, buffer@^5.4.2, buffer@^5.4.3, buffer@^5.5.0, buffe
     base64-js "^1.3.1"
     ieee754 "^1.1.13"
 
+buffer@^6.0.3:
+  version "6.0.3"
+  resolved "https://registry.yarnpkg.com/buffer/-/buffer-6.0.3.tgz#2ace578459cc8fbe2a70aaa8f52ee63b6a74c6c6"
+  integrity sha512-FTiCpNxtwiZZHEZbcbTIcZjERVICn9yq/pDFkTl95/AxzD1naBctN7YO68riM/gLSDY7sdrMby8hofADYuuqOA==
+  dependencies:
+    base64-js "^1.3.1"
+    ieee754 "^1.2.1"
+
 bufferutil@^4.0.1:
   version "4.0.3"
   resolved "https://registry.yarnpkg.com/bufferutil/-/bufferutil-4.0.3.tgz#66724b756bed23cd7c28c4d306d7994f9943cc6b"
@@ -3023,7 +3153,7 @@ bufferutil@^4.0.1:
 builtin-status-codes@^3.0.0:
   version "3.0.0"
   resolved "https://registry.yarnpkg.com/builtin-status-codes/-/builtin-status-codes-3.0.0.tgz#85982878e21b98e1c66425e03d0174788f569ee8"
-  integrity sha1-hZgoeOIbmOHGZCXgPQF0eI9Wnug=
+  integrity sha512-HpGFw18DgFWlncDfjTa2rcQ4W88O1mC8e8yZ2AvQY5KDaktSTwo+KRf6nHK6FRI5FyRyb/5T6+TSxfP7QyGsmQ==
 
 busboy@^0.3.1:
   version "0.3.1"
@@ -3115,7 +3245,7 @@ caniuse-lite@^1.0.30001181:
 caseless@^0.12.0, caseless@~0.12.0:
   version "0.12.0"
   resolved "https://registry.yarnpkg.com/caseless/-/caseless-0.12.0.tgz#1b681c21ff84033c826543090689420d187151dc"
-  integrity sha1-G2gcIf+EAzyCZUMJBolCDRhxUdw=
+  integrity sha512-4tYFyifaFfGacoiObjJegolkwSU4xQNGbVgUiNYVUxbQ2x2lUsFvY4hVgVzGiIe6WLOPqycWXA40l+PWsxthUw==
 
 cbor@^5.1.0:
   version "5.2.0"
@@ -3330,11 +3460,16 @@ cli-cursor@^3.1.0:
   dependencies:
     restore-cursor "^3.1.0"
 
-cli-spinners@^2.0.0, cli-spinners@^2.2.0:
+cli-spinners@^2.0.0:
   version "2.5.0"
   resolved "https://registry.yarnpkg.com/cli-spinners/-/cli-spinners-2.5.0.tgz#12763e47251bf951cb75c201dfa58ff1bcb2d047"
   integrity sha512-PC+AmIuK04E6aeSs/pUccSujsTzBhu4HzC2dL+CfJB/Jcc2qTRbEwZQDfIUpt2Xl8BodYBEq8w4fc0kU2I9DjQ==
 
+cli-spinners@^2.2.0:
+  version "2.7.0"
+  resolved "https://registry.yarnpkg.com/cli-spinners/-/cli-spinners-2.7.0.tgz#f815fd30b5f9eaac02db604c7a231ed7cb2f797a"
+  integrity sha512-qu3pN8Y3qHNgE2AFweciB1IfMnmZ/fsNTEE+NOFjmGB2F/7rLhnhzppvpCnN4FovtP26k8lHyy9ptEbNwWFLzw==
+
 cli-table3@~0.5.0:
   version "0.5.1"
   resolved "https://registry.yarnpkg.com/cli-table3/-/cli-table3-0.5.1.tgz#0252372d94dfc40dbd8df06005f48f31f656f202"
@@ -3436,7 +3571,7 @@ color-logger@0.0.6:
 color-name@1.1.3:
   version "1.1.3"
   resolved "https://registry.yarnpkg.com/color-name/-/color-name-1.1.3.tgz#a7d0558bd89c42f795dd42328f740831ca53bc25"
-  integrity sha1-p9BVi9icQveV3UIyj3QIMcpTvCU=
+  integrity sha512-72fSenhMw2HZMTVHeCA9KCmpEIbzWiQsjN+BHcBbS9vr1mtt+vJjPdksIBNUmKAW8TFUDPJK5SUU3QhE9NEXDw==
 
 color-name@~1.1.4:
   version "1.1.4"
@@ -3488,7 +3623,7 @@ component-emitter@1.2.1:
 concat-map@0.0.1:
   version "0.0.1"
   resolved "https://registry.yarnpkg.com/concat-map/-/concat-map-0.0.1.tgz#d8a96bd77fd68df7793a73036a3ba0d5405d477b"
-  integrity sha1-2Klr13/Wjfd5OnMDajug1UBdR3s=
+  integrity sha512-/Srv4dswyQNBfohGpz9o6Yb3Gz3SrUDqBH5rTuhGR7ahtlbYKnVxw2bCFMRljaA7EXHaXZ8wsHdodFvbkhKmqg==
 
 concat-stream@1.5.1:
   version "1.5.1"
@@ -3602,10 +3737,15 @@ core-js@^2.4.0, core-js@^2.5.0:
   resolved "https://registry.yarnpkg.com/core-js/-/core-js-2.6.12.tgz#d9333dfa7b065e347cc5682219d6f690859cc2ec"
   integrity sha512-Kb2wC0fvsWfQrgk8HU5lW6U/Lcs8+9aaYcy4ZFc6DDlo4nZ7n70dEgE5rtR0oG6ufKDUnrwfWL1mXR5ljDatrQ==
 
-core-util-is@1.0.2, core-util-is@~1.0.0:
+core-util-is@1.0.2:
   version "1.0.2"
   resolved "https://registry.yarnpkg.com/core-util-is/-/core-util-is-1.0.2.tgz#b5fd54220aa2bc5ab57aab7140c940754503c1a7"
-  integrity sha1-tf1UIgqivFq1eqtxQMlAdUUDwac=
+  integrity sha512-3lqz5YjWTYnW6dlDa5TLaTCcShfar1e40rmcJVwCBJC6mWlFuj0eCHIElmG1g5kyuJ/GD+8Wn4FFCcz4gJPfaQ==
+
+core-util-is@~1.0.0:
+  version "1.0.3"
+  resolved "https://registry.yarnpkg.com/core-util-is/-/core-util-is-1.0.3.tgz#a6042d3634c2b27e9328f837b965fac83808db85"
+  integrity sha512-ZQBvi1DcpJ4GDqanjucZ2Hj3wEO5pZDS89BWbkcrvdxksJorwUDDZamX9ldFkp9aw2lmBDLgkObEA4DWNJ9FYQ==
 
 cors@^2.8.1, cors@^2.8.5:
   version "2.8.5"
@@ -3785,7 +3925,7 @@ d@1, d@^1.0.1:
 dashdash@^1.12.0:
   version "1.14.1"
   resolved "https://registry.yarnpkg.com/dashdash/-/dashdash-1.14.1.tgz#853cfa0f7cbe2fed5de20326b8dd581035f6e2f0"
-  integrity sha1-hTz6D3y+L+1d4gMmuN1YEDX24vA=
+  integrity sha512-jRFi8UDGo6j+odZiEpjazZaWqEal3w/basFjQHQEwVtZJGDpxbH1MeYluwCS8Xq5wmLJooDlMgvVarmWfGM44g==
   dependencies:
     assert-plus "^1.0.0"
 
@@ -3824,7 +3964,7 @@ debug@4.1.1:
   dependencies:
     ms "^2.1.1"
 
-debug@4.3.1, debug@^4.1.0, debug@^4.1.1, debug@^4.3.1:
+debug@4.3.1, debug@^4.1.1, debug@^4.3.1:
   version "4.3.1"
   resolved "https://registry.yarnpkg.com/debug/-/debug-4.3.1.tgz#f0d229c505e0c6d8c49ac553d1b13dc183f6b2ee"
   integrity sha512-doEwdvm4PCeK4K3RQN2ZC2BYUBaxwLARCqZmMjtF8a51J2Rb0xpVloFRnCODwqjpwnAoao4pelN8l3RJdv3gRQ==
@@ -3838,6 +3978,13 @@ debug@^3.1.0, debug@^3.2.6:
   dependencies:
     ms "^2.1.1"
 
+debug@^4.1.0:
+  version "4.3.4"
+  resolved "https://registry.yarnpkg.com/debug/-/debug-4.3.4.tgz#1319f6579357f2338d3337d2cdd4914bb5dcc865"
+  integrity sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==
+  dependencies:
+    ms "2.1.2"
+
 decamelize@^1.1.1, decamelize@^1.2.0:
   version "1.2.0"
   resolved "https://registry.yarnpkg.com/decamelize/-/decamelize-1.2.0.tgz#f6534d15148269b20352e7bee26f501f9a191290"
@@ -3868,7 +4015,7 @@ deep-is@~0.1.3:
 defaults@^1.0.3:
   version "1.0.3"
   resolved "https://registry.yarnpkg.com/defaults/-/defaults-1.0.3.tgz#c656051e9817d9ff08ed881477f3fe4019f3ef7d"
-  integrity sha1-xlYFHpgX2f8I7YgUd/P+QBnz730=
+  integrity sha512-s82itHOnYrN0Ib8r+z7laQz3sdE+4FP3d9Q7VLO7U+KRT+CR0GsWuyHxzdAY82I7cXv0G/twrqomTJLOssO5HA==
   dependencies:
     clone "^1.0.2"
 
@@ -3915,7 +4062,7 @@ delay@^5.0.0:
 delayed-stream@~1.0.0:
   version "1.0.0"
   resolved "https://registry.yarnpkg.com/delayed-stream/-/delayed-stream-1.0.0.tgz#df3ae199acadfb7d440aaae0b29e2272b24ec619"
-  integrity sha1-3zrhmayt+31ECqrgsp4icrJOxhk=
+  integrity sha512-ZySD7Nf91aLB0RxL4KGrKHBXl7Eds1DAmEdcoVawXnLD7SDhpNgtuII2aAkg7a7QS41jxPSZ17p4VdGnMHk3MQ==
 
 delegates@^1.0.0:
   version "1.0.0"
@@ -3925,7 +4072,7 @@ delegates@^1.0.0:
 delimit-stream@0.1.0:
   version "0.1.0"
   resolved "https://registry.yarnpkg.com/delimit-stream/-/delimit-stream-0.1.0.tgz#9b8319477c0e5f8aeb3ce357ae305fc25ea1cd2b"
-  integrity sha1-m4MZR3wOX4rrPONXrjBfwl6hzSs=
+  integrity sha512-a02fiQ7poS5CnjiJBAsjGLPp5EwVoGHNeu9sziBd9huppRfsAFIpv5zNLv0V1gbop53ilngAf5Kf331AwcoRBQ==
 
 depd@~1.1.2:
   version "1.1.2"
@@ -3973,9 +4120,9 @@ detect-newline@2.X:
   integrity sha1-9B8cEL5LAOh7XxPaaAdZ8sW/0+I=
 
 detect-node@^2.0.4:
-  version "2.0.4"
-  resolved "https://registry.yarnpkg.com/detect-node/-/detect-node-2.0.4.tgz#014ee8f8f669c5c58023da64b8179c083a28c46c"
-  integrity sha512-ZIzRpLJrOj7jjP2miAtgqIfmzbxa4ZOr5jJc601zklsfEx9oTzmmj2nVpIPRpNlRTIh8lc1kyViIY7BWSGNmKw==
+  version "2.1.0"
+  resolved "https://registry.yarnpkg.com/detect-node/-/detect-node-2.1.0.tgz#c9c70775a49c3d03bc2c06d9a73be550f978f8b1"
+  integrity sha512-T0NIuQpnTvFDATNuHN5roPwSBG83rFsuO+MXXH9/3N1eFbn4wcPjttvjMLEPWJ0RGUYgQE7cGgS3tNxbqCGM7g==
 
 dicer@0.3.0:
   version "0.3.0"
@@ -4142,7 +4289,7 @@ double-ended-queue@2.1.0-0:
 drbg.js@^1.0.1:
   version "1.0.1"
   resolved "https://registry.yarnpkg.com/drbg.js/-/drbg.js-1.0.1.tgz#3e36b6c42b37043823cdbc332d58f31e2445480b"
-  integrity sha1-Pja2xCs3BDgjzbwzLVjzHiRFSAs=
+  integrity sha512-F4wZ06PvqxYLFEZKkFxTDcns9oFNk34hvmJSEwdzsxVQ8YI5YaxtACgQatkYgv2VI2CFkUd2Y+xosPQnHv809g==
   dependencies:
     browserify-aes "^1.0.6"
     create-hash "^1.1.2"
@@ -4166,7 +4313,7 @@ duplexify@^3.2.0:
 ecc-jsbn@~0.1.1:
   version "0.1.2"
   resolved "https://registry.yarnpkg.com/ecc-jsbn/-/ecc-jsbn-0.1.2.tgz#3a83a904e54353287874c564b7549386849a98c9"
-  integrity sha1-OoOpBOVDUyh4dMVkt1SThoSamMk=
+  integrity sha512-eh9O+hwRHNbG4BLTjEl3nw044CkGm5X6LoaCf7LPp7UU8Qrt47JYNi6nPX8xjW97TKGKm1ouctg0QSpZe9qrnw==
   dependencies:
     jsbn "~0.1.0"
     safer-buffer "^2.1.0"
@@ -4199,7 +4346,7 @@ elliptic@6.5.3:
     minimalistic-assert "^1.0.0"
     minimalistic-crypto-utils "^1.0.0"
 
-elliptic@6.5.4, elliptic@^6.4.0, elliptic@^6.5.2, elliptic@^6.5.3:
+elliptic@6.5.4, elliptic@^6.4.0, elliptic@^6.5.2, elliptic@^6.5.3, elliptic@^6.5.4:
   version "6.5.4"
   resolved "https://registry.yarnpkg.com/elliptic/-/elliptic-6.5.4.tgz#da37cebd31e79a1367e941b592ed1fbebd58abbb"
   integrity sha512-iLhC6ULemrljPZb+QutR5TQGB+pdW6KGD5RSegS+8sorOZT+rdQFbsQFJgvN3eRqNALqJer4oQ16YvJHlU8hzQ==
@@ -4293,7 +4440,7 @@ entities@~2.1.0:
 err-code@^1.1.2:
   version "1.1.2"
   resolved "https://registry.yarnpkg.com/err-code/-/err-code-1.1.2.tgz#06e0116d3028f6aef4806849eb0ea6a748ae6960"
-  integrity sha1-BuARbTAo9q70gGhJ6w6mp0iuaWA=
+  integrity sha512-CJAN+O0/yA1CKfRn9SXOGctSpEM7DCon/r/5r2eXFMY2zCCJBasFhcM5I+1kh3Ap11FsQCX+vGHceNPvpWKhoA==
 
 err-code@^2.0.0:
   version "2.0.3"
@@ -4421,7 +4568,7 @@ es6-promise@^4.0.3:
 es6-promisify@^5.0.0:
   version "5.0.0"
   resolved "https://registry.yarnpkg.com/es6-promisify/-/es6-promisify-5.0.0.tgz#5109d62f3e56ea967c4b63505aef08291c8a5203"
-  integrity sha1-UQnWLz5W6pZ8S2NQWu8IKRyKUgM=
+  integrity sha512-C+d6UdsYDk0lMebHNR4S2NybQMMngAOnOwYBQjTOiv0MkoJMP0Myw2mgpDLBcpfCmRLxyFqYhS/CfOENq4SJhQ==
   dependencies:
     es6-promise "^4.0.3"
 
@@ -4582,9 +4729,9 @@ eth-rpc-errors@^3.0.0:
     fast-safe-stringify "^2.0.6"
 
 ethereum-bloom-filters@^1.0.6:
-  version "1.0.9"
-  resolved "https://registry.yarnpkg.com/ethereum-bloom-filters/-/ethereum-bloom-filters-1.0.9.tgz#4a59dead803af0c9e33834170bd7695df67061ec"
-  integrity sha512-GiK/RQkAkcVaEdxKVkPcG07PQ5vD7v2MFSHgZmBJSfMzNRHimntdBithsHAT89tAXnIpzVDWt8iaCD1DvkaxGg==
+  version "1.0.10"
+  resolved "https://registry.yarnpkg.com/ethereum-bloom-filters/-/ethereum-bloom-filters-1.0.10.tgz#3ca07f4aed698e75bd134584850260246a5fed8a"
+  integrity sha512-rxJ5OFN3RwjQxDcFP2Z5+Q9ho4eIdEmSc2ht0fCu8Se9nbXjZ7/031uXoUYJ87KHCOdVeiUuwSnoS7hmYAGVHA==
   dependencies:
     js-sha3 "^0.8.0"
 
@@ -4794,7 +4941,7 @@ ethers@^4.0.32:
 ethjs-unit@0.1.6:
   version "0.1.6"
   resolved "https://registry.yarnpkg.com/ethjs-unit/-/ethjs-unit-0.1.6.tgz#c665921e476e87bce2a9d588a6fe0405b2c41699"
-  integrity sha1-xmWSHkduh7ziqdWIpv4EBbLEFpk=
+  integrity sha512-/Sn9Y0oKl0uqQuvgFk/zQgR7aw1g36qX/jzSQ5lSwlO0GigPymk4eGQfeNTD03w1dPOqfz8V77Cy43jH56pagw==
   dependencies:
     bn.js "4.11.6"
     number-to-bn "1.7.0"
@@ -4880,7 +5027,7 @@ expand-range@^1.8.1:
 explain-error@^1.0.4:
   version "1.0.4"
   resolved "https://registry.yarnpkg.com/explain-error/-/explain-error-1.0.4.tgz#a793d3ac0cad4c6ab571e9968fbbab6cb2532929"
-  integrity sha1-p5PTrAytTGq1cemWj7urbLJTKSk=
+  integrity sha512-/wSgNMxFusiYRy1rd19LT2SQlIXDppHpumpWo06wxjflD1OYxDLbl6rMVw+U3bxD5Nuhex4TKqv9Aem4D0lVzQ==
 
 express@^4.0.0, express@^4.14.0, express@^4.17.1:
   version "4.17.1"
@@ -4952,17 +5099,17 @@ extract-files@9.0.0, extract-files@^9.0.0:
 extsprintf@1.3.0:
   version "1.3.0"
   resolved "https://registry.yarnpkg.com/extsprintf/-/extsprintf-1.3.0.tgz#96918440e3041a7a414f8c52e3c574eb3c3e1e05"
-  integrity sha1-lpGEQOMEGnpBT4xS48V06zw+HgU=
+  integrity sha512-11Ndz7Nv+mvAC1j0ktTa7fAb0vLyGGX+rMHNBYQviQDGU0Hw7lhctJANqbPhu9nV9/izT/IntTgZ7Im/9LJs9g==
 
 extsprintf@^1.2.0:
-  version "1.4.0"
-  resolved "https://registry.yarnpkg.com/extsprintf/-/extsprintf-1.4.0.tgz#e2689f8f356fad62cca65a3a91c5df5f9551692f"
-  integrity sha1-4mifjzVvrWLMplo6kcXfX5VRaS8=
+  version "1.4.1"
+  resolved "https://registry.yarnpkg.com/extsprintf/-/extsprintf-1.4.1.tgz#8d172c064867f235c0c84a596806d279bf4bcc07"
+  integrity sha512-Wrk35e8ydCKDj/ArClo1VrPVmN8zph5V4AtHwIuHhvMXsKf73UT3BOD+azBIW+3wOJ4FhEH7zyaJCFvChjYvMA==
 
 eyes@^0.1.8:
   version "0.1.8"
   resolved "https://registry.yarnpkg.com/eyes/-/eyes-0.1.8.tgz#62cf120234c683785d902348a800ef3e0cc20bc0"
-  integrity sha1-Ys8SAjTGg3hdkCNIqADvPgzCC8A=
+  integrity sha512-GipyPsXO1anza0AOZdy69Im7hGFCNB7Y/NGjDlZGJ3GJJLtwNSb2vrzYrTYJRrRloVx7pl+bhUaTB8yiccPvFQ==
 
 fake-merkle-patricia-tree@^1.0.1:
   version "1.0.1"
@@ -5168,13 +5315,18 @@ flat@^4.1.0:
 flatmap@0.0.3:
   version "0.0.3"
   resolved "https://registry.yarnpkg.com/flatmap/-/flatmap-0.0.3.tgz#1f18a4d938152d495965f9c958d923ab2dd669b4"
-  integrity sha1-Hxik2TgVLUlZZfnJWNkjqy3WabQ=
+  integrity sha512-OuR+o7kHVe+x9RtIujPay7Uw3bvDZBZFSBXClEphZuSDLmZTqMdclasf4vFSsogC8baDz0eaC2NdO/2dlXHBKQ==
 
-follow-redirects@^1.12.1, follow-redirects@^1.14.0:
+follow-redirects@^1.12.1:
   version "1.14.8"
   resolved "https://registry.yarnpkg.com/follow-redirects/-/follow-redirects-1.14.8.tgz#016996fb9a11a100566398b1c6839337d7bfa8fc"
   integrity sha512-1x0S9UVJHsQprFcEC/qnNzBLcIxsjAV905f/UkQxbclCsoTWlacCNOpQa/anodLl2uaEKFhfWOvM2Qg77+15zA==
 
+follow-redirects@^1.14.0:
+  version "1.15.1"
+  resolved "https://registry.yarnpkg.com/follow-redirects/-/follow-redirects-1.15.1.tgz#0ca6a452306c9b276e4d3127483e29575e207ad5"
+  integrity sha512-yLAMQs+k0b2m7cVxpS1VKJVvoz7SS9Td1zss3XRwXj+ZDH00RJgnuLx7E44wx02kQLrdM3aOOy+FpzS7+8OizA==
+
 for-each@^0.3.3:
   version "0.3.3"
   resolved "https://registry.yarnpkg.com/for-each/-/for-each-0.3.3.tgz#69b447e88a0a5d32c3e7084f3f1710034b21376e"
@@ -5202,7 +5354,7 @@ foreach@^2.0.4, foreach@^2.0.5:
 forever-agent@~0.6.1:
   version "0.6.1"
   resolved "https://registry.yarnpkg.com/forever-agent/-/forever-agent-0.6.1.tgz#fbc71f0c41adeb37f96c577ad1ed42d8fdacca91"
-  integrity sha1-+8cfDEGt6zf5bFd60e1C2P2sypE=
+  integrity sha512-j0KLYPhm6zeac4lz3oJ3o65qvgQCcPubiyotZrXqEaG4hNagNYO8qdlUrX5vwqv9ohqeT/Z3j6+yW067yWWdUw==
 
 form-data@3.0.0:
   version "3.0.0"
@@ -5339,7 +5491,7 @@ fs-minipass@^2.0.0:
 fs.realpath@^1.0.0:
   version "1.0.0"
   resolved "https://registry.yarnpkg.com/fs.realpath/-/fs.realpath-1.0.0.tgz#1504ad2523158caa40db4a2787cb01411994ea4f"
-  integrity sha1-FQStJSMVjKpA20onh8sBQRmU6k8=
+  integrity sha512-OO0pH2lK6a0hZnAdau5ItzHPI6pUlvI7jMVnxUQRtw4owF2wk8lOSabtGDCTP4Ggrg2MbGnWO9X8K1t4+fGMDw==
 
 fsevents@~2.1.2:
   version "2.1.3"
@@ -5385,7 +5537,16 @@ get-caller-file@^2.0.1:
   resolved "https://registry.yarnpkg.com/get-caller-file/-/get-caller-file-2.0.5.tgz#4f94412a82db32f36e3b0b9741f8a97feb031f7e"
   integrity sha512-DyFP3BM/3YHTQOCUL/w0OZHR0lpKeGrxotcHWcqNEdnltqFwXVfhEBQ94eIo34AfQpo0rGki4cyIiftY06h2Fg==
 
-get-intrinsic@^1.0.2, get-intrinsic@^1.1.0, get-intrinsic@^1.1.1:
+get-intrinsic@^1.0.2:
+  version "1.1.2"
+  resolved "https://registry.yarnpkg.com/get-intrinsic/-/get-intrinsic-1.1.2.tgz#336975123e05ad0b7ba41f152ee4aadbea6cf598"
+  integrity sha512-Jfm3OyCxHh9DJyc28qGk+JmfkpO41A4XkneDSujN9MDXrm4oDKdHvndhZ2dN94+ERNfkYJWDclW6k2L/ZGHjXA==
+  dependencies:
+    function-bind "^1.1.1"
+    has "^1.0.3"
+    has-symbols "^1.0.3"
+
+get-intrinsic@^1.1.0, get-intrinsic@^1.1.1:
   version "1.1.1"
   resolved "https://registry.yarnpkg.com/get-intrinsic/-/get-intrinsic-1.1.1.tgz#15f59f376f855c446963948f0d24cd3637b4abc6"
   integrity sha512-kWZrnVM42QCiEA2Ig1bG8zjoIMOgxWwYCEeNdwY6Tv/cOSeGpcoX4pXHfKUxNKVoArnrEr2e9srnAxxGIraS9Q==
@@ -5434,7 +5595,7 @@ get-symbol-description@^1.0.0:
 getpass@^0.1.1:
   version "0.1.7"
   resolved "https://registry.yarnpkg.com/getpass/-/getpass-0.1.7.tgz#5eff8e3e684d569ae4cb2b1282604e8ba62149fa"
-  integrity sha1-Xv+OPmhNVprkyysSgmBOi6YhSfo=
+  integrity sha512-0fzj9JxOLfJ+XGLhR8ze3unN0KZCgZwiSSDz168VERjK8Wl8kVSdcu2kspd4s4wtAa1y/qrVRiAA0WclVsu0ng==
   dependencies:
     assert-plus "^1.0.0"
 
@@ -5482,7 +5643,7 @@ glob-stream@^5.3.2:
     to-absolute-glob "^0.1.1"
     unique-stream "^2.0.2"
 
-glob@7.1.6, glob@^7.1.1, glob@^7.1.2, glob@^7.1.3:
+glob@7.1.6, glob@^7.1.1, glob@^7.1.2:
   version "7.1.6"
   resolved "https://registry.yarnpkg.com/glob/-/glob-7.1.6.tgz#141f33b81a7c2492e125594307480c46679278a6"
   integrity sha512-LwaxwyZ72Lk7vZINtNNrywX0ZuLyStrdDtabefZKAY5ZGJhVtgdznluResxNmPitE0SAO+O26sWTHeKSI2wMBA==
@@ -5505,6 +5666,18 @@ glob@^5.0.3:
     once "^1.3.0"
     path-is-absolute "^1.0.0"
 
+glob@^7.1.3:
+  version "7.2.3"
+  resolved "https://registry.yarnpkg.com/glob/-/glob-7.2.3.tgz#b8df0fb802bbfa8e89bd1d938b4e16578ed44f2b"
+  integrity sha512-nFR0zLpU2YCaRxwoCJvL6UvCH2JFyFVIvwTLsIf21AuHlMskA1hhTdk+LlYJtOlYt9v6dvszD2BGRqBL+iQK9Q==
+  dependencies:
+    fs.realpath "^1.0.0"
+    inflight "^1.0.4"
+    inherits "2"
+    minimatch "^3.1.1"
+    once "^1.3.0"
+    path-is-absolute "^1.0.0"
+
 global@~4.4.0:
   version "4.4.0"
   resolved "https://registry.yarnpkg.com/global/-/global-4.4.0.tgz#3e7b105179006a323ed71aafca3e9c57a5cc6406"
@@ -5645,11 +5818,16 @@ got@^7.1.0:
     url-parse-lax "^1.0.0"
     url-to-options "^1.0.1"
 
-graceful-fs@4.X, graceful-fs@^4.0.0, graceful-fs@^4.1.11, graceful-fs@^4.1.2, graceful-fs@^4.1.6, graceful-fs@^4.1.9, graceful-fs@^4.2.0:
+graceful-fs@4.X, graceful-fs@^4.0.0, graceful-fs@^4.1.11, graceful-fs@^4.1.2, graceful-fs@^4.1.9:
   version "4.2.6"
   resolved "https://registry.yarnpkg.com/graceful-fs/-/graceful-fs-4.2.6.tgz#ff040b2b0853b23c3d31027523706f1885d76bee"
   integrity sha512-nTnJ528pbqxYanhpDYsi4Rd8MAeaBA67+RZ10CM1m3bTAVFEDcd5AuA4a6W5YkGZ1iNXHzZz8T6TBKLeBuNriQ==
 
+graceful-fs@^4.1.6, graceful-fs@^4.2.0:
+  version "4.2.10"
+  resolved "https://registry.yarnpkg.com/graceful-fs/-/graceful-fs-4.2.10.tgz#147d3a006da4ca3ce14728c7aefc287c367d7a6c"
+  integrity sha512-9ByhssR2fPVsNZj478qUUbKfmL0+t5BDVyjShtyZZLiK7ZDAArFFfopyOTj0M05wE2tJPisA4iTnnXl2YoPvOA==
+
 graphql-extensions@^0.15.0:
   version "0.15.0"
   resolved "https://registry.yarnpkg.com/graphql-extensions/-/graphql-extensions-0.15.0.tgz#3f291f9274876b0c289fa4061909a12678bd9817"
@@ -5766,7 +5944,7 @@ gulp-sourcemaps@^1.5.2:
 har-schema@^2.0.0:
   version "2.0.0"
   resolved "https://registry.yarnpkg.com/har-schema/-/har-schema-2.0.0.tgz#a94c2224ebcac04782a0d9035521f24735b7ec92"
-  integrity sha1-qUwiJOvKwEeCoNkDVSHyRzW37JI=
+  integrity sha512-Oqluz6zhGX8cyRaTQlFMPw80bSJVG2x/cFb8ZPhUILGgHka9SsokCCOQgpveePerqidZOrT14ipqfJb7ILcW5Q==
 
 har-validator@~5.1.3:
   version "5.1.5"
@@ -5791,7 +5969,7 @@ has-bigints@^1.0.0, has-bigints@^1.0.1:
 has-flag@^3.0.0:
   version "3.0.0"
   resolved "https://registry.yarnpkg.com/has-flag/-/has-flag-3.0.0.tgz#b5d454dc2199ae225699f3467e5a07f3b955bafd"
-  integrity sha1-tdRU3CGZriJWmfNGfloH87lVuv0=
+  integrity sha512-sKJf1+ceQBr4SMkvQnBDNDtf4TXpVhVGateu0t918bl30FnbE2m4vNLX+VWe/dpjlb+HugGYzW7uQXH98HPEYw==
 
 has-flag@^4.0.0:
   version "4.0.0"
@@ -5803,11 +5981,16 @@ has-symbol-support-x@^1.4.1:
   resolved "https://registry.yarnpkg.com/has-symbol-support-x/-/has-symbol-support-x-1.4.2.tgz#1409f98bc00247da45da67cee0a36f282ff26455"
   integrity sha512-3ToOva++HaW+eCpgqZrCfN51IPB+7bJNVT6CUATzueB5Heb8o6Nam0V3HG5dlDvZU1Gn5QLcbahiKw/XVk5JJw==
 
-has-symbols@^1.0.0, has-symbols@^1.0.1, has-symbols@^1.0.2:
+has-symbols@^1.0.0, has-symbols@^1.0.2:
   version "1.0.2"
   resolved "https://registry.yarnpkg.com/has-symbols/-/has-symbols-1.0.2.tgz#165d3070c00309752a1236a479331e3ac56f1423"
   integrity sha512-chXa79rL/UC2KlX17jo3vRGz0azaWEx5tGqZg5pO3NUyEJVB17dMruQlzCCOfUvElghKcm5194+BCRvi2Rv/Gw==
 
+has-symbols@^1.0.1, has-symbols@^1.0.3:
+  version "1.0.3"
+  resolved "https://registry.yarnpkg.com/has-symbols/-/has-symbols-1.0.3.tgz#bb7b2c4349251dce87b125f7bdf874aa7c8b39f8"
+  integrity sha512-l3LCuF6MgDNwTDKkdYGEihYjt5pRPbEg46rtlmnSPlUbgmB8LOIrKJbYYFBSbnPaJexMKtiPO8hmeRjRz2Td+A==
+
 has-to-string-tag-x@^1.2.0:
   version "1.4.1"
   resolved "https://registry.yarnpkg.com/has-to-string-tag-x/-/has-to-string-tag-x-1.4.1.tgz#a045ab383d7b4b2012a00148ab0aa5f290044d4d"
@@ -5851,7 +6034,7 @@ hash.js@1.1.3:
     inherits "^2.0.3"
     minimalistic-assert "^1.0.0"
 
-hash.js@^1.0.0, hash.js@^1.0.3, hash.js@^1.1.7:
+hash.js@1.1.7, hash.js@^1.0.0, hash.js@^1.0.3, hash.js@^1.1.7:
   version "1.1.7"
   resolved "https://registry.yarnpkg.com/hash.js/-/hash.js-1.1.7.tgz#0babca538e8d4ee4a0f8988d68866537a003cf42"
   integrity sha512-taOaskGt4z4SOANNseOviYDvjEJinIkRgmp7LbKP2YTTmVxWBl87s/uzK9r+44BclBSp2X7K1hqeNfz9JbBeXA==
@@ -5873,9 +6056,9 @@ header-case@^1.0.0:
     upper-case "^1.1.3"
 
 hi-base32@~0.5.0:
-  version "0.5.0"
-  resolved "https://registry.yarnpkg.com/hi-base32/-/hi-base32-0.5.0.tgz#61329f76a31f31008533f1c36f2473e259d64571"
-  integrity sha512-DDRmxSyoYuvjUb9EnXdoiMChBZ7ZcUVJsK5Frd3kqMhuBxvmZdnBeynAVfj7/ECbn++CekcoprvC/rprHPAtow==
+  version "0.5.1"
+  resolved "https://registry.yarnpkg.com/hi-base32/-/hi-base32-0.5.1.tgz#1279f2ddae2673219ea5870c2121d2a33132857e"
+  integrity sha512-EmBBpvdYh/4XxsnUybsPag6VikPYnN30td+vQk+GI3qpahVEG9+gTkG0aXVxTjBqQ5T6ijbWIu77O+C5WFWsnA==
 
 highlight.js@^10.4.0, highlight.js@^10.4.1:
   version "10.6.0"
@@ -6012,7 +6195,7 @@ http-response-object@^3.0.1:
 http-signature@~1.2.0:
   version "1.2.0"
   resolved "https://registry.yarnpkg.com/http-signature/-/http-signature-1.2.0.tgz#9aecd925114772f3d95b65a60abb8f7c18fbace1"
-  integrity sha1-muzZJRFHcvPZW2WmCruPfBj7rOE=
+  integrity sha512-CAbnr6Rz4CYQkLYUtSNXxQPUH2gK8f3iWexVlsnMeD+GjlsQ0Xsy1cOX+mN3dtxYomRy21CiOzU8Uhw6OwncEQ==
   dependencies:
     assert-plus "^1.0.0"
     jsprim "^1.2.2"
@@ -6052,7 +6235,7 @@ idna-uts46-hx@^2.3.1:
   dependencies:
     punycode "2.1.0"
 
-ieee754@^1.1.13:
+ieee754@^1.1.13, ieee754@^1.2.1:
   version "1.2.1"
   resolved "https://registry.yarnpkg.com/ieee754/-/ieee754-1.2.1.tgz#8eb7a10a63fff25d15a57b001586d177d1b0d352"
   integrity sha512-dcyqhDvX1C46lXZcVqCpK+FtMRQVdIMN6/Df5js2zouUsqG7I6sFxitIC+7KYK29KdXOLHdu9zL4sFnoVQnqaA==
@@ -6117,7 +6300,7 @@ imurmurhash@^0.1.4:
 inflight@^1.0.4:
   version "1.0.6"
   resolved "https://registry.yarnpkg.com/inflight/-/inflight-1.0.6.tgz#49bd6331d7d02d0c09bc910a1075ba8165b56df9"
-  integrity sha1-Sb1jMdfQLQwJvJEKEHW6gWW1bfk=
+  integrity sha512-k92I/b08q4wvFscXCLvqfsHCrjrF7yiXsQuIVvVE7N82W3+aqpzuUdBbfhWcy/FZR3/4IgflMgKLOsvPDrGCJA==
   dependencies:
     once "^1.3.0"
     wrappy "1"
@@ -6161,7 +6344,7 @@ invert-kv@^1.0.0:
 ip-regex@^2.0.0:
   version "2.1.0"
   resolved "https://registry.yarnpkg.com/ip-regex/-/ip-regex-2.1.0.tgz#fa78bf5d2e6913c911ce9f819ee5146bb6d844e9"
-  integrity sha1-+ni/XS5pE8kRzp+BnuUUa7bYROk=
+  integrity sha512-58yWmlHpp7VYfcdTwMTvwMmqx/Elfxjd9RXTDyMsbL7lLWmhMylLEqiYVLKuLzOZqVgiWXD9MfR62Vv89VRxkw==
 
 ip-regex@^4.0.0:
   version "4.3.0"
@@ -6169,9 +6352,9 @@ ip-regex@^4.0.0:
   integrity sha512-B9ZWJxHHOHUhUjCPrMpLD4xEq35bUTClHM1S6CBU5ixQnkZmwipwgc96vAd7AAGM9TGHvJR+Uss+/Ak6UphK+Q==
 
 ip@^1.1.5:
-  version "1.1.5"
-  resolved "https://registry.yarnpkg.com/ip/-/ip-1.1.5.tgz#bdded70114290828c0a039e72ef25f5aaec4354a"
-  integrity sha1-vd7XARQpCCjAoDnnLvJfWq7ENUo=
+  version "1.1.8"
+  resolved "https://registry.yarnpkg.com/ip/-/ip-1.1.8.tgz#ae05948f6b075435ed3307acce04629da8cdbf48"
+  integrity sha512-PuExPYUiu6qMBQb4l06ecm6T6ujzhmh+MeJcW9wa89PoAz5pvd4zPgN5WJV104mb6S2T1AwNIAaB70JNrLQWhg==
 
 ipaddr.js@1.9.1:
   version "1.9.1"
@@ -6299,7 +6482,7 @@ is-arguments@^1.0.4, is-arguments@^1.1.0:
 is-arrayish@^0.2.1:
   version "0.2.1"
   resolved "https://registry.yarnpkg.com/is-arrayish/-/is-arrayish-0.2.1.tgz#77c99840527aa8ecb1a8ba697b80645a7a926a9d"
-  integrity sha1-d8mYQFJ6qOyxqLppe4BkWnqSap0=
+  integrity sha512-zz06S8t0ozoDXMG+ube26zeCTNXcKIPJZJi8hBrF4idCLms4CG9QtK7qBl1boi5ODzFpjswb5JPmHCbMpjaYzg==
 
 is-bigint@^1.0.1:
   version "1.0.4"
@@ -6368,9 +6551,9 @@ is-dotfile@^1.0.0:
   integrity sha1-pqLzL/0t+wT1yiXs0Pa4PPeYoeE=
 
 is-electron@^2.2.0:
-  version "2.2.0"
-  resolved "https://registry.yarnpkg.com/is-electron/-/is-electron-2.2.0.tgz#8943084f09e8b731b3a7a0298a7b5d56f6b7eef0"
-  integrity sha512-SpMppC2XR3YdxSzczXReBjqs2zGscWQpBIKqwXYBFic0ERaxNVgwLCHwOLZeESfdJQjX0RDvrJ1lBXX2ij+G1Q==
+  version "2.2.1"
+  resolved "https://registry.yarnpkg.com/is-electron/-/is-electron-2.2.1.tgz#751b1dd8a74907422faa5c35aaa0cf66d98086e9"
+  integrity sha512-r8EEQQsqT+Gn0aXFx7lTFygYQhILLCB+wn0WCDL5LZRINeLH/Rvw1j2oKodELLXYNImQ3CRlVsY8wW4cGOsyuw==
 
 is-equal-shallow@^0.1.3:
   version "0.1.3"
@@ -6414,7 +6597,7 @@ is-fullwidth-code-point@^1.0.0:
 is-fullwidth-code-point@^2.0.0:
   version "2.0.0"
   resolved "https://registry.yarnpkg.com/is-fullwidth-code-point/-/is-fullwidth-code-point-2.0.0.tgz#a3b30a5c4f199183167aaab93beefae3ddfb654f"
-  integrity sha1-o7MKXE8ZkYMWeqq5O+764937ZU8=
+  integrity sha512-VHskAKYM8RfSFXwee5t5cbN5PZeq1Wrh6qd5bkyiXIf6UQcN6w/A0eXM9r6t8d+GYOh+o6ZhiEnb88LN/Y8m2w==
 
 is-fullwidth-code-point@^3.0.0:
   version "3.0.0"
@@ -6431,7 +6614,7 @@ is-generator-function@^1.0.7:
   resolved "https://registry.yarnpkg.com/is-generator-function/-/is-generator-function-1.0.8.tgz#dfb5c2b120e02b0a8d9d2c6806cd5621aa922f7b"
   integrity sha512-2Omr/twNtufVZFr1GhxjOMFPAj2sjc/dKaIqBhvo4qciXfJmITGH6ZGd8eZYNHza8t1y0e01AuqRhJwfWp26WQ==
 
-is-glob@4.0.1, is-glob@^4.0.1, is-glob@~4.0.1:
+is-glob@4.0.1:
   version "4.0.1"
   resolved "https://registry.yarnpkg.com/is-glob/-/is-glob-4.0.1.tgz#7567dbe9f2f5e2467bc77ab83c4a29482407a5dc"
   integrity sha512-5G0tKtBTFImOqDnLB2hG6Bp2qcKEFduo4tZu9MT/H6NQv/ghhy30o55ufafxJ/LdH79LLs2Kfrn85TLKyA7BUg==
@@ -6452,10 +6635,17 @@ is-glob@^3.1.0:
   dependencies:
     is-extglob "^2.1.0"
 
+is-glob@^4.0.1, is-glob@~4.0.1:
+  version "4.0.3"
+  resolved "https://registry.yarnpkg.com/is-glob/-/is-glob-4.0.3.tgz#64f61e42cbbb2eec2071a9dac0b28ba1e65d5084"
+  integrity sha512-xelSayHH36ZgE7ZWhli7pW34hNbNl8Ojv5KVmkJD4hBdD3th8Tfk9vYasLM+mXWOZhFkgZfxhLSnrwRr4elSSg==
+  dependencies:
+    is-extglob "^2.1.1"
+
 is-hex-prefixed@1.0.0:
   version "1.0.0"
   resolved "https://registry.yarnpkg.com/is-hex-prefixed/-/is-hex-prefixed-1.0.0.tgz#7d8d37e6ad77e5d127148913c573e082d777f554"
-  integrity sha1-fY035q135dEnFIkTxXPggtd39VQ=
+  integrity sha512-WvtOiug1VFrE9v1Cydwm+FnXd3+w9GaeVUss5W4v/SLy3UW00vP+6iNF2SdnfiBoLy4bTqVdkftNGTUeOFVsbA==
 
 is-interactive@^1.0.0:
   version "1.0.0"
@@ -6465,7 +6655,7 @@ is-interactive@^1.0.0:
 is-ip@^2.0.0:
   version "2.0.0"
   resolved "https://registry.yarnpkg.com/is-ip/-/is-ip-2.0.0.tgz#68eea07e8a0a0a94c2d080dd674c731ab2a461ab"
-  integrity sha1-aO6gfooKCpTC0IDdZ0xzGrKkYas=
+  integrity sha512-9MTn0dteHETtyUx8pxqMwg5hMBi3pvlyglJ+b79KOCca0po23337LbVV2Hl4xmMvfw++ljnO0/+5G6G+0Szh6g==
   dependencies:
     ip-regex "^2.0.0"
 
@@ -6562,12 +6752,12 @@ is-promise@4.0.0:
 is-promise@~1, is-promise@~1.0.0:
   version "1.0.1"
   resolved "https://registry.yarnpkg.com/is-promise/-/is-promise-1.0.1.tgz#31573761c057e33c2e91aab9e96da08cefbe76e5"
-  integrity sha1-MVc3YcBX4zwukaq56W2gjO++duU=
+  integrity sha512-mjWH5XxnhMA8cFnDchr6qRP9S/kLntKuEfIYku+PaN1CnS8v+OG9O/BKpRCVRJvpIkgAZm0Pf5Is3iSSOILlcg==
 
 is-pull-stream@0.0.0:
   version "0.0.0"
   resolved "https://registry.yarnpkg.com/is-pull-stream/-/is-pull-stream-0.0.0.tgz#a3bc3d1c6d3055151c46bde6f399efed21440ca9"
-  integrity sha1-o7w9HG0wVRUcRr3m85nv7SFEDKk=
+  integrity sha512-NWLwqCc95I6m8FZDYLAmVJc9Xgk8O+8pPOoDKFTC293FH4S7FBcbLCw3WWPCdiT8uUSdzPy47VM08WPDMJJrag==
 
 is-regex@^1.1.2:
   version "1.1.2"
@@ -6606,9 +6796,9 @@ is-stream@^1.0.0, is-stream@^1.0.1:
   integrity sha1-EtSj3U5o4Lec6428hBc66A2RykQ=
 
 is-stream@^2.0.0:
-  version "2.0.0"
-  resolved "https://registry.yarnpkg.com/is-stream/-/is-stream-2.0.0.tgz#bde9c32680d6fae04129d6ac9d921ce7815f78e3"
-  integrity sha512-XCoy+WlUr7d1+Z8GgSuXmpuUFC9fOhRXglJMx+dwLKTkL44Cjd4W1Z5P+BQZpr+cR93aGP4S/s7Ftw6Nd/kiEw==
+  version "2.0.1"
+  resolved "https://registry.yarnpkg.com/is-stream/-/is-stream-2.0.1.tgz#fac1e3d53b97ad5a9d0ae9cef2389f5810a5c077"
+  integrity sha512-hFoiJiTl63nn+kstHGBtewWSKnQLpyb155KHheA1l39uvtO9nWIop1p3udqPcUd/xbF1VLMO4n7OI6p7RbngDg==
 
 is-string@^1.0.5, is-string@^1.0.7:
   version "1.0.7"
@@ -6667,7 +6857,7 @@ is-weakref@^1.0.1:
 isarray@0.0.1:
   version "0.0.1"
   resolved "https://registry.yarnpkg.com/isarray/-/isarray-0.0.1.tgz#8a18acfca9a8f4177e09abfc6038939b05d1eedf"
-  integrity sha1-ihis/Kmo9Bd+Cav8YDiTmwXR7t8=
+  integrity sha512-D2S+3GLxWH+uhrNEcoh/fnmYeP8E8/zHl644d/jdA0g2uyXvy3sb0qxotE+ne0LtccHknQzWwZEzhak7oJ0COQ==
 
 isarray@1.0.0, isarray@~1.0.0:
   version "1.0.0"
@@ -6682,14 +6872,14 @@ isarray@^2.0.5:
 isexe@^2.0.0:
   version "2.0.0"
   resolved "https://registry.yarnpkg.com/isexe/-/isexe-2.0.0.tgz#e8fbf374dc556ff8947a10dcb0572d633f2cfa10"
-  integrity sha1-6PvzdNxVb/iUehDcsFctYz8s+hA=
+  integrity sha512-RHxMLp9lnKHGHRng9QFhRCMbYAcVpn69smSGcq3f36xjgVVWThj4qqLbTLlq7Ssj8B+fIQ1EuCEGI2lKsyQeIw==
 
 iso-random-stream@^1.1.0:
-  version "1.1.1"
-  resolved "https://registry.yarnpkg.com/iso-random-stream/-/iso-random-stream-1.1.1.tgz#83824bba77fbb3480dd6b35fbb06de7f9e93e80f"
-  integrity sha512-YEt/7xOwTdu4KXIgtdgGFkiLUsBaddbnkmHyaFdjJYIcD7V4gpQHPvYC5tyh3kA0PQ01y9lWm1ruVdf8Mqzovg==
+  version "1.1.2"
+  resolved "https://registry.yarnpkg.com/iso-random-stream/-/iso-random-stream-1.1.2.tgz#c703da2c518db573277c5678cc43c5298283d64c"
+  integrity sha512-7y0tsBBgQs544iTYjyrMp5xvgrbYR8b+plQq1Bryp+03p0LssrxC9C1M0oHv4QESDt7d95c74XvMk/yawKqX+A==
   dependencies:
-    buffer "^5.4.3"
+    buffer "^6.0.3"
     readable-stream "^3.4.0"
 
 iso-stream-http@~0.1.2:
@@ -6726,7 +6916,7 @@ isomorphic-ws@4.0.1, isomorphic-ws@^4.0.1:
 isstream@~0.1.2:
   version "0.1.2"
   resolved "https://registry.yarnpkg.com/isstream/-/isstream-0.1.2.tgz#47e63f7af55afa6f92e1500e690eb8b8529c099a"
-  integrity sha1-R+Y/evVa+m+S4VAOaQ64uFKcCZo=
+  integrity sha512-Yljz7ffyPbrLpLngrMtZ7NduUgVvi6wG9RJ9IUcyCd59YQ911PBJphODUcbOVbqYfxe1wuYf/LJ8PauMRwsM/g==
 
 isurl@^1.0.0-alpha5:
   version "1.0.0"
@@ -6851,7 +7041,7 @@ jsan@^3.1.13:
 jsbn@~0.1.0:
   version "0.1.1"
   resolved "https://registry.yarnpkg.com/jsbn/-/jsbn-0.1.1.tgz#a5e654c2e5a2deb5f201d96cefbca80c0ef2f513"
-  integrity sha1-peZUwuWi3rXyAdls77yoDA7y9RM=
+  integrity sha512-UVU9dibq2JcFWxQPA6KCqj5O42VOmAY3zQUfEKxU0KpTGXwNoCjkX1e13eHNvw/xPynt6pU0rZ1htjWTNTSXsg==
 
 jsdom@^7.0.2:
   version "7.2.2"
@@ -6919,10 +7109,10 @@ json-schema-traverse@^0.4.1:
   resolved "https://registry.yarnpkg.com/json-schema-traverse/-/json-schema-traverse-0.4.1.tgz#69f6a87d9513ab8bb8fe63bdb0979c448e684660"
   integrity sha512-xbbCH5dCYU5T8LcEhhuh7HJ88HXuW3qsI3Y0zOZFKfZEHcpWiHU/Jxzk629Brsab/mMiHQti9wMP+845RPe3Vg==
 
-json-schema@0.2.3:
-  version "0.2.3"
-  resolved "https://registry.yarnpkg.com/json-schema/-/json-schema-0.2.3.tgz#b480c892e59a2f05954ce727bd3f2a4e882f9e13"
-  integrity sha1-tIDIkuWaLwWVTOcnvT8qTogvnhM=
+json-schema@0.4.0:
+  version "0.4.0"
+  resolved "https://registry.yarnpkg.com/json-schema/-/json-schema-0.4.0.tgz#f7de4cf6efab838ebaeb3236474cbba5a1930ab5"
+  integrity sha512-es94M3nTIfsEPisRafak+HDLfHXnKBhV3vU5eqPcS3flIWqcxJWgXHXiey3YrpaNsanY5ei1VoYEbOzijuq9BA==
 
 json-stable-stringify-without-jsonify@^1.0.1:
   version "1.0.1"
@@ -6939,12 +7129,12 @@ json-stable-stringify@^1.0.1:
 json-stringify-safe@^5.0.1, json-stringify-safe@~5.0.1:
   version "5.0.1"
   resolved "https://registry.yarnpkg.com/json-stringify-safe/-/json-stringify-safe-5.0.1.tgz#1296a2d58fd45f19a0f6ce01d65701e2c735b6eb"
-  integrity sha1-Epai1Y/UXxmg9s4B1lcB4sc1tus=
+  integrity sha512-ZClg6AaYvamvYEE82d3Iyd3vSSIjQ+odgjaTzRuO3s7toCdFKczob2i0zCh7JE8kWn17yvAWhUVxvqGwUalsRA==
 
 json-text-sequence@~0.1.0:
   version "0.1.1"
   resolved "https://registry.yarnpkg.com/json-text-sequence/-/json-text-sequence-0.1.1.tgz#a72f217dc4afc4629fff5feb304dc1bd51a2f3d2"
-  integrity sha1-py8hfcSvxGKf/1/rME3BvVGi89I=
+  integrity sha512-L3mEegEWHRekSHjc7+sc8eJhba9Clq1PZ8kMkzf8OxElhXc8O4TS5MwcVlj9aEbm5dr81N90WHC5nAz3UO971w==
   dependencies:
     delimit-stream "0.1.0"
 
@@ -6999,16 +7189,16 @@ jsonify@~0.0.0:
 jsonparse@^1.2.0:
   version "1.3.1"
   resolved "https://registry.yarnpkg.com/jsonparse/-/jsonparse-1.3.1.tgz#3f4dae4a91fac315f71062f8521cc239f1366280"
-  integrity sha1-P02uSpH6wxX3EGL4UhzCOfE2YoA=
+  integrity sha512-POQXvpdL69+CluYsillJ7SUhKvytYjW9vG/GKpnf+xP8UWgYEM/RaMzHHofbALDiKbbP1W8UEYmgGl39WkPZsg==
 
 jsprim@^1.2.2:
-  version "1.4.1"
-  resolved "https://registry.yarnpkg.com/jsprim/-/jsprim-1.4.1.tgz#313e66bc1e5cc06e438bc1b7499c2e5c56acb6a2"
-  integrity sha1-MT5mvB5cwG5Di8G3SZwuXFastqI=
+  version "1.4.2"
+  resolved "https://registry.yarnpkg.com/jsprim/-/jsprim-1.4.2.tgz#712c65533a15c878ba59e9ed5f0e26d5b77c5feb"
+  integrity sha512-P2bSOMAc/ciLz6DzgjVlGJP9+BrJWu5UDGK70C2iweC5QBIeFf0ZXRvGjEj2uYgrY2MkAAhsSWHDWlFtEroZWw==
   dependencies:
     assert-plus "1.0.0"
     extsprintf "1.3.0"
-    json-schema "0.2.3"
+    json-schema "0.4.0"
     verror "1.10.0"
 
 just-kebab-case@^1.1.0:
@@ -7017,17 +7207,18 @@ just-kebab-case@^1.1.0:
   integrity sha512-QkuwuBMQ9BQHMUEkAtIA4INLrkmnnveqlFB1oFi09gbU0wBdZo6tTnyxNWMR84zHxBuwK7GLAwqN8nrvVxOLTA==
 
 just-map-keys@^1.1.0:
-  version "1.1.0"
-  resolved "https://registry.yarnpkg.com/just-map-keys/-/just-map-keys-1.1.0.tgz#9663c9f971ba46e17f2b05e66fec81149375f230"
-  integrity sha512-oNKi+4y7fr8lXnhKYpBbCkiwHRVkAnx0VDkCeTDtKKMzGr1Lz1Yym+RSieKUTKim68emC5Yxrb4YmiF9STDO+g==
+  version "1.2.1"
+  resolved "https://registry.yarnpkg.com/just-map-keys/-/just-map-keys-1.2.1.tgz#ef6e16133b7d34329962dfae9101d581abb1b143"
+  integrity sha512-Dmyz1Cy2SWM+PpqDPB1kdDglyexdzMthnAsvOIE9w4OPj8NDRuY1mh20x/JfG5w6fCGw9F0WmcofJhYZ4MiuyA==
 
 keccak@^3.0.0:
-  version "3.0.1"
-  resolved "https://registry.yarnpkg.com/keccak/-/keccak-3.0.1.tgz#ae30a0e94dbe43414f741375cff6d64c8bea0bff"
-  integrity sha512-epq90L9jlFWCW7+pQa6JOnKn2Xgl2mtI664seYR6MHskvI9agt7AnDqmAlp9TqU4/caMYbA08Hi5DMZAl5zdkA==
+  version "3.0.2"
+  resolved "https://registry.yarnpkg.com/keccak/-/keccak-3.0.2.tgz#4c2c6e8c54e04f2670ee49fa734eb9da152206e0"
+  integrity sha512-PyKKjkH53wDMLGrvmRGSNWgmSxZOUqbnXwKL9tmgbFYA1iAYqW21kfR7mZXV0MlESiefxQQE9X9fTa3X+2MPDQ==
   dependencies:
     node-addon-api "^2.0.0"
     node-gyp-build "^4.2.0"
+    readable-stream "^3.6.0"
 
 keypair@^1.0.1:
   version "1.0.4"
@@ -7279,9 +7470,9 @@ libp2p-crypto-secp256k1@~0.3.0:
     secp256k1 "^3.6.2"
 
 libp2p-crypto@~0.16.1:
-  version "0.16.3"
-  resolved "https://registry.yarnpkg.com/libp2p-crypto/-/libp2p-crypto-0.16.3.tgz#a4012361a6b6b3328d3d6b67cd1cb278e8d58f59"
-  integrity sha512-ro7/5Tu+f8p2+qDS1JrROnO++nNaAaBFs+VVXVHLuTMnbnMASu1eUtSlWPk1uOwikAlBFTvfqe5J1bK6Bpq6Pg==
+  version "0.16.4"
+  resolved "https://registry.yarnpkg.com/libp2p-crypto/-/libp2p-crypto-0.16.4.tgz#fb1a4ba39d56789303947784b5b0d6cefce12fdc"
+  integrity sha512-II8HxKc9jbmQp34pprlluNxsBCWJDjHRPYJzuRy7ragztNip9Zb7uJ4lCje6gGzz4DNAcHkAUn+GqCIK1592iA==
   dependencies:
     asmcrypto.js "^2.3.2"
     asn1.js "^5.0.1"
@@ -7293,7 +7484,7 @@ libp2p-crypto@~0.16.1:
     keypair "^1.0.1"
     libp2p-crypto-secp256k1 "~0.3.0"
     multihashing-async "~0.5.1"
-    node-forge "~0.9.1"
+    node-forge "^0.10.0"
     pem-jwk "^2.0.0"
     protons "^1.0.1"
     rsa-pem-to-jwk "^1.1.3"
@@ -7301,9 +7492,9 @@ libp2p-crypto@~0.16.1:
     ursa-optional "~0.10.0"
 
 lines-and-columns@^1.1.6:
-  version "1.1.6"
-  resolved "https://registry.yarnpkg.com/lines-and-columns/-/lines-and-columns-1.1.6.tgz#1c00c743b433cd0a4e80758f7b64a57440d9ff00"
-  integrity sha1-HADHQ7QzzQpOgHWPe2SldEDZ/wA=
+  version "1.2.4"
+  resolved "https://registry.yarnpkg.com/lines-and-columns/-/lines-and-columns-1.2.4.tgz#eca284f75d2965079309dc0ad9255abb2ebc1632"
+  integrity sha512-7ylylesZQ/PV29jhEDl3Ufjo6ZX7gCqJr5F7PKrqc93v7fzSymt1BpwEU8nAUXs8qzzvqhbjhK5QZg6Mt/HkBg==
 
 linked-list@0.1.0:
   version "0.1.0"
@@ -7379,7 +7570,7 @@ lodash.assigninwith@^4.0.0:
 lodash.camelcase@^4.3.0:
   version "4.3.0"
   resolved "https://registry.yarnpkg.com/lodash.camelcase/-/lodash.camelcase-4.3.0.tgz#b28aa6288a2b9fc651035c7711f65ab6190331a6"
-  integrity sha1-soqmKIorn8ZRA1x3EfZathkDMaY=
+  integrity sha512-TwuEnCnxbc3rAvhf/LbG7tJUDzhqXyFnv3dtzLOPgCG/hODL7WFnsbwktkD7yUV0RrreP/l1PALq/YSg6VvjlA==
 
 lodash.clonedeep@^4.5.0:
   version "4.5.0"
@@ -7414,7 +7605,7 @@ lodash.isequal@^4.0.0:
 lodash.kebabcase@^4.1.1:
   version "4.1.1"
   resolved "https://registry.yarnpkg.com/lodash.kebabcase/-/lodash.kebabcase-4.1.1.tgz#8489b1cb0d29ff88195cceca448ff6d6cc295c36"
-  integrity sha1-hImxyw0p/4gZXM7KRI/21swpXDY=
+  integrity sha512-N8XRTIMMqqDgSy4VLKPnJ/+hpGZN+PHQiJnSenYqPaVV/NCqEogTnAdZLQiGKhxX+JCs8waWq2t1XHWKOmlY8g==
 
 lodash.keys@^4.0.0:
   version "4.2.0"
@@ -7424,12 +7615,12 @@ lodash.keys@^4.0.0:
 lodash.lowercase@^4.3.0:
   version "4.3.0"
   resolved "https://registry.yarnpkg.com/lodash.lowercase/-/lodash.lowercase-4.3.0.tgz#46515aced4acb0b7093133333af068e4c3b14e9d"
-  integrity sha1-RlFaztSssLcJMTMzOvBo5MOxTp0=
+  integrity sha512-UcvP1IZYyDKyEL64mmrwoA1AbFu5ahojhTtkOUr1K9dbuxzS9ev8i4TxMMGCqRC9TE8uDaSoufNAXxRPNTseVA==
 
 lodash.lowerfirst@^4.3.1:
   version "4.3.1"
   resolved "https://registry.yarnpkg.com/lodash.lowerfirst/-/lodash.lowerfirst-4.3.1.tgz#de3c7b12e02c6524a0059c2f6cb7c5c52655a13d"
-  integrity sha1-3jx7EuAsZSSgBZwvbLfFxSZVoT0=
+  integrity sha512-UUKX7VhP1/JL54NXg2aq/E1Sfnjjes8fNYTNkPU8ZmsaVeBvPHKdbNaN79Re5XRL01u6wbq3j0cbYZj71Fcu5w==
 
 lodash.merge@^4.6.2:
   version "4.6.2"
@@ -7439,17 +7630,17 @@ lodash.merge@^4.6.2:
 lodash.pad@^4.5.1:
   version "4.5.1"
   resolved "https://registry.yarnpkg.com/lodash.pad/-/lodash.pad-4.5.1.tgz#4330949a833a7c8da22cc20f6a26c4d59debba70"
-  integrity sha1-QzCUmoM6fI2iLMIPaibE1Z3runA=
+  integrity sha512-mvUHifnLqM+03YNzeTBS1/Gr6JRFjd3rRx88FHWUvamVaT9k2O/kXha3yBSOwB9/DTQrSTLJNHvLBBt2FdX7Mg==
 
 lodash.padend@^4.6.1:
   version "4.6.1"
   resolved "https://registry.yarnpkg.com/lodash.padend/-/lodash.padend-4.6.1.tgz#53ccba047d06e158d311f45da625f4e49e6f166e"
-  integrity sha1-U8y6BH0G4VjTEfRdpiX05J5vFm4=
+  integrity sha512-sOQs2aqGpbl27tmCS1QNZA09Uqp01ZzWfDUoD+xzTii0E7dSQfRKcRetFwa+uXaxaqL+TKm7CgD2JdKP7aZBSw==
 
 lodash.padstart@^4.6.1:
   version "4.6.1"
   resolved "https://registry.yarnpkg.com/lodash.padstart/-/lodash.padstart-4.6.1.tgz#d2e3eebff0d9d39ad50f5cbd1b52a7bce6bb611b"
-  integrity sha1-0uPuv/DZ05rVD1y9G1KnvOa7YRs=
+  integrity sha512-sW73O6S8+Tg66eY56DBk85aQzzUJDtpoXFBgELMd5P/SotAguo+1kYO6RuYgXxA4HJH3LFTFPASX6ET6bjfriw==
 
 lodash.partition@^4.6.0:
   version "4.6.0"
@@ -7459,7 +7650,7 @@ lodash.partition@^4.6.0:
 lodash.repeat@^4.1.0:
   version "4.1.0"
   resolved "https://registry.yarnpkg.com/lodash.repeat/-/lodash.repeat-4.1.0.tgz#fc7de8131d8c8ac07e4b49f74ffe829d1f2bec44"
-  integrity sha1-/H3oEx2MisB+S0n3T/6CnR8r7EQ=
+  integrity sha512-eWsgQW89IewS95ZOcr15HHCX6FVDxq3f2PNUIng3fyzsPev9imFQxIYdFZ6crl8L56UR6ZlGDLcEb3RZsCSSqw==
 
 lodash.rest@^4.0.0:
   version "4.0.5"
@@ -7469,7 +7660,7 @@ lodash.rest@^4.0.0:
 lodash.snakecase@^4.1.1:
   version "4.1.1"
   resolved "https://registry.yarnpkg.com/lodash.snakecase/-/lodash.snakecase-4.1.1.tgz#39d714a35357147837aefd64b5dcbb16becd8f8d"
-  integrity sha1-OdcUo1NXFHg3rv1ktdy7Fr7Nj40=
+  integrity sha512-QZ1d4xoBHYUeuouhEq3lk3Uq7ldgyFXGBhg04+oRLnIz8o9T65Eh+8YdroUwn846zchkA9yDsDl5CVVaV2nqYw==
 
 lodash.sortby@^4.7.0:
   version "4.7.0"
@@ -7479,7 +7670,7 @@ lodash.sortby@^4.7.0:
 lodash.startcase@^4.4.0:
   version "4.4.0"
   resolved "https://registry.yarnpkg.com/lodash.startcase/-/lodash.startcase-4.4.0.tgz#9436e34ed26093ed7ffae1936144350915d9add8"
-  integrity sha1-lDbjTtJgk+1/+uGTYUQ1CRXZrdg=
+  integrity sha512-+WKqsK294HMSc2jEbNgpHpd0JfIBhp7rEV4aqXWqFr6AlXov+SlcgB1Fv01y2kGe3Gc8nMW7VA0SrGuSkRfIEg==
 
 lodash.sum@^4.0.2:
   version "4.0.2"
@@ -7513,27 +7704,27 @@ lodash.tostring@^4.0.0:
 lodash.trim@^4.5.1:
   version "4.5.1"
   resolved "https://registry.yarnpkg.com/lodash.trim/-/lodash.trim-4.5.1.tgz#36425e7ee90be4aa5e27bcebb85b7d11ea47aa57"
-  integrity sha1-NkJefukL5KpeJ7zruFt9EepHqlc=
+  integrity sha512-nJAlRl/K+eiOehWKDzoBVrSMhK0K3A3YQsUNXHQa5yIrKBAhsZgSu3KoAFoFT+mEgiyBHddZ0pRk1ITpIp90Wg==
 
 lodash.trimend@^4.5.1:
   version "4.5.1"
   resolved "https://registry.yarnpkg.com/lodash.trimend/-/lodash.trimend-4.5.1.tgz#12804437286b98cad8996b79414e11300114082f"
-  integrity sha1-EoBENyhrmMrYmWt5QU4RMAEUCC8=
+  integrity sha512-lsD+k73XztDsMBKPKvzHXRKFNMohTjoTKIIo4ADLn5dA65LZ1BqlAvSXhR2rPEC3BgAUQnzMnorqDtqn2z4IHA==
 
 lodash.trimstart@^4.5.1:
   version "4.5.1"
   resolved "https://registry.yarnpkg.com/lodash.trimstart/-/lodash.trimstart-4.5.1.tgz#8ff4dec532d82486af59573c39445914e944a7f1"
-  integrity sha1-j/TexTLYJIavWVc8OURZFOlEp/E=
+  integrity sha512-b/+D6La8tU76L/61/aN0jULWHkT0EeJCmVstPBn/K9MtD2qBW83AsBNrr63dKuWYwVMO7ucv13QNO/Ek/2RKaQ==
 
 lodash.uppercase@^4.3.0:
   version "4.3.0"
   resolved "https://registry.yarnpkg.com/lodash.uppercase/-/lodash.uppercase-4.3.0.tgz#c404abfd1469f93931f9bb24cf6cc7d57059bc73"
-  integrity sha1-xASr/RRp+Tkx+bskz2zH1XBZvHM=
+  integrity sha512-+Nbnxkj7s8K5U8z6KnEYPGUOGp3woZbB7Ecs7v3LkkjLQSm2kP9SKIILitN1ktn2mB/tmM9oSlku06I+/lH7QA==
 
 lodash.upperfirst@^4.3.1:
   version "4.3.1"
   resolved "https://registry.yarnpkg.com/lodash.upperfirst/-/lodash.upperfirst-4.3.1.tgz#1365edf431480481ef0d1c68957a5ed99d49f7ce"
-  integrity sha1-E2Xt9DFIBIHvDRxolXpe2Z1J984=
+  integrity sha512-sReKOYJIJf74dhJONhU4e0/shzi1trVbSWDOhKYE5XV2O+H7Sb2Dihwuc7xWxVl+DgFPyTqIN3zMfT9cq5iWDg==
 
 lodash.zipwith@^4.2.0:
   version "4.2.0"
@@ -7579,7 +7770,7 @@ long@^4.0.0:
 looper@^3.0.0:
   version "3.0.0"
   resolved "https://registry.yarnpkg.com/looper/-/looper-3.0.0.tgz#2efa54c3b1cbaba9b94aee2e5914b0be57fbb749"
-  integrity sha1-LvpUw7HLq6m5Su4uWRSwvlf7t0k=
+  integrity sha512-LJ9wplN/uSn72oJRsXTx+snxPet5c8XiZmOKCm906NVYu+ag6SB6vUcnJcWxgnl2NfbIyeobAn7Bwv6xRj2XJg==
 
 loose-envify@^1.0.0, loose-envify@^1.1.0, loose-envify@^1.4.0:
   version "1.4.0"
@@ -7789,7 +7980,19 @@ mime-db@1.50.0:
   resolved "https://registry.yarnpkg.com/mime-db/-/mime-db-1.50.0.tgz#abd4ac94e98d3c0e185016c67ab45d5fde40c11f"
   integrity sha512-9tMZCDlYHqeERXEHO9f/hKfNXhre5dK2eE/krIvUjZbS2KPcqGDfNShIWS1uW9XOTKQKqK6qbeOci18rbfW77A==
 
-mime-types@^2.1.12, mime-types@^2.1.16, mime-types@~2.1.19:
+mime-db@1.52.0:
+  version "1.52.0"
+  resolved "https://registry.yarnpkg.com/mime-db/-/mime-db-1.52.0.tgz#bbabcdc02859f4987301c856e3387ce5ec43bf70"
+  integrity sha512-sPU4uV7dYlvtWJxwwxHD0PuihVNiE7TyAbQ5SWxDCB9mUYvOgroQOwYQQOKPJ8CIbE+1ETVlOoK1UC2nU3gYvg==
+
+mime-types@^2.1.12, mime-types@~2.1.19:
+  version "2.1.35"
+  resolved "https://registry.yarnpkg.com/mime-types/-/mime-types-2.1.35.tgz#381a871b62a734450660ae3deee44813f70d959a"
+  integrity sha512-ZDY+bPm5zTTF+YpCrAU9nK0UgICYPT0QtT1NZWFv4s++TNkcgVaT0g6+4R2uI4MjQjzysHB1zxuWL50hzaeXiw==
+  dependencies:
+    mime-db "1.52.0"
+
+mime-types@^2.1.16:
   version "2.1.29"
   resolved "https://registry.yarnpkg.com/mime-types/-/mime-types-2.1.29.tgz#1d4ab77da64b91f5f72489df29236563754bb1b2"
   integrity sha512-Y/jMt/S5sR9OaqteJtslsFZKWOIIqMACsJSiHghlCAyhf7jfVYjKBmLiX8OgpWeW+fjJ2b+Az69aPFPkUOY6xQ==
@@ -7845,13 +8048,20 @@ minimalistic-crypto-utils@^1.0.0, minimalistic-crypto-utils@^1.0.1:
   resolved "https://registry.yarnpkg.com/minimalistic-crypto-utils/-/minimalistic-crypto-utils-1.0.1.tgz#f6c00c1c0b082246e5c4d99dfb8c7c083b2b582a"
   integrity sha1-9sAMHAsIIkblxNmd+4x8CDsrWCo=
 
-"minimatch@2 || 3", minimatch@3.0.4, minimatch@^3.0.2, minimatch@^3.0.4:
+"minimatch@2 || 3", minimatch@3.0.4:
   version "3.0.4"
   resolved "https://registry.yarnpkg.com/minimatch/-/minimatch-3.0.4.tgz#5166e286457f03306064be5497e8dbb0c3d32083"
   integrity sha512-yJHVQEhyqPLUTgt9B83PXu6W3rx4MvvHvSUvToogpwoGDOUQ+yDrR0HRot+yOCdCO7u4hX3pWft6kWBBcqh0UA==
   dependencies:
     brace-expansion "^1.1.7"
 
+minimatch@^3.0.2, minimatch@^3.0.4, minimatch@^3.1.1:
+  version "3.1.2"
+  resolved "https://registry.yarnpkg.com/minimatch/-/minimatch-3.1.2.tgz#19cd194bfd3e428f049a70817c038d89ab4be35b"
+  integrity sha512-J7p63hRiAjw1NDEww1W7i37+ByIrOWO5XQQAzZ3VOcL0PNybwpfmV/N05zFAzwQ9USyEcX6t3UO+K5aqBQOIHw==
+  dependencies:
+    brace-expansion "^1.1.7"
+
 minimist@0.0.8:
   version "0.0.8"
   resolved "https://registry.yarnpkg.com/minimist/-/minimist-0.0.8.tgz#857fcabfc3397d2625b8228262e86aa7a011b05d"
@@ -7862,10 +8072,10 @@ minimist@1.2.0:
   resolved "https://registry.yarnpkg.com/minimist/-/minimist-1.2.0.tgz#a35008b20f41383eec1fb914f4cd5df79a264284"
   integrity sha1-o1AIsg9BOD7sH7kU9M1d95omQoQ=
 
-minimist@^1.2.0, minimist@^1.2.5:
-  version "1.2.5"
-  resolved "https://registry.yarnpkg.com/minimist/-/minimist-1.2.5.tgz#67d66014b66a6a8aaa0c083c5fd58df4e4e97602"
-  integrity sha512-FM9nNUYrRBAELZQT3xeZQ7fmMOBg6nWNmJKTcgsJeaLstP/UODVpGsr5OhXhhXg6f+qtJ8uiZ+PUxkDWcgIXLw==
+minimist@^1.2.0, minimist@^1.2.5, minimist@^1.2.6:
+  version "1.2.6"
+  resolved "https://registry.yarnpkg.com/minimist/-/minimist-1.2.6.tgz#8637a5b759ea0d6e98702cfb3a9283323c93af44"
+  integrity sha512-Jsjnk4bw3YJqYzbdyBiNsPWHPfO++UGG749Cxs6peCu5Xg4nrena6OVxOYxrQTqww0Jmwt+Ref8rggumkTLz9Q==
 
 minipass@^2.6.0, minipass@^2.9.0:
   version "2.9.0"
@@ -7876,9 +8086,9 @@ minipass@^2.6.0, minipass@^2.9.0:
     yallist "^3.0.0"
 
 minipass@^3.0.0:
-  version "3.1.6"
-  resolved "https://registry.yarnpkg.com/minipass/-/minipass-3.1.6.tgz#3b8150aa688a711a1521af5e8779c1d3bb4f45ee"
-  integrity sha512-rty5kpw9/z8SX9dmxblFA6edItUmwJgMeYDZRrwlIVN27i8gysGbznJwUggw2V/FVqFSDdWy040ZPS811DYAqQ==
+  version "3.3.4"
+  resolved "https://registry.yarnpkg.com/minipass/-/minipass-3.3.4.tgz#ca99f95dd77c43c7a76bf51e6d200025eee0ffae"
+  integrity sha512-I9WPbWHCGu8W+6k1ZiGpPu0GkoKBeorkfKNuAFBNS1HNFJvke82sxvI5bzcCNpWPorkOO5QQ+zomzzwRxejXiw==
   dependencies:
     yallist "^4.0.0"
 
@@ -7916,13 +8126,20 @@ mkdirp@0.5.1:
   dependencies:
     minimist "0.0.8"
 
-mkdirp@^0.5.0, mkdirp@^0.5.1, mkdirp@^0.5.5:
+mkdirp@^0.5.0, mkdirp@^0.5.5:
   version "0.5.5"
   resolved "https://registry.yarnpkg.com/mkdirp/-/mkdirp-0.5.5.tgz#d91cefd62d1436ca0f41620e251288d420099def"
   integrity sha512-NKmAlESf6jMGym1++R0Ra7wvhV+wFW63FaSOFPwRahvea0gMUcGUhVeAg/0BC0wiv9ih5NYPB1Wn1UEI1/L+xQ==
   dependencies:
     minimist "^1.2.5"
 
+mkdirp@^0.5.1:
+  version "0.5.6"
+  resolved "https://registry.yarnpkg.com/mkdirp/-/mkdirp-0.5.6.tgz#7def03d2432dcae4ba1d611445c48396062255f6"
+  integrity sha512-FP+p8RB8OWpF3YZBCrP5gtADmtXApB5AMLn+vdyA+PyxCjrCs00mjyUozssO33cwDeT3wNGdLxJ5M//YqtHAJw==
+  dependencies:
+    minimist "^1.2.6"
+
 mocha@8.1.2:
   version "8.1.2"
   resolved "https://registry.yarnpkg.com/mocha/-/mocha-8.1.2.tgz#d67fad13300e4f5cd48135a935ea566f96caf827"
@@ -8127,18 +8344,23 @@ murmurhash3js-revisited@^3.0.0:
 murmurhash3js@^3.0.1:
   version "3.0.1"
   resolved "https://registry.yarnpkg.com/murmurhash3js/-/murmurhash3js-3.0.1.tgz#3e983e5b47c2a06f43a713174e7e435ca044b998"
-  integrity sha1-Ppg+W0fCoG9DpxMXTn5DXKBEuZg=
+  integrity sha512-KL8QYUaxq7kUbcl0Yto51rMcYt7E/4N4BG3/c96Iqw1PQrTRspu8Cpx4TZ4Nunib1d4bEkIH3gjCYlP2RLBdow==
 
 mute-stream@0.0.8:
   version "0.0.8"
   resolved "https://registry.yarnpkg.com/mute-stream/-/mute-stream-0.0.8.tgz#1630c42b2251ff81e2a283de96a5497ea92e5e0d"
   integrity sha512-nnbWWOkoWyUsTjKrhgD0dcz22mdkSnpYqbEjIm2nhwhuxlSkpywJmBo8h0ZqJdkp73mb90SssHkN4rsRaBAfAA==
 
-nan@^2.12.1, nan@^2.14.0, nan@^2.14.2:
+nan@^2.12.1:
   version "2.14.2"
   resolved "https://registry.yarnpkg.com/nan/-/nan-2.14.2.tgz#f5376400695168f4cc694ac9393d0c9585eeea19"
   integrity sha512-M2ufzIiINKCuDfBSAUr1vWQ+vuVcA9kqx8JJUsbQi6yf1uGRyb7HfpdfUr5qLXf3B/t8dPvcjhKMmlfnP47EzQ==
 
+nan@^2.14.0, nan@^2.14.2:
+  version "2.16.0"
+  resolved "https://registry.yarnpkg.com/nan/-/nan-2.16.0.tgz#664f43e45460fb98faf00edca0bb0d7b8dce7916"
+  integrity sha512-UdAqHyFngu7TfQKsCBgAA6pWDkT8MAO7d0jyOecVhN5354xbLqdn8mV9Tat9gepAupm0bt2DbeaSC8vS52MuFA==
+
 nano-json-stream-parser@^0.1.2:
   version "0.1.2"
   resolved "https://registry.yarnpkg.com/nano-json-stream-parser/-/nano-json-stream-parser-0.1.2.tgz#0cc8f6d0e2b622b479c40d499c46d64b755c6f5f"
@@ -8230,11 +8452,18 @@ node-fetch@2.6.0:
   resolved "https://registry.yarnpkg.com/node-fetch/-/node-fetch-2.6.0.tgz#e633456386d4aa55863f676a7ab0daa8fdecb0fd"
   integrity sha512-8dG4H5ujfvFiqDmVu9fQ5bOHUC15JMjMY/Zumv26oOvvVJjM67KF8koCWIabKQ1GJIa9r2mMZscBq/TbdOcmNA==
 
-node-fetch@2.6.1, node-fetch@^2.3.0:
+node-fetch@2.6.1:
   version "2.6.1"
   resolved "https://registry.yarnpkg.com/node-fetch/-/node-fetch-2.6.1.tgz#045bd323631f76ed2e2b55573394416b639a0052"
   integrity sha512-V4aYg89jEoVRxRb2fJdAg8FHvI7cEyYdVAh94HH0UIK8oJxUfkjlDQN9RbMx+bEjP7+ggMiFRprSti032Oipxw==
 
+node-fetch@^2.3.0:
+  version "2.6.7"
+  resolved "https://registry.yarnpkg.com/node-fetch/-/node-fetch-2.6.7.tgz#24de9fba827e3b4ae44dc8b20256a379160052ad"
+  integrity sha512-ZjMPFEfVx5j+y2yF35Kzx5sF7kDzxuDj6ziH4FFbOp87zKDZNx8yExJIb05OGF4Nlt9IHFIMBkRl41VdvcNdbQ==
+  dependencies:
+    whatwg-url "^5.0.0"
+
 node-fetch@^2.6.1:
   version "2.6.6"
   resolved "https://registry.yarnpkg.com/node-fetch/-/node-fetch-2.6.6.tgz#1751a7c01834e8e1697758732e9efb6eeadfaf89"
@@ -8242,15 +8471,15 @@ node-fetch@^2.6.1:
   dependencies:
     whatwg-url "^5.0.0"
 
-node-forge@~0.9.1:
-  version "0.9.2"
-  resolved "https://registry.yarnpkg.com/node-forge/-/node-forge-0.9.2.tgz#b35a44c28889b2ea55cabf8c79e3563f9676190a"
-  integrity sha512-naKSScof4Wn+aoHU6HBsifh92Zeicm1GDQKd1vp3Y/kOi8ub0DozCa9KpvYNCXslFHYRmLNiqRopGdTGwNLpNw==
+node-forge@^0.10.0:
+  version "0.10.0"
+  resolved "https://registry.yarnpkg.com/node-forge/-/node-forge-0.10.0.tgz#32dea2afb3e9926f02ee5ce8794902691a676bf3"
+  integrity sha512-PPmu8eEeG9saEUvI97fm4OYxXVB6bFvyNTyiUOBichBpFG8A1Ljw3bY62+5oOjDEMHRnd0Y7HQ+x7uzxOzC6JA==
 
 node-gyp-build@^4.2.0:
-  version "4.2.3"
-  resolved "https://registry.yarnpkg.com/node-gyp-build/-/node-gyp-build-4.2.3.tgz#ce6277f853835f718829efb47db20f3e4d9c4739"
-  integrity sha512-MN6ZpzmfNCRM+3t57PTJHgHyw/h4OWnZ6mR8P5j/uZtqQr46RRuDE/P+g3n0YR/AiYXeWixZZzaip77gdICfRg==
+  version "4.5.0"
+  resolved "https://registry.yarnpkg.com/node-gyp-build/-/node-gyp-build-4.5.0.tgz#7a64eefa0b21112f89f58379da128ac177f20e40"
+  integrity sha512-2iGbaQBV+ITgCz76ZEjmhUKAKVf7xfY1sRl4UiKQspfZMH2h06SyhNsnSVy50cwkFQDGLyif6m/6uFXHkOZ6rg==
 
 node-gyp-build@~3.8.0:
   version "3.8.0"
@@ -8298,7 +8527,7 @@ node-releases@^1.1.70:
 nodeify@^1.0.1:
   version "1.0.1"
   resolved "https://registry.yarnpkg.com/nodeify/-/nodeify-1.0.1.tgz#64ab69a7bdbaf03ce107b4f0335c87c0b9e91b1d"
-  integrity sha1-ZKtpp7268DzhB7TwM1yHwLnpGx0=
+  integrity sha512-n7C2NyEze8GCo/z73KdbjRsBiLbv6eBn1FxwYKQ23IqGo7pQY3mhQan61Sv7eEDJCiyUjTVrVkXTzJCo1dW7Aw==
   dependencies:
     is-promise "~1.0.0"
     promise "~1.3.0"
@@ -8413,7 +8642,7 @@ number-is-nan@^1.0.0:
 number-to-bn@1.7.0:
   version "1.7.0"
   resolved "https://registry.yarnpkg.com/number-to-bn/-/number-to-bn-1.7.0.tgz#bb3623592f7e5f9e0030b1977bd41a0c53fe1ea0"
-  integrity sha1-uzYjWS9+X54AMLGXe9QaDFP+HqA=
+  integrity sha512-wsJ9gfSz1/s4ZsJN01lyonwuxA1tml6X1yBDnfpMglypcBRFZZkus26EdPSlqS5GJfYddVZa22p3VNb3z5m5Ig==
   dependencies:
     bn.js "4.11.6"
     strip-hex-prefix "1.0.0"
@@ -8436,18 +8665,23 @@ object-assign@4.1.0:
 object-assign@^2.0.0:
   version "2.1.1"
   resolved "https://registry.yarnpkg.com/object-assign/-/object-assign-2.1.1.tgz#43c36e5d569ff8e4816c4efa8be02d26967c18aa"
-  integrity sha1-Q8NuXVaf+OSBbE76i+AtJpZ8GKo=
+  integrity sha512-CdsOUYIh5wIiozhJ3rLQgmUTgcyzFwZZrqhkKhODMoGtPKM+wt0h0CNIoauJWMsS9822EdzPsF/6mb4nLvPN5g==
 
 object-assign@^4, object-assign@^4.0.0, object-assign@^4.0.1, object-assign@^4.1.0, object-assign@^4.1.1:
   version "4.1.1"
   resolved "https://registry.yarnpkg.com/object-assign/-/object-assign-4.1.1.tgz#2109adc7965887cfc05cbbd442cac8bfbb360863"
   integrity sha1-IQmtx5ZYh8/AXLvUQsrIv7s2CGM=
 
-object-inspect@^1.11.0, object-inspect@^1.9.0:
+object-inspect@^1.11.0:
   version "1.11.0"
   resolved "https://registry.yarnpkg.com/object-inspect/-/object-inspect-1.11.0.tgz#9dceb146cedd4148a0d9e51ab88d34cf509922b1"
   integrity sha512-jp7ikS6Sd3GxQfZJPyH3cjcbJF6GZPClgdV+EFygjFLQ5FmW/dRUnTd9PQ9k0JhoNDabWFbpF1yCdSWCC6gexg==
 
+object-inspect@^1.9.0:
+  version "1.12.2"
+  resolved "https://registry.yarnpkg.com/object-inspect/-/object-inspect-1.12.2.tgz#c0641f26394532f28ab8d796ab954e43c009a8ea"
+  integrity sha512-z+cPxW0QGUp0mcqcsgQyLVRDoXFQbXOwBaqyF7VIgI4TWNQsDHrBpUQslRmIfAoYWdYzs6UlKJtB2XJpTaNSpQ==
+
 object-keys@^1.0.11, object-keys@^1.0.12, object-keys@^1.1.1:
   version "1.1.1"
   resolved "https://registry.yarnpkg.com/object-keys/-/object-keys-1.1.1.tgz#1c47f272df277f3b1daf061677d9c82e2322c60e"
@@ -8524,7 +8758,7 @@ on-finished@~2.3.0:
 once@^1.3.0, once@^1.3.1, once@^1.4.0:
   version "1.4.0"
   resolved "https://registry.yarnpkg.com/once/-/once-1.4.0.tgz#583b1aa775961d4b113ac17d9c50baef9dd76bd1"
-  integrity sha1-WDsap3WWHUsROsF9nFC6753Xa9E=
+  integrity sha512-lNaJgI+2Q5URQBkccEKHTQOPaXdUxnZZElQTZY0MFUAuaEqe1E+Nyvgdz/aIyNi6Z9MzO5dv1H8n58/GELp3+w==
   dependencies:
     wrappy "1"
 
@@ -8558,7 +8792,7 @@ optimism@^0.14.0:
 optimist@~0.3.5:
   version "0.3.7"
   resolved "https://registry.yarnpkg.com/optimist/-/optimist-0.3.7.tgz#c90941ad59e4273328923074d2cf2e7cbc6ec0d9"
-  integrity sha1-yQlBrVnkJzMokjB00s8ufLxuwNk=
+  integrity sha512-TCx0dXQzVtSCg2OgY/bO9hjM9cV4XYx09TVK+s3+FhkjT6LovsLe+pPMzpWf+6yXK/hUizs2gUoTw3jHM0VaTQ==
   dependencies:
     wordwrap "~0.0.2"
 
@@ -8906,7 +9140,18 @@ path-type@^4.0.0:
   resolved "https://registry.yarnpkg.com/path-type/-/path-type-4.0.0.tgz#84ed01c0a7ba380afe09d90a8c180dcd9d03043b"
   integrity sha512-gDKb8aZMDeD/tZWs9P6+q0J9Mwkdl6xMV8TjnGP3qJVJ06bdMgkbBlLU8IdfOsIsFz2BW1rNVT3XuNEl8zPAvw==
 
-pbkdf2@^3.0.17, pbkdf2@^3.0.3:
+pbkdf2@^3.0.17:
+  version "3.1.2"
+  resolved "https://registry.yarnpkg.com/pbkdf2/-/pbkdf2-3.1.2.tgz#dd822aa0887580e52f1a039dc3eda108efae3075"
+  integrity sha512-iuh7L6jA7JEGu2WxDwtQP1ddOpaJNC4KlDEFfdQajSGgGPNi4OyDc2R7QnbY2bR9QjBVGwgvTdNJZoE7RaxUMA==
+  dependencies:
+    create-hash "^1.1.2"
+    create-hmac "^1.1.4"
+    ripemd160 "^2.0.1"
+    safe-buffer "^5.0.1"
+    sha.js "^2.4.8"
+
+pbkdf2@^3.0.3:
   version "3.1.1"
   resolved "https://registry.yarnpkg.com/pbkdf2/-/pbkdf2-3.1.1.tgz#cb8724b0fada984596856d1a6ebafd3584654b94"
   integrity sha512-4Ejy1OPxi9f2tt1rRV7Go7zmfDQ+ZectEQz3VGUQhgq62HtIRPDyG/JtnwIxs6x3uNMwo2V7q1fMvKjb+Tnpqg==
@@ -8947,9 +9192,14 @@ pem-jwk@^2.0.0:
 performance-now@^2.1.0:
   version "2.1.0"
   resolved "https://registry.yarnpkg.com/performance-now/-/performance-now-2.1.0.tgz#6309f4e0e5fa913ec1c69307ae364b4b377c9e7b"
-  integrity sha1-Ywn04OX6kT7BxpMHrjZLSzd8nns=
+  integrity sha512-7EAHlyLHI56VEIdK57uwHdHKIaAGbnXPiw0yWbarQZOKaKpvUIgW0jWRVLiatnM+XXlSwsanIBH/hzGMJulMow==
+
+picomatch@^2.0.4, picomatch@^2.2.1:
+  version "2.3.1"
+  resolved "https://registry.yarnpkg.com/picomatch/-/picomatch-2.3.1.tgz#3ba3833733646d9d3e4995946c1365a67fb07a42"
+  integrity sha512-JU3teHTNjmE2VCGFzuY8EXzCDVwEqB2a8fsIvwaStHhAWJEeVd1o1QD80CU6+ZdEXXSLbSsuLwJjkCBWqRQUVA==
 
-picomatch@^2.0.4, picomatch@^2.0.5, picomatch@^2.2.1:
+picomatch@^2.0.5:
   version "2.2.2"
   resolved "https://registry.yarnpkg.com/picomatch/-/picomatch-2.2.2.tgz#21f333e9b6b8eaff02468f5146ea406d345f4dad"
   integrity sha512-q0M/9eZHzmr0AulXyPwNfZjtwZ/RBZlbN3K3CErVrk50T2ASYI7Bye0EvekFY3IP1Nt2DHu0re+V2ZHIpMkuWg==
@@ -9359,7 +9609,7 @@ promise@^8.0.0:
 promise@~1.3.0:
   version "1.3.0"
   resolved "https://registry.yarnpkg.com/promise/-/promise-1.3.0.tgz#e5cc9a4c8278e4664ffedc01c7da84842b040175"
-  integrity sha1-5cyaTIJ45GZP/twBx9qEhCsEAXU=
+  integrity sha512-R9WrbTF3EPkVtWjp7B7umQGVndpsi+rsDAfrR4xAALQpFLa/+2OriecLhawxzvii2gd9+DZFwROWDuUUaqS5yA==
   dependencies:
     is-promise "~1"
 
@@ -9378,9 +9628,9 @@ prop-types@^15.7.2:
     react-is "^16.8.1"
 
 protocol-buffers-schema@^3.3.1:
-  version "3.5.1"
-  resolved "https://registry.yarnpkg.com/protocol-buffers-schema/-/protocol-buffers-schema-3.5.1.tgz#8388e768d383ac8cbea23e1280dfadb79f4122ad"
-  integrity sha512-YVCvdhxWNDP8/nJDyXLuM+UFsuPk4+1PB7WGPVDzm3HTHbzFLxQYeW2iZpS4mmnXrQJGBzt230t/BbEb7PrQaw==
+  version "3.6.0"
+  resolved "https://registry.yarnpkg.com/protocol-buffers-schema/-/protocol-buffers-schema-3.6.0.tgz#77bc75a48b2ff142c1ad5b5b90c94cd0fa2efd03"
+  integrity sha512-TdDRD+/QNdrCGCE7v8340QyuXd4kIWIgapsE2+n/SaGiSSbomYl4TjHlvIoCWRpE7wFt02EpB35VVA2ImcBVqw==
 
 protons@^1.0.1:
   version "1.2.1"
@@ -9405,7 +9655,12 @@ prr@~1.0.1:
   resolved "https://registry.yarnpkg.com/prr/-/prr-1.0.1.tgz#d3fc114ba06995a45ec6893f484ceb1d78f5f476"
   integrity sha1-0/wRS6BplaRexok/SEzrHXj19HY=
 
-psl@^1.1.28, psl@^1.1.33:
+psl@^1.1.28:
+  version "1.9.0"
+  resolved "https://registry.yarnpkg.com/psl/-/psl-1.9.0.tgz#d0df2a137f00794565fcaf3b2c00cd09f8d5a5a7"
+  integrity sha512-E/ZsdU4HLs/68gYzgGTkMicWTLPdAftJLfJFlLUAAKZGkStNU72sZjT66SnMDVOfOWY/YAoiD7Jxa9iHvngcag==
+
+psl@^1.1.33:
   version "1.8.0"
   resolved "https://registry.yarnpkg.com/psl/-/psl-1.8.0.tgz#9326f8bcfb013adcc005fdff056acce020e51c24"
   integrity sha512-RIdOzyoavK+hA18OGGWDqUTsCLhtA7IcZ/6NCs4fFJaHBDab+pDDmDIByWFRQJq2Cd7r1OoQxBGKOaztq+hjIQ==
@@ -9475,22 +9730,17 @@ qs@6.7.0:
   resolved "https://registry.yarnpkg.com/qs/-/qs-6.7.0.tgz#41dc1a015e3d581f1621776be31afb2876a9b1bc"
   integrity sha512-VCdBRNFTX1fyE7Nb6FYoURo/SPe62QCaAyzJvUjwRaIsc+NePBEniHlvxFmmX56+HZphIGtV0XeCirBtpDrTyQ==
 
-qs@^6.4.0:
-  version "6.10.5"
-  resolved "https://registry.yarnpkg.com/qs/-/qs-6.10.5.tgz#974715920a80ff6a262264acd2c7e6c2a53282b4"
-  integrity sha512-O5RlPh0VFtR78y79rgcgKK4wbAI0C5zGVLztOIdpWX6ep368q5Hv6XRxDvXuZ9q3C6v+e3n8UfZZJw7IIG27eQ==
+qs@^6.4.0, qs@^6.5.2:
+  version "6.11.0"
+  resolved "https://registry.yarnpkg.com/qs/-/qs-6.11.0.tgz#fd0d963446f7a65e1367e01abd85429453f0c37a"
+  integrity sha512-MvjoMCJwEarSbUYk5O+nmoSzSutSsTwF85zcHPQ9OrlFoZOYIjaqBAJIqIXjptyD5vThxGq52Xu/MaJzRkIk4Q==
   dependencies:
     side-channel "^1.0.4"
 
-qs@^6.5.2:
-  version "6.9.6"
-  resolved "https://registry.yarnpkg.com/qs/-/qs-6.9.6.tgz#26ed3c8243a431b2924aca84cc90471f35d5a0ee"
-  integrity sha512-TIRk4aqYLNoJUbd+g2lEdz5kLWIuTMRagAXxl78Q0RiVjAOugHmeKNGdd3cwo/ktpf9aL9epCfFqWDEKysUlLQ==
-
 qs@~6.5.2:
-  version "6.5.2"
-  resolved "https://registry.yarnpkg.com/qs/-/qs-6.5.2.tgz#cb3ae806e8740444584ef154ce8ee98d403f3e36"
-  integrity sha512-N5ZAX4/LxJmF+7wN74pUD6qAh9/wnvdQcjq9TZjevvXzSUo7bfmw91saqMjzGS2xq91/odN2dW/WOl7qQHNDGA==
+  version "6.5.3"
+  resolved "https://registry.yarnpkg.com/qs/-/qs-6.5.3.tgz#3aeeffc91967ef6e35c0e488ef46fb296ab76aad"
+  integrity sha512-qxXIEh4pCGfHICj1mAJQ2/2XVZkjCDTcEgfoSQxc/fYivUZxTkk7L3bDBJSoNrEzXI17oUO5Dp07ktqE5KzczA==
 
 query-string@^5.0.1:
   version "5.1.1"
@@ -9524,7 +9774,7 @@ queue-microtask@^1.2.2:
 ramda@^0.24.1:
   version "0.24.1"
   resolved "https://registry.yarnpkg.com/ramda/-/ramda-0.24.1.tgz#c3b7755197f35b8dc3502228262c4c91ddb6b857"
-  integrity sha1-w7d1UZfzW43DUCIoJixMkd22uFc=
+  integrity sha512-HEm619G8PaZMfkqCa23qiOe7r3R0brPu7ZgOsgKUsnvLhd0qhc/vTjkUovomgPWa5ECBa08fJZixth9LaoBo5w==
 
 ramda@^0.25.0:
   version "0.25.0"
@@ -9764,9 +10014,9 @@ regenerator-runtime@^0.11.0:
   integrity sha512-MguG95oij0fC3QV3URf4V2SDYGJhJnJGqvIIgdECeODCT98wSWDAJ94SSuVpYQUoTcGUIL6L4yNB7j1DFFHSBg==
 
 regenerator-runtime@^0.13.4:
-  version "0.13.7"
-  resolved "https://registry.yarnpkg.com/regenerator-runtime/-/regenerator-runtime-0.13.7.tgz#cac2dacc8a1ea675feaabaeb8ae833898ae46f55"
-  integrity sha512-a54FxoJDIr27pgf7IgeQGxmqUNYrcV338lf/6gH456HZ/PhX+5BcwHXG9ajESmwe6WRO0tAzRUrRmNONWgkrew==
+  version "0.13.9"
+  resolved "https://registry.yarnpkg.com/regenerator-runtime/-/regenerator-runtime-0.13.9.tgz#8925742a98ffd90814988d7566ad30ca3b263b52"
+  integrity sha512-p3VT+cOEgxFsRRA9X4lkI1E+k2/CtnKtU4gcxyaCUreilL/vqI6CdZ3wxVUx3UOUg+gnUOQQcRI7BmSI656MYA==
 
 regex-cache@^0.4.2:
   version "0.4.4"
@@ -9996,13 +10246,20 @@ ripemd160@^2.0.0, ripemd160@^2.0.1:
     hash-base "^3.0.0"
     inherits "^2.0.1"
 
-rlp@^2.0.0, rlp@^2.2.3, rlp@^2.2.4:
+rlp@^2.0.0, rlp@^2.2.3:
   version "2.2.6"
   resolved "https://registry.yarnpkg.com/rlp/-/rlp-2.2.6.tgz#c80ba6266ac7a483ef1e69e8e2f056656de2fb2c"
   integrity sha512-HAfAmL6SDYNWPUOJNrM500x4Thn4PZsEy5pijPh40U9WfNk0z15hUYzO9xVIMAdIHdFtD8CBDHd75Td1g36Mjg==
   dependencies:
     bn.js "^4.11.1"
 
+rlp@^2.2.4:
+  version "2.2.7"
+  resolved "https://registry.yarnpkg.com/rlp/-/rlp-2.2.7.tgz#33f31c4afac81124ac4b283e2bd4d9720b30beaf"
+  integrity sha512-d5gdPmgQ0Z+AklL2NVXr/IoSjNZFfTVvQWzL/AM2AOcSzYP2xjlb0AC8YyCLc41MSNf6P6QVtjgPdmVtzb+4lQ==
+  dependencies:
+    bn.js "^5.2.0"
+
 rn-host-detect@^1.1.5:
   version "1.2.0"
   resolved "https://registry.yarnpkg.com/rn-host-detect/-/rn-host-detect-1.2.0.tgz#8b0396fc05631ec60c1cb8789e5070cdb04d0da0"
@@ -10011,7 +10268,7 @@ rn-host-detect@^1.1.5:
 rsa-pem-to-jwk@^1.1.3:
   version "1.1.3"
   resolved "https://registry.yarnpkg.com/rsa-pem-to-jwk/-/rsa-pem-to-jwk-1.1.3.tgz#245e76bdb7e7234cfee7ca032d31b54c38fab98e"
-  integrity sha1-JF52vbfnI0z+58oDLTG1TDj6uY4=
+  integrity sha512-ZlVavEvTnD8Rzh/pdB8NH4VF5GNEtF6biGQcTtC4GKFMsbZR08oHtOYefbhCN+JnJIuMItiCDCMycdcMrw6blA==
   dependencies:
     object-assign "^2.0.0"
     rsa-unpack "0.0.6"
@@ -10019,7 +10276,7 @@ rsa-pem-to-jwk@^1.1.3:
 rsa-unpack@0.0.6:
   version "0.0.6"
   resolved "https://registry.yarnpkg.com/rsa-unpack/-/rsa-unpack-0.0.6.tgz#f50ebd56a628378e631f297161026ce9ab4eddba"
-  integrity sha1-9Q69VqYoN45jHylxYQJs6atO3bo=
+  integrity sha512-HRrl8GHjjPziPFRDJPq/v5OxZ3IPdksV5h3cime/oHgcgM1k1toO5OdtzClgBqRf5dF6IgptOB0g/zFb0w5zQw==
   dependencies:
     optimist "~0.3.5"
 
@@ -10104,11 +10361,11 @@ secp256k1@^3.6.2:
     safe-buffer "^5.1.2"
 
 secp256k1@^4.0.1:
-  version "4.0.2"
-  resolved "https://registry.yarnpkg.com/secp256k1/-/secp256k1-4.0.2.tgz#15dd57d0f0b9fdb54ac1fa1694f40e5e9a54f4a1"
-  integrity sha512-UDar4sKvWAksIlfX3xIaQReADn+WFnHvbVujpcbr+9Sf/69odMwy2MUsz5CKLQgX9nsIyrjuxL2imVyoNHa3fg==
+  version "4.0.3"
+  resolved "https://registry.yarnpkg.com/secp256k1/-/secp256k1-4.0.3.tgz#c4559ecd1b8d3c1827ed2d1b94190d69ce267303"
+  integrity sha512-NLZVf+ROMxwtEj3Xa562qgv2BK5e2WNmXPiOdVIPLgs6lyTzMvBq0aWTYMI5XCP9jZMVKOcqZLw/Wc4vDkuxhA==
   dependencies:
-    elliptic "^6.5.2"
+    elliptic "^6.5.4"
     node-addon-api "^2.0.0"
     node-gyp-build "^4.2.0"
 
@@ -10139,7 +10396,14 @@ semver@^6.1.1, semver@^6.1.2, semver@^6.3.0:
   resolved "https://registry.yarnpkg.com/semver/-/semver-6.3.0.tgz#ee0a64c8af5e8ceea67687b133761e1becbd1d3d"
   integrity sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw==
 
-semver@^7.0.0, semver@^7.3.4:
+semver@^7.0.0:
+  version "7.3.7"
+  resolved "https://registry.yarnpkg.com/semver/-/semver-7.3.7.tgz#12c5b649afdbf9049707796e22a4028814ce523f"
+  integrity sha512-QlYTucUYOews+WeEujDoEGziz4K6c47V/Bd+LjSSYcA94p+DmINdf7ncaUinThfvZyu13lN9OY1XDxt8C0Tw0g==
+  dependencies:
+    lru-cache "^6.0.0"
+
+semver@^7.3.4:
   version "7.3.4"
   resolved "https://registry.yarnpkg.com/semver/-/semver-7.3.4.tgz#27aaa7d2e4ca76452f98d3add093a72c943edc97"
   integrity sha512-tCfb2WLjqFAtXn4KEdxIhalnRtoKFN7nAwj0B3ZXCbQloV2tq5eDbcTmT68JJD3nRJq24/XgxtQKFIpQdtvmVw==
@@ -10224,7 +10488,7 @@ setimmediate@1.0.4:
 setimmediate@^1.0.5:
   version "1.0.5"
   resolved "https://registry.yarnpkg.com/setimmediate/-/setimmediate-1.0.5.tgz#290cbb232e306942d7d7ea9b83732ab7856f8285"
-  integrity sha1-KQy7Iy4waULX1+qbg3Mqt4VvgoU=
+  integrity sha512-MATJdZp8sLqDl/68LfQmbP8zKPLQNV6BIZoIgrscFDQ+RsvK/BxeDQOgyxKKoh0y/8h3BqVFnCqQ/gd+reiIXA==
 
 setprototypeof@1.1.1:
   version "1.1.1"
@@ -10270,15 +10534,20 @@ side-channel@^1.0.4:
     get-intrinsic "^1.0.2"
     object-inspect "^1.9.0"
 
-signal-exit@^3.0.0, signal-exit@^3.0.2:
+signal-exit@^3.0.0:
   version "3.0.3"
   resolved "https://registry.yarnpkg.com/signal-exit/-/signal-exit-3.0.3.tgz#a1410c2edd8f077b08b4e253c8eacfcaf057461c"
   integrity sha512-VUJ49FC8U1OxwZLxIbTTrDvLnf/6TDgxZcK8wxR8zs13xpx7xbG60ndBlhNrFi2EMuFRoeDoJO7wthSLq42EjA==
 
+signal-exit@^3.0.2:
+  version "3.0.7"
+  resolved "https://registry.yarnpkg.com/signal-exit/-/signal-exit-3.0.7.tgz#a9a1767f8af84155114eaabd73f99273c8f59ad9"
+  integrity sha512-wnD2ZE+l+SPC/uoS0vXeE9L1+0wuaMqKlfz9AMUo38JsyLSBWSFcHR1Rri62LZc12vLr1gb3jl7iwQhgwpAbGQ==
+
 signed-varint@^2.0.1:
   version "2.0.1"
   resolved "https://registry.yarnpkg.com/signed-varint/-/signed-varint-2.0.1.tgz#50a9989da7c98c2c61dad119bc97470ef8528129"
-  integrity sha1-UKmYnafJjCxh2tEZvJdHDvhSgSk=
+  integrity sha512-abgDPg1106vuZZOvw7cFwdCABddfJRz5akcCcchzTbhyhYnsG31y4AlZEgp315T7W3nQq5P4xeOm186ZiPVFzw==
   dependencies:
     varint "~5.0.0"
 
@@ -10429,7 +10698,7 @@ spdx-license-ids@^3.0.0:
 split-ca@^1.0.0:
   version "1.0.1"
   resolved "https://registry.yarnpkg.com/split-ca/-/split-ca-1.0.1.tgz#6c83aff3692fa61256e0cd197e05e9de157691a6"
-  integrity sha1-bIOv82kvphJW4M0ZfgXp3hV2kaY=
+  integrity sha512-Q5thBSxp5t8WPTTJQS59LrGqOZqOsrhDGDVm8azCqIBjSBd7nd9o2PM+mDulQQkh8h//4U6hFZnc/mul8t5pWQ==
 
 split2@^3.1.0:
   version "3.2.2"
@@ -10441,7 +10710,7 @@ split2@^3.1.0:
 sprintf-js@~1.0.2:
   version "1.0.3"
   resolved "https://registry.yarnpkg.com/sprintf-js/-/sprintf-js-1.0.3.tgz#04e6926f662895354f3dd015203633b857297e2c"
-  integrity sha1-BOaSb2YolTVPPdAVIDYzuFcpfiw=
+  integrity sha512-D9cPgkvLlV3t3IzL0D0YLvGA9Ahk4PcvVwUbN0dSGr1aP0Nrt4AEnTUbuGvquEC0mA64Gqt1fzirlRs5ibXx8g==
 
 sqlite3@^4.0.0:
   version "4.2.0"
@@ -10457,9 +10726,9 @@ sse-z@0.3.0:
   integrity sha512-jfcXynl9oAOS9YJ7iqS2JMUEHOlvrRAD+54CENiWnc4xsuVLQVSgmwf7cwOTcBd/uq3XkQKBGojgvEtVXcJ/8w==
 
 sshpk@^1.7.0:
-  version "1.16.1"
-  resolved "https://registry.yarnpkg.com/sshpk/-/sshpk-1.16.1.tgz#fb661c0bef29b39db40769ee39fa70093d6f6877"
-  integrity sha512-HXXqVUq7+pcKeLqqZj6mHFUMvXtOJt1uoUx09pFW6011inTMxqI8BA8PM95myrIyyKwdnzjdFjLiE6KBPVtJIg==
+  version "1.17.0"
+  resolved "https://registry.yarnpkg.com/sshpk/-/sshpk-1.17.0.tgz#578082d92d4fe612b13007496e543fa0fbcbe4c5"
+  integrity sha512-/9HIEs1ZXGhSPE8X6Ccm7Nam1z8KcoCqPdI7ecm1N33EzAetWahvQWVqLZtaZQ+IDKX4IyA2o0gBzqIMkAagHQ==
   dependencies:
     asn1 "~0.2.3"
     assert-plus "^1.0.0"
@@ -10570,7 +10839,7 @@ string_decoder@^1.1.1, string_decoder@^1.2.0:
 string_decoder@~0.10.x:
   version "0.10.31"
   resolved "https://registry.yarnpkg.com/string_decoder/-/string_decoder-0.10.31.tgz#62e203bc41766c6c28c9fc84301dab1c5310fa94"
-  integrity sha1-YuIDvEF2bGwoyfyEMB2rHFMQ+pQ=
+  integrity sha512-ev2QzSzWPYmy9GuqfIVildA4OdcGLeFZQrq5ys6RtiuF+RQQiZWr8TZNyAcuVXyQRYfEO+MsoB/1BuQVhOJuoQ==
 
 string_decoder@~1.1.1:
   version "1.1.1"
@@ -10589,7 +10858,7 @@ strip-ansi@^3.0.0, strip-ansi@^3.0.1:
 strip-ansi@^4.0.0:
   version "4.0.0"
   resolved "https://registry.yarnpkg.com/strip-ansi/-/strip-ansi-4.0.0.tgz#a8479022eb1ac368a871389b635262c505ee368f"
-  integrity sha1-qEeQIusaw2iocTibY1JixQXuNo8=
+  integrity sha512-4XaJ2zQdCzROZDivEVIDPkcQn8LMFSa8kj8Gxb/Lnwzv9A8VctNZ+lfivC/sV3ivW8ElJTERXZoPBRrZKkNKow==
   dependencies:
     ansi-regex "^3.0.0"
 
@@ -10601,11 +10870,11 @@ strip-ansi@^5.0.0, strip-ansi@^5.1.0, strip-ansi@^5.2.0:
     ansi-regex "^4.1.0"
 
 strip-ansi@^6.0.0:
-  version "6.0.0"
-  resolved "https://registry.yarnpkg.com/strip-ansi/-/strip-ansi-6.0.0.tgz#0b1571dd7669ccd4f3e06e14ef1eed26225ae532"
-  integrity sha512-AuvKTrTfQNYNIctbR1K/YGTR1756GycPsg7b9bdV9Duqur4gv6aKqHXah67Z8ImS7WEz5QVcOtlfW2rZEugt6w==
+  version "6.0.1"
+  resolved "https://registry.yarnpkg.com/strip-ansi/-/strip-ansi-6.0.1.tgz#9e26c63d30f53443e9489495b2105d37b67a85d9"
+  integrity sha512-Y38VPSHcqkFrCpFnQ9vuSXmquuv5oXOKpGeT6aGrr3o3Gc9AlVa6JBfUSOCnbxGGZF+/0ooI7KrPuUSztUdU5A==
   dependencies:
-    ansi-regex "^5.0.0"
+    ansi-regex "^5.0.1"
 
 strip-bom-stream@^1.0.0:
   version "1.0.0"
@@ -10630,7 +10899,7 @@ strip-final-newline@^2.0.0:
 strip-hex-prefix@1.0.0:
   version "1.0.0"
   resolved "https://registry.yarnpkg.com/strip-hex-prefix/-/strip-hex-prefix-1.0.0.tgz#0c5f155fef1151373377de9dbb588da05500e36f"
-  integrity sha1-DF8VX+8RUTczd96du1iNoFUA428=
+  integrity sha512-q8d4ue7JGEiVcypji1bALTos+0pWtyGlivAWyPuTkHzuTCJqrK9sWxYQZUq6Nq3cuyv3bm734IhHvHtGGURU6A==
   dependencies:
     is-hex-prefixed "1.0.0"
 
@@ -10901,7 +11170,7 @@ through2@^0.6.0:
 "through@>=2.2.7 <3":
   version "2.3.8"
   resolved "https://registry.yarnpkg.com/through/-/through-2.3.8.tgz#0dd4c9ffaabc357960b1b724115d7e0e86a2e1f5"
-  integrity sha1-DdTJ/6q8NXlgsbckEV1+Doai4fU=
+  integrity sha512-w89qg7PI8wAdvX60bMDP+bFoD5Dvhm9oLheFp5O4a2QF0cSBGsBX4qZmadPMvVqlLJBBci+WqGGOAPvcDeNSVg==
 
 tildify@1.2.0:
   version "1.2.0"
@@ -11066,14 +11335,14 @@ tslib@~2.0.1:
 tunnel-agent@^0.6.0:
   version "0.6.0"
   resolved "https://registry.yarnpkg.com/tunnel-agent/-/tunnel-agent-0.6.0.tgz#27a5dea06b36b04a0a9966774b290868f0fc40fd"
-  integrity sha1-J6XeoGs2sEoKmWZ3SykIaPD8QP0=
+  integrity sha512-McnNiV1l8RYeY8tBgEpuodCC1mLUdbSN+CYBL7kJsJNInOP8UjDDEwdk6Mw60vdLLrr5NHKZhMAOSrR2NZuQ+w==
   dependencies:
     safe-buffer "^5.0.1"
 
 tweetnacl@^0.14.3, tweetnacl@~0.14.0:
   version "0.14.5"
   resolved "https://registry.yarnpkg.com/tweetnacl/-/tweetnacl-0.14.5.tgz#5ae68177f192d4456269d108afa93ff8743f4f64"
-  integrity sha1-WuaBd/GS1EViadEIr6k/+HQ/T2Q=
+  integrity sha512-KXXFFdAbFXY4geFIwoyNK+f5Z1b7swfXABfL7HXCmoIWMKU3dmS26672A4EeQtDzLKy7SXmfBu51JolvEKwtGA==
 
 tweetnacl@^1.0.0:
   version "1.0.3"
@@ -11179,7 +11448,7 @@ underscore@^1.8.3:
 unique-by@^1.0.0:
   version "1.0.0"
   resolved "https://registry.yarnpkg.com/unique-by/-/unique-by-1.0.0.tgz#5220c86ba7bc572fb713ad74651470cb644212bd"
-  integrity sha1-UiDIa6e8Vy+3E610ZRRwy2RCEr0=
+  integrity sha512-rJRXK5V0zL6TiSzhoGNpJp5dr+TZBLoPJFC06rLn17Ug++7Aa0Qnve5v+skXeQxx6/sI7rBsSesa6MAcmFi8Ew==
 
 unique-stream@^2.0.2:
   version "2.3.1"
@@ -11302,7 +11571,7 @@ utf8@3.0.0, utf8@^3.0.0:
 util-deprecate@^1.0.1, util-deprecate@~1.0.1:
   version "1.0.2"
   resolved "https://registry.yarnpkg.com/util-deprecate/-/util-deprecate-1.0.2.tgz#450d4dc9fa70de732762fbd2d4a28981419a0ccf"
-  integrity sha1-RQ1Nyfpw3nMnYvvS1KKJgUGaDM8=
+  integrity sha512-EPD5q1uXyFxJpCrLnCc1nHnq3gOa6DZBocAIiI2TaSCA7VCJ1UJDMagCzIkXNsUYfD1daK//LTEQ8xiIbrHtcw==
 
 util.promisify@^1.0.0:
   version "1.1.1"
@@ -11393,7 +11662,7 @@ vary@^1, vary@~1.1.2:
 verror@1.10.0:
   version "1.10.0"
   resolved "https://registry.yarnpkg.com/verror/-/verror-1.10.0.tgz#3a105ca17053af55d6e270c1f8288682e18da400"
-  integrity sha1-OhBcoXBTr1XW4nDB+CiGguGNpAA=
+  integrity sha512-ZZKSmDAEFOijERBLkmYfJ+vmk3w+7hOLYDNkRCuRuMJGEmqYNCNLyBBFwWKVMhfwaEF3WOd0Zlw86U/WC/+nYw==
   dependencies:
     assert-plus "^1.0.0"
     core-util-is "1.0.2"
@@ -11439,7 +11708,7 @@ vuvuzela@1.0.3:
 wcwidth@^1.0.1:
   version "1.0.1"
   resolved "https://registry.yarnpkg.com/wcwidth/-/wcwidth-1.0.1.tgz#f0b0dcf915bc5ff1528afadb2c0e17b532da2fe8"
-  integrity sha1-8LDc+RW8X/FSivrbLA4XtTLaL+g=
+  integrity sha512-XHPEwS0q6TaxcvG85+8EYkbiCux2XtWG2mkc47Ng2A77BQu9+DqIOJldST4HgPkuea7dvKSj5VgX3P1d4rW8Tg==
   dependencies:
     defaults "^1.0.3"
 
@@ -12057,7 +12326,7 @@ word-wrap@~1.2.3:
 wordwrap@~0.0.2:
   version "0.0.3"
   resolved "https://registry.yarnpkg.com/wordwrap/-/wordwrap-0.0.3.tgz#a3d5da6cd5c0bc0008d37234bbaf1bed63059107"
-  integrity sha1-o9XabNXAvAAI03I0u68b7WMFkQc=
+  integrity sha512-1tMA907+V4QmxV7dbRvb4/8MaRALK6q9Abid3ndMYnbyo8piisCmeONVqVSXqQA3KaP4SLt5b7ud6E2sqP8TFw==
 
 workerpool@6.0.0:
   version "6.0.0"
@@ -12093,7 +12362,7 @@ wrap-ansi@^6.2.0:
 wrappy@1:
   version "1.0.2"
   resolved "https://registry.yarnpkg.com/wrappy/-/wrappy-1.0.2.tgz#b5243d8f3ec1aa35f1364605bc0d1036e30ab69f"
-  integrity sha1-tSQ9jz7BqjXxNkYFvA0QNuMKtp8=
+  integrity sha512-l4Sp/DRseor9wL6EvV2+TuQn63dMkPjZ/sp9XkghTEbV9KlPS1xUsZ3u7/IQO4wxtcFB4bgpQPRcR3QCvezPcQ==
 
 write-file-atomic@^2.0.0:
   version "2.4.3"
@@ -12145,9 +12414,9 @@ ws@^5.1.1:
   integrity sha512-BAkMFcAzl8as1G/hArkxOxq3G7pjUqQ3gzYbLL0/5zNkph70e+lCoxBGnm6AW1+/aiNeV4fnKqZ8m4GZewmH2w==
 
 ws@^7.4.5:
-  version "7.5.8"
-  resolved "https://registry.yarnpkg.com/ws/-/ws-7.5.8.tgz#ac2729881ab9e7cbaf8787fe3469a48c5c7f636a"
-  integrity sha512-ri1Id1WinAX5Jqn9HejiGb8crfRio0Qgu8+MtL36rlTA6RLsMdWt1Az/19A2Qij6uSHUMphEFaTKa4WG+UNHNw==
+  version "7.5.9"
+  resolved "https://registry.yarnpkg.com/ws/-/ws-7.5.9.tgz#54fa7db29f4c7cec68b1ddd3a89de099942bb591"
+  integrity sha512-F+P9Jil7UiSKSkppIiD94dN07AwvFixvLIj1Og1Rl9GGMuNipJnV9JzjD6XuqmAeiswGvUmNLjr5cFuXwNS77Q==
 
 xdg-basedir@^3.0.0:
   version "3.0.0"
@@ -12253,11 +12522,16 @@ yaml@1.9.2:
   dependencies:
     "@babel/runtime" "^7.9.2"
 
-yaml@^1.5.1, yaml@^1.7.2:
+yaml@^1.5.1:
   version "1.10.0"
   resolved "https://registry.yarnpkg.com/yaml/-/yaml-1.10.0.tgz#3b593add944876077d4d683fee01081bd9fff31e"
   integrity sha512-yr2icI4glYaNG+KWONODapy2/jDdMSDnrONSjblABjD9B4Z5LgiircSt8m8sRZFNi08kG9Sm0uSHtEmP3zaEGg==
 
+yaml@^1.7.2:
+  version "1.10.2"
+  resolved "https://registry.yarnpkg.com/yaml/-/yaml-1.10.2.tgz#2301c5ffbf12b467de8da2333a459e29e7920e4b"
+  integrity sha512-r3vXyErRCYJ7wg28yvBY5VSoAF8ZvlcW9/BwUzEtUsjvX/DKs24dIkuwjtuprwJJHsbyUbLApepYTR1BN4uHrg==
+
 yargs-parser@13.1.2, yargs-parser@^13.1.2:
   version "13.1.2"
   resolved "https://registry.yarnpkg.com/yargs-parser/-/yargs-parser-13.1.2.tgz#130f09702ebaeef2650d54ce6e3e5706f7a4fb38"

From 149a14d430d9c8659d0d0eaceade71ee62a2694f Mon Sep 17 00:00:00 2001
From: Leonardo Yvens <leoyvens@gmail.com>
Date: Tue, 23 Aug 2022 21:07:39 +0100
Subject: [PATCH 33/33] fix: Update comments

---
 store/postgres/src/dynds/private.rs | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/store/postgres/src/dynds/private.rs b/store/postgres/src/dynds/private.rs
index 476846534f8..78710db066a 100644
--- a/store/postgres/src/dynds/private.rs
+++ b/store/postgres/src/dynds/private.rs
@@ -157,8 +157,8 @@ impl DataSourcesTable {
                 ));
             }
 
-            // Onchain data sources have the causality region explicitly set to 0, while offchain
-            // data sources have an unique causality region assigned from the sequence.
+            // Offchain data sources have a unique causality region assigned from a sequence in the
+            // database, while onchain data sources always have causality region 0.
             let query = match is_offchain {
                 false => format!(
                     "insert into {}(block_range, manifest_idx, param, context, causality_region) \
@@ -282,7 +282,7 @@ impl DataSourcesTable {
                 // Data source deduplication enforces this invariant.
                 // See also: data-source-is-duplicate-of
                 return Err(constraint_violation!(
-                    "expected to remove at most one offchain data source but removed {}, ds: {:?}",
+                    "expected to remove at most one offchain data source but would remove {}, ds: {:?}",
                     count,
                     ds
                 ));