From 973efd6c0a26685ac533af1e84d3b369a64dfd3e Mon Sep 17 00:00:00 2001 From: Morgan McCauley Date: Thu, 20 Jun 2024 08:35:21 +1200 Subject: [PATCH 01/10] refactor: Move Data Layer provisioning from Runner to Coordinator (#805) This PR updates Coordinator to handle Data Layer provisioning, removing the implicit step from Runner. Provisioning itself is still completed within Runner, but Coordinator will trigger and monitor it. Functionally, provisioning is the same, but there are some subtle differences around how it is handled: - Provisioning will now happen as soon as the Indexer is registered, rather than when the first matched block is executed. - Block Streams/Executors will not be started until provisioning is successful, neither will start when either pending or failed. A `provisioned_state` enum has been added to the Indexer state within Redis. This is used to persist what stage of provisioning the Data Layer is at, as well as ensuring we only provision once. ## Concerns with current implementation Overall, I'm not happy with the implementation here, but feel it is the best we can do given the current structure of Coordinator. As to not block this feature I decided to go forward with this approach, and will create a ticket to refactor/update later. Provisioning is triggered within the "new" handler, and then polled within the "existing" handler, which seems a little awkward. The separated handling is necessary since no operation within the control loop (i.e. `Synchroniser`) should block, as that would block synchronisation for all other Indexers. So we need to trigger the provisioning initially, and then poll the completion each subsequent control loop. I feel we have outgrown the current control loop, and am planning to refactor later. Rather than have a single control loop for _all_ Indexers, I'm thinking we can have dedicated loops for each of them. We could spawn a new task for each Indexer, which then manages its own lifecycle. Then each Indexer is free to wait for as long as it wants, without impacting other Indexers. This would allow us to handle the blocking provisioning step much more elegantly. --- .../block_streams.rs} | 0 coordinator/src/handlers/data_layer.rs | 96 ++++++ .../executors.rs} | 0 coordinator/src/handlers/mod.rs | 3 + coordinator/src/indexer_state.rs | 188 ++++------- coordinator/src/main.rs | 13 +- coordinator/src/synchroniser.rs | 312 +++++++++++------- runner-client/build.rs | 1 + .../check_provisioning_task_status.rs | 20 ++ .../examples/start_provisioning_task.rs | 21 ++ runner-client/proto/data-layer.proto | 33 ++ runner-client/src/lib.rs | 4 + runner/protos/data-layer.proto | 4 +- .../__snapshots__/indexer.test.ts.snap | 2 - runner/src/indexer/indexer.test.ts | 137 -------- runner/src/indexer/indexer.ts | 15 - .../data-layer/data-layer-service.test.ts | 18 +- .../services/data-layer/data-layer-service.ts | 10 +- runner/tests/integration.test.ts | 6 +- 19 files changed, 461 insertions(+), 422 deletions(-) rename coordinator/src/{block_streams_handler.rs => handlers/block_streams.rs} (100%) create mode 100644 coordinator/src/handlers/data_layer.rs rename coordinator/src/{executors_handler.rs => handlers/executors.rs} (100%) create mode 100644 coordinator/src/handlers/mod.rs create mode 100644 runner-client/examples/check_provisioning_task_status.rs create mode 100644 runner-client/examples/start_provisioning_task.rs create mode 100644 runner-client/proto/data-layer.proto diff --git a/coordinator/src/block_streams_handler.rs b/coordinator/src/handlers/block_streams.rs similarity index 100% rename from coordinator/src/block_streams_handler.rs rename to coordinator/src/handlers/block_streams.rs diff --git a/coordinator/src/handlers/data_layer.rs b/coordinator/src/handlers/data_layer.rs new file mode 100644 index 000000000..7db624648 --- /dev/null +++ b/coordinator/src/handlers/data_layer.rs @@ -0,0 +1,96 @@ +#![cfg_attr(test, allow(dead_code))] + +pub use runner::data_layer::ProvisioningStatus; + +use anyhow::Context; +use runner::data_layer::data_layer_client::DataLayerClient; +use runner::data_layer::{CheckProvisioningTaskStatusRequest, ProvisionRequest}; +use tonic::transport::channel::Channel; +use tonic::{Request, Status}; + +use crate::indexer_config::IndexerConfig; + +#[cfg(not(test))] +pub use DataLayerHandlerImpl as DataLayerHandler; +#[cfg(test)] +pub use MockDataLayerHandlerImpl as DataLayerHandler; + +pub struct DataLayerHandlerImpl { + client: DataLayerClient, +} + +#[cfg_attr(test, mockall::automock)] +impl DataLayerHandlerImpl { + pub fn from_env() -> anyhow::Result { + let runner_url = std::env::var("RUNNER_URL").context("RUNNER_URL is not set")?; + Self::connect(&runner_url) + } + + pub fn connect(runner_url: &str) -> anyhow::Result { + let channel = Channel::from_shared(runner_url.to_string()) + .context("Runner URL is invalid")? + .connect_lazy(); + let client = DataLayerClient::new(channel); + + Ok(Self { client }) + } + + pub async fn start_provisioning_task( + &self, + indexer_config: &IndexerConfig, + ) -> anyhow::Result { + let request = ProvisionRequest { + account_id: indexer_config.account_id.to_string(), + function_name: indexer_config.function_name.clone(), + schema: indexer_config.schema.clone(), + }; + + let response = self + .client + .clone() + .start_provisioning_task(Request::new(request)) + .await; + + if let Err(error) = response { + if error.code() == tonic::Code::AlreadyExists { + return Ok(ProvisioningStatus::Pending); + } + + return Err(error.into()); + } + + let status = match response.unwrap().into_inner().status { + 1 => ProvisioningStatus::Pending, + 2 => ProvisioningStatus::Complete, + 3 => ProvisioningStatus::Failed, + _ => ProvisioningStatus::Unspecified, + }; + + Ok(status) + } + + pub async fn check_provisioning_task_status( + &self, + indexer_config: &IndexerConfig, + ) -> anyhow::Result { + let request = CheckProvisioningTaskStatusRequest { + account_id: indexer_config.account_id.to_string(), + function_name: indexer_config.function_name.clone(), + }; + + let response = self + .client + .clone() + .check_provisioning_task_status(Request::new(request)) + .await?; + + let status = match response.into_inner().status { + 1 => ProvisioningStatus::Pending, + 2 => ProvisioningStatus::Complete, + 3 => ProvisioningStatus::Failed, + _ => ProvisioningStatus::Unspecified, + }; + + Ok(status) + } +} diff --git a/coordinator/src/executors_handler.rs b/coordinator/src/handlers/executors.rs similarity index 100% rename from coordinator/src/executors_handler.rs rename to coordinator/src/handlers/executors.rs diff --git a/coordinator/src/handlers/mod.rs b/coordinator/src/handlers/mod.rs new file mode 100644 index 000000000..ecabefea7 --- /dev/null +++ b/coordinator/src/handlers/mod.rs @@ -0,0 +1,3 @@ +pub mod block_streams; +pub mod data_layer; +pub mod executors; diff --git a/coordinator/src/indexer_state.rs b/coordinator/src/indexer_state.rs index 431fb3345..ce100354c 100644 --- a/coordinator/src/indexer_state.rs +++ b/coordinator/src/indexer_state.rs @@ -2,14 +2,22 @@ use anyhow::Context; use near_primitives::types::AccountId; -use std::str::FromStr; use crate::indexer_config::IndexerConfig; use crate::redis::RedisClient; -use crate::registry::IndexerRegistry; -#[derive(Debug, Clone, serde::Serialize, serde::Deserialize)] +#[derive(Debug, Clone, serde::Serialize, serde::Deserialize, PartialEq, Eq)] +pub enum ProvisionedState { + Unprovisioned, + Provisioning, + Provisioned, + Failed, +} + +#[derive(Debug, Clone, serde::Serialize, serde::Deserialize, PartialEq, Eq)] pub struct OldIndexerState { + pub account_id: AccountId, + pub function_name: String, pub block_stream_synced_at: Option, pub enabled: bool, } @@ -20,6 +28,7 @@ pub struct IndexerState { pub function_name: String, pub block_stream_synced_at: Option, pub enabled: bool, + pub provisioned_state: ProvisionedState, } impl IndexerState { @@ -31,17 +40,6 @@ impl IndexerState { } } -impl Default for IndexerState { - fn default() -> Self { - Self { - account_id: AccountId::from_str("morgs.near").unwrap(), - function_name: String::new(), - block_stream_synced_at: None, - enabled: true, - } - } -} - #[cfg(not(test))] pub use IndexerStateManagerImpl as IndexerStateManager; #[cfg(test)] @@ -57,41 +55,36 @@ impl IndexerStateManagerImpl { Self { redis_client } } - pub async fn migrate(&self, registry: &IndexerRegistry) -> anyhow::Result<()> { - if self.redis_client.indexer_states_set_exists().await? { - return Ok(()); - } + pub async fn migrate(&self) -> anyhow::Result<()> { + let raw_states = self.redis_client.list_indexer_states().await?; + + for raw_state in raw_states { + if let Ok(state) = serde_json::from_str::(&raw_state) { + tracing::info!( + "{}/{} already migrated, skipping", + state.account_id, + state.function_name + ); + continue; + } + + tracing::info!("Migrating {}", raw_state); - tracing::info!("Migrating indexer state"); - - for config in registry.iter() { - let raw_state = self.redis_client.get_indexer_state(config).await?; - - let state = if let Some(raw_state) = raw_state { - let old_state: OldIndexerState = - serde_json::from_str(&raw_state).context(format!( - "Failed to deserialize OldIndexerState for {}", - config.get_full_name() - ))?; - - IndexerState { - account_id: config.account_id.clone(), - function_name: config.function_name.clone(), - block_stream_synced_at: old_state.block_stream_synced_at, - enabled: old_state.enabled, - } - } else { - self.get_default_state(config) + let old_state: IndexerState = serde_json::from_str(&raw_state)?; + + let state = IndexerState { + account_id: old_state.account_id, + function_name: old_state.function_name, + block_stream_synced_at: old_state.block_stream_synced_at, + enabled: old_state.enabled, + provisioned_state: ProvisionedState::Provisioned, }; - self.set_state(config, state).await.context(format!( - "Failed to set state for {}", - config.get_full_name() - ))?; + self.redis_client + .set(state.get_state_key(), serde_json::to_string(&state)?) + .await?; } - tracing::info!("Migration complete"); - Ok(()) } @@ -101,6 +94,7 @@ impl IndexerStateManagerImpl { function_name: indexer_config.function_name.clone(), block_stream_synced_at: None, enabled: true, + provisioned_state: ProvisionedState::Unprovisioned, } } @@ -140,6 +134,38 @@ impl IndexerStateManagerImpl { Ok(()) } + pub async fn set_provisioning(&self, indexer_config: &IndexerConfig) -> anyhow::Result<()> { + let mut indexer_state = self.get_state(indexer_config).await?; + + indexer_state.provisioned_state = ProvisionedState::Provisioning; + + self.set_state(indexer_config, indexer_state).await?; + + Ok(()) + } + + pub async fn set_provisioned(&self, indexer_config: &IndexerConfig) -> anyhow::Result<()> { + let mut indexer_state = self.get_state(indexer_config).await?; + + indexer_state.provisioned_state = ProvisionedState::Provisioned; + + self.set_state(indexer_config, indexer_state).await?; + + Ok(()) + } + pub async fn set_provisioning_failure( + &self, + indexer_config: &IndexerConfig, + ) -> anyhow::Result<()> { + let mut indexer_state = self.get_state(indexer_config).await?; + + indexer_state.provisioned_state = ProvisionedState::Failed; + + self.set_state(indexer_config, indexer_state).await?; + + Ok(()) + } + pub async fn set_enabled( &self, indexer_config: &IndexerConfig, @@ -173,8 +199,6 @@ impl IndexerStateManagerImpl { mod tests { use super::*; - use std::collections::HashMap; - use mockall::predicate; use registry_types::{Rule, StartBlock, Status}; @@ -183,7 +207,7 @@ mod tests { let mut mock_redis_client = RedisClient::default(); mock_redis_client .expect_list_indexer_states() - .returning(|| Ok(vec![serde_json::json!({ "account_id": "morgs.near", "function_name": "test", "block_stream_synced_at": 200, "enabled": true }).to_string()])) + .returning(|| Ok(vec![serde_json::json!({ "account_id": "morgs.near", "function_name": "test", "block_stream_synced_at": 200, "enabled": true, "provisioned_state": "Provisioned" }).to_string()])) .once(); mock_redis_client .expect_list_indexer_states() @@ -196,72 +220,6 @@ mod tests { assert!(indexer_manager.list().await.is_err()); } - #[tokio::test] - async fn migrate() { - let config1 = IndexerConfig::default(); - let config2 = IndexerConfig { - account_id: "darunrs.near".parse().unwrap(), - function_name: "test".to_string(), - ..Default::default() - }; - - let registry = IndexerRegistry::from(&[ - ( - "morgs.near".parse().unwrap(), - HashMap::from([("test".to_string(), config1.clone())]), - ), - ( - "darunrs.near".parse().unwrap(), - HashMap::from([("test".to_string(), config2.clone())]), - ), - ]); - - let mut mock_redis_client = RedisClient::default(); - mock_redis_client - .expect_indexer_states_set_exists() - .returning(|| Ok(false)) - .once(); - mock_redis_client - .expect_get_indexer_state() - .with(predicate::eq(config1.clone())) - .returning(|_| { - Ok(Some( - serde_json::json!({ "block_stream_synced_at": 200, "enabled": false }) - .to_string(), - )) - }) - .once(); - mock_redis_client - .expect_get_indexer_state() - .with(predicate::eq(config2.clone())) - .returning(|_| Ok(None)) - .once(); - mock_redis_client - .expect_set_indexer_state() - .with( - predicate::eq(config1), - predicate::eq( - "{\"account_id\":\"morgs.near\",\"function_name\":\"test\",\"block_stream_synced_at\":200,\"enabled\":false}".to_string(), - ), - ) - .returning(|_, _| Ok(())) - .once(); - mock_redis_client - .expect_set_indexer_state() - .with( - predicate::eq(config2), - predicate::eq( - "{\"account_id\":\"darunrs.near\",\"function_name\":\"test\",\"block_stream_synced_at\":null,\"enabled\":true}".to_string() - ), - ) - .returning(|_, _| Ok(())) - .once(); - - let indexer_manager = IndexerStateManagerImpl::new(mock_redis_client); - - indexer_manager.migrate(®istry).await.unwrap(); - } - #[tokio::test] pub async fn disable_indexer() { let indexer_config = IndexerConfig { @@ -284,7 +242,7 @@ mod tests { .with(predicate::eq(indexer_config.clone())) .returning(|_| { Ok(Some( - serde_json::json!({ "account_id": "morgs.near", "function_name": "test", "block_stream_synced_at": 123, "enabled": true }) + serde_json::json!({ "account_id": "morgs.near", "function_name": "test", "block_stream_synced_at": 123, "enabled": true, "provisioned_state": "Provisioned" }) .to_string(), )) }); @@ -292,7 +250,7 @@ mod tests { .expect_set_indexer_state() .with( predicate::always(), - predicate::eq("{\"account_id\":\"morgs.near\",\"function_name\":\"test\",\"block_stream_synced_at\":123,\"enabled\":false}".to_string()), + predicate::eq("{\"account_id\":\"morgs.near\",\"function_name\":\"test\",\"block_stream_synced_at\":123,\"enabled\":false,\"provisioned_state\":\"Provisioned\"}".to_string()), ) .returning(|_, _| Ok(())) .once(); diff --git a/coordinator/src/main.rs b/coordinator/src/main.rs index 676d04196..a437e4187 100644 --- a/coordinator/src/main.rs +++ b/coordinator/src/main.rs @@ -4,15 +4,15 @@ use std::time::Duration; use near_primitives::types::AccountId; use tracing_subscriber::prelude::*; -use crate::block_streams_handler::BlockStreamsHandler; -use crate::executors_handler::ExecutorsHandler; +use crate::handlers::block_streams::BlockStreamsHandler; +use crate::handlers::data_layer::DataLayerHandler; +use crate::handlers::executors::ExecutorsHandler; use crate::indexer_state::IndexerStateManager; use crate::redis::RedisClient; use crate::registry::Registry; use crate::synchroniser::Synchroniser; -mod block_streams_handler; -mod executors_handler; +mod handlers; mod indexer_config; mod indexer_state; mod redis; @@ -60,10 +60,12 @@ async fn main() -> anyhow::Result<()> { let redis_client = RedisClient::connect(&redis_url).await?; let block_streams_handler = BlockStreamsHandler::connect(&block_streamer_url)?; let executors_handler = ExecutorsHandler::connect(&runner_url)?; + let data_layer_handler = DataLayerHandler::connect(&runner_url)?; let indexer_state_manager = Arc::new(IndexerStateManager::new(redis_client.clone())); let synchroniser = Synchroniser::new( &block_streams_handler, &executors_handler, + &data_layer_handler, ®istry, &indexer_state_manager, &redis_client, @@ -75,8 +77,7 @@ async fn main() -> anyhow::Result<()> { async move { server::init(grpc_port, indexer_state_manager, registry).await } }); - let indexer_registry = registry.fetch().await?; - indexer_state_manager.migrate(&indexer_registry).await?; + indexer_state_manager.migrate().await?; loop { tokio::try_join!(synchroniser.sync(), sleep(CONTROL_LOOP_THROTTLE_SECONDS))?; diff --git a/coordinator/src/synchroniser.rs b/coordinator/src/synchroniser.rs index b18185edd..ec200e10c 100644 --- a/coordinator/src/synchroniser.rs +++ b/coordinator/src/synchroniser.rs @@ -2,10 +2,13 @@ use registry_types::StartBlock; use tracing::instrument; use crate::{ - block_streams_handler::{BlockStreamsHandler, StreamInfo}, - executors_handler::{ExecutorInfo, ExecutorsHandler}, + handlers::{ + block_streams::{BlockStreamsHandler, StreamInfo}, + data_layer::{DataLayerHandler, ProvisioningStatus}, + executors::{ExecutorInfo, ExecutorsHandler}, + }, indexer_config::IndexerConfig, - indexer_state::{IndexerState, IndexerStateManager}, + indexer_state::{IndexerState, IndexerStateManager, ProvisionedState}, redis::RedisClient, registry::Registry, }; @@ -26,6 +29,7 @@ pub enum SynchronisationState { pub struct Synchroniser<'a> { block_streams_handler: &'a BlockStreamsHandler, executors_handler: &'a ExecutorsHandler, + data_layer_handler: &'a DataLayerHandler, registry: &'a Registry, state_manager: &'a IndexerStateManager, redis_client: &'a RedisClient, @@ -35,6 +39,7 @@ impl<'a> Synchroniser<'a> { pub fn new( block_streams_handler: &'a BlockStreamsHandler, executors_handler: &'a ExecutorsHandler, + data_layer_handler: &'a DataLayerHandler, registry: &'a Registry, state_manager: &'a IndexerStateManager, redis_client: &'a RedisClient, @@ -42,6 +47,7 @@ impl<'a> Synchroniser<'a> { Self { block_streams_handler, executors_handler, + data_layer_handler, registry, state_manager, redis_client, @@ -49,7 +55,7 @@ impl<'a> Synchroniser<'a> { } async fn start_new_block_stream(&self, config: &IndexerConfig) -> anyhow::Result<()> { - let start_block = match config.start_block { + let height = match config.start_block { StartBlock::Height(height) => height, StartBlock::Latest => config.get_registry_version(), StartBlock::Continue => { @@ -60,7 +66,9 @@ impl<'a> Synchroniser<'a> { } }; - self.block_streams_handler.start(start_block, config).await + tracing::info!(height, "Starting block stream"); + + self.block_streams_handler.start(height, config).await } #[instrument( @@ -72,23 +80,11 @@ impl<'a> Synchroniser<'a> { ) )] async fn sync_new_indexer(&self, config: &IndexerConfig) -> anyhow::Result<()> { - tracing::info!("Starting executor"); - - if let Err(err) = self.executors_handler.start(config).await { - tracing::error!(?err, "Failed to start Executor"); - return Ok(()); - } - - tracing::info!("Starting block stream"); - - // FIX if this fails, then subsequent control loops will perpetually fail since the - // above will error with ALREADY_EXISTS - if let Err(err) = self.start_new_block_stream(config).await { - tracing::error!(?err, "Failed to start Block Stream"); - return Ok(()); - } + tracing::info!("Starting data layer provisioning"); - self.state_manager.set_synced(config).await?; + self.data_layer_handler + .start_provisioning_task(config) + .await?; Ok(()) } @@ -191,10 +187,6 @@ impl<'a> Synchroniser<'a> { } if state.block_stream_synced_at.is_none() { - // NOTE: A value of `None` would suggest that `state` was created before initialisation, - // which is currently not possible, but may be in future - tracing::warn!("Existing block stream has no previous sync state, treating as new"); - self.start_new_block_stream(config).await?; return Ok(()); @@ -211,6 +203,34 @@ impl<'a> Synchroniser<'a> { Ok(()) } + async fn handle_provisioning(&self, config: &IndexerConfig) -> anyhow::Result<()> { + let task_status_result = self + .data_layer_handler + .check_provisioning_task_status(config) + .await; + + if let Err(error) = task_status_result { + tracing::warn!(?error, "Failed to check provisioning task status"); + + return Ok(()); + }; + + let _ = match task_status_result.unwrap() { + ProvisioningStatus::Complete => { + tracing::info!("Data layer provisioning complete"); + self.state_manager.set_provisioned(config).await + } + ProvisioningStatus::Pending => Ok(()), + _ => { + tracing::info!("Data layer provisioning failed"); + self.state_manager.set_provisioning_failure(config).await + } + } + .map_err(|err| tracing::warn!(?err, "Failed to set provisioning state")); + + Ok(()) + } + #[instrument( skip_all, fields( @@ -226,6 +246,15 @@ impl<'a> Synchroniser<'a> { executor: Option<&ExecutorInfo>, block_stream: Option<&StreamInfo>, ) -> anyhow::Result<()> { + match state.provisioned_state { + ProvisionedState::Unprovisioned | ProvisionedState::Provisioning => { + self.handle_provisioning(config).await?; + return Ok(()); + } + ProvisionedState::Failed => return Ok(()), + ProvisionedState::Provisioned => {} + } + if !state.enabled { if let Some(executor) = executor { self.executors_handler @@ -483,12 +512,14 @@ mod test { function_name: indexer.function_name.clone(), block_stream_synced_at: Some(indexer.get_registry_version()), enabled: true, + provisioned_state: ProvisionedState::Provisioned, }) .chain(deleted_indexer_configs.iter().map(|indexer| IndexerState { account_id: indexer.account_id.clone(), function_name: indexer.function_name.clone(), block_stream_synced_at: Some(indexer.get_registry_version()), enabled: true, + provisioned_state: ProvisionedState::Provisioned, })) .collect(); state_manager @@ -496,10 +527,12 @@ mod test { .returning(move || Ok(states.clone())); let redis_client = RedisClient::default(); + let data_layer_handler = DataLayerHandler::default(); let synchroniser = Synchroniser::new( &block_streams_handler, &executors_handler, + &data_layer_handler, ®istry, &state_manager, &redis_client, @@ -540,47 +573,19 @@ mod test { use super::*; #[tokio::test] - async fn start() { - let config1 = IndexerConfig::default(); - let config2 = IndexerConfig { - function_name: "test2".to_string(), - start_block: StartBlock::Latest, - ..Default::default() - }; + async fn triggers_data_layer_provisioning() { + let config = IndexerConfig::default(); let indexer_registry = IndexerRegistry::from(&[( - config1.account_id.clone(), - HashMap::from([ - (config1.function_name.clone(), config1.clone()), - (config2.function_name.clone(), config2.clone()), - ]), + config.account_id.clone(), + HashMap::from([(config.function_name.clone(), config.clone())]), )]); let mut block_streams_handler = BlockStreamsHandler::default(); block_streams_handler.expect_list().returning(|| Ok(vec![])); - block_streams_handler - .expect_start() - .with(eq(100), eq(config1.clone())) - .returning(|_, _| Ok(())) - .once(); - block_streams_handler - .expect_start() - .with(eq(config2.get_registry_version()), eq(config2.clone())) - .returning(|_, _| Ok(())) - .once(); let mut executors_handler = ExecutorsHandler::default(); executors_handler.expect_list().returning(|| Ok(vec![])); - executors_handler - .expect_start() - .with(eq(config1.clone())) - .returning(|_| Ok(())) - .once(); - executors_handler - .expect_start() - .with(eq(config2.clone())) - .returning(|_| Ok(())) - .once(); let mut registry = Registry::default(); registry @@ -589,15 +594,12 @@ mod test { let mut state_manager = IndexerStateManager::default(); state_manager.expect_list().returning(|| Ok(vec![])); - state_manager - .expect_set_synced() - .with(eq(config1)) - .returning(|_| Ok(())) - .once(); - state_manager - .expect_set_synced() - .with(eq(config2)) - .returning(|_| Ok(())) + + let mut data_layer_handler = DataLayerHandler::default(); + data_layer_handler + .expect_start_provisioning_task() + .with(eq(config)) + .returning(|_| Ok(ProvisioningStatus::Pending)) .once(); let redis_client = RedisClient::default(); @@ -605,6 +607,7 @@ mod test { let synchroniser = Synchroniser::new( &block_streams_handler, &executors_handler, + &data_layer_handler, ®istry, &state_manager, &redis_client, @@ -612,97 +615,126 @@ mod test { synchroniser.sync().await.unwrap(); } + } + + mod existing { + use super::*; #[tokio::test] - async fn configures_block_stream() { - let config_with_latest = IndexerConfig { - start_block: StartBlock::Latest, - ..IndexerConfig::default() - }; - let height = 5; - let config_with_height = IndexerConfig { - start_block: StartBlock::Height(height), - ..IndexerConfig::default() - }; - let config_with_continue = IndexerConfig { - start_block: StartBlock::Continue, - ..IndexerConfig::default() + async fn waits_for_provisioning_to_complete() { + let config = IndexerConfig::default(); + + let indexer_registry = IndexerRegistry::from(&[( + config.account_id.clone(), + HashMap::from([(config.function_name.clone(), config.clone())]), + )]); + + let state = IndexerState { + account_id: config.account_id.clone(), + function_name: config.function_name.clone(), + block_stream_synced_at: Some(config.get_registry_version()), + enabled: true, + provisioned_state: ProvisionedState::Provisioning, }; - let mut block_streams_handler = BlockStreamsHandler::default(); - block_streams_handler - .expect_start() - .with( - eq(config_with_continue.get_registry_version()), - eq(config_with_continue.clone()), - ) - .returning(|_, _| Ok(())) - .once(); - block_streams_handler - .expect_start() - .with( - eq(config_with_latest.get_registry_version()), - eq(config_with_latest.clone()), - ) - .returning(|_, _| Ok(())) - .once(); - block_streams_handler - .expect_start() - .with(eq(height), eq(config_with_height.clone())) - .returning(|_, _| Ok(())) - .once(); + let mut registry = Registry::default(); + registry + .expect_fetch() + .returning(move || Ok(indexer_registry.clone())); let mut state_manager = IndexerStateManager::default(); state_manager - .expect_set_synced() - .with(eq(config_with_continue.clone())) - .returning(|_| Ok(())) - .once(); - state_manager - .expect_set_synced() - .with(eq(config_with_latest.clone())) - .returning(|_| Ok(())) - .once(); - state_manager - .expect_set_synced() - .with(eq(config_with_height.clone())) + .expect_set_provisioned() + .with(eq(config.clone())) .returning(|_| Ok(())) .once(); + let mut data_layer_handler = DataLayerHandler::default(); + data_layer_handler + .expect_check_provisioning_task_status() + .with(eq(config.clone())) + .returning(|_| Ok(ProvisioningStatus::Complete)); + + let mut block_streams_handler = BlockStreamsHandler::default(); + block_streams_handler.expect_start().never(); + let mut executors_handler = ExecutorsHandler::default(); - executors_handler - .expect_start() - .returning(|_| Ok(())) - .times(3); + executors_handler.expect_start().never(); let redis_client = RedisClient::default(); - let registry = Registry::default(); let synchroniser = Synchroniser::new( &block_streams_handler, &executors_handler, + &data_layer_handler, ®istry, &state_manager, &redis_client, ); synchroniser - .sync_new_indexer(&config_with_latest) - .await - .unwrap(); - synchroniser - .sync_new_indexer(&config_with_height) + .sync_existing_indexer(&config, &state, None, None) .await .unwrap(); + } + + #[tokio::test] + async fn ignores_failed_provisioning() { + let config = IndexerConfig::default(); + + let indexer_registry = IndexerRegistry::from(&[( + config.account_id.clone(), + HashMap::from([(config.function_name.clone(), config.clone())]), + )]); + + let state = IndexerState { + account_id: config.account_id.clone(), + function_name: config.function_name.clone(), + block_stream_synced_at: Some(config.get_registry_version()), + enabled: true, + provisioned_state: ProvisionedState::Provisioning, + }; + + let mut registry = Registry::default(); + registry + .expect_fetch() + .returning(move || Ok(indexer_registry.clone())); + + let mut state_manager = IndexerStateManager::default(); + state_manager + .expect_set_provisioning_failure() + .with(eq(config.clone())) + .returning(|_| Ok(())) + .once(); + + let mut data_layer_handler = DataLayerHandler::default(); + data_layer_handler + .expect_check_provisioning_task_status() + .with(eq(config.clone())) + .returning(|_| Ok(ProvisioningStatus::Failed)); + + let mut block_streams_handler = BlockStreamsHandler::default(); + block_streams_handler.expect_start().never(); + + let mut executors_handler = ExecutorsHandler::default(); + executors_handler.expect_start().never(); + + let redis_client = RedisClient::default(); + + let synchroniser = Synchroniser::new( + &block_streams_handler, + &executors_handler, + &data_layer_handler, + ®istry, + &state_manager, + &redis_client, + ); + synchroniser - .sync_new_indexer(&config_with_continue) + .sync_existing_indexer(&config, &state, None, None) .await .unwrap(); } - } - - mod existing { - use super::*; #[tokio::test] async fn ignores_synced() { @@ -757,14 +789,17 @@ mod test { function_name: config.function_name.clone(), block_stream_synced_at: Some(config.get_registry_version()), enabled: true, + provisioned_state: ProvisionedState::Provisioned, }]) }); let redis_client = RedisClient::default(); + let data_layer_handler = DataLayerHandler::default(); let synchroniser = Synchroniser::new( &block_streams_handler, &executors_handler, + &data_layer_handler, ®istry, &state_manager, &redis_client, @@ -849,12 +884,16 @@ mod test { function_name: config.function_name.clone(), block_stream_synced_at: Some(config.get_registry_version()), enabled: true, + provisioned_state: ProvisionedState::Provisioned, }]) }); + let data_layer_handler = DataLayerHandler::default(); + let synchroniser = Synchroniser::new( &block_streams_handler, &executors_handler, + &data_layer_handler, ®istry, &state_manager, &redis_client, @@ -871,6 +910,7 @@ mod test { function_name: config.function_name.clone(), block_stream_synced_at: None, enabled: true, + provisioned_state: ProvisionedState::Provisioned, }; let mut block_streams_handler = BlockStreamsHandler::default(); @@ -884,10 +924,12 @@ mod test { let state_manager = IndexerStateManager::default(); let executors_handler = ExecutorsHandler::default(); let registry = Registry::default(); + let data_layer_handler = DataLayerHandler::default(); let synchroniser = Synchroniser::new( &block_streams_handler, &executors_handler, + &data_layer_handler, ®istry, &state_manager, &redis_client, @@ -907,6 +949,7 @@ mod test { function_name: config.function_name.clone(), block_stream_synced_at: Some(config.get_registry_version() - 1), enabled: true, + provisioned_state: ProvisionedState::Provisioned, }; let mut block_streams_handler = BlockStreamsHandler::default(); @@ -926,10 +969,12 @@ mod test { let state_manager = IndexerStateManager::default(); let executors_handler = ExecutorsHandler::default(); let registry = Registry::default(); + let data_layer_handler = DataLayerHandler::default(); let synchroniser = Synchroniser::new( &block_streams_handler, &executors_handler, + &data_layer_handler, ®istry, &state_manager, &redis_client, @@ -949,6 +994,7 @@ mod test { function_name: config.function_name.clone(), block_stream_synced_at: Some(config.get_registry_version()), enabled: true, + provisioned_state: ProvisionedState::Provisioned, }; let last_published_block = 1; @@ -970,10 +1016,12 @@ mod test { let state_manager = IndexerStateManager::default(); let executors_handler = ExecutorsHandler::default(); let registry = Registry::default(); + let data_layer_handler = DataLayerHandler::default(); let synchroniser = Synchroniser::new( &block_streams_handler, &executors_handler, + &data_layer_handler, ®istry, &state_manager, &redis_client, @@ -1044,10 +1092,12 @@ mod test { let state_manager = IndexerStateManager::default(); let executors_handler = ExecutorsHandler::default(); let registry = Registry::default(); + let data_layer_handler = DataLayerHandler::default(); let synchroniser = Synchroniser::new( &block_streams_handler, &executors_handler, + &data_layer_handler, ®istry, &state_manager, &redis_client, @@ -1075,6 +1125,7 @@ mod test { function_name: config.function_name.clone(), block_stream_synced_at: Some(config.get_registry_version()), enabled: false, + provisioned_state: ProvisionedState::Provisioned, }; let executor = ExecutorInfo { executor_id: "executor_id".to_string(), @@ -1113,10 +1164,12 @@ mod test { let registry = Registry::default(); let redis_client = RedisClient::default(); + let data_layer_handler = DataLayerHandler::default(); let synchroniser = Synchroniser::new( &block_streams_handler, &executors_handler, + &data_layer_handler, ®istry, &state_manager, &redis_client, @@ -1145,6 +1198,7 @@ mod test { function_name: config.function_name.clone(), block_stream_synced_at: Some(config.get_registry_version()), enabled: false, + provisioned_state: ProvisionedState::Provisioned, }; let executor = ExecutorInfo { executor_id: "executor_id".to_string(), @@ -1183,10 +1237,12 @@ mod test { let registry = Registry::default(); let redis_client = RedisClient::default(); + let data_layer_handler = DataLayerHandler::default(); let synchroniser = Synchroniser::new( &block_streams_handler, &executors_handler, + &data_layer_handler, ®istry, &state_manager, &redis_client, diff --git a/runner-client/build.rs b/runner-client/build.rs index 618200337..0ed89bc1c 100644 --- a/runner-client/build.rs +++ b/runner-client/build.rs @@ -1,5 +1,6 @@ fn main() -> Result<(), Box> { tonic_build::compile_protos("proto/runner.proto")?; + tonic_build::compile_protos("proto/data-layer.proto")?; Ok(()) } diff --git a/runner-client/examples/check_provisioning_task_status.rs b/runner-client/examples/check_provisioning_task_status.rs new file mode 100644 index 000000000..b5f6fc562 --- /dev/null +++ b/runner-client/examples/check_provisioning_task_status.rs @@ -0,0 +1,20 @@ +use tonic::Request; + +use runner::data_layer::data_layer_client::DataLayerClient; +use runner::data_layer::CheckProvisioningTaskStatusRequest; + +#[tokio::main] +async fn main() -> Result<(), Box> { + let mut client = DataLayerClient::connect("http://localhost:7001").await?; + + let response = client + .check_provisioning_task_status(Request::new(CheckProvisioningTaskStatusRequest { + account_id: "morgs.near".to_string(), + function_name: "test2".to_string(), + })) + .await?; + + println!("{:#?}", response.into_inner()); + + Ok(()) +} diff --git a/runner-client/examples/start_provisioning_task.rs b/runner-client/examples/start_provisioning_task.rs new file mode 100644 index 000000000..24a938d15 --- /dev/null +++ b/runner-client/examples/start_provisioning_task.rs @@ -0,0 +1,21 @@ +use tonic::Request; + +use runner::data_layer::data_layer_client::DataLayerClient; +use runner::data_layer::ProvisionRequest; + +#[tokio::main] +async fn main() -> Result<(), Box> { + let mut client = DataLayerClient::connect("http://localhost:7001").await?; + + let response = client + .start_provisioning_task(Request::new(ProvisionRequest { + account_id: "morgs.near".to_string(), + function_name: "test2".to_string(), + schema: "create table blocks();".to_string(), + })) + .await?; + + println!("{:#?}", response.into_inner()); + + Ok(()) +} diff --git a/runner-client/proto/data-layer.proto b/runner-client/proto/data-layer.proto new file mode 100644 index 000000000..909e3d28a --- /dev/null +++ b/runner-client/proto/data-layer.proto @@ -0,0 +1,33 @@ +syntax = "proto3"; + +package data_layer; + +service DataLayer { + // Starts async provisioning task + rpc StartProvisioningTask (ProvisionRequest) returns (ProvisionResponse); + + // Checks the provisioning status + rpc CheckProvisioningTaskStatus (CheckProvisioningTaskStatusRequest) returns (ProvisionResponse); +} + +message ProvisionRequest { + string account_id = 1; + string function_name = 2; + string schema = 3; +} + +message CheckProvisioningTaskStatusRequest { + string account_id = 1; + string function_name = 2; +} + +enum ProvisioningStatus { + UNSPECIFIED = 0; + PENDING = 1; + COMPLETE = 2; + FAILED = 3; +} + +message ProvisionResponse { + ProvisioningStatus status = 1; +} diff --git a/runner-client/src/lib.rs b/runner-client/src/lib.rs index c7a7a57fd..ca16435d8 100644 --- a/runner-client/src/lib.rs +++ b/runner-client/src/lib.rs @@ -3,3 +3,7 @@ mod runner { } pub use runner::*; + +pub mod data_layer { + tonic::include_proto!("data_layer"); +} diff --git a/runner/protos/data-layer.proto b/runner/protos/data-layer.proto index eaef8034a..909e3d28a 100644 --- a/runner/protos/data-layer.proto +++ b/runner/protos/data-layer.proto @@ -3,8 +3,8 @@ syntax = "proto3"; package data_layer; service DataLayer { - // Provisions the data layer (PostgreSQL + Hasura) - rpc Provision (ProvisionRequest) returns (ProvisionResponse); + // Starts async provisioning task + rpc StartProvisioningTask (ProvisionRequest) returns (ProvisionResponse); // Checks the provisioning status rpc CheckProvisioningTaskStatus (CheckProvisioningTaskStatusRequest) returns (ProvisionResponse); diff --git a/runner/src/indexer/__snapshots__/indexer.test.ts.snap b/runner/src/indexer/__snapshots__/indexer.test.ts.snap index a688b963a..bca20ca2d 100644 --- a/runner/src/indexer/__snapshots__/indexer.test.ts.snap +++ b/runner/src/indexer/__snapshots__/indexer.test.ts.snap @@ -35,8 +35,6 @@ exports[`Indexer unit tests Indexer.execute() allows imperative execution of Gra exports[`Indexer unit tests Indexer.execute() catches errors 1`] = `[]`; -exports[`Indexer unit tests Indexer.execute() logs provisioning failures 1`] = `[]`; - exports[`Indexer unit tests Indexer.execute() should execute all functions against the current block 1`] = ` [ [ diff --git a/runner/src/indexer/indexer.test.ts b/runner/src/indexer/indexer.test.ts index fced2b565..f2e4bad6c 100644 --- a/runner/src/indexer/indexer.test.ts +++ b/runner/src/indexer/indexer.test.ts @@ -909,92 +909,6 @@ describe('Indexer unit tests', () => { expect(indexerMeta.updateBlockHeight).not.toHaveBeenCalled(); }); - test('Indexer.execute() provisions a GraphQL endpoint with the specified schema', async () => { - const blockHeight = 82699904; - const mockFetch = jest.fn(() => ({ - status: 200, - json: async () => ({ - errors: null, - }), - })); - const mockBlock = Block.fromStreamerMessage({ - block: { - chunks: [0], - header: { - height: blockHeight - } - }, - shards: {} - } as unknown as StreamerMessage) as unknown as Block; - const provisioner: any = { - getPgBouncerConnectionParameters: jest.fn().mockReturnValue(genericDbCredentials), - fetchUserApiProvisioningStatus: jest.fn().mockReturnValue(false), - provisionUserApi: jest.fn(), - provisionLogsAndMetadataIfNeeded: jest.fn(), - ensureConsistentHasuraState: jest.fn(), - }; - const indexerMeta = { - writeLogs: jest.fn(), - setStatus: jest.fn(), - updateBlockHeight: jest.fn() - } as unknown as IndexerMeta; - const indexer = new Indexer(simpleSchemaConfig, { - fetch: mockFetch as unknown as typeof fetch, - provisioner, - dmlHandler: genericMockDmlHandler, - indexerMeta, - }, undefined, config); - - await indexer.execute(mockBlock); - - expect(provisioner.fetchUserApiProvisioningStatus).toHaveBeenCalledWith(simpleSchemaConfig); - expect(indexerMeta.setStatus).toHaveBeenNthCalledWith(1, IndexerStatus.RUNNING); - expect(provisioner.provisionUserApi).toHaveBeenCalledTimes(1); - expect(provisioner.provisionUserApi).toHaveBeenCalledWith(simpleSchemaConfig); - // expect(provisioner.provisionLogsAndMetadataIfNeeded).toHaveBeenCalledTimes(1); - // expect(provisioner.ensureConsistentHasuraState).toHaveBeenCalledTimes(1); - expect(provisioner.getPgBouncerConnectionParameters).toHaveBeenCalledTimes(1); - }); - - test('Indexer.execute() skips provisioning if the endpoint exists', async () => { - const blockHeight = 82699904; - const mockFetch = jest.fn(() => ({ - status: 200, - json: async () => ({ - errors: null, - }), - })); - const mockBlock = Block.fromStreamerMessage({ - block: { - chunks: [0], - header: { - height: blockHeight - } - }, - shards: {} - } as unknown as StreamerMessage) as unknown as Block; - const provisioner: any = { - getPgBouncerConnectionParameters: jest.fn().mockReturnValue(genericDbCredentials), - fetchUserApiProvisioningStatus: jest.fn().mockReturnValue(true), - provisionUserApi: jest.fn(), - provisionLogsAndMetadataIfNeeded: jest.fn(), - ensureConsistentHasuraState: jest.fn(), - }; - const indexer = new Indexer(simpleSchemaConfig, { - fetch: mockFetch as unknown as typeof fetch, - provisioner, - dmlHandler: genericMockDmlHandler, - indexerMeta: genericMockIndexerMeta, - }, undefined, config); - - await indexer.execute(mockBlock); - - expect(provisioner.provisionUserApi).not.toHaveBeenCalled(); - expect(provisioner.getPgBouncerConnectionParameters).toHaveBeenCalledTimes(1); - // expect(provisioner.provisionLogsAndMetadataIfNeeded).toHaveBeenCalledTimes(1); - // expect(provisioner.ensureConsistentHasuraState).toHaveBeenCalledTimes(1); - }); - test('Indexer.execute() skips database credentials fetch second time onward', async () => { const blockHeight = 82699904; const mockFetch = jest.fn(() => ({ @@ -1096,57 +1010,6 @@ describe('Indexer unit tests', () => { expect(indexerMeta.updateBlockHeight).toHaveBeenCalledWith(blockHeight); }); - test('Indexer.execute() logs provisioning failures', async () => { - const blockHeight = 82699904; - const mockFetch = jest.fn(() => ({ - status: 200, - json: async () => ({ - errors: null, - }), - })); - const mockBlock = Block.fromStreamerMessage({ - block: { - chunks: [0], - header: { - height: blockHeight - } - }, - shards: {} - } as unknown as StreamerMessage) as unknown as Block; - const error = new Error('something went wrong with provisioning'); - const provisioner: any = { - getPgBouncerConnectionParameters: jest.fn().mockReturnValue(genericDbCredentials), - fetchUserApiProvisioningStatus: jest.fn().mockReturnValue(false), - provisionUserApi: jest.fn().mockRejectedValue(error), - provisionLogsIfNeeded: jest.fn(), - provisionMetadataIfNeeded: jest.fn(), - ensureConsistentHasuraState: jest.fn(), - }; - const indexerMeta = { - writeLogs: jest.fn(), - setStatus: jest.fn(), - updateBlockHeight: jest.fn() - } as unknown as IndexerMeta; - const code = ` - context.graphql(\`mutation { set(functionName: "buildnear.testnet/test", key: "height", data: "\${block.blockHeight}")}\`); - `; - const indexerConfig = new IndexerConfig(SIMPLE_REDIS_STREAM, 'morgs.near', 'test', 0, code, 'schema', LogLevel.INFO); - const indexer = new Indexer(indexerConfig, { - fetch: mockFetch as unknown as typeof fetch, - provisioner, - dmlHandler: genericMockDmlHandler, - indexerMeta, - }, undefined, config); - - await expect(indexer.execute(mockBlock)).rejects.toThrow(error); - - expect(mockFetch.mock.calls).toMatchSnapshot(); - expect(indexerMeta.updateBlockHeight).not.toHaveBeenCalled(); - expect(provisioner.provisionLogsIfNeeded).not.toHaveBeenCalled(); - expect(provisioner.provisionMetadataIfNeeded).not.toHaveBeenCalled(); - expect(provisioner.getPgBouncerConnectionParameters).not.toHaveBeenCalled(); - }); - test('Indexer passes all relevant logs to writeLogs', async () => { const mockDebugIndexerMeta = { writeLogs: jest.fn(), diff --git a/runner/src/indexer/indexer.ts b/runner/src/indexer/indexer.ts index 344deae86..97d9f6289 100644 --- a/runner/src/indexer/indexer.ts +++ b/runner/src/indexer/indexer.ts @@ -92,21 +92,6 @@ export default class Indexer { try { const runningMessage = `Running function ${this.indexerConfig.fullName()} on block ${blockHeight}, lag is: ${lag?.toString()}ms from block timestamp`; - try { - if (!await this.deps.provisioner.fetchUserApiProvisioningStatus(this.indexerConfig)) { - logEntries.push(LogEntry.systemInfo('Provisioning endpoint: starting', blockHeight)); - await this.deps.provisioner.provisionUserApi(this.indexerConfig); - logEntries.push(LogEntry.systemInfo('Provisioning endpoint: successful', blockHeight)); - } - } catch (e) { - const error = e as Error; - if (this.IS_FIRST_EXECUTION) { - this.logger.error('Provisioning endpoint: failure', error); - } - logEntries.push(LogEntry.systemError(`Provisioning endpoint failure: ${error.message}`, blockHeight)); - throw error; - } - logEntries.push(LogEntry.systemInfo(runningMessage, blockHeight)); // Cache database credentials after provisioning await wrapSpan(async () => { diff --git a/runner/src/server/services/data-layer/data-layer-service.test.ts b/runner/src/server/services/data-layer/data-layer-service.test.ts index 99963d30e..15fc29590 100644 --- a/runner/src/server/services/data-layer/data-layer-service.test.ts +++ b/runner/src/server/services/data-layer/data-layer-service.test.ts @@ -67,7 +67,7 @@ describe('DataLayerService', () => { }); describe('Provision', () => { - it('should return ALREADY_EXISTS if the task is already pending', (done) => { + it('should return ALREADY_EXISTS if the task exists', (done) => { const tasks = { 'testAccount:testFunction': { pending: true, completed: false, failed: false } as unknown as ProvisioningTask }; @@ -80,23 +80,23 @@ describe('DataLayerService', () => { done(); }; - createDataLayerService(undefined, tasks).Provision(call, callback); + createDataLayerService(undefined, tasks).StartProvisioningTask(call, callback); }); - it('should return ALREADY_EXISTS if the task has already completed', (done) => { + it('should return Complete if the task has already completed', (done) => { + const tasks: Record = {}; const provisioner = { fetchUserApiProvisioningStatus: jest.fn().mockResolvedValue(true) } as unknown as Provisioner; const call = { request: { accountId: 'testAccount', functionName: 'testFunction', schema: 'testSchema' } } as unknown as ServerUnaryCall; - const callback = (error: any): void => { - expect(error.code).toBe(status.ALREADY_EXISTS); - expect(error.details).toBe('Provisioning task has already completed'); + const callback = (_error: any, response: any): void => { + expect(response.status).toBe(ProvisioningStatus.COMPLETE); done(); }; - createDataLayerService(provisioner).Provision(call, callback); + createDataLayerService(provisioner, tasks).StartProvisioningTask(call, callback); }); it('should start a new provisioning task and return PENDING', (done) => { @@ -115,7 +115,7 @@ describe('DataLayerService', () => { done(); }; - createDataLayerService(provisioner, tasks).Provision(call, callback); + createDataLayerService(provisioner, tasks).StartProvisioningTask(call, callback); }); it('should return INTERNAL error if checking provisioning status fails', (done) => { @@ -132,7 +132,7 @@ describe('DataLayerService', () => { done(); }; - createDataLayerService(provisioner, tasks).Provision(call, callback); + createDataLayerService(provisioner, tasks).StartProvisioningTask(call, callback); }); }); }); diff --git a/runner/src/server/services/data-layer/data-layer-service.ts b/runner/src/server/services/data-layer/data-layer-service.ts index 65176de65..ca62a5ea2 100644 --- a/runner/src/server/services/data-layer/data-layer-service.ts +++ b/runner/src/server/services/data-layer/data-layer-service.ts @@ -67,14 +67,14 @@ export function createDataLayerService ( callback(null, { status: ProvisioningStatus.PENDING }); }, - Provision (call: ServerUnaryCall, callback: sendUnaryData): void { + StartProvisioningTask (call: ServerUnaryCall, callback: sendUnaryData): void { const { accountId, functionName, schema } = call.request; const provisioningConfig = new ProvisioningConfig(accountId, functionName, schema); const task = tasks[generateTaskId(accountId, functionName)]; - if (task?.pending) { + if (task) { const exists = new StatusBuilder() .withCode(status.ALREADY_EXISTS) .withDetails('Provisioning task already exists') @@ -86,11 +86,7 @@ export function createDataLayerService ( provisioner.fetchUserApiProvisioningStatus(provisioningConfig).then((isProvisioned) => { if (isProvisioned) { - const exists = new StatusBuilder() - .withCode(status.ALREADY_EXISTS) - .withDetails('Provisioning task has already completed') - .build(); - callback(exists); + callback(null, { status: ProvisioningStatus.COMPLETE }); return; } diff --git a/runner/tests/integration.test.ts b/runner/tests/integration.test.ts index f7be5ef1b..dda4511b8 100644 --- a/runner/tests/integration.test.ts +++ b/runner/tests/integration.test.ts @@ -126,6 +126,8 @@ describe('Indexer integration', () => { } ); + await provisioner.provisionUserApi(indexerConfig); + await indexer.execute(Block.fromStreamerMessage(block_115185108 as any as StreamerMessage)); const firstHeight = await indexerBlockHeightQuery('morgs_near_test', graphqlClient); @@ -139,7 +141,7 @@ describe('Indexer integration', () => { expect(secondHeight.value).toEqual('115185109'); const logs: any = await indexerLogsQuery('morgs_near_test', graphqlClient); - expect(logs.length).toEqual(4); + expect(logs.length).toEqual(2); const { morgs_near_test_blocks: blocks }: any = await graphqlClient.request(blocksIndexerQuery); expect(blocks.map(({ height }: any) => height)).toEqual([115185108, 115185109]); @@ -232,6 +234,8 @@ describe('Indexer integration', () => { } ); + await provisioner.provisionUserApi(indexerConfig); + await indexer.execute(Block.fromStreamerMessage(block_115185108 as any as StreamerMessage)); await indexer.execute(Block.fromStreamerMessage(block_115185109 as any as StreamerMessage)); From 10a65f24ffdff986a8433c75b89bedbe9539ecb6 Mon Sep 17 00:00:00 2001 From: Morgan McCauley Date: Thu, 20 Jun 2024 10:14:33 +1200 Subject: [PATCH 02/10] fix: Deserialise old state correctly during migration (#816) --- coordinator/src/indexer_state.rs | 24 +++++++++++++++++++++++- 1 file changed, 23 insertions(+), 1 deletion(-) diff --git a/coordinator/src/indexer_state.rs b/coordinator/src/indexer_state.rs index ce100354c..2dabcaa96 100644 --- a/coordinator/src/indexer_state.rs +++ b/coordinator/src/indexer_state.rs @@ -70,7 +70,7 @@ impl IndexerStateManagerImpl { tracing::info!("Migrating {}", raw_state); - let old_state: IndexerState = serde_json::from_str(&raw_state)?; + let old_state: OldIndexerState = serde_json::from_str(&raw_state)?; let state = IndexerState { account_id: old_state.account_id, @@ -202,6 +202,28 @@ mod tests { use mockall::predicate; use registry_types::{Rule, StartBlock, Status}; + #[tokio::test] + async fn migrate() { + let config = IndexerConfig::default(); + let mut mock_redis_client = RedisClient::default(); + mock_redis_client + .expect_list_indexer_states() + .returning(|| Ok(vec![serde_json::json!({ "account_id": "morgs.near", "function_name": "test", "block_stream_synced_at": 200, "enabled": true }).to_string()])) + .once(); + mock_redis_client + .expect_set::() + .with( + predicate::eq(config.get_state_key()), + predicate::eq("{\"account_id\":\"morgs.near\",\"function_name\":\"test\",\"block_stream_synced_at\":200,\"enabled\":true,\"provisioned_state\":\"Provisioned\"}".to_string()), + ) + .returning(|_, _| Ok(())) + .once(); + + let indexer_manager = IndexerStateManagerImpl::new(mock_redis_client); + + indexer_manager.migrate().await.unwrap(); + } + #[tokio::test] async fn list_indexer_states() { let mut mock_redis_client = RedisClient::default(); From d88a4dc884bcf10abb0ade2043412d18e698fd5c Mon Sep 17 00:00:00 2001 From: Morgan McCauley Date: Thu, 20 Jun 2024 15:06:04 +1200 Subject: [PATCH 03/10] fix: Ensure state is updated after starting data layer provisioning (#817) - Need to write the state to move it out of "new" and in to "existing" - Added some logs to `DataLayerService` --- coordinator/src/synchroniser.rs | 13 ++++++++++++ .../services/data-layer/data-layer-service.ts | 21 ++++++++++++++++++- 2 files changed, 33 insertions(+), 1 deletion(-) diff --git a/coordinator/src/synchroniser.rs b/coordinator/src/synchroniser.rs index ec200e10c..1156d89af 100644 --- a/coordinator/src/synchroniser.rs +++ b/coordinator/src/synchroniser.rs @@ -86,6 +86,14 @@ impl<'a> Synchroniser<'a> { .start_provisioning_task(config) .await?; + self.state_manager + .set_provisioning(config) + .await + .map_err(|err| { + tracing::warn!(?err, "Failed to set provisioning state"); + err + })?; + Ok(()) } @@ -594,6 +602,11 @@ mod test { let mut state_manager = IndexerStateManager::default(); state_manager.expect_list().returning(|| Ok(vec![])); + state_manager + .expect_set_provisioning() + .with(eq(config.clone())) + .returning(|_| Ok(())) + .once(); let mut data_layer_handler = DataLayerHandler::default(); data_layer_handler diff --git a/runner/src/server/services/data-layer/data-layer-service.ts b/runner/src/server/services/data-layer/data-layer-service.ts index ca62a5ea2..18312568f 100644 --- a/runner/src/server/services/data-layer/data-layer-service.ts +++ b/runner/src/server/services/data-layer/data-layer-service.ts @@ -2,6 +2,7 @@ import { type ServerUnaryCall, type sendUnaryData, status, StatusBuilder } from import Provisioner from '../../../provisioner'; import { ProvisioningConfig } from '../../../indexer-config/indexer-config'; +import parentLogger from '../../../logger'; import { type CheckProvisioningTaskStatusRequest__Output } from '../../../generated/data_layer/CheckProvisioningTaskStatusRequest'; import { type DataLayerHandlers } from '../../../generated/data_layer/DataLayer'; @@ -72,6 +73,12 @@ export function createDataLayerService ( const provisioningConfig = new ProvisioningConfig(accountId, functionName, schema); + const logger = parentLogger.child({ + service: 'DataLayerService', + accountId: provisioningConfig.accountId, + functionName: provisioningConfig.functionName, + }); + const task = tasks[generateTaskId(accountId, functionName)]; if (task) { @@ -91,7 +98,19 @@ export function createDataLayerService ( return; } - tasks[generateTaskId(accountId, functionName)] = new ProvisioningTask(provisioner.provisionUserApi(provisioningConfig)); + logger.info('Provisioning Data Layer'); + + tasks[generateTaskId(accountId, functionName)] = new ProvisioningTask( + provisioner + .provisionUserApi(provisioningConfig) + .then(() => { + logger.info('Successfully provisioned Data Layer'); + }) + .catch((err) => { + logger.error('Failed to provision Data Layer', err); + throw err; + }) + ); callback(null, { status: ProvisioningStatus.PENDING }); }).catch((error) => { From 64d1ebbf12d4bce7bcbf54b560b530fe6197824f Mon Sep 17 00:00:00 2001 From: Morgan McCauley Date: Thu, 20 Jun 2024 20:32:21 +1200 Subject: [PATCH 04/10] feat: Deprovision Data Layer on delete (#808) This PR removes Data Layer resources on Indexer Delete. To achieve this, the following has been added: - `Provisioner.deprovision()` method which removes: schema, cron jobs, and if necessary, Hasura source, database, and role - `DataLayerService.StartDeprovisioningTask` gRPC method - Calling the above from Coordinator within the delete lifecycle hook In addition to the above, I've slightly refactored DataLayerService to make the addition of de-provisioning more accomodating: - `StartDeprovisioningTask` and `StartProvisioningTask` now return opaque IDs rather than using `accountId`/`functionName` - to avoid conflicts with eachother - There is a single `GetTaskStatus` method which is used for both, before it was specific to provisioning As mentioned in #805, the Coordinator implementation is a little awkward due to the shared/non-blocking Control Loop. I'll look to refactor this later and hopefully improve on this. --- coordinator/src/handlers/data_layer.rs | 81 ++++---- coordinator/src/indexer_state.rs | 43 +++- coordinator/src/redis.rs | 4 + coordinator/src/synchroniser.rs | 188 ++++++++++++++---- .../examples/start_provisioning_task.rs | 4 +- runner-client/proto/data-layer.proto | 27 ++- runner/protos/data-layer.proto | 27 ++- .../__snapshots__/hasura-client.test.ts.snap | 25 +++ .../src/hasura-client/hasura-client.test.ts | 29 +++ runner/src/hasura-client/hasura-client.ts | 14 ++ runner/src/provisioner/provisioner.test.ts | 98 +++++++++ runner/src/provisioner/provisioner.ts | 94 +++++++++ .../data-layer/data-layer-service.test.ts | 90 ++++----- .../services/data-layer/data-layer-service.ts | 135 ++++++++----- runner/tests/integration.test.ts | 46 +++++ 15 files changed, 711 insertions(+), 194 deletions(-) diff --git a/coordinator/src/handlers/data_layer.rs b/coordinator/src/handlers/data_layer.rs index 7db624648..68537b3c7 100644 --- a/coordinator/src/handlers/data_layer.rs +++ b/coordinator/src/handlers/data_layer.rs @@ -1,12 +1,14 @@ #![cfg_attr(test, allow(dead_code))] -pub use runner::data_layer::ProvisioningStatus; +use near_primitives::types::AccountId; + +pub use runner::data_layer::TaskStatus; use anyhow::Context; use runner::data_layer::data_layer_client::DataLayerClient; -use runner::data_layer::{CheckProvisioningTaskStatusRequest, ProvisionRequest}; +use runner::data_layer::{DeprovisionRequest, GetTaskStatusRequest, ProvisionRequest}; use tonic::transport::channel::Channel; -use tonic::{Request, Status}; +use tonic::Request; use crate::indexer_config::IndexerConfig; @@ -15,17 +17,14 @@ pub use DataLayerHandlerImpl as DataLayerHandler; #[cfg(test)] pub use MockDataLayerHandlerImpl as DataLayerHandler; +type TaskId = String; + pub struct DataLayerHandlerImpl { client: DataLayerClient, } #[cfg_attr(test, mockall::automock)] impl DataLayerHandlerImpl { - pub fn from_env() -> anyhow::Result { - let runner_url = std::env::var("RUNNER_URL").context("RUNNER_URL is not set")?; - Self::connect(&runner_url) - } - pub fn connect(runner_url: &str) -> anyhow::Result { let channel = Channel::from_shared(runner_url.to_string()) .context("Runner URL is invalid")? @@ -38,7 +37,7 @@ impl DataLayerHandlerImpl { pub async fn start_provisioning_task( &self, indexer_config: &IndexerConfig, - ) -> anyhow::Result { + ) -> anyhow::Result { let request = ProvisionRequest { account_id: indexer_config.account_id.to_string(), function_name: indexer_config.function_name.clone(), @@ -49,46 +48,52 @@ impl DataLayerHandlerImpl { .client .clone() .start_provisioning_task(Request::new(request)) - .await; - - if let Err(error) = response { - if error.code() == tonic::Code::AlreadyExists { - return Ok(ProvisioningStatus::Pending); - } - - return Err(error.into()); - } - - let status = match response.unwrap().into_inner().status { - 1 => ProvisioningStatus::Pending, - 2 => ProvisioningStatus::Complete, - 3 => ProvisioningStatus::Failed, - _ => ProvisioningStatus::Unspecified, - }; + .await?; - Ok(status) + Ok(response.into_inner().task_id) } - pub async fn check_provisioning_task_status( + pub async fn start_deprovisioning_task( &self, - indexer_config: &IndexerConfig, - ) -> anyhow::Result { - let request = CheckProvisioningTaskStatusRequest { - account_id: indexer_config.account_id.to_string(), - function_name: indexer_config.function_name.clone(), + account_id: AccountId, + function_name: String, + ) -> anyhow::Result { + let request = DeprovisionRequest { + account_id: account_id.to_string(), + function_name, }; let response = self .client .clone() - .check_provisioning_task_status(Request::new(request)) + .start_deprovisioning_task(Request::new(request)) .await?; - let status = match response.into_inner().status { - 1 => ProvisioningStatus::Pending, - 2 => ProvisioningStatus::Complete, - 3 => ProvisioningStatus::Failed, - _ => ProvisioningStatus::Unspecified, + Ok(response.into_inner().task_id) + } + + pub async fn get_task_status(&self, task_id: TaskId) -> anyhow::Result { + let request = GetTaskStatusRequest { task_id }; + + let response = self + .client + .clone() + .get_task_status(Request::new(request)) + .await; + + if let Err(error) = response { + if error.code() == tonic::Code::NotFound { + return Ok(TaskStatus::Failed); + } + + return Err(error.into()); + } + + let status = match response.unwrap().into_inner().status { + 1 => TaskStatus::Pending, + 2 => TaskStatus::Complete, + 3 => TaskStatus::Failed, + _ => anyhow::bail!("Received invalid task status"), }; Ok(status) diff --git a/coordinator/src/indexer_state.rs b/coordinator/src/indexer_state.rs index 2dabcaa96..b6cb20be5 100644 --- a/coordinator/src/indexer_state.rs +++ b/coordinator/src/indexer_state.rs @@ -9,8 +9,9 @@ use crate::redis::RedisClient; #[derive(Debug, Clone, serde::Serialize, serde::Deserialize, PartialEq, Eq)] pub enum ProvisionedState { Unprovisioned, - Provisioning, + Provisioning { task_id: String }, Provisioned, + Deprovisioning { task_id: String }, Failed, } @@ -31,13 +32,17 @@ pub struct IndexerState { pub provisioned_state: ProvisionedState, } +// FIX `IndexerConfig` does not exist after an Indexer is deleted, and we need a way to +// construct the state key without it. But, this isn't ideal as we now have two places which +// define this key - we need to consolidate these somehow. impl IndexerState { - // FIX `IndexerConfig` does not exist after an Indexer is deleted, and we need a way to - // construct the state key without it. But, this isn't ideal as we now have two places which - // define this key - we need to consolidate these somehow. pub fn get_state_key(&self) -> String { format!("{}/{}:state", self.account_id, self.function_name) } + + pub fn get_redis_stream_key(&self) -> String { + format!("{}/{}:block_stream", self.account_id, self.function_name) + } } #[cfg(not(test))] @@ -105,6 +110,12 @@ impl IndexerStateManagerImpl { return Ok(serde_json::from_str(&raw_state)?); } + tracing::info!( + account_id = indexer_config.account_id.to_string(), + function_name = indexer_config.function_name.as_str(), + "Creating new state using default" + ); + Ok(self.get_default_state(indexer_config)) } @@ -134,10 +145,30 @@ impl IndexerStateManagerImpl { Ok(()) } - pub async fn set_provisioning(&self, indexer_config: &IndexerConfig) -> anyhow::Result<()> { + pub async fn set_deprovisioning( + &self, + indexer_state: &IndexerState, + task_id: String, + ) -> anyhow::Result<()> { + let mut state = indexer_state.clone(); + + state.provisioned_state = ProvisionedState::Deprovisioning { task_id }; + + self.redis_client + .set(state.get_state_key(), serde_json::to_string(&state)?) + .await?; + + Ok(()) + } + + pub async fn set_provisioning( + &self, + indexer_config: &IndexerConfig, + task_id: String, + ) -> anyhow::Result<()> { let mut indexer_state = self.get_state(indexer_config).await?; - indexer_state.provisioned_state = ProvisionedState::Provisioning; + indexer_state.provisioned_state = ProvisionedState::Provisioning { task_id }; self.set_state(indexer_config, indexer_state).await?; diff --git a/coordinator/src/redis.rs b/coordinator/src/redis.rs index 74f536c70..191b92f91 100644 --- a/coordinator/src/redis.rs +++ b/coordinator/src/redis.rs @@ -227,6 +227,10 @@ mockall::mock! { K: ToRedisArgs + Debug + Send + Sync + 'static, V: ToRedisArgs + Debug + Send + Sync + 'static; + pub async fn del(&self, key: K) -> anyhow::Result<()> + where + K: ToRedisArgs + Debug + Send + Sync + 'static; + pub async fn indexer_states_set_exists(&self) -> anyhow::Result; pub async fn sadd(&self, set: S, value: V) -> anyhow::Result<()> diff --git a/coordinator/src/synchroniser.rs b/coordinator/src/synchroniser.rs index 1156d89af..e464e241f 100644 --- a/coordinator/src/synchroniser.rs +++ b/coordinator/src/synchroniser.rs @@ -4,7 +4,7 @@ use tracing::instrument; use crate::{ handlers::{ block_streams::{BlockStreamsHandler, StreamInfo}, - data_layer::{DataLayerHandler, ProvisioningStatus}, + data_layer::{DataLayerHandler, TaskStatus}, executors::{ExecutorInfo, ExecutorsHandler}, }, indexer_config::IndexerConfig, @@ -82,17 +82,12 @@ impl<'a> Synchroniser<'a> { async fn sync_new_indexer(&self, config: &IndexerConfig) -> anyhow::Result<()> { tracing::info!("Starting data layer provisioning"); - self.data_layer_handler + let task_id = self + .data_layer_handler .start_provisioning_task(config) .await?; - self.state_manager - .set_provisioning(config) - .await - .map_err(|err| { - tracing::warn!(?err, "Failed to set provisioning state"); - err - })?; + self.state_manager.set_provisioning(config, task_id).await?; Ok(()) } @@ -211,11 +206,12 @@ impl<'a> Synchroniser<'a> { Ok(()) } - async fn handle_provisioning(&self, config: &IndexerConfig) -> anyhow::Result<()> { - let task_status_result = self - .data_layer_handler - .check_provisioning_task_status(config) - .await; + async fn ensure_provisioned( + &self, + config: &IndexerConfig, + task_id: String, + ) -> anyhow::Result<()> { + let task_status_result = self.data_layer_handler.get_task_status(task_id).await; if let Err(error) = task_status_result { tracing::warn!(?error, "Failed to check provisioning task status"); @@ -224,13 +220,13 @@ impl<'a> Synchroniser<'a> { }; let _ = match task_status_result.unwrap() { - ProvisioningStatus::Complete => { + TaskStatus::Complete => { tracing::info!("Data layer provisioning complete"); self.state_manager.set_provisioned(config).await } - ProvisioningStatus::Pending => Ok(()), + TaskStatus::Pending => Ok(()), _ => { - tracing::info!("Data layer provisioning failed"); + tracing::warn!("Data layer provisioning failed"); self.state_manager.set_provisioning_failure(config).await } } @@ -254,13 +250,16 @@ impl<'a> Synchroniser<'a> { executor: Option<&ExecutorInfo>, block_stream: Option<&StreamInfo>, ) -> anyhow::Result<()> { - match state.provisioned_state { - ProvisionedState::Unprovisioned | ProvisionedState::Provisioning => { - self.handle_provisioning(config).await?; + match &state.provisioned_state { + ProvisionedState::Provisioning { task_id } => { + self.ensure_provisioned(config, task_id.clone()).await?; return Ok(()); } ProvisionedState::Failed => return Ok(()), ProvisionedState::Provisioned => {} + ProvisionedState::Unprovisioned | ProvisionedState::Deprovisioning { .. } => { + anyhow::bail!("Provisioning task should have been started") + } } if !state.enabled { @@ -326,6 +325,38 @@ impl<'a> Synchroniser<'a> { .await?; } + if let ProvisionedState::Deprovisioning { task_id } = &state.provisioned_state { + match self + .data_layer_handler + .get_task_status(task_id.clone()) + .await? + { + TaskStatus::Complete => { + tracing::info!("Data layer deprovisioning complete"); + } + TaskStatus::Failed => { + tracing::info!("Data layer deprovisioning failed"); + } + TaskStatus::Unspecified => { + tracing::info!("Encountered unspecified deprovisioning task status"); + } + TaskStatus::Pending => return Ok(()), + } + } else { + let task_id = self + .data_layer_handler + .start_deprovisioning_task(state.account_id.clone(), state.function_name.clone()) + .await?; + + self.state_manager + .set_deprovisioning(state, task_id.clone()) + .await?; + + return Ok(()); + } + + self.redis_client.del(state.get_redis_stream_key()).await?; + self.state_manager.delete_state(state).await?; Ok(()) @@ -604,15 +635,15 @@ mod test { state_manager.expect_list().returning(|| Ok(vec![])); state_manager .expect_set_provisioning() - .with(eq(config.clone())) - .returning(|_| Ok(())) + .with(eq(config.clone()), eq("task_id".to_string())) + .returning(|_, _| Ok(())) .once(); let mut data_layer_handler = DataLayerHandler::default(); data_layer_handler .expect_start_provisioning_task() .with(eq(config)) - .returning(|_| Ok(ProvisioningStatus::Pending)) + .returning(|_| Ok("task_id".to_string())) .once(); let redis_client = RedisClient::default(); @@ -642,12 +673,16 @@ mod test { HashMap::from([(config.function_name.clone(), config.clone())]), )]); + let task_id = "task_id".to_string(); + let state = IndexerState { account_id: config.account_id.clone(), function_name: config.function_name.clone(), block_stream_synced_at: Some(config.get_registry_version()), enabled: true, - provisioned_state: ProvisionedState::Provisioning, + provisioned_state: ProvisionedState::Provisioning { + task_id: task_id.clone().to_string(), + }, }; let mut registry = Registry::default(); @@ -664,9 +699,9 @@ mod test { let mut data_layer_handler = DataLayerHandler::default(); data_layer_handler - .expect_check_provisioning_task_status() - .with(eq(config.clone())) - .returning(|_| Ok(ProvisioningStatus::Complete)); + .expect_get_task_status() + .with(eq(task_id)) + .returning(|_| Ok(TaskStatus::Complete)); let mut block_streams_handler = BlockStreamsHandler::default(); block_streams_handler.expect_start().never(); @@ -705,7 +740,9 @@ mod test { function_name: config.function_name.clone(), block_stream_synced_at: Some(config.get_registry_version()), enabled: true, - provisioned_state: ProvisionedState::Provisioning, + provisioned_state: ProvisionedState::Provisioning { + task_id: "task_id".to_string(), + }, }; let mut registry = Registry::default(); @@ -722,9 +759,9 @@ mod test { let mut data_layer_handler = DataLayerHandler::default(); data_layer_handler - .expect_check_provisioning_task_status() - .with(eq(config.clone())) - .returning(|_| Ok(ProvisioningStatus::Failed)); + .expect_get_task_status() + .with(eq("task_id".to_string())) + .returning(|_| Ok(TaskStatus::Failed)); let mut block_streams_handler = BlockStreamsHandler::default(); block_streams_handler.expect_start().never(); @@ -1204,14 +1241,16 @@ mod test { use super::*; #[tokio::test] - async fn stops_and_deletes() { + async fn stops_block_stream_and_executor() { let config = IndexerConfig::default(); let state = IndexerState { account_id: config.account_id.clone(), function_name: config.function_name.clone(), block_stream_synced_at: Some(config.get_registry_version()), enabled: false, - provisioned_state: ProvisionedState::Provisioned, + provisioned_state: ProvisionedState::Deprovisioning { + task_id: "task_id".to_string(), + }, }; let executor = ExecutorInfo { executor_id: "executor_id".to_string(), @@ -1242,15 +1281,86 @@ mod test { .once(); let mut state_manager = IndexerStateManager::default(); + state_manager.expect_delete_state().never(); + + let mut data_layer_handler = DataLayerHandler::default(); + data_layer_handler + .expect_get_task_status() + .with(eq("task_id".to_string())) + .returning(|_| Ok(TaskStatus::Pending)); + + let registry = Registry::default(); + let redis_client = RedisClient::default(); + + let synchroniser = Synchroniser::new( + &block_streams_handler, + &executors_handler, + &data_layer_handler, + ®istry, + &state_manager, + &redis_client, + ); + + synchroniser + .sync_deleted_indexer(&state, Some(&executor), Some(&block_stream)) + .await + .unwrap(); + } + + #[tokio::test] + async fn cleans_indexer_resources() { + let config = IndexerConfig::default(); + let provisioned_state = IndexerState { + account_id: config.account_id.clone(), + function_name: config.function_name.clone(), + block_stream_synced_at: Some(config.get_registry_version()), + enabled: false, + provisioned_state: ProvisionedState::Provisioned, + }; + let deprovisioning_state = IndexerState { + account_id: config.account_id.clone(), + function_name: config.function_name.clone(), + block_stream_synced_at: Some(config.get_registry_version()), + enabled: false, + provisioned_state: ProvisionedState::Deprovisioning { + task_id: "task_id".to_string(), + }, + }; + + let mut state_manager = IndexerStateManager::default(); + state_manager + .expect_set_deprovisioning() + .with(eq(provisioned_state.clone()), eq("task_id".to_string())) + .returning(|_, _| Ok(())); state_manager .expect_delete_state() - .with(eq(state.clone())) + .with(eq(deprovisioning_state.clone())) + .returning(|_| Ok(())) + .once(); + + let mut data_layer_handler = DataLayerHandler::default(); + data_layer_handler + .expect_start_deprovisioning_task() + .with( + eq(config.clone().account_id), + eq(config.clone().function_name), + ) + .returning(|_, _| Ok("task_id".to_string())); + data_layer_handler + .expect_get_task_status() + .with(eq("task_id".to_string())) + .returning(|_| Ok(TaskStatus::Complete)); + + let mut redis_client = RedisClient::default(); + redis_client + .expect_del::() + .with(eq(config.get_redis_stream_key())) .returning(|_| Ok(())) .once(); let registry = Registry::default(); - let redis_client = RedisClient::default(); - let data_layer_handler = DataLayerHandler::default(); + let block_streams_handler = BlockStreamsHandler::default(); + let executors_handler = ExecutorsHandler::default(); let synchroniser = Synchroniser::new( &block_streams_handler, @@ -1262,7 +1372,11 @@ mod test { ); synchroniser - .sync_deleted_indexer(&state, Some(&executor), Some(&block_stream)) + .sync_deleted_indexer(&provisioned_state, None, None) + .await + .unwrap(); + synchroniser + .sync_deleted_indexer(&deprovisioning_state, None, None) .await .unwrap(); } diff --git a/runner-client/examples/start_provisioning_task.rs b/runner-client/examples/start_provisioning_task.rs index 24a938d15..316d62c6e 100644 --- a/runner-client/examples/start_provisioning_task.rs +++ b/runner-client/examples/start_provisioning_task.rs @@ -9,8 +9,8 @@ async fn main() -> Result<(), Box> { let response = client .start_provisioning_task(Request::new(ProvisionRequest { - account_id: "morgs.near".to_string(), - function_name: "test2".to_string(), + account_id: "test.near".to_string(), + function_name: "data_layer_example".to_string(), schema: "create table blocks();".to_string(), })) .await?; diff --git a/runner-client/proto/data-layer.proto b/runner-client/proto/data-layer.proto index 909e3d28a..cd8fb2c25 100644 --- a/runner-client/proto/data-layer.proto +++ b/runner-client/proto/data-layer.proto @@ -4,10 +4,18 @@ package data_layer; service DataLayer { // Starts async provisioning task - rpc StartProvisioningTask (ProvisionRequest) returns (ProvisionResponse); + rpc StartProvisioningTask (ProvisionRequest) returns (StartTaskResponse); - // Checks the provisioning status - rpc CheckProvisioningTaskStatus (CheckProvisioningTaskStatusRequest) returns (ProvisionResponse); + // Start async deprovisioning task + rpc StartDeprovisioningTask (DeprovisionRequest) returns (StartTaskResponse); + + // Checks the status of provisioning/deprovisioning + rpc GetTaskStatus (GetTaskStatusRequest) returns (GetTaskStatusResponse); + +} + +message StartTaskResponse { + string task_id = 1; } message ProvisionRequest { @@ -16,18 +24,23 @@ message ProvisionRequest { string schema = 3; } -message CheckProvisioningTaskStatusRequest { +message DeprovisionRequest { string account_id = 1; string function_name = 2; } -enum ProvisioningStatus { + +message GetTaskStatusRequest { + string task_id = 1; +} + +enum TaskStatus { UNSPECIFIED = 0; PENDING = 1; COMPLETE = 2; FAILED = 3; } -message ProvisionResponse { - ProvisioningStatus status = 1; +message GetTaskStatusResponse { + TaskStatus status = 1; } diff --git a/runner/protos/data-layer.proto b/runner/protos/data-layer.proto index 909e3d28a..cd8fb2c25 100644 --- a/runner/protos/data-layer.proto +++ b/runner/protos/data-layer.proto @@ -4,10 +4,18 @@ package data_layer; service DataLayer { // Starts async provisioning task - rpc StartProvisioningTask (ProvisionRequest) returns (ProvisionResponse); + rpc StartProvisioningTask (ProvisionRequest) returns (StartTaskResponse); - // Checks the provisioning status - rpc CheckProvisioningTaskStatus (CheckProvisioningTaskStatusRequest) returns (ProvisionResponse); + // Start async deprovisioning task + rpc StartDeprovisioningTask (DeprovisionRequest) returns (StartTaskResponse); + + // Checks the status of provisioning/deprovisioning + rpc GetTaskStatus (GetTaskStatusRequest) returns (GetTaskStatusResponse); + +} + +message StartTaskResponse { + string task_id = 1; } message ProvisionRequest { @@ -16,18 +24,23 @@ message ProvisionRequest { string schema = 3; } -message CheckProvisioningTaskStatusRequest { +message DeprovisionRequest { string account_id = 1; string function_name = 2; } -enum ProvisioningStatus { + +message GetTaskStatusRequest { + string task_id = 1; +} + +enum TaskStatus { UNSPECIFIED = 0; PENDING = 1; COMPLETE = 2; FAILED = 3; } -message ProvisionResponse { - ProvisioningStatus status = 1; +message GetTaskStatusResponse { + TaskStatus status = 1; } diff --git a/runner/src/hasura-client/__snapshots__/hasura-client.test.ts.snap b/runner/src/hasura-client/__snapshots__/hasura-client.test.ts.snap index b1b518094..a4aeba4d4 100644 --- a/runner/src/hasura-client/__snapshots__/hasura-client.test.ts.snap +++ b/runner/src/hasura-client/__snapshots__/hasura-client.test.ts.snap @@ -212,6 +212,31 @@ exports[`HasuraClient creates a schema 1`] = ` ] `; +exports[`HasuraClient drops a datasource 1`] = ` +{ + "args": { + "cascade": true, + "name": "morgs_near", + }, + "type": "pg_drop_source", +} +`; + +exports[`HasuraClient drops a schema 1`] = ` +[ + [ + "mock-hasura-endpoint/v2/query", + { + "body": "{"type":"run_sql","args":{"sql":"DROP schema IF EXISTS schemaName CASCADE","read_only":false,"source":"dbName"}}", + "headers": { + "X-Hasura-Admin-Secret": "mock-hasura-admin-secret", + }, + "method": "POST", + }, + ], +] +`; + exports[`HasuraClient gets table names within a schema 1`] = ` { "args": { diff --git a/runner/src/hasura-client/hasura-client.test.ts b/runner/src/hasura-client/hasura-client.test.ts index 5c9e17be8..819530b9d 100644 --- a/runner/src/hasura-client/hasura-client.test.ts +++ b/runner/src/hasura-client/hasura-client.test.ts @@ -41,6 +41,20 @@ describe('HasuraClient', () => { expect(mockFetch.mock.calls).toMatchSnapshot(); }); + it('drops a schema', async () => { + const mockFetch = jest + .fn() + .mockResolvedValue({ + status: 200, + text: () => JSON.stringify({}) + }); + const client = new HasuraClient({ fetch: mockFetch as unknown as typeof fetch }, config); + + await client.dropSchema('dbName', 'schemaName'); + + expect(mockFetch.mock.calls).toMatchSnapshot(); + }); + it('checks if a schema exists within source', async () => { const mockFetch = jest .fn() @@ -172,6 +186,21 @@ describe('HasuraClient', () => { expect(JSON.parse(mockFetch.mock.calls[0][1].body)).toMatchSnapshot(); }); + it('drops a datasource', async () => { + const mockFetch = jest + .fn() + .mockResolvedValue({ + status: 200, + text: () => JSON.stringify({}) + }); + const client = new HasuraClient({ fetch: mockFetch as unknown as typeof fetch }, config); + + await client.dropDatasource('morgs_near'); + + expect(mockFetch.mock.calls[0][1].headers['X-Hasura-Admin-Secret']).toBe(config.adminSecret); + expect(JSON.parse(mockFetch.mock.calls[0][1].body)).toMatchSnapshot(); + }); + it('adds a datasource', async () => { const mockFetch = jest .fn() diff --git a/runner/src/hasura-client/hasura-client.ts b/runner/src/hasura-client/hasura-client.ts index 6efb3c986..8b50242f2 100644 --- a/runner/src/hasura-client/hasura-client.ts +++ b/runner/src/hasura-client/hasura-client.ts @@ -209,6 +209,13 @@ export default class HasuraClient { return result.length > 1; } + async dropSchema (source: string, schemaName: string): Promise { + return await this.executeSql( + `DROP schema IF EXISTS ${schemaName} CASCADE`, + { source, readOnly: false } + ); + } + async createSchema (source: string, schemaName: string): Promise { return await this.executeSql(`CREATE schema ${schemaName}`, { source, @@ -443,4 +450,11 @@ export default class HasuraClient { }, }); } + + async dropDatasource (databaseName: string): Promise { + return await this.executeMetadataRequest('pg_drop_source', { + name: databaseName, + cascade: true, + }); + } } diff --git a/runner/src/provisioner/provisioner.test.ts b/runner/src/provisioner/provisioner.test.ts index ea6cf0b21..c27016832 100644 --- a/runner/src/provisioner/provisioner.test.ts +++ b/runner/src/provisioner/provisioner.test.ts @@ -40,8 +40,10 @@ describe('Provisioner', () => { trackForeignKeyRelationships: jest.fn().mockReturnValueOnce(null), addPermissionsToTables: jest.fn().mockReturnValueOnce(null), addDatasource: jest.fn().mockReturnValueOnce(null), + dropDatasource: jest.fn().mockReturnValueOnce(null), executeSqlOnSchema: jest.fn().mockReturnValueOnce(null), createSchema: jest.fn().mockReturnValueOnce(null), + dropSchema: jest.fn().mockReturnValueOnce(null), doesSourceExist: jest.fn().mockReturnValueOnce(false), doesSchemaExist: jest.fn().mockReturnValueOnce(false), untrackTables: jest.fn().mockReturnValueOnce(null), @@ -71,6 +73,102 @@ describe('Provisioner', () => { indexerConfig = new IndexerConfig('', accountId, functionName, 0, '', databaseSchema, LogLevel.INFO); }); + describe('deprovision', () => { + it('removes schema level resources', async () => { + userPgClientQuery = jest.fn() + .mockResolvedValueOnce(null) // unschedule create partition job + .mockResolvedValueOnce(null) // unschedule delete partition job + .mockResolvedValueOnce({ rows: [{ schema_name: 'another_one' }] }); // list schemas + + await provisioner.deprovision(indexerConfig); + + expect(hasuraClient.dropSchema).toBeCalledWith(indexerConfig.databaseName(), indexerConfig.schemaName()); + expect(userPgClientQuery.mock.calls).toEqual([ + ["SELECT cron.unschedule('morgs_near_test_function_sys_logs_create_partition');"], + ["SELECT cron.unschedule('morgs_near_test_function_sys_logs_delete_partition');"], + ["SELECT schema_name FROM information_schema.schemata WHERE schema_owner = 'morgs_near'"], + ]); + }); + + it('removes database level resources', async () => { + userPgClientQuery = jest.fn() + .mockResolvedValueOnce(null) // unschedule create partition job + .mockResolvedValueOnce(null) // unschedule delete partition job + .mockResolvedValueOnce({ rows: [] }); // list schemas + + await provisioner.deprovision(indexerConfig); + + expect(hasuraClient.dropSchema).toBeCalledWith(indexerConfig.databaseName(), indexerConfig.schemaName()); + expect(userPgClientQuery.mock.calls).toEqual([ + ["SELECT cron.unschedule('morgs_near_test_function_sys_logs_create_partition');"], + ["SELECT cron.unschedule('morgs_near_test_function_sys_logs_delete_partition');"], + ["SELECT schema_name FROM information_schema.schemata WHERE schema_owner = 'morgs_near'"], + ]); + expect(hasuraClient.dropDatasource).toBeCalledWith(indexerConfig.databaseName()); + expect(adminPgClient.query).toBeCalledWith('DROP DATABASE IF EXISTS morgs_near (FORCE)'); + expect(cronPgClient.query).toBeCalledWith('REVOKE USAGE ON SCHEMA cron FROM morgs_near CASCADE'); + expect(cronPgClient.query).toBeCalledWith('REVOKE EXECUTE ON FUNCTION cron.schedule_in_database FROM morgs_near;'); + expect(adminPgClient.query).toBeCalledWith('DROP ROLE IF EXISTS morgs_near'); + }); + + it('handles revoke cron failures', async () => { + userPgClientQuery = jest.fn() + .mockResolvedValueOnce(null) // unschedule create partition job + .mockResolvedValueOnce(null) // unschedule delete partition job + .mockResolvedValueOnce({ rows: [] }); // list schemas + + cronPgClient.query = jest.fn() + .mockRejectedValue(new Error('failed revoke')); + + await expect(provisioner.deprovision(indexerConfig)).rejects.toThrow('Failed to deprovision: Failed to revoke cron access: failed revoke'); + }); + + it('handles drop role failures', async () => { + userPgClientQuery = jest.fn() + .mockResolvedValueOnce(null) // unschedule create partition job + .mockResolvedValueOnce(null) // unschedule delete partition job + .mockResolvedValueOnce({ rows: [] }); // list schemas + + adminPgClient.query = jest.fn() + .mockResolvedValueOnce(null) + .mockRejectedValue(new Error('failed to drop role')); + + await expect(provisioner.deprovision(indexerConfig)).rejects.toThrow('Failed to deprovision: Failed to drop role: failed to drop role'); + }); + + it('handles drop database failures', async () => { + userPgClientQuery = jest.fn() + .mockResolvedValueOnce(null) // unschedule create partition job + .mockResolvedValueOnce(null) // unschedule delete partition job + .mockResolvedValueOnce({ rows: [] }); // list schemas + + adminPgClient.query = jest.fn().mockRejectedValue(new Error('failed to drop db')); + + await expect(provisioner.deprovision(indexerConfig)).rejects.toThrow('Failed to deprovision: Failed to drop database: failed to drop db'); + }); + + it('handles drop datasource failures', async () => { + userPgClientQuery = jest.fn() + .mockResolvedValueOnce(null) // unschedule create partition job + .mockResolvedValueOnce(null) // unschedule delete partition job + .mockResolvedValueOnce({ rows: [] }); // list schemas + + hasuraClient.dropDatasource = jest.fn().mockRejectedValue(new Error('failed to drop datasource')); + + await expect(provisioner.deprovision(indexerConfig)).rejects.toThrow('Failed to deprovision: Failed to drop datasource: failed to drop'); + }); + + it('handles drop schema failures', async () => { + hasuraClient.dropSchema = jest.fn().mockRejectedValue(new Error('failed to drop schema')); + await expect(provisioner.deprovision(indexerConfig)).rejects.toThrow('Failed to deprovision: Failed to drop schema: failed to drop'); + }); + + it('handles remove log job failures', async () => { + userPgClientQuery = jest.fn().mockResolvedValueOnce(null).mockRejectedValueOnce(new Error('failed to remove jobs')); + await expect(provisioner.deprovision(indexerConfig)).rejects.toThrow('Failed to deprovision: Failed to unschedule log partition jobs: failed to remove jobs'); + }); + }); + describe('isUserApiProvisioned', () => { it('returns false if datasource doesnt exists', async () => { hasuraClient.doesSourceExist = jest.fn().mockReturnValueOnce(false); diff --git a/runner/src/provisioner/provisioner.ts b/runner/src/provisioner/provisioner.ts index 04ab238e5..f23d8ca97 100644 --- a/runner/src/provisioner/provisioner.ts +++ b/runner/src/provisioner/provisioner.ts @@ -234,6 +234,100 @@ export default class Provisioner { return await wrapError(async () => await this.hasuraClient.addDatasource(userName, password, databaseName), 'Failed to add datasource'); } + async dropSchemaAndMetadata (databaseName: string, schemaName: string): Promise { + await wrapError(async () => { + // Need to drop via Hasura to ensure metadata is cleaned up + await this.hasuraClient.dropSchema(databaseName, schemaName); + }, 'Failed to drop schema'); + } + + async removeLogPartitionJobs (userName: string, schemaName: string): Promise { + await wrapError( + async () => { + const userCronConnectionParameters = { + ...(await this.getPostgresConnectionParameters(userName)), + database: this.config.cronDatabase + }; + const userCronPgClient = new this.PgClient(userCronConnectionParameters); + + await userCronPgClient.query( + this.pgFormat( + "SELECT cron.unschedule('%I_sys_logs_create_partition');", + schemaName, + ) + ); + await userCronPgClient.query( + this.pgFormat( + "SELECT cron.unschedule('%I_sys_logs_delete_partition');", + schemaName, + ) + ); + + await userCronPgClient.end(); + }, + 'Failed to unschedule log partition jobs' + ); + } + + async listUserOwnedSchemas (userName: string): Promise { + return await wrapError(async () => { + const userDbConnectionParameters = await this.getPostgresConnectionParameters(userName); + const userPgClient = new this.PgClient(userDbConnectionParameters); + + const result = await userPgClient.query( + this.pgFormat('SELECT schema_name FROM information_schema.schemata WHERE schema_owner = %L', userName) + ); + + await userPgClient.end(); + + return result.rows.map((row) => row.schema_name); + }, 'Failed to list schemas'); + } + + async dropDatabase (databaseName: string): Promise { + await wrapError(async () => { + await this.adminDefaultPgClient.query(this.pgFormat('DROP DATABASE IF EXISTS %I (FORCE)', databaseName)); + }, 'Failed to drop database'); + } + + async dropDatasource (databaseName: string): Promise { + await wrapError(async () => { + await this.hasuraClient.dropDatasource(databaseName); + }, 'Failed to drop datasource'); + } + + async dropRole (userName: string): Promise { + await wrapError(async () => { + await this.adminDefaultPgClient.query(this.pgFormat('DROP ROLE IF EXISTS %I', userName)); + }, 'Failed to drop role'); + } + + async revokeCronAccess (userName: string): Promise { + await wrapError( + async () => { + await this.adminCronPgClient.query(this.pgFormat('REVOKE USAGE ON SCHEMA cron FROM %I CASCADE', userName)); + await this.adminCronPgClient.query(this.pgFormat('REVOKE EXECUTE ON FUNCTION cron.schedule_in_database FROM %I;', userName)); + }, + 'Failed to revoke cron access' + ); + } + + public async deprovision (config: ProvisioningConfig): Promise { + await wrapError(async () => { + await this.dropSchemaAndMetadata(config.userName(), config.schemaName()); + await this.removeLogPartitionJobs(config.userName(), config.schemaName()); + + const schemas = await this.listUserOwnedSchemas(config.userName()); + + if (schemas.length === 0) { + await this.dropDatasource(config.databaseName()); + await this.dropDatabase(config.databaseName()); + await this.revokeCronAccess(config.userName()); + await this.dropRole(config.userName()); + } + }, 'Failed to deprovision'); + } + async provisionUserApi (indexerConfig: ProvisioningConfig): Promise { // replace any with actual type const userName = indexerConfig.userName(); const databaseName = indexerConfig.databaseName(); diff --git a/runner/src/server/services/data-layer/data-layer-service.test.ts b/runner/src/server/services/data-layer/data-layer-service.test.ts index 15fc29590..bc4904514 100644 --- a/runner/src/server/services/data-layer/data-layer-service.test.ts +++ b/runner/src/server/services/data-layer/data-layer-service.test.ts @@ -1,14 +1,14 @@ import { type ServerUnaryCall, status } from '@grpc/grpc-js'; -import { createDataLayerService, type ProvisioningTask } from './data-layer-service'; -import { ProvisioningStatus } from '../../../generated/data_layer/ProvisioningStatus'; +import { createDataLayerService, type AsyncTask } from './data-layer-service'; +import { TaskStatus } from '../../../generated/data_layer/TaskStatus'; import type Provisioner from '../../../provisioner'; describe('DataLayerService', () => { - describe('CheckProvisioningTaskStatus', () => { + describe('GetTaskStatus', () => { it('should return NOT_FOUND if the task does not exist', (done) => { const call = { - request: { accountId: 'testAccount', functionName: 'testFunction' } + request: { taskId: 'id' } } as unknown as ServerUnaryCall; const callback = (error: any): void => { @@ -17,122 +17,122 @@ describe('DataLayerService', () => { done(); }; - createDataLayerService().CheckProvisioningTaskStatus(call, callback); + createDataLayerService().GetTaskStatus(call, callback); }); it('should return PENDING if the task is pending', (done) => { const tasks = { - 'testAccount:testFunction': { pending: true, completed: false, failed: false } as unknown as ProvisioningTask + id: { pending: true, completed: false, failed: false } as unknown as AsyncTask }; const call = { - request: { accountId: 'testAccount', functionName: 'testFunction' } + request: { taskId: 'id' } } as unknown as ServerUnaryCall; const callback = (_error: any, response: any): void => { - expect(response.status).toBe(ProvisioningStatus.PENDING); + expect(response.status).toBe(TaskStatus.PENDING); done(); }; - createDataLayerService(undefined, tasks).CheckProvisioningTaskStatus(call, callback); + createDataLayerService(undefined, tasks).GetTaskStatus(call, callback); }); it('should return COMPLETE if the task is completed', (done) => { const tasks = { - 'testAccount:testFunction': { pending: false, completed: true, failed: false } as unknown as ProvisioningTask + id: { pending: false, completed: true, failed: false } as unknown as AsyncTask }; const call = { - request: { accountId: 'testAccount', functionName: 'testFunction' } + request: { taskId: 'id' } } as unknown as ServerUnaryCall; const callback = (_error: any, response: any): void => { - expect(response.status).toBe(ProvisioningStatus.COMPLETE); + expect(response.status).toBe(TaskStatus.COMPLETE); done(); }; - createDataLayerService(undefined, tasks).CheckProvisioningTaskStatus(call, callback); + createDataLayerService(undefined, tasks).GetTaskStatus(call, callback); }); it('should return FAILED if the task has failed', (done) => { const tasks = { - 'testAccount:testFunction': { pending: false, completed: false, failed: true } as unknown as ProvisioningTask + id: { pending: false, completed: false, failed: true } as unknown as AsyncTask }; const call = { - request: { accountId: 'testAccount', functionName: 'testFunction' } + request: { taskId: 'id' } } as unknown as ServerUnaryCall; const callback = (_error: any, response: any): void => { - expect(response.status).toBe(ProvisioningStatus.FAILED); + expect(response.status).toBe(TaskStatus.FAILED); done(); }; - createDataLayerService(undefined, tasks).CheckProvisioningTaskStatus(call, callback); + createDataLayerService(undefined, tasks).GetTaskStatus(call, callback); }); }); - describe('Provision', () => { - it('should return ALREADY_EXISTS if the task exists', (done) => { - const tasks = { - 'testAccount:testFunction': { pending: true, completed: false, failed: false } as unknown as ProvisioningTask + describe('StartProvisioningTask', () => { + it('should return the current task if it exists', (done) => { + const tasks: Record = { + '8291150845651941809f8f3db28eeb7fd8acdfeb422cb07c10178020070836b8': { pending: false, completed: true, failed: false } as unknown as AsyncTask }; const call = { request: { accountId: 'testAccount', functionName: 'testFunction', schema: 'schema' } } as unknown as ServerUnaryCall; - const callback = (error: any): void => { - expect(error.code).toBe(status.ALREADY_EXISTS); - expect(error.details).toBe('Provisioning task already exists'); + const callback = (_error: any, response: any): void => { + expect(tasks[response.taskId]).toBeDefined(); + expect(tasks[response.taskId].completed).toBe(true); done(); }; createDataLayerService(undefined, tasks).StartProvisioningTask(call, callback); }); - it('should return Complete if the task has already completed', (done) => { + it('should start a new provisioning task', (done) => { const tasks: Record = {}; const provisioner = { - fetchUserApiProvisioningStatus: jest.fn().mockResolvedValue(true) + provisionUserApi: jest.fn().mockResolvedValue(null) } as unknown as Provisioner; const call = { request: { accountId: 'testAccount', functionName: 'testFunction', schema: 'testSchema' } } as unknown as ServerUnaryCall; const callback = (_error: any, response: any): void => { - expect(response.status).toBe(ProvisioningStatus.COMPLETE); + expect(tasks[response.taskId]).toBeDefined(); + expect(tasks[response.taskId].pending).toBe(true); done(); }; createDataLayerService(provisioner, tasks).StartProvisioningTask(call, callback); }); + }); - it('should start a new provisioning task and return PENDING', (done) => { - const tasks: Record = {}; - const provisioner = { - fetchUserApiProvisioningStatus: jest.fn().mockResolvedValue(false), - provisionUserApi: jest.fn().mockResolvedValue(null) - } as unknown as Provisioner; + describe('StartDeprovisioningTask', () => { + it('should return ALREADY_EXISTS if the task exists', (done) => { + const tasks = { + f92a9f97d2609849e6837b483d8210c7b308c6f615a691449087ec00db1eef06: { pending: true, completed: false, failed: false } as unknown as AsyncTask + }; const call = { - request: { accountId: 'testAccount', functionName: 'testFunction', schema: 'testSchema' } + request: { accountId: 'testAccount', functionName: 'testFunction', schema: 'schema' } } as unknown as ServerUnaryCall; - const callback = (_error: any, response: any): void => { - expect(response.status).toBe(ProvisioningStatus.PENDING); - expect(tasks['testAccount:testFunction']).toBeDefined(); - expect(tasks['testAccount:testFunction'].pending).toBe(true); + const callback = (error: any): void => { + expect(error.code).toBe(status.ALREADY_EXISTS); + expect(error.details).toBe('Deprovisioning task already exists'); done(); }; - createDataLayerService(provisioner, tasks).StartProvisioningTask(call, callback); + createDataLayerService(undefined, tasks).StartDeprovisioningTask(call, callback); }); - it('should return INTERNAL error if checking provisioning status fails', (done) => { + it('should start a new deprovisioning task', (done) => { const tasks: Record = {}; const provisioner = { - fetchUserApiProvisioningStatus: jest.fn().mockRejectedValue(new Error('boom')) + deprovision: jest.fn().mockResolvedValue(null) } as unknown as Provisioner; const call = { request: { accountId: 'testAccount', functionName: 'testFunction', schema: 'testSchema' } } as unknown as ServerUnaryCall; - const callback = (error: any): void => { - expect(error.code).toBe(status.INTERNAL); - expect(error.details).toBe('boom'); + const callback = (_error: any, response: any): void => { + expect(tasks[response.taskId]).toBeDefined(); + expect(tasks[response.taskId].pending).toBe(true); done(); }; - createDataLayerService(provisioner, tasks).StartProvisioningTask(call, callback); + createDataLayerService(provisioner, tasks).StartDeprovisioningTask(call, callback); }); }); }); diff --git a/runner/src/server/services/data-layer/data-layer-service.ts b/runner/src/server/services/data-layer/data-layer-service.ts index 18312568f..3b3d46ece 100644 --- a/runner/src/server/services/data-layer/data-layer-service.ts +++ b/runner/src/server/services/data-layer/data-layer-service.ts @@ -1,21 +1,27 @@ +import crypto from 'crypto'; + import { type ServerUnaryCall, type sendUnaryData, status, StatusBuilder } from '@grpc/grpc-js'; import Provisioner from '../../../provisioner'; import { ProvisioningConfig } from '../../../indexer-config/indexer-config'; import parentLogger from '../../../logger'; -import { type CheckProvisioningTaskStatusRequest__Output } from '../../../generated/data_layer/CheckProvisioningTaskStatusRequest'; +import { type GetTaskStatusRequest__Output } from '../../../generated/data_layer/GetTaskStatusRequest'; +import { type GetTaskStatusResponse } from '../../../generated/data_layer/GetTaskStatusResponse'; import { type DataLayerHandlers } from '../../../generated/data_layer/DataLayer'; +import { type StartTaskResponse } from '../../../generated/data_layer/StartTaskResponse'; import { type ProvisionRequest__Output } from '../../../generated/data_layer/ProvisionRequest'; -import { type ProvisionResponse } from '../../../generated/data_layer/ProvisionResponse'; -import { ProvisioningStatus } from '../../../generated/data_layer/ProvisioningStatus'; +import { type DeprovisionRequest__Output } from '../../../generated/data_layer/DeprovisionRequest'; +import { TaskStatus } from '../../../generated/data_layer/TaskStatus'; -export class ProvisioningTask { +export class AsyncTask { public failed: boolean; public pending: boolean; public completed: boolean; - constructor (public readonly promise: Promise) { + constructor ( + public readonly promise: Promise + ) { promise.then(() => { this.completed = true; }).catch((error) => { @@ -31,19 +37,36 @@ export class ProvisioningTask { } } -type ProvisioningTasks = Record; +type AsyncTasks = Record; + +enum TaskType { + PROVISION = 'PROVISION', + DEPROVISION = 'DEPROVISION' +} + +const hash = (...args: string[]): string => { + const hash = crypto.createHash('sha256'); + hash.update(args.join(':')); + return hash.digest('hex'); +}; + +const createLogger = (config: ProvisioningConfig): typeof parentLogger => { + const logger = parentLogger.child({ + accountId: config.accountId, + functionName: config.functionName, + service: 'DataLayerService' + }); -const generateTaskId = (accountId: string, functionName: string): string => `${accountId}:${functionName}`; + return logger; +}; export function createDataLayerService ( provisioner: Provisioner = new Provisioner(), - tasks: ProvisioningTasks = {} + tasks: AsyncTasks = {} ): DataLayerHandlers { return { - CheckProvisioningTaskStatus (call: ServerUnaryCall, callback: sendUnaryData): void { - const { accountId, functionName } = call.request; - - const task = tasks[generateTaskId(accountId, functionName)]; + GetTaskStatus (call: ServerUnaryCall, callback: sendUnaryData): void { + const task = tasks[call.request.taskId]; if (!task) { const notFound = new StatusBuilder() @@ -56,70 +79,78 @@ export function createDataLayerService ( } if (task.completed) { - callback(null, { status: ProvisioningStatus.COMPLETE }); + callback(null, { status: TaskStatus.COMPLETE }); return; } if (task.failed) { - callback(null, { status: ProvisioningStatus.FAILED }); + callback(null, { status: TaskStatus.FAILED }); return; } - callback(null, { status: ProvisioningStatus.PENDING }); + callback(null, { status: TaskStatus.PENDING }); }, - StartProvisioningTask (call: ServerUnaryCall, callback: sendUnaryData): void { + StartProvisioningTask (call: ServerUnaryCall, callback: sendUnaryData): void { const { accountId, functionName, schema } = call.request; const provisioningConfig = new ProvisioningConfig(accountId, functionName, schema); - const logger = parentLogger.child({ - service: 'DataLayerService', - accountId: provisioningConfig.accountId, - functionName: provisioningConfig.functionName, - }); + const logger = createLogger(provisioningConfig); + + const taskId = hash(accountId, functionName, schema, TaskType.PROVISION); + + const task = tasks[taskId]; + + if (task) { + callback(null, { taskId }); + + return; + }; + + logger.info(`Starting provisioning task: ${taskId}`); + + tasks[taskId] = new AsyncTask( + provisioner + .provisionUserApi(provisioningConfig) + .then(() => { + logger.info('Successfully provisioned Data Layer'); + }) + .catch((err) => { + logger.error('Failed to provision Data Layer', err); + throw err; + }) + ); + + callback(null, { taskId }); + }, + + StartDeprovisioningTask (call: ServerUnaryCall, callback: sendUnaryData): void { + const { accountId, functionName } = call.request; - const task = tasks[generateTaskId(accountId, functionName)]; + const provisioningConfig = new ProvisioningConfig(accountId, functionName, 'todo'); + + const logger = createLogger(provisioningConfig); + + const taskId = hash(accountId, functionName, TaskType.DEPROVISION); + + const task = tasks[taskId]; if (task) { const exists = new StatusBuilder() .withCode(status.ALREADY_EXISTS) - .withDetails('Provisioning task already exists') + .withDetails('Deprovisioning task already exists') .build(); callback(exists); return; }; - provisioner.fetchUserApiProvisioningStatus(provisioningConfig).then((isProvisioned) => { - if (isProvisioned) { - callback(null, { status: ProvisioningStatus.COMPLETE }); - - return; - } - - logger.info('Provisioning Data Layer'); - - tasks[generateTaskId(accountId, functionName)] = new ProvisioningTask( - provisioner - .provisionUserApi(provisioningConfig) - .then(() => { - logger.info('Successfully provisioned Data Layer'); - }) - .catch((err) => { - logger.error('Failed to provision Data Layer', err); - throw err; - }) - ); - - callback(null, { status: ProvisioningStatus.PENDING }); - }).catch((error) => { - const internalError = new StatusBuilder() - .withCode(status.INTERNAL) - .withDetails(error.message) - .build(); - callback(internalError); - }); + logger.info(`Starting deprovisioning task: ${taskId}`); + + tasks[taskId] = new AsyncTask(provisioner.deprovision(provisioningConfig)); + + callback(null, { taskId }); } }; } diff --git a/runner/tests/integration.test.ts b/runner/tests/integration.test.ts index dda4511b8..3a74fbc8f 100644 --- a/runner/tests/integration.test.ts +++ b/runner/tests/integration.test.ts @@ -245,6 +245,52 @@ describe('Indexer integration', () => { const { morgs_near_test_context_db_indexer_storage: totalRows }: any = await graphqlClient.request(queryAllRows); expect(totalRows.length).toEqual(3); // Two inserts, and the overwritten upsert }); + + it('can provision and deprovision', async () => { + const testConfig1 = new IndexerConfig( + 'test:stream', + 'provisioning.near', // must be unique to prevent conflicts with other tests + 'test-provisioning1', + 0, + '', + 'CREATE TABLE blocks (height numeric)', + LogLevel.INFO + ); + + const testConfig2 = new IndexerConfig( + 'test:stream', + 'provisioning.near', // must be unique to prevent conflicts with other tests + 'test-provisioning2', + 0, + '', + 'CREATE TABLE blocks (height numeric)', + LogLevel.INFO + ); + + await provisioner.provisionUserApi(testConfig1); + await provisioner.provisionUserApi(testConfig2); + + const params = await provisioner.getPostgresConnectionParameters(testConfig1.userName()); + const userPgClient = new PgClient(params); + + await expect(pgClient.query('SELECT 1 FROM pg_database WHERE datname = $1', ['provisioning_near']).then(({ rows }) => rows)).resolves.toHaveLength(1); + await expect(userPgClient.query('SELECT schema_name FROM information_schema.schemata WHERE schema_name like $1', ['provisioning_near_test_provisioning%']).then(({ rows }) => rows)).resolves.toHaveLength(2); + await expect(pgClient.query('SELECT * FROM cron.job WHERE jobname like $1', ['provisioning_near_test_provisioning%']).then(({ rows }) => rows)).resolves.toHaveLength(4); + await expect(hasuraClient.doesSourceExist(testConfig1.databaseName())).resolves.toBe(true); + + await provisioner.deprovision(testConfig1); + + await expect(pgClient.query('SELECT 1 FROM pg_database WHERE datname = $1', ['provisioning_near']).then(({ rows }) => rows)).resolves.toHaveLength(1); + await expect(userPgClient.query('SELECT schema_name FROM information_schema.schemata WHERE schema_name like $1', ['provisioning_near_test_provisioning%']).then(({ rows }) => rows)).resolves.toHaveLength(1); + await expect(pgClient.query('SELECT * FROM cron.job WHERE jobname like $1', ['provisioning_near_test_provisioning%']).then(({ rows }) => rows)).resolves.toHaveLength(2); + await expect(hasuraClient.doesSourceExist(testConfig1.databaseName())).resolves.toBe(true); + + await provisioner.deprovision(testConfig2); + + await expect(pgClient.query('SELECT 1 FROM pg_database WHERE datname = $1', ['provisioning_near']).then(({ rows }) => rows)).resolves.toHaveLength(0); + await expect(pgClient.query('SELECT * FROM cron.job WHERE jobname like $1', ['provisioning_near_test_provisioning%']).then(({ rows }) => rows)).resolves.toHaveLength(0); + await expect(hasuraClient.doesSourceExist(testConfig1.databaseName())).resolves.toBe(false); + }); }); async function indexerLogsQuery (indexerSchemaName: string, graphqlClient: GraphQLClient): Promise { From f6ce3e56248156c4e7dfd7cd9343cdeba72417e9 Mon Sep 17 00:00:00 2001 From: Morgan McCauley Date: Thu, 20 Jun 2024 20:53:22 +1200 Subject: [PATCH 05/10] chore: Log deprovisioning success/failures (#818) --- .../server/services/data-layer/data-layer-service.ts | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/runner/src/server/services/data-layer/data-layer-service.ts b/runner/src/server/services/data-layer/data-layer-service.ts index 3b3d46ece..f82cbc1eb 100644 --- a/runner/src/server/services/data-layer/data-layer-service.ts +++ b/runner/src/server/services/data-layer/data-layer-service.ts @@ -148,7 +148,17 @@ export function createDataLayerService ( logger.info(`Starting deprovisioning task: ${taskId}`); - tasks[taskId] = new AsyncTask(provisioner.deprovision(provisioningConfig)); + tasks[taskId] = new AsyncTask( + provisioner + .deprovision(provisioningConfig) + .then(() => { + logger.info('Successfully deprovisioned Data Layer'); + }) + .catch((err) => { + logger.error('Failed to deprovision Data Layer', err); + throw err; + }) + ); callback(null, { taskId }); } From 8f7326bed30d0ab45d801db6028f9a3a350f88de Mon Sep 17 00:00:00 2001 From: Kevin Zhang <42101107+Kevin101Zhang@users.noreply.github.com> Date: Fri, 21 Jun 2024 00:23:19 -0400 Subject: [PATCH 06/10] fix: added polling to latest indexer block height (#823) --- frontend/src/components/Logs/LogsMenu.tsx | 29 +++++++++++++---------- 1 file changed, 16 insertions(+), 13 deletions(-) diff --git a/frontend/src/components/Logs/LogsMenu.tsx b/frontend/src/components/Logs/LogsMenu.tsx index 04539ffe7..5fe102cbe 100644 --- a/frontend/src/components/Logs/LogsMenu.tsx +++ b/frontend/src/components/Logs/LogsMenu.tsx @@ -40,6 +40,7 @@ const LogsMenu: React.FC = ({ `; const { loading, error, data, refetch } = useQuery(GET_METADATA, { + pollInterval: 1000, context: { headers: { 'x-hasura-role': hasuraRole, @@ -49,30 +50,32 @@ const LogsMenu: React.FC = ({ const [blockHeight, setBlockHeight] = useState(null); const [status, setStatus] = useState(null); - const [attributeMap, setAttributeMap] = useState>(new Map()); useEffect(() => { - if (!loading && data) { + if (data) { const newAttributeMap = new Map(data[queryName].map((item: any) => [item.attribute, item.value])); - setAttributeMap(newAttributeMap); - } - }, [data, queryName, loading]); - useEffect(() => { - if (attributeMap.has('LAST_PROCESSED_BLOCK_HEIGHT')) { - setBlockHeight(attributeMap.get('LAST_PROCESSED_BLOCK_HEIGHT') ?? 'N/A'); - } - if (attributeMap.has('STATUS')) { - setStatus(attributeMap.get('STATUS') ?? 'UNKNOWN'); + if (newAttributeMap.has('LAST_PROCESSED_BLOCK_HEIGHT')) { + setBlockHeight(newAttributeMap.get('LAST_PROCESSED_BLOCK_HEIGHT') ?? 'N/A'); + } + if (newAttributeMap.has('STATUS')) { + setStatus(newAttributeMap.get('STATUS') ?? 'UNKNOWN'); + } } - }, [attributeMap]); + }, [data, queryName]); const handleReload = async (): Promise => { try { const { data: refetchedData } = await refetch(); if (refetchedData) { const newAttributeMap = new Map(refetchedData[queryName].map((item: any) => [item.attribute, item.value])); - setAttributeMap(newAttributeMap); + + if (newAttributeMap.has('LAST_PROCESSED_BLOCK_HEIGHT')) { + setBlockHeight(newAttributeMap.get('LAST_PROCESSED_BLOCK_HEIGHT') ?? 'N/A'); + } + if (newAttributeMap.has('STATUS')) { + setStatus(newAttributeMap.get('STATUS') ?? 'UNKNOWN'); + } } reloadDataProp(); } catch (error) { From 7303f7a3484e356aa36e4985e991e1aa87b91b01 Mon Sep 17 00:00:00 2001 From: Morgan McCauley Date: Fri, 21 Jun 2024 21:08:30 +1200 Subject: [PATCH 07/10] fix: Allow re-provisioning of recently de-provisioned resources (#825) Provisioning a recently de-provisioned Data Layer would silently fail. This is due to the fact that de/provisioning tasks are stored in-memory, keyed by a hash of the config. So if a provisioning task was recently completed, attempting to re-provision would return that same task. This PR keys by random UUIDs, instead of hashes, so we can trigger multiple provisioning jobs for a given Indexer/Data Layer, allowing for the Provision > De-provision > Provision flow. To protect against re-provisioning an _existing_ Data Layer, we only start the task after verifying it doesn't already exist. Also removed cache from `Provisioner` to ensure we are getting accurate results. --- runner/src/provisioner/provisioner.test.ts | 10 +-- runner/src/provisioner/provisioner.ts | 24 +---- .../data-layer/data-layer-service.test.ts | 36 +++----- .../services/data-layer/data-layer-service.ts | 89 +++++++++---------- 4 files changed, 57 insertions(+), 102 deletions(-) diff --git a/runner/src/provisioner/provisioner.test.ts b/runner/src/provisioner/provisioner.test.ts index c27016832..90d932182 100644 --- a/runner/src/provisioner/provisioner.test.ts +++ b/runner/src/provisioner/provisioner.test.ts @@ -173,24 +173,21 @@ describe('Provisioner', () => { it('returns false if datasource doesnt exists', async () => { hasuraClient.doesSourceExist = jest.fn().mockReturnValueOnce(false); - await expect(provisioner.fetchUserApiProvisioningStatus(indexerConfig)).resolves.toBe(false); - expect(provisioner.isUserApiProvisioned(indexerConfig.accountId, indexerConfig.functionName)).toBe(false); + await expect(provisioner.isProvisioned(indexerConfig)).resolves.toBe(false); }); it('returns false if datasource and schema dont exists', async () => { hasuraClient.doesSourceExist = jest.fn().mockReturnValueOnce(false); hasuraClient.doesSchemaExist = jest.fn().mockReturnValueOnce(false); - await expect(provisioner.fetchUserApiProvisioningStatus(indexerConfig)).resolves.toBe(false); - expect(provisioner.isUserApiProvisioned(indexerConfig.accountId, indexerConfig.functionName)).toBe(false); + await expect(provisioner.isProvisioned(indexerConfig)).resolves.toBe(false); }); it('returns true if datasource and schema exists', async () => { hasuraClient.doesSourceExist = jest.fn().mockReturnValueOnce(true); hasuraClient.doesSchemaExist = jest.fn().mockReturnValueOnce(true); - await expect(provisioner.fetchUserApiProvisioningStatus(indexerConfig)).resolves.toBe(true); - expect(provisioner.isUserApiProvisioned(indexerConfig.accountId, indexerConfig.functionName)).toBe(true); + await expect(provisioner.isProvisioned(indexerConfig)).resolves.toBe(true); }); }); @@ -233,7 +230,6 @@ describe('Provisioner', () => { 'delete' ] ); - expect(provisioner.isUserApiProvisioned(accountId, functionName)).toBe(true); }); it('skips provisioning the datasource if it already exists', async () => { diff --git a/runner/src/provisioner/provisioner.ts b/runner/src/provisioner/provisioner.ts index f23d8ca97..5533c677e 100644 --- a/runner/src/provisioner/provisioner.ts +++ b/runner/src/provisioner/provisioner.ts @@ -49,7 +49,6 @@ const defaultConfig: Config = { export default class Provisioner { tracer: Tracer = trace.getTracer('queryapi-runner-provisioner'); - #hasBeenProvisioned: Record> = {}; constructor ( private readonly hasuraClient: HasuraClient = new HasuraClient(), @@ -70,17 +69,6 @@ export default class Provisioner { .replace(/\//g, '0'); } - isUserApiProvisioned (accountId: string, functionName: string): boolean { - const accountIndexers = this.#hasBeenProvisioned[accountId]; - if (!accountIndexers) { return false; } - return accountIndexers[functionName]; - } - - private setProvisioned (accountId: string, functionName: string): void { - this.#hasBeenProvisioned[accountId] ??= {}; - this.#hasBeenProvisioned[accountId][functionName] = true; - } - async createDatabase (name: string): Promise { await this.adminDefaultPgClient.query(this.pgFormat('CREATE DATABASE %I', name)); } @@ -156,12 +144,8 @@ export default class Provisioner { ); } - async fetchUserApiProvisioningStatus (indexerConfig: ProvisioningConfig): Promise { + async isProvisioned (indexerConfig: ProvisioningConfig): Promise { const checkProvisioningSpan = this.tracer.startSpan('Check if indexer is provisioned'); - if (this.isUserApiProvisioned(indexerConfig.accountId, indexerConfig.functionName)) { - checkProvisioningSpan.end(); - return true; - } const databaseName = indexerConfig.databaseName(); const schemaName = indexerConfig.schemaName(); @@ -172,10 +156,9 @@ export default class Provisioner { } const schemaExists = await this.hasuraClient.doesSchemaExist(databaseName, schemaName); - if (schemaExists) { - this.setProvisioned(indexerConfig.accountId, indexerConfig.functionName); - } + checkProvisioningSpan.end(); + return schemaExists; } @@ -356,7 +339,6 @@ export default class Provisioner { await this.trackForeignKeyRelationships(schemaName, databaseName); await this.addPermissionsToTables(indexerConfig, updatedTableNames, ['select', 'insert', 'update', 'delete']); - this.setProvisioned(indexerConfig.accountId, indexerConfig.functionName); }, 'Failed to provision endpoint' ); diff --git a/runner/src/server/services/data-layer/data-layer-service.test.ts b/runner/src/server/services/data-layer/data-layer-service.test.ts index bc4904514..88541de69 100644 --- a/runner/src/server/services/data-layer/data-layer-service.test.ts +++ b/runner/src/server/services/data-layer/data-layer-service.test.ts @@ -67,25 +67,27 @@ describe('DataLayerService', () => { }); describe('StartProvisioningTask', () => { - it('should return the current task if it exists', (done) => { - const tasks: Record = { - '8291150845651941809f8f3db28eeb7fd8acdfeb422cb07c10178020070836b8': { pending: false, completed: true, failed: false } as unknown as AsyncTask - }; + it('returns FAILED_PRECONDITION if already provisioned', (done) => { + const provisioner = { + isProvisioned: jest.fn().mockResolvedValue(true) + } as unknown as Provisioner; + const tasks = {}; const call = { - request: { accountId: 'testAccount', functionName: 'testFunction', schema: 'schema' } + request: { accountId: 'testAccount', functionName: 'testFunction', schema: 'testSchema' } } as unknown as ServerUnaryCall; - const callback = (_error: any, response: any): void => { - expect(tasks[response.taskId]).toBeDefined(); - expect(tasks[response.taskId].completed).toBe(true); + const callback = (error: any): void => { + expect(error.code).toBe(status.FAILED_PRECONDITION); + expect(error.details).toBe('Data Layer is already provisioned'); done(); }; - createDataLayerService(undefined, tasks).StartProvisioningTask(call, callback); + createDataLayerService(provisioner, tasks).StartProvisioningTask(call, callback); }); it('should start a new provisioning task', (done) => { const tasks: Record = {}; const provisioner = { + isProvisioned: jest.fn().mockResolvedValue(false), provisionUserApi: jest.fn().mockResolvedValue(null) } as unknown as Provisioner; const call = { @@ -102,22 +104,6 @@ describe('DataLayerService', () => { }); describe('StartDeprovisioningTask', () => { - it('should return ALREADY_EXISTS if the task exists', (done) => { - const tasks = { - f92a9f97d2609849e6837b483d8210c7b308c6f615a691449087ec00db1eef06: { pending: true, completed: false, failed: false } as unknown as AsyncTask - }; - const call = { - request: { accountId: 'testAccount', functionName: 'testFunction', schema: 'schema' } - } as unknown as ServerUnaryCall; - const callback = (error: any): void => { - expect(error.code).toBe(status.ALREADY_EXISTS); - expect(error.details).toBe('Deprovisioning task already exists'); - done(); - }; - - createDataLayerService(undefined, tasks).StartDeprovisioningTask(call, callback); - }); - it('should start a new deprovisioning task', (done) => { const tasks: Record = {}; const provisioner = { diff --git a/runner/src/server/services/data-layer/data-layer-service.ts b/runner/src/server/services/data-layer/data-layer-service.ts index f82cbc1eb..5f60c18dd 100644 --- a/runner/src/server/services/data-layer/data-layer-service.ts +++ b/runner/src/server/services/data-layer/data-layer-service.ts @@ -39,17 +39,6 @@ export class AsyncTask { type AsyncTasks = Record; -enum TaskType { - PROVISION = 'PROVISION', - DEPROVISION = 'DEPROVISION' -} - -const hash = (...args: string[]): string => { - const hash = crypto.createHash('sha256'); - hash.update(args.join(':')); - return hash.digest('hex'); -}; - const createLogger = (config: ProvisioningConfig): typeof parentLogger => { const logger = parentLogger.child({ accountId: config.accountId, @@ -98,31 +87,45 @@ export function createDataLayerService ( const logger = createLogger(provisioningConfig); - const taskId = hash(accountId, functionName, schema, TaskType.PROVISION); - - const task = tasks[taskId]; - - if (task) { - callback(null, { taskId }); - - return; - }; - - logger.info(`Starting provisioning task: ${taskId}`); - - tasks[taskId] = new AsyncTask( - provisioner - .provisionUserApi(provisioningConfig) - .then(() => { - logger.info('Successfully provisioned Data Layer'); - }) - .catch((err) => { - logger.error('Failed to provision Data Layer', err); - throw err; - }) - ); - - callback(null, { taskId }); + provisioner + .isProvisioned(provisioningConfig) + .then((isProvisioned) => { + if (isProvisioned) { + const failedPrecondition = new StatusBuilder() + .withCode(status.FAILED_PRECONDITION) + .withDetails('Data Layer is already provisioned') + .build(); + + callback(failedPrecondition); + + return; + } + + const taskId = crypto.randomUUID(); + + tasks[taskId] = new AsyncTask( + provisioner + .provisionUserApi(provisioningConfig) + .then(() => { + logger.info('Successfully deprovisioned Data Layer'); + }) + .catch((err) => { + logger.error('Failed to deprovision Data Layer', err); + throw err; + }) + ); + + callback(null, { taskId }); + }) + .catch((err) => { + logger.error('Failed to check if Data Layer is provisioned', err); + + const internal = new StatusBuilder() + .withCode(status.INTERNAL) + .withDetails('Failed to check Data Layer provisioned status') + .build(); + callback(internal); + }); }, StartDeprovisioningTask (call: ServerUnaryCall, callback: sendUnaryData): void { @@ -132,19 +135,7 @@ export function createDataLayerService ( const logger = createLogger(provisioningConfig); - const taskId = hash(accountId, functionName, TaskType.DEPROVISION); - - const task = tasks[taskId]; - - if (task) { - const exists = new StatusBuilder() - .withCode(status.ALREADY_EXISTS) - .withDetails('Deprovisioning task already exists') - .build(); - callback(exists); - - return; - }; + const taskId = crypto.randomUUID(); logger.info(`Starting deprovisioning task: ${taskId}`); From f8028fe73cf2268b9f11b6fc40b7d22837265f3e Mon Sep 17 00:00:00 2001 From: Morgan McCauley Date: Fri, 21 Jun 2024 21:41:04 +1200 Subject: [PATCH 08/10] chore: Fix data layer service logs (#826) --- runner/src/server/services/data-layer/data-layer-service.ts | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/runner/src/server/services/data-layer/data-layer-service.ts b/runner/src/server/services/data-layer/data-layer-service.ts index 5f60c18dd..d24befa02 100644 --- a/runner/src/server/services/data-layer/data-layer-service.ts +++ b/runner/src/server/services/data-layer/data-layer-service.ts @@ -103,14 +103,16 @@ export function createDataLayerService ( const taskId = crypto.randomUUID(); + logger.info(`Starting provisioning task: ${taskId}`); + tasks[taskId] = new AsyncTask( provisioner .provisionUserApi(provisioningConfig) .then(() => { - logger.info('Successfully deprovisioned Data Layer'); + logger.info('Successfully provisioned Data Layer'); }) .catch((err) => { - logger.error('Failed to deprovision Data Layer', err); + logger.error('Failed to provision Data Layer', err); throw err; }) ); From 518545732dc9dd406485f3917da59400b4c90def Mon Sep 17 00:00:00 2001 From: Kevin Zhang <42101107+Kevin101Zhang@users.noreply.github.com> Date: Mon, 24 Jun 2024 13:32:23 -0400 Subject: [PATCH 09/10] fix: contract filter is now editable (#824) Fixed logic to handle editable contract filter --- frontend/src/components/Modals/ModalsView/PublishFormView.jsx | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/frontend/src/components/Modals/ModalsView/PublishFormView.jsx b/frontend/src/components/Modals/ModalsView/PublishFormView.jsx index dc7bf0752..ba6817fe2 100644 --- a/frontend/src/components/Modals/ModalsView/PublishFormView.jsx +++ b/frontend/src/components/Modals/ModalsView/PublishFormView.jsx @@ -94,10 +94,9 @@ const PublishFormView = ({ id="contractFilter" type="text" placeholder="social.near" - value={startBlock === "startBlockContinue" ? contractFilter : indexerDetails.rule.affected_account_id} + value={startBlock === "startBlockContinue" ? indexerDetails.rule.affected_account_id : contractFilter} onChange={(e) => handleSetContractFilter(e.target.value)} required - disabled={startBlock === "startBlockContinue"} className={`px-3 py-2 border rounded-md focus:outline-none focus:ring-2 focus:ring-gray-500 focus:border-gray-500 ${isContractFilterValid ? 'border-gray-300' : 'border-red-500'}`} /> {!isContractFilterValid && ( From 056f27241d295c233bf0298eee06e882c68d7443 Mon Sep 17 00:00:00 2001 From: Kevin Zhang <42101107+Kevin101Zhang@users.noreply.github.com> Date: Mon, 24 Jun 2024 13:39:12 -0400 Subject: [PATCH 10/10] chore: run prettier and lint on frontend (#828) no functional changes. created scripts for prettier and lint in package.json. reinstalled modules/lock and ran prettier on all files. --- frontend/.eslintrc.js | 50 +- frontend/.prettierrc.js | 3 +- frontend/package-lock.json | 791 ++++---- frontend/package.json | 9 +- frontend/primitives.d.ts | 4 +- frontend/src/classes/ValidationError.js | 8 +- .../src/components/Common/CustomTooltip.tsx | 18 +- .../src/components/Common/LatestBlock.tsx | 35 +- .../CreateNewIndexer/CreateNewIndexer.js | 6 +- .../src/components/CreateNewIndexer/index.js | 2 +- .../components/Editor/DiffEditorComponent.jsx | 11 +- frontend/src/components/Editor/Editor.jsx | 246 +-- .../Editor/EditorView/BlockPickerView.jsx | 15 +- .../Editor/EditorView/DeveloperToolsView.jsx | 242 +-- .../Editor/EditorView/EditorMenuView.jsx | 64 +- .../BlockPickerContainer.tsx | 12 +- .../DeveloperToolsContainer.tsx | 60 +- .../EditorMenuContainer.jsx | 69 +- .../src/components/Editor/FileSwitcher.jsx | 35 +- .../src/components/Editor/GlyphContainer.js | 8 +- .../Editor/MonacoEditorComponent.jsx | 2 +- .../Editor/ResizableLayoutEditor.jsx | 54 +- .../Editor/__tests__/Editor.test.js | 113 +- .../src/components/Editor/block_details.js | 1645 ++++++++++++++++- .../Logs/GraphQL/QueryValidation.ts | 52 +- frontend/src/components/Logs/LogsMenu.tsx | 37 +- .../Logs/LogsView/ClearButtonView.jsx | 20 +- .../Logs/LogsView/DateSelectorView.jsx | 8 +- .../Logs/LogsView/Icons/CheckMarkIcon.js | 10 +- .../Logs/LogsView/Icons/ClearIcon.js | 22 +- .../Logs/LogsView/IndexerLogsView.jsx | 119 +- .../Logs/LogsView/LogFieldCardView.jsx | 104 +- .../Logs/LogsView/LogTypeSelectorView.jsx | 8 +- .../Logs/LogsView/OptionSelectorView.jsx | 46 +- .../Logs/LogsView/SeveritySelectorView.jsx | 8 +- .../ClearButtonContainer.tsx | 2 +- .../DateSelectorContainer.tsx | 12 +- .../IndexerLogsContainer.tsx | 53 +- .../LogTypeSelectorContainer.tsx | 10 +- .../OptionSelectorContainer.tsx | 12 +- .../SeveritySelectorContainer.tsx | 4 +- .../components/Modals/ForkIndexerModal.jsx | 28 +- .../ModalsContainer/PublishFormContainer.tsx | 8 +- .../Modals/ModalsView/PublishFormView.jsx | 209 +-- .../src/components/Modals/PublishModal.jsx | 46 +- .../src/components/Modals/resetChanges.jsx | 30 +- .../src/components/Playground/graphiql.jsx | 32 +- frontend/src/components/Playground/index.js | 6 +- frontend/src/constants/DurationMap.ts | 2 +- frontend/src/constants/RegexExp.js | 4 +- frontend/src/constants/Strings.js | 14 +- .../src/contexts/IndexerDetailsContext.js | 65 +- frontend/src/contexts/ModalContext.js | 44 +- frontend/src/core/InfoModal.jsx | 104 +- frontend/src/pages/_app.jsx | 13 +- .../src/pages/create-new-indexer/index.js | 6 +- frontend/src/pages/query-api-editor/index.js | 9 +- frontend/src/utils/fetchBlock.js | 7 +- frontend/src/utils/formatters.js | 27 +- frontend/src/utils/formatters.test.js | 21 +- frontend/src/utils/getLatestBlockHeight.js | 17 +- frontend/src/utils/indexerRunner.js | 207 ++- frontend/src/utils/pgSchemaTypeGen.js | 506 ++--- frontend/src/utils/queryIndexerFunction.js | 28 +- frontend/src/utils/resize.js | 136 +- frontend/src/utils/validators.js | 22 +- frontend/tsconfig.json | 2 +- 67 files changed, 3650 insertions(+), 1972 deletions(-) diff --git a/frontend/.eslintrc.js b/frontend/.eslintrc.js index 56315b89e..64a016015 100644 --- a/frontend/.eslintrc.js +++ b/frontend/.eslintrc.js @@ -1,43 +1,17 @@ module.exports = { parser: '@typescript-eslint/parser', - env: { - es2021: true, - jest: true - }, extends: [ - 'eslint:recommended', - 'plugin:react/recommended', - 'plugin:react-hooks/recommended', - 'plugin:@next/next/recommended', - 'next/core-web-vitals', - 'plugin:@typescript-eslint/recommended', - ], - overrides: [ - { - files: ['./src/**/*.js', './src/**/*.jsx'], - parser: 'espree', - rules: { - semi: ['error', 'always'], - 'comma-dangle': ['error', 'only-multiline'], - 'eol-last': ['error', 'always'], - '@typescript-eslint/no-empty-function': ['warn', { allow: ['methods'] }], - }, - }, - { - files: ['./src/**/*', './tests/**/*', './**/*.json'], - excludedFiles: ['./src/**/*.js', './src/**/*.jsx'], - parserOptions: { - project: './tsconfig.json', - }, - extends: ['standard-with-typescript'], - rules: { - '@typescript-eslint/semi': ['error', 'always'], - '@typescript-eslint/comma-dangle': ['error', 'only-multiline'], - '@typescript-eslint/strict-boolean-expressions': 'off', - 'eol-last': ['error', 'always'], - '@typescript-eslint/no-unused-vars': 'warn', - '@typescript-eslint/no-empty-function': ['warn', { allow: ['methods'] }], - }, - }, + "plugin:@typescript-eslint/recommended", + "next/core-web-vitals", // extended set of recommended rules from Next.js + "prettier", ], + plugins: ["simple-import-sort", "@typescript-eslint"], + root: true, + rules: { + "simple-import-sort/imports": "warn", + "@typescript-eslint/no-explicit-any": "off", // TODO: remove once refactor from JS is complete + "@typescript-eslint/consistent-type-imports": "error", + "@typescript-eslint/no-unused-vars": ['warn', { argsIgnorePattern: "^_", "varsIgnorePattern": "^_" }], + '@typescript-eslint/no-empty-function': ['warn', { allow: ['methods'] }], + } }; diff --git a/frontend/.prettierrc.js b/frontend/.prettierrc.js index 540c06fd9..e3388e325 100644 --- a/frontend/.prettierrc.js +++ b/frontend/.prettierrc.js @@ -2,7 +2,8 @@ module.exports = { semi: true, // Add semicolons at the end of statements singleQuote: true, // Use single quotes instead of double quotes tabWidth: 2, // Set the tab width to 2 spaces - printWidth: 100, // Wrap lines that exceed 100 characters + singleAttributePerLine: false, + printWidth: 120, // Wrap lines that exceed 120 characters trailingComma: 'all', // Use trailing commas wherever possible (multi-line objects and arrays) arrowParens: 'always', // Always include parentheses around arrow function parameters endOfLine: 'lf', // Use LF (line feed) as the line ending diff --git a/frontend/package-lock.json b/frontend/package-lock.json index 652a1ab36..7d50d65f6 100644 --- a/frontend/package-lock.json +++ b/frontend/package-lock.json @@ -48,11 +48,13 @@ "autoprefixer": "^10.4.17", "eslint": "^8.46.0", "eslint-config-next": "^14.2.4", + "eslint-config-prettier": "^9.1.0", "eslint-config-standard-with-typescript": "^37.0.0", "eslint-plugin-import": "^2.28.0", "eslint-plugin-n": "^16.0.1", "eslint-plugin-react": "^7.32.0", "eslint-plugin-react-hooks": "^4.3.0", + "eslint-plugin-simple-import-sort": "^12.1.0", "jest": "^29.6.2", "tailwindcss": "^3.0.21", "ts-jest": "^29.1.1", @@ -87,9 +89,9 @@ } }, "node_modules/@apollo/client": { - "version": "3.10.5", - "resolved": "https://registry.npmjs.org/@apollo/client/-/client-3.10.5.tgz", - "integrity": "sha512-bZh5wLAT8b4KdEmqnqiQeDUttnR+NJ+gDYSN8T+U0uFGN++5LO5PTwySih6kIU5ErGGGw4NHI94YdSET3uLuBA==", + "version": "3.10.6", + "resolved": "https://registry.npmjs.org/@apollo/client/-/client-3.10.6.tgz", + "integrity": "sha512-3lLFGJtzC1/mEnK11BRf+Bf8536kBQUSB1G9yMtcRsxmY+tCKdTPzsP3fMUKy10BPIE0sDUY1pux3iMPIn2vow==", "license": "MIT", "dependencies": { "@graphql-typed-document-node/core": "^3.1.1", @@ -712,9 +714,9 @@ "peer": true }, "node_modules/@codemirror/view": { - "version": "6.28.1", - "resolved": "https://registry.npmjs.org/@codemirror/view/-/view-6.28.1.tgz", - "integrity": "sha512-BUWr+zCJpMkA/u69HlJmR+YkV4yPpM81HeMkOMZuwFa8iM5uJdEPKAs1icIRZKkKmy0Ub1x9/G3PQLTXdpBxrQ==", + "version": "6.28.2", + "resolved": "https://registry.npmjs.org/@codemirror/view/-/view-6.28.2.tgz", + "integrity": "sha512-A3DmyVfjgPsGIjiJqM/zvODUAPQdQl3ci0ghehYNnbt5x+o76xq+dL5+mMBuysDXnI3kapgOkoeJ0sbtL/3qPw==", "license": "MIT", "peer": true, "dependencies": { @@ -2313,28 +2315,24 @@ } }, "node_modules/@radix-ui/primitive": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/@radix-ui/primitive/-/primitive-1.0.1.tgz", - "integrity": "sha512-yQ8oGX2GVsEYMWGxcovu1uGWPCxV5BFfeeYxqPmuAzUyLT9qmaMXSAhXpb0WrspIeqYzdJpkh2vHModJPgRIaw==", - "license": "MIT", - "dependencies": { - "@babel/runtime": "^7.13.10" - } + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@radix-ui/primitive/-/primitive-1.1.0.tgz", + "integrity": "sha512-4Z8dn6Upk0qk4P74xBhZ6Hd/w0mPEzOOLxy4xiPXOXqjF7jZS0VAKk7/x/H6FyY2zCkYJqePf1G5KmkmNJ4RBA==", + "license": "MIT" }, "node_modules/@radix-ui/react-arrow": { - "version": "1.0.3", - "resolved": "https://registry.npmjs.org/@radix-ui/react-arrow/-/react-arrow-1.0.3.tgz", - "integrity": "sha512-wSP+pHsB/jQRaL6voubsQ/ZlrGBHHrOjmBnr19hxYgtS0WvAFwZhK2WP/YY5yF9uKECCEEDGxuLxq1NBK51wFA==", + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@radix-ui/react-arrow/-/react-arrow-1.1.0.tgz", + "integrity": "sha512-FmlW1rCg7hBpEBwFbjHwCW6AmWLQM6g/v0Sn8XbP9NvmSZ2San1FpQeyPtufzOMSIx7Y4dzjlHoifhp+7NkZhw==", "license": "MIT", "dependencies": { - "@babel/runtime": "^7.13.10", - "@radix-ui/react-primitive": "1.0.3" + "@radix-ui/react-primitive": "2.0.0" }, "peerDependencies": { "@types/react": "*", "@types/react-dom": "*", - "react": "^16.8 || ^17.0 || ^18.0", - "react-dom": "^16.8 || ^17.0 || ^18.0" + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc", + "react-dom": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -2346,22 +2344,21 @@ } }, "node_modules/@radix-ui/react-collection": { - "version": "1.0.3", - "resolved": "https://registry.npmjs.org/@radix-ui/react-collection/-/react-collection-1.0.3.tgz", - "integrity": "sha512-3SzW+0PW7yBBoQlT8wNcGtaxaD0XSu0uLUFgrtHY08Acx05TaHaOmVLR73c0j/cqpDy53KBMO7s0dx2wmOIDIA==", + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@radix-ui/react-collection/-/react-collection-1.1.0.tgz", + "integrity": "sha512-GZsZslMJEyo1VKm5L1ZJY8tGDxZNPAoUeQUIbKeJfoi7Q4kmig5AsgLMYYuyYbfjd8fBmFORAIwYAkXMnXZgZw==", "license": "MIT", "dependencies": { - "@babel/runtime": "^7.13.10", - "@radix-ui/react-compose-refs": "1.0.1", - "@radix-ui/react-context": "1.0.1", - "@radix-ui/react-primitive": "1.0.3", - "@radix-ui/react-slot": "1.0.2" + "@radix-ui/react-compose-refs": "1.1.0", + "@radix-ui/react-context": "1.1.0", + "@radix-ui/react-primitive": "2.0.0", + "@radix-ui/react-slot": "1.1.0" }, "peerDependencies": { "@types/react": "*", "@types/react-dom": "*", - "react": "^16.8 || ^17.0 || ^18.0", - "react-dom": "^16.8 || ^17.0 || ^18.0" + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc", + "react-dom": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -2373,16 +2370,13 @@ } }, "node_modules/@radix-ui/react-compose-refs": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/@radix-ui/react-compose-refs/-/react-compose-refs-1.0.1.tgz", - "integrity": "sha512-fDSBgd44FKHa1FRMU59qBMPFcl2PZE+2nmqunj+BWFyYYjnhIDWL2ItDs3rrbJDQOtzt5nIebLCQc4QRfz6LJw==", + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@radix-ui/react-compose-refs/-/react-compose-refs-1.1.0.tgz", + "integrity": "sha512-b4inOtiaOnYf9KWyO3jAeeCG6FeyfY6ldiEPanbUjWd+xIk5wZeHa8yVwmrJ2vderhu/BQvzCrJI0lHd+wIiqw==", "license": "MIT", - "dependencies": { - "@babel/runtime": "^7.13.10" - }, "peerDependencies": { "@types/react": "*", - "react": "^16.8 || ^17.0 || ^18.0" + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -2391,16 +2385,13 @@ } }, "node_modules/@radix-ui/react-context": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/@radix-ui/react-context/-/react-context-1.0.1.tgz", - "integrity": "sha512-ebbrdFoYTcuZ0v4wG5tedGnp9tzcV8awzsxYph7gXUyvnNLuTIcCk1q17JEbnVhXAKG9oX3KtchwiMIAYp9NLg==", + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@radix-ui/react-context/-/react-context-1.1.0.tgz", + "integrity": "sha512-OKrckBy+sMEgYM/sMmqmErVn0kZqrHPJze+Ql3DzYsDDp0hl0L62nx/2122/Bvps1qz645jlcu2tD9lrRSdf8A==", "license": "MIT", - "dependencies": { - "@babel/runtime": "^7.13.10" - }, "peerDependencies": { "@types/react": "*", - "react": "^16.8 || ^17.0 || ^18.0" + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -2409,32 +2400,31 @@ } }, "node_modules/@radix-ui/react-dialog": { - "version": "1.0.5", - "resolved": "https://registry.npmjs.org/@radix-ui/react-dialog/-/react-dialog-1.0.5.tgz", - "integrity": "sha512-GjWJX/AUpB703eEBanuBnIWdIXg6NvJFCXcNlSZk4xdszCdhrJgBoUd1cGk67vFO+WdA2pfI/plOpqz/5GUP6Q==", - "license": "MIT", - "dependencies": { - "@babel/runtime": "^7.13.10", - "@radix-ui/primitive": "1.0.1", - "@radix-ui/react-compose-refs": "1.0.1", - "@radix-ui/react-context": "1.0.1", - "@radix-ui/react-dismissable-layer": "1.0.5", - "@radix-ui/react-focus-guards": "1.0.1", - "@radix-ui/react-focus-scope": "1.0.4", - "@radix-ui/react-id": "1.0.1", - "@radix-ui/react-portal": "1.0.4", - "@radix-ui/react-presence": "1.0.1", - "@radix-ui/react-primitive": "1.0.3", - "@radix-ui/react-slot": "1.0.2", - "@radix-ui/react-use-controllable-state": "1.0.1", + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/@radix-ui/react-dialog/-/react-dialog-1.1.1.tgz", + "integrity": "sha512-zysS+iU4YP3STKNS6USvFVqI4qqx8EpiwmT5TuCApVEBca+eRCbONi4EgzfNSuVnOXvC5UPHHMjs8RXO6DH9Bg==", + "license": "MIT", + "dependencies": { + "@radix-ui/primitive": "1.1.0", + "@radix-ui/react-compose-refs": "1.1.0", + "@radix-ui/react-context": "1.1.0", + "@radix-ui/react-dismissable-layer": "1.1.0", + "@radix-ui/react-focus-guards": "1.1.0", + "@radix-ui/react-focus-scope": "1.1.0", + "@radix-ui/react-id": "1.1.0", + "@radix-ui/react-portal": "1.1.1", + "@radix-ui/react-presence": "1.1.0", + "@radix-ui/react-primitive": "2.0.0", + "@radix-ui/react-slot": "1.1.0", + "@radix-ui/react-use-controllable-state": "1.1.0", "aria-hidden": "^1.1.1", - "react-remove-scroll": "2.5.5" + "react-remove-scroll": "2.5.7" }, "peerDependencies": { "@types/react": "*", "@types/react-dom": "*", - "react": "^16.8 || ^17.0 || ^18.0", - "react-dom": "^16.8 || ^17.0 || ^18.0" + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc", + "react-dom": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -2446,16 +2436,13 @@ } }, "node_modules/@radix-ui/react-direction": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/@radix-ui/react-direction/-/react-direction-1.0.1.tgz", - "integrity": "sha512-RXcvnXgyvYvBEOhCBuddKecVkoMiI10Jcm5cTI7abJRAHYfFxeu+FBQs/DvdxSYucxR5mna0dNsL6QFlds5TMA==", + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@radix-ui/react-direction/-/react-direction-1.1.0.tgz", + "integrity": "sha512-BUuBvgThEiAXh2DWu93XsT+a3aWrGqolGlqqw5VU1kG7p/ZH2cuDlM1sRLNnY3QcBS69UIz2mcKhMxDsdewhjg==", "license": "MIT", - "dependencies": { - "@babel/runtime": "^7.13.10" - }, "peerDependencies": { "@types/react": "*", - "react": "^16.8 || ^17.0 || ^18.0" + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -2464,23 +2451,22 @@ } }, "node_modules/@radix-ui/react-dismissable-layer": { - "version": "1.0.5", - "resolved": "https://registry.npmjs.org/@radix-ui/react-dismissable-layer/-/react-dismissable-layer-1.0.5.tgz", - "integrity": "sha512-aJeDjQhywg9LBu2t/At58hCvr7pEm0o2Ke1x33B+MhjNmmZ17sy4KImo0KPLgsnc/zN7GPdce8Cnn0SWvwZO7g==", + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@radix-ui/react-dismissable-layer/-/react-dismissable-layer-1.1.0.tgz", + "integrity": "sha512-/UovfmmXGptwGcBQawLzvn2jOfM0t4z3/uKffoBlj724+n3FvBbZ7M0aaBOmkp6pqFYpO4yx8tSVJjx3Fl2jig==", "license": "MIT", "dependencies": { - "@babel/runtime": "^7.13.10", - "@radix-ui/primitive": "1.0.1", - "@radix-ui/react-compose-refs": "1.0.1", - "@radix-ui/react-primitive": "1.0.3", - "@radix-ui/react-use-callback-ref": "1.0.1", - "@radix-ui/react-use-escape-keydown": "1.0.3" + "@radix-ui/primitive": "1.1.0", + "@radix-ui/react-compose-refs": "1.1.0", + "@radix-ui/react-primitive": "2.0.0", + "@radix-ui/react-use-callback-ref": "1.1.0", + "@radix-ui/react-use-escape-keydown": "1.1.0" }, "peerDependencies": { "@types/react": "*", "@types/react-dom": "*", - "react": "^16.8 || ^17.0 || ^18.0", - "react-dom": "^16.8 || ^17.0 || ^18.0" + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc", + "react-dom": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -2492,25 +2478,24 @@ } }, "node_modules/@radix-ui/react-dropdown-menu": { - "version": "2.0.6", - "resolved": "https://registry.npmjs.org/@radix-ui/react-dropdown-menu/-/react-dropdown-menu-2.0.6.tgz", - "integrity": "sha512-i6TuFOoWmLWq+M/eCLGd/bQ2HfAX1RJgvrBQ6AQLmzfvsLdefxbWu8G9zczcPFfcSPehz9GcpF6K9QYreFV8hA==", + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/@radix-ui/react-dropdown-menu/-/react-dropdown-menu-2.1.1.tgz", + "integrity": "sha512-y8E+x9fBq9qvteD2Zwa4397pUVhYsh9iq44b5RD5qu1GMJWBCBuVg1hMyItbc6+zH00TxGRqd9Iot4wzf3OoBQ==", "license": "MIT", "dependencies": { - "@babel/runtime": "^7.13.10", - "@radix-ui/primitive": "1.0.1", - "@radix-ui/react-compose-refs": "1.0.1", - "@radix-ui/react-context": "1.0.1", - "@radix-ui/react-id": "1.0.1", - "@radix-ui/react-menu": "2.0.6", - "@radix-ui/react-primitive": "1.0.3", - "@radix-ui/react-use-controllable-state": "1.0.1" + "@radix-ui/primitive": "1.1.0", + "@radix-ui/react-compose-refs": "1.1.0", + "@radix-ui/react-context": "1.1.0", + "@radix-ui/react-id": "1.1.0", + "@radix-ui/react-menu": "2.1.1", + "@radix-ui/react-primitive": "2.0.0", + "@radix-ui/react-use-controllable-state": "1.1.0" }, "peerDependencies": { "@types/react": "*", "@types/react-dom": "*", - "react": "^16.8 || ^17.0 || ^18.0", - "react-dom": "^16.8 || ^17.0 || ^18.0" + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc", + "react-dom": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -2522,16 +2507,13 @@ } }, "node_modules/@radix-ui/react-focus-guards": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/@radix-ui/react-focus-guards/-/react-focus-guards-1.0.1.tgz", - "integrity": "sha512-Rect2dWbQ8waGzhMavsIbmSVCgYxkXLxxR3ZvCX79JOglzdEy4JXMb98lq4hPxUbLr77nP0UOGf4rcMU+s1pUA==", + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@radix-ui/react-focus-guards/-/react-focus-guards-1.1.0.tgz", + "integrity": "sha512-w6XZNUPVv6xCpZUqb/yN9DL6auvpGX3C/ee6Hdi16v2UUy25HV2Q5bcflsiDyT/g5RwbPQ/GIT1vLkeRb+ITBw==", "license": "MIT", - "dependencies": { - "@babel/runtime": "^7.13.10" - }, "peerDependencies": { "@types/react": "*", - "react": "^16.8 || ^17.0 || ^18.0" + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -2540,21 +2522,20 @@ } }, "node_modules/@radix-ui/react-focus-scope": { - "version": "1.0.4", - "resolved": "https://registry.npmjs.org/@radix-ui/react-focus-scope/-/react-focus-scope-1.0.4.tgz", - "integrity": "sha512-sL04Mgvf+FmyvZeYfNu1EPAaaxD+aw7cYeIB9L9Fvq8+urhltTRaEo5ysKOpHuKPclsZcSUMKlN05x4u+CINpA==", + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@radix-ui/react-focus-scope/-/react-focus-scope-1.1.0.tgz", + "integrity": "sha512-200UD8zylvEyL8Bx+z76RJnASR2gRMuxlgFCPAe/Q/679a/r0eK3MBVYMb7vZODZcffZBdob1EGnky78xmVvcA==", "license": "MIT", "dependencies": { - "@babel/runtime": "^7.13.10", - "@radix-ui/react-compose-refs": "1.0.1", - "@radix-ui/react-primitive": "1.0.3", - "@radix-ui/react-use-callback-ref": "1.0.1" + "@radix-ui/react-compose-refs": "1.1.0", + "@radix-ui/react-primitive": "2.0.0", + "@radix-ui/react-use-callback-ref": "1.1.0" }, "peerDependencies": { "@types/react": "*", "@types/react-dom": "*", - "react": "^16.8 || ^17.0 || ^18.0", - "react-dom": "^16.8 || ^17.0 || ^18.0" + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc", + "react-dom": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -2566,17 +2547,16 @@ } }, "node_modules/@radix-ui/react-id": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/@radix-ui/react-id/-/react-id-1.0.1.tgz", - "integrity": "sha512-tI7sT/kqYp8p96yGWY1OAnLHrqDgzHefRBKQ2YAkBS5ja7QLcZ9Z/uY7bEjPUatf8RomoXM8/1sMj1IJaE5UzQ==", + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@radix-ui/react-id/-/react-id-1.1.0.tgz", + "integrity": "sha512-EJUrI8yYh7WOjNOqpoJaf1jlFIH2LvtgAl+YcFqNCa+4hj64ZXmPkAKOFs/ukjz3byN6bdb/AVUqHkI8/uWWMA==", "license": "MIT", "dependencies": { - "@babel/runtime": "^7.13.10", - "@radix-ui/react-use-layout-effect": "1.0.1" + "@radix-ui/react-use-layout-effect": "1.1.0" }, "peerDependencies": { "@types/react": "*", - "react": "^16.8 || ^17.0 || ^18.0" + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -2585,36 +2565,35 @@ } }, "node_modules/@radix-ui/react-menu": { - "version": "2.0.6", - "resolved": "https://registry.npmjs.org/@radix-ui/react-menu/-/react-menu-2.0.6.tgz", - "integrity": "sha512-BVkFLS+bUC8HcImkRKPSiVumA1VPOOEC5WBMiT+QAVsPzW1FJzI9KnqgGxVDPBcql5xXrHkD3JOVoXWEXD8SYA==", - "license": "MIT", - "dependencies": { - "@babel/runtime": "^7.13.10", - "@radix-ui/primitive": "1.0.1", - "@radix-ui/react-collection": "1.0.3", - "@radix-ui/react-compose-refs": "1.0.1", - "@radix-ui/react-context": "1.0.1", - "@radix-ui/react-direction": "1.0.1", - "@radix-ui/react-dismissable-layer": "1.0.5", - "@radix-ui/react-focus-guards": "1.0.1", - "@radix-ui/react-focus-scope": "1.0.4", - "@radix-ui/react-id": "1.0.1", - "@radix-ui/react-popper": "1.1.3", - "@radix-ui/react-portal": "1.0.4", - "@radix-ui/react-presence": "1.0.1", - "@radix-ui/react-primitive": "1.0.3", - "@radix-ui/react-roving-focus": "1.0.4", - "@radix-ui/react-slot": "1.0.2", - "@radix-ui/react-use-callback-ref": "1.0.1", + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/@radix-ui/react-menu/-/react-menu-2.1.1.tgz", + "integrity": "sha512-oa3mXRRVjHi6DZu/ghuzdylyjaMXLymx83irM7hTxutQbD+7IhPKdMdRHD26Rm+kHRrWcrUkkRPv5pd47a2xFQ==", + "license": "MIT", + "dependencies": { + "@radix-ui/primitive": "1.1.0", + "@radix-ui/react-collection": "1.1.0", + "@radix-ui/react-compose-refs": "1.1.0", + "@radix-ui/react-context": "1.1.0", + "@radix-ui/react-direction": "1.1.0", + "@radix-ui/react-dismissable-layer": "1.1.0", + "@radix-ui/react-focus-guards": "1.1.0", + "@radix-ui/react-focus-scope": "1.1.0", + "@radix-ui/react-id": "1.1.0", + "@radix-ui/react-popper": "1.2.0", + "@radix-ui/react-portal": "1.1.1", + "@radix-ui/react-presence": "1.1.0", + "@radix-ui/react-primitive": "2.0.0", + "@radix-ui/react-roving-focus": "1.1.0", + "@radix-ui/react-slot": "1.1.0", + "@radix-ui/react-use-callback-ref": "1.1.0", "aria-hidden": "^1.1.1", - "react-remove-scroll": "2.5.5" + "react-remove-scroll": "2.5.7" }, "peerDependencies": { "@types/react": "*", "@types/react-dom": "*", - "react": "^16.8 || ^17.0 || ^18.0", - "react-dom": "^16.8 || ^17.0 || ^18.0" + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc", + "react-dom": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -2626,28 +2605,27 @@ } }, "node_modules/@radix-ui/react-popper": { - "version": "1.1.3", - "resolved": "https://registry.npmjs.org/@radix-ui/react-popper/-/react-popper-1.1.3.tgz", - "integrity": "sha512-cKpopj/5RHZWjrbF2846jBNacjQVwkP068DfmgrNJXpvVWrOvlAmE9xSiy5OqeE+Gi8D9fP+oDhUnPqNMY8/5w==", + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/@radix-ui/react-popper/-/react-popper-1.2.0.tgz", + "integrity": "sha512-ZnRMshKF43aBxVWPWvbj21+7TQCvhuULWJ4gNIKYpRlQt5xGRhLx66tMp8pya2UkGHTSlhpXwmjqltDYHhw7Vg==", "license": "MIT", "dependencies": { - "@babel/runtime": "^7.13.10", "@floating-ui/react-dom": "^2.0.0", - "@radix-ui/react-arrow": "1.0.3", - "@radix-ui/react-compose-refs": "1.0.1", - "@radix-ui/react-context": "1.0.1", - "@radix-ui/react-primitive": "1.0.3", - "@radix-ui/react-use-callback-ref": "1.0.1", - "@radix-ui/react-use-layout-effect": "1.0.1", - "@radix-ui/react-use-rect": "1.0.1", - "@radix-ui/react-use-size": "1.0.1", - "@radix-ui/rect": "1.0.1" + "@radix-ui/react-arrow": "1.1.0", + "@radix-ui/react-compose-refs": "1.1.0", + "@radix-ui/react-context": "1.1.0", + "@radix-ui/react-primitive": "2.0.0", + "@radix-ui/react-use-callback-ref": "1.1.0", + "@radix-ui/react-use-layout-effect": "1.1.0", + "@radix-ui/react-use-rect": "1.1.0", + "@radix-ui/react-use-size": "1.1.0", + "@radix-ui/rect": "1.1.0" }, "peerDependencies": { "@types/react": "*", "@types/react-dom": "*", - "react": "^16.8 || ^17.0 || ^18.0", - "react-dom": "^16.8 || ^17.0 || ^18.0" + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc", + "react-dom": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -2659,19 +2637,19 @@ } }, "node_modules/@radix-ui/react-portal": { - "version": "1.0.4", - "resolved": "https://registry.npmjs.org/@radix-ui/react-portal/-/react-portal-1.0.4.tgz", - "integrity": "sha512-Qki+C/EuGUVCQTOTD5vzJzJuMUlewbzuKyUy+/iHM2uwGiru9gZeBJtHAPKAEkB5KWGi9mP/CHKcY0wt1aW45Q==", + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/@radix-ui/react-portal/-/react-portal-1.1.1.tgz", + "integrity": "sha512-A3UtLk85UtqhzFqtoC8Q0KvR2GbXF3mtPgACSazajqq6A41mEQgo53iPzY4i6BwDxlIFqWIhiQ2G729n+2aw/g==", "license": "MIT", "dependencies": { - "@babel/runtime": "^7.13.10", - "@radix-ui/react-primitive": "1.0.3" + "@radix-ui/react-primitive": "2.0.0", + "@radix-ui/react-use-layout-effect": "1.1.0" }, "peerDependencies": { "@types/react": "*", "@types/react-dom": "*", - "react": "^16.8 || ^17.0 || ^18.0", - "react-dom": "^16.8 || ^17.0 || ^18.0" + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc", + "react-dom": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -2683,20 +2661,19 @@ } }, "node_modules/@radix-ui/react-presence": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/@radix-ui/react-presence/-/react-presence-1.0.1.tgz", - "integrity": "sha512-UXLW4UAbIY5ZjcvzjfRFo5gxva8QirC9hF7wRE4U5gz+TP0DbRk+//qyuAQ1McDxBt1xNMBTaciFGvEmJvAZCg==", + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@radix-ui/react-presence/-/react-presence-1.1.0.tgz", + "integrity": "sha512-Gq6wuRN/asf9H/E/VzdKoUtT8GC9PQc9z40/vEr0VCJ4u5XvvhWIrSsCB6vD2/cH7ugTdSfYq9fLJCcM00acrQ==", "license": "MIT", "dependencies": { - "@babel/runtime": "^7.13.10", - "@radix-ui/react-compose-refs": "1.0.1", - "@radix-ui/react-use-layout-effect": "1.0.1" + "@radix-ui/react-compose-refs": "1.1.0", + "@radix-ui/react-use-layout-effect": "1.1.0" }, "peerDependencies": { "@types/react": "*", "@types/react-dom": "*", - "react": "^16.8 || ^17.0 || ^18.0", - "react-dom": "^16.8 || ^17.0 || ^18.0" + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc", + "react-dom": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -2708,19 +2685,18 @@ } }, "node_modules/@radix-ui/react-primitive": { - "version": "1.0.3", - "resolved": "https://registry.npmjs.org/@radix-ui/react-primitive/-/react-primitive-1.0.3.tgz", - "integrity": "sha512-yi58uVyoAcK/Nq1inRY56ZSjKypBNKTa/1mcL8qdl6oJeEaDbOldlzrGn7P6Q3Id5d+SYNGc5AJgc4vGhjs5+g==", + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/@radix-ui/react-primitive/-/react-primitive-2.0.0.tgz", + "integrity": "sha512-ZSpFm0/uHa8zTvKBDjLFWLo8dkr4MBsiDLz0g3gMUwqgLHz9rTaRRGYDgvZPtBJgYCBKXkS9fzmoySgr8CO6Cw==", "license": "MIT", "dependencies": { - "@babel/runtime": "^7.13.10", - "@radix-ui/react-slot": "1.0.2" + "@radix-ui/react-slot": "1.1.0" }, "peerDependencies": { "@types/react": "*", "@types/react-dom": "*", - "react": "^16.8 || ^17.0 || ^18.0", - "react-dom": "^16.8 || ^17.0 || ^18.0" + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc", + "react-dom": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -2732,27 +2708,26 @@ } }, "node_modules/@radix-ui/react-roving-focus": { - "version": "1.0.4", - "resolved": "https://registry.npmjs.org/@radix-ui/react-roving-focus/-/react-roving-focus-1.0.4.tgz", - "integrity": "sha512-2mUg5Mgcu001VkGy+FfzZyzbmuUWzgWkj3rvv4yu+mLw03+mTzbxZHvfcGyFp2b8EkQeMkpRQ5FiA2Vr2O6TeQ==", + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@radix-ui/react-roving-focus/-/react-roving-focus-1.1.0.tgz", + "integrity": "sha512-EA6AMGeq9AEeQDeSH0aZgG198qkfHSbvWTf1HvoDmOB5bBG/qTxjYMWUKMnYiV6J/iP/J8MEFSuB2zRU2n7ODA==", "license": "MIT", "dependencies": { - "@babel/runtime": "^7.13.10", - "@radix-ui/primitive": "1.0.1", - "@radix-ui/react-collection": "1.0.3", - "@radix-ui/react-compose-refs": "1.0.1", - "@radix-ui/react-context": "1.0.1", - "@radix-ui/react-direction": "1.0.1", - "@radix-ui/react-id": "1.0.1", - "@radix-ui/react-primitive": "1.0.3", - "@radix-ui/react-use-callback-ref": "1.0.1", - "@radix-ui/react-use-controllable-state": "1.0.1" + "@radix-ui/primitive": "1.1.0", + "@radix-ui/react-collection": "1.1.0", + "@radix-ui/react-compose-refs": "1.1.0", + "@radix-ui/react-context": "1.1.0", + "@radix-ui/react-direction": "1.1.0", + "@radix-ui/react-id": "1.1.0", + "@radix-ui/react-primitive": "2.0.0", + "@radix-ui/react-use-callback-ref": "1.1.0", + "@radix-ui/react-use-controllable-state": "1.1.0" }, "peerDependencies": { "@types/react": "*", "@types/react-dom": "*", - "react": "^16.8 || ^17.0 || ^18.0", - "react-dom": "^16.8 || ^17.0 || ^18.0" + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc", + "react-dom": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -2764,17 +2739,16 @@ } }, "node_modules/@radix-ui/react-slot": { - "version": "1.0.2", - "resolved": "https://registry.npmjs.org/@radix-ui/react-slot/-/react-slot-1.0.2.tgz", - "integrity": "sha512-YeTpuq4deV+6DusvVUW4ivBgnkHwECUu0BiN43L5UCDFgdhsRUWAghhTF5MbvNTPzmiFOx90asDSUjWuCNapwg==", + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@radix-ui/react-slot/-/react-slot-1.1.0.tgz", + "integrity": "sha512-FUCf5XMfmW4dtYl69pdS4DbxKy8nj4M7SafBgPllysxmdachynNflAdp/gCsnYWNDnge6tI9onzMp5ARYc1KNw==", "license": "MIT", "dependencies": { - "@babel/runtime": "^7.13.10", - "@radix-ui/react-compose-refs": "1.0.1" + "@radix-ui/react-compose-refs": "1.1.0" }, "peerDependencies": { "@types/react": "*", - "react": "^16.8 || ^17.0 || ^18.0" + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -2783,30 +2757,29 @@ } }, "node_modules/@radix-ui/react-tooltip": { - "version": "1.0.7", - "resolved": "https://registry.npmjs.org/@radix-ui/react-tooltip/-/react-tooltip-1.0.7.tgz", - "integrity": "sha512-lPh5iKNFVQ/jav/j6ZrWq3blfDJ0OH9R6FlNUHPMqdLuQ9vwDgFsRxvl8b7Asuy5c8xmoojHUxKHQSOAvMHxyw==", - "license": "MIT", - "dependencies": { - "@babel/runtime": "^7.13.10", - "@radix-ui/primitive": "1.0.1", - "@radix-ui/react-compose-refs": "1.0.1", - "@radix-ui/react-context": "1.0.1", - "@radix-ui/react-dismissable-layer": "1.0.5", - "@radix-ui/react-id": "1.0.1", - "@radix-ui/react-popper": "1.1.3", - "@radix-ui/react-portal": "1.0.4", - "@radix-ui/react-presence": "1.0.1", - "@radix-ui/react-primitive": "1.0.3", - "@radix-ui/react-slot": "1.0.2", - "@radix-ui/react-use-controllable-state": "1.0.1", - "@radix-ui/react-visually-hidden": "1.0.3" + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/@radix-ui/react-tooltip/-/react-tooltip-1.1.1.tgz", + "integrity": "sha512-LLE8nzNE4MzPMw3O2zlVlkLFid3y9hMUs7uCbSHyKSo+tCN4yMCf+ZCCcfrYgsOC0TiHBPQ1mtpJ2liY3ZT3SQ==", + "license": "MIT", + "dependencies": { + "@radix-ui/primitive": "1.1.0", + "@radix-ui/react-compose-refs": "1.1.0", + "@radix-ui/react-context": "1.1.0", + "@radix-ui/react-dismissable-layer": "1.1.0", + "@radix-ui/react-id": "1.1.0", + "@radix-ui/react-popper": "1.2.0", + "@radix-ui/react-portal": "1.1.1", + "@radix-ui/react-presence": "1.1.0", + "@radix-ui/react-primitive": "2.0.0", + "@radix-ui/react-slot": "1.1.0", + "@radix-ui/react-use-controllable-state": "1.1.0", + "@radix-ui/react-visually-hidden": "1.1.0" }, "peerDependencies": { "@types/react": "*", "@types/react-dom": "*", - "react": "^16.8 || ^17.0 || ^18.0", - "react-dom": "^16.8 || ^17.0 || ^18.0" + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc", + "react-dom": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -2818,16 +2791,13 @@ } }, "node_modules/@radix-ui/react-use-callback-ref": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/@radix-ui/react-use-callback-ref/-/react-use-callback-ref-1.0.1.tgz", - "integrity": "sha512-D94LjX4Sp0xJFVaoQOd3OO9k7tpBYNOXdVhkltUbGv2Qb9OXdrg/CpsjlZv7ia14Sylv398LswWBVVu5nqKzAQ==", + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@radix-ui/react-use-callback-ref/-/react-use-callback-ref-1.1.0.tgz", + "integrity": "sha512-CasTfvsy+frcFkbXtSJ2Zu9JHpN8TYKxkgJGWbjiZhFivxaeW7rMeZt7QELGVLaYVfFMsKHjb7Ak0nMEe+2Vfw==", "license": "MIT", - "dependencies": { - "@babel/runtime": "^7.13.10" - }, "peerDependencies": { "@types/react": "*", - "react": "^16.8 || ^17.0 || ^18.0" + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -2836,17 +2806,16 @@ } }, "node_modules/@radix-ui/react-use-controllable-state": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/@radix-ui/react-use-controllable-state/-/react-use-controllable-state-1.0.1.tgz", - "integrity": "sha512-Svl5GY5FQeN758fWKrjM6Qb7asvXeiZltlT4U2gVfl8Gx5UAv2sMR0LWo8yhsIZh2oQ0eFdZ59aoOOMV7b47VA==", + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@radix-ui/react-use-controllable-state/-/react-use-controllable-state-1.1.0.tgz", + "integrity": "sha512-MtfMVJiSr2NjzS0Aa90NPTnvTSg6C/JLCV7ma0W6+OMV78vd8OyRpID+Ng9LxzsPbLeuBnWBA1Nq30AtBIDChw==", "license": "MIT", "dependencies": { - "@babel/runtime": "^7.13.10", - "@radix-ui/react-use-callback-ref": "1.0.1" + "@radix-ui/react-use-callback-ref": "1.1.0" }, "peerDependencies": { "@types/react": "*", - "react": "^16.8 || ^17.0 || ^18.0" + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -2855,17 +2824,16 @@ } }, "node_modules/@radix-ui/react-use-escape-keydown": { - "version": "1.0.3", - "resolved": "https://registry.npmjs.org/@radix-ui/react-use-escape-keydown/-/react-use-escape-keydown-1.0.3.tgz", - "integrity": "sha512-vyL82j40hcFicA+M4Ex7hVkB9vHgSse1ZWomAqV2Je3RleKGO5iM8KMOEtfoSB0PnIelMd2lATjTGMYqN5ylTg==", + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@radix-ui/react-use-escape-keydown/-/react-use-escape-keydown-1.1.0.tgz", + "integrity": "sha512-L7vwWlR1kTTQ3oh7g1O0CBF3YCyyTj8NmhLR+phShpyA50HCfBFKVJTpshm9PzLiKmehsrQzTYTpX9HvmC9rhw==", "license": "MIT", "dependencies": { - "@babel/runtime": "^7.13.10", - "@radix-ui/react-use-callback-ref": "1.0.1" + "@radix-ui/react-use-callback-ref": "1.1.0" }, "peerDependencies": { "@types/react": "*", - "react": "^16.8 || ^17.0 || ^18.0" + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -2874,16 +2842,13 @@ } }, "node_modules/@radix-ui/react-use-layout-effect": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/@radix-ui/react-use-layout-effect/-/react-use-layout-effect-1.0.1.tgz", - "integrity": "sha512-v/5RegiJWYdoCvMnITBkNNx6bCj20fiaJnWtRkU18yITptraXjffz5Qbn05uOiQnOvi+dbkznkoaMltz1GnszQ==", + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@radix-ui/react-use-layout-effect/-/react-use-layout-effect-1.1.0.tgz", + "integrity": "sha512-+FPE0rOdziWSrH9athwI1R0HDVbWlEhd+FR+aSDk4uWGmSJ9Z54sdZVDQPZAinJhJXwfT+qnj969mCsT2gfm5w==", "license": "MIT", - "dependencies": { - "@babel/runtime": "^7.13.10" - }, "peerDependencies": { "@types/react": "*", - "react": "^16.8 || ^17.0 || ^18.0" + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -2892,17 +2857,16 @@ } }, "node_modules/@radix-ui/react-use-rect": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/@radix-ui/react-use-rect/-/react-use-rect-1.0.1.tgz", - "integrity": "sha512-Cq5DLuSiuYVKNU8orzJMbl15TXilTnJKUCltMVQg53BQOF1/C5toAaGrowkgksdBQ9H+SRL23g0HDmg9tvmxXw==", + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@radix-ui/react-use-rect/-/react-use-rect-1.1.0.tgz", + "integrity": "sha512-0Fmkebhr6PiseyZlYAOtLS+nb7jLmpqTrJyv61Pe68MKYW6OWdRE2kI70TaYY27u7H0lajqM3hSMMLFq18Z7nQ==", "license": "MIT", "dependencies": { - "@babel/runtime": "^7.13.10", - "@radix-ui/rect": "1.0.1" + "@radix-ui/rect": "1.1.0" }, "peerDependencies": { "@types/react": "*", - "react": "^16.8 || ^17.0 || ^18.0" + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -2911,17 +2875,16 @@ } }, "node_modules/@radix-ui/react-use-size": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/@radix-ui/react-use-size/-/react-use-size-1.0.1.tgz", - "integrity": "sha512-ibay+VqrgcaI6veAojjofPATwledXiSmX+C0KrBk/xgpX9rBzPV3OsfwlhQdUOFbh+LKQorLYT+xTXW9V8yd0g==", + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@radix-ui/react-use-size/-/react-use-size-1.1.0.tgz", + "integrity": "sha512-XW3/vWuIXHa+2Uwcc2ABSfcCledmXhhQPlGbfcRXbiUQI5Icjcg19BGCZVKKInYbvUCut/ufbbLLPFC5cbb1hw==", "license": "MIT", "dependencies": { - "@babel/runtime": "^7.13.10", - "@radix-ui/react-use-layout-effect": "1.0.1" + "@radix-ui/react-use-layout-effect": "1.1.0" }, "peerDependencies": { "@types/react": "*", - "react": "^16.8 || ^17.0 || ^18.0" + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -2930,19 +2893,18 @@ } }, "node_modules/@radix-ui/react-visually-hidden": { - "version": "1.0.3", - "resolved": "https://registry.npmjs.org/@radix-ui/react-visually-hidden/-/react-visually-hidden-1.0.3.tgz", - "integrity": "sha512-D4w41yN5YRKtu464TLnByKzMDG/JlMPHtfZgQAu9v6mNakUqGUI9vUrfQKz8NK41VMm/xbZbh76NUTVtIYqOMA==", + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@radix-ui/react-visually-hidden/-/react-visually-hidden-1.1.0.tgz", + "integrity": "sha512-N8MDZqtgCgG5S3aV60INAB475osJousYpZ4cTJ2cFbMpdHS5Y6loLTH8LPtkj2QN0x93J30HT/M3qJXM0+lyeQ==", "license": "MIT", "dependencies": { - "@babel/runtime": "^7.13.10", - "@radix-ui/react-primitive": "1.0.3" + "@radix-ui/react-primitive": "2.0.0" }, "peerDependencies": { "@types/react": "*", "@types/react-dom": "*", - "react": "^16.8 || ^17.0 || ^18.0", - "react-dom": "^16.8 || ^17.0 || ^18.0" + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc", + "react-dom": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -2954,13 +2916,10 @@ } }, "node_modules/@radix-ui/rect": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/@radix-ui/rect/-/rect-1.0.1.tgz", - "integrity": "sha512-fyrgCaedtvMg9NK3en0pnOYJdtfwxUcNolezkNPUsoX57X8oQk+NkqcvzHXD2uKNij6GXmWU9NDru2IWjrO4BQ==", - "license": "MIT", - "dependencies": { - "@babel/runtime": "^7.13.10" - } + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@radix-ui/rect/-/rect-1.1.0.tgz", + "integrity": "sha512-A9+lCBZoaMJlVKcRBz2YByCG+Cp2t6nAnMnNba+XiWxnj6r4JUFqfsgwocMBZU9LPtdxC6wB56ySYpc7LQIoJg==", + "license": "MIT" }, "node_modules/@react-aria/ssr": { "version": "3.9.4", @@ -3070,12 +3029,12 @@ } }, "node_modules/@tanstack/react-virtual": { - "version": "3.5.1", - "resolved": "https://registry.npmjs.org/@tanstack/react-virtual/-/react-virtual-3.5.1.tgz", - "integrity": "sha512-jIsuhfgy8GqA67PdWqg73ZB2LFE+HD9hjWL1L6ifEIZVyZVAKpYmgUG4WsKQ005aEyImJmbuimPiEvc57IY0Aw==", + "version": "3.7.0", + "resolved": "https://registry.npmjs.org/@tanstack/react-virtual/-/react-virtual-3.7.0.tgz", + "integrity": "sha512-3RtOwEU1HKS4iFBoTcCrV3Szqt4KoERMhZr8v57dvnh5o70sR9GAdF+0aE/qhiOmePrKujGwAayFNJSr/8Dbqw==", "license": "MIT", "dependencies": { - "@tanstack/virtual-core": "3.5.1" + "@tanstack/virtual-core": "3.7.0" }, "funding": { "type": "github", @@ -3087,9 +3046,9 @@ } }, "node_modules/@tanstack/virtual-core": { - "version": "3.5.1", - "resolved": "https://registry.npmjs.org/@tanstack/virtual-core/-/virtual-core-3.5.1.tgz", - "integrity": "sha512-046+AUSiDru/V9pajE1du8WayvBKeCvJ2NmKPy/mR8/SbKKrqmSbj7LJBfXE+nSq4f5TBXvnCzu0kcYebI9WdQ==", + "version": "3.7.0", + "resolved": "https://registry.npmjs.org/@tanstack/virtual-core/-/virtual-core-3.7.0.tgz", + "integrity": "sha512-p0CWuqn+n8iZmsL7/l0Xg7kbyIKnHNqkEJkMDOkg4x3Ni3LohszmnJY8FPhTgG7Ad9ZFGcdKmn1R1mKUGEh9Xg==", "license": "MIT", "funding": { "type": "github", @@ -3240,9 +3199,9 @@ "license": "MIT" }, "node_modules/@types/node": { - "version": "20.14.2", - "resolved": "https://registry.npmjs.org/@types/node/-/node-20.14.2.tgz", - "integrity": "sha512-xyu6WAMVwv6AKFLB+e/7ySZVr/0zLCzOa7rSpq6jNwpqOrUbcACDWC+53d4n2QHOnDou0fbIsg8wZu/sxrnI4Q==", + "version": "20.14.7", + "resolved": "https://registry.npmjs.org/@types/node/-/node-20.14.7.tgz", + "integrity": "sha512-uTr2m2IbJJucF3KUxgnGOZvYbN0QgkGyWxG6973HCpMYFy2KfcgYuIwkJQMQkt1VbBMlvWRbpshFTLxnxCZjKQ==", "license": "MIT", "dependencies": { "undici-types": "~5.26.4" @@ -3807,9 +3766,9 @@ "peer": true }, "node_modules/acorn": { - "version": "8.11.3", - "resolved": "https://registry.npmjs.org/acorn/-/acorn-8.11.3.tgz", - "integrity": "sha512-Y9rRfJG5jcKOE0CLisYbojUjIrIEE7AGMzA/Sm4BslANhbS+cDMpgBdcPT91oJ7OuJ9hYJBx59RjbhxVnrF8Xg==", + "version": "8.12.0", + "resolved": "https://registry.npmjs.org/acorn/-/acorn-8.12.0.tgz", + "integrity": "sha512-RTvkC4w+KNXrM39/lWCUaG0IbRkWdCv7W/IOW9oU6SawyxulvkQy5HQPVTKxEjczcUvapcrw3cFx/60VN/NRNw==", "license": "MIT", "bin": { "acorn": "bin/acorn" @@ -3949,13 +3908,13 @@ } }, "node_modules/aria-query": { - "version": "5.3.0", - "resolved": "https://registry.npmjs.org/aria-query/-/aria-query-5.3.0.tgz", - "integrity": "sha512-b0P0sZPKtyu8HkeRAfCq0IfURZK+SuwMjY1UXGBU27wpAiTwQAIlq56IbIO+ytk/JjS1fMR14ee5WBBfKi5J6A==", + "version": "5.1.3", + "resolved": "https://registry.npmjs.org/aria-query/-/aria-query-5.1.3.tgz", + "integrity": "sha512-R5iJ5lkuHybztUfuOAznmboyjWq8O6sqNqtK7CLOqdydi54VNbORp49mb14KbWgG1QD3JFO9hJdZ+y4KutfdOQ==", "dev": true, "license": "Apache-2.0", "dependencies": { - "dequal": "^2.0.3" + "deep-equal": "^2.0.5" } }, "node_modules/array-buffer-byte-length": { @@ -4201,9 +4160,9 @@ } }, "node_modules/axe-core": { - "version": "4.7.0", - "resolved": "https://registry.npmjs.org/axe-core/-/axe-core-4.7.0.tgz", - "integrity": "sha512-M0JtH+hlOL5pLQwHOLNYZaXuhqmvS8oExsqB1SBYgA4Dk7u/xx+YdGHXaK5pyUfed5mYXdlYiphWq3G8cRi5JQ==", + "version": "4.9.1", + "resolved": "https://registry.npmjs.org/axe-core/-/axe-core-4.9.1.tgz", + "integrity": "sha512-QbUdXJVTpvUTHU7871ppZkdOLBeGUKBQWHkHrvN2V9IQWGMt61zf3B45BtzjxEJzYuj0JBjBZP/hmYS/R9pmAw==", "dev": true, "license": "MPL-2.0", "engines": { @@ -4211,13 +4170,13 @@ } }, "node_modules/axobject-query": { - "version": "3.2.1", - "resolved": "https://registry.npmjs.org/axobject-query/-/axobject-query-3.2.1.tgz", - "integrity": "sha512-jsyHu61e6N4Vbz/v18DHwWYKK0bSWLqn47eeDSKPB7m8tqMHF9YJ+mhIk2lVteyZrY8tnSj/jHOv4YiTCuCJgg==", + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/axobject-query/-/axobject-query-3.1.1.tgz", + "integrity": "sha512-goKlv8DZrK9hUh975fnHzhNIO4jUnFCfv/dszV5VwUGDFjI6vQ2VwoyjYjYNEbBE8AH87TduWP5uyDR1D+Iteg==", "dev": true, "license": "Apache-2.0", "dependencies": { - "dequal": "^2.0.3" + "deep-equal": "^2.0.5" } }, "node_modules/babel-jest": { @@ -4742,9 +4701,9 @@ } }, "node_modules/caniuse-lite": { - "version": "1.0.30001632", - "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001632.tgz", - "integrity": "sha512-udx3o7yHJfUxMLkGohMlVHCvFvWmirKh9JAH/d7WOLPetlH+LTL5cocMZ0t7oZx/mdlOWXti97xLZWc8uURRHg==", + "version": "1.0.30001636", + "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001636.tgz", + "integrity": "sha512-bMg2vmr8XBsbL6Lr0UHXy/21m84FTxDLWn2FSqMd5PrlbMxwJlQnC2YWYxVgp66PZE+BBNF2jYQUBKCo1FDeZg==", "funding": [ { "type": "opencollective", @@ -5337,6 +5296,39 @@ } } }, + "node_modules/deep-equal": { + "version": "2.2.3", + "resolved": "https://registry.npmjs.org/deep-equal/-/deep-equal-2.2.3.tgz", + "integrity": "sha512-ZIwpnevOurS8bpT4192sqAowWM76JDKSHYzMLty3BZGSswgq6pBaH3DhCSW5xVAZICZyKdOBPjwww5wfgT/6PA==", + "dev": true, + "license": "MIT", + "dependencies": { + "array-buffer-byte-length": "^1.0.0", + "call-bind": "^1.0.5", + "es-get-iterator": "^1.1.3", + "get-intrinsic": "^1.2.2", + "is-arguments": "^1.1.1", + "is-array-buffer": "^3.0.2", + "is-date-object": "^1.0.5", + "is-regex": "^1.1.4", + "is-shared-array-buffer": "^1.0.2", + "isarray": "^2.0.5", + "object-is": "^1.1.5", + "object-keys": "^1.1.1", + "object.assign": "^4.1.4", + "regexp.prototype.flags": "^1.5.1", + "side-channel": "^1.0.4", + "which-boxed-primitive": "^1.0.2", + "which-collection": "^1.0.1", + "which-typed-array": "^1.1.13" + }, + "engines": { + "node": ">= 0.4" + }, + "funding": { + "url": "https://github.com/sponsors/ljharb" + } + }, "node_modules/deep-is": { "version": "0.1.4", "resolved": "https://registry.npmjs.org/deep-is/-/deep-is-0.1.4.tgz", @@ -5498,9 +5490,9 @@ "license": "MIT" }, "node_modules/electron-to-chromium": { - "version": "1.4.799", - "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.799.tgz", - "integrity": "sha512-3D3DwWkRTzrdEpntY0hMLYwj7SeBk1138CkPE8sBDSj3WzrzOiG2rHm3luw8jucpf+WiyLBCZyU9lMHyQI9M9Q==", + "version": "1.4.808", + "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.808.tgz", + "integrity": "sha512-0ItWyhPYnww2VOuCGF4s1LTfbrdAV2ajy/TN+ZTuhR23AHI6rWHCrBXJ/uxoXOvRRqw8qjYVrG81HFI7x/2wdQ==", "license": "ISC" }, "node_modules/emittery": { @@ -5659,6 +5651,27 @@ "node": ">= 0.4" } }, + "node_modules/es-get-iterator": { + "version": "1.1.3", + "resolved": "https://registry.npmjs.org/es-get-iterator/-/es-get-iterator-1.1.3.tgz", + "integrity": "sha512-sPZmqHBe6JIiTfN5q2pEi//TwxmAFHwj/XEuYjTuse78i8KxaqMTTzxPoFKuzRpDpTJ+0NAbpfenkmH2rePtuw==", + "dev": true, + "license": "MIT", + "dependencies": { + "call-bind": "^1.0.2", + "get-intrinsic": "^1.1.3", + "has-symbols": "^1.0.3", + "is-arguments": "^1.1.1", + "is-map": "^2.0.2", + "is-set": "^2.0.2", + "is-string": "^1.0.7", + "isarray": "^2.0.5", + "stop-iteration-iterator": "^1.0.0" + }, + "funding": { + "url": "https://github.com/sponsors/ljharb" + } + }, "node_modules/es-iterator-helpers": { "version": "1.0.19", "resolved": "https://registry.npmjs.org/es-iterator-helpers/-/es-iterator-helpers-1.0.19.tgz", @@ -5879,6 +5892,19 @@ } } }, + "node_modules/eslint-config-prettier": { + "version": "9.1.0", + "resolved": "https://registry.npmjs.org/eslint-config-prettier/-/eslint-config-prettier-9.1.0.tgz", + "integrity": "sha512-NSWl5BFQWEPi1j4TjVNItzYV7dZXZ+wP6I6ZhrBGpChQhZRUaElihE9uRRkcbRnNb76UMKDF3r+WTmNcGPKsqw==", + "dev": true, + "license": "MIT", + "bin": { + "eslint-config-prettier": "bin/cli.js" + }, + "peerDependencies": { + "eslint": ">=7.0.0" + } + }, "node_modules/eslint-config-standard": { "version": "17.1.0", "resolved": "https://registry.npmjs.org/eslint-config-standard/-/eslint-config-standard-17.1.0.tgz", @@ -6083,28 +6109,28 @@ } }, "node_modules/eslint-plugin-jsx-a11y": { - "version": "6.8.0", - "resolved": "https://registry.npmjs.org/eslint-plugin-jsx-a11y/-/eslint-plugin-jsx-a11y-6.8.0.tgz", - "integrity": "sha512-Hdh937BS3KdwwbBaKd5+PLCOmYY6U4f2h9Z2ktwtNKvIdIEu137rjYbcb9ApSbVJfWxANNuiKTD/9tOKjK9qOA==", + "version": "6.9.0", + "resolved": "https://registry.npmjs.org/eslint-plugin-jsx-a11y/-/eslint-plugin-jsx-a11y-6.9.0.tgz", + "integrity": "sha512-nOFOCaJG2pYqORjK19lqPqxMO/JpvdCZdPtNdxY3kvom3jTvkAbOvQvD8wuD0G8BYR0IGAGYDlzqWJOh/ybn2g==", "dev": true, "license": "MIT", "dependencies": { - "@babel/runtime": "^7.23.2", - "aria-query": "^5.3.0", - "array-includes": "^3.1.7", + "aria-query": "~5.1.3", + "array-includes": "^3.1.8", "array.prototype.flatmap": "^1.3.2", "ast-types-flow": "^0.0.8", - "axe-core": "=4.7.0", - "axobject-query": "^3.2.1", + "axe-core": "^4.9.1", + "axobject-query": "~3.1.1", "damerau-levenshtein": "^1.0.8", "emoji-regex": "^9.2.2", - "es-iterator-helpers": "^1.0.15", - "hasown": "^2.0.0", + "es-iterator-helpers": "^1.0.19", + "hasown": "^2.0.2", "jsx-ast-utils": "^3.3.5", "language-tags": "^1.0.9", "minimatch": "^3.1.2", - "object.entries": "^1.1.7", - "object.fromentries": "^2.0.7" + "object.fromentries": "^2.0.8", + "safe-regex-test": "^1.0.3", + "string.prototype.includes": "^2.0.0" }, "engines": { "node": ">=4.0" @@ -6202,9 +6228,9 @@ } }, "node_modules/eslint-plugin-react": { - "version": "7.34.2", - "resolved": "https://registry.npmjs.org/eslint-plugin-react/-/eslint-plugin-react-7.34.2.tgz", - "integrity": "sha512-2HCmrU+/JNigDN6tg55cRDKCQWicYAPB38JGSFDQt95jDm8rrvSUo7YPkOIm5l6ts1j1zCvysNcasvfTMQzUOw==", + "version": "7.34.3", + "resolved": "https://registry.npmjs.org/eslint-plugin-react/-/eslint-plugin-react-7.34.3.tgz", + "integrity": "sha512-aoW4MV891jkUulwDApQbPYTVZmeuSyFrudpbTAQuj5Fv8VL+o6df2xIGpw8B0hPjAaih1/Fb0om9grCdyFYemA==", "dev": true, "license": "MIT", "dependencies": { @@ -6212,7 +6238,7 @@ "array.prototype.findlast": "^1.2.5", "array.prototype.flatmap": "^1.3.2", "array.prototype.toreversed": "^1.1.2", - "array.prototype.tosorted": "^1.1.3", + "array.prototype.tosorted": "^1.1.4", "doctrine": "^2.1.0", "es-iterator-helpers": "^1.0.19", "estraverse": "^5.3.0", @@ -6278,6 +6304,16 @@ "url": "https://github.com/sponsors/ljharb" } }, + "node_modules/eslint-plugin-simple-import-sort": { + "version": "12.1.0", + "resolved": "https://registry.npmjs.org/eslint-plugin-simple-import-sort/-/eslint-plugin-simple-import-sort-12.1.0.tgz", + "integrity": "sha512-Y2fqAfC11TcG/WP3TrI1Gi3p3nc8XJyEOJYHyEPEGI/UAgNx6akxxlX74p7SbAQdLcgASKhj8M0GKvH3vq/+ig==", + "dev": true, + "license": "MIT", + "peerDependencies": { + "eslint": ">=5.0.0" + } + }, "node_modules/eslint-scope": { "version": "5.1.1", "resolved": "https://registry.npmjs.org/eslint-scope/-/eslint-scope-5.1.1.tgz", @@ -6751,9 +6787,9 @@ } }, "node_modules/foreground-child": { - "version": "3.2.0", - "resolved": "https://registry.npmjs.org/foreground-child/-/foreground-child-3.2.0.tgz", - "integrity": "sha512-CrWQNaEl1/6WeZoarcM9LHupTo3RpZO2Pdk1vktwzPiQTsJnAKJmm3TACKeG5UZbWDfaH2AbvYxzP96y0MT7fA==", + "version": "3.2.1", + "resolved": "https://registry.npmjs.org/foreground-child/-/foreground-child-3.2.1.tgz", + "integrity": "sha512-PXUUyLqrR2XCWICfv6ukppP96sdFwWbNEnfEMt7jNsISjMsvaLNinAHNDYyvkyU+SZG2BTSbT5NjG+vZslfGTA==", "dev": true, "license": "ISC", "dependencies": { @@ -7136,9 +7172,9 @@ } }, "node_modules/graphql": { - "version": "16.8.2", - "resolved": "https://registry.npmjs.org/graphql/-/graphql-16.8.2.tgz", - "integrity": "sha512-cvVIBILwuoSyD54U4cF/UXDh5yAobhNV/tPygI4lZhgOIJQE/WLWC4waBRb4I6bDVYb3OVx3lfHbaQOEoUD5sg==", + "version": "16.9.0", + "resolved": "https://registry.npmjs.org/graphql/-/graphql-16.9.0.tgz", + "integrity": "sha512-GGTKBX4SD7Wdb8mqeDLni2oaRGYQWjWHGKPQ24ZMnUtKfcsVoiv4uX8+LJr1K6U5VW2Lu1BwJnj7uiori0YtRw==", "license": "MIT", "engines": { "node": "^12.22.0 || ^14.16.0 || ^16.0.0 || >=17.0.0" @@ -7532,6 +7568,23 @@ "loose-envify": "^1.0.0" } }, + "node_modules/is-arguments": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/is-arguments/-/is-arguments-1.1.1.tgz", + "integrity": "sha512-8Q7EARjzEnKpt/PCD7e1cgUS0a6X8u5tdSiMqXhojOdoV9TsMsiO+9VLC5vAmO8N7/GmXn7yjR8qnA6bVAEzfA==", + "dev": true, + "license": "MIT", + "dependencies": { + "call-bind": "^1.0.2", + "has-tostringtag": "^1.0.0" + }, + "engines": { + "node": ">= 0.4" + }, + "funding": { + "url": "https://github.com/sponsors/ljharb" + } + }, "node_modules/is-array-buffer": { "version": "3.0.4", "resolved": "https://registry.npmjs.org/is-array-buffer/-/is-array-buffer-3.0.4.tgz", @@ -7645,13 +7698,16 @@ } }, "node_modules/is-core-module": { - "version": "2.13.1", - "resolved": "https://registry.npmjs.org/is-core-module/-/is-core-module-2.13.1.tgz", - "integrity": "sha512-hHrIjvZsftOsvKSn2TRYl63zvxsgE0K+0mYMoH6gD4omR5IWB2KynivBQczo3+wF1cCkjzvptnI9Q0sPU66ilw==", + "version": "2.14.0", + "resolved": "https://registry.npmjs.org/is-core-module/-/is-core-module-2.14.0.tgz", + "integrity": "sha512-a5dFJih5ZLYlRtDc0dZWP7RiKr6xIKzmn/oAYCDvdLThadVgyJwlaoQPmRtMSpz+rk0OGAgIu+TcM9HUF0fk1A==", "dev": true, "license": "MIT", "dependencies": { - "hasown": "^2.0.0" + "hasown": "^2.0.2" + }, + "engines": { + "node": ">= 0.4" }, "funding": { "url": "https://github.com/sponsors/ljharb" @@ -10707,6 +10763,23 @@ "url": "https://github.com/sponsors/ljharb" } }, + "node_modules/object-is": { + "version": "1.1.6", + "resolved": "https://registry.npmjs.org/object-is/-/object-is-1.1.6.tgz", + "integrity": "sha512-F8cZ+KfGlSGi09lJT7/Nd6KJZ9ygtvYC0/UYYLI9nmQKLMnydpB9yvbv9K1uSkEu7FU9vYPmVwLg328tX+ot3Q==", + "dev": true, + "license": "MIT", + "dependencies": { + "call-bind": "^1.0.7", + "define-properties": "^1.2.1" + }, + "engines": { + "node": ">= 0.4" + }, + "funding": { + "url": "https://github.com/sponsors/ljharb" + } + }, "node_modules/object-keys": { "version": "1.1.1", "resolved": "https://registry.npmjs.org/object-keys/-/object-keys-1.1.1.tgz", @@ -11581,14 +11654,14 @@ } }, "node_modules/react-bootstrap": { - "version": "2.10.2", - "resolved": "https://registry.npmjs.org/react-bootstrap/-/react-bootstrap-2.10.2.tgz", - "integrity": "sha512-UvB7mRqQjivdZNxJNEA2yOQRB7L9N43nBnKc33K47+cH90/ujmnMwatTCwQLu83gLhrzAl8fsa6Lqig/KLghaA==", + "version": "2.10.3", + "resolved": "https://registry.npmjs.org/react-bootstrap/-/react-bootstrap-2.10.3.tgz", + "integrity": "sha512-cc1KAaQyj6Gr3AfA0eRRiUMSlRi3brDVcjc/o0E9y9XNW7ISo8TITrq8G8G3QTFe7VIhCiDt38k99AEFoLOolw==", "license": "MIT", "dependencies": { - "@babel/runtime": "^7.22.5", + "@babel/runtime": "^7.24.7", "@restart/hooks": "^0.4.9", - "@restart/ui": "^1.6.8", + "@restart/ui": "^1.6.9", "@types/react-transition-group": "^4.4.6", "classnames": "^2.3.2", "dom-helpers": "^5.2.1", @@ -11648,12 +11721,12 @@ "license": "MIT" }, "node_modules/react-remove-scroll": { - "version": "2.5.5", - "resolved": "https://registry.npmjs.org/react-remove-scroll/-/react-remove-scroll-2.5.5.tgz", - "integrity": "sha512-ImKhrzJJsyXJfBZ4bzu8Bwpka14c/fQt0k+cyFp/PBhTfyDnU5hjOtM4AG/0AMyy8oKzOTR0lDgJIM7pYXI0kw==", + "version": "2.5.7", + "resolved": "https://registry.npmjs.org/react-remove-scroll/-/react-remove-scroll-2.5.7.tgz", + "integrity": "sha512-FnrTWO4L7/Bhhf3CYBNArEG/yROV0tKmTv7/3h9QCFvH6sndeFf1wPqOcbFVu5VAulS5dV1wGT3GZZ/1GawqiA==", "license": "MIT", "dependencies": { - "react-remove-scroll-bar": "^2.3.3", + "react-remove-scroll-bar": "^2.3.4", "react-style-singleton": "^2.2.1", "tslib": "^2.1.0", "use-callback-ref": "^1.3.0", @@ -12356,6 +12429,19 @@ "node": ">= 0.6" } }, + "node_modules/stop-iteration-iterator": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/stop-iteration-iterator/-/stop-iteration-iterator-1.0.0.tgz", + "integrity": "sha512-iCGQj+0l0HOdZ2AEeBADlsRC+vsnDsZsbdSiH1yNSjcfKM7fdpCMfqAL/dwF5BLiw/XhRft/Wax6zQbhq2BcjQ==", + "dev": true, + "license": "MIT", + "dependencies": { + "internal-slot": "^1.0.4" + }, + "engines": { + "node": ">= 0.4" + } + }, "node_modules/streamsearch": { "version": "1.1.0", "resolved": "https://registry.npmjs.org/streamsearch/-/streamsearch-1.1.0.tgz", @@ -12448,6 +12534,17 @@ "url": "https://github.com/chalk/strip-ansi?sponsor=1" } }, + "node_modules/string.prototype.includes": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/string.prototype.includes/-/string.prototype.includes-2.0.0.tgz", + "integrity": "sha512-E34CkBgyeqNDcrbU76cDjL5JLcVrtSdYq0MEh/B10r17pRP4ciHLwTgnuLV8Ay6cgEMLkcBkFCKyFZ43YldYzg==", + "dev": true, + "license": "MIT", + "dependencies": { + "define-properties": "^1.1.3", + "es-abstract": "^1.17.5" + } + }, "node_modules/string.prototype.matchall": { "version": "4.0.11", "resolved": "https://registry.npmjs.org/string.prototype.matchall/-/string.prototype.matchall-4.0.11.tgz", @@ -13035,9 +13132,9 @@ } }, "node_modules/ts-jest": { - "version": "29.1.4", - "resolved": "https://registry.npmjs.org/ts-jest/-/ts-jest-29.1.4.tgz", - "integrity": "sha512-YiHwDhSvCiItoAgsKtoLFCuakDzDsJ1DLDnSouTaTmdOcOwIkSzbLXduaQ6M5DRVhuZC/NYaaZ/mtHbWMv/S6Q==", + "version": "29.1.5", + "resolved": "https://registry.npmjs.org/ts-jest/-/ts-jest-29.1.5.tgz", + "integrity": "sha512-UuClSYxM7byvvYfyWdFI+/2UxMmwNyJb0NPkZPQE2hew3RurV7l7zURgOHAd/1I1ZdPpe3GUsXNXAcN8TFKSIg==", "dev": true, "license": "MIT", "dependencies": { @@ -13280,9 +13377,9 @@ } }, "node_modules/typescript": { - "version": "5.4.5", - "resolved": "https://registry.npmjs.org/typescript/-/typescript-5.4.5.tgz", - "integrity": "sha512-vcI4UpRgg81oIRUFwR0WSIHKt11nJ7SAVlYNIu+QpqeyXP+gpQJy/Z4+F0aGxSE4MqwjyXvW/TzgkLAx2AGHwQ==", + "version": "5.5.2", + "resolved": "https://registry.npmjs.org/typescript/-/typescript-5.5.2.tgz", + "integrity": "sha512-NcRtPEOsPFFWjobJEtfihkLCZCXZt/os3zf8nTxjVH3RvTSxjrCamJpbExGvYOF+tFHc3pA65qpdwPbzjohhew==", "dev": true, "license": "Apache-2.0", "bin": { @@ -13511,9 +13608,9 @@ "license": "BSD-2-Clause" }, "node_modules/webpack": { - "version": "5.92.0", - "resolved": "https://registry.npmjs.org/webpack/-/webpack-5.92.0.tgz", - "integrity": "sha512-Bsw2X39MYIgxouNATyVpCNVWBCuUwDgWtN78g6lSdPJRLaQ/PUVm/oXcaRAyY/sMFoKFQrsPeqvTizWtq7QPCA==", + "version": "5.92.1", + "resolved": "https://registry.npmjs.org/webpack/-/webpack-5.92.1.tgz", + "integrity": "sha512-JECQ7IwJb+7fgUFBlrJzbyu3GEuNBcdqr1LD7IbSzwkSmIevTm8PF+wej3Oxuz/JFBUZ6O1o43zsPkwm1C4TmA==", "license": "MIT", "peer": true, "dependencies": { diff --git a/frontend/package.json b/frontend/package.json index 713cb51b4..058404a5f 100644 --- a/frontend/package.json +++ b/frontend/package.json @@ -12,8 +12,11 @@ "start": "next start", "type-check": "tsc", "lint": "eslint . --config ./.eslintrc.js", - "lint-fix": "eslint . --fix --config ./.eslintrc.js", - "lint-quiet": "eslint . --config ./.eslintrc.js --quiet" + "lint:fix": "eslint . --fix --config ./.eslintrc.js", + "lint:quiet": "eslint . --config ./.eslintrc.js --quiet", + "prettier": "prettier --check 'src/**/*.{js,jsx,ts,tsx,json}'", + "prettier:write": "prettier --write 'src/**/*.{js,jsx,ts,tsx,json}'", + "ts:check": "tsc --noEmit --incremental" }, "dependencies": { "@apollo/client": "^3.10.4", @@ -56,11 +59,13 @@ "autoprefixer": "^10.4.17", "eslint": "^8.46.0", "eslint-config-next": "^14.2.4", + "eslint-config-prettier": "^9.1.0", "eslint-config-standard-with-typescript": "^37.0.0", "eslint-plugin-import": "^2.28.0", "eslint-plugin-n": "^16.0.1", "eslint-plugin-react": "^7.32.0", "eslint-plugin-react-hooks": "^4.3.0", + "eslint-plugin-simple-import-sort": "^12.1.0", "jest": "^29.6.2", "tailwindcss": "^3.0.21", "ts-jest": "^29.1.1", diff --git a/frontend/primitives.d.ts b/frontend/primitives.d.ts index f87673ed9..ab4ee9302 100644 --- a/frontend/primitives.d.ts +++ b/frontend/primitives.d.ts @@ -1,5 +1,5 @@ -import * as borsh_lib_types_types from 'borsh/lib/types/types'; -import * as borsh from 'borsh'; +import type * as borsh_lib_types_types from 'borsh/lib/types/types'; +import type * as borsh from 'borsh'; import * as borsher from 'borsher'; function _mergeNamespaces(n, m) { diff --git a/frontend/src/classes/ValidationError.js b/frontend/src/classes/ValidationError.js index 9325dd8fc..da308a9a2 100644 --- a/frontend/src/classes/ValidationError.js +++ b/frontend/src/classes/ValidationError.js @@ -1,6 +1,6 @@ export class ValidationError extends Error { - constructor(message, type) { - super(message); - this.type = type; - } + constructor(message, type) { + super(message); + this.type = type; + } } diff --git a/frontend/src/components/Common/CustomTooltip.tsx b/frontend/src/components/Common/CustomTooltip.tsx index e26c71d1b..fb273164c 100644 --- a/frontend/src/components/Common/CustomTooltip.tsx +++ b/frontend/src/components/Common/CustomTooltip.tsx @@ -8,16 +8,20 @@ export enum TooltipDirection { } interface CustomTooltipProps { - message: string - direction?: TooltipDirection - children: React.ReactElement + message: string; + direction?: TooltipDirection; + children: React.ReactElement; } const CustomTooltip: React.FC = ({ message, direction = TooltipDirection.Top, children }) => { const [visible, setVisible] = useState(false); - const showTooltip = (): void => { setVisible(true); }; - const hideTooltip = (): void => { setVisible(false); }; + const showTooltip = (): void => { + setVisible(true); + }; + const hideTooltip = (): void => { + setVisible(false); + }; const getTooltipPositionClass = (direction: TooltipDirection | undefined): string => { switch (direction) { @@ -46,7 +50,9 @@ const CustomTooltip: React.FC = ({ message, direction = Tool 'absolute', 'text-center', 'w-max', - ].filter(Boolean).join(' '); + ] + .filter(Boolean) + .join(' '); return (
diff --git a/frontend/src/components/Common/LatestBlock.tsx b/frontend/src/components/Common/LatestBlock.tsx index c32f297aa..b03394f3e 100644 --- a/frontend/src/components/Common/LatestBlock.tsx +++ b/frontend/src/components/Common/LatestBlock.tsx @@ -2,18 +2,18 @@ import React, { useState, useEffect } from 'react'; import { calculateBlockTimeDifference } from '@/utils/calculateBlockTimeDifference'; interface LatestBlockProps { - indexerBlockHeight?: number + indexerBlockHeight?: number; } interface BlockResponse { result?: { header?: { - height?: number - } - } + height?: number; + }; + }; error?: { - message?: string - } + message?: string; + }; } const LatestBlock: React.FC = (props) => { @@ -62,25 +62,28 @@ const LatestBlock: React.FC = (props) => { } }; - updateFinalBlock() - .catch(error => { - console.error('Failed to fetch or process data:', error); - }); + updateFinalBlock().catch((error) => { + console.error('Failed to fetch or process data:', error); + }); const intervalId = setInterval(() => { - updateFinalBlock() - .catch(error => { - console.error('Failed to update final block:', error); - }); + updateFinalBlock().catch((error) => { + console.error('Failed to update final block:', error); + }); }, 1000); - return () => { clearInterval(intervalId); }; + return () => { + clearInterval(intervalId); + }; }, []); return (
{latestFinalBlock !== null && props.indexerBlockHeight !== undefined - ? `Indexer is ${latestFinalBlock - props.indexerBlockHeight} blocks or ${calculateBlockTimeDifference(latestFinalBlock, props.indexerBlockHeight)} behind the blockchain tip` + ? `Indexer is ${latestFinalBlock - props.indexerBlockHeight} blocks or ${calculateBlockTimeDifference( + latestFinalBlock, + props.indexerBlockHeight, + )} behind the blockchain tip` : `Indexer is not yet synced Latest Final Block Height: ${latestFinalBlock as number}`} {errors &&
Error: {errors}
}
diff --git a/frontend/src/components/CreateNewIndexer/CreateNewIndexer.js b/frontend/src/components/CreateNewIndexer/CreateNewIndexer.js index 555a55765..0d28d02e7 100644 --- a/frontend/src/components/CreateNewIndexer/CreateNewIndexer.js +++ b/frontend/src/components/CreateNewIndexer/CreateNewIndexer.js @@ -1,11 +1,7 @@ import Editor from '@/components/Editor'; const CreateNewIndexer = () => { - return ( - ); + return ; }; export default CreateNewIndexer; diff --git a/frontend/src/components/CreateNewIndexer/index.js b/frontend/src/components/CreateNewIndexer/index.js index 2f4ebb89a..4ef4470ba 100644 --- a/frontend/src/components/CreateNewIndexer/index.js +++ b/frontend/src/components/CreateNewIndexer/index.js @@ -1,2 +1,2 @@ -import CreateNewIndexer from "./CreateNewIndexer"; +import CreateNewIndexer from './CreateNewIndexer'; export default CreateNewIndexer; diff --git a/frontend/src/components/Editor/DiffEditorComponent.jsx b/frontend/src/components/Editor/DiffEditorComponent.jsx index 42ca1fca6..1bffd6647 100644 --- a/frontend/src/components/Editor/DiffEditorComponent.jsx +++ b/frontend/src/components/Editor/DiffEditorComponent.jsx @@ -1,13 +1,6 @@ -import { DiffEditor } from "@monaco-editor/react"; +import { DiffEditor } from '@monaco-editor/react'; -export const DiffEditorComponent = ({ - original, - modified, - language, - readOnly, - options, - onMount, -}) => ( +export const DiffEditorComponent = ({ original, modified, language, readOnly, options, onMount }) => ( { - const { - indexerDetails, - setShowResetCodeModel, - setShowPublishModal, - debugMode, - isCreateNewIndexer, - setAccountId, - } = useContext(IndexerDetailsContext); - - const DEBUG_LIST_STORAGE_KEY = `QueryAPI:debugList:${indexerDetails.accountId}#${indexerDetails.indexerName || "new"}`; - const SCHEMA_STORAGE_KEY = `QueryAPI:Schema:${indexerDetails.accountId}#${indexerDetails.indexerName || "new"}`; - const SCHEMA_TYPES_STORAGE_KEY = `QueryAPI:Schema:Types:${indexerDetails.accountId}#${indexerDetails.indexerName || "new"}`; - const CODE_STORAGE_KEY = `QueryAPI:Code:${indexerDetails.accountId}#${indexerDetails.indexerName || "new"}`; - const SCHEMA_TAB_NAME = "schema.sql"; + const { indexerDetails, setShowResetCodeModel, setShowPublishModal, debugMode, isCreateNewIndexer, setAccountId } = + useContext(IndexerDetailsContext); + + const DEBUG_LIST_STORAGE_KEY = `QueryAPI:debugList:${indexerDetails.accountId}#${ + indexerDetails.indexerName || 'new' + }`; + const SCHEMA_STORAGE_KEY = `QueryAPI:Schema:${indexerDetails.accountId}#${indexerDetails.indexerName || 'new'}`; + const SCHEMA_TYPES_STORAGE_KEY = `QueryAPI:Schema:Types:${indexerDetails.accountId}#${ + indexerDetails.indexerName || 'new' + }`; + const CODE_STORAGE_KEY = `QueryAPI:Code:${indexerDetails.accountId}#${indexerDetails.indexerName || 'new'}`; + const SCHEMA_TAB_NAME = 'schema.sql'; const [blockHeightError, setBlockHeightError] = useState(undefined); const [error, setError] = useState(); - const [fileName, setFileName] = useState("indexingLogic.js"); + const [fileName, setFileName] = useState('indexingLogic.js'); const [originalSQLCode, setOriginalSQLCode] = useState(formatSQL(defaultSchema)); const [originalIndexingCode, setOriginalIndexingCode] = useState(formatIndexingCode(defaultCode)); @@ -85,7 +88,6 @@ const Editor = ({ actionButtonText }) => { const disposableRef = useRef(null); const monacoEditorRef = useRef(null); - const parseGlyphError = (error, line) => { const { line: startLine, column: startColumn } = line?.start || { line: 1, column: 1 }; const { line: endLine, column: endColumn } = line?.end || { line: 1, column: 1 }; @@ -99,11 +101,11 @@ const Editor = ({ actionButtonText }) => { range: new monaco.Range(startLine, startColumn, endLine, endColumn), options: { isWholeLine: true, - glyphMarginClassName: error ? "glyphError" : "glyphSuccess", + glyphMarginClassName: error ? 'glyphError' : 'glyphSuccess', glyphMarginHoverMessage: { value: displayedError }, }, }, - ] + ], ); }; @@ -120,9 +122,7 @@ const Editor = ({ actionButtonText }) => { useEffect(() => { if (indexerDetails.code != null) { - const { data: formattedCode, error: codeError } = validateJSCode( - indexerDetails.code - ); + const { data: formattedCode, error: codeError } = validateJSCode(indexerDetails.code); if (codeError) { setError(CODE_FORMATTING_ERROR_MESSAGE); @@ -135,9 +135,7 @@ const Editor = ({ actionButtonText }) => { useEffect(() => { if (indexerDetails.schema != null) { - const { data: formattedSchema, error: schemaError } = validateSQLSchema( - indexerDetails.schema - ); + const { data: formattedSchema, error: schemaError } = validateSQLSchema(indexerDetails.schema); if (schemaError?.type === FORMATTING_ERROR_TYPE) { setError(SCHEMA_FORMATTING_ERROR_MESSAGE); @@ -163,7 +161,6 @@ const Editor = ({ actionButtonText }) => { handleCodeGen(); } if (fileName === SCHEMA_TAB_NAME) debouncedValidateSQLSchema(schema); - }, [fileName]); useEffect(() => { @@ -202,7 +199,7 @@ const Editor = ({ actionButtonText }) => { // eslint-disable-next-line no-undef const newDisposable = monaco.languages.typescript.typescriptDefaults.addExtraLib(schemaTypes); if (newDisposable != null) { - console.log("Types successfully imported to Editor"); + console.log('Types successfully imported to Editor'); } disposableRef.current = newDisposable; @@ -212,44 +209,37 @@ const Editor = ({ actionButtonText }) => { const forkIndexer = async (indexerName) => { let code = indexingCode; setAccountId(currentUserAccountId); - let prevAccountId = indexerDetails.accountId.replaceAll(".", "_"); - let newAccountId = currentUserAccountId.replaceAll(".", "_"); - let prevIndexerName = indexerDetails.indexerName - .replaceAll("-", "_") - .trim() - .toLowerCase(); - let newIndexerName = indexerName.replaceAll("-", "_").trim().toLowerCase(); + let prevAccountId = indexerDetails.accountId.replaceAll('.', '_'); + let newAccountId = currentUserAccountId.replaceAll('.', '_'); + let prevIndexerName = indexerDetails.indexerName.replaceAll('-', '_').trim().toLowerCase(); + let newIndexerName = indexerName.replaceAll('-', '_').trim().toLowerCase(); code = code.replaceAll(prevAccountId, newAccountId); code = code.replaceAll(prevIndexerName, newIndexerName); setIndexingCode(formatIndexingCode(code)); }; const registerFunction = async (indexerName, indexerConfig) => { - const { data: validatedSchema, error: schemaValidationError } = - validateSQLSchema(schema); - const { data: validatedCode, error: codeValidationError } = - validateJSCode(indexingCode); + const { data: validatedSchema, error: schemaValidationError } = validateSQLSchema(schema); + const { data: validatedCode, error: codeValidationError } = validateJSCode(indexingCode); if (codeValidationError) { setError(CODE_FORMATTING_ERROR_MESSAGE); return; } - let innerCode = validatedCode.match( - /getBlock\s*\([^)]*\)\s*{([\s\S]*)}/ - )[1]; - indexerName = indexerName.replaceAll(" ", "_"); + let innerCode = validatedCode.match(/getBlock\s*\([^)]*\)\s*{([\s\S]*)}/)[1]; + indexerName = indexerName.replaceAll(' ', '_'); let forkedFrom = - (indexerDetails.forkedAccountId && indexerDetails.forkedIndexerName) + indexerDetails.forkedAccountId && indexerDetails.forkedIndexerName ? { account_id: indexerDetails.forkedAccountId, function_name: indexerDetails.forkedIndexerName } : null; const startBlock = - indexerConfig.startBlock === "startBlockHeight" + indexerConfig.startBlock === 'startBlockHeight' ? { HEIGHT: indexerConfig.height } - : indexerConfig.startBlock === "startBlockLatest" - ? "LATEST" - : "CONTINUE"; + : indexerConfig.startBlock === 'startBlockLatest' + ? 'LATEST' + : 'CONTINUE'; if (schemaValidationError?.type === FORMATTING_ERROR_TYPE) { setError(SCHEMA_FORMATTING_ERROR_MESSAGE); @@ -266,20 +256,20 @@ const Editor = ({ actionButtonText }) => { return; } - request("register-function", { + request('register-function', { indexerName: indexerName, code: innerCode, schema: validatedSchema, startBlock, contractFilter: indexerConfig.filter, - ...(forkedFrom && { forkedFrom }) + ...(forkedFrom && { forkedFrom }), }); setShowPublishModal(false); }; const handleDeleteIndexer = () => { - request("delete-indexer", { + request('delete-indexer', { accountId: indexerDetails.accountId, indexerName: indexerDetails.indexerName, }); @@ -294,10 +284,7 @@ const Editor = ({ actionButtonText }) => { return; } - const data = await queryIndexerFunctionDetails( - indexerDetails.accountId, - indexerDetails.indexerName - ); + const data = await queryIndexerFunctionDetails(indexerDetails.accountId, indexerDetails.indexerName); if (data == null) { setIndexingCode(defaultCode); setSchema(defaultSchema); @@ -315,10 +302,7 @@ const Editor = ({ actionButtonText }) => { setSchema(unformatted_schema); } - const { formattedCode, formattedSchema } = reformatAll( - unformatted_wrapped_indexing_code, - unformatted_schema - ); + const { formattedCode, formattedSchema } = reformatAll(unformatted_wrapped_indexing_code, unformatted_schema); setIndexingCode(formattedCode); setSchema(formattedSchema); } catch (formattingError) { @@ -330,15 +314,13 @@ const Editor = ({ actionButtonText }) => { const getActionButtonText = () => { const isUserIndexer = indexerDetails.accountId === currentUserAccountId; - if (isCreateNewIndexer) return "Create New Indexer"; - return isUserIndexer ? actionButtonText : "Fork Indexer"; + if (isCreateNewIndexer) return 'Create New Indexer'; + return isUserIndexer ? actionButtonText : 'Fork Indexer'; }; const reformatAll = (indexingCode, schema) => { - let { data: formattedCode, error: codeError } = - validateJSCode(indexingCode); - let { data: formattedSchema, error: schemaError } = - validateSQLSchema(schema); + let { data: formattedCode, error: codeError } = validateJSCode(indexingCode); + let { data: formattedSchema, error: schemaError } = validateSQLSchema(schema); if (codeError) { formattedCode = indexingCode; @@ -361,85 +343,57 @@ const Editor = ({ actionButtonText }) => { setSchemaTypes(pgSchemaTypeGen.generateTypes(schema)); attachTypesToMonaco(); // Just in case schema types have been updated but weren't added to monaco } catch (_error) { - console.error("Error generating types for saved schema.\n", _error); + console.error('Error generating types for saved schema.\n', _error); setError(SCHEMA_TYPE_GENERATION_ERROR_MESSAGE); } } function handleFormating() { - const { formattedCode, formattedSchema } = reformatAll( - indexingCode, - schema - ); + const { formattedCode, formattedSchema } = reformatAll(indexingCode, schema); setIndexingCode(formattedCode); setSchema(formattedSchema); } function handleEditorWillMount(editor, monaco) { if (!diffView) { - const decorations = editor.deltaDecorations([], + const decorations = editor.deltaDecorations( + [], [ { range: new monaco.Range(1, 1, 1, 1), options: {}, }, - ] + ], ); monacoEditorRef.current = editor; setDecorations(decorations); } monaco.languages.typescript.typescriptDefaults.addExtraLib( `${primitives}}`, - "file:///node_modules/@near-lake/primitives/index.d.ts" + 'file:///node_modules/@near-lake/primitives/index.d.ts', ); setMonacoMount(true); } - async function executeIndexerFunction( - option = "latest", - startingBlockHeight = null - ) { + async function executeIndexerFunction(option = 'latest', startingBlockHeight = null) { setIsExecutingIndexerFunction(() => true); - const schemaName = indexerDetails.accountId - .concat("_", indexerDetails.indexerName) - .replace(/[^a-zA-Z0-9]/g, "_"); + const schemaName = indexerDetails.accountId.concat('_', indexerDetails.indexerName).replace(/[^a-zA-Z0-9]/g, '_'); let latestHeight; switch (option) { - case "debugList": - await indexerRunner.executeIndexerFunctionOnHeights( - heights, - indexingCode, - schema, - schemaName, - option - ); + case 'debugList': + await indexerRunner.executeIndexerFunctionOnHeights(heights, indexingCode, schema, schemaName, option); break; - case "specific": + case 'specific': if (startingBlockHeight === null && Number(startingBlockHeight) === 0) { - console.log( - "Invalid Starting Block Height: starting block height is null or 0" - ); + console.log('Invalid Starting Block Height: starting block height is null or 0'); break; } - await indexerRunner.start( - startingBlockHeight, - indexingCode, - schema, - schemaName, - option - ); + await indexerRunner.start(startingBlockHeight, indexingCode, schema, schemaName, option); break; - case "latest": + case 'latest': latestHeight = await getLatestBlockHeight(); - if (latestHeight) - await indexerRunner.start( - latestHeight - 10, - indexingCode, - schema, - schemaName, - option - ); + if (latestHeight) await indexerRunner.start(latestHeight - 10, indexingCode, schema, schemaName, option); } setIsExecutingIndexerFunction(() => false); } @@ -455,17 +409,17 @@ const Editor = ({ actionButtonText }) => { } function handleRegisterIndexerWithErrors(args) { - request("register-function", args); + request('register-function', args); } return ( <>
{!indexerDetails.code && !isCreateNewIndexer && ( @@ -475,7 +429,6 @@ const Editor = ({ actionButtonText }) => { > Indexer Function could not be found. Are you sure this indexer exists? - )} {(indexerDetails.code || isCreateNewIndexer) && ( <> @@ -506,7 +459,6 @@ const Editor = ({ actionButtonText }) => { latestHeight={height} isUserIndexer={indexerDetails.accountId === currentUserAccountId} handleDeleteIndexer={handleDeleteIndexer} - fileName={fileName} setFileName={setFileName} diffView={diffView} @@ -524,10 +476,10 @@ const Editor = ({ actionButtonText }) => {
{error && ( @@ -556,9 +508,7 @@ const Editor = ({ actionButtonText }) => { diffView={diffView} setDiffView={setDiffView} /> - + { if (heights.length > 0) { - executeIndexerFunction("debugList"); + executeIndexerFunction('debugList'); } else if (inputValue) { - executeIndexerFunction("specific", inputValue); + executeIndexerFunction('specific', inputValue); } else { - executeIndexerFunction("latest"); + executeIndexerFunction('latest'); } }} > @@ -56,7 +56,7 @@ const BlockPickerView = ({ className="flex items-center justify-center px-2 py-1 bg-gray-200 hover:bg-gray-300 text-gray-700 text-xs rounded" onClick={() => { if (!heights.length) return; - executeIndexerFunction("latest"); + executeIndexerFunction('latest'); }} disabled={!heights.length} > @@ -65,7 +65,7 @@ const BlockPickerView = ({
- - - ); export default BlockPickerView; diff --git a/frontend/src/components/Editor/EditorView/DeveloperToolsView.jsx b/frontend/src/components/Editor/EditorView/DeveloperToolsView.jsx index a2308d23a..d9d05a2ac 100644 --- a/frontend/src/components/Editor/EditorView/DeveloperToolsView.jsx +++ b/frontend/src/components/Editor/EditorView/DeveloperToolsView.jsx @@ -1,126 +1,142 @@ -import { useState } from "react"; -import { ArrowCounterclockwise, Justify, Code } from "react-bootstrap-icons"; -import BlockPickerContainer from "../EditorViewContainer/BlockPickerContainer"; -import CustomTooltip, { TooltipDirection } from "@/components/Common/CustomTooltip"; +import { useState } from 'react'; +import { ArrowCounterclockwise, Justify, Code } from 'react-bootstrap-icons'; +import BlockPickerContainer from '../EditorViewContainer/BlockPickerContainer'; +import CustomTooltip, { TooltipDirection } from '@/components/Common/CustomTooltip'; const DeveloperToolsView = ({ - handleFormating, - handleCodeGen, - setShowResetCodeModel, - debugMode, - setDebugMode, - heights, - setHeights, - latestHeight, - isExecuting, - stopExecution, - removeHeight, - executeIndexerFunction, - diffView, - setDiffView, + handleFormating, + handleCodeGen, + setShowResetCodeModel, + debugMode, + setDebugMode, + heights, + setHeights, + latestHeight, + isExecuting, + stopExecution, + removeHeight, + executeIndexerFunction, + diffView, + setDiffView, }) => { - const [hoveredIndex, setHoveredIndex] = useState(null); + const [hoveredIndex, setHoveredIndex] = useState(null); - return ( -
-
- Developer Tools -
-
-
- - + return ( +
+
+ Developer Tools +
+
+
+ + - - - + + + +
-
- -
- Diff View - { - setDiffView(e.target.checked); - }} - /> -
-
- Debug Mode - { - setDebugMode(e.target.checked); - }} - /> -
-
+
+ Diff View + { + setDiffView(e.target.checked); + }} + /> +
+
+ Debug Mode + { + setDebugMode(e.target.checked); + }} + /> +
+
-
- {debugMode && ( - <> - {typeof debugMode === 'boolean' && debugMode && ( -
- -
- )} - - )} -
-
- {debugMode && heights.length > 0 && ( -
-
-

- Selected Block Heights: [ - - {heights.map((height, index) => ( - removeHeight(index)} - onMouseEnter={() => setHoveredIndex(index)} - onMouseLeave={() => setHoveredIndex(null)} - style={{ - color: index === hoveredIndex ? "#EF4444" : "", - }} - > - {height} - {index !== heights.length - 1 && ", "} - - ))} - - ] -

-
+
+ {debugMode && ( + <> + {typeof debugMode === 'boolean' && debugMode && ( +
+
- - )} + )} + + )}
- - ); +
+ {debugMode && heights.length > 0 && ( +
+
+

+ Selected Block Heights: [ + + {heights.map((height, index) => ( + removeHeight(index)} + onMouseEnter={() => setHoveredIndex(index)} + onMouseLeave={() => setHoveredIndex(null)} + style={{ + color: index === hoveredIndex ? '#EF4444' : '', + }} + > + {height} + {index !== heights.length - 1 && ', '} + + ))} + + ] +

+
+
+ )} +
+ ); }; export default DeveloperToolsView; diff --git a/frontend/src/components/Editor/EditorView/EditorMenuView.jsx b/frontend/src/components/Editor/EditorView/EditorMenuView.jsx index b9bbbcc40..f1df07d9e 100644 --- a/frontend/src/components/Editor/EditorView/EditorMenuView.jsx +++ b/frontend/src/components/Editor/EditorView/EditorMenuView.jsx @@ -1,7 +1,7 @@ -import React from "react"; +import React from 'react'; import { Navbar, Container, InputGroup, Form, ButtonGroup, Button } from 'react-bootstrap'; import { Braces, ArrowCounterclockwise, FileText, TrashFill } from 'react-bootstrap-icons'; -import CustomTooltip, { TooltipDirection } from "@/components/Common/CustomTooltip"; +import CustomTooltip, { TooltipDirection } from '@/components/Common/CustomTooltip'; const EditorMenuView = ({ indexerName, @@ -15,60 +15,90 @@ const EditorMenuView = ({ isCreateNewIndexer, setShowLogsView, isUserIndexer, - error + error, }) => { return (
- Indexer: {accountId}/{indexerName ? indexerName : "Null"} + Indexer: {accountId}/{indexerName ? indexerName : 'Null'} - Filter: {!isCreateNewIndexer ? indexerDetails.rule.affected_account_id : - - - Please provide a valid contract name. - - } + Filter:{' '} + {!isCreateNewIndexer ? ( + indexerDetails.rule.affected_account_id + ) : ( + + Please provide a valid contract name. + + )}
{isUserIndexer && !isCreateNewIndexer && ( - - )} {isUserIndexer && !isCreateNewIndexer && ( - )} {!isUserIndexer && !isCreateNewIndexer ? ( - ) : ( - )} {!isCreateNewIndexer && ( - )}
-
+ ); }; diff --git a/frontend/src/components/Editor/EditorViewContainer/BlockPickerContainer.tsx b/frontend/src/components/Editor/EditorViewContainer/BlockPickerContainer.tsx index 248c95977..754d1ad1f 100644 --- a/frontend/src/components/Editor/EditorViewContainer/BlockPickerContainer.tsx +++ b/frontend/src/components/Editor/EditorViewContainer/BlockPickerContainer.tsx @@ -2,12 +2,12 @@ import React, { useState } from 'react'; import BlockPickerView from '../EditorView/BlockPickerView'; interface BlockPickerContainerProps { - heights: string[] - setHeights: React.Dispatch> - executeIndexerFunction: () => void - latestHeight: number - isExecuting: boolean - stopExecution: () => void + heights: string[]; + setHeights: React.Dispatch>; + executeIndexerFunction: () => void; + latestHeight: number; + isExecuting: boolean; + stopExecution: () => void; } const BlockPickerContainer: React.FC = ({ diff --git a/frontend/src/components/Editor/EditorViewContainer/DeveloperToolsContainer.tsx b/frontend/src/components/Editor/EditorViewContainer/DeveloperToolsContainer.tsx index 7c914a939..c7013b565 100644 --- a/frontend/src/components/Editor/EditorViewContainer/DeveloperToolsContainer.tsx +++ b/frontend/src/components/Editor/EditorViewContainer/DeveloperToolsContainer.tsx @@ -3,19 +3,19 @@ import DeveloperToolsView from '../EditorView/DeveloperToolsView'; import { IndexerDetailsContext } from '../../../contexts/IndexerDetailsContext'; interface DeveloperToolsContainerProps { - handleFormating: () => void - handleCodeGen: () => void - executeIndexerFunction: () => void - isExecuting: boolean - stopExecution: () => void - heights: string[] - setHeights: React.Dispatch> - latestHeight: number - isUserIndexer: boolean - handleDeleteIndexer: () => void - error: string - diffView: boolean - setDiffView: React.Dispatch> + handleFormating: () => void; + handleCodeGen: () => void; + executeIndexerFunction: () => void; + isExecuting: boolean; + stopExecution: () => void; + heights: string[]; + setHeights: React.Dispatch>; + latestHeight: number; + isUserIndexer: boolean; + handleDeleteIndexer: () => void; + error: string; + diffView: boolean; + setDiffView: React.Dispatch>; } const DeveloperToolsContainer: React.FC = ({ @@ -51,22 +51,24 @@ const DeveloperToolsContainer: React.FC = ({ }; return ( - + ); }; diff --git a/frontend/src/components/Editor/EditorViewContainer/EditorMenuContainer.jsx b/frontend/src/components/Editor/EditorViewContainer/EditorMenuContainer.jsx index 7e9c28902..fa381d5cc 100644 --- a/frontend/src/components/Editor/EditorViewContainer/EditorMenuContainer.jsx +++ b/frontend/src/components/Editor/EditorViewContainer/EditorMenuContainer.jsx @@ -1,23 +1,37 @@ -import React, { useContext } from "react"; -import EditorMenuView from "../EditorView/EditorMenuView"; +import React, { useContext } from 'react'; +import EditorMenuView from '../EditorView/EditorMenuView'; import { IndexerDetailsContext } from '../../../contexts/IndexerDetailsContext'; const EditorMenuContainer = (props) => { - const { - handleFormating, - handleCodeGen, - error, - executeIndexerFunction, - heights, - setHeights, - isCreateNewIndexer, - isExecuting, - stopExecution, - latestHeight, - isUserIndexer, - handleDeleteIndexer, - } = props; - const { + const { + handleFormating, + handleCodeGen, + error, + executeIndexerFunction, + heights, + setHeights, + isCreateNewIndexer, + isExecuting, + stopExecution, + latestHeight, + isUserIndexer, + handleDeleteIndexer, + } = props; + const { + indexerName, + accountId, + indexerDetails, + setShowPublishModal, + setShowResetCodeModel, + setShowForkIndexerModal, + debugMode, + setShowLogsView, + } = useContext(IndexerDetailsContext); + + return ( + { setShowResetCodeModel, setShowForkIndexerModal, debugMode, + isCreateNewIndexer, setShowLogsView, - } = useContext(IndexerDetailsContext); - - return ( - - ); + handleDeleteIndexer, + }} + /> + ); }; export default EditorMenuContainer; diff --git a/frontend/src/components/Editor/FileSwitcher.jsx b/frontend/src/components/Editor/FileSwitcher.jsx index 50758055f..5d220cafe 100644 --- a/frontend/src/components/Editor/FileSwitcher.jsx +++ b/frontend/src/components/Editor/FileSwitcher.jsx @@ -1,43 +1,36 @@ -import React, { useContext } from "react"; +import React, { useContext } from 'react'; import { IndexerDetailsContext } from '@/contexts/IndexerDetailsContext'; -export function FileSwitcher({ - fileName, - setFileName, -}) { +export function FileSwitcher({ fileName, setFileName }) { const { isCreateNewIndexer } = useContext(IndexerDetailsContext); return (
{!isCreateNewIndexer && ( )}
- ); } diff --git a/frontend/src/components/Editor/GlyphContainer.js b/frontend/src/components/Editor/GlyphContainer.js index e1e74a470..251cc5124 100644 --- a/frontend/src/components/Editor/GlyphContainer.js +++ b/frontend/src/components/Editor/GlyphContainer.js @@ -1,12 +1,12 @@ // Used to render Glyphs in the Editor -import styled from "styled-components"; +import styled from 'styled-components'; const GlyphContainer = styled.div` -.glyphSuccess { + .glyphSuccess { background: transparent; - } + } -.glyphError { + .glyphError { background: red; } `; diff --git a/frontend/src/components/Editor/MonacoEditorComponent.jsx b/frontend/src/components/Editor/MonacoEditorComponent.jsx index 4dda234ef..041d7e08c 100644 --- a/frontend/src/components/Editor/MonacoEditorComponent.jsx +++ b/frontend/src/components/Editor/MonacoEditorComponent.jsx @@ -1,4 +1,4 @@ -import MonacoEditor from "@monaco-editor/react"; +import MonacoEditor from '@monaco-editor/react'; export const MonacoEditorComponent = ({ value, defaultValue, diff --git a/frontend/src/components/Editor/ResizableLayoutEditor.jsx b/frontend/src/components/Editor/ResizableLayoutEditor.jsx index c48c1e997..ed3e5be74 100644 --- a/frontend/src/components/Editor/ResizableLayoutEditor.jsx +++ b/frontend/src/components/Editor/ResizableLayoutEditor.jsx @@ -1,27 +1,27 @@ -import { DiffEditorComponent } from "./DiffEditorComponent"; -import { MonacoEditorComponent } from "./MonacoEditorComponent"; -import { defaultCode, defaultSchema } from "@/utils/formatters"; -import { useDragResize } from "@/utils/resize"; -import GraphqlPlayground from "./../Playground"; +import { DiffEditorComponent } from './DiffEditorComponent'; +import { MonacoEditorComponent } from './MonacoEditorComponent'; +import { defaultCode, defaultSchema } from '@/utils/formatters'; +import { useDragResize } from '@/utils/resize'; +import GraphqlPlayground from './../Playground'; const containerStyle = { - display: "flex", - flexDirection: "row", - width: "100%", - height: "100%", + display: 'flex', + flexDirection: 'row', + width: '100%', + height: '100%', }; const editorContainerStyle = { - width: "100%", - display: "flex", - justifyContent: "center", - minWidth: "100px", + width: '100%', + display: 'flex', + justifyContent: 'center', + minWidth: '100px', }; const dragBarStyle = { - width: "10px", - backgroundColor: "gray", - cursor: "col-resize", + width: '10px', + backgroundColor: 'gray', + cursor: 'col-resize', }; const ResizableEditor = ({ @@ -41,7 +41,7 @@ const ResizableEditor = ({ isCreateNewIndexer, }) => { const { firstRef, secondRef, dragBarRef } = useDragResize({ - direction: "horizontal", + direction: 'horizontal', initiallyHidden: null, defaultSizeRelation: 3, sizeThresholdFirst: 60, @@ -51,7 +51,7 @@ const ResizableEditor = ({ // Render logic based on fileName const editorComponents = { GraphiQL: () => , - "indexingLogic.js": () => + 'indexingLogic.js': () => diffView ? ( ), - "schema.sql": () => + 'schema.sql': () => diffView ? ( ), @@ -151,17 +151,17 @@ export default function ResizableLayoutEditor({ firstRef: firstRefEditor, secondRef: secondRefConsole, } = useDragResize({ - direction: "vertical", - initiallyHidden: "second", + direction: 'vertical', + initiallyHidden: 'second', defaultSizeRelation: 3, sizeThresholdFirst: 60, sizeThresholdSecond: 20, }); return ( -
+
{/* Code Editor */} -
+
{ - beforeEach(() => { - render(); - }); + beforeEach(() => { + render( + , + ); + }); - test('renders and displays the component without errors', () => { - expect(screen.getByTestId('action-button-group')).toBeInTheDocument(); - expect(screen.getByTestId('indexing-logic-file-button')).toBeInTheDocument(); - expect(screen.getByTestId('schema-file-button')).toBeInTheDocument(); - expect(screen.getByTestId('diff-view-switch')).toBeInTheDocument(); - }); + test('renders and displays the component without errors', () => { + expect(screen.getByTestId('action-button-group')).toBeInTheDocument(); + expect(screen.getByTestId('indexing-logic-file-button')).toBeInTheDocument(); + expect(screen.getByTestId('schema-file-button')).toBeInTheDocument(); + expect(screen.getByTestId('diff-view-switch')).toBeInTheDocument(); + }); - test('verifies the visibility and functionality of buttons: Reset, Format Code, and Register Function', async () => { - const resetButton = screen.getByTestId('reset-button'); - const formatButton = screen.getByTestId('format-code-button'); - const registerButton = screen.getByTestId('submit-code-button'); + test('verifies the visibility and functionality of buttons: Reset, Format Code, and Register Function', async () => { + const resetButton = screen.getByTestId('reset-button'); + const formatButton = screen.getByTestId('format-code-button'); + const registerButton = screen.getByTestId('submit-code-button'); - expect(resetButton).toBeInTheDocument(); - expect(formatButton).toBeInTheDocument(); - expect(registerButton).toBeInTheDocument(); + expect(resetButton).toBeInTheDocument(); + expect(formatButton).toBeInTheDocument(); + expect(registerButton).toBeInTheDocument(); - fireEvent.click(resetButton); - await waitFor(() => expect(screen.queryByText('Are you sure?')).toBeInTheDocument()); + fireEvent.click(resetButton); + await waitFor(() => expect(screen.queryByText('Are you sure?')).toBeInTheDocument()); - fireEvent.click(formatButton); - await waitFor(() => expect(screen.queryByText('Oh snap! We could not format your code. Make sure it is proper Javascript code.')).not.toBeInTheDocument()); + fireEvent.click(formatButton); + await waitFor(() => + expect( + screen.queryByText('Oh snap! We could not format your code. Make sure it is proper Javascript code.'), + ).not.toBeInTheDocument(), + ); - fireEvent.click(registerButton); - }); + fireEvent.click(registerButton); + }); - test('ensures that the component loads the default or stored values for the indexing code and SQL schema', () => { - expect(screen.getByTestId('code-editor-indexing-logic')).toBeInTheDocument(); - expect(screen.getByTestId('schema-editor-schema')).toBeInTheDocument(); - }); + test('ensures that the component loads the default or stored values for the indexing code and SQL schema', () => { + expect(screen.getByTestId('code-editor-indexing-logic')).toBeInTheDocument(); + expect(screen.getByTestId('schema-editor-schema')).toBeInTheDocument(); + }); - test('confirming that the component handles formatting errors and displays an error message when the indexing code or SQL schema is not valid', async () => { - const invalidCode = 'function invalidCode) {}'; - // await new Promise((r) => setTimeout(r, 3000)); - const indexingEditor = screen.getByTestId('code-editor-indexing-logic'); - // fireEvent.click(screen.getByTestId('indexing-logic-file-button')); - // userEvent.type(indexingEditor, invalidCode); + test('confirming that the component handles formatting errors and displays an error message when the indexing code or SQL schema is not valid', async () => { + const invalidCode = 'function invalidCode) {}'; + // await new Promise((r) => setTimeout(r, 3000)); + const indexingEditor = screen.getByTestId('code-editor-indexing-logic'); + // fireEvent.click(screen.getByTestId('indexing-logic-file-button')); + // userEvent.type(indexingEditor, invalidCode); - // fireEvent.click(screen.getByTestId('format-code-button')); - // await waitFor(() => expect(screen.queryByText('Oh snap! We could not format your code. Make sure it is proper Javascript code.')).toBeInTheDocument()); - }); + // fireEvent.click(screen.getByTestId('format-code-button')); + // await waitFor(() => expect(screen.queryByText('Oh snap! We could not format your code. Make sure it is proper Javascript code.')).toBeInTheDocument()); + }); - test('testing the Diff View switch and making sure the component switches between normal and diff view as expected', () => { - const diffViewSwitch = screen.getByTestId('diff-view-switch'); - fireEvent.click(diffViewSwitch); - expect(screen.getByTestId('diff-editor-indexing-logic')).toBeInTheDocument(); - fireEvent.click(diffViewSwitch); - expect(screen.getByTestId('code-editor-indexing-logic')).toBeInTheDocument(); - }); + test('testing the Diff View switch and making sure the component switches between normal and diff view as expected', () => { + const diffViewSwitch = screen.getByTestId('diff-view-switch'); + fireEvent.click(diffViewSwitch); + expect(screen.getByTestId('diff-editor-indexing-logic')).toBeInTheDocument(); + fireEvent.click(diffViewSwitch); + expect(screen.getByTestId('code-editor-indexing-logic')).toBeInTheDocument(); + }); - test('checking the component behavior when resetting the code and reloading the original code and schema', async () => { - fireEvent.click(screen.getByTestId('reset-button')); - await waitFor(() => expect(screen.queryByText('Are you sure?')).toBeInTheDocument()); - fireEvent.click(screen.getByText('Reload')); - await waitFor(() => expect(screen.queryByText('Are you sure?')).not.toBeInTheDocument()); - expect(screen.getByTestId('code-editor-indexing-logic')).toBeInTheDocument(); - expect(screen.getByTestId('schema-editor-schema')).toBeInTheDocument(); - }); + test('checking the component behavior when resetting the code and reloading the original code and schema', async () => { + fireEvent.click(screen.getByTestId('reset-button')); + await waitFor(() => expect(screen.queryByText('Are you sure?')).toBeInTheDocument()); + fireEvent.click(screen.getByText('Reload')); + await waitFor(() => expect(screen.queryByText('Are you sure?')).not.toBeInTheDocument()); + expect(screen.getByTestId('code-editor-indexing-logic')).toBeInTheDocument(); + expect(screen.getByTestId('schema-editor-schema')).toBeInTheDocument(); + }); }); diff --git a/frontend/src/components/Editor/block_details.js b/frontend/src/components/Editor/block_details.js index c92abb508..64780ad94 100644 --- a/frontend/src/components/Editor/block_details.js +++ b/frontend/src/components/Editor/block_details.js @@ -1,2 +1,1643 @@ -export const block_details = - { "block": { "author": "continue.poolv1.near", "chunks": [{ "balanceBurnt": "1486252472505500000000", "chunkHash": "3qxSWuuo5QLdgpYSuwtQcAcAoX91b6p55CLxjda2wNNw", "encodedLength": 1772, "encodedMerkleRoot": "ChpbnZdB2NdSaC2ir443zB4Bv6WUMSDBY6n3RfDyEVs", "gasLimit": 1000000000000000, "gasUsed": 16028665261860, "heightCreated": 85376002, "heightIncluded": 85376002, "outcomeRoot": "G5c89nTRQSRYPdJQFcP47YXccbmbrbutNwLZzKUeRG3f", "outgoingReceiptsRoot": "5GrcVRVw9njFAskhcEbUyPZAoKMEEjGnC6edUhupEDkj", "prevBlockHash": "H5X25nVZgx43tQSMcF5bRLphWGN92UVYdvaAhcgW4no7", "prevStateRoot": "CRaTBLfd55cw6i5GfVvVSo1rw5ioZ91hT3ZVo5eBmNCi", "rentPaid": "0", "shardId": 0, "signature": "ed25519:4Fzq1idZFShNtErzbK5uZWDcnWhHTMshVBvXG9c8cvc3YPkHF8SVgRsMym6V3WNHVY29gNaw1kXPCLBApcZ2bA1h", "txRoot": "5uqwhjC7wRJsvidh7gjtRRPvvUVwyefdQCivBMTL1hak", "validatorProposals": [], "validatorReward": "0" }, { "balanceBurnt": "1433581134734200000000", "chunkHash": "BqfFW2NyLJuyp1pGgXd2ffYPGptNswNVH6ZJuTq5zSNa", "encodedLength": 161, "encodedMerkleRoot": "EB5tG2a3aVfXv41ESuQVT1ZvQFvuwKnkYGLc2CDVCf5w", "gasLimit": 1000000000000000, "gasUsed": 19438956235189, "heightCreated": 85376002, "heightIncluded": 85376002, "outcomeRoot": "Cp7GeqLAqZ1Z1GbtCL5nG13wxy6mV6PaRdwLddDHjhVc", "outgoingReceiptsRoot": "DP1DT6QP1tNHM5ZjHMJ3C8G2mHJd8kGoFFNEdZPN8Hyx", "prevBlockHash": "H5X25nVZgx43tQSMcF5bRLphWGN92UVYdvaAhcgW4no7", "prevStateRoot": "73sn3JyP1LU58r1FsTo5ZVgvErdbEGFxtiGExkWwhQUd", "rentPaid": "0", "shardId": 1, "signature": "ed25519:3RYj6fpboT1ALHsVxYtYmofT8fGf1VRhNaJhWY45hsKdePcX3AZb3rdKBAZMThF5PVn5j5boKWHPb9zd1xZWYzex", "txRoot": "11111111111111111111111111111111", "validatorProposals": [], "validatorReward": "0" }, { "balanceBurnt": "242806857164400000000", "chunkHash": "6uDUqFuYwGohFMQ9UyRaHbmHALf8HS9F6WcNnZWypkka", "encodedLength": 1474, "encodedMerkleRoot": "9rtC25N5jqfLxMumUwvupL9MpMNUAEKuXZB8RmfkdD7p", "gasLimit": 1000000000000000, "gasUsed": 2428068571644, "heightCreated": 85376002, "heightIncluded": 85376002, "outcomeRoot": "AvRw1DBT1JFB2KNuahMvYucvttrem74GSEnwMacxbkv6", "outgoingReceiptsRoot": "C2QjTGsH8HcfR6W7ecLYwuYD3Vkw6mi9Pf9tFXuGdMLr", "prevBlockHash": "H5X25nVZgx43tQSMcF5bRLphWGN92UVYdvaAhcgW4no7", "prevStateRoot": "CtxzDGCGs1SAr7GEqrA8gSKCcbaPDMKegnTyPEJmU7mb", "rentPaid": "0", "shardId": 2, "signature": "ed25519:66LFkWB13BEUYRsmockx9LbiBzko9zohGkfbLwJertYWAa5TJY5ELFxPycps6poroh1C7Tfbh6NH7tQpgHZFr43d", "txRoot": "DVr9U4uYvJY1cz5awdSmKm4CovJmr1VcPyqCMonat8bD", "validatorProposals": [], "validatorReward": "0" }, { "balanceBurnt": "599537042985400000000", "chunkHash": "3Q39LzrowGBhqRZvkjDqQNjkfUEYEK6Zis5LUzqvrtFj", "encodedLength": 1837, "encodedMerkleRoot": "2d1EBsn1UspGLYjyFkrzPoawjghZe4fBbWkZfmsPJRbM", "gasLimit": 1000000000000000, "gasUsed": 7599093905353, "heightCreated": 85376002, "heightIncluded": 85376002, "outcomeRoot": "CUmJsZfPtNYEggezXf6wW7NX2i7Vn9yuPz7RWfVKxR7p", "outgoingReceiptsRoot": "8TG5j9ahcEtGUNREhCEJW3SgHyeCzochf4foYxSU7b8Q", "prevBlockHash": "H5X25nVZgx43tQSMcF5bRLphWGN92UVYdvaAhcgW4no7", "prevStateRoot": "EPbgnc5qJFvFGqqAyQQu2SrFcVr6mr3AVbSuXnHvZ1ie", "rentPaid": "0", "shardId": 3, "signature": "ed25519:RgPmNaXij3pNFdMVi1Ex8WXyQiYr12uUyRk5tuoddJvCTwJXcxQZZLCdy1XqB6uB9imMz9NsXQwT1M993NBMsjS", "txRoot": "8V1iVH9TSzWsPtNVorScHc4ha9ESNqNEG13tZiRSDzk7", "validatorProposals": [], "validatorReward": "0" }], "header": { "approvals": ["ed25519:3XJG7eriCesmFBP78Q6iECzr34boAvEKn1nMintoQyf8MGf6TUzkjH91jvnb3putUvCicXDzgQjYrjbqTrgQZdTB", "ed25519:238a4WVjhbpubpMhMi2wKzR6wjoVLZ7788KPE3a3RzkppLzd22CChmEHHEyqsFj5VpWFVr13iTmgcgh2MomMxhxQ", null, "ed25519:3hE7715teaeQE2UNWuZy9T6B7yA833NeBZuEsfYnHWGmGvQtApVwyX5QsVF1NV8cT24yQTus1u98JPjUr3X1XBdG", "ed25519:2JFfKfhtZFLrqNHSKWfojnQ63986w9BDVckL88ny3wzmEgextjkvAAmg12EjsmCwXUdc2VZ6zFBwto2pMVSovHcE", "ed25519:3V28c6KbuvuV21W5h8AsbjfZYQXA7aPJEsBi97xpJWAtULikSqqjSVhoLk6ckiAhG83G2nUWc12M8bcroyRB9Lso", "ed25519:5rp3ZGsLKo6oHrUix29cEDdfMAeiWuPmaQEh4T22pCPFEGMQ4jf5WRG8PZKv8zKGeia7NJrWy5iR88k2KF554c7A", "ed25519:2is1ybPjjGhJGJx7BTWbtXVbQNauwEAZLywtZFGrjLPyXdekxGqG695K3W9q7xnfCJYEXzUxpcANhcL1viG7fG3F", "ed25519:3h8Bx2e8Zb5wRw9Cph29FGm1pvWhopqSfbWzQeeawQAssSAd2fcy32yJGtZabHtLp2VRRznv6CgotS4Cc2JWRjUR", "ed25519:v7cD1AUXxnqamFqZU3mg3h7zAx6rWZvWARsZhCc28zAHrTM3YgEJiEJCbqcTDtRRCxC2uyBwzzmZftaDum8p5Py", "ed25519:5VDMQYTp59pwJ1eVHjcFSUeQnmnkBC9F8jEHwjxoJb1YewM8VAgQCb718r4EpV9Z41MHW57i9H3gP9JHSWJiZUL2", "ed25519:2zwMz7w2MJHpg8F4G9AAu5UEy6Yo2TFw1VGFCAUsQrP3WwGDwUbsCR63Y88M9v4aWFdyr38no6yzd2DEsrQQCq7b", "ed25519:3M3cA2X1DqWdiELg7PKuFDf8u9MJC2rNhiHccZBYJxZcvKeomgRhGLcgjniEiAmC4uaYq86ePKY7e68smydbXRFr", "ed25519:5hX41Uom35xmLD5pTVaxEXzzQMJiS5Vi4Eyd3bBHcd2dKbE4g49AyCmCJ3Mpqr1xEEMsWGKtcLAFpAkhEyiTS73H", "ed25519:42kqMAMQdA1b1No1PhLwp8FL2g789vkwHTshiHLjvqSfJ4t6B8krevBj3Z2AmeEYduGMZGNaMcDmFG43vifoX15F", null, "ed25519:4AxJBk1cK9sqHwP4F32KQ21tj37kroBqMyesqXy2ksF8MWmXpxC3R2UKwFJRp3EztNaD8EM2vy7oFRmdumZcM2Gf", "ed25519:3KJDnpo4vxwyBDE11AWVRrnyGh9fYgq1GrsinEUKdwkVSCGKGv3rQrm19y5drxZEsqRw39D1bws1scc1TniCv1AX", "ed25519:PapbiGeqfur5csDNK3C9Cky6eLUQxzQXDPmE441MqFv5x1VDmEe2ZqHQecjiU6p18o1sQMGwgMARQZxHFDenYLB", "ed25519:57Pz4cEpBK2WCMnSaqt4jswZ4QYyhcRx1xG9pD44mp8mLCG3ogJhrq23eFF2xMizUzEDSD7AVMG1tByRob8VCZYm", null, null, "ed25519:5qNA6byTPhr6GHyTGxEZfRX12tYbqiYy7dQ4kdZVx4PW3i7s5i4q3ZUmwqP5rq6bdxx5fGkdMUVc3sqomz2dErE7", "ed25519:5YcgeAxawhVaTnvuRTWbPCtDtS6NH2UrNjeXcn89eXbKHTqeWtHsXKAb1NptrLgKr8K5WUGXjTKicz1xJyv9HgNh", "ed25519:sqZFxPtQpBnWUzcuPJeQUqyp3z3vYDYYe7Fvgcu4pVA46ueaU1oyTBpoMakY7pkZGyPq4jmgEamS3Bupf1W8KW3", null, null, null, null, null, "ed25519:291jQZp5NAeogGttS75JYoqaenVKkQHP8M1YXSB8kpB2UJWN4wn7CCPZUxSPPSsPYmvYfpX3pkrxhJ7gAms9h743", null, "ed25519:W2igco7sqdc9am1aVYA59yfPUVCJ8v4y6G5GAgfN9xpHy9r5QCKmatHpVJ6FMzinhy9GkCiZ3qJizbQBSWScwcP", null, "ed25519:4q7PyxNTZtEdoSQf4azWVmXVs851JH9aYJc9B2H67AdkmM5sBz2gDFce1b9c36xWccUtf7cDDGBE6YPAf5eELax6", "ed25519:3ECDJscq5C3JLy6XeqAX5qKSvk8KXRjucJPjCiTW7vLmwb5C1MHR8VUZa6ufpN5chHMgFie2L4D82nMuPABPkXmz", "ed25519:29VK9ikBVSp2ZDBKhLDXC4QbA3XXAp5gdMRvVtANNDW29M6boo939HXpwhazhVSJydft8x4WFNfrJUykGrPHtQi6", null, null, null, null, "ed25519:5nUAvGput21GYcbAYuPncCvhyMSQuoKD2eLxb3pUVhLcPyYCxETWwtY4k5AL1p5aR7c9aBG1cfEqUXRB99Y2zwYJ", "ed25519:3mae6hCDVnY1WrV1N74xnTkvN5ihTn8Y9SKUPY32zS4jpqJWKzdopgX7KZniiFkC41JhD4vQjWwKChvxfaztBKq6", null, "ed25519:5XjF6gyHudHwy3b48eGsED47myBjxB2dxvYZG2m954eQquMyLcVwKRZJvAXbzbHKehXUP8r9eeL3rm2LriYsx1dD", "ed25519:5vtjieSMvoYXYYP3zit9x6Dd7qVRDPYmoBZfELvouSX5ur9dnHmUnjEAXsWSWB69obm6hnfJz1Bx5gkXjLvyen3X", null, null, null, "ed25519:3WK1bgn3qqzSiZcEphbKcw6UajNSDdLSYeGYMSMGH4YMdS7FnFKAhdigotRZ8z32adAzZp1URjd39yBHBibJVnDJ", "ed25519:3hkDFUQhi9pDWKtb6YtcuFy56QY9CkhAKvEwrg99wFYtmcYicN5N3tSu3bSKpFFEWop3qsvr5rCc3zWwkqKVT2nb", null, "ed25519:rpALm3QcZov6qnDscsyyKwtgLSXVjppugEgM1DAzMw6Ewk2ovVAtow5frCV7nc1umnDLuxfLSfRNxkjeDqgAr4S", "ed25519:282Bw4icWLoUEAa6KUmixVeGSjjt6o1GAMcLypc1ruCqneENnzPC9hVqUWWGe9PmHU6GCEDoe79Y8hGpP9GuDYdA", "ed25519:2JQENivaSR9iDsbEXB3gz6PdqBiNbWRKU1e9XvcChzwtHp3gcCMW6Gnr362ktodDcjtcxZzyR4mHKBgREm2iRDJC", null, "ed25519:35ffy6YAW7V1VSF1XcYNQUXnQxgYyNpsaUWEF3Gu7LgijZkdFqVdbPxfQpKCntWuC3LLrb8Vs1a3WNHgmEcub4Ng", null, null, "ed25519:53swEewwpM15AN4uBW3DPSdX4VTcx7XvP7d7LZNtxDga7JnNXZXVYJeNw31ui9LB1ffnHMNyAFoiQMLm5bnPQcsq", "ed25519:3Wb7R4K6DXA3o3DDAEwaYNY72SwqUadjqesMYvq9cmhoHm2UtyDMp1wPWCWMLQ4BfrshGdMgryvSn8cpeiVuv5HN", null, "ed25519:5gGB9iTkt31SvAYaR52WfYf51uhhUb3CTf9RDN3MXoD3VR4m8fwBgRpmeu2Txb1RQJCwczxfYByYuRJR3orKkXA", "ed25519:2L7h92UaYwSXGBTqVkuhpRKZwRpW2wB7568U6jcBnH3iQtj71jERJdAtru62dx17teRngxJjDzMAG2Hs273Nt4cn", "ed25519:DL2QVuWC6CWrr519nQiKRtgC2H59ebBKNb5MjAKyv64zF3VrguWKQCcfaRUGgugiVeUjx12m2QfQigHmrqHWPmi", "ed25519:2uRyW8QrTaxdeX11tKxFmCY1FZbHbz7iWKxrNzGV9jiJg56BJgRKSZKbFpzPTakWMsesUfUUq4HNB4YsE6pvJSsN", "ed25519:2JB1CzirhvYfDVhay5jdzW5zGhGCLftfAB2Ff1ByNRnrvN7osvswuzLGHYDbmdMPdhC1ZTqeNCmKnQHzqQxKSjzN", null, "ed25519:4YNGqdPNKGGDcuWAnW1DFJfJ6Qeqz5wh7rGLj7gSK2jXYUh9vnuXcKoC2fBWsTKtfawncDyKE7sAfW3THaSqrDDM", null, null, null, null, "ed25519:KhbmAUdmHgTuVkUsBX2PdbPsLgKMtr8ds7pWrb1otn5dKidQJc8Vdtq5MBZUrn6pRL2d96YWL9jxTBpgBqrbn2M", null, "ed25519:MZT25bZDDxDJP4CRhyTK17Qfr8oWroVdyRuTccpA3Ynhf3PwB1WcC31XwugD6L5LCtMU1hyCPbE16bk7NksHhHC", null, "ed25519:2BWXNtCGqwLG9yu8KkRnT1tui8GcynBN5DrhtfNQtFotSHphtABu1iNucrd82aPTegFogSMDA2cvhhrgpu4SRevc", null, "ed25519:4EFt6q2MUjvcvZrCFXY2amqBcd36gKezF3FFHWsGwtJcqbH4brTHMH5THio3pXFYCWY8mtQTpLZ9SnPUWDyVKxnA", "ed25519:57js3GUGprV3Rc5Aci4Ws2JciuuWMhmApPPmyhYJXcz67LuwTT4KCnhFNZYDzGnV2ueP1irNUFFUXhC6HXa7amUQ", null, "ed25519:2uJEgTHUV1qSLLDSjuA6NBDdJur5Xh9wKQ9sHWmpwpePyknc93J64YVZxnidG2EDAKVBSDm9kpBndMXGNEXjqEew", "ed25519:BzXpv857MvdsUkgCdZRAspbAhrKqibxXs7JqSt2KkzjCm9JgsxiCGf5GJCYC3gV3NJfC4p6zst8UYezQ43LAkT1", "ed25519:3hX4uGLHK2GRCTng7SBazenaXxSHSscJyRCKgUpKiQfHMJ7BsVvJb9obJfSNN3ynfCNmRtoUiB3tE46xHKqZ73rt", null, "ed25519:2VyXyzSMCS6nzA2rDVwoZJLt2k1Zx3c4wQMx15SBx3mYbjevfdt6ErHH4GvF7PMqnXAUEgCTzWoJ54NeV5nWbM9b", null, "ed25519:2FURA8ZJGN7gTtoaGgYvNkQE46rNLmrwSqqNVhzevUg3MP6GNt821vqMpsfrR5t7mFy5pk5inzrWYZcELZw5LHKz", null, "ed25519:3JHQcPV1nTLsqSTboA4NavexkhmYxDsiDun49KFVEZQRuxoNHuigJL1jBW78T4nRRPPowESosCZgmpgsWDb5UDAK", "ed25519:2yqmma3px8io4Dd7ezg5H5d3C4Rueyi3aok9agpzpowU4vPxj2Y58h3Mu2qPSYBUtT75DgEcTtiZaMjQsBMfjCj7", null, "ed25519:647bt2zPCma9kZqZmaR4HshKrERRUiFg3KT1G6S5aeowmoELvLhHcwZyAki4hW8BtgdW5GuEu2pPRXFsoTbEcASC", null, "ed25519:4kv1RTxeFCRxc6shxqwfmmL398dJDGjKvFPn9PuaJJoFZTBkkFDaNuMDagoBrfgZYBhb7eZ26vKAoPK3vBVUGZgp", "ed25519:4P183Ceq88jLBs1cKoANgrmTBDCsvbJVRYgBH4iYyK76bYrw795DkgMaQKsWo3ZFjMHH3fC9nZ8C4WNLwkJZVfE4", "ed25519:CbQsnkfYhEZ32hfTrj4d4MYd4duFep6zY1cqRUFqt74WtxQHWU4mVL4qoxgvQgfeWPLiev9FRVaFLK4VnL9jYsZ", "ed25519:4kFsUdepZzP9oVCYWdzNq3iHURJdNn9GHj4gJgeaHPVWFpbue52zagJpmh4hsNjhDXX5tkL3TQzcd5HTWjNcqgDC", null], "blockMerkleRoot": "FYpXxjbPZ5FoB57ZcPMvvs94x52wbCJdhxYs6qnGaZsm", "blockOrdinal": 75337992, "challengesResult": [], "challengesRoot": "11111111111111111111111111111111", "chunkHeadersRoot": "5AzFQd9Ds8yWynns4B4TFFcBGSh24ze2jApisECRYEib", "chunkMask": [true, true, true, true], "chunkReceiptsRoot": "CsBRzd4FJe8Qr22wDkQXtxaVs5z2Jmrk2ZQC1PBrHRoV", "chunkTxRoot": "9DwCvRfUYzaxHCjWEWDR3WxJfn9gPoignTYXmEN9D6wC", "chunksIncluded": 4, "epochId": "4GoC7oSPLfaXo3Gz9z2Rkb86nN1fM8qVd578gTAmf1fn", "epochSyncDataHash": null, "gasPrice": "100000000", "hash": "A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM", "height": 85376002, "lastDsFinalBlock": "H5X25nVZgx43tQSMcF5bRLphWGN92UVYdvaAhcgW4no7", "lastFinalBlock": "C3KNVyUKZ8uR166dc5vUKPQdsW5CL2mn4oerbMQFcQp5", "latestProtocolVersion": 58, "nextBpHash": "5BaNTEgDB11HxTGNKvQRJSytiv6ZM73YrJjxBymYau7", "nextEpochId": "3jvt2LuA1LZGQwKtbcThcjT4twe4juCfqwSEiuktBrqH", "outcomeRoot": "5dDKr6oN9bh1bxePyq2HbAWTA792ZRQSvfvcmTTrySi9", "prevHash": "H5X25nVZgx43tQSMcF5bRLphWGN92UVYdvaAhcgW4no7", "prevHeight": 85376001, "prevStateRoot": "6Y3XsdVvhHj3wcGVbXNQNg3qh46HHQA5FPKUoigHPVYr", "randomValue": "3SPgojnHE7CFj7bue5w8CsYDvg5Ffskue8qPDZohqQJh", "rentPaid": "0", "signature": "ed25519:5kjuiCWar11gt9BSRncbEaUnwoL1Nx5tzdnxrnBzgWhjHdfH2QfDajYqdy4bkXaU3UJ6TQDGcLz7pPiNLR3mmnQS", "timestamp": 1676579182274205000, "timestampNanosec": "1676579182274204901", "totalSupply": "1121201720917794037313140715777439", "validatorProposals": [], "validatorReward": "0" } }, "shards": [{ "chunk": { "author": "figment.poolv1.near", "header": { "balanceBurnt": "1486252472505500000000", "chunkHash": "3qxSWuuo5QLdgpYSuwtQcAcAoX91b6p55CLxjda2wNNw", "encodedLength": 1772, "encodedMerkleRoot": "ChpbnZdB2NdSaC2ir443zB4Bv6WUMSDBY6n3RfDyEVs", "gasLimit": 1000000000000000, "gasUsed": 16028665261860, "heightCreated": 85376002, "heightIncluded": 85376002, "outcomeRoot": "G5c89nTRQSRYPdJQFcP47YXccbmbrbutNwLZzKUeRG3f", "outgoingReceiptsRoot": "5GrcVRVw9njFAskhcEbUyPZAoKMEEjGnC6edUhupEDkj", "prevBlockHash": "H5X25nVZgx43tQSMcF5bRLphWGN92UVYdvaAhcgW4no7", "prevStateRoot": "CRaTBLfd55cw6i5GfVvVSo1rw5ioZ91hT3ZVo5eBmNCi", "rentPaid": "0", "shardId": 0, "signature": "ed25519:4Fzq1idZFShNtErzbK5uZWDcnWhHTMshVBvXG9c8cvc3YPkHF8SVgRsMym6V3WNHVY29gNaw1kXPCLBApcZ2bA1h", "txRoot": "5uqwhjC7wRJsvidh7gjtRRPvvUVwyefdQCivBMTL1hak", "validatorProposals": [], "validatorReward": "0" }, "receipts": [{ "predecessorId": "app.nearcrowd.near", "receipt": { "Action": { "actions": [{ "FunctionCall": { "args": "eyJhY2NvdW50X2lkIjoicmV6ZXJ2LTI5Lm5lYXIiLCJzb2x1dGlvbl9oYXNoIjpbMTY2LDIwNCw2NSw3NCwxMTksMiwyOSwzMywyMTMsMTA1LDE2OCw0LDE5NCwxNzksMTk3LDUyLDI0MSw1NSwxLDEwMSwxNTcsNjQsMTI0LDAsMTE4LDY3LDE4NywxMjEsMTMzLDQxLDIzOSwyNDNdfQ==", "deposit": "0", "gas": 200000000000000, "methodName": "approve_solution" } }], "gasPrice": "335989893", "inputDataIds": [], "outputDataReceivers": [], "signerId": "app.nearcrowd.near", "signerPublicKey": "ed25519:CXYkSHvK2rj6sCGbaKPbTtwhcMrPvWxs1ntVvA6vLtm2" } }, "receiptId": "8FWEuJ76SphAbWL2hgsJ36zD5jobWjqPNE2xQWaEFKNY", "receiverId": "app.nearcrowd.near" }, { "predecessorId": "7a8ba1bd804863883ace10ed76441e6ad2239ae7917b78c89700b701f75ce1ab", "receipt": { "Action": { "actions": [{ "FunctionCall": { "args": "eyJyZWNlaXZlcl9pZCI6InJld2FyZC1vcHRpbi5zd2VhdCIsImFtb3VudCI6IjEwMDAwMDAwMDAwMDAwMDAwMCIsIm1lbW8iOiJzdzpyZXc6b3B0aW46WUdSV3ZncFhLMC03YThiYTFiZDgwNDg2Mzg4M2FjZTEwZWQ3NjQ0MWU2YWQyMjM5YWU3OTE3Yjc4Yzg5NzAwYjcwMWY3NWNlMWFiIn0=", "deposit": "1", "gas": 14000000000000, "methodName": "ft_transfer" } }], "gasPrice": "109272700", "inputDataIds": [], "outputDataReceivers": [], "signerId": "7a8ba1bd804863883ace10ed76441e6ad2239ae7917b78c89700b701f75ce1ab", "signerPublicKey": "ed25519:9FNEMP92rkuvRVefVkBW38RmbZ648XvQGeWbFYtdC7hk" } }, "receiptId": "7Q29UAwrFbeMMM6niPSuMrqkp4Y21FYqJ1SXPvr9oZYU", "receiverId": "token.sweat" }, { "predecessorId": "system", "receipt": { "Action": { "actions": [{ "Transfer": { "deposit": "67682508099918060916120" } }], "gasPrice": "0", "inputDataIds": [], "outputDataReceivers": [], "signerId": "app.nearcrowd.near", "signerPublicKey": "ed25519:6MP4bCPHEud33eKXM9kg7f9fVNhmn97CNUyn5ZwM375U" } }, "receiptId": "5GMwBgfhgASLVXkCsqMn2riVbByJPh2pmyxCeQmbHHWe", "receiverId": "app.nearcrowd.near" }, { "predecessorId": "system", "receipt": { "Action": { "actions": [{ "Transfer": { "deposit": "13222602662027447938448" } }], "gasPrice": "0", "inputDataIds": [], "outputDataReceivers": [], "signerId": "olezha4.near", "signerPublicKey": "ed25519:EnLNUDncRuw4AYLcGmi1Hr2YQ1DLu12VutsubZ6k8c2F" } }, "receiptId": "BpZGWvbmx8sAMRsrhWyBnC5QHA9mFwist7orTceJTRvu", "receiverId": "olezha4.near" }, { "predecessorId": "system", "receipt": { "Action": { "actions": [{ "Transfer": { "deposit": "13239855304236542651168" } }], "gasPrice": "0", "inputDataIds": [], "outputDataReceivers": [], "signerId": "marthacecilia.near", "signerPublicKey": "ed25519:8W28RYFfXztNA7ENP4VTaitsqEjSbABQJDqELwbwXswq" } }, "receiptId": "3R2wjnktLwBKaceGiPetmShVAyHT2LuoQZMdNg6oWMwK", "receiverId": "marthacecilia.near" }], "transactions": [{ "outcome": { "executionOutcome": { "blockHash": "A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM", "id": "3W8KnZUsb7Mb3SZk7yygL5F2L2C6djxZ3fVumodB2KQe", "outcome": { "executorId": "app.nearcrowd.near", "gasBurnt": 2428314524384, "logs": [], "metadata": { "gasProfile": null, "version": 1 }, "receiptIds": ["8FWEuJ76SphAbWL2hgsJ36zD5jobWjqPNE2xQWaEFKNY"], "status": { "SuccessReceiptId": "8FWEuJ76SphAbWL2hgsJ36zD5jobWjqPNE2xQWaEFKNY" }, "tokensBurnt": "242831452438400000000" }, "proof": [{ "direction": "Right", "hash": "8AS32PCfmnnsgkmrm2QH6ULKdTBS4o3EjcU2FZ34dTux" }, { "direction": "Right", "hash": "8PVbKv5bqbBoTUkXbcQ8sWEYbUDKo8hv8S1JHKVrftCz" }, { "direction": "Right", "hash": "8VRHTUKsUcZRDgrKizV5sup3Zvzg6reRacn3KmeUVY9n" }] }, "receipt": null }, "transaction": { "actions": [{ "FunctionCall": { "args": "eyJhY2NvdW50X2lkIjoicmV6ZXJ2LTI5Lm5lYXIiLCJzb2x1dGlvbl9oYXNoIjpbMTY2LDIwNCw2NSw3NCwxMTksMiwyOSwzMywyMTMsMTA1LDE2OCw0LDE5NCwxNzksMTk3LDUyLDI0MSw1NSwxLDEwMSwxNTcsNjQsMTI0LDAsMTE4LDY3LDE4NywxMjEsMTMzLDQxLDIzOSwyNDNdfQ==", "deposit": "0", "gas": 200000000000000, "methodName": "approve_solution" } }], "hash": "3W8KnZUsb7Mb3SZk7yygL5F2L2C6djxZ3fVumodB2KQe", "nonce": 43616777771952, "publicKey": "ed25519:CXYkSHvK2rj6sCGbaKPbTtwhcMrPvWxs1ntVvA6vLtm2", "receiverId": "app.nearcrowd.near", "signature": "ed25519:5KcVvC1zzJZkk92uWzMoD9e2fvded6JdgUApuV6NDCcY7YEhzVtUQaVPomQ4ZpDydtUyZmgxTXcqaVp5c54wciEX", "signerId": "app.nearcrowd.near" } }, { "outcome": { "executionOutcome": { "blockHash": "A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM", "id": "3GYUTXHUQMH1kpccEms6yNVzckyAedUTWKPQpEN4tc5K", "outcome": { "executorId": "2fe75000c84efd3b64f88f61d88cd08c339b8c6654e6239f7d4e3592aa92c659", "gasBurnt": 2428312288450, "logs": [], "metadata": { "gasProfile": null, "version": 1 }, "receiptIds": ["7CmRnTXnY3LmX7PojrByXqZpirVNVSuHgL9k3uhbdKm1"], "status": { "SuccessReceiptId": "7CmRnTXnY3LmX7PojrByXqZpirVNVSuHgL9k3uhbdKm1" }, "tokensBurnt": "242831228845000000000" }, "proof": [{ "direction": "Left", "hash": "5Bf7Pbbnwt6THJBxQ3mBkpc8WD2GBLoXrte6HYJZU6yh" }, { "direction": "Right", "hash": "8PVbKv5bqbBoTUkXbcQ8sWEYbUDKo8hv8S1JHKVrftCz" }, { "direction": "Right", "hash": "8VRHTUKsUcZRDgrKizV5sup3Zvzg6reRacn3KmeUVY9n" }] }, "receipt": null }, "transaction": { "actions": [{ "FunctionCall": { "args": "eyJyZWNlaXZlcl9pZCI6InJld2FyZC1vcHRpbi5zd2VhdCIsImFtb3VudCI6IjEwMDAwMDAwMDAwMDAwMDAwMCIsIm1lbW8iOiJzdzpyZXc6b3B0aW46WUdSV3ZncFhLMC0yZmU3NTAwMGM4NGVmZDNiNjRmODhmNjFkODhjZDA4YzMzOWI4YzY2NTRlNjIzOWY3ZDRlMzU5MmFhOTJjNjU5In0=", "deposit": "1", "gas": 14000000000000, "methodName": "ft_transfer" } }], "hash": "3GYUTXHUQMH1kpccEms6yNVzckyAedUTWKPQpEN4tc5K", "nonce": 70681752000024, "publicKey": "ed25519:4Dzjg5EJX1RBYctnzmi1SNoq6fswSowsiWB5m3sEcC4Q", "receiverId": "token.sweat", "signature": "ed25519:5NuszWGgsVB5yPWqvGMGP3yCjs5YVh3QtWw79kch5mf3ApSVqpzTD42TzAsKcmhauDYDhzU8SUU9dGx2rHodJRLn", "signerId": "2fe75000c84efd3b64f88f61d88cd08c339b8c6654e6239f7d4e3592aa92c659" } }] }, "receiptExecutionOutcomes": [{ "executionOutcome": { "blockHash": "A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM", "id": "8FWEuJ76SphAbWL2hgsJ36zD5jobWjqPNE2xQWaEFKNY", "outcome": { "executorId": "app.nearcrowd.near", "gasBurnt": 3300309921959, "logs": [], "metadata": { "gasProfile": [{ "cost": "BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "3177217332" }, { "cost": "CONTRACT_LOADING_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "35445963" }, { "cost": "CONTRACT_LOADING_BYTES", "costCategory": "WASM_HOST_COST", "gasUsed": "104961404250" }, { "cost": "READ_CACHED_TRIE_NODE", "costCategory": "WASM_HOST_COST", "gasUsed": "86640000000" }, { "cost": "READ_MEMORY_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "13049316000" }, { "cost": "READ_MEMORY_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "315510639" }, { "cost": "READ_REGISTER_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "7551495558" }, { "cost": "READ_REGISTER_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "19712400" }, { "cost": "STORAGE_READ_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "56356845750" }, { "cost": "STORAGE_READ_KEY_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "154762665" }, { "cost": "STORAGE_READ_VALUE_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "123442110" }, { "cost": "STORAGE_WRITE_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "128393472000" }, { "cost": "STORAGE_WRITE_EVICTED_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "706580754" }, { "cost": "STORAGE_WRITE_KEY_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "1691588808" }, { "cost": "STORAGE_WRITE_VALUE_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "1675001106" }, { "cost": "TOUCHING_TRIE_NODE", "costCategory": "WASM_HOST_COST", "gasUsed": "418650854076" }, { "cost": "WASM_INSTRUCTION", "costCategory": "WASM_HOST_COST", "gasUsed": "24383196816" }, { "cost": "WRITE_MEMORY_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "11215179444" }, { "cost": "WRITE_MEMORY_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "588334752" }, { "cost": "WRITE_REGISTER_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "11462089944" }, { "cost": "WRITE_REGISTER_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "843947208" }], "version": 1 }, "receiptIds": ["8kGPN9gvU8dhZy3374rTa3LjtxcZcHcZkXRAf56si2vh"], "status": { "SuccessValue": "" }, "tokensBurnt": "330030992195900000000" }, "proof": [{ "direction": "Right", "hash": "8uFcqYe8azbspu5mofVCt5TC6K4n6kTbNBRwDt3CRT6Z" }, { "direction": "Left", "hash": "9C9aaY5ECUzq1t2NtXQVXTH6wZ8sBExs33KhVNmjFnQJ" }, { "direction": "Right", "hash": "8VRHTUKsUcZRDgrKizV5sup3Zvzg6reRacn3KmeUVY9n" }] }, "receipt": { "predecessorId": "app.nearcrowd.near", "receipt": { "Action": { "actions": [{ "FunctionCall": { "args": "eyJhY2NvdW50X2lkIjoicmV6ZXJ2LTI5Lm5lYXIiLCJzb2x1dGlvbl9oYXNoIjpbMTY2LDIwNCw2NSw3NCwxMTksMiwyOSwzMywyMTMsMTA1LDE2OCw0LDE5NCwxNzksMTk3LDUyLDI0MSw1NSwxLDEwMSwxNTcsNjQsMTI0LDAsMTE4LDY3LDE4NywxMjEsMTMzLDQxLDIzOSwyNDNdfQ==", "deposit": "0", "gas": 200000000000000, "methodName": "approve_solution" } }], "gasPrice": "335989893", "inputDataIds": [], "outputDataReceivers": [], "signerId": "app.nearcrowd.near", "signerPublicKey": "ed25519:CXYkSHvK2rj6sCGbaKPbTtwhcMrPvWxs1ntVvA6vLtm2" } }, "receiptId": "8FWEuJ76SphAbWL2hgsJ36zD5jobWjqPNE2xQWaEFKNY", "receiverId": "app.nearcrowd.near" } }, { "executionOutcome": { "blockHash": "A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM", "id": "6XN47idjeWhq2QpkjGs9w7pPg452Z1GwgwRxZjy2ny7T", "outcome": { "executorId": "app.nearcrowd.near", "gasBurnt": 5360036359518, "logs": [], "metadata": { "gasProfile": [{ "cost": "BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "15091782327" }, { "cost": "CONTRACT_LOADING_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "35445963" }, { "cost": "CONTRACT_LOADING_BYTES", "costCategory": "WASM_HOST_COST", "gasUsed": "104961404250" }, { "cost": "READ_CACHED_TRIE_NODE", "costCategory": "WASM_HOST_COST", "gasUsed": "592800000000" }, { "cost": "READ_MEMORY_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "62636716800" }, { "cost": "READ_MEMORY_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "2698946430" }, { "cost": "READ_REGISTER_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "42791808162" }, { "cost": "READ_REGISTER_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "125075178" }, { "cost": "STORAGE_HAS_KEY_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "54039896625" }, { "cost": "STORAGE_HAS_KEY_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "738980280" }, { "cost": "STORAGE_READ_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "281784228750" }, { "cost": "STORAGE_READ_KEY_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "4859547681" }, { "cost": "STORAGE_READ_VALUE_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "4253141790" }, { "cost": "STORAGE_REMOVE_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "160419091500" }, { "cost": "STORAGE_REMOVE_KEY_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "1605256128" }, { "cost": "STORAGE_REMOVE_RET_VALUE_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "1141624044" }, { "cost": "STORAGE_WRITE_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "449377152000" }, { "cost": "STORAGE_WRITE_EVICTED_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "9121315188" }, { "cost": "STORAGE_WRITE_KEY_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "6977803833" }, { "cost": "STORAGE_WRITE_VALUE_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "11880100437" }, { "cost": "TOUCHING_TRIE_NODE", "costCategory": "WASM_HOST_COST", "gasUsed": "933913443708" }, { "cost": "WASM_INSTRUCTION", "costCategory": "WASM_HOST_COST", "gasUsed": "83208605304" }, { "cost": "WRITE_MEMORY_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "50468307498" }, { "cost": "WRITE_MEMORY_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "3500047020" }, { "cost": "WRITE_REGISTER_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "48713882262" }, { "cost": "WRITE_REGISTER_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "4824184716" }], "version": 1 }, "receiptIds": ["HF5rs8fq4n8HygFMYiQ8nWZSpVncKBnCXj9aNpvzZsAP"], "status": { "SuccessValue": "ZmFsc2U=" }, "tokensBurnt": "536003635951800000000" }, "proof": [{ "direction": "Left", "hash": "2Lon5RSqBvnuGcXgkneYZTUUup6CotQYqEbi1eYfBfVK" }, { "direction": "Left", "hash": "9C9aaY5ECUzq1t2NtXQVXTH6wZ8sBExs33KhVNmjFnQJ" }, { "direction": "Right", "hash": "8VRHTUKsUcZRDgrKizV5sup3Zvzg6reRacn3KmeUVY9n" }] }, "receipt": { "predecessorId": "evaenergy.near", "receipt": { "Action": { "actions": [{ "FunctionCall": { "args": "eyJ0YXNrX29yZGluYWwiOjEsImJpZCI6IjQ0NDQ2ODcxMTE2MjIyODMzODczMDc2In0=", "deposit": "0", "gas": 30000000000000, "methodName": "claim_assignment" } }], "gasPrice": "122987387", "inputDataIds": [], "outputDataReceivers": [], "signerId": "evaenergy.near", "signerPublicKey": "ed25519:9Q53x5kiLChb4tX3J5dV2wNAdTetejEcnizyQ3n3PZdg" } }, "receiptId": "6XN47idjeWhq2QpkjGs9w7pPg452Z1GwgwRxZjy2ny7T", "receiverId": "app.nearcrowd.near" } }, { "executionOutcome": { "blockHash": "A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM", "id": "5GMwBgfhgASLVXkCsqMn2riVbByJPh2pmyxCeQmbHHWe", "outcome": { "executorId": "app.nearcrowd.near", "gasBurnt": 223182562500, "logs": [], "metadata": { "gasProfile": [], "version": 1 }, "receiptIds": [], "status": { "SuccessValue": "" }, "tokensBurnt": "0" }, "proof": [{ "direction": "Left", "hash": "BcJG6WkrSgW8dsc8iBJHNZcP7576vvRQGBagZZfuXkmG" }] }, "receipt": { "predecessorId": "system", "receipt": { "Action": { "actions": [{ "Transfer": { "deposit": "67682508099918060916120" } }], "gasPrice": "0", "inputDataIds": [], "outputDataReceivers": [], "signerId": "app.nearcrowd.near", "signerPublicKey": "ed25519:6MP4bCPHEud33eKXM9kg7f9fVNhmn97CNUyn5ZwM375U" } }, "receiptId": "5GMwBgfhgASLVXkCsqMn2riVbByJPh2pmyxCeQmbHHWe", "receiverId": "app.nearcrowd.near" } }], "shardId": 0, "stateChanges": [{ "cause": { "txHash": "3GYUTXHUQMH1kpccEms6yNVzckyAedUTWKPQpEN4tc5K", "type": "transaction_processing" }, "change": { "accountId": "2fe75000c84efd3b64f88f61d88cd08c339b8c6654e6239f7d4e3592aa92c659", "amount": "31778755366428489684978", "codeHash": "11111111111111111111111111111111", "locked": "0", "storagePaidAt": 0, "storageUsage": 182 }, "type": "account_update" }, { "cause": { "txHash": "3W8KnZUsb7Mb3SZk7yygL5F2L2C6djxZ3fVumodB2KQe", "type": "transaction_processing" }, "change": { "accountId": "app.nearcrowd.near", "amount": "6559406007009136306129113176", "codeHash": "DyHG2t99dBZWiPgX53Z4UbbBQR6JJoxmqXwaKD4hTeyP", "locked": "0", "storagePaidAt": 0, "storageUsage": 3976751 }, "type": "account_update" }, { "cause": { "receiptHash": "8FWEuJ76SphAbWL2hgsJ36zD5jobWjqPNE2xQWaEFKNY", "type": "receipt_processing" }, "change": { "accountId": "app.nearcrowd.near", "amount": "6559406007009136306129113176", "codeHash": "DyHG2t99dBZWiPgX53Z4UbbBQR6JJoxmqXwaKD4hTeyP", "locked": "0", "storagePaidAt": 0, "storageUsage": 3976842 }, "type": "account_update" }, { "cause": { "receiptHash": "8FWEuJ76SphAbWL2hgsJ36zD5jobWjqPNE2xQWaEFKNY", "type": "action_receipt_gas_reward" }, "change": { "accountId": "app.nearcrowd.near", "amount": "6559406033168998233329113176", "codeHash": "DyHG2t99dBZWiPgX53Z4UbbBQR6JJoxmqXwaKD4hTeyP", "locked": "0", "storagePaidAt": 0, "storageUsage": 3976842 }, "type": "account_update" }, { "cause": { "receiptHash": "6XN47idjeWhq2QpkjGs9w7pPg452Z1GwgwRxZjy2ny7T", "type": "receipt_processing" }, "change": { "accountId": "app.nearcrowd.near", "amount": "6559406033168998233329113176", "codeHash": "DyHG2t99dBZWiPgX53Z4UbbBQR6JJoxmqXwaKD4hTeyP", "locked": "0", "storagePaidAt": 0, "storageUsage": 3976788 }, "type": "account_update" }, { "cause": { "receiptHash": "6XN47idjeWhq2QpkjGs9w7pPg452Z1GwgwRxZjy2ny7T", "type": "action_receipt_gas_reward" }, "change": { "accountId": "app.nearcrowd.near", "amount": "6559406121128031869529113176", "codeHash": "DyHG2t99dBZWiPgX53Z4UbbBQR6JJoxmqXwaKD4hTeyP", "locked": "0", "storagePaidAt": 0, "storageUsage": 3976788 }, "type": "account_update" }, { "cause": { "receiptHash": "5GMwBgfhgASLVXkCsqMn2riVbByJPh2pmyxCeQmbHHWe", "type": "receipt_processing" }, "change": { "accountId": "app.nearcrowd.near", "amount": "6559473803636131787590029296", "codeHash": "DyHG2t99dBZWiPgX53Z4UbbBQR6JJoxmqXwaKD4hTeyP", "locked": "0", "storagePaidAt": 0, "storageUsage": 3976788 }, "type": "account_update" }, { "cause": { "txHash": "3GYUTXHUQMH1kpccEms6yNVzckyAedUTWKPQpEN4tc5K", "type": "transaction_processing" }, "change": { "accessKey": { "nonce": 70681752000024, "permission": "FullAccess" }, "accountId": "2fe75000c84efd3b64f88f61d88cd08c339b8c6654e6239f7d4e3592aa92c659", "publicKey": "ed25519:4Dzjg5EJX1RBYctnzmi1SNoq6fswSowsiWB5m3sEcC4Q" }, "type": "access_key_update" }, { "cause": { "txHash": "3W8KnZUsb7Mb3SZk7yygL5F2L2C6djxZ3fVumodB2KQe", "type": "transaction_processing" }, "change": { "accessKey": { "nonce": 43616777771952, "permission": "FullAccess" }, "accountId": "app.nearcrowd.near", "publicKey": "ed25519:CXYkSHvK2rj6sCGbaKPbTtwhcMrPvWxs1ntVvA6vLtm2" }, "type": "access_key_update" }, { "cause": { "receiptHash": "8FWEuJ76SphAbWL2hgsJ36zD5jobWjqPNE2xQWaEFKNY", "type": "receipt_processing" }, "change": { "accountId": "app.nearcrowd.near", "keyBase64": "U1RBVEU=", "valueBase64": "EgAAAGFwcC5uZWFyY3Jvd2QubmVhcg==" }, "type": "data_update" }, { "cause": { "receiptHash": "8FWEuJ76SphAbWL2hgsJ36zD5jobWjqPNE2xQWaEFKNY", "type": "receipt_processing" }, "change": { "accountId": "app.nearcrowd.near", "keyBase64": "YQ4AAAByZXplcnYtMjkubmVhcg==", "valueBase64": "psxBSncCHSHVaagEwrPFNPE3AWWdQHwAdkO7eYUp7/M=" }, "type": "data_update" }, { "cause": { "receiptHash": "6XN47idjeWhq2QpkjGs9w7pPg452Z1GwgwRxZjy2ny7T", "type": "receipt_processing" }, "change": { "accountId": "app.nearcrowd.near", "keyBase64": "cAEAAAA=", "valueBase64": "AABAe6XwY4GWCgAAAAAAAAAA9ESCkWNFAAAAAAAAAAAsAQAAAAAAABACoJtTF3zmCgkAAAAAAADcEUiC7mdEF6E7rAAAAAAABgAAAHQBAAAAYrtzrAAAAAAABgAAAHQBAAAAYh0EAAAAAAAABgAAAHQBAAAAYwYAAAB0AQAAAGUGAAAAdAEAAABmBgAAAHQBAAAAZwEGAAAAdAEAAABtAA==" }, "type": "data_update" }, { "cause": { "receiptHash": "6XN47idjeWhq2QpkjGs9w7pPg452Z1GwgwRxZjy2ny7T", "type": "receipt_processing" }, "change": { "accountId": "app.nearcrowd.near", "keyBase64": "dAEAAABjHQQAAAAAAAA=" }, "type": "data_deletion" }, { "cause": { "receiptHash": "6XN47idjeWhq2QpkjGs9w7pPg452Z1GwgwRxZjy2ny7T", "type": "receipt_processing" }, "change": { "accountId": "app.nearcrowd.near", "keyBase64": "dAEAAABjIgEAAAAAAAA=", "valueBase64": "epIvaamXn1S0leE2SA8w+TxNWM4rEC75i6V3XS7gjvAE" }, "type": "data_update" }, { "cause": { "receiptHash": "6XN47idjeWhq2QpkjGs9w7pPg452Z1GwgwRxZjy2ny7T", "type": "receipt_processing" }, "change": { "accountId": "app.nearcrowd.near", "keyBase64": "dAEAAABjmwAAAAAAAAA=", "valueBase64": "0h0vJmTB/V9IK0xVxNBid+JAmsnySRFNF3jnQNl2m8sB" }, "type": "data_update" }, { "cause": { "receiptHash": "6XN47idjeWhq2QpkjGs9w7pPg452Z1GwgwRxZjy2ny7T", "type": "receipt_processing" }, "change": { "accountId": "app.nearcrowd.near", "keyBase64": "dAEAAABjqQEAAAAAAAA=", "valueBase64": "s+OZU4d49W6Qv/gFmkBPsmVUX1WVbDiJAJzJse0tXVEE" }, "type": "data_update" }, { "cause": { "receiptHash": "6XN47idjeWhq2QpkjGs9w7pPg452Z1GwgwRxZjy2ny7T", "type": "receipt_processing" }, "change": { "accountId": "app.nearcrowd.near", "keyBase64": "dAEAAABnDgAAAGV2YWVuZXJneS5uZWFy", "valueBase64": "A/UhKZel+/gCMLF4XhBcxERDBIa48p9qlYBMRIbWALUgA+XURCXiZ0QXWujnX0t/H7y0BAAAAAAAAA==" }, "type": "data_update" }] }, { "chunk": { "author": "bisontrails.poolv1.near", "header": { "balanceBurnt": "1433581134734200000000", "chunkHash": "BqfFW2NyLJuyp1pGgXd2ffYPGptNswNVH6ZJuTq5zSNa", "encodedLength": 161, "encodedMerkleRoot": "EB5tG2a3aVfXv41ESuQVT1ZvQFvuwKnkYGLc2CDVCf5w", "gasLimit": 1000000000000000, "gasUsed": 19438956235189, "heightCreated": 85376002, "heightIncluded": 85376002, "outcomeRoot": "Cp7GeqLAqZ1Z1GbtCL5nG13wxy6mV6PaRdwLddDHjhVc", "outgoingReceiptsRoot": "DP1DT6QP1tNHM5ZjHMJ3C8G2mHJd8kGoFFNEdZPN8Hyx", "prevBlockHash": "H5X25nVZgx43tQSMcF5bRLphWGN92UVYdvaAhcgW4no7", "prevStateRoot": "73sn3JyP1LU58r1FsTo5ZVgvErdbEGFxtiGExkWwhQUd", "rentPaid": "0", "shardId": 1, "signature": "ed25519:3RYj6fpboT1ALHsVxYtYmofT8fGf1VRhNaJhWY45hsKdePcX3AZb3rdKBAZMThF5PVn5j5boKWHPb9zd1xZWYzex", "txRoot": "11111111111111111111111111111111", "validatorProposals": [], "validatorReward": "0" }, "receipts": [{ "predecessorId": "system", "receipt": { "Action": { "actions": [{ "Transfer": { "deposit": "187086049935277727891452" } }], "gasPrice": "0", "inputDataIds": [], "outputDataReceivers": [], "signerId": "relay.aurora", "signerPublicKey": "ed25519:8j2e8u9t1SoTgMPn5G65ChU5WskpcnemBjwzkAnxDtZq" } }, "receiptId": "CdUSBwV29D1P1qvxCwnbfNsTsdxfdRY6PxezsSWRFyqN", "receiverId": "relay.aurora" }], "transactions": [] }, "receiptExecutionOutcomes": [{ "executionOutcome": { "blockHash": "A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM", "id": "821S5L8Htxafh2DDuDKtA4qapudN3mRq2AMv2qPeBhpj", "outcome": { "executorId": "aurora", "gasBurnt": 3497076222351, "logs": ["signer_address Address(0x605ce19532c7121d1b5e632eb776e5aa7bfb8990)"], "metadata": { "gasProfile": [{ "cost": "BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "5295362220" }, { "cost": "CONTRACT_LOADING_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "35445963" }, { "cost": "CONTRACT_LOADING_BYTES", "costCategory": "WASM_HOST_COST", "gasUsed": "207231423750" }, { "cost": "ECRECOVER_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "278821988457" }, { "cost": "KECCAK256_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "17638473825" }, { "cost": "KECCAK256_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "8889037470" }, { "cost": "LOG_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "3543313050" }, { "cost": "LOG_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "871120206" }, { "cost": "READ_CACHED_TRIE_NODE", "costCategory": "WASM_HOST_COST", "gasUsed": "9120000000" }, { "cost": "READ_MEMORY_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "20878905600" }, { "cost": "READ_MEMORY_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "2128746480" }, { "cost": "READ_REGISTER_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "22654486674" }, { "cost": "READ_REGISTER_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "65642292" }, { "cost": "STORAGE_READ_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "112713691500" }, { "cost": "STORAGE_READ_KEY_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "897623457" }, { "cost": "STORAGE_READ_VALUE_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "482546430" }, { "cost": "TOUCHING_TRIE_NODE", "costCategory": "WASM_HOST_COST", "gasUsed": "209325427038" }, { "cost": "UTF8_DECODING_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "6223558122" }, { "cost": "UTF8_DECODING_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "24784340715" }, { "cost": "WASM_INSTRUCTION", "costCategory": "WASM_HOST_COST", "gasUsed": "84250214400" }, { "cost": "WRITE_MEMORY_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "22430358888" }, { "cost": "WRITE_MEMORY_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "1639710744" }, { "cost": "WRITE_REGISTER_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "25789702374" }, { "cost": "WRITE_REGISTER_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "2531841624" }], "version": 1 }, "receiptIds": ["H4FHCm5scAbLHhki995o2UoJsHYuJHfbbCx5egAjNmk9"], "status": { "Failure": { "ActionError": { "index": 0, "kind": { "FunctionCallError": { "ExecutionError": "Smart contract panicked: ERR_INCORRECT_NONCE" } } } } }, "tokensBurnt": "349707622235100000000" }, "proof": [] }, "receipt": { "predecessorId": "relay.aurora", "receipt": { "Action": { "actions": [{ "FunctionCall": { "args": "+QGPggpehAQsHYCCUgiU5ShUyG+2SwTdHXetVCh23cBAxPSAuQEkGMuv5QAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADGiUDx20tY/zAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAR/fZpFXwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAoAAAAAAAAAAAAAAAAGBc4ZUyxxIdG15jLrd25ap7+4mQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABhaeq/ToAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAADCXWsrHC113TW91as0DqbkM1NT1AAAAAAAAAAAAAAAASYiolrEichjkpob95eq9yr2RVx8AAAAAAAAAAAAAAADJve7TPNAVQeHu0Q+QUZ0sBv4/64ScioLIoNNwZYHzq18jJVWwzwd9vJY1JV4H29H/wOgwyQVM2FJzoCUAB6Nz+98pSh9AfalXBMbThpftEUVyF49fYhEKwmJf", "deposit": "0", "gas": 300000000000000, "methodName": "submit" } }], "gasPrice": "625040174", "inputDataIds": [], "outputDataReceivers": [], "signerId": "relay.aurora", "signerPublicKey": "ed25519:2WmTnJeWZYKhh6NC3z3xTDKU5sHHAmHF96V6KAB3eHK5" } }, "receiptId": "821S5L8Htxafh2DDuDKtA4qapudN3mRq2AMv2qPeBhpj", "receiverId": "aurora" } }], "shardId": 1, "stateChanges": [{ "cause": { "receiptHash": "821S5L8Htxafh2DDuDKtA4qapudN3mRq2AMv2qPeBhpj", "type": "action_receipt_gas_reward" }, "change": { "accountId": "aurora", "amount": "62420509105129651830808079298", "codeHash": "qorYWFPQKMbJGcmjtWUhD3ee7fJJakRRUYFk3cao4W3", "locked": "0", "storagePaidAt": 0, "storageUsage": 6176907778 }, "type": "account_update" }] }, { "chunk": { "author": "republic.poolv1.near", "header": { "balanceBurnt": "242806857164400000000", "chunkHash": "6uDUqFuYwGohFMQ9UyRaHbmHALf8HS9F6WcNnZWypkka", "encodedLength": 1474, "encodedMerkleRoot": "9rtC25N5jqfLxMumUwvupL9MpMNUAEKuXZB8RmfkdD7p", "gasLimit": 1000000000000000, "gasUsed": 2428068571644, "heightCreated": 85376002, "heightIncluded": 85376002, "outcomeRoot": "AvRw1DBT1JFB2KNuahMvYucvttrem74GSEnwMacxbkv6", "outgoingReceiptsRoot": "C2QjTGsH8HcfR6W7ecLYwuYD3Vkw6mi9Pf9tFXuGdMLr", "prevBlockHash": "H5X25nVZgx43tQSMcF5bRLphWGN92UVYdvaAhcgW4no7", "prevStateRoot": "CtxzDGCGs1SAr7GEqrA8gSKCcbaPDMKegnTyPEJmU7mb", "rentPaid": "0", "shardId": 2, "signature": "ed25519:66LFkWB13BEUYRsmockx9LbiBzko9zohGkfbLwJertYWAa5TJY5ELFxPycps6poroh1C7Tfbh6NH7tQpgHZFr43d", "txRoot": "DVr9U4uYvJY1cz5awdSmKm4CovJmr1VcPyqCMonat8bD", "validatorProposals": [], "validatorReward": "0" }, "receipts": [{ "predecessorId": "evaenergy.near", "receipt": { "Action": { "actions": [{ "FunctionCall": { "args": "eyJ0YXNrX29yZGluYWwiOjEsImJpZCI6IjQ0NDQ2ODcxMTE2MjIyODMzODczMDc2In0=", "deposit": "0", "gas": 30000000000000, "methodName": "claim_assignment" } }], "gasPrice": "122987387", "inputDataIds": [], "outputDataReceivers": [], "signerId": "evaenergy.near", "signerPublicKey": "ed25519:9Q53x5kiLChb4tX3J5dV2wNAdTetejEcnizyQ3n3PZdg" } }, "receiptId": "6XN47idjeWhq2QpkjGs9w7pPg452Z1GwgwRxZjy2ny7T", "receiverId": "app.nearcrowd.near" }], "transactions": [{ "outcome": { "executionOutcome": { "blockHash": "A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM", "id": "F4z3CFPjUGRJEF9rNSVhz4hZG2LpBvA3rR3AHv68432i", "outcome": { "executorId": "chadmoney.near", "gasBurnt": 2428372658668, "logs": [], "metadata": { "gasProfile": null, "version": 1 }, "receiptIds": ["Ht6wAxzHfyCwWmfkngi9n6DaJu6UtoD1skHpJrRJQGik"], "status": { "SuccessReceiptId": "Ht6wAxzHfyCwWmfkngi9n6DaJu6UtoD1skHpJrRJQGik" }, "tokensBurnt": "242837265866800000000" }, "proof": [{ "direction": "Right", "hash": "E4X99JiTuYjYW6WPMiBhZ61Eo9fDYXmDdv67ZdkSbcf3" }, { "direction": "Right", "hash": "2NXhS3PRa1H8UZEtrNo14QhYNTzk5ztBhrSe6rb64heg" }] }, "receipt": null }, "transaction": { "actions": [{ "FunctionCall": { "args": "eyJhY3Rpb25zIjpbeyJwb29sX2lkIjo0LCJ0b2tlbl9pbiI6IndyYXAubmVhciIsInRva2VuX291dCI6ImRhYzE3Zjk1OGQyZWU1MjNhMjIwNjIwNjk5NDU5N2MxM2Q4MzFlYzcuZmFjdG9yeS5icmlkZ2UubmVhciIsImFtb3VudF9pbiI6IjEwMDAwMDAwMDAwMDAwMDAwMDAwMDAwMDAwMDAiLCJtaW5fYW1vdW50X291dCI6IjI0NzE5ODgxNjIifV19", "deposit": "1", "gas": 10000000000000, "methodName": "swap" } }], "hash": "F4z3CFPjUGRJEF9rNSVhz4hZG2LpBvA3rR3AHv68432i", "nonce": 72650480016948, "publicKey": "ed25519:78fzZoSWw4yr7gRev6onwRFQPu5auDAPNuAhM2Ryy4EC", "receiverId": "v2.ref-finance.near", "signature": "ed25519:SYNT4UNqfEpAofZykZpZUugyPxXpSxYPwo8BqVg1fPG8BZvQKJgmHkoLsEbG2bsTfW1iGr886E6mw4nRDLJQ9e1", "signerId": "chadmoney.near" } }, { "outcome": { "executionOutcome": { "blockHash": "A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM", "id": "3Zgj4eFNymhW7LBsaKtUsCWAaoP9LBXKGFtmebYsfibU", "outcome": { "executorId": "fd305d28fa12cb2e2769b211248059b8f4033ef4d4c0072c7daf246b166106ba", "gasBurnt": 2428312288450, "logs": [], "metadata": { "gasProfile": null, "version": 1 }, "receiptIds": ["ENQk2JGnPNtpXPUFkpBhnD3XUyv67eXB4XfmDU2sjHB7"], "status": { "SuccessReceiptId": "ENQk2JGnPNtpXPUFkpBhnD3XUyv67eXB4XfmDU2sjHB7" }, "tokensBurnt": "242831228845000000000" }, "proof": [{ "direction": "Left", "hash": "GAdP76ExKf5yQH9BU5usAeAWxiX2HkmDP7g54Ff47Zgz" }, { "direction": "Right", "hash": "2NXhS3PRa1H8UZEtrNo14QhYNTzk5ztBhrSe6rb64heg" }] }, "receipt": null }, "transaction": { "actions": [{ "FunctionCall": { "args": "eyJyZWNlaXZlcl9pZCI6InJld2FyZC1vcHRpbi5zd2VhdCIsImFtb3VudCI6IjEwMDAwMDAwMDAwMDAwMDAwMCIsIm1lbW8iOiJzdzpyZXc6b3B0aW46WUdSV3ZncFhLMC1mZDMwNWQyOGZhMTJjYjJlMjc2OWIyMTEyNDgwNTliOGY0MDMzZWY0ZDRjMDA3MmM3ZGFmMjQ2YjE2NjEwNmJhIn0=", "deposit": "1", "gas": 14000000000000, "methodName": "ft_transfer" } }], "hash": "3Zgj4eFNymhW7LBsaKtUsCWAaoP9LBXKGFtmebYsfibU", "nonce": 64674524000017, "publicKey": "ed25519:J3LvQdaf6r6YVuTrVzo6em32R2ZG7jE9n4ysiLUpMa4y", "receiverId": "token.sweat", "signature": "ed25519:3gdWeCTcqbT3ezViXEPaRiRafz4NcRRsP7bAwgY1CTFenKcFQB5gyvQqSHPAtQbx7JdppByH4j6SrEMxmPedFyPY", "signerId": "fd305d28fa12cb2e2769b211248059b8f4033ef4d4c0072c7daf246b166106ba" } }, { "outcome": { "executionOutcome": { "blockHash": "A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM", "id": "DbCXaFDNfQ7LjM5Jh8BS5fKLF6UJG3vJZYbGtfpDaTEB", "outcome": { "executorId": "de9d1abf39de5fc4eac513b559cc825352d03c7b642a8af657113d61f8c195dc", "gasBurnt": 2428158009004, "logs": [], "metadata": { "gasProfile": null, "version": 1 }, "receiptIds": ["86Xs7Z6wiQNb89hV5j33UHbBZTXrbBYtqwBMBT3Hhq4h"], "status": { "SuccessReceiptId": "86Xs7Z6wiQNb89hV5j33UHbBZTXrbBYtqwBMBT3Hhq4h" }, "tokensBurnt": "242815800900400000000" }, "proof": [{ "direction": "Right", "hash": "AJhsQtiU5riKpBKSBbHYD9svR84eZ3oYqd8eRjipqTFZ" }, { "direction": "Left", "hash": "45QTFnLgDmAbiiVBFmVHnivqQFU2ATNgLNErLnvZYZq2" }] }, "receipt": null }, "transaction": { "actions": [{ "FunctionCall": { "args": "eyJyZWNlaXZlcl9pZCI6ImRlcG9zaXRzLmdyb3cuc3dlYXQiLCJhbW91bnQiOiIxMDIxOTAwMDAwMDAwMDAwMDAwMDAiLCJtZW1vIjoic3c6ZDpnV095RFk0TEVrIn0=", "deposit": "1", "gas": 14000000000000, "methodName": "ft_transfer" } }], "hash": "DbCXaFDNfQ7LjM5Jh8BS5fKLF6UJG3vJZYbGtfpDaTEB", "nonce": 64963031000010, "publicKey": "ed25519:FyzTGZb9mP8gcDbyhLCLumsA38a1JF3E5WELLJHnJsab", "receiverId": "token.sweat", "signature": "ed25519:25q5o12NchK9rnUmrddTFKzYYCdqmPfQWq1QgRhGJiDiZmazqS2B4mPu7PeGYvVAEqucgq2k41EmXwgN3vLExjW5", "signerId": "de9d1abf39de5fc4eac513b559cc825352d03c7b642a8af657113d61f8c195dc" } }] }, "receiptExecutionOutcomes": [{ "executionOutcome": { "blockHash": "A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM", "id": "CEgnqr5aYdpBErSzjvdtmdqnXdmXjXqe1AnPeamsLcvM", "outcome": { "executorId": "f53b45ec59e26a50160026bd4e3d232a6672e0ee812d46cfcd29c0ef44a089f5", "gasBurnt": 424555062500, "logs": [], "metadata": { "gasProfile": [], "version": 1 }, "receiptIds": [], "status": { "SuccessValue": "" }, "tokensBurnt": "0" }, "proof": [{ "direction": "Left", "hash": "DiN9q4rgCEpmykqXzoXWP2dNdjLyWEAgYzi7UmsQhEpx" }, { "direction": "Left", "hash": "45QTFnLgDmAbiiVBFmVHnivqQFU2ATNgLNErLnvZYZq2" }] }, "receipt": { "predecessorId": "system", "receipt": { "Action": { "actions": [{ "Transfer": { "deposit": "1389731257024010315000" } }], "gasPrice": "0", "inputDataIds": [], "outputDataReceivers": [], "signerId": "f53b45ec59e26a50160026bd4e3d232a6672e0ee812d46cfcd29c0ef44a089f5", "signerPublicKey": "ed25519:HWHJq9M3m232SGpRTXuycdvjPEaaxn9TFN72bnEXnESp" } }, "receiptId": "CEgnqr5aYdpBErSzjvdtmdqnXdmXjXqe1AnPeamsLcvM", "receiverId": "f53b45ec59e26a50160026bd4e3d232a6672e0ee812d46cfcd29c0ef44a089f5" } }], "shardId": 2, "stateChanges": [{ "cause": { "txHash": "F4z3CFPjUGRJEF9rNSVhz4hZG2LpBvA3rR3AHv68432i", "type": "transaction_processing" }, "change": { "accountId": "chadmoney.near", "amount": "3455422147665387737062056", "codeHash": "11111111111111111111111111111111", "locked": "0", "storagePaidAt": 0, "storageUsage": 2282 }, "type": "account_update" }, { "cause": { "txHash": "DbCXaFDNfQ7LjM5Jh8BS5fKLF6UJG3vJZYbGtfpDaTEB", "type": "transaction_processing" }, "change": { "accountId": "de9d1abf39de5fc4eac513b559cc825352d03c7b642a8af657113d61f8c195dc", "amount": "37838802127313908609194", "codeHash": "11111111111111111111111111111111", "locked": "0", "storagePaidAt": 0, "storageUsage": 182 }, "type": "account_update" }, { "cause": { "receiptHash": "CEgnqr5aYdpBErSzjvdtmdqnXdmXjXqe1AnPeamsLcvM", "type": "receipt_processing" }, "change": { "accountId": "f53b45ec59e26a50160026bd4e3d232a6672e0ee812d46cfcd29c0ef44a089f5", "amount": "44000142104620699999996", "codeHash": "11111111111111111111111111111111", "locked": "0", "storagePaidAt": 0, "storageUsage": 182 }, "type": "account_update" }, { "cause": { "txHash": "3Zgj4eFNymhW7LBsaKtUsCWAaoP9LBXKGFtmebYsfibU", "type": "transaction_processing" }, "change": { "accountId": "fd305d28fa12cb2e2769b211248059b8f4033ef4d4c0072c7daf246b166106ba", "amount": "33618268317256489684987", "codeHash": "11111111111111111111111111111111", "locked": "0", "storagePaidAt": 0, "storageUsage": 182 }, "type": "account_update" }, { "cause": { "txHash": "F4z3CFPjUGRJEF9rNSVhz4hZG2LpBvA3rR3AHv68432i", "type": "transaction_processing" }, "change": { "accessKey": { "nonce": 72650480016948, "permission": "FullAccess" }, "accountId": "chadmoney.near", "publicKey": "ed25519:78fzZoSWw4yr7gRev6onwRFQPu5auDAPNuAhM2Ryy4EC" }, "type": "access_key_update" }, { "cause": { "txHash": "DbCXaFDNfQ7LjM5Jh8BS5fKLF6UJG3vJZYbGtfpDaTEB", "type": "transaction_processing" }, "change": { "accessKey": { "nonce": 64963031000010, "permission": "FullAccess" }, "accountId": "de9d1abf39de5fc4eac513b559cc825352d03c7b642a8af657113d61f8c195dc", "publicKey": "ed25519:FyzTGZb9mP8gcDbyhLCLumsA38a1JF3E5WELLJHnJsab" }, "type": "access_key_update" }, { "cause": { "txHash": "3Zgj4eFNymhW7LBsaKtUsCWAaoP9LBXKGFtmebYsfibU", "type": "transaction_processing" }, "change": { "accessKey": { "nonce": 64674524000017, "permission": "FullAccess" }, "accountId": "fd305d28fa12cb2e2769b211248059b8f4033ef4d4c0072c7daf246b166106ba", "publicKey": "ed25519:J3LvQdaf6r6YVuTrVzo6em32R2ZG7jE9n4ysiLUpMa4y" }, "type": "access_key_update" }] }, { "chunk": { "author": "binancenode1.poolv1.near", "header": { "balanceBurnt": "599537042985400000000", "chunkHash": "3Q39LzrowGBhqRZvkjDqQNjkfUEYEK6Zis5LUzqvrtFj", "encodedLength": 1837, "encodedMerkleRoot": "2d1EBsn1UspGLYjyFkrzPoawjghZe4fBbWkZfmsPJRbM", "gasLimit": 1000000000000000, "gasUsed": 7599093905353, "heightCreated": 85376002, "heightIncluded": 85376002, "outcomeRoot": "CUmJsZfPtNYEggezXf6wW7NX2i7Vn9yuPz7RWfVKxR7p", "outgoingReceiptsRoot": "8TG5j9ahcEtGUNREhCEJW3SgHyeCzochf4foYxSU7b8Q", "prevBlockHash": "H5X25nVZgx43tQSMcF5bRLphWGN92UVYdvaAhcgW4no7", "prevStateRoot": "EPbgnc5qJFvFGqqAyQQu2SrFcVr6mr3AVbSuXnHvZ1ie", "rentPaid": "0", "shardId": 3, "signature": "ed25519:RgPmNaXij3pNFdMVi1Ex8WXyQiYr12uUyRk5tuoddJvCTwJXcxQZZLCdy1XqB6uB9imMz9NsXQwT1M993NBMsjS", "txRoot": "8V1iVH9TSzWsPtNVorScHc4ha9ESNqNEG13tZiRSDzk7", "validatorProposals": [], "validatorReward": "0" }, "receipts": [{ "predecessorId": "relay.aurora", "receipt": { "Action": { "actions": [{ "FunctionCall": { "args": "+QGPggpehAQsHYCCUgiU5ShUyG+2SwTdHXetVCh23cBAxPSAuQEkGMuv5QAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADGiUDx20tY/zAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAR/fZpFXwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAoAAAAAAAAAAAAAAAAGBc4ZUyxxIdG15jLrd25ap7+4mQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABhaeq/ToAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAADCXWsrHC113TW91as0DqbkM1NT1AAAAAAAAAAAAAAAASYiolrEichjkpob95eq9yr2RVx8AAAAAAAAAAAAAAADJve7TPNAVQeHu0Q+QUZ0sBv4/64ScioLIoNNwZYHzq18jJVWwzwd9vJY1JV4H29H/wOgwyQVM2FJzoCUAB6Nz+98pSh9AfalXBMbThpftEUVyF49fYhEKwmJf", "deposit": "0", "gas": 300000000000000, "methodName": "submit" } }], "gasPrice": "625040174", "inputDataIds": [], "outputDataReceivers": [], "signerId": "relay.aurora", "signerPublicKey": "ed25519:2WmTnJeWZYKhh6NC3z3xTDKU5sHHAmHF96V6KAB3eHK5" } }, "receiptId": "821S5L8Htxafh2DDuDKtA4qapudN3mRq2AMv2qPeBhpj", "receiverId": "aurora" }, { "predecessorId": "system", "receipt": { "Action": { "actions": [{ "Transfer": { "deposit": "1389731257024010315000" } }], "gasPrice": "0", "inputDataIds": [], "outputDataReceivers": [], "signerId": "f53b45ec59e26a50160026bd4e3d232a6672e0ee812d46cfcd29c0ef44a089f5", "signerPublicKey": "ed25519:HWHJq9M3m232SGpRTXuycdvjPEaaxn9TFN72bnEXnESp" } }, "receiptId": "CEgnqr5aYdpBErSzjvdtmdqnXdmXjXqe1AnPeamsLcvM", "receiverId": "f53b45ec59e26a50160026bd4e3d232a6672e0ee812d46cfcd29c0ef44a089f5" }], "transactions": [{ "outcome": { "executionOutcome": { "blockHash": "A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM", "id": "AMdmdjfHvKX6vv1fNEyCCio2jmSvepxSetxQFhgj826f", "outcome": { "executorId": "relay.aurora", "gasBurnt": 2429696331596, "logs": [], "metadata": { "gasProfile": null, "version": 1 }, "receiptIds": ["9p28ts6sz8Meg4vu3Fqff5QmfkwFc65C22WgAoKF1ELh"], "status": { "SuccessReceiptId": "9p28ts6sz8Meg4vu3Fqff5QmfkwFc65C22WgAoKF1ELh" }, "tokensBurnt": "242969633159600000000" }, "proof": [{ "direction": "Right", "hash": "CKbaXt3vrGUUt7Ydw1gjb9by6pbtMooXo7uWJrbWAXXX" }, { "direction": "Right", "hash": "4NXWzpp8uSWuhoAXgsJKCs2eyAVHL3wXxsAWocx1Fs5K" }, { "direction": "Right", "hash": "FmBQ954izf84EZ5P4GVvmUfc8T4bGXXqQruyDTtikwo1" }] }, "receipt": null }, "transaction": { "actions": [{ "FunctionCall": { "args": "+QMRgw65o4QELB2Agww1AJSQVQb22oFec8oTVHtF0ZmIZxBLIoC5AqS/zVdxAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAApRioAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAgAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADgAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAcAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQDIwNjdlMDg4OWUyYmJmM2ZjYTI3NmFjOWNiYzdiYjI5OThmNzQzMjM1ZjUwNjU4NDUwNjMyMzQxMDJhMGEzODYAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwGI2NmMxNTdiYTljMWM0YTAxOWEzOTQ4MjlkMzQyMGQ4YjIxZDg5OTQzZjMzYTdiYTExMDQ0MWFiNDExOGI4NTYzZWNiZTkxM2Q5YTJmZmNhNTNiNGU1NWJjNTFiZDdmNjEyZDQxZDkxNTY4M2MzNGNiZWVhYTg0Yzc0NWI2N2VmNjZlMTM2YjE2NTFkZTEwODk3OWM0YjE4NzFiMjIwMjIyYmI4ZTBkNTgwNmM0ZjFlZjliY2U1OWRlNzI1Y2Y0YwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAYTE5ZWJlNmFmZjRiM2MxZGIxZjY2Njc5Nzc1MDc1ZTZkMzY0NGFlYjBiZmYyZmQxMjQzNjg5ZTQ3ZmNjYTk5NzA4NzhlY2E5MzI4ZDQ5YmM1ZGZiYWI0ZDU4N2Y4OGQ5MTkyMTJiY2FjYzY0YzJhNGFmMDUxZmFkZWE4NjdhZDkwMDUzMzY4OWQ1ZDcwNzg5YTQyMjNmMzVkM2JhYzhlNzE0NzY2NDg5NjQ4ODAxMTk1NGRlMDRiZWM0NDM3N2U4hJyKgsegSqdU7My5fwrLI1Mc3JcSJSgvQcmZsI1Chymt8Ydj0gqgXGqO8uaGvpV0jbfOyplT73XLhyC4ON9ekQI+tWB+8go=", "deposit": "0", "gas": 300000000000000, "methodName": "submit" } }], "hash": "AMdmdjfHvKX6vv1fNEyCCio2jmSvepxSetxQFhgj826f", "nonce": 65788368017229, "publicKey": "ed25519:FptiMvPBN7QRJwroc2zJtTEAEo4h9CJAdGKQfd2bA3Qm", "receiverId": "aurora", "signature": "ed25519:LehYX21sAyodn5g7MxXPkztx9tQMHQ92gMg2KQYQV4pBKyGidLHz6y2aeLREUFt2ckxrdWNZ6VannAGBsPpiHPu", "signerId": "relay.aurora" } }] }, "receiptExecutionOutcomes": [{ "executionOutcome": { "blockHash": "A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM", "id": "CdUSBwV29D1P1qvxCwnbfNsTsdxfdRY6PxezsSWRFyqN", "outcome": { "executorId": "relay.aurora", "gasBurnt": 223182562500, "logs": [], "metadata": { "gasProfile": [], "version": 1 }, "receiptIds": [], "status": { "SuccessValue": "" }, "tokensBurnt": "0" }, "proof": [{ "direction": "Left", "hash": "3zQuvAtY9UVHq2ws8vvMLmPivCpBPWwDhmTtCHNy1roV" }, { "direction": "Right", "hash": "4NXWzpp8uSWuhoAXgsJKCs2eyAVHL3wXxsAWocx1Fs5K" }, { "direction": "Right", "hash": "FmBQ954izf84EZ5P4GVvmUfc8T4bGXXqQruyDTtikwo1" }] }, "receipt": { "predecessorId": "system", "receipt": { "Action": { "actions": [{ "Transfer": { "deposit": "187086049935277727891452" } }], "gasPrice": "0", "inputDataIds": [], "outputDataReceivers": [], "signerId": "relay.aurora", "signerPublicKey": "ed25519:8j2e8u9t1SoTgMPn5G65ChU5WskpcnemBjwzkAnxDtZq" } }, "receiptId": "CdUSBwV29D1P1qvxCwnbfNsTsdxfdRY6PxezsSWRFyqN", "receiverId": "relay.aurora" } }, { "executionOutcome": { "blockHash": "A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM", "id": "7Q29UAwrFbeMMM6niPSuMrqkp4Y21FYqJ1SXPvr9oZYU", "outcome": { "executorId": "token.sweat", "gasBurnt": 4072736369755, "logs": ["EVENT_JSON:{\"standard\":\"nep141\",\"version\":\"1.0.0\",\"event\":\"ft_transfer\",\"data\":[{\"old_owner_id\":\"7a8ba1bd804863883ace10ed76441e6ad2239ae7917b78c89700b701f75ce1ab\",\"new_owner_id\":\"reward-optin.sweat\",\"amount\":\"100000000000000000\",\"memo\":\"sw:rew:optin:YGRWvgpXK0-7a8ba1bd804863883ace10ed76441e6ad2239ae7917b78c89700b701f75ce1ab\"}]}"], "metadata": { "gasProfile": [{ "cost": "BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "7413507108" }, { "cost": "CONTRACT_LOADING_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "35445963" }, { "cost": "CONTRACT_LOADING_BYTES", "costCategory": "WASM_HOST_COST", "gasUsed": "44831486250" }, { "cost": "LOG_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "3543313050" }, { "cost": "LOG_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "4342402239" }, { "cost": "READ_CACHED_TRIE_NODE", "costCategory": "WASM_HOST_COST", "gasUsed": "280440000000" }, { "cost": "READ_MEMORY_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "36538084800" }, { "cost": "READ_MEMORY_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "2816787753" }, { "cost": "READ_REGISTER_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "27688817046" }, { "cost": "READ_REGISTER_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "48295380" }, { "cost": "SHA256_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "18163881000" }, { "cost": "SHA256_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "3955245564" }, { "cost": "STORAGE_READ_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "169070537250" }, { "cost": "STORAGE_READ_KEY_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "2259534909" }, { "cost": "STORAGE_READ_VALUE_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "572322510" }, { "cost": "STORAGE_WRITE_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "192590208000" }, { "cost": "STORAGE_WRITE_EVICTED_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "3275965314" }, { "cost": "STORAGE_WRITE_KEY_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "5145249291" }, { "cost": "STORAGE_WRITE_VALUE_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "3163890978" }, { "cost": "TOUCHING_TRIE_NODE", "costCategory": "WASM_HOST_COST", "gasUsed": "611874325188" }, { "cost": "UTF8_DECODING_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "3111779061" }, { "cost": "UTF8_DECODING_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "95929977591" }, { "cost": "WASM_INSTRUCTION", "costCategory": "WASM_HOST_COST", "gasUsed": "56074112424" }, { "cost": "WRITE_MEMORY_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "33645538332" }, { "cost": "WRITE_MEMORY_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "1378228632" }, { "cost": "WRITE_REGISTER_BASE", "costCategory": "WASM_HOST_COST", "gasUsed": "34386269832" }, { "cost": "WRITE_REGISTER_BYTE", "costCategory": "WASM_HOST_COST", "gasUsed": "2128875840" }], "version": 1 }, "receiptIds": ["BXhqz9Hk5eVb5ibaN3uZmNW4cgW6zBerUYpM7Zh19kXe"], "status": { "SuccessValue": "" }, "tokensBurnt": "407273636975500000000" }, "proof": [{ "direction": "Right", "hash": "Cbwr9BAamScEJV1aMoZ21CULLdjPTpY6QQmfZNAS8cUx" }, { "direction": "Left", "hash": "7x71otoV7zEDfh2v1hYcLD9GNFZkoza1d3uoJUc79GMV" }, { "direction": "Right", "hash": "FmBQ954izf84EZ5P4GVvmUfc8T4bGXXqQruyDTtikwo1" }] }, "receipt": { "predecessorId": "7a8ba1bd804863883ace10ed76441e6ad2239ae7917b78c89700b701f75ce1ab", "receipt": { "Action": { "actions": [{ "FunctionCall": { "args": "eyJyZWNlaXZlcl9pZCI6InJld2FyZC1vcHRpbi5zd2VhdCIsImFtb3VudCI6IjEwMDAwMDAwMDAwMDAwMDAwMCIsIm1lbW8iOiJzdzpyZXc6b3B0aW46WUdSV3ZncFhLMC03YThiYTFiZDgwNDg2Mzg4M2FjZTEwZWQ3NjQ0MWU2YWQyMjM5YWU3OTE3Yjc4Yzg5NzAwYjcwMWY3NWNlMWFiIn0=", "deposit": "1", "gas": 14000000000000, "methodName": "ft_transfer" } }], "gasPrice": "109272700", "inputDataIds": [], "outputDataReceivers": [], "signerId": "7a8ba1bd804863883ace10ed76441e6ad2239ae7917b78c89700b701f75ce1ab", "signerPublicKey": "ed25519:9FNEMP92rkuvRVefVkBW38RmbZ648XvQGeWbFYtdC7hk" } }, "receiptId": "7Q29UAwrFbeMMM6niPSuMrqkp4Y21FYqJ1SXPvr9oZYU", "receiverId": "token.sweat" } }, { "executionOutcome": { "blockHash": "A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM", "id": "BpZGWvbmx8sAMRsrhWyBnC5QHA9mFwist7orTceJTRvu", "outcome": { "executorId": "olezha4.near", "gasBurnt": 223182562500, "logs": [], "metadata": { "gasProfile": [], "version": 1 }, "receiptIds": [], "status": { "SuccessValue": "" }, "tokensBurnt": "0" }, "proof": [{ "direction": "Left", "hash": "GdEbJpKZ4iqem9VmS5Qbm2CTrt1qFZhfSD6zUv5JaYXT" }, { "direction": "Left", "hash": "7x71otoV7zEDfh2v1hYcLD9GNFZkoza1d3uoJUc79GMV" }, { "direction": "Right", "hash": "FmBQ954izf84EZ5P4GVvmUfc8T4bGXXqQruyDTtikwo1" }] }, "receipt": { "predecessorId": "system", "receipt": { "Action": { "actions": [{ "Transfer": { "deposit": "13222602662027447938448" } }], "gasPrice": "0", "inputDataIds": [], "outputDataReceivers": [], "signerId": "olezha4.near", "signerPublicKey": "ed25519:EnLNUDncRuw4AYLcGmi1Hr2YQ1DLu12VutsubZ6k8c2F" } }, "receiptId": "BpZGWvbmx8sAMRsrhWyBnC5QHA9mFwist7orTceJTRvu", "receiverId": "olezha4.near" } }, { "executionOutcome": { "blockHash": "A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM", "id": "3R2wjnktLwBKaceGiPetmShVAyHT2LuoQZMdNg6oWMwK", "outcome": { "executorId": "marthacecilia.near", "gasBurnt": 223182562500, "logs": [], "metadata": { "gasProfile": [], "version": 1 }, "receiptIds": [], "status": { "SuccessValue": "" }, "tokensBurnt": "0" }, "proof": [{ "direction": "Left", "hash": "6FWhYXFGBgzSPDuBCFrfSkdAqSvsMHPbnCJ2Q3UsmUV6" }] }, "receipt": { "predecessorId": "system", "receipt": { "Action": { "actions": [{ "Transfer": { "deposit": "13239855304236542651168" } }], "gasPrice": "0", "inputDataIds": [], "outputDataReceivers": [], "signerId": "marthacecilia.near", "signerPublicKey": "ed25519:8W28RYFfXztNA7ENP4VTaitsqEjSbABQJDqELwbwXswq" } }, "receiptId": "3R2wjnktLwBKaceGiPetmShVAyHT2LuoQZMdNg6oWMwK", "receiverId": "marthacecilia.near" } }], "shardId": 3, "stateChanges": [{ "cause": { "receiptHash": "3R2wjnktLwBKaceGiPetmShVAyHT2LuoQZMdNg6oWMwK", "type": "receipt_processing" }, "change": { "accountId": "marthacecilia.near", "amount": "376417715605581247237588", "codeHash": "7DcAdMUT1MjaZ9s7zhXdyxKvQsRsSfnmBGdzeZaquqDE", "locked": "0", "storagePaidAt": 0, "storageUsage": 13722 }, "type": "account_update" }, { "cause": { "receiptHash": "BpZGWvbmx8sAMRsrhWyBnC5QHA9mFwist7orTceJTRvu", "type": "receipt_processing" }, "change": { "accountId": "olezha4.near", "amount": "147297020835776249333187456", "codeHash": "E8jZ1giWcVrps8PcV75ATauu6gFRkcwjNtKp7NKmipZG", "locked": "0", "storagePaidAt": 0, "storageUsage": 25029 }, "type": "account_update" }, { "cause": { "txHash": "AMdmdjfHvKX6vv1fNEyCCio2jmSvepxSetxQFhgj826f", "type": "transaction_processing" }, "change": { "accountId": "relay.aurora", "amount": "2493412853875426176695038138", "codeHash": "11111111111111111111111111111111", "locked": "0", "storagePaidAt": 0, "storageUsage": 149094 }, "type": "account_update" }, { "cause": { "receiptHash": "CdUSBwV29D1P1qvxCwnbfNsTsdxfdRY6PxezsSWRFyqN", "type": "receipt_processing" }, "change": { "accountId": "relay.aurora", "amount": "2493599939925361454422929590", "codeHash": "11111111111111111111111111111111", "locked": "0", "storagePaidAt": 0, "storageUsage": 149094 }, "type": "account_update" }, { "cause": { "receiptHash": "7Q29UAwrFbeMMM6niPSuMrqkp4Y21FYqJ1SXPvr9oZYU", "type": "receipt_processing" }, "change": { "accountId": "token.sweat", "amount": "32545059778997347633373318383", "codeHash": "FMy4MTxATGtfxqTg5PZfGhQpRWej9Ppbttwo7FWF13wA", "locked": "0", "storagePaidAt": 0, "storageUsage": 1648821628 }, "type": "account_update" }, { "cause": { "receiptHash": "7Q29UAwrFbeMMM6niPSuMrqkp4Y21FYqJ1SXPvr9oZYU", "type": "action_receipt_gas_reward" }, "change": { "accountId": "token.sweat", "amount": "32545059828330070072473318383", "codeHash": "FMy4MTxATGtfxqTg5PZfGhQpRWej9Ppbttwo7FWF13wA", "locked": "0", "storagePaidAt": 0, "storageUsage": 1648821628 }, "type": "account_update" }, { "cause": { "receiptHash": "3R2wjnktLwBKaceGiPetmShVAyHT2LuoQZMdNg6oWMwK", "type": "receipt_processing" }, "change": { "accessKey": { "nonce": 85297952000430, "permission": { "FunctionCall": { "allowance": "117616350729529600000000", "methodNames": [], "receiverId": "app.nearcrowd.near" } } }, "accountId": "marthacecilia.near", "publicKey": "ed25519:8W28RYFfXztNA7ENP4VTaitsqEjSbABQJDqELwbwXswq" }, "type": "access_key_update" }, { "cause": { "receiptHash": "BpZGWvbmx8sAMRsrhWyBnC5QHA9mFwist7orTceJTRvu", "type": "receipt_processing" }, "change": { "accessKey": { "nonce": 84682948002117, "permission": { "FunctionCall": { "allowance": "330678248075353400000000", "methodNames": [], "receiverId": "app.nearcrowd.near" } } }, "accountId": "olezha4.near", "publicKey": "ed25519:EnLNUDncRuw4AYLcGmi1Hr2YQ1DLu12VutsubZ6k8c2F" }, "type": "access_key_update" }, { "cause": { "txHash": "AMdmdjfHvKX6vv1fNEyCCio2jmSvepxSetxQFhgj826f", "type": "transaction_processing" }, "change": { "accessKey": { "nonce": 65788368017229, "permission": "FullAccess" }, "accountId": "relay.aurora", "publicKey": "ed25519:FptiMvPBN7QRJwroc2zJtTEAEo4h9CJAdGKQfd2bA3Qm" }, "type": "access_key_update" }, { "cause": { "receiptHash": "7Q29UAwrFbeMMM6niPSuMrqkp4Y21FYqJ1SXPvr9oZYU", "type": "receipt_processing" }, "change": { "accountId": "token.sweat", "keyBase64": "U1RBVEU=", "valueBase64": "AgAAAHNpAQAAAAAAAAACAAAAc2UBAAAAdAEIAAAALnUuc3dlYXQUQEmH9+nGUg8Ez0cAAAAAWgAAAAAAAADiIbWN+QEAAA==" }, "type": "data_update" }, { "cause": { "receiptHash": "7Q29UAwrFbeMMM6niPSuMrqkp4Y21FYqJ1SXPvr9oZYU", "type": "receipt_processing" }, "change": { "accountId": "token.sweat", "keyBase64": "dAAEzAvKN2Ekj8gSsSzscbcviXEnrSF9E4Nn8XlqPYFyaw==", "valueBase64": "6Hk2larECkgAAAAAAAAAAA==" }, "type": "data_update" }, { "cause": { "receiptHash": "7Q29UAwrFbeMMM6niPSuMrqkp4Y21FYqJ1SXPvr9oZYU", "type": "receipt_processing" }, "change": { "accountId": "token.sweat", "keyBase64": "dADCUiigKPOTAl1jqrFvCFMV6YPDQuu8kGehwLHx0CgpIw==", "valueBase64": "AAAu48ylkU6WIwAAAAAAAA==" }, "type": "data_update" }] }] }; +export const block_details = { + block: { + author: 'continue.poolv1.near', + chunks: [ + { + balanceBurnt: '1486252472505500000000', + chunkHash: '3qxSWuuo5QLdgpYSuwtQcAcAoX91b6p55CLxjda2wNNw', + encodedLength: 1772, + encodedMerkleRoot: 'ChpbnZdB2NdSaC2ir443zB4Bv6WUMSDBY6n3RfDyEVs', + gasLimit: 1000000000000000, + gasUsed: 16028665261860, + heightCreated: 85376002, + heightIncluded: 85376002, + outcomeRoot: 'G5c89nTRQSRYPdJQFcP47YXccbmbrbutNwLZzKUeRG3f', + outgoingReceiptsRoot: '5GrcVRVw9njFAskhcEbUyPZAoKMEEjGnC6edUhupEDkj', + prevBlockHash: 'H5X25nVZgx43tQSMcF5bRLphWGN92UVYdvaAhcgW4no7', + prevStateRoot: 'CRaTBLfd55cw6i5GfVvVSo1rw5ioZ91hT3ZVo5eBmNCi', + rentPaid: '0', + shardId: 0, + signature: 'ed25519:4Fzq1idZFShNtErzbK5uZWDcnWhHTMshVBvXG9c8cvc3YPkHF8SVgRsMym6V3WNHVY29gNaw1kXPCLBApcZ2bA1h', + txRoot: '5uqwhjC7wRJsvidh7gjtRRPvvUVwyefdQCivBMTL1hak', + validatorProposals: [], + validatorReward: '0', + }, + { + balanceBurnt: '1433581134734200000000', + chunkHash: 'BqfFW2NyLJuyp1pGgXd2ffYPGptNswNVH6ZJuTq5zSNa', + encodedLength: 161, + encodedMerkleRoot: 'EB5tG2a3aVfXv41ESuQVT1ZvQFvuwKnkYGLc2CDVCf5w', + gasLimit: 1000000000000000, + gasUsed: 19438956235189, + heightCreated: 85376002, + heightIncluded: 85376002, + outcomeRoot: 'Cp7GeqLAqZ1Z1GbtCL5nG13wxy6mV6PaRdwLddDHjhVc', + outgoingReceiptsRoot: 'DP1DT6QP1tNHM5ZjHMJ3C8G2mHJd8kGoFFNEdZPN8Hyx', + prevBlockHash: 'H5X25nVZgx43tQSMcF5bRLphWGN92UVYdvaAhcgW4no7', + prevStateRoot: '73sn3JyP1LU58r1FsTo5ZVgvErdbEGFxtiGExkWwhQUd', + rentPaid: '0', + shardId: 1, + signature: 'ed25519:3RYj6fpboT1ALHsVxYtYmofT8fGf1VRhNaJhWY45hsKdePcX3AZb3rdKBAZMThF5PVn5j5boKWHPb9zd1xZWYzex', + txRoot: '11111111111111111111111111111111', + validatorProposals: [], + validatorReward: '0', + }, + { + balanceBurnt: '242806857164400000000', + chunkHash: '6uDUqFuYwGohFMQ9UyRaHbmHALf8HS9F6WcNnZWypkka', + encodedLength: 1474, + encodedMerkleRoot: '9rtC25N5jqfLxMumUwvupL9MpMNUAEKuXZB8RmfkdD7p', + gasLimit: 1000000000000000, + gasUsed: 2428068571644, + heightCreated: 85376002, + heightIncluded: 85376002, + outcomeRoot: 'AvRw1DBT1JFB2KNuahMvYucvttrem74GSEnwMacxbkv6', + outgoingReceiptsRoot: 'C2QjTGsH8HcfR6W7ecLYwuYD3Vkw6mi9Pf9tFXuGdMLr', + prevBlockHash: 'H5X25nVZgx43tQSMcF5bRLphWGN92UVYdvaAhcgW4no7', + prevStateRoot: 'CtxzDGCGs1SAr7GEqrA8gSKCcbaPDMKegnTyPEJmU7mb', + rentPaid: '0', + shardId: 2, + signature: 'ed25519:66LFkWB13BEUYRsmockx9LbiBzko9zohGkfbLwJertYWAa5TJY5ELFxPycps6poroh1C7Tfbh6NH7tQpgHZFr43d', + txRoot: 'DVr9U4uYvJY1cz5awdSmKm4CovJmr1VcPyqCMonat8bD', + validatorProposals: [], + validatorReward: '0', + }, + { + balanceBurnt: '599537042985400000000', + chunkHash: '3Q39LzrowGBhqRZvkjDqQNjkfUEYEK6Zis5LUzqvrtFj', + encodedLength: 1837, + encodedMerkleRoot: '2d1EBsn1UspGLYjyFkrzPoawjghZe4fBbWkZfmsPJRbM', + gasLimit: 1000000000000000, + gasUsed: 7599093905353, + heightCreated: 85376002, + heightIncluded: 85376002, + outcomeRoot: 'CUmJsZfPtNYEggezXf6wW7NX2i7Vn9yuPz7RWfVKxR7p', + outgoingReceiptsRoot: '8TG5j9ahcEtGUNREhCEJW3SgHyeCzochf4foYxSU7b8Q', + prevBlockHash: 'H5X25nVZgx43tQSMcF5bRLphWGN92UVYdvaAhcgW4no7', + prevStateRoot: 'EPbgnc5qJFvFGqqAyQQu2SrFcVr6mr3AVbSuXnHvZ1ie', + rentPaid: '0', + shardId: 3, + signature: 'ed25519:RgPmNaXij3pNFdMVi1Ex8WXyQiYr12uUyRk5tuoddJvCTwJXcxQZZLCdy1XqB6uB9imMz9NsXQwT1M993NBMsjS', + txRoot: '8V1iVH9TSzWsPtNVorScHc4ha9ESNqNEG13tZiRSDzk7', + validatorProposals: [], + validatorReward: '0', + }, + ], + header: { + approvals: [ + 'ed25519:3XJG7eriCesmFBP78Q6iECzr34boAvEKn1nMintoQyf8MGf6TUzkjH91jvnb3putUvCicXDzgQjYrjbqTrgQZdTB', + 'ed25519:238a4WVjhbpubpMhMi2wKzR6wjoVLZ7788KPE3a3RzkppLzd22CChmEHHEyqsFj5VpWFVr13iTmgcgh2MomMxhxQ', + null, + 'ed25519:3hE7715teaeQE2UNWuZy9T6B7yA833NeBZuEsfYnHWGmGvQtApVwyX5QsVF1NV8cT24yQTus1u98JPjUr3X1XBdG', + 'ed25519:2JFfKfhtZFLrqNHSKWfojnQ63986w9BDVckL88ny3wzmEgextjkvAAmg12EjsmCwXUdc2VZ6zFBwto2pMVSovHcE', + 'ed25519:3V28c6KbuvuV21W5h8AsbjfZYQXA7aPJEsBi97xpJWAtULikSqqjSVhoLk6ckiAhG83G2nUWc12M8bcroyRB9Lso', + 'ed25519:5rp3ZGsLKo6oHrUix29cEDdfMAeiWuPmaQEh4T22pCPFEGMQ4jf5WRG8PZKv8zKGeia7NJrWy5iR88k2KF554c7A', + 'ed25519:2is1ybPjjGhJGJx7BTWbtXVbQNauwEAZLywtZFGrjLPyXdekxGqG695K3W9q7xnfCJYEXzUxpcANhcL1viG7fG3F', + 'ed25519:3h8Bx2e8Zb5wRw9Cph29FGm1pvWhopqSfbWzQeeawQAssSAd2fcy32yJGtZabHtLp2VRRznv6CgotS4Cc2JWRjUR', + 'ed25519:v7cD1AUXxnqamFqZU3mg3h7zAx6rWZvWARsZhCc28zAHrTM3YgEJiEJCbqcTDtRRCxC2uyBwzzmZftaDum8p5Py', + 'ed25519:5VDMQYTp59pwJ1eVHjcFSUeQnmnkBC9F8jEHwjxoJb1YewM8VAgQCb718r4EpV9Z41MHW57i9H3gP9JHSWJiZUL2', + 'ed25519:2zwMz7w2MJHpg8F4G9AAu5UEy6Yo2TFw1VGFCAUsQrP3WwGDwUbsCR63Y88M9v4aWFdyr38no6yzd2DEsrQQCq7b', + 'ed25519:3M3cA2X1DqWdiELg7PKuFDf8u9MJC2rNhiHccZBYJxZcvKeomgRhGLcgjniEiAmC4uaYq86ePKY7e68smydbXRFr', + 'ed25519:5hX41Uom35xmLD5pTVaxEXzzQMJiS5Vi4Eyd3bBHcd2dKbE4g49AyCmCJ3Mpqr1xEEMsWGKtcLAFpAkhEyiTS73H', + 'ed25519:42kqMAMQdA1b1No1PhLwp8FL2g789vkwHTshiHLjvqSfJ4t6B8krevBj3Z2AmeEYduGMZGNaMcDmFG43vifoX15F', + null, + 'ed25519:4AxJBk1cK9sqHwP4F32KQ21tj37kroBqMyesqXy2ksF8MWmXpxC3R2UKwFJRp3EztNaD8EM2vy7oFRmdumZcM2Gf', + 'ed25519:3KJDnpo4vxwyBDE11AWVRrnyGh9fYgq1GrsinEUKdwkVSCGKGv3rQrm19y5drxZEsqRw39D1bws1scc1TniCv1AX', + 'ed25519:PapbiGeqfur5csDNK3C9Cky6eLUQxzQXDPmE441MqFv5x1VDmEe2ZqHQecjiU6p18o1sQMGwgMARQZxHFDenYLB', + 'ed25519:57Pz4cEpBK2WCMnSaqt4jswZ4QYyhcRx1xG9pD44mp8mLCG3ogJhrq23eFF2xMizUzEDSD7AVMG1tByRob8VCZYm', + null, + null, + 'ed25519:5qNA6byTPhr6GHyTGxEZfRX12tYbqiYy7dQ4kdZVx4PW3i7s5i4q3ZUmwqP5rq6bdxx5fGkdMUVc3sqomz2dErE7', + 'ed25519:5YcgeAxawhVaTnvuRTWbPCtDtS6NH2UrNjeXcn89eXbKHTqeWtHsXKAb1NptrLgKr8K5WUGXjTKicz1xJyv9HgNh', + 'ed25519:sqZFxPtQpBnWUzcuPJeQUqyp3z3vYDYYe7Fvgcu4pVA46ueaU1oyTBpoMakY7pkZGyPq4jmgEamS3Bupf1W8KW3', + null, + null, + null, + null, + null, + 'ed25519:291jQZp5NAeogGttS75JYoqaenVKkQHP8M1YXSB8kpB2UJWN4wn7CCPZUxSPPSsPYmvYfpX3pkrxhJ7gAms9h743', + null, + 'ed25519:W2igco7sqdc9am1aVYA59yfPUVCJ8v4y6G5GAgfN9xpHy9r5QCKmatHpVJ6FMzinhy9GkCiZ3qJizbQBSWScwcP', + null, + 'ed25519:4q7PyxNTZtEdoSQf4azWVmXVs851JH9aYJc9B2H67AdkmM5sBz2gDFce1b9c36xWccUtf7cDDGBE6YPAf5eELax6', + 'ed25519:3ECDJscq5C3JLy6XeqAX5qKSvk8KXRjucJPjCiTW7vLmwb5C1MHR8VUZa6ufpN5chHMgFie2L4D82nMuPABPkXmz', + 'ed25519:29VK9ikBVSp2ZDBKhLDXC4QbA3XXAp5gdMRvVtANNDW29M6boo939HXpwhazhVSJydft8x4WFNfrJUykGrPHtQi6', + null, + null, + null, + null, + 'ed25519:5nUAvGput21GYcbAYuPncCvhyMSQuoKD2eLxb3pUVhLcPyYCxETWwtY4k5AL1p5aR7c9aBG1cfEqUXRB99Y2zwYJ', + 'ed25519:3mae6hCDVnY1WrV1N74xnTkvN5ihTn8Y9SKUPY32zS4jpqJWKzdopgX7KZniiFkC41JhD4vQjWwKChvxfaztBKq6', + null, + 'ed25519:5XjF6gyHudHwy3b48eGsED47myBjxB2dxvYZG2m954eQquMyLcVwKRZJvAXbzbHKehXUP8r9eeL3rm2LriYsx1dD', + 'ed25519:5vtjieSMvoYXYYP3zit9x6Dd7qVRDPYmoBZfELvouSX5ur9dnHmUnjEAXsWSWB69obm6hnfJz1Bx5gkXjLvyen3X', + null, + null, + null, + 'ed25519:3WK1bgn3qqzSiZcEphbKcw6UajNSDdLSYeGYMSMGH4YMdS7FnFKAhdigotRZ8z32adAzZp1URjd39yBHBibJVnDJ', + 'ed25519:3hkDFUQhi9pDWKtb6YtcuFy56QY9CkhAKvEwrg99wFYtmcYicN5N3tSu3bSKpFFEWop3qsvr5rCc3zWwkqKVT2nb', + null, + 'ed25519:rpALm3QcZov6qnDscsyyKwtgLSXVjppugEgM1DAzMw6Ewk2ovVAtow5frCV7nc1umnDLuxfLSfRNxkjeDqgAr4S', + 'ed25519:282Bw4icWLoUEAa6KUmixVeGSjjt6o1GAMcLypc1ruCqneENnzPC9hVqUWWGe9PmHU6GCEDoe79Y8hGpP9GuDYdA', + 'ed25519:2JQENivaSR9iDsbEXB3gz6PdqBiNbWRKU1e9XvcChzwtHp3gcCMW6Gnr362ktodDcjtcxZzyR4mHKBgREm2iRDJC', + null, + 'ed25519:35ffy6YAW7V1VSF1XcYNQUXnQxgYyNpsaUWEF3Gu7LgijZkdFqVdbPxfQpKCntWuC3LLrb8Vs1a3WNHgmEcub4Ng', + null, + null, + 'ed25519:53swEewwpM15AN4uBW3DPSdX4VTcx7XvP7d7LZNtxDga7JnNXZXVYJeNw31ui9LB1ffnHMNyAFoiQMLm5bnPQcsq', + 'ed25519:3Wb7R4K6DXA3o3DDAEwaYNY72SwqUadjqesMYvq9cmhoHm2UtyDMp1wPWCWMLQ4BfrshGdMgryvSn8cpeiVuv5HN', + null, + 'ed25519:5gGB9iTkt31SvAYaR52WfYf51uhhUb3CTf9RDN3MXoD3VR4m8fwBgRpmeu2Txb1RQJCwczxfYByYuRJR3orKkXA', + 'ed25519:2L7h92UaYwSXGBTqVkuhpRKZwRpW2wB7568U6jcBnH3iQtj71jERJdAtru62dx17teRngxJjDzMAG2Hs273Nt4cn', + 'ed25519:DL2QVuWC6CWrr519nQiKRtgC2H59ebBKNb5MjAKyv64zF3VrguWKQCcfaRUGgugiVeUjx12m2QfQigHmrqHWPmi', + 'ed25519:2uRyW8QrTaxdeX11tKxFmCY1FZbHbz7iWKxrNzGV9jiJg56BJgRKSZKbFpzPTakWMsesUfUUq4HNB4YsE6pvJSsN', + 'ed25519:2JB1CzirhvYfDVhay5jdzW5zGhGCLftfAB2Ff1ByNRnrvN7osvswuzLGHYDbmdMPdhC1ZTqeNCmKnQHzqQxKSjzN', + null, + 'ed25519:4YNGqdPNKGGDcuWAnW1DFJfJ6Qeqz5wh7rGLj7gSK2jXYUh9vnuXcKoC2fBWsTKtfawncDyKE7sAfW3THaSqrDDM', + null, + null, + null, + null, + 'ed25519:KhbmAUdmHgTuVkUsBX2PdbPsLgKMtr8ds7pWrb1otn5dKidQJc8Vdtq5MBZUrn6pRL2d96YWL9jxTBpgBqrbn2M', + null, + 'ed25519:MZT25bZDDxDJP4CRhyTK17Qfr8oWroVdyRuTccpA3Ynhf3PwB1WcC31XwugD6L5LCtMU1hyCPbE16bk7NksHhHC', + null, + 'ed25519:2BWXNtCGqwLG9yu8KkRnT1tui8GcynBN5DrhtfNQtFotSHphtABu1iNucrd82aPTegFogSMDA2cvhhrgpu4SRevc', + null, + 'ed25519:4EFt6q2MUjvcvZrCFXY2amqBcd36gKezF3FFHWsGwtJcqbH4brTHMH5THio3pXFYCWY8mtQTpLZ9SnPUWDyVKxnA', + 'ed25519:57js3GUGprV3Rc5Aci4Ws2JciuuWMhmApPPmyhYJXcz67LuwTT4KCnhFNZYDzGnV2ueP1irNUFFUXhC6HXa7amUQ', + null, + 'ed25519:2uJEgTHUV1qSLLDSjuA6NBDdJur5Xh9wKQ9sHWmpwpePyknc93J64YVZxnidG2EDAKVBSDm9kpBndMXGNEXjqEew', + 'ed25519:BzXpv857MvdsUkgCdZRAspbAhrKqibxXs7JqSt2KkzjCm9JgsxiCGf5GJCYC3gV3NJfC4p6zst8UYezQ43LAkT1', + 'ed25519:3hX4uGLHK2GRCTng7SBazenaXxSHSscJyRCKgUpKiQfHMJ7BsVvJb9obJfSNN3ynfCNmRtoUiB3tE46xHKqZ73rt', + null, + 'ed25519:2VyXyzSMCS6nzA2rDVwoZJLt2k1Zx3c4wQMx15SBx3mYbjevfdt6ErHH4GvF7PMqnXAUEgCTzWoJ54NeV5nWbM9b', + null, + 'ed25519:2FURA8ZJGN7gTtoaGgYvNkQE46rNLmrwSqqNVhzevUg3MP6GNt821vqMpsfrR5t7mFy5pk5inzrWYZcELZw5LHKz', + null, + 'ed25519:3JHQcPV1nTLsqSTboA4NavexkhmYxDsiDun49KFVEZQRuxoNHuigJL1jBW78T4nRRPPowESosCZgmpgsWDb5UDAK', + 'ed25519:2yqmma3px8io4Dd7ezg5H5d3C4Rueyi3aok9agpzpowU4vPxj2Y58h3Mu2qPSYBUtT75DgEcTtiZaMjQsBMfjCj7', + null, + 'ed25519:647bt2zPCma9kZqZmaR4HshKrERRUiFg3KT1G6S5aeowmoELvLhHcwZyAki4hW8BtgdW5GuEu2pPRXFsoTbEcASC', + null, + 'ed25519:4kv1RTxeFCRxc6shxqwfmmL398dJDGjKvFPn9PuaJJoFZTBkkFDaNuMDagoBrfgZYBhb7eZ26vKAoPK3vBVUGZgp', + 'ed25519:4P183Ceq88jLBs1cKoANgrmTBDCsvbJVRYgBH4iYyK76bYrw795DkgMaQKsWo3ZFjMHH3fC9nZ8C4WNLwkJZVfE4', + 'ed25519:CbQsnkfYhEZ32hfTrj4d4MYd4duFep6zY1cqRUFqt74WtxQHWU4mVL4qoxgvQgfeWPLiev9FRVaFLK4VnL9jYsZ', + 'ed25519:4kFsUdepZzP9oVCYWdzNq3iHURJdNn9GHj4gJgeaHPVWFpbue52zagJpmh4hsNjhDXX5tkL3TQzcd5HTWjNcqgDC', + null, + ], + blockMerkleRoot: 'FYpXxjbPZ5FoB57ZcPMvvs94x52wbCJdhxYs6qnGaZsm', + blockOrdinal: 75337992, + challengesResult: [], + challengesRoot: '11111111111111111111111111111111', + chunkHeadersRoot: '5AzFQd9Ds8yWynns4B4TFFcBGSh24ze2jApisECRYEib', + chunkMask: [true, true, true, true], + chunkReceiptsRoot: 'CsBRzd4FJe8Qr22wDkQXtxaVs5z2Jmrk2ZQC1PBrHRoV', + chunkTxRoot: '9DwCvRfUYzaxHCjWEWDR3WxJfn9gPoignTYXmEN9D6wC', + chunksIncluded: 4, + epochId: '4GoC7oSPLfaXo3Gz9z2Rkb86nN1fM8qVd578gTAmf1fn', + epochSyncDataHash: null, + gasPrice: '100000000', + hash: 'A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM', + height: 85376002, + lastDsFinalBlock: 'H5X25nVZgx43tQSMcF5bRLphWGN92UVYdvaAhcgW4no7', + lastFinalBlock: 'C3KNVyUKZ8uR166dc5vUKPQdsW5CL2mn4oerbMQFcQp5', + latestProtocolVersion: 58, + nextBpHash: '5BaNTEgDB11HxTGNKvQRJSytiv6ZM73YrJjxBymYau7', + nextEpochId: '3jvt2LuA1LZGQwKtbcThcjT4twe4juCfqwSEiuktBrqH', + outcomeRoot: '5dDKr6oN9bh1bxePyq2HbAWTA792ZRQSvfvcmTTrySi9', + prevHash: 'H5X25nVZgx43tQSMcF5bRLphWGN92UVYdvaAhcgW4no7', + prevHeight: 85376001, + prevStateRoot: '6Y3XsdVvhHj3wcGVbXNQNg3qh46HHQA5FPKUoigHPVYr', + randomValue: '3SPgojnHE7CFj7bue5w8CsYDvg5Ffskue8qPDZohqQJh', + rentPaid: '0', + signature: 'ed25519:5kjuiCWar11gt9BSRncbEaUnwoL1Nx5tzdnxrnBzgWhjHdfH2QfDajYqdy4bkXaU3UJ6TQDGcLz7pPiNLR3mmnQS', + timestamp: 1676579182274205000, + timestampNanosec: '1676579182274204901', + totalSupply: '1121201720917794037313140715777439', + validatorProposals: [], + validatorReward: '0', + }, + }, + shards: [ + { + chunk: { + author: 'figment.poolv1.near', + header: { + balanceBurnt: '1486252472505500000000', + chunkHash: '3qxSWuuo5QLdgpYSuwtQcAcAoX91b6p55CLxjda2wNNw', + encodedLength: 1772, + encodedMerkleRoot: 'ChpbnZdB2NdSaC2ir443zB4Bv6WUMSDBY6n3RfDyEVs', + gasLimit: 1000000000000000, + gasUsed: 16028665261860, + heightCreated: 85376002, + heightIncluded: 85376002, + outcomeRoot: 'G5c89nTRQSRYPdJQFcP47YXccbmbrbutNwLZzKUeRG3f', + outgoingReceiptsRoot: '5GrcVRVw9njFAskhcEbUyPZAoKMEEjGnC6edUhupEDkj', + prevBlockHash: 'H5X25nVZgx43tQSMcF5bRLphWGN92UVYdvaAhcgW4no7', + prevStateRoot: 'CRaTBLfd55cw6i5GfVvVSo1rw5ioZ91hT3ZVo5eBmNCi', + rentPaid: '0', + shardId: 0, + signature: 'ed25519:4Fzq1idZFShNtErzbK5uZWDcnWhHTMshVBvXG9c8cvc3YPkHF8SVgRsMym6V3WNHVY29gNaw1kXPCLBApcZ2bA1h', + txRoot: '5uqwhjC7wRJsvidh7gjtRRPvvUVwyefdQCivBMTL1hak', + validatorProposals: [], + validatorReward: '0', + }, + receipts: [ + { + predecessorId: 'app.nearcrowd.near', + receipt: { + Action: { + actions: [ + { + FunctionCall: { + args: 'eyJhY2NvdW50X2lkIjoicmV6ZXJ2LTI5Lm5lYXIiLCJzb2x1dGlvbl9oYXNoIjpbMTY2LDIwNCw2NSw3NCwxMTksMiwyOSwzMywyMTMsMTA1LDE2OCw0LDE5NCwxNzksMTk3LDUyLDI0MSw1NSwxLDEwMSwxNTcsNjQsMTI0LDAsMTE4LDY3LDE4NywxMjEsMTMzLDQxLDIzOSwyNDNdfQ==', + deposit: '0', + gas: 200000000000000, + methodName: 'approve_solution', + }, + }, + ], + gasPrice: '335989893', + inputDataIds: [], + outputDataReceivers: [], + signerId: 'app.nearcrowd.near', + signerPublicKey: 'ed25519:CXYkSHvK2rj6sCGbaKPbTtwhcMrPvWxs1ntVvA6vLtm2', + }, + }, + receiptId: '8FWEuJ76SphAbWL2hgsJ36zD5jobWjqPNE2xQWaEFKNY', + receiverId: 'app.nearcrowd.near', + }, + { + predecessorId: '7a8ba1bd804863883ace10ed76441e6ad2239ae7917b78c89700b701f75ce1ab', + receipt: { + Action: { + actions: [ + { + FunctionCall: { + args: 'eyJyZWNlaXZlcl9pZCI6InJld2FyZC1vcHRpbi5zd2VhdCIsImFtb3VudCI6IjEwMDAwMDAwMDAwMDAwMDAwMCIsIm1lbW8iOiJzdzpyZXc6b3B0aW46WUdSV3ZncFhLMC03YThiYTFiZDgwNDg2Mzg4M2FjZTEwZWQ3NjQ0MWU2YWQyMjM5YWU3OTE3Yjc4Yzg5NzAwYjcwMWY3NWNlMWFiIn0=', + deposit: '1', + gas: 14000000000000, + methodName: 'ft_transfer', + }, + }, + ], + gasPrice: '109272700', + inputDataIds: [], + outputDataReceivers: [], + signerId: '7a8ba1bd804863883ace10ed76441e6ad2239ae7917b78c89700b701f75ce1ab', + signerPublicKey: 'ed25519:9FNEMP92rkuvRVefVkBW38RmbZ648XvQGeWbFYtdC7hk', + }, + }, + receiptId: '7Q29UAwrFbeMMM6niPSuMrqkp4Y21FYqJ1SXPvr9oZYU', + receiverId: 'token.sweat', + }, + { + predecessorId: 'system', + receipt: { + Action: { + actions: [{ Transfer: { deposit: '67682508099918060916120' } }], + gasPrice: '0', + inputDataIds: [], + outputDataReceivers: [], + signerId: 'app.nearcrowd.near', + signerPublicKey: 'ed25519:6MP4bCPHEud33eKXM9kg7f9fVNhmn97CNUyn5ZwM375U', + }, + }, + receiptId: '5GMwBgfhgASLVXkCsqMn2riVbByJPh2pmyxCeQmbHHWe', + receiverId: 'app.nearcrowd.near', + }, + { + predecessorId: 'system', + receipt: { + Action: { + actions: [{ Transfer: { deposit: '13222602662027447938448' } }], + gasPrice: '0', + inputDataIds: [], + outputDataReceivers: [], + signerId: 'olezha4.near', + signerPublicKey: 'ed25519:EnLNUDncRuw4AYLcGmi1Hr2YQ1DLu12VutsubZ6k8c2F', + }, + }, + receiptId: 'BpZGWvbmx8sAMRsrhWyBnC5QHA9mFwist7orTceJTRvu', + receiverId: 'olezha4.near', + }, + { + predecessorId: 'system', + receipt: { + Action: { + actions: [{ Transfer: { deposit: '13239855304236542651168' } }], + gasPrice: '0', + inputDataIds: [], + outputDataReceivers: [], + signerId: 'marthacecilia.near', + signerPublicKey: 'ed25519:8W28RYFfXztNA7ENP4VTaitsqEjSbABQJDqELwbwXswq', + }, + }, + receiptId: '3R2wjnktLwBKaceGiPetmShVAyHT2LuoQZMdNg6oWMwK', + receiverId: 'marthacecilia.near', + }, + ], + transactions: [ + { + outcome: { + executionOutcome: { + blockHash: 'A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM', + id: '3W8KnZUsb7Mb3SZk7yygL5F2L2C6djxZ3fVumodB2KQe', + outcome: { + executorId: 'app.nearcrowd.near', + gasBurnt: 2428314524384, + logs: [], + metadata: { gasProfile: null, version: 1 }, + receiptIds: ['8FWEuJ76SphAbWL2hgsJ36zD5jobWjqPNE2xQWaEFKNY'], + status: { SuccessReceiptId: '8FWEuJ76SphAbWL2hgsJ36zD5jobWjqPNE2xQWaEFKNY' }, + tokensBurnt: '242831452438400000000', + }, + proof: [ + { direction: 'Right', hash: '8AS32PCfmnnsgkmrm2QH6ULKdTBS4o3EjcU2FZ34dTux' }, + { direction: 'Right', hash: '8PVbKv5bqbBoTUkXbcQ8sWEYbUDKo8hv8S1JHKVrftCz' }, + { direction: 'Right', hash: '8VRHTUKsUcZRDgrKizV5sup3Zvzg6reRacn3KmeUVY9n' }, + ], + }, + receipt: null, + }, + transaction: { + actions: [ + { + FunctionCall: { + args: 'eyJhY2NvdW50X2lkIjoicmV6ZXJ2LTI5Lm5lYXIiLCJzb2x1dGlvbl9oYXNoIjpbMTY2LDIwNCw2NSw3NCwxMTksMiwyOSwzMywyMTMsMTA1LDE2OCw0LDE5NCwxNzksMTk3LDUyLDI0MSw1NSwxLDEwMSwxNTcsNjQsMTI0LDAsMTE4LDY3LDE4NywxMjEsMTMzLDQxLDIzOSwyNDNdfQ==', + deposit: '0', + gas: 200000000000000, + methodName: 'approve_solution', + }, + }, + ], + hash: '3W8KnZUsb7Mb3SZk7yygL5F2L2C6djxZ3fVumodB2KQe', + nonce: 43616777771952, + publicKey: 'ed25519:CXYkSHvK2rj6sCGbaKPbTtwhcMrPvWxs1ntVvA6vLtm2', + receiverId: 'app.nearcrowd.near', + signature: + 'ed25519:5KcVvC1zzJZkk92uWzMoD9e2fvded6JdgUApuV6NDCcY7YEhzVtUQaVPomQ4ZpDydtUyZmgxTXcqaVp5c54wciEX', + signerId: 'app.nearcrowd.near', + }, + }, + { + outcome: { + executionOutcome: { + blockHash: 'A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM', + id: '3GYUTXHUQMH1kpccEms6yNVzckyAedUTWKPQpEN4tc5K', + outcome: { + executorId: '2fe75000c84efd3b64f88f61d88cd08c339b8c6654e6239f7d4e3592aa92c659', + gasBurnt: 2428312288450, + logs: [], + metadata: { gasProfile: null, version: 1 }, + receiptIds: ['7CmRnTXnY3LmX7PojrByXqZpirVNVSuHgL9k3uhbdKm1'], + status: { SuccessReceiptId: '7CmRnTXnY3LmX7PojrByXqZpirVNVSuHgL9k3uhbdKm1' }, + tokensBurnt: '242831228845000000000', + }, + proof: [ + { direction: 'Left', hash: '5Bf7Pbbnwt6THJBxQ3mBkpc8WD2GBLoXrte6HYJZU6yh' }, + { direction: 'Right', hash: '8PVbKv5bqbBoTUkXbcQ8sWEYbUDKo8hv8S1JHKVrftCz' }, + { direction: 'Right', hash: '8VRHTUKsUcZRDgrKizV5sup3Zvzg6reRacn3KmeUVY9n' }, + ], + }, + receipt: null, + }, + transaction: { + actions: [ + { + FunctionCall: { + args: 'eyJyZWNlaXZlcl9pZCI6InJld2FyZC1vcHRpbi5zd2VhdCIsImFtb3VudCI6IjEwMDAwMDAwMDAwMDAwMDAwMCIsIm1lbW8iOiJzdzpyZXc6b3B0aW46WUdSV3ZncFhLMC0yZmU3NTAwMGM4NGVmZDNiNjRmODhmNjFkODhjZDA4YzMzOWI4YzY2NTRlNjIzOWY3ZDRlMzU5MmFhOTJjNjU5In0=', + deposit: '1', + gas: 14000000000000, + methodName: 'ft_transfer', + }, + }, + ], + hash: '3GYUTXHUQMH1kpccEms6yNVzckyAedUTWKPQpEN4tc5K', + nonce: 70681752000024, + publicKey: 'ed25519:4Dzjg5EJX1RBYctnzmi1SNoq6fswSowsiWB5m3sEcC4Q', + receiverId: 'token.sweat', + signature: + 'ed25519:5NuszWGgsVB5yPWqvGMGP3yCjs5YVh3QtWw79kch5mf3ApSVqpzTD42TzAsKcmhauDYDhzU8SUU9dGx2rHodJRLn', + signerId: '2fe75000c84efd3b64f88f61d88cd08c339b8c6654e6239f7d4e3592aa92c659', + }, + }, + ], + }, + receiptExecutionOutcomes: [ + { + executionOutcome: { + blockHash: 'A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM', + id: '8FWEuJ76SphAbWL2hgsJ36zD5jobWjqPNE2xQWaEFKNY', + outcome: { + executorId: 'app.nearcrowd.near', + gasBurnt: 3300309921959, + logs: [], + metadata: { + gasProfile: [ + { cost: 'BASE', costCategory: 'WASM_HOST_COST', gasUsed: '3177217332' }, + { cost: 'CONTRACT_LOADING_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '35445963' }, + { cost: 'CONTRACT_LOADING_BYTES', costCategory: 'WASM_HOST_COST', gasUsed: '104961404250' }, + { cost: 'READ_CACHED_TRIE_NODE', costCategory: 'WASM_HOST_COST', gasUsed: '86640000000' }, + { cost: 'READ_MEMORY_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '13049316000' }, + { cost: 'READ_MEMORY_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '315510639' }, + { cost: 'READ_REGISTER_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '7551495558' }, + { cost: 'READ_REGISTER_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '19712400' }, + { cost: 'STORAGE_READ_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '56356845750' }, + { cost: 'STORAGE_READ_KEY_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '154762665' }, + { cost: 'STORAGE_READ_VALUE_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '123442110' }, + { cost: 'STORAGE_WRITE_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '128393472000' }, + { cost: 'STORAGE_WRITE_EVICTED_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '706580754' }, + { cost: 'STORAGE_WRITE_KEY_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '1691588808' }, + { cost: 'STORAGE_WRITE_VALUE_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '1675001106' }, + { cost: 'TOUCHING_TRIE_NODE', costCategory: 'WASM_HOST_COST', gasUsed: '418650854076' }, + { cost: 'WASM_INSTRUCTION', costCategory: 'WASM_HOST_COST', gasUsed: '24383196816' }, + { cost: 'WRITE_MEMORY_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '11215179444' }, + { cost: 'WRITE_MEMORY_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '588334752' }, + { cost: 'WRITE_REGISTER_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '11462089944' }, + { cost: 'WRITE_REGISTER_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '843947208' }, + ], + version: 1, + }, + receiptIds: ['8kGPN9gvU8dhZy3374rTa3LjtxcZcHcZkXRAf56si2vh'], + status: { SuccessValue: '' }, + tokensBurnt: '330030992195900000000', + }, + proof: [ + { direction: 'Right', hash: '8uFcqYe8azbspu5mofVCt5TC6K4n6kTbNBRwDt3CRT6Z' }, + { direction: 'Left', hash: '9C9aaY5ECUzq1t2NtXQVXTH6wZ8sBExs33KhVNmjFnQJ' }, + { direction: 'Right', hash: '8VRHTUKsUcZRDgrKizV5sup3Zvzg6reRacn3KmeUVY9n' }, + ], + }, + receipt: { + predecessorId: 'app.nearcrowd.near', + receipt: { + Action: { + actions: [ + { + FunctionCall: { + args: 'eyJhY2NvdW50X2lkIjoicmV6ZXJ2LTI5Lm5lYXIiLCJzb2x1dGlvbl9oYXNoIjpbMTY2LDIwNCw2NSw3NCwxMTksMiwyOSwzMywyMTMsMTA1LDE2OCw0LDE5NCwxNzksMTk3LDUyLDI0MSw1NSwxLDEwMSwxNTcsNjQsMTI0LDAsMTE4LDY3LDE4NywxMjEsMTMzLDQxLDIzOSwyNDNdfQ==', + deposit: '0', + gas: 200000000000000, + methodName: 'approve_solution', + }, + }, + ], + gasPrice: '335989893', + inputDataIds: [], + outputDataReceivers: [], + signerId: 'app.nearcrowd.near', + signerPublicKey: 'ed25519:CXYkSHvK2rj6sCGbaKPbTtwhcMrPvWxs1ntVvA6vLtm2', + }, + }, + receiptId: '8FWEuJ76SphAbWL2hgsJ36zD5jobWjqPNE2xQWaEFKNY', + receiverId: 'app.nearcrowd.near', + }, + }, + { + executionOutcome: { + blockHash: 'A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM', + id: '6XN47idjeWhq2QpkjGs9w7pPg452Z1GwgwRxZjy2ny7T', + outcome: { + executorId: 'app.nearcrowd.near', + gasBurnt: 5360036359518, + logs: [], + metadata: { + gasProfile: [ + { cost: 'BASE', costCategory: 'WASM_HOST_COST', gasUsed: '15091782327' }, + { cost: 'CONTRACT_LOADING_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '35445963' }, + { cost: 'CONTRACT_LOADING_BYTES', costCategory: 'WASM_HOST_COST', gasUsed: '104961404250' }, + { cost: 'READ_CACHED_TRIE_NODE', costCategory: 'WASM_HOST_COST', gasUsed: '592800000000' }, + { cost: 'READ_MEMORY_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '62636716800' }, + { cost: 'READ_MEMORY_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '2698946430' }, + { cost: 'READ_REGISTER_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '42791808162' }, + { cost: 'READ_REGISTER_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '125075178' }, + { cost: 'STORAGE_HAS_KEY_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '54039896625' }, + { cost: 'STORAGE_HAS_KEY_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '738980280' }, + { cost: 'STORAGE_READ_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '281784228750' }, + { cost: 'STORAGE_READ_KEY_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '4859547681' }, + { cost: 'STORAGE_READ_VALUE_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '4253141790' }, + { cost: 'STORAGE_REMOVE_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '160419091500' }, + { cost: 'STORAGE_REMOVE_KEY_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '1605256128' }, + { cost: 'STORAGE_REMOVE_RET_VALUE_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '1141624044' }, + { cost: 'STORAGE_WRITE_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '449377152000' }, + { cost: 'STORAGE_WRITE_EVICTED_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '9121315188' }, + { cost: 'STORAGE_WRITE_KEY_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '6977803833' }, + { cost: 'STORAGE_WRITE_VALUE_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '11880100437' }, + { cost: 'TOUCHING_TRIE_NODE', costCategory: 'WASM_HOST_COST', gasUsed: '933913443708' }, + { cost: 'WASM_INSTRUCTION', costCategory: 'WASM_HOST_COST', gasUsed: '83208605304' }, + { cost: 'WRITE_MEMORY_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '50468307498' }, + { cost: 'WRITE_MEMORY_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '3500047020' }, + { cost: 'WRITE_REGISTER_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '48713882262' }, + { cost: 'WRITE_REGISTER_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '4824184716' }, + ], + version: 1, + }, + receiptIds: ['HF5rs8fq4n8HygFMYiQ8nWZSpVncKBnCXj9aNpvzZsAP'], + status: { SuccessValue: 'ZmFsc2U=' }, + tokensBurnt: '536003635951800000000', + }, + proof: [ + { direction: 'Left', hash: '2Lon5RSqBvnuGcXgkneYZTUUup6CotQYqEbi1eYfBfVK' }, + { direction: 'Left', hash: '9C9aaY5ECUzq1t2NtXQVXTH6wZ8sBExs33KhVNmjFnQJ' }, + { direction: 'Right', hash: '8VRHTUKsUcZRDgrKizV5sup3Zvzg6reRacn3KmeUVY9n' }, + ], + }, + receipt: { + predecessorId: 'evaenergy.near', + receipt: { + Action: { + actions: [ + { + FunctionCall: { + args: 'eyJ0YXNrX29yZGluYWwiOjEsImJpZCI6IjQ0NDQ2ODcxMTE2MjIyODMzODczMDc2In0=', + deposit: '0', + gas: 30000000000000, + methodName: 'claim_assignment', + }, + }, + ], + gasPrice: '122987387', + inputDataIds: [], + outputDataReceivers: [], + signerId: 'evaenergy.near', + signerPublicKey: 'ed25519:9Q53x5kiLChb4tX3J5dV2wNAdTetejEcnizyQ3n3PZdg', + }, + }, + receiptId: '6XN47idjeWhq2QpkjGs9w7pPg452Z1GwgwRxZjy2ny7T', + receiverId: 'app.nearcrowd.near', + }, + }, + { + executionOutcome: { + blockHash: 'A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM', + id: '5GMwBgfhgASLVXkCsqMn2riVbByJPh2pmyxCeQmbHHWe', + outcome: { + executorId: 'app.nearcrowd.near', + gasBurnt: 223182562500, + logs: [], + metadata: { gasProfile: [], version: 1 }, + receiptIds: [], + status: { SuccessValue: '' }, + tokensBurnt: '0', + }, + proof: [{ direction: 'Left', hash: 'BcJG6WkrSgW8dsc8iBJHNZcP7576vvRQGBagZZfuXkmG' }], + }, + receipt: { + predecessorId: 'system', + receipt: { + Action: { + actions: [{ Transfer: { deposit: '67682508099918060916120' } }], + gasPrice: '0', + inputDataIds: [], + outputDataReceivers: [], + signerId: 'app.nearcrowd.near', + signerPublicKey: 'ed25519:6MP4bCPHEud33eKXM9kg7f9fVNhmn97CNUyn5ZwM375U', + }, + }, + receiptId: '5GMwBgfhgASLVXkCsqMn2riVbByJPh2pmyxCeQmbHHWe', + receiverId: 'app.nearcrowd.near', + }, + }, + ], + shardId: 0, + stateChanges: [ + { + cause: { txHash: '3GYUTXHUQMH1kpccEms6yNVzckyAedUTWKPQpEN4tc5K', type: 'transaction_processing' }, + change: { + accountId: '2fe75000c84efd3b64f88f61d88cd08c339b8c6654e6239f7d4e3592aa92c659', + amount: '31778755366428489684978', + codeHash: '11111111111111111111111111111111', + locked: '0', + storagePaidAt: 0, + storageUsage: 182, + }, + type: 'account_update', + }, + { + cause: { txHash: '3W8KnZUsb7Mb3SZk7yygL5F2L2C6djxZ3fVumodB2KQe', type: 'transaction_processing' }, + change: { + accountId: 'app.nearcrowd.near', + amount: '6559406007009136306129113176', + codeHash: 'DyHG2t99dBZWiPgX53Z4UbbBQR6JJoxmqXwaKD4hTeyP', + locked: '0', + storagePaidAt: 0, + storageUsage: 3976751, + }, + type: 'account_update', + }, + { + cause: { receiptHash: '8FWEuJ76SphAbWL2hgsJ36zD5jobWjqPNE2xQWaEFKNY', type: 'receipt_processing' }, + change: { + accountId: 'app.nearcrowd.near', + amount: '6559406007009136306129113176', + codeHash: 'DyHG2t99dBZWiPgX53Z4UbbBQR6JJoxmqXwaKD4hTeyP', + locked: '0', + storagePaidAt: 0, + storageUsage: 3976842, + }, + type: 'account_update', + }, + { + cause: { receiptHash: '8FWEuJ76SphAbWL2hgsJ36zD5jobWjqPNE2xQWaEFKNY', type: 'action_receipt_gas_reward' }, + change: { + accountId: 'app.nearcrowd.near', + amount: '6559406033168998233329113176', + codeHash: 'DyHG2t99dBZWiPgX53Z4UbbBQR6JJoxmqXwaKD4hTeyP', + locked: '0', + storagePaidAt: 0, + storageUsage: 3976842, + }, + type: 'account_update', + }, + { + cause: { receiptHash: '6XN47idjeWhq2QpkjGs9w7pPg452Z1GwgwRxZjy2ny7T', type: 'receipt_processing' }, + change: { + accountId: 'app.nearcrowd.near', + amount: '6559406033168998233329113176', + codeHash: 'DyHG2t99dBZWiPgX53Z4UbbBQR6JJoxmqXwaKD4hTeyP', + locked: '0', + storagePaidAt: 0, + storageUsage: 3976788, + }, + type: 'account_update', + }, + { + cause: { receiptHash: '6XN47idjeWhq2QpkjGs9w7pPg452Z1GwgwRxZjy2ny7T', type: 'action_receipt_gas_reward' }, + change: { + accountId: 'app.nearcrowd.near', + amount: '6559406121128031869529113176', + codeHash: 'DyHG2t99dBZWiPgX53Z4UbbBQR6JJoxmqXwaKD4hTeyP', + locked: '0', + storagePaidAt: 0, + storageUsage: 3976788, + }, + type: 'account_update', + }, + { + cause: { receiptHash: '5GMwBgfhgASLVXkCsqMn2riVbByJPh2pmyxCeQmbHHWe', type: 'receipt_processing' }, + change: { + accountId: 'app.nearcrowd.near', + amount: '6559473803636131787590029296', + codeHash: 'DyHG2t99dBZWiPgX53Z4UbbBQR6JJoxmqXwaKD4hTeyP', + locked: '0', + storagePaidAt: 0, + storageUsage: 3976788, + }, + type: 'account_update', + }, + { + cause: { txHash: '3GYUTXHUQMH1kpccEms6yNVzckyAedUTWKPQpEN4tc5K', type: 'transaction_processing' }, + change: { + accessKey: { nonce: 70681752000024, permission: 'FullAccess' }, + accountId: '2fe75000c84efd3b64f88f61d88cd08c339b8c6654e6239f7d4e3592aa92c659', + publicKey: 'ed25519:4Dzjg5EJX1RBYctnzmi1SNoq6fswSowsiWB5m3sEcC4Q', + }, + type: 'access_key_update', + }, + { + cause: { txHash: '3W8KnZUsb7Mb3SZk7yygL5F2L2C6djxZ3fVumodB2KQe', type: 'transaction_processing' }, + change: { + accessKey: { nonce: 43616777771952, permission: 'FullAccess' }, + accountId: 'app.nearcrowd.near', + publicKey: 'ed25519:CXYkSHvK2rj6sCGbaKPbTtwhcMrPvWxs1ntVvA6vLtm2', + }, + type: 'access_key_update', + }, + { + cause: { receiptHash: '8FWEuJ76SphAbWL2hgsJ36zD5jobWjqPNE2xQWaEFKNY', type: 'receipt_processing' }, + change: { + accountId: 'app.nearcrowd.near', + keyBase64: 'U1RBVEU=', + valueBase64: 'EgAAAGFwcC5uZWFyY3Jvd2QubmVhcg==', + }, + type: 'data_update', + }, + { + cause: { receiptHash: '8FWEuJ76SphAbWL2hgsJ36zD5jobWjqPNE2xQWaEFKNY', type: 'receipt_processing' }, + change: { + accountId: 'app.nearcrowd.near', + keyBase64: 'YQ4AAAByZXplcnYtMjkubmVhcg==', + valueBase64: 'psxBSncCHSHVaagEwrPFNPE3AWWdQHwAdkO7eYUp7/M=', + }, + type: 'data_update', + }, + { + cause: { receiptHash: '6XN47idjeWhq2QpkjGs9w7pPg452Z1GwgwRxZjy2ny7T', type: 'receipt_processing' }, + change: { + accountId: 'app.nearcrowd.near', + keyBase64: 'cAEAAAA=', + valueBase64: + 'AABAe6XwY4GWCgAAAAAAAAAA9ESCkWNFAAAAAAAAAAAsAQAAAAAAABACoJtTF3zmCgkAAAAAAADcEUiC7mdEF6E7rAAAAAAABgAAAHQBAAAAYrtzrAAAAAAABgAAAHQBAAAAYh0EAAAAAAAABgAAAHQBAAAAYwYAAAB0AQAAAGUGAAAAdAEAAABmBgAAAHQBAAAAZwEGAAAAdAEAAABtAA==', + }, + type: 'data_update', + }, + { + cause: { receiptHash: '6XN47idjeWhq2QpkjGs9w7pPg452Z1GwgwRxZjy2ny7T', type: 'receipt_processing' }, + change: { accountId: 'app.nearcrowd.near', keyBase64: 'dAEAAABjHQQAAAAAAAA=' }, + type: 'data_deletion', + }, + { + cause: { receiptHash: '6XN47idjeWhq2QpkjGs9w7pPg452Z1GwgwRxZjy2ny7T', type: 'receipt_processing' }, + change: { + accountId: 'app.nearcrowd.near', + keyBase64: 'dAEAAABjIgEAAAAAAAA=', + valueBase64: 'epIvaamXn1S0leE2SA8w+TxNWM4rEC75i6V3XS7gjvAE', + }, + type: 'data_update', + }, + { + cause: { receiptHash: '6XN47idjeWhq2QpkjGs9w7pPg452Z1GwgwRxZjy2ny7T', type: 'receipt_processing' }, + change: { + accountId: 'app.nearcrowd.near', + keyBase64: 'dAEAAABjmwAAAAAAAAA=', + valueBase64: '0h0vJmTB/V9IK0xVxNBid+JAmsnySRFNF3jnQNl2m8sB', + }, + type: 'data_update', + }, + { + cause: { receiptHash: '6XN47idjeWhq2QpkjGs9w7pPg452Z1GwgwRxZjy2ny7T', type: 'receipt_processing' }, + change: { + accountId: 'app.nearcrowd.near', + keyBase64: 'dAEAAABjqQEAAAAAAAA=', + valueBase64: 's+OZU4d49W6Qv/gFmkBPsmVUX1WVbDiJAJzJse0tXVEE', + }, + type: 'data_update', + }, + { + cause: { receiptHash: '6XN47idjeWhq2QpkjGs9w7pPg452Z1GwgwRxZjy2ny7T', type: 'receipt_processing' }, + change: { + accountId: 'app.nearcrowd.near', + keyBase64: 'dAEAAABnDgAAAGV2YWVuZXJneS5uZWFy', + valueBase64: 'A/UhKZel+/gCMLF4XhBcxERDBIa48p9qlYBMRIbWALUgA+XURCXiZ0QXWujnX0t/H7y0BAAAAAAAAA==', + }, + type: 'data_update', + }, + ], + }, + { + chunk: { + author: 'bisontrails.poolv1.near', + header: { + balanceBurnt: '1433581134734200000000', + chunkHash: 'BqfFW2NyLJuyp1pGgXd2ffYPGptNswNVH6ZJuTq5zSNa', + encodedLength: 161, + encodedMerkleRoot: 'EB5tG2a3aVfXv41ESuQVT1ZvQFvuwKnkYGLc2CDVCf5w', + gasLimit: 1000000000000000, + gasUsed: 19438956235189, + heightCreated: 85376002, + heightIncluded: 85376002, + outcomeRoot: 'Cp7GeqLAqZ1Z1GbtCL5nG13wxy6mV6PaRdwLddDHjhVc', + outgoingReceiptsRoot: 'DP1DT6QP1tNHM5ZjHMJ3C8G2mHJd8kGoFFNEdZPN8Hyx', + prevBlockHash: 'H5X25nVZgx43tQSMcF5bRLphWGN92UVYdvaAhcgW4no7', + prevStateRoot: '73sn3JyP1LU58r1FsTo5ZVgvErdbEGFxtiGExkWwhQUd', + rentPaid: '0', + shardId: 1, + signature: 'ed25519:3RYj6fpboT1ALHsVxYtYmofT8fGf1VRhNaJhWY45hsKdePcX3AZb3rdKBAZMThF5PVn5j5boKWHPb9zd1xZWYzex', + txRoot: '11111111111111111111111111111111', + validatorProposals: [], + validatorReward: '0', + }, + receipts: [ + { + predecessorId: 'system', + receipt: { + Action: { + actions: [{ Transfer: { deposit: '187086049935277727891452' } }], + gasPrice: '0', + inputDataIds: [], + outputDataReceivers: [], + signerId: 'relay.aurora', + signerPublicKey: 'ed25519:8j2e8u9t1SoTgMPn5G65ChU5WskpcnemBjwzkAnxDtZq', + }, + }, + receiptId: 'CdUSBwV29D1P1qvxCwnbfNsTsdxfdRY6PxezsSWRFyqN', + receiverId: 'relay.aurora', + }, + ], + transactions: [], + }, + receiptExecutionOutcomes: [ + { + executionOutcome: { + blockHash: 'A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM', + id: '821S5L8Htxafh2DDuDKtA4qapudN3mRq2AMv2qPeBhpj', + outcome: { + executorId: 'aurora', + gasBurnt: 3497076222351, + logs: ['signer_address Address(0x605ce19532c7121d1b5e632eb776e5aa7bfb8990)'], + metadata: { + gasProfile: [ + { cost: 'BASE', costCategory: 'WASM_HOST_COST', gasUsed: '5295362220' }, + { cost: 'CONTRACT_LOADING_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '35445963' }, + { cost: 'CONTRACT_LOADING_BYTES', costCategory: 'WASM_HOST_COST', gasUsed: '207231423750' }, + { cost: 'ECRECOVER_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '278821988457' }, + { cost: 'KECCAK256_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '17638473825' }, + { cost: 'KECCAK256_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '8889037470' }, + { cost: 'LOG_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '3543313050' }, + { cost: 'LOG_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '871120206' }, + { cost: 'READ_CACHED_TRIE_NODE', costCategory: 'WASM_HOST_COST', gasUsed: '9120000000' }, + { cost: 'READ_MEMORY_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '20878905600' }, + { cost: 'READ_MEMORY_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '2128746480' }, + { cost: 'READ_REGISTER_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '22654486674' }, + { cost: 'READ_REGISTER_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '65642292' }, + { cost: 'STORAGE_READ_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '112713691500' }, + { cost: 'STORAGE_READ_KEY_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '897623457' }, + { cost: 'STORAGE_READ_VALUE_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '482546430' }, + { cost: 'TOUCHING_TRIE_NODE', costCategory: 'WASM_HOST_COST', gasUsed: '209325427038' }, + { cost: 'UTF8_DECODING_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '6223558122' }, + { cost: 'UTF8_DECODING_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '24784340715' }, + { cost: 'WASM_INSTRUCTION', costCategory: 'WASM_HOST_COST', gasUsed: '84250214400' }, + { cost: 'WRITE_MEMORY_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '22430358888' }, + { cost: 'WRITE_MEMORY_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '1639710744' }, + { cost: 'WRITE_REGISTER_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '25789702374' }, + { cost: 'WRITE_REGISTER_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '2531841624' }, + ], + version: 1, + }, + receiptIds: ['H4FHCm5scAbLHhki995o2UoJsHYuJHfbbCx5egAjNmk9'], + status: { + Failure: { + ActionError: { + index: 0, + kind: { FunctionCallError: { ExecutionError: 'Smart contract panicked: ERR_INCORRECT_NONCE' } }, + }, + }, + }, + tokensBurnt: '349707622235100000000', + }, + proof: [], + }, + receipt: { + predecessorId: 'relay.aurora', + receipt: { + Action: { + actions: [ + { + FunctionCall: { + args: '+QGPggpehAQsHYCCUgiU5ShUyG+2SwTdHXetVCh23cBAxPSAuQEkGMuv5QAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADGiUDx20tY/zAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAR/fZpFXwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAoAAAAAAAAAAAAAAAAGBc4ZUyxxIdG15jLrd25ap7+4mQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABhaeq/ToAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAADCXWsrHC113TW91as0DqbkM1NT1AAAAAAAAAAAAAAAASYiolrEichjkpob95eq9yr2RVx8AAAAAAAAAAAAAAADJve7TPNAVQeHu0Q+QUZ0sBv4/64ScioLIoNNwZYHzq18jJVWwzwd9vJY1JV4H29H/wOgwyQVM2FJzoCUAB6Nz+98pSh9AfalXBMbThpftEUVyF49fYhEKwmJf', + deposit: '0', + gas: 300000000000000, + methodName: 'submit', + }, + }, + ], + gasPrice: '625040174', + inputDataIds: [], + outputDataReceivers: [], + signerId: 'relay.aurora', + signerPublicKey: 'ed25519:2WmTnJeWZYKhh6NC3z3xTDKU5sHHAmHF96V6KAB3eHK5', + }, + }, + receiptId: '821S5L8Htxafh2DDuDKtA4qapudN3mRq2AMv2qPeBhpj', + receiverId: 'aurora', + }, + }, + ], + shardId: 1, + stateChanges: [ + { + cause: { receiptHash: '821S5L8Htxafh2DDuDKtA4qapudN3mRq2AMv2qPeBhpj', type: 'action_receipt_gas_reward' }, + change: { + accountId: 'aurora', + amount: '62420509105129651830808079298', + codeHash: 'qorYWFPQKMbJGcmjtWUhD3ee7fJJakRRUYFk3cao4W3', + locked: '0', + storagePaidAt: 0, + storageUsage: 6176907778, + }, + type: 'account_update', + }, + ], + }, + { + chunk: { + author: 'republic.poolv1.near', + header: { + balanceBurnt: '242806857164400000000', + chunkHash: '6uDUqFuYwGohFMQ9UyRaHbmHALf8HS9F6WcNnZWypkka', + encodedLength: 1474, + encodedMerkleRoot: '9rtC25N5jqfLxMumUwvupL9MpMNUAEKuXZB8RmfkdD7p', + gasLimit: 1000000000000000, + gasUsed: 2428068571644, + heightCreated: 85376002, + heightIncluded: 85376002, + outcomeRoot: 'AvRw1DBT1JFB2KNuahMvYucvttrem74GSEnwMacxbkv6', + outgoingReceiptsRoot: 'C2QjTGsH8HcfR6W7ecLYwuYD3Vkw6mi9Pf9tFXuGdMLr', + prevBlockHash: 'H5X25nVZgx43tQSMcF5bRLphWGN92UVYdvaAhcgW4no7', + prevStateRoot: 'CtxzDGCGs1SAr7GEqrA8gSKCcbaPDMKegnTyPEJmU7mb', + rentPaid: '0', + shardId: 2, + signature: 'ed25519:66LFkWB13BEUYRsmockx9LbiBzko9zohGkfbLwJertYWAa5TJY5ELFxPycps6poroh1C7Tfbh6NH7tQpgHZFr43d', + txRoot: 'DVr9U4uYvJY1cz5awdSmKm4CovJmr1VcPyqCMonat8bD', + validatorProposals: [], + validatorReward: '0', + }, + receipts: [ + { + predecessorId: 'evaenergy.near', + receipt: { + Action: { + actions: [ + { + FunctionCall: { + args: 'eyJ0YXNrX29yZGluYWwiOjEsImJpZCI6IjQ0NDQ2ODcxMTE2MjIyODMzODczMDc2In0=', + deposit: '0', + gas: 30000000000000, + methodName: 'claim_assignment', + }, + }, + ], + gasPrice: '122987387', + inputDataIds: [], + outputDataReceivers: [], + signerId: 'evaenergy.near', + signerPublicKey: 'ed25519:9Q53x5kiLChb4tX3J5dV2wNAdTetejEcnizyQ3n3PZdg', + }, + }, + receiptId: '6XN47idjeWhq2QpkjGs9w7pPg452Z1GwgwRxZjy2ny7T', + receiverId: 'app.nearcrowd.near', + }, + ], + transactions: [ + { + outcome: { + executionOutcome: { + blockHash: 'A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM', + id: 'F4z3CFPjUGRJEF9rNSVhz4hZG2LpBvA3rR3AHv68432i', + outcome: { + executorId: 'chadmoney.near', + gasBurnt: 2428372658668, + logs: [], + metadata: { gasProfile: null, version: 1 }, + receiptIds: ['Ht6wAxzHfyCwWmfkngi9n6DaJu6UtoD1skHpJrRJQGik'], + status: { SuccessReceiptId: 'Ht6wAxzHfyCwWmfkngi9n6DaJu6UtoD1skHpJrRJQGik' }, + tokensBurnt: '242837265866800000000', + }, + proof: [ + { direction: 'Right', hash: 'E4X99JiTuYjYW6WPMiBhZ61Eo9fDYXmDdv67ZdkSbcf3' }, + { direction: 'Right', hash: '2NXhS3PRa1H8UZEtrNo14QhYNTzk5ztBhrSe6rb64heg' }, + ], + }, + receipt: null, + }, + transaction: { + actions: [ + { + FunctionCall: { + args: 'eyJhY3Rpb25zIjpbeyJwb29sX2lkIjo0LCJ0b2tlbl9pbiI6IndyYXAubmVhciIsInRva2VuX291dCI6ImRhYzE3Zjk1OGQyZWU1MjNhMjIwNjIwNjk5NDU5N2MxM2Q4MzFlYzcuZmFjdG9yeS5icmlkZ2UubmVhciIsImFtb3VudF9pbiI6IjEwMDAwMDAwMDAwMDAwMDAwMDAwMDAwMDAwMDAiLCJtaW5fYW1vdW50X291dCI6IjI0NzE5ODgxNjIifV19', + deposit: '1', + gas: 10000000000000, + methodName: 'swap', + }, + }, + ], + hash: 'F4z3CFPjUGRJEF9rNSVhz4hZG2LpBvA3rR3AHv68432i', + nonce: 72650480016948, + publicKey: 'ed25519:78fzZoSWw4yr7gRev6onwRFQPu5auDAPNuAhM2Ryy4EC', + receiverId: 'v2.ref-finance.near', + signature: + 'ed25519:SYNT4UNqfEpAofZykZpZUugyPxXpSxYPwo8BqVg1fPG8BZvQKJgmHkoLsEbG2bsTfW1iGr886E6mw4nRDLJQ9e1', + signerId: 'chadmoney.near', + }, + }, + { + outcome: { + executionOutcome: { + blockHash: 'A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM', + id: '3Zgj4eFNymhW7LBsaKtUsCWAaoP9LBXKGFtmebYsfibU', + outcome: { + executorId: 'fd305d28fa12cb2e2769b211248059b8f4033ef4d4c0072c7daf246b166106ba', + gasBurnt: 2428312288450, + logs: [], + metadata: { gasProfile: null, version: 1 }, + receiptIds: ['ENQk2JGnPNtpXPUFkpBhnD3XUyv67eXB4XfmDU2sjHB7'], + status: { SuccessReceiptId: 'ENQk2JGnPNtpXPUFkpBhnD3XUyv67eXB4XfmDU2sjHB7' }, + tokensBurnt: '242831228845000000000', + }, + proof: [ + { direction: 'Left', hash: 'GAdP76ExKf5yQH9BU5usAeAWxiX2HkmDP7g54Ff47Zgz' }, + { direction: 'Right', hash: '2NXhS3PRa1H8UZEtrNo14QhYNTzk5ztBhrSe6rb64heg' }, + ], + }, + receipt: null, + }, + transaction: { + actions: [ + { + FunctionCall: { + args: 'eyJyZWNlaXZlcl9pZCI6InJld2FyZC1vcHRpbi5zd2VhdCIsImFtb3VudCI6IjEwMDAwMDAwMDAwMDAwMDAwMCIsIm1lbW8iOiJzdzpyZXc6b3B0aW46WUdSV3ZncFhLMC1mZDMwNWQyOGZhMTJjYjJlMjc2OWIyMTEyNDgwNTliOGY0MDMzZWY0ZDRjMDA3MmM3ZGFmMjQ2YjE2NjEwNmJhIn0=', + deposit: '1', + gas: 14000000000000, + methodName: 'ft_transfer', + }, + }, + ], + hash: '3Zgj4eFNymhW7LBsaKtUsCWAaoP9LBXKGFtmebYsfibU', + nonce: 64674524000017, + publicKey: 'ed25519:J3LvQdaf6r6YVuTrVzo6em32R2ZG7jE9n4ysiLUpMa4y', + receiverId: 'token.sweat', + signature: + 'ed25519:3gdWeCTcqbT3ezViXEPaRiRafz4NcRRsP7bAwgY1CTFenKcFQB5gyvQqSHPAtQbx7JdppByH4j6SrEMxmPedFyPY', + signerId: 'fd305d28fa12cb2e2769b211248059b8f4033ef4d4c0072c7daf246b166106ba', + }, + }, + { + outcome: { + executionOutcome: { + blockHash: 'A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM', + id: 'DbCXaFDNfQ7LjM5Jh8BS5fKLF6UJG3vJZYbGtfpDaTEB', + outcome: { + executorId: 'de9d1abf39de5fc4eac513b559cc825352d03c7b642a8af657113d61f8c195dc', + gasBurnt: 2428158009004, + logs: [], + metadata: { gasProfile: null, version: 1 }, + receiptIds: ['86Xs7Z6wiQNb89hV5j33UHbBZTXrbBYtqwBMBT3Hhq4h'], + status: { SuccessReceiptId: '86Xs7Z6wiQNb89hV5j33UHbBZTXrbBYtqwBMBT3Hhq4h' }, + tokensBurnt: '242815800900400000000', + }, + proof: [ + { direction: 'Right', hash: 'AJhsQtiU5riKpBKSBbHYD9svR84eZ3oYqd8eRjipqTFZ' }, + { direction: 'Left', hash: '45QTFnLgDmAbiiVBFmVHnivqQFU2ATNgLNErLnvZYZq2' }, + ], + }, + receipt: null, + }, + transaction: { + actions: [ + { + FunctionCall: { + args: 'eyJyZWNlaXZlcl9pZCI6ImRlcG9zaXRzLmdyb3cuc3dlYXQiLCJhbW91bnQiOiIxMDIxOTAwMDAwMDAwMDAwMDAwMDAiLCJtZW1vIjoic3c6ZDpnV095RFk0TEVrIn0=', + deposit: '1', + gas: 14000000000000, + methodName: 'ft_transfer', + }, + }, + ], + hash: 'DbCXaFDNfQ7LjM5Jh8BS5fKLF6UJG3vJZYbGtfpDaTEB', + nonce: 64963031000010, + publicKey: 'ed25519:FyzTGZb9mP8gcDbyhLCLumsA38a1JF3E5WELLJHnJsab', + receiverId: 'token.sweat', + signature: + 'ed25519:25q5o12NchK9rnUmrddTFKzYYCdqmPfQWq1QgRhGJiDiZmazqS2B4mPu7PeGYvVAEqucgq2k41EmXwgN3vLExjW5', + signerId: 'de9d1abf39de5fc4eac513b559cc825352d03c7b642a8af657113d61f8c195dc', + }, + }, + ], + }, + receiptExecutionOutcomes: [ + { + executionOutcome: { + blockHash: 'A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM', + id: 'CEgnqr5aYdpBErSzjvdtmdqnXdmXjXqe1AnPeamsLcvM', + outcome: { + executorId: 'f53b45ec59e26a50160026bd4e3d232a6672e0ee812d46cfcd29c0ef44a089f5', + gasBurnt: 424555062500, + logs: [], + metadata: { gasProfile: [], version: 1 }, + receiptIds: [], + status: { SuccessValue: '' }, + tokensBurnt: '0', + }, + proof: [ + { direction: 'Left', hash: 'DiN9q4rgCEpmykqXzoXWP2dNdjLyWEAgYzi7UmsQhEpx' }, + { direction: 'Left', hash: '45QTFnLgDmAbiiVBFmVHnivqQFU2ATNgLNErLnvZYZq2' }, + ], + }, + receipt: { + predecessorId: 'system', + receipt: { + Action: { + actions: [{ Transfer: { deposit: '1389731257024010315000' } }], + gasPrice: '0', + inputDataIds: [], + outputDataReceivers: [], + signerId: 'f53b45ec59e26a50160026bd4e3d232a6672e0ee812d46cfcd29c0ef44a089f5', + signerPublicKey: 'ed25519:HWHJq9M3m232SGpRTXuycdvjPEaaxn9TFN72bnEXnESp', + }, + }, + receiptId: 'CEgnqr5aYdpBErSzjvdtmdqnXdmXjXqe1AnPeamsLcvM', + receiverId: 'f53b45ec59e26a50160026bd4e3d232a6672e0ee812d46cfcd29c0ef44a089f5', + }, + }, + ], + shardId: 2, + stateChanges: [ + { + cause: { txHash: 'F4z3CFPjUGRJEF9rNSVhz4hZG2LpBvA3rR3AHv68432i', type: 'transaction_processing' }, + change: { + accountId: 'chadmoney.near', + amount: '3455422147665387737062056', + codeHash: '11111111111111111111111111111111', + locked: '0', + storagePaidAt: 0, + storageUsage: 2282, + }, + type: 'account_update', + }, + { + cause: { txHash: 'DbCXaFDNfQ7LjM5Jh8BS5fKLF6UJG3vJZYbGtfpDaTEB', type: 'transaction_processing' }, + change: { + accountId: 'de9d1abf39de5fc4eac513b559cc825352d03c7b642a8af657113d61f8c195dc', + amount: '37838802127313908609194', + codeHash: '11111111111111111111111111111111', + locked: '0', + storagePaidAt: 0, + storageUsage: 182, + }, + type: 'account_update', + }, + { + cause: { receiptHash: 'CEgnqr5aYdpBErSzjvdtmdqnXdmXjXqe1AnPeamsLcvM', type: 'receipt_processing' }, + change: { + accountId: 'f53b45ec59e26a50160026bd4e3d232a6672e0ee812d46cfcd29c0ef44a089f5', + amount: '44000142104620699999996', + codeHash: '11111111111111111111111111111111', + locked: '0', + storagePaidAt: 0, + storageUsage: 182, + }, + type: 'account_update', + }, + { + cause: { txHash: '3Zgj4eFNymhW7LBsaKtUsCWAaoP9LBXKGFtmebYsfibU', type: 'transaction_processing' }, + change: { + accountId: 'fd305d28fa12cb2e2769b211248059b8f4033ef4d4c0072c7daf246b166106ba', + amount: '33618268317256489684987', + codeHash: '11111111111111111111111111111111', + locked: '0', + storagePaidAt: 0, + storageUsage: 182, + }, + type: 'account_update', + }, + { + cause: { txHash: 'F4z3CFPjUGRJEF9rNSVhz4hZG2LpBvA3rR3AHv68432i', type: 'transaction_processing' }, + change: { + accessKey: { nonce: 72650480016948, permission: 'FullAccess' }, + accountId: 'chadmoney.near', + publicKey: 'ed25519:78fzZoSWw4yr7gRev6onwRFQPu5auDAPNuAhM2Ryy4EC', + }, + type: 'access_key_update', + }, + { + cause: { txHash: 'DbCXaFDNfQ7LjM5Jh8BS5fKLF6UJG3vJZYbGtfpDaTEB', type: 'transaction_processing' }, + change: { + accessKey: { nonce: 64963031000010, permission: 'FullAccess' }, + accountId: 'de9d1abf39de5fc4eac513b559cc825352d03c7b642a8af657113d61f8c195dc', + publicKey: 'ed25519:FyzTGZb9mP8gcDbyhLCLumsA38a1JF3E5WELLJHnJsab', + }, + type: 'access_key_update', + }, + { + cause: { txHash: '3Zgj4eFNymhW7LBsaKtUsCWAaoP9LBXKGFtmebYsfibU', type: 'transaction_processing' }, + change: { + accessKey: { nonce: 64674524000017, permission: 'FullAccess' }, + accountId: 'fd305d28fa12cb2e2769b211248059b8f4033ef4d4c0072c7daf246b166106ba', + publicKey: 'ed25519:J3LvQdaf6r6YVuTrVzo6em32R2ZG7jE9n4ysiLUpMa4y', + }, + type: 'access_key_update', + }, + ], + }, + { + chunk: { + author: 'binancenode1.poolv1.near', + header: { + balanceBurnt: '599537042985400000000', + chunkHash: '3Q39LzrowGBhqRZvkjDqQNjkfUEYEK6Zis5LUzqvrtFj', + encodedLength: 1837, + encodedMerkleRoot: '2d1EBsn1UspGLYjyFkrzPoawjghZe4fBbWkZfmsPJRbM', + gasLimit: 1000000000000000, + gasUsed: 7599093905353, + heightCreated: 85376002, + heightIncluded: 85376002, + outcomeRoot: 'CUmJsZfPtNYEggezXf6wW7NX2i7Vn9yuPz7RWfVKxR7p', + outgoingReceiptsRoot: '8TG5j9ahcEtGUNREhCEJW3SgHyeCzochf4foYxSU7b8Q', + prevBlockHash: 'H5X25nVZgx43tQSMcF5bRLphWGN92UVYdvaAhcgW4no7', + prevStateRoot: 'EPbgnc5qJFvFGqqAyQQu2SrFcVr6mr3AVbSuXnHvZ1ie', + rentPaid: '0', + shardId: 3, + signature: 'ed25519:RgPmNaXij3pNFdMVi1Ex8WXyQiYr12uUyRk5tuoddJvCTwJXcxQZZLCdy1XqB6uB9imMz9NsXQwT1M993NBMsjS', + txRoot: '8V1iVH9TSzWsPtNVorScHc4ha9ESNqNEG13tZiRSDzk7', + validatorProposals: [], + validatorReward: '0', + }, + receipts: [ + { + predecessorId: 'relay.aurora', + receipt: { + Action: { + actions: [ + { + FunctionCall: { + args: '+QGPggpehAQsHYCCUgiU5ShUyG+2SwTdHXetVCh23cBAxPSAuQEkGMuv5QAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADGiUDx20tY/zAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAR/fZpFXwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAoAAAAAAAAAAAAAAAAGBc4ZUyxxIdG15jLrd25ap7+4mQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAABhaeq/ToAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAADCXWsrHC113TW91as0DqbkM1NT1AAAAAAAAAAAAAAAASYiolrEichjkpob95eq9yr2RVx8AAAAAAAAAAAAAAADJve7TPNAVQeHu0Q+QUZ0sBv4/64ScioLIoNNwZYHzq18jJVWwzwd9vJY1JV4H29H/wOgwyQVM2FJzoCUAB6Nz+98pSh9AfalXBMbThpftEUVyF49fYhEKwmJf', + deposit: '0', + gas: 300000000000000, + methodName: 'submit', + }, + }, + ], + gasPrice: '625040174', + inputDataIds: [], + outputDataReceivers: [], + signerId: 'relay.aurora', + signerPublicKey: 'ed25519:2WmTnJeWZYKhh6NC3z3xTDKU5sHHAmHF96V6KAB3eHK5', + }, + }, + receiptId: '821S5L8Htxafh2DDuDKtA4qapudN3mRq2AMv2qPeBhpj', + receiverId: 'aurora', + }, + { + predecessorId: 'system', + receipt: { + Action: { + actions: [{ Transfer: { deposit: '1389731257024010315000' } }], + gasPrice: '0', + inputDataIds: [], + outputDataReceivers: [], + signerId: 'f53b45ec59e26a50160026bd4e3d232a6672e0ee812d46cfcd29c0ef44a089f5', + signerPublicKey: 'ed25519:HWHJq9M3m232SGpRTXuycdvjPEaaxn9TFN72bnEXnESp', + }, + }, + receiptId: 'CEgnqr5aYdpBErSzjvdtmdqnXdmXjXqe1AnPeamsLcvM', + receiverId: 'f53b45ec59e26a50160026bd4e3d232a6672e0ee812d46cfcd29c0ef44a089f5', + }, + ], + transactions: [ + { + outcome: { + executionOutcome: { + blockHash: 'A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM', + id: 'AMdmdjfHvKX6vv1fNEyCCio2jmSvepxSetxQFhgj826f', + outcome: { + executorId: 'relay.aurora', + gasBurnt: 2429696331596, + logs: [], + metadata: { gasProfile: null, version: 1 }, + receiptIds: ['9p28ts6sz8Meg4vu3Fqff5QmfkwFc65C22WgAoKF1ELh'], + status: { SuccessReceiptId: '9p28ts6sz8Meg4vu3Fqff5QmfkwFc65C22WgAoKF1ELh' }, + tokensBurnt: '242969633159600000000', + }, + proof: [ + { direction: 'Right', hash: 'CKbaXt3vrGUUt7Ydw1gjb9by6pbtMooXo7uWJrbWAXXX' }, + { direction: 'Right', hash: '4NXWzpp8uSWuhoAXgsJKCs2eyAVHL3wXxsAWocx1Fs5K' }, + { direction: 'Right', hash: 'FmBQ954izf84EZ5P4GVvmUfc8T4bGXXqQruyDTtikwo1' }, + ], + }, + receipt: null, + }, + transaction: { + actions: [ + { + FunctionCall: { + args: '+QMRgw65o4QELB2Agww1AJSQVQb22oFec8oTVHtF0ZmIZxBLIoC5AqS/zVdxAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAApRioAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAgAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADgAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAcAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAQDIwNjdlMDg4OWUyYmJmM2ZjYTI3NmFjOWNiYzdiYjI5OThmNzQzMjM1ZjUwNjU4NDUwNjMyMzQxMDJhMGEzODYAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwGI2NmMxNTdiYTljMWM0YTAxOWEzOTQ4MjlkMzQyMGQ4YjIxZDg5OTQzZjMzYTdiYTExMDQ0MWFiNDExOGI4NTYzZWNiZTkxM2Q5YTJmZmNhNTNiNGU1NWJjNTFiZDdmNjEyZDQxZDkxNTY4M2MzNGNiZWVhYTg0Yzc0NWI2N2VmNjZlMTM2YjE2NTFkZTEwODk3OWM0YjE4NzFiMjIwMjIyYmI4ZTBkNTgwNmM0ZjFlZjliY2U1OWRlNzI1Y2Y0YwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAADAYTE5ZWJlNmFmZjRiM2MxZGIxZjY2Njc5Nzc1MDc1ZTZkMzY0NGFlYjBiZmYyZmQxMjQzNjg5ZTQ3ZmNjYTk5NzA4NzhlY2E5MzI4ZDQ5YmM1ZGZiYWI0ZDU4N2Y4OGQ5MTkyMTJiY2FjYzY0YzJhNGFmMDUxZmFkZWE4NjdhZDkwMDUzMzY4OWQ1ZDcwNzg5YTQyMjNmMzVkM2JhYzhlNzE0NzY2NDg5NjQ4ODAxMTk1NGRlMDRiZWM0NDM3N2U4hJyKgsegSqdU7My5fwrLI1Mc3JcSJSgvQcmZsI1Chymt8Ydj0gqgXGqO8uaGvpV0jbfOyplT73XLhyC4ON9ekQI+tWB+8go=', + deposit: '0', + gas: 300000000000000, + methodName: 'submit', + }, + }, + ], + hash: 'AMdmdjfHvKX6vv1fNEyCCio2jmSvepxSetxQFhgj826f', + nonce: 65788368017229, + publicKey: 'ed25519:FptiMvPBN7QRJwroc2zJtTEAEo4h9CJAdGKQfd2bA3Qm', + receiverId: 'aurora', + signature: + 'ed25519:LehYX21sAyodn5g7MxXPkztx9tQMHQ92gMg2KQYQV4pBKyGidLHz6y2aeLREUFt2ckxrdWNZ6VannAGBsPpiHPu', + signerId: 'relay.aurora', + }, + }, + ], + }, + receiptExecutionOutcomes: [ + { + executionOutcome: { + blockHash: 'A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM', + id: 'CdUSBwV29D1P1qvxCwnbfNsTsdxfdRY6PxezsSWRFyqN', + outcome: { + executorId: 'relay.aurora', + gasBurnt: 223182562500, + logs: [], + metadata: { gasProfile: [], version: 1 }, + receiptIds: [], + status: { SuccessValue: '' }, + tokensBurnt: '0', + }, + proof: [ + { direction: 'Left', hash: '3zQuvAtY9UVHq2ws8vvMLmPivCpBPWwDhmTtCHNy1roV' }, + { direction: 'Right', hash: '4NXWzpp8uSWuhoAXgsJKCs2eyAVHL3wXxsAWocx1Fs5K' }, + { direction: 'Right', hash: 'FmBQ954izf84EZ5P4GVvmUfc8T4bGXXqQruyDTtikwo1' }, + ], + }, + receipt: { + predecessorId: 'system', + receipt: { + Action: { + actions: [{ Transfer: { deposit: '187086049935277727891452' } }], + gasPrice: '0', + inputDataIds: [], + outputDataReceivers: [], + signerId: 'relay.aurora', + signerPublicKey: 'ed25519:8j2e8u9t1SoTgMPn5G65ChU5WskpcnemBjwzkAnxDtZq', + }, + }, + receiptId: 'CdUSBwV29D1P1qvxCwnbfNsTsdxfdRY6PxezsSWRFyqN', + receiverId: 'relay.aurora', + }, + }, + { + executionOutcome: { + blockHash: 'A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM', + id: '7Q29UAwrFbeMMM6niPSuMrqkp4Y21FYqJ1SXPvr9oZYU', + outcome: { + executorId: 'token.sweat', + gasBurnt: 4072736369755, + logs: [ + 'EVENT_JSON:{"standard":"nep141","version":"1.0.0","event":"ft_transfer","data":[{"old_owner_id":"7a8ba1bd804863883ace10ed76441e6ad2239ae7917b78c89700b701f75ce1ab","new_owner_id":"reward-optin.sweat","amount":"100000000000000000","memo":"sw:rew:optin:YGRWvgpXK0-7a8ba1bd804863883ace10ed76441e6ad2239ae7917b78c89700b701f75ce1ab"}]}', + ], + metadata: { + gasProfile: [ + { cost: 'BASE', costCategory: 'WASM_HOST_COST', gasUsed: '7413507108' }, + { cost: 'CONTRACT_LOADING_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '35445963' }, + { cost: 'CONTRACT_LOADING_BYTES', costCategory: 'WASM_HOST_COST', gasUsed: '44831486250' }, + { cost: 'LOG_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '3543313050' }, + { cost: 'LOG_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '4342402239' }, + { cost: 'READ_CACHED_TRIE_NODE', costCategory: 'WASM_HOST_COST', gasUsed: '280440000000' }, + { cost: 'READ_MEMORY_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '36538084800' }, + { cost: 'READ_MEMORY_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '2816787753' }, + { cost: 'READ_REGISTER_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '27688817046' }, + { cost: 'READ_REGISTER_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '48295380' }, + { cost: 'SHA256_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '18163881000' }, + { cost: 'SHA256_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '3955245564' }, + { cost: 'STORAGE_READ_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '169070537250' }, + { cost: 'STORAGE_READ_KEY_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '2259534909' }, + { cost: 'STORAGE_READ_VALUE_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '572322510' }, + { cost: 'STORAGE_WRITE_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '192590208000' }, + { cost: 'STORAGE_WRITE_EVICTED_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '3275965314' }, + { cost: 'STORAGE_WRITE_KEY_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '5145249291' }, + { cost: 'STORAGE_WRITE_VALUE_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '3163890978' }, + { cost: 'TOUCHING_TRIE_NODE', costCategory: 'WASM_HOST_COST', gasUsed: '611874325188' }, + { cost: 'UTF8_DECODING_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '3111779061' }, + { cost: 'UTF8_DECODING_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '95929977591' }, + { cost: 'WASM_INSTRUCTION', costCategory: 'WASM_HOST_COST', gasUsed: '56074112424' }, + { cost: 'WRITE_MEMORY_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '33645538332' }, + { cost: 'WRITE_MEMORY_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '1378228632' }, + { cost: 'WRITE_REGISTER_BASE', costCategory: 'WASM_HOST_COST', gasUsed: '34386269832' }, + { cost: 'WRITE_REGISTER_BYTE', costCategory: 'WASM_HOST_COST', gasUsed: '2128875840' }, + ], + version: 1, + }, + receiptIds: ['BXhqz9Hk5eVb5ibaN3uZmNW4cgW6zBerUYpM7Zh19kXe'], + status: { SuccessValue: '' }, + tokensBurnt: '407273636975500000000', + }, + proof: [ + { direction: 'Right', hash: 'Cbwr9BAamScEJV1aMoZ21CULLdjPTpY6QQmfZNAS8cUx' }, + { direction: 'Left', hash: '7x71otoV7zEDfh2v1hYcLD9GNFZkoza1d3uoJUc79GMV' }, + { direction: 'Right', hash: 'FmBQ954izf84EZ5P4GVvmUfc8T4bGXXqQruyDTtikwo1' }, + ], + }, + receipt: { + predecessorId: '7a8ba1bd804863883ace10ed76441e6ad2239ae7917b78c89700b701f75ce1ab', + receipt: { + Action: { + actions: [ + { + FunctionCall: { + args: 'eyJyZWNlaXZlcl9pZCI6InJld2FyZC1vcHRpbi5zd2VhdCIsImFtb3VudCI6IjEwMDAwMDAwMDAwMDAwMDAwMCIsIm1lbW8iOiJzdzpyZXc6b3B0aW46WUdSV3ZncFhLMC03YThiYTFiZDgwNDg2Mzg4M2FjZTEwZWQ3NjQ0MWU2YWQyMjM5YWU3OTE3Yjc4Yzg5NzAwYjcwMWY3NWNlMWFiIn0=', + deposit: '1', + gas: 14000000000000, + methodName: 'ft_transfer', + }, + }, + ], + gasPrice: '109272700', + inputDataIds: [], + outputDataReceivers: [], + signerId: '7a8ba1bd804863883ace10ed76441e6ad2239ae7917b78c89700b701f75ce1ab', + signerPublicKey: 'ed25519:9FNEMP92rkuvRVefVkBW38RmbZ648XvQGeWbFYtdC7hk', + }, + }, + receiptId: '7Q29UAwrFbeMMM6niPSuMrqkp4Y21FYqJ1SXPvr9oZYU', + receiverId: 'token.sweat', + }, + }, + { + executionOutcome: { + blockHash: 'A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM', + id: 'BpZGWvbmx8sAMRsrhWyBnC5QHA9mFwist7orTceJTRvu', + outcome: { + executorId: 'olezha4.near', + gasBurnt: 223182562500, + logs: [], + metadata: { gasProfile: [], version: 1 }, + receiptIds: [], + status: { SuccessValue: '' }, + tokensBurnt: '0', + }, + proof: [ + { direction: 'Left', hash: 'GdEbJpKZ4iqem9VmS5Qbm2CTrt1qFZhfSD6zUv5JaYXT' }, + { direction: 'Left', hash: '7x71otoV7zEDfh2v1hYcLD9GNFZkoza1d3uoJUc79GMV' }, + { direction: 'Right', hash: 'FmBQ954izf84EZ5P4GVvmUfc8T4bGXXqQruyDTtikwo1' }, + ], + }, + receipt: { + predecessorId: 'system', + receipt: { + Action: { + actions: [{ Transfer: { deposit: '13222602662027447938448' } }], + gasPrice: '0', + inputDataIds: [], + outputDataReceivers: [], + signerId: 'olezha4.near', + signerPublicKey: 'ed25519:EnLNUDncRuw4AYLcGmi1Hr2YQ1DLu12VutsubZ6k8c2F', + }, + }, + receiptId: 'BpZGWvbmx8sAMRsrhWyBnC5QHA9mFwist7orTceJTRvu', + receiverId: 'olezha4.near', + }, + }, + { + executionOutcome: { + blockHash: 'A5Jb7zz4VPuQhBFo7rXn8UoAPUSokjpiV9deSoEz5ijM', + id: '3R2wjnktLwBKaceGiPetmShVAyHT2LuoQZMdNg6oWMwK', + outcome: { + executorId: 'marthacecilia.near', + gasBurnt: 223182562500, + logs: [], + metadata: { gasProfile: [], version: 1 }, + receiptIds: [], + status: { SuccessValue: '' }, + tokensBurnt: '0', + }, + proof: [{ direction: 'Left', hash: '6FWhYXFGBgzSPDuBCFrfSkdAqSvsMHPbnCJ2Q3UsmUV6' }], + }, + receipt: { + predecessorId: 'system', + receipt: { + Action: { + actions: [{ Transfer: { deposit: '13239855304236542651168' } }], + gasPrice: '0', + inputDataIds: [], + outputDataReceivers: [], + signerId: 'marthacecilia.near', + signerPublicKey: 'ed25519:8W28RYFfXztNA7ENP4VTaitsqEjSbABQJDqELwbwXswq', + }, + }, + receiptId: '3R2wjnktLwBKaceGiPetmShVAyHT2LuoQZMdNg6oWMwK', + receiverId: 'marthacecilia.near', + }, + }, + ], + shardId: 3, + stateChanges: [ + { + cause: { receiptHash: '3R2wjnktLwBKaceGiPetmShVAyHT2LuoQZMdNg6oWMwK', type: 'receipt_processing' }, + change: { + accountId: 'marthacecilia.near', + amount: '376417715605581247237588', + codeHash: '7DcAdMUT1MjaZ9s7zhXdyxKvQsRsSfnmBGdzeZaquqDE', + locked: '0', + storagePaidAt: 0, + storageUsage: 13722, + }, + type: 'account_update', + }, + { + cause: { receiptHash: 'BpZGWvbmx8sAMRsrhWyBnC5QHA9mFwist7orTceJTRvu', type: 'receipt_processing' }, + change: { + accountId: 'olezha4.near', + amount: '147297020835776249333187456', + codeHash: 'E8jZ1giWcVrps8PcV75ATauu6gFRkcwjNtKp7NKmipZG', + locked: '0', + storagePaidAt: 0, + storageUsage: 25029, + }, + type: 'account_update', + }, + { + cause: { txHash: 'AMdmdjfHvKX6vv1fNEyCCio2jmSvepxSetxQFhgj826f', type: 'transaction_processing' }, + change: { + accountId: 'relay.aurora', + amount: '2493412853875426176695038138', + codeHash: '11111111111111111111111111111111', + locked: '0', + storagePaidAt: 0, + storageUsage: 149094, + }, + type: 'account_update', + }, + { + cause: { receiptHash: 'CdUSBwV29D1P1qvxCwnbfNsTsdxfdRY6PxezsSWRFyqN', type: 'receipt_processing' }, + change: { + accountId: 'relay.aurora', + amount: '2493599939925361454422929590', + codeHash: '11111111111111111111111111111111', + locked: '0', + storagePaidAt: 0, + storageUsage: 149094, + }, + type: 'account_update', + }, + { + cause: { receiptHash: '7Q29UAwrFbeMMM6niPSuMrqkp4Y21FYqJ1SXPvr9oZYU', type: 'receipt_processing' }, + change: { + accountId: 'token.sweat', + amount: '32545059778997347633373318383', + codeHash: 'FMy4MTxATGtfxqTg5PZfGhQpRWej9Ppbttwo7FWF13wA', + locked: '0', + storagePaidAt: 0, + storageUsage: 1648821628, + }, + type: 'account_update', + }, + { + cause: { receiptHash: '7Q29UAwrFbeMMM6niPSuMrqkp4Y21FYqJ1SXPvr9oZYU', type: 'action_receipt_gas_reward' }, + change: { + accountId: 'token.sweat', + amount: '32545059828330070072473318383', + codeHash: 'FMy4MTxATGtfxqTg5PZfGhQpRWej9Ppbttwo7FWF13wA', + locked: '0', + storagePaidAt: 0, + storageUsage: 1648821628, + }, + type: 'account_update', + }, + { + cause: { receiptHash: '3R2wjnktLwBKaceGiPetmShVAyHT2LuoQZMdNg6oWMwK', type: 'receipt_processing' }, + change: { + accessKey: { + nonce: 85297952000430, + permission: { + FunctionCall: { + allowance: '117616350729529600000000', + methodNames: [], + receiverId: 'app.nearcrowd.near', + }, + }, + }, + accountId: 'marthacecilia.near', + publicKey: 'ed25519:8W28RYFfXztNA7ENP4VTaitsqEjSbABQJDqELwbwXswq', + }, + type: 'access_key_update', + }, + { + cause: { receiptHash: 'BpZGWvbmx8sAMRsrhWyBnC5QHA9mFwist7orTceJTRvu', type: 'receipt_processing' }, + change: { + accessKey: { + nonce: 84682948002117, + permission: { + FunctionCall: { + allowance: '330678248075353400000000', + methodNames: [], + receiverId: 'app.nearcrowd.near', + }, + }, + }, + accountId: 'olezha4.near', + publicKey: 'ed25519:EnLNUDncRuw4AYLcGmi1Hr2YQ1DLu12VutsubZ6k8c2F', + }, + type: 'access_key_update', + }, + { + cause: { txHash: 'AMdmdjfHvKX6vv1fNEyCCio2jmSvepxSetxQFhgj826f', type: 'transaction_processing' }, + change: { + accessKey: { nonce: 65788368017229, permission: 'FullAccess' }, + accountId: 'relay.aurora', + publicKey: 'ed25519:FptiMvPBN7QRJwroc2zJtTEAEo4h9CJAdGKQfd2bA3Qm', + }, + type: 'access_key_update', + }, + { + cause: { receiptHash: '7Q29UAwrFbeMMM6niPSuMrqkp4Y21FYqJ1SXPvr9oZYU', type: 'receipt_processing' }, + change: { + accountId: 'token.sweat', + keyBase64: 'U1RBVEU=', + valueBase64: + 'AgAAAHNpAQAAAAAAAAACAAAAc2UBAAAAdAEIAAAALnUuc3dlYXQUQEmH9+nGUg8Ez0cAAAAAWgAAAAAAAADiIbWN+QEAAA==', + }, + type: 'data_update', + }, + { + cause: { receiptHash: '7Q29UAwrFbeMMM6niPSuMrqkp4Y21FYqJ1SXPvr9oZYU', type: 'receipt_processing' }, + change: { + accountId: 'token.sweat', + keyBase64: 'dAAEzAvKN2Ekj8gSsSzscbcviXEnrSF9E4Nn8XlqPYFyaw==', + valueBase64: '6Hk2larECkgAAAAAAAAAAA==', + }, + type: 'data_update', + }, + { + cause: { receiptHash: '7Q29UAwrFbeMMM6niPSuMrqkp4Y21FYqJ1SXPvr9oZYU', type: 'receipt_processing' }, + change: { + accountId: 'token.sweat', + keyBase64: 'dADCUiigKPOTAl1jqrFvCFMV6YPDQuu8kGehwLHx0CgpIw==', + valueBase64: 'AAAu48ylkU6WIwAAAAAAAA==', + }, + type: 'data_update', + }, + ], + }, + ], +}; diff --git a/frontend/src/components/Logs/GraphQL/QueryValidation.ts b/frontend/src/components/Logs/GraphQL/QueryValidation.ts index 2f03f540d..0c5e64ff2 100644 --- a/frontend/src/components/Logs/GraphQL/QueryValidation.ts +++ b/frontend/src/components/Logs/GraphQL/QueryValidation.ts @@ -1,36 +1,46 @@ import { calculateTimestamp } from '@/utils/calculateTimestamp'; interface Variables { - limit: number - offset: number - order_by_timestamp: 'asc' | 'desc' - level?: string - type?: string - timestamp?: string - keyword?: string + limit: number; + offset: number; + order_by_timestamp: 'asc' | 'desc'; + level?: string; + type?: string; + timestamp?: string; + keyword?: string; } interface QueryFilter { - _eq?: string | number - _ilike?: string - _gte?: string + _eq?: string | number; + _ilike?: string; + _gte?: string; } interface QueryValidationResult { - limit: number - offset: number - order_by_timestamp: 'asc' | 'desc' - level?: QueryFilter - type?: QueryFilter - timestamp?: QueryFilter - keyword?: string - message?: { _ilike: string } - block_height?: { _eq: number } + limit: number; + offset: number; + order_by_timestamp: 'asc' | 'desc'; + level?: QueryFilter; + type?: QueryFilter; + timestamp?: QueryFilter; + keyword?: string; + message?: { _ilike: string }; + block_height?: { _eq: number }; } /* eslint-disable-next-line */ -export const QueryValidation = ({ limit, offset, order_by_timestamp, level, type, timestamp, keyword }: Variables): QueryValidationResult => { +export const QueryValidation = ({ + limit, + offset, + order_by_timestamp, + level, + type, + timestamp, + keyword, +}: Variables): QueryValidationResult => { const levelFormat: { level?: QueryFilter } = level ? { level: { _eq: level } } : {}; const typeFormat: { type?: QueryFilter } = type ? { type: { _eq: type } } : {}; - const timestampFormat: { timestamp?: QueryFilter } = timestamp ? { timestamp: { _gte: calculateTimestamp(timestamp) } } : {}; + const timestampFormat: { timestamp?: QueryFilter } = timestamp + ? { timestamp: { _gte: calculateTimestamp(timestamp) } } + : {}; const messageFormat = keyword ? { message: { _ilike: `%${keyword}%` } } : {}; const blockHeightFormat = keyword && !isNaN(Number(keyword)) ? { block_height: { _eq: Number(keyword) } } : {}; diff --git a/frontend/src/components/Logs/LogsMenu.tsx b/frontend/src/components/Logs/LogsMenu.tsx index 5fe102cbe..5d44e936d 100644 --- a/frontend/src/components/Logs/LogsMenu.tsx +++ b/frontend/src/components/Logs/LogsMenu.tsx @@ -6,14 +6,14 @@ import { IndexerDetailsContext } from '@/contexts/IndexerDetailsContext'; import LatestBlock from '../Common/LatestBlock'; interface LogsMenuProps { - currentUserAccountId: string - heights: any[] - setHeights: React.Dispatch> - latestHeight: string - isUserIndexer: boolean - accountId: string - reloadData: () => void - functionName: string + currentUserAccountId: string; + heights: any[]; + setHeights: React.Dispatch>; + latestHeight: string; + isUserIndexer: boolean; + accountId: string; + reloadData: () => void; + functionName: string; } const LogsMenu: React.FC = ({ @@ -68,7 +68,9 @@ const LogsMenu: React.FC = ({ try { const { data: refetchedData } = await refetch(); if (refetchedData) { - const newAttributeMap = new Map(refetchedData[queryName].map((item: any) => [item.attribute, item.value])); + const newAttributeMap = new Map( + refetchedData[queryName].map((item: any) => [item.attribute, item.value]), + ); if (newAttributeMap.has('LAST_PROCESSED_BLOCK_HEIGHT')) { setBlockHeight(newAttributeMap.get('LAST_PROCESSED_BLOCK_HEIGHT') ?? 'N/A'); @@ -87,14 +89,12 @@ const LogsMenu: React.FC = ({
- - Indexer: {functionName} - + Indexer: {functionName} Filter: {indexerDetails.rule.affected_account_id} - Status: {loading ? : status ?? 'UNKNOWN'} + Status: {loading ? : status ?? 'UNKNOWN'} Height: {loading ? : blockHeight ?? 'N/A'} @@ -113,13 +113,20 @@ const LogsMenu: React.FC = ({ Reload - - + ); }; diff --git a/frontend/src/components/Logs/LogsView/ClearButtonView.jsx b/frontend/src/components/Logs/LogsView/ClearButtonView.jsx index 776007dfb..28e00eee5 100644 --- a/frontend/src/components/Logs/LogsView/ClearButtonView.jsx +++ b/frontend/src/components/Logs/LogsView/ClearButtonView.jsx @@ -3,16 +3,16 @@ import { Button } from 'react-bootstrap'; import { ClearIcon } from '../LogsView/Icons/ClearIcon'; const ClearButtonView = ({ onClick }) => { - return ( - - ); + return ( + + ); }; export default ClearButtonView; diff --git a/frontend/src/components/Logs/LogsView/DateSelectorView.jsx b/frontend/src/components/Logs/LogsView/DateSelectorView.jsx index 6aa407883..a223c2308 100644 --- a/frontend/src/components/Logs/LogsView/DateSelectorView.jsx +++ b/frontend/src/components/Logs/LogsView/DateSelectorView.jsx @@ -2,13 +2,7 @@ import React from 'react'; import OptionSelectorContainer from '../LogsViewContainer/OptionSelectorContainer'; const DateSelectorView = ({ options, selectedOption, onOptionChange }) => { - return ( - - ); + return ; }; export default DateSelectorView; diff --git a/frontend/src/components/Logs/LogsView/Icons/CheckMarkIcon.js b/frontend/src/components/Logs/LogsView/Icons/CheckMarkIcon.js index e4e0597b1..a5d7da584 100644 --- a/frontend/src/components/Logs/LogsView/Icons/CheckMarkIcon.js +++ b/frontend/src/components/Logs/LogsView/Icons/CheckMarkIcon.js @@ -1,7 +1,7 @@ export const CheckmarkIcon = () => ( - - - - - + + + + + ); diff --git a/frontend/src/components/Logs/LogsView/Icons/ClearIcon.js b/frontend/src/components/Logs/LogsView/Icons/ClearIcon.js index 063b72e8b..4097bc91e 100644 --- a/frontend/src/components/Logs/LogsView/Icons/ClearIcon.js +++ b/frontend/src/components/Logs/LogsView/Icons/ClearIcon.js @@ -1,16 +1,10 @@ export const ClearIcon = () => { - return ( - - - - ); + return ( + + + + ); }; diff --git a/frontend/src/components/Logs/LogsView/IndexerLogsView.jsx b/frontend/src/components/Logs/LogsView/IndexerLogsView.jsx index bd7846620..761985006 100644 --- a/frontend/src/components/Logs/LogsView/IndexerLogsView.jsx +++ b/frontend/src/components/Logs/LogsView/IndexerLogsView.jsx @@ -1,92 +1,89 @@ import { Container, Row, Col } from 'react-bootstrap'; import LogsMenu from '../LogsMenu'; -import LogFieldCardView from "./LogFieldCardView"; -import "gridjs/dist/theme/mermaid.css"; -import styled from "styled-components"; +import LogFieldCardView from './LogFieldCardView'; +import 'gridjs/dist/theme/mermaid.css'; +import styled from 'styled-components'; const CustomGridContainer = styled.div` - .gridjs-wrapper{ + .gridjs-wrapper { border-radius: 0 !important; box-shadow: none !important; border: 1px solid #d2d2d2 !important; border-collapse: collapse !important; } - .gridjs-container{ + .gridjs-container { padding: 0 2px !important; } - .gridjs-table { + .gridjs-table { border-collapse: collapse; } - .gridjs-td { + .gridjs-td { border: none; } - .gridjs-search { + .gridjs-search { width: 100% !important; } - .gridjs-search-input{ + .gridjs-search-input { width: 100% !important; padding: 18px !important; border-radius: 4px 4px 0px 0 !important; border: 1px solid #d2d2d2 !important; } - .gridjs-head{ - padding:0 !important; - margin:0 !important; + .gridjs-head { + padding: 0 !important; + margin: 0 !important; } `; - const IndexerLogsView = ({ - severity, - setSeverity, - logType, - setLogType, - startTime, - setStartTime, - functionName, - tableName, - latestHeight, - currentIndexerDetails, - currentUserAccountId, - getIndexerLogsConfig, - getSearchConfig, - getPaginationConfig, - getGridStyle, - getGridConfig, - reloadData, - gridContainerRef + severity, + setSeverity, + logType, + setLogType, + startTime, + setStartTime, + functionName, + tableName, + latestHeight, + currentIndexerDetails, + currentUserAccountId, + getIndexerLogsConfig, + getSearchConfig, + getPaginationConfig, + getGridStyle, + getGridConfig, + reloadData, + gridContainerRef, }) => { - return ( - <> - + return ( + <> + - - - - - - - - - - - - ); + + + + + + + + + + + + ); }; export default IndexerLogsView; diff --git a/frontend/src/components/Logs/LogsView/LogFieldCardView.jsx b/frontend/src/components/Logs/LogsView/LogFieldCardView.jsx index 75041f925..61e5cc037 100644 --- a/frontend/src/components/Logs/LogsView/LogFieldCardView.jsx +++ b/frontend/src/components/Logs/LogsView/LogFieldCardView.jsx @@ -6,64 +6,60 @@ import DateSelectorContainer from '../LogsViewContainer/DateSelectorContainer'; import styled from 'styled-components'; const CustomAccordianWrapper = styled.div` - .accordion-button { - background-color: #f8f8f8 !important; - } - .accordion-item:first-of-type { - border-top-left-radius: 0 !important; - border-top-right-radius: 0 !important; - border-collapse: collapse !important;; - } - .accordion-item:first-of-type>.accordion-header .accordion-button { - border-top-left-radius: 0 !important; - border-top-right-radius: 0 !important; - } - .accordian{ - .--bs-accordion-border-width : 0 !important; - } + .accordion-button { + background-color: #f8f8f8 !important; + } + .accordion-item:first-of-type { + border-top-left-radius: 0 !important; + border-top-right-radius: 0 !important; + border-collapse: collapse !important; + } + .accordion-item:first-of-type > .accordion-header .accordion-button { + border-top-left-radius: 0 !important; + border-top-right-radius: 0 !important; + } + .accordian { + .--bs-accordion-border-width: 0 !important; + } `; -const LogFieldCardView = ({ severity, handleSeverityChange, logType, handleLogTypeChange, dateFilter, handleDateFilter }) => { - return ( - - Filters - - +const LogFieldCardView = ({ + severity, + handleSeverityChange, + logType, + handleLogTypeChange, + dateFilter, + handleDateFilter, +}) => { + return ( + + Filters + + + + Severity + + + + - - Severity - - - - + + Log Type + + + + - - Log Type - - - - - - - Date - - - - - - - - - ); + + Date + + + + + + + + ); }; export default LogFieldCardView; diff --git a/frontend/src/components/Logs/LogsView/LogTypeSelectorView.jsx b/frontend/src/components/Logs/LogsView/LogTypeSelectorView.jsx index e37e9e894..68d43da39 100644 --- a/frontend/src/components/Logs/LogsView/LogTypeSelectorView.jsx +++ b/frontend/src/components/Logs/LogsView/LogTypeSelectorView.jsx @@ -2,13 +2,7 @@ import React from 'react'; import OptionSelectorContainer from '../LogsViewContainer/OptionSelectorContainer'; const LogTypeSelectorView = ({ options, selectedOption, onOptionChange }) => { - return ( - - ); + return ; }; export default LogTypeSelectorView; diff --git a/frontend/src/components/Logs/LogsView/OptionSelectorView.jsx b/frontend/src/components/Logs/LogsView/OptionSelectorView.jsx index 60a4a62ab..0439d7140 100644 --- a/frontend/src/components/Logs/LogsView/OptionSelectorView.jsx +++ b/frontend/src/components/Logs/LogsView/OptionSelectorView.jsx @@ -4,29 +4,29 @@ import { CheckmarkIcon } from './Icons/CheckMarkIcon'; import ClearButtonContainer from '../LogsViewContainer/ClearButtonContainer'; const OptionSelectorView = ({ options, selectedOption, onOptionChange, handleOptionChange, handleClearSelection }) => { - return ( -
- - {options.map((option, index) => ( - -
handleOptionChange(option)} - > - - {selectedOption === option && } -
{option}
-
- {selectedOption === option && ( - - )} -
- - ))} -
-
- ); + return ( +
+ + {options.map((option, index) => ( + +
handleOptionChange(option)} + > + + {selectedOption === option && } +
{option}
+
+ {selectedOption === option && } +
+ + ))} +
+
+ ); }; export default OptionSelectorView; diff --git a/frontend/src/components/Logs/LogsView/SeveritySelectorView.jsx b/frontend/src/components/Logs/LogsView/SeveritySelectorView.jsx index 12df8a9d2..340c17034 100644 --- a/frontend/src/components/Logs/LogsView/SeveritySelectorView.jsx +++ b/frontend/src/components/Logs/LogsView/SeveritySelectorView.jsx @@ -2,13 +2,7 @@ import React from 'react'; import OptionSelectorContainer from '../LogsViewContainer/OptionSelectorContainer'; const SeveritySelectorView = ({ options, selectedOption, onOptionChange }) => { - return ( - - ); + return ; }; export default SeveritySelectorView; diff --git a/frontend/src/components/Logs/LogsViewContainer/ClearButtonContainer.tsx b/frontend/src/components/Logs/LogsViewContainer/ClearButtonContainer.tsx index 428761050..77edd1b8e 100644 --- a/frontend/src/components/Logs/LogsViewContainer/ClearButtonContainer.tsx +++ b/frontend/src/components/Logs/LogsViewContainer/ClearButtonContainer.tsx @@ -2,7 +2,7 @@ import React, { type MouseEvent, type FC } from 'react'; import ClearButtonView from '../LogsView/ClearButtonView'; interface ClearButtonProps { - onClick: () => void + onClick: () => void; } const ClearButton: FC = ({ onClick }) => { diff --git a/frontend/src/components/Logs/LogsViewContainer/DateSelectorContainer.tsx b/frontend/src/components/Logs/LogsViewContainer/DateSelectorContainer.tsx index a1aa76f4a..cb05ba389 100644 --- a/frontend/src/components/Logs/LogsViewContainer/DateSelectorContainer.tsx +++ b/frontend/src/components/Logs/LogsViewContainer/DateSelectorContainer.tsx @@ -3,20 +3,14 @@ import DateSelectorView from '../LogsView/DateSelectorView'; import { TIME_INTERVALS_MAP } from '@/constants/DurationMap'; interface DateSelectorProps { - selectedDate: string - onDateChange: (selectedDate: Date) => void + selectedDate: string; + onDateChange: (selectedDate: Date) => void; } const DateSelectorContainer: React.FC = ({ selectedDate, onDateChange }) => { const dateOptions: string[] = Array.from(TIME_INTERVALS_MAP.values()); - return ( - - ); + return ; }; export default DateSelectorContainer; diff --git a/frontend/src/components/Logs/LogsViewContainer/IndexerLogsContainer.tsx b/frontend/src/components/Logs/LogsViewContainer/IndexerLogsContainer.tsx index ebb5b376b..af358681f 100644 --- a/frontend/src/components/Logs/LogsViewContainer/IndexerLogsContainer.tsx +++ b/frontend/src/components/Logs/LogsViewContainer/IndexerLogsContainer.tsx @@ -10,16 +10,16 @@ import { Query } from '../GraphQL/Query'; import { formatTimestamp } from '@/utils/formatTimestamp'; interface GridConfig { - columns: string[] - search: any - pagination: any - server: any - style: any - sort: boolean + columns: string[]; + search: any; + pagination: any; + server: any; + style: any; + sort: boolean; } interface InitialPayload { - currentUserAccountId: string + currentUserAccountId: string; } const GRAPHQL_ENDPOINT = `${process.env.NEXT_PUBLIC_HASURA_ENDPOINT as string}/v1/graphql`; @@ -54,18 +54,24 @@ const IndexerLogsContainer: React.FC = () => { }, body: JSON.stringify({ query: Query(tableName), - variables: QueryValidation({ limit: LOGS_PER_PAGE, offset: 0, order_by_timestamp: 'desc', level: severity, type: logType, timestamp: startTime }) + variables: QueryValidation({ + limit: LOGS_PER_PAGE, + offset: 0, + order_by_timestamp: 'desc', + level: severity, + type: logType, + timestamp: startTime, + }), }), - then: ({ data }: any) => ( + then: ({ data }: any) => data[tableName].map((log: any) => [ log.level, log.type, formatTimestamp(log.timestamp) ?? log.timestamp, log.block_height, - log.message - ]) - ), - total: ({ data }: any) => (data[`${tableName}_aggregate`].aggregate.count), + log.message, + ]), + total: ({ data }: any) => data[`${tableName}_aggregate`].aggregate.count, }; }; @@ -77,19 +83,26 @@ const IndexerLogsContainer: React.FC = () => { body: (prev: string, keyword: string) => { return JSON.stringify({ query: Query(tableName), - variables: QueryValidation({ limit: LOGS_PER_PAGE, offset: 0, order_by_timestamp: 'desc', level: severity, type: logType, timestamp: startTime, keyword }) + variables: QueryValidation({ + limit: LOGS_PER_PAGE, + offset: 0, + order_by_timestamp: 'desc', + level: severity, + type: logType, + timestamp: startTime, + keyword, + }), }); }, - then: ({ data }: any) => ( + then: ({ data }: any) => data[tableName].map((log: any) => [ log.level, log.type, formatTimestamp(log.timestamp) ?? log.timestamp, log.block_height, - log.message - ]) - ), - total: ({ data }: any) => (data[`${tableName}_aggregate`].aggregate.count), + log.message, + ]), + total: ({ data }: any) => data[`${tableName}_aggregate`].aggregate.count, }, }; }; @@ -99,7 +112,7 @@ const IndexerLogsContainer: React.FC = () => { prevButton: false, nextButton: false, limit: LOGS_PER_PAGE, - buttonsCount: 0 + buttonsCount: 0, }; }; diff --git a/frontend/src/components/Logs/LogsViewContainer/LogTypeSelectorContainer.tsx b/frontend/src/components/Logs/LogsViewContainer/LogTypeSelectorContainer.tsx index 056490d91..4b8f7ffe2 100644 --- a/frontend/src/components/Logs/LogsViewContainer/LogTypeSelectorContainer.tsx +++ b/frontend/src/components/Logs/LogsViewContainer/LogTypeSelectorContainer.tsx @@ -2,19 +2,15 @@ import React from 'react'; import LogTypeSelectorView from '../LogsView/LogTypeSelectorView'; interface LogTypeSelectorContainerProps { - selectedLogType: string - onLogTypeChange: (logType: string) => void + selectedLogType: string; + onLogTypeChange: (logType: string) => void; } const LogTypeSelectorContainer: React.FC = ({ selectedLogType, onLogTypeChange }) => { const logTypeOptions: string[] = ['system', 'user']; return ( - + ); }; diff --git a/frontend/src/components/Logs/LogsViewContainer/OptionSelectorContainer.tsx b/frontend/src/components/Logs/LogsViewContainer/OptionSelectorContainer.tsx index e5bb77a1a..2cf3dd171 100644 --- a/frontend/src/components/Logs/LogsViewContainer/OptionSelectorContainer.tsx +++ b/frontend/src/components/Logs/LogsViewContainer/OptionSelectorContainer.tsx @@ -1,12 +1,16 @@ import React from 'react'; import OptionSelectorView from '../LogsView/OptionSelectorView'; interface OptionSelectorContainerProps { - options: string[] - selectedOption: string - onOptionChange: (option: string) => void + options: string[]; + selectedOption: string; + onOptionChange: (option: string) => void; } -const OptionSelectorContainer: React.FC = ({ options, selectedOption, onOptionChange }) => { +const OptionSelectorContainer: React.FC = ({ + options, + selectedOption, + onOptionChange, +}) => { const handleOptionChange = (value: string): void => { onOptionChange(value); }; diff --git a/frontend/src/components/Logs/LogsViewContainer/SeveritySelectorContainer.tsx b/frontend/src/components/Logs/LogsViewContainer/SeveritySelectorContainer.tsx index bba590f2b..cb7635ac6 100644 --- a/frontend/src/components/Logs/LogsViewContainer/SeveritySelectorContainer.tsx +++ b/frontend/src/components/Logs/LogsViewContainer/SeveritySelectorContainer.tsx @@ -2,8 +2,8 @@ import React from 'react'; import SeverityRadioButtonGroupView from '../LogsView/SeveritySelectorView'; interface SeveritySelectorProps { - selectedSeverity: string - onSeverityChange: (severity: string) => void + selectedSeverity: string; + onSeverityChange: (severity: string) => void; } const SeveritySelectorContainer: React.FC = ({ selectedSeverity, onSeverityChange }) => { diff --git a/frontend/src/components/Modals/ForkIndexerModal.jsx b/frontend/src/components/Modals/ForkIndexerModal.jsx index 92bf2349d..e1ef53a43 100644 --- a/frontend/src/components/Modals/ForkIndexerModal.jsx +++ b/frontend/src/components/Modals/ForkIndexerModal.jsx @@ -1,6 +1,6 @@ -import React, { useContext, useState } from "react"; -import { Button, Modal, Alert, InputGroup, Form } from "react-bootstrap"; -import { IndexerDetailsContext } from "../../contexts/IndexerDetailsContext"; +import React, { useContext, useState } from 'react'; +import { Button, Modal, Alert, InputGroup, Form } from 'react-bootstrap'; +import { IndexerDetailsContext } from '../../contexts/IndexerDetailsContext'; export const ForkIndexerModal = ({ registerFunction, forkIndexer }) => { const { @@ -14,19 +14,17 @@ export const ForkIndexerModal = ({ registerFunction, forkIndexer }) => { setIndexerConfig, isCreateNewIndexer, } = useContext(IndexerDetailsContext); - const [indexerName, setIndexerNameField] = useState(""); + const [indexerName, setIndexerNameField] = useState(''); const [error, setError] = useState(null); const fork = async () => { if (!indexerName) { - setError("Please provide an Indexer Name"); + setError('Please provide an Indexer Name'); return; } if (indexerName === indexerDetails.indexerName) { - setError( - "Please provide a different Indexer Name than the original Indexer" - ); + setError('Please provide a different Indexer Name than the original Indexer'); return; } @@ -47,15 +45,11 @@ export const ForkIndexerModal = ({ registerFunction, forkIndexer }) => { className="bg-gray-50" > - - Enter Indexer Details - + Enter Indexer Details - - Indexer Name - + Indexer Name { > Cancel - diff --git a/frontend/src/components/Modals/ModalsContainer/PublishFormContainer.tsx b/frontend/src/components/Modals/ModalsContainer/PublishFormContainer.tsx index b49727b44..ca56fc627 100644 --- a/frontend/src/components/Modals/ModalsContainer/PublishFormContainer.tsx +++ b/frontend/src/components/Modals/ModalsContainer/PublishFormContainer.tsx @@ -4,7 +4,7 @@ import { validateContractIds } from '../../../utils/validators'; import PublishFormView from '../ModalsView/PublishFormView'; interface Props { - updateConfig: (indexerName: string, contractFilter: string, blockHeight: string, startBlock: string) => void + updateConfig: (indexerName: string, contractFilter: string, blockHeight: string, startBlock: string) => void; } const GENESIS_BLOCK_HEIGHT = 9820210; @@ -57,7 +57,7 @@ const ViewContainer: React.FC = ({ updateConfig }) => { } }; - function handleSetContractFilter (contractFilter: string): void { + function handleSetContractFilter(contractFilter: string): void { setContractFilter(contractFilter); const isContractFilterValid = validateContractIds(contractFilter); setIsContractFilterValid(isContractFilterValid); @@ -65,7 +65,9 @@ const ViewContainer: React.FC = ({ updateConfig }) => { useEffect(() => { if (startBlock === START_BLOCK.HEIGHT && parseInt(blockHeight) <= GENESIS_BLOCK_HEIGHT) { - setBlockHeightError(`Choose a block height greater than the Genesis BlockHeight ${GENESIS_BLOCK_HEIGHT}. Latest Block Height is ${latestHeight}`); + setBlockHeightError( + `Choose a block height greater than the Genesis BlockHeight ${GENESIS_BLOCK_HEIGHT}. Latest Block Height is ${latestHeight}`, + ); return; } setBlockHeightError(null); diff --git a/frontend/src/components/Modals/ModalsView/PublishFormView.jsx b/frontend/src/components/Modals/ModalsView/PublishFormView.jsx index ba6817fe2..51311055d 100644 --- a/frontend/src/components/Modals/ModalsView/PublishFormView.jsx +++ b/frontend/src/components/Modals/ModalsView/PublishFormView.jsx @@ -1,116 +1,117 @@ -import React from "react"; +import React from 'react'; const PublishFormView = ({ - indexerNameField, - isCreateNewIndexer, - showPublishModal, - startBlock, - blockHeight, - contractFilter, - latestHeight, - blockHeightError, - isContractFilterValid, - onChangeStartBlock, - setIndexerNameField, - setBlockHeight, - handleSetContractFilter, - updateConfig, - indexerDetails, + indexerNameField, + isCreateNewIndexer, + showPublishModal, + startBlock, + blockHeight, + contractFilter, + latestHeight, + blockHeightError, + isContractFilterValid, + onChangeStartBlock, + setIndexerNameField, + setBlockHeight, + handleSetContractFilter, + updateConfig, + indexerDetails, }) => ( -
-
- - setIndexerNameField(e.target.value.toLowerCase().trim())} - className="px-3 py-2 border border-gray-300 rounded-md focus:outline-none focus:ring-2 focus:ring-gray-500 focus:border-gray-500" - /> -
+
+
+ + setIndexerNameField(e.target.value.toLowerCase().trim())} + className="px-3 py-2 border border-gray-300 rounded-md focus:outline-none focus:ring-2 focus:ring-gray-500 focus:border-gray-500" + /> +
-
- - -
+
+ + +
- {!isCreateNewIndexer && ( -
- - -
- )} + {!isCreateNewIndexer && ( +
+ + +
+ )} -
- - - setBlockHeight(parseInt(e.target.value))} - className="ml-3 px-3 py-2 border border-gray-300 rounded-md focus:outline-none focus:ring-2 focus:ring-gray-500 focus:border-gray-500" - /> -
- {blockHeightError && ( -
- {blockHeightError} -
- )} +
+ + + setBlockHeight(parseInt(e.target.value))} + className="ml-3 px-3 py-2 border border-gray-300 rounded-md focus:outline-none focus:ring-2 focus:ring-gray-500 focus:border-gray-500" + /> +
+ {blockHeightError &&
{blockHeightError}
} -
- - handleSetContractFilter(e.target.value)} - required - className={`px-3 py-2 border rounded-md focus:outline-none focus:ring-2 focus:ring-gray-500 focus:border-gray-500 ${isContractFilterValid ? 'border-gray-300' : 'border-red-500'}`} - /> - {!isContractFilterValid && ( -
- Please provide a valid contract name. -
- )} - {startBlock === "startBlockContinue" && ( -
- Contract filter cannot be changed for "Continue" option. -
- )} +
+ + handleSetContractFilter(e.target.value)} + required + disabled={startBlock === 'startBlockContinue'} + className={`px-3 py-2 border rounded-md focus:outline-none focus:ring-2 focus:ring-gray-500 focus:border-gray-500 ${ + isContractFilterValid ? 'border-gray-300' : 'border-red-500' + }`} + /> + {!isContractFilterValid &&
Please provide a valid contract name.
} + {startBlock === 'startBlockContinue' && ( +
+ Contract filter cannot be changed for "Continue" option.
+ )}
+
); export default PublishFormView; diff --git a/frontend/src/components/Modals/PublishModal.jsx b/frontend/src/components/Modals/PublishModal.jsx index 45f7c9be3..e494840f7 100644 --- a/frontend/src/components/Modals/PublishModal.jsx +++ b/frontend/src/components/Modals/PublishModal.jsx @@ -1,20 +1,13 @@ -import React, { useContext, useState } from "react"; -import { Button, Modal, Alert } from "react-bootstrap"; -import PublishFormContainer from "./ModalsContainer/PublishFormContainer"; +import React, { useContext, useState } from 'react'; +import { Button, Modal, Alert } from 'react-bootstrap'; +import PublishFormContainer from './ModalsContainer/PublishFormContainer'; import { IndexerDetailsContext } from '../../contexts/IndexerDetailsContext'; -import { validateContractIds } from "../../utils/validators"; +import { validateContractIds } from '../../utils/validators'; -export const PublishModal = ({ - registerFunction, - actionButtonText, -}) => { - const { - indexerDetails, - showPublishModal, - setShowPublishModal, - } = useContext(IndexerDetailsContext); - const [indexerConfig, setIndexerConfig] = useState({ filter: "social.near", startBlockHeight: null }); - const [indexerName, setIndexerName] = useState(""); +export const PublishModal = ({ registerFunction, actionButtonText }) => { + const { indexerDetails, showPublishModal, setShowPublishModal } = useContext(IndexerDetailsContext); + const [indexerConfig, setIndexerConfig] = useState({ filter: 'social.near', startBlockHeight: null }); + const [indexerName, setIndexerName] = useState(''); const [error, setError] = useState(null); const updateConfig = (indexerName, filter, height, startBlock) => { @@ -23,13 +16,13 @@ export const PublishModal = ({ }; const register = async () => { - if (indexerName === undefined || indexerName === "") { - setError(() => "Please provide an Indexer Name"); + if (indexerName === undefined || indexerName === '') { + setError(() => 'Please provide an Indexer Name'); return; } if (!validateContractIds(indexerConfig.filter)) { - setError(() => "Please provide a valid contract name"); + setError(() => 'Please provide a valid contract name'); return; } setError(null); @@ -38,16 +31,9 @@ export const PublishModal = ({ }; return ( - setShowPublishModal(false)} - className="bg-gray-50" - > + setShowPublishModal(false)} className="bg-gray-50"> - - Enter Indexer Details - + Enter Indexer Details @@ -68,11 +54,7 @@ export const PublishModal = ({ > Cancel - diff --git a/frontend/src/components/Modals/resetChanges.jsx b/frontend/src/components/Modals/resetChanges.jsx index 225375341..15c2369b7 100644 --- a/frontend/src/components/Modals/resetChanges.jsx +++ b/frontend/src/components/Modals/resetChanges.jsx @@ -1,30 +1,16 @@ -import { IndexerDetailsContext } from "../../contexts/IndexerDetailsContext"; -import React, { useContext } from "react"; -import { Button, Modal } from "react-bootstrap"; -export const ResetChangesModal = ({ - handleReload, -}) => { - const { - showResetCodeModel, - setShowResetCodeModel, - } = useContext(IndexerDetailsContext); +import { IndexerDetailsContext } from '../../contexts/IndexerDetailsContext'; +import React, { useContext } from 'react'; +import { Button, Modal } from 'react-bootstrap'; +export const ResetChangesModal = ({ handleReload }) => { + const { showResetCodeModel, setShowResetCodeModel } = useContext(IndexerDetailsContext); return ( - setShowResetCodeModel(false)} - > + setShowResetCodeModel(false)}> Are you sure? - - The changes you have made in the editor will be deleted. - + The changes you have made in the editor will be deleted. - - )} - {onOkButtonPressed && ( - - )} - - - ); + return ( + + + {title} + + +

{message}

+
+ + {onCancelButtonPressed && ( + + )} + {onOkButtonPressed && ( + + )} + +
+ ); }; InfoModal.propTypes = { - open: PropTypes.bool.isRequired, - title: PropTypes.string.isRequired, - message: PropTypes.string.isRequired, - okButtonText: PropTypes.string, - onOkButtonPressed: PropTypes.func, - cancelButtonText: PropTypes.string, - onCancelButtonPressed: PropTypes.func, - onClose: PropTypes.func, + open: PropTypes.bool.isRequired, + title: PropTypes.string.isRequired, + message: PropTypes.string.isRequired, + okButtonText: PropTypes.string, + onOkButtonPressed: PropTypes.func, + cancelButtonText: PropTypes.string, + onCancelButtonPressed: PropTypes.func, + onClose: PropTypes.func, }; diff --git a/frontend/src/pages/_app.jsx b/frontend/src/pages/_app.jsx index f93101bca..58882823a 100644 --- a/frontend/src/pages/_app.jsx +++ b/frontend/src/pages/_app.jsx @@ -1,12 +1,12 @@ -import "bootstrap/dist/css/bootstrap.min.css"; -import "near-social-bridge/near-social-bridge.css"; -import "./global.css"; +import 'bootstrap/dist/css/bootstrap.min.css'; +import 'near-social-bridge/near-social-bridge.css'; +import './global.css'; import 'regenerator-runtime/runtime'; import React from 'react'; -import { Spinner } from "near-social-bridge"; -import { overrideLocalStorage, NearSocialBridgeProvider } from "near-social-bridge"; +import { Spinner } from 'near-social-bridge'; +import { overrideLocalStorage, NearSocialBridgeProvider } from 'near-social-bridge'; import { ApolloClient, InMemoryCache, ApolloProvider } from '@apollo/client'; import { ModalProvider } from '@/contexts/ModalContext'; import { IndexerDetailsProvider } from '@/contexts/IndexerDetailsContext'; @@ -14,10 +14,9 @@ import { IndexerDetailsProvider } from '@/contexts/IndexerDetailsContext'; overrideLocalStorage(); export default function App({ Component, pageProps }) { - const client = new ApolloClient({ uri: `${process.env.NEXT_PUBLIC_HASURA_ENDPOINT}/v1/graphql`, - cache: new InMemoryCache() + cache: new InMemoryCache(), }); return ( diff --git a/frontend/src/pages/create-new-indexer/index.js b/frontend/src/pages/create-new-indexer/index.js index d9f34917d..6597ac1ed 100644 --- a/frontend/src/pages/create-new-indexer/index.js +++ b/frontend/src/pages/create-new-indexer/index.js @@ -1,8 +1,8 @@ -import React, { useContext, useEffect } from "react"; +import React, { useContext, useEffect } from 'react'; import { withRouter } from 'next/router'; import { Alert } from 'react-bootstrap'; import { IndexerDetailsContext } from '@/contexts/IndexerDetailsContext'; -import CreateNewIndexer from "@/components/CreateNewIndexer"; +import CreateNewIndexer from '@/components/CreateNewIndexer'; const CreateNewIndexerPage = ({ router }) => { const { accountId } = router.query; @@ -23,7 +23,7 @@ const CreateNewIndexerPage = ({ router }) => { ); } - return (); + return ; }; export default withRouter(CreateNewIndexerPage); diff --git a/frontend/src/pages/query-api-editor/index.js b/frontend/src/pages/query-api-editor/index.js index 2203ee1ce..887b0fe64 100644 --- a/frontend/src/pages/query-api-editor/index.js +++ b/frontend/src/pages/query-api-editor/index.js @@ -1,7 +1,7 @@ -import React, { useEffect, useContext } from "react"; +import React, { useEffect, useContext } from 'react'; import { withRouter } from 'next/router'; import { Alert } from 'react-bootstrap'; -import Editor from "@/components/Editor"; +import Editor from '@/components/Editor'; import IndexerLogsContainer from '@/components/Logs/LogsViewContainer/IndexerLogsContainer'; import { IndexerDetailsContext } from '@/contexts/IndexerDetailsContext'; @@ -31,7 +31,10 @@ const QueryApiEditorPage = ({ router }) => { {showLogsView ? ( ) : ( - + )} ); diff --git a/frontend/src/utils/fetchBlock.js b/frontend/src/utils/fetchBlock.js index 05a00468b..b03f87464 100644 --- a/frontend/src/utils/fetchBlock.js +++ b/frontend/src/utils/fetchBlock.js @@ -1,5 +1,4 @@ -const BLOCK_FETCHER_API = - "https://70jshyr5cb.execute-api.eu-central-1.amazonaws.com/block/"; +const BLOCK_FETCHER_API = 'https://70jshyr5cb.execute-api.eu-central-1.amazonaws.com/block/'; const GENESIS_BLOCK_HEIGHT = 52945886; export async function fetchBlockDetails(blockHeight) { @@ -7,9 +6,7 @@ export async function fetchBlockDetails(blockHeight) { throw new Error(`Block Height must be greater than genesis block height #${GENESIS_BLOCK_HEIGHT}`); } try { - const response = await fetch( - `${BLOCK_FETCHER_API}${String(blockHeight)}` - ); + const response = await fetch(`${BLOCK_FETCHER_API}${String(blockHeight)}`); const block_details = await response.json(); return block_details; } catch { diff --git a/frontend/src/utils/formatters.js b/frontend/src/utils/formatters.js index d9ddfcac4..a6de20b03 100644 --- a/frontend/src/utils/formatters.js +++ b/frontend/src/utils/formatters.js @@ -1,6 +1,6 @@ -import prettier from "prettier"; -import SqlPlugin from "prettier-plugin-sql"; -import parserBabel from "prettier/parser-babel"; +import prettier from 'prettier'; +import SqlPlugin from 'prettier-plugin-sql'; +import parserBabel from 'prettier/parser-babel'; let wrap_code = (code) => `import * as primitives from "@near-lake/primitives" /** @@ -23,34 +23,37 @@ async function getBlock(block: primitives.Block) { export const formatSQL = (schema) => { return prettier.format(schema, { - parser: "sql", - formatter: "sql-formatter", + parser: 'sql', + formatter: 'sql-formatter', plugins: [SqlPlugin], pluginSearchDirs: false, - language: "postgresql", - database: "postgresql", + language: 'postgresql', + database: 'postgresql', }); }; export const wrapCode = (code) => { - code = code.replace(/(?:\\[n])+/g, "\r\n"); + code = code.replace(/(?:\\[n])+/g, '\r\n'); const wrappedCode = wrap_code(code); return wrappedCode; }; export const formatIndexingCode = (code) => { return prettier.format(code, { - parser: "babel", + parser: 'babel', plugins: [parserBabel], }); }; -export const defaultCode = formatIndexingCode(wrapCode( - ` +export const defaultCode = formatIndexingCode( + wrapCode( + ` // Add your code here const h = block.header().height await context.set('height', h); -`)); +`, + ), +); export const defaultSchema = ` CREATE TABLE "indexer_storage" ("function_name" TEXT NOT NULL, "key_name" TEXT NOT NULL, "value" TEXT NOT NULL, PRIMARY KEY ("function_name", "key_name")) diff --git a/frontend/src/utils/formatters.test.js b/frontend/src/utils/formatters.test.js index 9c88c6994..5ed9f451b 100644 --- a/frontend/src/utils/formatters.test.js +++ b/frontend/src/utils/formatters.test.js @@ -1,8 +1,7 @@ -import { formatSQL, formatIndexingCode } from "./formatters"; +import { formatSQL, formatIndexingCode } from './formatters'; const inputSQL1 = `CREATE TABLE\n "indexer_storage" (\n "function_name" TEXT NOT NULL,\n "key_name" TEXT NOT NULL,\n "value" TEXT NOT NULL,\n PRIMARY KEY ("function_name", "key_name")\n )\n`; -const expectedOutput1 = - `CREATE TABLE +const expectedOutput1 = `CREATE TABLE "indexer_storage" ( "function_name" TEXT NOT NULL, "key_name" TEXT NOT NULL, @@ -11,27 +10,27 @@ const expectedOutput1 = ) `; -test("Basic formatting for SQL", () => { +test('Basic formatting for SQL', () => { expect(formatSQL(inputSQL1)).toEqual(expectedOutput1); }); const inputSQL2 = `CREATE INVALID TABLE indexer_storage"`; -test("Formatting invalid SQL input returns the invalid unformatted input", () => { +test('Formatting invalid SQL input returns the invalid unformatted input', () => { console.log(formatSQL(inputSQL2)); expect(formatSQL(inputSQL2)).toEqual(inputSQL2); }); -const inputJS2 = "\n const h = block.header().height;\n console.log(\"About to write demo_blockheight\", h);\n await context.set(\"demo_height\", h);\n"; +const inputJS2 = + '\n const h = block.header().height;\n console.log("About to write demo_blockheight", h);\n await context.set("demo_height", h);\n'; const expectedOutput2 = `const h = block.header().height; console.log("About to write demo_blockheight", h); await context.set("demo_height", h);\n`; -test("formatting for JS code without wrapCode", () => { +test('formatting for JS code without wrapCode', () => { expect(formatIndexingCode(inputJS2, false)).toEqual(expectedOutput2); }); - const expectedOutput3 = `import { Block } from "@near-lake/primitives"; /** * Note: We only support javascript at the moment. We will support Rust, Typescript in a further release. @@ -53,12 +52,12 @@ async function getBlock(block: Block) { } `; -test("formatting for JS code with wrapCode", () => { +test('formatting for JS code with wrapCode', () => { expect(formatIndexingCode(inputJS2, true)).toEqual(expectedOutput3); }); -const inputJS3 = "const a = block.header().height;\nawait context.set(\"demo_height\", h\n"; +const inputJS3 = 'const a = block.header().height;\nawait context.set("demo_height", h\n'; -test("Handling invalid JS input returns original", () => { +test('Handling invalid JS input returns original', () => { expect(formatIndexingCode(inputJS3, false)).toEqual(inputJS3); }); diff --git a/frontend/src/utils/getLatestBlockHeight.js b/frontend/src/utils/getLatestBlockHeight.js index 6036bb01a..a6bdd7e75 100644 --- a/frontend/src/utils/getLatestBlockHeight.js +++ b/frontend/src/utils/getLatestBlockHeight.js @@ -1,18 +1,13 @@ import { providers } from 'near-api-js'; //network config (replace testnet with mainnet or betanet) -const provider = new providers.JsonRpcProvider( - "https://archival-rpc.mainnet.near.org" -); - +const provider = new providers.JsonRpcProvider('https://archival-rpc.mainnet.near.org'); // get latest block height export const getLatestBlockHeight = async () => { - const provider = new providers.JsonRpcProvider( - "https://archival-rpc.mainnet.near.org" - ); - const latestBlock = await provider.block({ - finality: "final" - }); - return latestBlock.header.height; + const provider = new providers.JsonRpcProvider('https://archival-rpc.mainnet.near.org'); + const latestBlock = await provider.block({ + finality: 'final', + }); + return latestBlock.header.height; }; diff --git a/frontend/src/utils/indexerRunner.js b/frontend/src/utils/indexerRunner.js index 1bad1896f..b61cb7a9d 100644 --- a/frontend/src/utils/indexerRunner.js +++ b/frontend/src/utils/indexerRunner.js @@ -1,7 +1,7 @@ -import { Block } from "@near-lake/primitives"; -import { Buffer } from "buffer"; -import { fetchBlockDetails } from "./fetchBlock"; -import { PgSchemaTypeGen } from "./pgSchemaTypeGen"; +import { Block } from '@near-lake/primitives'; +import { Buffer } from 'buffer'; +import { fetchBlockDetails } from './fetchBlock'; +import { PgSchemaTypeGen } from './pgSchemaTypeGen'; global.Buffer = Buffer; export default class IndexerRunner { @@ -16,9 +16,12 @@ export default class IndexerRunner { this.currentHeight = startingHeight; this.shouldStop = false; console.clear(); - console.group('%c Welcome! Lets test your indexing logic on some Near Blocks!', 'color: white; background-color: navy; padding: 5px;'); - if (option == "specific" && !Number(startingHeight)) { - console.log("No Start Block Height Provided to Stream Blocks From"); + console.group( + '%c Welcome! Lets test your indexing logic on some Near Blocks!', + 'color: white; background-color: navy; padding: 5px;', + ); + if (option == 'specific' && !Number(startingHeight)) { + console.log('No Start Block Height Provided to Stream Blocks From'); this.stop(); console.groupEnd(); return; @@ -39,13 +42,12 @@ export default class IndexerRunner { await this.delay(1000); } console.groupEnd(); - } } stop() { this.shouldStop = true; - console.log("%c Stopping Block Processing", 'color: white; background-color: red; padding: 5px;'); + console.log('%c Stopping Block Processing', 'color: white; background-color: red; padding: 5px;'); } delay(ms) { @@ -67,12 +69,17 @@ export default class IndexerRunner { async executeIndexerFunctionOnHeights(heights, indexingCode, schema, schemaName) { console.clear(); - console.group('%c Welcome! Lets test your indexing logic on some Near Blocks!', 'color: white; background-color: navy; padding: 5px;'); + console.group( + '%c Welcome! Lets test your indexing logic on some Near Blocks!', + 'color: white; background-color: navy; padding: 5px;', + ); if (heights.length === 0) { - console.warn("No Block Heights Selected"); + console.warn('No Block Heights Selected'); return; } - console.log("Note: GraphQL Mutations & Queries will not be executed on your database. They will simply return an empty object. Please keep this in mind as this may cause unintended behavior of your indexer function."); + console.log( + 'Note: GraphQL Mutations & Queries will not be executed on your database. They will simply return an empty object. Please keep this in mind as this may cause unintended behavior of your indexer function.', + ); for await (const height of heights) { console.group(`Block Height #${height}`); let blockDetails; @@ -95,66 +102,55 @@ export default class IndexerRunner { const block = Block.fromStreamerMessage(streamerMessage); ` + indexerCode; - const modifiedFunction = this.transformIndexerFunction( - innerCodeWithBlockHelper - ); + const modifiedFunction = this.transformIndexerFunction(innerCodeWithBlockHelper); // Create a function wrapper around the evaluated code - const wrappedFunction = new Function( - "Block", - "streamerMessage", - "context", - modifiedFunction - ); + const wrappedFunction = new Function('Block', 'streamerMessage', 'context', modifiedFunction); // Define the custom context object const context = { set: async (key, value) => { - this.handleLog( - blockHeight, - "", - () => { - console.group(`Setting Key/Value`); - console.log({[key]: value}); - console.groupEnd(); - } - ); + this.handleLog(blockHeight, '', () => { + console.group(`Setting Key/Value`); + console.log({ [key]: value }); + console.groupEnd(); + }); return {}; }, graphql: async (query, mutationData) => { - this.handleLog( - blockHeight, - "", - () => { - let operationType, operationName; - const match = query.match(/(query|mutation)\s+(\w+)\s*(\(.*?\))?\s*\{([\s\S]*)\}/); - if (match) { - operationType = match[1]; - operationName = match[2]; - } - console.group(`Executing GraphQL ${operationType}: (${operationName})`); - if (operationType === 'mutation') console.log('%c Mutations in debug mode do not alter the database', 'color: black; background-color: yellow; padding: 5px;'); - console.group(`Data passed to ${operationType}`); - console.dir(mutationData); - console.groupEnd(); - console.group(`Data returned by ${operationType}`); - console.log({}); - console.groupEnd(); - console.groupEnd(); + this.handleLog(blockHeight, '', () => { + let operationType, operationName; + const match = query.match(/(query|mutation)\s+(\w+)\s*(\(.*?\))?\s*\{([\s\S]*)\}/); + if (match) { + operationType = match[1]; + operationName = match[2]; } - ); + console.group(`Executing GraphQL ${operationType}: (${operationName})`); + if (operationType === 'mutation') + console.log( + '%c Mutations in debug mode do not alter the database', + 'color: black; background-color: yellow; padding: 5px;', + ); + console.group(`Data passed to ${operationType}`); + console.dir(mutationData); + console.groupEnd(); + console.group(`Data returned by ${operationType}`); + console.log({}); + console.groupEnd(); + console.groupEnd(); + }); return {}; }, log: async (message) => { this.handleLog(blockHeight, message); }, - db: this.buildDatabaseContext(blockHeight, schemaName, schema) + db: this.buildDatabaseContext(blockHeight, schemaName, schema), }; wrappedFunction(Block, streamerMessage, context); } - buildDatabaseContext (blockHeight, schemaName, schema) { + buildDatabaseContext(blockHeight, schemaName, schema) { try { const tableNameToDefinitionNamesMapping = this.pgSchemaTypeGen.getTableNameToDefinitionNamesMapping(schema); const tableNames = Array.from(tableNameToDefinitionNamesMapping.keys()); @@ -166,7 +162,9 @@ export default class IndexerRunner { const sanitizedTableName = this.pgSchemaTypeGen.sanitizeTableName(tableName); const tableDefinitionNames = tableNameToDefinitionNamesMapping.get(tableName); if (sanitizedTableNames.has(sanitizedTableName)) { - throw new Error(`Table '${tableName}' has the same name as another table in the generated types. Special characters are removed to generate context.db methods. Please rename the table.`); + throw new Error( + `Table '${tableName}' has the same name as another table in the generated types. Special characters are removed to generate context.db methods. Please rename the table.`, + ); } else { sanitizedTableNames.add(sanitizedTableName); } @@ -174,62 +172,82 @@ export default class IndexerRunner { // Generate context.db methods for table const funcForTable = { [`${sanitizedTableName}`]: { - insert: async (rowsToInsert) => await this.dbOperationLog(blockHeight, - `Mocking: Insert of the following objects into table ${tableDefinitionNames.originalTableName} on schema ${schemaName}`, - rowsToInsert), + insert: async (rowsToInsert) => + await this.dbOperationLog( + blockHeight, + `Mocking: Insert of the following objects into table ${tableDefinitionNames.originalTableName} on schema ${schemaName}`, + rowsToInsert, + ), - select: async (whereObj, limit = null) => await this.dbOperationLog(blockHeight, - `Mocking: Selection of objects with the following values from table ${tableDefinitionNames.originalTableName} on schema ${schemaName} with ${limit === null ? 'no' : limit} limit`, - whereObj), - - update: async (whereObj, updateObj) => await this.dbOperationLog(blockHeight, - `Mocking: Update of objects that match the specified fields with the following values in table ${tableDefinitionNames.originalTableName} on schema ${schemaName}`, - { - matchingFields: whereObj.map(col => tableDefinitionNames.originalColumnNames.get(col) ?? col), - fieldsToUpdate: updateObj.map(col => tableDefinitionNames.originalColumnNames.get(col) ?? col) - }), + select: async (whereObj, limit = null) => + await this.dbOperationLog( + blockHeight, + `Mocking: Selection of objects with the following values from table ${ + tableDefinitionNames.originalTableName + } on schema ${schemaName} with ${limit === null ? 'no' : limit} limit`, + whereObj, + ), - upsert: async (rowsToUpsert, conflictColumns, updateColumns) => await this.dbOperationLog(blockHeight, - `Mocking: Insertion of the following objects into table ${tableDefinitionNames.originalTableName} on schema ${schemaName}. Conflict on the specified columns will update values in the specified columns`, - { - insertObjects: rowsToUpsert, - conflictColumns: conflictColumns.map(col => tableDefinitionNames.originalColumnNames.get(col) ?? col).join(', '), - updateColumns: updateColumns.map(col => tableDefinitionNames.originalColumnNames.get(col) ?? col).join(', ') - }), + update: async (whereObj, updateObj) => + await this.dbOperationLog( + blockHeight, + `Mocking: Update of objects that match the specified fields with the following values in table ${tableDefinitionNames.originalTableName} on schema ${schemaName}`, + { + matchingFields: whereObj.map((col) => tableDefinitionNames.originalColumnNames.get(col) ?? col), + fieldsToUpdate: updateObj.map((col) => tableDefinitionNames.originalColumnNames.get(col) ?? col), + }, + ), - delete: async (whereObj) => await this.dbOperationLog(blockHeight, - `Mocking: Deletion of objects which match the following object's values from table ${tableDefinitionNames.originalTableName} on schema ${schemaName}`, - whereObj) - } + upsert: async (rowsToUpsert, conflictColumns, updateColumns) => + await this.dbOperationLog( + blockHeight, + `Mocking: Insertion of the following objects into table ${tableDefinitionNames.originalTableName} on schema ${schemaName}. Conflict on the specified columns will update values in the specified columns`, + { + insertObjects: rowsToUpsert, + conflictColumns: conflictColumns + .map((col) => tableDefinitionNames.originalColumnNames.get(col) ?? col) + .join(', '), + updateColumns: updateColumns + .map((col) => tableDefinitionNames.originalColumnNames.get(col) ?? col) + .join(', '), + }, + ), + + delete: async (whereObj) => + await this.dbOperationLog( + blockHeight, + `Mocking: Deletion of objects which match the following object's values from table ${tableDefinitionNames.originalTableName} on schema ${schemaName}`, + whereObj, + ), + }, }; return { ...prev, - ...funcForTable + ...funcForTable, }; }, {}); return result; } catch (error) { - console.warn('Caught error when generating context.db methods. Building no functions. You can still use other context object methods.\n', error); + console.warn( + 'Caught error when generating context.db methods. Building no functions. You can still use other context object methods.\n', + error, + ); } } dbOperationLog(blockHeight, logMessage, data) { - this.handleLog( - blockHeight, - "", - () => { - console.group(logMessage); - console.log(data); - console.groupEnd(); - } - ); + this.handleLog(blockHeight, '', () => { + console.group(logMessage); + console.log(data); + console.groupEnd(); + }); return {}; } // deprecated replaceNewLines(code) { - return code.replace(/\\n/g, "\n").replace(/\\"/g, '"'); + return code.replace(/\\n/g, '\n').replace(/\\"/g, '"'); } enableAwaitTransform(indexerFunction) { @@ -242,26 +260,23 @@ export default class IndexerRunner { } transformIndexerFunction(indexerFunction) { - return [this.replaceNewLines, this.enableAwaitTransform].reduce( - (acc, val) => val(acc), - indexerFunction - ); + return [this.replaceNewLines, this.enableAwaitTransform].reduce((acc, val) => val(acc), indexerFunction); } renameUnderscoreFieldsToCamelCase(value) { - if (value && typeof value === "object" && !Array.isArray(value)) { + if (value && typeof value === 'object' && !Array.isArray(value)) { // It's a non-null, non-array object, create a replacement with the keys initially-capped const newValue = {}; for (const key in value) { const newKey = key - .split("_") + .split('_') .map((word, i) => { if (i > 0) { return word.charAt(0).toUpperCase() + word.slice(1); } return word; }) - .join(""); + .join(''); newValue[newKey] = value[key]; } return newValue; diff --git a/frontend/src/utils/pgSchemaTypeGen.js b/frontend/src/utils/pgSchemaTypeGen.js index 63f9ce4d0..d899de4aa 100644 --- a/frontend/src/utils/pgSchemaTypeGen.js +++ b/frontend/src/utils/pgSchemaTypeGen.js @@ -1,213 +1,227 @@ -import { Parser } from "node-sql-parser"; +import { Parser } from 'node-sql-parser'; export class PgSchemaTypeGen { - constructor() { - this.parser = new Parser(); - } - - getColumnDefinitionNames(columnDefs) { - const columnDefinitionNames = new Map(); - for (const columnDef of columnDefs) { - if (columnDef.column?.type === 'column_ref') { - const columnNameDef = columnDef.column.column.expr; - const actualColumnName = columnNameDef.type === 'double_quote_string' ? `"${columnNameDef.value}"` : columnNameDef.value; - columnDefinitionNames.set(columnNameDef.value, actualColumnName); - } - } - return columnDefinitionNames; - } - - retainOriginalQuoting(schema, tableName) { - const createTableQuotedRegex = `\\b(create|CREATE)\\s+(table|TABLE)\\s+"${tableName}"\\s*`; - return schema.match(new RegExp(createTableQuotedRegex, 'i')) ? `"${tableName}"` : tableName; - } - - getTableNameToDefinitionNamesMapping(schema) { - let schemaSyntaxTree = this.parser.astify(schema, { database: 'Postgresql' }); - schemaSyntaxTree = Array.isArray(schemaSyntaxTree) ? schemaSyntaxTree : [schemaSyntaxTree]; // Ensure iterable - const tableNameToDefinitionNamesMap = new Map(); - - for (const statement of schemaSyntaxTree) { - if (statement.type === 'create' && statement.keyword === 'table' && statement.table !== undefined) { - const tableName = statement.table[0].table; - - if (tableNameToDefinitionNamesMap.has(tableName)) { - throw new Error(`Table ${tableName} already exists in schema. Table names must be unique. Quotes are not allowed as a differentiator between table names.`); - } - - const createDefs = statement.create_definitions ?? []; - const tableDefinitionNames = { - originalTableName: this.retainOriginalQuoting(schema, tableName), - originalColumnNames: this.getColumnDefinitionNames(createDefs) - }; - tableNameToDefinitionNamesMap.set(tableName, tableDefinitionNames); - } - } - - if (tableNameToDefinitionNamesMap.size === 0) { - throw new Error('Schema does not have any tables. There should be at least one table.'); - } - - return tableNameToDefinitionNamesMap; - } - - sanitizeTableName(tableName) { - // Convert to PascalCase - let pascalCaseTableName = tableName - .replace(/[^a-zA-Z0-9_]/g, '_') // Replace special characters with underscores - .replace(/^([a-zA-Z])|_([a-zA-Z])/g, (match) => match.toUpperCase()) // PascalCase transformation - .replace(/_/g, ''); // Remove all underscores - - // Add underscore if first character is a number - if (/^[0-9]/.test(pascalCaseTableName)) { - pascalCaseTableName = '_' + pascalCaseTableName; - } - - return pascalCaseTableName; - } - - generateTypes(sqlSchema) { - const schemaSyntaxTree = this.parser.astify(sqlSchema, { database: "Postgresql" }); - const dbSchema = {}; - - const statements = Array.isArray(schemaSyntaxTree) ? schemaSyntaxTree : [schemaSyntaxTree]; - for (const statement of statements) { - if (statement.type === "create" && statement.keyword === "table") { - this.processCreateTableStatement(statement, dbSchema); - } else if (statement.type === "alter") { - this.processAlterTableStatement(statement, dbSchema); - } - } - - const tsTypes = this.generateTypeScriptDefinitions(dbSchema); - console.log(`Types successfully generated`); - return tsTypes; - } - - processCreateTableStatement(statement, dbSchema) { - const tableName = statement.table[0].table; - - if (Object.prototype.hasOwnProperty.call(dbSchema, tableName)) { - throw new Error(`Table ${tableName} already exists in schema. Table names must be unique. Quotes are not allowed as a differentiator between table names.`); - } - - let columns = {}; - for (const columnSpec of statement.create_definitions) { - if (Object.prototype.hasOwnProperty.call(columnSpec, "column") && Object.prototype.hasOwnProperty.call(columnSpec, "definition")) { - this.addColumn(columnSpec, columns); - } else if (Object.prototype.hasOwnProperty.call(columnSpec, "constraint") && columnSpec.constraint_type === "primary key") { - for (const foreignKeyDef of columnSpec.definition) { - columns[foreignKeyDef.column.expr.value].nullable = false; - } - } - } - - dbSchema[tableName] = columns; - } - - processAlterTableStatement(statement, dbSchema) { - const tableName = statement.table[0].table; - - let newConstraint = {}; - for (const alterSpec of statement.expr) { - switch (alterSpec.action) { - case "add": - switch (alterSpec.resource) { - case "column": - this.addColumn(alterSpec, dbSchema[tableName]); - break; - case "constraint": - newConstraint = alterSpec.create_definitions; - if (newConstraint.constraint_type == "primary key") { - for (const foreignKeyDef of newConstraint.definition) { - dbSchema[tableName][foreignKeyDef.column].nullable = false; - } - } - break; - } - break; - case "drop": - delete dbSchema[tableName][alterSpec.column.column]; - break; - } - } - } - - addColumn(columnDef, columns) { - const columnName = columnDef.column.column.expr.value; - const columnType = this.getTypescriptType(columnDef.definition.dataType); - const nullable = this.getNullableStatus(columnDef); - const required = this.getRequiredStatus(columnDef, nullable); - - if (Object.prototype.hasOwnProperty.call(columns, columnName)) { - console.warn(`Column ${columnName} already exists in table. Skipping.`); - return; - } - - columns[columnName] = { - type: columnType, - nullable: nullable, - required: required, - }; - } - - getNullableStatus(columnDef) { - const isPrimaryKey = Object.prototype.hasOwnProperty.call(columnDef, "unique_or_primary") && - columnDef.unique_or_primary && - columnDef.unique_or_primary === "primary key"; - const isNullable = Object.prototype.hasOwnProperty.call(columnDef, "nullable") && - columnDef.nullable && - columnDef.nullable.value === "not null"; - return isPrimaryKey || isNullable ? false : true; - } - - getRequiredStatus(columnDef, nullable) { - const hasDefaultValue = Object.prototype.hasOwnProperty.call(columnDef, "default_val") && - columnDef.default_val && - columnDef.default_val != null; - const isSerial = columnDef.definition.dataType - .toLowerCase() - .includes("serial"); - return hasDefaultValue || isSerial || nullable ? false : true; - } - - generateTypeScriptDefinitions(schema) { - let tsDefinitions = ""; - let contextObject = `declare const context: { + constructor() { + this.parser = new Parser(); + } + + getColumnDefinitionNames(columnDefs) { + const columnDefinitionNames = new Map(); + for (const columnDef of columnDefs) { + if (columnDef.column?.type === 'column_ref') { + const columnNameDef = columnDef.column.column.expr; + const actualColumnName = + columnNameDef.type === 'double_quote_string' ? `"${columnNameDef.value}"` : columnNameDef.value; + columnDefinitionNames.set(columnNameDef.value, actualColumnName); + } + } + return columnDefinitionNames; + } + + retainOriginalQuoting(schema, tableName) { + const createTableQuotedRegex = `\\b(create|CREATE)\\s+(table|TABLE)\\s+"${tableName}"\\s*`; + return schema.match(new RegExp(createTableQuotedRegex, 'i')) ? `"${tableName}"` : tableName; + } + + getTableNameToDefinitionNamesMapping(schema) { + let schemaSyntaxTree = this.parser.astify(schema, { database: 'Postgresql' }); + schemaSyntaxTree = Array.isArray(schemaSyntaxTree) ? schemaSyntaxTree : [schemaSyntaxTree]; // Ensure iterable + const tableNameToDefinitionNamesMap = new Map(); + + for (const statement of schemaSyntaxTree) { + if (statement.type === 'create' && statement.keyword === 'table' && statement.table !== undefined) { + const tableName = statement.table[0].table; + + if (tableNameToDefinitionNamesMap.has(tableName)) { + throw new Error( + `Table ${tableName} already exists in schema. Table names must be unique. Quotes are not allowed as a differentiator between table names.`, + ); + } + + const createDefs = statement.create_definitions ?? []; + const tableDefinitionNames = { + originalTableName: this.retainOriginalQuoting(schema, tableName), + originalColumnNames: this.getColumnDefinitionNames(createDefs), + }; + tableNameToDefinitionNamesMap.set(tableName, tableDefinitionNames); + } + } + + if (tableNameToDefinitionNamesMap.size === 0) { + throw new Error('Schema does not have any tables. There should be at least one table.'); + } + + return tableNameToDefinitionNamesMap; + } + + sanitizeTableName(tableName) { + // Convert to PascalCase + let pascalCaseTableName = tableName + .replace(/[^a-zA-Z0-9_]/g, '_') // Replace special characters with underscores + .replace(/^([a-zA-Z])|_([a-zA-Z])/g, (match) => match.toUpperCase()) // PascalCase transformation + .replace(/_/g, ''); // Remove all underscores + + // Add underscore if first character is a number + if (/^[0-9]/.test(pascalCaseTableName)) { + pascalCaseTableName = '_' + pascalCaseTableName; + } + + return pascalCaseTableName; + } + + generateTypes(sqlSchema) { + const schemaSyntaxTree = this.parser.astify(sqlSchema, { database: 'Postgresql' }); + const dbSchema = {}; + + const statements = Array.isArray(schemaSyntaxTree) ? schemaSyntaxTree : [schemaSyntaxTree]; + for (const statement of statements) { + if (statement.type === 'create' && statement.keyword === 'table') { + this.processCreateTableStatement(statement, dbSchema); + } else if (statement.type === 'alter') { + this.processAlterTableStatement(statement, dbSchema); + } + } + + const tsTypes = this.generateTypeScriptDefinitions(dbSchema); + console.log(`Types successfully generated`); + return tsTypes; + } + + processCreateTableStatement(statement, dbSchema) { + const tableName = statement.table[0].table; + + if (Object.prototype.hasOwnProperty.call(dbSchema, tableName)) { + throw new Error( + `Table ${tableName} already exists in schema. Table names must be unique. Quotes are not allowed as a differentiator between table names.`, + ); + } + + let columns = {}; + for (const columnSpec of statement.create_definitions) { + if ( + Object.prototype.hasOwnProperty.call(columnSpec, 'column') && + Object.prototype.hasOwnProperty.call(columnSpec, 'definition') + ) { + this.addColumn(columnSpec, columns); + } else if ( + Object.prototype.hasOwnProperty.call(columnSpec, 'constraint') && + columnSpec.constraint_type === 'primary key' + ) { + for (const foreignKeyDef of columnSpec.definition) { + columns[foreignKeyDef.column.expr.value].nullable = false; + } + } + } + + dbSchema[tableName] = columns; + } + + processAlterTableStatement(statement, dbSchema) { + const tableName = statement.table[0].table; + + let newConstraint = {}; + for (const alterSpec of statement.expr) { + switch (alterSpec.action) { + case 'add': + switch (alterSpec.resource) { + case 'column': + this.addColumn(alterSpec, dbSchema[tableName]); + break; + case 'constraint': + newConstraint = alterSpec.create_definitions; + if (newConstraint.constraint_type == 'primary key') { + for (const foreignKeyDef of newConstraint.definition) { + dbSchema[tableName][foreignKeyDef.column].nullable = false; + } + } + break; + } + break; + case 'drop': + delete dbSchema[tableName][alterSpec.column.column]; + break; + } + } + } + + addColumn(columnDef, columns) { + const columnName = columnDef.column.column.expr.value; + const columnType = this.getTypescriptType(columnDef.definition.dataType); + const nullable = this.getNullableStatus(columnDef); + const required = this.getRequiredStatus(columnDef, nullable); + + if (Object.prototype.hasOwnProperty.call(columns, columnName)) { + console.warn(`Column ${columnName} already exists in table. Skipping.`); + return; + } + + columns[columnName] = { + type: columnType, + nullable: nullable, + required: required, + }; + } + + getNullableStatus(columnDef) { + const isPrimaryKey = + Object.prototype.hasOwnProperty.call(columnDef, 'unique_or_primary') && + columnDef.unique_or_primary && + columnDef.unique_or_primary === 'primary key'; + const isNullable = + Object.prototype.hasOwnProperty.call(columnDef, 'nullable') && + columnDef.nullable && + columnDef.nullable.value === 'not null'; + return isPrimaryKey || isNullable ? false : true; + } + + getRequiredStatus(columnDef, nullable) { + const hasDefaultValue = + Object.prototype.hasOwnProperty.call(columnDef, 'default_val') && + columnDef.default_val && + columnDef.default_val != null; + const isSerial = columnDef.definition.dataType.toLowerCase().includes('serial'); + return hasDefaultValue || isSerial || nullable ? false : true; + } + + generateTypeScriptDefinitions(schema) { + let tsDefinitions = ''; + let contextObject = `declare const context: { graphql: (operation, variables) => Promise, set: (key, value) => Promise, log: (...log) => Promise, fetchFromSocialApi: (path, options) => Promise, db: {`; - const tableList = new Set(); - for (const [tableName, columns] of Object.entries(schema)) { - const sanitizedTableName = this.sanitizeTableName(tableName); + const tableList = new Set(); + for (const [tableName, columns] of Object.entries(schema)) { + const sanitizedTableName = this.sanitizeTableName(tableName); - if (tableList.has(sanitizedTableName)) { - throw new Error(`Table '${tableName}' has the same name as another table in the generated types. Special characters are removed to generate context.db methods. Please rename the table.`); - } + if (tableList.has(sanitizedTableName)) { + throw new Error( + `Table '${tableName}' has the same name as another table in the generated types. Special characters are removed to generate context.db methods. Please rename the table.`, + ); + } - tableList.add(sanitizedTableName); + tableList.add(sanitizedTableName); - let itemDefinition = `declare interface ${sanitizedTableName}Item {\n`; - let inputDefinition = `declare interface ${sanitizedTableName}Input {\n`; + let itemDefinition = `declare interface ${sanitizedTableName}Item {\n`; + let inputDefinition = `declare interface ${sanitizedTableName}Input {\n`; - for (const [columnName, columnDetails] of Object.entries(columns)) { - const tsType = columnDetails.nullable ? columnDetails.type + " | null" : columnDetails.type; - const optional = columnDetails.required ? "" : "?"; + for (const [columnName, columnDetails] of Object.entries(columns)) { + const tsType = columnDetails.nullable ? columnDetails.type + ' | null' : columnDetails.type; + const optional = columnDetails.required ? '' : '?'; - itemDefinition += ` ${columnName}?: ${tsType};\n`; - inputDefinition += ` ${columnName}${optional}: ${tsType};\n`; - } + itemDefinition += ` ${columnName}?: ${tsType};\n`; + inputDefinition += ` ${columnName}${optional}: ${tsType};\n`; + } - itemDefinition += "}\n\n"; - inputDefinition += "}\n\n"; - const columnNamesDef = `type ${sanitizedTableName}Columns = "${Object.keys(columns).join('" | "')}";\n\n`; + itemDefinition += '}\n\n'; + inputDefinition += '}\n\n'; + const columnNamesDef = `type ${sanitizedTableName}Columns = "${Object.keys(columns).join('" | "')}";\n\n`; - tsDefinitions += itemDefinition + inputDefinition + columnNamesDef; + tsDefinitions += itemDefinition + inputDefinition + columnNamesDef; - contextObject += ` + contextObject += ` ${sanitizedTableName}: { insert: (objectsToInsert: ${sanitizedTableName}Input | ${sanitizedTableName}Input[]) => Promise<${sanitizedTableName}Item[]>; select: (filterObj: ${sanitizedTableName}Item, limit = null) => Promise<${sanitizedTableName}Item[]>; @@ -215,56 +229,56 @@ export class PgSchemaTypeGen { upsert: (objectsToInsert: ${sanitizedTableName}Input | ${sanitizedTableName}Input[], conflictColumns: ${sanitizedTableName}Columns[], updateColumns: ${sanitizedTableName}Columns[]) => Promise<${sanitizedTableName}Item[]>; delete: (filterObj: ${sanitizedTableName}Item) => Promise<${sanitizedTableName}Item[]>; },`; - } - - contextObject += '\n }\n};'; - return tsDefinitions + contextObject; - } - getTypescriptType(pgType) { - const typeMap = { - // Numeric types - "smallint": "number", - "integer": "number", - "bigint": "number", - "decimal": "number", - "numeric": "number", - "real": "number", - "double precision": "number", - "serial": "number", - "bigserial": "number", - // Monetary types - "money": "number", - // Character types - "character varying": "string", - "varchar": "string", - "character": "string", - "char": "string", - "text": "string", - // Binary data types - "bytea": "Buffer", - // Boolean type - "boolean": "boolean", - // Date/Time types - "timestamp": "Date", - "timestamp without time zone": "Date", - "timestamp with time zone": "Date", - "date": "Date", - "time": "Date", - "time without time zone": "Date", - "time with time zone": "Date", - "interval": "Date", - // UUID type - "uuid": "string", - // JSON types - "json": "any", - "jsonb": "any", - // Arrays - "integer[]": "number[]", - "text[]": "string[]", - // Default - "default": "any" // Replace with appropriate default type - }; - const typeKey = pgType.toLowerCase(); - return typeMap[typeKey] || typeMap["default"]; - } + } + + contextObject += '\n }\n};'; + return tsDefinitions + contextObject; + } + getTypescriptType(pgType) { + const typeMap = { + // Numeric types + smallint: 'number', + integer: 'number', + bigint: 'number', + decimal: 'number', + numeric: 'number', + real: 'number', + 'double precision': 'number', + serial: 'number', + bigserial: 'number', + // Monetary types + money: 'number', + // Character types + 'character varying': 'string', + varchar: 'string', + character: 'string', + char: 'string', + text: 'string', + // Binary data types + bytea: 'Buffer', + // Boolean type + boolean: 'boolean', + // Date/Time types + timestamp: 'Date', + 'timestamp without time zone': 'Date', + 'timestamp with time zone': 'Date', + date: 'Date', + time: 'Date', + 'time without time zone': 'Date', + 'time with time zone': 'Date', + interval: 'Date', + // UUID type + uuid: 'string', + // JSON types + json: 'any', + jsonb: 'any', + // Arrays + 'integer[]': 'number[]', + 'text[]': 'string[]', + // Default + default: 'any', // Replace with appropriate default type + }; + const typeKey = pgType.toLowerCase(); + return typeMap[typeKey] || typeMap['default']; + } } diff --git a/frontend/src/utils/queryIndexerFunction.js b/frontend/src/utils/queryIndexerFunction.js index 46e5be500..fe18deb84 100644 --- a/frontend/src/utils/queryIndexerFunction.js +++ b/frontend/src/utils/queryIndexerFunction.js @@ -1,28 +1,22 @@ -import { providers } from "near-api-js"; -const REGISTRY_CONTRACT = - process.env.NEXT_PUBLIC_REGISTRY_CONTRACT_ID || - "dev-queryapi.dataplatform.near"; +import { providers } from 'near-api-js'; +const REGISTRY_CONTRACT = process.env.NEXT_PUBLIC_REGISTRY_CONTRACT_ID || 'dev-queryapi.dataplatform.near'; //network config (replace testnet with mainnet or betanet) -const provider = new providers.JsonRpcProvider( - "https://rpc.mainnet.near.org" -); +const provider = new providers.JsonRpcProvider('https://rpc.mainnet.near.org'); export const queryIndexerFunctionDetails = async (accountId, functionName) => { let args = { account_id: accountId }; try { const result = await provider.query({ - request_type: "call_function", + request_type: 'call_function', account_id: REGISTRY_CONTRACT, // TODO Create method to query single indexer - method_name: "list_by_account", - args_base64: Buffer.from(JSON.stringify(args)).toString("base64"), - finality: "optimistic", + method_name: 'list_by_account', + args_base64: Buffer.from(JSON.stringify(args)).toString('base64'), + finality: 'optimistic', }); - const indexers = result.result && - result.result.length > 0 && - JSON.parse(Buffer.from(result.result).toString()); + const indexers = result.result && result.result.length > 0 && JSON.parse(Buffer.from(result.result).toString()); if (!indexers) { return null; @@ -30,8 +24,10 @@ export const queryIndexerFunctionDetails = async (accountId, functionName) => { return indexers[functionName]; } catch (error) { - console.log(`Could not query indexer function details from registry ${REGISTRY_CONTRACT}, for ${accountId}/${functionName}`); - console.log(error, "error"); + console.log( + `Could not query indexer function details from registry ${REGISTRY_CONTRACT}, for ${accountId}/${functionName}`, + ); + console.log(error, 'error'); return null; } }; diff --git a/frontend/src/utils/resize.js b/frontend/src/utils/resize.js index 3aa82b140..84356477d 100644 --- a/frontend/src/utils/resize.js +++ b/frontend/src/utils/resize.js @@ -1,13 +1,6 @@ -import { - useCallback, - useEffect, - useLayoutEffect, - useMemo, - useRef, - useState, -} from "react"; +import { useCallback, useEffect, useLayoutEffect, useMemo, useRef, useState } from 'react'; -import debounce from "./debounce"; +import debounce from './debounce'; export function useDragResize({ defaultSizeRelation = DEFAULT_FLEX, @@ -27,16 +20,16 @@ export function useDragResize({ storage.set(storageKey, value); } }), - [storage, storageKey] + [storage, storageKey], ); const [hiddenElement, setHiddenElement] = useState(() => { const storedValue = storage && storageKey ? storage.get(storageKey) : null; - if (storedValue === HIDE_FIRST || initiallyHidden === "first") { - return "first"; + if (storedValue === HIDE_FIRST || initiallyHidden === 'first') { + return 'first'; } - if (storedValue === HIDE_SECOND || initiallyHidden === "second") { - return "second"; + if (storedValue === HIDE_SECOND || initiallyHidden === 'second') { + return 'second'; } return null; }); @@ -48,7 +41,7 @@ export function useDragResize({ onHiddenElementChange?.(element); } }, - [hiddenElement, onHiddenElementChange] + [hiddenElement, onHiddenElementChange], ); const firstRef = useRef(null); @@ -62,35 +55,30 @@ export function useDragResize({ */ useLayoutEffect(() => { const storedValue = - storage && storageKey - ? storage.get(storageKey) || defaultFlexRef.current - : defaultFlexRef.current; - const flexDirection = direction === "horizontal" ? "row" : "column"; + storage && storageKey ? storage.get(storageKey) || defaultFlexRef.current : defaultFlexRef.current; + const flexDirection = direction === 'horizontal' ? 'row' : 'column'; if (firstRef.current) { - firstRef.current.style.display = "flex"; + firstRef.current.style.display = 'flex'; firstRef.current.style.flexDirection = flexDirection; firstRef.current.style.flex = - storedValue === HIDE_FIRST || storedValue === HIDE_SECOND - ? defaultFlexRef.current - : storedValue; + storedValue === HIDE_FIRST || storedValue === HIDE_SECOND ? defaultFlexRef.current : storedValue; } if (secondRef.current) { - secondRef.current.style.display = "flex"; + secondRef.current.style.display = 'flex'; secondRef.current.style.flexDirection = flexDirection; - secondRef.current.style.flex = "1"; + secondRef.current.style.flex = '1'; } if (dragBarRef.current) { - dragBarRef.current.style.display = "flex"; + dragBarRef.current.style.display = 'flex'; dragBarRef.current.style.flexDirection = flexDirection; } }, [direction, storage, storageKey]); const hide = useCallback((resizableElement) => { - const element = - resizableElement === "first" ? firstRef.current : secondRef.current; + const element = resizableElement === 'first' ? firstRef.current : secondRef.current; if (!element) { return; } @@ -99,35 +87,34 @@ export function useDragResize({ // and the codemirror container would have zero width, the layout isn't // instant pretty. By always giving the editor some width we avoid any // layout shifts when the editor reappears. - element.style.left = "-1000px"; - element.style.position = "absolute"; - element.style.opacity = "0"; - element.style.height = "500px"; - element.style.width = "500px"; + element.style.left = '-1000px'; + element.style.position = 'absolute'; + element.style.opacity = '0'; + element.style.height = '500px'; + element.style.width = '500px'; // Make sure that the flex value of the first item is at least equal to one // so that the entire space of the parent element is filled up if (firstRef.current) { const flex = parseFloat(firstRef.current.style.flex); if (!Number.isFinite(flex) || flex < 1) { - firstRef.current.style.flex = "1"; + firstRef.current.style.flex = '1'; } } }, []); const show = useCallback( (resizableElement) => { - const element = - resizableElement === "first" ? firstRef.current : secondRef.current; + const element = resizableElement === 'first' ? firstRef.current : secondRef.current; if (!element) { return; } - element.style.width = ""; - element.style.height = ""; - element.style.opacity = ""; - element.style.position = ""; - element.style.left = ""; + element.style.width = ''; + element.style.height = ''; + element.style.opacity = ''; + element.style.position = ''; + element.style.left = ''; if (firstRef.current && storage && storageKey) { const storedValue = storage?.get(storageKey); @@ -136,22 +123,22 @@ export function useDragResize({ } } }, - [storage, storageKey] + [storage, storageKey], ); /** * Hide and show items when state changes */ useLayoutEffect(() => { - if (hiddenElement === "first") { - hide("first"); + if (hiddenElement === 'first') { + hide('first'); } else { - show("first"); + show('first'); } - if (hiddenElement === "second") { - hide("second"); + if (hiddenElement === 'second') { + hide('second'); } else { - show("second"); + show('second'); } }, [hiddenElement, hide, show]); @@ -163,31 +150,24 @@ export function useDragResize({ const firstContainer = firstRef.current; const wrapper = firstContainer.parentElement; - const eventProperty = direction === "horizontal" ? "clientX" : "clientY"; - const rectProperty = direction === "horizontal" ? "left" : "top"; - const adjacentRectProperty = - direction === "horizontal" ? "right" : "bottom"; - const sizeProperty = - direction === "horizontal" ? "clientWidth" : "clientHeight"; + const eventProperty = direction === 'horizontal' ? 'clientX' : 'clientY'; + const rectProperty = direction === 'horizontal' ? 'left' : 'top'; + const adjacentRectProperty = direction === 'horizontal' ? 'right' : 'bottom'; + const sizeProperty = direction === 'horizontal' ? 'clientWidth' : 'clientHeight'; function handleMouseDown(downEvent) { downEvent.preventDefault(); // Distance between the start of the drag bar and the exact point where // the user clicked on the drag bar. - const offset = - downEvent[eventProperty] - - dragBarContainer.getBoundingClientRect()[rectProperty]; + const offset = downEvent[eventProperty] - dragBarContainer.getBoundingClientRect()[rectProperty]; function handleMouseMove(moveEvent) { if (moveEvent.buttons === 0) { return handleMouseUp(); } - const firstSize = - moveEvent[eventProperty] - - wrapper.getBoundingClientRect()[rectProperty] - - offset; + const firstSize = moveEvent[eventProperty] - wrapper.getBoundingClientRect()[rectProperty] - offset; const secondSize = wrapper.getBoundingClientRect()[adjacentRectProperty] - moveEvent[eventProperty] + @@ -196,11 +176,11 @@ export function useDragResize({ if (firstSize < sizeThresholdFirst) { // Hide the first display - setHiddenElementWithCallback("first"); + setHiddenElementWithCallback('first'); store(HIDE_FIRST); } else if (secondSize < sizeThresholdSecond) { // Hide the second display - setHiddenElementWithCallback("second"); + setHiddenElementWithCallback('second'); store(HIDE_SECOND); } else { // Show both and adjust the flex value of the first one (the flex @@ -213,15 +193,15 @@ export function useDragResize({ } function handleMouseUp() { - document.removeEventListener("mousemove", handleMouseMove); - document.removeEventListener("mouseup", handleMouseUp); + document.removeEventListener('mousemove', handleMouseMove); + document.removeEventListener('mouseup', handleMouseUp); } - document.addEventListener("mousemove", handleMouseMove); - document.addEventListener("mouseup", handleMouseUp); + document.addEventListener('mousemove', handleMouseMove); + document.addEventListener('mouseup', handleMouseUp); } - dragBarContainer.addEventListener("mousedown", handleMouseDown); + dragBarContainer.addEventListener('mousedown', handleMouseDown); function reset() { if (firstRef.current) { @@ -231,19 +211,13 @@ export function useDragResize({ setHiddenElementWithCallback(null); } - dragBarContainer.addEventListener("dblclick", reset); + dragBarContainer.addEventListener('dblclick', reset); return () => { - dragBarContainer.removeEventListener("mousedown", handleMouseDown); - dragBarContainer.removeEventListener("dblclick", reset); + dragBarContainer.removeEventListener('mousedown', handleMouseDown); + dragBarContainer.removeEventListener('dblclick', reset); }; - }, [ - direction, - setHiddenElementWithCallback, - sizeThresholdFirst, - sizeThresholdSecond, - store, - ]); + }, [direction, setHiddenElementWithCallback, sizeThresholdFirst, sizeThresholdSecond, store]); return useMemo( () => ({ @@ -253,10 +227,10 @@ export function useDragResize({ setHiddenElement, secondRef, }), - [hiddenElement, setHiddenElement] + [hiddenElement, setHiddenElement], ); } const DEFAULT_FLEX = 1; -const HIDE_FIRST = "hide-first"; -const HIDE_SECOND = "hide-second"; +const HIDE_FIRST = 'hide-first'; +const HIDE_SECOND = 'hide-second'; diff --git a/frontend/src/utils/validators.js b/frontend/src/utils/validators.js index 2c78392cf..5b115ab23 100644 --- a/frontend/src/utils/validators.js +++ b/frontend/src/utils/validators.js @@ -1,8 +1,8 @@ -import { defaultSchema, formatIndexingCode, formatSQL } from "./formatters"; -import { PgSchemaTypeGen } from "./pgSchemaTypeGen"; -import { CONTRACT_NAME_REGEX, WILD_CARD_REGEX, WILD_CARD } from "../constants/RegexExp"; +import { defaultSchema, formatIndexingCode, formatSQL } from './formatters'; +import { PgSchemaTypeGen } from './pgSchemaTypeGen'; +import { CONTRACT_NAME_REGEX, WILD_CARD_REGEX, WILD_CARD } from '../constants/RegexExp'; import { ValidationError } from '../classes/ValidationError'; -import { FORMATTING_ERROR_TYPE, TYPE_GENERATION_ERROR_TYPE } from "../constants/Strings"; +import { FORMATTING_ERROR_TYPE, TYPE_GENERATION_ERROR_TYPE } from '../constants/Strings'; export const validateContractId = (accountId) => { accountId = accountId.trim(); @@ -21,8 +21,8 @@ export const validateContractId = (accountId) => { }; export const validateContractIds = (accountIds) => { - const ids = accountIds.split(',').map(id => id.trim()); - return ids.every(accountId => validateContractId(accountId)); + const ids = accountIds.split(',').map((id) => id.trim()); + return ids.every((accountId) => validateContractId(accountId)); }; /** @@ -51,25 +51,27 @@ export function validateSQLSchema(schema) { return { data: formattedSchema, error: null }; } catch (error) { console.log(error); - return { data: schema, error: new ValidationError(error.message, TYPE_GENERATION_ERROR_TYPE), location: error.location }; + return { + data: schema, + error: new ValidationError(error.message, TYPE_GENERATION_ERROR_TYPE), + location: error.location, + }; } } } /** * Asynchronously validates and formats JavaScript code. - * + * * @param {string} code - The JavaScript code to be validated and formatted. * @returns {{ data: string | null, error: string | null }} An object containing either the formatted code or an error. */ export function validateJSCode(code) { - if (!code) return { data: null, error: null }; try { const formattedCode = formatIndexingCode(code); return { data: formattedCode, error: null }; - } catch (error) { console.error(error.message); return { data: code, error }; diff --git a/frontend/tsconfig.json b/frontend/tsconfig.json index 89f0ee206..d8406bd01 100644 --- a/frontend/tsconfig.json +++ b/frontend/tsconfig.json @@ -30,7 +30,7 @@ "include": [ "next-env.d.ts", "**/*.ts", - "**/*.tsx", "src/utils/validators.js", + "**/*.tsx", ], "exclude": [ "node_modules",